Merge branch 'main' of github.com:apple/foundationdb into jfu-tenant-rename

This commit is contained in:
Jon Fu 2022-07-06 14:10:22 -07:00
commit df1f108b78
116 changed files with 3365 additions and 2695 deletions

View File

@ -288,7 +288,7 @@ class TestRunner(object):
tr = self.db.create_transaction()
try:
tr.options.set_special_key_space_enable_writes()
del tr[b'\xff\xff/management/tenant_map/' : b'\xff\xff/management/tenant_map0']
del tr[b'\xff\xff/management/tenant/map/' : b'\xff\xff/management/tenant/map0']
tr.commit().wait()
break
except fdb.FDBError as e:

View File

@ -119,6 +119,7 @@ if(NOT WIN32)
set(API_TESTER_SRCS
test/apitester/fdb_c_api_tester.cpp
test/apitester/TesterAtomicOpsCorrectnessWorkload.cpp
test/apitester/TesterApiWorkload.cpp
test/apitester/TesterApiWorkload.h
test/apitester/TesterTestSpec.cpp

View File

@ -0,0 +1,330 @@
/*
* TesterAtomicOpsCorrectnessWorkload.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 "TesterApiWorkload.h"
#include "TesterUtil.h"
#include "fdb_c_options.g.h"
#include "fmt/core.h"
#include "test/fdb_api.hpp"
#include <cctype>
#include <memory>
#include <fmt/format.h>
namespace FdbApiTester {
using fdb::Key;
using fdb::Value;
using fdb::ValueRef;
class AtomicOpsCorrectnessWorkload : public ApiWorkload {
public:
AtomicOpsCorrectnessWorkload(const WorkloadConfig& config) : ApiWorkload(config) {}
private:
typedef std::function<uint64_t(uint64_t, uint64_t)> IntAtomicOpFunction;
typedef std::function<Value(ValueRef, ValueRef)> AtomicOpFunction;
enum OpType {
OP_ATOMIC_ADD,
OP_ATOMIC_BIT_AND,
OP_ATOMIC_BIT_OR,
OP_ATOMIC_BIT_XOR,
OP_ATOMIC_APPEND_IF_FITS,
OP_ATOMIC_MAX,
OP_ATOMIC_MIN,
OP_ATOMIC_VERSIONSTAMPED_KEY,
OP_ATOMIC_VERSIONSTAMPED_VALUE,
OP_ATOMIC_BYTE_MIN,
OP_ATOMIC_BYTE_MAX,
OP_ATOMIC_COMPARE_AND_CLEAR,
OP_LAST = OP_ATOMIC_COMPARE_AND_CLEAR
};
void randomOperation(TTaskFct cont) override {
OpType txType = (OpType)Random::get().randomInt(0, OP_LAST);
switch (txType) {
case OP_ATOMIC_ADD:
testIntAtomicOp(
FDBMutationType::FDB_MUTATION_TYPE_ADD, [](uint64_t val1, uint64_t val2) { return val1 + val2; }, cont);
break;
case OP_ATOMIC_BIT_AND:
testIntAtomicOp(
FDBMutationType::FDB_MUTATION_TYPE_BIT_AND,
[](uint64_t val1, uint64_t val2) { return val1 & val2; },
cont);
break;
case OP_ATOMIC_BIT_OR:
testIntAtomicOp(
FDBMutationType::FDB_MUTATION_TYPE_BIT_OR,
[](uint64_t val1, uint64_t val2) { return val1 | val2; },
cont);
break;
case OP_ATOMIC_BIT_XOR:
testIntAtomicOp(
FDBMutationType::FDB_MUTATION_TYPE_BIT_XOR,
[](uint64_t val1, uint64_t val2) { return val1 ^ val2; },
cont);
break;
case OP_ATOMIC_APPEND_IF_FITS: {
Value val1 = randomValue();
Value val2 = randomValue();
testAtomicOp(
FDBMutationType::FDB_MUTATION_TYPE_APPEND_IF_FITS,
val1,
val2,
[](ValueRef val1, ValueRef val2) { return Value(val1) + Value(val2); },
cont);
break;
}
case OP_ATOMIC_MAX:
testIntAtomicOp(
FDBMutationType::FDB_MUTATION_TYPE_MAX,
[](uint64_t val1, uint64_t val2) { return std::max(val1, val2); },
cont);
break;
case OP_ATOMIC_MIN:
testIntAtomicOp(
FDBMutationType::FDB_MUTATION_TYPE_MIN,
[](uint64_t val1, uint64_t val2) { return std::min(val1, val2); },
cont);
break;
case OP_ATOMIC_VERSIONSTAMPED_KEY:
testAtomicVersionstampedKeyOp(cont);
break;
case OP_ATOMIC_VERSIONSTAMPED_VALUE:
testAtomicVersionstampedValueOp(cont);
break;
case OP_ATOMIC_BYTE_MIN: {
Value val1 = randomValue();
Value val2 = randomValue();
testAtomicOp(
FDBMutationType::FDB_MUTATION_TYPE_BYTE_MIN,
val1,
val2,
[](ValueRef val1, ValueRef val2) { return Value(std::min(val1, val2)); },
cont);
break;
}
case OP_ATOMIC_BYTE_MAX: {
Value val1 = randomValue();
Value val2 = randomValue();
testAtomicOp(
FDBMutationType::FDB_MUTATION_TYPE_BYTE_MAX,
val1,
val2,
[](ValueRef val1, ValueRef val2) { return Value(std::max(val1, val2)); },
cont);
break;
}
case OP_ATOMIC_COMPARE_AND_CLEAR:
testAtomicCompareAndClearOp(cont);
break;
}
}
void testIntAtomicOp(FDBMutationType opType, IntAtomicOpFunction opFunc, TTaskFct cont) {
uint64_t intValue1 = Random::get().randomInt(0, 10000000);
uint64_t intValue2 = Random::get().randomInt(0, 10000000);
Value val1 = toByteString(intValue1);
Value val2 = toByteString(intValue2);
testAtomicOp(
opType,
val1,
val2,
[opFunc](ValueRef val1, ValueRef val2) {
return toByteString(opFunc(toInteger<uint64_t>(val1), toInteger<uint64_t>(val2)));
},
cont);
}
void testAtomicOp(FDBMutationType opType, Value val1, Value val2, AtomicOpFunction opFunc, TTaskFct cont) {
Key key(randomKeyName());
execTransaction(
// 1. Set the key to val1
[key, val1](auto ctx) {
ctx->tx().set(key, val1);
ctx->commit();
},
[this, opType, opFunc, key, val1, val2, cont]() {
execTransaction(
// 2. Perform the given atomic operation to val2, but only if it hasn't been applied yet, otherwise
// retries of commit_unknown_result would cause the operation to be applied multiple times, see
// https://github.com/apple/foundationdb/issues/1321.
[key, opType, val1, val2](auto ctx) {
auto f = ctx->tx().get(key, false);
ctx->continueAfter(f, [ctx, f, opType, key, val1, val2]() {
auto outputVal = f.get();
ASSERT(outputVal.has_value());
if (outputVal.value() == val1) {
ctx->tx().atomicOp(key, val2, opType);
ctx->commit();
} else {
ctx->done();
}
});
},
[this, opFunc, key, val1, val2, cont]() {
auto result = std::make_shared<Value>();
execTransaction(
// 3. Fetch the final value.
[key, result](auto ctx) {
auto f = ctx->tx().get(key, false);
ctx->continueAfter(
f,
[ctx, f, result]() {
auto outputVal = f.get();
ASSERT(outputVal.has_value());
*result = outputVal.value();
ctx->done();
},
true);
},
[this, opFunc, key, val1, val2, result, cont]() {
// 4. Assert expectation.
auto expected = opFunc(val1, val2);
if (*result != expected) {
error(fmt::format("testAtomicOp expected: {} actual: {}",
fdb::toCharsRef(expected),
fdb::toCharsRef(*result)));
ASSERT(false);
}
schedule(cont);
});
});
});
}
void testAtomicVersionstampedKeyOp(TTaskFct cont) {
Key keyPrefix(randomKeyName());
Key key = keyPrefix + fdb::ByteString(10, '\0') + toByteString((uint32_t)keyPrefix.size());
Value val = randomValue();
auto versionstamp_f = std::make_shared<fdb::TypedFuture<fdb::future_var::KeyRef>>();
execTransaction(
// 1. Perform SetVersionstampedKey operation.
[key, val, versionstamp_f](auto ctx) {
ctx->tx().atomicOp(key, val, FDBMutationType::FDB_MUTATION_TYPE_SET_VERSIONSTAMPED_KEY);
*versionstamp_f = ctx->tx().getVersionstamp();
ctx->commit();
},
[this, keyPrefix, val, versionstamp_f, cont]() {
ASSERT(versionstamp_f->ready());
auto resultKey = keyPrefix + Key(versionstamp_f->get());
auto resultVal = std::make_shared<Value>();
execTransaction(
// 2. Fetch the resulting versionstamped key and value.
[keyPrefix, resultKey, resultVal](auto ctx) {
auto fv = ctx->tx().get(resultKey, false);
ctx->continueAfter(fv, [ctx, fv, resultVal]() {
auto outputVal = fv.get();
ASSERT(outputVal.has_value());
*resultVal = outputVal.value();
ctx->done();
});
},
[this, keyPrefix, val, resultVal, cont]() {
// 3. Assert expectation.
ASSERT(*resultVal == val);
schedule(cont);
});
});
}
void testAtomicVersionstampedValueOp(TTaskFct cont) {
Key key(randomKeyName());
Value valPrefix = randomValue();
Value val = valPrefix + fdb::ByteString(10, '\0') + toByteString((uint32_t)valPrefix.size());
auto versionstamp_f = std::make_shared<fdb::TypedFuture<fdb::future_var::KeyRef>>();
execTransaction(
// 1. Perform SetVersionstampedValue operation.
[key, val, versionstamp_f](auto ctx) {
ctx->tx().atomicOp(key, val, FDBMutationType::FDB_MUTATION_TYPE_SET_VERSIONSTAMPED_VALUE);
*versionstamp_f = ctx->tx().getVersionstamp();
ctx->commit();
},
[this, key, valPrefix, versionstamp_f, cont]() {
versionstamp_f->blockUntilReady();
auto versionstamp = Key(versionstamp_f->get());
auto result = std::make_shared<Value>();
execTransaction(
// 2. Fetch the resulting versionstamped value.
[key, result](auto ctx) {
auto f = ctx->tx().get(key, false);
ctx->continueAfter(
f,
[ctx, f, result]() {
auto outputVal = f.get();
ASSERT(outputVal.has_value());
*result = outputVal.value();
ctx->done();
},
true);
},
[this, key, valPrefix, result, versionstamp, cont]() {
// 3. Assert expectation.
ASSERT(*result == valPrefix + versionstamp);
schedule(cont);
});
});
}
void testAtomicCompareAndClearOp(TTaskFct cont) {
Key key(randomKeyName());
Value val = randomValue();
execTransaction(
// 1. Set the key to initial value
[key, val](auto ctx) {
ctx->tx().set(key, val);
ctx->commit();
},
[this, key, val, cont]() {
execTransaction(
// 2. Perform CompareAndClear operation.
[key, val](auto ctx) {
ctx->tx().atomicOp(key, val, FDBMutationType::FDB_MUTATION_TYPE_COMPARE_AND_CLEAR);
ctx->commit();
},
[this, key, cont]() {
execTransaction(
// 3. Verify that the key was cleared.
[key](auto ctx) {
auto f = ctx->tx().get(key, false);
ctx->continueAfter(
f,
[ctx, f]() {
auto outputVal = f.get();
ASSERT(!outputVal.has_value());
ctx->done();
},
true);
},
[this, cont]() { schedule(cont); });
});
});
}
};
WorkloadFactory<AtomicOpsCorrectnessWorkload> AtomicOpsCorrectnessWorkloadFactory("AtomicOpsCorrectness");
} // namespace FdbApiTester

View File

@ -30,7 +30,16 @@ public:
ApiCorrectnessWorkload(const WorkloadConfig& config) : ApiWorkload(config) {}
private:
enum OpType { OP_INSERT, OP_GET, OP_CLEAR, OP_GET_RANGE, OP_CLEAR_RANGE, OP_COMMIT_READ, OP_LAST = OP_COMMIT_READ };
enum OpType {
OP_INSERT,
OP_GET,
OP_GET_KEY,
OP_CLEAR,
OP_GET_RANGE,
OP_CLEAR_RANGE,
OP_COMMIT_READ,
OP_LAST = OP_COMMIT_READ
};
void randomCommitReadOp(TTaskFct cont) {
int numKeys = Random::get().randomInt(1, maxKeysPerTransaction);
@ -125,6 +134,70 @@ private:
});
}
void randomGetKeyOp(TTaskFct cont) {
int numKeys = Random::get().randomInt(1, maxKeysPerTransaction);
auto keysWithSelectors = std::make_shared<std::vector<std::pair<fdb::Key, fdb::KeySelector>>>();
auto results = std::make_shared<std::vector<fdb::Key>>();
keysWithSelectors->reserve(numKeys);
for (int i = 0; i < numKeys; i++) {
auto key = randomKey(readExistingKeysRatio);
fdb::KeySelector selector;
selector.keyLength = key.size();
selector.orEqual = Random::get().randomBool(0.5);
selector.offset = Random::get().randomInt(0, 4);
keysWithSelectors->emplace_back(std::move(key), std::move(selector));
// We would ideally do the following above:
// selector.key = key.data();
// but key.data() may become invalid after the key is moved to the vector.
// So instead, we update the pointer here to the string already in the vector.
keysWithSelectors->back().second.key = keysWithSelectors->back().first.data();
}
execTransaction(
[keysWithSelectors, results](auto ctx) {
auto futures = std::make_shared<std::vector<fdb::Future>>();
for (const auto& keyWithSelector : *keysWithSelectors) {
auto key = keyWithSelector.first;
auto selector = keyWithSelector.second;
futures->push_back(ctx->tx().getKey(selector, false));
}
ctx->continueAfterAll(*futures, [ctx, futures, results]() {
results->clear();
for (auto& f : *futures) {
results->push_back(fdb::Key(f.get<fdb::future_var::KeyRef>()));
}
ASSERT(results->size() == futures->size());
ctx->done();
});
},
[this, keysWithSelectors, results, cont]() {
ASSERT(results->size() == keysWithSelectors->size());
for (int i = 0; i < keysWithSelectors->size(); i++) {
auto const& key = (*keysWithSelectors)[i].first;
auto const& selector = (*keysWithSelectors)[i].second;
auto expected = store.getKey(key, selector.orEqual, selector.offset);
auto actual = (*results)[i];
// Local store only contains data for the current client, while fdb contains data from multiple
// clients. If getKey returned a key outside of the range for the current client, adjust the result
// to match what would be expected in the local store.
if (actual.substr(0, keyPrefix.size()) < keyPrefix) {
actual = store.startKey();
} else if ((*results)[i].substr(0, keyPrefix.size()) > keyPrefix) {
actual = store.endKey();
}
if (actual != expected) {
error(fmt::format("randomGetKeyOp mismatch. key: {}, orEqual: {}, offset: {}, expected: {} "
"actual: {}",
fdb::toCharsRef(key),
selector.orEqual,
selector.offset,
fdb::toCharsRef(expected),
fdb::toCharsRef(actual)));
}
}
schedule(cont);
});
}
void getRangeLoop(std::shared_ptr<ITransactionContext> ctx,
fdb::KeySelector begin,
fdb::KeySelector end,
@ -199,6 +272,9 @@ private:
case OP_GET:
randomGetOp(cont);
break;
case OP_GET_KEY:
randomGetKeyOp(cont);
break;
case OP_CLEAR:
randomClearOp(cont);
break;

View File

@ -120,6 +120,25 @@ KeyValueArray copyKeyValueArray(fdb::future_var::KeyValueRefArray::Type array);
using KeyRangeArray = std::vector<fdb::KeyRange>;
KeyRangeArray copyKeyRangeArray(fdb::future_var::KeyRangeRefArray::Type array);
static_assert(__BYTE_ORDER__ == __ORDER_LITTLE_ENDIAN__, "Do not support non-little-endian systems");
// Converts a little-endian encoded number into an integral type.
template <class T, typename = std::enable_if_t<std::is_integral<T>::value>>
static T toInteger(fdb::BytesRef value) {
ASSERT(value.size() == sizeof(T));
T output;
memcpy(&output, value.data(), value.size());
return output;
}
// Converts an integral type to a little-endian encoded byte string.
template <class T, typename = std::enable_if_t<std::is_integral<T>::value>>
static fdb::ByteString toByteString(T value) {
fdb::ByteString output(sizeof(T), 0);
memcpy(output.data(), (const uint8_t*)&value, sizeof(value));
return output;
}
} // namespace FdbApiTester
#endif

View File

@ -12,14 +12,18 @@ maxClientThreads = 8
minClients = 2
maxClients = 8
[[test.workload]]
name = 'ApiCorrectness'
minKeyLength = 1
maxKeyLength = 64
minValueLength = 1
maxValueLength = 1000
maxKeysPerTransaction = 50
initialSize = 100
numRandomOperations = 100
readExistingKeysRatio = 0.9
maxKeyLength = 64
minValueLength = 1
maxValueLength = 1000
maxKeysPerTransaction = 50
initialSize = 100
numRandomOperations = 100
readExistingKeysRatio = 0.9
[[test.workload]]
name = 'AtomicOpsCorrectness'
initialSize = 0
numRandomOperations = 100

View File

@ -15,10 +15,15 @@ maxClients = 8
[[test.workload]]
name = 'ApiCorrectness'
minKeyLength = 1
maxKeyLength = 64
minValueLength = 1
maxValueLength = 1000
maxKeysPerTransaction = 50
initialSize = 100
numRandomOperations = 100
readExistingKeysRatio = 0.9
maxKeyLength = 64
minValueLength = 1
maxValueLength = 1000
maxKeysPerTransaction = 50
initialSize = 100
numRandomOperations = 100
readExistingKeysRatio = 0.9
[[test.workload]]
name = 'AtomicOpsCorrectness'
initialSize = 0
numRandomOperations = 100

View File

@ -15,10 +15,15 @@ maxClients = 8
[[test.workload]]
name = 'ApiCorrectness'
minKeyLength = 1
maxKeyLength = 64
minValueLength = 1
maxValueLength = 1000
maxKeysPerTransaction = 50
initialSize = 100
numRandomOperations = 100
readExistingKeysRatio = 0.9
maxKeyLength = 64
minValueLength = 1
maxValueLength = 1000
maxKeysPerTransaction = 50
initialSize = 100
numRandomOperations = 100
readExistingKeysRatio = 0.9
[[test.workload]]
name = 'AtomicOpsCorrectness'
initialSize = 0
numRandomOperations = 100

View File

@ -14,10 +14,15 @@ maxClients = 8
[[test.workload]]
name = 'ApiCorrectness'
minKeyLength = 1
maxKeyLength = 64
minValueLength = 1
maxValueLength = 1000
maxKeysPerTransaction = 50
initialSize = 100
numRandomOperations = 100
readExistingKeysRatio = 0.9
maxKeyLength = 64
minValueLength = 1
maxValueLength = 1000
maxKeysPerTransaction = 50
initialSize = 100
numRandomOperations = 100
readExistingKeysRatio = 0.9
[[test.workload]]
name = 'AtomicOpsCorrectness'
initialSize = 0
numRandomOperations = 100

View File

@ -7,10 +7,15 @@ multiThreaded = false
[[test.workload]]
name = 'ApiCorrectness'
minKeyLength = 1
maxKeyLength = 64
minValueLength = 1
maxValueLength = 1000
maxKeysPerTransaction = 50
initialSize = 100
numRandomOperations = 100
readExistingKeysRatio = 0.9
maxKeyLength = 64
minValueLength = 1
maxValueLength = 1000
maxKeysPerTransaction = 50
initialSize = 100
numRandomOperations = 100
readExistingKeysRatio = 0.9
[[test.workload]]
name = 'AtomicOpsCorrectness'
initialSize = 0
numRandomOperations = 100

View File

@ -197,16 +197,6 @@ inline int maxApiVersion() {
return native::fdb_get_max_api_version();
}
inline Error selectApiVersionNothrow(int version) {
return Error(native::fdb_select_api_version(version));
}
inline void selectApiVersion(int version) {
if (auto err = selectApiVersionNothrow(version)) {
throwError(fmt::format("ERROR: fdb_select_api_version({}): ", version), err);
}
}
namespace network {
inline Error setOptionNothrow(FDBNetworkOption option, BytesRef str) noexcept {
@ -516,6 +506,8 @@ public:
return out;
}
TypedFuture<future_var::KeyRef> getVersionstamp() { return native::fdb_transaction_get_versionstamp(tr.get()); }
TypedFuture<future_var::KeyRef> getKey(KeySelector sel, bool snapshot) {
return native::fdb_transaction_get_key(tr.get(), sel.key, sel.keyLength, sel.orEqual, sel.offset, snapshot);
}
@ -577,6 +569,11 @@ public:
native::fdb_transaction_set(tr.get(), key.data(), intSize(key), value.data(), intSize(value));
}
void atomicOp(KeyRef key, ValueRef param, FDBMutationType operationType) {
native::fdb_transaction_atomic_op(
tr.get(), key.data(), intSize(key), param.data(), intSize(param), operationType);
}
void clear(KeyRef key) { native::fdb_transaction_clear(tr.get(), key.data(), intSize(key)); }
void clearRange(KeyRef begin, KeyRef end) {
@ -588,14 +585,15 @@ class Tenant final {
friend class Database;
std::shared_ptr<native::FDBTenant> tenant;
static constexpr CharsRef tenantManagementMapPrefix = "\xff\xff/management/tenant_map/";
explicit Tenant(native::FDBTenant* tenant_raw) {
if (tenant_raw)
tenant = std::shared_ptr<native::FDBTenant>(tenant_raw, &native::fdb_tenant_destroy);
}
public:
// This should only be mutated by API versioning
static inline CharsRef tenantManagementMapPrefix = "\xff\xff/management/tenant/map/";
Tenant(const Tenant&) noexcept = default;
Tenant& operator=(const Tenant&) noexcept = default;
Tenant() noexcept : tenant(nullptr) {}
@ -684,6 +682,19 @@ public:
}
};
inline Error selectApiVersionNothrow(int version) {
if (version < 720) {
Tenant::tenantManagementMapPrefix = "\xff\xff/management/tenant_map/";
}
return Error(native::fdb_select_api_version(version));
}
inline void selectApiVersion(int version) {
if (auto err = selectApiVersionNothrow(version)) {
throwError(fmt::format("ERROR: fdb_select_api_version({}): ", version), err);
}
}
} // namespace fdb
template <>

View File

@ -2624,7 +2624,7 @@ TEST_CASE("Tenant create, access, and delete") {
fdb::Transaction tr(db);
while (1) {
fdb_check(tr.set_option(FDB_TR_OPTION_SPECIAL_KEY_SPACE_ENABLE_WRITES, nullptr, 0));
tr.set("\xff\xff/management/tenant_map/" + tenantName, "");
tr.set("\xff\xff/management/tenant/map/" + tenantName, "");
fdb::EmptyFuture commitFuture = tr.commit();
fdb_error_t err = wait_future(commitFuture);
if (err) {
@ -2637,8 +2637,8 @@ TEST_CASE("Tenant create, access, and delete") {
}
while (1) {
StringRef begin = "\xff\xff/management/tenant_map/"_sr;
StringRef end = "\xff\xff/management/tenant_map0"_sr;
StringRef begin = "\xff\xff/management/tenant/map/"_sr;
StringRef end = "\xff\xff/management/tenant/map0"_sr;
fdb_check(tr.set_option(FDB_TR_OPTION_SPECIAL_KEY_SPACE_ENABLE_WRITES, nullptr, 0));
fdb::KeyValueArrayFuture f = tr.get_range(FDB_KEYSEL_FIRST_GREATER_OR_EQUAL(begin.begin(), begin.size()),
@ -2716,7 +2716,7 @@ TEST_CASE("Tenant create, access, and delete") {
while (1) {
fdb_check(tr.set_option(FDB_TR_OPTION_SPECIAL_KEY_SPACE_ENABLE_WRITES, nullptr, 0));
tr.clear("\xff\xff/management/tenant_map/" + tenantName);
tr.clear("\xff\xff/management/tenant/map/" + tenantName);
fdb::EmptyFuture commitFuture = tr.commit();
fdb_error_t err = wait_future(commitFuture);
if (err) {

View File

@ -27,6 +27,8 @@ import java.util.concurrent.Semaphore;
import java.util.concurrent.ThreadFactory;
import java.util.concurrent.atomic.AtomicInteger;
import com.apple.foundationdb.tuple.ByteArrayUtil;
/**
* The starting point for accessing FoundationDB.
* <br>
@ -189,6 +191,11 @@ 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

@ -40,8 +40,8 @@ import com.apple.foundationdb.tuple.Tuple;
* The FoundationDB API includes function to manage the set of tenants in a cluster.
*/
public class TenantManagement {
static final byte[] TENANT_MAP_PREFIX = ByteArrayUtil.join(new byte[] { (byte)255, (byte)255 },
"/management/tenant_map/".getBytes());
static byte[] TENANT_MAP_PREFIX = ByteArrayUtil.join(new byte[] { (byte)255, (byte)255 },
"/management/tenant/map/".getBytes());
/**
* Creates a new tenant in the cluster. If the tenant already exists, this operation will complete

View File

@ -102,6 +102,8 @@ def api_version(ver):
if ver >= 710:
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

@ -25,7 +25,7 @@ https://apple.github.io/foundationdb/api-python.html"""
from fdb import impl as _impl
_tenant_map_prefix = b'\xff\xff/management/tenant_map/'
_tenant_map_prefix = b'\xff\xff/management/tenant/map/'
# If the existence_check_marker is an empty list, then check whether the tenant exists.
# After the check, append an item to the existence_check_marker list so that subsequent

View File

@ -78,12 +78,12 @@ def test_tenant_operations(db):
tenant1[b'tenant_test_key'] = b'tenant1'
tenant2[b'tenant_test_key'] = b'tenant2'
tenant1_entry = db[b'\xff\xff/management/tenant_map/tenant1']
tenant1_entry = db[b'\xff\xff/management/tenant/map/tenant1']
tenant1_json = json.loads(tenant1_entry)
prefix1 = tenant1_json['prefix'].encode('utf8')
assert prefix1 == p1
tenant2_entry = db[b'\xff\xff/management/tenant_map/tenant2']
tenant2_entry = db[b'\xff\xff/management/tenant/map/tenant2']
tenant2_json = json.loads(tenant2_entry)
prefix2 = tenant2_json['prefix'].encode('utf8')
assert prefix2 == p2

View File

@ -193,10 +193,9 @@ endif()
find_package(toml11 QUIET)
if(toml11_FOUND)
add_library(toml11_target INTERFACE)
add_dependencies(toml11_target INTERFACE toml11::toml11)
target_link_libraries(toml11_target INTERFACE toml11::toml11)
else()
include(ExternalProject)
include(ExternalProject)
ExternalProject_add(toml11Project
URL "https://github.com/ToruNiina/toml11/archive/v3.4.0.tar.gz"
URL_HASH SHA256=bc6d733efd9216af8c119d8ac64a805578c79cc82b813e4d1d880ca128bd154d

View File

@ -221,16 +221,10 @@ configure_file(${CMAKE_SOURCE_DIR}/LICENSE ${CMAKE_BINARY_DIR}/License.txt COPYO
################################################################################
# Filename of packages
################################################################################
set(CPACK_RPM_PACKAGE_RELEASE 1)
if(NOT FDB_RELEASE)
if(CURRENT_GIT_VERSION)
string(SUBSTRING ${CURRENT_GIT_VERSION} 0 9 git_hash)
endif()
set(CPACK_RPM_PACKAGE_RELEASE 0)
set(package_version_postfix "-0.${git_hash}.SNAPSHOT")
set(git_string ".${git_hash}")
set(package_version_postfix "-1.SNAPSHOT")
else()
set(CPACK_RPM_PACKAGE_RELEASE 1)
set(package_version_postfix "-1")
endif()
@ -239,6 +233,7 @@ endif()
################################################################################
string(REPLACE "-" "_" FDB_PACKAGE_VERSION ${FDB_VERSION})
set(CPACK_RPM_PACKAGE_GROUP ${CURRENT_GIT_VERSION})
set(CPACK_RPM_PACKAGE_LICENSE "Apache 2.0")
set(CPACK_RPM_PACKAGE_NAME "foundationdb")
set(CPACK_RPM_CLIENTS-EL7_PACKAGE_NAME "${CPACK_RPM_PACKAGE_NAME}-clients")
@ -260,14 +255,14 @@ set(CPACK_RPM_SERVER-EL7_USER_FILELIST "%config(noreplace) /
"%attr(0700,foundationdb,foundationdb) /var/lib/foundationdb")
set(CPACK_RPM_CLIENTS-VERSIONED_PACKAGE_NAME "${CPACK_RPM_PACKAGE_NAME}${FDB_PACKAGE_VERSION}${FDB_BUILDTIME_STRING}${PRERELEASE_TAG}-clients")
set(CPACK_RPM_CLIENTS-VERSIONED_FILE_NAME "${CPACK_RPM_CLIENTS-VERSIONED_PACKAGE_NAME}${git_string}.versioned.${CMAKE_SYSTEM_PROCESSOR}.rpm")
set(CPACK_RPM_CLIENTS-VERSIONED_DEBUGINFO_FILE_NAME "${CPACK_RPM_CLIENTS-VERSIONED_PACKAGE_NAME}${git_string}.versioned-debuginfo.${CMAKE_SYSTEM_PROCESSOR}.rpm")
set(CPACK_RPM_CLIENTS-VERSIONED_FILE_NAME "${CPACK_RPM_CLIENTS-VERSIONED_PACKAGE_NAME}-${CPACK_RPM_PACKAGE_RELEASE}.versioned.${CMAKE_SYSTEM_PROCESSOR}.rpm")
set(CPACK_RPM_CLIENTS-VERSIONED_DEBUGINFO_FILE_NAME "${CPACK_RPM_CLIENTS-VERSIONED_PACKAGE_NAME}-${CPACK_RPM_PACKAGE_RELEASE}.versioned-debuginfo.${CMAKE_SYSTEM_PROCESSOR}.rpm")
set(CPACK_RPM_CLIENTS-VERSIONED_POST_INSTALL_SCRIPT_FILE ${CMAKE_BINARY_DIR}/packaging/multiversion/clients/postinst-el7)
set(CPACK_RPM_CLIENTS-VERSIONED_PRE_UNINSTALL_SCRIPT_FILE ${CMAKE_BINARY_DIR}/packaging/multiversion/clients/prerm)
set(CPACK_RPM_SERVER-VERSIONED_PACKAGE_NAME "${CPACK_RPM_PACKAGE_NAME}${FDB_PACKAGE_VERSION}${FDB_BUILDTIME_STRING}${PRERELEASE_TAG}-server")
set(CPACK_RPM_SERVER-VERSIONED_FILE_NAME "${CPACK_RPM_SERVER-VERSIONED_PACKAGE_NAME}${git_string}.versioned.${CMAKE_SYSTEM_PROCESSOR}.rpm")
set(CPACK_RPM_SERVER-VERSIONED_DEBUGINFO_FILE_NAME "${CPACK_RPM_SERVER-VERSIONED_PACKAGE_NAME}${git_string}.versioned-debuginfo.${CMAKE_SYSTEM_PROCESSOR}.rpm")
set(CPACK_RPM_SERVER-VERSIONED_FILE_NAME "${CPACK_RPM_SERVER-VERSIONED_PACKAGE_NAME}-${CPACK_RPM_PACKAGE_RELEASE}.versioned.${CMAKE_SYSTEM_PROCESSOR}.rpm")
set(CPACK_RPM_SERVER-VERSIONED_DEBUGINFO_FILE_NAME "${CPACK_RPM_SERVER-VERSIONED_PACKAGE_NAME}-${CPACK_RPM_PACKAGE_RELEASE}.versioned-debuginfo.${CMAKE_SYSTEM_PROCESSOR}.rpm")
set(CPACK_RPM_SERVER-VERSIONED_PACKAGE_REQUIRES "${CPACK_COMPONENT_CLIENTS-VERSIONED_DISPLAY_NAME}")
set(CPACK_RPM_SERVER-VERSIONED_POST_INSTALL_SCRIPT_FILE ${CMAKE_BINARY_DIR}/packaging/multiversion/server/postinst-rpm)
set(CPACK_RPM_SERVER-VERSIONED_PRE_UNINSTALL_SCRIPT_FILE ${CMAKE_BINARY_DIR}/packaging/multiversion/server/prerm)
@ -309,13 +304,13 @@ set(CPACK_RPM_COMPONENT_INSTALL ON)
if (CMAKE_SYSTEM_PROCESSOR MATCHES "x86_64")
set(CPACK_DEBIAN_CLIENTS-DEB_FILE_NAME "foundationdb-clients_${FDB_VERSION}${package_version_postfix}_amd64.deb")
set(CPACK_DEBIAN_SERVER-DEB_FILE_NAME "foundationdb-server_${FDB_VERSION}${package_version_postfix}_amd64.deb")
set(CPACK_DEBIAN_CLIENTS-VERSIONED_FILE_NAME "foundationdb${FDB_VERSION}${FDB_BUILDTIME_STRING}${PRERELEASE_TAG}-clients${git_string}.versioned_amd64.deb")
set(CPACK_DEBIAN_SERVER-VERSIONED_FILE_NAME "foundationdb${FDB_VERSION}${FDB_BUILDTIME_STRING}${PRERELEASE_TAG}-server${git_string}.versioned_amd64.deb")
set(CPACK_DEBIAN_CLIENTS-VERSIONED_FILE_NAME "foundationdb${FDB_VERSION}${FDB_BUILDTIME_STRING}${PRERELEASE_TAG}-clients-${CPACK_RPM_PACKAGE_RELEASE}.versioned_amd64.deb")
set(CPACK_DEBIAN_SERVER-VERSIONED_FILE_NAME "foundationdb${FDB_VERSION}${FDB_BUILDTIME_STRING}${PRERELEASE_TAG}-server-${CPACK_RPM_PACKAGE_RELEASE}.versioned_amd64.deb")
else()
set(CPACK_DEBIAN_CLIENTS-DEB_FILE_NAME "foundationdb-clients_${FDB_VERSION}${package_version_postfix}_${CMAKE_SYSTEM_PROCESSOR}.deb")
set(CPACK_DEBIAN_SERVER-DEB_FILE_NAME "foundationdb-server_${FDB_VERSION}${package_version_postfix}_${CMAKE_SYSTEM_PROCESSOR}.deb")
set(CPACK_DEBIAN_CLIENTS-VERSIONED_FILE_NAME "foundationdb${FDB_VERSION}${FDB_BUILDTIME_STRING}${PRERELEASE_TAG}-clients${git_string}.versioned_${CMAKE_SYSTEM_PROCESSOR}.deb")
set(CPACK_DEBIAN_SERVER-VERSIONED_FILE_NAME "foundationdb${FDB_VERSION}${FDB_BUILDTIME_STRING}${PRERELEASE_TAG}-server${git_string}.versioned_${CMAKE_SYSTEM_PROCESSOR}.deb")
set(CPACK_DEBIAN_CLIENTS-VERSIONED_FILE_NAME "foundationdb${FDB_VERSION}${FDB_BUILDTIME_STRING}${PRERELEASE_TAG}-clients-${CPACK_RPM_PACKAGE_RELEASE}.versioned_${CMAKE_SYSTEM_PROCESSOR}.deb")
set(CPACK_DEBIAN_SERVER-VERSIONED_FILE_NAME "foundationdb${FDB_VERSION}${FDB_BUILDTIME_STRING}${PRERELEASE_TAG}-server-${CPACK_RPM_PACKAGE_RELEASE}.versioned_${CMAKE_SYSTEM_PROCESSOR}.deb")
endif()
set(CPACK_DEB_COMPONENT_INSTALL ON)

View File

@ -4,6 +4,7 @@ target_include_directories(rapidjson INTERFACE ${CMAKE_CURRENT_SOURCE_DIR}/rapid
add_subdirectory(crc32)
add_subdirectory(stacktrace)
add_subdirectory(folly_memcpy)
add_subdirectory(rapidxml)
add_subdirectory(sqlite)
add_subdirectory(SimpleOpt)
add_subdirectory(fmt-8.1.1)

View File

@ -35,6 +35,10 @@
#endif
#endif
#ifdef _WIN32
#include <intrin.h>
#endif
#include "crc32/crc32c.h"
#if !defined(__aarch64__) && !defined(__powerpc64__)

View File

@ -0,0 +1,2 @@
add_library(rapidxml INTERFACE)
target_include_directories(rapidxml INTERFACE "${CMAKE_CURRENT_SOURCE_DIR}/include")

View File

@ -204,7 +204,7 @@ that process, and wait for necessary data to be moved away.
#. ``\xff\xff/management/failed_locality/<locality>`` Read/write. Indicates that the cluster should consider matching processes as permanently failed. This allows the cluster to avoid maintaining extra state and doing extra work in the hope that these processes come back. See :ref:`removing machines from a cluster <removing-machines-from-a-cluster>` for documentation for the corresponding fdbcli command.
#. ``\xff\xff/management/options/excluded_locality/force`` Read/write. Setting this key disables safety checks for writes to ``\xff\xff/management/excluded_locality/<locality>``. Setting this key only has an effect in the current transaction and is not persisted on commit.
#. ``\xff\xff/management/options/failed_locality/force`` Read/write. Setting this key disables safety checks for writes to ``\xff\xff/management/failed_locality/<locality>``. Setting this key only has an effect in the current transaction and is not persisted on commit.
#. ``\xff\xff/management/tenant_map/<tenant>`` Read/write. Setting a key in this range to any value will result in a tenant being created with name ``<tenant>``. Clearing a key in this range will delete the tenant with name ``<tenant>``. Reading all or a portion of this range will return the list of tenants currently present in the cluster, excluding any changes in this transaction. Values read in this range will be JSON objects containing the metadata for the associated tenants.
#. ``\xff\xff/management/tenant/map/<tenant>`` Read/write. Setting a key in this range to any value will result in a tenant being created with name ``<tenant>``. Clearing a key in this range will delete the tenant with name ``<tenant>``. Reading all or a portion of this range will return the list of tenants currently present in the cluster, excluding any changes in this transaction. Values read in this range will be JSON objects containing the metadata for the associated tenants.
An exclusion is syntactically either an ip address (e.g. ``127.0.0.1``), or
an ip address and port (e.g. ``127.0.0.1:4500``) or any locality (e.g ``locality_dcid:primary-satellite`` or
@ -270,7 +270,8 @@ 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 7.2. 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/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

@ -31,7 +31,7 @@ FoundationDB clusters support the following tenant modes:
Creating and deleting tenants
=============================
Tenants can be created and deleted using the ``\xff\xff/management/tenant_map/<tenant_name>`` :doc:`special key <special-keys>` range as well as by using APIs provided in some language bindings.
Tenants can be created and deleted using the ``\xff\xff/management/tenant/map/<tenant_name>`` :doc:`special key <special-keys>` range as well as by using APIs provided in some language bindings.
Tenants can be created with any byte-string name that does not begin with the ``\xff`` character. Once created, a tenant will be assigned an ID and a prefix where its data will reside.

View File

@ -1920,11 +1920,11 @@ ACTOR Future<Void> submitBackup(Database db,
if (dryRun) {
state KeyBackedTag tag = makeBackupTag(tagName);
Optional<UidAndAbortedFlagT> uidFlag = wait(tag.get(db));
Optional<UidAndAbortedFlagT> uidFlag = wait(tag.get(db.getReference()));
if (uidFlag.present()) {
BackupConfig config(uidFlag.get().first);
EBackupState backupStatus = wait(config.stateEnum().getOrThrow(db));
EBackupState backupStatus = wait(config.stateEnum().getOrThrow(db.getReference()));
// Throw error if a backup is currently running until we support parallel backups
if (BackupAgentBase::isRunnable(backupStatus)) {
@ -2883,7 +2883,7 @@ ACTOR Future<Void> modifyBackup(Database db, std::string tagName, BackupModifyOp
tr->setOption(FDBTransactionOptions::ACCESS_SYSTEM_KEYS);
tr->setOption(FDBTransactionOptions::LOCK_AWARE);
state Optional<UidAndAbortedFlagT> uidFlag = wait(tag.get(db));
state Optional<UidAndAbortedFlagT> uidFlag = wait(tag.get(db.getReference()));
if (!uidFlag.present()) {
fprintf(stderr, "No backup exists on tag '%s'\n", tagName.c_str());

View File

@ -35,8 +35,8 @@
namespace fdb_cli {
const KeyRangeRef tenantSpecialKeyRange(LiteralStringRef("\xff\xff/management/tenant_map/"),
LiteralStringRef("\xff\xff/management/tenant_map0"));
const KeyRangeRef tenantSpecialKeyRange(LiteralStringRef("\xff\xff/management/tenant/map/"),
LiteralStringRef("\xff\xff/management/tenant/map0"));
// createtenant command
ACTOR Future<bool> createTenantCommandActor(Reference<IDatabase> db, std::vector<StringRef> tokens) {

View File

@ -68,11 +68,6 @@ endif()
if(WITH_AWS_BACKUP)
add_compile_definitions(BUILD_AWS_BACKUP)
set(FDBCLIENT_SRCS
${FDBCLIENT_SRCS}
FDBAWSCredentialsProvider.h)
include(awssdk)
endif()
@ -80,7 +75,7 @@ add_flow_target(STATIC_LIBRARY NAME fdbclient SRCS ${FDBCLIENT_SRCS} ADDL_SRCS $
target_include_directories(fdbclient PUBLIC "${CMAKE_CURRENT_SOURCE_DIR}/include" "${CMAKE_CURRENT_BINARY_DIR}/include")
configure_file(${CMAKE_CURRENT_SOURCE_DIR}/versions.h.cmake ${CMAKE_CURRENT_BINARY_DIR}/include/fdbclient/versions.h)
add_dependencies(fdbclient fdboptions)
target_link_libraries(fdbclient PUBLIC fdbrpc msgpack)
target_link_libraries(fdbclient PUBLIC fdbrpc msgpack PRIVATE rapidxml)
# Create a separate fdbclient library with sampling enabled. This lets
# fdbserver retain sampling functionality in client code while disabling
@ -88,7 +83,7 @@ target_link_libraries(fdbclient PUBLIC fdbrpc msgpack)
add_flow_target(STATIC_LIBRARY NAME fdbclient_sampling SRCS ${FDBCLIENT_SRCS} ADDL_SRCS ${options_srcs})
target_include_directories(fdbclient_sampling PUBLIC "${CMAKE_CURRENT_SOURCE_DIR}/include" "${CMAKE_CURRENT_BINARY_DIR}/include")
add_dependencies(fdbclient_sampling fdboptions)
target_link_libraries(fdbclient_sampling PUBLIC fdbrpc_sampling msgpack)
target_link_libraries(fdbclient_sampling PUBLIC fdbrpc_sampling msgpack PRIVATE rapidxml)
target_compile_definitions(fdbclient_sampling PRIVATE -DENABLE_SAMPLING)
if(WIN32)
add_dependencies(fdbclient_sampling_actors fdbclient_actors)

View File

@ -0,0 +1,44 @@
/*
* FDBAWSCredentialsProvider.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/FDBAWSCredentialsProvider.h"
#include "fdbclient/Tracing.h"
#ifdef BUILD_AWS_BACKUP
// You're supposed to call AWS::ShutdownAPI(options); once done
// But we want this to live for the lifetime of the process, so we don't do that
namespace FDBAWSCredentialsProvider {
Aws::Auth::AWSCredentials getAwsCredentials() {
static bool doneInit = false;
if (!doneInit) {
doneInit = true;
Aws::SDKOptions options;
Aws::InitAPI(options);
TraceEvent("AWSSDKInitSuccessful");
}
Aws::Auth::DefaultAWSCredentialsProviderChain credProvider;
Aws::Auth::AWSCredentials creds = credProvider.GetAWSCredentials();
return creds;
}
} // namespace FDBAWSCredentialsProvider
#endif

View File

@ -1221,7 +1221,7 @@ struct BackupRangeTaskFunc : BackupTaskFuncBase {
// Don't need to check keepRunning(task) here because we will do that while finishing each output file, but if
// bc is false then clearly the backup is no longer in progress
state Reference<IBackupContainer> bc = wait(backup.backupContainer().getD(cx));
state Reference<IBackupContainer> bc = wait(backup.backupContainer().getD(cx.getReference()));
if (!bc) {
return Void();
}
@ -3617,8 +3617,8 @@ struct RestoreDispatchTaskFunc : RestoreTaskFuncBase {
// Get a batch of files. We're targeting batchSize blocks being dispatched so query for batchSize files (each
// of which is 0 or more blocks).
state int taskBatchSize = BUGGIFY ? 1 : CLIENT_KNOBS->RESTORE_DISPATCH_ADDTASK_SIZE;
state RestoreConfig::FileSetT::Values files =
wait(restore.fileSet().getRange(tr, { beginVersion, beginFile }, {}, taskBatchSize));
state RestoreConfig::FileSetT::Values files = wait(restore.fileSet().getRange(
tr, Optional<RestoreConfig::RestoreFile>({ beginVersion, beginFile }), {}, taskBatchSize));
// allPartsDone will be set once all block tasks in the current batch are finished.
state Reference<TaskFuture> allPartsDone;
@ -5434,7 +5434,7 @@ public:
try {
// We must get a commit version so add a conflict range that won't likely cause conflicts
// but will ensure that the transaction is actually submitted.
tr.addWriteConflictRange(backupConfig.snapshotRangeDispatchMap().space.range());
tr.addWriteConflictRange(backupConfig.snapshotRangeDispatchMap().subspace);
wait(lockDatabase(&tr, randomUid));
wait(tr.commit());
commitVersion = tr.getCommittedVersion();
@ -5497,7 +5497,7 @@ public:
}
}
Reference<IBackupContainer> bc = wait(backupConfig.backupContainer().getOrThrow(cx));
Reference<IBackupContainer> bc = wait(backupConfig.backupContainer().getOrThrow(cx.getReference()));
if (fastRestore) {
TraceEvent("AtomicParallelRestoreStartRestore").log();

View File

@ -61,6 +61,7 @@
#include "fdbclient/SpecialKeySpace.actor.h"
#include "fdbclient/StorageServerInterface.h"
#include "fdbclient/SystemData.h"
#include "fdbclient/TenantSpecialKeys.actor.h"
#include "fdbclient/TransactionLineage.h"
#include "fdbclient/versions.h"
#include "fdbrpc/WellKnownEndpoints.h"
@ -237,6 +238,10 @@ void DatabaseContext::getLatestCommitVersions(const Reference<LocationInfo>& loc
return;
}
if (ssVersionVectorCache.getMaxVersion() == invalidVersion) {
return;
}
if (ssVersionVectorCache.getMaxVersion() != invalidVersion && readVersion > ssVersionVectorCache.getMaxVersion()) {
if (!CLIENT_KNOBS->FORCE_GRV_CACHE_OFF && !info->options.skipGrvCache && info->options.useGrvCache) {
return;
@ -1491,11 +1496,17 @@ DatabaseContext::DatabaseContext(Reference<AsyncVar<Reference<IClusterConnection
smoothMidShardSize.reset(CLIENT_KNOBS->INIT_MID_SHARD_BYTES);
globalConfig = std::make_unique<GlobalConfig>(this);
if (apiVersionAtLeast(710)) {
if (apiVersionAtLeast(720)) {
registerSpecialKeysImpl(
SpecialKeySpace::MODULE::MANAGEMENT,
SpecialKeySpace::IMPLTYPE::READWRITE,
std::make_unique<TenantMapRangeImpl>(SpecialKeySpace::getManagementApiCommandRange("tenantmap")));
std::make_unique<TenantRangeImpl<true>>(SpecialKeySpace::getManagementApiCommandRange("tenant")));
}
if (apiVersionAtLeast(710) && !apiVersionAtLeast(720)) {
registerSpecialKeysImpl(
SpecialKeySpace::MODULE::MANAGEMENT,
SpecialKeySpace::IMPLTYPE::READWRITE,
std::make_unique<TenantRangeImpl<false>>(SpecialKeySpace::getManagementApiCommandRange("tenantmap")));
}
if (apiVersionAtLeast(700)) {
registerSpecialKeysImpl(SpecialKeySpace::MODULE::ERRORMSG,
@ -3474,10 +3485,12 @@ ACTOR Future<Version> waitForCommittedVersion(Database cx, Version version, Span
cx->minAcceptableReadVersion = std::min(cx->minAcceptableReadVersion, v.version);
if (v.midShardSize > 0)
cx->smoothMidShardSize.setTotal(v.midShardSize);
if (cx->isCurrentGrvProxy(v.proxyId)) {
cx->ssVersionVectorCache.applyDelta(v.ssVersionVectorDelta);
} else {
cx->ssVersionVectorCache.clear();
if (cx->versionVectorCacheActive(v.ssVersionVectorDelta)) {
if (cx->isCurrentGrvProxy(v.proxyId)) {
cx->ssVersionVectorCache.applyDelta(v.ssVersionVectorDelta);
} else {
cx->ssVersionVectorCache.clear();
}
}
if (v.version >= version)
return v.version;
@ -3506,10 +3519,12 @@ ACTOR Future<Version> getRawVersion(Reference<TransactionState> trState) {
TransactionPriority::IMMEDIATE,
trState->cx->ssVersionVectorCache.getMaxVersion()),
trState->cx->taskID))) {
if (trState->cx->isCurrentGrvProxy(v.proxyId)) {
trState->cx->ssVersionVectorCache.applyDelta(v.ssVersionVectorDelta);
} else {
trState->cx->ssVersionVectorCache.clear();
if (trState->cx->versionVectorCacheActive(v.ssVersionVectorDelta)) {
if (trState->cx->isCurrentGrvProxy(v.proxyId)) {
trState->cx->ssVersionVectorCache.applyDelta(v.ssVersionVectorDelta);
} else {
trState->cx->ssVersionVectorCache.clear();
}
}
return v.version;
}
@ -6645,10 +6660,12 @@ ACTOR Future<GetReadVersionReply> getConsistentReadVersion(SpanContext parentSpa
"TransactionDebug", debugID.get().first(), "NativeAPI.getConsistentReadVersion.After");
ASSERT(v.version > 0);
cx->minAcceptableReadVersion = std::min(cx->minAcceptableReadVersion, v.version);
if (cx->isCurrentGrvProxy(v.proxyId)) {
cx->ssVersionVectorCache.applyDelta(v.ssVersionVectorDelta);
} else {
continue; // stale GRV reply, retry
if (cx->versionVectorCacheActive(v.ssVersionVectorDelta)) {
if (cx->isCurrentGrvProxy(v.proxyId)) {
cx->ssVersionVectorCache.applyDelta(v.ssVersionVectorDelta);
} else {
continue; // stale GRV reply, retry
}
}
return v;
}
@ -6835,10 +6852,12 @@ ACTOR Future<Version> extractReadVersion(Reference<TransactionState> trState,
}
metadataVersion.send(rep.metadataVersion);
if (trState->cx->isCurrentGrvProxy(rep.proxyId)) {
trState->cx->ssVersionVectorCache.applyDelta(rep.ssVersionVectorDelta);
} else {
trState->cx->ssVersionVectorCache.clear();
if (trState->cx->versionVectorCacheActive(rep.ssVersionVectorDelta)) {
if (trState->cx->isCurrentGrvProxy(rep.proxyId)) {
trState->cx->ssVersionVectorCache.applyDelta(rep.ssVersionVectorDelta);
} else {
trState->cx->ssVersionVectorCache.clear();
}
}
return rep.version;
}
@ -7909,7 +7928,8 @@ ACTOR Future<Void> splitStorageMetricsStream(PromiseStream<Key> resultStream,
Database cx,
KeyRange keys,
StorageMetrics limit,
StorageMetrics estimated) {
StorageMetrics estimated,
Optional<int> minSplitBytes) {
state Span span("NAPI:SplitStorageMetricsStream"_loc);
state Key beginKey = keys.begin;
state Key globalLastKey = beginKey;
@ -7940,7 +7960,8 @@ ACTOR Future<Void> splitStorageMetricsStream(PromiseStream<Key> resultStream,
limit,
localUsed,
estimated,
i == locations.size() - 1 && keys.end <= locations.back().range.end);
i == locations.size() - 1 && keys.end <= locations.back().range.end,
minSplitBytes);
SplitMetricsReply res = wait(loadBalance(locations[i].locations->locations(),
&StorageServerInterface::splitMetrics,
req,
@ -8003,15 +8024,17 @@ ACTOR Future<Void> splitStorageMetricsStream(PromiseStream<Key> resultStream,
Future<Void> DatabaseContext::splitStorageMetricsStream(const PromiseStream<Key>& resultStream,
KeyRange const& keys,
StorageMetrics const& limit,
StorageMetrics const& estimated) {
StorageMetrics const& estimated,
Optional<int> const& minSplitBytes) {
return ::splitStorageMetricsStream(
resultStream, Database(Reference<DatabaseContext>::addRef(this)), keys, limit, estimated);
resultStream, Database(Reference<DatabaseContext>::addRef(this)), keys, limit, estimated, minSplitBytes);
}
ACTOR Future<Standalone<VectorRef<KeyRef>>> splitStorageMetrics(Database cx,
KeyRange keys,
StorageMetrics limit,
StorageMetrics estimated) {
StorageMetrics estimated,
Optional<int> minSplitBytes) {
state Span span("NAPI:SplitStorageMetrics"_loc);
loop {
state std::vector<KeyRangeLocationInfo> locations =
@ -8040,7 +8063,8 @@ ACTOR Future<Standalone<VectorRef<KeyRef>>> splitStorageMetrics(Database cx,
state int i = 0;
for (; i < locations.size(); i++) {
SplitMetricsRequest req(locations[i].range, limit, used, estimated, i == locations.size() - 1);
SplitMetricsRequest req(
locations[i].range, limit, used, estimated, i == locations.size() - 1, minSplitBytes);
SplitMetricsReply res = wait(loadBalance(locations[i].locations->locations(),
&StorageServerInterface::splitMetrics,
req,
@ -8084,8 +8108,10 @@ ACTOR Future<Standalone<VectorRef<KeyRef>>> splitStorageMetrics(Database cx,
Future<Standalone<VectorRef<KeyRef>>> DatabaseContext::splitStorageMetrics(KeyRange const& keys,
StorageMetrics const& limit,
StorageMetrics const& estimated) {
return ::splitStorageMetrics(Database(Reference<DatabaseContext>::addRef(this)), keys, limit, estimated);
StorageMetrics const& estimated,
Optional<int> const& minSplitBytes) {
return ::splitStorageMetrics(
Database(Reference<DatabaseContext>::addRef(this)), keys, limit, estimated, minSplitBytes);
}
void Transaction::checkDeferredError() const {

View File

@ -40,7 +40,7 @@
#include "flow/IAsyncFile.h"
#include "flow/Hostname.h"
#include "flow/UnitTest.h"
#include "fdbclient/rapidxml/rapidxml.hpp"
#include "rapidxml/rapidxml.hpp"
#ifdef BUILD_AWS_BACKUP
#include "fdbclient/FDBAWSCredentialsProvider.h"
#endif

View File

@ -575,7 +575,8 @@ const KeyRef JSONSchemas::statusSchema = LiteralStringRef(R"statusSchema(
"duplicate_mutation_streams",
"duplicate_mutation_fetch_timeout",
"primary_dc_missing",
"fetch_primary_dc_timeout"
"fetch_primary_dc_timeout",
"fetch_storage_wiggler_stats_timeout"
]
},
"issues":[

View File

@ -20,6 +20,7 @@
#include "fdbclient/ServerKnobs.h"
#include "flow/IRandom.h"
#include "flow/flow.h"
#define init(...) KNOB_FN(__VA_ARGS__, INIT_ATOMIC_KNOB, INIT_KNOB)(__VA_ARGS__)
@ -35,12 +36,13 @@ void ServerKnobs::initialize(Randomize randomize, ClientKnobs* clientKnobs, IsSi
init( MAX_VERSIONS_IN_FLIGHT, 100 * VERSIONS_PER_SECOND );
init( MAX_VERSIONS_IN_FLIGHT_FORCED, 6e5 * VERSIONS_PER_SECOND ); //one week of versions
init( ENABLE_VERSION_VECTOR, false );
init( ENABLE_VERSION_VECTOR_TLOG_UNICAST, false );
bool buggifyShortReadWindow = randomize && BUGGIFY && !ENABLE_VERSION_VECTOR;
init( MAX_READ_TRANSACTION_LIFE_VERSIONS, 5 * VERSIONS_PER_SECOND ); if (randomize && BUGGIFY) MAX_READ_TRANSACTION_LIFE_VERSIONS = VERSIONS_PER_SECOND; else if (buggifyShortReadWindow) MAX_READ_TRANSACTION_LIFE_VERSIONS = std::max<int>(1, 0.1 * VERSIONS_PER_SECOND); else if( randomize && BUGGIFY ) MAX_READ_TRANSACTION_LIFE_VERSIONS = 10 * VERSIONS_PER_SECOND;
init( MAX_WRITE_TRANSACTION_LIFE_VERSIONS, 5 * VERSIONS_PER_SECOND ); if (randomize && BUGGIFY) MAX_WRITE_TRANSACTION_LIFE_VERSIONS=std::max<int>(1, 1 * VERSIONS_PER_SECOND);
init( MAX_COMMIT_BATCH_INTERVAL, 2.0 ); if( randomize && BUGGIFY ) MAX_COMMIT_BATCH_INTERVAL = 0.5; // Each commit proxy generates a CommitTransactionBatchRequest at least this often, so that versions always advance smoothly
MAX_COMMIT_BATCH_INTERVAL = std::min(MAX_COMMIT_BATCH_INTERVAL, MAX_READ_TRANSACTION_LIFE_VERSIONS/double(2*VERSIONS_PER_SECOND)); // Ensure that the proxy commits 2 times every MAX_READ_TRANSACTION_LIFE_VERSIONS, otherwise the master will not give out versions fast enough
init( ENABLE_VERSION_VECTOR_TLOG_UNICAST, false );
init( MAX_VERSION_RATE_MODIFIER, 0.1 );
init( MAX_VERSION_RATE_OFFSET, VERSIONS_PER_SECOND ); // If the calculated version is more than this amount away from the expected version, it will be clamped to this value. This prevents huge version jumps.
init( ENABLE_VERSION_VECTOR_HA_OPTIMIZATION, false );
@ -732,6 +734,7 @@ void ServerKnobs::initialize(Randomize randomize, ClientKnobs* clientKnobs, IsSi
init( CHECKPOINT_TRANSFER_BLOCK_BYTES, 40e6 );
init( QUICK_GET_VALUE_FALLBACK, true );
init( QUICK_GET_KEY_VALUES_FALLBACK, true );
init( MAX_PARALLEL_QUICK_GET_VALUE, 50 ); if ( randomize && BUGGIFY ) MAX_PARALLEL_QUICK_GET_VALUE = deterministicRandom()->randomInt(1, 100);
init( QUICK_GET_KEY_VALUES_LIMIT, 2000 );
init( QUICK_GET_KEY_VALUES_LIMIT_BYTES, 1e7 );

View File

@ -55,8 +55,6 @@ static bool isAlphaNumeric(const std::string& key) {
}
} // namespace
const KeyRangeRef TenantMapRangeImpl::submoduleRange = KeyRangeRef("tenant_map/"_sr, "tenant_map0"_sr);
std::unordered_map<SpecialKeySpace::MODULE, KeyRange> SpecialKeySpace::moduleToBoundary = {
{ SpecialKeySpace::MODULE::TRANSACTION,
KeyRangeRef(LiteralStringRef("\xff\xff/transaction/"), LiteralStringRef("\xff\xff/transaction0")) },
@ -117,7 +115,9 @@ std::unordered_map<std::string, KeyRange> SpecialKeySpace::managementApiCommandT
{ "datadistribution",
KeyRangeRef(LiteralStringRef("data_distribution/"), LiteralStringRef("data_distribution0"))
.withPrefix(moduleToBoundary[MODULE::MANAGEMENT].begin) },
{ "tenantmap", TenantMapRangeImpl::submoduleRange.withPrefix(moduleToBoundary[MODULE::MANAGEMENT].begin) }
{ "tenant", KeyRangeRef("tenant/"_sr, "tenant0"_sr).withPrefix(moduleToBoundary[MODULE::MANAGEMENT].begin) },
{ "tenantmap",
KeyRangeRef("tenant_map/"_sr, "tenant_map0"_sr).withPrefix(moduleToBoundary[MODULE::MANAGEMENT].begin) }
};
std::unordered_map<std::string, KeyRange> SpecialKeySpace::actorLineageApiCommandToRange = {

View File

@ -1404,17 +1404,6 @@ BlobWorkerInterface decodeBlobWorkerListValue(ValueRef const& value) {
return interf;
}
Value encodeTenantEntry(TenantMapEntry const& tenantEntry) {
return ObjectWriter::toValue(tenantEntry, IncludeVersion());
}
TenantMapEntry decodeTenantEntry(ValueRef const& value) {
TenantMapEntry entry;
ObjectReader reader(value.begin(), IncludeVersion());
reader.deserialize(entry);
return entry;
}
const KeyRangeRef tenantMapKeys("\xff/tenantMap/"_sr, "\xff/tenantMap0"_sr);
const KeyRef tenantMapPrefix = tenantMapKeys.begin;
const KeyRef tenantMapPrivatePrefix = "\xff\xff/tenantMap/"_sr;

View File

@ -54,12 +54,12 @@ TenantMapEntry::TenantMapEntry(int64_t id, KeyRef subspace) : id(id) {
TEST_CASE("/fdbclient/TenantMapEntry/Serialization") {
TenantMapEntry entry1(1, ""_sr);
ASSERT(entry1.prefix == "\x00\x00\x00\x00\x00\x00\x00\x01"_sr);
TenantMapEntry entry2 = decodeTenantEntry(encodeTenantEntry(entry1));
TenantMapEntry entry2 = TenantMapEntry::decode(entry1.encode());
ASSERT(entry1.id == entry2.id && entry1.prefix == entry2.prefix);
TenantMapEntry entry3(std::numeric_limits<int64_t>::max(), "foo"_sr);
ASSERT(entry3.prefix == "foo\x7f\xff\xff\xff\xff\xff\xff\xff"_sr);
TenantMapEntry entry4 = decodeTenantEntry(encodeTenantEntry(entry3));
TenantMapEntry entry4 = TenantMapEntry::decode(entry3.encode());
ASSERT(entry3.id == entry4.id && entry3.prefix == entry4.prefix);
for (int i = 0; i < 100; ++i) {
@ -78,7 +78,7 @@ TEST_CASE("/fdbclient/TenantMapEntry/Serialization") {
entry.prefix.endsWith(StringRef(reinterpret_cast<uint8_t*>(&bigEndianId), 8)) &&
entry.prefix.size() == subspaceLength + 8);
TenantMapEntry decodedEntry = decodeTenantEntry(encodeTenantEntry(entry));
TenantMapEntry decodedEntry = TenantMapEntry::decode(entry.encode());
ASSERT(decodedEntry.id == entry.id && decodedEntry.prefix == entry.prefix);
}

View File

@ -1,149 +0,0 @@
/*
* TenantSpecialKeys.actor.cpp
*
* This source file is part of the FoundationDB open source project
*
* Copyright 2013-2022 Apple Inc. and the FoundationDB project authors
*
* Licensed under the Apache License, Version 2.0 (the "License");
* you may not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
#include "fdbclient/ActorLineageProfiler.h"
#include "fdbclient/FDBOptions.g.h"
#include "fdbclient/Knobs.h"
#include "fdbclient/DatabaseContext.h"
#include "fdbclient/SpecialKeySpace.actor.h"
#include "fdbclient/TenantManagement.actor.h"
#include "flow/Arena.h"
#include "flow/UnitTest.h"
#include "flow/actorcompiler.h" // This must be the last #include.
ACTOR Future<RangeResult> getTenantList(ReadYourWritesTransaction* ryw, KeyRangeRef kr, GetRangeLimits limitsHint) {
state KeyRef managementPrefix =
kr.begin.substr(0,
SpecialKeySpace::getModuleRange(SpecialKeySpace::MODULE::MANAGEMENT).begin.size() +
TenantMapRangeImpl::submoduleRange.begin.size());
kr = kr.removePrefix(SpecialKeySpace::getModuleRange(SpecialKeySpace::MODULE::MANAGEMENT).begin);
TenantNameRef beginTenant = kr.begin.removePrefix(TenantMapRangeImpl::submoduleRange.begin);
TenantNameRef endTenant = kr.end;
if (endTenant.startsWith(TenantMapRangeImpl::submoduleRange.begin)) {
endTenant = endTenant.removePrefix(TenantMapRangeImpl::submoduleRange.begin);
} else {
endTenant = "\xff"_sr;
}
std::map<TenantName, TenantMapEntry> tenants =
wait(TenantAPI::listTenantsTransaction(&ryw->getTransaction(), beginTenant, endTenant, limitsHint.rows));
RangeResult results;
for (auto tenant : tenants) {
json_spirit::mObject tenantEntry;
tenantEntry["id"] = tenant.second.id;
tenantEntry["prefix"] = tenant.second.prefix.toString();
std::string tenantEntryString = json_spirit::write_string(json_spirit::mValue(tenantEntry));
ValueRef tenantEntryBytes(results.arena(), tenantEntryString);
results.push_back(results.arena(),
KeyValueRef(tenant.first.withPrefix(managementPrefix, results.arena()), tenantEntryBytes));
}
return results;
}
TenantMapRangeImpl::TenantMapRangeImpl(KeyRangeRef kr) : SpecialKeyRangeRWImpl(kr) {}
Future<RangeResult> TenantMapRangeImpl::getRange(ReadYourWritesTransaction* ryw,
KeyRangeRef kr,
GetRangeLimits limitsHint) const {
return getTenantList(ryw, kr, limitsHint);
}
ACTOR Future<Void> createTenants(ReadYourWritesTransaction* ryw, std::vector<TenantNameRef> tenants) {
Optional<Value> lastIdVal = wait(ryw->getTransaction().get(tenantLastIdKey));
int64_t previousId = lastIdVal.present() ? TenantMapEntry::prefixToId(lastIdVal.get()) : -1;
std::vector<Future<Void>> createFutures;
for (auto tenant : tenants) {
createFutures.push_back(
success(TenantAPI::createTenantTransaction(&ryw->getTransaction(), tenant, ++previousId)));
}
ryw->getTransaction().set(tenantLastIdKey, TenantMapEntry::idToPrefix(previousId));
wait(waitForAll(createFutures));
return Void();
}
ACTOR Future<Void> deleteTenantRange(ReadYourWritesTransaction* ryw,
TenantNameRef beginTenant,
TenantNameRef endTenant) {
std::map<TenantName, TenantMapEntry> tenants =
wait(TenantAPI::listTenantsTransaction(&ryw->getTransaction(), beginTenant, endTenant, CLIENT_KNOBS->TOO_MANY));
if (tenants.size() == CLIENT_KNOBS->TOO_MANY) {
TraceEvent(SevWarn, "DeleteTenantRangeTooLange")
.detail("BeginTenant", beginTenant)
.detail("EndTenant", endTenant);
ryw->setSpecialKeySpaceErrorMsg("too many tenants to range delete");
throw special_keys_api_failure();
}
std::vector<Future<Void>> deleteFutures;
for (auto tenant : tenants) {
deleteFutures.push_back(TenantAPI::deleteTenantTransaction(&ryw->getTransaction(), tenant.first));
}
wait(waitForAll(deleteFutures));
return Void();
}
Future<Optional<std::string>> TenantMapRangeImpl::commit(ReadYourWritesTransaction* ryw) {
auto ranges = ryw->getSpecialKeySpaceWriteMap().containedRanges(range);
std::vector<TenantNameRef> tenantsToCreate;
std::vector<Future<Void>> tenantManagementFutures;
for (auto range : ranges) {
if (!range.value().first) {
continue;
}
TenantNameRef tenantName =
range.begin()
.removePrefix(SpecialKeySpace::getModuleRange(SpecialKeySpace::MODULE::MANAGEMENT).begin)
.removePrefix(TenantMapRangeImpl::submoduleRange.begin);
if (range.value().second.present()) {
tenantsToCreate.push_back(tenantName);
} else {
// For a single key clear, just issue the delete
if (KeyRangeRef(range.begin(), range.end()).singleKeyRange()) {
tenantManagementFutures.push_back(
TenantAPI::deleteTenantTransaction(&ryw->getTransaction(), tenantName));
} else {
TenantNameRef endTenant = range.end().removePrefix(
SpecialKeySpace::getModuleRange(SpecialKeySpace::MODULE::MANAGEMENT).begin);
if (endTenant.startsWith(submoduleRange.begin)) {
endTenant = endTenant.removePrefix(submoduleRange.begin);
} else {
endTenant = "\xff"_sr;
}
tenantManagementFutures.push_back(deleteTenantRange(ryw, tenantName, endTenant));
}
}
}
if (tenantsToCreate.size()) {
tenantManagementFutures.push_back(createTenants(ryw, tenantsToCreate));
}
return tag(waitForAll(tenantManagementFutures), Optional<std::string>());
}

View File

@ -0,0 +1,33 @@
/*
* 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);

View File

@ -359,12 +359,14 @@ public:
};
template <>
inline Tuple Codec<FileBackupAgent::ERestoreState>::pack(FileBackupAgent::ERestoreState const& val) {
return Tuple().append(val);
inline Standalone<StringRef> TupleCodec<FileBackupAgent::ERestoreState>::pack(
FileBackupAgent::ERestoreState const& val) {
return Tuple().append(val).pack();
}
template <>
inline FileBackupAgent::ERestoreState Codec<FileBackupAgent::ERestoreState>::unpack(Tuple const& val) {
return (FileBackupAgent::ERestoreState)val.getInt(0);
inline FileBackupAgent::ERestoreState TupleCodec<FileBackupAgent::ERestoreState>::unpack(
Standalone<StringRef> const& val) {
return (FileBackupAgent::ERestoreState)Tuple::unpack(val).getInt(0);
}
class DatabaseBackupAgent : public BackupAgentBase {
@ -575,12 +577,12 @@ ACTOR Future<Void> cleanupBackup(Database cx, DeleteData deleteData);
using EBackupState = BackupAgentBase::EnumState;
template <>
inline Tuple Codec<EBackupState>::pack(EBackupState const& val) {
return Tuple().append(static_cast<int>(val));
inline Standalone<StringRef> TupleCodec<EBackupState>::pack(EBackupState const& val) {
return Tuple().append(static_cast<int>(val)).pack();
}
template <>
inline EBackupState Codec<EBackupState>::unpack(Tuple const& val) {
return static_cast<EBackupState>(val.getInt(0));
inline EBackupState TupleCodec<EBackupState>::unpack(Standalone<StringRef> const& val) {
return static_cast<EBackupState>(Tuple::unpack(val).getInt(0));
}
// Key backed tags are a single-key slice of the TagUidMap, defined below.
@ -678,7 +680,7 @@ public:
throw restore_error();
// Validation contition is that the uidPair key must be exactly {u, false}
TaskBucket::setValidationCondition(
task, KeyBackedTag(tag.get(), p).key, Codec<UidAndAbortedFlagT>::pack({ u, false }).pack());
task, KeyBackedTag(tag.get(), p).key, TupleCodec<UidAndAbortedFlagT>::pack({ u, false }));
return Void();
});
}
@ -724,7 +726,7 @@ protected:
};
template <>
inline Tuple Codec<Reference<IBackupContainer>>::pack(Reference<IBackupContainer> const& bc) {
inline Standalone<StringRef> TupleCodec<Reference<IBackupContainer>>::pack(Reference<IBackupContainer> const& bc) {
Tuple tuple;
tuple.append(StringRef(bc->getURL()));
@ -740,21 +742,23 @@ inline Tuple Codec<Reference<IBackupContainer>>::pack(Reference<IBackupContainer
tuple.append(StringRef());
}
return tuple;
return tuple.pack();
}
template <>
inline Reference<IBackupContainer> Codec<Reference<IBackupContainer>>::unpack(Tuple const& val) {
ASSERT(val.size() >= 1);
auto url = val.getString(0).toString();
inline Reference<IBackupContainer> TupleCodec<Reference<IBackupContainer>>::unpack(Standalone<StringRef> const& val) {
Tuple t = Tuple::unpack(val);
ASSERT(t.size() >= 1);
auto url = t.getString(0).toString();
Optional<std::string> encryptionKeyFileName;
if (val.size() > 1 && !val.getString(1).empty()) {
encryptionKeyFileName = val.getString(1).toString();
if (t.size() > 1 && !t.getString(1).empty()) {
encryptionKeyFileName = t.getString(1).toString();
}
Optional<std::string> proxy;
if (val.size() > 2 && !val.getString(2).empty()) {
proxy = val.getString(2).toString();
if (t.size() > 2 && !t.getString(2).empty()) {
proxy = t.getString(2).toString();
}
return IBackupContainer::openContainer(url, proxy, encryptionKeyFileName);

View File

@ -79,6 +79,7 @@ struct MutationRef {
CompareAndClear,
Reserved_For_SpanContextMessage /* See fdbserver/SpanContextMessage.h */,
Reserved_For_OTELSpanContextMessage,
Reserved_For_EncryptedMutationMessage /* See fdbserver/EncryptedMutationMessage.actor.h */,
MAX_ATOMIC_OP
};
// This is stored this way for serialization purposes.

View File

@ -291,10 +291,12 @@ public:
Future<Void> splitStorageMetricsStream(PromiseStream<Key> const& resultsStream,
KeyRange const& keys,
StorageMetrics const& limit,
StorageMetrics const& estimated);
StorageMetrics const& estimated,
Optional<int> const& minSplitBytes = {});
Future<Standalone<VectorRef<KeyRef>>> splitStorageMetrics(KeyRange const& keys,
StorageMetrics const& limit,
StorageMetrics const& estimated);
StorageMetrics const& estimated,
Optional<int> const& minSplitBytes = {});
Future<Standalone<VectorRef<ReadHotRangeWithMetrics>>> getReadHotRanges(KeyRange const& keys);
@ -595,6 +597,14 @@ public:
// Cache of the latest commit versions of storage servers.
VersionVector ssVersionVectorCache;
// Introduced mainly to optimize out the version vector related code (on the client side)
// when the version vector feature is disabled (on the server side).
// @param ssVersionVectorDelta version vector changes sent by GRV proxy
inline bool versionVectorCacheActive(const VersionVector& ssVersionVectorDelta) {
return (ssVersionVectorCache.getMaxVersion() != invalidVersion ||
ssVersionVectorDelta.getMaxVersion() != invalidVersion);
}
// Adds or updates the specified (SS, TSS) pair in the TSS mapping (if not already present).
// Requests to the storage server will be duplicated to the TSS.
void addTssMapping(StorageServerInterface const& ssi, StorageServerInterface const& tssi);

View File

@ -22,26 +22,15 @@
#define FDB_AWS_CREDENTIALS_PROVIDER_H
#pragma once
#ifdef BUILD_AWS_BACKUP
#include "aws/core/Aws.h"
#include "aws/core/auth/AWSCredentialsProviderChain.h"
// Singleton
namespace FDBAWSCredentialsProvider {
bool doneInit = false;
// You're supposed to call AWS::ShutdownAPI(options); once done
// But we want this to live for the lifetime of the process, so we don't do that
static Aws::Auth::AWSCredentials getAwsCredentials() {
if (!doneInit) {
doneInit = true;
Aws::SDKOptions options;
Aws::InitAPI(options);
TraceEvent("AWSSDKInitSuccessful");
}
Aws::Auth::DefaultAWSCredentialsProviderChain credProvider;
Aws::Auth::AWSCredentials creds = credProvider.GetAWSCredentials();
return creds;
Aws::Auth::AWSCredentials getAwsCredentials();
}
} // namespace FDBAWSCredentialsProvider
#endif
#endif

View File

@ -1509,4 +1509,16 @@ struct StorageWiggleValue {
serializer(ar, id);
}
};
// Can be used to identify types (e.g. IDatabase) that can be used to create transactions with a `createTransaction`
// function
template <typename, typename = void>
struct transaction_creator_traits : std::false_type {};
template <typename T>
struct transaction_creator_traits<T, std::void_t<typename T::TransactionT>> : std::true_type {};
template <typename T>
constexpr bool is_transaction_creator = transaction_creator_traits<T>::value;
#endif

View File

@ -23,115 +23,118 @@
#include <utility>
#include <vector>
#include "fdbclient/GenericTransactionHelper.h"
#include "fdbclient/IClientApi.h"
#include "fdbclient/ReadYourWrites.h"
#include "fdbclient/DatabaseContext.h"
#include "fdbclient/Subspace.h"
#include "flow/ObjectSerializer.h"
#include "flow/genericactors.actor.h"
#include "flow/serialize.h"
// Codec is a utility struct to convert a type to and from a Tuple. It is used by the template
// classes below like KeyBackedProperty and KeyBackedMap to convert key parts and values
// from various types to Value strings and back.
// TupleCodec is a utility struct to convert a type to and from a value using Tuple encoding.
// It is used by the template classes below like KeyBackedProperty and KeyBackedMap to convert
// key parts and values from various types to Value strings and back.
// New types can be supported either by writing a new specialization or adding these
// methods to the type so that the default specialization can be used:
// static T T::unpack(Tuple const &t)
// Tuple T::pack() const
// Since Codec is a struct, partial specialization can be used, such as the std::pair
// static T T::unpack(Standalone<StringRef> const& val)
// Standalone<StringRef> T::pack(T const& val) const
// Since TupleCodec is a struct, partial specialization can be used, such as the std::pair
// partial specialization below allowing any std::pair<T1,T2> where T1 and T2 are already
// supported by Codec.
// supported by TupleCodec.
template <typename T>
struct Codec {
static inline Tuple pack(T const& val) { return val.pack(); }
static inline T unpack(Tuple const& t) { return T::unpack(t); }
struct TupleCodec {
static inline Standalone<StringRef> pack(T const& val) { return val.pack().pack(); }
static inline T unpack(Standalone<StringRef> const& val) { return T::unpack(Tuple::unpack(val)); }
};
// If T is Tuple then conversion is simple.
template <>
inline Tuple Codec<Tuple>::pack(Tuple const& val) {
return val;
inline Standalone<StringRef> TupleCodec<Tuple>::pack(Tuple const& val) {
return val.pack();
}
template <>
inline Tuple Codec<Tuple>::unpack(Tuple const& val) {
return val;
inline Tuple TupleCodec<Tuple>::unpack(Standalone<StringRef> const& val) {
return Tuple::unpack(val);
}
template <>
inline Tuple Codec<int64_t>::pack(int64_t const& val) {
return Tuple().append(val);
inline Standalone<StringRef> TupleCodec<int64_t>::pack(int64_t const& val) {
return Tuple().append(val).pack();
}
template <>
inline int64_t Codec<int64_t>::unpack(Tuple const& val) {
return val.getInt(0);
inline int64_t TupleCodec<int64_t>::unpack(Standalone<StringRef> const& val) {
return Tuple::unpack(val).getInt(0);
}
template <>
inline Tuple Codec<bool>::pack(bool const& val) {
return Tuple().append(val ? 1 : 0);
inline Standalone<StringRef> TupleCodec<bool>::pack(bool const& val) {
return Tuple().append(val ? 1 : 0).pack();
}
template <>
inline bool Codec<bool>::unpack(Tuple const& val) {
return val.getInt(0) == 1;
inline bool TupleCodec<bool>::unpack(Standalone<StringRef> const& val) {
return Tuple::unpack(val).getInt(0) == 1;
}
template <>
inline Tuple Codec<Standalone<StringRef>>::pack(Standalone<StringRef> const& val) {
return Tuple().append(val);
inline Standalone<StringRef> TupleCodec<Standalone<StringRef>>::pack(Standalone<StringRef> const& val) {
return Tuple().append(val).pack();
}
template <>
inline Standalone<StringRef> Codec<Standalone<StringRef>>::unpack(Tuple const& val) {
return val.getString(0);
inline Standalone<StringRef> TupleCodec<Standalone<StringRef>>::unpack(Standalone<StringRef> const& val) {
return Tuple::unpack(val).getString(0);
}
template <>
inline Tuple Codec<UID>::pack(UID const& val) {
return Codec<Standalone<StringRef>>::pack(BinaryWriter::toValue<UID>(val, Unversioned()));
inline Standalone<StringRef> TupleCodec<UID>::pack(UID const& val) {
return TupleCodec<Standalone<StringRef>>::pack(BinaryWriter::toValue<UID>(val, Unversioned()));
}
template <>
inline UID Codec<UID>::unpack(Tuple const& val) {
return BinaryReader::fromStringRef<UID>(Codec<Standalone<StringRef>>::unpack(val), Unversioned());
inline UID TupleCodec<UID>::unpack(Standalone<StringRef> const& val) {
return BinaryReader::fromStringRef<UID>(TupleCodec<Standalone<StringRef>>::unpack(val), Unversioned());
}
// This is backward compatible with Codec<Standalone<StringRef>>
// This is backward compatible with TupleCodec<Standalone<StringRef>>
template <>
inline Tuple Codec<std::string>::pack(std::string const& val) {
return Tuple().append(StringRef(val));
inline Standalone<StringRef> TupleCodec<std::string>::pack(std::string const& val) {
return Tuple().append(StringRef(val)).pack();
}
template <>
inline std::string Codec<std::string>::unpack(Tuple const& val) {
return val.getString(0).toString();
inline std::string TupleCodec<std::string>::unpack(Standalone<StringRef> const& val) {
return Tuple::unpack(val).getString(0).toString();
}
// Partial specialization to cover all std::pairs as long as the component types are Codec compatible
// Partial specialization to cover all std::pairs as long as the component types are TupleCodec compatible
template <typename First, typename Second>
struct Codec<std::pair<First, Second>> {
static Tuple pack(typename std::pair<First, Second> const& val) {
return Tuple().append(Codec<First>::pack(val.first)).append(Codec<Second>::pack(val.second));
struct TupleCodec<std::pair<First, Second>> {
static Standalone<StringRef> pack(typename std::pair<First, Second> const& val) {
// Packing a concatenated tuple is the same as concatenating two packed tuples
return TupleCodec<First>::pack(val.first).withSuffix(TupleCodec<Second>::pack(val.second));
}
static std::pair<First, Second> unpack(Tuple const& t) {
static std::pair<First, Second> unpack(Standalone<StringRef> const& val) {
Tuple t = Tuple::unpack(val);
ASSERT(t.size() == 2);
return { Codec<First>::unpack(t.subTuple(0, 1)), Codec<Second>::unpack(t.subTuple(1, 2)) };
return { TupleCodec<First>::unpack(t.subTupleRawString(0)),
TupleCodec<Second>::unpack(t.subTupleRawString(1)) };
}
};
template <typename T>
struct Codec<std::vector<T>> {
static Tuple pack(typename std::vector<T> const& val) {
struct TupleCodec<std::vector<T>> {
static Standalone<StringRef> pack(typename std::vector<T> const& val) {
Tuple t;
for (T item : val) {
Tuple itemTuple = Codec<T>::pack(item);
// fdbclient doesn't support nested tuples yet. For now, flatten the tuple into StringRef
t.append(itemTuple.pack());
t.append(TupleCodec<T>::pack(item));
}
return t;
return t.pack();
}
static std::vector<T> unpack(Tuple const& t) {
static std::vector<T> unpack(Standalone<StringRef> const& val) {
Tuple t = Tuple::unpack(val);
std::vector<T> v;
for (int i = 0; i < t.size(); i++) {
Tuple itemTuple = Tuple::unpack(t.getString(i));
v.push_back(Codec<T>::unpack(itemTuple));
v.push_back(TupleCodec<T>::unpack(t.getString(i)));
}
return v;
@ -139,37 +142,47 @@ struct Codec<std::vector<T>> {
};
template <>
inline Tuple Codec<KeyRange>::pack(KeyRange const& val) {
return Tuple().append(val.begin).append(val.end);
inline Standalone<StringRef> TupleCodec<KeyRange>::pack(KeyRange const& val) {
return Tuple().append(val.begin).append(val.end).pack();
}
template <>
inline KeyRange Codec<KeyRange>::unpack(Tuple const& val) {
return KeyRangeRef(val.getString(0), val.getString(1));
inline KeyRange TupleCodec<KeyRange>::unpack(Standalone<StringRef> const& val) {
Tuple t = Tuple::unpack(val);
return KeyRangeRef(t.getString(0), t.getString(1));
}
// Convenient read/write access to a single value of type T stored at key
// Even though 'this' is not actually mutated, methods that change the db key are not const.
template <typename T>
template <typename T, typename Codec = TupleCodec<T>>
class KeyBackedProperty {
public:
KeyBackedProperty(KeyRef key) : key(key) {}
Future<Optional<T>> get(Reference<ReadYourWritesTransaction> tr, Snapshot snapshot = Snapshot::False) const {
return map(tr->get(key, snapshot), [](Optional<Value> const& val) -> Optional<T> {
if (val.present())
return Codec<T>::unpack(Tuple::unpack(val.get()));
return {};
});
template <class Transaction>
typename std::enable_if<!is_transaction_creator<Transaction>, Future<Optional<T>>>::type get(
Transaction tr,
Snapshot snapshot = Snapshot::False) const {
typename transaction_future_type<Transaction, Optional<Value>>::type getFuture = tr->get(key, snapshot);
return holdWhile(getFuture,
map(safeThreadFutureToFuture(getFuture), [](Optional<Value> const& val) -> Optional<T> {
if (val.present())
return Codec::unpack(val.get());
return {};
}));
}
// Get property's value or defaultValue if it doesn't exist
Future<T> getD(Reference<ReadYourWritesTransaction> tr,
Snapshot snapshot = Snapshot::False,
T defaultValue = T()) const {
template <class Transaction>
typename std::enable_if<!is_transaction_creator<Transaction>, Future<T>>::type
getD(Transaction tr, Snapshot snapshot = Snapshot::False, T defaultValue = T()) const {
return map(get(tr, snapshot), [=](Optional<T> val) -> T { return val.present() ? val.get() : defaultValue; });
}
// Get property's value or throw error if it doesn't exist
Future<T> getOrThrow(Reference<ReadYourWritesTransaction> tr,
Snapshot snapshot = Snapshot::False,
Error err = key_not_found()) const {
template <class Transaction>
typename std::enable_if<!is_transaction_creator<Transaction>, Future<T>>::type
getOrThrow(Transaction tr, Snapshot snapshot = Snapshot::False, Error err = key_not_found()) const {
return map(get(tr, snapshot), [=](Optional<T> val) -> T {
if (!val.present()) {
throw err;
@ -179,8 +192,11 @@ public:
});
}
Future<Optional<T>> get(Database cx, Snapshot snapshot = Snapshot::False) const {
return runRYWTransaction(cx, [=, self = *this](Reference<ReadYourWritesTransaction> tr) {
template <class DB>
typename std::enable_if<is_transaction_creator<DB>, Future<Optional<T>>>::type get(
Reference<DB> db,
Snapshot snapshot = Snapshot::False) const {
return runTransaction(db, [=, self = *this](Reference<typename DB::TransactionT> tr) {
tr->setOption(FDBTransactionOptions::ACCESS_SYSTEM_KEYS);
tr->setOption(FDBTransactionOptions::LOCK_AWARE);
@ -188,8 +204,11 @@ public:
});
}
Future<T> getD(Database cx, Snapshot snapshot = Snapshot::False, T defaultValue = T()) const {
return runRYWTransaction(cx, [=, self = *this](Reference<ReadYourWritesTransaction> tr) {
template <class DB>
typename std::enable_if<is_transaction_creator<DB>, Future<T>>::type getD(Reference<DB> db,
Snapshot snapshot = Snapshot::False,
T defaultValue = T()) const {
return runTransaction(db, [=, self = *this](Reference<typename DB::TransactionT> tr) {
tr->setOption(FDBTransactionOptions::ACCESS_SYSTEM_KEYS);
tr->setOption(FDBTransactionOptions::LOCK_AWARE);
@ -197,8 +216,11 @@ public:
});
}
Future<T> getOrThrow(Database cx, Snapshot snapshot = Snapshot::False, Error err = key_not_found()) const {
return runRYWTransaction(cx, [=, self = *this](Reference<ReadYourWritesTransaction> tr) {
template <class DB>
typename std::enable_if<is_transaction_creator<DB>, Future<T>>::type getOrThrow(Reference<DB> db,
Snapshot snapshot = Snapshot::False,
Error err = key_not_found()) const {
return runTransaction(db, [=, self = *this](Reference<typename DB::TransactionT> tr) {
tr->setOption(FDBTransactionOptions::ACCESS_SYSTEM_KEYS);
tr->setOption(FDBTransactionOptions::LOCK_AWARE);
@ -206,10 +228,14 @@ public:
});
}
void set(Reference<ReadYourWritesTransaction> tr, T const& val) { return tr->set(key, Codec<T>::pack(val).pack()); }
template <class Transaction>
typename std::enable_if<!is_transaction_creator<Transaction>, void>::type set(Transaction tr, T const& val) {
return tr->set(key, Codec::pack(val));
}
Future<Void> set(Database cx, T const& val) {
return runRYWTransaction(cx, [=, self = *this](Reference<ReadYourWritesTransaction> tr) {
template <class DB>
typename std::enable_if<is_transaction_creator<DB>, Future<Void>>::type set(Reference<DB> db, T const& val) {
return runTransaction(db, [=, self = *this](Reference<typename DB::TransactionT> tr) {
tr->setOption(FDBTransactionOptions::ACCESS_SYSTEM_KEYS);
tr->setOption(FDBTransactionOptions::LOCK_AWARE);
self->set(tr, val);
@ -217,47 +243,67 @@ public:
});
}
void clear(Reference<ReadYourWritesTransaction> tr) { return tr->clear(key); }
template <class Transaction>
typename std::enable_if<!is_transaction_creator<Transaction>, void>::type clear(Transaction tr) {
return tr->clear(key);
}
Key key;
};
// This is just like KeyBackedProperty but instead of using Codec for conversion to/from values it
// This is just like KeyBackedProperty but instead of using a Codec for conversion to/from values it
// uses BinaryReader and BinaryWriter. This enables allows atomic ops with integer types, and also
// allows reading and writing of existing keys which use BinaryReader/Writer.
template <typename T>
class KeyBackedBinaryValue {
public:
KeyBackedBinaryValue(KeyRef key) : key(key) {}
Future<Optional<T>> get(Reference<ReadYourWritesTransaction> tr, Snapshot snapshot = Snapshot::False) const {
return map(tr->get(key, snapshot), [](Optional<Value> const& val) -> Optional<T> {
if (val.present())
return BinaryReader::fromStringRef<T>(val.get(), Unversioned());
return {};
});
template <class Transaction>
Future<Optional<T>> get(Transaction tr, Snapshot snapshot = Snapshot::False) const {
typename transaction_future_type<Transaction, Optional<Value>>::type getFuture = tr->get(key, snapshot);
return holdWhile(getFuture,
map(safeThreadFutureToFuture(getFuture), [](Optional<Value> const& val) -> Optional<T> {
if (val.present())
return BinaryReader::fromStringRef<T>(val.get(), Unversioned());
return {};
}));
}
// Get property's value or defaultValue if it doesn't exist
Future<T> getD(Reference<ReadYourWritesTransaction> tr,
Snapshot snapshot = Snapshot::False,
T defaultValue = T()) const {
template <class Transaction>
Future<T> getD(Transaction tr, Snapshot snapshot = Snapshot::False, T defaultValue = T()) const {
return map(get(tr, Snapshot::False),
[=](Optional<T> val) -> T { return val.present() ? val.get() : defaultValue; });
}
void set(Reference<ReadYourWritesTransaction> tr, T const& val) {
template <class Transaction>
void set(Transaction tr, T const& val) {
return tr->set(key, BinaryWriter::toValue<T>(val, Unversioned()));
}
void atomicOp(Reference<ReadYourWritesTransaction> tr, T const& val, MutationRef::Type type) {
template <class Transaction>
void atomicOp(Transaction tr, T const& val, MutationRef::Type type) {
return tr->atomicOp(key, BinaryWriter::toValue<T>(val, Unversioned()), type);
}
void clear(Reference<ReadYourWritesTransaction> tr) { return tr->clear(key); }
template <class Transaction>
void clear(Transaction tr) {
return tr->clear(key);
}
Key key;
};
// Convenient read/write access to a sorted map of KeyType to ValueType under prefix
// Even though 'this' is not actually mutated, methods that change db keys are not const.
template <typename _KeyType, typename _ValueType>
template <typename _KeyType,
typename _ValueType,
typename KeyCodec = TupleCodec<_KeyType>,
typename ValueCodec = TupleCodec<_ValueType>>
class KeyBackedMap {
public:
KeyBackedMap(KeyRef prefix) : space(prefix) {}
KeyBackedMap(KeyRef prefix) : subspace(prefixRange(prefix)) {}
typedef _KeyType KeyType;
typedef _ValueType ValueType;
@ -265,65 +311,77 @@ public:
typedef std::vector<PairType> PairsType;
// If end is not present one key past the end of the map is used.
Future<PairsType> getRange(Reference<ReadYourWritesTransaction> tr,
KeyType const& begin,
template <class Transaction>
Future<PairsType> getRange(Transaction tr,
Optional<KeyType> const& begin,
Optional<KeyType> const& end,
int limit,
Snapshot snapshot = Snapshot::False,
Reverse reverse = Reverse::False) const {
Subspace s = space; // 'this' could be invalid inside lambda
Key endKey = end.present() ? s.pack(Codec<KeyType>::pack(end.get())) : space.range().end;
return map(
tr->getRange(
KeyRangeRef(s.pack(Codec<KeyType>::pack(begin)), endKey), GetRangeLimits(limit), snapshot, reverse),
[s](RangeResult const& kvs) -> PairsType {
PairsType results;
for (int i = 0; i < kvs.size(); ++i) {
KeyType key = Codec<KeyType>::unpack(s.unpack(kvs[i].key));
ValueType val = Codec<ValueType>::unpack(Tuple::unpack(kvs[i].value));
results.push_back(PairType(key, val));
}
return results;
});
Key prefix = subspace.begin; // 'this' could be invalid inside lambda
Key beginKey = begin.present() ? prefix.withSuffix(KeyCodec::pack(begin.get())) : subspace.begin;
Key endKey = end.present() ? prefix.withSuffix(KeyCodec::pack(end.get())) : subspace.end;
typename transaction_future_type<Transaction, RangeResult>::type getRangeFuture =
tr->getRange(KeyRangeRef(beginKey, endKey), GetRangeLimits(limit), snapshot, reverse);
return holdWhile(getRangeFuture,
map(safeThreadFutureToFuture(getRangeFuture), [prefix](RangeResult const& kvs) -> PairsType {
PairsType results;
for (int i = 0; i < kvs.size(); ++i) {
KeyType key = KeyCodec::unpack(kvs[i].key.removePrefix(prefix));
ValueType val = ValueCodec::unpack(kvs[i].value);
results.push_back(PairType(key, val));
}
return results;
}));
}
Future<Optional<ValueType>> get(Reference<ReadYourWritesTransaction> tr,
KeyType const& key,
Snapshot snapshot = Snapshot::False) const {
return map(tr->get(space.pack(Codec<KeyType>::pack(key)), snapshot),
[](Optional<Value> const& val) -> Optional<ValueType> {
if (val.present())
return Codec<ValueType>::unpack(Tuple::unpack(val.get()));
return {};
});
template <class Transaction>
Future<Optional<ValueType>> get(Transaction tr, KeyType const& key, Snapshot snapshot = Snapshot::False) const {
typename transaction_future_type<Transaction, Optional<Value>>::type getFuture =
tr->get(subspace.begin.withSuffix(KeyCodec::pack(key)), snapshot);
return holdWhile(
getFuture, map(safeThreadFutureToFuture(getFuture), [](Optional<Value> const& val) -> Optional<ValueType> {
if (val.present())
return ValueCodec::unpack(val.get());
return {};
}));
}
// Returns a Property that can be get/set that represents key's entry in this this.
KeyBackedProperty<ValueType> getProperty(KeyType const& key) const { return space.pack(Codec<KeyType>::pack(key)); }
KeyBackedProperty<ValueType> getProperty(KeyType const& key) const {
return subspace.begin.withSuffix(KeyCodec::pack(key));
}
// Returns the expectedSize of the set key
int set(Reference<ReadYourWritesTransaction> tr, KeyType const& key, ValueType const& val) {
Key k = space.pack(Codec<KeyType>::pack(key));
Value v = Codec<ValueType>::pack(val).pack();
template <class Transaction>
int set(Transaction tr, KeyType const& key, ValueType const& val) {
Key k = subspace.begin.withSuffix(KeyCodec::pack(key));
Value v = ValueCodec::pack(val);
tr->set(k, v);
return k.expectedSize() + v.expectedSize();
}
void erase(Reference<ReadYourWritesTransaction> tr, KeyType const& key) {
return tr->clear(space.pack(Codec<KeyType>::pack(key)));
template <class Transaction>
void erase(Transaction tr, KeyType const& key) {
return tr->clear(subspace.begin.withSuffix(KeyCodec::pack(key)));
}
void erase(Reference<ITransaction> tr, KeyType const& key) {
return tr->clear(space.pack(Codec<KeyType>::pack(key)));
template <class Transaction>
void erase(Transaction tr, KeyType const& begin, KeyType const& end) {
return tr->clear(KeyRangeRef(subspace.begin.withSuffix(KeyCodec::pack(begin)),
subspace.begin.withSuffix(KeyCodec::pack(end))));
}
void erase(Reference<ReadYourWritesTransaction> tr, KeyType const& begin, KeyType const& end) {
return tr->clear(KeyRangeRef(space.pack(Codec<KeyType>::pack(begin)), space.pack(Codec<KeyType>::pack(end))));
template <class Transaction>
void clear(Transaction tr) {
return tr->clear(subspace);
}
void clear(Reference<ReadYourWritesTransaction> tr) { return tr->clear(space.range()); }
Subspace space;
KeyRange subspace;
};
// Convenient read/write access to a single value of type T stored at key
@ -332,25 +390,32 @@ template <typename T, typename VersionOptions>
class KeyBackedObjectProperty {
public:
KeyBackedObjectProperty(KeyRef key, VersionOptions versionOptions) : key(key), versionOptions(versionOptions) {}
Future<Optional<T>> get(Reference<ReadYourWritesTransaction> tr, Snapshot snapshot = Snapshot::False) const {
return map(tr->get(key, snapshot), [vo = versionOptions](Optional<Value> const& val) -> Optional<T> {
if (val.present())
return ObjectReader::fromStringRef<T>(val.get(), vo);
return {};
});
template <class Transaction>
typename std::enable_if<!is_transaction_creator<Transaction>, Future<Optional<T>>>::type get(
Transaction tr,
Snapshot snapshot = Snapshot::False) const {
typename transaction_future_type<Transaction, Optional<Value>>::type getFuture = tr->get(key, snapshot);
return holdWhile(
getFuture,
map(safeThreadFutureToFuture(getFuture), [vo = versionOptions](Optional<Value> const& val) -> Optional<T> {
if (val.present())
return ObjectReader::fromStringRef<T>(val.get(), vo);
return {};
}));
}
// Get property's value or defaultValue if it doesn't exist
Future<T> getD(Reference<ReadYourWritesTransaction> tr,
Snapshot snapshot = Snapshot::False,
T defaultValue = T()) const {
template <class Transaction>
typename std::enable_if<!is_transaction_creator<Transaction>, Future<T>>::type
getD(Transaction tr, Snapshot snapshot = Snapshot::False, T defaultValue = T()) const {
return map(get(tr, snapshot), [=](Optional<T> val) -> T { return val.present() ? val.get() : defaultValue; });
}
// Get property's value or throw error if it doesn't exist
Future<T> getOrThrow(Reference<ReadYourWritesTransaction> tr,
Snapshot snapshot = Snapshot::False,
Error err = key_not_found()) const {
template <class Transaction>
typename std::enable_if<!is_transaction_creator<Transaction>, Future<T>>::type
getOrThrow(Transaction tr, Snapshot snapshot = Snapshot::False, Error err = key_not_found()) const {
return map(get(tr, snapshot), [=](Optional<T> val) -> T {
if (!val.present()) {
throw err;
@ -360,8 +425,11 @@ public:
});
}
Future<Optional<T>> get(Database cx, Snapshot snapshot = Snapshot::False) const {
return runRYWTransaction(cx, [=, self = *this](Reference<ReadYourWritesTransaction> tr) {
template <class DB>
typename std::enable_if<is_transaction_creator<DB>, Future<Optional<T>>>::type get(
Reference<DB> db,
Snapshot snapshot = Snapshot::False) const {
return runTransaction(db, [=, self = *this](Reference<typename DB::TransactionT> tr) {
tr->setOption(FDBTransactionOptions::ACCESS_SYSTEM_KEYS);
tr->setOption(FDBTransactionOptions::LOCK_AWARE);
@ -369,8 +437,11 @@ public:
});
}
Future<T> getD(Database cx, Snapshot snapshot = Snapshot::False, T defaultValue = T()) const {
return runRYWTransaction(cx, [=, self = *this](Reference<ReadYourWritesTransaction> tr) {
template <class DB>
typename std::enable_if<is_transaction_creator<DB>, Future<T>>::type getD(Reference<DB> db,
Snapshot snapshot = Snapshot::False,
T defaultValue = T()) const {
return runTransaction(db, [=, self = *this](Reference<typename DB::TransactionT> tr) {
tr->setOption(FDBTransactionOptions::ACCESS_SYSTEM_KEYS);
tr->setOption(FDBTransactionOptions::LOCK_AWARE);
@ -378,8 +449,11 @@ public:
});
}
Future<T> getOrThrow(Database cx, Snapshot snapshot = Snapshot::False, Error err = key_not_found()) const {
return runRYWTransaction(cx, [=, self = *this](Reference<ReadYourWritesTransaction> tr) {
template <class DB>
typename std::enable_if<is_transaction_creator<DB>, Future<T>>::type getOrThrow(Reference<DB> db,
Snapshot snapshot = Snapshot::False,
Error err = key_not_found()) const {
return runTransaction(db, [=, self = *this](Reference<typename DB::TransactionT> tr) {
tr->setOption(FDBTransactionOptions::ACCESS_SYSTEM_KEYS);
tr->setOption(FDBTransactionOptions::LOCK_AWARE);
@ -387,12 +461,14 @@ public:
});
}
void set(Reference<ReadYourWritesTransaction> tr, T const& val) {
template <class Transaction>
typename std::enable_if<!is_transaction_creator<Transaction>, void>::type set(Transaction tr, T const& val) {
return tr->set(key, ObjectWriter::toValue(val, versionOptions));
}
Future<Void> set(Database cx, T const& val) {
return runRYWTransaction(cx, [=, self = *this](Reference<ReadYourWritesTransaction> tr) {
template <class DB>
typename std::enable_if<is_transaction_creator<DB>, Future<Void>>::type set(Reference<DB> db, T const& val) {
return runTransaction(db, [=, self = *this](Reference<typename DB::TransactionT> tr) {
tr->setOption(FDBTransactionOptions::ACCESS_SYSTEM_KEYS);
tr->setOption(FDBTransactionOptions::LOCK_AWARE);
self.set(tr, val);
@ -400,7 +476,10 @@ public:
});
}
void clear(Reference<ReadYourWritesTransaction> tr) { return tr->clear(key); }
template <class Transaction>
typename std::enable_if<!is_transaction_creator<Transaction>, void>::type clear(Transaction tr) {
return tr->clear(key);
}
Key key;
VersionOptions versionOptions;
@ -409,136 +488,161 @@ public:
// Convenient read/write access to a sorted map of KeyType to ValueType under key prefix
// ValueType is encoded / decoded with ObjectWriter/ObjectReader
// Even though 'this' is not actually mutated, methods that change db keys are not const.
template <typename _KeyType, typename _ValueType, typename VersionOptions>
template <typename _KeyType, typename _ValueType, typename VersionOptions, typename KeyCodec = TupleCodec<_KeyType>>
class KeyBackedObjectMap {
public:
KeyBackedObjectMap(KeyRef prefix, VersionOptions versionOptions) : space(prefix), versionOptions(versionOptions) {}
KeyBackedObjectMap(KeyRef prefix, VersionOptions versionOptions)
: subspace(prefixRange(prefix)), versionOptions(versionOptions) {}
typedef _KeyType KeyType;
typedef _ValueType ValueType;
typedef std::pair<KeyType, ValueType> PairType;
typedef std::vector<PairType> PairsType;
// If end is not present one key past the end of the map is used.
Future<PairsType> getRange(Reference<ReadYourWritesTransaction> tr,
KeyType const& begin,
template <class Transaction>
Future<PairsType> getRange(Transaction tr,
Optional<KeyType> const& begin,
Optional<KeyType> const& end,
int limit,
Snapshot snapshot = Snapshot::False,
Reverse reverse = Reverse::False) const {
Key endKey = end.present() ? space.pack(Codec<KeyType>::pack(end.get())) : space.range().end;
return map(
tr->getRange(
KeyRangeRef(space.pack(Codec<KeyType>::pack(begin)), endKey), GetRangeLimits(limit), snapshot, reverse),
[self = *this](RangeResult const& kvs) -> PairsType {
Key beginKey = begin.present() ? subspace.begin.withSuffix(KeyCodec::pack(begin.get())) : subspace.begin;
Key endKey = end.present() ? subspace.begin.withSuffix(KeyCodec::pack(end.get())) : subspace.end;
typename transaction_future_type<Transaction, RangeResult>::type getRangeFuture =
tr->getRange(KeyRangeRef(beginKey, endKey), GetRangeLimits(limit), snapshot, reverse);
return holdWhile(
getRangeFuture,
map(safeThreadFutureToFuture(getRangeFuture), [self = *this](RangeResult const& kvs) -> PairsType {
PairsType results;
for (int i = 0; i < kvs.size(); ++i) {
KeyType key = Codec<KeyType>::unpack(self.space.unpack(kvs[i].key));
KeyType key = KeyCodec::unpack(kvs[i].key.removePrefix(self.subspace.begin));
ValueType val = ObjectReader::fromStringRef<ValueType>(kvs[i].value, self.versionOptions);
results.push_back(PairType(key, val));
}
return results;
});
}));
}
Future<Optional<ValueType>> get(Reference<ReadYourWritesTransaction> tr,
KeyType const& key,
Snapshot snapshot = Snapshot::False) const {
return map(tr->get(space.pack(Codec<KeyType>::pack(key)), snapshot),
[vo = versionOptions](Optional<Value> const& val) -> Optional<ValueType> {
if (val.present())
return ObjectReader::fromStringRef<ValueType>(val.get(), vo);
return {};
});
template <class Transaction>
Future<Optional<ValueType>> get(Transaction tr, KeyType const& key, Snapshot snapshot = Snapshot::False) const {
typename transaction_future_type<Transaction, Optional<Value>>::type getFuture =
tr->get(subspace.begin.withSuffix(KeyCodec::pack(key)), snapshot);
return holdWhile(getFuture,
map(safeThreadFutureToFuture(getFuture),
[vo = versionOptions](Optional<Value> const& val) -> Optional<ValueType> {
if (val.present())
return ObjectReader::fromStringRef<ValueType>(val.get(), vo);
return {};
}));
}
// Returns a Property that can be get/set that represents key's entry in this this.
KeyBackedObjectProperty<ValueType, VersionOptions> getProperty(KeyType const& key) const {
return KeyBackedObjectProperty<ValueType, VersionOptions>(space.pack(Codec<KeyType>::pack(key)),
return KeyBackedObjectProperty<ValueType, VersionOptions>(subspace.begin.withSuffix(KeyCodec::pack(key)),
versionOptions);
}
// Returns the expectedSize of the set key
int set(Reference<ReadYourWritesTransaction> tr, KeyType const& key, ValueType const& val) {
Key k = space.pack(Codec<KeyType>::pack(key));
template <class Transaction>
int set(Transaction tr, KeyType const& key, ValueType const& val) {
Key k = subspace.begin.withSuffix(KeyCodec::pack(key));
Value v = ObjectWriter::toValue(val, versionOptions);
tr->set(k, v);
return k.expectedSize() + v.expectedSize();
}
Key serializeKey(KeyType const& key) { return space.pack(Codec<KeyType>::pack(key)); }
Key serializeKey(KeyType const& key) { return subspace.begin.withSuffix(KeyCodec::pack(key)); }
Value serializeValue(ValueType const& val) { return ObjectWriter::toValue(val, versionOptions); }
void erase(Reference<ReadYourWritesTransaction> tr, KeyType const& key) {
return tr->clear(space.pack(Codec<KeyType>::pack(key)));
template <class Transaction>
void erase(Transaction tr, KeyType const& key) {
return tr->clear(subspace.begin.withSuffix(KeyCodec::pack(key)));
}
void erase(Reference<ITransaction> tr, KeyType const& key) {
return tr->clear(space.pack(Codec<KeyType>::pack(key)));
template <class Transaction>
void erase(Transaction tr, KeyType const& begin, KeyType const& end) {
return tr->clear(KeyRangeRef(subspace.begin.withSuffix(KeyCodec::pack(begin)),
subspace.begin.withSuffix(KeyCodec::pack(end))));
}
void erase(Reference<ReadYourWritesTransaction> tr, KeyType const& begin, KeyType const& end) {
return tr->clear(KeyRangeRef(space.pack(Codec<KeyType>::pack(begin)), space.pack(Codec<KeyType>::pack(end))));
template <class Transaction>
void clear(Transaction tr) {
return tr->clear(subspace);
}
void clear(Reference<ReadYourWritesTransaction> tr) { return tr->clear(space.range()); }
Subspace space;
KeyRange subspace;
VersionOptions versionOptions;
};
template <typename _ValueType>
template <typename _ValueType, typename Codec = TupleCodec<_ValueType>>
class KeyBackedSet {
public:
KeyBackedSet(KeyRef key) : space(key) {}
KeyBackedSet(KeyRef key) : subspace(prefixRange(key)) {}
typedef _ValueType ValueType;
typedef std::vector<ValueType> Values;
// If end is not present one key past the end of the map is used.
Future<Values> getRange(Reference<ReadYourWritesTransaction> tr,
ValueType const& begin,
template <class Transaction>
Future<Values> getRange(Transaction tr,
Optional<ValueType> const& begin,
Optional<ValueType> const& end,
int limit,
Snapshot snapshot = Snapshot::False) const {
Subspace s = space; // 'this' could be invalid inside lambda
Key endKey = end.present() ? s.pack(Codec<ValueType>::pack(end.get())) : space.range().end;
return map(
tr->getRange(KeyRangeRef(s.pack(Codec<ValueType>::pack(begin)), endKey), GetRangeLimits(limit), snapshot),
[s](RangeResult const& kvs) -> Values {
Values results;
for (int i = 0; i < kvs.size(); ++i) {
results.push_back(Codec<ValueType>::unpack(s.unpack(kvs[i].key)));
}
return results;
});
Snapshot snapshot = Snapshot::False,
Reverse reverse = Reverse::False) const {
Key prefix = subspace.begin; // 'this' could be invalid inside lambda
Key beginKey = begin.present() ? prefix.withSuffix(Codec::pack(begin.get())) : subspace.begin;
Key endKey = end.present() ? prefix.withSuffix(Codec::pack(end.get())) : subspace.end;
typename transaction_future_type<Transaction, RangeResult>::type getRangeFuture =
tr->getRange(KeyRangeRef(beginKey, endKey), GetRangeLimits(limit), snapshot, reverse);
return holdWhile(getRangeFuture,
map(safeThreadFutureToFuture(getRangeFuture), [prefix](RangeResult const& kvs) -> Values {
Values results;
for (int i = 0; i < kvs.size(); ++i) {
results.push_back(Codec::unpack(kvs[i].key.removePrefix(prefix)));
}
return results;
}));
}
Future<bool> exists(Reference<ReadYourWritesTransaction> tr,
ValueType const& val,
Snapshot snapshot = Snapshot::False) const {
return map(tr->get(space.pack(Codec<ValueType>::pack(val)), snapshot),
[](Optional<Value> const& val) -> bool { return val.present(); });
template <class Transaction>
Future<bool> exists(Transaction tr, ValueType const& val, Snapshot snapshot = Snapshot::False) const {
typename transaction_future_type<Transaction, Optional<Value>>::type getFuture =
tr->get(subspace.begin.withSuffix(Codec::pack(val)), snapshot);
return holdWhile(getFuture, map(safeThreadFutureToFuture(getFuture), [](Optional<Value> const& val) -> bool {
return val.present();
}));
}
// Returns the expectedSize of the set key
int insert(Reference<ReadYourWritesTransaction> tr, ValueType const& val) {
Key k = space.pack(Codec<ValueType>::pack(val));
template <class Transaction>
int insert(Transaction tr, ValueType const& val) {
Key k = subspace.begin.withSuffix(Codec::pack(val));
tr->set(k, StringRef());
return k.expectedSize();
}
void erase(Reference<ReadYourWritesTransaction> tr, ValueType const& val) {
return tr->clear(space.pack(Codec<ValueType>::pack(val)));
template <class Transaction>
void erase(Transaction tr, ValueType const& val) {
return tr->clear(subspace.begin.withSuffix(Codec::pack(val)));
}
void erase(Reference<ReadYourWritesTransaction> tr, ValueType const& begin, ValueType const& end) {
template <class Transaction>
void erase(Transaction tr, ValueType const& begin, ValueType const& end) {
return tr->clear(
KeyRangeRef(space.pack(Codec<ValueType>::pack(begin)), space.pack(Codec<ValueType>::pack(end))));
KeyRangeRef(subspace.begin.withSuffix(Codec::pack(begin)), subspace.begin.withSuffix(Codec::pack(end))));
}
void clear(Reference<ReadYourWritesTransaction> tr) { return tr->clear(space.range()); }
template <class Transaction>
void clear(Transaction tr) {
return tr->clear(subspace);
}
Subspace space;
KeyRange subspace;
};

View File

@ -41,7 +41,7 @@ Future<decltype(std::declval<Function>()(Reference<ReadYourWritesTransaction>())
state Reference<ReadYourWritesTransaction> tr(new ReadYourWritesTransaction(cx));
loop {
try {
// func should be idempodent; otherwise, retry will get undefined result
// func should be idempotent; otherwise, retry will get undefined result
state decltype(std::declval<Function>()(Reference<ReadYourWritesTransaction>()).getValue()) result =
wait(func(tr));
wait(tr->commit());
@ -52,6 +52,24 @@ Future<decltype(std::declval<Function>()(Reference<ReadYourWritesTransaction>())
}
}
ACTOR template <class Function, class DB>
Future<decltype(std::declval<Function>()(Reference<typename DB::TransactionT>()).getValue())> runTransaction(
Reference<DB> db,
Function func) {
state Reference<typename DB::TransactionT> tr = db->createTransaction();
loop {
try {
// func should be idempotent; otherwise, retry will get undefined result
state decltype(std::declval<Function>()(Reference<typename DB::TransactionT>()).getValue()) result =
wait(func(tr));
wait(safeThreadFutureToFuture(tr->commit()));
return result;
} catch (Error& e) {
wait(safeThreadFutureToFuture(tr->onError(e)));
}
}
}
ACTOR template <class Function>
Future<decltype(std::declval<Function>()(Reference<ReadYourWritesTransaction>()).getValue())>
runRYWTransactionFailIfLocked(Database cx, Function func) {

View File

@ -689,6 +689,7 @@ public:
bool ENABLE_CLEAR_RANGE_EAGER_READS;
bool QUICK_GET_VALUE_FALLBACK;
bool QUICK_GET_KEY_VALUES_FALLBACK;
int MAX_PARALLEL_QUICK_GET_VALUE;
int CHECKPOINT_TRANSFER_BLOCK_BYTES;
int QUICK_GET_KEY_VALUES_LIMIT;
int QUICK_GET_KEY_VALUES_LIMIT_BYTES;

View File

@ -538,16 +538,5 @@ public:
Future<Optional<std::string>> commit(ReadYourWritesTransaction* ryw) override;
};
class TenantMapRangeImpl : public SpecialKeyRangeRWImpl {
public:
const static KeyRangeRef submoduleRange;
explicit TenantMapRangeImpl(KeyRangeRef kr);
Future<RangeResult> getRange(ReadYourWritesTransaction* ryw,
KeyRangeRef kr,
GetRangeLimits limitsHint) const override;
Future<Optional<std::string>> commit(ReadYourWritesTransaction* ryw) override;
};
#include "flow/unactorcompiler.h"
#endif

View File

@ -716,18 +716,21 @@ struct SplitMetricsRequest {
StorageMetrics estimated;
bool isLastShard;
ReplyPromise<SplitMetricsReply> reply;
Optional<int> minSplitBytes;
SplitMetricsRequest() {}
SplitMetricsRequest(KeyRangeRef const& keys,
StorageMetrics const& limits,
StorageMetrics const& used,
StorageMetrics const& estimated,
bool isLastShard)
: keys(arena, keys), limits(limits), used(used), estimated(estimated), isLastShard(isLastShard) {}
bool isLastShard,
Optional<int> minSplitBytes)
: keys(arena, keys), limits(limits), used(used), estimated(estimated), isLastShard(isLastShard),
minSplitBytes(minSplitBytes) {}
template <class Ar>
void serialize(Ar& ar) {
serializer(ar, keys, limits, used, estimated, isLastShard, reply, arena);
serializer(ar, keys, limits, used, estimated, isLastShard, reply, arena, minSplitBytes);
}
};

View File

@ -631,9 +631,6 @@ extern const KeyRef tenantMapPrivatePrefix;
extern const KeyRef tenantLastIdKey;
extern const KeyRef tenantDataPrefixKey;
Value encodeTenantEntry(TenantMapEntry const& tenantEntry);
TenantMapEntry decodeTenantEntry(ValueRef const& value);
#pragma clang diagnostic pop
#endif

View File

@ -103,8 +103,8 @@ template <typename T>
class TaskParam {
public:
TaskParam(StringRef key) : key(key) {}
T get(Reference<Task> task) const { return Codec<T>::unpack(Tuple::unpack(task->params[key])); }
void set(Reference<Task> task, T const& val) const { task->params[key] = Codec<T>::pack(val).pack(); }
T get(Reference<Task> task) const { return TupleCodec<T>::unpack(task->params[key]); }
void set(Reference<Task> task, T const& val) const { task->params[key] = TupleCodec<T>::pack(val); }
bool exists(Reference<Task> task) const { return task->params.find(key) != task->params.end(); }
T getOrDefault(Reference<Task> task, const T defaultValue = T()) const {
if (!exists(task))

View File

@ -47,6 +47,15 @@ public:
TenantMapEntry();
TenantMapEntry(int64_t id, KeyRef subspace);
Value encode() const { return ObjectWriter::toValue(*this, IncludeVersion(ProtocolVersion::withTenants())); }
static TenantMapEntry decode(ValueRef const& value) {
TenantMapEntry entry;
ObjectReader reader(value.begin(), IncludeVersion(ProtocolVersion::withTenants()));
reader.deserialize(entry);
return entry;
}
template <class Ar>
void serialize(Ar& ar) {
KeyRef subspace;

View File

@ -40,7 +40,7 @@ Future<Optional<TenantMapEntry>> tryGetTenantTransaction(Transaction tr, TenantN
state typename transaction_future_type<Transaction, Optional<Value>>::type tenantFuture = tr->get(tenantMapKey);
Optional<Value> val = wait(safeThreadFutureToFuture(tenantFuture));
return val.map<TenantMapEntry>([](Optional<Value> v) { return decodeTenantEntry(v.get()); });
return val.map<TenantMapEntry>([](Optional<Value> v) { return TenantMapEntry::decode(v.get()); });
}
ACTOR template <class DB>
@ -130,7 +130,7 @@ Future<std::pair<TenantMapEntry, bool>> createTenantTransaction(Transaction tr,
throw tenant_prefix_allocator_conflict();
}
tr->set(tenantMapKey, encodeTenantEntry(newTenant));
tr->set(tenantMapKey, newTenant.encode());
return std::make_pair(newTenant, true);
}
@ -260,7 +260,7 @@ Future<std::map<TenantName, TenantMapEntry>> listTenantsTransaction(Transaction
std::map<TenantName, TenantMapEntry> tenants;
for (auto kv : results) {
tenants[kv.key.removePrefix(tenantMapPrefix)] = decodeTenantEntry(kv.value);
tenants[kv.key.removePrefix(tenantMapPrefix)] = TenantMapEntry::decode(kv.value);
}
return tenants;

View File

@ -0,0 +1,209 @@
/*
* TenantSpecialKeys.actor.h
*
* This source file is part of the FoundationDB open source project
*
* Copyright 2013-2022 Apple Inc. and the FoundationDB project authors
*
* Licensed under the Apache License, Version 2.0 (the "License");
* you may not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
#pragma once
#if defined(NO_INTELLISENSE) && !defined(FDBCLIENT_TENANT_SPECIAL_KEYS_ACTOR_G_H)
#define FDBCLIENT_TENANT_SPECIAL_KEYS_ACTOR_G_H
#include "fdbclient/TenantSpecialKeys.actor.g.h"
#elif !defined(FDBCLIENT_TENANT_SPECIAL_KEYS_ACTOR_H)
#define FDBCLIENT_TENANT_SPECIAL_KEYS_ACTOR_H
#include "fdbclient/ActorLineageProfiler.h"
#include "fdbclient/FDBOptions.g.h"
#include "fdbclient/Knobs.h"
#include "fdbclient/DatabaseContext.h"
#include "fdbclient/SpecialKeySpace.actor.h"
#include "fdbclient/TenantManagement.actor.h"
#include "flow/Arena.h"
#include "flow/UnitTest.h"
#include "flow/actorcompiler.h" // This must be the last #include.
template <bool HasSubRanges = true>
class TenantRangeImpl : public SpecialKeyRangeRWImpl {
private:
static KeyRangeRef removePrefix(KeyRangeRef range, KeyRef prefix, KeyRef defaultEnd) {
KeyRef begin = range.begin.removePrefix(prefix);
KeyRef end;
if (range.end.startsWith(prefix)) {
end = range.end.removePrefix(prefix);
} else {
end = defaultEnd;
}
return KeyRangeRef(begin, end);
}
static KeyRef withTenantMapPrefix(KeyRef key, Arena& ar) {
int keySize = SpecialKeySpace::getModuleRange(SpecialKeySpace::MODULE::MANAGEMENT).begin.size() +
TenantRangeImpl::submoduleRange.begin.size() + TenantRangeImpl::mapSubRange.begin.size() +
key.size();
KeyRef prefixedKey = makeString(keySize, ar);
uint8_t* mutableKey = mutateString(prefixedKey);
mutableKey = SpecialKeySpace::getModuleRange(SpecialKeySpace::MODULE::MANAGEMENT).begin.copyTo(mutableKey);
mutableKey = TenantRangeImpl::submoduleRange.begin.copyTo(mutableKey);
mutableKey = TenantRangeImpl::mapSubRange.begin.copyTo(mutableKey);
key.copyTo(mutableKey);
return prefixedKey;
}
ACTOR static Future<Void> getTenantList(ReadYourWritesTransaction* ryw,
KeyRangeRef kr,
RangeResult* results,
GetRangeLimits limitsHint) {
std::map<TenantName, TenantMapEntry> tenants =
wait(TenantAPI::listTenantsTransaction(&ryw->getTransaction(), kr.begin, kr.end, limitsHint.rows));
for (auto tenant : tenants) {
json_spirit::mObject tenantEntry;
tenantEntry["id"] = tenant.second.id;
tenantEntry["prefix"] = tenant.second.prefix.toString();
std::string tenantEntryString = json_spirit::write_string(json_spirit::mValue(tenantEntry));
ValueRef tenantEntryBytes(results->arena(), tenantEntryString);
results->push_back(results->arena(),
KeyValueRef(withTenantMapPrefix(tenant.first, results->arena()), tenantEntryBytes));
}
return Void();
}
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::submoduleRange.begin);
if (kr.intersects(TenantRangeImpl::mapSubRange)) {
GetRangeLimits limits = limitsHint;
limits.decrement(results);
wait(getTenantList(
ryw,
removePrefix(kr & TenantRangeImpl::mapSubRange, TenantRangeImpl::mapSubRange.begin, "\xff"_sr),
&results,
limits));
}
return results;
}
ACTOR static Future<Void> createTenants(ReadYourWritesTransaction* ryw, std::vector<TenantNameRef> tenants) {
Optional<Value> lastIdVal = wait(ryw->getTransaction().get(tenantLastIdKey));
int64_t previousId = lastIdVal.present() ? TenantMapEntry::prefixToId(lastIdVal.get()) : -1;
std::vector<Future<Void>> createFutures;
for (auto tenant : tenants) {
createFutures.push_back(
success(TenantAPI::createTenantTransaction(&ryw->getTransaction(), tenant, ++previousId)));
}
ryw->getTransaction().set(tenantLastIdKey, TenantMapEntry::idToPrefix(previousId));
wait(waitForAll(createFutures));
return Void();
}
ACTOR static Future<Void> deleteTenantRange(ReadYourWritesTransaction* ryw,
TenantName beginTenant,
TenantName endTenant) {
state std::map<TenantName, TenantMapEntry> tenants = wait(
TenantAPI::listTenantsTransaction(&ryw->getTransaction(), beginTenant, endTenant, CLIENT_KNOBS->TOO_MANY));
if (tenants.size() == CLIENT_KNOBS->TOO_MANY) {
TraceEvent(SevWarn, "DeleteTenantRangeTooLange")
.detail("BeginTenant", beginTenant)
.detail("EndTenant", endTenant);
ryw->setSpecialKeySpaceErrorMsg(
ManagementAPIError::toJsonString(false, "delete tenants", "too many tenants to range delete"));
throw special_keys_api_failure();
}
std::vector<Future<Void>> deleteFutures;
for (auto tenant : tenants) {
deleteFutures.push_back(TenantAPI::deleteTenantTransaction(&ryw->getTransaction(), tenant.first));
}
wait(waitForAll(deleteFutures));
return Void();
}
public:
const static KeyRangeRef submoduleRange;
const static KeyRangeRef mapSubRange;
explicit TenantRangeImpl(KeyRangeRef kr) : SpecialKeyRangeRWImpl(kr) {}
Future<RangeResult> getRange(ReadYourWritesTransaction* ryw,
KeyRangeRef kr,
GetRangeLimits limitsHint) const override {
return getTenantRange(ryw, kr, limitsHint);
}
Future<Optional<std::string>> commit(ReadYourWritesTransaction* ryw) override {
auto ranges = ryw->getSpecialKeySpaceWriteMap().containedRanges(range);
std::vector<Future<Void>> tenantManagementFutures;
std::vector<std::pair<KeyRangeRef, Optional<Value>>> mapMutations;
for (auto range : ranges) {
if (!range.value().first) {
continue;
}
KeyRangeRef adjustedRange =
range.range()
.removePrefix(SpecialKeySpace::getModuleRange(SpecialKeySpace::MODULE::MANAGEMENT).begin)
.removePrefix(submoduleRange.begin);
if (mapSubRange.intersects(adjustedRange)) {
adjustedRange = mapSubRange & adjustedRange;
adjustedRange = removePrefix(adjustedRange, mapSubRange.begin, "\xff"_sr);
mapMutations.push_back(std::make_pair(adjustedRange, range.value().second));
}
}
std::vector<TenantNameRef> tenantsToCreate;
for (auto mapMutation : mapMutations) {
TenantNameRef tenantName = mapMutation.first.begin;
if (mapMutation.second.present()) {
tenantsToCreate.push_back(tenantName);
} else {
// For a single key clear, just issue the delete
if (mapMutation.first.singleKeyRange()) {
tenantManagementFutures.push_back(
TenantAPI::deleteTenantTransaction(&ryw->getTransaction(), tenantName));
} else {
tenantManagementFutures.push_back(deleteTenantRange(ryw, tenantName, mapMutation.first.end));
}
}
}
if (!tenantsToCreate.empty()) {
tenantManagementFutures.push_back(createTenants(ryw, tenantsToCreate));
}
return tag(waitForAll(tenantManagementFutures), Optional<std::string>());
}
};
#include "flow/unactorcompiler.h"
#endif

View File

@ -32,6 +32,7 @@
static const int InvalidEncodedSize = 0;
struct VersionVector {
constexpr static FileIdentifier file_identifier = 5253554;
friend struct serializable_traits<VersionVector>;
boost::container::flat_map<Tag, Version> versions; // An ordered map. (Note:
// changing this to an unordered

View File

@ -108,15 +108,17 @@ int decodedLength(int codeLength) noexcept {
return (codeLength / 4) * 3 + (r - 1);
}
std::pair<StringRef, bool> decode(Arena& arena, StringRef base64UrlStr) {
Optional<StringRef> decode(Arena& arena, StringRef base64UrlStr) {
auto decodedLen = decodedLength(base64UrlStr.size());
if (decodedLen <= 0) {
return { StringRef(), decodedLen == 0 };
if (decodedLen == 0)
return StringRef{};
return {};
}
auto out = new (arena) uint8_t[decodedLen];
auto actualLen = decode(base64UrlStr.begin(), base64UrlStr.size(), out);
ASSERT_EQ(decodedLen, actualLen);
return { StringRef(out, decodedLen), true };
return StringRef(out, decodedLen);
}
} // namespace base64url
@ -237,13 +239,13 @@ TEST_CASE("/fdbrpc/Base64UrlEncode") {
encodeOutput.toHexString());
ASSERT(false);
}
auto [decodeOutput, decodeOk] = base64url::decode(tmpArena, encodeOutputExpected);
ASSERT(decodeOk);
if (decodeOutput != decodeOutputExpected) {
auto decodeOutput = base64url::decode(tmpArena, encodeOutputExpected);
ASSERT(decodeOutput.present());
if (decodeOutput.get() != decodeOutputExpected) {
fmt::print("Fixed case {} (decode): expected '{}' got '{}'\n",
i + 1,
decodeOutputExpected.toHexString(),
decodeOutput.toHexString());
decodeOutput.get().toHexString());
ASSERT(false);
}
}
@ -259,12 +261,12 @@ TEST_CASE("/fdbrpc/Base64UrlEncode") {
// make sure output only contains legal characters
for (auto i = 0; i < output.size(); i++)
ASSERT_NE(base64url::decodeValue(output[i]), base64url::_X);
auto [decodedOutput, decodeOk] = base64url::decode(tmpArena, output);
ASSERT(decodeOk);
if (input != decodedOutput) {
auto decodeOutput = base64url::decode(tmpArena, output);
ASSERT(decodeOutput.present());
if (input != decodeOutput.get()) {
fmt::print("Dynamic case {} (decode) failed, expected '{}', got '{}'\n",
input.toHexString(),
decodedOutput.toHexString());
decodeOutput.get().toHexString());
ASSERT(false);
}
}

840
fdbrpc/JsonWebKeySet.cpp Normal file
View File

@ -0,0 +1,840 @@
/*
* JsonWebKeySet.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 "flow/Arena.h"
#include "flow/AutoCPointer.h"
#include "flow/Error.h"
#include "flow/IRandom.h"
#include "flow/MkCert.h"
#include "flow/PKey.h"
#include "flow/UnitTest.h"
#include "fdbrpc/Base64UrlEncode.h"
#include "fdbrpc/Base64UrlDecode.h"
#include "fdbrpc/JsonWebKeySet.h"
#if defined(HAVE_WOLFSSL)
#include <wolfssl/options.h>
#endif
#include <openssl/bn.h>
#include <openssl/ec.h>
#include <openssl/err.h>
#include <openssl/evp.h>
#include <openssl/opensslv.h>
#include <openssl/rsa.h>
#include <openssl/x509.h>
#if OPENSSL_VERSION_NUMBER >= 0x30000000L && !defined(_WIN32)
#define USE_V3_API 1
#else
#define USE_V3_API 0
#endif
#if USE_V3_API
#include <openssl/core_names.h>
#include <openssl/param_build.h>
#endif
#include <rapidjson/document.h>
#include <rapidjson/error/en.h>
#include <rapidjson/writer.h>
#include <rapidjson/stringbuffer.h>
#include <array>
#include <string_view>
#include <type_traits>
#define JWKS_ERROR(issue, op) \
TraceEvent(SevWarnAlways, "JsonWebKeySet" #op "Error").suppressFor(10).detail("Issue", issue)
#define JWKS_PARSE_ERROR(issue) JWKS_ERROR(issue, Parse)
#define JWKS_WRITE_ERROR(issue) JWKS_ERROR(issue, Write)
#define JWK_PARSE_ERROR(issue) \
TraceEvent(SevWarnAlways, "JsonWebKeyParseError") \
.suppressFor(10) \
.detail("Issue", issue) \
.detail("KeyIndexBase0", keyIndex)
#define JWK_WRITE_ERROR(issue) \
TraceEvent(SevWarnAlways, "JsonWebKeyWriteError") \
.suppressFor(10) \
.detail("Issue", issue) \
.detail("KeyName", keyName.toString())
#define JWK_ERROR_OSSL(issue, op) \
do { \
char buf[256]{ \
0, \
}; \
if (auto err = ::ERR_get_error()) { \
::ERR_error_string_n(err, buf, sizeof(buf)); \
} \
JWK_##op##_ERROR(issue).detail("OpenSSLError", static_cast<char const*>(buf)); \
} while (0)
#define JWK_PARSE_ERROR_OSSL(issue) JWK_ERROR_OSSL(issue, PARSE)
#define JWK_WRITE_ERROR_OSSL(issue) JWK_ERROR_OSSL(issue, WRITE)
namespace {
template <bool Required, class JsonValue>
bool getJwkStringMember(JsonValue const& value,
char const* memberName,
std::conditional_t<Required, StringRef, Optional<StringRef>>& out,
int keyIndex) {
auto itr = value.FindMember(memberName);
if (itr == value.MemberEnd()) {
if constexpr (Required) {
JWK_PARSE_ERROR("Missing required member").detail("Member", memberName);
return false;
} else {
return true;
}
}
auto const& member = itr->value;
if (!member.IsString()) {
JWK_PARSE_ERROR("Expected member is not a string").detail("MemberName", memberName);
return false;
}
out = StringRef(reinterpret_cast<uint8_t const*>(member.GetString()), member.GetStringLength());
return true;
}
#define DECLARE_JWK_REQUIRED_STRING_MEMBER(value, member) \
auto member = StringRef(); \
if (!getJwkStringMember<true>(value, #member, member, keyIndex)) \
return {}
#define DECLARE_JWK_OPTIONAL_STRING_MEMBER(value, member) \
auto member = Optional<StringRef>(); \
if (!getJwkStringMember<false>(value, #member, member, keyIndex)) \
return {}
template <bool Required, class AutoPtr>
bool getJwkBigNumMember(Arena& arena,
std::conditional_t<Required, StringRef, Optional<StringRef>> const& b64Member,
AutoPtr& ptr,
char const* memberName,
char const* algorithm,
int keyIndex) {
if constexpr (!Required) {
if (!b64Member.present())
return true;
}
auto data = StringRef();
if constexpr (Required) {
data = b64Member;
} else {
data = b64Member.get();
}
auto decoded = base64url::decode(arena, data);
if (!decoded.present()) {
JWK_PARSE_ERROR("Base64URL decoding for parameter failed")
.detail("Algorithm", algorithm)
.detail("Parameter", memberName);
return false;
}
data = decoded.get();
auto bn = ::BN_bin2bn(data.begin(), data.size(), nullptr);
if (!bn) {
JWK_PARSE_ERROR_OSSL("BN_bin2bn");
return false;
}
ptr.reset(bn);
return true;
}
#define DECL_DECODED_BN_MEMBER_REQUIRED(member, algo) \
auto member = AutoCPointer(nullptr, &::BN_free); \
if (!getJwkBigNumMember<true /*Required*/>(arena, b64##member, member, #member, algo, keyIndex)) \
return {}
#define DECL_DECODED_BN_MEMBER_OPTIONAL(member, algo) \
auto member = AutoCPointer(nullptr, &::BN_clear_free); \
if (!getJwkBigNumMember<false /*Required*/>(arena, b64##member, member, #member, algo, keyIndex)) \
return {}
#define EC_DECLARE_DECODED_REQUIRED_BN_MEMBER(member) DECL_DECODED_BN_MEMBER_REQUIRED(member, "EC")
#define EC_DECLARE_DECODED_OPTIONAL_BN_MEMBER(member) DECL_DECODED_BN_MEMBER_OPTIONAL(member, "EC")
#define RSA_DECLARE_DECODED_REQUIRED_BN_MEMBER(member) DECL_DECODED_BN_MEMBER_REQUIRED(member, "RSA")
#define RSA_DECLARE_DECODED_OPTIONAL_BN_MEMBER(member) DECL_DECODED_BN_MEMBER_OPTIONAL(member, "RSA")
StringRef bigNumToBase64Url(Arena& arena, const BIGNUM* bn) {
auto len = BN_num_bytes(bn);
auto buf = new (arena) uint8_t[len];
::BN_bn2bin(bn, buf);
return base64url::encode(arena, StringRef(buf, len));
}
Optional<PublicOrPrivateKey> parseEcP256Key(StringRef b64x, StringRef b64y, Optional<StringRef> b64d, int keyIndex) {
auto arena = Arena();
EC_DECLARE_DECODED_REQUIRED_BN_MEMBER(x);
EC_DECLARE_DECODED_REQUIRED_BN_MEMBER(y);
EC_DECLARE_DECODED_OPTIONAL_BN_MEMBER(d);
#if USE_V3_API
// avoid deprecated API
auto bld = AutoCPointer(::OSSL_PARAM_BLD_new(), &::OSSL_PARAM_BLD_free);
if (!bld) {
JWK_PARSE_ERROR_OSSL("OSSL_PARAM_BLD_new() for EC");
return {};
}
// since OSSL_PKEY_PARAM_EC_PUB_{X|Y} are not settable params, we'll need to build a EC_GROUP and serialize it
auto group = AutoCPointer(::EC_GROUP_new_by_curve_name(NID_X9_62_prime256v1), &::EC_GROUP_free);
if (!group) {
JWK_PARSE_ERROR_OSSL("EC_GROUP_new_by_curve_name()");
return {};
}
auto point = AutoCPointer(::EC_POINT_new(group), &::EC_POINT_free);
if (!point) {
JWK_PARSE_ERROR_OSSL("EC_POINT_new()");
return {};
}
if (1 != ::EC_POINT_set_affine_coordinates(group, point, x, y, nullptr)) {
JWK_PARSE_ERROR_OSSL("EC_POINT_set_affine_coordinates()");
return {};
}
auto pointBufLen = ::EC_POINT_point2oct(group, point, POINT_CONVERSION_UNCOMPRESSED, nullptr, 0, nullptr);
if (!pointBufLen) {
JWK_PARSE_ERROR_OSSL("EC_POINT_point2oct() for length");
return {};
}
auto pointBuf = new (arena) uint8_t[pointBufLen];
::EC_POINT_point2oct(group, point, POINT_CONVERSION_UNCOMPRESSED, pointBuf, pointBufLen, nullptr);
if (!::OSSL_PARAM_BLD_push_utf8_string(bld, OSSL_PKEY_PARAM_GROUP_NAME, "prime256v1", sizeof("prime256v1") - 1) ||
!::OSSL_PARAM_BLD_push_octet_string(bld, OSSL_PKEY_PARAM_PUB_KEY, pointBuf, pointBufLen)) {
JWK_PARSE_ERROR_OSSL("OSSL_PARAM_BLD_push_*() for EC (group, point)");
return {};
}
if (d && !::OSSL_PARAM_BLD_push_BN(bld, OSSL_PKEY_PARAM_PRIV_KEY, d)) {
JWK_PARSE_ERROR_OSSL("OSSL_PARAM_BLD_push_BN() for EC (d)");
return {};
}
auto params = AutoCPointer(::OSSL_PARAM_BLD_to_param(bld), &OSSL_PARAM_free);
if (!params) {
JWK_PARSE_ERROR_OSSL("OSSL_PARAM_BLD_to_param() for EC");
return {};
}
auto pctx = AutoCPointer(::EVP_PKEY_CTX_new_from_name(nullptr, "EC", nullptr), &::EVP_PKEY_CTX_free);
if (!pctx) {
JWK_PARSE_ERROR_OSSL("EVP_PKEY_CTX_new_from_name(EC)");
return {};
}
if (1 != ::EVP_PKEY_fromdata_init(pctx)) {
JWK_PARSE_ERROR_OSSL("EVP_PKEY_fromdata_init() for EC");
return {};
}
auto pkey = std::add_pointer_t<EVP_PKEY>();
if (1 != ::EVP_PKEY_fromdata(pctx, &pkey, (d ? EVP_PKEY_KEYPAIR : EVP_PKEY_PUBLIC_KEY), params) || !pkey) {
JWK_PARSE_ERROR_OSSL("EVP_PKEY_fromdata() for EC");
return {};
}
auto pkeyAutoPtr = AutoCPointer(pkey, &::EVP_PKEY_free);
#else // USE_V3_API
auto key = AutoCPointer(::EC_KEY_new_by_curve_name(NID_X9_62_prime256v1), &::EC_KEY_free);
if (!key) {
JWK_PARSE_ERROR_OSSL("EC_KEY_new()");
return {};
}
if (d) {
if (1 != ::EC_KEY_set_private_key(key, d)) {
JWK_PARSE_ERROR_OSSL("EC_KEY_set_private_key()");
return {};
}
}
if (1 != ::EC_KEY_set_public_key_affine_coordinates(key, x, y)) {
JWK_PARSE_ERROR_OSSL("EC_KEY_set_public_key_affine_coordinates(key, x, y)");
return {};
}
auto pkey = AutoCPointer(::EVP_PKEY_new(), &::EVP_PKEY_free);
if (!pkey) {
JWK_PARSE_ERROR_OSSL("EVP_PKEY_new() for EC");
return {};
}
if (1 != EVP_PKEY_set1_EC_KEY(pkey, key)) {
JWK_PARSE_ERROR_OSSL("EVP_PKEY_set1_EC_KEY()");
return {};
}
#endif // USE_V3_API
if (d) {
auto len = ::i2d_PrivateKey(pkey, nullptr);
if (len <= 0) {
JWK_PARSE_ERROR_OSSL("i2d_PrivateKey() for EC");
return {};
}
auto buf = new (arena) uint8_t[len];
auto out = std::add_pointer_t<uint8_t>(buf);
len = ::i2d_PrivateKey(pkey, &out);
// assign through public API, even if it means some parsing overhead
return PrivateKey(DerEncoded{}, StringRef(buf, len));
} else {
auto len = ::i2d_PUBKEY(pkey, nullptr);
if (len <= 0) {
JWK_PARSE_ERROR_OSSL("i2d_PUBKEY() for EC");
return {};
}
auto buf = new (arena) uint8_t[len];
auto out = std::add_pointer_t<uint8_t>(buf);
len = ::i2d_PUBKEY(pkey, &out);
// assign through public API, even if it means some parsing overhead
return PublicKey(DerEncoded{}, StringRef(buf, len));
}
}
Optional<PublicOrPrivateKey> parseRsaKey(StringRef b64n,
StringRef b64e,
Optional<StringRef> b64d,
Optional<StringRef> b64p,
Optional<StringRef> b64q,
Optional<StringRef> b64dp,
Optional<StringRef> b64dq,
Optional<StringRef> b64qi,
int keyIndex) {
auto arena = Arena();
RSA_DECLARE_DECODED_REQUIRED_BN_MEMBER(n);
RSA_DECLARE_DECODED_REQUIRED_BN_MEMBER(e);
RSA_DECLARE_DECODED_OPTIONAL_BN_MEMBER(d);
RSA_DECLARE_DECODED_OPTIONAL_BN_MEMBER(p);
RSA_DECLARE_DECODED_OPTIONAL_BN_MEMBER(q);
RSA_DECLARE_DECODED_OPTIONAL_BN_MEMBER(dp);
RSA_DECLARE_DECODED_OPTIONAL_BN_MEMBER(dq);
RSA_DECLARE_DECODED_OPTIONAL_BN_MEMBER(qi);
auto const isPublic = !d || !p || !q || !dp || !dq || !qi;
#if USE_V3_API
// avoid deprecated, algo-specific API
auto bld = AutoCPointer(::OSSL_PARAM_BLD_new(), &::OSSL_PARAM_BLD_free);
if (!bld) {
JWK_PARSE_ERROR_OSSL("OSSL_PARAM_BLD_new() for EC");
return {};
}
if (!::OSSL_PARAM_BLD_push_BN(bld, OSSL_PKEY_PARAM_RSA_N, n) ||
!::OSSL_PARAM_BLD_push_BN(bld, OSSL_PKEY_PARAM_RSA_E, e)) {
JWK_PARSE_ERROR_OSSL("OSSL_PARAM_BLD_push_BN() for RSA (n, e)");
return {};
}
if (!isPublic) {
if (!::OSSL_PARAM_BLD_push_BN(bld, OSSL_PKEY_PARAM_RSA_D, d) ||
!::OSSL_PARAM_BLD_push_BN(bld, OSSL_PKEY_PARAM_RSA_FACTOR1, p) ||
!::OSSL_PARAM_BLD_push_BN(bld, OSSL_PKEY_PARAM_RSA_FACTOR2, q) ||
!::OSSL_PARAM_BLD_push_BN(bld, OSSL_PKEY_PARAM_RSA_EXPONENT1, dp) ||
!::OSSL_PARAM_BLD_push_BN(bld, OSSL_PKEY_PARAM_RSA_EXPONENT2, dq) ||
!::OSSL_PARAM_BLD_push_BN(bld, OSSL_PKEY_PARAM_RSA_COEFFICIENT1, qi)) {
JWK_PARSE_ERROR_OSSL("OSSL_PARAM_BLD_push_BN() for RSA (d, p, q, dp, dq, qi)");
return {};
}
}
auto params = AutoCPointer(::OSSL_PARAM_BLD_to_param(bld), &::OSSL_PARAM_free);
if (!params) {
JWK_PARSE_ERROR_OSSL("OSSL_PARAM_BLD_to_param() for RSA");
return {};
}
auto pctx = AutoCPointer(::EVP_PKEY_CTX_new_from_name(nullptr, "RSA", nullptr), &::EVP_PKEY_CTX_free);
if (!pctx) {
JWK_PARSE_ERROR_OSSL("EVP_PKEY_CTX_new_from_name(RSA)");
return {};
}
if (1 != ::EVP_PKEY_fromdata_init(pctx)) {
JWK_PARSE_ERROR_OSSL("EVP_PKEY_fromdata_init() for RSA");
return {};
}
auto pkey = std::add_pointer_t<EVP_PKEY>();
if (1 != ::EVP_PKEY_fromdata(pctx, &pkey, (!isPublic ? EVP_PKEY_KEYPAIR : EVP_PKEY_PUBLIC_KEY), params)) {
JWK_PARSE_ERROR_OSSL("EVP_PKEY_fromdata() for EC");
return {};
}
auto pkeyAutoPtr = AutoCPointer(pkey, &::EVP_PKEY_free);
#else // USE_V3_API
auto rsa = AutoCPointer(RSA_new(), &::RSA_free);
if (!rsa) {
JWK_PARSE_ERROR_OSSL("RSA_new()");
return {};
}
if (1 != ::RSA_set0_key(rsa, n, e, d)) {
JWK_PARSE_ERROR_OSSL("RSA_set0_key()");
return {};
}
// set0 == ownership taken by rsa, no need to free
n.release();
e.release();
d.release();
if (!isPublic) {
if (1 != ::RSA_set0_factors(rsa, p, q)) {
JWK_PARSE_ERROR_OSSL("RSA_set0_factors()");
return {};
}
p.release();
q.release();
if (1 != ::RSA_set0_crt_params(rsa, dp, dq, qi)) {
JWK_PARSE_ERROR_OSSL("RSA_set0_crt_params()");
return {};
}
dp.release();
dq.release();
qi.release();
}
auto pkey = AutoCPointer(::EVP_PKEY_new(), &::EVP_PKEY_free);
if (!pkey) {
JWK_PARSE_ERROR_OSSL("EVP_PKEY_new() for RSA");
return {};
}
if (1 != ::EVP_PKEY_set1_RSA(pkey, rsa)) {
JWK_PARSE_ERROR_OSSL("EVP_PKEY_set1_RSA()");
return {};
}
#endif // USE_V3_API
if (!isPublic) {
auto len = ::i2d_PrivateKey(pkey, nullptr);
if (len <= 0) {
JWK_PARSE_ERROR_OSSL("i2d_PrivateKey() for RSA");
return {};
}
auto buf = new (arena) uint8_t[len];
auto out = std::add_pointer_t<uint8_t>(buf);
len = ::i2d_PrivateKey(pkey, &out);
// assign through public API, even if it means some parsing overhead
return PrivateKey(DerEncoded{}, StringRef(buf, len));
} else {
auto len = ::i2d_PUBKEY(pkey, nullptr);
if (len <= 0) {
JWK_PARSE_ERROR_OSSL("i2d_PUBKEY() for RSA");
return {};
}
auto buf = new (arena) uint8_t[len];
auto out = std::add_pointer_t<uint8_t>(buf);
len = ::i2d_PUBKEY(pkey, &out);
// assign through public API, even if it means some parsing overhead
return PublicKey(DerEncoded{}, StringRef(buf, len));
}
}
template <class Value>
Optional<PublicOrPrivateKey> parseKey(const Value& key, StringRef kty, int keyIndex) {
if (kty == "EC"_sr) {
DECLARE_JWK_REQUIRED_STRING_MEMBER(key, alg);
if (alg != "ES256"_sr) {
JWK_PARSE_ERROR("Unsupported EC algorithm").detail("Algorithm", alg.toString());
return {};
}
DECLARE_JWK_REQUIRED_STRING_MEMBER(key, crv);
if (crv != "P-256"_sr) {
JWK_PARSE_ERROR("Unsupported EC curve").detail("Curve", crv.toString());
return {};
}
DECLARE_JWK_REQUIRED_STRING_MEMBER(key, x);
DECLARE_JWK_REQUIRED_STRING_MEMBER(key, y);
DECLARE_JWK_OPTIONAL_STRING_MEMBER(key, d);
return parseEcP256Key(x, y, d, keyIndex);
} else if (kty == "RSA"_sr) {
DECLARE_JWK_REQUIRED_STRING_MEMBER(key, alg);
if (alg != "RS256"_sr) {
JWK_PARSE_ERROR("Unsupported RSA algorithm").detail("Algorithm", alg.toString());
return {};
}
DECLARE_JWK_REQUIRED_STRING_MEMBER(key, n);
DECLARE_JWK_REQUIRED_STRING_MEMBER(key, e);
DECLARE_JWK_OPTIONAL_STRING_MEMBER(key, d);
DECLARE_JWK_OPTIONAL_STRING_MEMBER(key, p);
DECLARE_JWK_OPTIONAL_STRING_MEMBER(key, q);
DECLARE_JWK_OPTIONAL_STRING_MEMBER(key, dp);
DECLARE_JWK_OPTIONAL_STRING_MEMBER(key, dq);
DECLARE_JWK_OPTIONAL_STRING_MEMBER(key, qi);
auto privKeyArgs = 0;
privKeyArgs += d.present();
privKeyArgs += p.present();
privKeyArgs += q.present();
privKeyArgs += dp.present();
privKeyArgs += dq.present();
privKeyArgs += qi.present();
if (privKeyArgs == 0 || privKeyArgs == 6) {
return parseRsaKey(n, e, d, p, q, dp, dq, qi, keyIndex);
} else {
JWK_PARSE_ERROR("Private key arguments partially exist").detail("NumMissingArgs", 6 - privKeyArgs);
return {};
}
} else {
JWK_PARSE_ERROR("Unsupported key type").detail("KeyType", kty.toString());
return {};
}
}
bool encodeEcKey(rapidjson::Writer<rapidjson::StringBuffer>& writer,
StringRef keyName,
EVP_PKEY* pKey,
const bool isPublic) {
auto arena = Arena();
writer.StartObject();
writer.Key("kty");
writer.String("EC");
writer.Key("alg");
writer.String("ES256");
writer.Key("kid");
writer.String(reinterpret_cast<char const*>(keyName.begin()), keyName.size());
#if USE_V3_API
auto curveNameBuf = std::array<char, 64>{};
auto curveNameLen = 0ul;
if (1 != EVP_PKEY_get_utf8_string_param(
pKey, OSSL_PKEY_PARAM_GROUP_NAME, curveNameBuf.begin(), sizeof(curveNameBuf), &curveNameLen)) {
JWK_WRITE_ERROR_OSSL("Get group name from EC PKey");
return false;
}
auto curveName = std::string_view(curveNameBuf.cbegin(), curveNameLen);
if (curveName != std::string_view("prime256v1")) {
JWK_WRITE_ERROR("Unsupported EC curve").detail("CurveName", curveName);
return false;
}
writer.Key("crv");
writer.String("P-256");
#define JWK_WRITE_BN_EC_PARAM(x, param) \
do { \
auto x = AutoCPointer(nullptr, &::BN_clear_free); \
auto rawX = std::add_pointer_t<BIGNUM>(); \
if (1 != ::EVP_PKEY_get_bn_param(pKey, param, &rawX)) { \
JWK_WRITE_ERROR_OSSL("EVP_PKEY_get_bn_param(" #param ")"); \
return false; \
} \
x.reset(rawX); \
auto b64##x = bigNumToBase64Url(arena, x); \
writer.Key(#x); \
writer.String(reinterpret_cast<char const*>(b64##x.begin()), b64##x.size()); \
} while (0)
// Get and write affine coordinates, X and Y
JWK_WRITE_BN_EC_PARAM(x, OSSL_PKEY_PARAM_EC_PUB_X);
JWK_WRITE_BN_EC_PARAM(y, OSSL_PKEY_PARAM_EC_PUB_Y);
if (!isPublic) {
JWK_WRITE_BN_EC_PARAM(d, OSSL_PKEY_PARAM_PRIV_KEY);
}
#undef JWK_WRITE_BN_EC_PARAM
#else // USE_V3_API
auto ecKey = ::EVP_PKEY_get0_EC_KEY(pKey); // get0 == no refcount, no need to free
if (!ecKey) {
JWK_WRITE_ERROR_OSSL("Could not extract EC_KEY from EVP_PKEY");
return false;
}
auto group = ::EC_KEY_get0_group(ecKey);
if (!group) {
JWK_WRITE_ERROR("Could not get EC_GROUP from EVP_PKEY");
return false;
}
auto curveName = ::EC_GROUP_get_curve_name(group);
if (curveName == NID_undef) {
JWK_WRITE_ERROR("Could not match EC_GROUP to known curve");
return false;
}
if (curveName != NID_X9_62_prime256v1) {
JWK_WRITE_ERROR("Unsupported curve, expected P-256 (prime256v1)").detail("curveName", ::OBJ_nid2sn(curveName));
return false;
}
writer.Key("crv");
writer.String("P-256");
auto point = ::EC_KEY_get0_public_key(ecKey);
if (!point) {
JWK_WRITE_ERROR_OSSL("EC_KEY_get0_public_key() returned null");
return false;
}
auto x = AutoCPointer(::BN_new(), &::BN_free);
if (!x) {
JWK_WRITE_ERROR_OSSL("x = BN_new()");
return false;
}
auto y = AutoCPointer(::BN_new(), &::BN_free);
if (!y) {
JWK_WRITE_ERROR_OSSL("y = BN_new()");
return false;
}
if (1 !=
#ifdef OPENSSL_IS_BORINGSSL
::EC_POINT_get_affine_coordinates_GFp(group, point, x, y, nullptr)
#else
::EC_POINT_get_affine_coordinates(group, point, x, y, nullptr)
#endif
) {
JWK_WRITE_ERROR_OSSL("EC_POINT_get_affine_coordinates()");
return false;
}
auto b64X = bigNumToBase64Url(arena, x);
auto b64Y = bigNumToBase64Url(arena, y);
writer.Key("x");
writer.String(reinterpret_cast<char const*>(b64X.begin()), b64X.size());
writer.Key("y");
writer.String(reinterpret_cast<char const*>(b64Y.begin()), b64Y.size());
if (!isPublic) {
auto d = ::EC_KEY_get0_private_key(ecKey);
if (!d) {
JWK_WRITE_ERROR("EC_KEY_get0_private_key()");
return false;
}
auto b64D = bigNumToBase64Url(arena, d);
writer.Key("d");
writer.String(reinterpret_cast<char const*>(b64D.begin()), b64D.size());
}
#endif // USE_V3_API
writer.EndObject();
return true;
}
bool encodeRsaKey(rapidjson::Writer<rapidjson::StringBuffer>& writer,
StringRef keyName,
EVP_PKEY* pKey,
const bool isPublic) {
auto arena = Arena();
writer.StartObject();
writer.Key("kty");
writer.String("RSA");
writer.Key("alg");
writer.String("RS256");
writer.Key("kid");
writer.String(reinterpret_cast<char const*>(keyName.begin()), keyName.size());
#if USE_V3_API
#define JWK_WRITE_BN_RSA_PARAM_V3(x, param) \
do { \
auto x = AutoCPointer(nullptr, &::BN_clear_free); \
auto rawX = std::add_pointer_t<BIGNUM>(); \
if (1 != ::EVP_PKEY_get_bn_param(pKey, param, &rawX)) { \
JWK_WRITE_ERROR_OSSL("EVP_PKEY_get_bn_param(" #x ")"); \
return false; \
} \
x.reset(rawX); \
auto b64##x = bigNumToBase64Url(arena, x); \
writer.Key(#x); \
writer.String(reinterpret_cast<char const*>(b64##x.begin()), b64##x.size()); \
} while (0)
JWK_WRITE_BN_RSA_PARAM_V3(n, OSSL_PKEY_PARAM_RSA_N);
JWK_WRITE_BN_RSA_PARAM_V3(e, OSSL_PKEY_PARAM_RSA_E);
if (!isPublic) {
JWK_WRITE_BN_RSA_PARAM_V3(d, OSSL_PKEY_PARAM_RSA_D);
JWK_WRITE_BN_RSA_PARAM_V3(p, OSSL_PKEY_PARAM_RSA_FACTOR1);
JWK_WRITE_BN_RSA_PARAM_V3(q, OSSL_PKEY_PARAM_RSA_FACTOR2);
JWK_WRITE_BN_RSA_PARAM_V3(dp, OSSL_PKEY_PARAM_RSA_EXPONENT1);
JWK_WRITE_BN_RSA_PARAM_V3(dq, OSSL_PKEY_PARAM_RSA_EXPONENT2);
JWK_WRITE_BN_RSA_PARAM_V3(qi, OSSL_PKEY_PARAM_RSA_COEFFICIENT1);
}
#undef JWK_WRITE_BN_RSA_PARAM_V3
#else // USE_V3_API
#define JWK_WRITE_BN_RSA_PARAM_V1(x) \
do { \
if (!x) { \
JWK_WRITE_ERROR_OSSL("RSA_get0_* returned null " #x); \
return false; \
} \
auto b64##x = bigNumToBase64Url(arena, x); \
writer.Key(#x); \
writer.String(reinterpret_cast<char const*>(b64##x.begin()), b64##x.size()); \
} while (0)
auto rsaKey = ::EVP_PKEY_get0_RSA(pKey); // get0 == no refcount, no need to free
if (!rsaKey) {
JWK_WRITE_ERROR_OSSL("Could not extract RSA key from EVP_PKEY");
return false;
}
auto n = std::add_pointer_t<const BIGNUM>();
auto e = std::add_pointer_t<const BIGNUM>();
auto d = std::add_pointer_t<const BIGNUM>();
auto p = std::add_pointer_t<const BIGNUM>();
auto q = std::add_pointer_t<const BIGNUM>();
auto dp = std::add_pointer_t<const BIGNUM>();
auto dq = std::add_pointer_t<const BIGNUM>();
auto qi = std::add_pointer_t<const BIGNUM>();
::RSA_get0_key(rsaKey, &n, &e, &d);
JWK_WRITE_BN_RSA_PARAM_V1(n);
JWK_WRITE_BN_RSA_PARAM_V1(e);
if (!isPublic) {
::RSA_get0_factors(rsaKey, &p, &q);
::RSA_get0_crt_params(rsaKey, &dp, &dq, &qi);
JWK_WRITE_BN_RSA_PARAM_V1(d);
JWK_WRITE_BN_RSA_PARAM_V1(p);
JWK_WRITE_BN_RSA_PARAM_V1(q);
JWK_WRITE_BN_RSA_PARAM_V1(dp);
JWK_WRITE_BN_RSA_PARAM_V1(dq);
JWK_WRITE_BN_RSA_PARAM_V1(qi);
}
#undef JWK_WRITE_BN_RSA_PARAM_V1
#endif // USE_V3_API
writer.EndObject();
return true;
}
// Add exactly one object to context of writer. Object shall contain JWK-encoded public or private key
bool encodeKey(rapidjson::Writer<rapidjson::StringBuffer>& writer, StringRef keyName, const PublicOrPrivateKey& key) {
auto const isPublic = key.isPublic();
auto pKey = std::add_pointer_t<EVP_PKEY>();
auto alg = PKeyAlgorithm{};
if (isPublic) {
auto const& keyObj = key.getPublic();
pKey = keyObj.nativeHandle();
alg = keyObj.algorithm();
} else {
auto const& keyObj = key.getPrivate();
pKey = key.getPrivate().nativeHandle();
alg = keyObj.algorithm();
}
if (!pKey) {
JWK_WRITE_ERROR("PKey object to encode is null");
return false;
}
if (alg == PKeyAlgorithm::EC) {
return encodeEcKey(writer, keyName, pKey, isPublic);
} else if (alg == PKeyAlgorithm::RSA) {
return encodeRsaKey(writer, keyName, pKey, isPublic);
} else {
JWK_WRITE_ERROR("Attempted to encode PKey with unsupported algorithm");
return false;
}
return true;
}
void testPublicKey(PrivateKey (*factory)()) {
// stringify-deserialize public key.
// sign some data using private key to see whether deserialized public key can verify it.
auto& rng = *deterministicRandom();
auto pubKeyName = Standalone<StringRef>("somePublicKey"_sr);
auto privKey = factory();
auto pubKey = privKey.toPublic();
auto jwks = JsonWebKeySet{};
jwks.keys.emplace(pubKeyName, pubKey);
auto arena = Arena();
auto jwksStr = jwks.toStringRef(arena).get();
fmt::print("Test JWKS: {}\n", jwksStr.toString());
auto jwksClone = JsonWebKeySet::parse(jwksStr, {});
ASSERT(jwksClone.present());
auto pubKeyClone = jwksClone.get().keys[pubKeyName].getPublic();
auto randByteStr = [&rng, &arena](int len) {
auto buf = new (arena) uint8_t[len];
for (auto i = 0; i < len; i++)
buf[i] = rng.randomUInt32() % 255u;
return StringRef(buf, len);
};
auto randData = randByteStr(rng.randomUInt32() % 128 + 16);
auto signature = privKey.sign(arena, randData, *::EVP_sha256());
ASSERT(pubKeyClone.verify(randData, signature, *::EVP_sha256()));
const_cast<uint8_t&>(*randData.begin())++;
ASSERT(!pubKeyClone.verify(randData, signature, *::EVP_sha256()));
fmt::print("TESTED OK FOR OPENSSL V{} API\n", (OPENSSL_VERSION_NUMBER >> 28));
}
void testPrivateKey(PrivateKey (*factory)()) {
// stringify-deserialize private key.
// sign some data using deserialized private key to see whether public key can verify it.
auto& rng = *deterministicRandom();
auto privKeyName = Standalone<StringRef>("somePrivateKey"_sr);
auto privKey = factory();
auto pubKey = privKey.toPublic();
auto jwks = JsonWebKeySet{};
jwks.keys.emplace(privKeyName, privKey);
auto arena = Arena();
auto jwksStr = jwks.toStringRef(arena).get();
fmt::print("Test JWKS: {}\n", jwksStr.toString());
auto jwksClone = JsonWebKeySet::parse(jwksStr, {});
ASSERT(jwksClone.present());
auto privKeyClone = jwksClone.get().keys[privKeyName].getPrivate();
auto randByteStr = [&rng, &arena](int len) {
auto buf = new (arena) uint8_t[len];
for (auto i = 0; i < len; i++)
buf[i] = rng.randomUInt32() % 255u;
return StringRef(buf, len);
};
auto randData = randByteStr(rng.randomUInt32() % 128 + 16);
auto signature = privKeyClone.sign(arena, randData, *::EVP_sha256());
ASSERT(pubKey.verify(randData, signature, *::EVP_sha256()));
const_cast<uint8_t&>(*randData.begin())++;
ASSERT(!pubKey.verify(randData, signature, *::EVP_sha256()));
fmt::print("TESTED OK FOR OPENSSL V{} API\n", (OPENSSL_VERSION_NUMBER >> 28));
}
} // anonymous namespace
Optional<JsonWebKeySet> JsonWebKeySet::parse(StringRef jwksString, VectorRef<StringRef> allowedUses) {
auto d = rapidjson::Document();
d.Parse(reinterpret_cast<const char*>(jwksString.begin()), jwksString.size());
if (d.HasParseError()) {
JWKS_PARSE_ERROR("ParseError")
.detail("Message", GetParseError_En(d.GetParseError()))
.detail("Offset", d.GetErrorOffset());
return {};
}
auto keysItr = d.FindMember("keys");
if (!d.IsObject() || keysItr == d.MemberEnd() || !keysItr->value.IsArray()) {
JWKS_PARSE_ERROR("JWKS must be an object and have 'keys' array member");
return {};
}
auto const& keys = keysItr->value;
auto ret = JsonWebKeySet{};
for (auto keyIndex = 0; keyIndex < keys.Size(); keyIndex++) {
if (!keys[keyIndex].IsObject()) {
JWKS_PARSE_ERROR("element of 'keys' array must be an object");
return {};
}
auto const& key = keys[keyIndex];
DECLARE_JWK_REQUIRED_STRING_MEMBER(key, kty);
DECLARE_JWK_REQUIRED_STRING_MEMBER(key, kid);
DECLARE_JWK_OPTIONAL_STRING_MEMBER(key, use);
if (use.present() && !allowedUses.empty()) {
auto allowed = false;
for (auto allowedUse : allowedUses) {
if (allowedUse == use.get()) {
allowed = true;
break;
}
}
if (!allowed) {
JWK_PARSE_ERROR("Illegal optional 'use' member found").detail("Use", use.get().toString());
return {};
}
}
auto parsedKey = parseKey(key, kty, keyIndex);
if (!parsedKey.present())
return {};
auto [iter, inserted] = ret.keys.insert({ Standalone<StringRef>(kid), parsedKey.get() });
if (!inserted) {
JWK_PARSE_ERROR("Duplicate key name").detail("KeyName", kid.toString());
return {};
}
}
return ret;
}
Optional<StringRef> JsonWebKeySet::toStringRef(Arena& arena) {
using Buffer = rapidjson::StringBuffer;
using Writer = rapidjson::Writer<Buffer>;
auto buffer = Buffer();
auto writer = Writer(buffer);
writer.StartObject();
writer.Key("keys");
writer.StartArray();
for (const auto& [keyName, key] : keys) {
if (!encodeKey(writer, keyName, key)) {
return {};
}
}
writer.EndArray();
writer.EndObject();
auto buf = new (arena) uint8_t[buffer.GetSize()];
::memcpy(buf, buffer.GetString(), buffer.GetSize());
return StringRef(buf, buffer.GetSize());
}
void forceLinkJsonWebKeySetTests() {}
TEST_CASE("/fdbrpc/JsonWebKeySet/EC/PublicKey") {
testPublicKey(&mkcert::makeEcP256);
return Void();
}
TEST_CASE("/fdbrpc/JsonWebKeySet/EC/PrivateKey") {
testPrivateKey(&mkcert::makeEcP256);
return Void();
}
TEST_CASE("/fdbrpc/JsonWebKeySet/RSA/PublicKey") {
testPublicKey(&mkcert::makeRsa2048Bit);
return Void();
}
TEST_CASE("/fdbrpc/JsonWebKeySet/RSA/PrivateKey") {
testPrivateKey(&mkcert::makeRsa2048Bit);
return Void();
}

View File

@ -54,23 +54,6 @@ constexpr int MaxIssuerNameLenPlus1 = 25;
constexpr int MaxTenantNameLenPlus1 = 17;
constexpr int MaxKeyNameLenPlus1 = 21;
void trace(const char* type) {
auto te = TraceEvent(SevWarnAlways, type);
te.suppressFor(60);
if (auto err = ::ERR_get_error()) {
char buf[256]{
0,
};
::ERR_error_string_n(err, buf, sizeof(buf));
te.detail("OpenSSLError", static_cast<const char*>(buf));
}
}
[[noreturn]] void traceAndThrow(const char* type) {
trace(type);
throw digital_signature_ops_error();
}
StringRef genRandomAlphanumStringRef(Arena& arena, IRandom& rng, int maxLenPlusOne) {
const auto len = rng.randomInt(1, maxLenPlusOne);
auto strRaw = new (arena) uint8_t[len];
@ -79,6 +62,30 @@ StringRef genRandomAlphanumStringRef(Arena& arena, IRandom& rng, int maxLenPlusO
return StringRef(strRaw, len);
}
bool checkVerifyAlgorithm(PKeyAlgorithm algo, PublicKey key) {
if (algo != key.algorithm()) {
TraceEvent(SevWarnAlways, "TokenVerifyAlgoMismatch")
.suppressFor(10)
.detail("Expected", pkeyAlgorithmName(algo))
.detail("PublicKeyAlgorithm", key.algorithmName());
return false;
} else {
return true;
}
}
bool checkSignAlgorithm(PKeyAlgorithm algo, PrivateKey key) {
if (algo != key.algorithm()) {
TraceEvent(SevWarnAlways, "TokenSignAlgoMismatch")
.suppressFor(10)
.detail("Expected", pkeyAlgorithmName(algo))
.detail("PublicKeyAlgorithm", key.algorithmName());
return false;
} else {
return true;
}
}
} // namespace
namespace authz {
@ -94,21 +101,13 @@ Algorithm algorithmFromString(StringRef s) noexcept {
return Algorithm::UNKNOWN;
}
StringRef signString(Arena& arena, StringRef string, PrivateKey privateKey, int keyAlgNid, MessageDigestMethod digest);
bool verifyStringSignature(StringRef string,
StringRef signature,
PublicKey publicKey,
int keyAlgNid,
MessageDigestMethod digest);
std::pair<int /*key algorithm nid*/, MessageDigestMethod> getMethod(Algorithm alg) {
std::pair<PKeyAlgorithm, MessageDigestMethod> getMethod(Algorithm alg) {
if (alg == Algorithm::RS256) {
return { EVP_PKEY_RSA, ::EVP_sha256() };
return { PKeyAlgorithm::RSA, ::EVP_sha256() };
} else if (alg == Algorithm::ES256) {
return { EVP_PKEY_EC, ::EVP_sha256() };
return { PKeyAlgorithm::EC, ::EVP_sha256() };
} else {
return { NID_undef, nullptr };
return { PKeyAlgorithm::UNSUPPORTED, nullptr };
}
}
@ -121,71 +120,6 @@ std::string_view getAlgorithmName(Algorithm alg) {
UNREACHABLE();
}
StringRef signString(Arena& arena, StringRef string, PrivateKey privateKey, int keyAlgNid, MessageDigestMethod digest) {
ASSERT_NE(keyAlgNid, NID_undef);
auto key = privateKey.nativeHandle();
auto const privateKeyAlgNid = ::EVP_PKEY_base_id(key);
if (privateKeyAlgNid != keyAlgNid) {
TraceEvent(SevWarnAlways, "TokenSignAlgoMismatch")
.suppressFor(10)
.detail("ExpectedAlg", OBJ_nid2sn(keyAlgNid))
.detail("PublicKeyAlg", OBJ_nid2sn(privateKeyAlgNid));
throw digital_signature_ops_error();
}
auto mdctx = ::EVP_MD_CTX_create();
if (!mdctx)
traceAndThrow("SignTokenInitFail");
auto mdctxGuard = ScopeExit([mdctx]() { ::EVP_MD_CTX_free(mdctx); });
if (1 != ::EVP_DigestSignInit(mdctx, nullptr, digest, nullptr, key))
traceAndThrow("SignTokenInitFail");
if (1 != ::EVP_DigestSignUpdate(mdctx, string.begin(), string.size()))
traceAndThrow("SignTokenUpdateFail");
auto sigLen = size_t{};
if (1 != ::EVP_DigestSignFinal(mdctx, nullptr, &sigLen)) // assess the length first
traceAndThrow("SignTokenGetSigLenFail");
auto sigBuf = new (arena) uint8_t[sigLen];
if (1 != ::EVP_DigestSignFinal(mdctx, sigBuf, &sigLen))
traceAndThrow("SignTokenFinalizeFail");
return StringRef(sigBuf, sigLen);
}
bool verifyStringSignature(StringRef string,
StringRef signature,
PublicKey publicKey,
int keyAlgNid,
MessageDigestMethod digest) {
ASSERT_NE(keyAlgNid, NID_undef);
auto key = publicKey.nativeHandle();
auto const publicKeyAlgNid = ::EVP_PKEY_base_id(key);
if (keyAlgNid != publicKeyAlgNid) {
TraceEvent(SevWarnAlways, "TokenVerifyAlgoMismatch")
.suppressFor(10)
.detail("ExpectedAlg", OBJ_nid2sn(keyAlgNid))
.detail("PublicKeyAlg", OBJ_nid2sn(publicKeyAlgNid));
return false; // public key's algorithm doesn't match string's
}
auto mdctx = ::EVP_MD_CTX_create();
if (!mdctx) {
trace("VerifyTokenInitFail");
return false;
}
auto mdctxGuard = ScopeExit([mdctx]() { ::EVP_MD_CTX_free(mdctx); });
if (1 != ::EVP_DigestVerifyInit(mdctx, nullptr, digest, nullptr, key)) {
trace("VerifyTokenInitFail");
return false;
}
if (1 != ::EVP_DigestVerifyUpdate(mdctx, string.begin(), string.size())) {
trace("VerifyTokenUpdateFail");
return false;
}
if (1 != ::EVP_DigestVerifyFinal(mdctx, signature.begin(), signature.size())) {
auto te = TraceEvent(SevWarnAlways, "VerifyTokenFail");
te.suppressFor(5);
return false;
}
return true;
}
} // namespace authz
namespace authz::flatbuffers {
@ -195,8 +129,11 @@ SignedTokenRef signToken(Arena& arena, TokenRef token, StringRef keyName, Privat
auto writer = ObjectWriter([&arena](size_t len) { return new (arena) uint8_t[len]; }, IncludeVersion());
writer.serialize(token);
auto tokenStr = writer.toStringRef();
auto [keyAlgNid, digest] = getMethod(Algorithm::ES256);
auto sig = signString(arena, tokenStr, privateKey, keyAlgNid, digest);
auto [signAlgo, digest] = getMethod(Algorithm::ES256);
if (!checkSignAlgorithm(signAlgo, privateKey)) {
throw digital_signature_ops_error();
}
auto sig = privateKey.sign(arena, tokenStr, *digest);
ret.token = tokenStr;
ret.signature = sig;
ret.keyName = StringRef(arena, keyName);
@ -204,8 +141,10 @@ SignedTokenRef signToken(Arena& arena, TokenRef token, StringRef keyName, Privat
}
bool verifyToken(SignedTokenRef signedToken, PublicKey publicKey) {
auto [keyAlgNid, digest] = getMethod(Algorithm::ES256);
return verifyStringSignature(signedToken.token, signedToken.signature, publicKey, keyAlgNid, digest);
auto [keyAlg, digest] = getMethod(Algorithm::ES256);
if (!checkVerifyAlgorithm(keyAlg, publicKey))
return false;
return publicKey.verify(signedToken.token, signedToken.signature, *digest);
}
TokenRef makeRandomTokenSpec(Arena& arena, IRandom& rng) {
@ -281,15 +220,14 @@ StringRef makeTokenPart(Arena& arena, TokenRef tokenSpec) {
return StringRef(out, totalLen);
}
StringRef makePlainSignature(Arena& arena, Algorithm alg, StringRef tokenPart, PrivateKey privateKey) {
auto [keyAlgNid, digest] = getMethod(alg);
return signString(arena, tokenPart, privateKey, keyAlgNid, digest);
}
StringRef signToken(Arena& arena, TokenRef tokenSpec, PrivateKey privateKey) {
auto tmpArena = Arena();
auto tokenPart = makeTokenPart(tmpArena, tokenSpec);
auto plainSig = makePlainSignature(tmpArena, tokenSpec.algorithm, tokenPart, privateKey);
auto [signAlgo, digest] = getMethod(tokenSpec.algorithm);
if (!checkSignAlgorithm(signAlgo, privateKey)) {
throw digital_signature_ops_error();
}
auto plainSig = privateKey.sign(tmpArena, tokenPart, *digest);
auto const sigPartLen = base64url::encodedLength(plainSig.size());
auto const totalLen = tokenPart.size() + 1 + sigPartLen;
auto out = new (arena) uint8_t[totalLen];
@ -304,9 +242,10 @@ StringRef signToken(Arena& arena, TokenRef tokenSpec, PrivateKey privateKey) {
bool parseHeaderPart(TokenRef& token, StringRef b64urlHeader) {
auto tmpArena = Arena();
auto [header, valid] = base64url::decode(tmpArena, b64urlHeader);
if (!valid)
auto optHeader = base64url::decode(tmpArena, b64urlHeader);
if (!optHeader.present())
return false;
auto header = optHeader.get();
auto d = rapidjson::Document();
d.Parse(reinterpret_cast<const char*>(header.begin()), header.size());
if (d.HasParseError()) {
@ -317,9 +256,11 @@ bool parseHeaderPart(TokenRef& token, StringRef b64urlHeader) {
.detail("Offset", d.GetErrorOffset());
return false;
}
if (d.IsObject() && d.HasMember("alg") && d.HasMember("typ")) {
auto const& alg = d["alg"];
auto const& typ = d["typ"];
auto algItr = d.FindMember("alg");
auto typItr = d.FindMember("typ");
if (d.IsObject() && algItr != d.MemberEnd() && typItr != d.MemberEnd()) {
auto const& alg = algItr->value;
auto const& typ = typItr->value;
if (alg.IsString() && typ.IsString()) {
auto algValue = StringRef(reinterpret_cast<const uint8_t*>(alg.GetString()), alg.GetStringLength());
auto algType = algorithmFromString(algValue);
@ -337,9 +278,10 @@ bool parseHeaderPart(TokenRef& token, StringRef b64urlHeader) {
template <class FieldType>
bool parseField(Arena& arena, Optional<FieldType>& out, const rapidjson::Document& d, const char* fieldName) {
if (!d.HasMember(fieldName))
auto fieldItr = d.FindMember(fieldName);
if (fieldItr == d.MemberEnd())
return true;
auto const& field = d[fieldName];
auto const& field = fieldItr->value;
static_assert(std::is_same_v<StringRef, FieldType> || std::is_same_v<FieldType, uint64_t> ||
std::is_same_v<FieldType, VectorRef<StringRef>>);
if constexpr (std::is_same_v<FieldType, StringRef>) {
@ -371,7 +313,10 @@ bool parseField(Arena& arena, Optional<FieldType>& out, const rapidjson::Documen
bool parsePayloadPart(Arena& arena, TokenRef& token, StringRef b64urlPayload) {
auto tmpArena = Arena();
auto [payload, valid] = base64url::decode(tmpArena, b64urlPayload);
auto optPayload = base64url::decode(tmpArena, b64urlPayload);
if (!optPayload.present())
return false;
auto payload = optPayload.get();
auto d = rapidjson::Document();
d.Parse(reinterpret_cast<const char*>(payload.begin()), payload.size());
if (d.HasParseError()) {
@ -406,10 +351,11 @@ bool parsePayloadPart(Arena& arena, TokenRef& token, StringRef b64urlPayload) {
}
bool parseSignaturePart(Arena& arena, TokenRef& token, StringRef b64urlSignature) {
auto [sig, valid] = base64url::decode(arena, b64urlSignature);
if (valid)
token.signature = sig;
return valid;
auto optSig = base64url::decode(arena, b64urlSignature);
if (!optSig.present())
return false;
token.signature = optSig.get();
return true;
}
bool parseToken(Arena& arena, TokenRef& token, StringRef signedToken) {
@ -436,14 +382,17 @@ bool verifyToken(StringRef signedToken, PublicKey publicKey) {
if (b64urlHeader.empty() || b64urlPayload.empty() || b64urlSignature.empty())
return false;
auto b64urlTokenPart = fullToken.substr(0, b64urlHeader.size() + 1 + b64urlPayload.size());
auto [sig, valid] = base64url::decode(arena, b64urlSignature);
if (!valid)
auto optSig = base64url::decode(arena, b64urlSignature);
if (!optSig.present())
return false;
auto sig = optSig.get();
auto parsedToken = TokenRef();
if (!parseHeaderPart(parsedToken, b64urlHeader))
return false;
auto [keyAlgNid, digest] = getMethod(parsedToken.algorithm);
return verifyStringSignature(b64urlTokenPart, sig, publicKey, keyAlgNid, digest);
auto [verifyAlgo, digest] = getMethod(parsedToken.algorithm);
if (!checkVerifyAlgorithm(verifyAlgo, publicKey))
return false;
return publicKey.verify(b64urlTokenPart, sig, *digest);
}
TokenRef makeRandomTokenSpec(Arena& arena, IRandom& rng, Algorithm alg) {
@ -485,14 +434,14 @@ TEST_CASE("/fdbrpc/TokenSign/FlatBuffer") {
auto tokenSpec = authz::flatbuffers::makeRandomTokenSpec(arena, rng);
auto keyName = genRandomAlphanumStringRef(arena, rng, MaxKeyNameLenPlus1);
auto signedToken = authz::flatbuffers::signToken(arena, tokenSpec, keyName, privateKey);
const auto verifyExpectOk = authz::flatbuffers::verifyToken(signedToken, privateKey.toPublicKey());
const auto verifyExpectOk = authz::flatbuffers::verifyToken(signedToken, privateKey.toPublic());
ASSERT(verifyExpectOk);
// try tampering with signed token by adding one more tenant
tokenSpec.tenants.push_back(arena, genRandomAlphanumStringRef(arena, rng, MaxTenantNameLenPlus1));
auto writer = ObjectWriter([&arena](size_t len) { return new (arena) uint8_t[len]; }, IncludeVersion());
writer.serialize(tokenSpec);
signedToken.token = writer.toStringRef();
const auto verifyExpectFail = authz::flatbuffers::verifyToken(signedToken, privateKey.toPublicKey());
const auto verifyExpectFail = authz::flatbuffers::verifyToken(signedToken, privateKey.toPublic());
ASSERT(!verifyExpectFail);
}
printf("%d runs OK\n", numIters);
@ -507,7 +456,7 @@ TEST_CASE("/fdbrpc/TokenSign/JWT") {
auto& rng = *deterministicRandom();
auto tokenSpec = authz::jwt::makeRandomTokenSpec(arena, rng, authz::Algorithm::ES256);
auto signedToken = authz::jwt::signToken(arena, tokenSpec, privateKey);
const auto verifyExpectOk = authz::jwt::verifyToken(signedToken, privateKey.toPublicKey());
const auto verifyExpectOk = authz::jwt::verifyToken(signedToken, privateKey.toPublic());
ASSERT(verifyExpectOk);
auto signaturePart = signedToken;
signaturePart.eat("."_sr);
@ -527,15 +476,15 @@ TEST_CASE("/fdbrpc/TokenSign/JWT") {
ASSERT_EQ(tokenSpec.expiresAtUnixTime.get(), parsedToken.expiresAtUnixTime.get());
ASSERT_EQ(tokenSpec.notBeforeUnixTime.get(), parsedToken.notBeforeUnixTime.get());
ASSERT(tokenSpec.tenants == parsedToken.tenants);
auto [sig, sigValid] = base64url::decode(tmpArena, signaturePart);
ASSERT(sigValid);
ASSERT(sig == parsedToken.signature);
auto optSig = base64url::decode(tmpArena, signaturePart);
ASSERT(optSig.present());
ASSERT(optSig.get() == parsedToken.signature);
}
// try tampering with signed token by adding one more tenant
tokenSpec.tenants.get().push_back(arena, genRandomAlphanumStringRef(arena, rng, MaxTenantNameLenPlus1));
auto tamperedTokenPart = makeTokenPart(arena, tokenSpec);
auto tamperedTokenString = fmt::format("{}.{}", tamperedTokenPart.toString(), signaturePart.toString());
const auto verifyExpectFail = authz::jwt::verifyToken(StringRef(tamperedTokenString), privateKey.toPublicKey());
const auto verifyExpectFail = authz::jwt::verifyToken(StringRef(tamperedTokenString), privateKey.toPublic());
ASSERT(!verifyExpectFail);
}
printf("%d runs OK\n", numIters);
@ -543,13 +492,13 @@ TEST_CASE("/fdbrpc/TokenSign/JWT") {
}
TEST_CASE("/fdbrpc/TokenSign/bench") {
constexpr auto repeat = 10;
constexpr auto repeat = 5;
constexpr auto numSamples = 10000;
auto keys = std::vector<PrivateKey>(numSamples);
auto pubKeys = std::vector<PublicKey>(numSamples);
for (auto i = 0; i < numSamples; i++) {
keys[i] = mkcert::makeEcP256();
pubKeys[i] = keys[i].toPublicKey();
pubKeys[i] = keys[i].toPublic();
}
fmt::print("{} keys generated\n", numSamples);
auto& rng = *deterministicRandom();
@ -581,7 +530,8 @@ TEST_CASE("/fdbrpc/TokenSign/bench") {
auto fbBegin = timer_monotonic();
for (auto rep = 0; rep < repeat; rep++) {
for (auto i = 0; i < numSamples; i++) {
auto signedToken = ObjectReader::fromStringRef<authz::flatbuffers::SignedTokenRef>(fbs[i], Unversioned());
auto signedToken =
ObjectReader::fromStringRef<Standalone<authz::flatbuffers::SignedTokenRef>>(fbs[i], Unversioned());
auto verifyOk = authz::flatbuffers::verifyToken(signedToken, pubKeys[i]);
ASSERT(verifyOk);
}

View File

@ -41,9 +41,9 @@ int decode(const uint8_t* __restrict codeIn, const int lengthIn, uint8_t* __rest
// Returns -1 for invalid length (4n-3)
int decodedLength(int codeLength) noexcept;
// return a) decoded valid string and b) whether input string was a valid URL-encoded base64 string
// return, if base64UrlStr is valid, a StringRef containing a valid decoded string
// Note: even if decoding fails by bad encoding, StringRef memory still stays allocated from arena
std::pair<StringRef, bool> decode(Arena& arena, StringRef base64UrlStr);
Optional<StringRef> decode(Arena& arena, StringRef base64UrlStr);
} // namespace base64url

View File

@ -0,0 +1,67 @@
/*
* JsonWebKeySet.h
*
* This source file is part of the FoundationDB open source project
*
* Copyright 2013-2022 Apple Inc. and the FoundationDB project authors
*
* Licensed under the Apache License, Version 2.0 (the "License");
* you may not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
#ifndef FDBRPC_JSON_WEB_KEY_SET_H
#define FDBRPC_JSON_WEB_KEY_SET_H
#include "flow/Arena.h"
#include "flow/PKey.h"
#include <map>
#include <variant>
struct PublicOrPrivateKey {
std::variant<PublicKey, PrivateKey> key;
PublicOrPrivateKey() noexcept = default;
PublicOrPrivateKey(PublicKey key) noexcept : key(std::in_place_type<PublicKey>, key) {}
PublicOrPrivateKey(PrivateKey key) noexcept : key(std::in_place_type<PrivateKey>, key) {}
bool isPublic() const noexcept { return std::holds_alternative<PublicKey>(key); }
bool isPrivate() const noexcept { return std::holds_alternative<PrivateKey>(key); }
PublicKey getPublic() const { return std::get<PublicKey>(key); }
PrivateKey getPrivate() const { return std::get<PrivateKey>(key); }
};
// Implements JWKS standard in restricted scope:
// - Parses and stores public/private keys (but not shared keys) as OpenSSL internal types
// - Accept only a) EC algorithm with P-256 curve or b) RSA algorithm.
// - Each key object must meet following requirements:
// - "alg" field is set to either "ES256" or "RS256"
// - "kty" field is set to "EC" or "RSA"
// - "kty" field matches the "alg" field: i.e. EC for "alg":"ES256" and RSA for "alg":"RS256"
struct JsonWebKeySet {
using KeyMap = std::map<Standalone<StringRef>, PublicOrPrivateKey>;
KeyMap keys;
// Parse JWKS string to map of KeyName-PKey
// If allowedUses is not empty, JWK's optional "use" member is verified against it.
// Otherwise, uses are all-inclusive.
static Optional<JsonWebKeySet> parse(StringRef jwksString, VectorRef<StringRef> allowedUses);
// Returns JSON string representing the JSON Web Key Set.
// Inverse operation of parse(). Only allows keys expected/accepted by parse().
Optional<StringRef> toStringRef(Arena& arena);
};
#endif // FDBRPC_JSON_WEB_KEY_SET_H

View File

@ -100,10 +100,10 @@ struct TokenRef {
StringRef makeTokenPart(Arena& arena, TokenRef tokenSpec);
// Generate plaintext signature of token part
StringRef makePlainSignature(Arena& arena, Algorithm alg, StringRef tokenPart, StringRef privateKeyDer);
StringRef makePlainSignature(Arena& arena, Algorithm alg, StringRef tokenPart, PrivateKey privateKey);
// One-stop function to make JWT from spec
StringRef signToken(Arena& arena, TokenRef tokenSpec, StringRef privateKeyDer);
StringRef signToken(Arena& arena, TokenRef tokenSpec, PrivateKey privateKey);
// Parse passed b64url-encoded header part and materialize its contents into tokenOut,
// using memory allocated from arena
@ -123,7 +123,7 @@ bool parseSignaturePart(Arena& arena, TokenRef& tokenOut, StringRef b64urlSignat
bool parseToken(Arena& arena, TokenRef& tokenOut, StringRef signedTokenIn);
// Verify only the signature part of signed token string against its token part, not its content
bool verifyToken(StringRef signedToken, StringRef publicKeyDer);
bool verifyToken(StringRef signedToken, PublicKey publicKey);
} // namespace authz::jwt

View File

@ -24,6 +24,7 @@
#include "fdbclient/Notified.h"
#include "fdbclient/SystemData.h"
#include "fdbserver/ApplyMetadataMutation.h"
#include "fdbserver/EncryptedMutationMessage.h"
#include "fdbserver/IKeyValueStore.h"
#include "fdbserver/LogProtocolMessage.h"
#include "fdbserver/LogSystem.h"
@ -67,13 +68,14 @@ public:
ProxyCommitData& proxyCommitData_,
Reference<ILogSystem> logSystem_,
LogPushData* toCommit_,
const std::unordered_map<EncryptCipherDomainId, Reference<BlobCipherKey>>* cipherKeys_,
bool& confChange_,
Version version,
Version popVersion_,
bool initialCommit_)
: spanContext(spanContext_), dbgid(proxyCommitData_.dbgid), arena(arena_), mutations(mutations_),
txnStateStore(proxyCommitData_.txnStateStore), toCommit(toCommit_), confChange(confChange_),
logSystem(logSystem_), version(version), popVersion(popVersion_),
txnStateStore(proxyCommitData_.txnStateStore), toCommit(toCommit_), cipherKeys(cipherKeys_),
confChange(confChange_), logSystem(logSystem_), version(version), popVersion(popVersion_),
vecBackupKeys(&proxyCommitData_.vecBackupKeys), keyInfo(&proxyCommitData_.keyInfo),
cacheInfo(&proxyCommitData_.cacheInfo),
uid_applyMutationsData(proxyCommitData_.firstProxy ? &proxyCommitData_.uid_applyMutationsData : nullptr),
@ -108,6 +110,9 @@ private:
// non-null if these mutations were part of a new commit handled by this commit proxy
LogPushData* toCommit = nullptr;
// Cipher keys used to encrypt to be committed mutations
const std::unordered_map<EncryptCipherDomainId, Reference<BlobCipherKey>>* cipherKeys = nullptr;
// Flag indicates if the configure is changed
bool& confChange;
@ -152,6 +157,16 @@ private:
bool dummyConfChange = false;
private:
void writeMutation(const MutationRef& m) {
if (forResolver || !SERVER_KNOBS->ENABLE_TLOG_ENCRYPTION) {
toCommit->writeTypedMessage(m);
} else {
ASSERT(cipherKeys != nullptr);
Arena arena;
toCommit->writeTypedMessage(EncryptedMutationMessage::encryptMetadata(arena, *cipherKeys, m));
}
}
void checkSetKeyServersPrefix(MutationRef m) {
if (!m.param1.startsWith(keyServersPrefix)) {
return;
@ -221,7 +236,7 @@ private:
.detail("Tag", tag.toString());
toCommit->addTag(tag);
toCommit->writeTypedMessage(privatized);
writeMutation(privatized);
}
}
@ -243,7 +258,7 @@ private:
toCommit->writeTypedMessage(LogProtocolMessage(), true);
TraceEvent(SevDebug, "SendingPrivatized_ServerTag", dbgid).detail("M", privatized);
toCommit->addTag(tag);
toCommit->writeTypedMessage(privatized);
writeMutation(privatized);
}
if (!initialCommit) {
txnStateStore->set(KeyValueRef(m.param1, m.param2));
@ -303,7 +318,7 @@ private:
privatized.param1 = m.param1.withPrefix(systemKeys.begin, arena);
TraceEvent(SevDebug, "SendingPrivatized_CacheTag", dbgid).detail("M", privatized);
toCommit->addTag(cacheTag);
toCommit->writeTypedMessage(privatized);
writeMutation(privatized);
}
void checkSetConfigKeys(MutationRef m) {
@ -354,7 +369,7 @@ private:
toCommit->addTags(allSources);
}
TraceEvent(SevDebug, "SendingPrivatized_ChangeFeed", dbgid).detail("M", privatized);
toCommit->writeTypedMessage(privatized);
writeMutation(privatized);
}
}
@ -390,8 +405,8 @@ private:
}
// Normally uses key backed map, so have to use same unpacking code here.
UID ssId = Codec<UID>::unpack(Tuple::unpack(m.param1.removePrefix(tssMappingKeys.begin)));
UID tssId = Codec<UID>::unpack(Tuple::unpack(m.param2));
UID ssId = TupleCodec<UID>::unpack(m.param1.removePrefix(tssMappingKeys.begin));
UID tssId = TupleCodec<UID>::unpack(m.param2);
if (!initialCommit) {
txnStateStore->set(KeyValueRef(m.param1, m.param2));
}
@ -408,7 +423,7 @@ private:
if (tagV.present()) {
TraceEvent(SevDebug, "SendingPrivatized_TSSID", dbgid).detail("M", privatized);
toCommit->addTag(decodeServerTagValue(tagV.get()));
toCommit->writeTypedMessage(privatized);
writeMutation(privatized);
}
}
}
@ -437,7 +452,7 @@ private:
privatized.param1 = m.param1.withPrefix(systemKeys.begin, arena);
TraceEvent(SevDebug, "SendingPrivatized_TSSQuarantine", dbgid).detail("M", privatized);
toCommit->addTag(decodeServerTagValue(tagV.get()));
toCommit->writeTypedMessage(privatized);
writeMutation(privatized);
}
}
@ -560,7 +575,7 @@ private:
privatized.param1 = m.param1.withPrefix(systemKeys.begin, arena);
TraceEvent(SevDebug, "SendingPrivatized_GlobalKeys", dbgid).detail("M", privatized);
toCommit->addTags(allTags);
toCommit->writeTypedMessage(privatized);
writeMutation(privatized);
}
// Generates private mutations for the target storage server, instructing it to create a checkpoint.
@ -582,7 +597,7 @@ private:
.detail("Checkpoint", checkpoint.toString());
toCommit->addTag(tag);
toCommit->writeTypedMessage(privatized);
writeMutation(privatized);
}
}
@ -639,7 +654,7 @@ private:
if (tenantMap) {
ASSERT(version != invalidVersion);
TenantName tenantName = m.param1.removePrefix(tenantMapPrefix);
TenantMapEntry tenantEntry = decodeTenantEntry(m.param2);
TenantMapEntry tenantEntry = TenantMapEntry::decode(m.param2);
TraceEvent("CommitProxyInsertTenant", dbgid).detail("Tenant", tenantName).detail("Version", version);
(*tenantMap)[tenantName] = tenantEntry;
@ -662,7 +677,7 @@ private:
MutationRef privatized = m;
privatized.param1 = m.param1.withPrefix(systemKeys.begin, arena);
toCommit->writeTypedMessage(privatized);
writeMutation(privatized);
}
TEST(true); // Tenant added to map
@ -760,7 +775,7 @@ private:
TraceEvent(SevDebug, "SendingPrivatized_ClearServerTag", dbgid).detail("M", privatized);
toCommit->addTag(decodeServerTagValue(kv.value));
toCommit->writeTypedMessage(privatized);
writeMutation(privatized);
}
}
// Might be a tss removal, which doesn't store a tag there.
@ -784,7 +799,7 @@ private:
TraceEvent(SevDebug, "SendingPrivatized_TSSClearServerTag", dbgid)
.detail("M", privatized);
toCommit->addTag(decodeServerTagValue(tagV.get()));
toCommit->writeTypedMessage(privatized);
writeMutation(privatized);
}
}
}
@ -950,7 +965,7 @@ private:
ASSERT(rangeToClear.singleKeyRange());
// Normally uses key backed map, so have to use same unpacking code here.
UID ssId = Codec<UID>::unpack(Tuple::unpack(m.param1.removePrefix(tssMappingKeys.begin)));
UID ssId = TupleCodec<UID>::unpack(m.param1.removePrefix(tssMappingKeys.begin));
if (!initialCommit) {
txnStateStore->clear(rangeToClear);
}
@ -969,7 +984,7 @@ private:
privatized.param2 = m.param2.withPrefix(systemKeys.begin, arena);
TraceEvent(SevDebug, "SendingPrivatized_ClearTSSMapping", dbgid).detail("M", privatized);
toCommit->addTag(decodeServerTagValue(tagV.get()));
toCommit->writeTypedMessage(privatized);
writeMutation(privatized);
}
}
@ -996,7 +1011,7 @@ private:
privatized.param2 = m.param2.withPrefix(systemKeys.begin, arena);
TraceEvent(SevDebug, "SendingPrivatized_ClearTSSQuarantine", dbgid).detail("M", privatized);
toCommit->addTag(decodeServerTagValue(tagV.get()));
toCommit->writeTypedMessage(privatized);
writeMutation(privatized);
}
}
}
@ -1050,7 +1065,7 @@ private:
privatized.type = MutationRef::ClearRange;
privatized.param1 = range.begin.withPrefix(systemKeys.begin, arena);
privatized.param2 = range.end.withPrefix(systemKeys.begin, arena);
toCommit->writeTypedMessage(privatized);
writeMutation(privatized);
}
TEST(true); // Tenant cleared from map
@ -1146,9 +1161,9 @@ private:
.detail("MBegin", mutationBegin)
.detail("MEnd", mutationEnd);
toCommit->addTags(allTags);
toCommit->writeTypedMessage(mutationBegin);
writeMutation(mutationBegin);
toCommit->addTags(allTags);
toCommit->writeTypedMessage(mutationEnd);
writeMutation(mutationEnd);
}
}
@ -1223,6 +1238,7 @@ void applyMetadataMutations(SpanContext const& spanContext,
Reference<ILogSystem> logSystem,
const VectorRef<MutationRef>& mutations,
LogPushData* toCommit,
const std::unordered_map<EncryptCipherDomainId, Reference<BlobCipherKey>>* pCipherKeys,
bool& confChange,
Version version,
Version popVersion,
@ -1234,6 +1250,7 @@ void applyMetadataMutations(SpanContext const& spanContext,
proxyCommitData,
logSystem,
toCommit,
pCipherKeys,
confChange,
version,
popVersion,

View File

@ -25,6 +25,8 @@
#include "fdbclient/SystemData.h"
#include "fdbserver/BackupInterface.h"
#include "fdbserver/BackupProgress.actor.h"
#include "fdbserver/EncryptedMutationMessage.h"
#include "fdbserver/GetEncryptCipherKeys.h"
#include "fdbserver/Knobs.h"
#include "fdbserver/LogProtocolMessage.h"
#include "fdbserver/LogSystem.h"
@ -44,6 +46,7 @@ struct VersionedMessage {
StringRef message;
VectorRef<Tag> tags;
Arena arena; // Keep a reference to the memory containing the message
Arena decryptArena; // Arena used for decrypt buffer.
size_t bytes; // arena's size when inserted, which can grow afterwards
VersionedMessage(LogMessageVersion v, StringRef m, const VectorRef<Tag>& t, const Arena& a)
@ -53,7 +56,8 @@ struct VersionedMessage {
// Returns true if the message is a mutation that should be backuped, i.e.,
// either key is not in system key space or is not a metadataVersionKey.
bool isBackupMessage(MutationRef* m) const {
bool isBackupMessage(MutationRef* m,
const std::unordered_map<BlobCipherDetails, Reference<BlobCipherKey>>& cipherKeys) {
for (Tag tag : tags) {
if (tag.locality == tagLocalitySpecial || tag.locality == tagLocalityTxs) {
return false; // skip Txs mutations
@ -71,10 +75,26 @@ struct VersionedMessage {
TEST(true); // Returning false for OTELSpanContextMessage
return false;
}
reader >> *m;
if (EncryptedMutationMessage::isNextIn(reader)) {
// 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 = EncryptedMutationMessage::decrypt(reader, decryptArena, cipherKeys, &message);
} else {
reader >> *m;
}
return normalKeys.contains(m->param1) || m->param1 == metadataVersionKey;
}
void collectCipherDetailIfEncrypted(std::unordered_set<BlobCipherDetails>& cipherDetails) {
ArenaReader reader(arena, message, AssumeVersion(g_network->protocolVersion()));
if (EncryptedMutationMessage::isNextIn(reader)) {
EncryptedMutationMessage emm;
reader >> emm;
cipherDetails.insert(emm.header.cipherTextDetails);
cipherDetails.insert(emm.header.cipherHeaderDetails);
}
}
};
struct BackupData {
@ -89,6 +109,7 @@ struct BackupData {
Version minKnownCommittedVersion;
Version savedVersion; // Largest version saved to blob storage
Version popVersion; // Largest version popped in NOOP mode, can be larger than savedVersion.
Reference<AsyncVar<ServerDBInfo> const> db;
AsyncVar<Reference<ILogSystem>> logSystem;
Database cx;
std::vector<VersionedMessage> messages;
@ -104,8 +125,8 @@ struct BackupData {
PerBackupInfo(BackupData* data, UID uid, Version v) : self(data), startVersion(v) {
// Open the container and get key ranges
BackupConfig config(uid);
container = config.backupContainer().get(data->cx);
ranges = config.backupRanges().get(data->cx);
container = config.backupContainer().get(data->cx.getReference());
ranges = config.backupRanges().get(data->cx.getReference());
if (self->backupEpoch == self->recruitedEpoch) {
// Only current epoch's worker update the number of backup workers.
updateWorker = _updateStartedWorkers(this, data, uid);
@ -245,7 +266,7 @@ struct BackupData {
: myId(id), tag(req.routerTag), totalTags(req.totalTags), startVersion(req.startVersion),
endVersion(req.endVersion), recruitedEpoch(req.recruitedEpoch), backupEpoch(req.backupEpoch),
minKnownCommittedVersion(invalidVersion), savedVersion(req.startVersion - 1), popVersion(req.startVersion - 1),
pulledVersion(0), paused(false), lock(new FlowLock(SERVER_KNOBS->BACKUP_LOCK_BYTES)),
db(db), pulledVersion(0), paused(false), lock(new FlowLock(SERVER_KNOBS->BACKUP_LOCK_BYTES)),
cc("BackupWorker", myId.toString()) {
cx = openDBOnServer(db, TaskPriority::DefaultEndpoint, LockAware::True);
@ -682,7 +703,10 @@ ACTOR static Future<Void> updateLogBytesWritten(BackupData* self,
// Saves messages in the range of [0, numMsg) to a file and then remove these
// messages. The file content format is a sequence of (Version, sub#, msgSize, message).
// Note only ready backups are saved.
ACTOR Future<Void> saveMutationsToFile(BackupData* self, Version popVersion, int numMsg) {
ACTOR Future<Void> saveMutationsToFile(BackupData* self,
Version popVersion,
int numMsg,
std::unordered_set<BlobCipherDetails> cipherDetails) {
state int blockSize = SERVER_KNOBS->BACKUP_FILE_BLOCK_BYTES;
state std::vector<Future<Reference<IBackupFile>>> logFileFutures;
state std::vector<Reference<IBackupFile>> logFiles;
@ -691,6 +715,7 @@ ACTOR Future<Void> saveMutationsToFile(BackupData* self, Version popVersion, int
state std::vector<Version> beginVersions; // logFiles' begin versions
state KeyRangeMap<std::set<int>> keyRangeMap; // range to index in logFileFutures, logFiles, & blockEnds
state std::vector<Standalone<StringRef>> mutations;
state std::unordered_map<BlobCipherDetails, Reference<BlobCipherKey>> cipherKeys;
state int idx;
// Make sure all backups are ready, otherwise mutations will be lost.
@ -742,11 +767,18 @@ ACTOR Future<Void> saveMutationsToFile(BackupData* self, Version popVersion, int
.detail("File", logFiles[i]->getFileName());
}
// 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));
cipherKeys = getCipherKeysResult;
}
blockEnds = std::vector<int64_t>(logFiles.size(), 0);
for (idx = 0; idx < numMsg; idx++) {
const auto& message = self->messages[idx];
auto& message = self->messages[idx];
MutationRef m;
if (!message.isBackupMessage(&m))
if (!message.isBackupMessage(&m, cipherKeys))
continue;
DEBUG_MUTATION("addMutation", message.version.version, m)
@ -815,6 +847,7 @@ ACTOR Future<Void> uploadData(BackupData* self) {
state Future<Void> uploadDelay = delay(SERVER_KNOBS->BACKUP_UPLOAD_DELAY);
state int numMsg = 0;
state std::unordered_set<BlobCipherDetails> cipherDetails;
Version lastPopVersion = popVersion;
// index of last version's end position in self->messages
int lastVersionIndex = 0;
@ -826,7 +859,7 @@ ACTOR Future<Void> uploadData(BackupData* self) {
popVersion = std::max(popVersion, self->minKnownCommittedVersion);
}
} else {
for (const auto& message : self->messages) {
for (auto& message : self->messages) {
// message may be prefetched in peek; uncommitted message should not be uploaded.
const Version version = message.getVersion();
if (version > self->maxPopVersion())
@ -836,6 +869,7 @@ ACTOR Future<Void> uploadData(BackupData* self) {
lastVersion = popVersion;
popVersion = version;
}
message.collectCipherDetailIfEncrypted(cipherDetails);
numMsg++;
}
}
@ -859,7 +893,7 @@ ACTOR Future<Void> uploadData(BackupData* self) {
.detail("NumMsg", numMsg)
.detail("MsgQ", self->messages.size());
// save an empty file for old epochs so that log file versions are continuous
wait(saveMutationsToFile(self, popVersion, numMsg));
wait(saveMutationsToFile(self, popVersion, numMsg, cipherDetails));
self->eraseMessages(numMsg);
}

View File

@ -354,8 +354,9 @@ ACTOR Future<Standalone<VectorRef<KeyRef>>> splitRange(Reference<BlobManagerData
state PromiseStream<Key> resultStream;
state Standalone<VectorRef<KeyRef>> keys;
state Future<Void> streamFuture =
bmData->db->splitStorageMetricsStream(resultStream, range, splitMetrics, estimated);
// SplitMetrics.bytes / 3 as min split size because of same splitThreshold logic above.
state Future<Void> streamFuture = bmData->db->splitStorageMetricsStream(
resultStream, range, splitMetrics, estimated, splitMetrics.bytes / 3);
loop {
try {
Key k = waitNext(resultStream.getFuture());
@ -845,7 +846,7 @@ ACTOR Future<Void> monitorClientRanges(Reference<BlobManagerData> bmData) {
std::vector<Key> prefixes;
for (auto& it : tenantResults) {
TenantNameRef tenantName = it.key.removePrefix(tenantMapPrefix);
TenantMapEntry entry = decodeTenantEntry(it.value);
TenantMapEntry entry = TenantMapEntry::decode(it.value);
tenants.push_back(std::pair(tenantName, entry));
prefixes.push_back(entry.prefix);
}

View File

@ -3198,7 +3198,7 @@ ACTOR Future<Void> monitorTenants(Reference<BlobWorkerData> bwData) {
for (auto& it : tenantResults) {
// FIXME: handle removing/moving tenants!
TenantNameRef tenantName = it.key.removePrefix(tenantMapPrefix);
TenantMapEntry entry = decodeTenantEntry(it.value);
TenantMapEntry entry = TenantMapEntry::decode(it.value);
tenants.push_back(std::pair(tenantName, entry));
}
bwData->tenantData.addTenants(tenants);

View File

@ -1013,11 +1013,6 @@ ACTOR Future<Void> updateLocalityForDcId(Optional<Key> dcId,
if (ver == invalidVersion) {
ver = oldLogSystem->getKnownCommittedVersion();
}
if (SERVER_KNOBS->ENABLE_VERSION_VECTOR_TLOG_UNICAST) {
// Do not try to split peeks between data centers in peekTxns() to recover mem kvstore.
// This recovery optimization won't work in UNICAST mode.
loc.first = -1;
}
locality->set(PeekTxsInfo(loc.first, loc.second, ver));
TraceEvent("UpdatedLocalityForDcId")

View File

@ -34,7 +34,9 @@
#include "fdbserver/ApplyMetadataMutation.h"
#include "fdbserver/ConflictSet.h"
#include "fdbserver/DataDistributorInterface.h"
#include "fdbserver/EncryptedMutationMessage.h"
#include "fdbserver/FDBExecHelper.actor.h"
#include "fdbserver/GetEncryptCipherKeys.h"
#include "fdbserver/IKeyValueStore.h"
#include "fdbserver/Knobs.h"
#include "fdbserver/LogSystem.h"
@ -48,6 +50,7 @@
#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"
@ -641,6 +644,9 @@ struct CommitBatchContext {
std::set<Tag> writtenTags; // final set tags written to in the batch
std::set<Tag> writtenTagsPreResolution; // tags written to in the batch not including any changes from the resolver.
// Cipher keys to be used to encrypt mutations
std::unordered_map<EncryptCipherDomainId, Reference<BlobCipherKey>> cipherKeys;
CommitBatchContext(ProxyCommitData*, const std::vector<CommitTransactionRequest>*, const int);
void setupTraceBatch();
@ -897,6 +903,27 @@ ACTOR Future<Void> getResolution(CommitBatchContext* self) {
self->transactionResolverMap.swap(requests.transactionResolverMap);
// Used to report conflicting keys
self->txReadConflictRangeIndexMap.swap(requests.txReadConflictRangeIndexMap);
// Fetch cipher keys if needed.
state Future<std::unordered_map<EncryptCipherDomainId, Reference<BlobCipherKey>>> getCipherKeys;
if (SERVER_KNOBS->ENABLE_TLOG_ENCRYPTION) {
static std::unordered_map<EncryptCipherDomainId, EncryptCipherDomainName> defaultDomains = {
{ SYSTEM_KEYSPACE_ENCRYPT_DOMAIN_ID, FDB_DEFAULT_ENCRYPT_DOMAIN_NAME },
{ ENCRYPT_HEADER_DOMAIN_ID, FDB_DEFAULT_ENCRYPT_DOMAIN_NAME }
};
std::unordered_map<EncryptCipherDomainId, EncryptCipherDomainName> encryptDomains = defaultDomains;
for (int t = 0; t < trs.size(); t++) {
int64_t tenantId = trs[t].tenantInfo.tenantId;
Optional<TenantName> tenantName = trs[t].tenantInfo.name;
// TODO(yiwu): In raw access mode, use tenant prefix to figure out tenant id for user data
if (tenantId != TenantInfo::INVALID_TENANT) {
ASSERT(tenantName.present());
encryptDomains[tenantId] = tenantName.get();
}
}
getCipherKeys = getLatestEncryptCipherKeys(pProxyCommitData->db, encryptDomains);
}
self->releaseFuture = releaseResolvingAfter(pProxyCommitData, self->releaseDelay, self->localBatchNumber);
if (self->localBatchNumber - self->pProxyCommitData->latestLocalCommitBatchLogging.get() >
@ -922,6 +949,11 @@ ACTOR Future<Void> getResolution(CommitBatchContext* self) {
"CommitDebug", self->debugID.get().first(), "CommitProxyServer.commitBatch.AfterResolution");
}
if (SERVER_KNOBS->ENABLE_TLOG_ENCRYPTION) {
std::unordered_map<EncryptCipherDomainId, Reference<BlobCipherKey>> cipherKeys = wait(getCipherKeys);
self->cipherKeys = cipherKeys;
}
return Void();
}
@ -961,6 +993,7 @@ void applyMetadataEffect(CommitBatchContext* self) {
self->pProxyCommitData->logSystem,
self->resolution[0].stateMutations[versionIndex][transactionIndex].mutations,
/* pToCommit= */ nullptr,
/* pCipherKeys= */ nullptr,
self->forceRecovery,
/* version= */ self->commitVersion,
/* popVersion= */ 0,
@ -1060,6 +1093,7 @@ ACTOR Future<Void> applyMetadataToCommittedTransactions(CommitBatchContext* self
pProxyCommitData->logSystem,
trs[t].transaction.mutations,
SERVER_KNOBS->PROXY_USE_RESOLVER_PRIVATE_MUTATIONS ? nullptr : &self->toCommit,
SERVER_KNOBS->ENABLE_TLOG_ENCRYPTION ? &self->cipherKeys : nullptr,
self->forceRecovery,
self->commitVersion,
self->commitVersion + 1,
@ -1111,6 +1145,22 @@ ACTOR Future<Void> applyMetadataToCommittedTransactions(CommitBatchContext* self
return Void();
}
void writeMutation(CommitBatchContext* self, int64_t tenantId, const MutationRef& mutation) {
static_assert(TenantInfo::INVALID_TENANT == ENCRYPT_INVALID_DOMAIN_ID);
if (!SERVER_KNOBS->ENABLE_TLOG_ENCRYPTION || tenantId == TenantInfo::INVALID_TENANT) {
// TODO(yiwu): In raw access mode, use tenant prefix to figure out tenant id for user data
bool isRawAccess = tenantId == TenantInfo::INVALID_TENANT && !isSystemKey(mutation.param1) &&
!(mutation.type == MutationRef::ClearRange && isSystemKey(mutation.param2)) &&
self->pProxyCommitData->db->get().client.tenantMode == TenantMode::REQUIRED;
TEST(isRawAccess); // Raw access to tenant key space
self->toCommit.writeTypedMessage(mutation);
} else {
Arena arena;
self->toCommit.writeTypedMessage(
EncryptedMutationMessage::encrypt(arena, self->cipherKeys, tenantId /*domainId*/, mutation));
}
}
/// This second pass through committed transactions assigns the actual mutations to the appropriate storage servers'
/// tags
ACTOR Future<Void> assignMutationsToStorageServers(CommitBatchContext* self) {
@ -1127,6 +1177,7 @@ ACTOR Future<Void> assignMutationsToStorageServers(CommitBatchContext* self) {
state Optional<ClientTrCommitCostEstimation>* trCost = &trs[self->transactionNum].commitCostEstimation;
state int mutationNum = 0;
state VectorRef<MutationRef>* pMutations = &trs[self->transactionNum].transaction.mutations;
state int64_t tenantId = trs[self->transactionNum].tenantInfo.tenantId;
self->toCommit.addTransactionInfo(trs[self->transactionNum].spanContext);
@ -1184,7 +1235,7 @@ ACTOR Future<Void> assignMutationsToStorageServers(CommitBatchContext* self) {
if (pProxyCommitData->cacheInfo[m.param1]) {
self->toCommit.addTag(cacheTag);
}
self->toCommit.writeTypedMessage(m);
writeMutation(self, tenantId, m);
} else if (m.type == MutationRef::ClearRange) {
KeyRangeRef clearRange(KeyRangeRef(m.param1, m.param2));
auto ranges = pProxyCommitData->keyInfo.intersectingRanges(clearRange);
@ -1237,7 +1288,7 @@ ACTOR Future<Void> assignMutationsToStorageServers(CommitBatchContext* self) {
if (pProxyCommitData->needsCacheTag(clearRange)) {
self->toCommit.addTag(cacheTag);
}
self->toCommit.writeTypedMessage(m);
writeMutation(self, tenantId, m);
} else {
UNREACHABLE();
}
@ -2308,6 +2359,7 @@ ACTOR Future<Void> processCompleteTransactionStateRequest(TransactionStateResolv
Reference<ILogSystem>(),
mutations,
/* pToCommit= */ nullptr,
/* pCipherKeys= */ nullptr,
confChanges,
/* version= */ 0,
/* popVersion= */ 0,
@ -2387,7 +2439,6 @@ ACTOR Future<Void> commitProxyServerCore(CommitProxyInterface proxy,
state PromiseStream<Future<Void>> addActor;
state Future<Void> onError = transformError(actorCollection(addActor.getFuture()), broken_promise(), tlog_failed());
state double lastCommit = 0;
state GetHealthMetricsReply healthMetricsReply;
state GetHealthMetricsReply detailedHealthMetricsReply;
@ -2399,7 +2450,8 @@ ACTOR Future<Void> commitProxyServerCore(CommitProxyInterface proxy,
// Wait until we can load the "real" logsystem, since we don't support switching them currently
while (!(masterLifetime.isEqual(commitData.db->get().masterLifetime) &&
commitData.db->get().recoveryState >= RecoveryState::RECOVERY_TRANSACTION)) {
commitData.db->get().recoveryState >= RecoveryState::RECOVERY_TRANSACTION &&
(!SERVER_KNOBS->ENABLE_TLOG_ENCRYPTION || commitData.db->get().encryptKeyProxy.present()))) {
//TraceEvent("ProxyInit2", proxy.id()).detail("LSEpoch", db->get().logSystemConfig.epoch).detail("Need", epoch);
wait(commitData.db->onChange());
}
@ -2488,9 +2540,7 @@ ACTOR Future<Void> commitProxyServerCore(CommitProxyInterface proxy,
//TraceEvent("CommitProxyCTR", proxy.id()).detail("CommitTransactions", trs.size()).detail("TransactionRate", transactionRate).detail("TransactionQueue", transactionQueue.size()).detail("ReleasedTransactionCount", transactionCount);
//TraceEvent("CommitProxyCore", commitData.dbgid).detail("TxSize", trs.size()).detail("MasterLifetime", masterLifetime.toString()).detail("DbMasterLifetime", commitData.db->get().masterLifetime.toString()).detail("RecoveryState", commitData.db->get().recoveryState).detail("CCInf", commitData.db->get().clusterInterface.id().toString());
if (trs.size() || (commitData.db->get().recoveryState >= RecoveryState::ACCEPTING_COMMITS &&
masterLifetime.isEqual(commitData.db->get().masterLifetime) &&
now() - lastCommit >= SERVER_KNOBS->MAX_COMMIT_BATCH_INTERVAL)) {
lastCommit = now();
masterLifetime.isEqual(commitData.db->get().masterLifetime))) {
if (trs.size() || lastCommitComplete.isReady()) {
lastCommitComplete = transformError(

View File

@ -380,7 +380,7 @@ ACTOR Future<Standalone<VectorRef<KeyRef>>> getSplitKeys(DataDistributionTracker
state Transaction tr(self->cx);
try {
Standalone<VectorRef<KeyRef>> keys =
wait(self->cx->splitStorageMetrics(splitRange, splitMetrics, estimated));
wait(self->cx->splitStorageMetrics(splitRange, splitMetrics, estimated, SERVER_KNOBS->MIN_SHARD_BYTES));
return keys;
} catch (Error& e) {
wait(tr.onError(e));

View File

@ -42,6 +42,7 @@
#include "flow/genericactors.actor.h"
#include "flow/network.h"
#include <boost/functional/hash.hpp>
#include <boost/mpl/not.hpp>
#include <limits>
#include <string>

View File

@ -1,5 +1,5 @@
/*
* GetCipherKeys.actor.cpp
* GetEncryptCipherKeys.actor.cpp
*
* This source file is part of the FoundationDB open source project
*
@ -38,7 +38,7 @@ ACTOR Future<Void> onEncryptKeyProxyChange(Reference<AsyncVar<ServerDBInfo> cons
break;
}
}
TraceEvent("GetCipherKeys_EncryptKeyProxyChanged")
TraceEvent("GetEncryptCipherKeys_EncryptKeyProxyChanged")
.detail("PreviousProxyId", previousProxyId.orDefault(UID()))
.detail("CurrentProxyId", currentProxyId.orDefault(UID()));
return Void();
@ -50,19 +50,19 @@ ACTOR Future<EKPGetLatestBaseCipherKeysReply> getUncachedLatestEncryptCipherKeys
Optional<EncryptKeyProxyInterface> proxy = db->get().encryptKeyProxy;
if (!proxy.present()) {
// Wait for onEncryptKeyProxyChange.
TraceEvent("GetLatestCipherKeys_EncryptKeyProxyNotPresent");
TraceEvent("GetLatestEncryptCipherKeys_EncryptKeyProxyNotPresent");
return Never();
}
request.reply.reset();
try {
EKPGetLatestBaseCipherKeysReply reply = wait(proxy.get().getLatestBaseCipherKeys.getReply(request));
if (reply.error.present()) {
TraceEvent(SevWarn, "GetLatestCipherKeys_RequestFailed").error(reply.error.get());
TraceEvent(SevWarn, "GetLatestEncryptCipherKeys_RequestFailed").error(reply.error.get());
throw encrypt_keys_fetch_failed();
}
return reply;
} catch (Error& e) {
TraceEvent("GetLatestCipherKeys_CaughtError").error(e);
TraceEvent("GetLatestEncryptCipherKeys_CaughtError").error(e);
if (e.code() == error_code_broken_promise) {
// Wait for onEncryptKeyProxyChange.
return Never();
@ -81,7 +81,7 @@ ACTOR Future<std::unordered_map<EncryptCipherDomainId, Reference<BlobCipherKey>>
state EKPGetLatestBaseCipherKeysRequest request;
if (!db.isValid()) {
TraceEvent(SevError, "GetLatestCipherKeys_ServerDBInfoNotAvailable");
TraceEvent(SevError, "GetLatestEncryptCipherKeys_ServerDBInfoNotAvailable");
throw encrypt_ops_error();
}
@ -114,7 +114,7 @@ ACTOR Future<std::unordered_map<EncryptCipherDomainId, Reference<BlobCipherKey>>
// Check for any missing cipher keys.
for (auto& domain : request.encryptDomainInfos) {
if (cipherKeys.count(domain.domainId) == 0) {
TraceEvent(SevWarn, "GetLatestCipherKeys_KeyMissing").detail("DomainId", domain.domainId);
TraceEvent(SevWarn, "GetLatestEncryptCipherKeys_KeyMissing").detail("DomainId", domain.domainId);
throw encrypt_key_not_found();
}
}
@ -133,19 +133,19 @@ ACTOR Future<EKPGetBaseCipherKeysByIdsReply> getUncachedEncryptCipherKeys(Refere
Optional<EncryptKeyProxyInterface> proxy = db->get().encryptKeyProxy;
if (!proxy.present()) {
// Wait for onEncryptKeyProxyChange.
TraceEvent("GetCipherKeys_EncryptKeyProxyNotPresent");
TraceEvent("GetEncryptCipherKeys_EncryptKeyProxyNotPresent");
return Never();
}
request.reply.reset();
try {
EKPGetBaseCipherKeysByIdsReply reply = wait(proxy.get().getBaseCipherKeysByIds.getReply(request));
if (reply.error.present()) {
TraceEvent(SevWarn, "GetCipherKeys_RequestFailed").error(reply.error.get());
TraceEvent(SevWarn, "GetEncryptCipherKeys_RequestFailed").error(reply.error.get());
throw encrypt_keys_fetch_failed();
}
return reply;
} catch (Error& e) {
TraceEvent("GetCipherKeys_CaughtError").error(e);
TraceEvent("GetEncryptCipherKeys_CaughtError").error(e);
if (e.code() == error_code_broken_promise) {
// Wait for onEncryptKeyProxyChange.
return Never();
@ -167,7 +167,7 @@ ACTOR Future<std::unordered_map<BlobCipherDetails, Reference<BlobCipherKey>>> ge
state EKPGetBaseCipherKeysByIdsRequest request;
if (!db.isValid()) {
TraceEvent(SevError, "GetCipherKeys_ServerDBInfoNotAvailable");
TraceEvent(SevError, "GetEncryptCipherKeys_ServerDBInfoNotAvailable");
throw encrypt_ops_error();
}
@ -204,7 +204,7 @@ ACTOR Future<std::unordered_map<BlobCipherDetails, Reference<BlobCipherKey>>> ge
BaseCipherIndex baseIdx = std::make_pair(details.encryptDomainId, details.baseCipherId);
const auto& itr = baseCipherKeys.find(baseIdx);
if (itr == baseCipherKeys.end()) {
TraceEvent(SevError, "GetCipherKeys_KeyMissing")
TraceEvent(SevError, "GetEncryptCipherKeys_KeyMissing")
.detail("DomainId", details.encryptDomainId)
.detail("BaseCipherId", details.baseCipherId);
throw encrypt_key_not_found();

View File

@ -356,7 +356,7 @@ ACTOR static Future<Void> monitorClientRates(GlobalTagThrottler* globalTagThrott
.detail("Tag", tag)
.detail("CurrentTPSRate", currentTPSLimit.get())
.detail("DesiredTPSRate", desiredTPSLimit);
if (abs(currentTPSLimit.get() - desiredTPSLimit) < 0.1) {
if (abs(currentTPSLimit.get() - desiredTPSLimit) < 1.0) {
if (++successes == 3) {
return Void();
}

View File

@ -1538,7 +1538,7 @@ void seedShardServers(Arena& arena, CommitTransactionRef& tr, std::vector<Storag
// THIS SHOULD NEVER BE ENABLED IN ANY NON-TESTING ENVIRONMENT
TraceEvent(SevError, "TSSIdentityMappingEnabled").log();
// hack key-backed map here since we can't really change CommitTransactionRef to a RYW transaction
Key uidRef = Codec<UID>::pack(s.id()).pack();
Key uidRef = TupleCodec<UID>::pack(s.id());
tr.set(arena, uidRef.withPrefix(tssMappingKeys.begin), uidRef);
}
}

View File

@ -21,6 +21,7 @@
#include <algorithm>
#include <vector>
#include "fdbclient/FDBTypes.h"
#include "fdbserver/EncryptedMutationMessage.h"
#include "fdbserver/MutationTracking.h"
#include "fdbserver/LogProtocolMessage.h"
#include "fdbserver/SpanContextMessage.h"
@ -102,6 +103,8 @@ TraceEvent debugTagsAndMessageEnabled(const char* context, Version version, Stri
BinaryReader br(mutationData, AssumeVersion(rdr.protocolVersion()));
OTELSpanContextMessage scm;
br >> scm;
} else if (EncryptedMutationMessage::startsEncryptedMutationMessage(mutationType)) {
throw encrypt_unsupported();
} else {
MutationRef m;
BinaryReader br(mutationData, AssumeVersion(rdr.protocolVersion()));

View File

@ -762,7 +762,7 @@ ACTOR static Future<Void> handleApplyToDBRequest(RestoreVersionBatchRequest req,
ASSERT(batchData->dbApplier.present());
ASSERT(!batchData->dbApplier.get().isError()); // writeMutationsToDB actor cannot have error.
// We cannot blindly retry because it is not idempodent
// We cannot blindly retry because it is not idempotent
wait(batchData->dbApplier.get());

View File

@ -24,6 +24,7 @@
#include "flow/UnitTest.h"
#include "fdbclient/BackupContainer.h"
#include "fdbclient/BackupAgent.actor.h"
#include "fdbserver/EncryptedMutationMessage.h"
#include "fdbserver/RestoreLoader.actor.h"
#include "fdbserver/RestoreRoleCommon.actor.h"
#include "fdbserver/MutationTracking.h"
@ -422,6 +423,9 @@ ACTOR static Future<Void> _parsePartitionedLogFileOnLoader(
ASSERT(inserted);
ArenaReader rd(buf.arena(), StringRef(message, msgSize), AssumeVersion(g_network->protocolVersion()));
if (EncryptedMutationMessage::isNextIn(rd)) {
throw encrypt_unsupported();
}
MutationRef mutation;
rd >> mutation;

View File

@ -139,7 +139,7 @@ ACTOR Future<Void> ekLookupByDomainIds(Reference<SimKmsConnectorContext> ctx,
req.debugId.present() ? TraceEvent("SimKmsGetsByDomIds", interf.id()) : Optional<TraceEvent>();
if (dbgDIdTrace.present()) {
dbgDIdTrace.get().detail("DbgId", req.debugId.get());
dbgDIdTrace.get().setMaxEventLength(16384).detail("DbgId", req.debugId.get());
}
// Map encryptionDomainId to corresponding EncryptKeyCtx element using a modulo operation. This

View File

@ -2793,19 +2793,11 @@ ACTOR Future<Optional<Value>> getActivePrimaryDC(Database cx, int* fullyReplicat
}
}
// read storageWigglerStats through Read-only tx, then convert it to JSON field
ACTOR Future<JsonBuilderObject> storageWigglerStatsFetcher(Optional<DataDistributorInterface> ddWorker,
DatabaseConfiguration conf,
Database cx,
bool use_system_priority) {
ACTOR Future<std::pair<Optional<Value>, Optional<Value>>> readStorageWiggleMetrics(Database cx,
bool use_system_priority) {
state Reference<ReadYourWritesTransaction> tr(new ReadYourWritesTransaction(cx));
state Optional<Value> primaryV;
state Optional<Value> remoteV;
state Future<ErrorOr<GetStorageWigglerStateReply>> stateFut;
if (ddWorker.present()) {
stateFut = ddWorker.get().storageWigglerState.tryGetReply(GetStorageWigglerStateRequest());
}
loop {
try {
if (use_system_priority) {
@ -2813,42 +2805,59 @@ ACTOR Future<JsonBuilderObject> storageWigglerStatsFetcher(Optional<DataDistribu
}
wait(store(primaryV, StorageWiggleMetrics::runGetTransaction(tr, true)) &&
store(remoteV, StorageWiggleMetrics::runGetTransaction(tr, false)));
wait(tr->commit());
break;
return std::make_pair(primaryV, remoteV);
} catch (Error& e) {
wait(tr->onError(e));
}
}
}
// read storageWigglerStats through Read-only tx, then convert it to JSON field
ACTOR Future<JsonBuilderObject> storageWigglerStatsFetcher(Optional<DataDistributorInterface> ddWorker,
DatabaseConfiguration conf,
Database cx,
bool use_system_priority,
JsonBuilderArray* messages) {
state Future<GetStorageWigglerStateReply> stateFut;
state Future<std::pair<Optional<Value>, Optional<Value>>> wiggleMetricsFut =
timeoutError(readStorageWiggleMetrics(cx, use_system_priority), 2.0);
state JsonBuilderObject res;
if (ddWorker.present()) {
stateFut = timeoutError(ddWorker.get().storageWigglerState.getReply(GetStorageWigglerStateRequest()), 2.0);
wait(ready(stateFut));
} else {
return res;
}
JsonBuilderObject res;
if (primaryV.present()) {
auto obj = ObjectReader::fromStringRef<StorageWiggleMetrics>(primaryV.get(), IncludeVersion()).toJSON();
if (stateFut.canGet() && stateFut.get().present()) {
auto& reply = stateFut.get().get();
try {
if (g_network->isSimulated() && BUGGIFY_WITH_PROB(0.01)) {
throw timed_out();
}
wait(success(wiggleMetricsFut) && success(stateFut));
auto [primaryV, remoteV] = wiggleMetricsFut.get();
if (primaryV.present()) {
auto obj = ObjectReader::fromStringRef<StorageWiggleMetrics>(primaryV.get(), IncludeVersion()).toJSON();
auto& reply = stateFut.get();
obj["state"] = StorageWiggler::getWiggleStateStr(static_cast<StorageWiggler::State>(reply.primary));
obj["last_state_change_timestamp"] = reply.lastStateChangePrimary;
obj["last_state_change_datetime"] = epochsToGMTString(reply.lastStateChangePrimary);
res["primary"] = obj;
}
res["primary"] = obj;
}
if (conf.regions.size() > 1 && remoteV.present()) {
auto obj = ObjectReader::fromStringRef<StorageWiggleMetrics>(remoteV.get(), IncludeVersion()).toJSON();
if (stateFut.canGet() && stateFut.get().present()) {
auto& reply = stateFut.get().get();
if (conf.regions.size() > 1 && remoteV.present()) {
auto obj = ObjectReader::fromStringRef<StorageWiggleMetrics>(remoteV.get(), IncludeVersion()).toJSON();
auto& reply = stateFut.get();
obj["state"] = StorageWiggler::getWiggleStateStr(static_cast<StorageWiggler::State>(reply.remote));
obj["last_state_change_timestamp"] = reply.lastStateChangeRemote;
obj["last_state_change_datetime"] = epochsToGMTString(reply.lastStateChangeRemote);
res["remote"] = obj;
}
res["remote"] = obj;
}
if (stateFut.canGet() && stateFut.isError()) {
res["error"] = std::string("Can't get storage wiggler state: ") + stateFut.getError().name();
TraceEvent(SevWarn, "StorageWigglerStatsFetcher").error(stateFut.getError());
} else if (stateFut.canGet() && stateFut.get().isError()) {
res["error"] = std::string("Can't get storage wiggler state: ") + stateFut.get().getError().name();
TraceEvent(SevWarn, "StorageWigglerStatsFetcher").error(stateFut.get().getError());
return res;
} catch (Error& e) {
if (e.code() == error_code_actor_cancelled)
throw;
messages->push_back(JsonString::makeMessage("fetch_storage_wiggler_stats_timeout",
"Fetching storage wiggler stats timed out."));
}
return res;
}
@ -3096,17 +3105,29 @@ ACTOR Future<StatusReply> clusterGetStatus(
if (configuration.get().perpetualStorageWiggleSpeed > 0) {
state Future<std::vector<std::pair<UID, StorageWiggleValue>>> primaryWiggleValues;
state Future<std::vector<std::pair<UID, StorageWiggleValue>>> remoteWiggleValues;
double timeout = g_network->isSimulated() && BUGGIFY_WITH_PROB(0.01) ? 0.0 : 2.0;
primaryWiggleValues = timeoutError(readStorageWiggleValues(cx, true, true), timeout);
remoteWiggleValues = timeoutError(readStorageWiggleValues(cx, false, true), timeout);
wait(store(
storageWiggler,
storageWigglerStatsFetcher(db->get().distributor, configuration.get(), cx, true, &messages)) &&
ready(primaryWiggleValues) && ready(remoteWiggleValues));
primaryWiggleValues = readStorageWiggleValues(cx, true, true);
remoteWiggleValues = readStorageWiggleValues(cx, false, true);
wait(store(storageWiggler,
storageWigglerStatsFetcher(db->get().distributor, configuration.get(), cx, true)) &&
success(primaryWiggleValues) && success(remoteWiggleValues));
for (auto& p : primaryWiggleValues.get())
wiggleServers.insert(p.first);
for (auto& p : remoteWiggleValues.get())
wiggleServers.insert(p.first);
if (primaryWiggleValues.canGet()) {
for (auto& p : primaryWiggleValues.get())
wiggleServers.insert(p.first);
} else {
messages.push_back(
JsonString::makeMessage("fetch_storage_wiggler_stats_timeout",
"Fetching wiggling servers in primary region timed out"));
}
if (remoteWiggleValues.canGet()) {
for (auto& p : remoteWiggleValues.get())
wiggleServers.insert(p.first);
} else {
messages.push_back(JsonString::makeMessage("fetch_storage_wiggler_stats_timeout",
"Fetching wiggling servers in remote region timed out"));
}
}
state std::vector<JsonBuilderObject> workerStatuses = wait(getAll(futures2));

View File

@ -23,6 +23,8 @@
#include "fdbclient/FDBOptions.g.h"
#include "fdbclient/NativeAPI.actor.h"
#include "fdbclient/SystemData.h"
#include "fdbserver/EncryptedMutationMessage.h"
#include "fdbserver/GetEncryptCipherKeys.h"
#include "fdbserver/Knobs.h"
#include "fdbserver/ServerDBInfo.h"
#include "fdbclient/StorageServerInterface.h"
@ -1874,6 +1876,9 @@ ACTOR Future<Void> pullAsyncData(StorageCacheData* data) {
state FetchInjectionInfo fii;
state Reference<ILogSystem::IPeekCursor> cloneCursor2;
state Optional<std::unordered_map<BlobCipherDetails, Reference<BlobCipherKey>>> cipherKeys;
state bool collectingCipherKeys = false;
// If encrypted mutation is encountered, we collect cipher details and fetch cipher keys, then start over.
loop {
state uint64_t changeCounter = data->cacheRangeChangeCounter;
bool epochEnd = false;
@ -1881,6 +1886,8 @@ ACTOR Future<Void> pullAsyncData(StorageCacheData* data) {
bool firstMutation = true;
bool dbgLastMessageWasProtocol = false;
std::unordered_set<BlobCipherDetails> cipherDetails;
Reference<ILogSystem::IPeekCursor> cloneCursor1 = cursor->cloneNoMore();
cloneCursor2 = cursor->cloneNoMore();
@ -1904,36 +1911,60 @@ ACTOR Future<Void> pullAsyncData(StorageCacheData* data) {
OTELSpanContextMessage::isNextIn(cloneReader)) {
OTELSpanContextMessage scm;
cloneReader >> scm;
} else if (cloneReader.protocolVersion().hasEncryptionAtRest() &&
EncryptedMutationMessage::isNextIn(cloneReader) && !cipherKeys.present()) {
// Encrypted mutation found, but cipher keys haven't been fetch.
// Collect cipher details to fetch cipher keys in one batch.
EncryptedMutationMessage emm;
cloneReader >> emm;
cipherDetails.insert(emm.header.cipherTextDetails);
cipherDetails.insert(emm.header.cipherHeaderDetails);
collectingCipherKeys = true;
} else {
MutationRef msg;
cloneReader >> msg;
if (firstMutation && msg.param1.startsWith(systemKeys.end))
hasPrivateData = true;
firstMutation = false;
if (msg.param1 == lastEpochEndPrivateKey) {
epochEnd = true;
// ASSERT(firstMutation);
ASSERT(dbgLastMessageWasProtocol);
if (cloneReader.protocolVersion().hasEncryptionAtRest() &&
EncryptedMutationMessage::isNextIn(cloneReader)) {
assert(cipherKeys.present());
msg = EncryptedMutationMessage::decrypt(cloneReader, cloneReader.arena(), cipherKeys.get());
} else {
cloneReader >> msg;
}
dbgLastMessageWasProtocol = false;
if (!collectingCipherKeys) {
if (firstMutation && msg.param1.startsWith(systemKeys.end))
hasPrivateData = true;
firstMutation = false;
if (msg.param1 == lastEpochEndPrivateKey) {
epochEnd = true;
// ASSERT(firstMutation);
ASSERT(dbgLastMessageWasProtocol);
}
dbgLastMessageWasProtocol = false;
}
}
}
// Any fetchKeys which are ready to transition their cacheRanges to the adding,transferred state do so
// now. If there is an epoch end we skip this step, to increase testability and to prevent inserting a
// version in the middle of a rolled back version range.
while (!hasPrivateData && !epochEnd && !data->readyFetchKeys.empty()) {
auto fk = data->readyFetchKeys.back();
data->readyFetchKeys.pop_back();
fk.send(&fii);
if (collectingCipherKeys) {
std::unordered_map<BlobCipherDetails, Reference<BlobCipherKey>> result =
wait(getEncryptCipherKeys(data->db, cipherDetails));
cipherKeys = result;
collectingCipherKeys = false;
} else {
// Any fetchKeys which are ready to transition their cacheRanges to the adding,transferred state do
// so now. If there is an epoch end we skip this step, to increase testability and to prevent
// inserting a version in the middle of a rolled back version range.
while (!hasPrivateData && !epochEnd && !data->readyFetchKeys.empty()) {
auto fk = data->readyFetchKeys.back();
data->readyFetchKeys.pop_back();
fk.send(&fii);
}
if (data->cacheRangeChangeCounter == changeCounter)
break;
// TEST(true); // A fetchKeys completed while we were doing this, so eager might be outdated. Read
// it again.
}
if (data->cacheRangeChangeCounter == changeCounter)
break;
// TEST(true); // A fetchKeys completed while we were doing this, so eager might be outdated. Read it
// again.
}
data->debug_inApplyUpdate = true;
@ -1988,7 +2019,11 @@ ACTOR Future<Void> pullAsyncData(StorageCacheData* data) {
reader >> oscm;
} else {
MutationRef msg;
reader >> msg;
if (reader.protocolVersion().hasEncryptionAtRest() && EncryptedMutationMessage::isNextIn(reader)) {
msg = EncryptedMutationMessage::decrypt(reader, reader.arena(), cipherKeys.get());
} else {
reader >> msg;
}
if (ver != invalidVersion) // This change belongs to a version < minVersion
{

View File

@ -28,7 +28,6 @@
#include "fdbclient/FDBTypes.h"
#include "fdbclient/ManagementAPI.actor.h"
#include "fdbserver/WorkerInterface.actor.h"
#include "fdbserver/LogProtocolMessage.h"
#include "fdbserver/SpanContextMessage.h"
#include "fdbserver/TLogInterface.h"
#include "fdbserver/Knobs.h"

View File

@ -44,8 +44,8 @@ ACTOR Future<Void> readTSSMapping(Transaction* tr, std::map<UID, StorageServerIn
ASSERT(!mappingList.more && mappingList.size() < CLIENT_KNOBS->TOO_MANY);
for (auto& it : mappingList) {
state UID ssId = Codec<UID>::unpack(Tuple::unpack(it.key.removePrefix(tssMappingKeys.begin)));
UID tssId = Codec<UID>::unpack(Tuple::unpack(it.value));
state UID ssId = TupleCodec<UID>::unpack(it.key.removePrefix(tssMappingKeys.begin));
UID tssId = TupleCodec<UID>::unpack(it.value);
Optional<Value> v = wait(tr->get(serverListKeyFor(tssId)));
(*tssMapping)[ssId] = decodeServerListValue(v.get());
}

View File

@ -48,7 +48,7 @@ public:
for (int i = 0; i < tenantList.size(); i++) {
TenantName tname = tenantList[i].key.removePrefix(tenantMapPrefix);
TenantMapEntry t = decodeTenantEntry(tenantList[i].value);
TenantMapEntry t = TenantMapEntry::decode(tenantList[i].value);
tenantCache->insert(tname, t);
@ -86,7 +86,7 @@ public:
for (int i = 0; i < tenantList.size(); i++) {
TenantName tname = tenantList[i].key.removePrefix(tenantMapPrefix);
TenantMapEntry t = decodeTenantEntry(tenantList[i].value);
TenantMapEntry t = TenantMapEntry::decode(tenantList[i].value);
if (tenantCache->update(tname, t)) {
tenantListUpdated = true;

View File

@ -33,6 +33,7 @@
#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.
@ -93,6 +94,7 @@ void applyMetadataMutations(SpanContext const& spanContext,
Reference<ILogSystem> logSystem,
const VectorRef<MutationRef>& mutations,
LogPushData* pToCommit,
const std::unordered_map<EncryptCipherDomainId, Reference<BlobCipherKey>>* pCipherKeys,
bool& confChange,
Version version,
Version popVersion,

View File

@ -0,0 +1,117 @@
/*
* EncryptedMutationMessage.h
*
* This source file is part of the FoundationDB open source project
*
* Copyright 2013-2022 Apple Inc. and the FoundationDB project authors
*
* Licensed under the Apache License, Version 2.0 (the "License");
* you may not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
#ifndef FDBSERVER_ENCRYPTEDMUTATIONMESSAGE_H
#define FDBSERVER_ENCRYPTEDMUTATIONMESSAGE_H
#pragma once
#include "fdbclient/CommitTransaction.h"
#include "flow/BlobCipher.h"
struct EncryptedMutationMessage {
BlobCipherEncryptHeader header;
StringRef encrypted;
EncryptedMutationMessage() {}
std::string toString() const {
return format("code: %d, encryption info: %s",
MutationRef::Reserved_For_EncryptedMutationMessage,
header.toString().c_str());
}
template <class Ar>
void serialize(Ar& ar) {
uint8_t poly = MutationRef::Reserved_For_EncryptedMutationMessage;
serializer(ar, poly, header, encrypted);
}
static bool startsEncryptedMutationMessage(uint8_t byte) {
return byte == MutationRef::Reserved_For_EncryptedMutationMessage;
}
template <class Ar>
static bool isNextIn(Ar& ar) {
return startsEncryptedMutationMessage(*(const uint8_t*)ar.peekBytes(1));
}
// Encrypt given mutation and return an EncryptedMutationMessage.
static EncryptedMutationMessage encrypt(
Arena& arena,
const std::unordered_map<EncryptCipherDomainId, Reference<BlobCipherKey>>& cipherKeys,
const EncryptCipherDomainId& domainId,
const MutationRef& mutation) {
ASSERT_NE(domainId, ENCRYPT_INVALID_DOMAIN_ID);
auto textCipherItr = cipherKeys.find(domainId);
auto headerCipherItr = cipherKeys.find(ENCRYPT_HEADER_DOMAIN_ID);
ASSERT(textCipherItr != cipherKeys.end() && textCipherItr->second.isValid());
ASSERT(headerCipherItr != cipherKeys.end() && headerCipherItr->second.isValid());
uint8_t iv[AES_256_IV_LENGTH];
generateRandomData(iv, AES_256_IV_LENGTH);
BinaryWriter bw(AssumeVersion(g_network->protocolVersion()));
bw << mutation;
EncryptedMutationMessage encrypted_mutation;
EncryptBlobCipherAes265Ctr cipher(textCipherItr->second,
headerCipherItr->second,
iv,
AES_256_IV_LENGTH,
ENCRYPT_HEADER_AUTH_TOKEN_MODE_SINGLE);
encrypted_mutation.encrypted =
cipher
.encrypt(static_cast<const uint8_t*>(bw.getData()), bw.getLength(), &encrypted_mutation.header, arena)
->toStringRef();
return encrypted_mutation;
}
// Encrypt system key space mutation and return an EncryptedMutationMessage.
static EncryptedMutationMessage encryptMetadata(
Arena& arena,
const std::unordered_map<EncryptCipherDomainId, Reference<BlobCipherKey>>& cipherKeys,
const MutationRef& mutation) {
return encrypt(arena, cipherKeys, SYSTEM_KEYSPACE_ENCRYPT_DOMAIN_ID, mutation);
}
// Read an EncryptedMutationMessage from given reader, decrypt and return the encrypted mutation.
// Also return decrypt buffer through buf, if it is specified.
template <class Ar>
static MutationRef decrypt(Ar& ar,
Arena& arena,
const std::unordered_map<BlobCipherDetails, Reference<BlobCipherKey>>& cipherKeys,
StringRef* buf = nullptr) {
EncryptedMutationMessage msg;
ar >> msg;
auto textCipherItr = cipherKeys.find(msg.header.cipherTextDetails);
auto headerCipherItr = cipherKeys.find(msg.header.cipherHeaderDetails);
ASSERT(textCipherItr != cipherKeys.end() && textCipherItr->second.isValid());
ASSERT(headerCipherItr != cipherKeys.end() && headerCipherItr->second.isValid());
DecryptBlobCipherAes256Ctr cipher(textCipherItr->second, headerCipherItr->second, msg.header.iv);
StringRef plaintext =
cipher.decrypt(msg.encrypted.begin(), msg.encrypted.size(), msg.header, arena)->toStringRef();
if (buf != nullptr) {
*buf = plaintext;
}
ArenaReader reader(arena, plaintext, AssumeVersion(g_network->protocolVersion()));
MutationRef mutation;
reader >> mutation;
return mutation;
}
};
#endif

View File

@ -1,5 +1,5 @@
/*
* GetCipherKeys.h
* GetEncryptCipherKeys.h
*
* This source file is part of the FoundationDB open source project
*

View File

@ -357,6 +357,7 @@ struct StorageServerMetrics {
}
void splitMetrics(SplitMetricsRequest req) const {
int minSplitBytes = req.minSplitBytes.present() ? req.minSplitBytes.get() : SERVER_KNOBS->MIN_SHARD_BYTES;
try {
SplitMetricsReply reply;
KeyRef lastKey = req.keys.begin;
@ -364,10 +365,10 @@ struct StorageServerMetrics {
StorageMetrics estimated = req.estimated;
StorageMetrics remaining = getMetrics(req.keys) + used;
//TraceEvent("SplitMetrics").detail("Begin", req.keys.begin).detail("End", req.keys.end).detail("Remaining", remaining.bytes).detail("Used", used.bytes);
//TraceEvent("SplitMetrics").detail("Begin", req.keys.begin).detail("End", req.keys.end).detail("Remaining", remaining.bytes).detail("Used", used.bytes).detail("MinSplitBytes", minSplitBytes);
while (true) {
if (remaining.bytes < 2 * SERVER_KNOBS->MIN_SHARD_BYTES)
if (remaining.bytes < 2 * minSplitBytes)
break;
KeyRef key = req.keys.end;
bool hasUsed = used.bytes != 0 || used.bytesPerKSecond != 0 || used.iosPerKSecond != 0;
@ -382,10 +383,9 @@ struct StorageServerMetrics {
lastKey,
key,
hasUsed);
if (used.bytes < SERVER_KNOBS->MIN_SHARD_BYTES)
key = std::max(key,
byteSample.splitEstimate(KeyRangeRef(lastKey, req.keys.end),
SERVER_KNOBS->MIN_SHARD_BYTES - used.bytes));
if (used.bytes < minSplitBytes)
key = std::max(
key, byteSample.splitEstimate(KeyRangeRef(lastKey, req.keys.end), minSplitBytes - used.bytes));
key = getSplitKey(remaining.iosPerKSecond,
estimated.iosPerKSecond,
req.limits.iosPerKSecond,

View File

@ -32,7 +32,6 @@
#include "fdbrpc/simulator.h"
#include "fdbserver/DBCoreState.h"
#include "fdbserver/Knobs.h"
#include "fdbserver/LogProtocolMessage.h"
#include "fdbserver/LogSystem.h"
#include "fdbserver/RecoveryState.h"
#include "fdbserver/ServerDBInfo.h"

View File

@ -1,321 +0,0 @@
/*
* TPCCWorkload.h
*
* This source file is part of the FoundationDB open source project
*
* Copyright 2013-2022 Apple Inc. and the FoundationDB project authors
*
* Licensed under the Apache License, Version 2.0 (the "License");
* you may not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
#ifndef FDBSERVER_TPCCWORKLOAD_H
#define FDBSERVER_TPCCWORKLOAD_H
#pragma once
#include "flow/Arena.h"
#include "fdbclient/FDBTypes.h"
#include <boost/preprocessor.hpp>
#include <iomanip>
namespace TPCCWorkload {
// Schema
#define EXPAND(...) __VA_ARGS__
#define EMPTY()
#define DEFER(x) x EMPTY()
// An indirection macro to avoid direct recursion
#define BOOST_PP_SEQ_FOR_EACH_ID() BOOST_PP_SEQ_FOR_EACH generators
#define ROW_CONCAT(prefix, name) prefix##name
#define ROW_TO_STRING(str) #str
#define ROW_ELEMENT_NAME(prefix, element) ROW_CONCAT(prefix, element)
#define ROW_MEMBER(r, data, elem) \
BOOST_PP_TUPLE_ELEM(0, elem) \
ROW_ELEMENT_NAME(data, BOOST_PP_TUPLE_ELEM(1, elem));
#define ROW_MEMBERS_SEQ(prefix, seq) BOOST_PP_SEQ_FOR_EACH(ROW_MEMBER, prefix, seq)
#define ROW_MEMBERS(prefix, tuple) ROW_MEMBERS_SEQ(prefix, BOOST_PP_TUPLE_TO_SEQ(tuple))
#define ROW_SERIALIZE_ELEMENT(r, data, elem) , ROW_ELEMENT_NAME(data, BOOST_PP_TUPLE_ELEM(1, elem))
#define ROW_SERIALIZE_ELEMENTS(prefix, seq) BOOST_PP_SEQ_FOR_EACH(ROW_SERIALIZE_ELEMENT, prefix, seq)
#define ROW_SERIALIZE(prefix, tuple) ar ROW_SERIALIZE_ELEMENTS(prefix, BOOST_PP_TUPLE_TO_SEQ(tuple))
#define ROW_KEY_STEP(r, data, elem) , ROW_ELEMENT_NAME(data, elem)
#define ROW_KEY_LIST_SEQ_EXP(prefix, seq) BOOST_PP_SEQ_FOR_EACH(ROW_KEY_STEP, prefix, seq)
#define ROW_KEY_LIST_SEQ(prefix, seq) ROW_KEY_LIST_SEQ_EXP(prefix, seq)
#define ROW_KEY_LIST(prefix, a) ROW_KEY_LIST_SEQ(prefix, BOOST_PP_ARRAY_TO_SEQ(a))
#define ROW_KEY_LIST_TUPLE(prefix, tuple) ROW_KEY_LIST_SEQ(prefix, BOOST_PP_TUPLE_TO_SEQ(tuple))
#define ROW_KEY_HAS_KEY(Name, prefix, primary_key) \
static constexpr bool HAS_KEY = true; \
StringRef key() { \
auto s = generateKey(#Name, KEY_SIZE ROW_KEY_LIST(prefix, primary_key)); \
return StringRef(arena, s); \
} \
KeyRangeRef keyRange(int dontInclude) { \
auto s = generateKey(#Name, KEY_SIZE - dontInclude ROW_KEY_LIST(prefix, primary_key)); \
KeyRef begin = StringRef(arena, reinterpret_cast<const uint8_t*>(s.c_str()), s.size() + 1); \
KeyRef end = StringRef(arena, reinterpret_cast<const uint8_t*>(s.c_str()), s.size() + 1); \
auto sBegin = mutateString(begin); \
sBegin[s.size()] = uint8_t('/'); \
auto sEnd = mutateString(end); \
sEnd[s.size()] = uint8_t('0'); \
return KeyRangeRef(begin, end); \
}
#define ROW_KEY_NO_KEY static constexpr bool HAS_KEY = false;
#define ROW_KEY_IMPL(Name, prefix, primary_key, sz) \
BOOST_PP_IF(sz, ROW_KEY_HAS_KEY(Name, prefix, primary_key), ROW_KEY_NO_KEY)
#define ROW_KEY(Name, prefix, primary_key) ROW_KEY_IMPL(Name, prefix, primary_key, BOOST_PP_ARRAY_SIZE(primary_key))
#define ROW_INDEX_NAME_KEY(name) ROW_CONCAT(name, Key)
#define ROW_INDEX_NAME_IMPL2(name) ROW_TO_STRING(name)
#define ROW_INDEX_NAME_IMPL(indexName, name) ROW_INDEX_NAME_IMPL2(ROW_CONCAT(indexName, name))
#define ROW_INDEX_NAME(nameTuple, index) \
ROW_INDEX_NAME_IMPL(BOOST_PP_TUPLE_ELEM(0, index), BOOST_PP_TUPLE_ELEM(0, nameTuple))
#define ROW_GENERATE_INDEX(r, data, index) \
StringRef ROW_INDEX_NAME_KEY(BOOST_PP_TUPLE_ELEM(0, index))(int dontInclude = 0) { \
auto s = generateKey(ROW_INDEX_NAME(data, index), \
BOOST_PP_TUPLE_SIZE(index) - dontInclude - \
1 ROW_KEY_LIST_TUPLE(BOOST_PP_TUPLE_ELEM(1, data), BOOST_PP_TUPLE_POP_FRONT(index))); \
return StringRef(arena, s); \
}
#define ROW_GENERATE_INDEXES_LIST(Name, prefix, indexes) \
BOOST_PP_LIST_FOR_EACH(ROW_GENERATE_INDEX, (Name, prefix), indexes)
#define ROW_GENERATE_INDEXES(Name, prefix, indexes) \
ROW_GENERATE_INDEXES_LIST(Name, prefix, BOOST_PP_ARRAY_TO_LIST(indexes))
#define ROW_INDEXES(Name, prefix, indexes) \
BOOST_PP_IF(BOOST_PP_ARRAY_SIZE(indexes), ROW_GENERATE_INDEXES(Name, prefix, indexes), BOOST_PP_EMPTY())
#define ROW(Name, prefix, tuple, primary_key, indexes) \
struct Name { \
constexpr static FileIdentifier file_identifier = __COUNTER__; \
Arena arena; \
ROW_MEMBERS(prefix, tuple) \
template <class Ar> \
void serialize(Ar& ar) { \
serializer(ROW_SERIALIZE(prefix, tuple)); \
} \
static constexpr int KEY_SIZE = BOOST_PP_ARRAY_SIZE(primary_key); \
ROW_KEY(Name, prefix, primary_key) \
ROW_INDEXES(Name, prefix, indexes) \
}
template <class Value>
struct KeyStreamer {
void operator()(std::stringstream& ss, const Value& v) { ss << v; }
};
template <>
struct KeyStreamer<StringRef> {
void operator()(std::stringstream& ss, const StringRef& v) { ss << v.toString(); }
};
template <>
struct KeyStreamer<int> {
void operator()(std::stringstream& ss, const int v) { ss << std::setfill('0') << std::setw(6) << v; }
};
template <>
struct KeyStreamer<short> {
void operator()(std::stringstream& ss, const int v) { ss << std::setfill('0') << std::setw(6) << v; }
};
template <class... Values>
struct KeyGenerator;
template <class Head, class... Tail>
struct KeyGenerator<Head, Tail...> {
static void generate(std::stringstream& ss, int max, Head h, Tail... tail) {
KeyStreamer<Head> streamer;
if (max > 0) {
ss << '/';
streamer(ss, h);
KeyGenerator<Tail...>::generate(ss, max - 1, tail...);
}
}
};
template <>
struct KeyGenerator<> {
static void generate(std::stringstream&, int) {}
};
template <class... Values>
std::string generateKey(const std::string& table, int max, Values... values) {
std::stringstream ss;
ss << table;
if (max > 0) {
KeyGenerator<Values...>::generate(ss, max, values...);
}
return ss.str();
}
ROW(Warehouse,
w_,
((int, id),
(StringRef, name),
(StringRef, street_1),
(StringRef, street_2),
(StringRef, city),
(StringRef, state),
(StringRef, zip),
(double, tax),
(double, ytd)),
(1, (id)),
(0, ()));
ROW(District,
d_,
((int, id),
(int, w_id),
(StringRef, name),
(StringRef, street_1),
(StringRef, street_2),
(StringRef, city),
(StringRef, state),
(StringRef, zip),
(double, tax),
(double, ytd),
(int, next_o_id)),
(2, (w_id, id)),
(0, ()));
ROW(Customer,
c_,
((int, id),
(int, d_id),
(int, w_id),
(StringRef, first),
(StringRef, last),
(StringRef, middle),
(StringRef, street_1),
(StringRef, street_2),
(StringRef, city),
(StringRef, state),
(StringRef, zip),
(StringRef, phone),
(double, since),
(StringRef, credit),
(double, credit_lim),
(double, discount),
(double, balance),
(double, ytd_payment),
(unsigned, payment_cnt),
(unsigned, delivery_count),
(StringRef, data)),
(3, (w_id, d_id, id)),
(1, ((indexLast, w_id, d_id, last, id))));
ROW(History,
h_,
((int, c_id),
(int, c_d_id),
(int, c_w_id),
(int, d_id),
(int, w_id),
(double, date),
(double, amount),
(StringRef, data)),
(0, ()),
(0, ()));
ROW(NewOrder, no_, ((int, o_id), (int, d_id), (int, w_id)), (3, (w_id, d_id, o_id)), (0, ()));
ROW(Order,
o_,
((int, id),
(int, d_id),
(int, w_id),
(int, c_id),
(double, entry_d),
(Optional<short>, carrier_id),
(short, ol_cnt),
(bool, all_local)),
(3, (w_id, d_id, id)),
(0, ()));
ROW(OrderLine,
ol_,
((int, o_id),
(int, d_id),
(int, w_id),
(short, number),
(int, i_id),
(int, supply_w_id),
(Optional<double>, delivery_d),
(short, quantity),
(double, amount),
(StringRef, dist_info)),
(4, (w_id, d_id, o_id, number)),
(0, ()));
ROW(Item, i_, ((int, id), (int, im_id), (StringRef, name), (double, price), (StringRef, data)), (1, (id)), (0, ()));
ROW(Stock,
s_,
((int, i_id),
(int, w_id),
(short, quantity),
(StringRef, dist_01),
(StringRef, dist_02),
(StringRef, dist_03),
(StringRef, dist_04),
(StringRef, dist_05),
(StringRef, dist_06),
(StringRef, dist_07),
(StringRef, dist_08),
(StringRef, dist_09),
(StringRef, dist_10),
(int, ytd),
(short, order_cnt),
(short, remote_cnt),
(StringRef, data)),
(2, (w_id, i_id)),
(0, ()));
#undef FLOW_ACOMPILER_STATE
#define FLOW_ACOMPILER_STATE 1
struct GlobalState {
constexpr static FileIdentifier file_identifier = 1064821;
int CLoad, CRun, CDelta, CId, COlIID;
GlobalState() {
CLoad = deterministicRandom()->randomInt(0, 256);
while (true) {
CDelta = deterministicRandom()->randomInt(65, 120);
if (!(CDelta == 96 || CDelta == 112)) {
break;
}
}
if (CDelta > CLoad) {
CRun = CLoad + CDelta;
} else {
CRun = deterministicRandom()->coinflip() ? CLoad + CDelta : CLoad - CDelta;
}
CId = deterministicRandom()->randomInt(1, 3001);
COlIID = deterministicRandom()->randomInt(1, 100001);
}
template <class Ar>
void serialize(Ar& ar) {
serializer(ar, CLoad, CRun, CDelta, CId, COlIID);
}
StringRef key() const { return LiteralStringRef("GlobalState"); }
};
const std::vector<std::string> syllables = {
"BAR", "UGHT", "ABLE", "RI", "PRES", "SE", "ANTI", "ALLY", "ATION", "ING",
};
} // namespace TPCCWorkload
#endif

View File

@ -74,7 +74,7 @@ struct MasterData : NonCopyable, ReferenceCounted<MasterData> {
Counter reportLiveCommittedVersionRequests;
// This counter gives an estimate of the number of non-empty peeks that storage servers
// should do from tlogs (in the worst case, ignoring blocking peek timeouts).
Counter versionVectorTagUpdates;
LatencySample versionVectorTagUpdates;
Counter waitForPrevCommitRequests;
Counter nonWaitForPrevCommitRequests;
LatencySample versionVectorSizeOnCVReply;
@ -99,7 +99,10 @@ struct MasterData : NonCopyable, ReferenceCounted<MasterData> {
getCommitVersionRequests("GetCommitVersionRequests", cc),
getLiveCommittedVersionRequests("GetLiveCommittedVersionRequests", cc),
reportLiveCommittedVersionRequests("ReportLiveCommittedVersionRequests", cc),
versionVectorTagUpdates("VersionVectorTagUpdates", cc),
versionVectorTagUpdates("VersionVectorTagUpdates",
dbgid,
SERVER_KNOBS->LATENCY_METRICS_LOGGING_INTERVAL,
SERVER_KNOBS->LATENCY_SAMPLE_SIZE),
waitForPrevCommitRequests("WaitForPrevCommitRequests", cc),
nonWaitForPrevCommitRequests("NonWaitForPrevCommitRequests", cc),
versionVectorSizeOnCVReply("VersionVectorSizeOnCVReply",
@ -249,7 +252,7 @@ void updateLiveCommittedVersion(Reference<MasterData> self, ReportRawCommittedVe
int8_t primaryLocality =
SERVER_KNOBS->ENABLE_VERSION_VECTOR_HA_OPTIMIZATION ? self->locality : tagLocalityInvalid;
self->ssVersionVector.setVersion(req.writtenTags.get(), req.version, primaryLocality);
self->versionVectorTagUpdates += req.writtenTags.get().size();
self->versionVectorTagUpdates.addMeasurement(req.writtenTags.get().size());
}
auto curTime = now();
// add debug here to change liveCommittedVersion to time bound of now()

View File

@ -24,6 +24,7 @@
#include <unordered_map>
#include "fmt/format.h"
#include "fdbclient/CommitTransaction.h"
#include "fdbclient/FDBTypes.h"
#include "fdbrpc/fdbrpc.h"
#include "fdbrpc/LoadBalance.h"
@ -52,7 +53,9 @@
#include "fdbclient/SystemData.h"
#include "fdbclient/TransactionLineage.h"
#include "fdbclient/VersionedMap.h"
#include "fdbserver/EncryptedMutationMessage.h"
#include "fdbserver/FDBExecHelper.actor.h"
#include "fdbserver/GetEncryptCipherKeys.h"
#include "fdbserver/IKeyValueStore.h"
#include "fdbserver/Knobs.h"
#include "fdbserver/LatencyBandConfig.h"
@ -112,7 +115,7 @@ bool canReplyWith(Error e) {
return true;
default:
return false;
};
}
}
} // namespace
@ -1673,7 +1676,7 @@ ACTOR Future<Void> getValueQ(StorageServer* data, GetValueRequest req) {
}
return Void();
};
}
// Pessimistic estimate the number of overhead bytes used by each
// watch. Watch key references are stored in an AsyncMap<Key,bool>, and actors
@ -2170,6 +2173,58 @@ MutationsAndVersionRef filterMutations(Arena& arena,
DEBUG_CF_MISSING_CF&& keyRange.contains(DEBUG_CF_MISSING_KEY) && \
beginVersion <= DEBUG_CF_MISSING_VERSION&& lastVersion >= DEBUG_CF_MISSING_VERSION
// efficiently searches for the change feed mutation start point at begin version
static std::deque<Standalone<MutationsAndVersionRef>>::const_iterator searchChangeFeedStart(
std::deque<Standalone<MutationsAndVersionRef>> const& mutations,
Version beginVersion,
bool atLatest) {
if (mutations.empty() || beginVersion > mutations.back().version) {
return mutations.end();
} else if (beginVersion <= mutations.front().version) {
return mutations.begin();
}
MutationsAndVersionRef searchKey;
searchKey.version = beginVersion;
if (atLatest) {
int jump = 1;
// exponential search backwards, because atLatest means the new mutations are likely only at the very end
auto lastEnd = mutations.end();
auto currentEnd = mutations.end() - 1;
while (currentEnd > mutations.begin()) {
if (beginVersion >= currentEnd->version) {
break;
}
lastEnd = currentEnd + 1;
currentEnd -= jump;
jump <<= 1;
}
if (currentEnd < mutations.begin()) {
currentEnd = mutations.begin();
}
auto ret = std::lower_bound(currentEnd, lastEnd, searchKey, MutationsAndVersionRef::OrderByVersion());
// TODO REMOVE: for validation
if (ret != mutations.end()) {
if (ret->version < beginVersion) {
fmt::print("ERROR: {0}) {1} < {2}\n", ret - mutations.begin(), ret->version, beginVersion);
}
ASSERT(ret->version >= beginVersion);
}
if (ret != mutations.begin()) {
if ((ret - 1)->version >= beginVersion) {
fmt::print("ERROR: {0}) {1} >= {2}\n", (ret - mutations.begin()) - 1, (ret - 1)->version, beginVersion);
}
ASSERT((ret - 1)->version < beginVersion);
}
return ret;
} else {
// binary search
return std::lower_bound(
mutations.begin(), mutations.end(), searchKey, MutationsAndVersionRef::OrderByVersion());
}
}
ACTOR Future<std::pair<ChangeFeedStreamReply, bool>> getChangeFeedMutations(StorageServer* data,
ChangeFeedStreamRequest req,
bool inverted,
@ -2235,19 +2290,18 @@ ACTOR Future<std::pair<ChangeFeedStreamReply, bool>> getChangeFeedMutations(Stor
}
if (req.end > emptyVersion + 1) {
// FIXME: do exponential backwards search from end to find beginVersion if atLatest to reduce cpu
for (auto& it : feedInfo->mutations) {
if (it.version >= req.end || it.version > dequeVersion || remainingLimitBytes <= 0) {
auto it = searchChangeFeedStart(feedInfo->mutations, req.begin, atLatest);
while (it != feedInfo->mutations.end()) {
if (it->version >= req.end || it->version > dequeVersion || remainingLimitBytes <= 0) {
break;
}
if (it.version >= req.begin) {
auto m = filterMutations(memoryReply.arena, it, req.range, inverted);
if (m.mutations.size()) {
memoryReply.arena.dependsOn(it.arena());
memoryReply.mutations.push_back(memoryReply.arena, m);
remainingLimitBytes -= sizeof(MutationsAndVersionRef) + m.expectedSize();
}
auto m = filterMutations(memoryReply.arena, *it, req.range, inverted);
if (m.mutations.size()) {
memoryReply.arena.dependsOn(it->arena());
memoryReply.mutations.push_back(memoryReply.arena, m);
remainingLimitBytes -= sizeof(MutationsAndVersionRef) + m.expectedSize();
}
it++;
}
}
@ -2935,7 +2989,7 @@ ACTOR Future<GetValueReqAndResultRef> quickGetValue(StorageServer* data,
} else {
throw quick_get_value_miss();
}
};
}
// If limit>=0, it returns the first rows in the range (sorted ascending), otherwise the last rows (sorted descending).
// readRange has O(|result|) + O(log |data|) cost
@ -3549,7 +3603,7 @@ ACTOR Future<GetRangeReqAndResultRef> quickGetKeyValues(
} else {
throw quick_get_key_values_miss();
}
};
}
void unpackKeyTuple(Tuple** referenceTuple, Optional<Tuple>& keyTuple, KeyValueRef* keyValue) {
if (!keyTuple.present()) {
@ -3798,6 +3852,36 @@ TEST_CASE("/fdbserver/storageserver/constructMappedKey") {
return Void();
}
// Issues a secondary query (either range and point read) and fills results into "kvm".
ACTOR Future<Void> mapSubquery(StorageServer* data,
Version version,
GetMappedKeyValuesRequest* pOriginalReq,
Arena* pArena,
int matchIndex,
bool isRangeQuery,
bool isBoundary,
KeyValueRef* it,
MappedKeyValueRef* kvm,
Key mappedKey) {
if (isRangeQuery) {
// Use the mappedKey as the prefix of the range query.
GetRangeReqAndResultRef getRange = wait(quickGetKeyValues(data, mappedKey, version, pArena, pOriginalReq));
if ((!getRange.result.empty() && matchIndex == MATCH_INDEX_MATCHED_ONLY) ||
(getRange.result.empty() && matchIndex == MATCH_INDEX_UNMATCHED_ONLY)) {
kvm->key = it->key;
kvm->value = it->value;
}
kvm->boundaryAndExist = isBoundary && !getRange.result.empty();
kvm->reqAndResult = getRange;
} else {
GetValueReqAndResultRef getValue = wait(quickGetValue(data, mappedKey, version, pArena, pOriginalReq));
kvm->reqAndResult = getValue;
kvm->boundaryAndExist = isBoundary && getValue.result.present();
}
return Void();
}
ACTOR Future<GetMappedKeyValuesReply> mapKeyValues(StorageServer* data,
GetKeyValuesReply input,
StringRef mapper,
@ -3827,43 +3911,49 @@ ACTOR Future<GetMappedKeyValuesReply> mapKeyValues(StorageServer* data,
preprocessMappedKey(mappedKeyFormatTuple, vt, isRangeQuery);
state int sz = input.data.size();
state int i = 0;
for (; i < sz; i++) {
state KeyValueRef* it = &input.data[i];
state MappedKeyValueRef kvm;
state bool isBoundary = i == 0 || i == sz - 1;
// need to keep the boundary, so that caller can use it as a continuation.
if (isBoundary || matchIndex == MATCH_INDEX_ALL) {
kvm.key = it->key;
kvm.value = it->value;
}
state Key mappedKey = constructMappedKey(it, vt, mappedKeyTuple, mappedKeyFormatTuple);
// Make sure the mappedKey is always available, so that it's good even we want to get key asynchronously.
result.arena.dependsOn(mappedKey.arena());
// std::cout << "key:" << printable(kvm.key) << ", value:" << printable(kvm.value)
// << ", mappedKey:" << printable(mappedKey) << std::endl;
if (isRangeQuery) {
// Use the mappedKey as the prefix of the range query.
GetRangeReqAndResultRef getRange =
wait(quickGetKeyValues(data, mappedKey, input.version, &(result.arena), pOriginalReq));
if ((!getRange.result.empty() && matchIndex == MATCH_INDEX_MATCHED_ONLY) ||
(getRange.result.empty() && matchIndex == MATCH_INDEX_UNMATCHED_ONLY)) {
kvm.key = it->key;
kvm.value = it->value;
const int k = std::min(sz, SERVER_KNOBS->MAX_PARALLEL_QUICK_GET_VALUE);
state std::vector<MappedKeyValueRef> kvms(k);
state std::vector<Future<Void>> subqueries;
state int offset = 0;
for (; offset < sz; offset += SERVER_KNOBS->MAX_PARALLEL_QUICK_GET_VALUE) {
// Divide into batches of MAX_PARALLEL_QUICK_GET_VALUE subqueries
for (int i = 0; i + offset < sz && i < SERVER_KNOBS->MAX_PARALLEL_QUICK_GET_VALUE; i++) {
KeyValueRef* it = &input.data[i + offset];
MappedKeyValueRef* kvm = &kvms[i];
bool isBoundary = (i + offset) == 0 || (i + offset) == sz - 1;
// need to keep the boundary, so that caller can use it as a continuation.
if (isBoundary || matchIndex == MATCH_INDEX_ALL) {
kvm->key = it->key;
kvm->value = it->value;
} else {
// Clear key value to the default.
kvm->key = ""_sr;
kvm->value = ""_sr;
}
kvm.boundaryAndExist = isBoundary && !getRange.result.empty();
kvm.reqAndResult = getRange;
} else {
GetValueReqAndResultRef getValue =
wait(quickGetValue(data, mappedKey, input.version, &(result.arena), pOriginalReq));
kvm.reqAndResult = getValue;
kvm.boundaryAndExist = isBoundary && getValue.result.present();
Key mappedKey = constructMappedKey(it, vt, mappedKeyTuple, mappedKeyFormatTuple);
// Make sure the mappedKey is always available, so that it's good even we want to get key asynchronously.
result.arena.dependsOn(mappedKey.arena());
// std::cout << "key:" << printable(kvm->key) << ", value:" << printable(kvm->value)
// << ", mappedKey:" << printable(mappedKey) << std::endl;
subqueries.push_back(mapSubquery(data,
input.version,
pOriginalReq,
&result.arena,
matchIndex,
isRangeQuery,
isBoundary,
it,
kvm,
mappedKey));
}
wait(waitForAll(subqueries));
subqueries.clear();
for (int i = 0; i + offset < sz && i < SERVER_KNOBS->MAX_PARALLEL_QUICK_GET_VALUE; i++) {
result.data.push_back(result.arena, kvms[i]);
}
result.data.push_back(result.arena, kvm);
}
return result;
}
@ -6223,7 +6313,7 @@ ACTOR Future<Void> fetchKeys(StorageServer* data, AddingShard* shard) {
}
return Void();
};
}
AddingShard::AddingShard(StorageServer* server, KeyRangeRef const& keys)
: keys(keys), server(server), transferredVersion(invalidVersion), fetchVersion(invalidVersion), phase(WaitPrevious) {
@ -6849,12 +6939,12 @@ private:
} else if (m.param1.substr(1).startsWith(tssMappingKeys.begin) &&
(m.type == MutationRef::SetValue || m.type == MutationRef::ClearRange)) {
if (!data->isTss()) {
UID ssId = Codec<UID>::unpack(Tuple::unpack(m.param1.substr(1).removePrefix(tssMappingKeys.begin)));
UID ssId = TupleCodec<UID>::unpack(m.param1.substr(1).removePrefix(tssMappingKeys.begin));
ASSERT(ssId == data->thisServerID);
// Add ss pair id change to mutation log to make durable
auto& mLV = data->addVersionToMutationLog(data->data().getLatestVersion());
if (m.type == MutationRef::SetValue) {
UID tssId = Codec<UID>::unpack(Tuple::unpack(m.param2));
UID tssId = TupleCodec<UID>::unpack(m.param2);
data->setSSWithTssPair(tssId);
data->addMutationToMutationLog(mLV,
MutationRef(MutationRef::SetValue,
@ -6946,7 +7036,7 @@ void StorageServer::insertTenant(TenantNameRef tenantName,
tenantMap.createNewVersion(version);
tenantPrefixIndex.createNewVersion(version);
TenantMapEntry tenantEntry = decodeTenantEntry(value);
TenantMapEntry tenantEntry = TenantMapEntry::decode(value);
tenantMap.insert(tenantName, tenantEntry);
tenantPrefixIndex.insert(tenantEntry.prefix, tenantName);
@ -7092,7 +7182,11 @@ ACTOR Future<Void> update(StorageServer* data, bool* pReceivedUpdate) {
state UpdateEagerReadInfo eager;
state FetchInjectionInfo fii;
state Reference<ILogSystem::IPeekCursor> cloneCursor2;
state Optional<std::unordered_map<BlobCipherDetails, Reference<BlobCipherKey>>> cipherKeys;
state bool collectingCipherKeys = false;
// Collect eager read keys.
// If encrypted mutation is encountered, we collect cipher details and fetch cipher keys, then start over.
loop {
state uint64_t changeCounter = data->shardChangeCounter;
bool epochEnd = false;
@ -7100,6 +7194,8 @@ ACTOR Future<Void> update(StorageServer* data, bool* pReceivedUpdate) {
bool firstMutation = true;
bool dbgLastMessageWasProtocol = false;
std::unordered_set<BlobCipherDetails> cipherDetails;
Reference<ILogSystem::IPeekCursor> cloneCursor1 = cursor->cloneNoMore();
cloneCursor2 = cursor->cloneNoMore();
@ -7122,47 +7218,72 @@ ACTOR Future<Void> update(StorageServer* data, bool* pReceivedUpdate) {
OTELSpanContextMessage::isNextIn(cloneReader)) {
OTELSpanContextMessage scm;
cloneReader >> scm;
} else if (cloneReader.protocolVersion().hasEncryptionAtRest() &&
EncryptedMutationMessage::isNextIn(cloneReader) && !cipherKeys.present()) {
// Encrypted mutation found, but cipher keys haven't been fetch.
// Collect cipher details to fetch cipher keys in one batch.
EncryptedMutationMessage emm;
cloneReader >> emm;
cipherDetails.insert(emm.header.cipherTextDetails);
cipherDetails.insert(emm.header.cipherHeaderDetails);
collectingCipherKeys = true;
} else {
MutationRef msg;
cloneReader >> msg;
if (cloneReader.protocolVersion().hasEncryptionAtRest() &&
EncryptedMutationMessage::isNextIn(cloneReader)) {
assert(cipherKeys.present());
msg = EncryptedMutationMessage::decrypt(cloneReader, eager.arena, cipherKeys.get());
} else {
cloneReader >> msg;
}
// TraceEvent(SevDebug, "SSReadingLog", data->thisServerID).detail("Mutation", msg);
if (firstMutation && msg.param1.startsWith(systemKeys.end))
hasPrivateData = true;
firstMutation = false;
if (!collectingCipherKeys) {
if (firstMutation && msg.param1.startsWith(systemKeys.end))
hasPrivateData = true;
firstMutation = false;
if (msg.param1 == lastEpochEndPrivateKey) {
epochEnd = true;
ASSERT(dbgLastMessageWasProtocol);
if (msg.param1 == lastEpochEndPrivateKey) {
epochEnd = true;
ASSERT(dbgLastMessageWasProtocol);
}
eager.addMutation(msg);
dbgLastMessageWasProtocol = false;
}
eager.addMutation(msg);
dbgLastMessageWasProtocol = false;
}
}
// Any fetchKeys which are ready to transition their shards to the adding,transferred state do so now.
// If there is an epoch end we skip this step, to increase testability and to prevent inserting a
// version in the middle of a rolled back version range.
while (!hasPrivateData && !epochEnd && !data->readyFetchKeys.empty()) {
auto fk = data->readyFetchKeys.back();
data->readyFetchKeys.pop_back();
fk.send(&fii);
// fetchKeys() would put the data it fetched into the fii. The thread will not return back to this
// actor until it was completed.
if (collectingCipherKeys) {
std::unordered_map<BlobCipherDetails, Reference<BlobCipherKey>> getCipherKeysResult =
wait(getEncryptCipherKeys(data->db, cipherDetails));
cipherKeys = getCipherKeysResult;
collectingCipherKeys = false;
eager = UpdateEagerReadInfo();
} else {
// Any fetchKeys which are ready to transition their shards to the adding,transferred state do so now.
// If there is an epoch end we skip this step, to increase testability and to prevent inserting a
// version in the middle of a rolled back version range.
while (!hasPrivateData && !epochEnd && !data->readyFetchKeys.empty()) {
auto fk = data->readyFetchKeys.back();
data->readyFetchKeys.pop_back();
fk.send(&fii);
// fetchKeys() would put the data it fetched into the fii. The thread will not return back to this
// actor until it was completed.
}
for (auto& c : fii.changes)
eager.addMutations(c.mutations);
wait(doEagerReads(data, &eager));
if (data->shardChangeCounter == changeCounter)
break;
TEST(true); // A fetchKeys completed while we were doing this, so eager might be outdated. Read it
// again.
// SOMEDAY: Theoretically we could check the change counters of individual shards and retry the reads
// only selectively
eager = UpdateEagerReadInfo();
}
for (auto& c : fii.changes)
eager.addMutations(c.mutations);
wait(doEagerReads(data, &eager));
if (data->shardChangeCounter == changeCounter)
break;
TEST(true); // A fetchKeys completed while we were doing this, so eager might be outdated. Read it
// again.
// SOMEDAY: Theoretically we could check the change counters of individual shards and retry the reads
// only selectively
eager = UpdateEagerReadInfo();
}
data->eagerReadsLatencyHistogram->sampleSeconds(now() - start);
@ -7255,7 +7376,12 @@ ACTOR Future<Void> update(StorageServer* data, bool* pReceivedUpdate) {
spanContext = scm.spanContext;
} else {
MutationRef msg;
rd >> msg;
if (rd.protocolVersion().hasEncryptionAtRest() && EncryptedMutationMessage::isNextIn(rd)) {
ASSERT(cipherKeys.present());
msg = EncryptedMutationMessage::decrypt(rd, rd.arena(), cipherKeys.get());
} else {
rd >> msg;
}
Span span("SS:update"_loc, spanContext);
span.addAttribute("key"_sr, msg.param1);
@ -7435,7 +7561,9 @@ ACTOR Future<Void> update(StorageServer* data, bool* pReceivedUpdate) {
return Void(); // update will get called again ASAP
} catch (Error& err) {
state Error e = err;
if (e.code() != error_code_worker_removed && e.code() != error_code_please_reboot) {
if (e.code() == error_code_encrypt_keys_fetch_failed) {
TraceEvent(SevWarn, "SSUpdateError", data->thisServerID).error(e).backtrace();
} else if (e.code() != error_code_worker_removed && e.code() != error_code_please_reboot) {
TraceEvent(SevError, "SSUpdateError", data->thisServerID).error(e).backtrace();
} else if (e.code() == error_code_please_reboot) {
wait(data->durableInProgress);
@ -7789,7 +7917,7 @@ void StorageServerDisk::makeNewStorageServerDurable() {
auto view = data->tenantMap.atLatest();
for (auto itr = view.begin(); itr != view.end(); ++itr) {
storage->set(KeyValueRef(itr.key().withPrefix(persistTenantMapKeys.begin), encodeTenantEntry(*itr)));
storage->set(KeyValueRef(itr.key().withPrefix(persistTenantMapKeys.begin), itr->encode()));
}
}
@ -8270,7 +8398,7 @@ ACTOR Future<bool> restoreDurableState(StorageServer* data, IKeyValueStore* stor
for (tenantMapLoc = 0; tenantMapLoc < tenantMap.size(); tenantMapLoc++) {
auto const& result = tenantMap[tenantMapLoc];
TenantName tenantName = result.key.substr(persistTenantMapKeys.begin.size());
TenantMapEntry tenantEntry = decodeTenantEntry(result.value);
TenantMapEntry tenantEntry = TenantMapEntry::decode(result.value);
data->tenantMap.insert(tenantName, tenantEntry);
data->tenantPrefixIndex.insert(tenantEntry.prefix, tenantName);

View File

@ -466,11 +466,11 @@ struct BackupAndParallelRestoreCorrectnessWorkload : TestWorkload {
.detail("AbortAndRestartAfter", self->abortAndRestartAfter);
state KeyBackedTag keyBackedTag = makeBackupTag(self->backupTag.toString());
UidAndAbortedFlagT uidFlag = wait(keyBackedTag.getOrThrow(cx));
UidAndAbortedFlagT uidFlag = wait(keyBackedTag.getOrThrow(cx.getReference()));
state UID logUid = uidFlag.first;
state Key destUidValue = wait(BackupConfig(logUid).destUidValue().getD(cx));
state Key destUidValue = wait(BackupConfig(logUid).destUidValue().getD(cx.getReference()));
state Reference<IBackupContainer> lastBackupContainer =
wait(BackupConfig(logUid).backupContainer().getD(cx));
wait(BackupConfig(logUid).backupContainer().getD(cx.getReference()));
// Occasionally start yet another backup that might still be running when we restore
if (!self->locked && BUGGIFY) {

View File

@ -513,11 +513,11 @@ struct BackupAndRestoreCorrectnessWorkload : TestWorkload {
.detail("AbortAndRestartAfter", self->abortAndRestartAfter);
state KeyBackedTag keyBackedTag = makeBackupTag(self->backupTag.toString());
UidAndAbortedFlagT uidFlag = wait(keyBackedTag.getOrThrow(cx));
UidAndAbortedFlagT uidFlag = wait(keyBackedTag.getOrThrow(cx.getReference()));
state UID logUid = uidFlag.first;
state Key destUidValue = wait(BackupConfig(logUid).destUidValue().getD(cx));
state Key destUidValue = wait(BackupConfig(logUid).destUidValue().getD(cx.getReference()));
state Reference<IBackupContainer> lastBackupContainer =
wait(BackupConfig(logUid).backupContainer().getD(cx));
wait(BackupConfig(logUid).backupContainer().getD(cx.getReference()));
// Occasionally start yet another backup that might still be running when we restore
if (!self->locked && BUGGIFY) {

View File

@ -1,521 +0,0 @@
/*
* TPCC.actor.cpp
*
* This source file is part of the FoundationDB open source project
*
* Copyright 2013-2022 Apple Inc. and the FoundationDB project authors
*
* Licensed under the Apache License, Version 2.0 (the "License");
* you may not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
#include "flow/Arena.h"
#include "fdbserver/workloads/workloads.actor.h"
#include "fdbserver/QuietDatabase.h"
#include "fdbserver/workloads/TPCCWorkload.h"
#include "fdbserver/ServerDBInfo.h"
#include "fdbclient/ReadYourWrites.h"
#include "flow/actorcompiler.h" // needs to be last include
#undef FLOW_ACOMPILER_STATE
#define FLOW_ACOMPILER_STATE 1
using namespace TPCCWorkload;
namespace {
constexpr char alphaNumerics[] = { 'a', 'b', 'c', 'd', 'e', 'f', 'g', 'h', 'i', 'j', 'k', 'l', 'm', 'n', 'o', 'p',
'q', 'r', 's', 't', 'u', 'v', 'w', 'x', 'y', 'z', 'A', 'B', 'C', 'D', 'E', 'F',
'G', 'H', 'I', 'J', 'K', 'L', 'M', 'N', 'O', 'P', 'Q', 'R', 'S', 'T', 'U', 'V',
'W', 'X', 'Y', 'Z', '1', '2', '3', '4', '5', '6', '7', '8', '9', '0' };
constexpr char numerics[] = { '0', '1', '2', '3', '4', '5', '6', '7', '8', '9' };
constexpr const char* originalString = "ORIGINAL";
struct PopulateTPCC : TestWorkload {
static constexpr const char* DESCRIPTION = "PopulateTPCC";
int actorsPerClient;
int warehousesPerActor;
int clientsUsed;
GlobalState gState;
PopulateTPCC(WorkloadContext const& ctx) : TestWorkload(ctx) {
std::string workloadName = DESCRIPTION;
actorsPerClient = getOption(options, LiteralStringRef("actorsPerClient"), 10);
warehousesPerActor = getOption(options, LiteralStringRef("warehousesPerActor"), 30);
clientsUsed = getOption(options, LiteralStringRef("clientsUsed"), 2);
}
int NURand(int C, int A, int x, int y) {
return (((deterministicRandom()->randomInt(0, A + 1) | deterministicRandom()->randomInt(x, y + 1)) + C) %
(y - x + 1)) +
x;
}
StringRef aString(Arena& arena, int x, int y) {
int length = deterministicRandom()->randomInt(x, y + 1);
char* res = new (arena) char[length];
for (int i = 0; i < length; ++i) {
res[i] = alphaNumerics[deterministicRandom()->randomInt(0, sizeof(alphaNumerics))];
}
return StringRef(reinterpret_cast<uint8_t*>(res), length);
}
StringRef nString(Arena& arena, int x, int y) {
int length = deterministicRandom()->randomInt(x, y + 1);
char* res = new (arena) char[length];
for (int i = 0; i < length; ++i) {
res[i] = numerics[deterministicRandom()->randomInt(0, sizeof(numerics))];
}
return StringRef(reinterpret_cast<uint8_t*>(res), length);
}
StringRef genCLast(Arena& arena, int x) {
int l = x % 10;
x /= 10;
int m = x % 10;
x /= 10;
int f = x % 10;
std::stringstream ss;
ss << syllables[f] << syllables[m] << syllables[l];
return StringRef(arena, ss.str());
}
StringRef rndZip(Arena& arena) {
char* result = new (arena) char[9];
for (int i = 0; i < 4; ++i) {
result[i] = numerics[deterministicRandom()->randomInt(0, sizeof(numerics))];
}
for (int i = 4; i < 9; ++i) {
result[i] = '1';
}
return StringRef(reinterpret_cast<uint8_t*>(result), 9);
}
StringRef dataString(Arena& arena) {
if (deterministicRandom()->random01() < 0.1) {
auto str = aString(arena, 26, 51 - strlen(originalString));
char* r = new (arena) char[str.size() + strlen(originalString)];
int pos = deterministicRandom()->randomInt(0, str.size());
std::copy(originalString, originalString + strlen(originalString), r + pos);
auto res = reinterpret_cast<uint8_t*>(r);
std::copy(str.begin(), str.begin() + pos, res);
std::copy(str.begin() + pos, str.end(), res + pos + strlen(originalString));
return StringRef(res, str.size() + strlen(originalString));
} else {
return aString(arena, 26, 51);
}
}
ACTOR static Future<Void> writeGlobalState(PopulateTPCC* self, Database cx) {
state ReadYourWritesTransaction tr(cx);
loop {
tr.reset();
try {
BinaryWriter writer(IncludeVersion());
serializer(writer, self->gState);
tr.set(self->gState.key(), writer.toValue());
wait(tr.commit());
return Void();
} catch (Error& e) {
wait(tr.onError(e));
}
}
}
ACTOR static Future<Void> readGlobalState(PopulateTPCC* self, Database cx) {
state ReadYourWritesTransaction tr(cx);
loop {
tr.reset();
try {
Optional<Value> val = wait(tr.get(self->gState.key()));
if (val.present()) {
BinaryReader reader(val.get(), IncludeVersion());
serializer(reader, self->gState);
} else {
wait(delay(1.0));
}
return Void();
} catch (Error& e) {
wait(tr.onError(e));
}
}
}
std::string description() const override { return DESCRIPTION; }
ACTOR static Future<Void> populateItems(PopulateTPCC* self, Database cx) {
state Transaction tr(cx);
state int itemStart = 0;
state int i_id;
for (; itemStart < 100000; itemStart += 100) {
TraceEvent("PopulateItems").detail("Status", itemStart);
loop {
try {
tr.reset();
for (i_id = itemStart; i_id < itemStart + 100; ++i_id) {
Item item;
item.i_id = i_id;
item.i_im_id = deterministicRandom()->randomInt(1, 10001);
item.i_name = self->aString(item.arena, 14, 25);
item.i_price = deterministicRandom()->randomInt64(1.0, 100.0);
item.i_data = self->dataString(item.arena);
BinaryWriter w(IncludeVersion());
serializer(w, item);
tr.set(item.key(), w.toValue(), AddConflictRange::False);
}
wait(tr.commit());
break;
} catch (Error& e) {
TraceEvent("PopulateItemsHandleError").error(e);
wait(tr.onError(e));
}
}
}
TraceEvent("PopulateItemsDone").log();
return Void();
}
ACTOR static Future<Void> populateCustomers(PopulateTPCC* self, Database cx, int w_id, int d_id) {
state Transaction tr(cx);
state int cStart;
state int c_id;
for (cStart = 0; cStart < 3000; cStart += 100) {
TraceEvent("PopulateCustomers")
.detail("Warehouse", w_id)
.detail("District", d_id)
.detail("Customer", cStart);
loop {
for (c_id = cStart; c_id < cStart + 100; ++c_id) {
Customer c;
History h;
c.c_id = c_id;
c.c_d_id = d_id;
c.c_w_id = w_id;
if (c_id < 1000) {
c.c_last = self->genCLast(c.arena, c_id);
} else {
c.c_last = self->genCLast(c.arena, self->NURand(self->gState.CLoad, 255, 0, 999));
}
c.c_middle = LiteralStringRef("OE");
c.c_first = self->aString(c.arena, 8, 16);
c.c_street_1 = self->aString(c.arena, 10, 20);
c.c_street_2 = self->aString(c.arena, 10, 20);
c.c_city = self->aString(c.arena, 10, 20);
c.c_state = self->aString(c.arena, 2, 2);
c.c_zip = self->rndZip(c.arena);
c.c_phone = self->nString(c.arena, 16, 16);
c.c_since = g_network->now();
if (deterministicRandom()->random01() < 0.1) {
c.c_credit = LiteralStringRef("BC");
} else {
c.c_credit = LiteralStringRef("GC");
}
c.c_credit_lim = 50000;
c.c_discount = deterministicRandom()->random01() / 2.0;
c.c_balance = -10.0;
c.c_ytd_payment = 10.0;
c.c_payment_cnt = 1;
c.c_delivery_count = 0;
c.c_data = self->aString(c.arena, 300, 500);
h.h_c_id = c_id;
h.h_c_d_id = d_id;
h.h_d_id = d_id;
h.h_w_id = w_id;
h.h_c_w_id = w_id;
h.h_date = g_network->now();
h.h_amount = 10.0;
h.h_data = self->aString(c.arena, 12, 24);
{
BinaryWriter w(IncludeVersion());
serializer(w, c);
tr.set(c.key(), w.toValue(), AddConflictRange::False);
}
{
// Write index
tr.set(c.indexLastKey(), c.key(), AddConflictRange::False);
}
{
BinaryWriter w(IncludeVersion());
serializer(w, h);
UID k = deterministicRandom()->randomUniqueID();
BinaryWriter kW(Unversioned());
serializer(kW, k);
auto key = kW.toValue().withPrefix(LiteralStringRef("History/"));
tr.set(key, w.toValue(), AddConflictRange::False);
}
}
try {
wait(tr.commit());
break;
} catch (Error& e) {
TraceEvent("PopulateCustomerHandleError").error(e);
wait(tr.onError(e));
}
}
}
TraceEvent("PopulateCustomersDone").detail("Warehouse", w_id).detail("District", d_id);
return Void();
}
ACTOR static Future<Void> populateOrders(PopulateTPCC* self, Database cx, int w_id, int d_id) {
state Transaction tr(cx);
state std::vector<int> customerIds;
state int idStart;
state int o_id;
customerIds.reserve(3000);
for (int i = 0; i < 3000; ++i) {
customerIds.push_back(i);
}
deterministicRandom()->randomShuffle(customerIds);
for (idStart = 0; idStart < 3000; idStart += 100) {
TraceEvent("PopulateOrders").detail("Warehouse", w_id).detail("District", d_id).detail("Order", idStart);
loop {
tr.reset();
for (o_id = idStart; o_id < idStart + 100; ++o_id) {
Order o;
o.o_id = o_id;
o.o_c_id = customerIds[o_id];
o.o_d_id = d_id;
o.o_w_id = w_id;
o.o_entry_d = g_network->now();
if (o_id < 2100) {
o.o_carrier_id = deterministicRandom()->randomInt(1, 11);
}
o.o_ol_cnt = deterministicRandom()->randomInt(5, 16);
o.o_all_local = true;
for (int ol_number = 0; ol_number < o.o_ol_cnt; ++ol_number) {
OrderLine ol;
ol.ol_o_id = o_id;
ol.ol_d_id = d_id;
ol.ol_w_id = w_id;
ol.ol_number = ol_number;
ol.ol_i_id = deterministicRandom()->randomInt(0, 100000);
ol.ol_supply_w_id = w_id;
if (o_id < 2100) {
ol.ol_delivery_d = g_network->now();
ol.ol_amount = 0.0;
} else {
ol.ol_amount = deterministicRandom()->random01() * 10000.0;
}
ol.ol_quantity = 5;
ol.ol_dist_info = self->aString(ol.arena, 24, 24);
BinaryWriter w(IncludeVersion());
serializer(w, ol);
tr.set(ol.key(), w.toValue(), AddConflictRange::False);
}
BinaryWriter w(IncludeVersion());
serializer(w, o);
tr.set(o.key(), w.toValue(), AddConflictRange::False);
}
try {
wait(tr.commit());
break;
} catch (Error& e) {
TraceEvent("PopulateOrderHandleError").error(e);
wait(tr.onError(e));
}
}
}
TraceEvent("PopulateOrdersDone").detail("Warehouse", w_id).detail("District", d_id);
return Void();
}
ACTOR static Future<Void> populateNewOrders(PopulateTPCC* self, Database cx, int w_id, int d_id) {
state Transaction tr(cx);
TraceEvent("PopulateNewOrders").detail("Warehouse", w_id).detail("District", d_id);
loop {
tr.reset();
for (int i = 2100; i < 3000; ++i) {
NewOrder no;
no.no_o_id = i;
no.no_d_id = d_id;
no.no_w_id = w_id;
BinaryWriter w(IncludeVersion());
serializer(w, no);
tr.set(no.key(), w.toValue(), AddConflictRange::False);
}
try {
wait(tr.commit());
break;
} catch (Error& e) {
TraceEvent("PopulateNewOrderHandleError").error(e);
wait(tr.onError(e));
}
}
TraceEvent("PopulateNewOrdersDone").detail("Warehouse", w_id).detail("District", d_id);
return Void();
}
ACTOR static Future<Void> populateDistricts(PopulateTPCC* self, Database cx, int w_id) {
state Transaction tr(cx);
state int d_id;
for (d_id = 0; d_id < 10; ++d_id) {
TraceEvent("PopulateDistricts").detail("Warehouse", w_id).detail("District", d_id);
loop {
tr.reset();
District d;
d.d_id = d_id;
d.d_w_id = w_id;
d.d_name = self->aString(d.arena, 6, 10);
d.d_street_1 = self->aString(d.arena, 10, 20);
d.d_street_2 = self->aString(d.arena, 10, 20);
d.d_city = self->aString(d.arena, 10, 20);
d.d_state = self->aString(d.arena, 2, 2);
d.d_zip = self->rndZip(d.arena);
d.d_tax = deterministicRandom()->random01() * 0.2;
d.d_ytd = 30000;
d.d_next_o_id = 3000;
BinaryWriter w(IncludeVersion());
serializer(w, d);
tr.set(d.key(), w.toValue(), AddConflictRange::False);
try {
wait(tr.commit());
wait(populateCustomers(self, cx, w_id, d_id));
wait(populateOrders(self, cx, w_id, d_id));
wait(populateNewOrders(self, cx, w_id, d_id));
break;
} catch (Error& e) {
TraceEvent("PopulateDistrictHandleError").error(e);
wait(tr.onError(e));
}
}
}
TraceEvent("PopulateDistrictsDone").detail("Warehouse", w_id);
return Void();
}
ACTOR static Future<Void> populateStock(PopulateTPCC* self, Database cx, int w_id) {
state Transaction tr(cx);
state int idStart;
for (idStart = 0; idStart < 100000; idStart += 100) {
TraceEvent("PopulateStock").detail("Warehouse", w_id).detail("i_id", idStart);
loop {
tr.reset();
for (int i = idStart; i < idStart + 100; ++i) {
Stock s;
s.s_i_id = i;
s.s_w_id = w_id;
s.s_quantity = deterministicRandom()->randomInt(1, 101);
s.s_dist_01 = self->aString(s.arena, 24, 25);
s.s_dist_02 = self->aString(s.arena, 24, 25);
s.s_dist_03 = self->aString(s.arena, 24, 25);
s.s_dist_04 = self->aString(s.arena, 24, 25);
s.s_dist_05 = self->aString(s.arena, 24, 25);
s.s_dist_06 = self->aString(s.arena, 24, 25);
s.s_dist_07 = self->aString(s.arena, 24, 25);
s.s_dist_08 = self->aString(s.arena, 24, 25);
s.s_dist_09 = self->aString(s.arena, 24, 25);
s.s_dist_10 = self->aString(s.arena, 24, 25);
s.s_ytd = 0;
s.s_order_cnt = 0;
s.s_remote_cnt = 0;
s.s_data = self->dataString(s.arena);
BinaryWriter w(IncludeVersion());
serializer(w, s);
tr.set(s.key(), w.toValue(), AddConflictRange::False);
}
try {
wait(tr.commit());
break;
} catch (Error& e) {
TraceEvent("PopulateStockHandleError").error(e).detail("Warehouse", w_id);
wait(tr.onError(e));
}
}
}
TraceEvent("PopulateStockDone").detail("Warehouse", w_id);
return Void();
}
ACTOR static Future<Void> populateWarehouse(PopulateTPCC* self, Database cx, int w_id) {
state Transaction tr(cx);
TraceEvent("PopulateWarehouse").detail("W_ID", w_id);
loop {
tr.reset();
try {
Warehouse w;
w.w_id = w_id;
w.w_name = self->aString(w.arena, 6, 11);
w.w_street_1 = self->aString(w.arena, 10, 21);
w.w_street_2 = self->aString(w.arena, 10, 21);
w.w_city = self->aString(w.arena, 10, 21);
w.w_state = self->aString(w.arena, 2, 3);
w.w_tax = deterministicRandom()->random01() * 0.2;
w.w_ytd = 300000;
BinaryWriter writer(IncludeVersion());
serializer(writer, w);
tr.set(w.key(), writer.toValue(), AddConflictRange::False);
wait(tr.commit());
break;
} catch (Error& e) {
TraceEvent("PopulateWarehouseHandleError").error(e).detail("Warehouse", w_id);
wait(tr.onError(e));
}
}
wait(populateStock(self, cx, w_id));
wait(populateDistricts(self, cx, w_id));
TraceEvent("PopulateWarehouseDone").detail("W_ID", w_id);
return Void();
}
ACTOR static Future<Void> populateActor(PopulateTPCC* self, Database cx, int actorId) {
state int startWID =
self->clientId * self->actorsPerClient * self->warehousesPerActor + actorId * self->warehousesPerActor;
state int endWID = startWID + self->warehousesPerActor;
state int wid;
for (wid = startWID; wid < endWID; ++wid) {
wait(populateWarehouse(self, cx, wid));
}
return Void();
}
ACTOR static Future<Void> populate(PopulateTPCC* self, Database cx) {
if (self->clientId == 0) {
wait(writeGlobalState(self, cx));
} else {
wait(readGlobalState(self, cx));
}
if (self->clientId == 0) {
wait(populateItems(self, cx));
}
state std::vector<Future<Void>> populateActors;
state int actorId;
for (actorId = 0; actorId < self->actorsPerClient; ++actorId) {
populateActors.push_back(populateActor(self, cx, actorId));
}
wait(waitForAll(populateActors));
wait(quietDatabase(cx, self->dbInfo, "PopulateTPCC"));
return Void();
}
Future<Void> setup(Database const& cx) override {
if (clientId >= clientsUsed)
return Void();
return populate(this, cx);
}
Future<Void> start(Database const& cx) override { return Void(); }
Future<bool> check(Database const& cx) override { return true; }
void getMetrics(std::vector<PerfMetric>& m) override {}
};
} // namespace
WorkloadFactory<PopulateTPCC> PopulateTPCCWorkloadFactory(PopulateTPCC::DESCRIPTION);

View File

@ -1,825 +0,0 @@
/*
* TPCC.actor.cpp
*
* This source file is part of the FoundationDB open source project
*
* Copyright 2013-2022 Apple Inc. and the FoundationDB project authors
*
* Licensed under the Apache License, Version 2.0 (the "License");
* you may not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
#include "fdbserver/workloads/workloads.actor.h"
#include "fdbserver/workloads/TPCCWorkload.h"
#include <fdbclient/ReadYourWrites.h>
#include "flow/actorcompiler.h" // has to be last include
using namespace TPCCWorkload;
namespace {
struct TPCCMetrics {
static constexpr int latenciesStored = 1000;
uint64_t successfulStockLevelTransactions{ 0 };
uint64_t failedStockLevelTransactions{ 0 };
uint64_t successfulDeliveryTransactions{ 0 };
uint64_t failedDeliveryTransactions{ 0 };
uint64_t successfulOrderStatusTransactions{ 0 };
uint64_t failedOrderStatusTransactions{ 0 };
uint64_t successfulPaymentTransactions{ 0 };
uint64_t failedPaymentTransactions{ 0 };
uint64_t successfulNewOrderTransactions{ 0 };
uint64_t failedNewOrderTransactions{ 0 };
double stockLevelResponseTime{ 0.0 };
double deliveryResponseTime{ 0.0 };
double orderStatusResponseTime{ 0.0 };
double paymentResponseTime{ 0.0 };
double newOrderResponseTime{ 0.0 };
std::vector<double> stockLevelLatencies, deliveryLatencies, orderStatusLatencies, paymentLatencies,
newOrderLatencies;
void sort() {
std::sort(stockLevelLatencies.begin(), stockLevelLatencies.end());
std::sort(deliveryLatencies.begin(), deliveryLatencies.end());
std::sort(orderStatusLatencies.begin(), orderStatusLatencies.end());
std::sort(paymentLatencies.begin(), paymentLatencies.end());
std::sort(newOrderLatencies.begin(), newOrderLatencies.end());
}
static double median(const std::vector<double>& latencies) {
// assumes latencies is sorted
return latencies[latencies.size() / 2];
}
static double percentile_90(const std::vector<double>& latencies) {
// assumes latencies is sorted
return latencies[(9 * latencies.size()) / 10];
}
static double percentile_99(const std::vector<double>& latencies) {
// assumes latencies is sorted
return latencies[(99 * latencies.size()) / 100];
}
static void updateMetrics(bool committed,
uint64_t& successCounter,
uint64_t& failedCounter,
double txnStartTime,
std::vector<double>& latencies,
double& totalLatency,
std::string txnType) {
auto responseTime = g_network->now() - txnStartTime;
if (committed) {
totalLatency += responseTime;
++successCounter;
if (successCounter <= latenciesStored)
latencies[successCounter - 1] = responseTime;
else {
auto index = deterministicRandom()->randomInt(0, successCounter);
if (index < latenciesStored) {
latencies[index] = responseTime;
}
}
} else {
++failedCounter;
}
TraceEvent("TransactionComplete")
.detail("TransactionType", txnType)
.detail("Latency", responseTime)
.detail("Begin", txnStartTime)
.detail("End", txnStartTime + responseTime)
.detail("Success", committed);
}
};
struct TPCC : TestWorkload {
static constexpr const char* DESCRIPTION = "TPCC";
int warehousesPerClient;
int expectedTransactionsPerMinute;
int testDuration;
int warmupTime;
int clientsUsed;
double startTime;
GlobalState gState;
TPCCMetrics metrics;
TPCC(WorkloadContext const& ctx) : TestWorkload(ctx) {
std::string workloadName = DESCRIPTION;
warehousesPerClient = getOption(options, LiteralStringRef("warehousesPerClient"), 100);
expectedTransactionsPerMinute = getOption(options, LiteralStringRef("expectedTransactionsPerMinute"), 1000);
testDuration = getOption(options, LiteralStringRef("testDuration"), 600);
warmupTime = getOption(options, LiteralStringRef("warmupTime"), 30);
clientsUsed = getOption(options, LiteralStringRef("clientsUsed"), 40);
}
int NURand(int C, int A, int x, int y) {
return (((deterministicRandom()->randomInt(0, A + 1) | deterministicRandom()->randomInt(x, y + 1)) + C) %
(y - x + 1)) +
x;
}
StringRef genCLast(Arena& arena, int x) {
int l = x % 10;
x /= 10;
int m = x % 10;
x /= 10;
int f = x % 10;
std::stringstream ss;
ss << syllables[f] << syllables[m] << syllables[l];
return StringRef(arena, ss.str());
}
// Should call in setup
ACTOR static Future<Void> readGlobalState(TPCC* self, Database cx) {
state ReadYourWritesTransaction tr(cx);
loop {
tr.reset();
try {
Optional<Value> val = wait(tr.get(self->gState.key()));
if (val.present()) {
BinaryReader reader(val.get(), IncludeVersion());
serializer(reader, self->gState);
} else {
wait(delay(1.0));
}
return Void();
} catch (Error& e) {
wait(tr.onError(e));
}
}
}
std::string description() const override { return DESCRIPTION; }
// Transactions
ACTOR static Future<bool> newOrder(TPCC* self, Database cx, int w_id) {
state int d_id = deterministicRandom()->randomInt(0, 10);
state int c_id = self->NURand(self->gState.CRun, 1023, 1, 3000) - 1;
state int ol_cnt = deterministicRandom()->randomInt(5, 16);
state bool willRollback = deterministicRandom()->randomInt(1, 100) == 1;
state ReadYourWritesTransaction tr(cx);
try {
state Warehouse warehouse;
warehouse.w_id = w_id;
Optional<Value> wValue = wait(tr.get(warehouse.key()));
ASSERT(wValue.present());
{
BinaryReader r(wValue.get(), IncludeVersion());
serializer(r, warehouse);
}
state District district;
district.d_w_id = w_id;
district.d_id = d_id;
Optional<Value> dValue = wait(tr.get(district.key()));
ASSERT(dValue.present());
{
BinaryReader r(dValue.get(), IncludeVersion());
serializer(r, district);
}
state Customer customer;
customer.c_id = c_id;
customer.c_w_id = w_id;
customer.c_d_id = d_id;
Optional<Value> cValue = wait(tr.get(customer.key()));
ASSERT(cValue.present());
{
BinaryReader r(cValue.get(), IncludeVersion());
serializer(r, customer);
}
state Order order;
order.o_entry_d = g_network->now();
order.o_c_id = c_id;
order.o_d_id = d_id;
order.o_w_id = w_id;
order.o_ol_cnt = ol_cnt;
order.o_id = district.d_next_o_id;
++district.d_next_o_id;
{
BinaryWriter w(IncludeVersion());
serializer(w, district);
tr.set(district.key(), w.toValue());
}
state NewOrder newOrder;
newOrder.no_w_id = w_id;
newOrder.no_d_id = d_id;
newOrder.no_o_id = order.o_id;
state int ol_id = 0;
state bool allLocal = true;
for (; ol_id < order.o_ol_cnt; ++ol_id) {
if (ol_id + 1 == order.o_ol_cnt && willRollback) {
// Simulated abort - order item not found
return false;
}
state OrderLine orderLine;
orderLine.ol_number = ol_id;
orderLine.ol_w_id = w_id;
orderLine.ol_d_id = d_id;
orderLine.ol_supply_w_id = w_id;
orderLine.ol_o_id = order.o_id;
orderLine.ol_i_id = self->NURand(self->gState.CRun, 8191, 1, 100000) - 1;
orderLine.ol_quantity = deterministicRandom()->randomInt(1, 11);
if (deterministicRandom()->randomInt(0, 100) == 0) {
orderLine.ol_supply_w_id =
deterministicRandom()->randomInt(0, self->clientsUsed * self->warehousesPerClient);
}
state Item item;
item.i_id = orderLine.ol_i_id;
orderLine.ol_i_id = item.i_id;
Optional<Value> iValue = wait(tr.get(item.key()));
ASSERT(iValue.present());
{
BinaryReader r(iValue.get(), IncludeVersion());
serializer(r, item);
}
state Stock stock;
stock.s_i_id = item.i_id;
stock.s_w_id = orderLine.ol_supply_w_id;
Optional<Value> sValue = wait(tr.get(stock.key()));
ASSERT(sValue.present());
{
BinaryReader r(sValue.get(), IncludeVersion());
serializer(r, stock);
}
if (stock.s_quantity - orderLine.ol_quantity >= 10) {
stock.s_quantity -= orderLine.ol_quantity;
} else {
stock.s_quantity = (stock.s_quantity - orderLine.ol_quantity) + 91;
}
stock.s_ytd += orderLine.ol_quantity;
stock.s_order_cnt += 1;
if (orderLine.ol_supply_w_id != w_id) {
stock.s_remote_cnt += 1;
allLocal = false;
}
{
BinaryWriter w(IncludeVersion());
serializer(w, stock);
tr.set(stock.key(), w.toValue());
}
orderLine.ol_amount = orderLine.ol_quantity * item.i_price;
switch (orderLine.ol_d_id) {
case 0:
orderLine.ol_dist_info = stock.s_dist_01;
break;
case 1:
orderLine.ol_dist_info = stock.s_dist_02;
break;
case 2:
orderLine.ol_dist_info = stock.s_dist_03;
break;
case 3:
orderLine.ol_dist_info = stock.s_dist_04;
break;
case 4:
orderLine.ol_dist_info = stock.s_dist_05;
break;
case 5:
orderLine.ol_dist_info = stock.s_dist_06;
break;
case 6:
orderLine.ol_dist_info = stock.s_dist_07;
break;
case 7:
orderLine.ol_dist_info = stock.s_dist_08;
break;
case 8:
orderLine.ol_dist_info = stock.s_dist_09;
break;
case 9:
orderLine.ol_dist_info = stock.s_dist_10;
break;
}
{
BinaryWriter w(IncludeVersion());
serializer(w, orderLine);
tr.set(orderLine.key(), w.toValue());
}
}
order.o_all_local = allLocal;
{
BinaryWriter w(IncludeVersion());
serializer(w, order);
tr.set(order.key(), w.toValue());
}
{
BinaryWriter w(IncludeVersion());
serializer(w, newOrder);
tr.set(newOrder.key(), w.toValue());
}
wait(tr.commit());
} catch (Error& e) {
return false;
}
return true;
}
ACTOR static Future<Customer> getRandomCustomer(TPCC* self, ReadYourWritesTransaction* tr, int w_id, int d_id) {
state Customer result;
result.c_w_id = w_id;
result.c_d_id = d_id;
if (deterministicRandom()->randomInt(0, 100) >= 85) {
result.c_d_id = deterministicRandom()->randomInt(0, 10);
result.c_w_id = deterministicRandom()->randomInt(0, self->clientsUsed * self->warehousesPerClient);
}
if (deterministicRandom()->randomInt(0, 100) < 60) {
// select through last name
result.c_last = self->genCLast(result.arena, self->NURand(self->gState.CRun, 1023, 1, 3000) - 1);
auto s = result.indexLastKey(1);
auto begin = new (result.arena) uint8_t[s.size() + 1];
auto end = new (result.arena) uint8_t[s.size() + 1];
memcpy(begin, s.begin(), s.size());
memcpy(end, s.begin(), s.size());
begin[s.size()] = '/';
end[s.size()] = '0';
state RangeResult range =
wait(tr->getRange(KeyRangeRef(StringRef(begin, s.size() + 1), StringRef(end, s.size() + 1)), 1000));
ASSERT(range.size() > 0);
state std::vector<Customer> customers;
state int i = 0;
for (; i < range.size(); ++i) {
Optional<Value> cValue = wait(tr->get(range[i].value));
ASSERT(cValue.present());
BinaryReader r(cValue.get(), IncludeVersion());
state Customer customer;
serializer(r, customer);
customers.push_back(customer);
}
// Sort customers by first name and choose median
std::sort(customers.begin(), customers.end(), [](const Customer& cus1, const Customer& cus2) {
const std::string cus1Name = cus1.c_first.toString();
const std::string cus2Name = cus2.c_first.toString();
return (cus1Name.compare(cus2Name) < 0);
});
result = customers[customers.size() / 2];
} else {
// select through random id
result.c_id = self->NURand(self->gState.CRun, 1023, 1, 3000) - 1;
Optional<Value> val = wait(tr->get(result.key()));
ASSERT(val.present());
BinaryReader r(val.get(), IncludeVersion());
serializer(r, result);
}
return result;
}
ACTOR static Future<bool> payment(TPCC* self, Database cx, int w_id) {
state ReadYourWritesTransaction tr(cx);
state int d_id = deterministicRandom()->randomInt(0, 10);
state History history;
state Warehouse warehouse;
state District district;
history.h_amount = deterministicRandom()->random01() * 4999.0 + 1.0;
history.h_date = g_network->now();
try {
// get the customer
state Customer customer = wait(getRandomCustomer(self, &tr, w_id, d_id));
warehouse.w_id = w_id;
Optional<Value> wValue = wait(tr.get(warehouse.key()));
ASSERT(wValue.present());
{
BinaryReader r(wValue.get(), IncludeVersion());
serializer(r, warehouse);
}
warehouse.w_ytd += history.h_amount;
{
BinaryWriter w(IncludeVersion());
serializer(w, warehouse);
tr.set(warehouse.key(), w.toValue());
}
district.d_w_id = w_id;
district.d_id = d_id;
Optional<Value> dValue = wait(tr.get(district.key()));
ASSERT(dValue.present());
{
BinaryReader r(dValue.get(), IncludeVersion());
serializer(r, district);
}
district.d_ytd += history.h_amount;
customer.c_balance -= history.h_amount;
customer.c_ytd_payment += history.h_amount;
customer.c_payment_cnt += 1;
if (customer.c_credit == LiteralStringRef("BC")) {
// we must update c_data
std::stringstream ss;
ss << customer.c_id << "," << customer.c_d_id << "," << customer.c_w_id << "," << district.d_id << ","
<< w_id << history.h_amount << ";";
auto s = ss.str();
auto len = std::min(int(s.size()) + customer.c_data.size(), 500);
auto data = new (customer.arena) uint8_t[len];
std::copy(s.begin(), s.end(), reinterpret_cast<char*>(data));
std::copy(customer.c_data.begin(), customer.c_data.begin() + len - s.size(), data);
customer.c_data = StringRef(data, len);
}
{
BinaryWriter w(IncludeVersion());
serializer(w, customer);
tr.set(customer.key(), w.toValue());
}
std::stringstream ss;
ss << warehouse.w_name.toString() << " " << district.d_name.toString();
history.h_data = StringRef(history.arena, ss.str());
history.h_c_id = customer.c_id;
history.h_c_d_id = customer.c_d_id;
history.h_c_w_id = customer.c_w_id;
history.h_d_id = d_id;
history.h_w_id = w_id;
{
BinaryWriter w(IncludeVersion());
serializer(w, history);
UID k = deterministicRandom()->randomUniqueID();
BinaryWriter kW(Unversioned());
serializer(kW, k);
auto key = kW.toValue().withPrefix(LiteralStringRef("History/"));
tr.set(key, w.toValue());
}
wait(tr.commit());
} catch (Error& e) {
return false;
}
return true;
}
ACTOR static Future<bool> orderStatus(TPCC* self, Database cx, int w_id) {
state ReadYourWritesTransaction tr(cx);
state int d_id = deterministicRandom()->randomInt(0, 10);
state int i;
state Order order;
state std::vector<OrderLine> orderLines;
try {
state Customer customer = wait(getRandomCustomer(self, &tr, w_id, d_id));
order.o_w_id = customer.c_w_id;
order.o_d_id = customer.c_d_id;
order.o_c_id = customer.c_id;
RangeResult range = wait(tr.getRange(order.keyRange(1), 1, Snapshot::False, Reverse::True));
ASSERT(range.size() > 0);
{
BinaryReader r(range[0].value, IncludeVersion());
serializer(r, order);
}
for (i = 0; i < order.o_ol_cnt; ++i) {
OrderLine orderLine;
orderLine.ol_w_id = order.o_w_id;
orderLine.ol_d_id = order.o_d_id;
orderLine.ol_o_id = order.o_id;
orderLine.ol_number = i;
Optional<Value> olValue = wait(tr.get(orderLine.key()));
ASSERT(olValue.present());
BinaryReader r(olValue.get(), IncludeVersion());
OrderLine ol;
serializer(r, ol);
orderLines.push_back(ol);
}
} catch (Error& e) {
return false;
}
return true;
}
ACTOR static Future<bool> delivery(TPCC* self, Database cx, int w_id) {
state ReadYourWritesTransaction tr(cx);
state int carrier_id = deterministicRandom()->randomInt(0, 10);
state int d_id;
state NewOrder newOrder;
state Order order;
state double sumAmount = 0.0;
state Customer customer;
state int i;
try {
for (d_id = 0; d_id < 10; ++d_id) {
newOrder.no_w_id = w_id;
newOrder.no_d_id = d_id;
RangeResult range = wait(tr.getRange(newOrder.keyRange(1), 1));
if (range.size() > 0) {
{
BinaryReader r(range[0].value, IncludeVersion());
serializer(r, newOrder);
}
tr.clear(newOrder.key());
order.o_w_id = w_id;
order.o_d_id = d_id;
order.o_id = newOrder.no_o_id;
Optional<Value> oValue = wait(tr.get(order.key()));
ASSERT(oValue.present());
{
BinaryReader r(oValue.get(), IncludeVersion());
serializer(r, order);
}
order.o_carrier_id = carrier_id;
{
BinaryWriter w(IncludeVersion());
serializer(w, order);
tr.set(order.key(), w.toValue());
}
for (i = 0; i < order.o_ol_cnt; ++i) {
state OrderLine orderLine;
orderLine.ol_w_id = order.o_w_id;
orderLine.ol_d_id = order.o_d_id;
orderLine.ol_o_id = order.o_id;
orderLine.ol_number = i;
Optional<Value> olV = wait(tr.get(orderLine.key()));
ASSERT(olV.present());
BinaryReader r(olV.get(), IncludeVersion());
serializer(r, orderLine);
orderLine.ol_delivery_d = g_network->now();
sumAmount += orderLine.ol_amount;
}
customer.c_w_id = w_id;
customer.c_d_id = d_id;
customer.c_id = order.o_c_id;
Optional<Value> cV = wait(tr.get(customer.key()));
ASSERT(cV.present());
{
BinaryReader r(cV.get(), IncludeVersion());
serializer(r, customer);
}
customer.c_balance += sumAmount;
customer.c_delivery_count += 1;
{
BinaryWriter w(IncludeVersion());
serializer(w, customer);
tr.set(customer.key(), w.toValue());
}
wait(tr.commit());
}
}
} catch (Error& e) {
return false;
}
return true;
}
ACTOR static Future<bool> stockLevel(TPCC* self, Database cx, int w_id, int d_id) {
state int threshold = deterministicRandom()->randomInt(10, 21);
state Transaction tr(cx);
state District district;
state OrderLine orderLine;
state Stock stock;
state int ol_o_id;
state int low_stock = 0;
state int i;
try {
district.d_w_id = w_id;
district.d_id = d_id;
Optional<Value> dV = wait(tr.get(district.key()));
ASSERT(dV.present());
{
BinaryReader r(dV.get(), IncludeVersion());
serializer(r, district);
}
for (ol_o_id = district.d_next_o_id - 20; ol_o_id < district.d_next_o_id; ++ol_o_id) {
orderLine.ol_w_id = w_id;
orderLine.ol_d_id = d_id;
orderLine.ol_o_id = ol_o_id;
state RangeResult range = wait(tr.getRange(orderLine.keyRange(1), CLIENT_KNOBS->TOO_MANY));
ASSERT(!range.more);
ASSERT(range.size() > 0);
for (i = 0; i < range.size(); ++i) {
{
BinaryReader r(range[i].value, IncludeVersion());
serializer(r, orderLine);
}
stock.s_i_id = orderLine.ol_i_id;
stock.s_w_id = orderLine.ol_w_id;
Optional<Value> sV = wait(tr.get(stock.key()));
ASSERT(sV.present());
{
BinaryReader r(sV.get(), IncludeVersion());
serializer(r, stock);
}
if (stock.s_quantity < threshold) {
++low_stock;
}
}
}
} catch (Error& e) {
return false;
}
return true;
}
ACTOR static Future<Void> emulatedUser(TPCC* self, Database cx, int w_id, int d_id) {
// stagger users
wait(delay(20.0 * deterministicRandom()->random01()));
TraceEvent("StartingEmulatedUser").detail("Warehouse", w_id).detail("District", d_id);
loop {
auto type = deterministicRandom()->randomInt(0, 100);
Future<bool> tx;
state double txnStartTime = g_network->now();
if (type < 4) {
tx = stockLevel(self, cx, w_id, d_id);
bool committed = wait(tx);
if (self->recordMetrics()) {
TPCCMetrics::updateMetrics(committed,
self->metrics.successfulStockLevelTransactions,
self->metrics.failedStockLevelTransactions,
txnStartTime,
self->metrics.stockLevelLatencies,
self->metrics.stockLevelResponseTime,
"StockLevel");
}
wait(delay(2 + deterministicRandom()->random01() * 10));
} else if (type < 8) {
tx = delivery(self, cx, w_id);
bool committed = wait(tx);
if (self->recordMetrics()) {
TPCCMetrics::updateMetrics(committed,
self->metrics.successfulDeliveryTransactions,
self->metrics.failedDeliveryTransactions,
txnStartTime,
self->metrics.deliveryLatencies,
self->metrics.deliveryResponseTime,
"Delivery");
}
wait(delay(2 + deterministicRandom()->random01() * 10));
} else if (type < 12) {
tx = orderStatus(self, cx, w_id);
bool committed = wait(tx);
if (self->recordMetrics()) {
TPCCMetrics::updateMetrics(committed,
self->metrics.successfulOrderStatusTransactions,
self->metrics.failedOrderStatusTransactions,
txnStartTime,
self->metrics.orderStatusLatencies,
self->metrics.orderStatusResponseTime,
"OrderStatus");
}
wait(delay(2 + deterministicRandom()->random01() * 20));
} else if (type < 55) {
tx = payment(self, cx, w_id);
bool committed = wait(tx);
if (self->recordMetrics()) {
TPCCMetrics::updateMetrics(committed,
self->metrics.successfulPaymentTransactions,
self->metrics.failedPaymentTransactions,
txnStartTime,
self->metrics.paymentLatencies,
self->metrics.paymentResponseTime,
"Payment");
}
wait(delay(3 + deterministicRandom()->random01() * 24));
} else {
tx = newOrder(self, cx, w_id);
bool committed = wait(tx);
if (self->recordMetrics()) {
TPCCMetrics::updateMetrics(committed,
self->metrics.successfulNewOrderTransactions,
self->metrics.failedNewOrderTransactions,
txnStartTime,
self->metrics.newOrderLatencies,
self->metrics.newOrderResponseTime,
"NewOrder");
}
wait(delay(18 + deterministicRandom()->random01() * 24));
}
}
}
double transactionsPerMinute() const {
return metrics.successfulNewOrderTransactions * 60.0 / (testDuration - 2 * warmupTime);
}
bool recordMetrics() const {
auto now = g_network->now();
return (now > startTime + warmupTime && now < startTime + testDuration - warmupTime);
}
Future<Void> start(Database const& cx) override {
if (clientId >= clientsUsed)
return Void();
return _start(cx, this);
}
ACTOR Future<Void> _start(Database cx, TPCC* self) {
wait(readGlobalState(self, cx));
self->startTime = g_network->now();
int startWID = self->clientId * self->warehousesPerClient;
int endWID = startWID + self->warehousesPerClient;
state int w_id;
state int d_id;
state std::vector<Future<Void>> emulatedUsers;
for (w_id = startWID; w_id < endWID; ++w_id) {
for (d_id = 0; d_id < 10; ++d_id) {
emulatedUsers.push_back(timeout(emulatedUser(self, cx, w_id, d_id), self->testDuration, Void()));
}
}
wait(waitForAll(emulatedUsers));
return Void();
}
Future<bool> check(Database const& cx) override {
return (transactionsPerMinute() > expectedTransactionsPerMinute);
}
void getMetrics(std::vector<PerfMetric>& m) override {
double multiplier = static_cast<double>(clientCount) / static_cast<double>(clientsUsed);
m.emplace_back("Transactions Per Minute", transactionsPerMinute(), Averaged::False);
m.emplace_back("Successful StockLevel Transactions", metrics.successfulStockLevelTransactions, Averaged::False);
m.emplace_back("Successful Delivery Transactions", metrics.successfulDeliveryTransactions, Averaged::False);
m.emplace_back(
"Successful OrderStatus Transactions", metrics.successfulOrderStatusTransactions, Averaged::False);
m.emplace_back("Successful Payment Transactions", metrics.successfulPaymentTransactions, Averaged::False);
m.emplace_back("Successful NewOrder Transactions", metrics.successfulNewOrderTransactions, Averaged::False);
m.emplace_back("Failed StockLevel Transactions", metrics.failedStockLevelTransactions, Averaged::False);
m.emplace_back("Failed Delivery Transactions", metrics.failedDeliveryTransactions, Averaged::False);
m.emplace_back("Failed OrderStatus Transactions", metrics.failedOrderStatusTransactions, Averaged::False);
m.emplace_back("Failed Payment Transactions", metrics.failedPaymentTransactions, Averaged::False);
m.emplace_back("Failed NewOrder Transactions", metrics.failedNewOrderTransactions, Averaged::False);
m.emplace_back("Mean StockLevel Latency",
(clientId < clientsUsed)
? (multiplier * metrics.stockLevelResponseTime / metrics.successfulStockLevelTransactions)
: 0.0,
Averaged::True);
m.emplace_back("Mean Delivery Latency",
(clientId < clientsUsed)
? (multiplier * metrics.deliveryResponseTime / metrics.successfulDeliveryTransactions)
: 0.0,
Averaged::True);
m.emplace_back("Mean OrderStatus Repsonse Time",
(clientId < clientsUsed)
? (multiplier * metrics.orderStatusResponseTime / metrics.successfulOrderStatusTransactions)
: 0.0,
Averaged::True);
m.emplace_back("Mean Payment Latency",
(clientId < clientsUsed)
? (multiplier * metrics.paymentResponseTime / metrics.successfulPaymentTransactions)
: 0.0,
Averaged::True);
m.emplace_back("Mean NewOrder Latency",
(clientId < clientsUsed)
? (multiplier * metrics.newOrderResponseTime / metrics.successfulNewOrderTransactions)
: 0.0,
Averaged::True);
metrics.sort();
m.emplace_back(
"Median StockLevel Latency", multiplier * TPCCMetrics::median(metrics.stockLevelLatencies), Averaged::True);
m.emplace_back(
"Median Delivery Latency", multiplier * TPCCMetrics::median(metrics.deliveryLatencies), Averaged::True);
m.emplace_back("Median OrderStatus Latency",
multiplier * TPCCMetrics::median(metrics.orderStatusLatencies),
Averaged::True);
m.emplace_back(
"Median Payment Latency", multiplier * TPCCMetrics::median(metrics.paymentLatencies), Averaged::True);
m.emplace_back(
"Median NewOrder Latency", multiplier * TPCCMetrics::median(metrics.newOrderLatencies), Averaged::True);
m.emplace_back("90th Percentile StockLevel Latency",
multiplier * TPCCMetrics::percentile_90(metrics.stockLevelLatencies),
Averaged::True);
m.emplace_back("90th Percentile Delivery Latency",
multiplier * TPCCMetrics::percentile_90(metrics.deliveryLatencies),
Averaged::True);
m.emplace_back("90th Percentile OrderStatus Latency",
multiplier * TPCCMetrics::percentile_90(metrics.orderStatusLatencies),
Averaged::True);
m.emplace_back("90th Percentile Payment Latency",
multiplier * TPCCMetrics::percentile_90(metrics.paymentLatencies),
Averaged::True);
m.emplace_back("90th Percentile NewOrder Latency",
multiplier * TPCCMetrics::percentile_90(metrics.newOrderLatencies),
Averaged::True);
m.emplace_back("99th Percentile StockLevel Latency",
multiplier * TPCCMetrics::percentile_99(metrics.stockLevelLatencies),
Averaged::True);
m.emplace_back("99th Percentile Delivery Latency",
multiplier * TPCCMetrics::percentile_99(metrics.deliveryLatencies),
Averaged::True);
m.emplace_back("99th Percentile OrderStatus Latency",
multiplier * TPCCMetrics::percentile_99(metrics.orderStatusLatencies),
Averaged::True);
m.emplace_back("99th Percentile Payment Latency",
multiplier * TPCCMetrics::percentile_99(metrics.paymentLatencies),
Averaged::True);
m.emplace_back("99th Percentile NewOrder Latency",
multiplier * TPCCMetrics::percentile_99(metrics.newOrderLatencies),
Averaged::True);
}
};
} // namespace
WorkloadFactory<TPCC> TPCCWorkloadFactory(TPCC::DESCRIPTION);

View File

@ -22,6 +22,7 @@
#include <limits>
#include "fdbclient/FDBOptions.g.h"
#include "fdbclient/TenantManagement.actor.h"
#include "fdbclient/TenantSpecialKeys.actor.h"
#include "fdbrpc/simulator.h"
#include "fdbserver/workloads/workloads.actor.h"
#include "fdbserver/Knobs.h"
@ -49,8 +50,9 @@ struct TenantManagementWorkload : TestWorkload {
const TenantName tenantNamePrefix = "tenant_management_workload_"_sr;
TenantName localTenantNamePrefix;
const Key specialKeysTenantMapPrefix = TenantMapRangeImpl::submoduleRange.begin.withPrefix(
SpecialKeySpace::getModuleRange(SpecialKeySpace::MODULE::MANAGEMENT).begin);
const Key specialKeysTenantMapPrefix = SpecialKeySpace::getModuleRange(SpecialKeySpace::MODULE::MANAGEMENT)
.begin.withSuffix(TenantRangeImpl<true>::submoduleRange.begin)
.withSuffix(TenantRangeImpl<true>::mapSubRange.begin);
int maxTenants;
double testDuration;

View File

@ -36,6 +36,7 @@ void forceLinkMutationLogReaderTests();
void forceLinkSimKmsConnectorTests();
void forceLinkIThreadPoolTests();
void forceLinkTokenSignTests();
void forceLinkJsonWebKeySetTests();
void forceLinkVersionVectorTests();
void forceLinkRESTClientTests();
void forceLinkRESTUtilsTests();
@ -86,6 +87,7 @@ struct UnitTestWorkload : TestWorkload {
forceLinkSimKmsConnectorTests();
forceLinkIThreadPoolTests();
forceLinkTokenSignTests();
forceLinkJsonWebKeySetTests();
forceLinkVersionVectorTests();
forceLinkRESTClientTests();
forceLinkRESTUtilsTests();

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