Merge commit 'a93eb20bde57e334707cda64189dc969387825b4' into storageserver-pml

This commit is contained in:
Steve Atherton 2022-11-01 17:21:49 -07:00
commit 8c90c0d69a
86 changed files with 2003 additions and 649 deletions

View File

@ -1,5 +1,5 @@
[flake8]
ignore = E203, E266, E501, W503, F403, F401, E711
ignore = E203, E266, E501, W503, F403, F401, E711, C901
max-line-length = 79
max-complexity = 18
select = B,C,E,F,W,T4,B9

View File

@ -577,7 +577,7 @@ class ApiTest(Test):
key1, key2 = key2, key1
# TODO: randomize chunkSize but should not exceed 100M(shard limit)
chunkSize = 10000000 # 10M
chunkSize = 10000000 # 10M
instructions.push_args(key1, key2, chunkSize)
instructions.append(op)
self.add_strings(1)

View File

@ -114,7 +114,7 @@ class DirectoryTest(Test):
instructions.push_args(layer)
instructions.push_args(*test_util.with_length(path))
instructions.append('DIRECTORY_OPEN')
self.dir_list.append(self.root.add_child(path, DirectoryStateTreeNode(True, True, has_known_prefix=False, is_partition=(layer==b'partition'))))
self.dir_list.append(self.root.add_child(path, DirectoryStateTreeNode(True, True, has_known_prefix=False, is_partition=(layer == b'partition'))))
# print('%d. Selected %s, dir=%s, dir_id=%s, has_known_prefix=%s, dir_list_len=%d' \
# % (len(instructions), 'DIRECTORY_OPEN', repr(self.dir_index), self.dir_list[-1].dir_id, False, len(self.dir_list)-1))
@ -163,8 +163,8 @@ class DirectoryTest(Test):
elif root_op == 'DIRECTORY_CREATE_LAYER':
indices = []
prefixes = [generate_prefix(require_unique=args.concurrency==1, is_partition=True) for i in range(2)]
prefixes = [generate_prefix(require_unique=args.concurrency == 1, is_partition=True) for i in range(2)]
for i in range(2):
instructions.push_args(prefixes[i])
instructions.push_args(*test_util.with_length(generate_path()))
@ -184,9 +184,9 @@ class DirectoryTest(Test):
test_util.blocking_commit(instructions)
path = generate_path()
# Partitions that use the high-contention allocator can result in non-determinism if they fail to commit,
# Partitions that use the high-contention allocator can result in non-determinism if they fail to commit,
# so we disallow them in comparison tests
op_args = test_util.with_length(path) + (self.generate_layer(allow_partition=args.concurrency>1),)
op_args = test_util.with_length(path) + (self.generate_layer(allow_partition=args.concurrency > 1),)
directory_util.push_instruction_and_record_prefix(instructions, op, op_args, path, len(self.dir_list), self.random, self.prefix_log)
if not op.endswith('_DATABASE') and args.concurrency == 1:
@ -196,14 +196,14 @@ class DirectoryTest(Test):
if child_entry is None:
child_entry = DirectoryStateTreeNode(True, True)
child_entry.state.has_known_prefix = False
child_entry.state.has_known_prefix = False
self.dir_list.append(dir_entry.add_child(path, child_entry))
elif root_op == 'DIRECTORY_CREATE':
layer = self.generate_layer()
is_partition = layer == b'partition'
prefix = generate_prefix(require_unique=is_partition and args.concurrency==1, is_partition=is_partition, min_length=0)
prefix = generate_prefix(require_unique=is_partition and args.concurrency == 1, is_partition=is_partition, min_length=0)
# Because allocated prefixes are non-deterministic, we cannot have overlapping
# transactions that allocate/remove these prefixes in a comparison test
@ -409,7 +409,7 @@ def generate_prefix(require_unique=False, is_partition=False, min_length=1):
if require_unique:
min_length = max(min_length, 16)
length = random.randint(min_length, min_length+5)
length = random.randint(min_length, min_length + 5)
if length == 0:
return b''
@ -419,6 +419,6 @@ def generate_prefix(require_unique=False, is_partition=False, min_length=1):
else:
return bytes([random.randrange(ord('\x02'), ord('\x14')) for i in range(0, length)])
else:
prefix = fixed_prefix
prefix = fixed_prefix
generated = prefix[0:random.randrange(min_length, len(prefix))]
return generated

View File

@ -1,5 +1,6 @@
import sys
class TreeNodeState:
def __init__(self, node, dir_id, is_directory, is_subspace, has_known_prefix, root, is_partition):
self.dir_id = dir_id
@ -9,10 +10,11 @@ class TreeNodeState:
self.root = root
self.is_partition = is_partition
self.parents = { node }
self.parents = {node}
self.children = {}
self.deleted = False
# Represents an element of the directory hierarchy. As a result of various operations (e.g. moves) that
# may or may not have succeeded, a node can represent multiple possible states.
class DirectoryStateTreeNode:
@ -25,7 +27,7 @@ class DirectoryStateTreeNode:
default_directory = None
# Used for debugging
dir_id = 0
dir_id = 0
@classmethod
def reset(cls):
@ -62,7 +64,7 @@ class DirectoryStateTreeNode:
if default is not None:
default_child = default.state.children.get(subpath[0])
self_child = self.state.children.get(subpath[0])
self_child = self.state.children.get(subpath[0])
if self_child is None:
if default_child is None:
@ -143,13 +145,15 @@ class DirectoryStateTreeNode:
child = self.get_descendent(path)
if child:
child._delete_impl()
def validate_dir(dir, root):
if dir.state.is_directory:
assert dir.state.root == root
else:
assert dir.state.root == dir
def run_test():
all_entries = []
@ -249,11 +253,11 @@ def run_test():
# Test moving an entry
assert not entry.state.has_known_prefix
assert not entry.state.is_subspace
assert list(entry.state.children.keys()) == ['1']
assert list(entry.state.children.keys()) == ['1']
for e in all_entries:
validate_dir(e, root)
if __name__ == '__main__':
sys.exit(run_test())

View File

@ -18,7 +18,6 @@
# limitations under the License.
#
import random
import struct
import fdb
@ -35,6 +34,7 @@ DEFAULT_DIRECTORY_INDEX = 4
DEFAULT_DIRECTORY_PREFIX = b'default'
DIRECTORY_ERROR_STRING = b'DIRECTORY_ERROR'
def setup_directories(instructions, default_path, random):
# Clients start with the default directory layer in the directory list
DirectoryStateTreeNode.reset()

View File

@ -107,7 +107,7 @@ class RandomGenerator(object):
user_version = random.randint(0, 0xffff)
tup.append(fdb.tuple.Versionstamp(tr_version, user_version))
else:
assert false
assert False
return tuple(tup)

View File

@ -31,6 +31,7 @@ from bindingtester.tests import test_util
fdb.api_version(FDB_API_VERSION)
class TupleTest(Test):
def __init__(self, subspace):
super(TupleTest, self).__init__(subspace)
@ -44,14 +45,14 @@ class TupleTest(Test):
def generate(self, args, thread_number):
instructions = InstructionSet()
min_value = -2**self.max_int_bits+1
max_value = 2**self.max_int_bits-1
min_value = -2**self.max_int_bits + 1
max_value = 2**self.max_int_bits - 1
instructions.append('NEW_TRANSACTION')
# Test integer encoding
mutations = 0
for i in range(0, self.max_int_bits+1):
for i in range(0, self.max_int_bits + 1):
for sign in [-1, 1]:
sign_str = '' if sign == 1 else '-'
for offset in range(-10, 11):

View File

@ -21,7 +21,7 @@
#include "fdbclient/FDBTypes.h"
#include "flow/ProtocolVersion.h"
#include <cstdint>
#define FDB_API_VERSION 720
#define FDB_API_VERSION 730
#define FDB_INCLUDE_LEGACY_TYPES
#include "fdbclient/MultiVersionTransaction.h"
@ -905,6 +905,10 @@ extern "C" DLLEXPORT fdb_error_t fdb_transaction_get_committed_version(FDBTransa
CATCH_AND_RETURN(*out_version = TXN(tr)->getCommittedVersion(););
}
extern "C" DLLEXPORT FDBFuture* fdb_transaction_get_total_cost(FDBTransaction* tr) {
return (FDBFuture*)TXN(tr)->getTotalCost().extractPtr();
}
extern "C" DLLEXPORT FDBFuture* fdb_transaction_get_approximate_size(FDBTransaction* tr) {
return (FDBFuture*)TXN(tr)->getApproximateSize().extractPtr();
}

View File

@ -27,10 +27,10 @@
#endif
#if !defined(FDB_API_VERSION)
#error You must #define FDB_API_VERSION prior to including fdb_c.h (current version is 720)
#error You must #define FDB_API_VERSION prior to including fdb_c.h (current version is 730)
#elif FDB_API_VERSION < 13
#error API version no longer supported (upgrade to 13)
#elif FDB_API_VERSION > 720
#elif FDB_API_VERSION > 730
#error Requested API version requires a newer version of this header
#endif
@ -514,12 +514,14 @@ DLLEXPORT WARN_UNUSED_RESULT fdb_error_t fdb_transaction_get_committed_version(F
int64_t* out_version);
/*
* This function intentionally returns an FDBFuture instead of an integer
* directly, so that calling this API can see the effect of previous
* These functions intentionally return an FDBFuture instead of an integer
* directly, so that calling the API can see the effect of previous
* mutations on the transaction. Specifically, mutations are applied
* asynchronously by the main thread. In order to see them, this call has to
* be serviced by the main thread too.
*/
DLLEXPORT WARN_UNUSED_RESULT FDBFuture* fdb_transaction_get_total_cost(FDBTransaction* tr);
DLLEXPORT WARN_UNUSED_RESULT FDBFuture* fdb_transaction_get_approximate_size(FDBTransaction* tr);
DLLEXPORT WARN_UNUSED_RESULT FDBFuture* fdb_transaction_get_versionstamp(FDBTransaction* tr);

View File

@ -20,11 +20,14 @@
#include "TesterApiWorkload.h"
#include "TesterBlobGranuleUtil.h"
#include "TesterUtil.h"
#include <unordered_set>
#include <memory>
#include <fmt/format.h>
namespace FdbApiTester {
#define BG_API_DEBUG_VERBOSE false
class ApiBlobGranuleCorrectnessWorkload : public ApiWorkload {
public:
ApiBlobGranuleCorrectnessWorkload(const WorkloadConfig& config) : ApiWorkload(config) {
@ -35,7 +38,7 @@ public:
}
private:
// FIXME: use other new blob granule apis!
// FIXME: add tenant support for DB operations
enum OpType {
OP_INSERT,
OP_CLEAR,
@ -51,7 +54,27 @@ private:
// Allow reads at the start to get blob_granule_transaction_too_old if BG data isn't initialized yet
// FIXME: should still guarantee a read succeeds eventually somehow
bool seenReadSuccess = false;
// FIXME: this needs to be per tenant if tenant ids are set
std::unordered_set<std::optional<int>> tenantsWithReadSuccess;
inline void setReadSuccess(std::optional<int> tenantId) { tenantsWithReadSuccess.insert(tenantId); }
inline bool seenReadSuccess(std::optional<int> tenantId) { return tenantsWithReadSuccess.count(tenantId); }
std::string tenantDebugString(std::optional<int> tenantId) {
return tenantId.has_value() ? fmt::format(" (tenant {0})", tenantId.value()) : "";
}
void debugOp(std::string opName, fdb::Key begin, fdb::Key end, std::optional<int> tenantId, std::string message) {
if (BG_API_DEBUG_VERBOSE) {
info(fmt::format("{0}: [{1} - {2}){3}: {4}",
opName,
fdb::toCharsRef(begin),
fdb::toCharsRef(end),
tenantDebugString(tenantId),
message));
}
}
void randomReadOp(TTaskFct cont, std::optional<int> tenantId) {
fdb::Key begin = randomKeyName();
@ -63,8 +86,10 @@ private:
auto results = std::make_shared<std::vector<fdb::KeyValue>>();
auto tooOld = std::make_shared<bool>(false);
debugOp("Read", begin, end, tenantId, "starting");
execTransaction(
[this, begin, end, results, tooOld](auto ctx) {
[this, begin, end, tenantId, results, tooOld](auto ctx) {
ctx->tx().setOption(FDB_TR_OPTION_READ_YOUR_WRITES_DISABLE);
TesterGranuleContext testerContext(ctx->getBGBasePath());
fdb::native::FDBReadBlobGranuleContext granuleContext = createGranuleContext(&testerContext);
@ -74,8 +99,13 @@ private:
auto out = fdb::Result::KeyValueRefArray{};
fdb::Error err = res.getKeyValueArrayNothrow(out);
if (err.code() == error_code_blob_granule_transaction_too_old) {
info("BlobGranuleCorrectness::randomReadOp bg too old\n");
ASSERT(!seenReadSuccess);
bool previousSuccess = seenReadSuccess(tenantId);
if (previousSuccess) {
error("Read bg too old after read success!\n");
} else {
info("Read bg too old\n");
}
ASSERT(!previousSuccess);
*tooOld = true;
ctx->done();
} else if (err.code() != error_code_success) {
@ -85,10 +115,13 @@ private:
auto& [resVector, out_more] = resCopy;
ASSERT(!out_more);
results.get()->assign(resVector.begin(), resVector.end());
if (!seenReadSuccess) {
info("BlobGranuleCorrectness::randomReadOp first success\n");
bool previousSuccess = seenReadSuccess(tenantId);
if (!previousSuccess) {
info(fmt::format("Read{0}: first success\n", tenantDebugString(tenantId)));
setReadSuccess(tenantId);
} else {
debugOp("Read", begin, end, tenantId, "complete");
}
seenReadSuccess = true;
ctx->done();
}
},
@ -97,7 +130,7 @@ private:
std::vector<fdb::KeyValue> expected =
stores[tenantId].getRange(begin, end, stores[tenantId].size(), false);
if (results->size() != expected.size()) {
error(fmt::format("randomReadOp result size mismatch. expected: {} actual: {}",
error(fmt::format("randomReadOp result size mismatch. expected: {0} actual: {1}",
expected.size(),
results->size()));
}
@ -105,7 +138,7 @@ private:
for (int i = 0; i < results->size(); i++) {
if ((*results)[i].key != expected[i].key) {
error(fmt::format("randomReadOp key mismatch at {}/{}. expected: {} actual: {}",
error(fmt::format("randomReadOp key mismatch at {0}/{1}. expected: {2} actual: {3}",
i,
results->size(),
fdb::toCharsRef(expected[i].key),
@ -138,6 +171,8 @@ private:
}
auto results = std::make_shared<std::vector<fdb::KeyRange>>();
debugOp("GetGranules", begin, end, tenantId, "starting");
execTransaction(
[begin, end, results](auto ctx) {
fdb::Future f = ctx->tx().getBlobGranuleRanges(begin, end, 1000).eraseType();
@ -149,15 +184,17 @@ private:
},
true);
},
[this, begin, end, results, cont]() {
this->validateRanges(results, begin, end, seenReadSuccess);
[this, begin, end, tenantId, results, cont]() {
debugOp(
"GetGranules", begin, end, tenantId, fmt::format("complete with {0} granules", results->size()));
this->validateRanges(results, begin, end, seenReadSuccess(tenantId));
schedule(cont);
},
getTenant(tenantId));
}
void randomSummarizeOp(TTaskFct cont, std::optional<int> tenantId) {
if (!seenReadSuccess) {
if (!seenReadSuccess(tenantId)) {
// tester can't handle this throwing bg_txn_too_old, so just don't call it unless we have already seen a
// read success
schedule(cont);
@ -169,6 +206,9 @@ private:
std::swap(begin, end);
}
auto results = std::make_shared<std::vector<fdb::GranuleSummary>>();
debugOp("Summarize", begin, end, tenantId, "starting");
execTransaction(
[begin, end, results](auto ctx) {
fdb::Future f = ctx->tx().summarizeBlobGranules(begin, end, -2 /*latest version*/, 1000).eraseType();
@ -180,10 +220,11 @@ private:
},
true);
},
[this, begin, end, results, cont]() {
ASSERT(results->size() > 0);
ASSERT(results->front().keyRange.beginKey <= begin);
ASSERT(results->back().keyRange.endKey >= end);
[this, begin, end, tenantId, results, cont]() {
debugOp("Summarize", begin, end, tenantId, fmt::format("complete with {0} granules", results->size()));
// use validateRanges to share validation
auto ranges = std::make_shared<std::vector<fdb::KeyRange>>();
for (int i = 0; i < results->size(); i++) {
// TODO: could do validation of subsequent calls and ensure snapshot version never decreases
@ -191,12 +232,11 @@ private:
ASSERT((*results)[i].snapshotVersion <= (*results)[i].deltaVersion);
ASSERT((*results)[i].snapshotSize > 0);
ASSERT((*results)[i].deltaSize >= 0);
ranges->push_back((*results)[i].keyRange);
}
for (int i = 1; i < results->size(); i++) {
// ranges contain entire requested key range
ASSERT((*results)[i].keyRange.beginKey == (*results)[i - 1].keyRange.endKey);
}
this->validateRanges(ranges, begin, end, true);
schedule(cont);
},
@ -208,18 +248,29 @@ private:
fdb::Key end,
bool shouldBeRanges) {
if (shouldBeRanges) {
if (results->size() == 0) {
error(fmt::format(
"ValidateRanges: [{0} - {1}): No ranges returned!", fdb::toCharsRef(begin), fdb::toCharsRef(end)));
}
ASSERT(results->size() > 0);
if (results->front().beginKey > begin || results->back().endKey < end) {
error(fmt::format("ValidateRanges: [{0} - {1}): Incomplete range(s) returned [{2} - {3})!",
fdb::toCharsRef(begin),
fdb::toCharsRef(end),
fdb::toCharsRef(results->front().beginKey),
fdb::toCharsRef(results->back().endKey)));
}
ASSERT(results->front().beginKey <= begin);
ASSERT(results->back().endKey >= end);
}
for (int i = 0; i < results->size(); i++) {
// no empty or inverted ranges
if ((*results)[i].beginKey >= (*results)[i].endKey) {
error(fmt::format("Empty/inverted range [{0} - {1}) for getBlobGranuleRanges({2} - {3})",
fdb::toCharsRef((*results)[i].beginKey),
fdb::toCharsRef((*results)[i].endKey),
error(fmt::format("ValidateRanges: [{0} - {1}): Empty/inverted range [{2} - {3})",
fdb::toCharsRef(begin),
fdb::toCharsRef(end)));
fdb::toCharsRef(end),
fdb::toCharsRef((*results)[i].beginKey),
fdb::toCharsRef((*results)[i].endKey)));
}
ASSERT((*results)[i].beginKey < (*results)[i].endKey);
}
@ -227,16 +278,17 @@ private:
for (int i = 1; i < results->size(); i++) {
// ranges contain entire requested key range
if ((*results)[i].beginKey != (*results)[i].endKey) {
error(fmt::format("Non-contiguous range [{0} - {1}) for getBlobGranuleRanges({2} - {3})",
fdb::toCharsRef((*results)[i].beginKey),
fdb::toCharsRef((*results)[i].endKey),
error(fmt::format("ValidateRanges: [{0} - {1}): Non-covereed range [{2} - {3})",
fdb::toCharsRef(begin),
fdb::toCharsRef(end)));
fdb::toCharsRef(end),
fdb::toCharsRef((*results)[i - 1].endKey),
fdb::toCharsRef((*results)[i].endKey)));
}
ASSERT((*results)[i].beginKey == (*results)[i - 1].endKey);
}
}
// TODO: tenant support
void randomGetBlobRangesOp(TTaskFct cont) {
fdb::Key begin = randomKeyName();
fdb::Key end = randomKeyName();
@ -244,6 +296,10 @@ private:
if (begin > end) {
std::swap(begin, end);
}
std::optional<int> tenantId = {};
debugOp("GetBlobRanges", begin, end, tenantId, "starting");
execOperation(
[begin, end, results](auto ctx) {
fdb::Future f = ctx->db().listBlobbifiedRanges(begin, end, 1000).eraseType();
@ -252,22 +308,27 @@ private:
ctx->done();
});
},
[this, begin, end, results, cont]() {
this->validateRanges(results, begin, end, seenReadSuccess);
[this, begin, end, tenantId, results, cont]() {
debugOp(
"GetBlobRanges", begin, end, tenantId, fmt::format("complete with {0} ranges", results->size()));
this->validateRanges(results, begin, end, seenReadSuccess(tenantId));
schedule(cont);
},
/* failOnError = */ false);
}
// TODO: tenant support
void randomVerifyOp(TTaskFct cont) {
fdb::Key begin = randomKeyName();
fdb::Key end = randomKeyName();
std::optional<int> tenantId;
if (begin > end) {
std::swap(begin, end);
}
auto verifyVersion = std::make_shared<int64_t>(false);
// info("Verify op starting");
debugOp("Verify", begin, end, tenantId, "starting");
execOperation(
[begin, end, verifyVersion](auto ctx) {
@ -277,16 +338,15 @@ private:
ctx->done();
});
},
[this, begin, end, verifyVersion, cont]() {
[this, begin, end, tenantId, verifyVersion, cont]() {
debugOp("Verify", begin, end, tenantId, fmt::format("Complete @ {0}", *verifyVersion));
bool previousSuccess = seenReadSuccess(tenantId);
if (*verifyVersion == -1) {
ASSERT(!seenReadSuccess);
} else {
if (!seenReadSuccess) {
info("BlobGranuleCorrectness::randomVerifyOp first success");
}
seenReadSuccess = true;
ASSERT(!previousSuccess);
} else if (!previousSuccess) {
info(fmt::format("Verify{0}: first success\n", tenantDebugString(tenantId)));
setReadSuccess(tenantId);
}
// info(fmt::format("verify op done @ {}", *verifyVersion));
schedule(cont);
},
/* failOnError = */ false);

View File

@ -27,7 +27,7 @@
#include <unordered_map>
#include <vector>
#define FDB_API_VERSION 720
#define FDB_API_VERSION 730
namespace FdbApiTester {

View File

@ -36,7 +36,7 @@ namespace FdbApiTester {
namespace {
#define API_VERSION_CLIENT_TMP_DIR 720
#define API_VERSION_CLIENT_TMP_DIR 730
enum TesterOptionId {
OPT_CONNFILE,

View File

@ -18,7 +18,7 @@
* limitations under the License.
*/
#define FDB_API_VERSION 720
#define FDB_API_VERSION 730
#include <foundationdb/fdb_c.h>
#include "unit/fdb_api.hpp"

View File

@ -23,7 +23,7 @@
#pragma once
#ifndef FDB_API_VERSION
#define FDB_API_VERSION 720
#define FDB_API_VERSION 730
#endif
#include <cassert>
@ -716,6 +716,12 @@ public:
throwError("Failed to create transaction: ", err);
return Transaction(tx_native);
}
TypedFuture<future_var::Bool> blobbifyRange(KeyRef begin, KeyRef end) {
if (!tenant)
throw std::runtime_error("blobbifyRange from null tenant");
return native::fdb_tenant_blobbify_range(tenant.get(), begin.data(), intSize(begin), end.data(), intSize(end));
}
};
class Database {

View File

@ -283,24 +283,60 @@ int populate(Database db,
int batch_size = args.tenant_batch_size;
int batches = (args.total_tenants + batch_size - 1) / batch_size;
for (int batch = 0; batch < batches; ++batch) {
while (1) {
for (int i = batch * batch_size; i < args.total_tenants && i < (batch + 1) * batch_size; ++i) {
std::string tenant_str = "tenant" + std::to_string(i);
Tenant::createTenant(systemTx, toBytesRef(tenant_str));
}
auto future_commit = systemTx.commit();
const auto rc = waitAndHandleError(systemTx, future_commit, "CREATE_TENANT");
if (rc == FutureRC::OK) {
// Keep going with reset transaction if commit was successful
systemTx.reset();
break;
} else if (rc == FutureRC::RETRY) {
// We want to retry this batch. Transaction is already reset
} else {
// Abort
return -1;
}
}
Tenant tenants[batch_size];
fdb::TypedFuture<fdb::future_var::Bool> blobbifyResults[batch_size];
// blobbify tenant ranges explicitly
// FIXME: skip if database not configured for blob granules?
for (int i = batch * batch_size; i < args.total_tenants && i < (batch + 1) * batch_size; ++i) {
std::string tenant_name = "tenant" + std::to_string(i);
Tenant::createTenant(systemTx, toBytesRef(tenant_name));
std::string tenant_str = "tenant" + std::to_string(i);
BytesRef tenant_name = toBytesRef(tenant_str);
tenants[i] = db.openTenant(tenant_name);
std::string rangeEnd = "\xff";
blobbifyResults[i - (batch * batch_size)] =
tenants[i].blobbifyRange(BytesRef(), toBytesRef(rangeEnd));
}
auto future_commit = systemTx.commit();
const auto rc = waitAndHandleError(systemTx, future_commit, "CREATE_TENANT");
if (rc == FutureRC::OK) {
// Keep going with reset transaction if commit was successful
systemTx.reset();
} else if (rc == FutureRC::RETRY) {
// We want to retry this batch, so decrement the number
// and go back through the loop to get the same value
// Transaction is already reset
--batch;
} else {
// Abort
return -1;
for (int i = batch * batch_size; i < args.total_tenants && i < (batch + 1) * batch_size; ++i) {
while (true) {
// not technically an operation that's part of systemTx, but it works
const auto rc =
waitAndHandleError(systemTx, blobbifyResults[i - (batch * batch_size)], "BLOBBIFY_TENANT");
if (rc == FutureRC::OK) {
if (!blobbifyResults[i - (batch * batch_size)].get()) {
fmt::print("Blobbifying tenant {0} failed!\n", i);
return -1;
}
break;
} else if (rc == FutureRC::RETRY) {
continue;
} else {
// Abort
return -1;
}
}
}
systemTx.reset();
}
} else {
std::string last_tenant_name = "tenant" + std::to_string(args.total_tenants - 1);

View File

@ -22,7 +22,7 @@
#define MAKO_HPP
#ifndef FDB_API_VERSION
#define FDB_API_VERSION 720
#define FDB_API_VERSION 730
#endif
#include <array>

View File

@ -29,7 +29,7 @@
#include <inttypes.h>
#ifndef FDB_API_VERSION
#define FDB_API_VERSION 720
#define FDB_API_VERSION 730
#endif
#include <foundationdb/fdb_c.h>

View File

@ -20,7 +20,7 @@
// Unit tests that test the timeouts for a disconnected cluster
#define FDB_API_VERSION 720
#define FDB_API_VERSION 730
#include <foundationdb/fdb_c.h>
#include <chrono>

View File

@ -231,6 +231,10 @@ Int64Future Transaction::get_approximate_size() {
return Int64Future(fdb_transaction_get_approximate_size(tr_));
}
Int64Future Transaction::get_total_cost() {
return Int64Future(fdb_transaction_get_total_cost(tr_));
}
KeyFuture Transaction::get_versionstamp() {
return KeyFuture(fdb_transaction_get_versionstamp(tr_));
}

View File

@ -39,7 +39,7 @@
#pragma once
#define FDB_API_VERSION 720
#define FDB_API_VERSION 730
#include <foundationdb/fdb_c.h>
#include <string>
@ -276,6 +276,9 @@ public:
// Returns a future which will be set to the approximate transaction size so far.
Int64Future get_approximate_size();
// Returns a future which will be set tot the transaction's total cost so far.
Int64Future get_total_cost();
// Returns a future which will be set to the versionstamp which was used by
// any versionstamp operations in the transaction.
KeyFuture get_versionstamp();

View File

@ -20,7 +20,7 @@
// Unit tests for API setup, network initialization functions from the FDB C API.
#define FDB_API_VERSION 720
#define FDB_API_VERSION 730
#include <foundationdb/fdb_c.h>
#include <iostream>
#include <thread>

View File

@ -21,7 +21,7 @@
// Unit tests for the FoundationDB C API.
#include "fdb_c_options.g.h"
#define FDB_API_VERSION 720
#define FDB_API_VERSION 730
#include <foundationdb/fdb_c.h>
#include <assert.h>
#include <string.h>
@ -1945,6 +1945,30 @@ TEST_CASE("fdb_transaction_get_committed_version") {
}
}
TEST_CASE("fdb_transaction_get_total_cost") {
fdb::Transaction tr(db);
while (1) {
fdb::ValueFuture f1 = tr.get("foo", /*snapshot*/ false);
fdb_error_t err = wait_future(f1);
if (err) {
fdb::EmptyFuture fOnError = tr.on_error(err);
fdb_check(wait_future(fOnError));
continue;
}
fdb::Int64Future f2 = tr.get_total_cost();
err = wait_future(f2);
if (err) {
fdb::EmptyFuture fOnError = tr.on_error(err);
fdb_check(wait_future(fOnError));
continue;
}
int64_t cost;
fdb_check(f2.get(&cost));
CHECK(cost > 0);
break;
}
}
TEST_CASE("fdb_transaction_get_approximate_size") {
fdb::Transaction tr(db);
while (1) {

View File

@ -18,7 +18,7 @@
* limitations under the License.
*/
#define FDB_API_VERSION 720
#define FDB_API_VERSION 730
#include "foundationdb/fdb_c.h"
#undef DLLEXPORT
#include "workloads.h"

View File

@ -303,7 +303,6 @@ class TestRun:
self.stats: str | None = stats
self.expected_unseed: int | None = expected_unseed
self.use_valgrind: bool = config.use_valgrind
self.long_running: bool = config.long_running
self.old_binary_path: Path = config.old_binaries_path
self.buggify_enabled: bool = buggify_enabled
self.fault_injection_enabled: bool = True
@ -315,7 +314,7 @@ class TestRun:
# state for the run
self.retryable_error: bool = False
self.summary: Summary = Summary(binary, uid=self.uid, stats=self.stats, expected_unseed=self.expected_unseed,
will_restart=will_restart)
will_restart=will_restart, long_running=config.long_running)
self.run_time: int = 0
self.success = self.run()
@ -367,6 +366,11 @@ class TestRun:
command += ['-b', 'on']
if config.crash_on_error:
command.append('--crash')
if config.long_running:
# disable simulation speedup
command += ['--knob-sim-speedup-after-seconds=36000']
# disable traceTooManyLines Error MAX_TRACE_LINES
command += ['--knob-max-trace-lines=1000000000']
self.temp_path.mkdir(parents=True, exist_ok=True)
@ -376,7 +380,8 @@ class TestRun:
process = subprocess.Popen(command, stdout=subprocess.DEVNULL, stderr=subprocess.PIPE, cwd=self.temp_path,
text=True, env=env)
did_kill = False
timeout = 20 * config.kill_seconds if self.use_valgrind or self.long_running else config.kill_seconds
# No timeout for long running tests
timeout = 20 * config.kill_seconds if self.use_valgrind else (None if config.long_running else config.kill_seconds)
err_out: str
try:
_, err_out = process.communicate(timeout=timeout)

View File

@ -291,11 +291,12 @@ class Summary:
def __init__(self, binary: Path, runtime: float = 0, max_rss: int | None = None,
was_killed: bool = False, uid: uuid.UUID | None = None, expected_unseed: int | None = None,
exit_code: int = 0, valgrind_out_file: Path | None = None, stats: str | None = None,
error_out: str = None, will_restart: bool = False):
error_out: str = None, will_restart: bool = False, long_running: bool = False):
self.binary = binary
self.runtime: float = runtime
self.max_rss: int | None = max_rss
self.was_killed: bool = was_killed
self.long_running = long_running
self.expected_unseed: int | None = expected_unseed
self.exit_code: int = exit_code
self.out: SummaryTree = SummaryTree('Test')
@ -396,6 +397,10 @@ class Summary:
if self.was_killed:
child = SummaryTree('ExternalTimeout')
child.attributes['Severity'] = '40'
if self.long_running:
# debugging info for long-running tests
child.attributes['LongRunning'] = '1'
child.attributes['Runtime'] = str(self.runtime)
self.out.append(child)
self.error = True
if self.max_rss is not None:

View File

@ -55,6 +55,6 @@ if __name__ == '__main__':
summary.summarize_files(files)
summary.out.dump(sys.stdout)
else:
summary = Summary(Path('bin/fdbserver'), was_killed=True)
summary = Summary(Path('bin/fdbserver'), was_killed=True, long_running=config.long_running)
summary.summarize_files(files)
summary.out.dump(sys.stdout)

View File

@ -11,16 +11,16 @@ The global tag throttler bases throttling decisions on "quotas" provided by clie
The global tag throttler cannot throttle tags to a throughput below the reserved quota, and it cannot allow throughput to exceed the total quota.
### Cost
Internally, the units for these quotas are "page costs", computed as follows. The "page cost" of a read operation is computed as:
Internally, the units for these quotas are bytes. The cost of an operation is rounded up to the nearest page size. The cost of a read operation is computed as:
```
readCost = ceiling(bytesRead / CLIENT_KNOBS->READ_COST_BYTE_FACTOR);
readCost = ceiling(bytesRead / CLIENT_KNOBS->READ_COST_BYTE_FACTOR) * CLIENT_KNOBS->READ_COST_BYTE_FACTOR;
```
The "page cost" of a write operation is computed as:
The cost of a write operation is computed as:
```
writeCost = SERVER_KNOBS->GLOBAL_TAG_THROTTLING_RW_FUNGIBILITY_RATIO * ceiling(bytesWritten / CLIENT_KNOBS->WRITE_COST_BYTE_FACTOR);
writeCost = CLIENT_KNOBS->GLOBAL_TAG_THROTTLING_RW_FUNGIBILITY_RATIO * ceiling(bytesWritten / CLIENT_KNOBS->WRITE_COST_BYTE_FACTOR) * CLIENT_KNOBS->WRITE_COST_BYTE_FACTOR;
```
Here `bytesWritten` includes cleared bytes. The size of range clears is estimated at commit time.
@ -41,12 +41,6 @@ To set the quota through `fdbcli`, run:
fdbcli> quota set <tag> [reserved_throughput|total_throughput] <bytes_per_second>
```
Note that the quotas are specified in terms of bytes/second, and internally converted to page costs:
```
page_cost_quota = ceiling(byte_quota / CLIENT_KNOBS->READ_COST_BYTE_FACTOR)
```
To clear a both reserved and total throughput quotas for a tag, run:
```

View File

@ -43,9 +43,9 @@ Optional<LimitType> parseLimitType(StringRef token) {
}
}
Optional<double> parseLimitValue(StringRef token) {
Optional<int64_t> parseLimitValue(StringRef token) {
try {
return std::stod(token.toString());
return std::stol(token.toString());
} catch (...) {
return {};
}
@ -63,9 +63,9 @@ ACTOR Future<Void> getQuota(Reference<IDatabase> db, TransactionTag tag, LimitTy
} else {
auto const quota = ThrottleApi::TagQuotaValue::fromValue(v.get());
if (limitType == LimitType::TOTAL) {
fmt::print("{}\n", quota.totalQuota * CLIENT_KNOBS->READ_COST_BYTE_FACTOR);
fmt::print("{}\n", quota.totalQuota);
} else if (limitType == LimitType::RESERVED) {
fmt::print("{}\n", quota.reservedQuota * CLIENT_KNOBS->READ_COST_BYTE_FACTOR);
fmt::print("{}\n", quota.reservedQuota);
}
}
return Void();
@ -75,7 +75,7 @@ ACTOR Future<Void> getQuota(Reference<IDatabase> db, TransactionTag tag, LimitTy
}
}
ACTOR Future<Void> setQuota(Reference<IDatabase> db, TransactionTag tag, LimitType limitType, double value) {
ACTOR Future<Void> setQuota(Reference<IDatabase> db, TransactionTag tag, LimitType limitType, int64_t value) {
state Reference<ITransaction> tr = db->createTransaction();
loop {
tr->setOption(FDBTransactionOptions::ACCESS_SYSTEM_KEYS);
@ -89,9 +89,13 @@ ACTOR Future<Void> setQuota(Reference<IDatabase> db, TransactionTag tag, LimitTy
// Internally, costs are stored in terms of pages, but in the API,
// costs are specified in terms of bytes
if (limitType == LimitType::TOTAL) {
quota.totalQuota = (value - 1) / CLIENT_KNOBS->READ_COST_BYTE_FACTOR + 1;
// Round up to nearest page size
quota.totalQuota =
((value - 1) / CLIENT_KNOBS->READ_COST_BYTE_FACTOR + 1) * CLIENT_KNOBS->READ_COST_BYTE_FACTOR;
} else if (limitType == LimitType::RESERVED) {
quota.reservedQuota = (value - 1) / CLIENT_KNOBS->READ_COST_BYTE_FACTOR + 1;
// Round up to nearest page size
quota.reservedQuota =
((value - 1) / CLIENT_KNOBS->READ_COST_BYTE_FACTOR + 1) * CLIENT_KNOBS->READ_COST_BYTE_FACTOR;
}
if (!quota.isValid()) {
throw invalid_throttle_quota_value();

View File

@ -232,10 +232,10 @@ void validateEncryptionHeaderDetails(const BlobGranuleFileEncryptionKeys& eKeys,
.detail("ExpectedHeaderSalt", header.cipherHeaderDetails.salt);
throw encrypt_header_metadata_mismatch();
}
// Validate encryption header 'cipherHeader' details sanity
if (!(header.cipherHeaderDetails.baseCipherId == eKeys.headerCipherKey->getBaseCipherId() &&
header.cipherHeaderDetails.encryptDomainId == eKeys.headerCipherKey->getDomainId() &&
header.cipherHeaderDetails.salt == eKeys.headerCipherKey->getSalt())) {
// Validate encryption header 'cipherText' details sanity
if (!(header.cipherTextDetails.baseCipherId == eKeys.textCipherKey->getBaseCipherId() &&
header.cipherTextDetails.encryptDomainId == eKeys.textCipherKey->getDomainId() &&
header.cipherTextDetails.salt == eKeys.textCipherKey->getSalt())) {
TraceEvent(SevError, "EncryptionHeader_CipherTextMismatch")
.detail("TextDomainId", eKeys.textCipherKey->getDomainId())
.detail("ExpectedTextDomainId", header.cipherTextDetails.encryptDomainId)

View File

@ -272,7 +272,7 @@ void ClientKnobs::initialize(Randomize randomize) {
init( TAG_THROTTLE_EXPIRATION_INTERVAL, 60.0 ); if( randomize && BUGGIFY ) TAG_THROTTLE_EXPIRATION_INTERVAL = 1.0;
init( WRITE_COST_BYTE_FACTOR, 16384 ); if( randomize && BUGGIFY ) WRITE_COST_BYTE_FACTOR = 4096;
init( READ_COST_BYTE_FACTOR, 16384 ); if( randomize && BUGGIFY ) READ_COST_BYTE_FACTOR = 4096;
init( PROXY_MAX_TAG_THROTTLE_DURATION, 5.0 ); if( randomize && BUGGIFY ) PROXY_MAX_TAG_THROTTLE_DURATION = 0.5;
init( GLOBAL_TAG_THROTTLING_RW_FUNGIBILITY_RATIO, 5.0 );
// busyness reporting
init( BUSYNESS_SPIKE_START_THRESHOLD, 0.100 );

View File

@ -414,6 +414,20 @@ Version DLTransaction::getCommittedVersion() {
return version;
}
ThreadFuture<int64_t> DLTransaction::getTotalCost() {
if (!api->transactionGetTotalCost) {
return unsupported_operation();
}
FdbCApi::FDBFuture* f = api->transactionGetTotalCost(tr);
return toThreadFuture<int64_t>(api, f, [](FdbCApi::FDBFuture* f, FdbCApi* api) {
int64_t size = 0;
FdbCApi::fdb_error_t error = api->futureGetInt64(f, &size);
ASSERT(!error);
return size;
});
}
ThreadFuture<int64_t> DLTransaction::getApproximateSize() {
if (!api->transactionGetApproximateSize) {
return unsupported_operation();
@ -950,6 +964,11 @@ void DLApi::init() {
fdbCPath,
"fdb_transaction_get_committed_version",
headerVersion >= 0);
loadClientFunction(&api->transactionGetTotalCost,
lib,
fdbCPath,
"fdb_transaction_get_total_cost",
headerVersion >= ApiVersion::withGetTotalCost().version());
loadClientFunction(&api->transactionGetApproximateSize,
lib,
fdbCPath,
@ -1486,6 +1505,12 @@ ThreadFuture<SpanContext> MultiVersionTransaction::getSpanContext() {
return SpanContext();
}
ThreadFuture<int64_t> MultiVersionTransaction::getTotalCost() {
auto tr = getTransaction();
auto f = tr.transaction ? tr.transaction->getTotalCost() : makeTimeout<int64_t>();
return abortableFuture(f, tr.onChange);
}
ThreadFuture<int64_t> MultiVersionTransaction::getApproximateSize() {
auto tr = getTransaction();
auto f = tr.transaction ? tr.transaction->getApproximateSize() : makeTimeout<int64_t>();

View File

@ -3456,6 +3456,8 @@ ACTOR Future<Optional<Value>> getValue(Reference<TransactionState> trState,
}
trState->cx->getValueCompleted->latency = timer_int() - startTime;
trState->cx->getValueCompleted->log();
trState->totalCost +=
getReadOperationCost(key.size() + (reply.value.present() ? reply.value.get().size() : 0));
if (getValueID.present()) {
g_traceBatch.addEvent("GetValueDebug",
@ -4285,6 +4287,7 @@ void getRangeFinished(Reference<TransactionState> trState,
RangeResultFamily result) {
int64_t bytes = getRangeResultFamilyBytes(result);
trState->totalCost += getReadOperationCost(bytes);
trState->cx->transactionBytesRead += bytes;
trState->cx->transactionKeysRead += result.size();
@ -5767,6 +5770,7 @@ void Transaction::set(const KeyRef& key, const ValueRef& value, AddConflictRange
auto r = singleKeyRange(key, req.arena);
auto v = ValueRef(req.arena, value);
t.mutations.emplace_back(req.arena, MutationRef::SetValue, r.begin, v);
trState->totalCost += getWriteOperationCost(key.expectedSize() + value.expectedSize());
if (addConflictRange) {
t.write_conflict_ranges.push_back(req.arena, r);
@ -5796,6 +5800,7 @@ void Transaction::atomicOp(const KeyRef& key,
auto v = ValueRef(req.arena, operand);
t.mutations.emplace_back(req.arena, operationType, r.begin, v);
trState->totalCost += getWriteOperationCost(key.expectedSize());
if (addConflictRange && operationType != MutationRef::SetVersionstampedKey)
t.write_conflict_ranges.push_back(req.arena, r);
@ -5827,7 +5832,10 @@ void Transaction::clear(const KeyRangeRef& range, AddConflictRange addConflictRa
return;
t.mutations.emplace_back(req.arena, MutationRef::ClearRange, r.begin, r.end);
// NOTE: The throttling cost of each clear is assumed to be one page.
// This makes compuation fast, but can be inaccurate and may
// underestimate the cost of large clears.
trState->totalCost += CLIENT_KNOBS->WRITE_COST_BYTE_FACTOR;
if (addConflictRange)
t.write_conflict_ranges.push_back(req.arena, r);
}
@ -6240,14 +6248,14 @@ ACTOR Future<Optional<ClientTrCommitCostEstimation>> estimateCommitCosts(Referen
state int i = 0;
for (; i < transaction->mutations.size(); ++i) {
auto* it = &transaction->mutations[i];
auto const& mutation = transaction->mutations[i];
if (it->type == MutationRef::Type::SetValue || it->isAtomicOp()) {
if (mutation.type == MutationRef::Type::SetValue || mutation.isAtomicOp()) {
trCommitCosts.opsCount++;
trCommitCosts.writeCosts += getWriteOperationCost(it->expectedSize());
} else if (it->type == MutationRef::Type::ClearRange) {
trCommitCosts.writeCosts += getWriteOperationCost(mutation.expectedSize());
} else if (mutation.type == MutationRef::Type::ClearRange) {
trCommitCosts.opsCount++;
keyRange = KeyRangeRef(it->param1, it->param2);
keyRange = KeyRangeRef(mutation.param1, mutation.param2);
if (trState->options.expensiveClearCostEstimation) {
StorageMetrics m = wait(trState->cx->getStorageMetrics(keyRange, CLIENT_KNOBS->TOO_MANY, trState));
trCommitCosts.clearIdxCosts.emplace_back(i, getWriteOperationCost(m.bytes));
@ -7994,6 +8002,21 @@ ACTOR Future<TenantMapEntry> blobGranuleGetTenantEntry(Transaction* self,
return tme;
}
// Tenant's are supposed to be unique and therefore can be loaded once.
// There is an assumption that a tenant exists as long as operations are happening against said tenant.
ACTOR Future<TenantMapEntry> blobLoadTenantMapEntry(Database* db, Key rangeStartKey, Optional<TenantName> tenantName) {
state Transaction tr(*db);
loop {
try {
TenantMapEntry tme = wait(blobGranuleGetTenantEntry(&tr, rangeStartKey, tenantName));
return tme;
} catch (Error& e) {
wait(tr.onError(e));
}
}
}
Future<Standalone<VectorRef<KeyRef>>> Transaction::getRangeSplitPoints(KeyRange const& keys, int64_t chunkSize) {
return ::getRangeSplitPoints(
trState, keys, chunkSize, readVersion.isValid() && readVersion.isReady() ? readVersion.get() : latestVersion);
@ -8465,7 +8488,6 @@ ACTOR Future<Version> verifyBlobRangeActor(Reference<DatabaseContext> cx,
state Version readVersionOut = invalidVersion;
state int batchSize = BUGGIFY ? deterministicRandom()->randomInt(2, 10) : CLIENT_KNOBS->BG_TOO_MANY_GRANULES / 2;
state int loadSize = (BUGGIFY ? deterministicRandom()->randomInt(1, 20) : 20) * batchSize;
state bool loadedTenantEntry = false;
if (version.present()) {
if (version.get() == latestVersion) {
@ -8485,16 +8507,16 @@ ACTOR Future<Version> verifyBlobRangeActor(Reference<DatabaseContext> cx,
}
}
if (tenantName.present()) {
TenantMapEntry tme = wait(blobLoadTenantMapEntry(&db, range.begin, tenantName));
range = range.withPrefix(tme.prefix);
curRegion = KeyRangeRef(range.begin, range.begin);
}
loop {
if (curRegion.begin >= range.end) {
return readVersionOut;
}
if (tenantName.present() && !loadedTenantEntry) {
TenantMapEntry tenantEntry = wait(blobGranuleGetTenantEntry(&tr, range.begin, tenantName));
loadedTenantEntry = true;
range = range.withPrefix(tenantEntry.prefix);
curRegion = KeyRangeRef(range.begin, range.begin);
}
loop {
try {
wait(store(allRanges, tr.getBlobGranuleRanges(KeyRangeRef(curRegion.begin, range.end), loadSize)));
@ -10618,70 +10640,28 @@ ACTOR Future<Standalone<VectorRef<KeyRangeRef>>> getBlobRanges(Transaction* tr,
state Standalone<VectorRef<KeyRangeRef>> blobRanges;
state Key beginKey = range.begin;
tr->setOption(FDBTransactionOptions::ACCESS_SYSTEM_KEYS);
loop {
try {
tr->setOption(FDBTransactionOptions::ACCESS_SYSTEM_KEYS);
state RangeResult results = wait(
krmGetRangesUnaligned(tr, blobRangeKeys.begin, KeyRangeRef(beginKey, range.end), 2 * batchLimit + 2));
state RangeResult results =
wait(krmGetRangesUnaligned(tr, blobRangeKeys.begin, KeyRangeRef(beginKey, range.end), 2 * batchLimit + 2));
blobRanges.arena().dependsOn(results.arena());
for (int i = 0; i < results.size() - 1; i++) {
if (results[i].value == blobRangeActive) {
blobRanges.push_back(blobRanges.arena(), KeyRangeRef(results[i].key, results[i + 1].key));
}
if (blobRanges.size() == batchLimit) {
return blobRanges;
}
blobRanges.arena().dependsOn(results.arena());
for (int i = 0; i < results.size() - 1; i++) {
if (results[i].value == blobRangeActive) {
blobRanges.push_back(blobRanges.arena(), KeyRangeRef(results[i].key, results[i + 1].key));
}
if (!results.more) {
if (blobRanges.size() == batchLimit) {
return blobRanges;
}
beginKey = results.back().key;
} catch (Error& e) {
wait(tr->onError(e));
}
}
}
ACTOR Future<Standalone<VectorRef<KeyRangeRef>>> getBlobbifiedRanges(Transaction* tr,
KeyRange range,
int rangeLimit,
Optional<TenantName> tenantName) {
state TenantMapEntry tme;
loop {
try {
if (tenantName.present()) {
wait(store(tme, blobGranuleGetTenantEntry(tr, range.begin, tenantName)));
range = range.withPrefix(tme.prefix);
}
break;
} catch (Error& e) {
wait(tr->onError(e));
if (!results.more) {
return blobRanges;
}
beginKey = results.back().key;
}
state Standalone<VectorRef<KeyRangeRef>> blobRanges = wait(getBlobRanges(tr, range, rangeLimit));
if (!tenantName.present()) {
return blobRanges;
}
// Strip tenant prefix out.
state Standalone<VectorRef<KeyRangeRef>> tenantBlobRanges;
for (auto& blobRange : blobRanges) {
// Filter out blob ranges that span tenants for some reason.
if (!blobRange.begin.startsWith(tme.prefix) || !blobRange.end.startsWith(tme.prefix)) {
TraceEvent("ListBlobbifiedRangeSpansTenants")
.suppressFor(/*seconds=*/5)
.detail("Tenant", tenantName.get())
.detail("Range", blobRange);
continue;
}
tenantBlobRanges.push_back_deep(tenantBlobRanges.arena(), blobRange.removePrefix(tme.prefix));
}
return tenantBlobRanges;
}
ACTOR Future<Key> purgeBlobGranulesActor(Reference<DatabaseContext> db,
@ -10693,7 +10673,6 @@ ACTOR Future<Key> purgeBlobGranulesActor(Reference<DatabaseContext> db,
state Transaction tr(cx);
state Key purgeKey;
state KeyRange purgeRange = range;
state bool loadedTenantPrefix = false;
tr.setOption(FDBTransactionOptions::PRIORITY_SYSTEM_IMMEDIATE);
if (purgeVersion == latestVersion) {
@ -10713,23 +10692,22 @@ ACTOR Future<Key> purgeBlobGranulesActor(Reference<DatabaseContext> db,
throw unsupported_operation();
}
if (tenant.present()) {
TenantMapEntry tme = wait(blobLoadTenantMapEntry(&cx, range.begin, tenant));
purgeRange = purgeRange.withPrefix(tme.prefix);
}
loop {
try {
tr.setOption(FDBTransactionOptions::ACCESS_SYSTEM_KEYS);
tr.setOption(FDBTransactionOptions::PRIORITY_SYSTEM_IMMEDIATE);
tr.setOption(FDBTransactionOptions::LOCK_AWARE);
if (tenant.present() && !loadedTenantPrefix) {
TenantMapEntry tenantEntry = wait(blobGranuleGetTenantEntry(&tr, range.begin, tenant));
loadedTenantPrefix = true;
purgeRange = purgeRange.withPrefix(tenantEntry.prefix);
}
// must be aligned to blob range(s)
state Future<Standalone<VectorRef<KeyRangeRef>>> blobbifiedBegin =
getBlobbifiedRanges(&tr, KeyRangeRef(purgeRange.begin, purgeRange.begin), 2, {});
getBlobRanges(&tr, KeyRangeRef(purgeRange.begin, purgeRange.begin), 2);
state Future<Standalone<VectorRef<KeyRangeRef>>> blobbifiedEnd =
getBlobbifiedRanges(&tr, KeyRangeRef(purgeRange.end, purgeRange.end), 2, {});
getBlobRanges(&tr, KeyRangeRef(purgeRange.end, purgeRange.end), 2);
wait(success(blobbifiedBegin) && success(blobbifiedEnd));
if ((!blobbifiedBegin.get().empty() && blobbifiedBegin.get().front().begin < purgeRange.begin) ||
(!blobbifiedEnd.get().empty() && blobbifiedEnd.get().back().end > purgeRange.end)) {
@ -10815,7 +10793,11 @@ ACTOR Future<bool> setBlobRangeActor(Reference<DatabaseContext> cx,
Optional<TenantName> tenantName) {
state Database db(cx);
state Reference<ReadYourWritesTransaction> tr = makeReference<ReadYourWritesTransaction>(db);
state bool loadedTenantEntry = false;
if (tenantName.present()) {
TenantMapEntry tme = wait(blobLoadTenantMapEntry(&db, range.begin, tenantName));
range = range.withPrefix(tme.prefix);
}
state Value value = active ? blobRangeActive : blobRangeInactive;
loop {
@ -10823,13 +10805,6 @@ ACTOR Future<bool> setBlobRangeActor(Reference<DatabaseContext> cx,
tr->setOption(FDBTransactionOptions::ACCESS_SYSTEM_KEYS);
tr->setOption(FDBTransactionOptions::PRIORITY_SYSTEM_IMMEDIATE);
if (tenantName.present() && !loadedTenantEntry) {
TenantMapEntry tenantEntry =
wait(blobGranuleGetTenantEntry(&tr->getTransaction(), range.begin, tenantName));
loadedTenantEntry = true;
range = range.withPrefix(tenantEntry.prefix);
}
Standalone<VectorRef<KeyRangeRef>> startBlobRanges = wait(getBlobRanges(&tr->getTransaction(), range, 1));
if (active) {
@ -10881,10 +10856,41 @@ ACTOR Future<Standalone<VectorRef<KeyRangeRef>>> listBlobbifiedRangesActor(Refer
state Database db(cx);
state Transaction tr(db);
state TenantMapEntry tme;
state Standalone<VectorRef<KeyRangeRef>> blobRanges;
Standalone<VectorRef<KeyRangeRef>> blobbifiedRanges = wait(getBlobbifiedRanges(&tr, range, rangeLimit, tenantName));
if (tenantName.present()) {
wait(store(tme, blobLoadTenantMapEntry(&db, range.begin, tenantName)));
range = range.withPrefix(tme.prefix);
}
return blobbifiedRanges;
loop {
try {
wait(store(blobRanges, getBlobRanges(&tr, range, rangeLimit)));
break;
} catch (Error& e) {
wait(tr.onError(e));
}
}
if (!tenantName.present()) {
return blobRanges;
}
// Strip tenant prefix out.
state Standalone<VectorRef<KeyRangeRef>> tenantBlobRanges;
for (auto& blobRange : blobRanges) {
// Filter out blob ranges that span tenants for some reason.
if (!blobRange.begin.startsWith(tme.prefix) || !blobRange.end.startsWith(tme.prefix)) {
TraceEvent("ListBlobbifiedRangeSpansTenants")
.suppressFor(/*seconds=*/5)
.detail("Tenant", tenantName.get())
.detail("Range", blobRange);
continue;
}
tenantBlobRanges.push_back_deep(tenantBlobRanges.arena(), blobRange.removePrefix(tme.prefix));
}
return tenantBlobRanges;
}
Future<Standalone<VectorRef<KeyRangeRef>>> DatabaseContext::listBlobbifiedRanges(KeyRange range,

View File

@ -564,6 +564,10 @@ Version PaxosConfigTransaction::getCommittedVersion() const {
return impl->getCommittedVersion();
}
int64_t PaxosConfigTransaction::getTotalCost() const {
return 0;
}
int64_t PaxosConfigTransaction::getApproximateSize() const {
return impl->getApproximateSize();
}

View File

@ -383,6 +383,8 @@ void ServerKnobs::initialize(Randomize randomize, ClientKnobs* clientKnobs, IsSi
init( ROCKSDB_WRITER_THREAD_PRIORITY, 0 );
init( ROCKSDB_BACKGROUND_PARALLELISM, 4 );
init( ROCKSDB_READ_PARALLELISM, 4 );
// If true, do not process and store RocksDB logs
init( ROCKSDB_MUTE_LOGS, false );
// Use a smaller memtable in simulation to avoid OOMs.
int64_t memtableBytes = isSimulated ? 32 * 1024 : 512 * 1024 * 1024;
init( ROCKSDB_MEMTABLE_BYTES, memtableBytes );
@ -422,10 +424,11 @@ void ServerKnobs::initialize(Randomize randomize, ClientKnobs* clientKnobs, IsSi
// Enable this knob only for experminatal purpose, never enable this in production.
// If enabled, all the committed in-memory memtable writes are lost on a crash.
init( ROCKSDB_DISABLE_WAL_EXPERIMENTAL, false );
// If ROCKSDB_SINGLEKEY_DELETES_ON_CLEARRANGE is enabled, disable ENABLE_CLEAR_RANGE_EAGER_READS knob.
// If ROCKSDB_SINGLEKEY_DELETES_ON_CLEARRANGE is enabled, disable ROCKSDB_ENABLE_CLEAR_RANGE_EAGER_READS knob.
// These knobs have contrary functionality.
init( ROCKSDB_SINGLEKEY_DELETES_ON_CLEARRANGE, false ); if( randomize && BUGGIFY ) ROCKSDB_SINGLEKEY_DELETES_ON_CLEARRANGE = deterministicRandom()->coinflip() ? false : true;
init( ROCKSDB_SINGLEKEY_DELETES_BYTES_LIMIT, 200000 ); // 200KB
init( ROCKSDB_ENABLE_CLEAR_RANGE_EAGER_READS, true ); if( randomize && BUGGIFY ) ROCKSDB_ENABLE_CLEAR_RANGE_EAGER_READS = deterministicRandom()->coinflip() ? false : true;
// Can commit will delay ROCKSDB_CAN_COMMIT_DELAY_ON_OVERLOAD seconds for
// ROCKSDB_CAN_COMMIT_DELAY_TIMES_ON_OVERLOAD times, if rocksdb overloaded.
// Set ROCKSDB_CAN_COMMIT_DELAY_TIMES_ON_OVERLOAD to 0, to disable
@ -731,9 +734,10 @@ void ServerKnobs::initialize(Randomize randomize, ClientKnobs* clientKnobs, IsSi
init( ENFORCE_TAG_THROTTLING_ON_PROXIES, GLOBAL_TAG_THROTTLING );
init( GLOBAL_TAG_THROTTLING_MIN_RATE, 1.0 );
init( GLOBAL_TAG_THROTTLING_FOLDING_TIME, 10.0 );
init( GLOBAL_TAG_THROTTLING_RW_FUNGIBILITY_RATIO, 5.0 );
init( GLOBAL_TAG_THROTTLING_MAX_TAGS_TRACKED, 10 );
init( GLOBAL_TAG_THROTTLING_TAG_EXPIRE_AFTER, 240.0 );
init( PROXY_MAX_TAG_THROTTLE_DURATION, 5.0 ); if( randomize && BUGGIFY ) PROXY_MAX_TAG_THROTTLE_DURATION = 0.5;
init( GLOBAL_TAG_THROTTLING_PROXY_LOGGING_INTERVAL, 60.0 );
//Storage Metrics
init( STORAGE_METRICS_AVERAGE_INTERVAL, 120.0 );
@ -942,9 +946,9 @@ void ServerKnobs::initialize(Randomize randomize, ClientKnobs* clientKnobs, IsSi
init( ENCRYPTION_MODE, "AES-256-CTR" );
init( SIM_KMS_MAX_KEYS, 4096 );
init( ENCRYPT_PROXY_MAX_DBG_TRACE_LENGTH, 100000 );
init( ENABLE_TLOG_ENCRYPTION, ENABLE_ENCRYPTION ); if ( randomize && BUGGIFY && ENABLE_ENCRYPTION && !PROXY_USE_RESOLVER_PRIVATE_MUTATIONS ) ENABLE_TLOG_ENCRYPTION = true;
init( ENABLE_STORAGE_SERVER_ENCRYPTION, ENABLE_ENCRYPTION ); if ( randomize && BUGGIFY) ENABLE_STORAGE_SERVER_ENCRYPTION = !ENABLE_STORAGE_SERVER_ENCRYPTION;
init( ENABLE_BLOB_GRANULE_ENCRYPTION, ENABLE_ENCRYPTION ); if ( randomize && BUGGIFY) ENABLE_BLOB_GRANULE_ENCRYPTION = !ENABLE_BLOB_GRANULE_ENCRYPTION;
init( ENABLE_TLOG_ENCRYPTION, ENABLE_ENCRYPTION ); if ( randomize && BUGGIFY && ENABLE_ENCRYPTION ) ENABLE_TLOG_ENCRYPTION = false;
init( ENABLE_STORAGE_SERVER_ENCRYPTION, ENABLE_ENCRYPTION ); if ( randomize && BUGGIFY && ENABLE_ENCRYPTION) ENABLE_STORAGE_SERVER_ENCRYPTION = false;
init( ENABLE_BLOB_GRANULE_ENCRYPTION, ENABLE_ENCRYPTION ); if ( randomize && BUGGIFY && ENABLE_ENCRYPTION) ENABLE_BLOB_GRANULE_ENCRYPTION = false;
// encrypt key proxy
init( ENABLE_BLOB_GRANULE_COMPRESSION, false ); if ( randomize && BUGGIFY ) { ENABLE_BLOB_GRANULE_COMPRESSION = deterministicRandom()->coinflip(); }

View File

@ -296,6 +296,10 @@ Version SimpleConfigTransaction::getCommittedVersion() const {
return impl->getCommittedVersion();
}
int64_t SimpleConfigTransaction::getTotalCost() const {
return 0;
}
int64_t SimpleConfigTransaction::getApproximateSize() const {
return impl->getApproximateSize();
}

View File

@ -626,6 +626,14 @@ ThreadFuture<SpanContext> ThreadSafeTransaction::getSpanContext() {
});
}
ThreadFuture<int64_t> ThreadSafeTransaction::getTotalCost() {
ISingleThreadTransaction* tr = this->tr;
return onMainThread([tr]() -> Future<int64_t> {
tr->checkDeferredError();
return tr->getTotalCost();
});
}
ThreadFuture<int64_t> ThreadSafeTransaction::getApproximateSize() {
ISingleThreadTransaction* tr = this->tr;
return onMainThread([tr]() -> Future<int64_t> {

View File

@ -262,8 +262,8 @@ public:
double TAG_THROTTLE_EXPIRATION_INTERVAL;
int64_t WRITE_COST_BYTE_FACTOR; // Used to round up the cost of write operations
int64_t READ_COST_BYTE_FACTOR; // Used to round up the cost of read operations
double PROXY_MAX_TAG_THROTTLE_DURATION; // Maximum duration that a transaction can be tag throttled by proxy before
// being rejected
// Cost multiplier for writes (because write operations are more expensive than reads):
double GLOBAL_TAG_THROTTLING_RW_FUNGIBILITY_RATIO;
// busyness reporting
double BUSYNESS_SPIKE_START_THRESHOLD;

View File

@ -120,6 +120,7 @@ public:
// later if they are not really needed.
virtual ThreadFuture<VersionVector> getVersionVector() = 0;
virtual ThreadFuture<SpanContext> getSpanContext() = 0;
virtual ThreadFuture<int64_t> getTotalCost() = 0;
virtual ThreadFuture<int64_t> getApproximateSize() = 0;
virtual void setOption(FDBTransactionOptions::Option option, Optional<StringRef> value = Optional<StringRef>()) = 0;

View File

@ -101,6 +101,7 @@ public:
virtual Version getCommittedVersion() const = 0;
virtual VersionVector getVersionVector() const = 0;
virtual SpanContext getSpanContext() const = 0;
virtual int64_t getTotalCost() const = 0;
virtual int64_t getApproximateSize() const = 0;
virtual Future<Standalone<StringRef>> getVersionstamp() = 0;
virtual void setOption(FDBTransactionOptions::Option option, Optional<StringRef> value = Optional<StringRef>()) = 0;

View File

@ -377,6 +377,7 @@ struct FdbCApi : public ThreadSafeReferenceCounted<FdbCApi> {
FDBFuture* (*transactionCommit)(FDBTransaction* tr);
fdb_error_t (*transactionGetCommittedVersion)(FDBTransaction* tr, int64_t* outVersion);
FDBFuture* (*transactionGetTotalCost)(FDBTransaction* tr);
FDBFuture* (*transactionGetApproximateSize)(FDBTransaction* tr);
FDBFuture* (*transactionWatch)(FDBTransaction* tr, uint8_t const* keyName, int keyNameLength);
FDBFuture* (*transactionOnError)(FDBTransaction* tr, fdb_error_t error);
@ -505,6 +506,7 @@ public:
Version getCommittedVersion() override;
ThreadFuture<VersionVector> getVersionVector() override;
ThreadFuture<SpanContext> getSpanContext() override { return SpanContext(); };
ThreadFuture<int64_t> getTotalCost() override;
ThreadFuture<int64_t> getApproximateSize() override;
void setOption(FDBTransactionOptions::Option option, Optional<StringRef> value = Optional<StringRef>()) override;
@ -732,6 +734,7 @@ public:
Version getCommittedVersion() override;
ThreadFuture<VersionVector> getVersionVector() override;
ThreadFuture<SpanContext> getSpanContext() override;
ThreadFuture<int64_t> getTotalCost() override;
ThreadFuture<int64_t> getApproximateSize() override;
void setOption(FDBTransactionOptions::Option option, Optional<StringRef> value = Optional<StringRef>()) override;

View File

@ -249,6 +249,9 @@ struct TransactionState : ReferenceCounted<TransactionState> {
SpanContext spanContext;
UseProvisionalProxies useProvisionalProxies = UseProvisionalProxies::False;
bool readVersionObtainedFromGrvProxy;
// Measured by summing the bytes accessed by each read and write operation
// after rounding up to the nearest page size and applying a write penalty
int64_t totalCost = 0;
// Special flag to skip prepending tenant prefix to mutations and conflict ranges
// when a dummy, internal transaction gets commited. The sole purpose of commitDummyTransaction() is to
@ -447,6 +450,8 @@ public:
// May be called only after commit() returns success
Version getCommittedVersion() const { return trState->committedVersion; }
int64_t getTotalCost() const { return trState->totalCost; }
// Will be fulfilled only after commit() returns success
[[nodiscard]] Future<Standalone<StringRef>> getVersionstamp();
@ -566,9 +571,16 @@ ACTOR Future<std::vector<CheckpointMetaData>> getCheckpointMetaData(Database cx,
// Checks with Data Distributor that it is safe to mark all servers in exclusions as failed
ACTOR Future<bool> checkSafeExclusions(Database cx, std::vector<AddressExclusion> exclusions);
// Round up to the nearest page size
// Measured in bytes, rounded up to the nearest page size. Multiply by fungibility ratio
// because writes are more expensive than reads.
inline uint64_t getWriteOperationCost(uint64_t bytes) {
return (bytes - 1) / CLIENT_KNOBS->WRITE_COST_BYTE_FACTOR + 1;
return CLIENT_KNOBS->GLOBAL_TAG_THROTTLING_RW_FUNGIBILITY_RATIO * CLIENT_KNOBS->WRITE_COST_BYTE_FACTOR *
((bytes - 1) / CLIENT_KNOBS->WRITE_COST_BYTE_FACTOR + 1);
}
// Measured in bytes, rounded up to the nearest page size.
inline uint64_t getReadOperationCost(uint64_t bytes) {
return ((bytes - 1) / CLIENT_KNOBS->READ_COST_BYTE_FACTOR + 1) * CLIENT_KNOBS->READ_COST_BYTE_FACTOR;
}
// Create a transaction to set the value of system key \xff/conf/perpetual_storage_wiggle. If enable == true, the value

View File

@ -64,6 +64,7 @@ public:
void clear(KeyRef const&) override;
Future<Void> commit() override;
Version getCommittedVersion() const override;
int64_t getTotalCost() const override;
int64_t getApproximateSize() const override;
void setOption(FDBTransactionOptions::Option option, Optional<StringRef> value = Optional<StringRef>()) override;
Future<Void> onError(Error const& e) override;

View File

@ -149,6 +149,7 @@ public:
VersionVector getVersionVector() const override { return tr.getVersionVector(); }
SpanContext getSpanContext() const override { return tr.getSpanContext(); }
int64_t getTotalCost() const override { return tr.getTotalCost(); }
int64_t getApproximateSize() const override { return approximateSize; }
[[nodiscard]] Future<Standalone<StringRef>> getVersionstamp() override;

View File

@ -316,6 +316,7 @@ public:
int64_t ROCKSDB_MEMTABLE_BYTES;
bool ROCKSDB_LEVEL_STYLE_COMPACTION;
bool ROCKSDB_UNSAFE_AUTO_FSYNC;
bool ROCKSDB_MUTE_LOGS;
int64_t ROCKSDB_PERIODIC_COMPACTION_SECONDS;
int ROCKSDB_PREFIX_LEN;
int64_t ROCKSDB_BLOCK_CACHE_SIZE;
@ -349,6 +350,7 @@ public:
bool ROCKSDB_DISABLE_WAL_EXPERIMENTAL;
bool ROCKSDB_SINGLEKEY_DELETES_ON_CLEARRANGE;
int64_t ROCKSDB_SINGLEKEY_DELETES_BYTES_LIMIT;
bool ROCKSDB_ENABLE_CLEAR_RANGE_EAGER_READS;
int64_t ROCKSDB_COMPACTION_READAHEAD_SIZE;
int64_t ROCKSDB_BLOCK_SIZE;
bool ENABLE_SHARDED_ROCKSDB;
@ -628,14 +630,16 @@ public:
double GLOBAL_TAG_THROTTLING_MIN_RATE;
// Used by global tag throttling counters
double GLOBAL_TAG_THROTTLING_FOLDING_TIME;
// Cost multiplier for writes (because write operations are more expensive than reads)
double GLOBAL_TAG_THROTTLING_RW_FUNGIBILITY_RATIO;
// Maximum number of tags tracked by global tag throttler. Additional tags will be ignored
// until some existing tags expire
int64_t GLOBAL_TAG_THROTTLING_MAX_TAGS_TRACKED;
// Global tag throttler forgets about throughput from a tag once no new transactions from that
// tag have been received for this duration (in seconds):
int64_t GLOBAL_TAG_THROTTLING_TAG_EXPIRE_AFTER;
// Maximum duration that a transaction can be tag throttled by proxy before being rejected
double PROXY_MAX_TAG_THROTTLE_DURATION;
// Interval at which latency bands are logged for each tag on grv proxy
double GLOBAL_TAG_THROTTLING_PROXY_LOGGING_INTERVAL;
double MAX_TRANSACTIONS_PER_BYTE;
@ -739,7 +743,6 @@ public:
int64_t MIN_TAG_READ_PAGES_RATE;
int64_t MIN_TAG_WRITE_PAGES_RATE;
double TAG_MEASUREMENT_INTERVAL;
int64_t READ_COST_BYTE_FACTOR;
bool PREFIX_COMPRESS_KVS_MEM_SNAPSHOTS;
bool REPORT_DD_METRICS;
double DD_METRICS_REPORT_INTERVAL;

View File

@ -76,6 +76,7 @@ public:
void reset() override;
void debugTransaction(UID dID) override;
void checkDeferredError() const override;
int64_t getTotalCost() const override;
int64_t getApproximateSize() const override;
void set(KeyRef const&, ValueRef const&) override;
void clear(KeyRangeRef const&) override { throw client_invalid_operation(); }

View File

@ -45,7 +45,7 @@ struct CheckpointMetaData {
constexpr static FileIdentifier file_identifier = 13804342;
Version version;
KeyRange range;
std::vector<KeyRange> ranges;
int16_t format; // CheckpointFormat.
UID ssID; // Storage server ID on which this checkpoint is created.
UID checkpointID; // A unique id for this checkpoint.
@ -58,11 +58,15 @@ struct CheckpointMetaData {
CheckpointMetaData() = default;
CheckpointMetaData(KeyRange const& range, CheckpointFormat format, UID const& ssID, UID const& checkpointID)
: version(invalidVersion), range(range), format(format), ssID(ssID), checkpointID(checkpointID), state(Pending),
referenceCount(0), gcTime(0) {}
: version(invalidVersion), format(format), ssID(ssID), checkpointID(checkpointID), state(Pending),
referenceCount(0), gcTime(0) {
this->ranges.push_back(range);
}
CheckpointMetaData(Version version, KeyRange const& range, CheckpointFormat format, UID checkpointID)
: version(version), range(range), format(format), ssID(UID()), checkpointID(checkpointID), state(Pending),
referenceCount(0), gcTime(0) {}
: version(version), format(format), ssID(UID()), checkpointID(checkpointID), state(Pending), referenceCount(0),
gcTime(0) {
this->ranges.push_back(range);
}
CheckpointState getState() const { return static_cast<CheckpointState>(state); }
@ -73,7 +77,7 @@ struct CheckpointMetaData {
void setFormat(CheckpointFormat format) { this->format = static_cast<int16_t>(format); }
std::string toString() const {
std::string res = "Checkpoint MetaData:\nRange: " + range.toString() + "\nVersion: " + std::to_string(version) +
std::string res = "Checkpoint MetaData:\nRange: " + describe(ranges) + "\nVersion: " + std::to_string(version) +
"\nFormat: " + std::to_string(format) + "\nServer: " + ssID.toString() +
"\nID: " + checkpointID.toString() + "\nState: " + std::to_string(static_cast<int>(state)) +
"\n";
@ -82,7 +86,7 @@ struct CheckpointMetaData {
template <class Ar>
void serialize(Ar& ar) {
serializer(ar, version, range, format, state, checkpointID, ssID, gcTime, serializedCheckpoint);
serializer(ar, version, ranges, format, state, checkpointID, ssID, gcTime, serializedCheckpoint);
}
};
@ -99,23 +103,28 @@ struct DataMoveMetaData {
constexpr static FileIdentifier file_identifier = 13804362;
UID id; // A unique id for this data move.
Version version;
KeyRange range;
std::vector<KeyRange> ranges;
int priority;
std::set<UID> src;
std::set<UID> dest;
std::set<UID> checkpoints;
int16_t phase; // DataMoveMetaData::Phase.
int8_t mode;
DataMoveMetaData() = default;
DataMoveMetaData(UID id, Version version, KeyRange range)
: id(id), version(version), range(std::move(range)), priority(0) {}
DataMoveMetaData(UID id, KeyRange range) : id(id), version(invalidVersion), range(std::move(range)), priority(0) {}
DataMoveMetaData(UID id, Version version, KeyRange range) : id(id), version(version), priority(0), mode(0) {
this->ranges.push_back(range);
}
DataMoveMetaData(UID id, KeyRange range) : id(id), version(invalidVersion), priority(0), mode(0) {
this->ranges.push_back(range);
}
Phase getPhase() const { return static_cast<Phase>(phase); }
void setPhase(Phase phase) { this->phase = static_cast<int16_t>(phase); }
std::string toString() const {
std::string res = "DataMoveMetaData: [ID]: " + id.shortString() + " [Range]: " + range.toString() +
std::string res = "DataMoveMetaData: [ID]: " + id.shortString() + " [Range]: " + describe(ranges) +
" [Phase]: " + std::to_string(static_cast<int>(phase)) +
" [Source Servers]: " + describe(src) + " [Destination Servers]: " + describe(dest);
return res;
@ -123,7 +132,7 @@ struct DataMoveMetaData {
template <class Ar>
void serialize(Ar& ar) {
serializer(ar, id, version, range, phase, src, dest);
serializer(ar, id, version, ranges, priority, src, dest, checkpoints, phase, mode);
}
};

View File

@ -205,6 +205,7 @@ public:
Version getCommittedVersion() override;
ThreadFuture<VersionVector> getVersionVector() override;
ThreadFuture<SpanContext> getSpanContext() override;
ThreadFuture<int64_t> getTotalCost() override;
ThreadFuture<int64_t> getApproximateSize() override;
ThreadFuture<uint64_t> getProtocolVersion();

View File

@ -133,3 +133,50 @@ Future<Void> CounterCollection::traceCounters(std::string const& traceEventName,
return CounterCollectionImpl::traceCounters(
this, traceEventName, traceEventID, interval, trackLatestName, decorator);
}
void LatencyBands::insertBand(double value) {
bands.emplace(std::make_pair(value, std::make_unique<Counter>(format("Band%f", value), *cc)));
}
FDB_DEFINE_BOOLEAN_PARAM(Filtered);
LatencyBands::LatencyBands(std::string const& name,
UID id,
double loggingInterval,
std::function<void(TraceEvent&)> const& decorator)
: name(name), id(id), loggingInterval(loggingInterval), decorator(decorator) {}
void LatencyBands::addThreshold(double value) {
if (value > 0 && bands.count(value) == 0) {
if (bands.size() == 0) {
ASSERT(!cc && !filteredCount);
cc = std::make_unique<CounterCollection>(name, id.toString());
logger = cc->traceCounters(name, id, loggingInterval, id.toString() + "/" + name, decorator);
filteredCount = std::make_unique<Counter>("Filtered", *cc);
insertBand(std::numeric_limits<double>::infinity());
}
insertBand(value);
}
}
void LatencyBands::addMeasurement(double measurement, int count, Filtered filtered) {
if (filtered && filteredCount) {
(*filteredCount) += count;
} else if (bands.size() > 0) {
auto itr = bands.upper_bound(measurement);
ASSERT(itr != bands.end());
(*itr->second) += count;
}
}
void LatencyBands::clearBands() {
logger = Void();
bands.clear();
filteredCount.reset();
cc.reset();
}
LatencyBands::~LatencyBands() {
clearBands();
}

View File

@ -182,47 +182,12 @@ static void specialCounter(CounterCollection& collection, std::string const& nam
new SpecialCounter<F>(collection, name, std::move(f));
}
FDB_DECLARE_BOOLEAN_PARAM(Filtered);
class LatencyBands {
public:
LatencyBands(std::string name, UID id, double loggingInterval)
: name(name), id(id), loggingInterval(loggingInterval) {}
void addThreshold(double value) {
if (value > 0 && bands.count(value) == 0) {
if (bands.size() == 0) {
ASSERT(!cc && !filteredCount);
cc = std::make_unique<CounterCollection>(name, id.toString());
logger = cc->traceCounters(name, id, loggingInterval, id.toString() + "/" + name);
filteredCount = std::make_unique<Counter>("Filtered", *cc);
insertBand(std::numeric_limits<double>::infinity());
}
insertBand(value);
}
}
void addMeasurement(double measurement, bool filtered = false) {
if (filtered && filteredCount) {
++(*filteredCount);
} else if (bands.size() > 0) {
auto itr = bands.upper_bound(measurement);
ASSERT(itr != bands.end());
++(*itr->second);
}
}
void clearBands() {
logger = Void();
bands.clear();
filteredCount.reset();
cc.reset();
}
~LatencyBands() { clearBands(); }
private:
std::map<double, std::unique_ptr<Counter>> bands;
std::unique_ptr<Counter> filteredCount;
std::function<void(TraceEvent&)> decorator;
std::string name;
UID id;
@ -231,9 +196,22 @@ private:
std::unique_ptr<CounterCollection> cc;
Future<Void> logger;
void insertBand(double value) {
bands.emplace(std::make_pair(value, std::make_unique<Counter>(format("Band%f", value), *cc)));
}
void insertBand(double value);
public:
LatencyBands(
std::string const& name,
UID id,
double loggingInterval,
std::function<void(TraceEvent&)> const& decorator = [](auto&) {});
LatencyBands(LatencyBands&&) = default;
LatencyBands& operator=(LatencyBands&&) = default;
void addThreshold(double value);
void addMeasurement(double measurement, int count = 1, Filtered = Filtered::False);
void clearBands();
~LatencyBands();
};
class LatencySample {

View File

@ -27,6 +27,7 @@
#include "fdbserver/ApplyMetadataMutation.h"
#include "fdbserver/EncryptionOpsUtils.h"
#include "fdbserver/IKeyValueStore.h"
#include "fdbserver/Knobs.h"
#include "fdbserver/LogProtocolMessage.h"
#include "fdbserver/LogSystem.h"
#include "flow/Error.h"
@ -87,9 +88,10 @@ public:
ApplyMetadataMutationsImpl(const SpanContext& spanContext_,
ResolverData& resolverData_,
const VectorRef<MutationRef>& mutations_)
const VectorRef<MutationRef>& mutations_,
const std::unordered_map<EncryptCipherDomainId, Reference<BlobCipherKey>>* cipherKeys_)
: spanContext(spanContext_), dbgid(resolverData_.dbgid), arena(resolverData_.arena), mutations(mutations_),
txnStateStore(resolverData_.txnStateStore), toCommit(resolverData_.toCommit),
cipherKeys(cipherKeys_), txnStateStore(resolverData_.txnStateStore), toCommit(resolverData_.toCommit),
confChange(resolverData_.confChanges), logSystem(resolverData_.logSystem), popVersion(resolverData_.popVersion),
keyInfo(resolverData_.keyInfo), storageCache(resolverData_.storageCache),
initialCommit(resolverData_.initialCommit), forResolver(true) {}
@ -160,11 +162,13 @@ private:
private:
void writeMutation(const MutationRef& m) {
if (forResolver || !isEncryptionOpSupported(EncryptOperationType::TLOG_ENCRYPTION)) {
if (!isEncryptionOpSupported(EncryptOperationType::TLOG_ENCRYPTION)) {
toCommit->writeTypedMessage(m);
} else {
ASSERT(cipherKeys != nullptr);
Arena arena;
CODE_PROBE(!forResolver, "encrypting metadata mutations");
CODE_PROBE(forResolver, "encrypting resolver mutations");
toCommit->writeTypedMessage(m.encryptMetadata(*cipherKeys, arena, BlobCipherMetrics::TLOG));
}
}
@ -799,7 +803,7 @@ private:
.detail("Tag", tag.toString())
.detail("Server", decodeServerTagKey(kv.key));
if (!forResolver) {
logSystem->pop(popVersion, decodeServerTagValue(kv.value));
logSystem->pop(popVersion, tag);
(*tag_popped)[tag] = popVersion;
}
ASSERT_WE_THINK(forResolver ^ (tag_popped != nullptr));
@ -807,11 +811,11 @@ private:
if (toCommit) {
MutationRef privatized = m;
privatized.param1 = kv.key.withPrefix(systemKeys.begin, arena);
privatized.param2 = keyAfter(kv.key, arena).withPrefix(systemKeys.begin, arena);
privatized.param2 = keyAfter(privatized.param1, arena);
TraceEvent(SevDebug, "SendingPrivatized_ClearServerTag", dbgid).detail("M", privatized);
toCommit->addTag(decodeServerTagValue(kv.value));
toCommit->addTag(tag);
writeMutation(privatized);
}
}
@ -1343,8 +1347,9 @@ void applyMetadataMutations(SpanContext const& spanContext,
void applyMetadataMutations(SpanContext const& spanContext,
ResolverData& resolverData,
const VectorRef<MutationRef>& mutations) {
ApplyMetadataMutationsImpl(spanContext, resolverData, mutations).apply();
const VectorRef<MutationRef>& mutations,
const std::unordered_map<EncryptCipherDomainId, Reference<BlobCipherKey>>* pCipherKeys) {
ApplyMetadataMutationsImpl(spanContext, resolverData, mutations, pCipherKeys).apply();
}
void applyMetadataMutations(SpanContext const& spanContext,

View File

@ -2776,6 +2776,7 @@ ACTOR Future<Void> haltBlobWorker(Reference<BlobManagerData> bmData, BlobWorkerI
if (bmData->iAmReplaced.canBeSet()) {
bmData->iAmReplaced.send(Void());
}
throw;
}
}
@ -2896,6 +2897,7 @@ ACTOR Future<Void> monitorBlobWorkerStatus(Reference<BlobManagerData> bmData, Bl
if (bmData->iAmReplaced.canBeSet()) {
bmData->iAmReplaced.send(Void());
}
throw blob_manager_replaced();
}
BoundaryEvaluation newEval(rep.continueEpoch,
@ -5299,6 +5301,7 @@ ACTOR Future<Void> blobManager(BlobManagerInterface bmInterf,
fmt::print("BM {} exiting because it is replaced\n", self->epoch);
}
TraceEvent("BlobManagerReplaced", bmInterf.id()).detail("Epoch", epoch);
wait(delay(0.0));
break;
}
when(HaltBlobManagerRequest req = waitNext(bmInterf.haltBlobManager.getFuture())) {

View File

@ -1266,8 +1266,14 @@ ACTOR Future<MutationRef> writeMutation(CommitBatchContext* self,
if (self->pProxyCommitData->isEncryptionEnabled) {
state EncryptCipherDomainId domainId = tenantId;
state MutationRef encryptedMutation;
CODE_PROBE(self->pProxyCommitData->db->get().client.tenantMode == TenantMode::DISABLED,
"using disabled tenant mode");
CODE_PROBE(self->pProxyCommitData->db->get().client.tenantMode == TenantMode::OPTIONAL_TENANT,
"using optional tenant mode");
CODE_PROBE(self->pProxyCommitData->db->get().client.tenantMode == TenantMode::REQUIRED,
"using required tenant mode");
if (encryptedMutationOpt->present()) {
if (encryptedMutationOpt && encryptedMutationOpt->present()) {
CODE_PROBE(true, "using already encrypted mutation");
encryptedMutation = encryptedMutationOpt->get();
ASSERT(encryptedMutation.isEncrypted());
@ -1299,6 +1305,8 @@ ACTOR Future<MutationRef> writeMutation(CommitBatchContext* self,
ASSERT_NE(domainId, INVALID_ENCRYPT_DOMAIN_ID);
encryptedMutation = mutation->encrypt(self->cipherKeys, domainId, *arena, BlobCipherMetrics::TLOG);
}
ASSERT(encryptedMutation.isEncrypted());
CODE_PROBE(true, "encrypting non-metadata mutations");
self->toCommit.writeTypedMessage(encryptedMutation);
return encryptedMutation;
} else {
@ -1473,12 +1481,12 @@ ACTOR Future<Void> assignMutationsToStorageServers(CommitBatchContext* self) {
if (!hasCandidateBackupKeys) {
continue;
}
if (m.type != MutationRef::Type::ClearRange) {
// 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());
CODE_PROBE(writtenMutation.isEncrypted(), "using encrypted backup mutation");
self->logRangeMutations[backupName].push_back_deep(self->logRangeMutationsArena, writtenMutation);
}
} else {
@ -1500,6 +1508,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) {
CODE_PROBE(true, "encrypting clear range backup mutation");
if (backupMutation.param1 == m.param1 && backupMutation.param2 == m.param2 &&
encryptedMutation.present()) {
backupMutation = encryptedMutation.get();
@ -1510,6 +1519,7 @@ ACTOR Future<Void> assignMutationsToStorageServers(CommitBatchContext* self) {
backupMutation =
backupMutation.encrypt(self->cipherKeys, domainId, arena, BlobCipherMetrics::BACKUP);
}
ASSERT(backupMutation.isEncrypted());
}
// Add the mutation to the relevant backup tag
@ -1613,14 +1623,25 @@ ACTOR Future<Void> postResolution(CommitBatchContext* self) {
idempotencyIdSet.param2 = kv.value;
auto& tags = pProxyCommitData->tagsForKey(kv.key);
self->toCommit.addTags(tags);
self->toCommit.writeTypedMessage(idempotencyIdSet);
if (self->pProxyCommitData->isEncryptionEnabled) {
CODE_PROBE(true, "encrypting idempotency mutation");
std::pair<EncryptCipherDomainName, EncryptCipherDomainId> p =
getEncryptDetailsFromMutationRef(self->pProxyCommitData, idempotencyIdSet);
Arena arena;
MutationRef encryptedMutation = idempotencyIdSet.encrypt(
self->cipherKeys, p.second, arena, BlobCipherMetrics::TLOG);
self->toCommit.writeTypedMessage(encryptedMutation);
} else {
self->toCommit.writeTypedMessage(idempotencyIdSet);
}
});
for (const auto& m : pProxyCommitData->idempotencyClears) {
state int i = 0;
for (i = 0; i < pProxyCommitData->idempotencyClears.size(); i++) {
MutationRef& m = pProxyCommitData->idempotencyClears[i];
auto& tags = pProxyCommitData->tagsForKey(m.param1);
self->toCommit.addTags(tags);
// TODO(nwijetunga): Encrypt these mutations
self->toCommit.writeTypedMessage(m);
Arena arena;
wait(success(writeMutation(self, SYSTEM_KEYSPACE_ENCRYPT_DOMAIN_ID, &m, nullptr, &arena)));
}
pProxyCommitData->idempotencyClears = Standalone<VectorRef<MutationRef>>();
@ -1922,7 +1943,7 @@ ACTOR Future<Void> reply(CommitBatchContext* self) {
bool filter = self->maxTransactionBytes >
pProxyCommitData->latencyBandConfig.get().commitConfig.maxCommitBytes.orDefault(
std::numeric_limits<int>::max());
pProxyCommitData->stats.commitLatencyBands.addMeasurement(duration, filter);
pProxyCommitData->stats.commitLatencyBands.addMeasurement(duration, 1, Filtered(filter));
}
}

View File

@ -304,6 +304,7 @@ class DDTxnProcessorImpl {
for (int i = 0; i < dms.size(); ++i) {
auto dataMove = std::make_shared<DataMove>(decodeDataMoveValue(dms[i].value), true);
const DataMoveMetaData& meta = dataMove->meta;
ASSERT(!meta.ranges.empty());
for (const UID& id : meta.src) {
auto& dc = server_dc[id];
if (std::find(remoteDcIds.begin(), remoteDcIds.end(), dc) != remoteDcIds.end()) {
@ -325,11 +326,11 @@ class DDTxnProcessorImpl {
std::sort(dataMove->primaryDest.begin(), dataMove->primaryDest.end());
std::sort(dataMove->remoteDest.begin(), dataMove->remoteDest.end());
auto ranges = result->dataMoveMap.intersectingRanges(meta.range);
auto ranges = result->dataMoveMap.intersectingRanges(meta.ranges.front());
for (auto& r : ranges) {
ASSERT(!r.value()->valid);
}
result->dataMoveMap.insert(meta.range, std::move(dataMove));
result->dataMoveMap.insert(meta.ranges.front(), std::move(dataMove));
++numDataMoves;
}

View File

@ -90,7 +90,7 @@ void DataMove::validateShard(const DDShardInfo& shard, KeyRangeRef range, int pr
return;
}
ASSERT(this->meta.range.contains(range));
ASSERT(!this->meta.ranges.empty() && this->meta.ranges.front().contains(range));
if (!shard.hasDest) {
TraceEvent(SevError, "DataMoveValidationError")
@ -494,17 +494,21 @@ public:
for (; it != self->initData->dataMoveMap.ranges().end(); ++it) {
const DataMoveMetaData& meta = it.value()->meta;
if (meta.ranges.empty()) {
TraceEvent(SevWarnAlways, "EmptyDataMoveRange", self->ddId).detail("DataMoveMetaData", meta.toString());
continue;
}
if (it.value()->isCancelled() || (it.value()->valid && !SERVER_KNOBS->SHARD_ENCODE_LOCATION_METADATA)) {
RelocateShard rs(meta.range, DataMovementReason::RECOVER_MOVE, RelocateReason::OTHER);
RelocateShard rs(meta.ranges.front(), DataMovementReason::RECOVER_MOVE, RelocateReason::OTHER);
rs.dataMoveId = meta.id;
rs.cancelled = true;
self->relocationProducer.send(rs);
TraceEvent("DDInitScheduledCancelDataMove", self->ddId).detail("DataMove", meta.toString());
} else if (it.value()->valid) {
TraceEvent(SevDebug, "DDInitFoundDataMove", self->ddId).detail("DataMove", meta.toString());
ASSERT(meta.range == it.range());
ASSERT(meta.ranges.front() == it.range());
// TODO: Persist priority in DataMoveMetaData.
RelocateShard rs(meta.range, DataMovementReason::RECOVER_MOVE, RelocateReason::OTHER);
RelocateShard rs(meta.ranges.front(), DataMovementReason::RECOVER_MOVE, RelocateReason::OTHER);
rs.dataMoveId = meta.id;
rs.dataMove = it.value();
std::vector<ShardsAffectedByTeamFailure::Team> teams;

View File

@ -55,7 +55,7 @@ struct StringBuffer {
StringBuffer(UID fromFileID) : reserved(0), id(fromFileID) {}
int size() const { return str.size(); }
StringRef& ref() { return str; }
Standalone<StringRef> get() { return str; }
void clear() {
str = Standalone<StringRef>();
reserved = 0;
@ -63,19 +63,19 @@ struct StringBuffer {
void clearReserve(int size) {
str = Standalone<StringRef>();
reserved = size;
ref() = StringRef(new (str.arena()) uint8_t[size], 0);
str.contents() = StringRef(new (str.arena()) uint8_t[size], 0);
}
void append(StringRef x) { memcpy(append(x.size()), x.begin(), x.size()); }
void* append(int bytes) {
ASSERT(str.size() + bytes <= reserved);
void* p = const_cast<uint8_t*>(str.end());
ref() = StringRef(str.begin(), str.size() + bytes);
str.contents() = StringRef(str.begin(), str.size() + bytes);
return p;
}
StringRef pop_front(int bytes) {
ASSERT(bytes <= str.size());
StringRef result = str.substr(0, bytes);
ref() = str.substr(bytes);
str.contents() = str.substr(bytes);
return result;
}
void alignReserve(int alignment, int size) {
@ -101,7 +101,7 @@ struct StringBuffer {
if (str.size() > 0) {
memcpy(p, str.begin(), str.size());
}
ref() = StringRef(p, str.size());
str.contents() = StringRef(p, str.size());
}
}
};
@ -196,7 +196,7 @@ public:
stallCount.init("RawDiskQueue.StallCount"_sr);
}
Future<Void> pushAndCommit(StringRef pageData, StringBuffer* pageMem, uint64_t poppedPages) {
Future<Void> pushAndCommit(Standalone<StringRef> pageData, StringBuffer* pageMem, uint64_t poppedPages) {
return pushAndCommit(this, pageData, pageMem, poppedPages);
}
@ -332,13 +332,13 @@ public:
}
#endif
Future<Future<Void>> push(StringRef pageData, std::vector<Reference<SyncQueue>>* toSync) {
Future<Future<Void>> push(Standalone<StringRef> pageData, std::vector<Reference<SyncQueue>>* toSync) {
return push(this, pageData, toSync);
}
ACTOR static Future<Future<Void>> push(RawDiskQueue_TwoFiles* self,
StringRef pageData,
std::vector<Reference<SyncQueue>>* toSync) {
ACTOR static UNCANCELLABLE Future<Future<Void>> push(RawDiskQueue_TwoFiles* self,
Standalone<StringRef> pageData,
std::vector<Reference<SyncQueue>>* toSync) {
// Write the given data (pageData) to the queue files, swapping or extending them if necessary.
// Don't do any syncs, but push the modified file(s) onto toSync.
ASSERT(self->readingFile == 2);
@ -357,8 +357,9 @@ public:
toSync->push_back(self->files[1].syncQueue);
/*TraceEvent("RDQWriteAndSwap", this->dbgid).detail("File1name", self->files[1].dbgFilename).detail("File1size", self->files[1].size)
.detail("WritingPos", self->writingPos).detail("WritingBytes", p);*/
waitfor.push_back(self->files[1].f->write(pageData.begin(), p, self->writingPos));
pageData = pageData.substr(p);
waitfor.push_back(uncancellable(
holdWhile(pageData, self->files[1].f->write(pageData.begin(), p, self->writingPos))));
pageData.contents() = pageData.substr(p);
}
self->dbg_file0BeginSeq += self->files[0].size;
@ -426,7 +427,8 @@ public:
.detail("WritingPos", self->writingPos).detail("WritingBytes", pageData.size());*/
self->files[1].size = std::max(self->files[1].size, self->writingPos + pageData.size());
toSync->push_back(self->files[1].syncQueue);
waitfor.push_back(self->files[1].f->write(pageData.begin(), pageData.size(), self->writingPos));
waitfor.push_back(uncancellable(
holdWhile(pageData, self->files[1].f->write(pageData.begin(), pageData.size(), self->writingPos))));
self->writingPos += pageData.size();
return waitForAllReadyThenThrow(waitfor);
@ -435,7 +437,7 @@ public:
// Write the given data (pageData) to the queue files of self, sync data to disk, and delete the memory (pageMem)
// that hold the pageData
ACTOR static UNCANCELLABLE Future<Void> pushAndCommit(RawDiskQueue_TwoFiles* self,
StringRef pageData,
Standalone<StringRef> pageData,
StringBuffer* pageMem,
uint64_t poppedPages) {
state Promise<Void> pushing, committed;
@ -983,7 +985,7 @@ public:
lastCommittedSeq = backPage().endSeq();
auto f = rawQueue->pushAndCommit(
pushed_page_buffer->ref(), pushed_page_buffer, poppedSeq / sizeof(Page) - lastPoppedSeq / sizeof(Page));
pushed_page_buffer->get(), pushed_page_buffer, poppedSeq / sizeof(Page) - lastPoppedSeq / sizeof(Page));
lastPoppedSeq = poppedSeq;
pushed_page_buffer = 0;
return f;
@ -1179,7 +1181,7 @@ private:
Standalone<StringRef> pagedData = wait(readPages(self, start, end));
const int startOffset = start % _PAGE_SIZE;
const int dataLen = end - start;
ASSERT(pagedData.substr(startOffset, dataLen).compare(buffer->ref().substr(0, dataLen)) == 0);
ASSERT(pagedData.substr(startOffset, dataLen).compare(buffer->get().substr(0, dataLen)) == 0);
} catch (Error& e) {
if (e.code() != error_code_io_error) {
delete buffer;
@ -1546,9 +1548,9 @@ private:
StringBuffer* pushed_page_buffer;
Page& backPage() {
ASSERT(pushedPageCount());
return ((Page*)pushed_page_buffer->ref().end())[-1];
return ((Page*)pushed_page_buffer->get().end())[-1];
}
Page const& backPage() const { return ((Page*)pushed_page_buffer->ref().end())[-1]; }
Page const& backPage() const { return ((Page*)pushed_page_buffer->get().end())[-1]; }
int pushedPageCount() const { return pushed_page_buffer ? pushed_page_buffer->size() / sizeof(Page) : 0; }
// Recovery state
@ -1662,3 +1664,43 @@ IDiskQueue* openDiskQueue(std::string basename,
int64_t fileSizeWarningLimit) {
return new DiskQueue_PopUncommitted(basename, ext, dbgid, dqv, fileSizeWarningLimit);
}
TEST_CASE("performance/fdbserver/DiskQueue") {
state IDiskQueue* queue =
openDiskQueue("test-", "fdq", deterministicRandom()->randomUniqueID(), DiskQueueVersion::V2);
state std::string valueString = std::string(10e6, '.');
state StringRef valueStr((uint8_t*)valueString.c_str(), 10e6);
state std::deque<IDiskQueue::location> locations;
state int loopCount = 0;
state Future<Void> lastCommit = Void();
bool fullyRecovered = wait(queue->initializeRecovery(0));
if (!fullyRecovered) {
loop {
Standalone<StringRef> h = wait(queue->readNext(1e6));
if (h.size() < 1e6) {
break;
}
}
}
while (loopCount < 4000) {
if (loopCount % 100 == 0) {
printf("loop count: %d\n", loopCount);
}
if (++loopCount % 2 == 0) {
state IDiskQueue::location frontLocation = locations.front();
locations.pop_front();
if (locations.size() > 10) {
Standalone<StringRef> r = wait(queue->read(frontLocation, locations.front(), CheckHashes::True));
}
queue->pop(frontLocation);
}
wait(delay(0.001));
locations.push_back(queue->push(valueStr));
Future<Void> prevCommit = lastCommit;
lastCommit = queue->commit();
wait(prevCommit);
}
queue->dispose();
wait(queue->onClosed());
return Void();
}

View File

@ -107,7 +107,7 @@ class GlobalTagThrottlerImpl {
if (opType == OpType::READ) {
readCost.setTotal(newCost);
} else {
writeCost.setTotal(SERVER_KNOBS->GLOBAL_TAG_THROTTLING_RW_FUNGIBILITY_RATIO * newCost);
writeCost.setTotal(CLIENT_KNOBS->GLOBAL_TAG_THROTTLING_RW_FUNGIBILITY_RATIO * newCost);
}
}
@ -161,12 +161,18 @@ class GlobalTagThrottlerImpl {
}
};
struct StorageServerInfo {
Optional<Standalone<StringRef>> zoneId;
Optional<double> throttlingRatio;
};
Database db;
UID id;
int maxFallingBehind{ 0 };
uint64_t throttledTagChangeId{ 0 };
uint32_t lastBusyTagCount{ 0 };
std::unordered_map<UID, Optional<double>> throttlingRatios;
std::unordered_map<UID, StorageServerInfo> ssInfos;
std::unordered_map<TransactionTag, PerTagStatistics> tagStatistics;
std::unordered_map<UID, std::unordered_map<TransactionTag, ThroughputCounters>> throughput;
@ -304,19 +310,20 @@ class GlobalTagThrottlerImpl {
// Returns the desired cost for a storage server, based on its current
// cost and throttling ratio
Optional<double> getLimitingCost(UID storageServerId) const {
auto const throttlingRatio = tryGet(throttlingRatios, storageServerId);
auto const currentCost = getCurrentCost(storageServerId);
if (!throttlingRatio.present() || !currentCost.present() || !throttlingRatio.get().present()) {
auto const ssInfo = tryGet(ssInfos, storageServerId);
Optional<double> const throttlingRatio = ssInfo.present() ? ssInfo.get().throttlingRatio : Optional<double>{};
Optional<double> const currentCost = getCurrentCost(storageServerId);
if (!throttlingRatio.present() || !currentCost.present()) {
return {};
}
return throttlingRatio.get().get() * currentCost.get();
return throttlingRatio.get() * currentCost.get();
}
// For a given storage server and tag combination, return the limiting transaction rate.
Optional<double> getLimitingTps(UID storageServerId, TransactionTag tag) const {
auto const quotaRatio = getQuotaRatio(tag, storageServerId);
auto const limitingCost = getLimitingCost(storageServerId);
auto const averageTransactionCost = getAverageTransactionCost(tag, storageServerId);
Optional<double> const limitingCost = getLimitingCost(storageServerId);
Optional<double> const averageTransactionCost = getAverageTransactionCost(tag, storageServerId);
if (!limitingCost.present() || !averageTransactionCost.present()) {
return {};
}
@ -325,14 +332,36 @@ class GlobalTagThrottlerImpl {
return limitingCostForTag / averageTransactionCost.get();
}
// Return the limiting transaction rate, aggregated across all storage servers
// Return the limiting transaction rate, aggregated across all storage servers.
// The limits from the worst maxFallingBehind zones are
// ignored, because we do not non-workload related issues (e.g. slow disks)
// to affect tag throttling. If more than maxFallingBehind zones are at
// or near saturation, this indicates that throttling should take place.
Optional<double> getLimitingTps(TransactionTag tag) const {
Optional<double> result;
for (const auto& [id, _] : throttlingRatios) {
auto const targetTpsForSS = getLimitingTps(id, tag);
result = getMin(result, targetTpsForSS);
// TODO: The algorithm for ignoring the worst zones can be made more efficient
std::unordered_map<Optional<Standalone<StringRef>>, double> zoneIdToLimitingTps;
for (const auto& [id, ssInfo] : ssInfos) {
auto const limitingTpsForSS = getLimitingTps(id, tag);
if (limitingTpsForSS.present()) {
auto it = zoneIdToLimitingTps.find(ssInfo.zoneId);
if (it != zoneIdToLimitingTps.end()) {
auto& limitingTpsForZone = it->second;
limitingTpsForZone = std::min<double>(limitingTpsForZone, limitingTpsForSS.get());
} else {
zoneIdToLimitingTps[ssInfo.zoneId] = limitingTpsForSS.get();
}
}
}
if (zoneIdToLimitingTps.size() <= maxFallingBehind) {
return {};
} else {
std::vector<double> zoneLimits;
for (const auto& [_, limit] : zoneIdToLimitingTps) {
zoneLimits.push_back(limit);
}
std::nth_element(zoneLimits.begin(), zoneLimits.begin() + maxFallingBehind, zoneLimits.end());
return zoneLimits[maxFallingBehind];
}
return result;
}
Optional<double> getTps(TransactionTag tag, LimitType limitType, double averageTransactionCost) const {
@ -408,7 +437,8 @@ class GlobalTagThrottlerImpl {
}
public:
GlobalTagThrottlerImpl(Database db, UID id) : db(db), id(id) {}
GlobalTagThrottlerImpl(Database db, UID id, int maxFallingBehind)
: db(db), id(id), maxFallingBehind(maxFallingBehind) {}
Future<Void> monitorThrottlingChanges() { return monitorThrottlingChanges(this); }
void addRequests(TransactionTag tag, int count) {
auto it = tagStatistics.find(tag);
@ -492,8 +522,11 @@ public:
int64_t manualThrottleCount() const { return 0; }
Future<Void> tryUpdateAutoThrottling(StorageQueueInfo const& ss) {
throttlingRatios[ss.id] = ss.getThrottlingRatio(SERVER_KNOBS->TARGET_BYTES_PER_STORAGE_SERVER,
SERVER_KNOBS->SPRING_BYTES_STORAGE_SERVER);
auto& ssInfo = ssInfos[ss.id];
ssInfo.throttlingRatio = ss.getTagThrottlingRatio(SERVER_KNOBS->TARGET_BYTES_PER_STORAGE_SERVER,
SERVER_KNOBS->SPRING_BYTES_STORAGE_SERVER);
ssInfo.zoneId = ss.locality.zoneId();
for (const auto& busyReadTag : ss.busiestReadTags) {
if (tagStatistics.find(busyReadTag.tag) != tagStatistics.end()) {
throughput[ss.id][busyReadTag.tag].updateCost(busyReadTag.rate, OpType::READ);
@ -530,7 +563,8 @@ public:
uint32_t tagsTracked() const { return tagStatistics.size(); }
};
GlobalTagThrottler::GlobalTagThrottler(Database db, UID id) : impl(PImpl<GlobalTagThrottlerImpl>::create(db, id)) {}
GlobalTagThrottler::GlobalTagThrottler(Database db, UID id, int maxFallingBehind)
: impl(PImpl<GlobalTagThrottlerImpl>::create(db, id, maxFallingBehind)) {}
GlobalTagThrottler::~GlobalTagThrottler() = default;
@ -584,7 +618,7 @@ void GlobalTagThrottler::removeExpiredTags() {
return impl->removeExpiredTags();
}
namespace GlobalTagThrottlerTesting {
namespace {
enum class LimitType { RESERVED, TOTAL };
enum class OpType { READ, WRITE };
@ -615,12 +649,13 @@ class MockStorageServer {
};
UID id;
double targetCost;
// bytes/second that this storage server can handle
double capacity;
std::map<TransactionTag, Cost> readCosts, writeCosts;
Cost totalReadCost, totalWriteCost;
public:
explicit MockStorageServer(UID id, double targetCost) : id(id), targetCost(targetCost) { ASSERT_GT(targetCost, 0); }
explicit MockStorageServer(UID id, double capacity) : id(id), capacity(capacity) { ASSERT_GT(capacity, 0); }
void addReadCost(TransactionTag tag, double cost) {
readCosts[tag] += cost;
totalReadCost += cost;
@ -630,8 +665,10 @@ public:
totalWriteCost += cost;
}
void setCapacity(double value) { capacity = value; }
StorageQueueInfo getStorageQueueInfo() const {
StorageQueueInfo result(id, LocalityData{});
StorageQueueInfo result(id, LocalityData({}, Value(id.toString()), {}, {}));
for (const auto& [tag, readCost] : readCosts) {
double fractionalBusyness{ 0.0 }; // unused for global tag throttling
result.busiestReadTags.emplace_back(tag, readCost.smoothRate(), fractionalBusyness);
@ -640,7 +677,7 @@ public:
double fractionalBusyness{ 0.0 }; // unused for global tag throttling
result.busiestWriteTags.emplace_back(tag, writeCost.smoothRate(), fractionalBusyness);
}
result.lastReply.bytesInput = ((totalReadCost.smoothRate() + totalWriteCost.smoothRate()) / targetCost) *
result.lastReply.bytesInput = ((totalReadCost.smoothRate() + totalWriteCost.smoothRate()) / capacity) *
SERVER_KNOBS->TARGET_BYTES_PER_STORAGE_SERVER;
return result;
}
@ -680,6 +717,8 @@ public:
}
}
void setCapacity(int index, double value) { storageServers[index].setCapacity(value); }
std::vector<StorageQueueInfo> getStorageQueueInfos() const {
std::vector<StorageQueueInfo> result;
result.reserve(storageServers.size());
@ -707,12 +746,14 @@ ACTOR Future<Void> runClient(GlobalTagThrottler* globalTagThrottler,
}
ACTOR template <class Check>
Future<Void> monitor(GlobalTagThrottler* globalTagThrottler, Check check) {
Future<Void> monitorActor(GlobalTagThrottler* globalTagThrottler, Check check) {
state int successes = 0;
loop {
wait(delay(1.0));
if (check(*globalTagThrottler)) {
if (++successes == 3) {
// Wait for 10 consecutive successes so we're certain
// than a stable equilibrium has been reached
if (++successes == 10) {
return Void();
}
} else {
@ -775,47 +816,55 @@ ACTOR Future<Void> updateGlobalTagThrottler(GlobalTagThrottler* globalTagThrottl
}
}
} // namespace GlobalTagThrottlerTesting
} // namespace
// 10 storage servers can handle 100 bytes/second each.
// Total quota set to 100 bytes/second.
// Client attempts 5 6-byte read transactions per second.
// Limit should adjust to allow 100/6 transactions per second.
TEST_CASE("/GlobalTagThrottler/Simple") {
state GlobalTagThrottler globalTagThrottler(Database{}, UID{});
state GlobalTagThrottlerTesting::StorageServerCollection storageServers(10, 100);
state GlobalTagThrottler globalTagThrottler(Database{}, UID{}, 0);
state StorageServerCollection storageServers(10, 100);
ThrottleApi::TagQuotaValue tagQuotaValue;
TransactionTag testTag = "sampleTag1"_sr;
tagQuotaValue.totalQuota = 100.0;
globalTagThrottler.setQuota(testTag, tagQuotaValue);
state Future<Void> client = GlobalTagThrottlerTesting::runClient(
&globalTagThrottler, &storageServers, testTag, 5.0, 6.0, GlobalTagThrottlerTesting::OpType::READ);
state Future<Void> monitor = GlobalTagThrottlerTesting::monitor(&globalTagThrottler, [testTag](auto& gtt) {
return GlobalTagThrottlerTesting::targetRateIsNear(gtt, testTag, 100.0 / 6.0);
});
state Future<Void> updater =
GlobalTagThrottlerTesting::updateGlobalTagThrottler(&globalTagThrottler, &storageServers);
state Future<Void> client = runClient(&globalTagThrottler, &storageServers, testTag, 5.0, 6.0, OpType::READ);
state Future<Void> monitor =
monitorActor(&globalTagThrottler, [testTag](auto& gtt) { return targetRateIsNear(gtt, testTag, 100.0 / 6.0); });
state Future<Void> updater = updateGlobalTagThrottler(&globalTagThrottler, &storageServers);
wait(timeoutError(monitor || client || updater, 600.0));
return Void();
}
// 10 storage servers can handle 100 bytes/second each.
// Total quota set to 100 bytes/second.
// Client attempts 5 6-byte write transactions per second.
// Limit should adjust to allow 100/(6*<fungibility_ratio>) transactions per second.
TEST_CASE("/GlobalTagThrottler/WriteThrottling") {
state GlobalTagThrottler globalTagThrottler(Database{}, UID{});
state GlobalTagThrottlerTesting::StorageServerCollection storageServers(10, 100);
state GlobalTagThrottler globalTagThrottler(Database{}, UID{}, 0);
state StorageServerCollection storageServers(10, 100);
ThrottleApi::TagQuotaValue tagQuotaValue;
TransactionTag testTag = "sampleTag1"_sr;
tagQuotaValue.totalQuota = 100.0;
globalTagThrottler.setQuota(testTag, tagQuotaValue);
state Future<Void> client = GlobalTagThrottlerTesting::runClient(
&globalTagThrottler, &storageServers, testTag, 5.0, 6.0, GlobalTagThrottlerTesting::OpType::WRITE);
state Future<Void> monitor = GlobalTagThrottlerTesting::monitor(&globalTagThrottler, [testTag](auto& gtt) {
return GlobalTagThrottlerTesting::targetRateIsNear(gtt, testTag, 100.0 / 6.0);
state Future<Void> client = runClient(&globalTagThrottler, &storageServers, testTag, 5.0, 6.0, OpType::WRITE);
state Future<Void> monitor = monitorActor(&globalTagThrottler, [testTag](auto& gtt) {
return targetRateIsNear(gtt, testTag, 100.0 / (6.0 * CLIENT_KNOBS->GLOBAL_TAG_THROTTLING_RW_FUNGIBILITY_RATIO));
});
state Future<Void> updater =
GlobalTagThrottlerTesting::updateGlobalTagThrottler(&globalTagThrottler, &storageServers);
state Future<Void> updater = updateGlobalTagThrottler(&globalTagThrottler, &storageServers);
wait(timeoutError(monitor || client || updater, 600.0));
return Void();
}
// 10 storage servers can handle 100 bytes/second each.
// Total quota set to 100 bytes/second for each tag.
// 2 clients each attempt 5 6-byte read transactions per second.
// Both limits should adjust to allow 100/6 transactions per second.
TEST_CASE("/GlobalTagThrottler/MultiTagThrottling") {
state GlobalTagThrottler globalTagThrottler(Database{}, UID{});
state GlobalTagThrottlerTesting::StorageServerCollection storageServers(10, 100);
state GlobalTagThrottler globalTagThrottler(Database{}, UID{}, 0);
state StorageServerCollection storageServers(10, 100);
ThrottleApi::TagQuotaValue tagQuotaValue;
TransactionTag testTag1 = "sampleTag1"_sr;
TransactionTag testTag2 = "sampleTag2"_sr;
@ -824,171 +873,181 @@ TEST_CASE("/GlobalTagThrottler/MultiTagThrottling") {
globalTagThrottler.setQuota(testTag2, tagQuotaValue);
state std::vector<Future<Void>> futures;
state std::vector<Future<Void>> monitorFutures;
futures.push_back(GlobalTagThrottlerTesting::runClient(
&globalTagThrottler, &storageServers, testTag1, 5.0, 6.0, GlobalTagThrottlerTesting::OpType::READ));
futures.push_back(GlobalTagThrottlerTesting::runClient(
&globalTagThrottler, &storageServers, testTag2, 5.0, 6.0, GlobalTagThrottlerTesting::OpType::READ));
futures.push_back(GlobalTagThrottlerTesting::updateGlobalTagThrottler(&globalTagThrottler, &storageServers));
state Future<Void> monitor =
GlobalTagThrottlerTesting::monitor(&globalTagThrottler, [testTag1, testTag2](auto& gtt) {
return GlobalTagThrottlerTesting::targetRateIsNear(gtt, testTag1, 100.0 / 6.0) &&
GlobalTagThrottlerTesting::targetRateIsNear(gtt, testTag2, 100.0 / 6.0);
});
futures.push_back(runClient(&globalTagThrottler, &storageServers, testTag1, 5.0, 6.0, OpType::READ));
futures.push_back(runClient(&globalTagThrottler, &storageServers, testTag2, 5.0, 6.0, OpType::READ));
futures.push_back(updateGlobalTagThrottler(&globalTagThrottler, &storageServers));
state Future<Void> monitor = monitorActor(&globalTagThrottler, [testTag1, testTag2](auto& gtt) {
return targetRateIsNear(gtt, testTag1, 100.0 / 6.0) && targetRateIsNear(gtt, testTag2, 100.0 / 6.0);
});
wait(timeoutError(waitForAny(futures) || monitor, 600.0));
return Void();
}
// 10 storage servers can handle 100 bytes/second each.
// Total quota set to 100 bytes/second.
// Client attempts 20 10-byte read transactions per second.
// Limit should adjust to allow 100/10 transactions per second.
TEST_CASE("/GlobalTagThrottler/AttemptWorkloadAboveQuota") {
state GlobalTagThrottler globalTagThrottler(Database{}, UID{});
state GlobalTagThrottlerTesting::StorageServerCollection storageServers(10, 100);
state GlobalTagThrottler globalTagThrottler(Database{}, UID{}, 0);
state StorageServerCollection storageServers(10, 100);
ThrottleApi::TagQuotaValue tagQuotaValue;
TransactionTag testTag = "sampleTag1"_sr;
tagQuotaValue.totalQuota = 100.0;
globalTagThrottler.setQuota(testTag, tagQuotaValue);
state Future<Void> client = GlobalTagThrottlerTesting::runClient(
&globalTagThrottler, &storageServers, testTag, 20.0, 10.0, GlobalTagThrottlerTesting::OpType::READ);
state Future<Void> monitor = GlobalTagThrottlerTesting::monitor(&globalTagThrottler, [testTag](auto& gtt) {
return GlobalTagThrottlerTesting::targetRateIsNear(gtt, testTag, 10.0);
});
state Future<Void> updater =
GlobalTagThrottlerTesting::updateGlobalTagThrottler(&globalTagThrottler, &storageServers);
state Future<Void> client = runClient(&globalTagThrottler, &storageServers, testTag, 20.0, 10.0, OpType::READ);
state Future<Void> monitor =
monitorActor(&globalTagThrottler, [testTag](auto& gtt) { return targetRateIsNear(gtt, testTag, 10.0); });
state Future<Void> updater = updateGlobalTagThrottler(&globalTagThrottler, &storageServers);
wait(timeoutError(monitor || client || updater, 600.0));
return Void();
}
// 10 storage servers can handle 100 bytes/second each.
// Total quota set to 100 bytes/second.
// 2 clients each attempt 5 6-byte transactions per second.
// Limit should adjust to allow 100/6 transactions per second.
TEST_CASE("/GlobalTagThrottler/MultiClientThrottling") {
state GlobalTagThrottler globalTagThrottler(Database{}, UID{});
state GlobalTagThrottlerTesting::StorageServerCollection storageServers(10, 100);
state GlobalTagThrottler globalTagThrottler(Database{}, UID{}, 0);
state StorageServerCollection storageServers(10, 100);
ThrottleApi::TagQuotaValue tagQuotaValue;
TransactionTag testTag = "sampleTag1"_sr;
tagQuotaValue.totalQuota = 100.0;
globalTagThrottler.setQuota(testTag, tagQuotaValue);
state Future<Void> client = GlobalTagThrottlerTesting::runClient(
&globalTagThrottler, &storageServers, testTag, 5.0, 6.0, GlobalTagThrottlerTesting::OpType::READ);
state Future<Void> client2 = GlobalTagThrottlerTesting::runClient(
&globalTagThrottler, &storageServers, testTag, 5.0, 6.0, GlobalTagThrottlerTesting::OpType::READ);
state Future<Void> monitor = GlobalTagThrottlerTesting::monitor(&globalTagThrottler, [testTag](auto& gtt) {
return GlobalTagThrottlerTesting::targetRateIsNear(gtt, testTag, 100.0 / 6.0) &&
GlobalTagThrottlerTesting::clientRateIsNear(gtt, testTag, 100.0 / 6.0);
state Future<Void> client = runClient(&globalTagThrottler, &storageServers, testTag, 5.0, 6.0, OpType::READ);
state Future<Void> client2 = runClient(&globalTagThrottler, &storageServers, testTag, 5.0, 6.0, OpType::READ);
state Future<Void> monitor = monitorActor(&globalTagThrottler, [testTag](auto& gtt) {
return targetRateIsNear(gtt, testTag, 100.0 / 6.0) && clientRateIsNear(gtt, testTag, 100.0 / 6.0);
});
state Future<Void> updater =
GlobalTagThrottlerTesting::updateGlobalTagThrottler(&globalTagThrottler, &storageServers);
state Future<Void> updater = updateGlobalTagThrottler(&globalTagThrottler, &storageServers);
wait(timeoutError(monitor || client || client2 || updater, 600.0));
return Void();
}
// 10 storage servers can handle 100 bytes/second each.
// Total quota set to 100 bytes/second.
// 2 clients each attempt 20 10-byte transactions per second.
// Target rate should adjust to allow 100/10 transactions per second.
// Each client is throttled to only perform 100/20 transactions per second.
TEST_CASE("/GlobalTagThrottler/MultiClientThrottling2") {
state GlobalTagThrottler globalTagThrottler(Database{}, UID{});
state GlobalTagThrottlerTesting::StorageServerCollection storageServers(10, 100);
state GlobalTagThrottler globalTagThrottler(Database{}, UID{}, 0);
state StorageServerCollection storageServers(10, 100);
ThrottleApi::TagQuotaValue tagQuotaValue;
TransactionTag testTag = "sampleTag1"_sr;
tagQuotaValue.totalQuota = 100.0;
globalTagThrottler.setQuota(testTag, tagQuotaValue);
state Future<Void> client = GlobalTagThrottlerTesting::runClient(
&globalTagThrottler, &storageServers, testTag, 20.0, 10.0, GlobalTagThrottlerTesting::OpType::READ);
state Future<Void> client2 = GlobalTagThrottlerTesting::runClient(
&globalTagThrottler, &storageServers, testTag, 20.0, 10.0, GlobalTagThrottlerTesting::OpType::READ);
state Future<Void> monitor = GlobalTagThrottlerTesting::monitor(&globalTagThrottler, [testTag](auto& gtt) {
return GlobalTagThrottlerTesting::targetRateIsNear(gtt, testTag, 10.0) &&
GlobalTagThrottlerTesting::clientRateIsNear(gtt, testTag, 5.0);
state Future<Void> client = runClient(&globalTagThrottler, &storageServers, testTag, 20.0, 10.0, OpType::READ);
state Future<Void> client2 = runClient(&globalTagThrottler, &storageServers, testTag, 20.0, 10.0, OpType::READ);
state Future<Void> monitor = monitorActor(&globalTagThrottler, [testTag](auto& gtt) {
return targetRateIsNear(gtt, testTag, 10.0) && clientRateIsNear(gtt, testTag, 5.0);
});
state Future<Void> updater =
GlobalTagThrottlerTesting::updateGlobalTagThrottler(&globalTagThrottler, &storageServers);
state Future<Void> updater = updateGlobalTagThrottler(&globalTagThrottler, &storageServers);
wait(timeoutError(monitor || client || updater, 600.0));
return Void();
}
// Global transaction rate should be 20.0, with a distribution of (5, 15) between the 2 clients
// 10 storage servers can handle 100 bytes/second each.
// Total quota set to 100 bytes/second.
// One client attempts 5 5-byte read transactions per second.
// Another client attempts 25 5-byte read transactions per second.
// Target rate should adjust to allow 100/5 transactions per second.
// This 20 transactions/second limit is split with a distribution of (5, 15) between the 2 clients.
TEST_CASE("/GlobalTagThrottler/SkewedMultiClientThrottling") {
state GlobalTagThrottler globalTagThrottler(Database{}, UID{});
state GlobalTagThrottlerTesting::StorageServerCollection storageServers(10, 100);
state GlobalTagThrottler globalTagThrottler(Database{}, UID{}, 0);
state StorageServerCollection storageServers(10, 100);
ThrottleApi::TagQuotaValue tagQuotaValue;
TransactionTag testTag = "sampleTag1"_sr;
tagQuotaValue.totalQuota = 100.0;
globalTagThrottler.setQuota(testTag, tagQuotaValue);
state Future<Void> client = GlobalTagThrottlerTesting::runClient(
&globalTagThrottler, &storageServers, testTag, 5.0, 5.0, GlobalTagThrottlerTesting::OpType::READ);
state Future<Void> client2 = GlobalTagThrottlerTesting::runClient(
&globalTagThrottler, &storageServers, testTag, 25.0, 5.0, GlobalTagThrottlerTesting::OpType::READ);
state Future<Void> monitor = GlobalTagThrottlerTesting::monitor(&globalTagThrottler, [testTag](auto& gtt) {
return GlobalTagThrottlerTesting::targetRateIsNear(gtt, testTag, 20.0) &&
GlobalTagThrottlerTesting::clientRateIsNear(gtt, testTag, 15.0);
state Future<Void> client = runClient(&globalTagThrottler, &storageServers, testTag, 5.0, 5.0, OpType::READ);
state Future<Void> client2 = runClient(&globalTagThrottler, &storageServers, testTag, 25.0, 5.0, OpType::READ);
state Future<Void> monitor = monitorActor(&globalTagThrottler, [testTag](auto& gtt) {
return targetRateIsNear(gtt, testTag, 20.0) && clientRateIsNear(gtt, testTag, 15.0);
});
state Future<Void> updater =
GlobalTagThrottlerTesting::updateGlobalTagThrottler(&globalTagThrottler, &storageServers);
state Future<Void> updater = updateGlobalTagThrottler(&globalTagThrottler, &storageServers);
wait(timeoutError(monitor || client || updater, 600.0));
return Void();
}
// 10 storage servers can handle 100 bytes/second each.
// Total quota is initially set to 100 bytes/second.
// Client attempts 5 6-byte transactions per second.
// Test that the tag throttler can reach equilibrium, then adjust to a new equilibrium once the quota is changed
// Target rate should adjust to allow 100/6 transactions per second.
// Total quota is modified to 50 bytes/second.
// Target rate should adjust to allow 50/6 transactions per second.
TEST_CASE("/GlobalTagThrottler/UpdateQuota") {
state GlobalTagThrottler globalTagThrottler(Database{}, UID{});
state GlobalTagThrottlerTesting::StorageServerCollection storageServers(10, 100);
state GlobalTagThrottler globalTagThrottler(Database{}, UID{}, 0);
state StorageServerCollection storageServers(10, 100);
state ThrottleApi::TagQuotaValue tagQuotaValue;
state TransactionTag testTag = "sampleTag1"_sr;
tagQuotaValue.totalQuota = 100.0;
globalTagThrottler.setQuota(testTag, tagQuotaValue);
state Future<Void> client = GlobalTagThrottlerTesting::runClient(
&globalTagThrottler, &storageServers, testTag, 5.0, 6.0, GlobalTagThrottlerTesting::OpType::READ);
state Future<Void> monitor = GlobalTagThrottlerTesting::monitor(&globalTagThrottler, [](auto& gtt) {
return GlobalTagThrottlerTesting::targetRateIsNear(gtt, "sampleTag1"_sr, 100.0 / 6.0);
});
state Future<Void> updater =
GlobalTagThrottlerTesting::updateGlobalTagThrottler(&globalTagThrottler, &storageServers);
state Future<Void> client = runClient(&globalTagThrottler, &storageServers, testTag, 5.0, 6.0, OpType::READ);
state Future<Void> monitor = monitorActor(
&globalTagThrottler, [](auto& gtt) { return targetRateIsNear(gtt, "sampleTag1"_sr, 100.0 / 6.0); });
state Future<Void> updater = updateGlobalTagThrottler(&globalTagThrottler, &storageServers);
wait(timeoutError(monitor || client || updater, 600.0));
tagQuotaValue.totalQuota = 50.0;
globalTagThrottler.setQuota(testTag, tagQuotaValue);
monitor = GlobalTagThrottlerTesting::monitor(&globalTagThrottler, [](auto& gtt) {
return GlobalTagThrottlerTesting::targetRateIsNear(gtt, "sampleTag1"_sr, 50.0 / 6.0);
});
monitor =
monitorActor(&globalTagThrottler, [](auto& gtt) { return targetRateIsNear(gtt, "sampleTag1"_sr, 50.0 / 6.0); });
wait(timeoutError(monitor || client || updater, 600.0));
return Void();
}
// 10 storage servers can handle 100 bytes/second each.
// Total quota is initially set to 100 bytes/second.
// Client attempts 5 6-byte read transactions per second.
// Target limit adjusts to allow 100/6 transactions per second.
// Then Quota is removed.
// Target limit is removed as a result.
TEST_CASE("/GlobalTagThrottler/RemoveQuota") {
state GlobalTagThrottler globalTagThrottler(Database{}, UID{});
state GlobalTagThrottlerTesting::StorageServerCollection storageServers(10, 100);
state GlobalTagThrottler globalTagThrottler(Database{}, UID{}, 0);
state StorageServerCollection storageServers(10, 100);
state ThrottleApi::TagQuotaValue tagQuotaValue;
state TransactionTag testTag = "sampleTag1"_sr;
tagQuotaValue.totalQuota = 100.0;
globalTagThrottler.setQuota(testTag, tagQuotaValue);
state Future<Void> client = GlobalTagThrottlerTesting::runClient(
&globalTagThrottler, &storageServers, testTag, 5.0, 6.0, GlobalTagThrottlerTesting::OpType::READ);
state Future<Void> monitor = GlobalTagThrottlerTesting::monitor(&globalTagThrottler, [](auto& gtt) {
return GlobalTagThrottlerTesting::targetRateIsNear(gtt, "sampleTag1"_sr, 100.0 / 6.0);
});
state Future<Void> updater =
GlobalTagThrottlerTesting::updateGlobalTagThrottler(&globalTagThrottler, &storageServers);
state Future<Void> client = runClient(&globalTagThrottler, &storageServers, testTag, 5.0, 6.0, OpType::READ);
state Future<Void> monitor = monitorActor(
&globalTagThrottler, [](auto& gtt) { return targetRateIsNear(gtt, "sampleTag1"_sr, 100.0 / 6.0); });
state Future<Void> updater = updateGlobalTagThrottler(&globalTagThrottler, &storageServers);
wait(timeoutError(monitor || client || updater, 600.0));
globalTagThrottler.removeQuota(testTag);
monitor = GlobalTagThrottlerTesting::monitor(&globalTagThrottler, [](auto& gtt) {
return GlobalTagThrottlerTesting::targetRateIsNear(gtt, "sampleTag1"_sr, {});
});
monitor = monitorActor(&globalTagThrottler, [](auto& gtt) { return targetRateIsNear(gtt, "sampleTag1"_sr, {}); });
wait(timeoutError(monitor || client || updater, 600.0));
return Void();
}
// 10 storage servers can handle 5 bytes/second each.
// Total quota is set to 100 bytes/second.
// Client attempts 10 6-byte transactions per second
// Target is adjusted to 50/6 transactions per second, to match the total capacity all storage servers.
TEST_CASE("/GlobalTagThrottler/ActiveThrottling") {
state GlobalTagThrottler globalTagThrottler(Database{}, UID{});
state GlobalTagThrottlerTesting::StorageServerCollection storageServers(10, 5);
state GlobalTagThrottler globalTagThrottler(Database{}, UID{}, 0);
state StorageServerCollection storageServers(10, 5);
state ThrottleApi::TagQuotaValue tagQuotaValue;
TransactionTag testTag = "sampleTag1"_sr;
tagQuotaValue.totalQuota = 100.0;
globalTagThrottler.setQuota(testTag, tagQuotaValue);
state Future<Void> client = GlobalTagThrottlerTesting::runClient(
&globalTagThrottler, &storageServers, testTag, 10.0, 6.0, GlobalTagThrottlerTesting::OpType::READ);
state Future<Void> monitor = GlobalTagThrottlerTesting::monitor(&globalTagThrottler, [testTag](auto& gtt) {
return GlobalTagThrottlerTesting::targetRateIsNear(gtt, testTag, 50 / 6.0) && gtt.busyReadTagCount() == 1;
state Future<Void> client = runClient(&globalTagThrottler, &storageServers, testTag, 10.0, 6.0, OpType::READ);
state Future<Void> monitor = monitorActor(&globalTagThrottler, [testTag](auto& gtt) {
return targetRateIsNear(gtt, testTag, 50 / 6.0) && gtt.busyReadTagCount() == 1;
});
state Future<Void> updater =
GlobalTagThrottlerTesting::updateGlobalTagThrottler(&globalTagThrottler, &storageServers);
state Future<Void> updater = updateGlobalTagThrottler(&globalTagThrottler, &storageServers);
wait(timeoutError(monitor || client || updater, 600.0));
return Void();
}
// 10 storage servers can handle 5 bytes/second each.
// Total quota is set to 50 bytes/second for one tag, 100 bytes/second for another.
// For each tag, a client attempts to execute 10 6-byte read transactions per second.
// Target rates are adjusted to utilize the full 50 bytes/second capacity of the
// add storage servers. The two tags receive this capacity with a 2:1 ratio,
// matching the ratio of their total quotas.
TEST_CASE("/GlobalTagThrottler/MultiTagActiveThrottling") {
state GlobalTagThrottler globalTagThrottler(Database{}, UID{});
state GlobalTagThrottlerTesting::StorageServerCollection storageServers(10, 5);
state GlobalTagThrottler globalTagThrottler(Database{}, UID{}, 0);
state StorageServerCollection storageServers(10, 5);
state ThrottleApi::TagQuotaValue tagQuotaValue1;
state ThrottleApi::TagQuotaValue tagQuotaValue2;
TransactionTag testTag1 = "sampleTag1"_sr;
@ -998,24 +1057,26 @@ TEST_CASE("/GlobalTagThrottler/MultiTagActiveThrottling") {
globalTagThrottler.setQuota(testTag1, tagQuotaValue1);
globalTagThrottler.setQuota(testTag2, tagQuotaValue2);
std::vector<Future<Void>> futures;
futures.push_back(GlobalTagThrottlerTesting::runClient(
&globalTagThrottler, &storageServers, testTag1, 10.0, 6.0, GlobalTagThrottlerTesting::OpType::READ));
futures.push_back(GlobalTagThrottlerTesting::runClient(
&globalTagThrottler, &storageServers, testTag2, 10.0, 6.0, GlobalTagThrottlerTesting::OpType::READ));
state Future<Void> monitor =
GlobalTagThrottlerTesting::monitor(&globalTagThrottler, [testTag1, testTag2](auto& gtt) {
return GlobalTagThrottlerTesting::targetRateIsNear(gtt, testTag1, (50 / 6.0) / 3) &&
GlobalTagThrottlerTesting::targetRateIsNear(gtt, testTag2, 2 * (50 / 6.0) / 3) &&
gtt.busyReadTagCount() == 2;
});
futures.push_back(GlobalTagThrottlerTesting::updateGlobalTagThrottler(&globalTagThrottler, &storageServers));
futures.push_back(runClient(&globalTagThrottler, &storageServers, testTag1, 10.0, 6.0, OpType::READ));
futures.push_back(runClient(&globalTagThrottler, &storageServers, testTag2, 10.0, 6.0, OpType::READ));
state Future<Void> monitor = monitorActor(&globalTagThrottler, [testTag1, testTag2](auto& gtt) {
return targetRateIsNear(gtt, testTag1, (50 / 6.0) / 3) && targetRateIsNear(gtt, testTag2, 2 * (50 / 6.0) / 3) &&
gtt.busyReadTagCount() == 2;
});
futures.push_back(updateGlobalTagThrottler(&globalTagThrottler, &storageServers));
wait(timeoutError(waitForAny(futures) || monitor, 600.0));
return Void();
}
// 3 storage servers can handle 50 bytes/second each.
// Total quota is set to 100 bytes/second for each tag.
// Each client attempts 10 6-byte read transactions per second.
// This workload is sent to 2 storage servers per client (with an overlap of one storage server).
// Target rates for both tags are adjusted to 50/6 transactions per second to match the throughput
// that the busiest server can handle.
TEST_CASE("/GlobalTagThrottler/MultiTagActiveThrottling2") {
state GlobalTagThrottler globalTagThrottler(Database{}, UID{});
state GlobalTagThrottlerTesting::StorageServerCollection storageServers(3, 50);
state GlobalTagThrottler globalTagThrottler(Database{}, UID{}, 0);
state StorageServerCollection storageServers(3, 50);
state ThrottleApi::TagQuotaValue tagQuotaValue1;
state ThrottleApi::TagQuotaValue tagQuotaValue2;
TransactionTag testTag1 = "sampleTag1"_sr;
@ -1025,23 +1086,27 @@ TEST_CASE("/GlobalTagThrottler/MultiTagActiveThrottling2") {
globalTagThrottler.setQuota(testTag1, tagQuotaValue1);
globalTagThrottler.setQuota(testTag2, tagQuotaValue2);
std::vector<Future<Void>> futures;
futures.push_back(GlobalTagThrottlerTesting::runClient(
&globalTagThrottler, &storageServers, testTag1, 10.0, 6.0, GlobalTagThrottlerTesting::OpType::READ, { 0, 1 }));
futures.push_back(GlobalTagThrottlerTesting::runClient(
&globalTagThrottler, &storageServers, testTag2, 10.0, 6.0, GlobalTagThrottlerTesting::OpType::READ, { 1, 2 }));
state Future<Void> monitor =
GlobalTagThrottlerTesting::monitor(&globalTagThrottler, [testTag1, testTag2](auto& gtt) {
return GlobalTagThrottlerTesting::targetRateIsNear(gtt, testTag1, 50 / 6.0) &&
GlobalTagThrottlerTesting::targetRateIsNear(gtt, testTag2, 50 / 6.0) && gtt.busyReadTagCount() == 2;
});
futures.push_back(GlobalTagThrottlerTesting::updateGlobalTagThrottler(&globalTagThrottler, &storageServers));
futures.push_back(runClient(&globalTagThrottler, &storageServers, testTag1, 10.0, 6.0, OpType::READ, { 0, 1 }));
futures.push_back(runClient(&globalTagThrottler, &storageServers, testTag2, 10.0, 6.0, OpType::READ, { 1, 2 }));
state Future<Void> monitor = monitorActor(&globalTagThrottler, [testTag1, testTag2](auto& gtt) {
return targetRateIsNear(gtt, testTag1, 50 / 6.0) && targetRateIsNear(gtt, testTag2, 50 / 6.0) &&
gtt.busyReadTagCount() == 2;
});
futures.push_back(updateGlobalTagThrottler(&globalTagThrottler, &storageServers));
wait(timeoutError(waitForAny(futures) || monitor, 600.0));
return Void();
}
// 3 storage servers can handle 50 bytes/second each.
// Total quota is set to 100 bytes/second for each tag.
// One client attempts 10 6-byte read transactions per second, all directed towards a single storage server.
// Another client, using a different tag, attempts 10 6-byte read transactions split across the other two storage
// servers. Target rates adjust to 50/6 and 100/6 transactions per second for the two clients, based on the capacities
// of the
// storage servers being accessed.
TEST_CASE("/GlobalTagThrottler/MultiTagActiveThrottling3") {
state GlobalTagThrottler globalTagThrottler(Database{}, UID{});
state GlobalTagThrottlerTesting::StorageServerCollection storageServers(3, 50);
state GlobalTagThrottler globalTagThrottler(Database{}, UID{}, 0);
state StorageServerCollection storageServers(3, 50);
state ThrottleApi::TagQuotaValue tagQuotaValue1;
state ThrottleApi::TagQuotaValue tagQuotaValue2;
TransactionTag testTag1 = "sampleTag1"_sr;
@ -1051,35 +1116,35 @@ TEST_CASE("/GlobalTagThrottler/MultiTagActiveThrottling3") {
globalTagThrottler.setQuota(testTag1, tagQuotaValue1);
globalTagThrottler.setQuota(testTag2, tagQuotaValue2);
std::vector<Future<Void>> futures;
futures.push_back(GlobalTagThrottlerTesting::runClient(
&globalTagThrottler, &storageServers, testTag1, 10.0, 6.0, GlobalTagThrottlerTesting::OpType::READ, { 0 }));
futures.push_back(GlobalTagThrottlerTesting::runClient(
&globalTagThrottler, &storageServers, testTag2, 10.0, 6.0, GlobalTagThrottlerTesting::OpType::READ, { 1, 2 }));
state Future<Void> monitor =
GlobalTagThrottlerTesting::monitor(&globalTagThrottler, [testTag1, testTag2](auto& gtt) {
return GlobalTagThrottlerTesting::targetRateIsNear(gtt, testTag1, 50 / 6.0) &&
GlobalTagThrottlerTesting::targetRateIsNear(gtt, testTag2, 100 / 6.0) && gtt.busyReadTagCount() == 1;
});
futures.push_back(GlobalTagThrottlerTesting::updateGlobalTagThrottler(&globalTagThrottler, &storageServers));
futures.push_back(runClient(&globalTagThrottler, &storageServers, testTag1, 10.0, 6.0, OpType::READ, { 0 }));
futures.push_back(runClient(&globalTagThrottler, &storageServers, testTag2, 10.0, 6.0, OpType::READ, { 1, 2 }));
state Future<Void> monitor = monitorActor(&globalTagThrottler, [testTag1, testTag2](auto& gtt) {
return targetRateIsNear(gtt, testTag1, 50 / 6.0) && targetRateIsNear(gtt, testTag2, 100 / 6.0) &&
gtt.busyReadTagCount() == 1;
});
futures.push_back(updateGlobalTagThrottler(&globalTagThrottler, &storageServers));
wait(timeoutError(waitForAny(futures) || monitor, 600.0));
return Void();
}
// 10 storage servers can serve 5 bytes/second each.
// Total quota is set to 100 bytes/second.
// Reserved quota is set to 70 bytes/second.
// A client attempts to execute 10 6-byte read transactions per second.
// Despite the storage server only having capacity to serve 50/6 transactions per second,
// the reserved quota will ensure the target rate adjusts to 70/6 transactions per second.
TEST_CASE("/GlobalTagThrottler/ReservedQuota") {
state GlobalTagThrottler globalTagThrottler(Database{}, UID{});
state GlobalTagThrottlerTesting::StorageServerCollection storageServers(10, 5);
state GlobalTagThrottler globalTagThrottler(Database{}, UID{}, 0);
state StorageServerCollection storageServers(10, 5);
state ThrottleApi::TagQuotaValue tagQuotaValue;
TransactionTag testTag = "sampleTag1"_sr;
tagQuotaValue.totalQuota = 100.0;
tagQuotaValue.reservedQuota = 70.0;
globalTagThrottler.setQuota(testTag, tagQuotaValue);
state Future<Void> client = GlobalTagThrottlerTesting::runClient(
&globalTagThrottler, &storageServers, testTag, 10.0, 6.0, GlobalTagThrottlerTesting::OpType::READ);
state Future<Void> monitor = GlobalTagThrottlerTesting::monitor(&globalTagThrottler, [testTag](auto& gtt) {
return GlobalTagThrottlerTesting::targetRateIsNear(gtt, testTag, 70 / 6.0);
});
state Future<Void> updater =
GlobalTagThrottlerTesting::updateGlobalTagThrottler(&globalTagThrottler, &storageServers);
state Future<Void> client = runClient(&globalTagThrottler, &storageServers, testTag, 10.0, 6.0, OpType::READ);
state Future<Void> monitor =
monitorActor(&globalTagThrottler, [testTag](auto& gtt) { return targetRateIsNear(gtt, testTag, 70 / 6.0); });
state Future<Void> updater = updateGlobalTagThrottler(&globalTagThrottler, &storageServers);
wait(timeoutError(monitor || client || updater, 600.0));
return Void();
}
@ -1087,17 +1152,13 @@ TEST_CASE("/GlobalTagThrottler/ReservedQuota") {
// Test that tags are expired iff a sufficient amount of time has passed since the
// last transaction with that tag
TEST_CASE("/GlobalTagThrottler/ExpireTags") {
state GlobalTagThrottler globalTagThrottler(Database{}, UID{});
state GlobalTagThrottlerTesting::StorageServerCollection storageServers(10, 5);
state GlobalTagThrottler globalTagThrottler(Database{}, UID{}, 0);
state StorageServerCollection storageServers(10, 5);
TransactionTag testTag = "sampleTag1"_sr;
state Future<Void> client =
timeout(GlobalTagThrottlerTesting::runClient(
&globalTagThrottler, &storageServers, testTag, 10.0, 6.0, GlobalTagThrottlerTesting::OpType::READ),
60.0,
Void());
state Future<Void> updater = timeout(
GlobalTagThrottlerTesting::updateGlobalTagThrottler(&globalTagThrottler, &storageServers), 60.0, Void());
timeout(runClient(&globalTagThrottler, &storageServers, testTag, 10.0, 6.0, OpType::READ), 60.0, Void());
state Future<Void> updater = timeout(updateGlobalTagThrottler(&globalTagThrottler, &storageServers), 60.0, Void());
wait(client && updater);
client.cancel();
updater.cancel();
@ -1113,17 +1174,43 @@ TEST_CASE("/GlobalTagThrottler/ExpireTags") {
// Test that the number of tags tracked does not grow beyond SERVER_KNOBS->GLOBAL_TAG_THROTTLING_MAX_TAGS_TRACKED
TEST_CASE("/GlobalTagThrottler/TagLimit") {
state GlobalTagThrottler globalTagThrottler(Database{}, UID{});
state GlobalTagThrottlerTesting::StorageServerCollection storageServers(10, 5);
state GlobalTagThrottler globalTagThrottler(Database{}, UID{}, 0);
state StorageServerCollection storageServers(10, 5);
std::vector<Future<Void>> futures;
for (int i = 0; i < 2 * SERVER_KNOBS->GLOBAL_TAG_THROTTLING_MAX_TAGS_TRACKED; ++i) {
Arena arena;
TransactionTag tag = makeString(8, arena);
deterministicRandom()->randomBytes(mutateString(tag), tag.size());
futures.push_back(GlobalTagThrottlerTesting::runClient(
&globalTagThrottler, &storageServers, tag, 1.0, 6.0, GlobalTagThrottlerTesting::OpType::READ));
futures.push_back(runClient(&globalTagThrottler, &storageServers, tag, 1.0, 6.0, OpType::READ));
}
wait(timeout(waitForAll(futures), 60.0, Void()));
ASSERT_EQ(globalTagThrottler.tagsTracked(), SERVER_KNOBS->GLOBAL_TAG_THROTTLING_MAX_TAGS_TRACKED);
return Void();
}
// 9 storage servers can handle 100 bytes/second each.
// 1 unhealthy storage server can only handle 1 byte/second.
// Total quota is set to 100 bytes/second.
// Client attempts 5 6-byte transactions per second.
// Target rate adjusts to 100/6 transactions per second, ignoring the worst storage server.
// Then, a second storage server becomes unhealthy and can only handle 1 byte/second.
// Target rate adjusts down to 1/6 transactions per second, because only one bad zone can be ignored.
TEST_CASE("/GlobalTagThrottler/IgnoreWorstZone") {
state GlobalTagThrottler globalTagThrottler(Database{}, UID{}, 1);
state StorageServerCollection storageServers(10, 100);
state TransactionTag testTag = "sampleTag1"_sr;
storageServers.setCapacity(0, 1);
ThrottleApi::TagQuotaValue tagQuotaValue;
tagQuotaValue.totalQuota = 100.0;
globalTagThrottler.setQuota(testTag, tagQuotaValue);
state Future<Void> client = runClient(&globalTagThrottler, &storageServers, testTag, 5.0, 6.0, OpType::READ);
state Future<Void> monitor = monitorActor(
&globalTagThrottler, [](auto& gtt) { return targetRateIsNear(gtt, "sampleTag1"_sr, 100.0 / 6.0); });
state Future<Void> updater = updateGlobalTagThrottler(&globalTagThrottler, &storageServers);
wait(timeoutError(monitor || client || updater, 600.0));
storageServers.setCapacity(1, 1);
monitor =
monitorActor(&globalTagThrottler, [](auto& gtt) { return targetRateIsNear(gtt, "sampleTag1"_sr, 1.0 / 6.0); });
wait(timeoutError(monitor || client || updater, 600.0));
return Void();
}

View File

@ -183,6 +183,8 @@ struct GrvProxyData {
Version version;
Version minKnownCommittedVersion; // we should ask master for this version.
GrvProxyTransactionTagThrottler tagThrottler;
// Cache of the latest commit versions of storage servers.
VersionVector ssVersionVectorCache;
@ -195,6 +197,7 @@ struct GrvProxyData {
if (newLatencyBandConfig.present()) {
for (auto band : newLatencyBandConfig.get().grvConfig.bands) {
stats.grvLatencyBands.addThreshold(band);
tagThrottler.addLatencyBandThreshold(band);
}
}
}
@ -213,7 +216,8 @@ struct GrvProxyData {
dbgid,
SERVER_KNOBS->LATENCY_METRICS_LOGGING_INTERVAL,
SERVER_KNOBS->LATENCY_SAMPLE_SIZE),
updateCommitRequests(0), lastCommitTime(0), version(0), minKnownCommittedVersion(invalidVersion) {}
updateCommitRequests(0), lastCommitTime(0), version(0), minKnownCommittedVersion(invalidVersion),
tagThrottler(SERVER_KNOBS->PROXY_MAX_TAG_THROTTLE_DURATION) {}
};
ACTOR Future<Void> healthMetricsRequestServer(GrvProxyInterface grvProxy,
@ -363,7 +367,6 @@ ACTOR Future<Void> getRate(UID myID,
GetHealthMetricsReply* detailedHealthMetricsReply,
TransactionTagMap<uint64_t>* transactionTagCounter,
PrioritizedTransactionTagMap<ClientTagThrottleLimits>* clientThrottledTags,
GrvProxyTransactionTagThrottler* tagThrottler,
GrvProxyStats* stats,
GrvProxyData* proxyData) {
state Future<Void> nextRequestTimer = Never();
@ -424,7 +427,7 @@ ACTOR Future<Void> getRate(UID myID,
*clientThrottledTags = std::move(rep.clientThrottledTags.get());
}
if (rep.proxyThrottledTags.present()) {
tagThrottler->updateRates(rep.proxyThrottledTags.get());
proxyData->tagThrottler.updateRates(rep.proxyThrottledTags.get());
}
}
when(wait(leaseTimeout)) {
@ -721,7 +724,7 @@ ACTOR Future<Void> sendGrvReplies(Future<GetReadVersionReply> replyFuture,
reply.proxyId = grvProxyData->dbgid;
reply.proxyTagThrottledDuration = request.proxyTagThrottledDuration;
if (!request.tags.empty()) {
if (request.isTagged()) {
auto& priorityThrottledTags = clientThrottledTags[request.priority];
for (auto tag : request.tags) {
auto tagItr = priorityThrottledTags.find(tag.first);
@ -824,7 +827,6 @@ ACTOR static Future<Void> transactionStarter(GrvProxyInterface proxy,
state int64_t batchTransactionCount = 0;
state GrvTransactionRateInfo normalRateInfo(10);
state GrvTransactionRateInfo batchRateInfo(0);
state GrvProxyTransactionTagThrottler tagThrottler;
state Deque<GetReadVersionRequest> systemQueue;
state Deque<GetReadVersionRequest> defaultQueue;
@ -850,7 +852,6 @@ ACTOR static Future<Void> transactionStarter(GrvProxyInterface proxy,
detailedHealthMetricsReply,
&transactionTagCounter,
&clientThrottledTags,
&tagThrottler,
&grvProxyData->stats,
grvProxyData));
addActor.send(queueGetReadVersionRequests(db,
@ -865,7 +866,7 @@ ACTOR static Future<Void> transactionStarter(GrvProxyInterface proxy,
&grvProxyData->stats,
&batchRateInfo,
&transactionTagCounter,
&tagThrottler));
&grvProxyData->tagThrottler));
while (std::find(db->get().client.grvProxies.begin(), db->get().client.grvProxies.end(), proxy) ==
db->get().client.grvProxies.end()) {
@ -888,7 +889,7 @@ ACTOR static Future<Void> transactionStarter(GrvProxyInterface proxy,
elapsed = 1e-15;
}
tagThrottler.releaseTransactions(elapsed, defaultQueue, batchQueue);
grvProxyData->tagThrottler.releaseTransactions(elapsed, defaultQueue, batchQueue);
normalRateInfo.startReleaseWindow();
batchRateInfo.startReleaseWindow();

View File

@ -20,17 +20,21 @@
#include "fdbclient/Knobs.h"
#include "fdbserver/GrvProxyTransactionTagThrottler.h"
#include "fdbserver/Knobs.h"
#include "flow/UnitTest.h"
#include "flow/actorcompiler.h" // must be last include
uint64_t GrvProxyTransactionTagThrottler::DelayedRequest::lastSequenceNumber = 0;
void GrvProxyTransactionTagThrottler::DelayedRequest::updateProxyTagThrottledDuration() {
void GrvProxyTransactionTagThrottler::DelayedRequest::updateProxyTagThrottledDuration(
LatencyBandsMap& latencyBandsMap) {
req.proxyTagThrottledDuration = now() - startTime;
auto const& [tag, count] = *req.tags.begin();
latencyBandsMap.addMeasurement(tag, req.proxyTagThrottledDuration, count);
}
bool GrvProxyTransactionTagThrottler::DelayedRequest::isMaxThrottled() const {
return now() - startTime > CLIENT_KNOBS->PROXY_MAX_TAG_THROTTLE_DURATION;
bool GrvProxyTransactionTagThrottler::DelayedRequest::isMaxThrottled(double maxThrottleDuration) const {
return now() - startTime > maxThrottleDuration;
}
void GrvProxyTransactionTagThrottler::TagQueue::setRate(double rate) {
@ -41,20 +45,27 @@ void GrvProxyTransactionTagThrottler::TagQueue::setRate(double rate) {
}
}
bool GrvProxyTransactionTagThrottler::TagQueue::isMaxThrottled() const {
return !requests.empty() && requests.front().isMaxThrottled();
bool GrvProxyTransactionTagThrottler::TagQueue::isMaxThrottled(double maxThrottleDuration) const {
return !requests.empty() && requests.front().isMaxThrottled(maxThrottleDuration);
}
void GrvProxyTransactionTagThrottler::TagQueue::rejectRequests() {
void GrvProxyTransactionTagThrottler::TagQueue::rejectRequests(LatencyBandsMap& latencyBandsMap) {
CODE_PROBE(true, "GrvProxyTransactionTagThrottler rejecting requests");
while (!requests.empty()) {
auto& delayedReq = requests.front();
delayedReq.updateProxyTagThrottledDuration();
delayedReq.updateProxyTagThrottledDuration(latencyBandsMap);
delayedReq.req.reply.sendError(proxy_tag_throttled());
requests.pop_front();
}
}
GrvProxyTransactionTagThrottler::GrvProxyTransactionTagThrottler(double maxThrottleDuration)
: maxThrottleDuration(maxThrottleDuration),
latencyBandsMap("GrvProxyTagThrottler",
deterministicRandom()->randomUniqueID(),
SERVER_KNOBS->GLOBAL_TAG_THROTTLING_PROXY_LOGGING_INTERVAL,
SERVER_KNOBS->GLOBAL_TAG_THROTTLING_MAX_TAGS_TRACKED) {}
void GrvProxyTransactionTagThrottler::updateRates(TransactionTagMap<double> const& newRates) {
for (const auto& [tag, rate] : newRates) {
auto it = queues.find(tag);
@ -160,17 +171,17 @@ void GrvProxyTransactionTagThrottler::releaseTransactions(double elapsed,
// Cannot release any more transaction from this tag (don't push the tag queue handle back into
// pqOfQueues)
CODE_PROBE(true, "GrvProxyTransactionTagThrottler throttling transaction");
if (tagQueueHandle.queue->isMaxThrottled()) {
if (tagQueueHandle.queue->isMaxThrottled(maxThrottleDuration)) {
// Requests in this queue have been throttled too long and errors
// should be sent to clients.
tagQueueHandle.queue->rejectRequests();
tagQueueHandle.queue->rejectRequests(latencyBandsMap);
}
break;
} else {
if (tagQueueHandle.nextSeqNo < nextQueueSeqNo) {
// Releasing transaction
*(tagQueueHandle.numReleased) += count;
delayedReq.updateProxyTagThrottledDuration();
delayedReq.updateProxyTagThrottledDuration(latencyBandsMap);
if (delayedReq.req.priority == TransactionPriority::BATCH) {
outBatchPriority.push_back(delayedReq.req);
} else if (delayedReq.req.priority == TransactionPriority::DEFAULT) {
@ -209,7 +220,12 @@ void GrvProxyTransactionTagThrottler::releaseTransactions(double elapsed,
ASSERT_EQ(transactionsReleased.capacity(), transactionsReleasedInitialCapacity);
}
uint32_t GrvProxyTransactionTagThrottler::size() {
void GrvProxyTransactionTagThrottler::addLatencyBandThreshold(double value) {
CODE_PROBE(size() > 0, "GrvProxyTransactionTagThrottler adding latency bands while actively throttling");
latencyBandsMap.addThreshold(value);
}
uint32_t GrvProxyTransactionTagThrottler::size() const {
return queues.size();
}
@ -290,6 +306,7 @@ ACTOR static Future<Void> mockServer(GrvProxyTransactionTagThrottler* throttler)
outBatchPriority.front().reply.send(GetReadVersionReply{});
outBatchPriority.pop_front();
}
TraceEvent("HERE_ServerProcessing").detail("Size", outDefaultPriority.size());
while (!outDefaultPriority.empty()) {
outDefaultPriority.front().reply.send(GetReadVersionReply{});
outDefaultPriority.pop_front();
@ -314,7 +331,7 @@ static bool isNear(double desired, int64_t actual) {
// Rate limit set at 10, but client attempts 20 transactions per second.
// Client should be throttled to only 10 transactions per second.
TEST_CASE("/GrvProxyTransactionTagThrottler/Simple") {
state GrvProxyTransactionTagThrottler throttler;
state GrvProxyTransactionTagThrottler throttler(5.0);
state TagSet tagSet;
state TransactionTagMap<uint32_t> counters;
{
@ -334,7 +351,7 @@ TEST_CASE("/GrvProxyTransactionTagThrottler/Simple") {
// Clients share the available 30 transaction/second budget
TEST_CASE("/GrvProxyTransactionTagThrottler/MultiClient") {
state GrvProxyTransactionTagThrottler throttler;
state GrvProxyTransactionTagThrottler throttler(5.0);
state TagSet tagSet;
state TransactionTagMap<uint32_t> counters;
{
@ -359,7 +376,7 @@ TEST_CASE("/GrvProxyTransactionTagThrottler/MultiClient") {
// Test processing GetReadVersionRequests that batch several transactions
TEST_CASE("/GrvProxyTransactionTagThrottler/Batch") {
state GrvProxyTransactionTagThrottler throttler;
state GrvProxyTransactionTagThrottler throttler(5.0);
state TagSet tagSet;
state TransactionTagMap<uint32_t> counters;
{
@ -380,7 +397,7 @@ TEST_CASE("/GrvProxyTransactionTagThrottler/Batch") {
// Tests cleanup of tags that are no longer throttled.
TEST_CASE("/GrvProxyTransactionTagThrottler/Cleanup1") {
GrvProxyTransactionTagThrottler throttler;
GrvProxyTransactionTagThrottler throttler(5.0);
for (int i = 0; i < 1000; ++i) {
auto const tag = getRandomTag();
TransactionTagMap<double> rates;
@ -393,7 +410,7 @@ TEST_CASE("/GrvProxyTransactionTagThrottler/Cleanup1") {
// Tests cleanup of tags once queues have been emptied
TEST_CASE("/GrvProxyTransactionTagThrottler/Cleanup2") {
GrvProxyTransactionTagThrottler throttler;
GrvProxyTransactionTagThrottler throttler(5.0);
{
GetReadVersionRequest req;
req.tags["sampleTag"_sr] = 1;
@ -417,7 +434,7 @@ TEST_CASE("/GrvProxyTransactionTagThrottler/Cleanup2") {
// Tests that unthrottled transactions are released in FIFO order, even when they
// have different tags
TEST_CASE("/GrvProxyTransactionTagThrottler/Fifo") {
state GrvProxyTransactionTagThrottler throttler;
state GrvProxyTransactionTagThrottler throttler(5.0);
state Future<Void> server = mockServer(&throttler);
wait(mockFifoClient(&throttler));
return Void();

View File

@ -48,6 +48,7 @@
#endif
#include "fdbclient/SystemData.h"
#include "fdbserver/CoroFlow.h"
#include "fdbserver/RocksDBLogForwarder.h"
#include "flow/ActorCollection.h"
#include "flow/flow.h"
#include "flow/IThreadPool.h"
@ -202,6 +203,13 @@ rocksdb::DBOptions SharedRocksDBState::initialDbOptions() {
options.statistics->set_stats_level(rocksdb::kExceptHistogramOrTimers);
options.db_log_dir = SERVER_KNOBS->LOG_DIRECTORY;
if (SERVER_KNOBS->ROCKSDB_MUTE_LOGS) {
options.info_log = std::make_shared<NullRocksDBLogForwarder>();
} else {
options.info_log = std::make_shared<RocksDBLogForwarder>(id, options.info_log_level);
}
return options;
}

View File

@ -224,7 +224,7 @@ Error statusToError(const rocksdb::Status& s) {
if (s.IsIOError()) {
return io_error();
} else if (s.IsTimedOut()) {
return transaction_too_old();
return key_value_store_deadline_exceeded();
} else {
return unknown_error();
}
@ -518,17 +518,10 @@ int readRangeInDb(PhysicalShard* shard, const KeyRangeRef& range, int rowLimit,
int accumulatedRows = 0;
int accumulatedBytes = 0;
// TODO: Pass read timeout.
const int readRangeTimeout = SERVER_KNOBS->ROCKSDB_READ_RANGE_TIMEOUT;
rocksdb::Status s;
auto options = getReadOptions();
// TODO: define single shard read timeout.
const uint64_t deadlineMircos = shard->db->GetEnv()->NowMicros() + readRangeTimeout * 1000000;
options.deadline = std::chrono::microseconds(deadlineMircos / 1000000);
// When using a prefix extractor, ensure that keys are returned in order even if they cross
// a prefix boundary.
options.auto_prefix_mode = (SERVER_KNOBS->ROCKSDB_PREFIX_LEN > 0);
if (rowLimit >= 0) {
ReadIterator readIter = shard->readIterPool->getIterator();
auto cursor = readIter.iter;
@ -2023,7 +2016,7 @@ struct ShardedRocksDBKeyValueStore : IKeyValueStore {
.detail("Error", "Read value request timedout")
.detail("Method", "ReadValueAction")
.detail("Timeout value", readValueTimeout);
a.result.sendError(transaction_too_old());
a.result.sendError(key_value_store_deadline_exceeded());
return;
}
@ -2101,7 +2094,7 @@ struct ShardedRocksDBKeyValueStore : IKeyValueStore {
.detail("Error", "Read value prefix request timedout")
.detail("Method", "ReadValuePrefixAction")
.detail("Timeout value", readValuePrefixTimeout);
a.result.sendError(transaction_too_old());
a.result.sendError(key_value_store_deadline_exceeded());
return;
}
@ -2176,7 +2169,7 @@ struct ShardedRocksDBKeyValueStore : IKeyValueStore {
.detail("Error", "Read range request timedout")
.detail("Method", "ReadRangeAction")
.detail("Timeout value", readRangeTimeout);
a.result.sendError(transaction_too_old());
a.result.sendError(key_value_store_deadline_exceeded());
return;
}
@ -2217,6 +2210,15 @@ struct ShardedRocksDBKeyValueStore : IKeyValueStore {
if (result.size() >= abs(a.rowLimit) || accumulatedBytes >= a.byteLimit) {
break;
}
if (timer_monotonic() - a.startTime > readRangeTimeout) {
TraceEvent(SevInfo, "ShardedRocksDBTimeout")
.detail("Action", "ReadRange")
.detail("ShardsRead", numShards)
.detail("BytesRead", accumulatedBytes);
a.result.sendError(key_value_store_deadline_exceeded());
return;
}
}
result.more =
@ -2229,6 +2231,9 @@ struct ShardedRocksDBKeyValueStore : IKeyValueStore {
double currTime = timer_monotonic();
rocksDBMetrics->getReadRangeActionHistogram(threadIndex)->sampleSeconds(currTime - readBeginTime);
rocksDBMetrics->getReadRangeLatencyHistogram(threadIndex)->sampleSeconds(currTime - a.startTime);
if (a.shardRanges.size() > 1) {
TraceEvent(SevInfo, "ShardedRocksDB").detail("ReadRangeShards", a.shardRanges.size());
}
}
sample();

View File

@ -0,0 +1,131 @@
/*
* LatencyBandsMap.actor.cpp
*
* This source file is part of the FoundationDB open source project
*
* Copyright 2013-2022 Apple Inc. and the FoundationDB project authors
*
* Licensed under the Apache License, Version 2.0 (the "License");
* you may not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
#include "fdbserver/LatencyBandsMap.h"
#include "flow/UnitTest.h"
#include "flow/actorcompiler.h" // must be last include
class LatencyBandsMapImpl {
public:
ACTOR static Future<Void> expireOldTagsActor(LatencyBandsMap* self) {
loop {
wait(delay(5.0));
for (auto it = self->map.begin(); it != self->map.end();) {
const auto& [tag, expirableBands] = *it;
if (now() - expirableBands.lastUpdated > SERVER_KNOBS->GLOBAL_TAG_THROTTLING_TAG_EXPIRE_AFTER) {
CODE_PROBE(true, "LatencyBandsMap erasing expired tag");
it = self->map.erase(it);
} else {
++it;
}
}
}
}
};
LatencyBandsMap::ExpirableBands::ExpirableBands(LatencyBands&& bands)
: latencyBands(std::move(bands)), lastUpdated(now()) {}
Optional<LatencyBands*> LatencyBandsMap::getLatencyBands(TransactionTag tag) {
if (map.size() == maxSize && !map.count(tag)) {
CODE_PROBE(true, "LatencyBandsMap reached maxSize");
return {};
}
auto const [it, inserted] = map.try_emplace(
tag, LatencyBands(name, id, loggingInterval, [tag](auto& te) { te.detail("Tag", printable(tag)); }));
auto& expirableBands = it->second;
if (inserted) {
for (const auto& threshold : thresholds) {
expirableBands.latencyBands.addThreshold(threshold);
}
}
expirableBands.lastUpdated = now();
return &expirableBands.latencyBands;
}
LatencyBandsMap::LatencyBandsMap(std::string const& name, UID id, double loggingInterval, int maxSize)
: name(name), id(id), loggingInterval(loggingInterval), maxSize(maxSize) {
expireOldTags = LatencyBandsMapImpl::expireOldTagsActor(this);
}
void LatencyBandsMap::addMeasurement(TransactionTag tag, double value, int count) {
auto bands = getLatencyBands(tag);
if (bands.present()) {
bands.get()->addMeasurement(value, count);
}
}
void LatencyBandsMap::addThreshold(double value) {
thresholds.push_back(value);
for (auto& [tag, expirableBands] : map) {
expirableBands.latencyBands.addThreshold(value);
}
}
TEST_CASE("/fdbserver/LatencyBandsMap/Simple") {
state LatencyBandsMap latencyBandsMap("TestLatencyBandsMap", deterministicRandom()->randomUniqueID(), 10.0, 100);
state Standalone<VectorRef<TransactionTagRef>> tags;
tags.push_back_deep(tags.arena(), "a"_sr);
tags.push_back_deep(tags.arena(), "b"_sr);
tags.push_back_deep(tags.arena(), "c"_sr);
latencyBandsMap.addThreshold(0.1);
latencyBandsMap.addThreshold(0.2);
latencyBandsMap.addThreshold(0.4);
state int measurements = 0;
loop {
wait(delayJittered(0.1));
auto const tag = deterministicRandom()->randomChoice(tags);
latencyBandsMap.addMeasurement(tag, deterministicRandom()->random01());
if (++measurements == 1000) {
return Void();
}
}
}
TEST_CASE("/fdbserver/LatencyBandsMap/MaxSize") {
LatencyBandsMap latencyBandsMap("TestLatencyBandsMap", deterministicRandom()->randomUniqueID(), 10.0, 2);
latencyBandsMap.addMeasurement("a"_sr, deterministicRandom()->random01());
latencyBandsMap.addMeasurement("b"_sr, deterministicRandom()->random01());
latencyBandsMap.addMeasurement("c"_sr, deterministicRandom()->random01());
ASSERT_EQ(latencyBandsMap.size(), 2);
return Void();
}
TEST_CASE("/fdbserver/LatencyBandsMap/Expire") {
state LatencyBandsMap latencyBandsMap("TestLatencyBandsMap", deterministicRandom()->randomUniqueID(), 10.0, 100);
latencyBandsMap.addMeasurement("a"_sr, deterministicRandom()->random01());
latencyBandsMap.addMeasurement("b"_sr, deterministicRandom()->random01());
latencyBandsMap.addMeasurement("c"_sr, deterministicRandom()->random01());
latencyBandsMap.addThreshold(0.1);
latencyBandsMap.addThreshold(0.2);
latencyBandsMap.addThreshold(0.4);
ASSERT_EQ(latencyBandsMap.size(), 3);
state int waitIterations = 0;
loop {
wait(delay(1.0));
latencyBandsMap.addMeasurement("a"_sr, deterministicRandom()->random01());
latencyBandsMap.addMeasurement("b"_sr, deterministicRandom()->random01());
if (++waitIterations == 2 * SERVER_KNOBS->GLOBAL_TAG_THROTTLING_TAG_EXPIRE_AFTER) {
break;
}
}
ASSERT_EQ(latencyBandsMap.size(), 2);
return Void();
}

View File

@ -1287,7 +1287,7 @@ ACTOR static Future<Void> startMoveShards(Database occ,
TraceEvent(SevVerbose, "StartMoveShardsFoundDataMove", relocationIntervalId)
.detail("DataMoveID", dataMoveId)
.detail("DataMove", dataMove.toString());
ASSERT(dataMove.range.begin == keys.begin);
ASSERT(!dataMove.ranges.empty() && dataMove.ranges.front().begin == keys.begin);
if (dataMove.getPhase() == DataMoveMetaData::Deleting) {
TraceEvent(SevVerbose, "StartMoveShardsDataMove", relocationIntervalId)
.detail("DataMoveBeingDeleted", dataMoveId);
@ -1296,10 +1296,10 @@ ACTOR static Future<Void> startMoveShards(Database occ,
if (dataMove.getPhase() == DataMoveMetaData::Running) {
TraceEvent(SevVerbose, "StartMoveShardsDataMove", relocationIntervalId)
.detail("DataMoveAlreadyCommitted", dataMoveId);
ASSERT(keys == dataMove.range);
ASSERT(keys == dataMove.ranges.front());
return Void();
}
begin = dataMove.range.end;
begin = dataMove.ranges.front().end;
} else {
dataMove.id = dataMoveId;
TraceEvent(SevVerbose, "StartMoveKeysNewDataMove", relocationIntervalId)
@ -1441,7 +1441,8 @@ ACTOR static Future<Void> startMoveShards(Database occ,
&tr, serverKeysPrefixFor(servers[i]), currentKeys, allKeys, serverKeysValue(dataMoveId)));
}
dataMove.range = KeyRangeRef(keys.begin, currentKeys.end);
dataMove.ranges.clear();
dataMove.ranges.push_back(KeyRangeRef(keys.begin, currentKeys.end));
dataMove.dest.insert(servers.begin(), servers.end());
}
@ -1471,7 +1472,7 @@ ACTOR static Future<Void> startMoveShards(Database occ,
.detail("DataMoveKey", dataMoveKeyFor(dataMoveId))
.detail("CommitVersion", tr.getCommittedVersion())
.detail("DeltaRange", currentKeys.toString())
.detail("Range", dataMove.range.toString())
.detail("Range", describe(dataMove.ranges))
.detail("DataMove", dataMove.toString());
dataMove = DataMoveMetaData();
@ -1628,7 +1629,8 @@ ACTOR static Future<Void> finishMoveShards(Database occ,
throw data_move_cancelled();
}
ASSERT(dataMove.getPhase() == DataMoveMetaData::Running);
range = dataMove.range;
ASSERT(!dataMove.ranges.empty());
range = dataMove.ranges.front();
} else {
TraceEvent(SevWarn, "FinishMoveShardsDataMoveDeleted", relocationIntervalId)
.detail("DataMoveID", dataMoveId);
@ -1766,7 +1768,7 @@ ACTOR static Future<Void> finishMoveShards(Database occ,
wait(waitForAll(actors));
if (range.end == dataMove.range.end) {
if (range.end == dataMove.ranges.front().end) {
tr.clear(dataMoveKeyFor(dataMoveId));
complete = true;
TraceEvent(SevVerbose, "FinishMoveShardsDeleteMetaData", dataMoveId)
@ -1776,7 +1778,7 @@ ACTOR static Future<Void> finishMoveShards(Database occ,
.detail("DataMoveID", dataMoveId)
.detail("CurrentRange", range)
.detail("NewDataMoveMetaData", dataMove.toString());
dataMove.range = KeyRangeRef(range.end, dataMove.range.end);
dataMove.ranges.front() = KeyRangeRef(range.end, dataMove.ranges.front().end);
tr.set(dataMoveKeyFor(dataMoveId), dataMoveValue(dataMove));
}
@ -2229,9 +2231,10 @@ ACTOR Future<Void> removeKeysFromFailedServer(Database cx,
Optional<Value> val = wait(tr.get(dataMoveKeyFor(destId)));
if (val.present()) {
state DataMoveMetaData dataMove = decodeDataMoveValue(val.get());
ASSERT(!dataMove.ranges.empty());
TraceEvent(SevVerbose, "RemoveRangeFoundDataMove", serverID)
.detail("DataMoveMetaData", dataMove.toString());
if (range == dataMove.range) {
if (range == dataMove.ranges.front()) {
tr.clear(dataMoveKeyFor(destId));
} else {
dataMove.setPhase(DataMoveMetaData::Deleting);
@ -2350,10 +2353,11 @@ ACTOR Future<Void> cleanUpDataMove(Database occ,
Optional<Value> val = wait(tr.get(dataMoveKeyFor(dataMoveId)));
if (val.present()) {
dataMove = decodeDataMoveValue(val.get());
ASSERT(!dataMove.ranges.empty());
TraceEvent(SevVerbose, "CleanUpDataMoveMetaData", dataMoveId)
.detail("DataMoveID", dataMoveId)
.detail("DataMoveMetaData", dataMove.toString());
range = dataMove.range;
range = dataMove.ranges.front();
ASSERT(!range.empty());
} else {
TraceEvent(SevDebug, "CleanUpDataMoveNotExist", dataMoveId).detail("DataMoveID", dataMoveId);
@ -2397,6 +2401,9 @@ ACTOR Future<Void> cleanUpDataMove(Database occ,
.detail("ReadVersion", tr.getReadVersion().get());
if (destId != dataMoveId) {
for (const auto& uid : dest) {
physicalShardMap[uid].push_back(Shard(rangeIntersectKeys, destId));
}
TraceEvent(SevVerbose, "CleanUpDataMoveSkipShard", dataMoveId)
.detail("DataMoveID", dataMoveId)
.detail("ShardRange", rangeIntersectKeys)
@ -2419,14 +2426,14 @@ ACTOR Future<Void> cleanUpDataMove(Database occ,
currentShards[i + 1].value);
}
if (range.end == dataMove.range.end) {
if (range.end == dataMove.ranges.front().end) {
tr.clear(dataMoveKeyFor(dataMoveId));
complete = true;
TraceEvent(SevVerbose, "CleanUpDataMoveDeleteMetaData", dataMoveId)
.detail("DataMoveID", dataMove.toString());
} else {
dataMove.range = KeyRangeRef(range.end, dataMove.range.end);
dataMove.ranges.front() = KeyRangeRef(range.end, dataMove.ranges.front().end);
dataMove.setPhase(DataMoveMetaData::Deleting);
tr.set(dataMoveKeyFor(dataMoveId), dataMoveValue(dataMove));
TraceEvent(SevVerbose, "CleanUpDataMovePartial", dataMoveId)

View File

@ -637,7 +637,7 @@ Ratekeeper::Ratekeeper(UID id, Database db)
SERVER_KNOBS->TARGET_BW_LAG_BATCH),
maxVersion(0), blobWorkerTime(now()), unblockedAssignmentTime(now()) {
if (SERVER_KNOBS->GLOBAL_TAG_THROTTLING) {
tagThrottler = std::make_unique<GlobalTagThrottler>(db, id);
tagThrottler = std::make_unique<GlobalTagThrottler>(db, id, SERVER_KNOBS->MAX_MACHINES_FALLING_BEHIND);
} else {
tagThrottler = std::make_unique<TagThrottler>(db, id);
}
@ -695,17 +695,16 @@ void Ratekeeper::updateRate(RatekeeperLimits* limits) {
limitReason_t ssLimitReason = limitReason_t::unlimited;
int64_t minFreeSpace =
std::max(SERVER_KNOBS->MIN_AVAILABLE_SPACE,
(int64_t)(SERVER_KNOBS->MIN_AVAILABLE_SPACE_RATIO * ss.smoothTotalSpace.smoothTotal()));
int64_t minFreeSpace = std::max(SERVER_KNOBS->MIN_AVAILABLE_SPACE,
(int64_t)(SERVER_KNOBS->MIN_AVAILABLE_SPACE_RATIO * ss.getSmoothTotalSpace()));
worstFreeSpaceStorageServer =
std::min(worstFreeSpaceStorageServer, (int64_t)ss.smoothFreeSpace.smoothTotal() - minFreeSpace);
std::min(worstFreeSpaceStorageServer, (int64_t)ss.getSmoothFreeSpace() - minFreeSpace);
int64_t springBytes = std::max<int64_t>(
1, std::min<int64_t>(limits->storageSpringBytes, (ss.smoothFreeSpace.smoothTotal() - minFreeSpace) * 0.2));
int64_t targetBytes = std::max<int64_t>(
1, std::min(limits->storageTargetBytes, (int64_t)ss.smoothFreeSpace.smoothTotal() - minFreeSpace));
1, std::min<int64_t>(limits->storageSpringBytes, (ss.getSmoothFreeSpace() - minFreeSpace) * 0.2));
int64_t targetBytes =
std::max<int64_t>(1, std::min(limits->storageTargetBytes, (int64_t)ss.getSmoothFreeSpace() - minFreeSpace));
if (targetBytes != limits->storageTargetBytes) {
if (minFreeSpace == SERVER_KNOBS->MIN_AVAILABLE_SPACE) {
ssLimitReason = limitReason_t::storage_server_min_free_space;
@ -716,8 +715,8 @@ void Ratekeeper::updateRate(RatekeeperLimits* limits) {
TraceEvent("RatekeeperLimitReasonDetails")
.detail("Reason", ssLimitReason)
.detail("SSID", ss.id)
.detail("SSSmoothTotalSpace", ss.smoothTotalSpace.smoothTotal())
.detail("SSSmoothFreeSpace", ss.smoothFreeSpace.smoothTotal())
.detail("SSSmoothTotalSpace", ss.getSmoothTotalSpace())
.detail("SSSmoothFreeSpace", ss.getSmoothFreeSpace())
.detail("TargetBytes", targetBytes)
.detail("LimitsStorageTargetBytes", limits->storageTargetBytes)
.detail("MinFreeSpace", minFreeSpace);
@ -744,7 +743,7 @@ void Ratekeeper::updateRate(RatekeeperLimits* limits) {
addActor.send(tagThrottler->tryUpdateAutoThrottling(ss));
}
double inputRate = ss.smoothInputBytes.smoothRate();
double inputRate = ss.getSmoothInputBytesRate();
// inputRate = std::max( inputRate, actualTps / SERVER_KNOBS->MAX_TRANSACTIONS_PER_BYTE );
/*if( deterministicRandom()->random01() < 0.1 ) {
@ -753,15 +752,15 @@ void Ratekeeper::updateRate(RatekeeperLimits* limits) {
.detail("MinFreeSpace", minFreeSpace)
.detail("SpringBytes", springBytes)
.detail("TargetBytes", targetBytes)
.detail("SmoothTotalSpaceTotal", ss.smoothTotalSpace.smoothTotal())
.detail("SmoothFreeSpaceTotal", ss.smoothFreeSpace.smoothTotal())
.detail("SmoothTotalSpaceTotal", ss.getSmoothTotalSpace())
.detail("SmoothFreeSpaceTotal", ss.getSmoothFreeSpace())
.detail("LastReplyBytesInput", ss.lastReply.bytesInput)
.detail("SmoothDurableBytesTotal", ss.smoothDurableBytes.smoothTotal())
.detail("SmoothDurableBytesTotal", ss.getSmoothDurableBytes())
.detail("TargetRateRatio", targetRateRatio)
.detail("SmoothInputBytesRate", ss.smoothInputBytes.smoothRate())
.detail("SmoothInputBytesRate", ss.getSmoothInputBytesRate())
.detail("ActualTPS", actualTps)
.detail("InputRate", inputRate)
.detail("VerySmoothDurableBytesRate", ss.verySmoothDurableBytes.smoothRate())
.detail("VerySmoothDurableBytesRate", ss.getVerySmoothDurableBytesRate())
.detail("B", b);
}*/
@ -777,7 +776,7 @@ void Ratekeeper::updateRate(RatekeeperLimits* limits) {
if (targetRateRatio > 0 && inputRate > 0) {
ASSERT(inputRate != 0);
double smoothedRate =
std::max(ss.verySmoothDurableBytes.smoothRate(), actualTps / SERVER_KNOBS->MAX_TRANSACTIONS_PER_BYTE);
std::max(ss.getVerySmoothDurableBytesRate(), actualTps / SERVER_KNOBS->MAX_TRANSACTIONS_PER_BYTE);
double x = smoothedRate / (inputRate * targetRateRatio);
double lim = actualTps * x;
if (lim < limitTps) {
@ -790,17 +789,17 @@ void Ratekeeper::updateRate(RatekeeperLimits* limits) {
.detail("Reason", limitReason_t::storage_server_write_queue_size)
.detail("FromReason", ssLimitReason)
.detail("SSID", ss.id)
.detail("SSSmoothTotalSpace", ss.smoothTotalSpace.smoothTotal())
.detail("SSSmoothTotalSpace", ss.getSmoothTotalSpace())
.detail("LimitsStorageTargetBytes", limits->storageTargetBytes)
.detail("LimitsStorageSpringBytes", limits->storageSpringBytes)
.detail("SSSmoothFreeSpace", ss.smoothFreeSpace.smoothTotal())
.detail("SSSmoothFreeSpace", ss.getSmoothFreeSpace())
.detail("MinFreeSpace", minFreeSpace)
.detail("SSLastReplyBytesInput", ss.lastReply.bytesInput)
.detail("SSSmoothDurableBytes", ss.smoothDurableBytes.smoothTotal())
.detail("SSSmoothDurableBytes", ss.getSmoothDurableBytes())
.detail("StorageQueue", storageQueue)
.detail("TargetBytes", targetBytes)
.detail("SpringBytes", springBytes)
.detail("SSVerySmoothDurableBytesSmoothRate", ss.verySmoothDurableBytes.smoothRate())
.detail("SSVerySmoothDurableBytesRate", ss.getVerySmoothDurableBytesRate())
.detail("SmoothedRate", smoothedRate)
.detail("X", x)
.detail("ActualTps", actualTps)
@ -839,8 +838,7 @@ void Ratekeeper::updateRate(RatekeeperLimits* limits) {
continue;
}
limitingStorageQueueStorageServer =
ss->second->lastReply.bytesInput - ss->second->smoothDurableBytes.smoothTotal();
limitingStorageQueueStorageServer = ss->second->lastReply.bytesInput - ss->second->getSmoothDurableBytes();
limits->tpsLimit = ss->first;
reasonID = storageTpsLimitReverseIndex.begin()->second->id; // Although we aren't controlling based on the worst
// SS, we still report it as the limiting process
@ -1086,16 +1084,15 @@ void Ratekeeper::updateRate(RatekeeperLimits* limits) {
limitReason_t tlogLimitReason = limitReason_t::log_server_write_queue;
int64_t minFreeSpace =
std::max(SERVER_KNOBS->MIN_AVAILABLE_SPACE,
(int64_t)(SERVER_KNOBS->MIN_AVAILABLE_SPACE_RATIO * tl.smoothTotalSpace.smoothTotal()));
int64_t minFreeSpace = std::max(SERVER_KNOBS->MIN_AVAILABLE_SPACE,
(int64_t)(SERVER_KNOBS->MIN_AVAILABLE_SPACE_RATIO * tl.getSmoothTotalSpace()));
worstFreeSpaceTLog = std::min(worstFreeSpaceTLog, (int64_t)tl.smoothFreeSpace.smoothTotal() - minFreeSpace);
worstFreeSpaceTLog = std::min(worstFreeSpaceTLog, (int64_t)tl.getSmoothFreeSpace() - minFreeSpace);
int64_t springBytes = std::max<int64_t>(
1, std::min<int64_t>(limits->logSpringBytes, (tl.smoothFreeSpace.smoothTotal() - minFreeSpace) * 0.2));
int64_t targetBytes = std::max<int64_t>(
1, std::min(limits->logTargetBytes, (int64_t)tl.smoothFreeSpace.smoothTotal() - minFreeSpace));
1, std::min<int64_t>(limits->logSpringBytes, (tl.getSmoothFreeSpace() - minFreeSpace) * 0.2));
int64_t targetBytes =
std::max<int64_t>(1, std::min(limits->logTargetBytes, (int64_t)tl.getSmoothFreeSpace() - minFreeSpace));
if (targetBytes != limits->logTargetBytes) {
if (minFreeSpace == SERVER_KNOBS->MIN_AVAILABLE_SPACE) {
tlogLimitReason = limitReason_t::log_server_min_free_space;
@ -1106,15 +1103,15 @@ void Ratekeeper::updateRate(RatekeeperLimits* limits) {
TraceEvent("RatekeeperLimitReasonDetails")
.detail("TLogID", tl.id)
.detail("Reason", tlogLimitReason)
.detail("TLSmoothFreeSpace", tl.smoothFreeSpace.smoothTotal())
.detail("TLSmoothTotalSpace", tl.smoothTotalSpace.smoothTotal())
.detail("TLSmoothFreeSpace", tl.getSmoothFreeSpace())
.detail("TLSmoothTotalSpace", tl.getSmoothTotalSpace())
.detail("LimitsLogTargetBytes", limits->logTargetBytes)
.detail("TargetBytes", targetBytes)
.detail("MinFreeSpace", minFreeSpace);
}
}
int64_t queue = tl.lastReply.bytesInput - tl.smoothDurableBytes.smoothTotal();
int64_t queue = tl.lastReply.bytesInput - tl.getSmoothDurableBytes();
healthMetrics.tLogQueue[tl.id] = queue;
int64_t b = queue - targetBytes;
worstStorageQueueTLog = std::max(worstStorageQueueTLog, queue);
@ -1136,14 +1133,14 @@ void Ratekeeper::updateRate(RatekeeperLimits* limits) {
TraceEvent("RatekeeperLimitReasonDetails")
.detail("TLogID", tl.id)
.detail("Reason", limitReason_t::storage_server_readable_behind)
.detail("TLSmoothFreeSpace", tl.smoothFreeSpace.smoothTotal())
.detail("TLSmoothTotalSpace", tl.smoothTotalSpace.smoothTotal())
.detail("TLSmoothFreeSpace", tl.getSmoothFreeSpace())
.detail("TLSmoothTotalSpace", tl.getSmoothTotalSpace())
.detail("LimitsLogSpringBytes", limits->logSpringBytes)
.detail("LimitsLogTargetBytes", limits->logTargetBytes)
.detail("SpringBytes", springBytes)
.detail("TargetBytes", targetBytes)
.detail("TLLastReplyBytesInput", tl.lastReply.bytesInput)
.detail("TLSmoothDurableBytes", tl.smoothDurableBytes.smoothTotal())
.detail("TLSmoothDurableBytes", tl.getSmoothDurableBytes())
.detail("Queue", queue)
.detail("B", b)
.detail("TargetRateRatio", targetRateRatio)
@ -1155,11 +1152,11 @@ void Ratekeeper::updateRate(RatekeeperLimits* limits) {
tlogLimitReason = limitReason_t::storage_server_readable_behind;
}
double inputRate = tl.smoothInputBytes.smoothRate();
double inputRate = tl.getSmoothInputBytesRate();
if (targetRateRatio > 0) {
double smoothedRate =
std::max(tl.verySmoothDurableBytes.smoothRate(), actualTps / SERVER_KNOBS->MAX_TRANSACTIONS_PER_BYTE);
std::max(tl.getVerySmoothDurableBytesRate(), actualTps / SERVER_KNOBS->MAX_TRANSACTIONS_PER_BYTE);
double x = smoothedRate / (inputRate * targetRateRatio);
if (targetRateRatio < .75) //< FIXME: KNOB for 2.0
x = std::max(x, 0.95);
@ -1184,8 +1181,8 @@ void Ratekeeper::updateRate(RatekeeperLimits* limits) {
.detail("Reason", limitReason_t::log_server_mvcc_write_bandwidth)
.detail("TLogID", tl.id)
.detail("MinFreeSpace", minFreeSpace)
.detail("TLSmoothFreeSpace", tl.smoothFreeSpace.smoothTotal())
.detail("TLSmoothTotalSpace", tl.smoothTotalSpace.smoothTotal())
.detail("TLSmoothFreeSpace", tl.getSmoothFreeSpace())
.detail("TLSmoothTotalSpace", tl.getSmoothTotalSpace())
.detail("LimitsLogSpringBytes", limits->logSpringBytes)
.detail("LimitsLogTargetBytes", limits->logTargetBytes)
.detail("SpringBytes", springBytes)
@ -1366,7 +1363,7 @@ UpdateCommitCostRequest StorageQueueInfo::refreshCommitCost(double elapsed) {
return updateCommitCostRequest;
}
Optional<double> StorageQueueInfo::getThrottlingRatio(int64_t storageTargetBytes, int64_t storageSpringBytes) const {
Optional<double> StorageQueueInfo::getTagThrottlingRatio(int64_t storageTargetBytes, int64_t storageSpringBytes) const {
auto const storageQueue = getStorageQueueBytes();
if (storageQueue < storageTargetBytes - storageSpringBytes) {
return {};

View File

@ -205,6 +205,17 @@ ACTOR Future<Void> resolveBatch(Reference<Resolver> self,
req.prevVersion >= 0 ? req.reply.getEndpoint().getPrimaryAddress() : NetworkAddress();
state ProxyRequestsInfo& proxyInfo = self->proxyInfoMap[proxyAddress];
state std::unordered_map<EncryptCipherDomainId, Reference<BlobCipherKey>> cipherKeys;
if (isEncryptionOpSupported(EncryptOperationType::TLOG_ENCRYPTION)) {
static const std::unordered_map<EncryptCipherDomainId, EncryptCipherDomainName> metadataDomains = {
{ SYSTEM_KEYSPACE_ENCRYPT_DOMAIN_ID, FDB_SYSTEM_KEYSPACE_ENCRYPT_DOMAIN_NAME },
{ ENCRYPT_HEADER_DOMAIN_ID, FDB_ENCRYPT_HEADER_DOMAIN_NAME }
};
std::unordered_map<EncryptCipherDomainId, Reference<BlobCipherKey>> cks =
wait(getLatestEncryptCipherKeys(db, metadataDomains, BlobCipherMetrics::TLOG));
cipherKeys = cks;
}
++self->resolveBatchIn;
if (req.debugID.present()) {
@ -351,7 +362,11 @@ ACTOR Future<Void> resolveBatch(Reference<Resolver> self,
SpanContext spanContext =
req.transactions[t].spanContext.present() ? req.transactions[t].spanContext.get() : SpanContext();
applyMetadataMutations(spanContext, *resolverData, req.transactions[t].mutations);
applyMetadataMutations(spanContext,
*resolverData,
req.transactions[t].mutations,
isEncryptionOpSupported(EncryptOperationType::TLOG_ENCRYPTION) ? &cipherKeys
: nullptr);
}
CODE_PROBE(self->forceRecovery, "Resolver detects forced recovery");
}
@ -506,8 +521,10 @@ struct TransactionStateResolveContext {
}
};
ACTOR Future<Void> processCompleteTransactionStateRequest(TransactionStateResolveContext* pContext,
Reference<AsyncVar<ServerDBInfo> const> db) {
ACTOR Future<Void> processCompleteTransactionStateRequest(
TransactionStateResolveContext* pContext,
Reference<AsyncVar<ServerDBInfo> const> db,
std::unordered_map<EncryptCipherDomainId, Reference<BlobCipherKey>>* cipherKeys) {
state KeyRange txnKeys = allKeys;
state std::map<Tag, UID> tag_uid;
@ -574,7 +591,7 @@ ACTOR Future<Void> processCompleteTransactionStateRequest(TransactionStateResolv
bool confChanges; // Ignore configuration changes for initial commits.
ResolverData resolverData(
pContext->pResolverData->dbgid, pContext->pTxnStateStore, &pContext->pResolverData->keyInfo, confChanges);
applyMetadataMutations(SpanContext(), resolverData, mutations);
applyMetadataMutations(SpanContext(), resolverData, mutations, cipherKeys);
} // loop
auto lockedKey = pContext->pTxnStateStore->readValue(databaseLockedKey).get();
@ -615,7 +632,18 @@ ACTOR Future<Void> processTransactionStateRequestPart(TransactionStateResolveCon
if (pContext->receivedSequences.size() == pContext->maxSequence) {
// Received all components of the txnStateRequest
ASSERT(!pContext->processed);
wait(processCompleteTransactionStateRequest(pContext, db));
state std::unordered_map<EncryptCipherDomainId, Reference<BlobCipherKey>> cipherKeys;
if (isEncryptionOpSupported(EncryptOperationType::TLOG_ENCRYPTION)) {
static const std::unordered_map<EncryptCipherDomainId, EncryptCipherDomainName> metadataDomains = {
{ SYSTEM_KEYSPACE_ENCRYPT_DOMAIN_ID, FDB_SYSTEM_KEYSPACE_ENCRYPT_DOMAIN_NAME },
{ ENCRYPT_HEADER_DOMAIN_ID, FDB_ENCRYPT_HEADER_DOMAIN_NAME }
};
std::unordered_map<EncryptCipherDomainId, Reference<BlobCipherKey>> cks =
wait(getLatestEncryptCipherKeys(db, metadataDomains, BlobCipherMetrics::TLOG));
cipherKeys = cks;
}
wait(processCompleteTransactionStateRequest(
pContext, db, isEncryptionOpSupported(EncryptOperationType::TLOG_ENCRYPTION) ? &cipherKeys : nullptr));
pContext->processed = true;
}

View File

@ -753,6 +753,8 @@ ACTOR Future<CheckpointMetaData> fetchRocksDBCheckpoint(Database cx,
.detail("InitialState", initialState.toString())
.detail("CheckpointDir", dir);
ASSERT(!initialState.ranges.empty());
state std::shared_ptr<CheckpointMetaData> metaData = std::make_shared<CheckpointMetaData>(initialState);
if (metaData->format == RocksDBColumnFamily) {
@ -771,7 +773,7 @@ ACTOR Future<CheckpointMetaData> fetchRocksDBCheckpoint(Database cx,
} 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->range, dir, writer, cFun));
wait(fetchCheckpointRange(cx, metaData, metaData->ranges.front(), dir, writer, cFun));
}
return *metaData;

View File

@ -0,0 +1,144 @@
/*
* RocksDBLogForwarder.actor.cpp
*
* This source file is part of the FoundationDB open source project
*
* Copyright 2013-2022 Apple Inc. and the FoundationDB project authors
*
* Licensed under the Apache License, Version 2.0 (the "License");
* you may not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
#include "fdbserver/RocksDBLogForwarder.h"
#include "flow/network.h"
#include "flow/Trace.h"
#include "fdbrpc/simulator.h"
#include "flow/actorcompiler.h" // This must be the last include file
using InfoLogLevel = rocksdb::InfoLogLevel;
namespace {
Severity getSeverityFromLogLevel(const InfoLogLevel& log_level) {
switch (log_level) {
case InfoLogLevel::DEBUG_LEVEL:
return SevDebug;
case InfoLogLevel::INFO_LEVEL:
return SevInfo;
case InfoLogLevel::WARN_LEVEL:
return SevWarn;
case InfoLogLevel::ERROR_LEVEL:
return SevError;
case InfoLogLevel::FATAL_LEVEL:
return SevError;
case InfoLogLevel::HEADER_LEVEL:
return SevVerbose;
case InfoLogLevel::NUM_INFO_LOG_LEVELS:
ASSERT(false);
}
UNREACHABLE();
}
} // namespace
namespace details {
void logTraceEvent(const RocksDBLogRecord& record) {
TraceEvent event = TraceEvent(record.severity, "RocksDBLogRecord", record.uid);
event.detail("RocksDBLogTime", record.logReceiveTime);
{
std::stringstream ss;
ss << record.threadID;
event.detail("RocksDBThreadID", ss.str());
}
for (const auto& [k, v] : record.kvPairs) {
event.detail(k.c_str(), v);
}
}
ACTOR Future<Void> rocksDBPeriodicallyLogger(RocksDBLogger* pRecords) {
loop choose {
when(wait(delay(0.1))) { pRecords->consume(); }
}
}
RocksDBLogger::RocksDBLogger()
: mainThreadId(std::this_thread::get_id()), periodicLogger(rocksDBPeriodicallyLogger(this)) {}
void RocksDBLogger::inject(RocksDBLogRecord&& record) {
const std::thread::id threadId = std::this_thread::get_id();
if (threadId == mainThreadId) {
// In the main thread, it is *NOT* necessary to cache the record.
logTraceEvent(record);
consume();
} else {
const std::lock_guard<std::mutex> lockGuard(recordsMutex);
logRecords.emplace_back();
logRecords.back() = std::move(record);
}
}
void RocksDBLogger::consume() {
std::vector<RocksDBLogRecord> currentRecords;
{
const std::lock_guard<std::mutex> lockGuard(recordsMutex);
currentRecords.swap(logRecords);
}
for (const auto& record : currentRecords) {
logTraceEvent(record);
}
}
} // namespace details
RocksDBLogForwarder::RocksDBLogForwarder(const UID& id_, const InfoLogLevel log_level)
: rocksdb::Logger(log_level), id(id_), logger() {
TraceEvent(SevInfo, "RocksDBLoggerStart", id);
}
RocksDBLogForwarder::~RocksDBLogForwarder() {
TraceEvent(SevInfo, "RocksDBLoggerStop", id);
}
void RocksDBLogForwarder::Logv(const char* format, va_list ap) {
Logv(InfoLogLevel::INFO_LEVEL, format, ap);
}
void RocksDBLogForwarder::Logv(const InfoLogLevel log_level, const char* format, va_list ap) {
const std::thread::id threadID = std::this_thread::get_id();
// FIXME: Restrict the RocksDB log level to warn in order to prevent almost all simulation test failure. This has to
// be reconsidered.
const Severity severity = std::min(getSeverityFromLogLevel(log_level), SevWarn);
// TODO: Parse the log information into KV pairs
// At this stage vsnprintf is used
char buf[1024];
vsnprintf(buf, 1024, format, ap);
if (severity < SevError) {
logger.inject(details::RocksDBLogRecord{ now(), severity, id, threadID, { { "Text", std::string(buf) } } });
} else {
logger.inject(details::RocksDBLogRecord{
now(),
severity,
id,
threadID,
{ { "Text", std::string(buf) }, { "OriginalBacktrace", platform::get_backtrace() } } });
}
}

View File

@ -18,6 +18,7 @@
* limitations under the License.
*/
#include "fdbclient/NativeAPI.actor.h"
#include "fdbserver/Knobs.h"
#include "fdbserver/TransactionTagCounter.h"
#include "flow/Trace.h"
@ -90,9 +91,6 @@ class TransactionTagCounterImpl {
std::vector<StorageQueuingMetricsReply::TagInfo> previousBusiestTags;
Reference<EventCacheHolder> busiestReadTagEventHolder;
// Round up to the nearest page size
static int64_t costFunction(int64_t bytes) { return (bytes - 1) / CLIENT_KNOBS->READ_COST_BYTE_FACTOR + 1; }
public:
TransactionTagCounterImpl(UID thisServerID)
: thisServerID(thisServerID), topTags(SERVER_KNOBS->SS_THROTTLE_TAGS_TRACKED),
@ -101,7 +99,7 @@ public:
void addRequest(Optional<TagSet> const& tags, int64_t bytes) {
if (tags.present()) {
CODE_PROBE(true, "Tracking transaction tag in counter");
double cost = costFunction(bytes);
auto const cost = getReadOperationCost(bytes);
for (auto& tag : tags.get()) {
int64_t& count = intervalCounts[TransactionTag(tag, tags.get().getArena())];
topTags.incrementCount(tag, count, cost);

View File

@ -144,6 +144,7 @@ inline bool containsMetadataMutation(const VectorRef<MutationRef>& mutations) {
// Resolver's version
void applyMetadataMutations(SpanContext const& spanContext,
ResolverData& resolverData,
const VectorRef<MutationRef>& mutations);
const VectorRef<MutationRef>& mutations,
const std::unordered_map<EncryptCipherDomainId, Reference<BlobCipherKey>>* pCipherKeys);
#endif

View File

@ -23,6 +23,7 @@
#include "fdbclient/CommitProxyInterface.h"
#include "fdbclient/TagThrottle.actor.h"
#include "fdbserver/GrvTransactionRateInfo.h"
#include "fdbserver/LatencyBandsMap.h"
// GrvProxyTransactionTagThrottler is used to throttle GetReadVersionRequests based on tag quotas
// before they're pushed into priority-partitioned queues.
@ -45,8 +46,8 @@ class GrvProxyTransactionTagThrottler {
explicit DelayedRequest(GetReadVersionRequest const& req)
: req(req), startTime(now()), sequenceNumber(++lastSequenceNumber) {}
void updateProxyTagThrottledDuration();
bool isMaxThrottled() const;
void updateProxyTagThrottledDuration(LatencyBandsMap&);
bool isMaxThrottled(double maxThrottleDuration) const;
};
struct TagQueue {
@ -57,14 +58,20 @@ class GrvProxyTransactionTagThrottler {
explicit TagQueue(double rate) : rateInfo(rate) {}
void setRate(double rate);
bool isMaxThrottled() const;
void rejectRequests();
bool isMaxThrottled(double maxThrottleDuration) const;
void rejectRequests(LatencyBandsMap&);
};
// Track the budgets for each tag
TransactionTagMap<TagQueue> queues;
double maxThrottleDuration;
// Track latency bands for each tag
LatencyBandsMap latencyBandsMap;
public:
explicit GrvProxyTransactionTagThrottler(double maxThrottleDuration);
// Called with rates received from ratekeeper
void updateRates(TransactionTagMap<double> const& newRates);
@ -77,7 +84,9 @@ public:
void addRequest(GetReadVersionRequest const&);
void addLatencyBandThreshold(double value);
public: // testing
// Returns number of tags tracked
uint32_t size();
uint32_t size() const;
};

View File

@ -0,0 +1,60 @@
/*
* LatencyBandsMap.h
*
* This source file is part of the FoundationDB open source project
*
* Copyright 2013-2022 Apple Inc. and the FoundationDB project authors
*
* Licensed under the Apache License, Version 2.0 (the "License");
* you may not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
#pragma once
#include "fdbclient/TagThrottle.actor.h"
#include "fdbrpc/Stats.h"
#include "fdbserver/Knobs.h"
class LatencyBandsMap {
friend class LatencyBandsMapImpl;
std::string name;
UID id;
double loggingInterval;
int maxSize;
Future<Void> expireOldTags;
struct ExpirableBands {
LatencyBands latencyBands;
double lastUpdated;
explicit ExpirableBands(LatencyBands&&);
};
TransactionTagMap<ExpirableBands> map;
// Manually added thresholds (does not include "infinite" threshold automatically
// added by LatencyBands)
std::vector<double> thresholds;
// Get or create an LatencyBands object stored in map.
// Updates the lastUpdated field corresponding to this LatencyBands object.
// Returns pointer to this object, or an empty optional if object
// cannot be created.
Optional<LatencyBands*> getLatencyBands(TransactionTag tag);
public:
LatencyBandsMap(std::string const& name, UID id, double loggingInterval, int maxSize);
void addMeasurement(TransactionTag tag, double measurement, int count = 1);
void addThreshold(double value);
int size() const { return map.size(); }
};

View File

@ -58,17 +58,19 @@ class StorageQueueInfo {
// refresh periodically
TransactionTagMap<TransactionCommitCostEstimation> tagCostEst;
UID ratekeeperID;
Smoother smoothFreeSpace, smoothTotalSpace;
Smoother smoothDurableBytes, smoothInputBytes, verySmoothDurableBytes;
// Currently unused
Smoother smoothDurableVersion, smoothLatestVersion;
public:
bool valid;
UID ratekeeperID;
UID id;
LocalityData locality;
StorageQueuingMetricsReply lastReply;
bool acceptingRequests;
Smoother smoothDurableBytes, smoothInputBytes, verySmoothDurableBytes;
Smoother smoothDurableVersion, smoothLatestVersion;
Smoother smoothFreeSpace;
Smoother smoothTotalSpace;
limitReason_t limitReason;
std::vector<StorageQueuingMetricsReply::TagInfo> busiestReadTags, busiestWriteTags;
@ -81,18 +83,34 @@ public:
void update(StorageQueuingMetricsReply const&, Smoother& smoothTotalDurableBytes);
void addCommitCost(TransactionTagRef tagName, TransactionCommitCostEstimation const& cost);
// Accessor methods for Smoothers
double getSmoothFreeSpace() const { return smoothFreeSpace.smoothTotal(); }
double getSmoothTotalSpace() const { return smoothTotalSpace.smoothTotal(); }
double getSmoothDurableBytes() const { return smoothDurableBytes.smoothTotal(); }
double getSmoothInputBytesRate() const { return smoothInputBytes.smoothRate(); }
double getVerySmoothDurableBytesRate() const { return verySmoothDurableBytes.smoothRate(); }
// Determine the ratio (limit / current throughput) for throttling based on write queue size
Optional<double> getThrottlingRatio(int64_t storageTargetBytes, int64_t storageSpringBytes) const;
Optional<double> getTagThrottlingRatio(int64_t storageTargetBytes, int64_t storageSpringBytes) const;
};
struct TLogQueueInfo {
TLogQueuingMetricsReply lastReply;
bool valid;
UID id;
class TLogQueueInfo {
Smoother smoothDurableBytes, smoothInputBytes, verySmoothDurableBytes;
Smoother smoothFreeSpace;
Smoother smoothTotalSpace;
public:
TLogQueuingMetricsReply lastReply;
bool valid;
UID id;
// Accessor methods for Smoothers
double getSmoothFreeSpace() const { return smoothFreeSpace.smoothTotal(); }
double getSmoothTotalSpace() const { return smoothTotalSpace.smoothTotal(); }
double getSmoothDurableBytes() const { return smoothDurableBytes.smoothTotal(); }
double getSmoothInputBytesRate() const { return smoothInputBytes.smoothRate(); }
double getVerySmoothDurableBytesRate() const { return verySmoothDurableBytes.smoothRate(); }
TLogQueueInfo(UID id);
Version getLastCommittedVersion() const { return lastReply.v; }
void update(TLogQueuingMetricsReply const& reply, Smoother& smoothTotalDurableBytes);
@ -225,4 +243,4 @@ public:
static Future<Void> run(RatekeeperInterface rkInterf, Reference<AsyncVar<ServerDBInfo> const> dbInfo);
};
#endif // FDBSERVER_RATEKEEPER_H
#endif // FDBSERVER_RATEKEEPER_H

View File

@ -0,0 +1,105 @@
/*
* RocksDBLogForwarder.h
*
* This source file is part of the FoundationDB open source project
*
* Copyright 2013-2022 Apple Inc. and the FoundationDB project authors
*
* Licensed under the Apache License, Version 2.0 (the "License");
* you may not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
#ifndef __ROCKSDB_LOG_FORWARDER_H__
#define __ROCKSDB_LOG_FORWARDER_H__
#include <cstdarg>
#include <thread>
#include <rocksdb/env.h>
#include "flow/genericactors.actor.h"
#include "flow/IRandom.h"
#include "flow/Trace.h"
namespace details {
// Stores a RocksDB log line, transformed into Key/Value pairs
struct RocksDBLogRecord {
double logReceiveTime;
Severity severity;
UID uid;
std::thread::id threadID;
std::vector<std::pair<std::string, std::string>> kvPairs;
};
// Stores RocksDB log lines for furthur consumption.
// *NOTE* This logger *MUST* run in a thread that is able to generate TraceEvents, e.g. in the event loop thread.
class RocksDBLogger {
// The mutex that protects log records, as RocksDB is multi-threaded
std::mutex recordsMutex;
// Main thread ID. Only triggers TraceEvent when on main thread. In FDB only the main thread contains information
// that could thread.
const std::thread::id mainThreadId;
// The log record
std::vector<RocksDBLogRecord> logRecords;
// An ACTOR that logs the non-main thread data periodically
Future<Void> periodicLogger;
public:
// Constructor
RocksDBLogger();
// *Moves* the record to internal records
void inject(RocksDBLogRecord&& record);
// Consumes all the records
void consume();
};
} // namespace details
class NullRocksDBLogForwarder : public rocksdb::Logger {
public:
virtual void Logv(const char*, va_list) { /* intended to be blank */
}
virtual void Logv(const rocksdb::InfoLogLevel, const char*, va_list) { /* intended to be blank */
}
};
class RocksDBLogForwarder : public rocksdb::Logger {
// The ID of the RocksDB instance
const UID id;
// The cache that stores the logs from RocksDB
details::RocksDBLogger logger;
public:
// Constructor
// id is the UID of the logger
// log_level specifies the log level
explicit RocksDBLogForwarder(const UID& id,
const rocksdb::InfoLogLevel log_level = rocksdb::InfoLogLevel::INFO_LEVEL);
// Destructor
virtual ~RocksDBLogForwarder();
// Writes an entry to the log file
virtual void Logv(const char* format, va_list ap);
// Writes an entry to the log file, with a specificied log level
virtual void Logv(const rocksdb::InfoLogLevel log_level, const char* format, va_list ap);
};
#endif // __ROCKSDB_LOG_FORWARDER_H__

View File

@ -79,7 +79,7 @@ class GlobalTagThrottler : public ITagThrottler {
PImpl<class GlobalTagThrottlerImpl> impl;
public:
GlobalTagThrottler(Database db, UID id);
GlobalTagThrottler(Database db, UID id, int maxFallingBehind);
~GlobalTagThrottler();
Future<Void> monitorThrottlingChanges() override;

View File

@ -461,6 +461,9 @@ struct UpdateEagerReadInfo {
std::vector<Optional<Value>> value;
Arena arena;
bool enableClearRangeEagerReads;
UpdateEagerReadInfo(bool enableClearRangeEagerReads) : enableClearRangeEagerReads(enableClearRangeEagerReads) {}
void addMutations(VectorRef<MutationRef> const& mutations) {
for (auto& m : mutations)
@ -469,11 +472,10 @@ struct UpdateEagerReadInfo {
void addMutation(MutationRef const& m) {
// SOMEDAY: Theoretically we can avoid a read if there is an earlier overlapping ClearRange
if (m.type == MutationRef::ClearRange && !m.param2.startsWith(systemKeys.end) &&
SERVER_KNOBS->ENABLE_CLEAR_RANGE_EAGER_READS)
if (m.type == MutationRef::ClearRange && !m.param2.startsWith(systemKeys.end) && enableClearRangeEagerReads)
keyBegin.push_back(m.param2);
else if (m.type == MutationRef::CompareAndClear) {
if (SERVER_KNOBS->ENABLE_CLEAR_RANGE_EAGER_READS)
if (enableClearRangeEagerReads)
keyBegin.push_back(keyAfter(m.param1, arena));
if (keys.size() > 0 && keys.back().first == m.param1) {
// Don't issue a second read, if the last read was equal to the current key.
@ -490,7 +492,7 @@ struct UpdateEagerReadInfo {
}
void finishKeyBegin() {
if (SERVER_KNOBS->ENABLE_CLEAR_RANGE_EAGER_READS) {
if (enableClearRangeEagerReads) {
std::sort(keyBegin.begin(), keyBegin.end());
keyBegin.resize(std::unique(keyBegin.begin(), keyBegin.end()) - keyBegin.begin());
}
@ -1999,7 +2001,9 @@ ACTOR Future<Void> getValueQ(StorageServer* data, GetValueRequest req) {
data->sendErrorWithPenalty(req.reply, e, data->getPenalty());
}
data->transactionTagCounter.addRequest(req.tags, resultSize);
// Key size is not included in "BytesQueried", but still contributes to cost,
// so it must be accounted for here.
data->transactionTagCounter.addRequest(req.tags, req.key.size() + resultSize);
++data->counters.finishedQueries;
@ -2009,7 +2013,7 @@ ACTOR Future<Void> getValueQ(StorageServer* data, GetValueRequest req) {
if (data->latencyBandConfig.present()) {
int maxReadBytes =
data->latencyBandConfig.get().readConfig.maxReadBytes.orDefault(std::numeric_limits<int>::max());
data->counters.readLatencyBands.addMeasurement(duration, resultSize > maxReadBytes);
data->counters.readLatencyBands.addMeasurement(duration, 1, Filtered(resultSize > maxReadBytes));
}
return Void();
@ -2194,8 +2198,8 @@ ACTOR Future<Void> getCheckpointQ(StorageServer* self, GetCheckpointRequest req)
std::unordered_map<UID, CheckpointMetaData>::iterator it = self->checkpoints.begin();
for (; it != self->checkpoints.end(); ++it) {
const CheckpointMetaData& md = it->second;
if (md.version == req.version && md.format == req.format && md.range.contains(req.range) &&
md.getState() == CheckpointMetaData::Complete) {
if (md.version == req.version && md.format == req.format && !md.ranges.empty() &&
md.ranges.front().contains(req.range) && md.getState() == CheckpointMetaData::Complete) {
req.reply.send(md);
TraceEvent(SevDebug, "ServeGetCheckpointEnd", self->thisServerID).detail("Checkpoint", md.toString());
break;
@ -3012,7 +3016,11 @@ ACTOR Future<Void> changeFeedStreamQ(StorageServer* data, ChangeFeedStreamReques
req.reply.setByteLimit(std::min((int64_t)req.replyBufferSize, SERVER_KNOBS->CHANGEFEEDSTREAM_LIMIT_BYTES));
}
wait(delay(0, TaskPriority::DefaultEndpoint));
// Change feeds that are not atLatest must have a lower priority than UpdateStorage to not starve it out, and
// change feed disk reads generally only happen on blob worker recovery or data movement, so they should be
// lower priority. AtLatest change feeds are triggered directly from the SS update loop with no waits, so they
// will still be low latency
wait(delay(0, TaskPriority::SSSpilledChangeFeedReply));
if (DEBUG_CF_TRACE) {
TraceEvent(SevDebug, "TraceChangeFeedStreamStart", data->thisServerID)
@ -3933,9 +3941,10 @@ ACTOR Future<Void> getKeyValuesQ(StorageServer* data, GetKeyValuesRequest req)
int maxSelectorOffset =
data->latencyBandConfig.get().readConfig.maxKeySelectorOffset.orDefault(std::numeric_limits<int>::max());
data->counters.readLatencyBands.addMeasurement(duration,
resultSize > maxReadBytes ||
abs(req.begin.offset) > maxSelectorOffset ||
abs(req.end.offset) > maxSelectorOffset);
1,
Filtered(resultSize > maxReadBytes ||
abs(req.begin.offset) > maxSelectorOffset ||
abs(req.end.offset) > maxSelectorOffset));
}
return Void();
@ -5015,9 +5024,10 @@ ACTOR Future<Void> getMappedKeyValuesQ(StorageServer* data, GetMappedKeyValuesRe
int maxSelectorOffset =
data->latencyBandConfig.get().readConfig.maxKeySelectorOffset.orDefault(std::numeric_limits<int>::max());
data->counters.readLatencyBands.addMeasurement(duration,
resultSize > maxReadBytes ||
abs(req.begin.offset) > maxSelectorOffset ||
abs(req.end.offset) > maxSelectorOffset);
1,
Filtered(resultSize > maxReadBytes ||
abs(req.begin.offset) > maxSelectorOffset ||
abs(req.end.offset) > maxSelectorOffset));
}
return Void();
@ -5335,7 +5345,7 @@ ACTOR Future<Void> getKeyQ(StorageServer* data, GetKeyRequest req) {
int maxSelectorOffset =
data->latencyBandConfig.get().readConfig.maxKeySelectorOffset.orDefault(std::numeric_limits<int>::max());
data->counters.readLatencyBands.addMeasurement(
duration, resultSize > maxReadBytes || abs(req.sel.offset) > maxSelectorOffset);
duration, 1, Filtered(resultSize > maxReadBytes || abs(req.sel.offset) > maxSelectorOffset));
}
return Void();
@ -5374,7 +5384,7 @@ ACTOR Future<Void> doEagerReads(StorageServer* data, UpdateEagerReadInfo* eager)
eager->finishKeyBegin();
state ReadOptions options;
options.type = ReadType::EAGER;
if (SERVER_KNOBS->ENABLE_CLEAR_RANGE_EAGER_READS) {
if (eager->enableClearRangeEagerReads) {
std::vector<Future<Key>> keyEnd(eager->keyBegin.size());
for (int i = 0; i < keyEnd.size(); i++)
keyEnd[i] = data->storage.readNextKeyInclusive(eager->keyBegin[i], options);
@ -5578,7 +5588,7 @@ void expandClear(MutationRef& m,
i = d.lastLessOrEqual(m.param2);
if (i && i->isClearTo() && i->getEndKey() >= m.param2) {
m.param2 = i->getEndKey();
} else if (SERVER_KNOBS->ENABLE_CLEAR_RANGE_EAGER_READS) {
} else if (eager->enableClearRangeEagerReads) {
// Expand to the next set or clear (from storage or latestVersion), and if it
// is a clear, engulf it as well
i = d.lower_bound(m.param2);
@ -8415,6 +8425,12 @@ ACTOR Future<Void> tssDelayForever() {
ACTOR Future<Void> update(StorageServer* data, bool* pReceivedUpdate) {
state double updateStart = g_network->timer();
state double start;
state bool enableClearRangeEagerReads =
(data->storage.getKeyValueStoreType() == KeyValueStoreType::SSD_ROCKSDB_V1 ||
data->storage.getKeyValueStoreType() == KeyValueStoreType::SSD_SHARDED_ROCKSDB)
? SERVER_KNOBS->ROCKSDB_ENABLE_CLEAR_RANGE_EAGER_READS
: SERVER_KNOBS->ENABLE_CLEAR_RANGE_EAGER_READS;
state UpdateEagerReadInfo eager(enableClearRangeEagerReads);
try {
// If we are disk bound and durableVersion is very old, we need to block updates or we could run out of
@ -8517,7 +8533,6 @@ ACTOR Future<Void> update(StorageServer* data, bool* pReceivedUpdate) {
data->ssVersionLockLatencyHistogram->sampleSeconds(now() - start);
start = now();
state UpdateEagerReadInfo eager;
state FetchInjectionInfo fii;
state Reference<ILogSystem::IPeekCursor> cloneCursor2 = cursor->cloneNoMore();
state Optional<std::unordered_map<BlobCipherDetails, Reference<BlobCipherKey>>> cipherKeys;
@ -8558,6 +8573,11 @@ ACTOR Future<Void> update(StorageServer* data, bool* pReceivedUpdate) {
} else {
MutationRef msg;
cloneReader >> msg;
if (isEncryptionOpSupported(EncryptOperationType::TLOG_ENCRYPTION) && !msg.isEncrypted() &&
!(isSingleKeyMutation((MutationRef::Type)msg.type) &&
(backupLogKeys.contains(msg.param1) || (applyLogKeys.contains(msg.param1))))) {
ASSERT(false);
}
if (msg.isEncrypted()) {
if (!cipherKeys.present()) {
const BlobCipherEncryptHeader* header = msg.encryptionHeader();
@ -8591,7 +8611,7 @@ ACTOR Future<Void> update(StorageServer* data, bool* pReceivedUpdate) {
wait(getEncryptCipherKeys(data->db, cipherDetails, BlobCipherMetrics::TLOG));
cipherKeys = getCipherKeysResult;
collectingCipherKeys = false;
eager = UpdateEagerReadInfo();
eager = UpdateEagerReadInfo(enableClearRangeEagerReads);
} else {
// Any fetchKeys which are ready to transition their shards to the adding,transferred state do so now.
// If there is an epoch end we skip this step, to increase testability and to prevent inserting a
@ -8615,7 +8635,7 @@ ACTOR Future<Void> update(StorageServer* data, bool* pReceivedUpdate) {
"A fetchKeys completed while we were doing this, so eager might be outdated. Read it again.");
// SOMEDAY: Theoretically we could check the change counters of individual shards and retry the reads
// only selectively
eager = UpdateEagerReadInfo();
eager = UpdateEagerReadInfo(enableClearRangeEagerReads);
cloneCursor2 = cursor->cloneNoMore();
}
}
@ -8711,6 +8731,11 @@ ACTOR Future<Void> update(StorageServer* data, bool* pReceivedUpdate) {
} else {
MutationRef msg;
rd >> msg;
if (isEncryptionOpSupported(EncryptOperationType::TLOG_ENCRYPTION) && !msg.isEncrypted() &&
!(isSingleKeyMutation((MutationRef::Type)msg.type) &&
(backupLogKeys.contains(msg.param1) || (applyLogKeys.contains(msg.param1))))) {
ASSERT(false);
}
if (msg.isEncrypted()) {
ASSERT(cipherKeys.present());
msg = msg.decrypt(cipherKeys.get(), rd.arena(), BlobCipherMetrics::TLOG);
@ -8911,9 +8936,9 @@ ACTOR Future<Void> update(StorageServer* data, bool* pReceivedUpdate) {
}
ACTOR Future<Void> createCheckpoint(StorageServer* data, CheckpointMetaData metaData) {
ASSERT(metaData.ssID == data->thisServerID);
ASSERT(metaData.ssID == data->thisServerID && !metaData.ranges.empty());
const CheckpointRequest req(metaData.version,
metaData.range,
metaData.ranges.front(),
static_cast<CheckpointFormat>(metaData.format),
metaData.checkpointID,
data->folder + rocksdbCheckpointDirPrefix + metaData.checkpointID.toString());
@ -9405,7 +9430,7 @@ void setAvailableStatus(StorageServer* self, KeyRangeRef keys, bool available) {
// When a shard is moved out, delete all related checkpoints created for data move.
if (!available) {
for (auto& [id, checkpoint] : self->checkpoints) {
if (checkpoint.range.intersects(keys)) {
if (!checkpoint.ranges.empty() && checkpoint.ranges.front().intersects(keys)) {
Key persistCheckpointKey(persistCheckpointKeys.begin.toString() + checkpoint.checkpointID.toString());
checkpoint.setState(CheckpointMetaData::Deleting);
self->addMutationToMutationLog(

View File

@ -334,7 +334,7 @@ struct PhysicalShardMoveWorkLoad : TestWorkload {
dataMoveId,
moveKeysLock,
&self->cleanUpDataMoveParallelismLock,
dataMove.range,
dataMove.ranges.front(),
&ddEnabledState));
TraceEvent("TestCancelDataMoveEnd").detail("DataMove", dataMove.toString());
}

View File

@ -0,0 +1,323 @@
/*
* TransactionCost.actor.cpp
*
* This source file is part of the FoundationDB open source project
*
* Copyright 2013-2022 Apple Inc. and the FoundationDB project authors
*
* Licensed under the Apache License, Version 2.0 (the "License");
* you may not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
#include "fdbclient/ReadYourWrites.h"
#include "fdbserver/workloads/workloads.actor.h"
#include "flow/actorcompiler.h"
class TransactionCostWorkload : public TestWorkload {
int iterations{ 1000 };
Key prefix;
bool debugTransactions{ false };
Key getKey(uint64_t testNumber, uint64_t index = 0) const {
BinaryWriter bw(Unversioned());
bw << bigEndian64(testNumber);
bw << bigEndian64(index);
return bw.toValue().withPrefix(prefix);
}
static Value getValue(uint32_t size) { return makeString(size); }
static UID getDebugID(uint64_t testNumber) { return UID(testNumber << 32, testNumber << 32); }
class ITest {
protected:
uint64_t testNumber;
explicit ITest(uint64_t testNumber) : testNumber(testNumber) {}
public:
void debugTransaction(ReadYourWritesTransaction& tr) { tr.debugTransaction(getDebugID(testNumber)); }
virtual Future<Void> setup(TransactionCostWorkload const& workload, Database const&) { return Void(); }
virtual Future<Void> exec(TransactionCostWorkload const& workload, Reference<ReadYourWritesTransaction>) = 0;
virtual int64_t expectedFinalCost() const = 0;
virtual ~ITest() = default;
};
class ReadEmptyTest : public ITest {
public:
explicit ReadEmptyTest(uint64_t testNumber) : ITest(testNumber) {}
Future<Void> exec(TransactionCostWorkload const& workload, Reference<ReadYourWritesTransaction> tr) override {
return success(tr->get(workload.getKey(testNumber)));
}
int64_t expectedFinalCost() const override { return CLIENT_KNOBS->READ_COST_BYTE_FACTOR; }
};
class ReadLargeValueTest : public ITest {
ACTOR static Future<Void> setup(TransactionCostWorkload const* workload,
ReadLargeValueTest* self,
Database cx) {
state Transaction tr(cx);
loop {
try {
tr.set(workload->getKey(self->testNumber), getValue(CLIENT_KNOBS->READ_COST_BYTE_FACTOR));
wait(tr.commit());
return Void();
} catch (Error& e) {
wait(tr.onError(e));
}
}
}
public:
explicit ReadLargeValueTest(int64_t testNumber) : ITest(testNumber) {}
Future<Void> setup(TransactionCostWorkload const& workload, Database const& cx) override {
return setup(&workload, this, cx);
}
Future<Void> exec(TransactionCostWorkload const& workload, Reference<ReadYourWritesTransaction> tr) override {
return success(tr->get(workload.getKey(testNumber)));
}
int64_t expectedFinalCost() const override { return 2 * CLIENT_KNOBS->READ_COST_BYTE_FACTOR; }
};
class WriteTest : public ITest {
public:
explicit WriteTest(int64_t testNumber) : ITest(testNumber) {}
Future<Void> exec(TransactionCostWorkload const& workload, Reference<ReadYourWritesTransaction> tr) override {
tr->set(workload.getKey(testNumber), getValue(20));
return Void();
}
int64_t expectedFinalCost() const override {
return CLIENT_KNOBS->GLOBAL_TAG_THROTTLING_RW_FUNGIBILITY_RATIO * CLIENT_KNOBS->WRITE_COST_BYTE_FACTOR;
}
};
class WriteLargeValueTest : public ITest {
public:
explicit WriteLargeValueTest(int64_t testNumber) : ITest(testNumber) {}
Future<Void> exec(TransactionCostWorkload const& workload, Reference<ReadYourWritesTransaction> tr) override {
tr->set(workload.getKey(testNumber), getValue(CLIENT_KNOBS->WRITE_COST_BYTE_FACTOR));
return Void();
}
int64_t expectedFinalCost() const override {
return 2 * CLIENT_KNOBS->GLOBAL_TAG_THROTTLING_RW_FUNGIBILITY_RATIO * CLIENT_KNOBS->WRITE_COST_BYTE_FACTOR;
}
};
class WriteMultipleValuesTest : public ITest {
public:
explicit WriteMultipleValuesTest(int64_t testNumber) : ITest(testNumber) {}
Future<Void> exec(TransactionCostWorkload const& workload, Reference<ReadYourWritesTransaction> tr) override {
for (int i = 0; i < 10; ++i) {
tr->set(workload.getKey(testNumber, i), getValue(20));
}
return Void();
}
int64_t expectedFinalCost() const override {
return 10 * CLIENT_KNOBS->GLOBAL_TAG_THROTTLING_RW_FUNGIBILITY_RATIO * CLIENT_KNOBS->WRITE_COST_BYTE_FACTOR;
}
};
class ClearTest : public ITest {
public:
explicit ClearTest(int64_t testNumber) : ITest(testNumber) {}
Future<Void> exec(TransactionCostWorkload const& workload, Reference<ReadYourWritesTransaction> tr) override {
tr->clear(singleKeyRange(workload.getKey(testNumber)));
return Void();
}
int64_t expectedFinalCost() const override { return CLIENT_KNOBS->WRITE_COST_BYTE_FACTOR; }
};
class ReadRangeTest : public ITest {
ACTOR static Future<Void> setup(ReadRangeTest* self, TransactionCostWorkload const* workload, Database cx) {
state Transaction tr(cx);
loop {
try {
for (int i = 0; i < 10; ++i) {
tr.set(workload->getKey(self->testNumber, i), workload->getValue(20));
}
wait(tr.commit());
return Void();
} catch (Error& e) {
wait(tr.onError(e));
}
}
}
public:
explicit ReadRangeTest(int64_t testNumber) : ITest(testNumber) {}
Future<Void> setup(TransactionCostWorkload const& workload, Database const& cx) override {
return setup(this, &workload, cx);
}
Future<Void> exec(TransactionCostWorkload const& workload, Reference<ReadYourWritesTransaction> tr) override {
KeyRange const keys = KeyRangeRef(workload.getKey(testNumber, 0), workload.getKey(testNumber, 10));
return success(tr->getRange(keys, 10));
}
int64_t expectedFinalCost() const override { return CLIENT_KNOBS->READ_COST_BYTE_FACTOR; }
};
class ReadMultipleValuesTest : public ITest {
ACTOR static Future<Void> setup(ReadMultipleValuesTest* self,
TransactionCostWorkload const* workload,
Database cx) {
state Transaction tr(cx);
loop {
try {
for (int i = 0; i < 10; ++i) {
tr.set(workload->getKey(self->testNumber, i), workload->getValue(20));
}
wait(tr.commit());
return Void();
} catch (Error& e) {
wait(tr.onError(e));
}
}
}
public:
explicit ReadMultipleValuesTest(int64_t testNumber) : ITest(testNumber) {}
Future<Void> setup(TransactionCostWorkload const& workload, Database const& cx) override {
return setup(this, &workload, cx);
}
Future<Void> exec(TransactionCostWorkload const& workload, Reference<ReadYourWritesTransaction> tr) override {
std::vector<Future<Void>> futures;
for (int i = 0; i < 10; ++i) {
futures.push_back(success(tr->get(workload.getKey(testNumber, i))));
}
return waitForAll(futures);
}
int64_t expectedFinalCost() const override { return 10 * CLIENT_KNOBS->READ_COST_BYTE_FACTOR; }
};
class LargeReadRangeTest : public ITest {
ACTOR static Future<Void> setup(LargeReadRangeTest* self,
TransactionCostWorkload const* workload,
Database cx) {
state Transaction tr(cx);
loop {
try {
for (int i = 0; i < 10; ++i) {
tr.set(workload->getKey(self->testNumber, i),
workload->getValue(CLIENT_KNOBS->READ_COST_BYTE_FACTOR));
}
wait(tr.commit());
return Void();
} catch (Error& e) {
wait(tr.onError(e));
}
}
}
public:
explicit LargeReadRangeTest(int64_t testNumber) : ITest(testNumber) {}
Future<Void> setup(TransactionCostWorkload const& workload, Database const& cx) override {
return setup(this, &workload, cx);
}
Future<Void> exec(TransactionCostWorkload const& workload, Reference<ReadYourWritesTransaction> tr) override {
KeyRange const keys = KeyRangeRef(workload.getKey(testNumber, 0), workload.getKey(testNumber, 10));
return success(tr->getRange(keys, 10));
}
int64_t expectedFinalCost() const override { return 11 * CLIENT_KNOBS->READ_COST_BYTE_FACTOR; }
};
static std::unique_ptr<ITest> createRandomTest(int64_t testNumber) {
auto const rand = deterministicRandom()->randomInt(0, 9);
if (rand == 0) {
return std::make_unique<ReadEmptyTest>(testNumber);
} else if (rand == 1) {
return std::make_unique<ReadLargeValueTest>(testNumber);
} else if (rand == 2) {
return std::make_unique<ReadMultipleValuesTest>(testNumber);
} else if (rand == 3) {
return std::make_unique<WriteTest>(testNumber);
} else if (rand == 4) {
return std::make_unique<WriteLargeValueTest>(testNumber);
} else if (rand == 5) {
return std::make_unique<WriteMultipleValuesTest>(testNumber);
} else if (rand == 6) {
return std::make_unique<ClearTest>(testNumber);
} else if (rand == 7) {
return std::make_unique<ReadRangeTest>(testNumber);
} else {
return std::make_unique<LargeReadRangeTest>(testNumber);
}
}
ACTOR static Future<Void> runTest(TransactionCostWorkload* self, Database cx, ITest* test) {
wait(test->setup(*self, cx));
state Reference<ReadYourWritesTransaction> tr = makeReference<ReadYourWritesTransaction>(cx);
if (self->debugTransactions) {
test->debugTransaction(*tr);
}
loop {
try {
wait(test->exec(*self, tr));
wait(tr->commit());
ASSERT_EQ(tr->getTotalCost(), test->expectedFinalCost());
return Void();
} catch (Error& e) {
wait(tr->onError(e));
}
}
}
ACTOR static Future<Void> start(TransactionCostWorkload* self, Database cx) {
state uint64_t testNumber = 0;
state Future<Void> f;
// Must use shared_ptr because Flow doesn't support perfect forwarding into actors
state std::shared_ptr<ITest> test;
for (; testNumber < self->iterations; ++testNumber) {
test = createRandomTest(testNumber);
wait(runTest(self, cx, test.get()));
}
return Void();
}
public:
TransactionCostWorkload(WorkloadContext const& wcx) : TestWorkload(wcx) {
iterations = getOption(options, "iterations"_sr, 1000);
prefix = getOption(options, "prefix"_sr, "transactionCost/"_sr);
debugTransactions = getOption(options, "debug"_sr, false);
}
static constexpr auto NAME = "TransactionCost";
Future<Void> setup(Database const& cx) override { return Void(); }
Future<Void> start(Database const& cx) override { return clientId ? Void() : start(this, cx); }
Future<bool> check(Database const& cx) override { return true; }
void getMetrics(std::vector<PerfMetric>& m) override {}
};
WorkloadFactory<TransactionCostWorkload> TransactionCostWorkloadFactory;

View File

@ -71,6 +71,7 @@ public: // introduced features
API_VERSION_FEATURE(@FDB_AV_FUTURE_GET_BOOL@, FutureGetBool);
API_VERSION_FEATURE(@FDB_AV_FUTURE_PROTOCOL_VERSION_API@, FutureProtocolVersionApi);
API_VERSION_FEATURE(@FDB_AV_TENANT_BLOB_RANGE_API@, TenantBlobRangeApi);
API_VERSION_FEATURE(@FDB_AV_GET_TOTAL_COST@, GetTotalCost);
};
#endif // FLOW_CODE_API_VERSION_H

View File

@ -1,5 +1,5 @@
# API Versions
set(FDB_AV_LATEST_VERSION "720")
set(FDB_AV_LATEST_VERSION "730")
# Features
set(FDB_AV_SNAPSHOT_RYW "300")
@ -11,4 +11,5 @@ set(FDB_AV_BLOB_RANGE_API "720")
set(FDB_AV_CREATE_DB_FROM_CONN_STRING "720")
set(FDB_AV_FUTURE_GET_BOOL "720")
set(FDB_AV_FUTURE_PROTOCOL_VERSION_API "720")
set(FDB_AV_TENANT_BLOB_RANGE_API "720")
set(FDB_AV_TENANT_BLOB_RANGE_API "720")
set(FDB_AV_GET_TOTAL_COST "730")

View File

@ -881,8 +881,8 @@ public:
try {
Future<Void> onHandshook;
ConfigureSSLStream(N2::g_net2->activeTlsPolicy, self->ssl_sock, [self = self](bool verifyOk) {
self->has_trusted_peer = verifyOk;
ConfigureSSLStream(N2::g_net2->activeTlsPolicy, self->ssl_sock, [conn = self.getPtr()](bool verifyOk) {
conn->has_trusted_peer = verifyOk;
});
// If the background handshakers are not all busy, use one
@ -959,8 +959,8 @@ public:
try {
Future<Void> onHandshook;
ConfigureSSLStream(N2::g_net2->activeTlsPolicy, self->ssl_sock, [self = self](bool verifyOk) {
self->has_trusted_peer = verifyOk;
ConfigureSSLStream(N2::g_net2->activeTlsPolicy, self->ssl_sock, [conn = self.getPtr()](bool verifyOk) {
conn->has_trusted_peer = verifyOk;
});
// If the background handshakers are not all busy, use one

View File

@ -133,6 +133,7 @@ ERROR( incompatible_software_version, 1220, "Current software does not support d
ERROR( audit_storage_failed, 1221, "Validate storage consistency operation failed" )
ERROR( audit_storage_exceeded_request_limit, 1222, "Exceeded the max number of allowed concurrent audit storage requests" )
ERROR( proxy_tag_throttled, 1223, "Exceeded maximum proxy tag throttling duration" )
ERROR( key_value_store_deadline_exceeded, 1224, "Exceeded maximum time allowed to read or write.")
// 15xx Platform errors
ERROR( platform_error, 1500, "Platform error" )

View File

@ -101,6 +101,7 @@ enum class TaskPriority {
UpdateStorage = 3000,
CompactCache = 2900,
TLogSpilledPeekReply = 2800,
SSSpilledChangeFeedReply = 2730,
BlobWorkerReadChangeFeed = 2720,
BlobWorkerUpdateFDB = 2710,
BlobWorkerUpdateStorage = 2700,

View File

@ -20,6 +20,7 @@
#include "benchmark/benchmark.h"
#include "flow/IRandom.h"
#include "flow/flow.h"
#include "flow/DeterministicRandom.h"
#include "flow/network.h"
@ -61,3 +62,32 @@ static void bench_net2(benchmark::State& benchState) {
}
BENCHMARK(bench_net2)->Range(1, 1 << 16)->ReportAggregatesOnly(true);
static constexpr bool DELAY = false;
static constexpr bool YIELD = true;
ACTOR template <bool useYield>
static Future<Void> benchDelay(benchmark::State* benchState) {
// Number of random delays to start to just to populate the run loop
// priority queue
state int64_t timerCount = benchState->range(0);
state std::vector<Future<Void>> futures;
state DeterministicRandom rand(platform::getRandomSeed());
while (--timerCount > 0) {
futures.push_back(delay(1.0 + rand.random01(), getRandomTaskPriority(rand)));
}
while (benchState->KeepRunning()) {
wait(useYield ? yield() : delay(0));
}
benchState->SetItemsProcessed(static_cast<long>(benchState->iterations()));
return Void();
}
template <bool useYield>
static void bench_delay(benchmark::State& benchState) {
onMainThread([&benchState] { return benchDelay<useYield>(&benchState); }).blockUntilReady();
}
BENCHMARK_TEMPLATE(bench_delay, DELAY)->Range(0, 1 << 16)->ReportAggregatesOnly(true);
BENCHMARK_TEMPLATE(bench_delay, YIELD)->Range(0, 1 << 16)->ReportAggregatesOnly(true);

View File

@ -240,6 +240,7 @@ if(WITH_PYTHON)
add_fdb_test(TEST_FILES rare/RedwoodDeltaTree.toml)
add_fdb_test(TEST_FILES rare/Throttling.toml)
add_fdb_test(TEST_FILES rare/ThroughputQuota.toml)
add_fdb_test(TEST_FILES rare/TransactionCost.toml)
add_fdb_test(TEST_FILES rare/TransactionTagApiCorrectness.toml)
add_fdb_test(TEST_FILES rare/TransactionTagSwizzledApiCorrectness.toml)
add_fdb_test(TEST_FILES rare/WriteTagThrottling.toml)

View File

@ -6,8 +6,13 @@ testTitle='ThroughputQuotaTest'
transactionTag='a'
totalQuota=1.0
[[test.workload]]
testName='Status'
enableLatencyBands = true
testDuration = 60.0
[[test.workload]]
testName = 'Cycle'
transactionsPerSecond = 2500.0
testDuration = 10.0
testDuration = 60.0
expectedRate = 0

View File

@ -0,0 +1,6 @@
[[test]]
testTitle = 'TransactionCostTest'
[[test.workload]]
testName = 'TransactionCost'
iterations = 1000

View File

@ -1,3 +1,8 @@
[[knobs]]
rocksdb_read_value_timeout=300.0
rocksdb_read_value_prefix_timeout=300.0
rocksdb_read_range_timeout=300.0
[[test]]
testTitle = 'ApiCorrectnessTest'
clearAfterTest = false