Merge remote-tracking branch 'origin/main' into mako-authz-token-generation

This commit is contained in:
Junhyun Shim 2022-12-15 17:02:19 -08:00
commit 77e523ce91
58 changed files with 1746 additions and 575 deletions

View File

@ -228,7 +228,11 @@ if(NOT WIN32)
# Make sure that fdb_c.h is compatible with c90
add_executable(fdb_c90_test test/fdb_c90_test.c)
set_property(TARGET fdb_c90_test PROPERTY C_STANDARD 90)
target_compile_options(fdb_c90_test PRIVATE -Wall -Wextra -Wpedantic -Werror)
if (CMAKE_CXX_COMPILER_ID STREQUAL "Clang")
target_compile_options(fdb_c90_test PRIVATE -Wall -Wextra -Wpedantic -Wno-gnu-line-marker -Werror)
else ()
target_compile_options(fdb_c90_test PRIVATE -Wall -Wextra -Wpedantic -Werror)
endif ()
target_link_libraries(fdb_c90_test PRIVATE fdb_c)
endif()

View File

@ -166,20 +166,26 @@ else()
set(SANITIZER_COMPILE_OPTIONS)
set(SANITIZER_LINK_OPTIONS)
# we always compile with debug symbols. For release builds CPack will strip them out
# and create a debuginfo rpm
add_compile_options(-fno-omit-frame-pointer -gz)
add_link_options(-gz)
add_compile_options(-fno-omit-frame-pointer)
if(FDB_RELEASE OR FULL_DEBUG_SYMBOLS OR CMAKE_BUILD_TYPE STREQUAL "Debug")
# Configure with FULL_DEBUG_SYMBOLS=ON to generate all symbols for debugging with gdb
# Also generating full debug symbols in release builds, because they are packaged
# separately and installed optionally
# Also generating full debug symbols in release builds. CPack will strip them out
# and create a debuginfo rpm
add_compile_options(-ggdb)
else()
# Generating minimal debug symbols by default. They are sufficient for testing purposes
add_compile_options(-ggdb1)
endif()
if(NOT FDB_RELEASE)
# Enable compression of the debug sections. This reduces the size of the binaries several times.
# We do not enable it release builds, because CPack fails to generate debuginfo packages when
# compression is enabled
add_compile_options(-gz)
add_link_options(-gz)
endif()
if(TRACE_PC_GUARD_INSTRUMENTATION_LIB)
add_compile_options(-fsanitize-coverage=trace-pc-guard)
link_libraries(${TRACE_PC_GUARD_INSTRUMENTATION_LIB})

View File

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

View File

@ -541,9 +541,11 @@ void initHelp() {
"Fetch the current read version",
"Displays the current read version of the database or currently running transaction.");
helpMap["quota"] = CommandHelp("quota",
"quota [get <tag> [reserved_throughput|total_throughput] | set <tag> "
"[reserved_throughput|total_throughput] <value> | clear <tag>]",
"Get, modify, or clear the throughput quota for the specified tag.");
"quota [get <tag> [reserved_throughput|total_throughput|storage] | "
"set <tag> [reserved_throughput|total_throughput|storage] <value> | "
"clear <tag>]",
"Get, modify, or clear the reserved/total throughput quota (in bytes/s) or "
"storage quota (in bytes) for the specified tag.");
helpMap["reset"] =
CommandHelp("reset",
"reset the current transaction",

View File

@ -162,3 +162,51 @@ std::string describe(const std::string& s) {
std::string describe(UID const& item) {
return item.shortString();
}
TEST_CASE("/KeyRangeUtil/KeyRangeComplement") {
Key begin = "b"_sr;
Key end = "y"_sr;
KeyRangeRef range(begin, end);
{
Key b = "c"_sr;
Key e = "f"_sr;
std::vector<KeyRangeRef> result = range - KeyRangeRef(b, e);
ASSERT(result.size() == 2);
ASSERT(result[0] == KeyRangeRef("b"_sr, "c"_sr));
ASSERT(result[1] == KeyRangeRef("f"_sr, "y"_sr));
}
{
Key b = "1"_sr;
Key e = "9"_sr;
std::vector<KeyRangeRef> result = range - KeyRangeRef(b, e);
ASSERT(result.size() == 1);
ASSERT(result[0] == KeyRangeRef("b"_sr, "y"_sr));
}
{
Key b = "a"_sr;
Key e = "f"_sr;
std::vector<KeyRangeRef> result = range - KeyRangeRef(b, e);
ASSERT(result.size() == 1);
ASSERT(result[0] == KeyRangeRef("f"_sr, "y"_sr));
}
{
Key b = "f"_sr;
Key e = "z"_sr;
std::vector<KeyRangeRef> result = range - KeyRangeRef(b, e);
ASSERT(result.size() == 1);
ASSERT(result[0] == KeyRangeRef("b"_sr, "f"_sr));
}
{
Key b = "a"_sr;
Key e = "z"_sr;
std::vector<KeyRangeRef> result = range - KeyRangeRef(b, e);
ASSERT(result.size() == 0);
}
return Void();
}

View File

@ -21,6 +21,7 @@
#include "fdbclient/IdempotencyId.actor.h"
#include "fdbclient/ReadYourWrites.h"
#include "fdbclient/SystemData.h"
#include "flow/BooleanParam.h"
#include "flow/UnitTest.h"
#include "flow/actorcompiler.h" // this has to be the last include
@ -202,4 +203,172 @@ void decodeIdempotencyKey(KeyRef key, Version& commitVersion, uint8_t& highOrder
reader >> commitVersion;
commitVersion = bigEndian64(commitVersion);
reader >> highOrderBatchIndex;
}
}
FDB_BOOLEAN_PARAM(Oldest);
// Find the youngest or oldest idempotency id key in `range` (depending on `oldest`)
// Write the timestamp to `*time` and the version to `*version` when non-null.
ACTOR static Future<Optional<Key>> getBoundary(Reference<ReadYourWritesTransaction> tr,
KeyRange range,
Oldest oldest,
Version* version,
int64_t* time) {
RangeResult result =
wait(tr->getRange(range, /*limit*/ 1, Snapshot::False, oldest ? Reverse::False : Reverse::True));
if (!result.size()) {
return Optional<Key>();
}
if (version != nullptr) {
BinaryReader rd(result.front().key, Unversioned());
rd.readBytes(idempotencyIdKeys.begin.size());
rd >> *version;
*version = bigEndian64(*version);
}
if (time != nullptr) {
BinaryReader rd(result.front().value, IncludeVersion());
rd >> *time;
}
return result.front().key;
}
ACTOR Future<JsonBuilderObject> getIdmpKeyStatus(Database db) {
state Reference<ReadYourWritesTransaction> tr = makeReference<ReadYourWritesTransaction>(db);
state int64_t size;
state IdempotencyIdsExpiredVersion expired;
state KeyBackedObjectProperty<IdempotencyIdsExpiredVersion, _Unversioned> expiredKey(idempotencyIdsExpiredVersion,
Unversioned());
state int64_t oldestIdVersion = 0;
state int64_t oldestIdTime = 0;
loop {
try {
tr->setOption(FDBTransactionOptions::READ_SYSTEM_KEYS);
tr->setOption(FDBTransactionOptions::READ_LOCK_AWARE);
wait(store(size, tr->getEstimatedRangeSizeBytes(idempotencyIdKeys)) &&
store(expired, expiredKey.getD(tr)) &&
success(getBoundary(tr, idempotencyIdKeys, Oldest::True, &oldestIdVersion, &oldestIdTime)));
JsonBuilderObject result;
result["size_bytes"] = size;
if (expired.expired != 0) {
result["expired_version"] = expired.expired;
}
if (expired.expiredTime != 0) {
result["expired_age"] = int64_t(now()) - expired.expiredTime;
}
if (oldestIdVersion != 0) {
result["oldest_id_version"] = oldestIdVersion;
}
if (oldestIdTime != 0) {
result["oldest_id_age"] = int64_t(now()) - oldestIdTime;
}
return result;
} catch (Error& e) {
wait(tr->onError(e));
}
}
}
ACTOR Future<Void> cleanIdempotencyIds(Database db, double minAgeSeconds) {
state int64_t idmpKeySize;
state int64_t candidateDeleteSize;
state KeyRange finalRange;
state Reference<ReadYourWritesTransaction> tr;
// Only assigned to once
state Key oldestKey;
state Version oldestVersion;
state int64_t oldestTime;
// Assigned to multiple times looking for a suitable range
state Version candidateDeleteVersion;
state int64_t candidateDeleteTime;
state KeyRange candidateRangeToClean;
tr = makeReference<ReadYourWritesTransaction>(db);
loop {
try {
tr->setOption(FDBTransactionOptions::ACCESS_SYSTEM_KEYS);
tr->setOption(FDBTransactionOptions::LOCK_AWARE);
// Check if any keys are older than minAgeSeconds
Optional<Key> oldestKey_ =
wait(getBoundary(tr, idempotencyIdKeys, Oldest::True, &oldestVersion, &oldestTime));
if (!oldestKey_.present()) {
break;
}
oldestKey = oldestKey_.get();
if (int64_t(now()) - oldestTime < minAgeSeconds) {
break;
}
// Only used for a trace event
wait(store(idmpKeySize, tr->getEstimatedRangeSizeBytes(idempotencyIdKeys)));
// Get the version of the most recent idempotency ID
wait(success(
getBoundary(tr, idempotencyIdKeys, Oldest::False, &candidateDeleteVersion, &candidateDeleteTime)));
// Keep dividing the candidate range until clearing it would not delete something younger than
// minAgeSeconds
loop {
candidateRangeToClean =
KeyRangeRef(oldestKey,
BinaryWriter::toValue(bigEndian64(candidateDeleteVersion + 1), Unversioned())
.withPrefix(idempotencyIdKeys.begin));
// We know that we're okay deleting oldestVersion at this point. Go ahead and do that.
if (oldestVersion == candidateDeleteVersion) {
break;
}
// Find the youngest key in candidate range
wait(success(getBoundary(
tr, candidateRangeToClean, Oldest::False, &candidateDeleteVersion, &candidateDeleteTime)));
// Update the range so that it ends at an idempotency id key. Since we're binary searching, the
// candidate range was probably too large before.
candidateRangeToClean =
KeyRangeRef(oldestKey,
BinaryWriter::toValue(bigEndian64(candidateDeleteVersion + 1), Unversioned())
.withPrefix(idempotencyIdKeys.begin));
wait(store(candidateDeleteSize, tr->getEstimatedRangeSizeBytes(candidateRangeToClean)));
int64_t youngestAge = int64_t(now()) - candidateDeleteTime;
TraceEvent("IdempotencyIdsCleanerCandidateDelete")
.detail("Range", candidateRangeToClean.toString())
.detail("IdmpKeySizeEstimate", idmpKeySize)
.detail("YoungestIdAge", youngestAge)
.detail("MinAgeSeconds", minAgeSeconds)
.detail("ClearRangeSizeEstimate", candidateDeleteSize);
if (youngestAge > minAgeSeconds) {
break;
}
candidateDeleteVersion = (oldestVersion + candidateDeleteVersion) / 2;
}
finalRange = KeyRangeRef(idempotencyIdKeys.begin, candidateRangeToClean.end);
if (!finalRange.empty()) {
tr->addReadConflictRange(finalRange);
tr->clear(finalRange);
tr->set(
idempotencyIdsExpiredVersion,
ObjectWriter::toValue(IdempotencyIdsExpiredVersion{ candidateDeleteVersion, candidateDeleteTime },
Unversioned()));
TraceEvent("IdempotencyIdsCleanerAttempt")
.detail("Range", finalRange.toString())
.detail("IdmpKeySizeEstimate", idmpKeySize)
.detail("ClearRangeSizeEstimate", candidateDeleteSize)
.detail("ExpiredVersion", candidateDeleteVersion)
.detail("ExpiredVersionAgeEstimate", static_cast<int64_t>(now()) - candidateDeleteTime);
wait(tr->commit());
}
break;
} catch (Error& e) {
TraceEvent("IdempotencyIdsCleanerError").error(e);
wait(tr->onError(e));
}
}
return Void();
}

View File

@ -1186,7 +1186,7 @@ MultiVersionTransaction::MultiVersionTransaction(Reference<MultiVersionDatabase>
UniqueOrderedOptionList<FDBTransactionOptions> defaultOptions)
: db(db), tenant(tenant), startTime(timer_monotonic()), timeoutTsav(new ThreadSingleAssignmentVar<Void>()) {
setDefaultOptions(defaultOptions);
updateTransaction();
updateTransaction(false);
}
void MultiVersionTransaction::setDefaultOptions(UniqueOrderedOptionList<FDBTransactionOptions> options) {
@ -1194,7 +1194,7 @@ void MultiVersionTransaction::setDefaultOptions(UniqueOrderedOptionList<FDBTrans
std::copy(options.begin(), options.end(), std::back_inserter(persistentOptions));
}
void MultiVersionTransaction::updateTransaction() {
void MultiVersionTransaction::updateTransaction(bool setPersistentOptions) {
TransactionInfo newTr;
if (tenant.present()) {
ASSERT(tenant.get());
@ -1211,25 +1211,32 @@ void MultiVersionTransaction::updateTransaction() {
newTr.onChange = currentDb.onChange;
}
Optional<StringRef> timeout;
for (auto option : persistentOptions) {
if (option.first == FDBTransactionOptions::TIMEOUT) {
timeout = option.second.castTo<StringRef>();
} else if (newTr.transaction) {
newTr.transaction->setOption(option.first, option.second.castTo<StringRef>());
// When called from the constructor or from reset(), all persistent options are database options and therefore
// alredy set on newTr.transaction if it got created sucessfully. If newTr.transaction could not be created (i.e.,
// because no database with a matching version is present), the local timeout set in setTimeout() applies, so we
// need to set it.
if (setPersistentOptions || !newTr.transaction) {
Optional<StringRef> timeout;
for (auto option : persistentOptions) {
if (option.first == FDBTransactionOptions::TIMEOUT) {
timeout = option.second.castTo<StringRef>();
} else if (newTr.transaction) {
newTr.transaction->setOption(option.first, option.second.castTo<StringRef>());
}
}
// Setting a timeout can immediately cause a transaction to fail. The only timeout
// that matters is the one most recently set, so we ignore any earlier set timeouts
// that might inadvertently fail the transaction.
if (timeout.present()) {
if (newTr.transaction) {
newTr.transaction->setOption(FDBTransactionOptions::TIMEOUT, timeout);
resetTimeout();
} else {
setTimeout(timeout);
}
}
}
// Setting a timeout can immediately cause a transaction to fail. The only timeout
// that matters is the one most recently set, so we ignore any earlier set timeouts
// that might inadvertently fail the transaction.
if (timeout.present()) {
setTimeout(timeout);
if (newTr.transaction) {
newTr.transaction->setOption(FDBTransactionOptions::TIMEOUT, timeout);
}
}
lock.enter();
transaction = newTr;
lock.leave();
@ -1543,7 +1550,7 @@ void MultiVersionTransaction::setOption(FDBTransactionOptions::Option option, Op
ThreadFuture<Void> MultiVersionTransaction::onError(Error const& e) {
if (e.code() == error_code_cluster_version_changed) {
updateTransaction();
updateTransaction(true);
return ThreadFuture<Void>(Void());
} else {
auto f = executeOperation(&ITransaction::onError, e);
@ -1551,7 +1558,7 @@ ThreadFuture<Void> MultiVersionTransaction::onError(Error const& e) {
if (ready.isError() && ready.getError().code() == error_code_cluster_version_changed) {
// In case of a cluster version change, upgrade (or downgrade) the transaction
// and let it to be retried independently of the original error
updateTransaction();
updateTransaction(true);
return ErrorOr<ThreadFuture<Void>>(Void());
}
// In all other cases forward the result of the inner onError call
@ -1638,6 +1645,18 @@ void MultiVersionTransaction::setTimeout(Optional<StringRef> value) {
}
}
// Removes timeout if set. This timeout only applies if we don't have an underlying database object to connect with.
void MultiVersionTransaction::resetTimeout() {
ThreadFuture<Void> prevTimeout;
{ // lock scope
ThreadSpinLockHolder holder(timeoutLock);
prevTimeout = currentTimeout;
currentTimeout = ThreadFuture<Void>();
}
if (prevTimeout.isValid()) {
prevTimeout.cancel();
}
}
// Creates a ThreadFuture<T> that will signal an error if the transaction times out.
template <class T>
ThreadFuture<T> MultiVersionTransaction::makeTimeout() {
@ -1696,7 +1715,7 @@ void MultiVersionTransaction::reset() {
}
setDefaultOptions(db->dbState->transactionDefaultOptions);
updateTransaction();
updateTransaction(false);
}
MultiVersionTransaction::~MultiVersionTransaction() {

View File

@ -9545,6 +9545,9 @@ ACTOR Future<Void> changeFeedTSSValidator(ChangeFeedStreamRequest req,
Version next = waitNext(data->get().ssStreamSummary.getFuture());
ssSummary.push_back(next);
} catch (Error& e) {
if (e.code() == error_code_actor_cancelled) {
throw;
}
if (e.code() != error_code_end_of_stream) {
data->get().complete();
if (e.code() != error_code_operation_cancelled) {

View File

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

View File

@ -245,7 +245,11 @@ void ServerKnobs::initialize(Randomize randomize, ClientKnobs* clientKnobs, IsSi
init( ALL_DATA_REMOVED_DELAY, 1.0 );
init( INITIAL_FAILURE_REACTION_DELAY, 30.0 ); if( randomize && BUGGIFY ) INITIAL_FAILURE_REACTION_DELAY = 0.0;
init( CHECK_TEAM_DELAY, 30.0 );
init( PERPETUAL_WIGGLE_DELAY, 50.0 );
// This is a safety knob to avoid busy spinning and the case a small cluster don't have enough space when excluding and including too fast. The basic idea is let PW wait for the re-included storage to take on data before wiggling the next one.
// This knob's ideal value would vary by cluster based on its size and disk type. In the meanwhile, the wiggle will also wait until the storage load is almost (85%) balanced.
init( PERPETUAL_WIGGLE_DELAY, 60 );
init( PERPETUAL_WIGGLE_SMALL_LOAD_RATIO, 10 );
init( PERPETUAL_WIGGLE_MIN_BYTES_BALANCE_RATIO, 0.85 );
init( PERPETUAL_WIGGLE_DISABLE_REMOVER, true );
init( LOG_ON_COMPLETION_DELAY, DD_QUEUE_LOGGING_INTERVAL );
init( BEST_TEAM_MAX_TEAM_TRIES, 10 );
@ -388,6 +392,7 @@ void ServerKnobs::initialize(Randomize randomize, ClientKnobs* clientKnobs, IsSi
init( ROCKSDB_WRITER_THREAD_PRIORITY, 0 );
init( ROCKSDB_BACKGROUND_PARALLELISM, 4 );
init( ROCKSDB_READ_PARALLELISM, 4 );
init( ROCKSDB_CHECKPOINT_READER_PARALLELISM, 4 );
// If true, do not process and store RocksDB logs
init( ROCKSDB_MUTE_LOGS, true );
// Use a smaller memtable in simulation to avoid OOMs.
@ -407,6 +412,8 @@ void ServerKnobs::initialize(Randomize randomize, ClientKnobs* clientKnobs, IsSi
init( ROCKSDB_READ_VALUE_TIMEOUT, 5.0 ); if (isSimulated) ROCKSDB_READ_VALUE_TIMEOUT = 5 * 60;
init( ROCKSDB_READ_VALUE_PREFIX_TIMEOUT, 5.0 ); if (isSimulated) ROCKSDB_READ_VALUE_PREFIX_TIMEOUT = 5 * 60;
init( ROCKSDB_READ_RANGE_TIMEOUT, 5.0 ); if (isSimulated) ROCKSDB_READ_RANGE_TIMEOUT = 5 * 60;
init( ROCKSDB_READ_CHECKPOINT_TIMEOUT, 60.0 ); if (isSimulated) ROCKSDB_READ_CHECKPOINT_TIMEOUT = 5 * 60;
init( ROCKSDB_CHECKPOINT_READ_AHEAD_SIZE, 2 << 20 ); // 2M
init( ROCKSDB_READ_QUEUE_WAIT, 1.0 );
init( ROCKSDB_READ_QUEUE_HARD_MAX, 1000 );
init( ROCKSDB_READ_QUEUE_SOFT_MAX, 500 );
@ -1062,6 +1069,10 @@ void ServerKnobs::initialize(Randomize randomize, ClientKnobs* clientKnobs, IsSi
// Drop in-memory state associated with an idempotency id after this many seconds. Once dropped, this id cannot be
// expired proactively, but will eventually get cleaned up by the idempotency id cleaner.
init( IDEMPOTENCY_ID_IN_MEMORY_LIFETIME, 10);
// Attempt to clean old idempotency ids automatically this often
init( IDEMPOTENCY_IDS_CLEANER_POLLING_INTERVAL, 10);
// Don't clean idempotency ids younger than this
init( IDEMPOTENCY_IDS_MIN_AGE_SECONDS, 3600 * 24 * 7);
// clang-format on

View File

@ -425,6 +425,22 @@ inline KeyRangeRef operator&(const KeyRangeRef& lhs, const KeyRangeRef& rhs) {
return KeyRangeRef(b, e);
}
// Calculates the complement of `lhs` from `rhs`.
inline std::vector<KeyRangeRef> operator-(const KeyRangeRef& lhs, const KeyRangeRef& rhs) {
if ((lhs & rhs).empty()) {
return { lhs };
}
std::vector<KeyRangeRef> result;
if (lhs.begin < rhs.begin) {
result.push_back(KeyRangeRef(lhs.begin, rhs.begin));
}
if (lhs.end > rhs.end) {
result.push_back(KeyRangeRef(rhs.end, lhs.end));
}
return result;
}
struct KeyValueRef {
KeyRef key;
ValueRef value;
@ -1505,6 +1521,8 @@ struct EncryptionAtRestMode {
bool operator==(const EncryptionAtRestMode& e) const { return isEquals(e); }
bool operator!=(const EncryptionAtRestMode& e) const { return !isEquals(e); }
bool isEncryptionEnabled() const { return mode != EncryptionAtRestMode::DISABLED; }
static EncryptionAtRestMode fromValueRef(Optional<ValueRef> val) {
if (!val.present()) {
return DISABLED;

View File

@ -327,13 +327,8 @@ Future<ConfigurationResult> changeConfig(Reference<DB> db, std::map<std::string,
for (auto kv : m) {
newConfig.set(kv.first, kv.second);
}
if (!newConfig.isValid()) {
return ConfigurationResult::INVALID_CONFIGURATION;
}
if (!isEncryptionAtRestModeConfigValid(oldConfig, m, creating)) {
return ConfigurationResult::INVALID_CONFIGURATION;
}
if (!isTenantModeModeConfigValid(oldConfig, newConfig)) {
if (!newConfig.isValid() || !isEncryptionAtRestModeConfigValid(oldConfig, m, creating) ||
!isTenantModeModeConfigValid(oldConfig, newConfig)) {
return ConfigurationResult::INVALID_CONFIGURATION;
}

View File

@ -29,6 +29,7 @@
#pragma once
#include "fdbclient/FDBTypes.h"
#include "fdbclient/JsonBuilder.h"
#include "fdbclient/PImpl.h"
#include "flow/Arena.h"
#include "flow/IRandom.h"
@ -43,11 +44,14 @@ struct CommitResult {
// The type of the value stored at the key |idempotencyIdsExpiredVersion|
struct IdempotencyIdsExpiredVersion {
static constexpr auto file_identifier = 3746945;
// Any version at or below expired might have had its idempotency id expired. Any version greater than `expired`
// definitely has not had it's idempotency id expired.
Version expired = 0;
int64_t expiredTime = 0;
template <class Archive>
void serialize(Archive& ar) {
serializer(ar, expired);
serializer(ar, expired, expiredTime);
}
};
@ -185,5 +189,14 @@ KeyRangeRef makeIdempotencySingleKeyRange(Arena& arena, Version version, uint8_t
void decodeIdempotencyKey(KeyRef key, Version& commitVersion, uint8_t& highOrderBatchIndex);
ACTOR Future<JsonBuilderObject> getIdmpKeyStatus(Database db);
// Delete zero or more idempotency ids older than minAgeSeconds
//
// Normally idempotency ids are deleted as part of the normal commit process, so this only needs to clean ids that
// leaked during a failure scenario. The rate of leaked idempotency ids should be low. The rate is zero during normal
// operation, and proportional to the number of in-flight transactions during a failure scenario.
ACTOR Future<Void> cleanIdempotencyIds(Database db, double minAgeSeconds);
#include "flow/unactorcompiler.h"
#endif

View File

@ -1444,20 +1444,25 @@ struct DeleteTenantImpl {
throw tenant_not_found();
}
if (tenantEntry.get().renamePair.present()) {
ASSERT(tenantEntry.get().tenantState == TenantState::RENAMING_FROM ||
tenantEntry.get().tenantState == TenantState::REMOVING);
self->pairName = tenantEntry.get().renamePair.get();
}
if (tenantEntry.get().tenantState != TenantState::REMOVING) {
// Disallow removing the "new" name of a renamed tenant before it completes
if (tenantEntry.get().tenantState == TenantState::RENAMING_TO) {
throw tenant_not_found();
}
state TenantMapEntry updatedEntry = tenantEntry.get();
// Check if we are deleting a tenant in the middle of a rename
if (updatedEntry.renamePair.present()) {
ASSERT(updatedEntry.tenantState == TenantState::RENAMING_FROM);
self->pairName = updatedEntry.renamePair.get();
}
updatedEntry.tenantState = TenantState::REMOVING;
ManagementClusterMetadata::tenantMetadata().tenantMap.set(tr, self->tenantName, updatedEntry);
ManagementClusterMetadata::tenantMetadata().lastTenantModification.setVersionstamp(tr, Versionstamp(), 0);
// If this has a rename pair, also mark the other entry for deletion
if (self->pairName.present()) {
state Optional<TenantMapEntry> pairEntry = wait(tryGetTenantTransaction(tr, self->pairName.get()));
@ -1494,7 +1499,8 @@ struct DeleteTenantImpl {
return Void();
}
ASSERT(tenantEntry.get().tenantState == TenantState::REMOVING);
ASSERT(tenantEntry.get().tenantState == TenantState::REMOVING &&
(pairDelete || tenantEntry.get().renamePair == self->pairName));
// Erase the tenant entry itself
ManagementClusterMetadata::tenantMetadata().tenantMap.erase(tr, tenantName);

View File

@ -779,6 +779,8 @@ private:
// if we don't have an underlying database object to connect with.
void setTimeout(Optional<StringRef> value);
void resetTimeout();
// Creates a ThreadFuture<T> that will signal an error if the transaction times out.
template <class T>
ThreadFuture<T> makeTimeout();
@ -792,7 +794,7 @@ private:
TransactionInfo transaction;
TransactionInfo getTransaction();
void updateTransaction();
void updateTransaction(bool setPersistentOptions);
void setDefaultOptions(UniqueOrderedOptionList<FDBTransactionOptions> options);
template <class T, class... Args>

View File

@ -186,9 +186,14 @@ public:
double METRIC_DELAY;
double ALL_DATA_REMOVED_DELAY;
double INITIAL_FAILURE_REACTION_DELAY;
double CHECK_TEAM_DELAY;
double PERPETUAL_WIGGLE_DELAY;
bool PERPETUAL_WIGGLE_DISABLE_REMOVER;
double CHECK_TEAM_DELAY; // Perpetual wiggle check cluster team healthy
double PERPETUAL_WIGGLE_SMALL_LOAD_RATIO; // If the average load of storage server is less than this ratio * average
// shard bytes, the perpetual wiggle won't consider the available space
// load balance in the cluster
double PERPETUAL_WIGGLE_MIN_BYTES_BALANCE_RATIO; // target min : average space load balance ratio after re-include
// before perpetual wiggle will start the next wiggle
double PERPETUAL_WIGGLE_DELAY; // The max interval between the last wiggle finish and the next wiggle start
bool PERPETUAL_WIGGLE_DISABLE_REMOVER; // Whether the start of perpetual wiggle replace team remover
double LOG_ON_COMPLETION_DELAY;
int BEST_TEAM_MAX_TEAM_TRIES;
int BEST_TEAM_OPTION_COUNT;
@ -321,6 +326,7 @@ public:
int ROCKSDB_WRITER_THREAD_PRIORITY;
int ROCKSDB_BACKGROUND_PARALLELISM;
int ROCKSDB_READ_PARALLELISM;
int ROCKSDB_CHECKPOINT_READER_PARALLELISM;
int64_t ROCKSDB_MEMTABLE_BYTES;
bool ROCKSDB_LEVEL_STYLE_COMPACTION;
bool ROCKSDB_UNSAFE_AUTO_FSYNC;
@ -332,6 +338,8 @@ public:
double ROCKSDB_READ_VALUE_TIMEOUT;
double ROCKSDB_READ_VALUE_PREFIX_TIMEOUT;
double ROCKSDB_READ_RANGE_TIMEOUT;
double ROCKSDB_READ_CHECKPOINT_TIMEOUT;
int64_t ROCKSDB_CHECKPOINT_READ_AHEAD_SIZE;
double ROCKSDB_READ_QUEUE_WAIT;
int ROCKSDB_READ_QUEUE_SOFT_MAX;
int ROCKSDB_READ_QUEUE_HARD_MAX;
@ -1018,6 +1026,8 @@ public:
// Idempotency ids
double IDEMPOTENCY_ID_IN_MEMORY_LIFETIME;
double IDEMPOTENCY_IDS_CLEANER_POLLING_INTERVAL;
double IDEMPOTENCY_IDS_MIN_AGE_SECONDS;
ServerKnobs(Randomize, ClientKnobs*, IsSimulated);
void initialize(Randomize, ClientKnobs*, IsSimulated);

View File

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

View File

@ -204,32 +204,4 @@ struct TenantMetadata {
static Key tenantMapPrivatePrefix();
};
typedef VersionedMap<TenantName, TenantMapEntry> TenantMap;
// A set of tenant names that is generally expected to have one item in it. The set can have more than one item in it
// during certain periods when the set is being updated (e.g. while restoring a backup), but it is expected to have
// one item at the end. It is not possible to use the set while it contains more than one item.
struct TenantNameUniqueSet {
std::unordered_set<TenantName> tenantNames;
// Returns the single tenant name stored in the set
// It is an error to call this function if the set holds more than one name
TenantName get() const {
ASSERT(tenantNames.size() == 1);
return *tenantNames.begin();
}
void insert(TenantName const& name) { tenantNames.insert(name); }
// Removes a tenant name from the set. Returns true if the set is now empty.
bool remove(TenantName const& name) {
auto itr = tenantNames.find(name);
ASSERT(itr != tenantNames.end());
tenantNames.erase(itr);
return tenantNames.empty();
}
};
typedef VersionedMap<Key, TenantNameUniqueSet> TenantPrefixIndex;
#endif

View File

@ -259,16 +259,23 @@ void LatencySample::logSample() {
std::string port_str = std::to_string(addr.port);
switch (model) {
case MetricsDataModel::OTLP: {
if (metrics->histMap.find(IMetric::id) != metrics->histMap.end()) {
metrics->histMap[IMetric::id].points.emplace_back(
sketch.getErrorGuarantee(), sketch.getSamples(), sketch.min(), sketch.max(), sketch.getSum());
} else {
metrics->histMap[IMetric::id] = OTEL::OTELHistogram(
name, sketch.getErrorGuarantee(), sketch.getSamples(), sketch.min(), sketch.max(), sketch.getSum());
// We only want to emit the entire DDSketch if the knob is set
if (FLOW_KNOBS->METRICS_EMIT_DDSKETCH) {
if (metrics->histMap.find(IMetric::id) != metrics->histMap.end()) {
metrics->histMap[IMetric::id].points.emplace_back(
sketch.getErrorGuarantee(), sketch.getSamples(), sketch.min(), sketch.max(), sketch.getSum());
} else {
metrics->histMap[IMetric::id] = OTEL::OTELHistogram(name,
sketch.getErrorGuarantee(),
sketch.getSamples(),
sketch.min(),
sketch.max(),
sketch.getSum());
}
metrics->histMap[IMetric::id].points.back().addAttribute("ip", ip_str);
metrics->histMap[IMetric::id].points.back().addAttribute("port", port_str);
metrics->histMap[IMetric::id].points.back().startTime = sampleEmit;
}
metrics->histMap[IMetric::id].points.back().addAttribute("ip", ip_str);
metrics->histMap[IMetric::id].points.back().addAttribute("port", port_str);
metrics->histMap[IMetric::id].points.back().startTime = sampleEmit;
createOtelGauge(p50id, name + "p50", p50);
createOtelGauge(p90id, name + "p90", p90);
createOtelGauge(p95id, name + "p95", p95);

View File

@ -84,7 +84,7 @@ public:
commit(proxyCommitData_.commit), cx(proxyCommitData_.cx), committedVersion(&proxyCommitData_.committedVersion),
storageCache(&proxyCommitData_.storageCache), tag_popped(&proxyCommitData_.tag_popped),
tssMapping(&proxyCommitData_.tssMapping), tenantMap(&proxyCommitData_.tenantMap),
tenantIdIndex(&proxyCommitData_.tenantIdIndex), initialCommit(initialCommit_) {}
tenantNameIndex(&proxyCommitData_.tenantNameIndex), initialCommit(initialCommit_) {}
ApplyMetadataMutationsImpl(const SpanContext& spanContext_,
ResolverData& resolverData_,
@ -133,8 +133,8 @@ private:
std::map<Tag, Version>* tag_popped = nullptr;
std::unordered_map<UID, StorageServerInterface>* tssMapping = nullptr;
std::map<TenantName, TenantMapEntry>* tenantMap = nullptr;
std::unordered_map<int64_t, TenantNameUniqueSet>* tenantIdIndex = nullptr;
std::unordered_map<int64_t, TenantName>* tenantMap = nullptr;
std::map<TenantName, int64_t>* tenantNameIndex = nullptr;
// true if the mutations were already written to the txnStateStore as part of recovery
bool initialCommit = false;
@ -673,9 +673,9 @@ private:
.detail("Id", tenantEntry.id)
.detail("Version", version);
(*tenantMap)[tenantName] = tenantEntry;
if (tenantIdIndex) {
(*tenantIdIndex)[tenantEntry.id].insert(tenantName);
(*tenantMap)[tenantEntry.id] = tenantName;
if (tenantNameIndex) {
(*tenantNameIndex)[tenantName] = tenantEntry.id;
}
}
@ -1082,7 +1082,7 @@ private:
void checkClearTenantMapPrefix(KeyRangeRef range) {
KeyRangeRef subspace = TenantMetadata::tenantMap().subspace;
if (subspace.intersects(range)) {
if (tenantMap) {
if (tenantMap && tenantNameIndex) {
ASSERT(version != invalidVersion);
StringRef startTenant = std::max(range.begin, subspace.begin).removePrefix(subspace.begin);
@ -1094,24 +1094,16 @@ private:
.detail("EndTenant", endTenant)
.detail("Version", version);
auto startItr = tenantMap->lower_bound(startTenant);
auto endItr = tenantMap->lower_bound(endTenant);
auto startItr = tenantNameIndex->lower_bound(startTenant);
auto endItr = tenantNameIndex->lower_bound(endTenant);
if (tenantIdIndex) {
// Iterate over iterator-range and remove entries from TenantIdName map
// TODO: O(n) operation, optimize cpu
auto itr = startItr;
while (itr != endItr) {
auto indexItr = tenantIdIndex->find(itr->second.id);
ASSERT(indexItr != tenantIdIndex->end());
if (indexItr->second.remove(itr->first)) {
tenantIdIndex->erase(indexItr);
}
itr++;
}
auto itr = startItr;
while (itr != endItr) {
tenantMap->erase(itr->second);
itr++;
}
tenantMap->erase(startItr, endItr);
tenantNameIndex->erase(startItr, endItr);
}
if (!initialCommit) {

View File

@ -32,6 +32,13 @@
#include "flow/ProtocolVersion.h"
#include "flow/actorcompiler.h" // This must be the last #include.
namespace {
EncryptionAtRestMode getEncryptionAtRest(DatabaseConfiguration config) {
TraceEvent(SevDebug, "CREncryptionAtRestMode").detail("Mode", config.encryptionAtRestMode.toString());
return config.encryptionAtRestMode;
}
} // namespace
static std::set<int> const& normalClusterRecoveryErrors() {
static std::set<int> s;
if (s.empty()) {
@ -193,9 +200,11 @@ ACTOR Future<Void> newCommitProxies(Reference<ClusterRecoveryData> self, Recruit
req.recoveryCount = self->cstate.myDBState.recoveryCount + 1;
req.recoveryTransactionVersion = self->recoveryTransactionVersion;
req.firstProxy = i == 0;
req.encryptMode = getEncryptionAtRest(self->configuration);
TraceEvent("CommitProxyReplies", self->dbgid)
.detail("WorkerID", recr.commitProxies[i].id())
.detail("RecoveryTxnVersion", self->recoveryTransactionVersion)
.detail("EncryptMode", req.encryptMode.toString())
.detail("FirstProxy", req.firstProxy ? "True" : "False");
initializationReplies.push_back(
transformErrors(throwErrorOr(recr.commitProxies[i].commitProxy.getReplyUnlessFailedFor(
@ -433,24 +442,13 @@ ACTOR Future<Void> rejoinRequestHandler(Reference<ClusterRecoveryData> self) {
}
}
namespace {
EncryptionAtRestMode getEncryptionAtRest() {
// TODO: Use db-config encryption config to determine cluster encryption status
if (SERVER_KNOBS->ENABLE_ENCRYPTION) {
return EncryptionAtRestMode(EncryptionAtRestMode::Mode::DOMAIN_AWARE);
} else {
return EncryptionAtRestMode();
}
}
} // namespace
// Keeps the coordinated state (cstate) updated as the set of recruited tlogs change through recovery.
ACTOR Future<Void> trackTlogRecovery(Reference<ClusterRecoveryData> self,
Reference<AsyncVar<Reference<ILogSystem>>> oldLogSystems,
Future<Void> minRecoveryDuration) {
state Future<Void> rejoinRequests = Never();
state DBRecoveryCount recoverCount = self->cstate.myDBState.recoveryCount + 1;
state EncryptionAtRestMode encryptionAtRestMode = getEncryptionAtRest();
state EncryptionAtRestMode encryptionAtRestMode = getEncryptionAtRest(self->configuration);
state DatabaseConfiguration configuration =
self->configuration; // self-configuration can be changed by configurationMonitor so we need a copy
loop {
@ -972,7 +970,7 @@ ACTOR Future<std::vector<Standalone<CommitTransactionRef>>> recruitEverything(
// The cluster's EncryptionAtRest status is now readable.
if (self->controllerData->encryptionAtRestMode.canBeSet()) {
self->controllerData->encryptionAtRestMode.send(getEncryptionAtRest());
self->controllerData->encryptionAtRestMode.send(getEncryptionAtRest(self->configuration));
}
}

View File

@ -251,52 +251,45 @@ struct ResolutionRequestBuilder {
}
};
ErrorOr<Optional<TenantMapEntry>> getTenantEntry(ProxyCommitData* commitData,
Optional<TenantNameRef> tenant,
Optional<int64_t> tenantId,
bool logOnFailure) {
if (tenant.present()) {
auto itr = commitData->tenantMap.find(tenant.get());
if (itr == commitData->tenantMap.end()) {
if (logOnFailure) {
TraceEvent(SevWarn, "CommitProxyUnknownTenant", commitData->dbgid).detail("Tenant", tenant.get());
}
return unknown_tenant();
} else if (tenantId.present() && tenantId.get() != itr->second.id) {
if (logOnFailure) {
TraceEvent(SevWarn, "CommitProxyTenantIdMismatch", commitData->dbgid)
.detail("Tenant", tenant.get())
.detail("TenantId", tenantId)
.detail("ExistingId", itr->second.id);
}
return unknown_tenant();
ErrorOr<int64_t> lookupTenant(ProxyCommitData* commitData, TenantNameRef tenant, bool logOnFailure) {
auto itr = commitData->tenantNameIndex.find(tenant);
if (itr == commitData->tenantNameIndex.end()) {
if (logOnFailure) {
TraceEvent(SevWarn, "CommitProxyTenantNotFound", commitData->dbgid).detail("TenantName", tenant);
}
return ErrorOr<Optional<TenantMapEntry>>(Optional<TenantMapEntry>(itr->second));
return tenant_not_found();
}
return Optional<TenantMapEntry>();
return itr->second;
}
bool verifyTenantPrefix(ProxyCommitData* const commitData, const CommitTransactionRequest& req) {
ErrorOr<Optional<TenantMapEntry>> tenantEntry =
getTenantEntry(commitData, req.tenantInfo.name.castTo<TenantNameRef>(), req.tenantInfo.tenantId, true);
bool checkTenant(ProxyCommitData* commitData, int64_t tenant, Optional<TenantNameRef> tenantName) {
if (tenant != TenantInfo::INVALID_TENANT) {
auto itr = commitData->tenantMap.find(tenant);
if (itr == commitData->tenantMap.end()) {
TraceEvent(SevWarn, "CommitProxyTenantNotFound", commitData->dbgid).detail("Tenant", tenant);
return false;
} else if (tenantName.present() && itr->second != tenantName.get()) {
// This is temporary and will be removed when the client stops caching the tenant name -> ID mapping
TraceEvent(SevWarn, "CommitProxyTenantNotFound", commitData->dbgid).detail("Tenant", tenant);
return false;
}
if (tenantEntry.isError()) {
return true;
}
if (tenantEntry.get().present()) {
Key tenantPrefix = tenantEntry.get().get().prefix;
return true;
}
bool verifyTenantPrefix(ProxyCommitData* const commitData, const CommitTransactionRequest& req) {
if (req.tenantInfo.hasTenant()) {
KeyRef tenantPrefix = req.tenantInfo.prefix.get();
for (auto& m : req.transaction.mutations) {
if (m.param1 != metadataVersionKey) {
if (!m.param1.startsWith(tenantPrefix)) {
TraceEvent(SevWarnAlways, "TenantPrefixMismatch")
.suppressFor(60)
.detail("Tenant", req.tenantInfo.name)
.detail("TenantID", req.tenantInfo.tenantId)
.detail("Tenant", req.tenantInfo.tenantId)
.detail("Prefix", tenantPrefix)
.detail("Key", m.param1);
return false;
@ -305,8 +298,7 @@ bool verifyTenantPrefix(ProxyCommitData* const commitData, const CommitTransacti
if (m.type == MutationRef::ClearRange && !m.param2.startsWith(tenantPrefix)) {
TraceEvent(SevWarnAlways, "TenantClearRangePrefixMismatch")
.suppressFor(60)
.detail("Tenant", req.tenantInfo.name)
.detail("TenantID", req.tenantInfo.tenantId)
.detail("Tenant", req.tenantInfo.tenantId)
.detail("Prefix", tenantPrefix)
.detail("Key", m.param2);
return false;
@ -317,8 +309,7 @@ bool verifyTenantPrefix(ProxyCommitData* const commitData, const CommitTransacti
if (*offset < tenantPrefix.size()) {
TraceEvent(SevWarnAlways, "TenantVersionstampInvalidOffset")
.suppressFor(60)
.detail("Tenant", req.tenantInfo.name)
.detail("TenantID", req.tenantInfo.tenantId)
.detail("Tenant", req.tenantInfo.tenantId)
.detail("Prefix", tenantPrefix)
.detail("Key", m.param1)
.detail("Offset", *offset);
@ -333,8 +324,7 @@ bool verifyTenantPrefix(ProxyCommitData* const commitData, const CommitTransacti
(!rc.begin.startsWith(tenantPrefix) || !rc.end.startsWith(tenantPrefix))) {
TraceEvent(SevWarnAlways, "TenantReadConflictPrefixMismatch")
.suppressFor(60)
.detail("Tenant", req.tenantInfo.name)
.detail("TenantID", req.tenantInfo.tenantId)
.detail("Tenant", req.tenantInfo.tenantId)
.detail("Prefix", tenantPrefix)
.detail("BeginKey", rc.begin)
.detail("EndKey", rc.end);
@ -347,8 +337,7 @@ bool verifyTenantPrefix(ProxyCommitData* const commitData, const CommitTransacti
(!wc.begin.startsWith(tenantPrefix) || !wc.end.startsWith(tenantPrefix))) {
TraceEvent(SevWarnAlways, "TenantWriteConflictPrefixMismatch")
.suppressFor(60)
.detail("Tenant", req.tenantInfo.name)
.detail("TenantID", req.tenantInfo.tenantId)
.detail("Tenant", req.tenantInfo.tenantId)
.detail("Prefix", tenantPrefix)
.detail("BeginKey", wc.begin)
.detail("EndKey", wc.end);
@ -786,7 +775,7 @@ bool canReject(const std::vector<CommitTransactionRequest>& trs) {
for (const auto& tr : trs) {
if (tr.transaction.mutations.empty())
continue;
if (!tr.tenantInfo.name.present() &&
if (!tr.tenantInfo.hasTenant() &&
(tr.transaction.mutations[0].param1.startsWith("\xff"_sr) || tr.transaction.read_conflict_ranges.empty())) {
return false;
}
@ -896,19 +885,6 @@ ACTOR Future<Void> preresolutionProcessing(CommitBatchContext* self) {
}
namespace {
// Routine allows caller to find TenantName for a given TenantId. It returns empty optional when either TenantId is
// invalid or tenant is unknown
Optional<TenantName> getTenantName(ProxyCommitData* commitData, int64_t tenantId) {
if (tenantId != TenantInfo::INVALID_TENANT) {
auto itr = commitData->tenantIdIndex.find(tenantId);
if (itr != commitData->tenantIdIndex.end()) {
return Optional<TenantName>(itr->second.get());
}
}
return Optional<TenantName>();
}
EncryptCipherDomainId getEncryptDetailsFromMutationRef(ProxyCommitData* commitData, MutationRef m) {
EncryptCipherDomainId domainId = INVALID_ENCRYPT_DOMAIN_ID;
@ -920,7 +896,7 @@ EncryptCipherDomainId getEncryptDetailsFromMutationRef(ProxyCommitData* commitDa
if (isSystemKey(m.param1)) {
// Encryption domain == FDB SystemKeyspace encryption domain
domainId = SYSTEM_KEYSPACE_ENCRYPT_DOMAIN_ID;
} else if (commitData->tenantMap.empty()) {
} else if (commitData->tenantMap.empty() || commitData->encryptMode.mode == EncryptionAtRestMode::CLUSTER_AWARE) {
// Cluster serves no-tenants; use 'default encryption domain'
} else if (isSingleKeyMutation((MutationRef::Type)m.type)) {
ASSERT_NE((MutationRef::Type)m.type, MutationRef::Type::ClearRange);
@ -929,11 +905,8 @@ EncryptCipherDomainId getEncryptDetailsFromMutationRef(ProxyCommitData* commitDa
// Parse mutation key to determine mutation encryption domain
StringRef prefix = m.param1.substr(0, TenantAPI::PREFIX_SIZE);
int64_t tenantId = TenantAPI::prefixToId(prefix, EnforceValidTenantId::False);
if (tenantId != TenantInfo::INVALID_TENANT) {
Optional<TenantName> tenantName = getTenantName(commitData, tenantId);
if (tenantName.present()) {
domainId = tenantId;
}
if (commitData->tenantMap.count(tenantId)) {
domainId = tenantId;
} else {
// Leverage 'default encryption domain'
}
@ -1004,29 +977,32 @@ ACTOR Future<Void> getResolution(CommitBatchContext* self) {
// Fetch cipher keys if needed.
state Future<std::unordered_map<EncryptCipherDomainId, Reference<BlobCipherKey>>> getCipherKeys;
if (pProxyCommitData->isEncryptionEnabled) {
if (pProxyCommitData->encryptMode.isEncryptionEnabled()) {
static const std::unordered_set<EncryptCipherDomainId> defaultDomainIds = { SYSTEM_KEYSPACE_ENCRYPT_DOMAIN_ID,
ENCRYPT_HEADER_DOMAIN_ID,
FDB_DEFAULT_ENCRYPT_DOMAIN_ID };
std::unordered_set<EncryptCipherDomainId> encryptDomainIds = defaultDomainIds;
for (int t = 0; t < trs.size(); t++) {
TenantInfo const& tenantInfo = trs[t].tenantInfo;
int64_t tenantId = tenantInfo.tenantId;
Optional<TenantNameRef> const& tenantName = tenantInfo.name;
if (tenantId != TenantInfo::INVALID_TENANT) {
ASSERT(tenantName.present());
encryptDomainIds.emplace(tenantId);
} else {
// Optimization: avoid enumerating mutations if cluster only serves default encryption domains
if (pProxyCommitData->tenantMap.size() > 0) {
for (auto m : trs[t].transaction.mutations) {
EncryptCipherDomainId domainId = getEncryptDetailsFromMutationRef(pProxyCommitData, m);
encryptDomainIds.emplace(domainId);
}
// For cluster aware encryption only the default domain id is needed
if (pProxyCommitData->encryptMode.mode == EncryptionAtRestMode::DOMAIN_AWARE) {
for (int t = 0; t < trs.size(); t++) {
TenantInfo const& tenantInfo = trs[t].tenantInfo;
int64_t tenantId = tenantInfo.tenantId;
Optional<TenantNameRef> const& tenantName = tenantInfo.name;
if (tenantId != TenantInfo::INVALID_TENANT) {
ASSERT(tenantName.present());
encryptDomainIds.emplace(tenantId);
} else {
// Ensure default encryption domain-ids are present.
ASSERT_EQ(encryptDomainIds.count(SYSTEM_KEYSPACE_ENCRYPT_DOMAIN_ID), 1);
ASSERT_EQ(encryptDomainIds.count(FDB_DEFAULT_ENCRYPT_DOMAIN_ID), 1);
// Optimization: avoid enumerating mutations if cluster only serves default encryption domains
if (pProxyCommitData->tenantMap.size() > 0) {
for (auto m : trs[t].transaction.mutations) {
EncryptCipherDomainId domainId = getEncryptDetailsFromMutationRef(pProxyCommitData, m);
encryptDomainIds.emplace(domainId);
}
} else {
// Ensure default encryption domain-ids are present.
ASSERT_EQ(encryptDomainIds.count(SYSTEM_KEYSPACE_ENCRYPT_DOMAIN_ID), 1);
ASSERT_EQ(encryptDomainIds.count(FDB_DEFAULT_ENCRYPT_DOMAIN_ID), 1);
}
}
}
}
@ -1057,7 +1033,7 @@ ACTOR Future<Void> getResolution(CommitBatchContext* self) {
g_traceBatch.addEvent(
"CommitDebug", self->debugID.get().first(), "CommitProxyServer.commitBatch.AfterResolution");
}
if (pProxyCommitData->isEncryptionEnabled) {
if (pProxyCommitData->encryptMode.isEncryptionEnabled()) {
std::unordered_map<EncryptCipherDomainId, Reference<BlobCipherKey>> cipherKeys = wait(getCipherKeys);
self->cipherKeys = cipherKeys;
}
@ -1188,12 +1164,11 @@ ACTOR Future<Void> applyMetadataToCommittedTransactions(CommitBatchContext* self
int t;
for (t = 0; t < trs.size() && !self->forceRecovery; t++) {
if (self->committed[t] == ConflictBatch::TransactionCommitted && (!self->locked || trs[t].isLockAware())) {
ErrorOr<Optional<TenantMapEntry>> result = getTenantEntry(
pProxyCommitData, trs[t].tenantInfo.name.castTo<TenantNameRef>(), trs[t].tenantInfo.tenantId, true);
bool isValid = checkTenant(pProxyCommitData, trs[t].tenantInfo.tenantId, trs[t].tenantInfo.name);
if (result.isError()) {
if (!isValid) {
self->committed[t] = ConflictBatch::TransactionTenantFailure;
trs[t].reply.sendError(result.getError());
trs[t].reply.sendError(unknown_tenant());
} else {
self->commitCount++;
applyMetadataMutations(trs[t].spanContext,
@ -1202,7 +1177,8 @@ ACTOR Future<Void> applyMetadataToCommittedTransactions(CommitBatchContext* self
pProxyCommitData->logSystem,
trs[t].transaction.mutations,
SERVER_KNOBS->PROXY_USE_RESOLVER_PRIVATE_MUTATIONS ? nullptr : &self->toCommit,
pProxyCommitData->isEncryptionEnabled ? &self->cipherKeys : nullptr,
pProxyCommitData->encryptMode.isEncryptionEnabled() ? &self->cipherKeys
: nullptr,
self->forceRecovery,
self->commitVersion,
self->commitVersion + 1,
@ -1264,7 +1240,7 @@ ACTOR Future<WriteMutationRefVar> writeMutationEncryptedMutation(CommitBatchCont
state const BlobCipherEncryptHeader* header;
static_assert(TenantInfo::INVALID_TENANT == INVALID_ENCRYPT_DOMAIN_ID);
ASSERT(self->pProxyCommitData->isEncryptionEnabled);
ASSERT(self->pProxyCommitData->encryptMode.isEncryptionEnabled());
ASSERT(g_network && g_network->isSimulated());
ASSERT(encryptedMutation.isEncrypted());
@ -1289,7 +1265,7 @@ ACTOR Future<WriteMutationRefVar> writeMutationFetchEncryptKey(CommitBatchContex
state MutationRef encryptedMutation;
static_assert(TenantInfo::INVALID_TENANT == INVALID_ENCRYPT_DOMAIN_ID);
ASSERT(self->pProxyCommitData->isEncryptionEnabled);
ASSERT(self->pProxyCommitData->encryptMode.isEncryptionEnabled());
ASSERT_NE((MutationRef::Type)mutation->type, MutationRef::Type::ClearRange);
domainId = getEncryptDetailsFromMutationRef(self->pProxyCommitData, *mutation);
@ -1305,7 +1281,7 @@ ACTOR Future<WriteMutationRefVar> writeMutationFetchEncryptKey(CommitBatchContex
}
Future<WriteMutationRefVar> writeMutation(CommitBatchContext* self,
int64_t tenantId,
int64_t domainId,
const MutationRef* mutation,
Optional<MutationRef>* encryptedMutationOpt,
Arena* arena) {
@ -1322,8 +1298,10 @@ Future<WriteMutationRefVar> writeMutation(CommitBatchContext* self,
// the penalty happens iff any of above conditions are met. Otherwise, corresponding handle routine (ACTOR
// compliant) gets invoked ("slow path").
if (self->pProxyCommitData->isEncryptionEnabled) {
EncryptCipherDomainId domainId = tenantId;
if (self->pProxyCommitData->encryptMode.isEncryptionEnabled()) {
if (self->pProxyCommitData->encryptMode.mode == EncryptionAtRestMode::CLUSTER_AWARE) {
ASSERT(domainId == FDB_DEFAULT_ENCRYPT_DOMAIN_ID || domainId == SYSTEM_KEYSPACE_ENCRYPT_DOMAIN_ID);
}
MutationRef encryptedMutation;
CODE_PROBE(self->pProxyCommitData->db->get().client.tenantMode == TenantMode::DISABLED,
"using disabled tenant mode");
@ -1338,13 +1316,13 @@ Future<WriteMutationRefVar> writeMutation(CommitBatchContext* self,
ASSERT(encryptedMutation.isEncrypted());
// During simulation check whether the encrypted mutation matches the decrpyted mutation
if (g_network && g_network->isSimulated()) {
return writeMutationEncryptedMutation(self, tenantId, mutation, encryptedMutationOpt, arena);
return writeMutationEncryptedMutation(self, domainId, mutation, encryptedMutationOpt, arena);
}
} else {
if (domainId == INVALID_ENCRYPT_DOMAIN_ID) {
domainId = getEncryptDetailsFromMutationRef(self->pProxyCommitData, *mutation);
if (self->cipherKeys.find(domainId) == self->cipherKeys.end()) {
return writeMutationFetchEncryptKey(self, tenantId, mutation, arena);
return writeMutationFetchEncryptKey(self, domainId, mutation, arena);
}
CODE_PROBE(true, "Raw access mutation encryption");
@ -1382,7 +1360,11 @@ ACTOR Future<Void> assignMutationsToStorageServers(CommitBatchContext* self) {
state VectorRef<Optional<MutationRef>>* encryptedMutations =
&trs[self->transactionNum].transaction.encryptedMutations;
ASSERT(encryptedMutations->size() == 0 || encryptedMutations->size() == pMutations->size());
state int64_t tenantId = trs[self->transactionNum].tenantInfo.tenantId;
state int64_t encryptDomain = trs[self->transactionNum].tenantInfo.tenantId;
if (self->pProxyCommitData->encryptMode.mode == EncryptionAtRestMode::CLUSTER_AWARE &&
encryptDomain != SYSTEM_KEYSPACE_ENCRYPT_DOMAIN_ID) {
encryptDomain = FDB_DEFAULT_ENCRYPT_DOMAIN_ID;
}
self->toCommit.addTransactionInfo(trs[self->transactionNum].spanContext);
@ -1449,7 +1431,7 @@ ACTOR Future<Void> assignMutationsToStorageServers(CommitBatchContext* self) {
if (encryptedMutation.present()) {
ASSERT(encryptedMutation.get().isEncrypted());
}
WriteMutationRefVar var = wait(writeMutation(self, tenantId, &m, &encryptedMutation, &arena));
WriteMutationRefVar var = wait(writeMutation(self, encryptDomain, &m, &encryptedMutation, &arena));
// FIXME: Remove assert once ClearRange RAW_ACCESS usecase handling is done
ASSERT(std::holds_alternative<MutationRef>(var));
writtenMutation = std::get<MutationRef>(var);
@ -1505,7 +1487,7 @@ ACTOR Future<Void> assignMutationsToStorageServers(CommitBatchContext* self) {
if (pProxyCommitData->needsCacheTag(clearRange)) {
self->toCommit.addTag(cacheTag);
}
WriteMutationRefVar var = wait(writeMutation(self, tenantId, &m, &encryptedMutation, &arena));
WriteMutationRefVar var = wait(writeMutation(self, encryptDomain, &m, &encryptedMutation, &arena));
// FIXME: Remove assert once ClearRange RAW_ACCESS usecase handling is done
ASSERT(std::holds_alternative<MutationRef>(var));
writtenMutation = std::get<MutationRef>(var);
@ -1539,7 +1521,7 @@ ACTOR Future<Void> assignMutationsToStorageServers(CommitBatchContext* self) {
// Add the mutation to the relevant backup tag
for (auto backupName : pProxyCommitData->vecBackupKeys[m.param1]) {
// If encryption is enabled make sure the mutation we are writing is also encrypted
ASSERT(!self->pProxyCommitData->isEncryptionEnabled || writtenMutation.isEncrypted());
ASSERT(!self->pProxyCommitData->encryptMode.isEncryptionEnabled() || writtenMutation.isEncrypted());
CODE_PROBE(writtenMutation.isEncrypted(), "using encrypted backup mutation");
self->logRangeMutations[backupName].push_back_deep(self->logRangeMutationsArena, writtenMutation);
}
@ -1561,7 +1543,7 @@ ACTOR Future<Void> assignMutationsToStorageServers(CommitBatchContext* self) {
// TODO (Nim): Currently clear ranges are encrypted using the default encryption key, this must
// be changed to account for clear ranges which span tenant boundaries
if (self->pProxyCommitData->isEncryptionEnabled) {
if (self->pProxyCommitData->encryptMode.isEncryptionEnabled()) {
CODE_PROBE(true, "encrypting clear range backup mutation");
if (backupMutation.param1 == m.param1 && backupMutation.param2 == m.param2 &&
encryptedMutation.present()) {
@ -1676,7 +1658,7 @@ ACTOR Future<Void> postResolution(CommitBatchContext* self) {
idempotencyIdSet.param2 = kv.value;
auto& tags = pProxyCommitData->tagsForKey(kv.key);
self->toCommit.addTags(tags);
if (self->pProxyCommitData->isEncryptionEnabled) {
if (self->pProxyCommitData->encryptMode.isEncryptionEnabled()) {
CODE_PROBE(true, "encrypting idempotency mutation");
EncryptCipherDomainId domainId =
getEncryptDetailsFromMutationRef(self->pProxyCommitData, idempotencyIdSet);
@ -2133,7 +2115,7 @@ ACTOR static Future<Void> doKeyServerLocationRequest(GetKeyServerLocationsReques
wait(commitData->validState.getFuture());
wait(delay(0, TaskPriority::DefaultEndpoint));
state ErrorOr<Optional<TenantMapEntry>> tenantEntry;
state ErrorOr<int64_t> tenantId;
state Version minTenantVersion =
req.minTenantVersion == latestVersion ? commitData->stats.lastCommitVersionAssigned + 1 : req.minTenantVersion;
@ -2142,15 +2124,13 @@ ACTOR static Future<Void> doKeyServerLocationRequest(GetKeyServerLocationsReques
? delay(SERVER_KNOBS->FUTURE_VERSION_DELAY)
: Never();
while (tenantEntry.isError()) {
while (req.tenant.name.present() && tenantId.isError()) {
bool finalQuery = commitData->version.get() >= minTenantVersion;
ErrorOr<Optional<TenantMapEntry>> _tenantEntry =
getTenantEntry(commitData, req.tenant.name, Optional<int64_t>(), finalQuery);
tenantEntry = _tenantEntry;
tenantId = lookupTenant(commitData, req.tenant.name.get(), finalQuery);
if (tenantEntry.isError()) {
if (tenantId.isError()) {
if (finalQuery) {
req.reply.sendError(tenant_not_found());
req.reply.sendError(tenantId.getError());
return Void();
} else {
choose {
@ -2170,8 +2150,8 @@ ACTOR static Future<Void> doKeyServerLocationRequest(GetKeyServerLocationsReques
std::unordered_set<UID> tssMappingsIncluded;
GetKeyServerLocationsReply rep;
if (tenantEntry.get().present()) {
rep.tenantEntry = tenantEntry.get().get();
if (req.tenant.name.present()) {
rep.tenantEntry = TenantMapEntry(tenantId.get(), req.tenant.name.get(), TenantState::READY);
req.begin = req.begin.withPrefix(rep.tenantEntry.prefix, req.arena);
if (req.end.present()) {
req.end = req.end.get().withPrefix(rep.tenantEntry.prefix, req.arena);
@ -2731,7 +2711,7 @@ ACTOR Future<Void> processCompleteTransactionStateRequest(TransactionStateResolv
}
state std::unordered_map<EncryptCipherDomainId, Reference<BlobCipherKey>> cipherKeys;
if (pContext->pCommitData->isEncryptionEnabled) {
if (pContext->pCommitData->encryptMode.isEncryptionEnabled()) {
static const std::unordered_set<EncryptCipherDomainId> metadataDomainIds = { SYSTEM_KEYSPACE_ENCRYPT_DOMAIN_ID,
ENCRYPT_HEADER_DOMAIN_ID };
std::unordered_map<EncryptCipherDomainId, Reference<BlobCipherKey>> cks =
@ -2797,7 +2777,7 @@ ACTOR Future<Void> processCompleteTransactionStateRequest(TransactionStateResolv
Arena arena;
bool confChanges;
CODE_PROBE(
pContext->pCommitData->isEncryptionEnabled,
pContext->pCommitData->encryptMode.isEncryptionEnabled(),
"Commit proxy apply metadata mutations from txnStateStore on recovery, with encryption-at-rest enabled");
applyMetadataMutations(SpanContext(),
*pContext->pCommitData,
@ -2805,7 +2785,7 @@ ACTOR Future<Void> processCompleteTransactionStateRequest(TransactionStateResolv
Reference<ILogSystem>(),
mutations,
/* pToCommit= */ nullptr,
pContext->pCommitData->isEncryptionEnabled ? &cipherKeys : nullptr,
pContext->pCommitData->encryptMode.isEncryptionEnabled() ? &cipherKeys : nullptr,
confChanges,
/* version= */ 0,
/* popVersion= */ 0,
@ -2874,9 +2854,16 @@ ACTOR Future<Void> commitProxyServerCore(CommitProxyInterface proxy,
LogEpoch epoch,
Version recoveryTransactionVersion,
bool firstProxy,
std::string whitelistBinPaths) {
state ProxyCommitData commitData(
proxy.id(), master, proxy.getConsistentReadVersion, recoveryTransactionVersion, proxy.commit, db, firstProxy);
std::string whitelistBinPaths,
EncryptionAtRestMode encryptMode) {
state ProxyCommitData commitData(proxy.id(),
master,
proxy.getConsistentReadVersion,
recoveryTransactionVersion,
proxy.commit,
db,
firstProxy,
encryptMode);
state Future<Sequence> sequenceFuture = (Sequence)0;
state PromiseStream<std::pair<std::vector<CommitTransactionRequest>, int>> batchedCommits;
@ -2889,6 +2876,8 @@ ACTOR Future<Void> commitProxyServerCore(CommitProxyInterface proxy,
state GetHealthMetricsReply healthMetricsReply;
state GetHealthMetricsReply detailedHealthMetricsReply;
TraceEvent("CPEncryptionAtRestMode").detail("Mode", commitData.encryptMode.toString());
addActor.send(waitFailureServer(proxy.waitFailure.getFuture()));
addActor.send(traceRole(Role::COMMIT_PROXY, proxy.id()));
@ -2897,8 +2886,7 @@ 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 &&
(!isEncryptionOpSupported(EncryptOperationType::TLOG_ENCRYPTION) ||
commitData.db->get().encryptKeyProxy.present()))) {
(!commitData.encryptMode.isEncryptionEnabled() || commitData.db->get().encryptKeyProxy.present()))) {
//TraceEvent("ProxyInit2", proxy.id()).detail("LSEpoch", db->get().logSystemConfig.epoch).detail("Need", epoch);
wait(commitData.db->onChange());
}
@ -2922,6 +2910,7 @@ ACTOR Future<Void> commitProxyServerCore(CommitProxyInterface proxy,
commitData.logSystem = ILogSystem::fromServerDBInfo(proxy.id(), commitData.db->get(), false, addActor);
commitData.logAdapter =
new LogSystemDiskQueueAdapter(commitData.logSystem, Reference<AsyncVar<PeekTxsInfo>>(), 1, false);
// TODO: Pass the encrypt mode once supported in IKeyValueStore
commitData.txnStateStore = keyValueStoreLogSystem(commitData.logAdapter,
commitData.db,
proxy.id(),
@ -2951,10 +2940,19 @@ ACTOR Future<Void> commitProxyServerCore(CommitProxyInterface proxy,
addActor.send(rejoinServer(proxy, &commitData));
addActor.send(ddMetricsRequestServer(proxy, db));
addActor.send(reportTxnTagCommitCost(proxy.id(), db, &commitData.ssTrTagCommitCost));
addActor.send(idempotencyIdsExpireServer(openDBOnServer(db),
proxy.expireIdempotencyId,
commitData.expectedIdempotencyIdCountForKey,
&commitData.idempotencyClears));
auto openDb = openDBOnServer(db);
if (firstProxy) {
addActor.send(recurringAsync(
[openDb = openDb]() { return cleanIdempotencyIds(openDb, SERVER_KNOBS->IDEMPOTENCY_IDS_MIN_AGE_SECONDS); },
SERVER_KNOBS->IDEMPOTENCY_IDS_CLEANER_POLLING_INTERVAL,
true,
SERVER_KNOBS->IDEMPOTENCY_IDS_CLEANER_POLLING_INTERVAL));
}
addActor.send(idempotencyIdsExpireServer(
openDb, proxy.expireIdempotencyId, commitData.expectedIdempotencyIdCountForKey, &commitData.idempotencyClears));
if (SERVER_KNOBS->STORAGE_QUOTA_ENABLED) {
addActor.send(monitorTenantsOverStorageQuota(proxy.id(), db, &commitData));
}
@ -3050,7 +3048,8 @@ ACTOR Future<Void> commitProxyServer(CommitProxyInterface proxy,
req.recoveryCount,
req.recoveryTransactionVersion,
req.firstProxy,
whitelistBinPaths);
whitelistBinPaths,
req.encryptMode);
wait(core || checkRemoved(db, req.recoveryCount, proxy));
} catch (Error& e) {
TraceEvent("CommitProxyTerminated", proxy.id()).errorUnsuppressed(e);

View File

@ -208,11 +208,13 @@ int64_t getMaxShardSize(double dbSizeEstimate) {
(int64_t)SERVER_KNOBS->MAX_SHARD_BYTES);
}
ShardSizeBounds calculateShardSizeBounds(const KeyRange& keys,
const Reference<AsyncVar<Optional<ShardMetrics>>>& shardMetrics,
const BandwidthStatus& bandwidthStatus,
PromiseStream<KeyRange> readHotShard) {
// Returns the shard size bounds as well as whether `keys` a read hot shard.
std::pair<ShardSizeBounds, bool> calculateShardSizeBounds(
const KeyRange& keys,
const Reference<AsyncVar<Optional<ShardMetrics>>>& shardMetrics,
const BandwidthStatus& bandwidthStatus) {
ShardSizeBounds bounds = ShardSizeBounds::shardSizeBoundsBeforeTrack();
bool readHotShard = false;
if (shardMetrics->get().present()) {
auto bytes = shardMetrics->get().get().metrics.bytes;
auto readBandwidthStatus = getReadBandwidthStatus(shardMetrics->get().get().metrics);
@ -252,15 +254,13 @@ ShardSizeBounds calculateShardSizeBounds(const KeyRange& keys,
SERVER_KNOBS->STORAGE_METRICS_AVERAGE_INTERVAL_PER_KSECONDS *
(1.0 - SERVER_KNOBS->SHARD_MAX_BYTES_READ_PER_KSEC_JITTER);
bounds.permittedError.bytesReadPerKSecond = bounds.min.bytesReadPerKSecond / 4;
// TraceEvent("RHDTriggerReadHotLoggingForShard")
// .detail("ShardBegin", keys.begin.printable().c_str())
// .detail("ShardEnd", keys.end.printable().c_str());
readHotShard.send(keys);
readHotShard = true;
} else {
ASSERT(false);
}
}
return bounds;
return { bounds, readHotShard };
}
ACTOR Future<Void> trackShardMetrics(DataDistributionTracker::SafeAccessor self,
@ -285,7 +285,15 @@ ACTOR Future<Void> trackShardMetrics(DataDistributionTracker::SafeAccessor self,
try {
loop {
state ShardSizeBounds bounds;
bounds = calculateShardSizeBounds(keys, shardMetrics, bandwidthStatus, self()->readHotShard);
bool readHotShard;
std::tie(bounds, readHotShard) = calculateShardSizeBounds(keys, shardMetrics, bandwidthStatus);
if (readHotShard) {
// TraceEvent("RHDTriggerReadHotLoggingForShard")
// .detail("ShardBegin", keys.begin.printable().c_str())
// .detail("ShardEnd", keys.end.printable().c_str());
self()->readHotShard.send(keys);
}
loop {
// metrics.second is the number of key-ranges (i.e., shards) in the 'keys' key-range
@ -327,6 +335,7 @@ ACTOR Future<Void> trackShardMetrics(DataDistributionTracker::SafeAccessor self,
keys, metrics.first.get(), shardMetrics->get().get().metrics, initWithNewMetrics);
if (needToMove) {
// Do we need to update shardsAffectedByTeamFailure here?
// TODO(zhewu): move this to physical shard tracker that does shard split based on size.
self()->output.send(
RelocateShard(keys,
DataMovementReason::ENFORCE_MOVE_OUT_OF_PHYSICAL_SHARD,
@ -1544,6 +1553,94 @@ FDB_DEFINE_BOOLEAN_PARAM(InOverSizePhysicalShard);
FDB_DEFINE_BOOLEAN_PARAM(PhysicalShardAvailable);
FDB_DEFINE_BOOLEAN_PARAM(MoveKeyRangeOutPhysicalShard);
// Tracks storage metrics for `keys`. This function is similar to `trackShardMetrics()` and altered for physical shard.
// This meant to be temporary. Eventually, we want a new interface to track physical shard metrics more efficiently.
ACTOR Future<Void> trackKeyRangeInPhysicalShardMetrics(Reference<IDDTxnProcessor> db,
KeyRange keys,
Reference<AsyncVar<Optional<ShardMetrics>>> shardMetrics) {
state BandwidthStatus bandwidthStatus =
shardMetrics->get().present() ? getBandwidthStatus(shardMetrics->get().get().metrics) : BandwidthStatusNormal;
state double lastLowBandwidthStartTime =
shardMetrics->get().present() ? shardMetrics->get().get().lastLowBandwidthStartTime : now();
state int shardCount = shardMetrics->get().present() ? shardMetrics->get().get().shardCount : 1;
wait(delay(0, TaskPriority::DataDistribution));
/*TraceEvent("trackKeyRangeInPhysicalShardMetricsStarting")
.detail("Keys", keys)
.detail("TrackedBytesInitiallyPresent", shardMetrics->get().present())
.detail("StartingMetrics", shardMetrics->get().present() ? shardMetrics->get().get().metrics.bytes : 0)
.detail("StartingMerges", shardMetrics->get().present() ? shardMetrics->get().get().merges : 0);*/
loop {
state ShardSizeBounds bounds;
bool readHotShard;
std::tie(bounds, readHotShard) = calculateShardSizeBounds(keys, shardMetrics, bandwidthStatus);
loop {
// metrics.second is the number of key-ranges (i.e., shards) in the 'keys' key-range
std::pair<Optional<StorageMetrics>, int> metrics =
wait(db->waitStorageMetrics(keys,
bounds.min,
bounds.max,
bounds.permittedError,
CLIENT_KNOBS->STORAGE_METRICS_SHARD_LIMIT,
shardCount));
if (metrics.first.present()) {
BandwidthStatus newBandwidthStatus = getBandwidthStatus(metrics.first.get());
if (newBandwidthStatus == BandwidthStatusLow && bandwidthStatus != BandwidthStatusLow) {
lastLowBandwidthStartTime = now();
}
bandwidthStatus = newBandwidthStatus;
shardMetrics->set(ShardMetrics(metrics.first.get(), lastLowBandwidthStartTime, shardCount));
break;
} else {
shardCount = metrics.second;
if (shardMetrics->get().present()) {
auto newShardMetrics = shardMetrics->get().get();
newShardMetrics.shardCount = shardCount;
shardMetrics->set(newShardMetrics);
}
}
}
}
}
void PhysicalShardCollection::PhysicalShard::addRange(const KeyRange& newRange) {
if (g_network->isSimulated()) {
// Test that new range must not overlap with any existing range in this shard.
for (const auto& [range, data] : rangeData) {
ASSERT(!range.intersects(newRange));
}
}
RangeData data;
data.stats = makeReference<AsyncVar<Optional<ShardMetrics>>>();
data.trackMetrics = trackKeyRangeInPhysicalShardMetrics(txnProcessor, newRange, data.stats);
rangeData.emplace(newRange, data);
}
void PhysicalShardCollection::PhysicalShard::removeRange(const KeyRange& outRange) {
std::vector<KeyRangeRef> updateRanges;
for (auto& [range, data] : rangeData) {
if (range.intersects(outRange)) {
updateRanges.push_back(range);
}
}
for (auto& range : updateRanges) {
std::vector<KeyRangeRef> remainingRanges = range - outRange;
for (auto& r : remainingRanges) {
ASSERT(r != range);
RangeData data;
data.stats = makeReference<AsyncVar<Optional<ShardMetrics>>>();
data.trackMetrics = trackKeyRangeInPhysicalShardMetrics(txnProcessor, r, data.stats);
rangeData.emplace(r, data);
}
// Must erase last since `remainingRanges` uses data in `range`.
rangeData.erase(range);
}
}
// Decide whether a physical shard is available at the moment when the func is calling
PhysicalShardAvailable PhysicalShardCollection::checkPhysicalShardAvailable(uint64_t physicalShardID,
StorageMetrics const& moveInMetrics) {
@ -1589,16 +1686,69 @@ void PhysicalShardCollection::insertPhysicalShardToCollection(uint64_t physicalS
ASSERT(physicalShardID != anonymousShardId.first() && physicalShardID != UID().first());
ASSERT(physicalShardInstances.count(physicalShardID) == 0);
physicalShardInstances.insert(
std::make_pair(physicalShardID, PhysicalShard(physicalShardID, metrics, teams, whenCreated)));
std::make_pair(physicalShardID, PhysicalShard(txnProcessor, physicalShardID, metrics, teams, whenCreated)));
return;
}
// This method maintains the consistency between keyRangePhysicalShardIDMap and the RangeData in physicalShardInstances.
// They are all updated in this method.
void PhysicalShardCollection::updatekeyRangePhysicalShardIDMap(KeyRange keyRange,
uint64_t physicalShardID,
uint64_t debugID) {
ASSERT(physicalShardID != UID().first());
auto ranges = keyRangePhysicalShardIDMap.intersectingRanges(keyRange);
std::set<uint64_t> physicalShardIDSet;
// If there are any existing physical shards own `keyRange`, remove the overlaping ranges from existing physical
// shards.
for (auto it = ranges.begin(); it != ranges.end(); ++it) {
uint64_t shardID = it->value();
if (shardID == UID().first()) {
continue;
}
ASSERT(physicalShardInstances.find(shardID) != physicalShardInstances.end());
physicalShardInstances[shardID].removeRange(keyRange);
}
// Insert `keyRange` to the new physical shard.
keyRangePhysicalShardIDMap.insert(keyRange, physicalShardID);
return;
ASSERT(physicalShardInstances.find(physicalShardID) != physicalShardInstances.end());
physicalShardInstances[physicalShardID].addRange(keyRange);
// KeyRange to physical shard mapping consistency sanity check.
checkKeyRangePhysicalShardMapping();
}
void PhysicalShardCollection::checkKeyRangePhysicalShardMapping() {
if (!g_network->isSimulated()) {
return;
}
// Check the invariant that keyRangePhysicalShardIDMap and physicalShardInstances should be consistent.
KeyRangeMap<uint64_t>::Ranges keyRangePhysicalShardIDRanges = keyRangePhysicalShardIDMap.ranges();
KeyRangeMap<uint64_t>::iterator it = keyRangePhysicalShardIDRanges.begin();
for (; it != keyRangePhysicalShardIDRanges.end(); ++it) {
uint64_t shardID = it->value();
if (shardID == UID().first()) {
continue;
}
auto keyRangePiece = KeyRangeRef(it->range().begin, it->range().end);
ASSERT(physicalShardInstances.find(shardID) != physicalShardInstances.end());
bool exist = false;
for (const auto& [range, data] : physicalShardInstances[shardID].rangeData) {
if (range == keyRangePiece) {
exist = true;
break;
}
}
ASSERT(exist);
}
for (auto& [shardID, physicalShard] : physicalShardInstances) {
for (auto& [range, data] : physicalShard.rangeData) {
ASSERT(keyRangePhysicalShardIDMap[range.begin] == shardID);
}
}
}
// At beginning of the transition from the initial state without physical shard notion

View File

@ -1957,6 +1957,40 @@ public:
return Void();
}
ACTOR static Future<Void> perpetualStorageWiggleRest(DDTeamCollection* self) {
state bool takeRest = true;
state Promise<int64_t> avgShardBytes;
while (takeRest) {
// a minimal delay to avoid excluding and including SS too fast
wait(delay(SERVER_KNOBS->PERPETUAL_WIGGLE_DELAY));
avgShardBytes.reset();
self->getAverageShardBytes.send(avgShardBytes);
int64_t avgBytes = wait(avgShardBytes.getFuture());
double ratio = self->loadBytesBalanceRatio(avgBytes * SERVER_KNOBS->PERPETUAL_WIGGLE_SMALL_LOAD_RATIO);
bool imbalance = ratio < SERVER_KNOBS->PERPETUAL_WIGGLE_MIN_BYTES_BALANCE_RATIO;
CODE_PROBE(imbalance, "Perpetual Wiggle pause because cluster is imbalance.");
// there must not have other teams to place wiggled data
takeRest = self->server_info.size() <= self->configuration.storageTeamSize ||
self->machine_info.size() < self->configuration.storageTeamSize || imbalance;
// log the extra delay and change the wiggler state
if (takeRest) {
self->storageWiggler->setWiggleState(StorageWiggler::PAUSE);
if (self->configuration.storageMigrationType == StorageMigrationType::GRADUAL) {
TraceEvent(SevWarn, "PerpetualStorageWiggleSleep", self->distributorId)
.suppressFor(SERVER_KNOBS->PERPETUAL_WIGGLE_DELAY * 4)
.detail("BytesBalanceRatio", ratio)
.detail("ServerSize", self->server_info.size())
.detail("MachineSize", self->machine_info.size())
.detail("StorageTeamSize", self->configuration.storageTeamSize);
}
}
}
return Void();
}
ACTOR static Future<Void> perpetualStorageWiggleIterator(DDTeamCollection* teamCollection,
AsyncVar<bool>* stopSignal,
FutureStream<Void> finishStorageWiggleSignal) {
@ -1964,24 +1998,9 @@ public:
choose {
when(wait(stopSignal->onChange())) {}
when(waitNext(finishStorageWiggleSignal)) {
state bool takeRest = true; // delay to avoid delete and update ServerList too frequently
while (takeRest) {
wait(delayJittered(SERVER_KNOBS->PERPETUAL_WIGGLE_DELAY));
// there must not have other teams to place wiggled data
takeRest =
teamCollection->server_info.size() <= teamCollection->configuration.storageTeamSize ||
teamCollection->machine_info.size() < teamCollection->configuration.storageTeamSize;
if (takeRest) {
teamCollection->storageWiggler->setWiggleState(StorageWiggler::PAUSE);
if (teamCollection->configuration.storageMigrationType == StorageMigrationType::GRADUAL) {
TraceEvent(SevWarn, "PerpetualStorageWiggleSleep", teamCollection->distributorId)
.suppressFor(SERVER_KNOBS->PERPETUAL_WIGGLE_DELAY * 4)
.detail("ServerSize", teamCollection->server_info.size())
.detail("MachineSize", teamCollection->machine_info.size())
.detail("StorageTeamSize", teamCollection->configuration.storageTeamSize);
}
}
}
// delay to avoid delete and update ServerList too frequently, which could result busy loop or over
// utilize the disk of other active SS
wait(perpetualStorageWiggleRest(teamCollection));
wait(updateNextWigglingStorageID(teamCollection));
}
}
@ -3394,6 +3413,39 @@ Future<Void> DDTeamCollection::removeBadTeams() {
return DDTeamCollectionImpl::removeBadTeams(this);
}
double DDTeamCollection::loadBytesBalanceRatio(int64_t smallLoadThreshold) const {
double minLoadBytes = std::numeric_limits<double>::max();
double totalLoadBytes = 0;
int count = 0;
for (auto& [id, s] : server_info) {
// If a healthy SS don't have storage metrics, skip this round
if (server_status.get(s->getId()).isUnhealthy() || !s->metricsPresent()) {
TraceEvent(SevDebug, "LoadBytesBalanceRatioNoMetrics").detail("Server", id);
return 0;
}
double load = s->loadBytes();
totalLoadBytes += load;
++count;
minLoadBytes = std::min(minLoadBytes, load);
}
TraceEvent(SevDebug, "LoadBytesBalanceRatioMetrics")
.detail("TotalLoad", totalLoadBytes)
.detail("MinLoadBytes", minLoadBytes)
.detail("SmallLoadThreshold", smallLoadThreshold)
.detail("Count", count);
// avoid division-by-zero
double avgLoad = totalLoadBytes / count;
if (totalLoadBytes == 0 || avgLoad < smallLoadThreshold) {
CODE_PROBE(true, "The cluster load is small enough to ignore load bytes balance.");
return 1;
}
return minLoadBytes / avgLoad;
}
Future<Void> DDTeamCollection::storageServerFailureTracker(TCServerInfo* server,
ServerStatus* status,
Version addedVersion) {
@ -3542,7 +3594,8 @@ DDTeamCollection::DDTeamCollection(DDTeamCollectionInitParams const& params)
restartRecruiting(SERVER_KNOBS->DEBOUNCE_RECRUITING_DELAY), healthyTeamCount(0),
zeroHealthyTeams(params.zeroHealthyTeams), optimalTeamCount(0), zeroOptimalTeams(true), isTssRecruiting(false),
includedDCs(params.includedDCs), otherTrackedDCs(params.otherTrackedDCs),
processingUnhealthy(params.processingUnhealthy), readyToStart(params.readyToStart),
processingUnhealthy(params.processingUnhealthy), getAverageShardBytes(params.getAverageShardBytes),
readyToStart(params.readyToStart),
checkTeamDelay(delay(SERVER_KNOBS->CHECK_TEAM_DELAY, TaskPriority::DataDistribution)), badTeamRemover(Void()),
checkInvalidLocalities(Void()), wrongStoreTypeRemover(Void()), clearHealthyZoneFuture(true),
medianAvailableSpace(SERVER_KNOBS->MIN_AVAILABLE_SPACE_RATIO), lastMedianAvailableSpaceUpdate(0),
@ -5104,7 +5157,8 @@ public:
makeReference<AsyncVar<bool>>(false),
PromiseStream<GetMetricsRequest>(),
Promise<UID>(),
PromiseStream<Promise<int>>() }));
PromiseStream<Promise<int>>(),
PromiseStream<Promise<int64_t>>() }));
for (int id = 1; id <= processCount; ++id) {
UID uid(id, 0);
@ -5148,7 +5202,8 @@ public:
makeReference<AsyncVar<bool>>(false),
PromiseStream<GetMetricsRequest>(),
Promise<UID>(),
PromiseStream<Promise<int>>() }));
PromiseStream<Promise<int>>(),
PromiseStream<Promise<int64_t>>() }));
for (int id = 1; id <= processCount; id++) {
UID uid(id, 0);

View File

@ -626,7 +626,7 @@ ACTOR Future<Void> dataDistribution(Reference<DataDistributor> self,
}
self->shardsAffectedByTeamFailure = makeReference<ShardsAffectedByTeamFailure>();
self->physicalShardCollection = makeReference<PhysicalShardCollection>();
self->physicalShardCollection = makeReference<PhysicalShardCollection>(self->txnProcessor);
wait(self->resumeRelocations());
std::vector<TeamCollectionInterface> tcis; // primary and remote region interface
@ -723,7 +723,8 @@ ACTOR Future<Void> dataDistribution(Reference<DataDistributor> self,
processingWiggle,
getShardMetrics,
removeFailedServer,
getUnhealthyRelocationCount });
getUnhealthyRelocationCount,
getAverageShardBytes });
teamCollectionsPtrs.push_back(primaryTeamCollection.getPtr());
auto recruitStorage = IAsyncListener<RequestStream<RecruitStorageRequest>>::create(
self->dbInfo, [](auto const& info) { return info.clusterInterface.recruitStorage; });
@ -744,7 +745,8 @@ ACTOR Future<Void> dataDistribution(Reference<DataDistributor> self,
processingWiggle,
getShardMetrics,
removeFailedServer,
getUnhealthyRelocationCount });
getUnhealthyRelocationCount,
getAverageShardBytes });
teamCollectionsPtrs.push_back(remoteTeamCollection.getPtr());
remoteTeamCollection->teamCollections = teamCollectionsPtrs;
actors.push_back(reportErrorsExcept(DDTeamCollection::run(remoteTeamCollection,

View File

@ -435,7 +435,7 @@ gets deleted as the ref count becomes 0.
class ReadIteratorPool {
public:
ReadIteratorPool(UID id, DB& db, CF& cf, const rocksdb::ReadOptions readOptions)
: db(db), cf(cf), index(0), iteratorsReuseCount(0), readRangeOptions(readOptions) {
: db(db), cf(cf), index(0), deletedUptoIndex(0), iteratorsReuseCount(0), readRangeOptions(readOptions) {
readRangeOptions.background_purge_on_iterator_cleanup = true;
readRangeOptions.auto_prefix_mode = (SERVER_KNOBS->ROCKSDB_PREFIX_LEN > 0);
TraceEvent("ReadIteratorPool", id)
@ -455,8 +455,12 @@ public:
void update() {
if (SERVER_KNOBS->ROCKSDB_READ_RANGE_REUSE_ITERATORS ||
SERVER_KNOBS->ROCKSDB_READ_RANGE_REUSE_BOUNDED_ITERATORS) {
std::lock_guard<std::mutex> lock(mutex);
iteratorsMap.clear();
mutex.lock();
// The latest index might contain the current iterator which is getting created.
// But, that should be ok to avoid adding more code complexity.
deletedUptoIndex = index;
mutex.unlock();
deleteIteratorsPromise.send(Void());
}
}
@ -465,7 +469,7 @@ public:
if (SERVER_KNOBS->ROCKSDB_READ_RANGE_REUSE_ITERATORS) {
mutex.lock();
for (it = iteratorsMap.begin(); it != iteratorsMap.end(); it++) {
if (!it->second.inUse) {
if (!it->second.inUse && it->second.index > deletedUptoIndex) {
it->second.inUse = true;
iteratorsReuseCount++;
ReadIterator iter = it->second;
@ -486,7 +490,8 @@ public:
// TODO: Based on the datasize in the keyrange, decide whether to store the iterator for reuse.
mutex.lock();
for (it = iteratorsMap.begin(); it != iteratorsMap.end(); it++) {
if (!it->second.inUse && it->second.keyRange.contains(keyRange)) {
if (!it->second.inUse && it->second.index > deletedUptoIndex &&
it->second.keyRange.contains(keyRange)) {
it->second.inUse = true;
iteratorsReuseCount++;
ReadIterator iter = it->second;
@ -533,8 +538,10 @@ public:
void refreshIterators() {
std::lock_guard<std::mutex> lock(mutex);
it = iteratorsMap.begin();
auto currTime = now();
while (it != iteratorsMap.end()) {
if (now() - it->second.creationTime > SERVER_KNOBS->ROCKSDB_READ_RANGE_ITERATOR_REFRESH_TIME) {
if ((it->second.index <= deletedUptoIndex) ||
((currTime - it->second.creationTime) > SERVER_KNOBS->ROCKSDB_READ_RANGE_ITERATOR_REFRESH_TIME)) {
it = iteratorsMap.erase(it);
} else {
it++;
@ -546,6 +553,8 @@ public:
uint64_t numTimesReadIteratorsReused() { return iteratorsReuseCount; }
FutureStream<Void> getDeleteIteratorsFutureStream() { return deleteIteratorsPromise.getFuture(); }
private:
std::unordered_map<int, ReadIterator> iteratorsMap;
std::unordered_map<int, ReadIterator>::iterator it;
@ -555,7 +564,9 @@ private:
std::mutex mutex;
// incrementing counter for every new iterator creation, to uniquely identify the iterator in returnIterator().
uint64_t index;
uint64_t deletedUptoIndex;
uint64_t iteratorsReuseCount;
ThreadReturnPromiseStream<Void> deleteIteratorsPromise;
};
class PerfContextMetrics {
@ -825,9 +836,19 @@ uint64_t PerfContextMetrics::getRocksdbPerfcontextMetric(int metric) {
ACTOR Future<Void> refreshReadIteratorPool(std::shared_ptr<ReadIteratorPool> readIterPool) {
if (SERVER_KNOBS->ROCKSDB_READ_RANGE_REUSE_ITERATORS || SERVER_KNOBS->ROCKSDB_READ_RANGE_REUSE_BOUNDED_ITERATORS) {
state FutureStream<Void> deleteIteratorsFutureStream = readIterPool->getDeleteIteratorsFutureStream();
loop {
wait(delay(SERVER_KNOBS->ROCKSDB_READ_RANGE_ITERATOR_REFRESH_TIME));
readIterPool->refreshIterators();
choose {
when(wait(delay(SERVER_KNOBS->ROCKSDB_READ_RANGE_ITERATOR_REFRESH_TIME))) {
readIterPool->refreshIterators();
}
when(waitNext(deleteIteratorsFutureStream)) {
// Add a delay(0.0) to ensure the rest of the caller code runs before refreshing iterators,
// i.e., making the refreshIterators() call here asynchronous.
wait(delay(0.0));
readIterPool->refreshIterators();
}
}
}
}
return Void();
@ -2574,15 +2595,23 @@ TEST_CASE("noSim/fdbserver/KeyValueStoreRocksDB/CheckpointRestoreKeyValues") {
std::string checkpointDir = cwd + "checkpoint";
CheckpointRequest request(
latestVersion, { allKeys }, RocksDB, deterministicRandom()->randomUniqueID(), checkpointDir);
latestVersion, { allKeys }, DataMoveRocksCF, deterministicRandom()->randomUniqueID(), checkpointDir);
CheckpointMetaData metaData = wait(kvStore->checkpoint(request));
state ICheckpointReader* cpReader = newCheckpointReader(metaData, deterministicRandom()->randomUniqueID());
wait(cpReader->init(BinaryWriter::toValue(KeyRangeRef("foo"_sr, "foobar"_sr), IncludeVersion())));
TraceEvent(SevDebug, "RocksDBCreatedCheckpoint");
state KeyRange testRange = KeyRangeRef("foo"_sr, "foobar"_sr);
state Standalone<StringRef> token = BinaryWriter::toValue(testRange, IncludeVersion());
state ICheckpointReader* cpReader =
newCheckpointReader(metaData, CheckpointAsKeyValues::True, deterministicRandom()->randomUniqueID());
TraceEvent(SevDebug, "RocksDBCheckpointReaderCreated");
ASSERT(cpReader != nullptr);
wait(cpReader->init(token));
TraceEvent(SevDebug, "RocksDBCheckpointReaderInited");
state std::unique_ptr<ICheckpointIterator> iter = cpReader->getIterator(testRange);
loop {
try {
state RangeResult res =
wait(cpReader->nextKeyValues(CLIENT_KNOBS->REPLY_BYTE_LIMIT, CLIENT_KNOBS->REPLY_BYTE_LIMIT));
wait(iter->nextBatch(CLIENT_KNOBS->REPLY_BYTE_LIMIT, CLIENT_KNOBS->REPLY_BYTE_LIMIT));
state int i = 0;
for (; i < res.size(); ++i) {
Optional<Value> val = wait(kvStore->readValue(res[i].key));
@ -2597,6 +2626,7 @@ TEST_CASE("noSim/fdbserver/KeyValueStoreRocksDB/CheckpointRestoreKeyValues") {
}
}
iter.reset();
std::vector<Future<Void>> closes;
closes.push_back(cpReader->close());
closes.push_back(kvStore->onClosed());

View File

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

View File

@ -28,7 +28,9 @@
#include "flow/Trace.h"
#include "flow/flow.h"
#include <cstddef>
#ifndef WIN32
#include <sys/socket.h>
#endif
#include "flow/actorcompiler.h"
#include "flow/network.h"

View File

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

View File

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

View File

@ -42,6 +42,7 @@
#include "fdbclient/NativeAPI.actor.h"
#include "fdbclient/BackupAgent.actor.h"
#include "fdbclient/versions.h"
#include "flow/IRandom.h"
#include "flow/MkCert.h"
#include "fdbrpc/WellKnownEndpoints.h"
#include "flow/ProtocolVersion.h"
@ -403,6 +404,7 @@ public:
// remote key value store is a child process spawned by the SS process to run the storage engine
bool disableRemoteKVS = false;
// 7.2 cannot be downgraded to 7.1 or below after enabling encryption-at-rest.
// TODO: Remove this bool once the encryption knobs are removed
bool disableEncryption = false;
// By default, encryption mode is set randomly (based on the tenant mode)
// If provided, set using EncryptionAtRestMode::fromString
@ -2045,6 +2047,15 @@ void SimulationConfig::generateNormalConfig(const TestConfig& testConfig) {
setConfigDB(testConfig);
}
bool validateEncryptAndTenantModePair(EncryptionAtRestMode encryptMode, TenantMode tenantMode) {
// Domain aware encryption is only allowed when the tenant mode is required. Other encryption modes (disabled or
// cluster aware) are allowed regardless of the tenant mode
if (encryptMode.mode == EncryptionAtRestMode::DISABLED || encryptMode.mode == EncryptionAtRestMode::CLUSTER_AWARE) {
return true;
}
return tenantMode == TenantMode::REQUIRED;
}
// Configures the system according to the given specifications in order to run
// simulation under the correct conditions
void setupSimulatedSystem(std::vector<Future<Void>>* systemActors,
@ -2064,16 +2075,28 @@ void setupSimulatedSystem(std::vector<Future<Void>>* systemActors,
simconfig.db.tenantMode = tenantMode;
simconfig.db.encryptionAtRestMode = EncryptionAtRestMode::DISABLED;
if (!testConfig.encryptModes.empty()) {
simconfig.db.encryptionAtRestMode =
EncryptionAtRestMode::fromString(deterministicRandom()->randomChoice(testConfig.encryptModes));
} else if (!testConfig.disableEncryption && deterministicRandom()->coinflip()) {
if (tenantMode == TenantMode::DISABLED || tenantMode == TenantMode::OPTIONAL_TENANT ||
deterministicRandom()->coinflip()) {
// optional and disabled tenant modes currently only support cluster aware encryption
simconfig.db.encryptionAtRestMode = EncryptionAtRestMode::CLUSTER_AWARE;
// TODO: Remove check on the ENABLE_ENCRYPTION knob once the EKP can start using the db config
if (!testConfig.disableEncryption && (SERVER_KNOBS->ENABLE_ENCRYPTION || !testConfig.encryptModes.empty())) {
if (!testConfig.encryptModes.empty()) {
std::vector<EncryptionAtRestMode> validEncryptModes;
// Get the subset of valid encrypt modes given the tenant mode
for (int i = 0; i < testConfig.encryptModes.size(); i++) {
EncryptionAtRestMode encryptMode = EncryptionAtRestMode::fromString(testConfig.encryptModes.at(i));
if (validateEncryptAndTenantModePair(encryptMode, tenantMode)) {
validEncryptModes.push_back(encryptMode);
}
}
if (validEncryptModes.size() > 0) {
simconfig.db.encryptionAtRestMode = deterministicRandom()->randomChoice(validEncryptModes);
}
} else {
simconfig.db.encryptionAtRestMode = EncryptionAtRestMode::DOMAIN_AWARE;
// TODO: These cases should only trigger with probability (BUGGIFY) once the server knob is removed
if (tenantMode == TenantMode::DISABLED || tenantMode == TenantMode::OPTIONAL_TENANT || BUGGIFY) {
// optional and disabled tenant modes currently only support cluster aware encryption
simconfig.db.encryptionAtRestMode = EncryptionAtRestMode::CLUSTER_AWARE;
} else {
simconfig.db.encryptionAtRestMode = EncryptionAtRestMode::DOMAIN_AWARE;
}
}
}
TraceEvent("SimulatedClusterEncryptionMode").detail("Mode", simconfig.db.encryptionAtRestMode.toString());

View File

@ -3124,6 +3124,8 @@ ACTOR Future<StatusReply> clusterGetStatus(
state JsonBuilderObject recoveryStateStatus = wait(
recoveryStateStatusFetcher(cx, ccWorker, mWorker, workers.size(), &status_incomplete_reasons, &statusCode));
state JsonBuilderObject idmpKeyStatus = wait(getIdmpKeyStatus(cx));
// machine metrics
state WorkerEvents mMetrics = workerEventsVec[0].present() ? workerEventsVec[0].get().first : WorkerEvents();
// process metrics
@ -3505,6 +3507,8 @@ ACTOR Future<StatusReply> clusterGetStatus(
if (!recoveryStateStatus.empty())
statusObj["recovery_state"] = recoveryStateStatus;
statusObj["idempotency_ids"] = idmpKeyStatus;
// cluster messages subsection;
JsonBuilderArray clientIssuesArr = getClientIssuesAsMessages(clientStatus);
if (clientIssuesArr.size() > 0) {

View File

@ -509,8 +509,8 @@ Future<Void> startSystemMonitor(std::string dataFolder,
Optional<Standalone<StringRef>> dcId,
Optional<Standalone<StringRef>> zoneId,
Optional<Standalone<StringRef>> machineId) {
initializeSystemMonitorMachineState(
SystemMonitorMachineState(dataFolder, dcId, zoneId, machineId, g_network->getLocalAddress().ip));
initializeSystemMonitorMachineState(SystemMonitorMachineState(
dataFolder, dcId, zoneId, machineId, g_network->getLocalAddress().ip, FDB_VT_VERSION));
systemMonitor();
return recurring(&systemMonitor, SERVER_KNOBS->SYSTEM_MONITOR_FREQUENCY, TaskPriority::FlushTrace);

View File

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

View File

@ -248,27 +248,64 @@ FDB_DECLARE_BOOLEAN_PARAM(InOverSizePhysicalShard);
FDB_DECLARE_BOOLEAN_PARAM(PhysicalShardAvailable);
FDB_DECLARE_BOOLEAN_PARAM(MoveKeyRangeOutPhysicalShard);
struct ShardMetrics {
StorageMetrics metrics;
double lastLowBandwidthStartTime;
int shardCount; // number of smaller shards whose metrics are aggregated in the ShardMetrics
bool operator==(ShardMetrics const& rhs) const {
return metrics == rhs.metrics && lastLowBandwidthStartTime == rhs.lastLowBandwidthStartTime &&
shardCount == rhs.shardCount;
}
ShardMetrics(StorageMetrics const& metrics, double lastLowBandwidthStartTime, int shardCount)
: metrics(metrics), lastLowBandwidthStartTime(lastLowBandwidthStartTime), shardCount(shardCount) {}
};
struct ShardTrackedData {
Future<Void> trackShard;
Future<Void> trackBytes;
Reference<AsyncVar<Optional<ShardMetrics>>> stats;
};
class PhysicalShardCollection : public ReferenceCounted<PhysicalShardCollection> {
public:
PhysicalShardCollection() : lastTransitionStartTime(now()), requireTransition(false) {}
PhysicalShardCollection(Reference<IDDTxnProcessor> db)
: txnProcessor(db), lastTransitionStartTime(now()), requireTransition(false) {}
enum class PhysicalShardCreationTime { DDInit, DDRelocator };
struct PhysicalShard {
PhysicalShard() : id(UID().first()) {}
PhysicalShard(uint64_t id,
PhysicalShard(Reference<IDDTxnProcessor> txnProcessor,
uint64_t id,
StorageMetrics const& metrics,
std::vector<ShardsAffectedByTeamFailure::Team> teams,
PhysicalShardCreationTime whenCreated)
: id(id), metrics(metrics), teams(teams), whenCreated(whenCreated) {}
: txnProcessor(txnProcessor), id(id), metrics(metrics), teams(teams), whenCreated(whenCreated) {}
// Adds `newRange` to this physical shard and starts monitoring the shard.
void addRange(const KeyRange& newRange);
// Removes `outRange` from this physical shard and updates monitored shards.
void removeRange(const KeyRange& outRange);
std::string toString() const { return fmt::format("{}", std::to_string(id)); }
Reference<IDDTxnProcessor> txnProcessor;
uint64_t id; // physical shard id (never changed)
StorageMetrics metrics; // current metrics, updated by shardTracker
std::vector<ShardsAffectedByTeamFailure::Team> teams; // which team owns this physical shard (never changed)
PhysicalShardCreationTime whenCreated; // when this physical shard is created (never changed)
struct RangeData {
Future<Void> trackMetrics;
Reference<AsyncVar<Optional<ShardMetrics>>>
stats; // TODO(zhewu): aggregate all metrics to a single physical shard metrics.
};
std::unordered_map<KeyRange, RangeData> rangeData;
};
// Generate a random physical shard ID, which is not UID().first() nor anonymousShardId.first()
@ -373,6 +410,9 @@ private:
// In keyRangePhysicalShardIDMap, set the input physical shard id to the input key range
void updatekeyRangePhysicalShardIDMap(KeyRange keyRange, uint64_t physicalShardID, uint64_t debugID);
// Checks the consistency between the mapping of physical shards and key ranges.
void checkKeyRangePhysicalShardMapping();
// Return a string concating the input IDs interleaving with " "
std::string convertIDsToString(std::set<uint64_t> ids);
@ -402,6 +442,8 @@ private:
inline bool requireTransitionCheck() { return requireTransition; }
Reference<IDDTxnProcessor> txnProcessor;
// Core data structures
// Physical shard instances indexed by physical shard id
std::unordered_map<uint64_t, PhysicalShard> physicalShardInstances;
@ -443,26 +485,6 @@ struct InitialDataDistribution : ReferenceCounted<InitialDataDistribution> {
KeyRangeMap<std::shared_ptr<DataMove>> dataMoveMap;
};
struct ShardMetrics {
StorageMetrics metrics;
double lastLowBandwidthStartTime;
int shardCount; // number of smaller shards whose metrics are aggregated in the ShardMetrics
bool operator==(ShardMetrics const& rhs) const {
return metrics == rhs.metrics && lastLowBandwidthStartTime == rhs.lastLowBandwidthStartTime &&
shardCount == rhs.shardCount;
}
ShardMetrics(StorageMetrics const& metrics, double lastLowBandwidthStartTime, int shardCount)
: metrics(metrics), lastLowBandwidthStartTime(lastLowBandwidthStartTime), shardCount(shardCount) {}
};
struct ShardTrackedData {
Future<Void> trackShard;
Future<Void> trackBytes;
Reference<AsyncVar<Optional<ShardMetrics>>> stats;
};
// Holds the permitted size and IO Bounds for a shard
struct ShardSizeBounds {
StorageMetrics max;

View File

@ -107,7 +107,7 @@ struct ProxyStats {
NotifiedVersion* pVersion,
NotifiedVersion* pCommittedVersion,
int64_t* commitBatchesMemBytesCountPtr,
std::map<TenantName, TenantMapEntry>* pTenantMap)
std::unordered_map<int64_t, TenantName>* pTenantMap)
: cc("ProxyStats", id.toString()), txnCommitIn("TxnCommitIn", cc),
txnCommitVersionAssigned("TxnCommitVersionAssigned", cc), txnCommitResolving("TxnCommitResolving", cc),
txnCommitResolved("TxnCommitResolved", cc), txnCommitOut("TxnCommitOut", cc),
@ -173,8 +173,8 @@ struct ExpectedIdempotencyIdCountForKey {
struct ProxyCommitData {
UID dbgid;
int64_t commitBatchesMemBytesCount;
std::map<TenantName, TenantMapEntry> tenantMap;
std::unordered_map<int64_t, TenantNameUniqueSet> tenantIdIndex;
std::map<TenantName, int64_t> tenantNameIndex;
std::unordered_map<int64_t, TenantName> tenantMap;
std::unordered_set<TenantName> tenantsOverStorageQuota;
ProxyStats stats;
MasterInterface master;
@ -235,7 +235,7 @@ struct ProxyCommitData {
double lastResolverReset;
int localTLogCount = -1;
bool isEncryptionEnabled = false;
EncryptionAtRestMode encryptMode;
PromiseStream<ExpectedIdempotencyIdCountForKey> expectedIdempotencyIdCountForKey;
Standalone<VectorRef<MutationRef>> idempotencyClears;
@ -297,18 +297,19 @@ struct ProxyCommitData {
Version recoveryTransactionVersion,
PublicRequestStream<CommitTransactionRequest> commit,
Reference<AsyncVar<ServerDBInfo> const> db,
bool firstProxy)
bool firstProxy,
EncryptionAtRestMode encryptMode)
: dbgid(dbgid), commitBatchesMemBytesCount(0),
stats(dbgid, &version, &committedVersion, &commitBatchesMemBytesCount, &tenantMap), master(master),
logAdapter(nullptr), txnStateStore(nullptr), committedVersion(recoveryTransactionVersion),
minKnownCommittedVersion(0), version(0), lastVersionTime(0), commitVersionRequestNumber(1),
mostRecentProcessedRequestNumber(0), firstProxy(firstProxy), lastCoalesceTime(0), locked(false),
commitBatchInterval(SERVER_KNOBS->COMMIT_TRANSACTION_BATCH_INTERVAL_MIN), localCommitBatchesStarted(0),
getConsistentReadVersion(getConsistentReadVersion), commit(commit),
mostRecentProcessedRequestNumber(0), firstProxy(firstProxy), encryptMode(encryptMode), lastCoalesceTime(0),
locked(false), commitBatchInterval(SERVER_KNOBS->COMMIT_TRANSACTION_BATCH_INTERVAL_MIN),
localCommitBatchesStarted(0), getConsistentReadVersion(getConsistentReadVersion), commit(commit),
cx(openDBOnServer(db, TaskPriority::DefaultEndpoint, LockAware::True)), db(db),
singleKeyMutationEvent("SingleKeyMutation"_sr), lastTxsPop(0), popRemoteTxs(false), lastStartCommit(0),
lastCommitLatency(SERVER_KNOBS->REQUIRED_MIN_RECOVERY_DURATION), lastCommitTime(0), lastMasterReset(now()),
lastResolverReset(now()), isEncryptionEnabled(isEncryptionOpSupported(EncryptOperationType::TLOG_ENCRYPTION)) {
lastResolverReset(now()) {
commitComputePerOperation.resize(SERVER_KNOBS->PROXY_COMPUTE_BUCKETS, 0.0);
}
};

View File

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

View File

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

View File

@ -725,10 +725,12 @@ struct InitializeCommitProxyRequest {
Version recoveryTransactionVersion;
bool firstProxy;
ReplyPromise<CommitProxyInterface> reply;
EncryptionAtRestMode encryptMode;
template <class Ar>
void serialize(Ar& ar) {
serializer(ar, master, masterLifetime, recoveryCount, recoveryTransactionVersion, firstProxy, reply);
serializer(
ar, master, masterLifetime, recoveryCount, recoveryTransactionVersion, firstProxy, reply, encryptMode);
}
};

View File

@ -126,7 +126,7 @@ private:
std::map<TenantGroupName, TenantGroupEntry>(tenantGroupList.results.begin(), tenantGroupList.results.end());
for (auto t : tenantGroupTenantTuples.results) {
ASSERT(t.size() == 2);
ASSERT_EQ(t.size(), 2);
TenantGroupName tenantGroupName = t.getString(0);
TenantName tenantName = t.getString(1);
ASSERT(self->metadata.tenantGroupMap.count(tenantGroupName));
@ -134,24 +134,24 @@ private:
self->metadata.tenantGroupIndex[tenantGroupName].insert(tenantName);
ASSERT(self->metadata.tenantsInTenantGroupIndex.insert(tenantName).second);
}
ASSERT(self->metadata.tenantGroupIndex.size() == self->metadata.tenantGroupMap.size());
ASSERT_EQ(self->metadata.tenantGroupIndex.size(), self->metadata.tenantGroupMap.size());
return Void();
}
void validateTenantMetadata() {
if (metadata.clusterType == ClusterType::METACLUSTER_MANAGEMENT) {
ASSERT(metadata.tenantMap.size() <= metaclusterMaxTenants);
ASSERT_LE(metadata.tenantMap.size(), metaclusterMaxTenants);
} else {
ASSERT(metadata.tenantMap.size() <= CLIENT_KNOBS->MAX_TENANTS_PER_CLUSTER);
ASSERT_LE(metadata.tenantMap.size(), CLIENT_KNOBS->MAX_TENANTS_PER_CLUSTER);
}
ASSERT(metadata.tenantMap.size() == metadata.tenantCount);
ASSERT(metadata.tenantIdIndex.size() == metadata.tenantCount);
ASSERT_EQ(metadata.tenantMap.size(), metadata.tenantCount);
ASSERT_EQ(metadata.tenantIdIndex.size(), metadata.tenantCount);
for (auto [tenantName, tenantMapEntry] : metadata.tenantMap) {
if (metadata.clusterType != ClusterType::METACLUSTER_DATA) {
ASSERT(tenantMapEntry.id <= metadata.lastTenantId);
ASSERT_LE(tenantMapEntry.id, metadata.lastTenantId);
}
ASSERT(metadata.tenantIdIndex[tenantMapEntry.id] == tenantName);
@ -169,25 +169,25 @@ private:
// If the rename pair is present, it should be in the map and match our current entry
if (tenantMapEntry.renamePair.present()) {
auto pairMapEntry = metadata.tenantMap[tenantMapEntry.renamePair.get()];
ASSERT(pairMapEntry.id == tenantMapEntry.id);
ASSERT_EQ(pairMapEntry.id, tenantMapEntry.id);
ASSERT(pairMapEntry.prefix == tenantMapEntry.prefix);
ASSERT(pairMapEntry.configurationSequenceNum == tenantMapEntry.configurationSequenceNum);
ASSERT_EQ(pairMapEntry.configurationSequenceNum, tenantMapEntry.configurationSequenceNum);
ASSERT(pairMapEntry.assignedCluster.present());
ASSERT(pairMapEntry.assignedCluster.get() == tenantMapEntry.assignedCluster.get());
ASSERT(pairMapEntry.renamePair.present());
ASSERT(pairMapEntry.renamePair.get() == tenantName);
if (tenantMapEntry.tenantState == TenantState::RENAMING_FROM) {
ASSERT(pairMapEntry.tenantState == TenantState::RENAMING_TO);
ASSERT_EQ(pairMapEntry.tenantState, TenantState::RENAMING_TO);
} else if (tenantMapEntry.tenantState == TenantState::RENAMING_TO) {
ASSERT(pairMapEntry.tenantState == TenantState::RENAMING_FROM);
ASSERT_EQ(pairMapEntry.tenantState, TenantState::RENAMING_FROM);
} else if (tenantMapEntry.tenantState == TenantState::REMOVING) {
ASSERT(pairMapEntry.tenantState == TenantState::REMOVING);
ASSERT_EQ(pairMapEntry.tenantState, TenantState::REMOVING);
} else {
ASSERT(false); // Entry in an invalid state if we have a rename pair
}
}
} else {
ASSERT(tenantMapEntry.tenantState == TenantState::READY);
ASSERT_EQ(tenantMapEntry.tenantState, TenantState::READY);
ASSERT(!tenantMapEntry.assignedCluster.present());
ASSERT(!tenantMapEntry.renamePair.present());
}

View File

@ -2425,7 +2425,7 @@ ACTOR Future<Void> fetchCheckpointQ(StorageServer* self, FetchCheckpointRequest
}
try {
reader = newCheckpointReader(it->second, deterministicRandom()->randomUniqueID());
reader = newCheckpointReader(it->second, CheckpointAsKeyValues::False, deterministicRandom()->randomUniqueID());
wait(reader->init(req.token));
loop {
@ -2479,13 +2479,15 @@ ACTOR Future<Void> fetchCheckpointKeyValuesQ(StorageServer* self, FetchCheckpoin
}
state ICheckpointReader* reader = nullptr;
state std::unique_ptr<ICheckpointIterator> iter;
try {
reader = newCheckpointReader(it->second, self->thisServerID);
reader = newCheckpointReader(it->second, CheckpointAsKeyValues::True, self->thisServerID);
wait(reader->init(BinaryWriter::toValue(req.range, IncludeVersion())));
iter = reader->getIterator(req.range);
loop {
state RangeResult res =
wait(reader->nextKeyValues(CLIENT_KNOBS->REPLY_BYTE_LIMIT, CLIENT_KNOBS->REPLY_BYTE_LIMIT));
wait(iter->nextBatch(CLIENT_KNOBS->REPLY_BYTE_LIMIT, CLIENT_KNOBS->REPLY_BYTE_LIMIT));
if (!res.empty()) {
TraceEvent(SevDebug, "FetchCheckpontKeyValuesReadRange", self->thisServerID)
.detail("CheckpointID", req.checkpointID)
@ -2524,7 +2526,10 @@ ACTOR Future<Void> fetchCheckpointKeyValuesQ(StorageServer* self, FetchCheckpoin
}
}
wait(reader->close());
iter.reset();
if (!reader->inUse()) {
wait(reader->close());
}
return Void();
}
@ -2751,10 +2756,35 @@ static std::deque<Standalone<MutationsAndVersionRef>>::const_iterator searchChan
}
}
// The normal read case for a change feed stream query is that it will first read the disk portion, which is at a lower
// version than the memory portion, and then will effectively switch to reading only the memory portion. The complexity
// lies in the fact that the feed does not know the switchover point ahead of time before reading from disk, and the
// switchover point is constantly changing as the SS persists the in-memory data to disk. As a result, the
// implementation first reads from memory, then reads from disk if necessary, then merges the result and potentially
// discards the in-memory read data if the disk data is large and behind the in-memory data. The goal of
// FeedDiskReadState is that we want to skip doing the full memory read if we still have a lot of disk reads to catch up
// on. In the DISK_CATCHUP phase, the feed query will read only the first row from memory, to
// determine if it's hit the switchover point, instead of reading (potentially) both in the normal phase. We also want
// to default to the normal behavior at the start in case there is not a lot of disk data. This guarantees that if we
// somehow incorrectly went into DISK_CATCHUP when there wasn't much more data on disk, we only have one cycle of
// getChangeFeedMutations in the incorrect mode that returns a smaller result before switching to NORMAL mode.
//
// Put another way, the state transitions are:
//
// STARTING ->
// DISK_CATCHUP (if after the first read, there is more disk data to read before the first memory data)
// NORMAL (otherwise)
// DISK_CATCHUP ->
// still DISK_CATCHUP (if there is still more disk data to read before the first memory data)
// NORMAL (otherwise)
// NORMAL -> NORMAL (always)
enum FeedDiskReadState { STARTING, NORMAL, DISK_CATCHUP };
ACTOR Future<std::pair<ChangeFeedStreamReply, bool>> getChangeFeedMutations(StorageServer* data,
ChangeFeedStreamRequest req,
bool inverted,
bool atLatest) {
bool atLatest,
FeedDiskReadState* feedDiskReadState) {
state ChangeFeedStreamReply reply;
state ChangeFeedStreamReply memoryReply;
state int remainingLimitBytes = CLIENT_KNOBS->REPLY_BYTE_LIMIT;
@ -2823,9 +2853,15 @@ ACTOR Future<std::pair<ChangeFeedStreamReply, bool>> getChangeFeedMutations(Stor
if (req.end > emptyVersion + 1) {
auto it = searchChangeFeedStart(feedInfo->mutations, req.begin, atLatest);
while (it != feedInfo->mutations.end()) {
// If DISK_CATCHUP, only read 1 mutation from the memory queue
if (it->version >= req.end || it->version > dequeVersion || remainingLimitBytes <= 0) {
break;
}
if ((*feedDiskReadState) == FeedDiskReadState::DISK_CATCHUP && !memoryReply.mutations.empty()) {
// so we don't add an empty mutation at the end
remainingLimitBytes = -1;
break;
}
MutationsAndVersionRef m = *it;
if (doFilterMutations) {
m = filterMutations(memoryReply.arena, *it, req.range, inverted);
@ -2980,6 +3016,28 @@ ACTOR Future<std::pair<ChangeFeedStreamReply, bool>> getChangeFeedMutations(Stor
lastVersion = version;
lastKnownCommitted = knownCommittedVersion;
}
if ((*feedDiskReadState) == FeedDiskReadState::STARTING ||
(*feedDiskReadState) == FeedDiskReadState::DISK_CATCHUP) {
if (!memoryReply.mutations.empty() && !reply.mutations.empty() &&
reply.mutations.back().version < memoryReply.mutations.front().version && remainingDurableBytes <= 0) {
// if we read a full batch from disk and the entire disk read was still less than the first memory
// mutation, switch to disk_catchup mode
*feedDiskReadState = FeedDiskReadState::DISK_CATCHUP;
CODE_PROBE(true, "Feed switching to disk_catchup mode");
} else {
// for testing
if ((*feedDiskReadState) == FeedDiskReadState::STARTING && BUGGIFY_WITH_PROB(0.001)) {
*feedDiskReadState = FeedDiskReadState::DISK_CATCHUP;
CODE_PROBE(true, "Feed forcing disk_catchup mode");
} else {
// else switch to normal mode
CODE_PROBE(true, "Feed switching to normal mode");
*feedDiskReadState = FeedDiskReadState::NORMAL;
}
}
}
if (remainingDurableBytes > 0) {
reply.arena.dependsOn(memoryReply.arena);
auto it = memoryReply.mutations.begin();
@ -3001,6 +3059,7 @@ ACTOR Future<std::pair<ChangeFeedStreamReply, bool>> getChangeFeedMutations(Stor
}
} else {
reply = memoryReply;
*feedDiskReadState = FeedDiskReadState::NORMAL;
}
bool gotAll = remainingLimitBytes > 0 && remainingDurableBytes > 0 && data->version.get() == startVersion;
@ -3159,6 +3218,7 @@ ACTOR Future<Void> changeFeedStreamQ(StorageServer* data, ChangeFeedStreamReques
state Span span("SS:getChangeFeedStream"_loc, req.spanContext);
state bool atLatest = false;
state bool removeUID = false;
state FeedDiskReadState feedDiskReadState = STARTING;
state Optional<Version> blockedVersion;
try {
@ -3244,7 +3304,7 @@ ACTOR Future<Void> changeFeedStreamQ(StorageServer* data, ChangeFeedStreamReques
// keep this as not state variable so it is freed after sending to reduce memory
Future<std::pair<ChangeFeedStreamReply, bool>> feedReplyFuture =
getChangeFeedMutations(data, req, false, atLatest);
getChangeFeedMutations(data, req, false, atLatest, &feedDiskReadState);
if (atLatest && !removeUID && !feedReplyFuture.isReady()) {
data->changeFeedClientVersions[req.reply.getEndpoint().getPrimaryAddress()][req.id] =
blockedVersion.present() ? blockedVersion.get() : data->prevVersion;

View File

@ -30,6 +30,7 @@
#include "fdbclient/GlobalConfig.actor.h"
#include "fdbclient/ProcessInterface.h"
#include "fdbclient/StorageServerInterface.h"
#include "fdbclient/versions.h"
#include "fdbserver/Knobs.h"
#include "flow/ActorCollection.h"
#include "flow/Error.h"
@ -1828,8 +1829,12 @@ ACTOR Future<Void> workerServer(Reference<IClusterConnectionRecord> connRecord,
filesClosed.add(stopping.getFuture());
initializeSystemMonitorMachineState(SystemMonitorMachineState(
folder, locality.dcId(), locality.zoneId(), locality.machineId(), g_network->getLocalAddress().ip));
initializeSystemMonitorMachineState(SystemMonitorMachineState(folder,
locality.dcId(),
locality.zoneId(),
locality.machineId(),
g_network->getLocalAddress().ip,
FDB_VT_VERSION));
{
auto recruited = interf;

View File

@ -46,14 +46,19 @@ struct AutomaticIdempotencyWorkload : TestWorkload {
static constexpr auto NAME = "AutomaticIdempotencyCorrectness";
int64_t numTransactions;
Key keyPrefix;
int64_t minMinAgeSeconds;
double automaticPercentage;
constexpr static double slop = 2.0;
double pollingInterval;
bool ok = true;
AutomaticIdempotencyWorkload(WorkloadContext const& wcx) : TestWorkload(wcx) {
numTransactions = getOption(options, "numTransactions"_sr, 2500);
keyPrefix = KeyRef(getOption(options, "keyPrefix"_sr, "/autoIdempotency/"_sr));
minMinAgeSeconds = getOption(options, "minMinAgeSeconds"_sr, 15);
automaticPercentage = getOption(options, "automaticPercentage"_sr, 0.1);
pollingInterval = getOption(options, "pollingInterval"_sr, 5.0);
}
Future<Void> setup(Database const& cx) override { return Void(); }
@ -101,6 +106,7 @@ struct AutomaticIdempotencyWorkload : TestWorkload {
wait(runRYWTransaction(db,
[=](Reference<ReadYourWritesTransaction> tr) { return logIdempotencyIds(self, tr); }));
wait(runRYWTransaction(db, [=](Reference<ReadYourWritesTransaction> tr) { return testIdempotency(self, tr); }));
wait(testCleaner(self, db));
return self->ok;
}
@ -164,6 +170,167 @@ struct AutomaticIdempotencyWorkload : TestWorkload {
return Void();
}
ACTOR static Future<int64_t> getOldestCreatedTime(AutomaticIdempotencyWorkload* self, Database db) {
state ReadYourWritesTransaction tr(db);
state RangeResult result;
state Key key;
state Version commitVersion;
loop {
try {
tr.setOption(FDBTransactionOptions::READ_SYSTEM_KEYS);
wait(store(result, tr.getRange(idempotencyIdKeys, /*limit*/ 1)));
if (result.empty()) {
TraceEvent("AutomaticIdempotencyNoIdsLeft").log();
return -1;
}
for (const auto& [k, v] : result) {
uint8_t highOrderBatchIndex;
decodeIdempotencyKey(k, commitVersion, highOrderBatchIndex);
// Decode the first idempotency id in the value
BinaryReader valReader(v.begin(), v.size(), IncludeVersion());
int64_t timeStamp; // ignored
valReader >> timeStamp;
uint8_t length;
valReader >> length;
StringRef id{ reinterpret_cast<const uint8_t*>(valReader.readBytes(length)), length };
uint8_t lowOrderBatchIndex;
valReader >> lowOrderBatchIndex;
// Recover the key written in the transaction associated with this idempotency id
BinaryWriter keyWriter(Unversioned());
keyWriter.serializeBytes(self->keyPrefix);
keyWriter.serializeBinaryItem(bigEndian64(commitVersion));
keyWriter.serializeBinaryItem(highOrderBatchIndex);
keyWriter.serializeBinaryItem(lowOrderBatchIndex);
key = keyWriter.toValue();
// We need to use a different transaction because we set READ_SYSTEM_KEYS on this one, and we might
// be using a tenant.
Optional<Value> entry = wait(runRYWTransaction(
db, [key = key](Reference<ReadYourWritesTransaction> tr) { return tr->get(key); }));
if (!entry.present()) {
TraceEvent(SevError, "AutomaticIdempotencyKeyMissing")
.detail("Key", key)
.detail("CommitVersion", commitVersion)
.detail("ReadVersion", tr.getReadVersion().get());
}
ASSERT(entry.present());
auto e = ObjectReader::fromStringRef<ValueType>(entry.get(), Unversioned());
return e.createdTime;
}
ASSERT(false);
} catch (Error& e) {
wait(tr.onError(e));
}
}
}
ACTOR static Future<bool> testCleanerOneIteration(AutomaticIdempotencyWorkload* self,
Database db,
ActorCollection* actors,
int64_t minAgeSeconds,
const std::vector<int64_t>* createdTimes) {
state Future<Void> cleaner = recurringAsync(
[db = db, minAgeSeconds = minAgeSeconds]() { return cleanIdempotencyIds(db, minAgeSeconds); },
self->pollingInterval,
true,
self->pollingInterval);
state int64_t oldestCreatedTime;
state int64_t successes = 0;
actors->add(cleaner);
loop {
// Oldest created time of a transaction from the workload which still has an idempotency id
wait(store(oldestCreatedTime, getOldestCreatedTime(self, db)));
if (oldestCreatedTime == -1) {
return true; // Test can't make meaningful progress anymore
}
// oldestCreatedTime could seem too high if there's a large gap in the age
// of entries, so account for this by making oldestCreatedTime one more than
// the youngest entry that actually got deleted.
auto iter = std::lower_bound(createdTimes->begin(), createdTimes->end(), oldestCreatedTime);
if (iter != createdTimes->begin()) {
--iter;
oldestCreatedTime = *iter + 1;
}
auto maxActualAge = int64_t(now()) - oldestCreatedTime;
if (maxActualAge > minAgeSeconds * self->slop) {
CODE_PROBE(true, "Idempotency cleaner more to clean");
TraceEvent("AutomaticIdempotencyCleanerMoreToClean")
.detail("MaxActualAge", maxActualAge)
.detail("MinAgePolicy", minAgeSeconds);
successes = 0;
// Cleaning should happen eventually
} else if (maxActualAge < minAgeSeconds / self->slop) {
TraceEvent(SevError, "AutomaticIdempotencyCleanedTooMuch")
.detail("MaxActualAge", maxActualAge)
.detail("MinAgePolicy", minAgeSeconds);
self->ok = false;
ASSERT(false);
} else {
++successes;
TraceEvent("AutomaticIdempotencyCleanerSuccess")
.detail("MaxActualAge", maxActualAge)
.detail("MinAgePolicy", minAgeSeconds)
.detail("Successes", successes);
if (successes >= 10) {
break;
}
}
wait(delay(self->pollingInterval));
}
cleaner.cancel();
return false;
}
ACTOR static Future<std::vector<int64_t>> getCreatedTimes(AutomaticIdempotencyWorkload* self,
Reference<ReadYourWritesTransaction> tr) {
RangeResult result = wait(tr->getRange(prefixRange(self->keyPrefix), CLIENT_KNOBS->TOO_MANY));
ASSERT(!result.more);
std::vector<int64_t> createdTimes;
for (const auto& [k, v] : result) {
auto e = ObjectReader::fromStringRef<ValueType>(v, Unversioned());
createdTimes.emplace_back(e.createdTime);
}
std::sort(createdTimes.begin(), createdTimes.end());
return createdTimes;
}
// Check that min age is respected. Also test that we can tolerate concurrent cleaners.
ACTOR static Future<Void> testCleaner(AutomaticIdempotencyWorkload* self, Database db) {
state ActorCollection actors;
state int64_t minAgeSeconds;
state std::vector<int64_t> createdTimes;
// Initialize minAgeSeconds to match the current status
wait(store(minAgeSeconds, fmap([](int64_t t) { return int64_t(now()) - t; }, getOldestCreatedTime(self, db))) &&
store(createdTimes, runRYWTransaction(db, [self = self](Reference<ReadYourWritesTransaction> tr) {
return getCreatedTimes(self, tr);
})));
// Slowly and somewhat randomly allow the cleaner to do more cleaning. Observe that it cleans some, but not too
// much.
loop {
minAgeSeconds *= 1 / (self->slop * 2);
if (minAgeSeconds < self->minMinAgeSeconds) {
break;
}
choose {
when(bool done = wait(testCleanerOneIteration(self, db, &actors, minAgeSeconds, &createdTimes))) {
if (done) {
break;
}
}
when(wait(actors.getResult())) {
ASSERT(false);
}
}
}
return Void();
}
void getMetrics(std::vector<PerfMetric>& m) override {}
};

View File

@ -36,6 +36,7 @@ struct BulkSetupWorkload : TestWorkload {
double maxNumTenants;
double minNumTenants;
std::vector<TenantName> tenantNames;
double testDuration;
BulkSetupWorkload(WorkloadContext const& wcx) : TestWorkload(wcx) {
transactionsPerSecond = getOption(options, "transactionsPerSecond"_sr, 5000.0) / clientCount;
@ -45,6 +46,7 @@ struct BulkSetupWorkload : TestWorkload {
maxNumTenants = getOption(options, "maxNumTenants"_sr, 0);
minNumTenants = getOption(options, "minNumTenants"_sr, 0);
ASSERT(minNumTenants <= maxNumTenants);
testDuration = getOption(options, "testDuration"_sr, 10.0);
}
void getMetrics(std::vector<PerfMetric>& m) override {}
@ -73,32 +75,28 @@ struct BulkSetupWorkload : TestWorkload {
}
wait(waitForAll(tenantFutures));
}
wait(bulkSetup(cx,
workload,
workload->nodeCount,
Promise<double>(),
false,
0.0,
1e12,
std::vector<uint64_t>(),
Promise<std::vector<std::pair<uint64_t, double>>>(),
0,
0.1,
0,
0,
workload->tenantNames));
return Void();
}
Future<Void> start(Database const& cx) override {
if (clientId == 0) {
return bulkSetup(cx,
this,
nodeCount,
Promise<double>(),
false,
0.0,
1e12,
std::vector<uint64_t>(),
Promise<std::vector<std::pair<uint64_t, double>>>(),
0,
0.1,
0,
0,
tenantNames);
}
return Void();
}
Future<Void> start(Database const& cx) override { return Void(); }
Future<Void> setup(Database const& cx) override {
if (clientId == 0) {
return _setup(this, cx);
return timeout(_setup(this, cx), testDuration, Void());
}
return Void();
}

View File

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

View File

@ -95,6 +95,7 @@ void FlowKnobs::initialize(Randomize randomize, IsSimulated isSimulated) {
init( STATSD_UDP_EMISSION_PORT, 8125 );
init( OTEL_UDP_EMISSION_ADDR, "127.0.0.1");
init( OTEL_UDP_EMISSION_PORT, 8903 );
init( METRICS_EMIT_DDSKETCH, false ); // Determines if DDSketch buckets will get emitted
//connectionMonitor
init( CONNECTION_MONITOR_LOOP_TIME, isSimulated ? 0.75 : 1.0 ); if( randomize && BUGGIFY ) CONNECTION_MONITOR_LOOP_TIME = 6.0;

View File

@ -151,6 +151,7 @@ SystemStatistics customSystemMonitor(std::string const& eventName, StatisticsSta
.detail("DCID", machineState.dcId)
.detail("ZoneID", machineState.zoneId)
.detail("MachineID", machineState.machineId)
.detail("Version", machineState.fdbVersion)
.detail("AIOSubmitCount", netData.countAIOSubmit - statState->networkState.countAIOSubmit)
.detail("AIOCollectCount", netData.countAIOCollect - statState->networkState.countAIOCollect)
.detail("AIOSubmitLag",

View File

@ -222,7 +222,7 @@ LoadedTLSConfig TLSConfig::loadSync() const {
try {
loaded.tlsCertBytes = readFileBytes(certPath, FLOW_KNOBS->CERT_FILE_MAX_SIZE);
} catch (Error& e) {
fprintf(stderr, "Error reading TLS Certificate [%s]: %s\n", certPath.c_str(), e.what());
fprintf(stderr, "Warning: Error reading TLS Certificate [%s]: %s\n", certPath.c_str(), e.what());
throw;
}
} else {
@ -234,7 +234,7 @@ LoadedTLSConfig TLSConfig::loadSync() const {
try {
loaded.tlsKeyBytes = readFileBytes(keyPath, FLOW_KNOBS->CERT_FILE_MAX_SIZE);
} catch (Error& e) {
fprintf(stderr, "Error reading TLS Key [%s]: %s\n", keyPath.c_str(), e.what());
fprintf(stderr, "Warning: Error reading TLS Key [%s]: %s\n", keyPath.c_str(), e.what());
throw;
}
} else {
@ -246,7 +246,7 @@ LoadedTLSConfig TLSConfig::loadSync() const {
try {
loaded.tlsCABytes = readFileBytes(CAPath, FLOW_KNOBS->CERT_FILE_MAX_SIZE);
} catch (Error& e) {
fprintf(stderr, "Error reading TLS CA [%s]: %s\n", CAPath.c_str(), e.what());
fprintf(stderr, "Warning: Error reading TLS CA [%s]: %s\n", CAPath.c_str(), e.what());
throw;
}
} else {
@ -315,13 +315,13 @@ ACTOR Future<LoadedTLSConfig> TLSConfig::loadAsync(const TLSConfig* self) {
wait(waitForAll(reads));
} catch (Error& e) {
if (certIdx != -1 && reads[certIdx].isError()) {
fprintf(stderr, "Failure reading TLS Certificate [%s]: %s\n", certPath.c_str(), e.what());
fprintf(stderr, "Warning: Error reading TLS Certificate [%s]: %s\n", certPath.c_str(), e.what());
} else if (keyIdx != -1 && reads[keyIdx].isError()) {
fprintf(stderr, "Failure reading TLS Key [%s]: %s\n", keyPath.c_str(), e.what());
fprintf(stderr, "Warning: Error reading TLS Key [%s]: %s\n", keyPath.c_str(), e.what());
} else if (caIdx != -1 && reads[caIdx].isError()) {
fprintf(stderr, "Failure reading TLS Key [%s]: %s\n", CAPath.c_str(), e.what());
fprintf(stderr, "Warning: Error reading TLS Key [%s]: %s\n", CAPath.c_str(), e.what());
} else {
fprintf(stderr, "Failure reading TLS needed file: %s\n", e.what());
fprintf(stderr, "Warning: Error reading TLS needed file: %s\n", e.what());
}
throw;

View File

@ -152,6 +152,7 @@ public:
std::string OTEL_UDP_EMISSION_ADDR;
int STATSD_UDP_EMISSION_PORT;
int OTEL_UDP_EMISSION_PORT;
bool METRICS_EMIT_DDSKETCH;
// run loop profiling
double RUN_LOOP_PROFILING_INTERVAL;

View File

@ -66,7 +66,7 @@ enum DataPointFlags { FLAG_NONE = 0, FLAG_NO_RECORDED_VALUE };
class NumberDataPoint {
public:
double startTime; // 9 bytes in msgpack
double startTime = -1; // 9 bytes in msgpack
double recordTime; // 9 bytes in msgpack
std::vector<Attribute> attributes; // Variable size: assume to be 23 bytes
std::variant<int64_t, double> val; // 9 bytes in msgpack

View File

@ -31,6 +31,7 @@ struct SystemMonitorMachineState {
Optional<Standalone<StringRef>> zoneId;
Optional<Standalone<StringRef>> machineId;
Optional<IPAddress> ip;
Optional<std::string> fdbVersion;
double monitorStartTime;
@ -40,8 +41,10 @@ struct SystemMonitorMachineState {
Optional<Standalone<StringRef>> const& dcId,
Optional<Standalone<StringRef>> const& zoneId,
Optional<Standalone<StringRef>> const& machineId,
IPAddress const& ip)
: folder(folder), dcId(dcId), zoneId(zoneId), machineId(machineId), ip(ip), monitorStartTime(0) {}
IPAddress const& ip,
std::string const& fdbVersion)
: folder(folder), dcId(dcId), zoneId(zoneId), machineId(machineId), ip(ip), monitorStartTime(0),
fdbVersion(fdbVersion) {}
};
void initializeSystemMonitorMachineState(SystemMonitorMachineState machineState);

View File

@ -1,3 +1,6 @@
[configuration]
encryptModes = ['domain_aware', 'cluster_aware']
[[knobs]]
enable_encryption = true
enable_blob_file_encryption = true

View File

@ -3,6 +3,8 @@ testTitle = 'AutomaticIdempotency'
[[test.workload]]
testName = 'AutomaticIdempotencyCorrectness'
minMinAgeSeconds = 15
pollingInterval = 5.0
[[test.workload]]
testName='Attrition'

View File

@ -1,5 +1,6 @@
[configuration]
testClass = "Encryption"
encryptModes = ['domain_aware', 'cluster_aware']
[[knobs]]
enable_encryption = true

View File

@ -19,6 +19,7 @@ simBackupAgents = 'BackupToFile'
maxNumTenants = 100
minNumTenants = 0
transactionsPerSecond = 2500.0
testDuration = 30.0
[[test.workload]]
testName = 'BackupAndRestoreCorrectness'

View File

@ -1,9 +1,7 @@
[configuration]
buggify = false
testClass = "Encryption"
[[knobs]]
enable_encryption = false
disableEncryption = true
[[test]]
testTitle = 'EncryptDecrypt'

View File

@ -6,6 +6,7 @@ injectTargetedSSRestart = true
injectSSDelay = true
# FIXME: re-enable rocks at some point
storageEngineExcludeTypes = [4, 5]
encryptModes = ['domain_aware', 'cluster_aware']
[[knobs]]
bg_metadata_source = "tenant"

View File

@ -4,6 +4,7 @@ allowDefaultTenant = false
tenantModes = ['optional', 'required']
# FIXME: re-enable rocks at some point
storageEngineExcludeTypes = [4, 5]
encryptModes = ['domain_aware', 'cluster_aware']
[[knobs]]
bg_metadata_source = "tenant"