Merge branch 'apple:main' into tenantteams

This commit is contained in:
Bharadwaj V.R 2022-09-12 09:18:54 -07:00 committed by GitHub
commit ac4a70ee45
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
99 changed files with 1341 additions and 517 deletions
bindings
cmake
documentation/sphinx/source
fdbcli
fdbclient
fdbrpc/include/fdbrpc
fdbserver
flow
tests

View File

@ -198,7 +198,7 @@ if(NOT WIN32)
target_link_libraries(fdb_c_client_memory_test PRIVATE fdb_c Threads::Threads)
target_include_directories(fdb_c_api_tester_impl PUBLIC ${CMAKE_CURRENT_SOURCE_DIR} ${CMAKE_CURRENT_BINARY_DIR}/foundationdb/ ${CMAKE_SOURCE_DIR}/flow/include ${CMAKE_BINARY_DIR}/flow/include)
target_link_libraries(fdb_c_api_tester_impl PRIVATE fdb_cpp toml11_target Threads::Threads fmt::fmt boost_target)
target_link_libraries(fdb_c_api_tester_impl PRIVATE fdb_cpp toml11_target Threads::Threads fmt::fmt boost_target stdc++fs)
target_link_libraries(fdb_c_api_tester_impl PRIVATE SimpleOpt)
target_include_directories(fdb_c_unit_tests_impl PUBLIC ${CMAKE_CURRENT_SOURCE_DIR} ${CMAKE_CURRENT_BINARY_DIR}/foundationdb/)

View File

@ -107,11 +107,11 @@ void ApiWorkload::randomOperation(TTaskFct cont) {
}
fdb::Key ApiWorkload::randomKeyName() {
return keyPrefix + Random::get().randomStringLowerCase(minKeyLength, maxKeyLength);
return keyPrefix + Random::get().randomByteStringLowerCase(minKeyLength, maxKeyLength);
}
fdb::Value ApiWorkload::randomValue() {
return Random::get().randomStringLowerCase(minValueLength, maxValueLength);
return Random::get().randomByteStringLowerCase(minValueLength, maxValueLength);
}
fdb::Key ApiWorkload::randomNotExistingKey() {

View File

@ -78,7 +78,7 @@ private:
seenReadSuccess = true;
}
if (err.code() != expectedError) {
info(fmt::format("incorrect error. Expected {}, Got {}", err.code(), expectedError));
info(fmt::format("incorrect error. Expected {}, Got {}", expectedError, err.code()));
if (err.code() == error_code_blob_granule_transaction_too_old) {
ASSERT(!seenReadSuccess);
ctx->done();

View File

@ -35,8 +35,8 @@ public:
void start() override { setAndGet(NO_OP_TASK); }
void setAndGet(TTaskFct cont) {
fdb::Key key = keyPrefix + random.randomStringLowerCase(10, 100);
fdb::Value value = random.randomStringLowerCase(10, 1000);
fdb::Key key = keyPrefix + random.randomByteStringLowerCase(10, 100);
fdb::Value value = random.randomByteStringLowerCase(10, 1000);
execTransaction(
[key, value](auto ctx) {
ctx->tx().set(key, value);

View File

@ -65,6 +65,10 @@ std::unordered_map<std::string, std::function<void(const std::string& value, Tes
[](const std::string& value, TestSpec* spec) { //
spec->databasePerTransaction = (value == "true");
} },
{ "tamperClusterFile",
[](const std::string& value, TestSpec* spec) { //
spec->tamperClusterFile = (value == "true");
} },
{ "minFdbThreads",
[](const std::string& value, TestSpec* spec) { //
processIntOption(value, "minFdbThreads", spec->minFdbThreads, 1, 1000);

View File

@ -58,6 +58,9 @@ struct TestSpec {
// Execute each transaction in a separate database instance
bool databasePerTransaction = false;
// Test tampering the cluster file
bool tamperClusterFile = false;
// Size of the FDB client thread pool (a random number in the [min,max] range)
int minFdbThreads = 1;
int maxFdbThreads = 1;

View File

@ -31,6 +31,7 @@
#include <chrono>
#include <thread>
#include <fmt/format.h>
#include <filesystem>
namespace FdbApiTester {
@ -72,14 +73,24 @@ void ITransactionContext::continueAfterAll(std::vector<fdb::Future> futures, TTa
*/
class TransactionContextBase : public ITransactionContext {
public:
TransactionContextBase(fdb::Transaction tx,
TransactionContextBase(ITransactionExecutor* executor,
std::shared_ptr<ITransactionActor> txActor,
TTaskFct cont,
IScheduler* scheduler,
int retryLimit,
std::string bgBasePath)
: fdbTx(tx), txActor(txActor), contAfterDone(cont), scheduler(scheduler), retryLimit(retryLimit),
txState(TxState::IN_PROGRESS), commitCalled(false), bgBasePath(bgBasePath) {}
: executor(executor), txActor(txActor), contAfterDone(cont), scheduler(scheduler), retryLimit(retryLimit),
txState(TxState::IN_PROGRESS), commitCalled(false), bgBasePath(bgBasePath) {
databaseCreateErrorInjected = executor->getOptions().injectDatabaseCreateErrors &&
Random::get().randomBool(executor->getOptions().databaseCreateErrorRatio);
fdb::Database db;
if (databaseCreateErrorInjected) {
db = fdb::Database(executor->getClusterFileForErrorInjection());
} else {
db = executor->selectDatabase();
}
fdbTx = db.createTransaction();
}
virtual ~TransactionContextBase() { ASSERT(txState == TxState::DONE); }
@ -87,7 +98,7 @@ public:
// IN_PROGRESS -> (ON_ERROR -> IN_PROGRESS)* [-> ON_ERROR] -> DONE
enum class TxState { IN_PROGRESS, ON_ERROR, DONE };
fdb::Transaction tx() override { return fdbTx; }
fdb::Transaction tx() override { return fdbTx.atomic_load(); }
// Set a continuation to be executed when a future gets ready
void continueAfter(fdb::Future f, TTaskFct cont, bool retryOnError) override {
@ -125,9 +136,11 @@ public:
retriedErrors.size(),
fmt::join(retriedErrorCodes(), ", "));
}
// cancel transaction so that any pending operations on it
// fail gracefully
fdbTx.cancel();
txActor->complete(fdb::Error::success());
cleanUp();
ASSERT(txState == TxState::DONE);
@ -136,16 +149,58 @@ public:
std::string getBGBasePath() override { return bgBasePath; }
virtual void onError(fdb::Error err) override {
std::unique_lock<std::mutex> lock(mutex);
if (txState != TxState::IN_PROGRESS) {
// Ignore further errors, if the transaction is in the error handing mode or completed
return;
}
txState = TxState::ON_ERROR;
lock.unlock();
// No need to hold the lock from here on, because ON_ERROR state is handled sequentially, and
// other callbacks are simply ignored while it stays in this state
if (!canRetry(err)) {
return;
}
ASSERT(!onErrorFuture);
if (databaseCreateErrorInjected && canBeInjectedDatabaseCreateError(err.code())) {
// Failed to create a database because of failure injection
// Restart by recreating the transaction in a valid database
scheduler->schedule([this]() {
fdb::Database db = executor->selectDatabase();
fdbTx.atomic_store(db.createTransaction());
restartTransaction();
});
} else {
onErrorArg = err;
onErrorFuture = tx().onError(err);
handleOnErrorFuture();
}
}
protected:
virtual void doContinueAfter(fdb::Future f, TTaskFct cont, bool retryOnError) = 0;
virtual void handleOnErrorFuture() = 0;
// Clean up transaction state after completing the transaction
// Note that the object may live longer, because it is referenced
// by not yet triggered callbacks
virtual void cleanUp() {
void cleanUp() {
ASSERT(txState == TxState::DONE);
ASSERT(!onErrorFuture);
txActor = {};
cancelPendingFutures();
}
virtual void cancelPendingFutures() {}
bool canBeInjectedDatabaseCreateError(fdb::Error::CodeType errCode) {
return errCode == error_code_no_cluster_file_found || errCode == error_code_connection_string_invalid;
}
// Complete the transaction with an (unretriable) error
@ -174,15 +229,20 @@ protected:
if (err) {
transactionFailed(err);
} else {
std::unique_lock<std::mutex> lock(mutex);
ASSERT(txState == TxState::ON_ERROR);
txState = TxState::IN_PROGRESS;
commitCalled = false;
lock.unlock();
txActor->start();
restartTransaction();
}
}
void restartTransaction() {
ASSERT(txState == TxState::ON_ERROR);
cancelPendingFutures();
std::unique_lock<std::mutex> lock(mutex);
txState = TxState::IN_PROGRESS;
commitCalled = false;
lock.unlock();
txActor->start();
}
// Checks if a transaction can be retried. Fails the transaction if the check fails
bool canRetry(fdb::Error lastErr) {
ASSERT(txState == TxState::ON_ERROR);
@ -208,6 +268,10 @@ protected:
return retriedErrorCodes;
}
// Pointer to the transaction executor interface
// Set in contructor, stays immutable
ITransactionExecutor* const executor;
// FDB transaction
// Provides a thread safe interface by itself (no need for mutex)
fdb::Transaction fdbTx;
@ -261,6 +325,10 @@ protected:
// blob granule base path
// Set in contructor, stays immutable
const std::string bgBasePath;
// Indicates if the database error was injected
// Accessed on initialization and in ON_ERROR state only (no need for mutex)
bool databaseCreateErrorInjected;
};
/**
@ -268,13 +336,13 @@ protected:
*/
class BlockingTransactionContext : public TransactionContextBase {
public:
BlockingTransactionContext(fdb::Transaction tx,
BlockingTransactionContext(ITransactionExecutor* executor,
std::shared_ptr<ITransactionActor> txActor,
TTaskFct cont,
IScheduler* scheduler,
int retryLimit,
std::string bgBasePath)
: TransactionContextBase(tx, txActor, cont, scheduler, retryLimit, bgBasePath) {}
: TransactionContextBase(executor, txActor, cont, scheduler, retryLimit, bgBasePath) {}
protected:
void doContinueAfter(fdb::Future f, TTaskFct cont, bool retryOnError) override {
@ -315,22 +383,8 @@ protected:
onError(err);
}
virtual void onError(fdb::Error err) override {
std::unique_lock<std::mutex> lock(mutex);
if (txState != TxState::IN_PROGRESS) {
// Ignore further errors, if the transaction is in the error handing mode or completed
return;
}
txState = TxState::ON_ERROR;
lock.unlock();
if (!canRetry(err)) {
return;
}
ASSERT(!onErrorFuture);
onErrorFuture = fdbTx.onError(err);
onErrorArg = err;
virtual void handleOnErrorFuture() override {
ASSERT(txState == TxState::ON_ERROR);
auto start = timeNow();
fdb::Error err2 = onErrorFuture.blockUntilReady();
@ -357,13 +411,13 @@ protected:
*/
class AsyncTransactionContext : public TransactionContextBase {
public:
AsyncTransactionContext(fdb::Transaction tx,
AsyncTransactionContext(ITransactionExecutor* executor,
std::shared_ptr<ITransactionActor> txActor,
TTaskFct cont,
IScheduler* scheduler,
int retryLimit,
std::string bgBasePath)
: TransactionContextBase(tx, txActor, cont, scheduler, retryLimit, bgBasePath) {}
: TransactionContextBase(executor, txActor, cont, scheduler, retryLimit, bgBasePath) {}
protected:
void doContinueAfter(fdb::Future f, TTaskFct cont, bool retryOnError) override {
@ -371,7 +425,7 @@ protected:
if (txState != TxState::IN_PROGRESS) {
return;
}
callbackMap[f] = CallbackInfo{ f, cont, shared_from_this(), retryOnError, timeNow() };
callbackMap[f] = CallbackInfo{ f, cont, shared_from_this(), retryOnError, timeNow(), false };
lock.unlock();
try {
f.then([this](fdb::Future f) { futureReadyCallback(f, this); });
@ -418,7 +472,7 @@ protected:
err.code(),
err.what());
}
if (err.code() == error_code_transaction_cancelled) {
if (err.code() == error_code_transaction_cancelled || cbInfo.cancelled) {
return;
}
if (err.code() == error_code_success || !cbInfo.retryOnError) {
@ -432,25 +486,9 @@ protected:
onError(err);
}
virtual void onError(fdb::Error err) override {
std::unique_lock<std::mutex> lock(mutex);
if (txState != TxState::IN_PROGRESS) {
// Ignore further errors, if the transaction is in the error handing mode or completed
return;
}
txState = TxState::ON_ERROR;
lock.unlock();
virtual void handleOnErrorFuture() override {
ASSERT(txState == TxState::ON_ERROR);
// No need to hold the lock from here on, because ON_ERROR state is handled sequentially, and
// other callbacks are simply ignored while it stays in this state
if (!canRetry(err)) {
return;
}
ASSERT(!onErrorFuture);
onErrorArg = err;
onErrorFuture = tx().onError(err);
onErrorCallTimePoint = timeNow();
onErrorThisRef = std::static_pointer_cast<AsyncTransactionContext>(shared_from_this());
try {
@ -490,17 +528,17 @@ protected:
scheduler->schedule([thisRef]() { thisRef->handleOnErrorResult(); });
}
void cleanUp() override {
TransactionContextBase::cleanUp();
void cancelPendingFutures() override {
// Cancel all pending operations
// Note that the callbacks of the cancelled futures will still be called
std::unique_lock<std::mutex> lock(mutex);
std::vector<fdb::Future> futures;
for (auto& iter : callbackMap) {
iter.second.cancelled = true;
futures.push_back(iter.second.future);
}
lock.unlock();
for (auto& f : futures) {
f.cancel();
}
@ -520,6 +558,7 @@ protected:
std::shared_ptr<ITransactionContext> thisRef;
bool retryOnError;
TimePoint startTime;
bool cancelled;
};
// Map for keeping track of future waits and holding necessary object references
@ -539,24 +578,66 @@ class TransactionExecutorBase : public ITransactionExecutor {
public:
TransactionExecutorBase(const TransactionExecutorOptions& options) : options(options), scheduler(nullptr) {}
~TransactionExecutorBase() {
if (tamperClusterFileThread.joinable()) {
tamperClusterFileThread.join();
}
}
void init(IScheduler* scheduler, const char* clusterFile, const std::string& bgBasePath) override {
this->scheduler = scheduler;
this->clusterFile = clusterFile;
this->bgBasePath = bgBasePath;
ASSERT(!options.tmpDir.empty());
emptyClusterFile.create(options.tmpDir, "fdbempty.cluster");
invalidClusterFile.create(options.tmpDir, "fdbinvalid.cluster");
invalidClusterFile.write(Random().get().randomStringLowerCase<std::string>(1, 100));
emptyListClusterFile.create(options.tmpDir, "fdbemptylist.cluster");
emptyListClusterFile.write(fmt::format("{}:{}@",
Random().get().randomStringLowerCase<std::string>(3, 8),
Random().get().randomStringLowerCase<std::string>(1, 100)));
if (options.tamperClusterFile) {
tamperedClusterFile.create(options.tmpDir, "fdb.cluster");
originalClusterFile = clusterFile;
this->clusterFile = tamperedClusterFile.getFileName();
// begin with a valid cluster file, but with non existing address
tamperedClusterFile.write(fmt::format("{}:{}@192.168.{}.{}:{}",
Random().get().randomStringLowerCase<std::string>(3, 8),
Random().get().randomStringLowerCase<std::string>(1, 100),
Random().get().randomInt(1, 254),
Random().get().randomInt(1, 254),
Random().get().randomInt(2000, 10000)));
tamperClusterFileThread = std::thread([this]() {
std::this_thread::sleep_for(std::chrono::seconds(2));
// now write an invalid connection string
tamperedClusterFile.write(fmt::format("{}:{}@",
Random().get().randomStringLowerCase<std::string>(3, 8),
Random().get().randomStringLowerCase<std::string>(1, 100)));
std::this_thread::sleep_for(std::chrono::seconds(2));
// finally use correct cluster file contents
std::filesystem::copy_file(std::filesystem::path(originalClusterFile),
std::filesystem::path(tamperedClusterFile.getFileName()),
std::filesystem::copy_options::overwrite_existing);
});
}
}
protected:
// Execute the transaction on the given database instance
void executeOnDatabase(fdb::Database db, std::shared_ptr<ITransactionActor> txActor, TTaskFct cont) {
const TransactionExecutorOptions& getOptions() override { return options; }
void execute(std::shared_ptr<ITransactionActor> txActor, TTaskFct cont) override {
try {
fdb::Transaction tx = db.createTransaction();
std::shared_ptr<ITransactionContext> ctx;
if (options.blockOnFutures) {
ctx = std::make_shared<BlockingTransactionContext>(
tx, txActor, cont, scheduler, options.transactionRetryLimit, bgBasePath);
this, txActor, cont, scheduler, options.transactionRetryLimit, bgBasePath);
} else {
ctx = std::make_shared<AsyncTransactionContext>(
tx, txActor, cont, scheduler, options.transactionRetryLimit, bgBasePath);
this, txActor, cont, scheduler, options.transactionRetryLimit, bgBasePath);
}
txActor->init(ctx);
txActor->start();
@ -566,11 +647,30 @@ protected:
}
}
std::string getClusterFileForErrorInjection() override {
switch (Random::get().randomInt(0, 3)) {
case 0:
return fmt::format("{}{}", "not-existing-file", Random::get().randomStringLowerCase<std::string>(0, 2));
case 1:
return emptyClusterFile.getFileName();
case 2:
return invalidClusterFile.getFileName();
default: // case 3
return emptyListClusterFile.getFileName();
}
}
protected:
TransactionExecutorOptions options;
std::string bgBasePath;
std::string clusterFile;
IScheduler* scheduler;
TmpFile emptyClusterFile;
TmpFile invalidClusterFile;
TmpFile emptyListClusterFile;
TmpFile tamperedClusterFile;
std::thread tamperClusterFileThread;
std::string originalClusterFile;
};
/**
@ -585,19 +685,19 @@ public:
void init(IScheduler* scheduler, const char* clusterFile, const std::string& bgBasePath) override {
TransactionExecutorBase::init(scheduler, clusterFile, bgBasePath);
for (int i = 0; i < options.numDatabases; i++) {
fdb::Database db(clusterFile);
fdb::Database db(this->clusterFile);
databases.push_back(db);
}
}
void execute(std::shared_ptr<ITransactionActor> txActor, TTaskFct cont) override {
fdb::Database selectDatabase() override {
int idx = Random::get().randomInt(0, options.numDatabases - 1);
executeOnDatabase(databases[idx], txActor, cont);
return databases[idx];
}
private:
void release() { databases.clear(); }
private:
std::vector<fdb::Database> databases;
};
@ -608,10 +708,7 @@ class DBPerTransactionExecutor : public TransactionExecutorBase {
public:
DBPerTransactionExecutor(const TransactionExecutorOptions& options) : TransactionExecutorBase(options) {}
void execute(std::shared_ptr<ITransactionActor> txActor, TTaskFct cont) override {
fdb::Database db(clusterFile.c_str());
executeOnDatabase(db, txActor, cont);
}
fdb::Database selectDatabase() override { return fdb::Database(clusterFile.c_str()); }
};
std::unique_ptr<ITransactionExecutor> createTransactionExecutor(const TransactionExecutorOptions& options) {

View File

@ -124,11 +124,24 @@ struct TransactionExecutorOptions {
// Create each transaction in a separate database instance
bool databasePerTransaction = false;
// Enable injection of database create errors
bool injectDatabaseCreateErrors = false;
// Test tampering cluster file contents
bool tamperClusterFile = false;
// The probability of injected database create errors
// Used if injectDatabaseCreateErrors = true
double databaseCreateErrorRatio = 0.1;
// The size of the database instance pool
int numDatabases = 1;
// Maximum number of retries per transaction (0 - unlimited)
int transactionRetryLimit = 0;
// Temporary directory
std::string tmpDir;
};
/**
@ -141,6 +154,9 @@ public:
virtual ~ITransactionExecutor() {}
virtual void init(IScheduler* sched, const char* clusterFile, const std::string& bgBasePath) = 0;
virtual void execute(std::shared_ptr<ITransactionActor> tx, TTaskFct cont) = 0;
virtual fdb::Database selectDatabase() = 0;
virtual std::string getClusterFileForErrorInjection() = 0;
virtual const TransactionExecutorOptions& getOptions() = 0;
};
// Create a transaction executor for the given options

View File

@ -23,6 +23,9 @@
#include <algorithm>
#include <ctype.h>
#include <chrono>
#include <filesystem>
#include <fstream>
#include <string>
namespace FdbApiTester {
@ -46,16 +49,6 @@ Random& Random::get() {
return random;
}
fdb::ByteString Random::randomStringLowerCase(int minLength, int maxLength) {
int length = randomInt(minLength, maxLength);
fdb::ByteString str;
str.reserve(length);
for (int i = 0; i < length; i++) {
str += (char)randomInt('a', 'z');
}
return str;
}
bool Random::randomBool(double trueRatio) {
return std::uniform_real_distribution<double>(0.0, 1.0)(random) <= trueRatio;
}
@ -119,4 +112,39 @@ GranuleSummaryArray copyGranuleSummaryArray(fdb::future_var::GranuleSummaryRefAr
return out;
};
TmpFile::~TmpFile() {
if (!filename.empty()) {
remove();
}
}
void TmpFile::create(std::string_view dir, std::string_view prefix) {
while (true) {
filename = fmt::format("{}/{}-{}", dir, prefix, Random::get().randomStringLowerCase<std::string>(6, 6));
if (!std::filesystem::exists(std::filesystem::path(filename))) {
break;
}
}
// Create an empty tmp file
std::fstream tmpFile(filename, std::fstream::out);
if (!tmpFile.good()) {
throw TesterError(fmt::format("Failed to create temporary file {}\n", filename));
}
}
void TmpFile::write(std::string_view data) {
std::ofstream ofs(filename, std::fstream::out | std::fstream::binary);
if (!ofs.good()) {
throw TesterError(fmt::format("Failed to write to the temporary file {}\n", filename));
}
ofs.write(data.data(), data.size());
}
void TmpFile::remove() {
if (!std::filesystem::remove(std::filesystem::path(filename))) {
fmt::print(stderr, "Failed to remove file {}\n", filename);
}
}
} // namespace FdbApiTester

View File

@ -66,7 +66,20 @@ public:
int randomInt(int min, int max);
fdb::ByteString randomStringLowerCase(int minLength, int maxLength);
template <class StringType>
StringType randomStringLowerCase(int minLength, int maxLength) {
int length = randomInt(minLength, maxLength);
StringType str;
str.reserve(length);
for (int i = 0; i < length; i++) {
str += (char)randomInt('a', 'z');
}
return str;
}
fdb::ByteString randomByteStringLowerCase(int minLength, int maxLength) {
return randomStringLowerCase<fdb::ByteString>(minLength, maxLength);
}
bool randomBool(double trueRatio);
@ -142,6 +155,19 @@ static fdb::ByteString toByteString(T value) {
return output;
}
// Creates a temporary file; file gets destroyed/deleted along with object destruction.
struct TmpFile {
public:
~TmpFile();
void create(std::string_view dir, std::string_view prefix);
void write(std::string_view data);
void remove();
const std::string& getFileName() const { return filename; }
private:
std::string filename;
};
} // namespace FdbApiTester
#endif

View File

@ -358,7 +358,11 @@ bool runWorkloads(TesterOptions& options) {
txExecOptions.blockOnFutures = options.testSpec.blockOnFutures;
txExecOptions.numDatabases = options.numDatabases;
txExecOptions.databasePerTransaction = options.testSpec.databasePerTransaction;
// 7.1 and older releases crash on database create errors
txExecOptions.injectDatabaseCreateErrors = options.testSpec.buggify && options.apiVersion > 710;
txExecOptions.transactionRetryLimit = options.transactionRetryLimit;
txExecOptions.tmpDir = options.tmpDir.empty() ? std::string("/tmp") : options.tmpDir;
txExecOptions.tamperClusterFile = options.testSpec.tamperClusterFile;
std::vector<std::shared_ptr<IWorkload>> workloads;
workloads.reserve(options.testSpec.workloads.size() * options.numClients);

View File

@ -0,0 +1,24 @@
[[test]]
title = 'Test tampering the cluster file'
multiThreaded = true
buggify = true
tamperClusterFile = true
minFdbThreads = 2
maxFdbThreads = 4
minDatabases = 2
maxDatabases = 4
minClientThreads = 2
maxClientThreads = 4
minClients = 2
maxClients = 4
[[test.workload]]
name = 'ApiCorrectness'
minKeyLength = 1
maxKeyLength = 64
minValueLength = 1
maxValueLength = 1000
maxKeysPerTransaction = 50
initialSize = 100
numRandomOperations = 100
readExistingKeysRatio = 0.9

View File

@ -505,6 +505,14 @@ public:
Transaction(const Transaction&) noexcept = default;
Transaction& operator=(const Transaction&) noexcept = default;
void atomic_store(Transaction other) { std::atomic_store(&tr, other.tr); }
Transaction atomic_load() {
Transaction retVal;
retVal.tr = std::atomic_load(&tr);
return retVal;
}
bool valid() const noexcept { return tr != nullptr; }
explicit operator bool() const noexcept { return valid(); }
@ -666,6 +674,7 @@ public:
static void createTenant(Transaction tr, BytesRef name) {
tr.setOption(FDBTransactionOption::FDB_TR_OPTION_SPECIAL_KEY_SPACE_ENABLE_WRITES, BytesRef());
tr.setOption(FDBTransactionOption::FDB_TR_OPTION_LOCK_AWARE, BytesRef());
tr.setOption(FDBTransactionOption::FDB_TR_OPTION_RAW_ACCESS, BytesRef());
tr.set(toBytesRef(fmt::format("{}{}", tenantManagementMapPrefix, toCharsRef(name))), BytesRef());
}

View File

@ -191,11 +191,6 @@ public class FDB {
Select_API_version(version);
singleton = new FDB(version);
if (version < 720) {
TenantManagement.TENANT_MAP_PREFIX = ByteArrayUtil.join(new byte[] { (byte)255, (byte)255 },
"/management/tenant_map/".getBytes());
}
return singleton;
}

View File

@ -52,6 +52,10 @@ public class TenantManagement {
* @param tenantName The name of the tenant. Can be any byte string that does not begin a 0xFF byte.
*/
public static void createTenant(Transaction tr, byte[] tenantName) {
if (FDB.instance().getAPIVersion() < 720) {
tr.options().setSpecialKeySpaceRelaxed();
}
tr.options().setSpecialKeySpaceEnableWrites();
tr.set(ByteArrayUtil.join(TENANT_MAP_PREFIX, tenantName), new byte[0]);
}
@ -86,6 +90,7 @@ public class TenantManagement {
final AtomicBoolean checkedExistence = new AtomicBoolean(false);
final byte[] key = ByteArrayUtil.join(TENANT_MAP_PREFIX, tenantName);
return db.runAsync(tr -> {
tr.options().setSpecialKeySpaceRelaxed();
tr.options().setSpecialKeySpaceEnableWrites();
if(checkedExistence.get()) {
tr.set(key, new byte[0]);
@ -133,6 +138,10 @@ public class TenantManagement {
* @param tenantName The name of the tenant being deleted.
*/
public static void deleteTenant(Transaction tr, byte[] tenantName) {
if (FDB.instance().getAPIVersion() < 720) {
tr.options().setSpecialKeySpaceRelaxed();
}
tr.options().setSpecialKeySpaceEnableWrites();
tr.clear(ByteArrayUtil.join(TENANT_MAP_PREFIX, tenantName));
}
@ -173,6 +182,7 @@ public class TenantManagement {
final AtomicBoolean checkedExistence = new AtomicBoolean(false);
final byte[] key = ByteArrayUtil.join(TENANT_MAP_PREFIX, tenantName);
return db.runAsync(tr -> {
tr.options().setSpecialKeySpaceRelaxed();
tr.options().setSpecialKeySpaceEnableWrites();
if(checkedExistence.get()) {
tr.clear(key);
@ -238,7 +248,12 @@ public class TenantManagement {
* and the value is the unprocessed JSON string containing the tenant's metadata
*/
public static CloseableAsyncIterator<KeyValue> listTenants(Database db, Tuple begin, Tuple end, int limit) {
return listTenants_internal(db.createTransaction(), begin.pack(), end.pack(), limit);
Transaction tr = db.createTransaction();
if (FDB.instance().getAPIVersion() < 720) {
tr.options().setSpecialKeySpaceRelaxed();
}
return listTenants_internal(tr, begin.pack(), end.pack(), limit);
}
private static CloseableAsyncIterator<KeyValue> listTenants_internal(Transaction tr, byte[] begin, byte[] end,
@ -262,7 +277,7 @@ public class TenantManagement {
this.begin = ByteArrayUtil.join(TENANT_MAP_PREFIX, begin);
this.end = ByteArrayUtil.join(TENANT_MAP_PREFIX, end);
tr.options().setReadSystemKeys();
tr.options().setRawAccess();
tr.options().setLockAware();
firstGet = tr.getRange(this.begin, this.end, limit);

View File

@ -100,10 +100,8 @@ def api_version(ver):
_add_symbols(fdb.impl, list)
if ver >= 710:
if ver >= 630:
import fdb.tenant_management
if ver < 720:
fdb.tenant_management._tenant_map_prefix = b'\xff\xff/management/tenant_map/'
if ver < 610:
globals()["init"] = getattr(fdb.impl, "init")

View File

@ -23,6 +23,7 @@
"""Documentation for this API can be found at
https://apple.github.io/foundationdb/api-python.html"""
import fdb
from fdb import impl as _impl
_tenant_map_prefix = b'\xff\xff/management/tenant/map/'
@ -52,6 +53,9 @@ def _check_tenant_existence(tr, key, existence_check_marker, force_maybe_commite
# If the existence_check_marker is a non-empty list, then the existence check is skipped.
@_impl.transactional
def _create_tenant_impl(tr, tenant_name, existence_check_marker, force_existence_check_maybe_committed=False):
if fdb._version < 720:
tr.options.set_special_key_space_relaxed()
tr.options.set_special_key_space_enable_writes()
key = b'%s%s' % (_tenant_map_prefix, tenant_name)
@ -70,6 +74,9 @@ def _create_tenant_impl(tr, tenant_name, existence_check_marker, force_existence
# If the existence_check_marker is a non-empty list, then the existence check is skipped.
@_impl.transactional
def _delete_tenant_impl(tr, tenant_name, existence_check_marker, force_existence_check_maybe_committed=False):
if fdb._version < 720:
tr.options.set_special_key_space_relaxed()
tr.options.set_special_key_space_enable_writes()
key = b'%s%s' % (_tenant_map_prefix, tenant_name)
@ -103,7 +110,10 @@ class FDBTenantList(object):
# JSON strings of the tenant metadata
@_impl.transactional
def _list_tenants_impl(tr, begin, end, limit):
tr.options.set_read_system_keys()
if fdb._version < 720:
tr.options.set_special_key_space_relaxed()
tr.options.set_raw_access()
begin_key = b'%s%s' % (_tenant_map_prefix, begin)
end_key = b'%s%s' % (_tenant_map_prefix, end)

View File

@ -200,6 +200,9 @@ else()
URL "https://github.com/ToruNiina/toml11/archive/v3.4.0.tar.gz"
URL_HASH SHA256=bc6d733efd9216af8c119d8ac64a805578c79cc82b813e4d1d880ca128bd154d
CMAKE_CACHE_ARGS
-DCMAKE_BUILD_TYPE:STRING=${CMAKE_BUILD_TYPE}
-DCMAKE_C_COMPILER:FILEPATH=${CMAKE_C_COMPILER}
-DCMAKE_CXX_COMPILER:FILEPATH=${CMAKE_CXX_COMPILER}
-DCMAKE_INSTALL_PREFIX:PATH=${CMAKE_CURRENT_BINARY_DIR}/toml11
-Dtoml11_BUILD_TEST:BOOL=OFF
BUILD_ALWAYS ON)

View File

@ -14,7 +14,7 @@ ExternalProject_add(Jemalloc_project
BUILD_BYPRODUCTS "${JEMALLOC_DIR}/include/jemalloc/jemalloc.h"
"${JEMALLOC_DIR}/lib/libjemalloc.a"
"${JEMALLOC_DIR}/lib/libjemalloc_pic.a"
CONFIGURE_COMMAND ./configure --prefix=${JEMALLOC_DIR} --enable-static --disable-cxx --enable-prof
CONFIGURE_COMMAND CC=${CMAKE_C_COMPILER} CXX=${CMAKE_CXX_COMPILER} ./configure --prefix=${JEMALLOC_DIR} --enable-static --disable-cxx --enable-prof
BUILD_IN_SOURCE ON
BUILD_COMMAND make
INSTALL_DIR "${JEMALLOC_DIR}"
@ -24,4 +24,4 @@ add_dependencies(im_jemalloc_pic Jemalloc_project)
set_target_properties(im_jemalloc_pic PROPERTIES IMPORTED_LOCATION "${JEMALLOC_DIR}/lib/libjemalloc_pic.a")
set_target_properties(im_jemalloc PROPERTIES IMPORTED_LOCATION "${JEMALLOC_DIR}/lib/libjemalloc.a")
target_include_directories(jemalloc INTERFACE "${JEMALLOC_DIR}/include")
target_link_libraries(jemalloc INTERFACE im_jemalloc_pic im_jemalloc)
target_link_libraries(jemalloc INTERFACE im_jemalloc_pic im_jemalloc)

View File

@ -27,13 +27,10 @@ Each special key that existed before api version 630 is its own module. These ar
Prior to api version 630, it was also possible to read a range starting at ``\xff\xff/worker_interfaces``. This is mostly an implementation detail of fdbcli,
but it's available in api version 630 as a module with prefix ``\xff\xff/worker_interfaces/``.
Api version 630 includes two new modules:
Api version 630 includes three new modules:
#. ``\xff\xff/transaction/`` - information about the current transaction
#. ``\xff\xff/metrics/`` - various metrics, not transactional
Api version 720 includes one new module:
#. ``\xff\xff/clusterId`` - returns an immutable unique ID for a cluster
Transaction module
@ -279,7 +276,6 @@ Deprecated Keys
Listed below are the special keys that have been deprecated. Special key(s) will no longer be accessible when the client specifies an API version equal to or larger than the version where they were deprecated. Clients specifying older API versions will be able to continue using the deprecated key(s).
#. ``\xff\xff/management/profiling/<client_txn_sample_rate|client_txn_size_limit>`` Deprecated as of API version 720. The corresponding functionalities are now covered by the global configuration module. For details, see :doc:`global-configuration`. Read/write. Changing these two keys will change the corresponding system keys ``\xff\x02/fdbClientInfo/<client_txn_sample_rate|client_txn_size_limit>``, respectively. The value of ``\xff\xff/management/client_txn_sample_rate`` is a literal text of ``double``, and the value of ``\xff\xff/management/client_txn_size_limit`` is a literal text of ``int64_t``. A special value ``default`` can be set to or read from these two keys, representing the client profiling is disabled. In addition, ``clear`` in this range is not allowed. For more details, see help text of ``fdbcli`` command ``profile client``.
#. ``\xff\xff/management/tenant_map/<tenant>`` Removed as of API version 720 and renamed to ``\xff\xff/management/tenant/map/<tenant>``.
Versioning
==========

View File

@ -127,19 +127,24 @@ ACTOR Future<bool> blobRangeCommandActor(Database localDb,
}
fmt::print("{0} blobbify range for [{1} - {2})\n",
starting ? "Starting" : "Stopping",
tokens[2].printable().c_str(),
tokens[3].printable().c_str());
tokens[2].printable(),
tokens[3].printable());
state bool success = false;
if (starting) {
wait(store(success, localDb->blobbifyRange(KeyRangeRef(begin, end))));
} else {
wait(store(success, localDb->unblobbifyRange(KeyRangeRef(begin, end))));
}
if (!success) {
if (success) {
fmt::print("{0} updated blob range [{1} - {2}) succeeded\n",
starting ? "Starting" : "Stopping",
tokens[2].printable(),
tokens[3].printable());
} else {
fmt::print("{0} blobbify range for [{1} - {2}) failed\n",
starting ? "Starting" : "Stopping",
tokens[2].printable().c_str(),
tokens[3].printable().c_str());
tokens[2].printable(),
tokens[3].printable());
}
return success;
} else if (tokencmp(tokens[1], "purge") || tokencmp(tokens[1], "forcepurge") || tokencmp(tokens[1], "check")) {

View File

@ -3,6 +3,13 @@ fdb_find_sources(FDBCLI_SRCS)
add_flow_target(EXECUTABLE NAME fdbcli SRCS ${FDBCLI_SRCS})
target_include_directories(fdbcli PRIVATE "${CMAKE_CURRENT_SOURCE_DIR}/include" "${CMAKE_CURRENT_BINARY_DIR}/include")
target_link_libraries(fdbcli PRIVATE fdbclient SimpleOpt)
if (USE_UBSAN)
# The intent is to put typeinfo symbols in the dynamic symbol table so that
# the types in fdbcli and external libfdb_c clients agree for ubsan's vptr
# check. This would not be a good idea for the normal build, or if we ever
# start testing old libfdb_c's that are ubsan-instrumented.
target_link_options(fdbcli PRIVATE "-rdynamic")
endif()
if(NOT WIN32)
target_link_libraries(fdbcli PRIVATE linenoise)

View File

@ -259,7 +259,6 @@ ACTOR Future<bool> configureCommandActor(Reference<IDatabase> db,
fprintf(stderr,
"Type `configure perpetual_storage_wiggle=1' to enable the perpetual wiggle, or `configure "
"storage_migration_type=gradual' to set the gradual migration type.\n");
ret = false;
break;
case ConfigurationResult::SUCCESS_WARN_ROCKSDB_EXPERIMENTAL:
printf("Configuration changed\n");

View File

@ -36,24 +36,12 @@
namespace fdb_cli {
const KeyRangeRef tenantMapSpecialKeyRange720("\xff\xff/management/tenant/map/"_sr,
"\xff\xff/management/tenant/map0"_sr);
const KeyRangeRef tenantMapSpecialKeyRange("\xff\xff/management/tenant/map/"_sr, "\xff\xff/management/tenant/map0"_sr);
const KeyRangeRef tenantConfigSpecialKeyRange("\xff\xff/management/tenant/configure/"_sr,
"\xff\xff/management/tenant/configure0"_sr);
const KeyRangeRef tenantRenameSpecialKeyRange("\xff\xff/management/tenant/rename/"_sr,
"\xff\xff/management/tenant/rename0"_sr);
const KeyRangeRef tenantMapSpecialKeyRange710("\xff\xff/management/tenant_map/"_sr,
"\xff\xff/management/tenant_map0"_sr);
KeyRangeRef const& tenantMapSpecialKeyRange(int apiVersion) {
if (apiVersion >= 720) {
return tenantMapSpecialKeyRange720;
} else {
return tenantMapSpecialKeyRange710;
}
}
Optional<std::map<Standalone<StringRef>, Optional<Value>>>
parseTenantConfiguration(std::vector<StringRef> const& tokens, int startIndex, bool allowUnset) {
std::map<Standalone<StringRef>, Optional<Value>> configParams;
@ -114,13 +102,13 @@ void applyConfigurationToSpecialKeys(Reference<ITransaction> tr,
}
// createtenant command
ACTOR Future<bool> createTenantCommandActor(Reference<IDatabase> db, std::vector<StringRef> tokens, int apiVersion) {
ACTOR Future<bool> createTenantCommandActor(Reference<IDatabase> db, std::vector<StringRef> tokens) {
if (tokens.size() < 2 || tokens.size() > 3) {
printUsage(tokens[0]);
return false;
}
state Key tenantNameKey = tenantMapSpecialKeyRange(apiVersion).begin.withSuffix(tokens[1]);
state Key tenantNameKey = tenantMapSpecialKeyRange.begin.withSuffix(tokens[1]);
state Reference<ITransaction> tr = db->createTransaction();
state bool doneExistenceCheck = false;
@ -131,11 +119,6 @@ ACTOR Future<bool> createTenantCommandActor(Reference<IDatabase> db, std::vector
return false;
}
if (apiVersion < 720 && !configuration.get().empty()) {
fmt::print(stderr, "ERROR: tenants do not accept configuration options before API version 720.\n");
return false;
}
loop {
try {
tr->setOption(FDBTransactionOptions::SPECIAL_KEY_SPACE_ENABLE_WRITES);
@ -187,13 +170,13 @@ CommandFactory createTenantFactory(
"that will require this tenant to be placed on the same cluster as other tenants in the same group."));
// deletetenant command
ACTOR Future<bool> deleteTenantCommandActor(Reference<IDatabase> db, std::vector<StringRef> tokens, int apiVersion) {
ACTOR Future<bool> deleteTenantCommandActor(Reference<IDatabase> db, std::vector<StringRef> tokens) {
if (tokens.size() != 2) {
printUsage(tokens[0]);
return false;
}
state Key tenantNameKey = tenantMapSpecialKeyRange(apiVersion).begin.withSuffix(tokens[1]);
state Key tenantNameKey = tenantMapSpecialKeyRange.begin.withSuffix(tokens[1]);
state Reference<ITransaction> tr = db->createTransaction();
state bool doneExistenceCheck = false;
@ -243,7 +226,7 @@ CommandFactory deleteTenantFactory(
"Deletes a tenant from the cluster. Deletion will be allowed only if the specified tenant contains no data."));
// listtenants command
ACTOR Future<bool> listTenantsCommandActor(Reference<IDatabase> db, std::vector<StringRef> tokens, int apiVersion) {
ACTOR Future<bool> listTenantsCommandActor(Reference<IDatabase> db, std::vector<StringRef> tokens) {
if (tokens.size() > 4) {
printUsage(tokens[0]);
return false;
@ -271,8 +254,8 @@ ACTOR Future<bool> listTenantsCommandActor(Reference<IDatabase> db, std::vector<
}
}
state Key beginTenantKey = tenantMapSpecialKeyRange(apiVersion).begin.withSuffix(beginTenant);
state Key endTenantKey = tenantMapSpecialKeyRange(apiVersion).begin.withSuffix(endTenant);
state Key beginTenantKey = tenantMapSpecialKeyRange.begin.withSuffix(beginTenant);
state Key endTenantKey = tenantMapSpecialKeyRange.begin.withSuffix(endTenant);
state Reference<ITransaction> tr = db->createTransaction();
loop {
@ -292,7 +275,7 @@ ACTOR Future<bool> listTenantsCommandActor(Reference<IDatabase> db, std::vector<
tr->getRange(firstGreaterOrEqual(beginTenantKey), firstGreaterOrEqual(endTenantKey), limit);
RangeResult tenants = wait(safeThreadFutureToFuture(kvsFuture));
for (auto tenant : tenants) {
tenantNames.push_back(tenant.key.removePrefix(tenantMapSpecialKeyRange(apiVersion).begin));
tenantNames.push_back(tenant.key.removePrefix(tenantMapSpecialKeyRange.begin));
}
}
@ -330,14 +313,14 @@ CommandFactory listTenantsFactory(
"The number of tenants to print can be specified using the [LIMIT] parameter, which defaults to 100."));
// gettenant command
ACTOR Future<bool> getTenantCommandActor(Reference<IDatabase> db, std::vector<StringRef> tokens, int apiVersion) {
ACTOR Future<bool> getTenantCommandActor(Reference<IDatabase> db, std::vector<StringRef> tokens) {
if (tokens.size() < 2 || tokens.size() > 3 || (tokens.size() == 3 && tokens[2] != "JSON"_sr)) {
printUsage(tokens[0]);
return false;
}
state bool useJson = tokens.size() == 3;
state Key tenantNameKey = tenantMapSpecialKeyRange(apiVersion).begin.withSuffix(tokens[1]);
state Key tenantNameKey = tenantMapSpecialKeyRange.begin.withSuffix(tokens[1]);
state Reference<ITransaction> tr = db->createTransaction();
loop {
@ -347,7 +330,7 @@ ACTOR Future<bool> getTenantCommandActor(Reference<IDatabase> db, std::vector<St
state std::string tenantJson;
if (clusterType == ClusterType::METACLUSTER_MANAGEMENT) {
TenantMapEntry entry = wait(MetaclusterAPI::getTenantTransaction(tr, tokens[1]));
tenantJson = entry.toJson(apiVersion);
tenantJson = entry.toJson();
} else {
// Hold the reference to the standalone's memory
state ThreadFuture<Optional<Value>> tenantFuture = tr->get(tenantNameKey);
@ -378,12 +361,7 @@ ACTOR Future<bool> getTenantCommandActor(Reference<IDatabase> db, std::vector<St
std::string assignedCluster;
doc.get("id", id);
if (apiVersion >= 720) {
doc.get("prefix.printable", prefix);
} else {
doc.get("prefix", prefix);
}
doc.get("prefix.printable", prefix);
doc.get("tenant_state", tenantState);
bool hasTenantGroup = doc.tryGet("tenant_group.printable", tenantGroup);
@ -499,15 +477,15 @@ int64_t getTenantId(Value metadata) {
}
// renametenant command
ACTOR Future<bool> renameTenantCommandActor(Reference<IDatabase> db, std::vector<StringRef> tokens, int apiVersion) {
ACTOR Future<bool> renameTenantCommandActor(Reference<IDatabase> db, std::vector<StringRef> tokens) {
if (tokens.size() != 3) {
printUsage(tokens[0]);
return false;
}
state Reference<ITransaction> tr = db->createTransaction();
state Key tenantRenameKey = tenantRenameSpecialKeyRange.begin.withSuffix(tokens[1]);
state Key tenantOldNameKey = tenantMapSpecialKeyRange(apiVersion).begin.withSuffix(tokens[1]);
state Key tenantNewNameKey = tenantMapSpecialKeyRange(apiVersion).begin.withSuffix(tokens[2]);
state Key tenantOldNameKey = tenantMapSpecialKeyRange.begin.withSuffix(tokens[1]);
state Key tenantNewNameKey = tenantMapSpecialKeyRange.begin.withSuffix(tokens[2]);
state bool firstTry = true;
state int64_t id = -1;
loop {

View File

@ -1875,14 +1875,14 @@ ACTOR Future<int> cli(CLIOptions opt, LineNoise* plinenoise, Reference<ClusterCo
}
if (tokencmp(tokens[0], "createtenant")) {
bool _result = wait(makeInterruptable(createTenantCommandActor(db, tokens, opt.apiVersion)));
bool _result = wait(makeInterruptable(createTenantCommandActor(db, tokens)));
if (!_result)
is_error = true;
continue;
}
if (tokencmp(tokens[0], "deletetenant")) {
bool _result = wait(makeInterruptable(deleteTenantCommandActor(db, tokens, opt.apiVersion)));
bool _result = wait(makeInterruptable(deleteTenantCommandActor(db, tokens)));
if (!_result)
is_error = true;
else if (tenantName.present() && tokens[1] == tenantName.get()) {
@ -1894,26 +1894,20 @@ ACTOR Future<int> cli(CLIOptions opt, LineNoise* plinenoise, Reference<ClusterCo
}
if (tokencmp(tokens[0], "listtenants")) {
bool _result = wait(makeInterruptable(listTenantsCommandActor(db, tokens, opt.apiVersion)));
bool _result = wait(makeInterruptable(listTenantsCommandActor(db, tokens)));
if (!_result)
is_error = true;
continue;
}
if (tokencmp(tokens[0], "gettenant")) {
bool _result = wait(makeInterruptable(getTenantCommandActor(db, tokens, opt.apiVersion)));
bool _result = wait(makeInterruptable(getTenantCommandActor(db, tokens)));
if (!_result)
is_error = true;
continue;
}
if (tokencmp(tokens[0], "configuretenant")) {
if (opt.apiVersion < 720) {
fmt::print(stderr, "ERROR: tenants cannot be configured before API version 720.\n");
is_error = true;
continue;
}
bool _result = wait(makeInterruptable(configureTenantCommandActor(db, tokens)));
if (!_result)
is_error = true;
@ -1921,13 +1915,7 @@ ACTOR Future<int> cli(CLIOptions opt, LineNoise* plinenoise, Reference<ClusterCo
}
if (tokencmp(tokens[0], "renametenant")) {
if (opt.apiVersion < 720) {
fmt::print(stderr, "ERROR: tenants cannot be renamed before API version 720.\n");
is_error = true;
continue;
}
bool _result = wait(makeInterruptable(renameTenantCommandActor(db, tokens, opt.apiVersion)));
bool _result = wait(makeInterruptable(renameTenantCommandActor(db, tokens)));
if (!_result)
is_error = true;
continue;

View File

@ -168,11 +168,11 @@ ACTOR Future<bool> consistencyCheckCommandActor(Reference<ITransaction> tr,
// coordinators command
ACTOR Future<bool> coordinatorsCommandActor(Reference<IDatabase> db, std::vector<StringRef> tokens);
// createtenant command
ACTOR Future<bool> createTenantCommandActor(Reference<IDatabase> db, std::vector<StringRef> tokens, int apiVersion);
ACTOR Future<bool> createTenantCommandActor(Reference<IDatabase> db, std::vector<StringRef> tokens);
// datadistribution command
ACTOR Future<bool> dataDistributionCommandActor(Reference<IDatabase> db, std::vector<StringRef> tokens);
// deletetenant command
ACTOR Future<bool> deleteTenantCommandActor(Reference<IDatabase> db, std::vector<StringRef> tokens, int apiVersion);
ACTOR Future<bool> deleteTenantCommandActor(Reference<IDatabase> db, std::vector<StringRef> tokens);
// exclude command
ACTOR Future<bool> excludeCommandActor(Reference<IDatabase> db, std::vector<StringRef> tokens, Future<Void> warn);
// expensive_data_check command
@ -189,7 +189,7 @@ ACTOR Future<bool> fileConfigureCommandActor(Reference<IDatabase> db,
// force_recovery_with_data_loss command
ACTOR Future<bool> forceRecoveryWithDataLossCommandActor(Reference<IDatabase> db, std::vector<StringRef> tokens);
// gettenant command
ACTOR Future<bool> getTenantCommandActor(Reference<IDatabase> db, std::vector<StringRef> tokens, int apiVersion);
ACTOR Future<bool> getTenantCommandActor(Reference<IDatabase> db, std::vector<StringRef> tokens);
// include command
ACTOR Future<bool> includeCommandActor(Reference<IDatabase> db, std::vector<StringRef> tokens);
// kill command
@ -198,7 +198,7 @@ ACTOR Future<bool> killCommandActor(Reference<IDatabase> db,
std::vector<StringRef> tokens,
std::map<Key, std::pair<Value, ClientLeaderRegInterface>>* address_interface);
// listtenants command
ACTOR Future<bool> listTenantsCommandActor(Reference<IDatabase> db, std::vector<StringRef> tokens, int apiVersion);
ACTOR Future<bool> listTenantsCommandActor(Reference<IDatabase> db, std::vector<StringRef> tokens);
// lock/unlock command
ACTOR Future<bool> lockCommandActor(Reference<IDatabase> db, std::vector<StringRef> tokens);
ACTOR Future<bool> unlockDatabaseActor(Reference<IDatabase> db, UID uid);
@ -227,7 +227,7 @@ ACTOR Future<bool> profileCommandActor(Database db,
std::vector<StringRef> tokens,
bool intrans);
// renametenant command
ACTOR Future<bool> renameTenantCommandActor(Reference<IDatabase> db, std::vector<StringRef> tokens, int apiVersion);
ACTOR Future<bool> renameTenantCommandActor(Reference<IDatabase> db, std::vector<StringRef> tokens);
// quota command
ACTOR Future<bool> quotaCommandActor(Reference<IDatabase> db, std::vector<StringRef> tokens);
// setclass command

View File

@ -18,8 +18,9 @@
* limitations under the License.
*/
#include "flow/BlobCipher.h"
#include "fdbclient/BlobCipher.h"
#include "fdbclient/Knobs.h"
#include "flow/Arena.h"
#include "flow/EncryptUtils.h"
#include "flow/Knobs.h"
@ -27,6 +28,7 @@
#include "flow/FastRef.h"
#include "flow/IRandom.h"
#include "flow/ITrace.h"
#include "flow/Platform.h"
#include "flow/flow.h"
#include "flow/network.h"
#include "flow/Trace.h"
@ -54,6 +56,42 @@ bool isEncryptHeaderAuthTokenModeValid(const EncryptAuthTokenMode mode) {
}
} // namespace
// BlobCipherMetrics methods
BlobCipherMetrics::CounterSet::CounterSet(CounterCollection& cc, std::string name)
: encryptCPUTimeNS(name + "EncryptCPUTimeNS", cc), decryptCPUTimeNS(name + "DecryptCPUTimeNS", cc),
getCipherKeysLatency(name + "GetCipherKeysLatency",
UID(),
FLOW_KNOBS->ENCRYPT_KEY_CACHE_LOGGING_INTERVAL,
FLOW_KNOBS->ENCRYPT_KEY_CACHE_LOGGING_SAMPLE_SIZE),
getLatestCipherKeysLatency(name + "GetLatestCipherKeysLatency",
UID(),
FLOW_KNOBS->ENCRYPT_KEY_CACHE_LOGGING_INTERVAL,
FLOW_KNOBS->ENCRYPT_KEY_CACHE_LOGGING_SAMPLE_SIZE) {}
BlobCipherMetrics::BlobCipherMetrics()
: cc("BlobCipher"), cipherKeyCacheHit("CipherKeyCacheHit", cc), cipherKeyCacheMiss("CipherKeyCacheMiss", cc),
cipherKeyCacheExpired("CipherKeyCacheExpired", cc), latestCipherKeyCacheHit("LatestCipherKeyCacheHit", cc),
latestCipherKeyCacheMiss("LatestCipherKeyCacheMiss", cc),
latestCipherKeyCacheNeedsRefresh("LatestCipherKeyCacheNeedsRefresh", cc),
getCipherKeysLatency("GetCipherKeysLatency",
UID(),
FLOW_KNOBS->ENCRYPT_KEY_CACHE_LOGGING_INTERVAL,
FLOW_KNOBS->ENCRYPT_KEY_CACHE_LOGGING_SAMPLE_SIZE),
getLatestCipherKeysLatency("GetLatestCipherKeysLatency",
UID(),
FLOW_KNOBS->ENCRYPT_KEY_CACHE_LOGGING_INTERVAL,
FLOW_KNOBS->ENCRYPT_KEY_CACHE_LOGGING_SAMPLE_SIZE),
counterSets({ CounterSet(cc, "TLog"),
CounterSet(cc, "KVMemory"),
CounterSet(cc, "KVRedwood"),
CounterSet(cc, "BlobGranule"),
CounterSet(cc, "Backup"),
CounterSet(cc, "Test") }) {
specialCounter(cc, "CacheSize", []() { return BlobCipherKeyCache::getInstance()->getSize(); });
traceFuture = traceCounters("BlobCipherMetrics", UID(), FLOW_KNOBS->ENCRYPT_KEY_CACHE_LOGGING_INTERVAL, &cc);
}
// BlobCipherKey class methods
BlobCipherKey::BlobCipherKey(const EncryptCipherDomainId& domainId,
@ -144,11 +182,9 @@ void BlobCipherKey::reset() {
// BlobKeyIdCache class methods
BlobCipherKeyIdCache::BlobCipherKeyIdCache()
: domainId(ENCRYPT_INVALID_DOMAIN_ID), latestBaseCipherKeyId(), latestRandomSalt() {}
BlobCipherKeyIdCache::BlobCipherKeyIdCache(EncryptCipherDomainId dId)
: domainId(dId), latestBaseCipherKeyId(), latestRandomSalt() {
BlobCipherKeyIdCache::BlobCipherKeyIdCache(EncryptCipherDomainId dId, size_t* sizeStat)
: domainId(dId), latestBaseCipherKeyId(), latestRandomSalt(), sizeStat(sizeStat) {
ASSERT(sizeStat != nullptr);
TraceEvent(SevInfo, "BlobCipher.KeyIdCacheInit").detail("DomainId", domainId);
}
@ -223,6 +259,7 @@ Reference<BlobCipherKey> BlobCipherKeyIdCache::insertBaseCipherKey(const Encrypt
latestBaseCipherKeyId = baseCipherId;
latestRandomSalt = cipherKey->getSalt();
(*sizeStat)++;
return cipherKey;
}
@ -267,6 +304,7 @@ Reference<BlobCipherKey> BlobCipherKeyIdCache::insertBaseCipherKey(const Encrypt
Reference<BlobCipherKey> cipherKey =
makeReference<BlobCipherKey>(domainId, baseCipherId, baseCipher, baseCipherLen, salt, refreshAt, expireAt);
keyIdCache.emplace(cacheKey, cipherKey);
(*sizeStat)++;
return cipherKey;
}
@ -298,19 +336,19 @@ Reference<BlobCipherKey> BlobCipherKeyCache::insertCipherKey(const EncryptCipher
throw encrypt_invalid_id();
}
Reference<BlobCipherKey> cipherKey;
try {
auto domainItr = domainCacheMap.find(domainId);
if (domainItr == domainCacheMap.end()) {
// Add mapping to track new encryption domain
Reference<BlobCipherKeyIdCache> keyIdCache = makeReference<BlobCipherKeyIdCache>(domainId);
Reference<BlobCipherKey> cipherKey =
keyIdCache->insertBaseCipherKey(baseCipherId, baseCipher, baseCipherLen, refreshAt, expireAt);
Reference<BlobCipherKeyIdCache> keyIdCache = makeReference<BlobCipherKeyIdCache>(domainId, &size);
cipherKey = keyIdCache->insertBaseCipherKey(baseCipherId, baseCipher, baseCipherLen, refreshAt, expireAt);
domainCacheMap.emplace(domainId, keyIdCache);
return cipherKey;
} else {
// Track new baseCipher keys
Reference<BlobCipherKeyIdCache> keyIdCache = domainItr->second;
return keyIdCache->insertBaseCipherKey(baseCipherId, baseCipher, baseCipherLen, refreshAt, expireAt);
cipherKey = keyIdCache->insertBaseCipherKey(baseCipherId, baseCipher, baseCipherLen, refreshAt, expireAt);
}
} catch (Error& e) {
TraceEvent(SevWarn, "BlobCipher.InsertCipherKeyFailed")
@ -318,6 +356,7 @@ Reference<BlobCipherKey> BlobCipherKeyCache::insertCipherKey(const EncryptCipher
.detail("DomainId", domainId);
throw;
}
return cipherKey;
}
Reference<BlobCipherKey> BlobCipherKeyCache::insertCipherKey(const EncryptCipherDomainId& domainId,
@ -337,7 +376,7 @@ Reference<BlobCipherKey> BlobCipherKeyCache::insertCipherKey(const EncryptCipher
auto domainItr = domainCacheMap.find(domainId);
if (domainItr == domainCacheMap.end()) {
// Add mapping to track new encryption domain
Reference<BlobCipherKeyIdCache> keyIdCache = makeReference<BlobCipherKeyIdCache>(domainId);
Reference<BlobCipherKeyIdCache> keyIdCache = makeReference<BlobCipherKeyIdCache>(domainId, &size);
cipherKey =
keyIdCache->insertBaseCipherKey(baseCipherId, baseCipher, baseCipherLen, salt, refreshAt, expireAt);
domainCacheMap.emplace(domainId, keyIdCache);
@ -354,7 +393,6 @@ Reference<BlobCipherKey> BlobCipherKeyCache::insertCipherKey(const EncryptCipher
.detail("Salt", salt);
throw;
}
return cipherKey;
}
@ -373,15 +411,20 @@ Reference<BlobCipherKey> BlobCipherKeyCache::getLatestCipherKey(const EncryptCip
Reference<BlobCipherKey> cipherKey = keyIdCache->getLatestCipherKey();
// Ensure 'freshness' guarantees for the latestCipher
if (cipherKey.isValid() && cipherKey->needsRefresh()) {
if (cipherKey.isValid()) {
if (cipherKey->needsRefresh()) {
#if BLOB_CIPHER_DEBUG
TraceEvent("SevDebug, BlobCipher.GetLatestNeedsRefresh")
.detail("DomainId", domainId)
.detail("Now", now())
.detail("RefreshAt", cipherKey->getRefreshAtTS());
TraceEvent("SevDebug, BlobCipher.GetLatestNeedsRefresh")
.detail("DomainId", domainId)
.detail("Now", now())
.detail("RefreshAt", cipherKey->getRefreshAtTS());
#endif
return Reference<BlobCipherKey>();
++BlobCipherMetrics::getInstance()->latestCipherKeyCacheNeedsRefresh;
return Reference<BlobCipherKey>();
}
++BlobCipherMetrics::getInstance()->latestCipherKeyCacheHit;
} else {
++BlobCipherMetrics::getInstance()->latestCipherKeyCacheMiss;
}
return cipherKey;
@ -399,16 +442,21 @@ Reference<BlobCipherKey> BlobCipherKeyCache::getCipherKey(const EncryptCipherDom
Reference<BlobCipherKey> cipherKey = keyIdCache->getCipherByBaseCipherId(baseCipherId, salt);
// Ensure 'liveness' guarantees for the cipher
if (cipherKey.isValid() && cipherKey->isExpired()) {
if (cipherKey.isValid()) {
if (cipherKey->isExpired()) {
#if BLOB_CIPHER_DEBUG
TraceEvent(SevDebug, "BlobCipher.GetCipherExpired")
.detail("DomainId", domainId)
.detail("BaseCipherId", baseCipherId)
.detail("Now", now())
.detail("ExpireAt", cipherKey->getExpireAtTS());
TraceEvent(SevDebug, "BlobCipher.GetCipherExpired")
.detail("DomainId", domainId)
.detail("BaseCipherId", baseCipherId)
.detail("Now", now())
.detail("ExpireAt", cipherKey->getExpireAtTS());
#endif
return Reference<BlobCipherKey>();
++BlobCipherMetrics::getInstance()->cipherKeyCacheExpired;
return Reference<BlobCipherKey>();
}
++BlobCipherMetrics::getInstance()->cipherKeyCacheHit;
} else {
++BlobCipherMetrics::getInstance()->cipherKeyCacheMiss;
}
return cipherKey;
@ -421,6 +469,8 @@ void BlobCipherKeyCache::resetEncryptDomainId(const EncryptCipherDomainId domain
}
Reference<BlobCipherKeyIdCache> keyIdCache = domainItr->second;
ASSERT(keyIdCache->getSize() <= size);
size -= keyIdCache->getSize();
keyIdCache->cleanup();
TraceEvent(SevInfo, "BlobCipher.ResetEncryptDomainId").detail("DomainId", domainId);
}
@ -437,6 +487,7 @@ void BlobCipherKeyCache::cleanup() noexcept {
}
instance->domainCacheMap.clear();
instance->size = 0;
}
std::vector<Reference<BlobCipherKey>> BlobCipherKeyCache::getAllCiphers(const EncryptCipherDomainId& domainId) {
@ -455,8 +506,10 @@ EncryptBlobCipherAes265Ctr::EncryptBlobCipherAes265Ctr(Reference<BlobCipherKey>
Reference<BlobCipherKey> hCipherKey,
const uint8_t* cipherIV,
const int ivLen,
const EncryptAuthTokenMode mode)
: ctx(EVP_CIPHER_CTX_new()), textCipherKey(tCipherKey), headerCipherKey(hCipherKey), authTokenMode(mode) {
const EncryptAuthTokenMode mode,
BlobCipherMetrics::UsageType usageType)
: ctx(EVP_CIPHER_CTX_new()), textCipherKey(tCipherKey), headerCipherKey(hCipherKey), authTokenMode(mode),
usageType(usageType) {
ASSERT(isEncryptHeaderAuthTokenModeValid(mode));
ASSERT_EQ(ivLen, AES_256_IV_LENGTH);
memcpy(&iv[0], cipherIV, ivLen);
@ -465,8 +518,10 @@ EncryptBlobCipherAes265Ctr::EncryptBlobCipherAes265Ctr(Reference<BlobCipherKey>
EncryptBlobCipherAes265Ctr::EncryptBlobCipherAes265Ctr(Reference<BlobCipherKey> tCipherKey,
Reference<BlobCipherKey> hCipherKey,
const EncryptAuthTokenMode mode)
: ctx(EVP_CIPHER_CTX_new()), textCipherKey(tCipherKey), headerCipherKey(hCipherKey), authTokenMode(mode) {
const EncryptAuthTokenMode mode,
BlobCipherMetrics::UsageType usageType)
: ctx(EVP_CIPHER_CTX_new()), textCipherKey(tCipherKey), headerCipherKey(hCipherKey), authTokenMode(mode),
usageType(usageType) {
ASSERT(isEncryptHeaderAuthTokenModeValid(mode));
deterministicRandom()->randomBytes(iv, AES_256_IV_LENGTH);
init();
@ -488,6 +543,10 @@ Reference<EncryptBuf> EncryptBlobCipherAes265Ctr::encrypt(const uint8_t* plainte
const int plaintextLen,
BlobCipherEncryptHeader* header,
Arena& arena) {
double startTime = 0.0;
if (CLIENT_KNOBS->ENABLE_ENCRYPTION_CPU_TIME_LOGGING) {
startTime = timer_monotonic();
}
CODE_PROBE(true, "Encrypting data with BlobCipher");
memset(reinterpret_cast<uint8_t*>(header), 0, sizeof(BlobCipherEncryptHeader));
@ -577,11 +636,18 @@ Reference<EncryptBuf> EncryptBlobCipherAes265Ctr::encrypt(const uint8_t* plainte
}
encryptBuf->setLogicalSize(plaintextLen);
if (CLIENT_KNOBS->ENABLE_ENCRYPTION_CPU_TIME_LOGGING) {
BlobCipherMetrics::counters(usageType).encryptCPUTimeNS += int64_t((timer_monotonic() - startTime) * 1e9);
}
return encryptBuf;
}
Standalone<StringRef> EncryptBlobCipherAes265Ctr::encryptBlobGranuleChunk(const uint8_t* plaintext,
const int plaintextLen) {
double startTime = 0.0;
if (CLIENT_KNOBS->ENABLE_ENCRYPTION_CPU_TIME_LOGGING) {
startTime = timer_monotonic();
}
Standalone<StringRef> encrypted = makeString(plaintextLen);
uint8_t* ciphertext = mutateString(encrypted);
int bytes{ 0 };
@ -605,6 +671,9 @@ Standalone<StringRef> EncryptBlobCipherAes265Ctr::encryptBlobGranuleChunk(const
.detail("EncryptedBufLen", bytes + finalBytes);
throw encrypt_ops_error();
}
if (CLIENT_KNOBS->ENABLE_ENCRYPTION_CPU_TIME_LOGGING) {
BlobCipherMetrics::counters(usageType).encryptCPUTimeNS += int64_t((timer_monotonic() - startTime) * 1e9);
}
return encrypted;
}
@ -618,9 +687,10 @@ EncryptBlobCipherAes265Ctr::~EncryptBlobCipherAes265Ctr() {
DecryptBlobCipherAes256Ctr::DecryptBlobCipherAes256Ctr(Reference<BlobCipherKey> tCipherKey,
Reference<BlobCipherKey> hCipherKey,
const uint8_t* iv)
const uint8_t* iv,
BlobCipherMetrics::UsageType usageType)
: ctx(EVP_CIPHER_CTX_new()), textCipherKey(tCipherKey), headerCipherKey(hCipherKey),
headerAuthTokenValidationDone(false), authTokensValidationDone(false) {
headerAuthTokenValidationDone(false), authTokensValidationDone(false), usageType(usageType) {
if (ctx == nullptr) {
throw encrypt_ops_error();
}
@ -754,6 +824,11 @@ Reference<EncryptBuf> DecryptBlobCipherAes256Ctr::decrypt(const uint8_t* ciphert
Arena& arena) {
CODE_PROBE(true, "Decrypting data with BlobCipher");
double startTime = 0.0;
if (CLIENT_KNOBS->ENABLE_ENCRYPTION_CPU_TIME_LOGGING) {
startTime = timer_monotonic();
}
verifyEncryptHeaderMetadata(header);
if (header.flags.authTokenMode != ENCRYPT_HEADER_AUTH_TOKEN_MODE_NONE && !headerCipherKey.isValid()) {
@ -797,6 +872,9 @@ Reference<EncryptBuf> DecryptBlobCipherAes256Ctr::decrypt(const uint8_t* ciphert
}
decrypted->setLogicalSize(ciphertextLen);
if (CLIENT_KNOBS->ENABLE_ENCRYPTION_CPU_TIME_LOGGING) {
BlobCipherMetrics::counters(usageType).decryptCPUTimeNS += int64_t((timer_monotonic() - startTime) * 1e9);
}
return decrypted;
}
@ -1017,8 +1095,12 @@ TEST_CASE("flow/BlobCipher") {
{
TraceEvent("NoneAuthMode.Start").log();
EncryptBlobCipherAes265Ctr encryptor(
cipherKey, Reference<BlobCipherKey>(), iv, AES_256_IV_LENGTH, ENCRYPT_HEADER_AUTH_TOKEN_MODE_NONE);
EncryptBlobCipherAes265Ctr encryptor(cipherKey,
Reference<BlobCipherKey>(),
iv,
AES_256_IV_LENGTH,
ENCRYPT_HEADER_AUTH_TOKEN_MODE_NONE,
BlobCipherMetrics::TEST);
BlobCipherEncryptHeader header;
Reference<EncryptBuf> encrypted = encryptor.encrypt(&orgData[0], bufLen, &header, arena);
@ -1038,7 +1120,8 @@ TEST_CASE("flow/BlobCipher") {
header.cipherTextDetails.baseCipherId,
header.cipherTextDetails.salt);
ASSERT(tCipherKeyKey->isEqual(cipherKey));
DecryptBlobCipherAes256Ctr decryptor(tCipherKeyKey, Reference<BlobCipherKey>(), &header.iv[0]);
DecryptBlobCipherAes256Ctr decryptor(
tCipherKeyKey, Reference<BlobCipherKey>(), &header.iv[0], BlobCipherMetrics::TEST);
Reference<EncryptBuf> decrypted = decryptor.decrypt(encrypted->begin(), bufLen, header, arena);
ASSERT_EQ(decrypted->getLogicalSize(), bufLen);
@ -1053,7 +1136,8 @@ TEST_CASE("flow/BlobCipher") {
headerCopy.flags.headerVersion += 1;
try {
encrypted = encryptor.encrypt(&orgData[0], bufLen, &header, arena);
DecryptBlobCipherAes256Ctr decryptor(tCipherKeyKey, Reference<BlobCipherKey>(), header.iv);
DecryptBlobCipherAes256Ctr decryptor(
tCipherKeyKey, Reference<BlobCipherKey>(), header.iv, BlobCipherMetrics::TEST);
decrypted = decryptor.decrypt(encrypted->begin(), bufLen, headerCopy, arena);
ASSERT(false); // error expected
} catch (Error& e) {
@ -1069,7 +1153,8 @@ TEST_CASE("flow/BlobCipher") {
headerCopy.flags.encryptMode += 1;
try {
encrypted = encryptor.encrypt(&orgData[0], bufLen, &header, arena);
DecryptBlobCipherAes256Ctr decryptor(tCipherKeyKey, Reference<BlobCipherKey>(), header.iv);
DecryptBlobCipherAes256Ctr decryptor(
tCipherKeyKey, Reference<BlobCipherKey>(), header.iv, BlobCipherMetrics::TEST);
decrypted = decryptor.decrypt(encrypted->begin(), bufLen, headerCopy, arena);
ASSERT(false); // error expected
} catch (Error& e) {
@ -1085,7 +1170,8 @@ TEST_CASE("flow/BlobCipher") {
memcpy(encrypted->begin(), &temp[0], bufLen);
int tIdx = deterministicRandom()->randomInt(0, bufLen - 1);
temp[tIdx] += 1;
DecryptBlobCipherAes256Ctr decryptor(tCipherKeyKey, Reference<BlobCipherKey>(), header.iv);
DecryptBlobCipherAes256Ctr decryptor(
tCipherKeyKey, Reference<BlobCipherKey>(), header.iv, BlobCipherMetrics::TEST);
decrypted = decryptor.decrypt(&temp[0], bufLen, header, arena);
} catch (Error& e) {
// No authToken, hence, no corruption detection supported
@ -1099,8 +1185,12 @@ TEST_CASE("flow/BlobCipher") {
{
TraceEvent("SingleAuthMode.Start").log();
EncryptBlobCipherAes265Ctr encryptor(
cipherKey, headerCipherKey, iv, AES_256_IV_LENGTH, ENCRYPT_HEADER_AUTH_TOKEN_MODE_SINGLE);
EncryptBlobCipherAes265Ctr encryptor(cipherKey,
headerCipherKey,
iv,
AES_256_IV_LENGTH,
ENCRYPT_HEADER_AUTH_TOKEN_MODE_SINGLE,
BlobCipherMetrics::TEST);
BlobCipherEncryptHeader header;
Reference<EncryptBuf> encrypted = encryptor.encrypt(&orgData[0], bufLen, &header, arena);
@ -1125,7 +1215,7 @@ TEST_CASE("flow/BlobCipher") {
header.cipherHeaderDetails.baseCipherId,
header.cipherHeaderDetails.salt);
ASSERT(tCipherKeyKey->isEqual(cipherKey));
DecryptBlobCipherAes256Ctr decryptor(tCipherKeyKey, hCipherKey, header.iv);
DecryptBlobCipherAes256Ctr decryptor(tCipherKeyKey, hCipherKey, header.iv, BlobCipherMetrics::TEST);
Reference<EncryptBuf> decrypted = decryptor.decrypt(encrypted->begin(), bufLen, header, arena);
ASSERT_EQ(decrypted->getLogicalSize(), bufLen);
@ -1140,7 +1230,7 @@ TEST_CASE("flow/BlobCipher") {
sizeof(BlobCipherEncryptHeader));
headerCopy.flags.headerVersion += 1;
try {
DecryptBlobCipherAes256Ctr decryptor(tCipherKeyKey, hCipherKey, header.iv);
DecryptBlobCipherAes256Ctr decryptor(tCipherKeyKey, hCipherKey, header.iv, BlobCipherMetrics::TEST);
decrypted = decryptor.decrypt(encrypted->begin(), bufLen, headerCopy, arena);
ASSERT(false); // error expected
} catch (Error& e) {
@ -1156,7 +1246,7 @@ TEST_CASE("flow/BlobCipher") {
sizeof(BlobCipherEncryptHeader));
headerCopy.flags.encryptMode += 1;
try {
DecryptBlobCipherAes256Ctr decryptor(tCipherKeyKey, hCipherKey, header.iv);
DecryptBlobCipherAes256Ctr decryptor(tCipherKeyKey, hCipherKey, header.iv, BlobCipherMetrics::TEST);
decrypted = decryptor.decrypt(encrypted->begin(), bufLen, headerCopy, arena);
ASSERT(false); // error expected
} catch (Error& e) {
@ -1173,7 +1263,7 @@ TEST_CASE("flow/BlobCipher") {
int hIdx = deterministicRandom()->randomInt(0, AUTH_TOKEN_SIZE - 1);
headerCopy.singleAuthToken.authToken[hIdx] += 1;
try {
DecryptBlobCipherAes256Ctr decryptor(tCipherKeyKey, hCipherKey, header.iv);
DecryptBlobCipherAes256Ctr decryptor(tCipherKeyKey, hCipherKey, header.iv, BlobCipherMetrics::TEST);
decrypted = decryptor.decrypt(encrypted->begin(), bufLen, headerCopy, arena);
ASSERT(false); // error expected
} catch (Error& e) {
@ -1189,7 +1279,7 @@ TEST_CASE("flow/BlobCipher") {
memcpy(encrypted->begin(), &temp[0], bufLen);
int tIdx = deterministicRandom()->randomInt(0, bufLen - 1);
temp[tIdx] += 1;
DecryptBlobCipherAes256Ctr decryptor(tCipherKeyKey, hCipherKey, header.iv);
DecryptBlobCipherAes256Ctr decryptor(tCipherKeyKey, hCipherKey, header.iv, BlobCipherMetrics::TEST);
decrypted = decryptor.decrypt(&temp[0], bufLen, header, arena);
} catch (Error& e) {
if (e.code() != error_code_encrypt_header_authtoken_mismatch) {
@ -1204,8 +1294,12 @@ TEST_CASE("flow/BlobCipher") {
{
TraceEvent("MultiAuthMode.Start").log();
EncryptBlobCipherAes265Ctr encryptor(
cipherKey, headerCipherKey, iv, AES_256_IV_LENGTH, ENCRYPT_HEADER_AUTH_TOKEN_MODE_MULTI);
EncryptBlobCipherAes265Ctr encryptor(cipherKey,
headerCipherKey,
iv,
AES_256_IV_LENGTH,
ENCRYPT_HEADER_AUTH_TOKEN_MODE_MULTI,
BlobCipherMetrics::TEST);
BlobCipherEncryptHeader header;
Reference<EncryptBuf> encrypted = encryptor.encrypt(&orgData[0], bufLen, &header, arena);
@ -1231,7 +1325,7 @@ TEST_CASE("flow/BlobCipher") {
header.cipherHeaderDetails.salt);
ASSERT(tCipherKey->isEqual(cipherKey));
DecryptBlobCipherAes256Ctr decryptor(tCipherKey, hCipherKey, header.iv);
DecryptBlobCipherAes256Ctr decryptor(tCipherKey, hCipherKey, header.iv, BlobCipherMetrics::TEST);
Reference<EncryptBuf> decrypted = decryptor.decrypt(encrypted->begin(), bufLen, header, arena);
ASSERT_EQ(decrypted->getLogicalSize(), bufLen);
@ -1246,7 +1340,7 @@ TEST_CASE("flow/BlobCipher") {
sizeof(BlobCipherEncryptHeader));
headerCopy.flags.headerVersion += 1;
try {
DecryptBlobCipherAes256Ctr decryptor(tCipherKey, hCipherKey, header.iv);
DecryptBlobCipherAes256Ctr decryptor(tCipherKey, hCipherKey, header.iv, BlobCipherMetrics::TEST);
decrypted = decryptor.decrypt(encrypted->begin(), bufLen, headerCopy, arena);
ASSERT(false); // error expected
} catch (Error& e) {
@ -1262,7 +1356,7 @@ TEST_CASE("flow/BlobCipher") {
sizeof(BlobCipherEncryptHeader));
headerCopy.flags.encryptMode += 1;
try {
DecryptBlobCipherAes256Ctr decryptor(tCipherKey, hCipherKey, header.iv);
DecryptBlobCipherAes256Ctr decryptor(tCipherKey, hCipherKey, header.iv, BlobCipherMetrics::TEST);
decrypted = decryptor.decrypt(encrypted->begin(), bufLen, headerCopy, arena);
ASSERT(false); // error expected
} catch (Error& e) {
@ -1279,7 +1373,7 @@ TEST_CASE("flow/BlobCipher") {
int hIdx = deterministicRandom()->randomInt(0, AUTH_TOKEN_SIZE - 1);
headerCopy.multiAuthTokens.cipherTextAuthToken[hIdx] += 1;
try {
DecryptBlobCipherAes256Ctr decryptor(tCipherKey, hCipherKey, header.iv);
DecryptBlobCipherAes256Ctr decryptor(tCipherKey, hCipherKey, header.iv, BlobCipherMetrics::TEST);
decrypted = decryptor.decrypt(encrypted->begin(), bufLen, headerCopy, arena);
ASSERT(false); // error expected
} catch (Error& e) {
@ -1296,7 +1390,7 @@ TEST_CASE("flow/BlobCipher") {
hIdx = deterministicRandom()->randomInt(0, AUTH_TOKEN_SIZE - 1);
headerCopy.multiAuthTokens.headerAuthToken[hIdx] += 1;
try {
DecryptBlobCipherAes256Ctr decryptor(tCipherKey, hCipherKey, header.iv);
DecryptBlobCipherAes256Ctr decryptor(tCipherKey, hCipherKey, header.iv, BlobCipherMetrics::TEST);
decrypted = decryptor.decrypt(encrypted->begin(), bufLen, headerCopy, arena);
ASSERT(false); // error expected
} catch (Error& e) {
@ -1311,7 +1405,7 @@ TEST_CASE("flow/BlobCipher") {
memcpy(encrypted->begin(), &temp[0], bufLen);
int tIdx = deterministicRandom()->randomInt(0, bufLen - 1);
temp[tIdx] += 1;
DecryptBlobCipherAes256Ctr decryptor(tCipherKey, hCipherKey, header.iv);
DecryptBlobCipherAes256Ctr decryptor(tCipherKey, hCipherKey, header.iv, BlobCipherMetrics::TEST);
decrypted = decryptor.decrypt(&temp[0], bufLen, header, arena);
} catch (Error& e) {
if (e.code() != error_code_encrypt_header_authtoken_mismatch) {

View File

@ -20,6 +20,7 @@
#include "fdbclient/BlobGranuleFiles.h"
#include "fdbclient/BlobCipher.h"
#include "fdbclient/BlobGranuleCommon.h"
#include "fdbclient/ClientKnobs.h"
#include "fdbclient/CommitTransaction.h"
@ -27,7 +28,6 @@
#include "fdbclient/SystemData.h" // for allKeys unit test - could remove
#include "flow/Arena.h"
#include "flow/BlobCipher.h"
#include "flow/CompressionUtils.h"
#include "flow/DeterministicRandom.h"
#include "flow/IRandom.h"
@ -304,7 +304,8 @@ struct IndexBlockRef {
eKeys.headerCipherKey,
cipherKeysCtx.ivRef.begin(),
AES_256_IV_LENGTH,
ENCRYPT_HEADER_AUTH_TOKEN_MODE_SINGLE);
ENCRYPT_HEADER_AUTH_TOKEN_MODE_SINGLE,
BlobCipherMetrics::BLOB_GRANULE);
Value serializedBuff = ObjectWriter::toValue(block, IncludeVersion(ProtocolVersion::withBlobGranuleFile()));
BlobCipherEncryptHeader header;
buffer = encryptor.encrypt(serializedBuff.contents().begin(), serializedBuff.contents().size(), &header, arena)
@ -332,7 +333,8 @@ struct IndexBlockRef {
validateEncryptionHeaderDetails(eKeys, header, cipherKeysCtx.ivRef);
DecryptBlobCipherAes256Ctr decryptor(eKeys.textCipherKey, eKeys.headerCipherKey, cipherKeysCtx.ivRef.begin());
DecryptBlobCipherAes256Ctr decryptor(
eKeys.textCipherKey, eKeys.headerCipherKey, cipherKeysCtx.ivRef.begin(), BlobCipherMetrics::BLOB_GRANULE);
StringRef decrypted =
decryptor.decrypt(idxRef.buffer.begin(), idxRef.buffer.size(), header, arena)->toStringRef();
@ -425,7 +427,8 @@ struct IndexBlobGranuleFileChunkRef {
eKeys.headerCipherKey,
cipherKeysCtx.ivRef.begin(),
AES_256_IV_LENGTH,
ENCRYPT_HEADER_AUTH_TOKEN_MODE_SINGLE);
ENCRYPT_HEADER_AUTH_TOKEN_MODE_SINGLE,
BlobCipherMetrics::BLOB_GRANULE);
BlobCipherEncryptHeader header;
chunkRef.buffer =
encryptor.encrypt(chunkRef.buffer.begin(), chunkRef.buffer.size(), &header, arena)->toStringRef();
@ -454,7 +457,8 @@ struct IndexBlobGranuleFileChunkRef {
validateEncryptionHeaderDetails(eKeys, header, cipherKeysCtx.ivRef);
DecryptBlobCipherAes256Ctr decryptor(eKeys.textCipherKey, eKeys.headerCipherKey, cipherKeysCtx.ivRef.begin());
DecryptBlobCipherAes256Ctr decryptor(
eKeys.textCipherKey, eKeys.headerCipherKey, cipherKeysCtx.ivRef.begin(), BlobCipherMetrics::BLOB_GRANULE);
StringRef decrypted =
decryptor.decrypt(chunkRef.buffer.begin(), chunkRef.buffer.size(), header, arena)->toStringRef();

View File

@ -291,6 +291,8 @@ void ClientKnobs::initialize(Randomize randomize) {
init( METACLUSTER_ASSIGNMENT_FIRST_CHOICE_DELAY, 1.0 ); if ( randomize && BUGGIFY ) METACLUSTER_ASSIGNMENT_FIRST_CHOICE_DELAY = deterministicRandom()->random01() * 60;
init( METACLUSTER_ASSIGNMENT_AVAILABILITY_TIMEOUT, 10.0 ); if ( randomize && BUGGIFY ) METACLUSTER_ASSIGNMENT_AVAILABILITY_TIMEOUT = 1 + deterministicRandom()->random01() * 59;
init( TENANT_ENTRY_CACHE_LIST_REFRESH_INTERVAL, 2 ); if( randomize && BUGGIFY ) TENANT_ENTRY_CACHE_LIST_REFRESH_INTERVAL = deterministicRandom()->randomInt(1, 10);
init( ENABLE_ENCRYPTION_CPU_TIME_LOGGING, false );
// clang-format on
}

View File

@ -127,7 +127,10 @@ ACTOR Future<RangeResult> krmGetRangesUnaligned(Transaction* tr,
state GetRangeLimits limits(limit, limitBytes);
limits.minRows = 2;
RangeResult kv = wait(tr->getRange(lastLessOrEqual(withPrefix.begin), firstGreaterThan(withPrefix.end), limits));
// wait to include the next highest row >= keys.end in the result, so since end is exclusive, we need +2 and
// !orEqual
RangeResult kv =
wait(tr->getRange(lastLessOrEqual(withPrefix.begin), KeySelectorRef(withPrefix.end, false, +2), limits));
return krmDecodeRanges(mapPrefix, keys, kv, false);
}
@ -142,7 +145,10 @@ ACTOR Future<RangeResult> krmGetRangesUnaligned(Reference<ReadYourWritesTransact
state GetRangeLimits limits(limit, limitBytes);
limits.minRows = 2;
RangeResult kv = wait(tr->getRange(lastLessOrEqual(withPrefix.begin), firstGreaterThan(withPrefix.end), limits));
// wait to include the next highest row >= keys.end in the result, so since end is exclusive, we need +2 and
// !orEqual
RangeResult kv =
wait(tr->getRange(lastLessOrEqual(withPrefix.begin), KeySelectorRef(withPrefix.end, false, +2), limits));
return krmDecodeRanges(mapPrefix, keys, kv, false);
}
@ -323,17 +329,27 @@ TEST_CASE("/keyrangemap/decoderange/aligned") {
StringRef keyD = StringRef(arena, LiteralStringRef("d"));
StringRef keyE = StringRef(arena, LiteralStringRef("e"));
StringRef keyAB = StringRef(arena, LiteralStringRef("ab"));
StringRef keyAC = StringRef(arena, LiteralStringRef("ac"));
StringRef keyCD = StringRef(arena, LiteralStringRef("cd"));
// Fake getRange() call.
RangeResult kv;
kv.push_back(arena, KeyValueRef(fullKeyA, keyA));
kv.push_back(arena, KeyValueRef(fullKeyB, keyB));
// [A, AB(start), AC(start), B]
RangeResult decodedRanges = krmDecodeRanges(prefix, KeyRangeRef(keyAB, keyAC), kv);
ASSERT(decodedRanges.size() == 2);
ASSERT(decodedRanges.front().key == keyAB);
ASSERT(decodedRanges.front().value == keyA);
ASSERT(decodedRanges.back().key == keyAC);
ASSERT(decodedRanges.back().value == keyA);
kv.push_back(arena, KeyValueRef(fullKeyC, keyC));
kv.push_back(arena, KeyValueRef(fullKeyD, keyD));
// [A, AB(start), B, C, CD(end), D]
RangeResult decodedRanges = krmDecodeRanges(prefix, KeyRangeRef(keyAB, keyCD), kv);
decodedRanges = krmDecodeRanges(prefix, KeyRangeRef(keyAB, keyCD), kv);
ASSERT(decodedRanges.size() == 4);
ASSERT(decodedRanges.front().key == keyAB);
ASSERT(decodedRanges.front().value == keyA);
@ -365,17 +381,27 @@ TEST_CASE("/keyrangemap/decoderange/unaligned") {
StringRef keyD = StringRef(arena, LiteralStringRef("d"));
StringRef keyE = StringRef(arena, LiteralStringRef("e"));
StringRef keyAB = StringRef(arena, LiteralStringRef("ab"));
StringRef keyAC = StringRef(arena, LiteralStringRef("ac"));
StringRef keyCD = StringRef(arena, LiteralStringRef("cd"));
// Fake getRange() call.
RangeResult kv;
kv.push_back(arena, KeyValueRef(fullKeyA, keyA));
kv.push_back(arena, KeyValueRef(fullKeyB, keyB));
// [A, AB(start), AC(start), B]
RangeResult decodedRanges = krmDecodeRanges(prefix, KeyRangeRef(keyAB, keyAC), kv, false);
ASSERT(decodedRanges.size() == 2);
ASSERT(decodedRanges.front().key == keyA);
ASSERT(decodedRanges.front().value == keyA);
ASSERT(decodedRanges.back().key == keyB);
ASSERT(decodedRanges.back().value == keyB);
kv.push_back(arena, KeyValueRef(fullKeyC, keyC));
kv.push_back(arena, KeyValueRef(fullKeyD, keyD));
// [A, AB(start), B, C, CD(end), D]
RangeResult decodedRanges = krmDecodeRanges(prefix, KeyRangeRef(keyAB, keyCD), kv, false);
decodedRanges = krmDecodeRanges(prefix, KeyRangeRef(keyAB, keyCD), kv, false);
ASSERT(decodedRanges.size() == 4);
ASSERT(decodedRanges.front().key == keyA);
ASSERT(decodedRanges.front().value == keyA);

View File

@ -1355,6 +1355,7 @@ ACTOR Future<Void> excludeServers(Database cx, std::vector<AddressExclusion> ser
state ReadYourWritesTransaction ryw(cx);
loop {
try {
ryw.setOption(FDBTransactionOptions::RAW_ACCESS);
ryw.setOption(FDBTransactionOptions::SPECIAL_KEY_SPACE_ENABLE_WRITES);
ryw.set(
SpecialKeySpace::getManagementApiCommandOptionSpecialKey(failed ? "failed" : "excluded", "force"),
@ -1417,6 +1418,7 @@ ACTOR Future<Void> excludeLocalities(Database cx, std::unordered_set<std::string
state ReadYourWritesTransaction ryw(cx);
loop {
try {
ryw.setOption(FDBTransactionOptions::RAW_ACCESS);
ryw.setOption(FDBTransactionOptions::SPECIAL_KEY_SPACE_ENABLE_WRITES);
ryw.set(SpecialKeySpace::getManagementApiCommandOptionSpecialKey(
failed ? "failed_locality" : "excluded_locality", "force"),
@ -1459,6 +1461,7 @@ ACTOR Future<Void> includeServers(Database cx, std::vector<AddressExclusion> ser
state ReadYourWritesTransaction ryw(cx);
loop {
try {
ryw.setOption(FDBTransactionOptions::RAW_ACCESS);
ryw.setOption(FDBTransactionOptions::SPECIAL_KEY_SPACE_ENABLE_WRITES);
for (auto& s : servers) {
if (!s.isValid()) {
@ -1562,6 +1565,7 @@ ACTOR Future<Void> includeLocalities(Database cx, std::vector<std::string> local
state ReadYourWritesTransaction ryw(cx);
loop {
try {
ryw.setOption(FDBTransactionOptions::RAW_ACCESS);
ryw.setOption(FDBTransactionOptions::SPECIAL_KEY_SPACE_ENABLE_WRITES);
if (includeAll) {
if (failed) {

View File

@ -861,6 +861,7 @@ ACTOR Future<MonitorLeaderInfo> monitorProxiesOneGeneration(
for (const auto& c : cs.coords) {
clientLeaderServers.push_back(ClientLeaderRegInterface(c));
}
ASSERT(clientLeaderServers.size() > 0);
deterministicRandom()->randomShuffle(clientLeaderServers);
@ -880,7 +881,7 @@ ACTOR Future<MonitorLeaderInfo> monitorProxiesOneGeneration(
bool upToDate = wait(connRecord->upToDate(storedConnectionString));
if (upToDate) {
incorrectTime = Optional<double>();
} else if (allConnectionsFailed) {
} else if (allConnectionsFailed && storedConnectionString.getNumberOfCoordinators() > 0) {
// Failed to connect to all coordinators from the current connection string,
// so it is not possible to get any new updates from the cluster. It can be that
// all the coordinators have changed, but the client missed that, because it had
@ -938,6 +939,7 @@ ACTOR Future<MonitorLeaderInfo> monitorProxiesOneGeneration(
.detail("OldConnStr", info.intermediateConnRecord->getConnectionString().toString());
info.intermediateConnRecord = connRecord->makeIntermediateRecord(
ClusterConnectionString(rep.get().read().forward.get().toString()));
ASSERT(info.intermediateConnRecord->getConnectionString().getNumberOfCoordinators() > 0);
return info;
}
if (connRecord != info.intermediateConnRecord) {
@ -984,6 +986,7 @@ ACTOR Future<Void> monitorProxies(
Key traceLogGroup) {
state MonitorLeaderInfo info(connRecord->get());
loop {
ASSERT(connRecord->get().isValid());
choose {
when(MonitorLeaderInfo _info = wait(monitorProxiesOneGeneration(
connRecord->get(), clientInfo, coordinator, info, supportedVersions, traceLogGroup))) {

View File

@ -652,7 +652,7 @@ ThreadFuture<bool> DLDatabase::blobbifyRange(const KeyRangeRef& keyRange) {
db, keyRange.begin.begin(), keyRange.begin.size(), keyRange.end.begin(), keyRange.end.size());
return toThreadFuture<bool>(api, f, [](FdbCApi::FDBFuture* f, FdbCApi* api) {
bool ret = false;
FdbCApi::fdb_bool_t ret = false;
ASSERT(!api->futureGetBool(f, &ret));
return ret;
});
@ -667,7 +667,7 @@ ThreadFuture<bool> DLDatabase::unblobbifyRange(const KeyRangeRef& keyRange) {
db, keyRange.begin.begin(), keyRange.begin.size(), keyRange.end.begin(), keyRange.end.size());
return toThreadFuture<bool>(api, f, [](FdbCApi::FDBFuture* f, FdbCApi* api) {
bool ret = false;
FdbCApi::fdb_bool_t ret = false;
ASSERT(!api->futureGetBool(f, &ret));
return ret;
});
@ -2756,12 +2756,13 @@ ACTOR Future<std::string> updateClusterSharedStateMapImpl(MultiVersionApi* self,
ProtocolVersion dbProtocolVersion,
Reference<IDatabase> db) {
// The cluster ID will be the connection record string (either a filename or the connection string itself)
// in API versions before we could read the cluster ID.
// in versions before we could read the cluster ID.
state std::string clusterId = connectionRecord.toString();
if (MultiVersionApi::api->getApiVersion().hasCreateDBFromConnString()) {
if (dbProtocolVersion.hasClusterIdSpecialKey()) {
state Reference<ITransaction> tr = db->createTransaction();
loop {
try {
tr->setOption(FDBTransactionOptions::SPECIAL_KEY_SPACE_RELAXED);
state ThreadFuture<Optional<Value>> clusterIdFuture = tr->get("\xff\xff/cluster_id"_sr);
Optional<Value> clusterIdVal = wait(safeThreadFutureToFuture(clusterIdFuture));
ASSERT(clusterIdVal.present());

View File

@ -1296,12 +1296,17 @@ struct SingleSpecialKeyImpl : SpecialKeyRangeReadImpl {
});
}
SingleSpecialKeyImpl(KeyRef k, const std::function<Future<Optional<Value>>(ReadYourWritesTransaction*)>& f)
: SpecialKeyRangeReadImpl(singleKeyRange(k)), k(k), f(f) {}
SingleSpecialKeyImpl(KeyRef k,
const std::function<Future<Optional<Value>>(ReadYourWritesTransaction*)>& f,
bool supportsTenants = false)
: SpecialKeyRangeReadImpl(singleKeyRange(k)), k(k), f(f), tenantSupport(supportsTenants) {}
bool supportsTenants() const override { return tenantSupport; };
private:
Key k;
std::function<Future<Optional<Value>>(ReadYourWritesTransaction*)> f;
bool tenantSupport;
};
class HealthMetricsRangeImpl : public SpecialKeyRangeAsyncImpl {
@ -1501,32 +1506,6 @@ DatabaseContext::DatabaseContext(Reference<AsyncVar<Reference<IClusterConnection
smoothMidShardSize.reset(CLIENT_KNOBS->INIT_MID_SHARD_BYTES);
globalConfig = std::make_unique<GlobalConfig>(this);
if (apiVersion.hasTenantsV2()) {
registerSpecialKeysImpl(
SpecialKeySpace::MODULE::CLUSTERID,
SpecialKeySpace::IMPLTYPE::READONLY,
std::make_unique<SingleSpecialKeyImpl>(
LiteralStringRef("\xff\xff/cluster_id"), [](ReadYourWritesTransaction* ryw) -> Future<Optional<Value>> {
try {
if (ryw->getDatabase().getPtr()) {
return map(getClusterId(ryw->getDatabase()),
[](UID id) { return Optional<Value>(StringRef(id.toString())); });
}
} catch (Error& e) {
return e;
}
return Optional<Value>();
}));
registerSpecialKeysImpl(
SpecialKeySpace::MODULE::MANAGEMENT,
SpecialKeySpace::IMPLTYPE::READWRITE,
std::make_unique<TenantRangeImpl<true>>(SpecialKeySpace::getManagementApiCommandRange("tenant")));
} else if (apiVersion.hasTenantsV1()) {
registerSpecialKeysImpl(
SpecialKeySpace::MODULE::MANAGEMENT,
SpecialKeySpace::IMPLTYPE::READWRITE,
std::make_unique<TenantRangeImpl<false>>(SpecialKeySpace::getManagementApiCommandRange("tenantmap")));
}
if (apiVersion.version() >= 700) {
registerSpecialKeysImpl(SpecialKeySpace::MODULE::ERRORMSG,
SpecialKeySpace::IMPLTYPE::READONLY,
@ -1537,7 +1516,8 @@ DatabaseContext::DatabaseContext(Reference<AsyncVar<Reference<IClusterConnection
return Optional<Value>(ryw->getSpecialKeySpaceErrorMsg().get());
else
return Optional<Value>();
}));
},
true));
registerSpecialKeysImpl(
SpecialKeySpace::MODULE::MANAGEMENT,
SpecialKeySpace::IMPLTYPE::READWRITE,
@ -1684,7 +1664,8 @@ DatabaseContext::DatabaseContext(Reference<AsyncVar<Reference<IClusterConnection
} else {
return Optional<Value>();
}
}));
},
true));
registerSpecialKeysImpl(SpecialKeySpace::MODULE::CLUSTERFILEPATH,
SpecialKeySpace::IMPLTYPE::READONLY,
std::make_unique<SingleSpecialKeyImpl>(
@ -1701,7 +1682,8 @@ DatabaseContext::DatabaseContext(Reference<AsyncVar<Reference<IClusterConnection
return e;
}
return Optional<Value>();
}));
},
true));
registerSpecialKeysImpl(
SpecialKeySpace::MODULE::CONNECTIONSTRING,
@ -1719,7 +1701,30 @@ DatabaseContext::DatabaseContext(Reference<AsyncVar<Reference<IClusterConnection
return e;
}
return Optional<Value>();
}));
},
true));
registerSpecialKeysImpl(SpecialKeySpace::MODULE::CLUSTERID,
SpecialKeySpace::IMPLTYPE::READONLY,
std::make_unique<SingleSpecialKeyImpl>(
LiteralStringRef("\xff\xff/cluster_id"),
[](ReadYourWritesTransaction* ryw) -> Future<Optional<Value>> {
try {
if (ryw->getDatabase().getPtr()) {
return map(getClusterId(ryw->getDatabase()), [](UID id) {
return Optional<Value>(StringRef(id.toString()));
});
}
} catch (Error& e) {
return e;
}
return Optional<Value>();
},
true));
registerSpecialKeysImpl(
SpecialKeySpace::MODULE::MANAGEMENT,
SpecialKeySpace::IMPLTYPE::READWRITE,
std::make_unique<TenantRangeImpl>(SpecialKeySpace::getManagementApiCommandRange("tenant")));
}
throttleExpirer = recurring([this]() { expireThrottles(); }, CLIENT_KNOBS->TAG_THROTTLE_EXPIRATION_INTERVAL);
@ -1761,7 +1766,7 @@ DatabaseContext::DatabaseContext(const Error& err)
transactionGrvFullBatches("NumGrvFullBatches", cc), transactionGrvTimedOutBatches("NumGrvTimedOutBatches", cc),
transactionCommitVersionNotFoundForSS("CommitVersionNotFoundForSS", cc), latencies(1000), readLatencies(1000),
commitLatencies(1000), GRVLatencies(1000), mutationsPerCommit(1000), bytesPerCommit(1000), bgLatencies(1000),
bgGranulesPerRequest(1000), transactionTracingSample(false),
bgGranulesPerRequest(1000), sharedStatePtr(nullptr), transactionTracingSample(false),
smoothMidShardSize(CLIENT_KNOBS->SHARD_STAT_SMOOTH_AMOUNT),
connectToDatabaseEventCacheHolder(format("ConnectToDatabase/%s", dbId.toString().c_str())) {}
@ -6118,30 +6123,46 @@ ACTOR Future<Optional<ClientTrCommitCostEstimation>> estimateCommitCosts(Referen
// TODO: send the prefix as part of the commit request and ship it all the way
// through to the storage servers
void applyTenantPrefix(CommitTransactionRequest& req, Key tenantPrefix) {
VectorRef<MutationRef> updatedMutations;
updatedMutations.reserve(req.arena, req.transaction.mutations.size());
for (auto& m : req.transaction.mutations) {
StringRef param1 = m.param1;
StringRef param2 = m.param2;
if (m.param1 != metadataVersionKey) {
m.param1 = m.param1.withPrefix(tenantPrefix, req.arena);
param1 = m.param1.withPrefix(tenantPrefix, req.arena);
if (m.type == MutationRef::ClearRange) {
m.param2 = m.param2.withPrefix(tenantPrefix, req.arena);
param2 = m.param2.withPrefix(tenantPrefix, req.arena);
} else if (m.type == MutationRef::SetVersionstampedKey) {
uint8_t* key = mutateString(m.param1);
int* offset = reinterpret_cast<int*>(&key[m.param1.size() - 4]);
uint8_t* key = mutateString(param1);
int* offset = reinterpret_cast<int*>(&key[param1.size() - 4]);
*offset += tenantPrefix.size();
}
}
updatedMutations.push_back(req.arena, MutationRef(MutationRef::Type(m.type), param1, param2));
}
req.transaction.mutations = updatedMutations;
for (auto& rc : req.transaction.read_conflict_ranges) {
VectorRef<KeyRangeRef> updatedReadConflictRanges;
updatedReadConflictRanges.reserve(req.arena, req.transaction.read_conflict_ranges.size());
for (auto const& rc : req.transaction.read_conflict_ranges) {
if (rc.begin != metadataVersionKey) {
rc = rc.withPrefix(tenantPrefix, req.arena);
updatedReadConflictRanges.push_back(req.arena, rc.withPrefix(tenantPrefix, req.arena));
} else {
updatedReadConflictRanges.push_back(req.arena, rc);
}
}
req.transaction.read_conflict_ranges = updatedReadConflictRanges;
VectorRef<KeyRangeRef> updatedWriteConflictRanges;
updatedWriteConflictRanges.reserve(req.arena, req.transaction.write_conflict_ranges.size());
for (auto& wc : req.transaction.write_conflict_ranges) {
if (wc.begin != metadataVersionKey) {
wc = wc.withPrefix(tenantPrefix, req.arena);
updatedWriteConflictRanges.push_back(req.arena, wc.withPrefix(tenantPrefix, req.arena));
} else {
updatedWriteConflictRanges.push_back(req.arena, wc);
}
}
req.transaction.write_conflict_ranges = updatedWriteConflictRanges;
}
ACTOR static Future<Void> tryCommit(Reference<TransactionState> trState,
@ -6776,10 +6797,12 @@ ACTOR Future<GetReadVersionReply> getConsistentReadVersion(SpanContext parentSpa
if (e.code() != error_code_broken_promise && e.code() != error_code_batch_transaction_throttled &&
e.code() != error_code_grv_proxy_memory_limit_exceeded)
TraceEvent(SevError, "GetConsistentReadVersionError").error(e);
if ((e.code() == error_code_batch_transaction_throttled ||
e.code() == error_code_grv_proxy_memory_limit_exceeded) &&
!cx->apiVersionAtLeast(630)) {
if (e.code() == error_code_batch_transaction_throttled && !cx->apiVersionAtLeast(630)) {
wait(delayJittered(5.0));
} else if (e.code() == error_code_grv_proxy_memory_limit_exceeded) {
// FIXME(xwang): the better way is to let this error broadcast to transaction.onError(e), otherwise the
// txn->cx counter doesn't make sense
wait(delayJittered(CLIENT_KNOBS->GRV_ERROR_RETRY_DELAY));
} else {
throw;
}
@ -6816,7 +6839,6 @@ ACTOR Future<Void> readVersionBatcher(DatabaseContext* cx,
// dynamic batching
state PromiseStream<double> replyTimes;
state PromiseStream<Error> _errorStream;
state double batchTime = 0;
state Span span("NAPI:readVersionBatcher"_loc);
loop {
@ -9985,31 +10007,32 @@ ACTOR Future<bool> setBlobRangeActor(Reference<DatabaseContext> cx, KeyRange ran
state Reference<ReadYourWritesTransaction> tr = makeReference<ReadYourWritesTransaction>(db);
state Value value = active ? blobRangeActive : blobRangeInactive;
loop {
try {
tr->setOption(FDBTransactionOptions::ACCESS_SYSTEM_KEYS);
tr->setOption(FDBTransactionOptions::PRIORITY_SYSTEM_IMMEDIATE);
state Standalone<VectorRef<KeyRangeRef>> startBlobRanges = wait(getBlobRanges(tr, range, 10));
state Standalone<VectorRef<KeyRangeRef>> endBlobRanges =
wait(getBlobRanges(tr, KeyRangeRef(range.end, keyAfter(range.end)), 10));
Standalone<VectorRef<KeyRangeRef>> startBlobRanges = wait(getBlobRanges(tr, range, 1));
if (active) {
// Idempotent request.
if (!startBlobRanges.empty() && !endBlobRanges.empty()) {
return startBlobRanges.front().begin == range.begin && endBlobRanges.front().end == range.end;
if (!startBlobRanges.empty()) {
return startBlobRanges.front().begin == range.begin && startBlobRanges.front().end == range.end;
}
} else {
// An unblobbify request must be aligned to boundaries.
// It is okay to unblobbify multiple regions all at once.
if (startBlobRanges.empty() && endBlobRanges.empty()) {
if (startBlobRanges.empty()) {
// already unblobbified
return true;
} else if (startBlobRanges.front().begin != range.begin) {
// If there is a blob at the beginning of the range and it isn't aligned
return false;
}
// If there is a blob at the beginning of the range and it isn't aligned,
// or there is a blob range that begins before the end of the range, then fail.
if ((!startBlobRanges.empty() && startBlobRanges.front().begin != range.begin) ||
(!endBlobRanges.empty() && endBlobRanges.front().begin < range.end)) {
// if blob range does start at the specified, key, we need to make sure the end of also a boundary of a
// blob range
Optional<Value> endPresent = wait(tr->get(range.end.withPrefix(blobRangeKeys.begin)));
if (!endPresent.present()) {
return false;
}
}
@ -10018,10 +10041,6 @@ ACTOR Future<bool> setBlobRangeActor(Reference<DatabaseContext> cx, KeyRange ran
// This is not coalescing because we want to keep each range logically separate.
wait(krmSetRange(tr, blobRangeKeys.begin, range, value));
wait(tr->commit());
printf("Successfully updated blob range [%s - %s) to %s\n",
range.begin.printable().c_str(),
range.end.printable().c_str(),
value.printable().c_str());
return true;
} catch (Error& e) {
wait(tr->onError(e));

View File

@ -642,6 +642,7 @@ void ServerKnobs::initialize(Randomize randomize, ClientKnobs* clientKnobs, IsSi
init( STORAGE_HARD_LIMIT_VERSION_OVERAGE, VERSIONS_PER_SECOND / 4.0 );
init( STORAGE_DURABILITY_LAG_HARD_MAX, 2000e6 ); if( smallStorageTarget ) STORAGE_DURABILITY_LAG_HARD_MAX = 100e6;
init( STORAGE_DURABILITY_LAG_SOFT_MAX, 250e6 ); if( smallStorageTarget ) STORAGE_DURABILITY_LAG_SOFT_MAX = 10e6;
init( STORAGE_INCLUDE_FEED_STORAGE_QUEUE, true ); if ( randomize && BUGGIFY ) STORAGE_INCLUDE_FEED_STORAGE_QUEUE = false;
//FIXME: Low priority reads are disabled by assigning very high knob values, reduce knobs for 7.0
init( LOW_PRIORITY_STORAGE_QUEUE_BYTES, 775e8 ); if( smallStorageTarget ) LOW_PRIORITY_STORAGE_QUEUE_BYTES = 1750e3;

View File

@ -156,6 +156,11 @@ ACTOR Future<Void> moveKeySelectorOverRangeActor(const SpecialKeyRangeReadImpl*
// never being called if KeySelector is already normalized
ASSERT(ks->offset != 1);
// Throw error if module doesn't support tenants and we have a tenant
if (ryw->getTenant().present() && !skrImpl->supportsTenants()) {
throw illegal_tenant_access();
}
state Key startKey(skrImpl->getKeyRange().begin);
state Key endKey(skrImpl->getKeyRange().end);
state RangeResult result;
@ -376,6 +381,21 @@ ACTOR Future<RangeResult> SpecialKeySpace::getRangeAggregationActor(SpecialKeySp
}
state RangeMap<Key, SpecialKeyRangeReadImpl*, KeyRangeRef>::Ranges ranges =
sks->getReadImpls().intersectingRanges(KeyRangeRef(begin.getKey(), end.getKey()));
// Check tenant legality separately from below iterations
// because it may be partially completed and returned
// before illegal range is checked due to the limits handler
if (ryw->getTenant().present()) {
for (auto iter : ranges) {
if (iter->value() == nullptr) {
continue;
}
if (!iter->value()->supportsTenants()) {
throw illegal_tenant_access();
}
}
}
// TODO : workaround to write this two together to make the code compact
// The issue here is boost::iterator_range<> doest not provide rbegin(), rend()
iter = reverse ? ranges.end() : ranges.begin();
@ -501,6 +521,9 @@ void SpecialKeySpace::set(ReadYourWritesTransaction* ryw, const KeyRef& key, con
.detail("Value", value.toString());
throw special_keys_no_write_module_found();
}
if (!impl->supportsTenants() && ryw->getTenant().present()) {
throw illegal_tenant_access();
}
return impl->set(ryw, key, value);
}
@ -518,6 +541,9 @@ void SpecialKeySpace::clear(ReadYourWritesTransaction* ryw, const KeyRangeRef& r
TraceEvent(SevDebug, "SpecialKeySpaceNoWriteModuleFound").detail("Range", range);
throw special_keys_no_write_module_found();
}
if (!begin->supportsTenants() && ryw->getTenant().present()) {
throw illegal_tenant_access();
}
return begin->clear(ryw, range);
}
@ -527,6 +553,9 @@ void SpecialKeySpace::clear(ReadYourWritesTransaction* ryw, const KeyRef& key) {
auto impl = writeImpls[key];
if (impl == nullptr)
throw special_keys_no_write_module_found();
if (!impl->supportsTenants() && ryw->getTenant().present()) {
throw illegal_tenant_access();
}
return impl->clear(ryw, key);
}
@ -614,6 +643,16 @@ ACTOR Future<Void> commitActor(SpecialKeySpace* sks, ReadYourWritesTransaction*
++iter;
}
state std::vector<SpecialKeyRangeRWImpl*>::const_iterator it;
// Check validity of tenant support before iterating through
// module ptrs and potentially getting partial commits
if (ryw->getTenant().present()) {
for (it = writeModulePtrs.begin(); it != writeModulePtrs.end(); ++it) {
if (!(*it)->supportsTenants()) {
throw illegal_tenant_access();
}
}
}
for (it = writeModulePtrs.begin(); it != writeModulePtrs.end(); ++it) {
Optional<std::string> msg = wait((*it)->commit(ryw));
if (msg.present()) {

View File

@ -123,24 +123,19 @@ void TenantMapEntry::setId(int64_t id) {
prefix = idToPrefix(id);
}
std::string TenantMapEntry::toJson(int apiVersion) const {
std::string TenantMapEntry::toJson() const {
json_spirit::mObject tenantEntry;
tenantEntry["id"] = id;
tenantEntry["encrypted"] = encrypted;
if (apiVersion >= ApiVersion::withTenantsV2().version()) {
json_spirit::mObject prefixObject;
std::string encodedPrefix = base64::encoder::from_string(prefix.toString());
// Remove trailing newline
encodedPrefix.resize(encodedPrefix.size() - 1);
json_spirit::mObject prefixObject;
std::string encodedPrefix = base64::encoder::from_string(prefix.toString());
// Remove trailing newline
encodedPrefix.resize(encodedPrefix.size() - 1);
prefixObject["base64"] = encodedPrefix;
prefixObject["printable"] = printable(prefix);
tenantEntry["prefix"] = prefixObject;
} else {
// This is not a standard encoding in JSON, and some libraries may not be able to easily decode it
tenantEntry["prefix"] = prefix.toString();
}
prefixObject["base64"] = encodedPrefix;
prefixObject["printable"] = printable(prefix);
tenantEntry["prefix"] = prefixObject;
tenantEntry["tenant_state"] = TenantMapEntry::tenantStateToString(tenantState);
if (assignedCluster.present()) {

View File

@ -1,43 +0,0 @@
/*
* TenantSpecialKeys.cpp
*
* This source file is part of the FoundationDB open source project
*
* Copyright 2013-2022 Apple Inc. and the FoundationDB project authors
*
* Licensed under the Apache License, Version 2.0 (the "License");
* you may not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
#include "fdbclient/TenantSpecialKeys.actor.h"
template <>
const KeyRangeRef TenantRangeImpl<true>::submoduleRange = KeyRangeRef("tenant/"_sr, "tenant0"_sr);
template <>
const KeyRangeRef TenantRangeImpl<true>::mapSubRange = KeyRangeRef("map/"_sr, "map0"_sr);
template <>
const KeyRangeRef TenantRangeImpl<false>::submoduleRange = KeyRangeRef(""_sr, "\xff"_sr);
template <>
const KeyRangeRef TenantRangeImpl<false>::mapSubRange = KeyRangeRef("tenant_map/"_sr, "tenant_map0"_sr);
template <>
bool TenantRangeImpl<true>::subRangeIntersects(KeyRangeRef subRange, KeyRangeRef range) {
return subRange.intersects(range);
}
template <>
bool TenantRangeImpl<false>::subRangeIntersects(KeyRangeRef subRange, KeyRangeRef range) {
return subRange == mapSubRange;
}

View File

@ -88,6 +88,7 @@ ThreadFuture<int64_t> ThreadSafeDatabase::rebootWorker(const StringRef& address,
DatabaseContext* db = this->db;
Key addressKey = address;
return onMainThread([db, addressKey, check, duration]() -> Future<int64_t> {
db->checkDeferredError();
return db->rebootWorker(addressKey, check, duration);
});
}
@ -95,14 +96,20 @@ ThreadFuture<int64_t> ThreadSafeDatabase::rebootWorker(const StringRef& address,
ThreadFuture<Void> ThreadSafeDatabase::forceRecoveryWithDataLoss(const StringRef& dcid) {
DatabaseContext* db = this->db;
Key dcidKey = dcid;
return onMainThread([db, dcidKey]() -> Future<Void> { return db->forceRecoveryWithDataLoss(dcidKey); });
return onMainThread([db, dcidKey]() -> Future<Void> {
db->checkDeferredError();
return db->forceRecoveryWithDataLoss(dcidKey);
});
}
ThreadFuture<Void> ThreadSafeDatabase::createSnapshot(const StringRef& uid, const StringRef& snapshot_command) {
DatabaseContext* db = this->db;
Key snapUID = uid;
Key cmd = snapshot_command;
return onMainThread([db, snapUID, cmd]() -> Future<Void> { return db->createSnapshot(snapUID, cmd); });
return onMainThread([db, snapUID, cmd]() -> Future<Void> {
db->checkDeferredError();
return db->createSnapshot(snapUID, cmd);
});
}
ThreadFuture<DatabaseSharedState*> ThreadSafeDatabase::createSharedState() {
@ -126,14 +133,17 @@ double ThreadSafeDatabase::getMainThreadBusyness() {
// Note: this will never return if the server is running a protocol from FDB 5.0 or older
ThreadFuture<ProtocolVersion> ThreadSafeDatabase::getServerProtocol(Optional<ProtocolVersion> expectedVersion) {
DatabaseContext* db = this->db;
return onMainThread(
[db, expectedVersion]() -> Future<ProtocolVersion> { return db->getClusterProtocol(expectedVersion); });
return onMainThread([db, expectedVersion]() -> Future<ProtocolVersion> {
db->checkDeferredError();
return db->getClusterProtocol(expectedVersion);
});
}
ThreadFuture<Key> ThreadSafeDatabase::purgeBlobGranules(const KeyRangeRef& keyRange, Version purgeVersion, bool force) {
DatabaseContext* db = this->db;
KeyRange range = keyRange;
return onMainThread([db, range, purgeVersion, force]() -> Future<Key> {
db->checkDeferredError();
return db->purgeBlobGranules(range, purgeVersion, {}, force);
});
}
@ -141,33 +151,47 @@ ThreadFuture<Key> ThreadSafeDatabase::purgeBlobGranules(const KeyRangeRef& keyRa
ThreadFuture<Void> ThreadSafeDatabase::waitPurgeGranulesComplete(const KeyRef& purgeKey) {
DatabaseContext* db = this->db;
Key key = purgeKey;
return onMainThread([db, key]() -> Future<Void> { return db->waitPurgeGranulesComplete(key); });
return onMainThread([db, key]() -> Future<Void> {
db->checkDeferredError();
return db->waitPurgeGranulesComplete(key);
});
}
ThreadFuture<bool> ThreadSafeDatabase::blobbifyRange(const KeyRangeRef& keyRange) {
DatabaseContext* db = this->db;
KeyRange range = keyRange;
return onMainThread([=]() -> Future<bool> { return db->blobbifyRange(range); });
return onMainThread([=]() -> Future<bool> {
db->checkDeferredError();
return db->blobbifyRange(range);
});
}
ThreadFuture<bool> ThreadSafeDatabase::unblobbifyRange(const KeyRangeRef& keyRange) {
DatabaseContext* db = this->db;
KeyRange range = keyRange;
return onMainThread([=]() -> Future<bool> { return db->blobbifyRange(range); });
return onMainThread([=]() -> Future<bool> {
db->checkDeferredError();
return db->unblobbifyRange(range);
});
}
ThreadFuture<Standalone<VectorRef<KeyRangeRef>>> ThreadSafeDatabase::listBlobbifiedRanges(const KeyRangeRef& keyRange,
int rangeLimit) {
DatabaseContext* db = this->db;
KeyRange range = keyRange;
return onMainThread(
[=]() -> Future<Standalone<VectorRef<KeyRangeRef>>> { return db->listBlobbifiedRanges(range, rangeLimit); });
return onMainThread([=]() -> Future<Standalone<VectorRef<KeyRangeRef>>> {
db->checkDeferredError();
return db->listBlobbifiedRanges(range, rangeLimit);
});
}
ThreadFuture<Version> ThreadSafeDatabase::verifyBlobRange(const KeyRangeRef& keyRange, Optional<Version> version) {
DatabaseContext* db = this->db;
KeyRange range = keyRange;
return onMainThread([=]() -> Future<Version> { return db->verifyBlobRange(range, version); });
return onMainThread([=]() -> Future<Version> {
db->checkDeferredError();
return db->verifyBlobRange(range, version);
});
}
ThreadSafeDatabase::ThreadSafeDatabase(ConnectionRecordType connectionRecordType,
@ -216,7 +240,10 @@ ThreadFuture<Key> ThreadSafeTenant::purgeBlobGranules(const KeyRangeRef& keyRang
ThreadFuture<Void> ThreadSafeTenant::waitPurgeGranulesComplete(const KeyRef& purgeKey) {
DatabaseContext* db = this->db->db;
Key key = purgeKey;
return onMainThread([db, key]() -> Future<Void> { return db->waitPurgeGranulesComplete(key); });
return onMainThread([db, key]() -> Future<Void> {
db->checkDeferredError();
return db->waitPurgeGranulesComplete(key);
});
}
ThreadSafeTenant::~ThreadSafeTenant() {}
@ -538,22 +565,34 @@ Version ThreadSafeTransaction::getCommittedVersion() {
ThreadFuture<VersionVector> ThreadSafeTransaction::getVersionVector() {
ISingleThreadTransaction* tr = this->tr;
return onMainThread([tr]() -> Future<VersionVector> { return tr->getVersionVector(); });
return onMainThread([tr]() -> Future<VersionVector> {
tr->checkDeferredError();
return tr->getVersionVector();
});
}
ThreadFuture<SpanContext> ThreadSafeTransaction::getSpanContext() {
ISingleThreadTransaction* tr = this->tr;
return onMainThread([tr]() -> Future<SpanContext> { return tr->getSpanContext(); });
return onMainThread([tr]() -> Future<SpanContext> {
tr->checkDeferredError();
return tr->getSpanContext();
});
}
ThreadFuture<int64_t> ThreadSafeTransaction::getApproximateSize() {
ISingleThreadTransaction* tr = this->tr;
return onMainThread([tr]() -> Future<int64_t> { return tr->getApproximateSize(); });
return onMainThread([tr]() -> Future<int64_t> {
tr->checkDeferredError();
return tr->getApproximateSize();
});
}
ThreadFuture<Standalone<StringRef>> ThreadSafeTransaction::getVersionstamp() {
ISingleThreadTransaction* tr = this->tr;
return onMainThread([tr]() -> Future<Standalone<StringRef>> { return tr->getVersionstamp(); });
return onMainThread([tr]() -> Future<Standalone<StringRef>> {
tr->checkDeferredError();
return tr->getVersionstamp();
});
}
void ThreadSafeTransaction::setOption(FDBTransactionOptions::Option option, Optional<StringRef> value) {

View File

@ -17,10 +17,11 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
#ifndef FLOW_BLOB_CIPHER_H
#define FLOW_BLOB_CIPHER_H
#ifndef FDBCLIENT_BLOB_CIPHER_H
#define FDBCLIENT_BLOB_CIPHER_H
#pragma once
#include "fdbrpc/Stats.h"
#include "flow/Arena.h"
#include "flow/EncryptUtils.h"
#include "flow/FastRef.h"
@ -28,6 +29,7 @@
#include "flow/genericactors.actor.h"
#include "flow/Knobs.h"
#include "flow/network.h"
#include "flow/Platform.h"
#include "flow/ProtocolVersion.h"
#include "flow/serialize.h"
@ -50,6 +52,59 @@
#define AES_256_KEY_LENGTH 32
#define AES_256_IV_LENGTH 16
class BlobCipherMetrics : public NonCopyable {
public:
static BlobCipherMetrics* getInstance() {
static BlobCipherMetrics* instance = nullptr;
if (instance == nullptr) {
instance = new BlobCipherMetrics;
}
return instance;
}
// Order of this enum has to match initializer of counterSets.
enum UsageType : int {
TLOG = 0,
KV_MEMORY,
KV_REDWOOD,
BLOB_GRANULE,
BACKUP,
TEST,
MAX,
};
struct CounterSet {
Counter encryptCPUTimeNS;
Counter decryptCPUTimeNS;
LatencySample getCipherKeysLatency;
LatencySample getLatestCipherKeysLatency;
CounterSet(CounterCollection& cc, std::string name);
};
static CounterSet& counters(UsageType t) {
ASSERT(t < UsageType::MAX);
return getInstance()->counterSets[int(t)];
}
private:
BlobCipherMetrics();
CounterCollection cc;
Future<Void> traceFuture;
public:
Counter cipherKeyCacheHit;
Counter cipherKeyCacheMiss;
Counter cipherKeyCacheExpired;
Counter latestCipherKeyCacheHit;
Counter latestCipherKeyCacheMiss;
Counter latestCipherKeyCacheNeedsRefresh;
LatencySample getCipherKeysLatency;
LatencySample getLatestCipherKeysLatency;
std::array<CounterSet, int(UsageType::MAX)> counterSets;
};
// Encryption operations buffer management
// Approach limits number of copies needed during encryption or decryption operations.
// For encryption EncryptBuf is allocated using client supplied Arena and provided to AES library to capture
@ -324,8 +379,7 @@ using BlobCipherKeyIdCacheMapCItr =
struct BlobCipherKeyIdCache : ReferenceCounted<BlobCipherKeyIdCache> {
public:
BlobCipherKeyIdCache();
explicit BlobCipherKeyIdCache(EncryptCipherDomainId dId);
explicit BlobCipherKeyIdCache(EncryptCipherDomainId dId, size_t* sizeStat);
BlobCipherKeyIdCacheKey getCacheKey(const EncryptCipherBaseKeyId& baseCipherId,
const EncryptCipherRandomSalt& salt);
@ -378,11 +432,15 @@ public:
// API returns list of all 'cached' cipherKeys
std::vector<Reference<BlobCipherKey>> getAllCipherKeys();
// Return number of cipher keys in the cahce.
size_t getSize() const { return keyIdCache.size(); }
private:
EncryptCipherDomainId domainId;
BlobCipherKeyIdCacheMap keyIdCache;
Optional<EncryptCipherBaseKeyId> latestBaseCipherKeyId;
Optional<EncryptCipherRandomSalt> latestRandomSalt;
size_t* sizeStat; // pointer to the outer BlobCipherKeyCache size count.
};
using BlobCipherDomainCacheMap = std::unordered_map<EncryptCipherDomainId, Reference<BlobCipherKeyIdCache>>;
@ -447,10 +505,19 @@ public:
// API enables dropping all 'cached' cipherKeys for a given encryption domain Id.
// Useful to cleanup cache if an encryption domain gets removed/destroyed etc.
void resetEncryptDomainId(const EncryptCipherDomainId domainId);
// Total number of cipher keys in the cache.
size_t getSize() const { return size; }
static Reference<BlobCipherKeyCache> getInstance() {
static bool cleanupRegistered = false;
if (!cleanupRegistered) {
// We try to avoid cipher keys appear in core dumps, so we clean them up before crash.
// TODO(yiwu): use of MADV_DONTDUMP instead of the crash handler.
registerCrashHandlerCallback(BlobCipherKeyCache::cleanup);
cleanupRegistered = true;
}
if (g_network->isSimulated()) {
return FlowSingleton<BlobCipherKeyCache>::getInstance(
[]() { return makeReference<BlobCipherKeyCache>(g_network->isSimulated()); });
@ -466,6 +533,7 @@ public:
private:
BlobCipherDomainCacheMap domainCacheMap;
size_t size = 0;
BlobCipherKeyCache() {}
};
@ -483,10 +551,12 @@ public:
Reference<BlobCipherKey> hCipherKey,
const uint8_t* iv,
const int ivLen,
const EncryptAuthTokenMode mode);
const EncryptAuthTokenMode mode,
BlobCipherMetrics::UsageType usageType);
EncryptBlobCipherAes265Ctr(Reference<BlobCipherKey> tCipherKey,
Reference<BlobCipherKey> hCipherKey,
const EncryptAuthTokenMode mode);
const EncryptAuthTokenMode mode,
BlobCipherMetrics::UsageType usageType);
~EncryptBlobCipherAes265Ctr();
Reference<EncryptBuf> encrypt(const uint8_t* plaintext,
@ -501,6 +571,7 @@ private:
Reference<BlobCipherKey> headerCipherKey;
EncryptAuthTokenMode authTokenMode;
uint8_t iv[AES_256_IV_LENGTH];
BlobCipherMetrics::UsageType usageType;
void init();
};
@ -512,7 +583,8 @@ class DecryptBlobCipherAes256Ctr final : NonCopyable, public ReferenceCounted<De
public:
DecryptBlobCipherAes256Ctr(Reference<BlobCipherKey> tCipherKey,
Reference<BlobCipherKey> hCipherKey,
const uint8_t* iv);
const uint8_t* iv,
BlobCipherMetrics::UsageType usageType);
~DecryptBlobCipherAes256Ctr();
Reference<EncryptBuf> decrypt(const uint8_t* ciphertext,
@ -531,6 +603,7 @@ private:
Reference<BlobCipherKey> headerCipherKey;
bool headerAuthTokenValidationDone;
bool authTokensValidationDone;
BlobCipherMetrics::UsageType usageType;
void verifyEncryptHeaderMetadata(const BlobCipherEncryptHeader& header);
void verifyAuthTokens(const uint8_t* ciphertext,
@ -567,4 +640,4 @@ StringRef computeAuthToken(const uint8_t* payload,
const int keyLen,
Arena& arena);
#endif // FLOW_BLOB_CIPHER_H
#endif // FDBCLIENT_BLOB_CIPHER_H

View File

@ -22,10 +22,10 @@
#define FDBCLIENT_BLOBGRANULECOMMON_H
#pragma once
#include "fdbclient/BlobCipher.h"
#include "fdbclient/CommitTransaction.h"
#include "fdbclient/FDBTypes.h"
#include "flow/BlobCipher.h"
#include "flow/EncryptUtils.h"
#include "flow/IRandom.h"
#include "flow/serialize.h"

View File

@ -286,6 +286,9 @@ public:
double METACLUSTER_ASSIGNMENT_AVAILABILITY_TIMEOUT;
int TENANT_ENTRY_CACHE_LIST_REFRESH_INTERVAL; // How often the TenantEntryCache is refreshed
// Encryption-at-rest
bool ENABLE_ENCRYPTION_CPU_TIME_LOGGING;
ClientKnobs(Randomize randomize);
void initialize(Randomize randomize);
};

View File

@ -197,7 +197,7 @@ struct CommitTransactionRequest : TimedRequest {
template <class Ar>
void serialize(Ar& ar) {
serializer(
ar, transaction, reply, arena, flags, debugID, commitCostEstimation, tagSet, spanContext, tenantInfo);
ar, transaction, reply, flags, debugID, commitCostEstimation, tagSet, spanContext, tenantInfo, arena);
}
};
@ -339,7 +339,7 @@ struct GetKeyServerLocationsReply {
template <class Ar>
void serialize(Ar& ar) {
serializer(ar, results, resultsTssMapping, tenantEntry, arena, resultsTagMapping);
serializer(ar, results, resultsTssMapping, tenantEntry, resultsTagMapping, arena);
}
};
@ -543,7 +543,7 @@ struct ProxySnapRequest {
template <class Ar>
void serialize(Ar& ar) {
serializer(ar, snapPayload, snapUID, reply, arena, debugID);
serializer(ar, snapPayload, snapUID, reply, debugID, arena);
}
};

View File

@ -22,10 +22,10 @@
#define FLOW_FDBCLIENT_COMMITTRANSACTION_H
#pragma once
#include "fdbclient/BlobCipher.h"
#include "fdbclient/FDBTypes.h"
#include "fdbclient/Knobs.h"
#include "fdbclient/Tracing.h"
#include "flow/BlobCipher.h"
// The versioned message has wire format : -1, version, messages
static const int32_t VERSION_HEADER = -1;
@ -141,7 +141,8 @@ struct MutationRef {
MutationRef encrypt(const std::unordered_map<EncryptCipherDomainId, Reference<BlobCipherKey>>& cipherKeys,
const EncryptCipherDomainId& domainId,
Arena& arena) const {
Arena& arena,
BlobCipherMetrics::UsageType usageType) const {
ASSERT_NE(domainId, ENCRYPT_INVALID_DOMAIN_ID);
auto textCipherItr = cipherKeys.find(domainId);
auto headerCipherItr = cipherKeys.find(ENCRYPT_HEADER_DOMAIN_ID);
@ -155,7 +156,8 @@ struct MutationRef {
headerCipherItr->second,
iv,
AES_256_IV_LENGTH,
ENCRYPT_HEADER_AUTH_TOKEN_MODE_SINGLE);
ENCRYPT_HEADER_AUTH_TOKEN_MODE_SINGLE,
usageType);
BlobCipherEncryptHeader* header = new (arena) BlobCipherEncryptHeader;
StringRef headerRef(reinterpret_cast<const uint8_t*>(header), sizeof(BlobCipherEncryptHeader));
StringRef payload =
@ -164,19 +166,21 @@ struct MutationRef {
}
MutationRef encryptMetadata(const std::unordered_map<EncryptCipherDomainId, Reference<BlobCipherKey>>& cipherKeys,
Arena& arena) const {
return encrypt(cipherKeys, SYSTEM_KEYSPACE_ENCRYPT_DOMAIN_ID, arena);
Arena& arena,
BlobCipherMetrics::UsageType usageType) const {
return encrypt(cipherKeys, SYSTEM_KEYSPACE_ENCRYPT_DOMAIN_ID, arena, usageType);
}
MutationRef decrypt(const std::unordered_map<BlobCipherDetails, Reference<BlobCipherKey>>& cipherKeys,
Arena& arena,
BlobCipherMetrics::UsageType usageType,
StringRef* buf = nullptr) const {
const BlobCipherEncryptHeader* header = encryptionHeader();
auto textCipherItr = cipherKeys.find(header->cipherTextDetails);
auto headerCipherItr = cipherKeys.find(header->cipherHeaderDetails);
ASSERT(textCipherItr != cipherKeys.end() && textCipherItr->second.isValid());
ASSERT(headerCipherItr != cipherKeys.end() && headerCipherItr->second.isValid());
DecryptBlobCipherAes256Ctr cipher(textCipherItr->second, headerCipherItr->second, header->iv);
DecryptBlobCipherAes256Ctr cipher(textCipherItr->second, headerCipherItr->second, header->iv, usageType);
StringRef plaintext = cipher.decrypt(param2.begin(), param2.size(), *header, arena)->toStringRef();
if (buf != nullptr) {
*buf = plaintext;

View File

@ -86,6 +86,8 @@ public:
std::vector<NetworkAddress> coords;
std::vector<Hostname> hostnames;
size_t getNumberOfCoordinators() const { return coords.size() + hostnames.size(); }
bool operator==(const ClusterConnectionString& other) const noexcept {
return key == other.key && keyDesc == other.keyDesc && coords == other.coords && hostnames == other.hostnames;
}

View File

@ -132,7 +132,7 @@ struct EKPGetBaseCipherKeysByIdsReply {
template <class Ar>
void serialize(Ar& ar) {
serializer(ar, arena, baseCipherDetails, numHits, error);
serializer(ar, baseCipherDetails, numHits, error, arena);
}
};

View File

@ -26,7 +26,7 @@
#define FDBCLIENT_EVENTTYPES_ACTOR_G_H
#include "fdbclient/EventTypes.actor.g.h"
#elif !defined(FDBCLIENT_EVENTTYPES_ACTOR_H)
#define FDBCLIENT_EVENTTYPESS_ACTOR_H
#define FDBCLIENT_EVENTTYPES_ACTOR_H
#include "flow/flow.h"
#include "flow/TDMetric.actor.h"

View File

@ -24,8 +24,10 @@
#elif !defined(FDBCLIENT_GETCIPHERKEYS_ACTOR_H)
#define FDBCLIENT_GETCIPHERKEYS_ACTOR_H
#include "fdbclient/BlobCipher.h"
#include "fdbclient/EncryptKeyProxyInterface.h"
#include "flow/BlobCipher.h"
#include "fdbrpc/Stats.h"
#include "flow/Knobs.h"
#include "flow/IRandom.h"
#include <unordered_map>
@ -88,7 +90,8 @@ Future<EKPGetLatestBaseCipherKeysReply> getUncachedLatestEncryptCipherKeys(Refer
ACTOR template <class T>
Future<std::unordered_map<EncryptCipherDomainId, Reference<BlobCipherKey>>> getLatestEncryptCipherKeys(
Reference<AsyncVar<T> const> db,
std::unordered_map<EncryptCipherDomainId, EncryptCipherDomainNameRef> domains) {
std::unordered_map<EncryptCipherDomainId, EncryptCipherDomainNameRef> domains,
BlobCipherMetrics::UsageType usageType) {
state Reference<BlobCipherKeyCache> cipherKeyCache = BlobCipherKeyCache::getInstance();
state std::unordered_map<EncryptCipherDomainId, Reference<BlobCipherKey>> cipherKeys;
state EKPGetLatestBaseCipherKeysRequest request;
@ -112,6 +115,7 @@ Future<std::unordered_map<EncryptCipherDomainId, Reference<BlobCipherKey>>> getL
return cipherKeys;
}
// Fetch any uncached cipher keys.
state double startTime = now();
loop choose {
when(EKPGetLatestBaseCipherKeysReply reply = wait(getUncachedLatestEncryptCipherKeys(db, request))) {
// Insert base cipher keys into cache and construct result.
@ -140,6 +144,9 @@ Future<std::unordered_map<EncryptCipherDomainId, Reference<BlobCipherKey>>> getL
// In case encryptKeyProxy has changed, retry the request.
when(wait(onEncryptKeyProxyChange(db))) {}
}
double elapsed = now() - startTime;
BlobCipherMetrics::getInstance()->getLatestCipherKeysLatency.addMeasurement(elapsed);
BlobCipherMetrics::counters(usageType).getLatestCipherKeysLatency.addMeasurement(elapsed);
return cipherKeys;
}
@ -178,7 +185,8 @@ using BaseCipherIndex = std::pair<EncryptCipherDomainId, EncryptCipherBaseKeyId>
ACTOR template <class T>
Future<std::unordered_map<BlobCipherDetails, Reference<BlobCipherKey>>> getEncryptCipherKeys(
Reference<AsyncVar<T> const> db,
std::unordered_set<BlobCipherDetails> cipherDetails) {
std::unordered_set<BlobCipherDetails> cipherDetails,
BlobCipherMetrics::UsageType usageType) {
state Reference<BlobCipherKeyCache> cipherKeyCache = BlobCipherKeyCache::getInstance();
state std::unordered_map<BlobCipherDetails, Reference<BlobCipherKey>> cipherKeys;
state std::unordered_set<BaseCipherIndex, boost::hash<BaseCipherIndex>> uncachedBaseCipherIds;
@ -207,6 +215,7 @@ Future<std::unordered_map<BlobCipherDetails, Reference<BlobCipherKey>>> getEncry
id.first /*domainId*/, id.second /*baseCipherId*/, StringRef() /*domainName*/, request.arena);
}
// Fetch any uncached cipher keys.
state double startTime = now();
loop choose {
when(EKPGetBaseCipherKeysByIdsReply reply = wait(getUncachedEncryptCipherKeys(db, request))) {
std::unordered_map<BaseCipherIndex, EKPBaseCipherDetails, boost::hash<BaseCipherIndex>> baseCipherKeys;
@ -242,6 +251,9 @@ Future<std::unordered_map<BlobCipherDetails, Reference<BlobCipherKey>>> getEncry
// In case encryptKeyProxy has changed, retry the request.
when(wait(onEncryptKeyProxyChange(db))) {}
}
double elapsed = now() - startTime;
BlobCipherMetrics::getInstance()->getCipherKeysLatency.addMeasurement(elapsed);
BlobCipherMetrics::counters(usageType).getCipherKeysLatency.addMeasurement(elapsed);
return cipherKeys;
}
@ -253,12 +265,13 @@ struct TextAndHeaderCipherKeys {
ACTOR template <class T>
Future<TextAndHeaderCipherKeys> getLatestEncryptCipherKeysForDomain(Reference<AsyncVar<T> const> db,
EncryptCipherDomainId domainId,
EncryptCipherDomainNameRef domainName) {
EncryptCipherDomainNameRef domainName,
BlobCipherMetrics::UsageType usageType) {
std::unordered_map<EncryptCipherDomainId, EncryptCipherDomainNameRef> domains;
domains[domainId] = domainName;
domains[ENCRYPT_HEADER_DOMAIN_ID] = FDB_DEFAULT_ENCRYPT_DOMAIN_NAME;
std::unordered_map<EncryptCipherDomainId, Reference<BlobCipherKey>> cipherKeys =
wait(getLatestEncryptCipherKeys(db, domains));
wait(getLatestEncryptCipherKeys(db, domains, usageType));
ASSERT(cipherKeys.count(domainId) > 0);
ASSERT(cipherKeys.count(ENCRYPT_HEADER_DOMAIN_ID) > 0);
TextAndHeaderCipherKeys result{ cipherKeys.at(domainId), cipherKeys.at(ENCRYPT_HEADER_DOMAIN_ID) };
@ -268,15 +281,19 @@ Future<TextAndHeaderCipherKeys> getLatestEncryptCipherKeysForDomain(Reference<As
}
template <class T>
Future<TextAndHeaderCipherKeys> getLatestSystemEncryptCipherKeys(const Reference<AsyncVar<T> const>& db) {
return getLatestEncryptCipherKeysForDomain(db, SYSTEM_KEYSPACE_ENCRYPT_DOMAIN_ID, FDB_DEFAULT_ENCRYPT_DOMAIN_NAME);
Future<TextAndHeaderCipherKeys> getLatestSystemEncryptCipherKeys(const Reference<AsyncVar<T> const>& db,
BlobCipherMetrics::UsageType usageType) {
return getLatestEncryptCipherKeysForDomain(
db, SYSTEM_KEYSPACE_ENCRYPT_DOMAIN_ID, FDB_DEFAULT_ENCRYPT_DOMAIN_NAME, usageType);
}
ACTOR template <class T>
Future<TextAndHeaderCipherKeys> getEncryptCipherKeys(Reference<AsyncVar<T> const> db, BlobCipherEncryptHeader header) {
Future<TextAndHeaderCipherKeys> getEncryptCipherKeys(Reference<AsyncVar<T> const> db,
BlobCipherEncryptHeader header,
BlobCipherMetrics::UsageType usageType) {
std::unordered_set<BlobCipherDetails> cipherDetails{ header.cipherTextDetails, header.cipherHeaderDetails };
std::unordered_map<BlobCipherDetails, Reference<BlobCipherKey>> cipherKeys =
wait(getEncryptCipherKeys(db, cipherDetails));
wait(getEncryptCipherKeys(db, cipherDetails, usageType));
ASSERT(cipherKeys.count(header.cipherTextDetails) > 0);
ASSERT(cipherKeys.count(header.cipherHeaderDetails) > 0);
TextAndHeaderCipherKeys result{ cipherKeys.at(header.cipherTextDetails),

View File

@ -368,7 +368,7 @@ struct FdbCApi : public ThreadSafeReferenceCounted<FdbCApi> {
fdb_error_t (*futureGetDatabase)(FDBFuture* f, FDBDatabase** outDb);
fdb_error_t (*futureGetInt64)(FDBFuture* f, int64_t* outValue);
fdb_error_t (*futureGetUInt64)(FDBFuture* f, uint64_t* outValue);
fdb_error_t (*futureGetBool)(FDBFuture* f, bool* outValue);
fdb_error_t (*futureGetBool)(FDBFuture* f, fdb_bool_t* outValue);
fdb_error_t (*futureGetError)(FDBFuture* f);
fdb_error_t (*futureGetKey)(FDBFuture* f, uint8_t const** outKey, int* outKeyLength);
fdb_error_t (*futureGetValue)(FDBFuture* f, fdb_bool_t* outPresent, uint8_t const** outValue, int* outValueLength);

View File

@ -573,6 +573,7 @@ public:
int64_t STORAGE_HARD_LIMIT_VERSION_OVERAGE;
int64_t STORAGE_DURABILITY_LAG_HARD_MAX;
int64_t STORAGE_DURABILITY_LAG_SOFT_MAX;
bool STORAGE_INCLUDE_FEED_STORAGE_QUEUE;
int64_t LOW_PRIORITY_STORAGE_QUEUE_BYTES;
int64_t LOW_PRIORITY_DURABILITY_LAG;

View File

@ -60,6 +60,8 @@ public:
// TODO : give this function a more descriptive name
virtual bool isAsync() const { return false; }
virtual bool supportsTenants() const { return false; }
virtual ~SpecialKeyRangeReadImpl() {}
protected:
@ -301,6 +303,7 @@ public:
Future<RangeResult> getRange(ReadYourWritesTransaction* ryw,
KeyRangeRef kr,
GetRangeLimits limitsHint) const override;
bool supportsTenants() const override { return true; };
};
class ReadConflictRangeImpl : public SpecialKeyRangeReadImpl {
@ -309,6 +312,7 @@ public:
Future<RangeResult> getRange(ReadYourWritesTransaction* ryw,
KeyRangeRef kr,
GetRangeLimits limitsHint) const override;
bool supportsTenants() const override { return true; };
};
class WriteConflictRangeImpl : public SpecialKeyRangeReadImpl {
@ -317,6 +321,7 @@ public:
Future<RangeResult> getRange(ReadYourWritesTransaction* ryw,
KeyRangeRef kr,
GetRangeLimits limitsHint) const override;
bool supportsTenants() const override { return true; };
};
class DDStatsRangeImpl : public SpecialKeyRangeAsyncImpl {

View File

@ -416,8 +416,8 @@ struct GetKeyValuesRequest : TimedRequest {
spanContext,
tenantInfo,
options,
arena,
ssLatestCommitVersions);
ssLatestCommitVersions,
arena);
}
};
@ -474,9 +474,9 @@ struct GetMappedKeyValuesRequest : TimedRequest {
spanContext,
tenantInfo,
options,
arena,
ssLatestCommitVersions,
matchIndex);
matchIndex,
arena);
}
};
@ -539,8 +539,8 @@ struct GetKeyValuesStreamRequest {
spanContext,
tenantInfo,
options,
arena,
ssLatestCommitVersions);
ssLatestCommitVersions,
arena);
}
};
@ -588,7 +588,7 @@ struct GetKeyRequest : TimedRequest {
template <class Ar>
void serialize(Ar& ar) {
serializer(ar, sel, version, tags, reply, spanContext, tenantInfo, options, arena, ssLatestCommitVersions);
serializer(ar, sel, version, tags, reply, spanContext, tenantInfo, options, ssLatestCommitVersions, arena);
}
};
@ -758,7 +758,7 @@ struct SplitMetricsRequest {
template <class Ar>
void serialize(Ar& ar) {
serializer(ar, keys, limits, used, estimated, isLastShard, reply, arena, minSplitBytes);
serializer(ar, keys, limits, used, estimated, isLastShard, reply, minSplitBytes, arena);
}
};
@ -1038,7 +1038,7 @@ struct OverlappingChangeFeedsReply {
template <class Ar>
void serialize(Ar& ar) {
serializer(ar, feeds, arena, feedMetadataVersion);
serializer(ar, feeds, feedMetadataVersion, arena);
}
};

View File

@ -96,7 +96,7 @@ struct TenantMapEntry {
TenantMapEntry(int64_t id, TenantState tenantState, Optional<TenantGroupName> tenantGroup, bool encrypted);
void setId(int64_t id);
std::string toJson(int apiVersion) const;
std::string toJson() const;
bool matchesConfiguration(TenantMapEntry const& other) const;
void configure(Standalone<StringRef> parameter, Optional<Value> value);

View File

@ -36,11 +36,8 @@
#include "flow/UnitTest.h"
#include "flow/actorcompiler.h" // This must be the last #include.
template <bool HasSubRanges>
class TenantRangeImpl : public SpecialKeyRangeRWImpl {
private:
static bool subRangeIntersects(KeyRangeRef subRange, KeyRangeRef range);
static KeyRangeRef removePrefix(KeyRangeRef range, KeyRef prefix, KeyRef defaultEnd) {
KeyRef begin = range.begin.removePrefix(prefix);
KeyRef end;
@ -76,7 +73,7 @@ private:
wait(TenantAPI::listTenantsTransaction(&ryw->getTransaction(), kr.begin, kr.end, limitsHint.rows));
for (auto tenant : tenants) {
std::string jsonString = tenant.second.toJson(ryw->getDatabase()->apiVersion.version());
std::string jsonString = tenant.second.toJson();
ValueRef tenantEntryBytes(results->arena(), jsonString);
results->push_back(results->arena(),
KeyValueRef(withTenantMapPrefix(tenant.first, results->arena()), tenantEntryBytes));
@ -85,21 +82,20 @@ private:
return Void();
}
ACTOR template <bool B>
static Future<RangeResult> getTenantRange(ReadYourWritesTransaction* ryw,
KeyRangeRef kr,
GetRangeLimits limitsHint) {
ACTOR static Future<RangeResult> getTenantRange(ReadYourWritesTransaction* ryw,
KeyRangeRef kr,
GetRangeLimits limitsHint) {
state RangeResult results;
kr = kr.removePrefix(SpecialKeySpace::getModuleRange(SpecialKeySpace::MODULE::MANAGEMENT).begin)
.removePrefix(TenantRangeImpl<B>::submoduleRange.begin);
.removePrefix(TenantRangeImpl::submoduleRange.begin);
if (kr.intersects(TenantRangeImpl<B>::mapSubRange)) {
if (kr.intersects(TenantRangeImpl::mapSubRange)) {
GetRangeLimits limits = limitsHint;
limits.decrement(results);
wait(getTenantList(
ryw,
removePrefix(kr & TenantRangeImpl<B>::mapSubRange, TenantRangeImpl<B>::mapSubRange.begin, "\xff"_sr),
removePrefix(kr & TenantRangeImpl::mapSubRange, TenantRangeImpl::mapSubRange.begin, "\xff"_sr),
&results,
limits));
}
@ -254,11 +250,8 @@ private:
}
public:
// These ranges vary based on the template parameter
const static KeyRangeRef submoduleRange;
const static KeyRangeRef mapSubRange;
// These sub-ranges should only be used if HasSubRanges=true
const inline static KeyRangeRef submoduleRange = KeyRangeRef("tenant/"_sr, "tenant0"_sr);
const inline static KeyRangeRef mapSubRange = KeyRangeRef("map/"_sr, "map0"_sr);
const inline static KeyRangeRef configureSubRange = KeyRangeRef("configure/"_sr, "configure0"_sr);
const inline static KeyRangeRef renameSubRange = KeyRangeRef("rename/"_sr, "rename0"_sr);
@ -267,7 +260,7 @@ public:
Future<RangeResult> getRange(ReadYourWritesTransaction* ryw,
KeyRangeRef kr,
GetRangeLimits limitsHint) const override {
return getTenantRange<HasSubRanges>(ryw, kr, limitsHint);
return getTenantRange(ryw, kr, limitsHint);
}
ACTOR static Future<Optional<std::string>> commitImpl(TenantRangeImpl* self, ReadYourWritesTransaction* ryw) {
@ -301,11 +294,11 @@ public:
.removePrefix(SpecialKeySpace::getModuleRange(SpecialKeySpace::MODULE::MANAGEMENT).begin)
.removePrefix(submoduleRange.begin);
if (subRangeIntersects(mapSubRange, adjustedRange)) {
if (mapSubRange.intersects(adjustedRange)) {
adjustedRange = mapSubRange & adjustedRange;
adjustedRange = removePrefix(adjustedRange, mapSubRange.begin, "\xff"_sr);
mapMutations.push_back(std::make_pair(adjustedRange, range.value().second));
} else if (subRangeIntersects(configureSubRange, adjustedRange) && adjustedRange.singleKeyRange()) {
} else if (configureSubRange.intersects(adjustedRange) && adjustedRange.singleKeyRange()) {
StringRef configTupleStr = adjustedRange.begin.removePrefix(configureSubRange.begin);
try {
Tuple tuple = Tuple::unpack(configTupleStr);
@ -320,7 +313,7 @@ public:
false, "configure tenant", "invalid tenant configuration key"));
throw special_keys_api_failure();
}
} else if (subRangeIntersects(renameSubRange, adjustedRange)) {
} else if (renameSubRange.intersects(adjustedRange)) {
StringRef oldName = adjustedRange.begin.removePrefix(renameSubRange.begin);
StringRef newName = range.value().second.get();
// Do not allow overlapping renames in the same commit

View File

@ -26,6 +26,7 @@
#include "fdbrpc/TokenCache.h"
#include "fdbrpc/FlowTransport.h"
#include "flow/Arena.h"
#include "flow/Knobs.h"
struct TenantInfo {
static constexpr const int64_t INVALID_TENANT = -1;
@ -69,8 +70,8 @@ struct serializable_traits<TenantInfo> : std::true_type {
static void serialize(Archiver& ar, TenantInfo& v) {
serializer(ar, v.name, v.tenantId, v.token, v.arena);
if constexpr (Archiver::isDeserializing) {
bool tenantAuthorized = false;
if (v.name.present() && v.token.present()) {
bool tenantAuthorized = FLOW_KNOBS->ALLOW_TOKENLESS_TENANT_ACCESS;
if (!tenantAuthorized && v.name.present() && v.token.present()) {
tenantAuthorized = TokenCache::instance().validate(v.name.get(), v.token.get());
}
v.trusted = FlowTransport::transport().currentDeliveryPeerIsTrusted();

View File

@ -168,7 +168,7 @@ private:
} else {
ASSERT(cipherKeys != nullptr);
Arena arena;
toCommit->writeTypedMessage(m.encryptMetadata(*cipherKeys, arena));
toCommit->writeTypedMessage(m.encryptMetadata(*cipherKeys, arena, BlobCipherMetrics::TLOG));
}
}

View File

@ -20,6 +20,7 @@
#include "fdbclient/BackupAgent.actor.h"
#include "fdbclient/BackupContainer.h"
#include "fdbclient/BlobCipher.h"
#include "fdbclient/DatabaseContext.h"
#include "fdbclient/CommitProxyInterface.h"
#include "fdbclient/SystemData.h"
@ -79,7 +80,7 @@ struct VersionedMessage {
// In case the mutation is encrypted, get the decrypted mutation and also update message to point to
// the decrypted mutation.
// We use dedicated arena for decrypt buffer, as the other arena is used to count towards backup lock bytes.
*m = m->decrypt(cipherKeys, decryptArena, &message);
*m = m->decrypt(cipherKeys, decryptArena, BlobCipherMetrics::BACKUP, &message);
}
return normalKeys.contains(m->param1) || m->param1 == metadataVersionKey;
}
@ -780,7 +781,7 @@ ACTOR Future<Void> saveMutationsToFile(BackupData* self,
// Fetch cipher keys if any of the messages are encrypted.
if (!cipherDetails.empty()) {
std::unordered_map<BlobCipherDetails, Reference<BlobCipherKey>> getCipherKeysResult =
wait(getEncryptCipherKeys(self->db, cipherDetails));
wait(getEncryptCipherKeys(self->db, cipherDetails, BlobCipherMetrics::BLOB_GRANULE));
cipherKeys = getCipherKeysResult;
}

View File

@ -19,6 +19,7 @@
*/
#include "fdbclient/ClientBooleanParams.h"
#include "fdbclient/BlobCipher.h"
#include "fdbclient/BlobGranuleFiles.h"
#include "fdbclient/FDBTypes.h"
#include "fdbclient/KeyRangeMap.h"
@ -44,7 +45,6 @@
#include "fdbserver/WaitFailure.h"
#include "flow/Arena.h"
#include "flow/BlobCipher.h"
#include "flow/CompressionUtils.h"
#include "flow/EncryptUtils.h"
#include "flow/Error.h"
@ -360,13 +360,14 @@ ACTOR Future<BlobGranuleCipherKeysCtx> getLatestGranuleCipherKeys(Reference<Blob
std::unordered_map<EncryptCipherDomainId, EncryptCipherDomainNameRef> domains;
domains.emplace(tenantData->entry.id, StringRef(*arena, tenantData->name));
std::unordered_map<EncryptCipherDomainId, Reference<BlobCipherKey>> domainKeyMap =
wait(getLatestEncryptCipherKeys(bwData->dbInfo, domains));
wait(getLatestEncryptCipherKeys(bwData->dbInfo, domains, BlobCipherMetrics::BLOB_GRANULE));
auto domainKeyItr = domainKeyMap.find(tenantData->entry.id);
ASSERT(domainKeyItr != domainKeyMap.end());
cipherKeysCtx.textCipherKey = BlobGranuleCipherKey::fromBlobCipherKey(domainKeyItr->second, *arena);
TextAndHeaderCipherKeys systemCipherKeys = wait(getLatestSystemEncryptCipherKeys(bwData->dbInfo));
TextAndHeaderCipherKeys systemCipherKeys =
wait(getLatestSystemEncryptCipherKeys(bwData->dbInfo, BlobCipherMetrics::BLOB_GRANULE));
cipherKeysCtx.headerCipherKey = BlobGranuleCipherKey::fromBlobCipherKey(systemCipherKeys.cipherHeaderKey, *arena);
cipherKeysCtx.ivRef = makeString(AES_256_IV_LENGTH, *arena);
@ -392,7 +393,7 @@ ACTOR Future<BlobGranuleCipherKey> lookupCipherKey(Reference<BlobWorkerData> bwD
std::unordered_set<BlobCipherDetails> cipherDetailsSet;
cipherDetailsSet.emplace(cipherDetails);
state std::unordered_map<BlobCipherDetails, Reference<BlobCipherKey>> cipherKeyMap =
wait(getEncryptCipherKeys(bwData->dbInfo, cipherDetailsSet));
wait(getEncryptCipherKeys(bwData->dbInfo, cipherDetailsSet, BlobCipherMetrics::BLOB_GRANULE));
ASSERT(cipherKeyMap.size() == 1);
@ -1873,6 +1874,7 @@ ACTOR Future<Void> blobGranuleUpdateFiles(Reference<BlobWorkerData> bwData,
state int pendingSnapshots = 0;
state Version lastForceFlushVersion = invalidVersion;
state std::deque<Version> forceFlushVersions;
state Future<Void> nextForceFlush = metadata->forceFlushVersion.whenAtLeast(1);
state std::deque<std::pair<Version, Version>> rollbacksInProgress;
state std::deque<std::pair<Version, Version>> rollbacksCompleted;
@ -2113,7 +2115,7 @@ ACTOR Future<Void> blobGranuleUpdateFiles(Reference<BlobWorkerData> bwData,
}
}
when(wait(inFlightFiles.empty() ? Never() : success(inFlightFiles.front().future))) {}
when(wait(metadata->forceFlushVersion.whenAtLeast(lastForceFlushVersion + 1))) {
when(wait(nextForceFlush)) {
if (forceFlushVersions.empty() ||
forceFlushVersions.back() < metadata->forceFlushVersion.get()) {
forceFlushVersions.push_back(metadata->forceFlushVersion.get());
@ -2121,6 +2123,7 @@ ACTOR Future<Void> blobGranuleUpdateFiles(Reference<BlobWorkerData> bwData,
if (metadata->forceFlushVersion.get() > lastForceFlushVersion) {
lastForceFlushVersion = metadata->forceFlushVersion.get();
}
nextForceFlush = metadata->forceFlushVersion.whenAtLeast(lastForceFlushVersion + 1);
}
}
} catch (Error& e) {
@ -2255,6 +2258,7 @@ ACTOR Future<Void> blobGranuleUpdateFiles(Reference<BlobWorkerData> bwData,
forceFlushVersions.clear();
lastForceFlushVersion = 0;
metadata->forceFlushVersion = NotifiedVersion();
nextForceFlush = metadata->forceFlushVersion.whenAtLeast(1);
Reference<ChangeFeedData> cfData =
makeReference<ChangeFeedData>(bwData->db.getPtr());

View File

@ -22,6 +22,7 @@
#include <tuple>
#include "fdbclient/Atomic.h"
#include "fdbclient/BlobCipher.h"
#include "fdbclient/CommitTransaction.h"
#include "fdbclient/DatabaseContext.h"
#include "fdbclient/FDBTypes.h"
@ -50,7 +51,6 @@
#include "fdbserver/WaitFailure.h"
#include "fdbserver/WorkerInterface.actor.h"
#include "flow/ActorCollection.h"
#include "flow/BlobCipher.h"
#include "flow/Error.h"
#include "flow/IRandom.h"
#include "flow/Knobs.h"
@ -932,7 +932,7 @@ ACTOR Future<Void> getResolution(CommitBatchContext* self) {
encryptDomains[tenantId] = tenantName.get();
}
}
getCipherKeys = getLatestEncryptCipherKeys(pProxyCommitData->db, encryptDomains);
getCipherKeys = getLatestEncryptCipherKeys(pProxyCommitData->db, encryptDomains, BlobCipherMetrics::TLOG);
}
self->releaseFuture = releaseResolvingAfter(pProxyCommitData, self->releaseDelay, self->localBatchNumber);
@ -1167,7 +1167,8 @@ void writeMutation(CommitBatchContext* self, int64_t tenantId, const MutationRef
self->toCommit.writeTypedMessage(mutation);
} else {
Arena arena;
self->toCommit.writeTypedMessage(mutation.encrypt(self->cipherKeys, tenantId /*domainId*/, arena));
self->toCommit.writeTypedMessage(
mutation.encrypt(self->cipherKeys, tenantId /*domainId*/, arena, BlobCipherMetrics::TLOG));
}
}

View File

@ -21,7 +21,6 @@
#include "fdbclient/EncryptKeyProxyInterface.h"
#include "fdbrpc/Locality.h"
#include "fdbrpc/Stats.h"
#include "fdbserver/KmsConnector.h"
#include "fdbserver/KmsConnectorInterface.h"
#include "fdbserver/Knobs.h"
@ -223,6 +222,10 @@ public:
Counter blobMetadataRefreshed;
Counter numBlobMetadataRefreshErrors;
LatencySample kmsLookupByIdsReqLatency;
LatencySample kmsLookupByDomainIdsReqLatency;
LatencySample kmsBlobMetadataReqLatency;
explicit EncryptKeyProxyData(UID id)
: myId(id), ekpCacheMetrics("EKPMetrics", myId.toString()),
baseCipherKeyIdCacheMisses("EKPCipherIdCacheMisses", ekpCacheMetrics),
@ -235,7 +238,19 @@ public:
blobMetadataCacheHits("EKPBlobMetadataCacheHits", ekpCacheMetrics),
blobMetadataCacheMisses("EKPBlobMetadataCacheMisses", ekpCacheMetrics),
blobMetadataRefreshed("EKPBlobMetadataRefreshed", ekpCacheMetrics),
numBlobMetadataRefreshErrors("EKPBlobMetadataRefreshErrors", ekpCacheMetrics) {}
numBlobMetadataRefreshErrors("EKPBlobMetadataRefreshErrors", ekpCacheMetrics),
kmsLookupByIdsReqLatency("EKPKmsLookupByIdsReqLatency",
id,
SERVER_KNOBS->LATENCY_METRICS_LOGGING_INTERVAL,
SERVER_KNOBS->LATENCY_SAMPLE_SIZE),
kmsLookupByDomainIdsReqLatency("EKPKmsLookupByDomainIdsReqLatency",
id,
SERVER_KNOBS->LATENCY_METRICS_LOGGING_INTERVAL,
SERVER_KNOBS->LATENCY_SAMPLE_SIZE),
kmsBlobMetadataReqLatency("EKPKmsBlobMetadataReqLatency",
id,
SERVER_KNOBS->LATENCY_METRICS_LOGGING_INTERVAL,
SERVER_KNOBS->LATENCY_SAMPLE_SIZE) {}
EncryptBaseCipherDomainIdKeyIdCacheKey getBaseCipherDomainIdKeyIdCacheKey(
const EncryptCipherDomainId domainId,
@ -375,7 +390,9 @@ ACTOR Future<Void> getCipherKeysByBaseCipherKeyIds(Reference<EncryptKeyProxyData
keysByIdsReq.arena, item.second.domainId, item.second.baseCipherId, item.second.domainName);
}
keysByIdsReq.debugId = keysByIds.debugId;
state double startTime = now();
KmsConnLookupEKsByKeyIdsRep keysByIdsRep = wait(kmsConnectorInf.ekLookupByIds.getReply(keysByIdsReq));
ekpProxyData->kmsLookupByIdsReqLatency.addMeasurement(now() - startTime);
for (const auto& item : keysByIdsRep.cipherKeyDetails) {
keyIdsReply.baseCipherDetails.emplace_back(
@ -511,8 +528,10 @@ ACTOR Future<Void> getLatestCipherKeys(Reference<EncryptKeyProxyData> ekpProxyDa
}
keysByDomainIdReq.debugId = latestKeysReq.debugId;
state double startTime = now();
KmsConnLookupEKsByDomainIdsRep keysByDomainIdRep =
wait(kmsConnectorInf.ekLookupByDomainIds.getReply(keysByDomainIdReq));
ekpProxyData->kmsLookupByDomainIdsReqLatency.addMeasurement(now() - startTime);
for (auto& item : keysByDomainIdRep.cipherKeyDetails) {
CipherKeyValidityTS validityTS = getCipherKeyValidityTS(item.refreshAfterSec, item.expireAfterSec);
@ -609,7 +628,9 @@ ACTOR Future<Void> refreshEncryptionKeysCore(Reference<EncryptKeyProxyData> ekpP
}
}
state double startTime = now();
KmsConnLookupEKsByDomainIdsRep rep = wait(kmsConnectorInf.ekLookupByDomainIds.getReply(req));
ekpProxyData->kmsLookupByDomainIdsReqLatency.addMeasurement(now() - startTime);
for (const auto& item : rep.cipherKeyDetails) {
const auto itr = ekpProxyData->baseCipherDomainIdCache.find(item.encryptDomainId);
if (itr == ekpProxyData->baseCipherDomainIdCache.end()) {
@ -707,7 +728,9 @@ ACTOR Future<Void> getLatestBlobMetadata(Reference<EncryptKeyProxyData> ekpProxy
if (!lookupDomains.empty()) {
try {
KmsConnBlobMetadataReq kmsReq(lookupDomains, req.debugId);
state double startTime = now();
KmsConnBlobMetadataRep kmsRep = wait(kmsConnectorInf.blobMetadataReq.getReply(kmsReq));
ekpProxyData->kmsBlobMetadataReqLatency.addMeasurement(now() - startTime);
metadataDetails.arena().dependsOn(kmsRep.metadataDetails.arena());
for (auto& item : kmsRep.metadataDetails) {
@ -753,7 +776,9 @@ ACTOR Future<Void> refreshBlobMetadataCore(Reference<EncryptKeyProxyData> ekpPro
for (auto& item : ekpProxyData->blobMetadataDomainIdCache) {
req.domainIds.emplace_back(item.first);
}
state double startTime = now();
KmsConnBlobMetadataRep rep = wait(kmsConnectorInf.blobMetadataReq.getReply(req));
ekpProxyData->kmsBlobMetadataReqLatency.addMeasurement(now() - startTime);
for (auto& item : rep.metadataDetails) {
ekpProxyData->insertIntoBlobMetadataCache(item.domainId, item);
t.detail("BM" + std::to_string(item.domainId), "");

View File

@ -18,6 +18,7 @@
* limitations under the License.
*/
#include "fdbclient/BlobCipher.h"
#include "fdbclient/Knobs.h"
#include "fdbclient/Notified.h"
#include "fdbclient/SystemData.h"
@ -488,8 +489,10 @@ private:
ASSERT(cipherKeys.cipherTextKey.isValid());
ASSERT(cipherKeys.cipherHeaderKey.isValid());
EncryptBlobCipherAes265Ctr cipher(
cipherKeys.cipherTextKey, cipherKeys.cipherHeaderKey, ENCRYPT_HEADER_AUTH_TOKEN_MODE_SINGLE);
EncryptBlobCipherAes265Ctr cipher(cipherKeys.cipherTextKey,
cipherKeys.cipherHeaderKey,
ENCRYPT_HEADER_AUTH_TOKEN_MODE_SINGLE,
BlobCipherMetrics::KV_MEMORY);
BlobCipherEncryptHeader cipherHeader;
Arena arena;
StringRef ciphertext =
@ -527,8 +530,10 @@ private:
return data;
}
state BlobCipherEncryptHeader cipherHeader = *(BlobCipherEncryptHeader*)data.begin();
TextAndHeaderCipherKeys cipherKeys = wait(getEncryptCipherKeys(self->db, cipherHeader));
DecryptBlobCipherAes256Ctr cipher(cipherKeys.cipherTextKey, cipherKeys.cipherHeaderKey, cipherHeader.iv);
TextAndHeaderCipherKeys cipherKeys =
wait(getEncryptCipherKeys(self->db, cipherHeader, BlobCipherMetrics::KV_MEMORY));
DecryptBlobCipherAes256Ctr cipher(
cipherKeys.cipherTextKey, cipherKeys.cipherHeaderKey, cipherHeader.iv, BlobCipherMetrics::KV_MEMORY);
Arena arena;
StringRef plaintext = cipher
.decrypt(data.begin() + BlobCipherEncryptHeader::headerSize,
@ -961,7 +966,8 @@ private:
}
ACTOR static Future<Void> updateCipherKeys(KeyValueStoreMemory* self) {
TextAndHeaderCipherKeys cipherKeys = wait(getLatestSystemEncryptCipherKeys(self->db));
TextAndHeaderCipherKeys cipherKeys =
wait(getLatestSystemEncryptCipherKeys(self->db, BlobCipherMetrics::KV_MEMORY));
self->cipherKeys = cipherKeys;
return Void();
}

View File

@ -22,6 +22,7 @@
#include <array>
#include "flow/Knobs.h"
#include "fdbclient/Knobs.h"
#include "fdbclient/ServerKnobCollection.h"
#include "fdbserver/Knobs.h"
@ -52,6 +53,9 @@ void KnobProtectiveGroup::snapshotOriginalKnobs() {
if (std::get_if<NoKnobFound>(&value)) {
value = SERVER_KNOBS->getKnob(name);
}
if (std::get_if<NoKnobFound>(&value)) {
value = FLOW_KNOBS->getKnob(name);
}
if (std::get_if<NoKnobFound>(&value)) {
ASSERT(false);
}
@ -70,4 +74,4 @@ void KnobProtectiveGroup::assignKnobs(const KnobKeyValuePairs& overrideKnobs) {
ASSERT(mutableServerKnobs.trySetKnob(name, valueRef));
TraceEvent(SevInfo, "AssignKnobValue").detail("KnobName", name).detail("KnobValue", valueRef.toString());
}
}
}

View File

@ -22,11 +22,11 @@
#include "fmt/format.h"
#include "fdbrpc/sim_validation.h"
#include "fdbclient/BlobCipher.h"
#include "fdbserver/KmsConnectorInterface.h"
#include "fdbserver/Knobs.h"
#include "flow/ActorCollection.h"
#include "flow/Arena.h"
#include "flow/BlobCipher.h"
#include "flow/EncryptUtils.h"
#include "flow/Error.h"
#include "flow/FastRef.h"

View File

@ -18,6 +18,7 @@
* limitations under the License.
*/
#include "fdbclient/BlobCipher.h"
#include "fdbclient/FDBTypes.h"
#include "fdbserver/OTELSpanContextMessage.h"
#include "flow/Arena.h"
@ -1925,7 +1926,7 @@ ACTOR Future<Void> pullAsyncData(StorageCacheData* data) {
cipherDetails.insert(header->cipherHeaderDetails);
collectingCipherKeys = true;
} else {
msg = msg.decrypt(cipherKeys.get(), cloneReader.arena());
msg = msg.decrypt(cipherKeys.get(), cloneReader.arena(), BlobCipherMetrics::TLOG);
}
}
if (!collectingCipherKeys) {
@ -1946,7 +1947,7 @@ ACTOR Future<Void> pullAsyncData(StorageCacheData* data) {
if (collectingCipherKeys) {
std::unordered_map<BlobCipherDetails, Reference<BlobCipherKey>> result =
wait(getEncryptCipherKeys(data->db, cipherDetails));
wait(getEncryptCipherKeys(data->db, cipherDetails, BlobCipherMetrics::TLOG));
cipherKeys = result;
collectingCipherKeys = false;
} else {
@ -2021,7 +2022,7 @@ ACTOR Future<Void> pullAsyncData(StorageCacheData* data) {
MutationRef msg;
reader >> msg;
if (msg.isEncrypted()) {
msg = msg.decrypt(cipherKeys.get(), reader.arena());
msg = msg.decrypt(cipherKeys.get(), reader.arena(), BlobCipherMetrics::TLOG);
}
if (ver != invalidVersion) // This change belongs to a version < minVersion

View File

@ -25,6 +25,7 @@
#include <cstddef>
#include "fdbclient/BackupAgent.actor.h"
#include "fdbclient/BlobCipher.h"
#include "fdbclient/MutationList.h"
#include "fdbclient/Notified.h"
#include "fdbclient/StorageServerInterface.h"
@ -33,7 +34,6 @@
#include "fdbserver/LogProtocolMessage.h"
#include "fdbserver/LogSystem.h"
#include "fdbserver/ProxyCommitData.actor.h"
#include "flow/BlobCipher.h"
#include "flow/FastRef.h"
// Resolver's data for applyMetadataMutations() calls.

View File

@ -120,7 +120,7 @@ struct DistributorSnapRequest {
template <class Ar>
void serialize(Ar& ar) {
serializer(ar, snapPayload, snapUID, reply, arena, debugID);
serializer(ar, snapPayload, snapUID, reply, debugID, arena);
}
};

View File

@ -18,6 +18,7 @@
* limitations under the License.
*/
#include "fdbclient/BlobCipher.h"
#if defined(NO_INTELLISENSE) && !defined(FDBSERVER_IENCRYPTIONKEYPROVIDER_ACTOR_G_H)
#define FDBSERVER_IENCRYPTIONKEYPROVIDER_ACTOR_G_H
#include "fdbserver/IEncryptionKeyProvider.actor.g.h"
@ -207,7 +208,8 @@ public:
TraceEvent("TenantAwareEncryptionKeyProvider_CipherHeaderMissing");
throw encrypt_ops_error();
}
TextAndHeaderCipherKeys cipherKeys = wait(getEncryptCipherKeys(self->db, key.cipherHeader.get()));
TextAndHeaderCipherKeys cipherKeys =
wait(getEncryptCipherKeys(self->db, key.cipherHeader.get(), BlobCipherMetrics::KV_REDWOOD));
EncryptionKey s = key;
s.cipherKeys = cipherKeys;
return s;
@ -218,7 +220,8 @@ public:
ACTOR static Future<EncryptionKey> getByRange(TenantAwareEncryptionKeyProvider* self, KeyRef begin, KeyRef end) {
EncryptCipherDomainNameRef domainName;
EncryptCipherDomainId domainId = self->getEncryptionDomainId(begin, end, &domainName);
TextAndHeaderCipherKeys cipherKeys = wait(getLatestEncryptCipherKeysForDomain(self->db, domainId, domainName));
TextAndHeaderCipherKeys cipherKeys =
wait(getLatestEncryptCipherKeysForDomain(self->db, domainId, domainName, BlobCipherMetrics::KV_REDWOOD));
EncryptionKey s;
s.cipherKeys = cipherKeys;
return s;

View File

@ -19,15 +19,16 @@
*/
#pragma once
#include "fdbclient/BlobCipher.h"
#ifndef FDBSERVER_IPAGER_H
#define FDBSERVER_IPAGER_H
#include <cstddef>
#include <stdint.h>
#include "fdbclient/BlobCipher.h"
#include "fdbclient/FDBTypes.h"
#include "fdbclient/Tenant.h"
#include "fdbserver/IEncryptionKeyProvider.actor.h"
#include "fdbserver/IKeyValueStore.h"
#include "flow/BlobCipher.h"
#include "flow/Error.h"
#include "flow/FastAlloc.h"
#include "flow/flow.h"
@ -341,8 +342,10 @@ public:
BlobCipherEncryptHeader header;
void encode(const TextAndHeaderCipherKeys& cipherKeys, uint8_t* payload, int len) {
EncryptBlobCipherAes265Ctr cipher(
cipherKeys.cipherTextKey, cipherKeys.cipherHeaderKey, ENCRYPT_HEADER_AUTH_TOKEN_MODE_SINGLE);
EncryptBlobCipherAes265Ctr cipher(cipherKeys.cipherTextKey,
cipherKeys.cipherHeaderKey,
ENCRYPT_HEADER_AUTH_TOKEN_MODE_SINGLE,
BlobCipherMetrics::KV_REDWOOD);
Arena arena;
StringRef ciphertext = cipher.encrypt(payload, len, &header, arena)->toStringRef();
ASSERT_EQ(len, ciphertext.size());
@ -350,7 +353,8 @@ public:
}
void decode(const TextAndHeaderCipherKeys& cipherKeys, uint8_t* payload, int len) {
DecryptBlobCipherAes256Ctr cipher(cipherKeys.cipherTextKey, cipherKeys.cipherHeaderKey, header.iv);
DecryptBlobCipherAes256Ctr cipher(
cipherKeys.cipherTextKey, cipherKeys.cipherHeaderKey, header.iv, BlobCipherMetrics::KV_REDWOOD);
Arena arena;
StringRef plaintext = cipher.decrypt(payload, len, header, arena)->toStringRef();
ASSERT_EQ(len, plaintext.size());

View File

@ -139,10 +139,10 @@ struct ResolveTransactionBatchRequest {
transactions,
txnStateTransactions,
reply,
arena,
debugID,
writtenTags,
spanContext);
spanContext,
arena);
}
};

View File

@ -22,8 +22,8 @@
#define SIM_KMS_CONNECTOR_H
#pragma once
#include "fdbclient/BlobCipher.h"
#include "fdbserver/KmsConnector.h"
#include "flow/BlobCipher.h"
class SimKmsConnector : public KmsConnector {
public:

View File

@ -328,10 +328,10 @@ struct TLogCommitRequest {
minKnownCommittedVersion,
messages,
reply,
arena,
debugID,
tLogCount,
spanContext);
spanContext,
arena);
}
};

View File

@ -140,7 +140,7 @@ struct WorkerDetails {
bool degraded;
bool recoveredDiskFiles;
WorkerDetails() : degraded(false) {}
WorkerDetails() : degraded(false), recoveredDiskFiles(false) {}
WorkerDetails(const WorkerInterface& interf, ProcessClass processClass, bool degraded, bool recoveredDiskFiles)
: interf(interf), processClass(processClass), degraded(degraded), recoveredDiskFiles(recoveredDiskFiles) {}
@ -148,7 +148,7 @@ struct WorkerDetails {
template <class Ar>
void serialize(Ar& ar) {
serializer(ar, interf, processClass, degraded);
serializer(ar, interf, processClass, degraded, recoveredDiskFiles);
}
};

View File

@ -43,7 +43,7 @@ template <class T>
struct sfinae_true : std::true_type {};
template <class T>
auto testAuthToken(int) -> sfinae_true<decltype(std::declval<T>().getAuthToken())>;
auto testAuthToken(int) -> sfinae_true<decltype(std::declval<T>().setAuthToken(std::declval<Transaction&>()))>;
template <class>
auto testAuthToken(long) -> std::false_type;
@ -53,7 +53,7 @@ struct hasAuthToken : decltype(testAuthToken<T>(0)) {};
template <class T>
void setAuthToken(T const& self, Transaction& tr) {
if constexpr (hasAuthToken<T>::value) {
tr.setOption(FDBTransactionOptions::AUTHORIZATION_TOKEN, self.getAuthToken());
self.setAuthToken(tr);
}
}

View File

@ -23,6 +23,7 @@
#include <type_traits>
#include <unordered_map>
#include "fdbclient/BlobCipher.h"
#include "fdbclient/BlobGranuleCommon.h"
#include "flow/ApiVersion.h"
#include "fmt/format.h"
@ -940,6 +941,8 @@ public:
std::unordered_map<NetworkAddress, std::map<UID, Version>> changeFeedClientVersions;
std::unordered_map<Key, Version> changeFeedCleanupDurable;
int64_t activeFeedQueries = 0;
int64_t changeFeedMemoryBytes = 0;
std::deque<std::pair<Version, int64_t>> feedMemoryBytesByVersion;
// newestAvailableVersion[k]
// == invalidVersion -> k is unavailable at all versions
@ -1201,6 +1204,7 @@ public:
specialCounter(cc, "KvstoreInlineKey", [self]() { return std::get<2>(self->storage.getSize()); });
specialCounter(cc, "ActiveChangeFeeds", [self]() { return self->uidChangeFeed.size(); });
specialCounter(cc, "ActiveChangeFeedQueries", [self]() { return self->activeFeedQueries; });
specialCounter(cc, "ChangeFeedMemoryBytes", [self]() { return self->changeFeedMemoryBytes; });
}
} counters;
@ -1440,6 +1444,20 @@ public:
return minVersion;
}
// count in-memory change feed bytes towards storage queue size, for the purposes of memory management and
// throttling
void addFeedBytesAtVersion(int64_t bytes, Version version) {
if (feedMemoryBytesByVersion.empty() || version != feedMemoryBytesByVersion.back().first) {
ASSERT(feedMemoryBytesByVersion.empty() || version >= feedMemoryBytesByVersion.back().first);
feedMemoryBytesByVersion.push_back({ version, 0 });
}
feedMemoryBytesByVersion.back().second += bytes;
changeFeedMemoryBytes += bytes;
if (SERVER_KNOBS->STORAGE_INCLUDE_FEED_STORAGE_QUEUE) {
counters.bytesInput += bytes;
}
}
void getSplitPoints(SplitRangeRequest const& req) {
try {
Optional<TenantMapEntry> entry = getTenantEntry(version.get(), req.tenantInfo);
@ -5076,6 +5094,17 @@ bool changeDurableVersion(StorageServer* data, Version desiredDurableVersion) {
data->counters.bytesDurable += bytesDurable;
}
int64_t feedBytesDurable = 0;
while (!data->feedMemoryBytesByVersion.empty() &&
data->feedMemoryBytesByVersion.front().first <= desiredDurableVersion) {
feedBytesDurable += data->feedMemoryBytesByVersion.front().second;
data->feedMemoryBytesByVersion.pop_front();
}
data->changeFeedMemoryBytes -= feedBytesDurable;
if (SERVER_KNOBS->STORAGE_INCLUDE_FEED_STORAGE_QUEUE) {
data->counters.bytesDurable += feedBytesDurable;
}
if (EXPENSIVE_VALIDATION) {
// Check that the above loop did its job
auto view = data->data().atLatest();
@ -5261,6 +5290,7 @@ void applyChangeFeedMutation(StorageServer* self, MutationRef const& m, Version
}
it->mutations.back().mutations.push_back_deep(it->mutations.back().arena(), m);
self->currentChangeFeeds.insert(it->id);
self->addFeedBytesAtVersion(m.totalSize(), version);
DEBUG_MUTATION("ChangeFeedWriteSet", version, m, self->thisServerID)
.detail("Range", it->range)
@ -5289,6 +5319,8 @@ void applyChangeFeedMutation(StorageServer* self, MutationRef const& m, Version
}
it->mutations.back().mutations.push_back_deep(it->mutations.back().arena(), m);
self->currentChangeFeeds.insert(it->id);
self->addFeedBytesAtVersion(m.totalSize(), version);
DEBUG_MUTATION("ChangeFeedWriteClear", version, m, self->thisServerID)
.detail("Range", it->range)
.detail("ChangeFeedID", it->id);
@ -8187,7 +8219,7 @@ ACTOR Future<Void> update(StorageServer* data, bool* pReceivedUpdate) {
cipherDetails.insert(header->cipherHeaderDetails);
collectingCipherKeys = true;
} else {
msg = msg.decrypt(cipherKeys.get(), eager.arena);
msg = msg.decrypt(cipherKeys.get(), eager.arena, BlobCipherMetrics::TLOG);
}
}
// TraceEvent(SevDebug, "SSReadingLog", data->thisServerID).detail("Mutation", msg);
@ -8210,7 +8242,7 @@ ACTOR Future<Void> update(StorageServer* data, bool* pReceivedUpdate) {
if (collectingCipherKeys) {
std::unordered_map<BlobCipherDetails, Reference<BlobCipherKey>> getCipherKeysResult =
wait(getEncryptCipherKeys(data->db, cipherDetails));
wait(getEncryptCipherKeys(data->db, cipherDetails, BlobCipherMetrics::TLOG));
cipherKeys = getCipherKeysResult;
collectingCipherKeys = false;
eager = UpdateEagerReadInfo();
@ -8335,7 +8367,7 @@ ACTOR Future<Void> update(StorageServer* data, bool* pReceivedUpdate) {
rd >> msg;
if (msg.isEncrypted()) {
ASSERT(cipherKeys.present());
msg = msg.decrypt(cipherKeys.get(), rd.arena());
msg = msg.decrypt(cipherKeys.get(), rd.arena(), BlobCipherMetrics::TLOG);
}
Span span("SS:update"_loc, spanContext);

View File

@ -1418,6 +1418,9 @@ KnobKeyValuePairs getOverriddenKnobKeyValues(const toml::value& context) {
if (std::get_if<NoKnobFound>(&parsedValue)) {
parsedValue = SERVER_KNOBS->parseKnobValue(key, value);
}
if (std::get_if<NoKnobFound>(&parsedValue)) {
parsedValue = FLOW_KNOBS->parseKnobValue(key, value);
}
if (std::get_if<NoKnobFound>(&parsedValue)) {
TraceEvent(SevError, "TestSpecUnrecognizedKnob")
.detail("KnobName", key)

View File

@ -466,6 +466,16 @@ struct BlobGranuleRangesWorkload : TestWorkload {
state Key middleKey = range.begin.withSuffix("AF"_sr);
state Key middleKey2 = range.begin.withSuffix("AG"_sr);
if (BGRW_DEBUG) {
fmt::print("IdempotentUnit: [{0} - {1})\n", range.begin.printable(), range.end.printable());
}
// unblobbifying range that already doesn't exist should be no-op
if (deterministicRandom()->coinflip()) {
bool unblobbifyStartSuccess = wait(self->setRange(cx, activeRange, false));
ASSERT(unblobbifyStartSuccess);
}
bool success = wait(self->setRange(cx, activeRange, true));
ASSERT(success);
wait(self->checkRange(cx, self, activeRange, true));
@ -544,8 +554,11 @@ struct BlobGranuleRangesWorkload : TestWorkload {
bool unblobbifyFail8 = wait(self->setRange(cx, KeyRangeRef(middleKey, middleKey2), false));
ASSERT(!unblobbifyFail8);
bool unblobbifySuccess = wait(self->setRange(cx, activeRange, false));
ASSERT(!unblobbifySuccess);
bool unblobbifySuccess = wait(self->setRange(cx, activeRange, true));
ASSERT(unblobbifySuccess);
bool unblobbifySuccessAgain = wait(self->setRange(cx, activeRange, true));
ASSERT(unblobbifySuccessAgain);
return Void();
}
@ -592,7 +605,6 @@ struct BlobGranuleRangesWorkload : TestWorkload {
// FIXME: fix bugs and enable these tests!
excludedTypes.insert(RANGES_MISALIGNED); // TODO - fix in blob manager
excludedTypes.insert(BLOBBIFY_IDEMPOTENT); // fix already in progress in a separate PR
excludedTypes.insert(RE_BLOBBIFY); // TODO - fix is non-trivial, is desired behavior eventually
std::string nextRangeKey = "U_" + self->newKey();

View File

@ -140,6 +140,9 @@ struct ChangeConfigWorkload : TestWorkload {
if (autoChange) { // if auto, we first get the desired addresses by read \xff\xff/management/auto_coordinators
loop {
try {
// Set RAW_ACCESS to explicitly avoid using tenants because
// access to management keys is denied for tenant transactions
tr.setOption(FDBTransactionOptions::RAW_ACCESS);
Optional<Value> newCoordinatorsKey = wait(tr.get(
LiteralStringRef("auto_coordinators")
.withPrefix(SpecialKeySpace::getModuleRange(SpecialKeySpace::MODULE::MANAGEMENT).begin)));

View File

@ -43,6 +43,7 @@ struct CycleMembers<true> {
TenantName tenant;
authz::jwt::TokenRef token;
StringRef signedToken;
bool useToken;
};
template <bool MultiTenancy>
@ -67,6 +68,7 @@ struct CycleWorkload : TestWorkload, CycleMembers<MultiTenancy> {
minExpectedTransactionsPerSecond = transactionsPerSecond * getOption(options, "expectedRate"_sr, 0.7);
if constexpr (MultiTenancy) {
ASSERT(g_network->isSimulated());
this->useToken = getOption(options, "useToken"_sr, true);
auto k = g_simulator.authKeys.begin();
this->tenant = getOption(options, "tenant"_sr, "CycleTenant"_sr);
// make it comfortably longer than the timeout of the workload
@ -85,11 +87,6 @@ struct CycleWorkload : TestWorkload, CycleMembers<MultiTenancy> {
}
}
template <bool MT = MultiTenancy>
std::enable_if_t<MT, StringRef> getAuthToken() const {
return this->signedToken;
}
std::string description() const override {
if constexpr (MultiTenancy) {
return "TenantCycleWorkload";
@ -151,12 +148,13 @@ struct CycleWorkload : TestWorkload, CycleMembers<MultiTenancy> {
}
template <bool B = MultiTenancy>
std::enable_if_t<B> setAuthToken(Transaction& tr) {
tr.setOption(FDBTransactionOptions::AUTHORIZATION_TOKEN, this->signedToken);
std::enable_if_t<B> setAuthToken(Transaction& tr) const {
if (this->useToken)
tr.setOption(FDBTransactionOptions::AUTHORIZATION_TOKEN, this->signedToken);
}
template <bool B = MultiTenancy>
std::enable_if_t<!B> setAuthToken(Transaction& tr) {}
std::enable_if_t<!B> setAuthToken(Transaction& tr) const {}
ACTOR Future<Void> cycleClient(Database cx, CycleWorkload* self, double delay) {
state double lastTime = now();

View File

@ -18,12 +18,12 @@
* limitations under the License.
*/
#include "fdbclient/BlobCipher.h"
#include "fdbclient/DatabaseContext.h"
#include "fdbclient/NativeAPI.actor.h"
#include "flow/EncryptUtils.h"
#include "flow/Error.h"
#include "flow/IRandom.h"
#include "flow/BlobCipher.h"
#include "fdbserver/workloads/workloads.actor.h"
#include "flow/flow.h"
#include "flow/ITrace.h"
@ -274,7 +274,8 @@ struct EncryptionOpsWorkload : TestWorkload {
BlobCipherEncryptHeader* header) {
uint8_t iv[AES_256_IV_LENGTH];
deterministicRandom()->randomBytes(&iv[0], AES_256_IV_LENGTH);
EncryptBlobCipherAes265Ctr encryptor(textCipherKey, headerCipherKey, &iv[0], AES_256_IV_LENGTH, authMode);
EncryptBlobCipherAes265Ctr encryptor(
textCipherKey, headerCipherKey, &iv[0], AES_256_IV_LENGTH, authMode, BlobCipherMetrics::TEST);
auto start = std::chrono::high_resolution_clock::now();
Reference<EncryptBuf> encrypted = encryptor.encrypt(payload, len, header, arena);
@ -306,7 +307,7 @@ struct EncryptionOpsWorkload : TestWorkload {
ASSERT(cipherKey.isValid());
ASSERT(cipherKey->isEqual(orgCipherKey));
DecryptBlobCipherAes256Ctr decryptor(cipherKey, headerCipherKey, header.iv);
DecryptBlobCipherAes256Ctr decryptor(cipherKey, headerCipherKey, header.iv, BlobCipherMetrics::TEST);
const bool validateHeaderAuthToken = deterministicRandom()->randomInt(0, 100) < 65;
auto start = std::chrono::high_resolution_clock::now();

View File

@ -749,6 +749,8 @@ struct FuzzApiCorrectnessWorkload : TestWorkload {
std::make_pair(error_code_tenant_not_found,
ExceptionContract::possibleIf(!workload->canUseTenant(tr->getTenant()))),
std::make_pair(error_code_invalid_option,
ExceptionContract::possibleIf(tr->getTenant().present() && specialKeys.contains(key))),
std::make_pair(error_code_illegal_tenant_access,
ExceptionContract::possibleIf(tr->getTenant().present() && specialKeys.contains(key)))
};
}
@ -829,6 +831,8 @@ struct FuzzApiCorrectnessWorkload : TestWorkload {
std::make_pair(error_code_tenant_not_found,
ExceptionContract::possibleIf(!workload->canUseTenant(tr->getTenant()))),
std::make_pair(error_code_invalid_option,
ExceptionContract::possibleIf(tr->getTenant().present() && isSpecialKeyRange)),
std::make_pair(error_code_illegal_tenant_access,
ExceptionContract::possibleIf(tr->getTenant().present() && isSpecialKeyRange))
};
}
@ -875,7 +879,7 @@ struct FuzzApiCorrectnessWorkload : TestWorkload {
std::make_pair(error_code_accessed_unreadable, ExceptionContract::Possible),
std::make_pair(error_code_tenant_not_found,
ExceptionContract::possibleIf(!workload->canUseTenant(tr->getTenant()))),
std::make_pair(error_code_invalid_option,
std::make_pair(error_code_illegal_tenant_access,
ExceptionContract::possibleIf(tr->getTenant().present() && isSpecialKeyRange))
};
}
@ -945,6 +949,8 @@ struct FuzzApiCorrectnessWorkload : TestWorkload {
std::make_pair(error_code_tenant_not_found,
ExceptionContract::possibleIf(!workload->canUseTenant(tr->getTenant()))),
std::make_pair(error_code_invalid_option,
ExceptionContract::possibleIf(tr->getTenant().present() && isSpecialKeyRange)),
std::make_pair(error_code_illegal_tenant_access,
ExceptionContract::possibleIf(tr->getTenant().present() && isSpecialKeyRange))
};
}
@ -1002,6 +1008,8 @@ struct FuzzApiCorrectnessWorkload : TestWorkload {
std::make_pair(error_code_tenant_not_found,
ExceptionContract::possibleIf(!workload->canUseTenant(tr->getTenant()))),
std::make_pair(error_code_invalid_option,
ExceptionContract::possibleIf(tr->getTenant().present() && isSpecialKeyRange)),
std::make_pair(error_code_illegal_tenant_access,
ExceptionContract::possibleIf(tr->getTenant().present() && isSpecialKeyRange))
};
}

View File

@ -28,6 +28,7 @@
#include "fdbclient/Schemas.h"
#include "fdbclient/SpecialKeySpace.actor.h"
#include "fdbserver/Knobs.h"
#include "fdbclient/TenantManagement.actor.h"
#include "fdbserver/TesterInterface.actor.h"
#include "fdbserver/workloads/workloads.actor.h"
#include "flow/IRandom.h"
@ -298,8 +299,133 @@ struct SpecialKeySpaceCorrectnessWorkload : TestWorkload {
}
ACTOR Future<Void> testSpecialKeySpaceErrors(Database cx_, SpecialKeySpaceCorrectnessWorkload* self) {
Database cx = cx_->clone();
state Database cx = cx_->clone();
try {
wait(success(TenantAPI::createTenant(cx.getReference(), TenantName("foo"_sr))));
} catch (Error& e) {
ASSERT(e.code() == error_code_tenant_already_exists || e.code() == error_code_actor_cancelled);
}
state Reference<ReadYourWritesTransaction> tx = makeReference<ReadYourWritesTransaction>(cx);
state Reference<ReadYourWritesTransaction> tenantTx =
makeReference<ReadYourWritesTransaction>(cx, TenantName("foo"_sr));
// Use new transactions that may use default tenant rather than re-use tx
// This is because tx will reject raw access for later tests if default tenant is set
state Reference<ReadYourWritesTransaction> defaultTx1 = makeReference<ReadYourWritesTransaction>(cx);
state Reference<ReadYourWritesTransaction> defaultTx2 = makeReference<ReadYourWritesTransaction>(cx);
state bool disableRyw = deterministicRandom()->coinflip();
// tenant transaction accessing modules that do not support tenants
// tenant getRange
try {
wait(success(tenantTx->getRange(SpecialKeySpace::getModuleRange(SpecialKeySpace::MODULE::MANAGEMENT),
CLIENT_KNOBS->TOO_MANY)));
ASSERT(false);
} catch (Error& e) {
if (e.code() == error_code_actor_cancelled)
throw;
ASSERT(e.code() == error_code_illegal_tenant_access);
tenantTx->reset();
}
// tenant set + commit
try {
tenantTx->setOption(FDBTransactionOptions::SPECIAL_KEY_SPACE_ENABLE_WRITES);
tenantTx->set(SpecialKeySpace::getManagementApiCommandPrefix("consistencycheck"), ValueRef());
wait(tenantTx->commit());
ASSERT(false);
} catch (Error& e) {
if (e.code() == error_code_actor_cancelled)
throw;
ASSERT(e.code() == error_code_illegal_tenant_access);
tenantTx->reset();
}
// tenant clear
try {
tenantTx->setOption(FDBTransactionOptions::SPECIAL_KEY_SPACE_ENABLE_WRITES);
tenantTx->clear(SpecialKeySpace::getManagementApiCommandRange("exclude"));
ASSERT(false);
} catch (Error& e) {
if (e.code() == error_code_actor_cancelled)
throw;
ASSERT(e.code() == error_code_illegal_tenant_access);
tenantTx->reset();
}
// tenant check that conflict ranges stay the same after commit
// and depending on if RYW is disabled
{
state RangeResult readresult1;
state RangeResult readresult2;
state RangeResult writeResult1;
state RangeResult writeResult2;
try {
if (disableRyw) {
defaultTx1->setOption(FDBTransactionOptions::READ_YOUR_WRITES_DISABLE);
}
defaultTx1->addReadConflictRange(singleKeyRange(LiteralStringRef("testKeylll")));
defaultTx1->addWriteConflictRange(singleKeyRange(LiteralStringRef("testKeylll")));
wait(store(readresult1, defaultTx1->getRange(readConflictRangeKeysRange, CLIENT_KNOBS->TOO_MANY)));
wait(store(writeResult1, defaultTx1->getRange(writeConflictRangeKeysRange, CLIENT_KNOBS->TOO_MANY)));
wait(defaultTx1->commit());
CODE_PROBE(true, "conflict range tenant commit succeeded");
} catch (Error& e) {
if (e.code() == error_code_actor_cancelled)
throw;
CODE_PROBE(true, "conflict range tenant commit error thrown");
}
wait(store(readresult2, defaultTx1->getRange(readConflictRangeKeysRange, CLIENT_KNOBS->TOO_MANY)));
wait(store(writeResult2, defaultTx1->getRange(writeConflictRangeKeysRange, CLIENT_KNOBS->TOO_MANY)));
ASSERT(readresult1 == readresult2);
ASSERT(writeResult1 == writeResult2);
defaultTx1->reset();
}
// proper conflict ranges
loop {
try {
if (disableRyw) {
defaultTx1->setOption(FDBTransactionOptions::READ_YOUR_WRITES_DISABLE);
defaultTx2->setOption(FDBTransactionOptions::READ_YOUR_WRITES_DISABLE);
}
defaultTx1->setOption(FDBTransactionOptions::REPORT_CONFLICTING_KEYS);
defaultTx2->setOption(FDBTransactionOptions::REPORT_CONFLICTING_KEYS);
wait(success(defaultTx1->getReadVersion()));
wait(success(defaultTx2->getReadVersion()));
defaultTx1->addReadConflictRange(singleKeyRange(LiteralStringRef("foo")));
defaultTx1->addWriteConflictRange(singleKeyRange(LiteralStringRef("foo")));
defaultTx2->addWriteConflictRange(singleKeyRange(LiteralStringRef("foo")));
wait(defaultTx2->commit());
try {
wait(defaultTx1->commit());
ASSERT(false);
} catch (Error& e) {
state Error err = e;
if (err.code() != error_code_not_committed) {
wait(defaultTx1->onError(err));
wait(defaultTx2->onError(err));
continue;
}
// Read conflict ranges of defaultTx1 and check for "foo" with no tenant prefix
state RangeResult readConflictRange =
wait(defaultTx1->getRange(readConflictRangeKeysRange, CLIENT_KNOBS->TOO_MANY));
state RangeResult writeConflictRange =
wait(defaultTx1->getRange(writeConflictRangeKeysRange, CLIENT_KNOBS->TOO_MANY));
state RangeResult conflictKeys =
wait(defaultTx1->getRange(conflictingKeysRange, CLIENT_KNOBS->TOO_MANY));
// size is 2 because singleKeyRange includes the key after
ASSERT(readConflictRange.size() == 2 &&
readConflictRange.begin()->key == readConflictRangeKeysRange.begin.withSuffix("foo"_sr));
ASSERT(writeConflictRange.size() == 2 &&
writeConflictRange.begin()->key == writeConflictRangeKeysRange.begin.withSuffix("foo"_sr));
ASSERT(conflictKeys.size() == 2 &&
conflictKeys.begin()->key == conflictingKeysRange.begin.withSuffix("foo"_sr));
defaultTx1->reset();
defaultTx2->reset();
break;
}
} catch (Error& e) {
if (e.code() == error_code_actor_cancelled)
throw;
wait(defaultTx2->onError(e));
}
}
// begin key outside module range
try {
tx->setOption(FDBTransactionOptions::RAW_ACCESS);

View File

@ -73,14 +73,14 @@ struct TenantManagementWorkload : TestWorkload {
TenantName localTenantGroupNamePrefix;
const Key specialKeysTenantMapPrefix = SpecialKeySpace::getModuleRange(SpecialKeySpace::MODULE::MANAGEMENT)
.begin.withSuffix(TenantRangeImpl<true>::submoduleRange.begin)
.withSuffix(TenantRangeImpl<true>::mapSubRange.begin);
.begin.withSuffix(TenantRangeImpl::submoduleRange.begin)
.withSuffix(TenantRangeImpl::mapSubRange.begin);
const Key specialKeysTenantConfigPrefix = SpecialKeySpace::getModuleRange(SpecialKeySpace::MODULE::MANAGEMENT)
.begin.withSuffix(TenantRangeImpl<true>::submoduleRange.begin)
.withSuffix(TenantRangeImpl<true>::configureSubRange.begin);
.begin.withSuffix(TenantRangeImpl::submoduleRange.begin)
.withSuffix(TenantRangeImpl::configureSubRange.begin);
const Key specialKeysTenantRenamePrefix = SpecialKeySpace::getModuleRange(SpecialKeySpace::MODULE::MANAGEMENT)
.begin.withSuffix(TenantRangeImpl<true>::submoduleRange.begin)
.withSuffix(TenantRangeImpl<true>::renameSubRange.begin);
.begin.withSuffix(TenantRangeImpl::submoduleRange.begin)
.withSuffix(TenantRangeImpl::renameSubRange.begin);
int maxTenants;
int maxTenantGroups;

View File

@ -66,12 +66,10 @@ public: // introduced features
API_VERSION_FEATURE(@FDB_AV_PERSISTENT_OPTIONS@, PersistentOptions);
API_VERSION_FEATURE(@FDB_AV_TRACE_FILE_IDENTIFIER@, TraceFileIdentifier);
API_VERSION_FEATURE(@FDB_AV_CLUSTER_SHARED_STATE_MAP@, ClusterSharedStateMap);
API_VERSION_FEATURE(@FDB_AV_TENANTS_V1@, TenantsV1);
API_VERSION_FEATURE(@FDB_AV_BLOB_RANGE_API@, BlobRangeApi);
API_VERSION_FEATURE(@FDB_AV_CREATE_DB_FROM_CONN_STRING@, CreateDBFromConnString);
API_VERSION_FEATURE(@FDB_AV_FUTURE_GET_BOOL@, FutureGetBool);
API_VERSION_FEATURE(@FDB_AV_FUTURE_PROTOCOL_VERSION_API@, FutureProtocolVersionApi);
API_VERSION_FEATURE(@FDB_AV_TENANTS_V2@, TenantsV2);
};
#endif // FLOW_CODE_API_VERSION_H

View File

@ -7,9 +7,7 @@ set(FDB_AV_INLINE_UPDATE_DATABASE "610")
set(FDB_AV_PERSISTENT_OPTIONS "610")
set(FDB_AV_TRACE_FILE_IDENTIFIER "630")
set(FDB_AV_CLUSTER_SHARED_STATE_MAP "710")
set(FDB_AV_TENANTS_V1 "720")
set(FDB_AV_BLOB_RANGE_API "720")
set(FDB_AV_CREATE_DB_FROM_CONN_STRING "720")
set(FDB_AV_FUTURE_GET_BOOL "720")
set(FDB_AV_FUTURE_PROTOCOL_VERSION_API "720")
set(FDB_AV_TENANTS_V2 "720")
set(FDB_AV_FUTURE_PROTOCOL_VERSION_API "720")

View File

@ -131,6 +131,7 @@ void FlowKnobs::initialize(Randomize randomize, IsSimulated isSimulated) {
init( NETWORK_TEST_SCRIPT_MODE, false );
//Authorization
init( ALLOW_TOKENLESS_TENANT_ACCESS, false );
init( PUBLIC_KEY_FILE_MAX_SIZE, 1024 * 1024 );
init( PUBLIC_KEY_FILE_REFRESH_INTERVAL_SECONDS, 30 );
init( MAX_CACHED_EXPIRED_TOKENS, 1024 );
@ -295,6 +296,8 @@ void FlowKnobs::initialize(Randomize randomize, IsSimulated isSimulated) {
init( ENCRYPT_KEY_REFRESH_INTERVAL, isSimulated ? 60 : 8 * 60 );
if ( randomize && BUGGIFY) { ENCRYPT_KEY_REFRESH_INTERVAL = deterministicRandom()->randomInt(2, 10); }
init( TOKEN_CACHE_SIZE, 100 );
init( ENCRYPT_KEY_CACHE_LOGGING_INTERVAL, 5.0 );
init( ENCRYPT_KEY_CACHE_LOGGING_SAMPLE_SIZE, 1000 );
// REST Client
init( RESTCLIENT_MAX_CONNECTIONPOOL_SIZE, 10 );

View File

@ -32,7 +32,6 @@
#include "flow/Arena.h"
#include "flow/StreamCipher.h"
#include "flow/BlobCipher.h"
#include "flow/ScopeExit.h"
#include "flow/Trace.h"
#include "flow/Error.h"
@ -3623,6 +3622,12 @@ void platformInit() {
#endif
}
std::vector<std::function<void()>> g_crashHandlerCallbacks;
void registerCrashHandlerCallback(void (*f)()) {
g_crashHandlerCallbacks.push_back(f);
}
// The crashHandler function is registered to handle signals before the process terminates.
// Basic information about the crash is printed/traced, and stdout and trace events are flushed.
void crashHandler(int sig) {
@ -3635,7 +3640,10 @@ void crashHandler(int sig) {
StreamCipherKey::cleanup();
StreamCipher::cleanup();
BlobCipherKeyCache::cleanup();
for (auto& f : g_crashHandlerCallbacks) {
f();
}
fprintf(error ? stderr : stdout, "SIGNAL: %s (%d)\n", strsignal(sig), sig);
if (error) {

View File

@ -172,6 +172,7 @@ public: // introduced features
PROTOCOL_VERSION_FEATURE(@FDB_PV_SHARD_ENCODE_LOCATION_METADATA@, ShardEncodeLocationMetaData);
PROTOCOL_VERSION_FEATURE(@FDB_PV_TENANTS@, Tenants);
PROTOCOL_VERSION_FEATURE(@FDB_PV_BLOB_GRANULE_FILE@, BlobGranuleFile);
PROTOCOL_VERSION_FEATURE(@FDB_PV_CLUSTER_ID_SPECIAL_KEY@, ClusterIdSpecialKey);
};
template <>

View File

@ -87,4 +87,5 @@ set(FDB_PV_SW_VERSION_TRACKING "0x0FDB00B072000000LL")
set(FDB_PV_ENCRYPTION_AT_REST "0x0FDB00B072000000LL")
set(FDB_PV_SHARD_ENCODE_LOCATION_METADATA "0x0FDB00B072000000LL")
set(FDB_PV_TENANTS "0x0FDB00B072000000LL")
set(FDB_PV_BLOB_GRANULE_FILE "0x0FDB00B072000000LL")
set(FDB_PV_BLOB_GRANULE_FILE "0x0FDB00B072000000LL")
set(FDB_PV_CLUSTER_ID_SPECIAL_KEY "0x0FDB00B072000000LL")

View File

@ -473,13 +473,13 @@ TEST_CASE("/flow/FlatBuffers/VectorRef") {
vec.push_back(arena, str);
}
ObjectWriter writer(Unversioned());
writer.serialize(FileIdentifierFor<decltype(vec)>::value, arena, vec);
writer.serialize(FileIdentifierFor<decltype(vec)>::value, vec);
serializedVector = StringRef(readerArena, writer.toStringRef());
}
ArenaObjectReader reader(readerArena, serializedVector, Unversioned());
// The VectorRef and Arena arguments are intentionally in a different order from the serialize call above.
// The Arena argument is intentionally missing from the serialize call above.
// Arenas need to get serialized after any Ref types whose memory they own. In order for schema evolution to be
// possible, it needs to be okay to reorder an Arena so that it appears after a newly added Ref type. For this
// possible, it needs to be okay to move/add an Arena so that it appears after a newly added Ref type. For this
// reason, Arenas are ignored by the wire protocol entirely. We test that behavior here.
reader.deserialize(FileIdentifierFor<decltype(outVec)>::value, outVec, vecArena);
}

View File

@ -100,6 +100,7 @@ FDB_DECLARE_BOOLEAN_PARAM(FastInaccurateEstimate);
// memory is freed by deleting the entire Arena at once. See flow/README.md for details on using Arenas.
class Arena {
public:
constexpr static auto fb_must_appear_last = true;
Arena();
explicit Arena(size_t reservedSize);
//~Arena();
@ -365,6 +366,8 @@ class Standalone : private Arena, public T {
public:
using RefType = T;
constexpr static auto fb_must_appear_last = false;
// T must have no destructor
Arena& arena() { return *(Arena*)this; }
const Arena& arena() const { return *(const Arena*)this; }

View File

@ -22,11 +22,11 @@
// When actually compiled (NO_INTELLISENSE), include the generated version of this file. In intellisense use the source
// version.
#if defined(NO_INTELLISENSE) && !defined(FDBCLIENT_EVENTTYPES_ACTOR_G_H)
#define FDBCLIENT_EVENTTYPES_ACTOR_G_H
#if defined(NO_INTELLISENSE) && !defined(FLOW_EVENTTYPES_ACTOR_G_H)
#define FLOW_EVENTTYPES_ACTOR_G_H
#include "flow/EventTypes.actor.g.h"
#elif !defined(FDBCLIENT_EVENTTYPES_ACTOR_H)
#define FDBCLIENT_EVENTTYPESS_ACTOR_H
#elif !defined(FLOW_EVENTTYPES_ACTOR_H)
#define FLOW_EVENTTYPES_ACTOR_H
#include "flow/flow.h"
#include "flow/TDMetric.actor.h"

View File

@ -196,6 +196,7 @@ public:
bool NETWORK_TEST_SCRIPT_MODE;
// Authorization
bool ALLOW_TOKENLESS_TENANT_ACCESS;
int PUBLIC_KEY_FILE_MAX_SIZE;
int PUBLIC_KEY_FILE_REFRESH_INTERVAL_SECONDS;
int MAX_CACHED_EXPIRED_TOKENS;
@ -358,6 +359,8 @@ public:
// Encryption
int64_t ENCRYPT_CIPHER_KEY_CACHE_TTL;
int64_t ENCRYPT_KEY_REFRESH_INTERVAL;
double ENCRYPT_KEY_CACHE_LOGGING_INTERVAL;
double ENCRYPT_KEY_CACHE_LOGGING_SAMPLE_SIZE;
// Authorization
int TOKEN_CACHE_SIZE;

View File

@ -37,11 +37,6 @@ struct is_fb_function_t<T, typename std::enable_if<T::is_fb_visitor>::type> : st
template <class T>
constexpr bool is_fb_function = is_fb_function_t<T>::value;
template <class Visitor, class... Items>
typename std::enable_if<is_fb_function<Visitor>, void>::type serializer(Visitor& visitor, Items&... items) {
visitor(items...);
}
template <class... Ts>
struct pack {};
@ -61,6 +56,40 @@ struct index_impl<0, pack<T, Ts...>> {
template <int i, class Pack>
using index_t = typename index_impl<i, Pack>::type;
template <class T, typename = void>
struct fb_must_appear_last_t : std::false_type {};
template <class T>
struct fb_must_appear_last_t<T, typename std::enable_if<T::fb_must_appear_last>::type>
: std::conditional_t<T::fb_must_appear_last, std::true_type, std::false_type> {};
template <class T>
constexpr bool fb_must_appear_last = fb_must_appear_last_t<T>::value;
template <class Item, class... Items>
constexpr bool fb_appears_last_property_helper(pack<Item, Items...>) {
if constexpr (sizeof...(Items) == 0) {
return true;
} else {
return !fb_must_appear_last<Item> && fb_appears_last_property_helper(pack<Items...>{});
}
}
template <class... Items>
constexpr bool fb_appears_last_property(pack<Items...>) {
if constexpr (sizeof...(Items) == 0) {
return true;
} else {
return fb_appears_last_property_helper(pack<Items...>{});
}
}
template <class Visitor, class... Items>
typename std::enable_if<is_fb_function<Visitor>, void>::type serializer(Visitor& visitor, Items&... items) {
static_assert(fb_appears_last_property(pack<Items...>{}),
"An argument to a serializer call that must appear last (Arena?) does not appear last");
visitor(items...);
}
template <class T, typename = void>
struct scalar_traits : std::false_type {
constexpr static size_t size = 0;

View File

@ -794,7 +794,14 @@ EXTERNC void flushAndExit(int exitCode);
// Initilization code that's run at the beginning of every entry point (except fdbmonitor)
void platformInit();
// Register a callback which will run as part of the crash handler. Use in conjunction with registerCrashHandler.
// The callback being added should be simple and unlikely to fail, otherwise it will fail the crash handler,
// preventing necessary logging being printed. Also, the crash handler may not be comprehensive in handling all
// failure cases.
void registerCrashHandlerCallback(void (*f)());
void registerCrashHandler();
void setupRunLoopProfiler();
EXTERNC void setProfilingEnabled(int enabled);

View File

@ -91,6 +91,8 @@ inline typename Archive::READER& operator>>(Archive& ar, Item& item) {
template <class Archive, class Item, class... Items>
typename Archive::WRITER& serializer(Archive& ar, const Item& item, const Items&... items) {
static_assert(fb_appears_last_property(pack<Item, Items...>{}),
"An argument to a serializer call that must appear last (Arena?) does not appear last");
save(ar, item);
if constexpr (sizeof...(Items) > 0) {
serializer(ar, items...);
@ -100,6 +102,8 @@ typename Archive::WRITER& serializer(Archive& ar, const Item& item, const Items&
template <class Archive, class Item, class... Items>
typename Archive::READER& serializer(Archive& ar, Item& item, Items&... items) {
static_assert(fb_appears_last_property(pack<Item, Items...>{}),
"An argument to a serializer call that must appear last (Arena?) does not appear last");
load(ar, item);
if constexpr (sizeof...(Items) > 0) {
serializer(ar, items...);

View File

@ -184,6 +184,7 @@ if(WITH_PYTHON)
add_fdb_test(TEST_FILES fast/SystemRebootTestCycle.toml)
add_fdb_test(TEST_FILES fast/TaskBucketCorrectness.toml)
add_fdb_test(TEST_FILES fast/TenantCycle.toml)
add_fdb_test(TEST_FILES fast/TenantCycleTokenless.toml)
add_fdb_test(TEST_FILES fast/TenantEntryCache.toml)
add_fdb_test(TEST_FILES fast/TimeKeeperCorrectness.toml)
add_fdb_test(TEST_FILES fast/TxnStateStoreCycleTest.toml)

View File

@ -20,6 +20,7 @@ PROGRESS_CHECK_TIMEOUT_SEC = 30
TESTER_STATS_INTERVAL_SEC = 5
TRANSACTION_RETRY_LIMIT = 100
RUN_WITH_GDB = False
CLEANUP_ON_EXIT = True
def version_from_str(ver_str):
@ -166,7 +167,8 @@ class UpgradeTest:
def __exit__(self, xc_type, exc_value, traceback):
self.cluster.stop_cluster()
shutil.rmtree(self.tmp_dir)
if CLEANUP_ON_EXIT:
shutil.rmtree(self.tmp_dir)
# Determine FDB API version matching the upgrade path
def determine_api_version(self):
@ -425,6 +427,11 @@ if __name__ == "__main__":
help="Do not dump cluster log on error",
action="store_true",
)
parser.add_argument(
"--no-cleanup-on-error",
help="In case of an error do not remove any of the generated files",
action="store_true",
)
parser.add_argument("--blob-granules-enabled", help="Enable blob granules", action="store_true")
parser.add_argument("--run-with-gdb", help="Execute the tester binary from gdb", action="store_true")
args = parser.parse_args()
@ -450,5 +457,7 @@ if __name__ == "__main__":
test.dump_warnings_in_logs()
if errcode != 0 and not args.disable_log_dump:
test.dump_cluster_logs()
if errcode != 0 and args.no_cleanup_on_error:
CLEANUP_ON_EXIT = False
sys.exit(errcode)

View File

@ -1,3 +1,6 @@
[configuration]
allowDisablingTenants = false
[[test]]
testTitle = 'SpecialKeySpaceCorrectnessTest'

View File

@ -0,0 +1,36 @@
[configuration]
allowDefaultTenant = false
allowDisablingTenants = false
[[knobs]]
allow_tokenless_tenant_access = true
[[test]]
testTitle = 'TenantCreation'
[[test.workload]]
testName = 'CreateTenant'
name = 'First'
[[test.workload]]
testName = 'CreateTenant'
name = 'Second'
[[test]]
testTitle = 'Cycle'
[[test.workload]]
testName = 'TenantCycle'
tenant = 'First'
transactionsPerSecond = 250.0
testDuration = 10.0
expectedRate = 0.80
useToken = false
[[test.workload]]
testName = 'TenantCycle'
tenant = 'Second'
transactionsPerSecond = 2500.0
testDuration = 10.0
expectedRate = 0.80
useToken = false