Merge branch 'master' into feature-changefeed-empty-versions
# Conflicts: # fdbclient/StorageServerInterface.h
This commit is contained in:
commit
94a51e57a5
|
@ -436,21 +436,12 @@ extern "C" DLLEXPORT FDBFuture* fdb_transaction_get_addresses_for_key(FDBTransac
|
|||
return (FDBFuture*)(TXN(tr)->getAddressesForKey(KeyRef(key_name, key_name_length)).extractPtr());
|
||||
}
|
||||
|
||||
FDBFuture* fdb_transaction_get_range_impl(FDBTransaction* tr,
|
||||
uint8_t const* begin_key_name,
|
||||
int begin_key_name_length,
|
||||
fdb_bool_t begin_or_equal,
|
||||
int begin_offset,
|
||||
uint8_t const* end_key_name,
|
||||
int end_key_name_length,
|
||||
fdb_bool_t end_or_equal,
|
||||
int end_offset,
|
||||
int limit,
|
||||
int target_bytes,
|
||||
// Set to the actual limit, target_bytes, and reverse.
|
||||
FDBFuture* validate_and_update_parameters(int& limit,
|
||||
int& target_bytes,
|
||||
FDBStreamingMode mode,
|
||||
int iteration,
|
||||
fdb_bool_t snapshot,
|
||||
fdb_bool_t reverse) {
|
||||
fdb_bool_t& reverse) {
|
||||
/* This method may be called with a runtime API version of 13, in
|
||||
which negative row limits are a reverse range read */
|
||||
if (g_api_version <= 13 && limit < 0) {
|
||||
|
@ -500,6 +491,27 @@ FDBFuture* fdb_transaction_get_range_impl(FDBTransaction* tr,
|
|||
else if (mode_bytes != GetRangeLimits::BYTE_LIMIT_UNLIMITED)
|
||||
target_bytes = std::min(target_bytes, mode_bytes);
|
||||
|
||||
return nullptr;
|
||||
}
|
||||
|
||||
FDBFuture* fdb_transaction_get_range_impl(FDBTransaction* tr,
|
||||
uint8_t const* begin_key_name,
|
||||
int begin_key_name_length,
|
||||
fdb_bool_t begin_or_equal,
|
||||
int begin_offset,
|
||||
uint8_t const* end_key_name,
|
||||
int end_key_name_length,
|
||||
fdb_bool_t end_or_equal,
|
||||
int end_offset,
|
||||
int limit,
|
||||
int target_bytes,
|
||||
FDBStreamingMode mode,
|
||||
int iteration,
|
||||
fdb_bool_t snapshot,
|
||||
fdb_bool_t reverse) {
|
||||
FDBFuture* r = validate_and_update_parameters(limit, target_bytes, mode, iteration, reverse);
|
||||
if (r != nullptr)
|
||||
return r;
|
||||
return (
|
||||
FDBFuture*)(TXN(tr)
|
||||
->getRange(
|
||||
|
@ -511,6 +523,60 @@ FDBFuture* fdb_transaction_get_range_impl(FDBTransaction* tr,
|
|||
.extractPtr());
|
||||
}
|
||||
|
||||
FDBFuture* fdb_transaction_get_range_and_flat_map_impl(FDBTransaction* tr,
|
||||
uint8_t const* begin_key_name,
|
||||
int begin_key_name_length,
|
||||
fdb_bool_t begin_or_equal,
|
||||
int begin_offset,
|
||||
uint8_t const* end_key_name,
|
||||
int end_key_name_length,
|
||||
fdb_bool_t end_or_equal,
|
||||
int end_offset,
|
||||
uint8_t const* mapper_name,
|
||||
int mapper_name_length,
|
||||
int limit,
|
||||
int target_bytes,
|
||||
FDBStreamingMode mode,
|
||||
int iteration,
|
||||
fdb_bool_t snapshot,
|
||||
fdb_bool_t reverse) {
|
||||
FDBFuture* r = validate_and_update_parameters(limit, target_bytes, mode, iteration, reverse);
|
||||
if (r != nullptr)
|
||||
return r;
|
||||
return (
|
||||
FDBFuture*)(TXN(tr)
|
||||
->getRangeAndFlatMap(
|
||||
KeySelectorRef(KeyRef(begin_key_name, begin_key_name_length), begin_or_equal, begin_offset),
|
||||
KeySelectorRef(KeyRef(end_key_name, end_key_name_length), end_or_equal, end_offset),
|
||||
StringRef(mapper_name, mapper_name_length),
|
||||
GetRangeLimits(limit, target_bytes),
|
||||
snapshot,
|
||||
reverse)
|
||||
.extractPtr());
|
||||
}
|
||||
|
||||
// TODO: Support FDB_API_ADDED in generate_asm.py and then this can be replaced with fdb_api_ptr_unimpl.
|
||||
FDBFuture* fdb_transaction_get_range_and_flat_map_v699(FDBTransaction* tr,
|
||||
uint8_t const* begin_key_name,
|
||||
int begin_key_name_length,
|
||||
fdb_bool_t begin_or_equal,
|
||||
int begin_offset,
|
||||
uint8_t const* end_key_name,
|
||||
int end_key_name_length,
|
||||
fdb_bool_t end_or_equal,
|
||||
int end_offset,
|
||||
uint8_t const* mapper_name,
|
||||
int mapper_name_length,
|
||||
int limit,
|
||||
int target_bytes,
|
||||
FDBStreamingMode mode,
|
||||
int iteration,
|
||||
fdb_bool_t snapshot,
|
||||
fdb_bool_t reverse) {
|
||||
fprintf(stderr, "UNIMPLEMENTED FDB API FUNCTION\n");
|
||||
abort();
|
||||
}
|
||||
|
||||
FDBFuture* fdb_transaction_get_range_selector_v13(FDBTransaction* tr,
|
||||
uint8_t const* begin_key_name,
|
||||
int begin_key_name_length,
|
||||
|
@ -702,6 +768,7 @@ extern "C" DLLEXPORT fdb_error_t fdb_select_api_version_impl(int runtime_version
|
|||
// WARNING: use caution when implementing removed functions by calling public API functions. This can lead to
|
||||
// undesired behavior when using the multi-version API. Instead, it is better to have both the removed and public
|
||||
// functions call an internal implementation function. See fdb_create_database_impl for an example.
|
||||
FDB_API_CHANGED(fdb_transaction_get_range_and_flat_map, 700);
|
||||
FDB_API_REMOVED(fdb_future_get_version, 620);
|
||||
FDB_API_REMOVED(fdb_create_cluster, 610);
|
||||
FDB_API_REMOVED(fdb_cluster_create_database, 610);
|
||||
|
|
|
@ -244,6 +244,24 @@ DLLEXPORT WARN_UNUSED_RESULT FDBFuture* fdb_transaction_get_range(FDBTransaction
|
|||
fdb_bool_t reverse);
|
||||
#endif
|
||||
|
||||
DLLEXPORT WARN_UNUSED_RESULT FDBFuture* fdb_transaction_get_range_and_flat_map(FDBTransaction* tr,
|
||||
uint8_t const* begin_key_name,
|
||||
int begin_key_name_length,
|
||||
fdb_bool_t begin_or_equal,
|
||||
int begin_offset,
|
||||
uint8_t const* end_key_name,
|
||||
int end_key_name_length,
|
||||
fdb_bool_t end_or_equal,
|
||||
int end_offset,
|
||||
uint8_t const* mapper_name,
|
||||
int mapper_name_length,
|
||||
int limit,
|
||||
int target_bytes,
|
||||
FDBStreamingMode mode,
|
||||
int iteration,
|
||||
fdb_bool_t snapshot,
|
||||
fdb_bool_t reverse);
|
||||
|
||||
DLLEXPORT void fdb_transaction_set(FDBTransaction* tr,
|
||||
uint8_t const* key_name,
|
||||
int key_name_length,
|
||||
|
|
|
@ -193,6 +193,41 @@ KeyValueArrayFuture Transaction::get_range(const uint8_t* begin_key_name,
|
|||
reverse));
|
||||
}
|
||||
|
||||
KeyValueArrayFuture Transaction::get_range_and_flat_map(const uint8_t* begin_key_name,
|
||||
int begin_key_name_length,
|
||||
fdb_bool_t begin_or_equal,
|
||||
int begin_offset,
|
||||
const uint8_t* end_key_name,
|
||||
int end_key_name_length,
|
||||
fdb_bool_t end_or_equal,
|
||||
int end_offset,
|
||||
const uint8_t* mapper_name,
|
||||
int mapper_name_length,
|
||||
int limit,
|
||||
int target_bytes,
|
||||
FDBStreamingMode mode,
|
||||
int iteration,
|
||||
fdb_bool_t snapshot,
|
||||
fdb_bool_t reverse) {
|
||||
return KeyValueArrayFuture(fdb_transaction_get_range_and_flat_map(tr_,
|
||||
begin_key_name,
|
||||
begin_key_name_length,
|
||||
begin_or_equal,
|
||||
begin_offset,
|
||||
end_key_name,
|
||||
end_key_name_length,
|
||||
end_or_equal,
|
||||
end_offset,
|
||||
mapper_name,
|
||||
mapper_name_length,
|
||||
limit,
|
||||
target_bytes,
|
||||
mode,
|
||||
iteration,
|
||||
snapshot,
|
||||
reverse));
|
||||
}
|
||||
|
||||
EmptyFuture Transaction::watch(std::string_view key) {
|
||||
return EmptyFuture(fdb_transaction_watch(tr_, (const uint8_t*)key.data(), key.size()));
|
||||
}
|
||||
|
|
|
@ -219,6 +219,25 @@ public:
|
|||
fdb_bool_t snapshot,
|
||||
fdb_bool_t reverse);
|
||||
|
||||
// WARNING: This feature is considered experimental at this time. It is only allowed when using snapshot isolation
|
||||
// AND disabling read-your-writes. Returns a future which will be set to an FDBKeyValue array.
|
||||
KeyValueArrayFuture get_range_and_flat_map(const uint8_t* begin_key_name,
|
||||
int begin_key_name_length,
|
||||
fdb_bool_t begin_or_equal,
|
||||
int begin_offset,
|
||||
const uint8_t* end_key_name,
|
||||
int end_key_name_length,
|
||||
fdb_bool_t end_or_equal,
|
||||
int end_offset,
|
||||
const uint8_t* mapper_name,
|
||||
int mapper_name_length,
|
||||
int limit,
|
||||
int target_bytes,
|
||||
FDBStreamingMode mode,
|
||||
int iteration,
|
||||
fdb_bool_t snapshot,
|
||||
fdb_bool_t reverse);
|
||||
|
||||
// Wrapper around fdb_transaction_watch. Returns a future representing an
|
||||
// empty value.
|
||||
EmptyFuture watch(std::string_view key);
|
||||
|
|
|
@ -40,6 +40,7 @@
|
|||
#define DOCTEST_CONFIG_IMPLEMENT
|
||||
#include "doctest.h"
|
||||
#include "fdbclient/rapidjson/document.h"
|
||||
#include "fdbclient/Tuple.h"
|
||||
|
||||
#include "flow/config.h"
|
||||
|
||||
|
@ -76,7 +77,7 @@ fdb_error_t wait_future(fdb::Future& f) {
|
|||
// Given a string s, returns the "lowest" string greater than any string that
|
||||
// starts with s. Taken from
|
||||
// https://github.com/apple/foundationdb/blob/e7d72f458c6a985fdfa677ae021f357d6f49945b/flow/flow.cpp#L223.
|
||||
std::string strinc(const std::string& s) {
|
||||
std::string strinc_str(const std::string& s) {
|
||||
int index = -1;
|
||||
for (index = s.size() - 1; index >= 0; --index) {
|
||||
if ((uint8_t)s[index] != 255) {
|
||||
|
@ -92,16 +93,16 @@ std::string strinc(const std::string& s) {
|
|||
return r;
|
||||
}
|
||||
|
||||
TEST_CASE("strinc") {
|
||||
CHECK(strinc("a").compare("b") == 0);
|
||||
CHECK(strinc("y").compare("z") == 0);
|
||||
CHECK(strinc("!").compare("\"") == 0);
|
||||
CHECK(strinc("*").compare("+") == 0);
|
||||
CHECK(strinc("fdb").compare("fdc") == 0);
|
||||
CHECK(strinc("foundation database 6").compare("foundation database 7") == 0);
|
||||
TEST_CASE("strinc_str") {
|
||||
CHECK(strinc_str("a").compare("b") == 0);
|
||||
CHECK(strinc_str("y").compare("z") == 0);
|
||||
CHECK(strinc_str("!").compare("\"") == 0);
|
||||
CHECK(strinc_str("*").compare("+") == 0);
|
||||
CHECK(strinc_str("fdb").compare("fdc") == 0);
|
||||
CHECK(strinc_str("foundation database 6").compare("foundation database 7") == 0);
|
||||
|
||||
char terminated[] = { 'a', 'b', '\xff' };
|
||||
CHECK(strinc(std::string(terminated, 3)).compare("ac") == 0);
|
||||
CHECK(strinc_str(std::string(terminated, 3)).compare("ac") == 0);
|
||||
}
|
||||
|
||||
// Helper function to add `prefix` to all keys in the given map. Returns a new
|
||||
|
@ -117,7 +118,7 @@ std::map<std::string, std::string> create_data(std::map<std::string, std::string
|
|||
// Clears all data in the database, then inserts the given key value pairs.
|
||||
void insert_data(FDBDatabase* db, const std::map<std::string, std::string>& data) {
|
||||
fdb::Transaction tr(db);
|
||||
auto end_key = strinc(prefix);
|
||||
auto end_key = strinc_str(prefix);
|
||||
while (1) {
|
||||
tr.clear_range(prefix, end_key);
|
||||
for (const auto& [key, val] : data) {
|
||||
|
@ -224,6 +225,59 @@ GetRangeResult get_range(fdb::Transaction& tr,
|
|||
return GetRangeResult{ results, out_more != 0, 0 };
|
||||
}
|
||||
|
||||
GetRangeResult get_range_and_flat_map(fdb::Transaction& tr,
|
||||
const uint8_t* begin_key_name,
|
||||
int begin_key_name_length,
|
||||
fdb_bool_t begin_or_equal,
|
||||
int begin_offset,
|
||||
const uint8_t* end_key_name,
|
||||
int end_key_name_length,
|
||||
fdb_bool_t end_or_equal,
|
||||
int end_offset,
|
||||
const uint8_t* mapper_name,
|
||||
int mapper_name_length,
|
||||
int limit,
|
||||
int target_bytes,
|
||||
FDBStreamingMode mode,
|
||||
int iteration,
|
||||
fdb_bool_t snapshot,
|
||||
fdb_bool_t reverse) {
|
||||
fdb::KeyValueArrayFuture f1 = tr.get_range_and_flat_map(begin_key_name,
|
||||
begin_key_name_length,
|
||||
begin_or_equal,
|
||||
begin_offset,
|
||||
end_key_name,
|
||||
end_key_name_length,
|
||||
end_or_equal,
|
||||
end_offset,
|
||||
mapper_name,
|
||||
mapper_name_length,
|
||||
limit,
|
||||
target_bytes,
|
||||
mode,
|
||||
iteration,
|
||||
snapshot,
|
||||
reverse);
|
||||
|
||||
fdb_error_t err = wait_future(f1);
|
||||
if (err) {
|
||||
return GetRangeResult{ {}, false, err };
|
||||
}
|
||||
|
||||
const FDBKeyValue* out_kv;
|
||||
int out_count;
|
||||
fdb_bool_t out_more;
|
||||
fdb_check(f1.get(&out_kv, &out_count, &out_more));
|
||||
|
||||
std::vector<std::pair<std::string, std::string>> results;
|
||||
for (int i = 0; i < out_count; ++i) {
|
||||
std::string key((const char*)out_kv[i].key, out_kv[i].key_length);
|
||||
std::string value((const char*)out_kv[i].value, out_kv[i].value_length);
|
||||
results.emplace_back(key, value);
|
||||
}
|
||||
return GetRangeResult{ results, out_more != 0, 0 };
|
||||
}
|
||||
|
||||
// Clears all data in the database.
|
||||
void clear_data(FDBDatabase* db) {
|
||||
insert_data(db, {});
|
||||
|
@ -819,6 +873,86 @@ TEST_CASE("fdb_transaction_set_read_version future_version") {
|
|||
CHECK(err == 1009); // future_version
|
||||
}
|
||||
|
||||
const std::string EMPTY = Tuple().pack().toString();
|
||||
const KeyRef RECORD = "RECORD"_sr;
|
||||
const KeyRef INDEX = "INDEX"_sr;
|
||||
static Key primaryKey(const int i) {
|
||||
return Key(format("primary-key-of-record-%08d", i));
|
||||
}
|
||||
static Key indexKey(const int i) {
|
||||
return Key(format("index-key-of-record-%08d", i));
|
||||
}
|
||||
static Value dataOfRecord(const int i) {
|
||||
return Value(format("data-of-record-%08d", i));
|
||||
}
|
||||
static std::string indexEntryKey(const int i) {
|
||||
return Tuple().append(StringRef(prefix)).append(INDEX).append(indexKey(i)).append(primaryKey(i)).pack().toString();
|
||||
}
|
||||
static std::string recordKey(const int i) {
|
||||
return Tuple().append(prefix).append(RECORD).append(primaryKey(i)).pack().toString();
|
||||
}
|
||||
static std::string recordValue(const int i) {
|
||||
return Tuple().append(dataOfRecord(i)).pack().toString();
|
||||
}
|
||||
|
||||
TEST_CASE("fdb_transaction_get_range_and_flat_map") {
|
||||
// Note: The user requested `prefix` should be added as the first element of the tuple that forms the key, rather
|
||||
// than the prefix of the key. So we don't use key() or create_data() in this test.
|
||||
std::map<std::string, std::string> data;
|
||||
for (int i = 0; i < 3; i++) {
|
||||
data[indexEntryKey(i)] = EMPTY;
|
||||
data[recordKey(i)] = recordValue(i);
|
||||
}
|
||||
insert_data(db, data);
|
||||
|
||||
std::string mapper = Tuple().append(prefix).append(RECORD).append("{K[3]}"_sr).pack().toString();
|
||||
|
||||
fdb::Transaction tr(db);
|
||||
// get_range_and_flat_map is only support without RYW. This is a must!!!
|
||||
fdb_check(tr.set_option(FDB_TR_OPTION_READ_YOUR_WRITES_DISABLE, nullptr, 0));
|
||||
while (1) {
|
||||
auto result = get_range_and_flat_map(
|
||||
tr,
|
||||
// [0, 1]
|
||||
FDB_KEYSEL_FIRST_GREATER_OR_EQUAL((const uint8_t*)indexEntryKey(0).c_str(), indexEntryKey(0).size()),
|
||||
FDB_KEYSEL_FIRST_GREATER_THAN((const uint8_t*)indexEntryKey(1).c_str(), indexEntryKey(1).size()),
|
||||
(const uint8_t*)mapper.c_str(),
|
||||
mapper.size(),
|
||||
/* limit */ 0,
|
||||
/* target_bytes */ 0,
|
||||
/* FDBStreamingMode */ FDB_STREAMING_MODE_WANT_ALL,
|
||||
/* iteration */ 0,
|
||||
/* snapshot */ true,
|
||||
/* reverse */ 0);
|
||||
|
||||
if (result.err) {
|
||||
fdb::EmptyFuture f1 = tr.on_error(result.err);
|
||||
fdb_check(wait_future(f1));
|
||||
continue;
|
||||
}
|
||||
|
||||
// Only the first 2 records are supposed to be returned.
|
||||
if (result.kvs.size() < 2) {
|
||||
CHECK(result.more);
|
||||
// Retry.
|
||||
continue;
|
||||
}
|
||||
|
||||
CHECK(result.kvs.size() == 2);
|
||||
CHECK(!result.more);
|
||||
for (int i = 0; i < 2; i++) {
|
||||
const auto& [key, value] = result.kvs[i];
|
||||
std::cout << "result[" << i << "]: key=" << key << ", value=" << value << std::endl;
|
||||
// OUTPUT:
|
||||
// result[0]: key=fdbRECORDprimary-key-of-record-00000000, value=data-of-record-00000000
|
||||
// result[1]: key=fdbRECORDprimary-key-of-record-00000001, value=data-of-record-00000001
|
||||
CHECK(recordKey(i).compare(key) == 0);
|
||||
CHECK(recordValue(i).compare(value) == 0);
|
||||
}
|
||||
break;
|
||||
}
|
||||
}
|
||||
|
||||
TEST_CASE("fdb_transaction_get_range reverse") {
|
||||
std::map<std::string, std::string> data = create_data({ { "a", "1" }, { "b", "2" }, { "c", "3" }, { "d", "4" } });
|
||||
insert_data(db, data);
|
||||
|
@ -1726,7 +1860,7 @@ TEST_CASE("fdb_transaction_add_conflict_range") {
|
|||
|
||||
fdb::Transaction tr2(db);
|
||||
while (1) {
|
||||
fdb_check(tr2.add_conflict_range(key("a"), strinc(key("a")), FDB_CONFLICT_RANGE_TYPE_WRITE));
|
||||
fdb_check(tr2.add_conflict_range(key("a"), strinc_str(key("a")), FDB_CONFLICT_RANGE_TYPE_WRITE));
|
||||
fdb::EmptyFuture f1 = tr2.commit();
|
||||
|
||||
fdb_error_t err = wait_future(f1);
|
||||
|
@ -1739,8 +1873,8 @@ TEST_CASE("fdb_transaction_add_conflict_range") {
|
|||
}
|
||||
|
||||
while (1) {
|
||||
fdb_check(tr.add_conflict_range(key("a"), strinc(key("a")), FDB_CONFLICT_RANGE_TYPE_READ));
|
||||
fdb_check(tr.add_conflict_range(key("a"), strinc(key("a")), FDB_CONFLICT_RANGE_TYPE_WRITE));
|
||||
fdb_check(tr.add_conflict_range(key("a"), strinc_str(key("a")), FDB_CONFLICT_RANGE_TYPE_READ));
|
||||
fdb_check(tr.add_conflict_range(key("a"), strinc_str(key("a")), FDB_CONFLICT_RANGE_TYPE_WRITE));
|
||||
fdb::EmptyFuture f1 = tr.commit();
|
||||
|
||||
fdb_error_t err = wait_future(f1);
|
||||
|
@ -1828,41 +1962,6 @@ TEST_CASE("special-key-space set transaction ID after write") {
|
|||
}
|
||||
}
|
||||
|
||||
TEST_CASE("special-key-space set token after write") {
|
||||
fdb::Transaction tr(db);
|
||||
fdb_check(tr.set_option(FDB_TR_OPTION_SPECIAL_KEY_SPACE_ENABLE_WRITES, nullptr, 0));
|
||||
while (1) {
|
||||
tr.set(key("foo"), "bar");
|
||||
tr.set("\xff\xff/tracing/token", "false");
|
||||
fdb::ValueFuture f1 = tr.get("\xff\xff/tracing/token",
|
||||
/* snapshot */ false);
|
||||
|
||||
fdb_error_t err = wait_future(f1);
|
||||
if (err) {
|
||||
fdb::EmptyFuture f2 = tr.on_error(err);
|
||||
fdb_check(wait_future(f2));
|
||||
continue;
|
||||
}
|
||||
|
||||
int out_present;
|
||||
char* val;
|
||||
int vallen;
|
||||
fdb_check(f1.get(&out_present, (const uint8_t**)&val, &vallen));
|
||||
|
||||
REQUIRE(out_present);
|
||||
uint64_t token = std::stoul(std::string(val, vallen));
|
||||
CHECK(token != 0);
|
||||
break;
|
||||
}
|
||||
}
|
||||
|
||||
TEST_CASE("special-key-space valid token") {
|
||||
auto value = get_value("\xff\xff/tracing/token", /* snapshot */ false, {});
|
||||
REQUIRE(value.has_value());
|
||||
uint64_t token = std::stoul(value.value());
|
||||
CHECK(token > 0);
|
||||
}
|
||||
|
||||
TEST_CASE("special-key-space disable tracing") {
|
||||
fdb::Transaction tr(db);
|
||||
fdb_check(tr.set_option(FDB_TR_OPTION_SPECIAL_KEY_SPACE_ENABLE_WRITES, nullptr, 0));
|
||||
|
@ -1890,48 +1989,6 @@ TEST_CASE("special-key-space disable tracing") {
|
|||
}
|
||||
}
|
||||
|
||||
TEST_CASE("FDB_DB_OPTION_DISTRIBUTED_TRANSACTION_TRACE_DISABLE") {
|
||||
fdb_check(fdb_database_set_option(db, FDB_DB_OPTION_DISTRIBUTED_TRANSACTION_TRACE_DISABLE, nullptr, 0));
|
||||
|
||||
auto value = get_value("\xff\xff/tracing/token", /* snapshot */ false, {});
|
||||
REQUIRE(value.has_value());
|
||||
uint64_t token = std::stoul(value.value());
|
||||
CHECK(token == 0);
|
||||
|
||||
fdb_check(fdb_database_set_option(db, FDB_DB_OPTION_DISTRIBUTED_TRANSACTION_TRACE_ENABLE, nullptr, 0));
|
||||
}
|
||||
|
||||
TEST_CASE("FDB_DB_OPTION_DISTRIBUTED_TRANSACTION_TRACE_DISABLE enable tracing for transaction") {
|
||||
fdb_check(fdb_database_set_option(db, FDB_DB_OPTION_DISTRIBUTED_TRANSACTION_TRACE_DISABLE, nullptr, 0));
|
||||
|
||||
fdb::Transaction tr(db);
|
||||
fdb_check(tr.set_option(FDB_TR_OPTION_SPECIAL_KEY_SPACE_ENABLE_WRITES, nullptr, 0));
|
||||
while (1) {
|
||||
tr.set("\xff\xff/tracing/token", "true");
|
||||
fdb::ValueFuture f1 = tr.get("\xff\xff/tracing/token",
|
||||
/* snapshot */ false);
|
||||
|
||||
fdb_error_t err = wait_future(f1);
|
||||
if (err) {
|
||||
fdb::EmptyFuture f2 = tr.on_error(err);
|
||||
fdb_check(wait_future(f2));
|
||||
continue;
|
||||
}
|
||||
|
||||
int out_present;
|
||||
char* val;
|
||||
int vallen;
|
||||
fdb_check(f1.get(&out_present, (const uint8_t**)&val, &vallen));
|
||||
|
||||
REQUIRE(out_present);
|
||||
uint64_t token = std::stoul(std::string(val, vallen));
|
||||
CHECK(token > 0);
|
||||
break;
|
||||
}
|
||||
|
||||
fdb_check(fdb_database_set_option(db, FDB_DB_OPTION_DISTRIBUTED_TRANSACTION_TRACE_ENABLE, nullptr, 0));
|
||||
}
|
||||
|
||||
TEST_CASE("special-key-space tracing get range") {
|
||||
std::string tracingBegin = "\xff\xff/tracing/";
|
||||
std::string tracingEnd = "\xff\xff/tracing0";
|
||||
|
@ -1964,8 +2021,6 @@ TEST_CASE("special-key-space tracing get range") {
|
|||
CHECK(!out_more);
|
||||
CHECK(out_count == 2);
|
||||
|
||||
CHECK(std::string((char*)out_kv[0].key, out_kv[0].key_length) == tracingBegin + "token");
|
||||
CHECK(std::stoul(std::string((char*)out_kv[0].value, out_kv[0].value_length)) > 0);
|
||||
CHECK(std::string((char*)out_kv[1].key, out_kv[1].key_length) == tracingBegin + "transaction_id");
|
||||
CHECK(std::stoul(std::string((char*)out_kv[1].value, out_kv[1].value_length)) > 0);
|
||||
break;
|
||||
|
@ -2217,7 +2272,7 @@ TEST_CASE("commit_does_not_reset") {
|
|||
continue;
|
||||
}
|
||||
|
||||
fdb_check(tr2.add_conflict_range(key("foo"), strinc(key("foo")), FDB_CONFLICT_RANGE_TYPE_READ));
|
||||
fdb_check(tr2.add_conflict_range(key("foo"), strinc_str(key("foo")), FDB_CONFLICT_RANGE_TYPE_READ));
|
||||
tr2.set(key("foo"), "bar");
|
||||
fdb::EmptyFuture tr2CommitFuture = tr2.commit();
|
||||
err = wait_future(tr2CommitFuture);
|
||||
|
|
|
@ -756,6 +756,76 @@ JNIEXPORT jlong JNICALL Java_com_apple_foundationdb_FDBTransaction_Transaction_1
|
|||
return (jlong)f;
|
||||
}
|
||||
|
||||
JNIEXPORT jlong JNICALL
|
||||
Java_com_apple_foundationdb_FDBTransaction_Transaction_1getRangeAndFlatMap(JNIEnv* jenv,
|
||||
jobject,
|
||||
jlong tPtr,
|
||||
jbyteArray keyBeginBytes,
|
||||
jboolean orEqualBegin,
|
||||
jint offsetBegin,
|
||||
jbyteArray keyEndBytes,
|
||||
jboolean orEqualEnd,
|
||||
jint offsetEnd,
|
||||
jbyteArray mapperBytes,
|
||||
jint rowLimit,
|
||||
jint targetBytes,
|
||||
jint streamingMode,
|
||||
jint iteration,
|
||||
jboolean snapshot,
|
||||
jboolean reverse) {
|
||||
if (!tPtr || !keyBeginBytes || !keyEndBytes || !mapperBytes) {
|
||||
throwParamNotNull(jenv);
|
||||
return 0;
|
||||
}
|
||||
FDBTransaction* tr = (FDBTransaction*)tPtr;
|
||||
|
||||
uint8_t* barrBegin = (uint8_t*)jenv->GetByteArrayElements(keyBeginBytes, JNI_NULL);
|
||||
if (!barrBegin) {
|
||||
if (!jenv->ExceptionOccurred())
|
||||
throwRuntimeEx(jenv, "Error getting handle to native resources");
|
||||
return 0;
|
||||
}
|
||||
|
||||
uint8_t* barrEnd = (uint8_t*)jenv->GetByteArrayElements(keyEndBytes, JNI_NULL);
|
||||
if (!barrEnd) {
|
||||
jenv->ReleaseByteArrayElements(keyBeginBytes, (jbyte*)barrBegin, JNI_ABORT);
|
||||
if (!jenv->ExceptionOccurred())
|
||||
throwRuntimeEx(jenv, "Error getting handle to native resources");
|
||||
return 0;
|
||||
}
|
||||
|
||||
uint8_t* barrMapper = (uint8_t*)jenv->GetByteArrayElements(mapperBytes, JNI_NULL);
|
||||
if (!barrMapper) {
|
||||
jenv->ReleaseByteArrayElements(keyBeginBytes, (jbyte*)barrBegin, JNI_ABORT);
|
||||
jenv->ReleaseByteArrayElements(keyEndBytes, (jbyte*)barrEnd, JNI_ABORT);
|
||||
if (!jenv->ExceptionOccurred())
|
||||
throwRuntimeEx(jenv, "Error getting handle to native resources");
|
||||
return 0;
|
||||
}
|
||||
|
||||
FDBFuture* f = fdb_transaction_get_range_and_flat_map(tr,
|
||||
barrBegin,
|
||||
jenv->GetArrayLength(keyBeginBytes),
|
||||
orEqualBegin,
|
||||
offsetBegin,
|
||||
barrEnd,
|
||||
jenv->GetArrayLength(keyEndBytes),
|
||||
orEqualEnd,
|
||||
offsetEnd,
|
||||
barrMapper,
|
||||
jenv->GetArrayLength(mapperBytes),
|
||||
rowLimit,
|
||||
targetBytes,
|
||||
(FDBStreamingMode)streamingMode,
|
||||
iteration,
|
||||
snapshot,
|
||||
reverse);
|
||||
jenv->ReleaseByteArrayElements(keyBeginBytes, (jbyte*)barrBegin, JNI_ABORT);
|
||||
jenv->ReleaseByteArrayElements(keyEndBytes, (jbyte*)barrEnd, JNI_ABORT);
|
||||
jenv->ReleaseByteArrayElements(mapperBytes, (jbyte*)barrMapper, JNI_ABORT);
|
||||
return (jlong)f;
|
||||
}
|
||||
|
||||
JNIEXPORT void JNICALL Java_com_apple_foundationdb_FutureResults_FutureResults_1getDirect(JNIEnv* jenv,
|
||||
jobject,
|
||||
jlong future,
|
||||
|
|
|
@ -0,0 +1,256 @@
|
|||
/*
|
||||
* RangeAndFlatMapQueryIntegrationTest.java
|
||||
*
|
||||
* This source file is part of the FoundationDB open source project
|
||||
*
|
||||
* Copyright 2013-2018 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.
|
||||
*/
|
||||
package com.apple.foundationdb;
|
||||
|
||||
import java.util.ArrayList;
|
||||
import java.util.Collection;
|
||||
import java.util.Collections;
|
||||
import java.util.Iterator;
|
||||
import java.util.List;
|
||||
import java.util.concurrent.CompletableFuture;
|
||||
import java.util.concurrent.ExecutionException;
|
||||
import java.util.concurrent.ThreadLocalRandom;
|
||||
import java.util.concurrent.atomic.AtomicReference;
|
||||
|
||||
import com.apple.foundationdb.async.AsyncIterable;
|
||||
import com.apple.foundationdb.async.AsyncUtil;
|
||||
import com.apple.foundationdb.tuple.ByteArrayUtil;
|
||||
import com.apple.foundationdb.tuple.Tuple;
|
||||
import org.junit.jupiter.api.AfterEach;
|
||||
import org.junit.jupiter.api.Assertions;
|
||||
import org.junit.jupiter.api.BeforeEach;
|
||||
import org.junit.jupiter.api.Test;
|
||||
import org.junit.jupiter.api.extension.ExtendWith;
|
||||
|
||||
@ExtendWith(RequiresDatabase.class)
|
||||
class RangeAndFlatMapQueryIntegrationTest {
|
||||
private static final FDB fdb = FDB.selectAPIVersion(710);
|
||||
public String databaseArg = null;
|
||||
private Database openFDB() { return fdb.open(databaseArg); }
|
||||
|
||||
@BeforeEach
|
||||
@AfterEach
|
||||
void clearDatabase() throws Exception {
|
||||
/*
|
||||
* Empty the database before and after each run, just in case
|
||||
*/
|
||||
try (Database db = openFDB()) {
|
||||
db.run(tr -> {
|
||||
tr.clear(Range.startsWith(new byte[] { (byte)0x00 }));
|
||||
return null;
|
||||
});
|
||||
}
|
||||
}
|
||||
|
||||
static private final byte[] EMPTY = Tuple.from().pack();
|
||||
static private final String PREFIX = "prefix";
|
||||
static private final String RECORD = "RECORD";
|
||||
static private final String INDEX = "INDEX";
|
||||
static private String primaryKey(int i) { return String.format("primary-key-of-record-%08d", i); }
|
||||
static private String indexKey(int i) { return String.format("index-key-of-record-%08d", i); }
|
||||
static private String dataOfRecord(int i) { return String.format("data-of-record-%08d", i); }
|
||||
|
||||
static byte[] MAPPER = Tuple.from(PREFIX, RECORD, "{K[3]}").pack();
|
||||
static private byte[] indexEntryKey(final int i) {
|
||||
return Tuple.from(PREFIX, INDEX, indexKey(i), primaryKey(i)).pack();
|
||||
}
|
||||
static private byte[] recordKey(final int i) { return Tuple.from(PREFIX, RECORD, primaryKey(i)).pack(); }
|
||||
static private byte[] recordValue(final int i) { return Tuple.from(dataOfRecord(i)).pack(); }
|
||||
|
||||
static private void insertRecordWithIndex(final Transaction tr, final int i) {
|
||||
tr.set(indexEntryKey(i), EMPTY);
|
||||
tr.set(recordKey(i), recordValue(i));
|
||||
}
|
||||
|
||||
private static String getArgFromEnv() {
|
||||
String[] clusterFiles = MultiClientHelper.readClusterFromEnv();
|
||||
String cluster = clusterFiles[0];
|
||||
System.out.printf("Using Cluster: %s\n", cluster);
|
||||
return cluster;
|
||||
}
|
||||
public static void main(String[] args) throws Exception {
|
||||
final RangeAndFlatMapQueryIntegrationTest test = new RangeAndFlatMapQueryIntegrationTest();
|
||||
test.databaseArg = getArgFromEnv();
|
||||
test.clearDatabase();
|
||||
test.comparePerformance();
|
||||
test.clearDatabase();
|
||||
}
|
||||
|
||||
int numRecords = 10000;
|
||||
int numQueries = 10000;
|
||||
int numRecordsPerQuery = 100;
|
||||
boolean validate = false;
|
||||
@Test
|
||||
void comparePerformance() {
|
||||
FDB fdb = FDB.selectAPIVersion(710);
|
||||
try (Database db = openFDB()) {
|
||||
insertRecordsWithIndexes(numRecords, db);
|
||||
instrument(rangeQueryAndGet, "rangeQueryAndGet", db);
|
||||
instrument(rangeQueryAndFlatMap, "rangeQueryAndFlatMap", db);
|
||||
}
|
||||
}
|
||||
|
||||
private void instrument(final RangeQueryWithIndex query, final String name, final Database db) {
|
||||
System.out.printf("Starting %s (numQueries:%d, numRecordsPerQuery:%d)\n", name, numQueries, numRecordsPerQuery);
|
||||
long startTime = System.currentTimeMillis();
|
||||
for (int queryId = 0; queryId < numQueries; queryId++) {
|
||||
int begin = ThreadLocalRandom.current().nextInt(numRecords - numRecordsPerQuery);
|
||||
query.run(begin, begin + numRecordsPerQuery, db);
|
||||
}
|
||||
long time = System.currentTimeMillis() - startTime;
|
||||
System.out.printf("Finished %s, it takes %d ms for %d queries (%d qps)\n", name, time, numQueries,
|
||||
numQueries * 1000L / time);
|
||||
}
|
||||
|
||||
static private final int RECORDS_PER_TXN = 100;
|
||||
static private void insertRecordsWithIndexes(int n, Database db) {
|
||||
int i = 0;
|
||||
while (i < n) {
|
||||
int begin = i;
|
||||
int end = Math.min(n, i + RECORDS_PER_TXN);
|
||||
// insert [begin, end) in one transaction
|
||||
db.run(tr -> {
|
||||
for (int t = begin; t < end; t++) {
|
||||
insertRecordWithIndex(tr, t);
|
||||
}
|
||||
return null;
|
||||
});
|
||||
i = end;
|
||||
}
|
||||
}
|
||||
|
||||
public interface RangeQueryWithIndex {
|
||||
void run(int begin, int end, Database db);
|
||||
}
|
||||
|
||||
RangeQueryWithIndex rangeQueryAndGet = (int begin, int end, Database db) -> db.run(tr -> {
|
||||
try {
|
||||
List<KeyValue> kvs = tr.getRange(KeySelector.firstGreaterOrEqual(indexEntryKey(begin)),
|
||||
KeySelector.firstGreaterOrEqual(indexEntryKey(end)),
|
||||
ReadTransaction.ROW_LIMIT_UNLIMITED, false, StreamingMode.WANT_ALL)
|
||||
.asList()
|
||||
.get();
|
||||
Assertions.assertEquals(end - begin, kvs.size());
|
||||
|
||||
// Get the records of each index entry IN PARALLEL.
|
||||
List<CompletableFuture<byte[]>> resultFutures = new ArrayList<>();
|
||||
// In reality, we need to get the record key by parsing the index entry key. But considering this is a
|
||||
// performance test, we just ignore the returned key and simply generate it from recordKey.
|
||||
for (int id = begin; id < end; id++) {
|
||||
resultFutures.add(tr.get(recordKey(id)));
|
||||
}
|
||||
AsyncUtil.whenAll(resultFutures).get();
|
||||
|
||||
if (validate) {
|
||||
final Iterator<KeyValue> indexes = kvs.iterator();
|
||||
final Iterator<CompletableFuture<byte[]>> records = resultFutures.iterator();
|
||||
for (int id = begin; id < end; id++) {
|
||||
Assertions.assertTrue(indexes.hasNext());
|
||||
assertByteArrayEquals(indexEntryKey(id), indexes.next().getKey());
|
||||
Assertions.assertTrue(records.hasNext());
|
||||
assertByteArrayEquals(recordValue(id), records.next().get());
|
||||
}
|
||||
Assertions.assertFalse(indexes.hasNext());
|
||||
Assertions.assertFalse(records.hasNext());
|
||||
}
|
||||
} catch (Exception e) {
|
||||
Assertions.fail("Unexpected exception", e);
|
||||
}
|
||||
return null;
|
||||
});
|
||||
|
||||
RangeQueryWithIndex rangeQueryAndFlatMap = (int begin, int end, Database db) -> db.run(tr -> {
|
||||
try {
|
||||
tr.options().setReadYourWritesDisable();
|
||||
List<KeyValue> kvs =
|
||||
tr.snapshot()
|
||||
.getRangeAndFlatMap(KeySelector.firstGreaterOrEqual(indexEntryKey(begin)),
|
||||
KeySelector.firstGreaterOrEqual(indexEntryKey(end)), MAPPER,
|
||||
ReadTransaction.ROW_LIMIT_UNLIMITED, false, StreamingMode.WANT_ALL)
|
||||
.asList()
|
||||
.get();
|
||||
Assertions.assertEquals(end - begin, kvs.size());
|
||||
|
||||
if (validate) {
|
||||
final Iterator<KeyValue> results = kvs.iterator();
|
||||
for (int id = begin; id < end; id++) {
|
||||
Assertions.assertTrue(results.hasNext());
|
||||
assertByteArrayEquals(recordValue(id), results.next().getValue());
|
||||
}
|
||||
Assertions.assertFalse(results.hasNext());
|
||||
}
|
||||
} catch (Exception e) {
|
||||
Assertions.fail("Unexpected exception", e);
|
||||
}
|
||||
return null;
|
||||
});
|
||||
|
||||
void assertByteArrayEquals(byte[] expected, byte[] actual) {
|
||||
Assertions.assertEquals(ByteArrayUtil.printable(expected), ByteArrayUtil.printable(actual));
|
||||
}
|
||||
|
||||
@Test
|
||||
void rangeAndFlatMapQueryOverMultipleRows() throws Exception {
|
||||
try (Database db = openFDB()) {
|
||||
insertRecordsWithIndexes(3, db);
|
||||
|
||||
List<byte[]> expected_data_of_records = new ArrayList<>();
|
||||
for (int i = 0; i <= 1; i++) {
|
||||
expected_data_of_records.add(recordValue(i));
|
||||
}
|
||||
|
||||
db.run(tr -> {
|
||||
// getRangeAndFlatMap is only support without RYW. This is a must!!!
|
||||
tr.options().setReadYourWritesDisable();
|
||||
|
||||
// getRangeAndFlatMap is only supported with snapshot.
|
||||
Iterator<KeyValue> kvs =
|
||||
tr.snapshot()
|
||||
.getRangeAndFlatMap(KeySelector.firstGreaterOrEqual(indexEntryKey(0)),
|
||||
KeySelector.firstGreaterThan(indexEntryKey(1)), MAPPER,
|
||||
ReadTransaction.ROW_LIMIT_UNLIMITED, false, StreamingMode.WANT_ALL)
|
||||
.iterator();
|
||||
Iterator<byte[]> expected_data_of_records_iter = expected_data_of_records.iterator();
|
||||
while (expected_data_of_records_iter.hasNext()) {
|
||||
Assertions.assertTrue(kvs.hasNext(), "iterator ended too early");
|
||||
KeyValue kv = kvs.next();
|
||||
byte[] actual_data_of_record = kv.getValue();
|
||||
byte[] expected_data_of_record = expected_data_of_records_iter.next();
|
||||
|
||||
// System.out.println("result key:" + ByteArrayUtil.printable(kv.getKey()) + " value:" +
|
||||
// ByteArrayUtil.printable(kv.getValue())); Output:
|
||||
// result
|
||||
// key:\x02prefix\x00\x02INDEX\x00\x02index-key-of-record-0\x00\x02primary-key-of-record-0\x00
|
||||
// value:\x02data-of-record-0\x00
|
||||
// result
|
||||
// key:\x02prefix\x00\x02INDEX\x00\x02index-key-of-record-1\x00\x02primary-key-of-record-1\x00
|
||||
// value:\x02data-of-record-1\x00
|
||||
|
||||
// For now, we don't guarantee what that the returned keys mean.
|
||||
Assertions.assertArrayEquals(expected_data_of_record, actual_data_of_record,
|
||||
"Incorrect data of record!");
|
||||
}
|
||||
Assertions.assertFalse(kvs.hasNext(), "Iterator returned too much data");
|
||||
|
||||
return null;
|
||||
});
|
||||
}
|
||||
}
|
||||
}
|
|
@ -88,8 +88,11 @@ public class FakeFDBTransaction extends FDBTransaction {
|
|||
public int getNumRangeCalls() { return numRangeCalls; }
|
||||
|
||||
@Override
|
||||
protected FutureResults getRange_internal(KeySelector begin, KeySelector end, int rowLimit, int targetBytes,
|
||||
int streamingMode, int iteration, boolean isSnapshot, boolean reverse) {
|
||||
protected FutureResults getRange_internal(KeySelector begin, KeySelector end,
|
||||
// TODO: map is not supported in FakeFDBTransaction yet.
|
||||
byte[] mapper, // Nullable
|
||||
int rowLimit, int targetBytes, int streamingMode, int iteration,
|
||||
boolean isSnapshot, boolean reverse) {
|
||||
numRangeCalls++;
|
||||
// TODO this is probably not correct for all KeySelector instances--we'll want to match with real behavior
|
||||
NavigableMap<byte[], byte[]> range =
|
||||
|
|
|
@ -91,6 +91,15 @@ class FDBTransaction extends NativeObjectWrapper implements Transaction, OptionC
|
|||
return FDBTransaction.this.getRangeSplitPoints(range, chunkSize);
|
||||
}
|
||||
|
||||
@Override
|
||||
public AsyncIterable<KeyValue> getRangeAndFlatMap(KeySelector begin, KeySelector end, byte[] mapper, int limit,
|
||||
boolean reverse, StreamingMode mode) {
|
||||
if (mapper == null) {
|
||||
throw new IllegalArgumentException("Mapper must be non-null");
|
||||
}
|
||||
return new RangeQuery(FDBTransaction.this, true, begin, end, mapper, limit, reverse, mode, eventKeeper);
|
||||
}
|
||||
|
||||
///////////////////
|
||||
// getRange -> KeySelectors
|
||||
///////////////////
|
||||
|
@ -338,6 +347,12 @@ class FDBTransaction extends NativeObjectWrapper implements Transaction, OptionC
|
|||
return this.getRangeSplitPoints(range.begin, range.end, chunkSize);
|
||||
}
|
||||
|
||||
@Override
|
||||
public AsyncIterable<KeyValue> getRangeAndFlatMap(KeySelector begin, KeySelector end, byte[] mapper, int limit,
|
||||
boolean reverse, StreamingMode mode) {
|
||||
throw new UnsupportedOperationException("getRangeAndFlatMap is only supported in snapshot");
|
||||
}
|
||||
|
||||
///////////////////
|
||||
// getRange -> KeySelectors
|
||||
///////////////////
|
||||
|
@ -415,10 +430,10 @@ class FDBTransaction extends NativeObjectWrapper implements Transaction, OptionC
|
|||
}
|
||||
|
||||
// Users of this function must close the returned FutureResults when finished
|
||||
protected FutureResults getRange_internal(
|
||||
KeySelector begin, KeySelector end,
|
||||
int rowLimit, int targetBytes, int streamingMode,
|
||||
int iteration, boolean isSnapshot, boolean reverse) {
|
||||
protected FutureResults getRange_internal(KeySelector begin, KeySelector end,
|
||||
byte[] mapper, // Nullable
|
||||
int rowLimit, int targetBytes, int streamingMode, int iteration,
|
||||
boolean isSnapshot, boolean reverse) {
|
||||
if (eventKeeper != null) {
|
||||
eventKeeper.increment(Events.JNI_CALL);
|
||||
}
|
||||
|
@ -429,10 +444,14 @@ class FDBTransaction extends NativeObjectWrapper implements Transaction, OptionC
|
|||
begin.toString(), end.toString(), rowLimit, targetBytes, streamingMode,
|
||||
iteration, Boolean.toString(isSnapshot), Boolean.toString(reverse)));*/
|
||||
return new FutureResults(
|
||||
Transaction_getRange(getPtr(), begin.getKey(), begin.orEqual(), begin.getOffset(),
|
||||
end.getKey(), end.orEqual(), end.getOffset(), rowLimit, targetBytes,
|
||||
streamingMode, iteration, isSnapshot, reverse),
|
||||
FDB.instance().isDirectBufferQueriesEnabled(), executor, eventKeeper);
|
||||
mapper == null
|
||||
? Transaction_getRange(getPtr(), begin.getKey(), begin.orEqual(), begin.getOffset(), end.getKey(),
|
||||
end.orEqual(), end.getOffset(), rowLimit, targetBytes, streamingMode,
|
||||
iteration, isSnapshot, reverse)
|
||||
: Transaction_getRangeAndFlatMap(getPtr(), begin.getKey(), begin.orEqual(), begin.getOffset(),
|
||||
end.getKey(), end.orEqual(), end.getOffset(), mapper, rowLimit,
|
||||
targetBytes, streamingMode, iteration, isSnapshot, reverse),
|
||||
FDB.instance().isDirectBufferQueriesEnabled(), executor, eventKeeper);
|
||||
} finally {
|
||||
pointerReadLock.unlock();
|
||||
}
|
||||
|
@ -771,6 +790,12 @@ class FDBTransaction extends NativeObjectWrapper implements Transaction, OptionC
|
|||
byte[] keyEnd, boolean orEqualEnd, int offsetEnd,
|
||||
int rowLimit, int targetBytes, int streamingMode, int iteration,
|
||||
boolean isSnapshot, boolean reverse);
|
||||
private native long Transaction_getRangeAndFlatMap(long cPtr, byte[] keyBegin, boolean orEqualBegin,
|
||||
int offsetBegin, byte[] keyEnd, boolean orEqualEnd,
|
||||
int offsetEnd,
|
||||
byte[] mapper, // Nonnull
|
||||
int rowLimit, int targetBytes, int streamingMode, int iteration,
|
||||
boolean isSnapshot, boolean reverse);
|
||||
private native void Transaction_addConflictRange(long cPtr,
|
||||
byte[] keyBegin, byte[] keyEnd, int conflictRangeType);
|
||||
private native void Transaction_set(long cPtr, byte[] key, byte[] value);
|
||||
|
|
|
@ -49,17 +49,19 @@ class RangeQuery implements AsyncIterable<KeyValue> {
|
|||
private final FDBTransaction tr;
|
||||
private final KeySelector begin;
|
||||
private final KeySelector end;
|
||||
private final byte[] mapper; // Nullable
|
||||
private final boolean snapshot;
|
||||
private final int rowLimit;
|
||||
private final boolean reverse;
|
||||
private final StreamingMode streamingMode;
|
||||
private final EventKeeper eventKeeper;
|
||||
|
||||
RangeQuery(FDBTransaction transaction, boolean isSnapshot, KeySelector begin, KeySelector end, int rowLimit,
|
||||
boolean reverse, StreamingMode streamingMode, EventKeeper eventKeeper) {
|
||||
RangeQuery(FDBTransaction transaction, boolean isSnapshot, KeySelector begin, KeySelector end, byte[] mapper,
|
||||
int rowLimit, boolean reverse, StreamingMode streamingMode, EventKeeper eventKeeper) {
|
||||
this.tr = transaction;
|
||||
this.begin = begin;
|
||||
this.end = end;
|
||||
this.mapper = mapper;
|
||||
this.snapshot = isSnapshot;
|
||||
this.rowLimit = rowLimit;
|
||||
this.reverse = reverse;
|
||||
|
@ -67,6 +69,12 @@ class RangeQuery implements AsyncIterable<KeyValue> {
|
|||
this.eventKeeper = eventKeeper;
|
||||
}
|
||||
|
||||
// RangeQueryAndFlatMap
|
||||
RangeQuery(FDBTransaction transaction, boolean isSnapshot, KeySelector begin, KeySelector end, int rowLimit,
|
||||
boolean reverse, StreamingMode streamingMode, EventKeeper eventKeeper) {
|
||||
this(transaction, isSnapshot, begin, end, null, rowLimit, reverse, streamingMode, eventKeeper);
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns all the results from the range requested as a {@code List}. If there were no
|
||||
* limits on the original query and there is a large amount of data in the database
|
||||
|
@ -83,16 +91,16 @@ class RangeQuery implements AsyncIterable<KeyValue> {
|
|||
|
||||
// if the streaming mode is EXACT, try and grab things as one chunk
|
||||
if(mode == StreamingMode.EXACT) {
|
||||
FutureResults range = tr.getRange_internal(
|
||||
this.begin, this.end, this.rowLimit, 0, StreamingMode.EXACT.code(),
|
||||
1, this.snapshot, this.reverse);
|
||||
|
||||
FutureResults range = tr.getRange_internal(this.begin, this.end, this.mapper, this.rowLimit, 0,
|
||||
StreamingMode.EXACT.code(), 1, this.snapshot, this.reverse);
|
||||
return range.thenApply(result -> result.get().values)
|
||||
.whenComplete((result, e) -> range.close());
|
||||
}
|
||||
|
||||
// If the streaming mode is not EXACT, simply collect the results of an
|
||||
// iteration into a list
|
||||
return AsyncUtil.collect(new RangeQuery(tr, snapshot, begin, end, rowLimit, reverse, mode, eventKeeper),
|
||||
return AsyncUtil.collect(new RangeQuery(tr, snapshot, begin, end, mapper, rowLimit, reverse, mode, eventKeeper),
|
||||
tr.getExecutor());
|
||||
}
|
||||
|
||||
|
@ -221,8 +229,8 @@ class RangeQuery implements AsyncIterable<KeyValue> {
|
|||
|
||||
nextFuture = new CompletableFuture<>();
|
||||
final long sTime = System.nanoTime();
|
||||
fetchingChunk = tr.getRange_internal(begin, end, rowsLimited ? rowsRemaining : 0, 0, streamingMode.code(),
|
||||
++iteration, snapshot, reverse);
|
||||
fetchingChunk = tr.getRange_internal(begin, end, mapper, rowsLimited ? rowsRemaining : 0, 0,
|
||||
streamingMode.code(), ++iteration, snapshot, reverse);
|
||||
|
||||
BiConsumer<RangeResultInfo,Throwable> cons = new FetchComplete(fetchingChunk,nextFuture);
|
||||
if(eventKeeper!=null){
|
||||
|
|
|
@ -424,6 +424,41 @@ public interface ReadTransaction extends ReadTransactionContext {
|
|||
AsyncIterable<KeyValue> getRange(Range range,
|
||||
int limit, boolean reverse, StreamingMode mode);
|
||||
|
||||
/**
|
||||
* WARNING: This feature is considered experimental at this time. It is only allowed when using snapshot isolation
|
||||
* AND disabling read-your-writes.
|
||||
*
|
||||
* @see KeySelector
|
||||
* @see AsyncIterator
|
||||
*
|
||||
* @param begin the beginning of the range (inclusive)
|
||||
* @param end the end of the range (exclusive)
|
||||
* @param mapper TODO
|
||||
* @param limit the maximum number of results to return. Limits results to the
|
||||
* <i>first</i> keys in the range. Pass {@link #ROW_LIMIT_UNLIMITED} if this query
|
||||
* should not limit the number of results. If {@code reverse} is {@code true} rows
|
||||
* will be limited starting at the end of the range.
|
||||
* @param reverse return results starting at the end of the range in reverse order.
|
||||
* Reading ranges in reverse is supported natively by the database and should
|
||||
* have minimal extra cost.
|
||||
* @param mode provide a hint about how the results are to be used. This
|
||||
* can provide speed improvements or efficiency gains based on the caller's
|
||||
* knowledge of the upcoming access pattern.
|
||||
*
|
||||
* <p>
|
||||
* When converting the result of this query to a list using {@link AsyncIterable#asList()} with the {@code
|
||||
* ITERATOR} streaming mode, the query is automatically modified to fetch results in larger batches. This is done
|
||||
* because it is known in advance that the {@link AsyncIterable#asList()} function will fetch all results in the
|
||||
* range. If a limit is specified, the {@code EXACT} streaming mode will be used, and otherwise it will use {@code
|
||||
* WANT_ALL}.
|
||||
*
|
||||
* To achieve comparable performance when iterating over an entire range without using {@link
|
||||
* AsyncIterable#asList()}, the same streaming mode would need to be used.
|
||||
* </p>
|
||||
* @return a handle to access the results of the asynchronous call
|
||||
*/
|
||||
AsyncIterable<KeyValue> getRangeAndFlatMap(KeySelector begin, KeySelector end, byte[] mapper, int limit,
|
||||
boolean reverse, StreamingMode mode);
|
||||
|
||||
/**
|
||||
* Gets an estimate for the number of bytes stored in the given range.
|
||||
|
|
|
@ -52,6 +52,7 @@ set(JAVA_INTEGRATION_TESTS
|
|||
src/integration/com/apple/foundationdb/CycleMultiClientIntegrationTest.java
|
||||
src/integration/com/apple/foundationdb/SidebandMultiThreadClientTest.java
|
||||
src/integration/com/apple/foundationdb/RepeatableReadMultiThreadClientTest.java
|
||||
src/integration/com/apple/foundationdb/RangeAndFlatMapQueryIntegrationTest.java
|
||||
)
|
||||
|
||||
# Resources that are used in integration testing, but are not explicitly test files (JUnit rules,
|
||||
|
|
|
@ -30,6 +30,7 @@ Features
|
|||
* Improved the efficiency with which storage servers replicate data between themselves. `(PR #5017) <https://github.com/apple/foundationdb/pull/5017>`_
|
||||
* Added support to ``exclude command`` to exclude based on locality match. `(PR #5113) <https://github.com/apple/foundationdb/pull/5113>`_
|
||||
* Add the ``trace_partial_file_suffix`` network option. This option will give unfinished trace files a special suffix to indicate they're not complete yet. When the trace file is complete, it is renamed to remove the suffix. `(PR #5328) <https://github.com/apple/foundationdb/pull/5328>`_
|
||||
* Added "get range and flat map" feature with new APIs (see Bindings section). Storage servers are able to generate the keys in the queries based on another query. With this, upper layer can push some computations down to FDB, to improve latency and bandwidth when read. `(PR #5609) <https://github.com/apple/foundationdb/pull/5609>`_
|
||||
|
||||
Performance
|
||||
-----------
|
||||
|
@ -86,6 +87,8 @@ Bindings
|
|||
* C: Added a function, ``fdb_database_create_snapshot``, to create a snapshot of the database. `(PR #4241) <https://github.com/apple/foundationdb/pull/4241/files>`_
|
||||
* C: Added ``fdb_database_get_main_thread_busyness`` function to report how busy a client's main thread is. `(PR #4504) <https://github.com/apple/foundationdb/pull/4504>`_
|
||||
* Java: Added ``Database.getMainThreadBusyness`` function to report how busy a client's main thread is. `(PR #4564) <https://github.com/apple/foundationdb/pull/4564>`_
|
||||
* C: Added ``fdb_transaction_get_range_and_flat_map`` function to support running queries based on another query in one request. `(PR #5609) <https://github.com/apple/foundationdb/pull/5609>`_
|
||||
* Java: Added ``Transaction.getRangeAndFlatMap`` function to support running queries based on another query in one request. `(PR #5609) <https://github.com/apple/foundationdb/pull/5609>`_
|
||||
|
||||
Other Changes
|
||||
-------------
|
||||
|
|
|
@ -85,23 +85,12 @@ Control options
|
|||
In addition to the command line parameter described above, tracing can be set
|
||||
at a database and transaction level.
|
||||
|
||||
Tracing can be globally disabled by setting the
|
||||
``distributed_transaction_trace_disable`` database option. It can be enabled by
|
||||
setting the ``distributed_transaction_trace_enable`` database option. If
|
||||
neither option is specified but a tracer option is set as described above,
|
||||
tracing will be enabled.
|
||||
Tracing can be controlled on a global level by setting the
|
||||
``TRACING_SAMPLE_RATE`` knob. Set the knob to 0.0 to record no traces, to 1.0
|
||||
to record all traces, or somewhere in the middle. Traces are sampled as a unit.
|
||||
All individual spans in the trace will be included in the sample.
|
||||
|
||||
Tracing can be enabled or disabled for individual transactions. The special key
|
||||
space exposes an API to set a custom trace ID for a transaction, or to disable
|
||||
tracing for the transaction. See the special key space :ref:`tracing module
|
||||
documentation <special-key-space-tracing-module>` to learn more.
|
||||
|
||||
^^^^^^^^^^^^^^
|
||||
Trace sampling
|
||||
^^^^^^^^^^^^^^
|
||||
|
||||
By default, all traces are recorded. If tracing is producing too much data,
|
||||
adjust the trace sample rate with the ``TRACING_SAMPLE_RATE`` knob. Set the
|
||||
knob to 0.0 to record no traces, to 1.0 to record all traces, or somewhere in
|
||||
the middle. Traces are sampled as a unit. All individual spans in the trace
|
||||
will be included in the sample.
|
||||
|
|
|
@ -30,6 +30,7 @@ set(FDBCLIENT_SRCS
|
|||
ClientKnobs.cpp
|
||||
ClientKnobs.h
|
||||
ClientLogEvents.h
|
||||
ClientVersion.h
|
||||
ClientWorkerInterface.h
|
||||
ClusterConnectionFile.actor.cpp
|
||||
ClusterConnectionFile.h
|
||||
|
@ -73,6 +74,8 @@ set(FDBCLIENT_SRCS
|
|||
Knobs.h
|
||||
IKnobCollection.cpp
|
||||
IKnobCollection.h
|
||||
LocalClientAPI.cpp
|
||||
LocalClientAPI.h
|
||||
ManagementAPI.actor.cpp
|
||||
ManagementAPI.actor.h
|
||||
MonitorLeader.actor.cpp
|
||||
|
|
|
@ -23,6 +23,7 @@
|
|||
#include "fdbclient/NativeAPI.actor.h"
|
||||
#include "fdbclient/ManagementAPI.actor.h"
|
||||
#include "fdbclient/ClientKnobs.h"
|
||||
#include "fdbclient/SystemData.h"
|
||||
#include "fdbclient/versions.h"
|
||||
#include "fdbrpc/IAsyncFile.h"
|
||||
#include "flow/Platform.h"
|
||||
|
@ -46,7 +47,7 @@ struct ClientLibBinaryInfo {
|
|||
#define ASSERT_INDEX_IN_RANGE(idx, arr) ASSERT(idx >= 0 && idx < sizeof(arr) / sizeof(arr[0]))
|
||||
|
||||
const std::string& getStatusName(ClientLibStatus status) {
|
||||
static const std::string statusNames[] = { "disabled", "available", "uploading" };
|
||||
static const std::string statusNames[] = { "disabled", "uploading", "download", "active" };
|
||||
int idx = static_cast<int>(status);
|
||||
ASSERT_INDEX_IN_RANGE(idx, statusNames);
|
||||
return statusNames[idx];
|
||||
|
@ -123,7 +124,23 @@ ClientLibChecksumAlg getChecksumAlgByName(std::string_view checksumAlgName) {
|
|||
namespace {
|
||||
|
||||
bool isValidTargetStatus(ClientLibStatus status) {
|
||||
return status == ClientLibStatus::AVAILABLE || status == ClientLibStatus::DISABLED;
|
||||
return status == ClientLibStatus::DISABLED || status == ClientLibStatus::DOWNLOAD ||
|
||||
status == ClientLibStatus::ACTIVE;
|
||||
}
|
||||
|
||||
bool isAvailableForDownload(ClientLibStatus status) {
|
||||
return status == ClientLibStatus::DOWNLOAD || status == ClientLibStatus::ACTIVE;
|
||||
}
|
||||
|
||||
void updateClientLibChangeCounter(Transaction& tr, ClientLibStatus prevStatus, ClientLibStatus newStatus) {
|
||||
static const int64_t counterIncVal = 1;
|
||||
if ((prevStatus != newStatus) &&
|
||||
(newStatus == ClientLibStatus::DOWNLOAD || newStatus == ClientLibStatus::ACTIVE ||
|
||||
prevStatus == ClientLibStatus::DOWNLOAD || prevStatus == ClientLibStatus::ACTIVE)) {
|
||||
tr.atomicOp(clientLibChangeCounterKey,
|
||||
StringRef(reinterpret_cast<const uint8_t*>(&counterIncVal), sizeof(counterIncVal)),
|
||||
MutationRef::AddValue);
|
||||
}
|
||||
}
|
||||
|
||||
json_spirit::mObject parseMetadataJson(StringRef metadataString) {
|
||||
|
@ -432,6 +449,7 @@ ACTOR Future<Void> uploadClientLibrary(Database db,
|
|||
tr.setOption(FDBTransactionOptions::ACCESS_SYSTEM_KEYS);
|
||||
tr.setOption(FDBTransactionOptions::LOCK_AWARE);
|
||||
tr.set(clientLibMetaKey, ValueRef(jsStr));
|
||||
updateClientLibChangeCounter(tr, ClientLibStatus::DISABLED, targetStatus);
|
||||
wait(tr.commit());
|
||||
break;
|
||||
} catch (Error& e) {
|
||||
|
@ -488,8 +506,8 @@ ACTOR Future<Void> downloadClientLibrary(Database db,
|
|||
}
|
||||
}
|
||||
|
||||
// Allow downloading only libraries in the available state
|
||||
if (getStatusByName(getMetadataStrAttr(metadataJson, CLIENTLIB_ATTR_STATUS)) != ClientLibStatus::AVAILABLE) {
|
||||
// Prevent downloading not yet uploaded and disabled libraries
|
||||
if (!isAvailableForDownload(getStatusByName(getMetadataStrAttr(metadataJson, CLIENTLIB_ATTR_STATUS)))) {
|
||||
throw client_lib_not_available();
|
||||
}
|
||||
|
||||
|
@ -620,8 +638,11 @@ ACTOR Future<Void> deleteClientLibrary(Database db, Standalone<StringRef> client
|
|||
TraceEvent(SevWarnAlways, "ClientLibraryNotFound").detail("Key", clientLibMetaKey);
|
||||
throw client_lib_not_found();
|
||||
}
|
||||
json_spirit::mObject metadataJson = parseMetadataJson(metadataOpt.get());
|
||||
ClientLibStatus status = getStatusByName(getMetadataStrAttr(metadataJson, CLIENTLIB_ATTR_STATUS));
|
||||
tr.clear(prefixRange(chunkKeyPrefix));
|
||||
tr.clear(clientLibMetaKey);
|
||||
updateClientLibChangeCounter(tr, status, ClientLibStatus::DISABLED);
|
||||
wait(tr.commit());
|
||||
break;
|
||||
} catch (Error& e) {
|
||||
|
@ -641,8 +662,8 @@ void applyClientLibFilter(const ClientLibFilter& filter,
|
|||
for (const auto& [k, v] : scanResults) {
|
||||
try {
|
||||
json_spirit::mObject metadataJson = parseMetadataJson(v);
|
||||
if (filter.matchAvailableOnly && getStatusByName(getMetadataStrAttr(metadataJson, CLIENTLIB_ATTR_STATUS)) !=
|
||||
ClientLibStatus::AVAILABLE) {
|
||||
if (filter.matchAvailableOnly &&
|
||||
!isAvailableForDownload(getStatusByName(getMetadataStrAttr(metadataJson, CLIENTLIB_ATTR_STATUS)))) {
|
||||
continue;
|
||||
}
|
||||
if (filter.matchCompatibleAPI &&
|
||||
|
@ -707,4 +728,74 @@ ACTOR Future<Standalone<VectorRef<StringRef>>> listClientLibraries(Database db,
|
|||
return result;
|
||||
}
|
||||
|
||||
ACTOR Future<ClientLibStatus> getClientLibraryStatus(Database db, Standalone<StringRef> clientLibId) {
|
||||
state Key clientLibMetaKey = metadataKeyFromId(clientLibId);
|
||||
state Transaction tr(db);
|
||||
loop {
|
||||
try {
|
||||
tr.setOption(FDBTransactionOptions::READ_SYSTEM_KEYS);
|
||||
tr.setOption(FDBTransactionOptions::READ_LOCK_AWARE);
|
||||
Optional<Value> metadataOpt = wait(tr.get(clientLibMetaKey));
|
||||
if (!metadataOpt.present()) {
|
||||
TraceEvent(SevWarnAlways, "ClientLibraryNotFound").detail("Key", clientLibMetaKey);
|
||||
throw client_lib_not_found();
|
||||
}
|
||||
json_spirit::mObject metadataJson = parseMetadataJson(metadataOpt.get());
|
||||
return getStatusByName(getMetadataStrAttr(metadataJson, CLIENTLIB_ATTR_STATUS));
|
||||
} catch (Error& e) {
|
||||
wait(tr.onError(e));
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
ACTOR Future<Void> changeClientLibraryStatus(Database db,
|
||||
Standalone<StringRef> clientLibId,
|
||||
ClientLibStatus newStatus) {
|
||||
state Key clientLibMetaKey = metadataKeyFromId(clientLibId);
|
||||
state json_spirit::mObject metadataJson;
|
||||
state std::string jsStr;
|
||||
state Transaction tr;
|
||||
|
||||
if (!isValidTargetStatus(newStatus)) {
|
||||
TraceEvent(SevWarnAlways, "ClientLibraryInvalidMetadata")
|
||||
.detail("Reason", "InvalidTargetStatus")
|
||||
.detail("Status", getStatusName(newStatus));
|
||||
throw client_lib_invalid_metadata();
|
||||
}
|
||||
|
||||
loop {
|
||||
tr = Transaction(db);
|
||||
try {
|
||||
tr.setOption(FDBTransactionOptions::ACCESS_SYSTEM_KEYS);
|
||||
tr.setOption(FDBTransactionOptions::LOCK_AWARE);
|
||||
Optional<Value> metadataOpt = wait(tr.get(clientLibMetaKey));
|
||||
if (!metadataOpt.present()) {
|
||||
TraceEvent(SevWarnAlways, "ClientLibraryNotFound").detail("Key", clientLibMetaKey);
|
||||
throw client_lib_not_found();
|
||||
}
|
||||
metadataJson = parseMetadataJson(metadataOpt.get());
|
||||
ClientLibStatus prevStatus = getStatusByName(getMetadataStrAttr(metadataJson, CLIENTLIB_ATTR_STATUS));
|
||||
if (prevStatus == newStatus) {
|
||||
return Void();
|
||||
}
|
||||
metadataJson[CLIENTLIB_ATTR_STATUS] = getStatusName(newStatus);
|
||||
jsStr = json_spirit::write_string(json_spirit::mValue(metadataJson));
|
||||
tr.set(clientLibMetaKey, ValueRef(jsStr));
|
||||
|
||||
updateClientLibChangeCounter(tr, prevStatus, newStatus);
|
||||
|
||||
wait(tr.commit());
|
||||
break;
|
||||
} catch (Error& e) {
|
||||
if (e.code() == error_code_client_lib_not_found) {
|
||||
throw;
|
||||
}
|
||||
wait(tr.onError(e));
|
||||
}
|
||||
}
|
||||
|
||||
TraceEvent("ClientLibraryStatusChanged").detail("Key", clientLibMetaKey).detail("Status", getStatusName(newStatus));
|
||||
return Void();
|
||||
}
|
||||
|
||||
} // namespace ClientLibManagement
|
||||
|
|
|
@ -35,8 +35,9 @@ namespace ClientLibManagement {
|
|||
|
||||
enum class ClientLibStatus {
|
||||
DISABLED = 0,
|
||||
AVAILABLE, // 1
|
||||
UPLOADING, // 2
|
||||
UPLOADING, // 1
|
||||
DOWNLOAD, // 2
|
||||
ACTIVE, // 3
|
||||
COUNT // must be the last one
|
||||
};
|
||||
|
||||
|
@ -133,6 +134,12 @@ ACTOR Future<Void> deleteClientLibrary(Database db, Standalone<StringRef> client
|
|||
// Returns metadata JSON of each library
|
||||
ACTOR Future<Standalone<VectorRef<StringRef>>> listClientLibraries(Database db, ClientLibFilter filter);
|
||||
|
||||
// Get the current status of an uploaded client library
|
||||
ACTOR Future<ClientLibStatus> getClientLibraryStatus(Database db, Standalone<StringRef> clientLibId);
|
||||
|
||||
// Change client library metadata status
|
||||
ACTOR Future<Void> changeClientLibraryStatus(Database db, Standalone<StringRef> clientLibId, ClientLibStatus newStatus);
|
||||
|
||||
} // namespace ClientLibManagement
|
||||
|
||||
#include "flow/unactorcompiler.h"
|
||||
|
|
|
@ -0,0 +1,77 @@
|
|||
/*
|
||||
* ClientVersion.h
|
||||
*
|
||||
* This source file is part of the FoundationDB open source project
|
||||
*
|
||||
* Copyright 2013-2021 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 FDBCLIENT_CLIENT_VERSION_H
|
||||
#define FDBCLIENT_CLIENT_VERSION_H
|
||||
#pragma once
|
||||
|
||||
#include "flow/Arena.h"
|
||||
|
||||
struct ClientVersionRef {
|
||||
StringRef clientVersion;
|
||||
StringRef sourceVersion;
|
||||
StringRef protocolVersion;
|
||||
|
||||
ClientVersionRef() { initUnknown(); }
|
||||
|
||||
ClientVersionRef(Arena& arena, ClientVersionRef const& cv)
|
||||
: clientVersion(arena, cv.clientVersion), sourceVersion(arena, cv.sourceVersion),
|
||||
protocolVersion(arena, cv.protocolVersion) {}
|
||||
ClientVersionRef(StringRef clientVersion, StringRef sourceVersion, StringRef protocolVersion)
|
||||
: clientVersion(clientVersion), sourceVersion(sourceVersion), protocolVersion(protocolVersion) {}
|
||||
ClientVersionRef(StringRef versionString) {
|
||||
std::vector<StringRef> parts = versionString.splitAny(LiteralStringRef(","));
|
||||
if (parts.size() != 3) {
|
||||
initUnknown();
|
||||
return;
|
||||
}
|
||||
clientVersion = parts[0];
|
||||
sourceVersion = parts[1];
|
||||
protocolVersion = parts[2];
|
||||
}
|
||||
|
||||
void initUnknown() {
|
||||
clientVersion = LiteralStringRef("Unknown");
|
||||
sourceVersion = LiteralStringRef("Unknown");
|
||||
protocolVersion = LiteralStringRef("Unknown");
|
||||
}
|
||||
|
||||
template <class Ar>
|
||||
void serialize(Ar& ar) {
|
||||
serializer(ar, clientVersion, sourceVersion, protocolVersion);
|
||||
}
|
||||
|
||||
size_t expectedSize() const { return clientVersion.size() + sourceVersion.size() + protocolVersion.size(); }
|
||||
|
||||
bool operator<(const ClientVersionRef& rhs) const {
|
||||
if (protocolVersion != rhs.protocolVersion) {
|
||||
return protocolVersion < rhs.protocolVersion;
|
||||
}
|
||||
|
||||
// These comparisons are arbitrary because they aren't ordered
|
||||
if (clientVersion != rhs.clientVersion) {
|
||||
return clientVersion < rhs.clientVersion;
|
||||
}
|
||||
|
||||
return sourceVersion < rhs.sourceVersion;
|
||||
}
|
||||
};
|
||||
|
||||
#endif
|
|
@ -27,6 +27,7 @@
|
|||
#include "fdbclient/Status.h"
|
||||
#include "fdbclient/CommitProxyInterface.h"
|
||||
#include "fdbclient/ClientWorkerInterface.h"
|
||||
#include "fdbclient/ClientVersion.h"
|
||||
|
||||
struct ClusterInterface {
|
||||
constexpr static FileIdentifier file_identifier = 15888863;
|
||||
|
@ -38,6 +39,7 @@ struct ClusterInterface {
|
|||
RequestStream<struct ForceRecoveryRequest> forceRecovery;
|
||||
RequestStream<struct MoveShardRequest> moveShard;
|
||||
RequestStream<struct RepairSystemDataRequest> repairSystemData;
|
||||
RequestStream<struct SplitShardRequest> splitShard;
|
||||
|
||||
bool operator==(ClusterInterface const& r) const { return id() == r.id(); }
|
||||
bool operator!=(ClusterInterface const& r) const { return id() != r.id(); }
|
||||
|
@ -48,7 +50,8 @@ struct ClusterInterface {
|
|||
return openDatabase.getFuture().isReady() || failureMonitoring.getFuture().isReady() ||
|
||||
databaseStatus.getFuture().isReady() || ping.getFuture().isReady() ||
|
||||
getClientWorkers.getFuture().isReady() || forceRecovery.getFuture().isReady() ||
|
||||
moveShard.getFuture().isReady() || repairSystemData.getFuture().isReady();
|
||||
moveShard.getFuture().isReady() || repairSystemData.getFuture().isReady() ||
|
||||
splitShard.getFuture().isReady();
|
||||
}
|
||||
|
||||
void initEndpoints() {
|
||||
|
@ -60,6 +63,7 @@ struct ClusterInterface {
|
|||
forceRecovery.getEndpoint(TaskPriority::ClusterController);
|
||||
moveShard.getEndpoint(TaskPriority::ClusterController);
|
||||
repairSystemData.getEndpoint(TaskPriority::ClusterController);
|
||||
splitShard.getEndpoint(TaskPriority::ClusterController);
|
||||
}
|
||||
|
||||
template <class Ar>
|
||||
|
@ -72,7 +76,8 @@ struct ClusterInterface {
|
|||
getClientWorkers,
|
||||
forceRecovery,
|
||||
moveShard,
|
||||
repairSystemData);
|
||||
repairSystemData,
|
||||
splitShard);
|
||||
}
|
||||
};
|
||||
|
||||
|
@ -93,56 +98,6 @@ struct ClusterControllerClientInterface {
|
|||
}
|
||||
};
|
||||
|
||||
struct ClientVersionRef {
|
||||
StringRef clientVersion;
|
||||
StringRef sourceVersion;
|
||||
StringRef protocolVersion;
|
||||
|
||||
ClientVersionRef() { initUnknown(); }
|
||||
|
||||
ClientVersionRef(Arena& arena, ClientVersionRef const& cv)
|
||||
: clientVersion(arena, cv.clientVersion), sourceVersion(arena, cv.sourceVersion),
|
||||
protocolVersion(arena, cv.protocolVersion) {}
|
||||
ClientVersionRef(StringRef clientVersion, StringRef sourceVersion, StringRef protocolVersion)
|
||||
: clientVersion(clientVersion), sourceVersion(sourceVersion), protocolVersion(protocolVersion) {}
|
||||
ClientVersionRef(StringRef versionString) {
|
||||
std::vector<StringRef> parts = versionString.splitAny(LiteralStringRef(","));
|
||||
if (parts.size() != 3) {
|
||||
initUnknown();
|
||||
return;
|
||||
}
|
||||
clientVersion = parts[0];
|
||||
sourceVersion = parts[1];
|
||||
protocolVersion = parts[2];
|
||||
}
|
||||
|
||||
void initUnknown() {
|
||||
clientVersion = LiteralStringRef("Unknown");
|
||||
sourceVersion = LiteralStringRef("Unknown");
|
||||
protocolVersion = LiteralStringRef("Unknown");
|
||||
}
|
||||
|
||||
template <class Ar>
|
||||
void serialize(Ar& ar) {
|
||||
serializer(ar, clientVersion, sourceVersion, protocolVersion);
|
||||
}
|
||||
|
||||
size_t expectedSize() const { return clientVersion.size() + sourceVersion.size() + protocolVersion.size(); }
|
||||
|
||||
bool operator<(const ClientVersionRef& rhs) const {
|
||||
if (protocolVersion != rhs.protocolVersion) {
|
||||
return protocolVersion < rhs.protocolVersion;
|
||||
}
|
||||
|
||||
// These comparisons are arbitrary because they aren't ordered
|
||||
if (clientVersion != rhs.clientVersion) {
|
||||
return clientVersion < rhs.clientVersion;
|
||||
}
|
||||
|
||||
return sourceVersion < rhs.sourceVersion;
|
||||
}
|
||||
};
|
||||
|
||||
template <class T>
|
||||
struct ItemWithExamples {
|
||||
T item;
|
||||
|
@ -337,4 +292,35 @@ struct RepairSystemDataRequest {
|
|||
serializer(ar, reply);
|
||||
}
|
||||
};
|
||||
|
||||
// Returns the actual shards generated by the SplitShardRequest.
|
||||
struct SplitShardReply {
|
||||
constexpr static FileIdentifier file_identifier = 1384440;
|
||||
std::vector<KeyRange> shards;
|
||||
|
||||
SplitShardReply() {}
|
||||
explicit SplitShardReply(std::vector<KeyRange> shards) : shards{ std::move(shards) } {}
|
||||
|
||||
template <class Ar>
|
||||
void serialize(Ar& ar) {
|
||||
serializer(ar, shards);
|
||||
}
|
||||
};
|
||||
|
||||
// Split keyrange [shard.begin, shard.end) into num shards.
|
||||
// Split points are chosen as the arithmeticlly equal division points of the given range.
|
||||
struct SplitShardRequest {
|
||||
constexpr static FileIdentifier file_identifier = 1384443;
|
||||
KeyRange shard;
|
||||
int num;
|
||||
ReplyPromise<SplitShardReply> reply;
|
||||
|
||||
SplitShardRequest() : num(0) {}
|
||||
SplitShardRequest(KeyRange shard, int num) : shard{ std::move(shard) }, num(num) {}
|
||||
|
||||
template <class Ar>
|
||||
void serialize(Ar& ar) {
|
||||
serializer(ar, shard, num, reply);
|
||||
}
|
||||
};
|
||||
#endif
|
||||
|
|
|
@ -115,6 +115,9 @@ struct ClientDBInfo {
|
|||
firstCommitProxy; // not serialized, used for commitOnFirstProxy when the commit proxies vector has been shrunk
|
||||
Optional<Value> forward;
|
||||
std::vector<VersionHistory> history;
|
||||
// a counter increased every time a change of uploaded client libraries
|
||||
// happens, the clients need to be aware of
|
||||
uint64_t clientLibChangeCounter = 0;
|
||||
|
||||
ClientDBInfo() {}
|
||||
|
||||
|
@ -126,7 +129,7 @@ struct ClientDBInfo {
|
|||
if constexpr (!is_fb_function<Archive>) {
|
||||
ASSERT(ar.protocolVersion().isValid());
|
||||
}
|
||||
serializer(ar, grvProxies, commitProxies, id, forward, history);
|
||||
serializer(ar, grvProxies, commitProxies, id, forward, history, clientLibChangeCounter);
|
||||
}
|
||||
};
|
||||
|
||||
|
|
|
@ -217,6 +217,7 @@ public:
|
|||
Future<Reference<CommitProxyInfo>> getCommitProxiesFuture(bool useProvisionalProxies);
|
||||
Reference<GrvProxyInfo> getGrvProxies(bool useProvisionalProxies);
|
||||
Future<Void> onProxiesChanged() const;
|
||||
Future<Void> onClientLibStatusChanged() const;
|
||||
Future<HealthMetrics> getHealthMetrics(bool detailed);
|
||||
|
||||
// Returns the protocol version reported by the coordinator this client is connected to
|
||||
|
@ -307,7 +308,8 @@ public:
|
|||
// Key DB-specific information
|
||||
Reference<AsyncVar<Reference<IClusterConnectionRecord>>> connectionRecord;
|
||||
AsyncTrigger proxiesChangeTrigger;
|
||||
Future<Void> monitorProxiesInfoChange;
|
||||
AsyncTrigger clientLibChangeTrigger;
|
||||
Future<Void> clientDBInfoMonitor;
|
||||
Future<Void> monitorTssInfoChange;
|
||||
Future<Void> tssMismatchHandler;
|
||||
PromiseStream<std::pair<UID, std::vector<DetailedTSSMismatch>>> tssMismatchStream;
|
||||
|
@ -392,6 +394,7 @@ public:
|
|||
Counter transactionGetKeyRequests;
|
||||
Counter transactionGetValueRequests;
|
||||
Counter transactionGetRangeRequests;
|
||||
Counter transactionGetRangeAndFlatMapRequests;
|
||||
Counter transactionGetRangeStreamRequests;
|
||||
Counter transactionWatchRequests;
|
||||
Counter transactionGetAddressesForKeyRequests;
|
||||
|
@ -427,7 +430,7 @@ public:
|
|||
|
||||
int snapshotRywEnabled;
|
||||
|
||||
int transactionTracingEnabled;
|
||||
bool transactionTracingSample;
|
||||
double verifyCausalReadsProp = 0.0;
|
||||
|
||||
Future<Void> logger;
|
||||
|
|
|
@ -59,6 +59,12 @@ public:
|
|||
GetRangeLimits limits,
|
||||
bool snapshot = false,
|
||||
bool reverse = false) = 0;
|
||||
virtual ThreadFuture<RangeResult> getRangeAndFlatMap(const KeySelectorRef& begin,
|
||||
const KeySelectorRef& end,
|
||||
const StringRef& mapper,
|
||||
GetRangeLimits limits,
|
||||
bool snapshot = false,
|
||||
bool reverse = false) = 0;
|
||||
virtual ThreadFuture<Standalone<VectorRef<const char*>>> getAddressesForKey(const KeyRef& key) = 0;
|
||||
virtual ThreadFuture<Standalone<StringRef>> getVersionstamp() = 0;
|
||||
|
||||
|
|
|
@ -63,6 +63,12 @@ public:
|
|||
GetRangeLimits limits,
|
||||
Snapshot = Snapshot::False,
|
||||
Reverse = Reverse::False) = 0;
|
||||
virtual Future<RangeResult> getRangeAndFlatMap(KeySelector begin,
|
||||
KeySelector end,
|
||||
Key mapper,
|
||||
GetRangeLimits limits,
|
||||
Snapshot = Snapshot::False,
|
||||
Reverse = Reverse::False) = 0;
|
||||
virtual Future<Standalone<VectorRef<const char*>>> getAddressesForKey(Key const& key) = 0;
|
||||
virtual Future<Standalone<VectorRef<KeyRef>>> getRangeSplitPoints(KeyRange const& range, int64_t chunkSize) = 0;
|
||||
virtual Future<int64_t> getEstimatedRangeSizeBytes(KeyRange const& keys) = 0;
|
||||
|
|
|
@ -26,7 +26,9 @@
|
|||
#include "fdbclient/IClientApi.h"
|
||||
#include "fdbclient/ReadYourWrites.h"
|
||||
#include "fdbclient/Subspace.h"
|
||||
#include "flow/ObjectSerializer.h"
|
||||
#include "flow/genericactors.actor.h"
|
||||
#include "flow/serialize.h"
|
||||
|
||||
// Codec is a utility struct to convert a type to and from a Tuple. It is used by the template
|
||||
// classes below like KeyBackedProperty and KeyBackedMap to convert key parts and values
|
||||
|
@ -168,14 +170,8 @@ public:
|
|||
Future<T> getOrThrow(Reference<ReadYourWritesTransaction> tr,
|
||||
Snapshot snapshot = Snapshot::False,
|
||||
Error err = key_not_found()) const {
|
||||
auto keyCopy = key;
|
||||
auto backtrace = platform::get_backtrace();
|
||||
return map(get(tr, snapshot), [=](Optional<T> val) -> T {
|
||||
if (!val.present()) {
|
||||
TraceEvent(SevInfo, "KeyBackedProperty_KeyNotFound")
|
||||
.detail("Key", keyCopy)
|
||||
.detail("Err", err.code())
|
||||
.detail("ParentTrace", backtrace.c_str());
|
||||
throw err;
|
||||
}
|
||||
|
||||
|
@ -184,45 +180,39 @@ public:
|
|||
}
|
||||
|
||||
Future<Optional<T>> get(Database cx, Snapshot snapshot = Snapshot::False) const {
|
||||
auto& copy = *this;
|
||||
return runRYWTransaction(cx, [=](Reference<ReadYourWritesTransaction> tr) {
|
||||
return runRYWTransaction(cx, [=, self = *this](Reference<ReadYourWritesTransaction> tr) {
|
||||
tr->setOption(FDBTransactionOptions::ACCESS_SYSTEM_KEYS);
|
||||
tr->setOption(FDBTransactionOptions::LOCK_AWARE);
|
||||
|
||||
return copy.get(tr, snapshot);
|
||||
return self.get(tr, snapshot);
|
||||
});
|
||||
}
|
||||
|
||||
Future<T> getD(Database cx, Snapshot snapshot = Snapshot::False, T defaultValue = T()) const {
|
||||
auto& copy = *this;
|
||||
return runRYWTransaction(cx, [=](Reference<ReadYourWritesTransaction> tr) {
|
||||
return runRYWTransaction(cx, [=, self = *this](Reference<ReadYourWritesTransaction> tr) {
|
||||
tr->setOption(FDBTransactionOptions::ACCESS_SYSTEM_KEYS);
|
||||
tr->setOption(FDBTransactionOptions::LOCK_AWARE);
|
||||
|
||||
return copy.getD(tr, snapshot, defaultValue);
|
||||
return self.getD(tr, snapshot, defaultValue);
|
||||
});
|
||||
}
|
||||
|
||||
Future<T> getOrThrow(Database cx, Snapshot snapshot = Snapshot::False, Error err = key_not_found()) const {
|
||||
auto& copy = *this;
|
||||
return runRYWTransaction(cx, [=](Reference<ReadYourWritesTransaction> tr) {
|
||||
return runRYWTransaction(cx, [=, self = *this](Reference<ReadYourWritesTransaction> tr) {
|
||||
tr->setOption(FDBTransactionOptions::ACCESS_SYSTEM_KEYS);
|
||||
tr->setOption(FDBTransactionOptions::LOCK_AWARE);
|
||||
|
||||
return copy.getOrThrow(tr, snapshot, err);
|
||||
return self.getOrThrow(tr, snapshot, err);
|
||||
});
|
||||
}
|
||||
|
||||
void set(Reference<ReadYourWritesTransaction> tr, T const& val) { return tr->set(key, Codec<T>::pack(val).pack()); }
|
||||
|
||||
Future<Void> set(Database cx, T const& val) {
|
||||
auto _key = key;
|
||||
Value _val = Codec<T>::pack(val).pack();
|
||||
return runRYWTransaction(cx, [_key, _val](Reference<ReadYourWritesTransaction> tr) {
|
||||
return runRYWTransaction(cx, [=, self = *this](Reference<ReadYourWritesTransaction> tr) {
|
||||
tr->setOption(FDBTransactionOptions::ACCESS_SYSTEM_KEYS);
|
||||
tr->setOption(FDBTransactionOptions::LOCK_AWARE);
|
||||
tr->set(_key, _val);
|
||||
|
||||
self->set(tr, val);
|
||||
return Future<Void>(Void());
|
||||
});
|
||||
}
|
||||
|
@ -262,12 +252,12 @@ public:
|
|||
Key key;
|
||||
};
|
||||
|
||||
// Convenient read/write access to a sorted map of KeyType to ValueType that has key as its prefix
|
||||
// Convenient read/write access to a sorted map of KeyType to ValueType under prefix
|
||||
// Even though 'this' is not actually mutated, methods that change db keys are not const.
|
||||
template <typename _KeyType, typename _ValueType>
|
||||
class KeyBackedMap {
|
||||
public:
|
||||
KeyBackedMap(KeyRef key) : space(key) {}
|
||||
KeyBackedMap(KeyRef prefix) : space(prefix) {}
|
||||
|
||||
typedef _KeyType KeyType;
|
||||
typedef _ValueType ValueType;
|
||||
|
@ -336,6 +326,164 @@ public:
|
|||
Subspace space;
|
||||
};
|
||||
|
||||
// Convenient read/write access to a single value of type T stored at key
|
||||
// Even though 'this' is not actually mutated, methods that change the db key are not const.
|
||||
template <typename T, typename VersionOptions>
|
||||
class KeyBackedObjectProperty {
|
||||
public:
|
||||
KeyBackedObjectProperty(KeyRef key, VersionOptions versionOptions) : key(key), versionOptions(versionOptions) {}
|
||||
Future<Optional<T>> get(Reference<ReadYourWritesTransaction> tr, Snapshot snapshot = Snapshot::False) const {
|
||||
|
||||
return map(tr->get(key, snapshot), [vo = versionOptions](Optional<Value> const& val) -> Optional<T> {
|
||||
if (val.present())
|
||||
return ObjectReader::fromStringRef<T>(val.get(), vo);
|
||||
return {};
|
||||
});
|
||||
}
|
||||
|
||||
// Get property's value or defaultValue if it doesn't exist
|
||||
Future<T> getD(Reference<ReadYourWritesTransaction> tr,
|
||||
Snapshot snapshot = Snapshot::False,
|
||||
T defaultValue = T()) const {
|
||||
return map(get(tr, snapshot), [=](Optional<T> val) -> T { return val.present() ? val.get() : defaultValue; });
|
||||
}
|
||||
// Get property's value or throw error if it doesn't exist
|
||||
Future<T> getOrThrow(Reference<ReadYourWritesTransaction> tr,
|
||||
Snapshot snapshot = Snapshot::False,
|
||||
Error err = key_not_found()) const {
|
||||
return map(get(tr, snapshot), [=](Optional<T> val) -> T {
|
||||
if (!val.present()) {
|
||||
throw err;
|
||||
}
|
||||
|
||||
return val.get();
|
||||
});
|
||||
}
|
||||
|
||||
Future<Optional<T>> get(Database cx, Snapshot snapshot = Snapshot::False) const {
|
||||
return runRYWTransaction(cx, [=, self = *this](Reference<ReadYourWritesTransaction> tr) {
|
||||
tr->setOption(FDBTransactionOptions::ACCESS_SYSTEM_KEYS);
|
||||
tr->setOption(FDBTransactionOptions::LOCK_AWARE);
|
||||
|
||||
return self.get(tr, snapshot);
|
||||
});
|
||||
}
|
||||
|
||||
Future<T> getD(Database cx, Snapshot snapshot = Snapshot::False, T defaultValue = T()) const {
|
||||
return runRYWTransaction(cx, [=, self = *this](Reference<ReadYourWritesTransaction> tr) {
|
||||
tr->setOption(FDBTransactionOptions::ACCESS_SYSTEM_KEYS);
|
||||
tr->setOption(FDBTransactionOptions::LOCK_AWARE);
|
||||
|
||||
return self.getD(tr, snapshot, defaultValue);
|
||||
});
|
||||
}
|
||||
|
||||
Future<T> getOrThrow(Database cx, Snapshot snapshot = Snapshot::False, Error err = key_not_found()) const {
|
||||
return runRYWTransaction(cx, [=, self = *this](Reference<ReadYourWritesTransaction> tr) {
|
||||
tr->setOption(FDBTransactionOptions::ACCESS_SYSTEM_KEYS);
|
||||
tr->setOption(FDBTransactionOptions::LOCK_AWARE);
|
||||
|
||||
return self.getOrThrow(tr, snapshot, err);
|
||||
});
|
||||
}
|
||||
|
||||
void set(Reference<ReadYourWritesTransaction> tr, T const& val) {
|
||||
return tr->set(key, ObjectWriter::toValue(val, versionOptions));
|
||||
}
|
||||
|
||||
Future<Void> set(Database cx, T const& val) {
|
||||
return runRYWTransaction(cx, [=, self = *this](Reference<ReadYourWritesTransaction> tr) {
|
||||
tr->setOption(FDBTransactionOptions::ACCESS_SYSTEM_KEYS);
|
||||
tr->setOption(FDBTransactionOptions::LOCK_AWARE);
|
||||
self.set(tr, val);
|
||||
return Future<Void>(Void());
|
||||
});
|
||||
}
|
||||
|
||||
void clear(Reference<ReadYourWritesTransaction> tr) { return tr->clear(key); }
|
||||
|
||||
Key key;
|
||||
VersionOptions versionOptions;
|
||||
};
|
||||
|
||||
// Convenient read/write access to a sorted map of KeyType to ValueType under key prefix
|
||||
// ValueType is encoded / decoded with ObjectWriter/ObjectReader
|
||||
// Even though 'this' is not actually mutated, methods that change db keys are not const.
|
||||
template <typename _KeyType, typename _ValueType, typename VersionOptions>
|
||||
class KeyBackedObjectMap {
|
||||
public:
|
||||
KeyBackedObjectMap(KeyRef prefix, VersionOptions versionOptions) : space(prefix), versionOptions(versionOptions) {}
|
||||
|
||||
typedef _KeyType KeyType;
|
||||
typedef _ValueType ValueType;
|
||||
typedef std::pair<KeyType, ValueType> PairType;
|
||||
typedef std::vector<PairType> PairsType;
|
||||
|
||||
// If end is not present one key past the end of the map is used.
|
||||
Future<PairsType> getRange(Reference<ReadYourWritesTransaction> tr,
|
||||
KeyType const& begin,
|
||||
Optional<KeyType> const& end,
|
||||
int limit,
|
||||
Snapshot snapshot = Snapshot::False,
|
||||
Reverse reverse = Reverse::False) const {
|
||||
Key endKey = end.present() ? space.pack(Codec<KeyType>::pack(end.get())) : space.range().end;
|
||||
return map(
|
||||
tr->getRange(
|
||||
KeyRangeRef(space.pack(Codec<KeyType>::pack(begin)), endKey), GetRangeLimits(limit), snapshot, reverse),
|
||||
[self = *this](RangeResult const& kvs) -> PairsType {
|
||||
PairsType results;
|
||||
for (int i = 0; i < kvs.size(); ++i) {
|
||||
KeyType key = Codec<KeyType>::unpack(self.space.unpack(kvs[i].key));
|
||||
ValueType val = ObjectReader::fromStringRef<ValueType>(kvs[i].value, self.versionOptions);
|
||||
results.push_back(PairType(key, val));
|
||||
}
|
||||
return results;
|
||||
});
|
||||
}
|
||||
|
||||
Future<Optional<ValueType>> get(Reference<ReadYourWritesTransaction> tr,
|
||||
KeyType const& key,
|
||||
Snapshot snapshot = Snapshot::False) const {
|
||||
return map(tr->get(space.pack(Codec<KeyType>::pack(key)), snapshot),
|
||||
[vo = versionOptions](Optional<Value> const& val) -> Optional<ValueType> {
|
||||
if (val.present())
|
||||
return ObjectReader::fromStringRef<ValueType>(val.get(), vo);
|
||||
return {};
|
||||
});
|
||||
}
|
||||
|
||||
// Returns a Property that can be get/set that represents key's entry in this this.
|
||||
KeyBackedObjectProperty<ValueType, VersionOptions> getProperty(KeyType const& key) const {
|
||||
return KeyBackedObjectProperty<ValueType, VersionOptions>(space.pack(Codec<KeyType>::pack(key)),
|
||||
versionOptions);
|
||||
}
|
||||
|
||||
// Returns the expectedSize of the set key
|
||||
int set(Reference<ReadYourWritesTransaction> tr, KeyType const& key, ValueType const& val) {
|
||||
Key k = space.pack(Codec<KeyType>::pack(key));
|
||||
Value v = ObjectWriter::toValue(val, versionOptions);
|
||||
tr->set(k, v);
|
||||
return k.expectedSize() + v.expectedSize();
|
||||
}
|
||||
|
||||
void erase(Reference<ReadYourWritesTransaction> tr, KeyType const& key) {
|
||||
return tr->clear(space.pack(Codec<KeyType>::pack(key)));
|
||||
}
|
||||
|
||||
void erase(Reference<ITransaction> tr, KeyType const& key) {
|
||||
return tr->clear(space.pack(Codec<KeyType>::pack(key)));
|
||||
}
|
||||
|
||||
void erase(Reference<ReadYourWritesTransaction> tr, KeyType const& begin, KeyType const& end) {
|
||||
return tr->clear(KeyRangeRef(space.pack(Codec<KeyType>::pack(begin)), space.pack(Codec<KeyType>::pack(end))));
|
||||
}
|
||||
|
||||
void clear(Reference<ReadYourWritesTransaction> tr) { return tr->clear(space.range()); }
|
||||
|
||||
Subspace space;
|
||||
VersionOptions versionOptions;
|
||||
};
|
||||
|
||||
template <typename _ValueType>
|
||||
class KeyBackedSet {
|
||||
public:
|
||||
|
|
|
@ -0,0 +1,27 @@
|
|||
/*
|
||||
* LocalClientAPI.cpp
|
||||
*
|
||||
* This source file is part of the FoundationDB open source project
|
||||
*
|
||||
* Copyright 2013-2021 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/LocalClientAPI.h"
|
||||
#include "fdbclient/ThreadSafeTransaction.h"
|
||||
|
||||
IClientApi* getLocalClientAPI() {
|
||||
static IClientApi* api = new ThreadSafeApi();
|
||||
return api;
|
||||
}
|
|
@ -0,0 +1,28 @@
|
|||
/*
|
||||
* LocalClientAPI.h
|
||||
*
|
||||
* This source file is part of the FoundationDB open source project
|
||||
*
|
||||
* Copyright 2013-2021 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 FDBCLIENT_LOCALCLIENTAPI_H
|
||||
#define FDBCLIENT_LOCALCLIENTAPI_H
|
||||
#pragma once
|
||||
|
||||
#include "fdbclient/IClientApi.h"
|
||||
|
||||
IClientApi* getLocalClientAPI();
|
||||
#endif
|
|
@ -18,10 +18,10 @@
|
|||
* limitations under the License.
|
||||
*/
|
||||
|
||||
#include "fdbclient/CoordinationInterface.h"
|
||||
#include "fdbclient/MultiVersionTransaction.h"
|
||||
#include "fdbclient/MultiVersionAssignmentVars.h"
|
||||
#include "fdbclient/ThreadSafeTransaction.h"
|
||||
#include "fdbclient/ClientVersion.h"
|
||||
#include "fdbclient/LocalClientAPI.h"
|
||||
|
||||
#include "flow/network.h"
|
||||
#include "flow/Platform.h"
|
||||
|
@ -30,6 +30,10 @@
|
|||
|
||||
#include "flow/actorcompiler.h" // This must be the last #include.
|
||||
|
||||
#ifdef FDBCLIENT_NATIVEAPI_ACTOR_H
|
||||
#error "MVC should not depend on the Native API"
|
||||
#endif
|
||||
|
||||
void throwIfError(FdbCApi::fdb_error_t e) {
|
||||
if (e) {
|
||||
throw Error(e);
|
||||
|
@ -141,6 +145,41 @@ ThreadFuture<RangeResult> DLTransaction::getRange(const KeyRangeRef& keys,
|
|||
return getRange(firstGreaterOrEqual(keys.begin), firstGreaterOrEqual(keys.end), limits, snapshot, reverse);
|
||||
}
|
||||
|
||||
ThreadFuture<RangeResult> DLTransaction::getRangeAndFlatMap(const KeySelectorRef& begin,
|
||||
const KeySelectorRef& end,
|
||||
const StringRef& mapper,
|
||||
GetRangeLimits limits,
|
||||
bool snapshot,
|
||||
bool reverse) {
|
||||
FdbCApi::FDBFuture* f = api->transactionGetRangeAndFlatMap(tr,
|
||||
begin.getKey().begin(),
|
||||
begin.getKey().size(),
|
||||
begin.orEqual,
|
||||
begin.offset,
|
||||
end.getKey().begin(),
|
||||
end.getKey().size(),
|
||||
end.orEqual,
|
||||
end.offset,
|
||||
mapper.begin(),
|
||||
mapper.size(),
|
||||
limits.rows,
|
||||
limits.bytes,
|
||||
FDB_STREAMING_MODE_EXACT,
|
||||
0,
|
||||
snapshot,
|
||||
reverse);
|
||||
return toThreadFuture<RangeResult>(api, f, [](FdbCApi::FDBFuture* f, FdbCApi* api) {
|
||||
const FdbCApi::FDBKeyValue* kvs;
|
||||
int count;
|
||||
FdbCApi::fdb_bool_t more;
|
||||
FdbCApi::fdb_error_t error = api->futureGetKeyValueArray(f, &kvs, &count, &more);
|
||||
ASSERT(!error);
|
||||
|
||||
// The memory for this is stored in the FDBFuture and is released when the future gets destroyed
|
||||
return RangeResult(RangeResultRef(VectorRef<KeyValueRef>((KeyValueRef*)kvs, count), more), Arena());
|
||||
});
|
||||
}
|
||||
|
||||
ThreadFuture<Standalone<VectorRef<const char*>>> DLTransaction::getAddressesForKey(const KeyRef& key) {
|
||||
FdbCApi::FDBFuture* f = api->transactionGetAddressesForKey(tr, key.begin(), key.size());
|
||||
|
||||
|
@ -452,6 +491,7 @@ void DLApi::init() {
|
|||
loadClientFunction(&api->transactionGetKey, lib, fdbCPath, "fdb_transaction_get_key");
|
||||
loadClientFunction(&api->transactionGetAddressesForKey, lib, fdbCPath, "fdb_transaction_get_addresses_for_key");
|
||||
loadClientFunction(&api->transactionGetRange, lib, fdbCPath, "fdb_transaction_get_range");
|
||||
loadClientFunction(&api->transactionGetRangeAndFlatMap, lib, fdbCPath, "fdb_transaction_get_range_and_flat_map");
|
||||
loadClientFunction(
|
||||
&api->transactionGetVersionstamp, lib, fdbCPath, "fdb_transaction_get_versionstamp", headerVersion >= 410);
|
||||
loadClientFunction(&api->transactionSet, lib, fdbCPath, "fdb_transaction_set");
|
||||
|
@ -731,6 +771,18 @@ ThreadFuture<RangeResult> MultiVersionTransaction::getRange(const KeyRangeRef& k
|
|||
return abortableFuture(f, tr.onChange);
|
||||
}
|
||||
|
||||
ThreadFuture<RangeResult> MultiVersionTransaction::getRangeAndFlatMap(const KeySelectorRef& begin,
|
||||
const KeySelectorRef& end,
|
||||
const StringRef& mapper,
|
||||
GetRangeLimits limits,
|
||||
bool snapshot,
|
||||
bool reverse) {
|
||||
auto tr = getTransaction();
|
||||
auto f = tr.transaction ? tr.transaction->getRangeAndFlatMap(begin, end, mapper, limits, snapshot, reverse)
|
||||
: makeTimeout<RangeResult>();
|
||||
return abortableFuture(f, tr.onChange);
|
||||
}
|
||||
|
||||
ThreadFuture<Standalone<StringRef>> MultiVersionTransaction::getVersionstamp() {
|
||||
auto tr = getTransaction();
|
||||
auto f = tr.transaction ? tr.transaction->getVersionstamp() : makeTimeout<Standalone<StringRef>>();
|
||||
|
@ -888,6 +940,30 @@ ACTOR Future<Void> timeoutImpl(Reference<ThreadSingleAssignmentVar<Void>> tsav,
|
|||
return Void();
|
||||
}
|
||||
|
||||
namespace {
|
||||
|
||||
void validateOptionValuePresent(Optional<StringRef> value) {
|
||||
if (!value.present()) {
|
||||
throw invalid_option_value();
|
||||
}
|
||||
}
|
||||
|
||||
int64_t extractIntOption(Optional<StringRef> value, int64_t minValue, int64_t maxValue) {
|
||||
validateOptionValuePresent(value);
|
||||
if (value.get().size() != 8) {
|
||||
throw invalid_option_value();
|
||||
}
|
||||
|
||||
int64_t passed = *((int64_t*)(value.get().begin()));
|
||||
if (passed > maxValue || passed < minValue) {
|
||||
throw invalid_option_value();
|
||||
}
|
||||
|
||||
return passed;
|
||||
}
|
||||
|
||||
} // namespace
|
||||
|
||||
// Configure a timeout based on the options set for this transaction. This timeout only applies
|
||||
// if we don't have an underlying database object to connect with.
|
||||
void MultiVersionTransaction::setTimeout(Optional<StringRef> value) {
|
||||
|
@ -1467,7 +1543,8 @@ Reference<ClientInfo> MultiVersionApi::getLocalClient() {
|
|||
|
||||
void MultiVersionApi::selectApiVersion(int apiVersion) {
|
||||
if (!localClient) {
|
||||
localClient = makeReference<ClientInfo>(ThreadSafeApi::api);
|
||||
localClient = makeReference<ClientInfo>(getLocalClientAPI());
|
||||
ASSERT(localClient);
|
||||
}
|
||||
|
||||
if (this->apiVersion != 0 && this->apiVersion != apiVersion) {
|
||||
|
@ -1482,6 +1559,8 @@ const char* MultiVersionApi::getClientVersion() {
|
|||
return localClient->api->getClientVersion();
|
||||
}
|
||||
|
||||
namespace {
|
||||
|
||||
void validateOption(Optional<StringRef> value, bool canBePresent, bool canBeAbsent, bool canBeEmpty = true) {
|
||||
ASSERT(canBePresent || canBeAbsent);
|
||||
|
||||
|
@ -1493,6 +1572,8 @@ void validateOption(Optional<StringRef> value, bool canBePresent, bool canBeAbse
|
|||
}
|
||||
}
|
||||
|
||||
} // namespace
|
||||
|
||||
void MultiVersionApi::disableMultiVersionClientApi() {
|
||||
MutexHolder holder(lock);
|
||||
if (networkStartSetup || localClientDisabled) {
|
||||
|
|
|
@ -118,6 +118,23 @@ struct FdbCApi : public ThreadSafeReferenceCounted<FdbCApi> {
|
|||
int iteration,
|
||||
fdb_bool_t snapshot,
|
||||
fdb_bool_t reverse);
|
||||
FDBFuture* (*transactionGetRangeAndFlatMap)(FDBTransaction* tr,
|
||||
uint8_t const* beginKeyName,
|
||||
int beginKeyNameLength,
|
||||
fdb_bool_t beginOrEqual,
|
||||
int beginOffset,
|
||||
uint8_t const* endKeyName,
|
||||
int endKeyNameLength,
|
||||
fdb_bool_t endOrEqual,
|
||||
int endOffset,
|
||||
uint8_t const* mapper_name,
|
||||
int mapper_name_length,
|
||||
int limit,
|
||||
int targetBytes,
|
||||
FDBStreamingMode mode,
|
||||
int iteration,
|
||||
fdb_bool_t snapshot,
|
||||
fdb_bool_t reverse);
|
||||
FDBFuture* (*transactionGetVersionstamp)(FDBTransaction* tr);
|
||||
|
||||
void (*transactionSet)(FDBTransaction* tr,
|
||||
|
@ -219,6 +236,12 @@ public:
|
|||
GetRangeLimits limits,
|
||||
bool snapshot = false,
|
||||
bool reverse = false) override;
|
||||
ThreadFuture<RangeResult> getRangeAndFlatMap(const KeySelectorRef& begin,
|
||||
const KeySelectorRef& end,
|
||||
const StringRef& mapper,
|
||||
GetRangeLimits limits,
|
||||
bool snapshot,
|
||||
bool reverse) override;
|
||||
ThreadFuture<Standalone<VectorRef<const char*>>> getAddressesForKey(const KeyRef& key) override;
|
||||
ThreadFuture<Standalone<StringRef>> getVersionstamp() override;
|
||||
ThreadFuture<int64_t> getEstimatedRangeSizeBytes(const KeyRangeRef& keys) override;
|
||||
|
@ -360,6 +383,12 @@ public:
|
|||
GetRangeLimits limits,
|
||||
bool snapshot = false,
|
||||
bool reverse = false) override;
|
||||
ThreadFuture<RangeResult> getRangeAndFlatMap(const KeySelectorRef& begin,
|
||||
const KeySelectorRef& end,
|
||||
const StringRef& mapper,
|
||||
GetRangeLimits limits,
|
||||
bool snapshot,
|
||||
bool reverse) override;
|
||||
ThreadFuture<Standalone<VectorRef<const char*>>> getAddressesForKey(const KeyRef& key) override;
|
||||
ThreadFuture<Standalone<StringRef>> getVersionstamp() override;
|
||||
|
||||
|
|
|
@ -161,6 +161,8 @@ void DatabaseContext::addTssMapping(StorageServerInterface const& ssi, StorageSe
|
|||
TSSEndpointData(tssi.id(), tssi.getKey.getEndpoint(), metrics));
|
||||
queueModel.updateTssEndpoint(ssi.getKeyValues.getEndpoint().token.first(),
|
||||
TSSEndpointData(tssi.id(), tssi.getKeyValues.getEndpoint(), metrics));
|
||||
queueModel.updateTssEndpoint(ssi.getKeyValuesAndFlatMap.getEndpoint().token.first(),
|
||||
TSSEndpointData(tssi.id(), tssi.getKeyValuesAndFlatMap.getEndpoint(), metrics));
|
||||
queueModel.updateTssEndpoint(ssi.getKeyValuesStream.getEndpoint().token.first(),
|
||||
TSSEndpointData(tssi.id(), tssi.getKeyValuesStream.getEndpoint(), metrics));
|
||||
|
||||
|
@ -184,6 +186,7 @@ void DatabaseContext::removeTssMapping(StorageServerInterface const& ssi) {
|
|||
queueModel.removeTssEndpoint(ssi.getValue.getEndpoint().token.first());
|
||||
queueModel.removeTssEndpoint(ssi.getKey.getEndpoint().token.first());
|
||||
queueModel.removeTssEndpoint(ssi.getKeyValues.getEndpoint().token.first());
|
||||
queueModel.removeTssEndpoint(ssi.getKeyValuesAndFlatMap.getEndpoint().token.first());
|
||||
queueModel.removeTssEndpoint(ssi.getKeyValuesStream.getEndpoint().token.first());
|
||||
|
||||
queueModel.removeTssEndpoint(ssi.watchValue.getEndpoint().token.first());
|
||||
|
@ -718,14 +721,17 @@ Future<Void> attemptGRVFromOldProxies(std::vector<GrvProxyInterface> oldProxies,
|
|||
return waitForAll(replies);
|
||||
}
|
||||
|
||||
ACTOR static Future<Void> monitorProxiesChange(DatabaseContext* cx,
|
||||
Reference<AsyncVar<ClientDBInfo> const> clientDBInfo,
|
||||
AsyncTrigger* triggerVar) {
|
||||
ACTOR static Future<Void> monitorClientDBInfoChange(DatabaseContext* cx,
|
||||
Reference<AsyncVar<ClientDBInfo> const> clientDBInfo,
|
||||
AsyncTrigger* proxyChangeTrigger,
|
||||
AsyncTrigger* clientLibChangeTrigger) {
|
||||
state std::vector<CommitProxyInterface> curCommitProxies;
|
||||
state std::vector<GrvProxyInterface> curGrvProxies;
|
||||
state ActorCollection actors(false);
|
||||
state uint64_t curClientLibChangeCounter;
|
||||
curCommitProxies = clientDBInfo->get().commitProxies;
|
||||
curGrvProxies = clientDBInfo->get().grvProxies;
|
||||
curClientLibChangeCounter = clientDBInfo->get().clientLibChangeCounter;
|
||||
|
||||
loop {
|
||||
choose {
|
||||
|
@ -746,7 +752,10 @@ ACTOR static Future<Void> monitorProxiesChange(DatabaseContext* cx,
|
|||
}
|
||||
curCommitProxies = clientDBInfo->get().commitProxies;
|
||||
curGrvProxies = clientDBInfo->get().grvProxies;
|
||||
triggerVar->trigger();
|
||||
proxyChangeTrigger->trigger();
|
||||
}
|
||||
if (curClientLibChangeCounter != clientDBInfo->get().clientLibChangeCounter) {
|
||||
clientLibChangeTrigger->trigger();
|
||||
}
|
||||
}
|
||||
when(wait(actors.getResult())) { UNSTOPPABLE_ASSERT(false); }
|
||||
|
@ -1197,6 +1206,7 @@ DatabaseContext::DatabaseContext(Reference<AsyncVar<Reference<IClusterConnection
|
|||
transactionPhysicalReadsCompleted("PhysicalReadRequestsCompleted", cc),
|
||||
transactionGetKeyRequests("GetKeyRequests", cc), transactionGetValueRequests("GetValueRequests", cc),
|
||||
transactionGetRangeRequests("GetRangeRequests", cc),
|
||||
transactionGetRangeAndFlatMapRequests("GetRangeAndFlatMapRequests", cc),
|
||||
transactionGetRangeStreamRequests("GetRangeStreamRequests", cc), transactionWatchRequests("WatchRequests", cc),
|
||||
transactionGetAddressesForKeyRequests("GetAddressesForKeyRequests", cc), transactionBytesRead("BytesRead", cc),
|
||||
transactionKeysRead("KeysRead", cc), transactionMetadataVersionReads("MetadataVersionReads", cc),
|
||||
|
@ -1213,7 +1223,7 @@ DatabaseContext::DatabaseContext(Reference<AsyncVar<Reference<IClusterConnection
|
|||
transactionsExpensiveClearCostEstCount("ExpensiveClearCostEstCount", cc),
|
||||
transactionGrvFullBatches("NumGrvFullBatches", cc), transactionGrvTimedOutBatches("NumGrvTimedOutBatches", cc),
|
||||
latencies(1000), readLatencies(1000), commitLatencies(1000), GRVLatencies(1000), mutationsPerCommit(1000),
|
||||
bytesPerCommit(1000), outstandingWatches(0), transactionTracingEnabled(true), taskID(taskID),
|
||||
bytesPerCommit(1000), outstandingWatches(0), transactionTracingSample(false), taskID(taskID),
|
||||
clientInfo(clientInfo), clientInfoMonitor(clientInfoMonitor), coordinator(coordinator), apiVersion(apiVersion),
|
||||
mvCacheInsertLocation(0), healthMetricsLastUpdated(0), detailedHealthMetricsLastUpdated(0),
|
||||
smoothMidShardSize(CLIENT_KNOBS->SHARD_STAT_SMOOTH_AMOUNT),
|
||||
|
@ -1235,7 +1245,7 @@ DatabaseContext::DatabaseContext(Reference<AsyncVar<Reference<IClusterConnection
|
|||
getValueSubmitted.init(LiteralStringRef("NativeAPI.GetValueSubmitted"));
|
||||
getValueCompleted.init(LiteralStringRef("NativeAPI.GetValueCompleted"));
|
||||
|
||||
monitorProxiesInfoChange = monitorProxiesChange(this, clientInfo, &proxiesChangeTrigger);
|
||||
clientDBInfoMonitor = monitorClientDBInfoChange(this, clientInfo, &proxiesChangeTrigger, &clientLibChangeTrigger);
|
||||
tssMismatchHandler = handleTssMismatches(this);
|
||||
clientStatusUpdater.actor = clientStatusUpdateActor(this);
|
||||
cacheListMonitor = monitorCacheList(this);
|
||||
|
@ -1452,6 +1462,7 @@ DatabaseContext::DatabaseContext(const Error& err)
|
|||
transactionPhysicalReadsCompleted("PhysicalReadRequestsCompleted", cc),
|
||||
transactionGetKeyRequests("GetKeyRequests", cc), transactionGetValueRequests("GetValueRequests", cc),
|
||||
transactionGetRangeRequests("GetRangeRequests", cc),
|
||||
transactionGetRangeAndFlatMapRequests("GetRangeAndFlatMapRequests", cc),
|
||||
transactionGetRangeStreamRequests("GetRangeStreamRequests", cc), transactionWatchRequests("WatchRequests", cc),
|
||||
transactionGetAddressesForKeyRequests("GetAddressesForKeyRequests", cc), transactionBytesRead("BytesRead", cc),
|
||||
transactionKeysRead("KeysRead", cc), transactionMetadataVersionReads("MetadataVersionReads", cc),
|
||||
|
@ -1468,7 +1479,7 @@ DatabaseContext::DatabaseContext(const Error& err)
|
|||
transactionsExpensiveClearCostEstCount("ExpensiveClearCostEstCount", cc),
|
||||
transactionGrvFullBatches("NumGrvFullBatches", cc), transactionGrvTimedOutBatches("NumGrvTimedOutBatches", cc),
|
||||
latencies(1000), readLatencies(1000), commitLatencies(1000), GRVLatencies(1000), mutationsPerCommit(1000),
|
||||
bytesPerCommit(1000), transactionTracingEnabled(true), smoothMidShardSize(CLIENT_KNOBS->SHARD_STAT_SMOOTH_AMOUNT) {}
|
||||
bytesPerCommit(1000), transactionTracingSample(false), smoothMidShardSize(CLIENT_KNOBS->SHARD_STAT_SMOOTH_AMOUNT) {}
|
||||
|
||||
// Static constructor used by server processes to create a DatabaseContext
|
||||
// For internal (fdbserver) use only
|
||||
|
@ -1495,7 +1506,7 @@ Database DatabaseContext::create(Reference<AsyncVar<ClientDBInfo>> clientInfo,
|
|||
|
||||
DatabaseContext::~DatabaseContext() {
|
||||
cacheListMonitor.cancel();
|
||||
monitorProxiesInfoChange.cancel();
|
||||
clientDBInfoMonitor.cancel();
|
||||
monitorTssInfoChange.cancel();
|
||||
tssMismatchHandler.cancel();
|
||||
for (auto it = server_interf.begin(); it != server_interf.end(); it = server_interf.erase(it))
|
||||
|
@ -1584,6 +1595,10 @@ Future<Void> DatabaseContext::onProxiesChanged() const {
|
|||
return this->proxiesChangeTrigger.onTrigger();
|
||||
}
|
||||
|
||||
Future<Void> DatabaseContext::onClientLibStatusChanged() const {
|
||||
return this->clientLibChangeTrigger.onTrigger();
|
||||
}
|
||||
|
||||
bool DatabaseContext::sampleReadTags() const {
|
||||
double sampleRate = GlobalConfig::globalConfig().get(transactionTagSampleRate, CLIENT_KNOBS->READ_TAG_SAMPLE_RATE);
|
||||
return sampleRate > 0 && deterministicRandom()->random01() <= sampleRate;
|
||||
|
@ -1667,14 +1682,6 @@ void DatabaseContext::setOption(FDBDatabaseOptions::Option option, Optional<Stri
|
|||
validateOptionValueNotPresent(value);
|
||||
snapshotRywEnabled--;
|
||||
break;
|
||||
case FDBDatabaseOptions::DISTRIBUTED_TRANSACTION_TRACE_ENABLE:
|
||||
validateOptionValueNotPresent(value);
|
||||
transactionTracingEnabled++;
|
||||
break;
|
||||
case FDBDatabaseOptions::DISTRIBUTED_TRANSACTION_TRACE_DISABLE:
|
||||
validateOptionValueNotPresent(value);
|
||||
transactionTracingEnabled--;
|
||||
break;
|
||||
case FDBDatabaseOptions::USE_CONFIG_DATABASE:
|
||||
validateOptionValueNotPresent(value);
|
||||
useConfigDatabase = true;
|
||||
|
@ -3030,7 +3037,8 @@ ACTOR Future<Void> watchValueMap(Future<Version> version,
|
|||
return Void();
|
||||
}
|
||||
|
||||
void transformRangeLimits(GetRangeLimits limits, Reverse reverse, GetKeyValuesRequest& req) {
|
||||
template <class GetKeyValuesFamilyRequest>
|
||||
void transformRangeLimits(GetRangeLimits limits, Reverse reverse, GetKeyValuesFamilyRequest& req) {
|
||||
if (limits.bytes != 0) {
|
||||
if (!limits.hasRowLimit())
|
||||
req.limit = CLIENT_KNOBS->REPLY_BYTE_LIMIT; // Can't get more than this many rows anyway
|
||||
|
@ -3050,26 +3058,47 @@ void transformRangeLimits(GetRangeLimits limits, Reverse reverse, GetKeyValuesRe
|
|||
}
|
||||
}
|
||||
|
||||
ACTOR Future<RangeResult> getExactRange(Database cx,
|
||||
Version version,
|
||||
KeyRange keys,
|
||||
GetRangeLimits limits,
|
||||
Reverse reverse,
|
||||
TransactionInfo info,
|
||||
TagSet tags) {
|
||||
template <class GetKeyValuesFamilyRequest>
|
||||
RequestStream<GetKeyValuesFamilyRequest> StorageServerInterface::*getRangeRequestStream() {
|
||||
if constexpr (std::is_same<GetKeyValuesFamilyRequest, GetKeyValuesRequest>::value) {
|
||||
return &StorageServerInterface::getKeyValues;
|
||||
} else if (std::is_same<GetKeyValuesFamilyRequest, GetKeyValuesAndFlatMapRequest>::value) {
|
||||
return &StorageServerInterface::getKeyValuesAndFlatMap;
|
||||
} else {
|
||||
UNREACHABLE();
|
||||
}
|
||||
}
|
||||
|
||||
ACTOR template <class GetKeyValuesFamilyRequest, class GetKeyValuesFamilyReply>
|
||||
Future<RangeResult> getExactRange(Database cx,
|
||||
Version version,
|
||||
KeyRange keys,
|
||||
Key mapper,
|
||||
GetRangeLimits limits,
|
||||
Reverse reverse,
|
||||
TransactionInfo info,
|
||||
TagSet tags) {
|
||||
state RangeResult output;
|
||||
state Span span("NAPI:getExactRange"_loc, info.spanID);
|
||||
|
||||
// printf("getExactRange( '%s', '%s' )\n", keys.begin.toString().c_str(), keys.end.toString().c_str());
|
||||
loop {
|
||||
state std::vector<std::pair<KeyRange, Reference<LocationInfo>>> locations = wait(getKeyRangeLocations(
|
||||
cx, keys, CLIENT_KNOBS->GET_RANGE_SHARD_LIMIT, reverse, &StorageServerInterface::getKeyValues, info));
|
||||
state std::vector<std::pair<KeyRange, Reference<LocationInfo>>> locations =
|
||||
wait(getKeyRangeLocations(cx,
|
||||
keys,
|
||||
CLIENT_KNOBS->GET_RANGE_SHARD_LIMIT,
|
||||
reverse,
|
||||
getRangeRequestStream<GetKeyValuesFamilyRequest>(),
|
||||
info));
|
||||
ASSERT(locations.size());
|
||||
state int shard = 0;
|
||||
loop {
|
||||
const KeyRangeRef& range = locations[shard].first;
|
||||
|
||||
GetKeyValuesRequest req;
|
||||
GetKeyValuesFamilyRequest req;
|
||||
req.mapper = mapper;
|
||||
req.arena.dependsOn(mapper.arena());
|
||||
|
||||
req.version = version;
|
||||
req.begin = firstGreaterOrEqual(range.begin);
|
||||
req.end = firstGreaterOrEqual(range.end);
|
||||
|
@ -3099,14 +3128,14 @@ ACTOR Future<RangeResult> getExactRange(Database cx,
|
|||
.detail("Servers", locations[shard].second->description());*/
|
||||
}
|
||||
++cx->transactionPhysicalReads;
|
||||
state GetKeyValuesReply rep;
|
||||
state GetKeyValuesFamilyReply rep;
|
||||
try {
|
||||
choose {
|
||||
when(wait(cx->connectionFileChanged())) { throw transaction_too_old(); }
|
||||
when(GetKeyValuesReply _rep =
|
||||
when(GetKeyValuesFamilyReply _rep =
|
||||
wait(loadBalance(cx.getPtr(),
|
||||
locations[shard].second,
|
||||
&StorageServerInterface::getKeyValues,
|
||||
getRangeRequestStream<GetKeyValuesFamilyRequest>(),
|
||||
req,
|
||||
TaskPriority::DefaultPromiseEndpoint,
|
||||
AtMostOnce::False,
|
||||
|
@ -3156,7 +3185,7 @@ ACTOR Future<RangeResult> getExactRange(Database cx,
|
|||
.detail("BlockBytes", rep.data.expectedSize());
|
||||
ASSERT(false);
|
||||
}
|
||||
TEST(true); // GetKeyValuesReply.more in getExactRange
|
||||
TEST(true); // GetKeyValuesFamilyReply.more in getExactRange
|
||||
// Make next request to the same shard with a beginning key just after the last key returned
|
||||
if (reverse)
|
||||
locations[shard].first =
|
||||
|
@ -3232,14 +3261,16 @@ Future<Key> resolveKey(Database const& cx,
|
|||
return getKey(cx, key, version, info, tags);
|
||||
}
|
||||
|
||||
ACTOR Future<RangeResult> getRangeFallback(Database cx,
|
||||
Version version,
|
||||
KeySelector begin,
|
||||
KeySelector end,
|
||||
GetRangeLimits limits,
|
||||
Reverse reverse,
|
||||
TransactionInfo info,
|
||||
TagSet tags) {
|
||||
ACTOR template <class GetKeyValuesFamilyRequest, class GetKeyValuesFamilyReply>
|
||||
Future<RangeResult> getRangeFallback(Database cx,
|
||||
Version version,
|
||||
KeySelector begin,
|
||||
KeySelector end,
|
||||
Key mapper,
|
||||
GetRangeLimits limits,
|
||||
Reverse reverse,
|
||||
TransactionInfo info,
|
||||
TagSet tags) {
|
||||
if (version == latestVersion) {
|
||||
state Transaction transaction(cx);
|
||||
transaction.setOption(FDBTransactionOptions::CAUSAL_READ_RISKY);
|
||||
|
@ -3262,7 +3293,8 @@ ACTOR Future<RangeResult> getRangeFallback(Database cx,
|
|||
// if b is allKeys.begin, we have either read through the beginning of the database,
|
||||
// or allKeys.begin exists in the database and will be part of the conflict range anyways
|
||||
|
||||
RangeResult _r = wait(getExactRange(cx, version, KeyRangeRef(b, e), limits, reverse, info, tags));
|
||||
RangeResult _r = wait(getExactRange<GetKeyValuesFamilyRequest, GetKeyValuesFamilyReply>(
|
||||
cx, version, KeyRangeRef(b, e), mapper, limits, reverse, info, tags));
|
||||
RangeResult r = _r;
|
||||
|
||||
if (b == allKeys.begin && ((reverse && !r.more) || !reverse))
|
||||
|
@ -3287,6 +3319,7 @@ ACTOR Future<RangeResult> getRangeFallback(Database cx,
|
|||
return r;
|
||||
}
|
||||
|
||||
// TODO: Client should add mapped keys to conflict ranges.
|
||||
void getRangeFinished(Database cx,
|
||||
Reference<TransactionLogInfo> trLogInfo,
|
||||
double startTime,
|
||||
|
@ -3341,17 +3374,23 @@ void getRangeFinished(Database cx,
|
|||
}
|
||||
}
|
||||
|
||||
ACTOR Future<RangeResult> getRange(Database cx,
|
||||
Reference<TransactionLogInfo> trLogInfo,
|
||||
Future<Version> fVersion,
|
||||
KeySelector begin,
|
||||
KeySelector end,
|
||||
GetRangeLimits limits,
|
||||
Promise<std::pair<Key, Key>> conflictRange,
|
||||
Snapshot snapshot,
|
||||
Reverse reverse,
|
||||
TransactionInfo info,
|
||||
TagSet tags) {
|
||||
// GetKeyValuesFamilyRequest: GetKeyValuesRequest or GetKeyValuesAndFlatMapRequest
|
||||
// GetKeyValuesFamilyReply: GetKeyValuesReply or GetKeyValuesAndFlatMapReply
|
||||
// Sadly we need GetKeyValuesFamilyReply because cannot do something like: state
|
||||
// REPLY_TYPE(GetKeyValuesFamilyRequest) rep;
|
||||
ACTOR template <class GetKeyValuesFamilyRequest, class GetKeyValuesFamilyReply>
|
||||
Future<RangeResult> getRange(Database cx,
|
||||
Reference<TransactionLogInfo> trLogInfo,
|
||||
Future<Version> fVersion,
|
||||
KeySelector begin,
|
||||
KeySelector end,
|
||||
Key mapper,
|
||||
GetRangeLimits limits,
|
||||
Promise<std::pair<Key, Key>> conflictRange,
|
||||
Snapshot snapshot,
|
||||
Reverse reverse,
|
||||
TransactionInfo info,
|
||||
TagSet tags) {
|
||||
state GetRangeLimits originalLimits(limits);
|
||||
state KeySelector originalBegin = begin;
|
||||
state KeySelector originalEnd = end;
|
||||
|
@ -3385,11 +3424,13 @@ ACTOR Future<RangeResult> getRange(Database cx,
|
|||
|
||||
Key locationKey = reverse ? Key(end.getKey(), end.arena()) : Key(begin.getKey(), begin.arena());
|
||||
Reverse locationBackward{ reverse ? (end - 1).isBackward() : begin.isBackward() };
|
||||
state std::pair<KeyRange, Reference<LocationInfo>> beginServer =
|
||||
wait(getKeyLocation(cx, locationKey, &StorageServerInterface::getKeyValues, info, locationBackward));
|
||||
state std::pair<KeyRange, Reference<LocationInfo>> beginServer = wait(getKeyLocation(
|
||||
cx, locationKey, getRangeRequestStream<GetKeyValuesFamilyRequest>(), info, locationBackward));
|
||||
state KeyRange shard = beginServer.first;
|
||||
state bool modifiedSelectors = false;
|
||||
state GetKeyValuesRequest req;
|
||||
state GetKeyValuesFamilyRequest req;
|
||||
req.mapper = mapper;
|
||||
req.arena.dependsOn(mapper.arena());
|
||||
|
||||
req.isFetchKeys = (info.taskID == TaskPriority::FetchKeys);
|
||||
req.version = readVersion;
|
||||
|
@ -3448,17 +3489,17 @@ ACTOR Future<RangeResult> getRange(Database cx,
|
|||
}
|
||||
|
||||
++cx->transactionPhysicalReads;
|
||||
state GetKeyValuesReply rep;
|
||||
state GetKeyValuesFamilyReply rep;
|
||||
try {
|
||||
if (CLIENT_BUGGIFY_WITH_PROB(.01)) {
|
||||
throw deterministicRandom()->randomChoice(
|
||||
std::vector<Error>{ transaction_too_old(), future_version() });
|
||||
}
|
||||
// state AnnotateActor annotation(currentLineage);
|
||||
GetKeyValuesReply _rep =
|
||||
GetKeyValuesFamilyReply _rep =
|
||||
wait(loadBalance(cx.getPtr(),
|
||||
beginServer.second,
|
||||
&StorageServerInterface::getKeyValues,
|
||||
getRangeRequestStream<GetKeyValuesFamilyRequest>(),
|
||||
req,
|
||||
TaskPriority::DefaultPromiseEndpoint,
|
||||
AtMostOnce::False,
|
||||
|
@ -3558,11 +3599,11 @@ ACTOR Future<RangeResult> getRange(Database cx,
|
|||
|
||||
if (!rep.more) {
|
||||
ASSERT(modifiedSelectors);
|
||||
TEST(true); // !GetKeyValuesReply.more and modifiedSelectors in getRange
|
||||
TEST(true); // !GetKeyValuesFamilyReply.more and modifiedSelectors in getRange
|
||||
|
||||
if (!rep.data.size()) {
|
||||
RangeResult result = wait(getRangeFallback(
|
||||
cx, version, originalBegin, originalEnd, originalLimits, reverse, info, tags));
|
||||
RangeResult result = wait(getRangeFallback<GetKeyValuesFamilyRequest, GetKeyValuesFamilyReply>(
|
||||
cx, version, originalBegin, originalEnd, mapper, originalLimits, reverse, info, tags));
|
||||
getRangeFinished(cx,
|
||||
trLogInfo,
|
||||
startTime,
|
||||
|
@ -3580,7 +3621,7 @@ ACTOR Future<RangeResult> getRange(Database cx,
|
|||
else
|
||||
begin = firstGreaterOrEqual(shard.end);
|
||||
} else {
|
||||
TEST(true); // GetKeyValuesReply.more in getRange
|
||||
TEST(true); // GetKeyValuesFamilyReply.more in getRange
|
||||
if (reverse)
|
||||
end = firstGreaterOrEqual(output[output.size() - 1].key);
|
||||
else
|
||||
|
@ -3598,8 +3639,8 @@ ACTOR Future<RangeResult> getRange(Database cx,
|
|||
Reverse{ reverse ? (end - 1).isBackward() : begin.isBackward() });
|
||||
|
||||
if (e.code() == error_code_wrong_shard_server) {
|
||||
RangeResult result = wait(getRangeFallback(
|
||||
cx, version, originalBegin, originalEnd, originalLimits, reverse, info, tags));
|
||||
RangeResult result = wait(getRangeFallback<GetKeyValuesFamilyRequest, GetKeyValuesFamilyReply>(
|
||||
cx, version, originalBegin, originalEnd, mapper, originalLimits, reverse, info, tags));
|
||||
getRangeFinished(cx,
|
||||
trLogInfo,
|
||||
startTime,
|
||||
|
@ -4165,17 +4206,18 @@ Future<RangeResult> getRange(Database const& cx,
|
|||
Reverse const& reverse,
|
||||
TransactionInfo const& info,
|
||||
TagSet const& tags) {
|
||||
return getRange(cx,
|
||||
Reference<TransactionLogInfo>(),
|
||||
fVersion,
|
||||
begin,
|
||||
end,
|
||||
limits,
|
||||
Promise<std::pair<Key, Key>>(),
|
||||
Snapshot::True,
|
||||
reverse,
|
||||
info,
|
||||
tags);
|
||||
return getRange<GetKeyValuesRequest, GetKeyValuesReply>(cx,
|
||||
Reference<TransactionLogInfo>(),
|
||||
fVersion,
|
||||
begin,
|
||||
end,
|
||||
""_sr,
|
||||
limits,
|
||||
Promise<std::pair<Key, Key>>(),
|
||||
Snapshot::True,
|
||||
reverse,
|
||||
info,
|
||||
tags);
|
||||
}
|
||||
|
||||
bool DatabaseContext::debugUseTags = false;
|
||||
|
@ -4206,9 +4248,15 @@ void debugAddTags(Transaction* tr) {
|
|||
}
|
||||
}
|
||||
|
||||
SpanID generateSpanID(int transactionTracingEnabled) {
|
||||
SpanID generateSpanID(bool transactionTracingSample, SpanID parentContext = SpanID()) {
|
||||
uint64_t txnId = deterministicRandom()->randomUInt64();
|
||||
if (transactionTracingEnabled > 0) {
|
||||
if (parentContext.isValid()) {
|
||||
if (parentContext.first() > 0) {
|
||||
txnId = parentContext.first();
|
||||
}
|
||||
uint64_t tokenId = parentContext.second() > 0 ? deterministicRandom()->randomUInt64() : 0;
|
||||
return SpanID(txnId, tokenId);
|
||||
} else if (transactionTracingSample) {
|
||||
uint64_t tokenId = deterministicRandom()->random01() <= FLOW_KNOBS->TRACING_SAMPLE_RATE
|
||||
? deterministicRandom()->randomUInt64()
|
||||
: 0;
|
||||
|
@ -4218,10 +4266,10 @@ SpanID generateSpanID(int transactionTracingEnabled) {
|
|||
}
|
||||
}
|
||||
|
||||
Transaction::Transaction() : info(TaskPriority::DefaultEndpoint, generateSpanID(true)) {}
|
||||
Transaction::Transaction() : info(TaskPriority::DefaultEndpoint, generateSpanID(false)) {}
|
||||
|
||||
Transaction::Transaction(Database const& cx)
|
||||
: info(cx->taskID, generateSpanID(cx->transactionTracingEnabled)), numErrors(0), options(cx),
|
||||
: info(cx->taskID, generateSpanID(cx->transactionTracingSample)), numErrors(0), options(cx),
|
||||
span(info.spanID, "Transaction"_loc), trLogInfo(createTrLogInfoProbabilistically(cx)), cx(cx),
|
||||
backoff(CLIENT_KNOBS->DEFAULT_BACKOFF), committedVersion(invalidVersion), tr(info.spanID) {
|
||||
if (DatabaseContext::debugUseTags) {
|
||||
|
@ -4470,13 +4518,26 @@ Future<Key> Transaction::getKey(const KeySelector& key, Snapshot snapshot) {
|
|||
return getKeyAndConflictRange(cx, key, getReadVersion(), conflictRange, info, options.readTags);
|
||||
}
|
||||
|
||||
Future<RangeResult> Transaction::getRange(const KeySelector& begin,
|
||||
const KeySelector& end,
|
||||
GetRangeLimits limits,
|
||||
Snapshot snapshot,
|
||||
Reverse reverse) {
|
||||
template <class GetKeyValuesFamilyRequest>
|
||||
void increaseCounterForRequest(Database cx) {
|
||||
if constexpr (std::is_same<GetKeyValuesFamilyRequest, GetKeyValuesRequest>::value) {
|
||||
++cx->transactionGetRangeRequests;
|
||||
} else if (std::is_same<GetKeyValuesFamilyRequest, GetKeyValuesAndFlatMapRequest>::value) {
|
||||
++cx->transactionGetRangeAndFlatMapRequests;
|
||||
} else {
|
||||
UNREACHABLE();
|
||||
}
|
||||
}
|
||||
|
||||
template <class GetKeyValuesFamilyRequest, class GetKeyValuesFamilyReply>
|
||||
Future<RangeResult> Transaction::getRangeInternal(const KeySelector& begin,
|
||||
const KeySelector& end,
|
||||
const Key& mapper,
|
||||
GetRangeLimits limits,
|
||||
Snapshot snapshot,
|
||||
Reverse reverse) {
|
||||
++cx->transactionLogicalReads;
|
||||
++cx->transactionGetRangeRequests;
|
||||
increaseCounterForRequest<GetKeyValuesFamilyRequest>(cx);
|
||||
|
||||
if (limits.isReached())
|
||||
return RangeResult();
|
||||
|
@ -4508,8 +4569,37 @@ Future<RangeResult> Transaction::getRange(const KeySelector& begin,
|
|||
extraConflictRanges.push_back(conflictRange.getFuture());
|
||||
}
|
||||
|
||||
return ::getRange(
|
||||
cx, trLogInfo, getReadVersion(), b, e, limits, conflictRange, snapshot, reverse, info, options.readTags);
|
||||
return ::getRange<GetKeyValuesFamilyRequest, GetKeyValuesFamilyReply>(cx,
|
||||
trLogInfo,
|
||||
getReadVersion(),
|
||||
b,
|
||||
e,
|
||||
mapper,
|
||||
limits,
|
||||
conflictRange,
|
||||
snapshot,
|
||||
reverse,
|
||||
info,
|
||||
options.readTags);
|
||||
}
|
||||
|
||||
Future<RangeResult> Transaction::getRange(const KeySelector& begin,
|
||||
const KeySelector& end,
|
||||
GetRangeLimits limits,
|
||||
Snapshot snapshot,
|
||||
Reverse reverse) {
|
||||
return getRangeInternal<GetKeyValuesRequest, GetKeyValuesReply>(begin, end, ""_sr, limits, snapshot, reverse);
|
||||
}
|
||||
|
||||
Future<RangeResult> Transaction::getRangeAndFlatMap(const KeySelector& begin,
|
||||
const KeySelector& end,
|
||||
const Key& mapper,
|
||||
GetRangeLimits limits,
|
||||
Snapshot snapshot,
|
||||
Reverse reverse) {
|
||||
|
||||
return getRangeInternal<GetKeyValuesAndFlatMapRequest, GetKeyValuesAndFlatMapReply>(
|
||||
begin, end, mapper, limits, snapshot, reverse);
|
||||
}
|
||||
|
||||
Future<RangeResult> Transaction::getRange(const KeySelector& begin,
|
||||
|
@ -4846,8 +4936,8 @@ void Transaction::reset() {
|
|||
|
||||
void Transaction::fullReset() {
|
||||
reset();
|
||||
span = Span(span.location);
|
||||
info.spanID = span.context;
|
||||
info.spanID = generateSpanID(cx->transactionTracingSample);
|
||||
span = Span(info.spanID, "Transaction"_loc);
|
||||
backoff = CLIENT_KNOBS->DEFAULT_BACKOFF;
|
||||
}
|
||||
|
||||
|
@ -5369,6 +5459,9 @@ ACTOR Future<Void> commitAndWatch(Transaction* self) {
|
|||
try {
|
||||
wait(self->commitMutations());
|
||||
|
||||
self->getDatabase()->transactionTracingSample =
|
||||
(self->getCommittedVersion() % 60000000) < (60000000 * FLOW_KNOBS->TRACING_SAMPLE_RATE);
|
||||
|
||||
if (!self->watches.empty()) {
|
||||
self->setupWatches();
|
||||
}
|
||||
|
@ -5872,7 +5965,7 @@ Future<Version> Transaction::getReadVersion(uint32_t flags) {
|
|||
}
|
||||
|
||||
Location location = "NAPI:getReadVersion"_loc;
|
||||
UID spanContext = generateSpanID(cx->transactionTracingEnabled);
|
||||
UID spanContext = generateSpanID(cx->transactionTracingSample, info.spanID);
|
||||
auto const req = DatabaseContext::VersionRequest(spanContext, options.tags, info.debugID);
|
||||
batcher.stream.send(req);
|
||||
startTime = now();
|
||||
|
@ -6235,7 +6328,7 @@ ACTOR Future<std::pair<Optional<StorageMetrics>, int>> waitStorageMetrics(Databa
|
|||
StorageMetrics permittedError,
|
||||
int shardLimit,
|
||||
int expectedShardCount) {
|
||||
state Span span("NAPI:WaitStorageMetrics"_loc, generateSpanID(cx->transactionTracingEnabled));
|
||||
state Span span("NAPI:WaitStorageMetrics"_loc, generateSpanID(cx->transactionTracingSample));
|
||||
loop {
|
||||
std::vector<std::pair<KeyRange, Reference<LocationInfo>>> locations =
|
||||
wait(getKeyRangeLocations(cx,
|
||||
|
|
|
@ -289,6 +289,23 @@ public:
|
|||
reverse);
|
||||
}
|
||||
|
||||
[[nodiscard]] Future<RangeResult> getRangeAndFlatMap(const KeySelector& begin,
|
||||
const KeySelector& end,
|
||||
const Key& mapper,
|
||||
GetRangeLimits limits,
|
||||
Snapshot = Snapshot::False,
|
||||
Reverse = Reverse::False);
|
||||
|
||||
private:
|
||||
template <class GetKeyValuesFamilyRequest, class GetKeyValuesFamilyReply>
|
||||
Future<RangeResult> getRangeInternal(const KeySelector& begin,
|
||||
const KeySelector& end,
|
||||
const Key& mapper,
|
||||
GetRangeLimits limits,
|
||||
Snapshot snapshot,
|
||||
Reverse reverse);
|
||||
|
||||
public:
|
||||
// A method for streaming data from the storage server that is more efficient than getRange when reading large
|
||||
// amounts of data
|
||||
[[nodiscard]] Future<Void> getRangeStream(const PromiseStream<Standalone<RangeResultRef>>& results,
|
||||
|
|
|
@ -50,6 +50,14 @@ public:
|
|||
GetRangeLimits limits,
|
||||
Snapshot = Snapshot::False,
|
||||
Reverse = Reverse::False) override;
|
||||
Future<RangeResult> getRangeAndFlatMap(KeySelector begin,
|
||||
KeySelector end,
|
||||
Key mapper,
|
||||
GetRangeLimits limits,
|
||||
Snapshot = Snapshot::False,
|
||||
Reverse = Reverse::False) override {
|
||||
throw client_invalid_operation();
|
||||
}
|
||||
void set(KeyRef const& key, ValueRef const& value) override;
|
||||
void clear(KeyRangeRef const&) override { throw client_invalid_operation(); }
|
||||
void clear(KeyRef const&) override;
|
||||
|
|
|
@ -74,6 +74,16 @@ public:
|
|||
using Result = RangeResult;
|
||||
};
|
||||
|
||||
template <bool reverse>
|
||||
struct GetRangeAndFlatMapReq {
|
||||
GetRangeAndFlatMapReq(KeySelector begin, KeySelector end, Key mapper, GetRangeLimits limits)
|
||||
: begin(begin), end(end), mapper(mapper), limits(limits) {}
|
||||
KeySelector begin, end;
|
||||
Key mapper;
|
||||
GetRangeLimits limits;
|
||||
using Result = RangeResult;
|
||||
};
|
||||
|
||||
// read() Performs a read (get, getKey, getRange, etc), in the context of the given transaction. Snapshot or RYW
|
||||
// reads are distingushed by the type Iter being SnapshotCache::iterator or RYWIterator. Fills in the snapshot cache
|
||||
// as a side effect but does not affect conflict ranges. Some (indicated) overloads of read are required to update
|
||||
|
@ -203,6 +213,36 @@ public:
|
|||
return v;
|
||||
}
|
||||
|
||||
ACTOR template <bool backwards>
|
||||
static Future<RangeResult> readThroughAndFlatMap(ReadYourWritesTransaction* ryw,
|
||||
GetRangeAndFlatMapReq<backwards> read,
|
||||
Snapshot snapshot) {
|
||||
if (backwards && read.end.offset > 1) {
|
||||
// FIXME: Optimistically assume that this will not run into the system keys, and only reissue if the result
|
||||
// actually does.
|
||||
Key key = wait(ryw->tr.getKey(read.end, snapshot));
|
||||
if (key > ryw->getMaxReadKey())
|
||||
read.end = firstGreaterOrEqual(ryw->getMaxReadKey());
|
||||
else
|
||||
read.end = KeySelector(firstGreaterOrEqual(key), key.arena());
|
||||
}
|
||||
|
||||
RangeResult v = wait(ryw->tr.getRangeAndFlatMap(
|
||||
read.begin, read.end, read.mapper, read.limits, snapshot, backwards ? Reverse::True : Reverse::False));
|
||||
KeyRef maxKey = ryw->getMaxReadKey();
|
||||
if (v.size() > 0) {
|
||||
if (!backwards && v[v.size() - 1].key >= maxKey) {
|
||||
state RangeResult _v = v;
|
||||
int i = _v.size() - 2;
|
||||
for (; i >= 0 && _v[i].key >= maxKey; --i) {
|
||||
}
|
||||
return RangeResult(RangeResultRef(VectorRef<KeyValueRef>(&_v[0], i + 1), false), _v.arena());
|
||||
}
|
||||
}
|
||||
|
||||
return v;
|
||||
}
|
||||
|
||||
// addConflictRange(ryw,read,result) is called after a serializable read and is responsible for adding the relevant
|
||||
// conflict range
|
||||
|
||||
|
@ -309,6 +349,15 @@ public:
|
|||
}
|
||||
}
|
||||
ACTOR template <class Req>
|
||||
static Future<typename Req::Result> readWithConflictRangeThroughAndFlatMap(ReadYourWritesTransaction* ryw,
|
||||
Req req,
|
||||
Snapshot snapshot) {
|
||||
choose {
|
||||
when(typename Req::Result result = wait(readThroughAndFlatMap(ryw, req, snapshot))) { return result; }
|
||||
when(wait(ryw->resetPromise.getFuture())) { throw internal_error(); }
|
||||
}
|
||||
}
|
||||
ACTOR template <class Req>
|
||||
static Future<typename Req::Result> readWithConflictRangeSnapshot(ReadYourWritesTransaction* ryw, Req req) {
|
||||
state SnapshotCache::iterator it(&ryw->cache, &ryw->writes);
|
||||
choose {
|
||||
|
@ -344,6 +393,19 @@ public:
|
|||
return readWithConflictRangeRYW(ryw, req, snapshot);
|
||||
}
|
||||
|
||||
template <class Req>
|
||||
static inline Future<typename Req::Result> readWithConflictRangeAndFlatMap(ReadYourWritesTransaction* ryw,
|
||||
Req const& req,
|
||||
Snapshot snapshot) {
|
||||
// For now, getRangeAndFlatMap is only supported if transaction use snapshot isolation AND read-your-writes is
|
||||
// disabled.
|
||||
if (snapshot && ryw->options.readYourWritesDisabled) {
|
||||
return readWithConflictRangeThroughAndFlatMap(ryw, req, snapshot);
|
||||
}
|
||||
TEST(true); // readWithConflictRangeRYW not supported for getRangeAndFlatMap
|
||||
throw client_invalid_operation();
|
||||
}
|
||||
|
||||
template <class Iter>
|
||||
static void resolveKeySelectorFromCache(KeySelector& key,
|
||||
Iter& it,
|
||||
|
@ -1509,6 +1571,65 @@ Future<RangeResult> ReadYourWritesTransaction::getRange(const KeySelector& begin
|
|||
return getRange(begin, end, GetRangeLimits(limit), snapshot, reverse);
|
||||
}
|
||||
|
||||
Future<RangeResult> ReadYourWritesTransaction::getRangeAndFlatMap(KeySelector begin,
|
||||
KeySelector end,
|
||||
Key mapper,
|
||||
GetRangeLimits limits,
|
||||
Snapshot snapshot,
|
||||
Reverse reverse) {
|
||||
if (getDatabase()->apiVersionAtLeast(630)) {
|
||||
if (specialKeys.contains(begin.getKey()) && specialKeys.begin <= end.getKey() &&
|
||||
end.getKey() <= specialKeys.end) {
|
||||
TEST(true); // Special key space get range (FlatMap)
|
||||
throw client_invalid_operation(); // Not support special keys.
|
||||
}
|
||||
} else {
|
||||
if (begin.getKey() == LiteralStringRef("\xff\xff/worker_interfaces")) {
|
||||
throw client_invalid_operation(); // Not support special keys.
|
||||
}
|
||||
}
|
||||
|
||||
if (checkUsedDuringCommit()) {
|
||||
return used_during_commit();
|
||||
}
|
||||
|
||||
if (resetPromise.isSet())
|
||||
return resetPromise.getFuture().getError();
|
||||
|
||||
KeyRef maxKey = getMaxReadKey();
|
||||
if (begin.getKey() > maxKey || end.getKey() > maxKey)
|
||||
return key_outside_legal_range();
|
||||
|
||||
// This optimization prevents nullptr operations from being added to the conflict range
|
||||
if (limits.isReached()) {
|
||||
TEST(true); // RYW range read limit 0 (FlatMap)
|
||||
return RangeResult();
|
||||
}
|
||||
|
||||
if (!limits.isValid())
|
||||
return range_limits_invalid();
|
||||
|
||||
if (begin.orEqual)
|
||||
begin.removeOrEqual(begin.arena());
|
||||
|
||||
if (end.orEqual)
|
||||
end.removeOrEqual(end.arena());
|
||||
|
||||
if (begin.offset >= end.offset && begin.getKey() >= end.getKey()) {
|
||||
TEST(true); // RYW range inverted (FlatMap)
|
||||
return RangeResult();
|
||||
}
|
||||
|
||||
Future<RangeResult> result =
|
||||
reverse ? RYWImpl::readWithConflictRangeAndFlatMap(
|
||||
this, RYWImpl::GetRangeAndFlatMapReq<true>(begin, end, mapper, limits), snapshot)
|
||||
: RYWImpl::readWithConflictRangeAndFlatMap(
|
||||
this, RYWImpl::GetRangeAndFlatMapReq<false>(begin, end, mapper, limits), snapshot);
|
||||
|
||||
reading.add(success(result));
|
||||
return result;
|
||||
}
|
||||
|
||||
Future<Standalone<VectorRef<const char*>>> ReadYourWritesTransaction::getAddressesForKey(const Key& key) {
|
||||
if (checkUsedDuringCommit()) {
|
||||
return used_during_commit();
|
||||
|
|
|
@ -104,6 +104,12 @@ public:
|
|||
snapshot,
|
||||
reverse);
|
||||
}
|
||||
Future<RangeResult> getRangeAndFlatMap(KeySelector begin,
|
||||
KeySelector end,
|
||||
Key mapper,
|
||||
GetRangeLimits limits,
|
||||
Snapshot = Snapshot::False,
|
||||
Reverse = Reverse::False) override;
|
||||
|
||||
[[nodiscard]] Future<Standalone<VectorRef<const char*>>> getAddressesForKey(const Key& key) override;
|
||||
Future<Standalone<VectorRef<KeyRef>>> getRangeSplitPoints(const KeyRange& range, int64_t chunkSize) override;
|
||||
|
|
|
@ -349,6 +349,11 @@ void ServerKnobs::initialize(Randomize randomize, ClientKnobs* clientKnobs, IsSi
|
|||
init( ROCKSDB_READ_VALUE_TIMEOUT, 5.0 );
|
||||
init( ROCKSDB_READ_VALUE_PREFIX_TIMEOUT, 5.0 );
|
||||
init( ROCKSDB_READ_RANGE_TIMEOUT, 5.0 );
|
||||
init( ROCKSDB_READ_QUEUE_WAIT, 1.0 );
|
||||
init( ROCKSDB_READ_QUEUE_HARD_MAX, 1000 );
|
||||
init( ROCKSDB_READ_QUEUE_SOFT_MAX, 500 );
|
||||
init( ROCKSDB_FETCH_QUEUE_HARD_MAX, 100 );
|
||||
init( ROCKSDB_FETCH_QUEUE_SOFT_MAX, 50 );
|
||||
|
||||
// Leader election
|
||||
bool longLeaderElection = randomize && BUGGIFY;
|
||||
|
@ -644,6 +649,8 @@ void ServerKnobs::initialize(Randomize randomize, ClientKnobs* clientKnobs, IsSi
|
|||
init( MAX_STORAGE_COMMIT_TIME, 120.0 ); //The max fsync stall time on the storage server and tlog before marking a disk as failed
|
||||
init( RANGESTREAM_LIMIT_BYTES, 2e6 ); if( randomize && BUGGIFY ) RANGESTREAM_LIMIT_BYTES = 1;
|
||||
init( ENABLE_CLEAR_RANGE_EAGER_READS, true );
|
||||
init( QUICK_GET_VALUE_FALLBACK, true );
|
||||
init( QUICK_GET_KEY_VALUES_FALLBACK, true );
|
||||
|
||||
//Wait Failure
|
||||
init( MAX_OUTSTANDING_WAIT_FAILURE_REQUESTS, 250 ); if( randomize && BUGGIFY ) MAX_OUTSTANDING_WAIT_FAILURE_REQUESTS = 2;
|
||||
|
@ -764,6 +771,7 @@ void ServerKnobs::initialize(Randomize randomize, ClientKnobs* clientKnobs, IsSi
|
|||
init( REDWOOD_LAZY_CLEAR_MAX_PAGES, 1e6 );
|
||||
init( REDWOOD_REMAP_CLEANUP_WINDOW, 50 );
|
||||
init( REDWOOD_REMAP_CLEANUP_LAG, 0.1 );
|
||||
init( REDWOOD_PAGEFILE_GROWTH_SIZE_PAGES, 20000 ); if( randomize && BUGGIFY ) { REDWOOD_PAGEFILE_GROWTH_SIZE_PAGES = deterministicRandom()->randomInt(200, 1000); }
|
||||
init( REDWOOD_METRICS_INTERVAL, 5.0 );
|
||||
init( REDWOOD_HISTOGRAM_INTERVAL, 30.0 );
|
||||
|
||||
|
|
|
@ -281,6 +281,11 @@ public:
|
|||
double ROCKSDB_READ_VALUE_TIMEOUT;
|
||||
double ROCKSDB_READ_VALUE_PREFIX_TIMEOUT;
|
||||
double ROCKSDB_READ_RANGE_TIMEOUT;
|
||||
double ROCKSDB_READ_QUEUE_WAIT;
|
||||
int ROCKSDB_READ_QUEUE_SOFT_MAX;
|
||||
int ROCKSDB_READ_QUEUE_HARD_MAX;
|
||||
int ROCKSDB_FETCH_QUEUE_SOFT_MAX;
|
||||
int ROCKSDB_FETCH_QUEUE_HARD_MAX;
|
||||
|
||||
// Leader election
|
||||
int MAX_NOTIFICATIONS;
|
||||
|
@ -585,6 +590,8 @@ public:
|
|||
double MAX_STORAGE_COMMIT_TIME;
|
||||
int64_t RANGESTREAM_LIMIT_BYTES;
|
||||
bool ENABLE_CLEAR_RANGE_EAGER_READS;
|
||||
bool QUICK_GET_VALUE_FALLBACK;
|
||||
bool QUICK_GET_KEY_VALUES_FALLBACK;
|
||||
|
||||
// Wait Failure
|
||||
int MAX_OUTSTANDING_WAIT_FAILURE_REQUESTS;
|
||||
|
@ -717,6 +724,7 @@ public:
|
|||
int64_t REDWOOD_REMAP_CLEANUP_WINDOW; // Remap remover lag interval in which to coalesce page writes
|
||||
double REDWOOD_REMAP_CLEANUP_LAG; // Maximum allowed remap remover lag behind the cleanup window as a multiple of
|
||||
// the window size
|
||||
int REDWOOD_PAGEFILE_GROWTH_SIZE_PAGES; // Number of pages to grow page file by
|
||||
double REDWOOD_METRICS_INTERVAL;
|
||||
double REDWOOD_HISTOGRAM_INTERVAL;
|
||||
|
||||
|
|
|
@ -59,6 +59,14 @@ public:
|
|||
GetRangeLimits limits,
|
||||
Snapshot = Snapshot::False,
|
||||
Reverse = Reverse::False) override;
|
||||
Future<RangeResult> getRangeAndFlatMap(KeySelector begin,
|
||||
KeySelector end,
|
||||
Key mapper,
|
||||
GetRangeLimits limits,
|
||||
Snapshot = Snapshot::False,
|
||||
Reverse = Reverse::False) override {
|
||||
throw client_invalid_operation();
|
||||
}
|
||||
Future<Void> commit() override;
|
||||
Version getCommittedVersion() const override;
|
||||
void setOption(FDBTransactionOptions::Option option, Optional<StringRef> value = Optional<StringRef>()) override;
|
||||
|
|
|
@ -152,6 +152,45 @@ void TSS_traceMismatch(TraceEvent& event,
|
|||
.detail("TSSReply", tssResultsString);
|
||||
}
|
||||
|
||||
// range reads and flat map
|
||||
template <>
|
||||
bool TSS_doCompare(const GetKeyValuesAndFlatMapReply& src, const GetKeyValuesAndFlatMapReply& tss) {
|
||||
return src.more == tss.more && src.data == tss.data;
|
||||
}
|
||||
|
||||
template <>
|
||||
const char* TSS_mismatchTraceName(const GetKeyValuesAndFlatMapRequest& req) {
|
||||
return "TSSMismatchGetKeyValuesAndFlatMap";
|
||||
}
|
||||
|
||||
template <>
|
||||
void TSS_traceMismatch(TraceEvent& event,
|
||||
const GetKeyValuesAndFlatMapRequest& req,
|
||||
const GetKeyValuesAndFlatMapReply& src,
|
||||
const GetKeyValuesAndFlatMapReply& tss) {
|
||||
std::string ssResultsString = format("(%d)%s:\n", src.data.size(), src.more ? "+" : "");
|
||||
for (auto& it : src.data) {
|
||||
ssResultsString += "\n" + it.key.printable() + "=" + traceChecksumValue(it.value);
|
||||
}
|
||||
|
||||
std::string tssResultsString = format("(%d)%s:\n", tss.data.size(), tss.more ? "+" : "");
|
||||
for (auto& it : tss.data) {
|
||||
tssResultsString += "\n" + it.key.printable() + "=" + traceChecksumValue(it.value);
|
||||
}
|
||||
event
|
||||
.detail(
|
||||
"Begin",
|
||||
format("%s%s:%d", req.begin.orEqual ? "=" : "", req.begin.getKey().printable().c_str(), req.begin.offset))
|
||||
.detail("End",
|
||||
format("%s%s:%d", req.end.orEqual ? "=" : "", req.end.getKey().printable().c_str(), req.end.offset))
|
||||
.detail("Version", req.version)
|
||||
.detail("Limit", req.limit)
|
||||
.detail("LimitBytes", req.limitBytes)
|
||||
.setMaxFieldLength(FLOW_KNOBS->TSS_LARGE_TRACE_SIZE * 4 / 10)
|
||||
.detail("SSReply", ssResultsString)
|
||||
.detail("TSSReply", tssResultsString);
|
||||
}
|
||||
|
||||
// streaming range reads
|
||||
template <>
|
||||
bool TSS_doCompare(const GetKeyValuesStreamReply& src, const GetKeyValuesStreamReply& tss) {
|
||||
|
@ -356,6 +395,12 @@ void TSSMetrics::recordLatency(const GetKeyValuesRequest& req, double ssLatency,
|
|||
TSSgetKeyValuesLatency.addSample(tssLatency);
|
||||
}
|
||||
|
||||
template <>
|
||||
void TSSMetrics::recordLatency(const GetKeyValuesAndFlatMapRequest& req, double ssLatency, double tssLatency) {
|
||||
SSgetKeyValuesAndFlatMapLatency.addSample(ssLatency);
|
||||
TSSgetKeyValuesAndFlatMapLatency.addSample(tssLatency);
|
||||
}
|
||||
|
||||
template <>
|
||||
void TSSMetrics::recordLatency(const WatchValueRequest& req, double ssLatency, double tssLatency) {}
|
||||
|
||||
|
|
|
@ -22,6 +22,7 @@
|
|||
#define FDBCLIENT_STORAGESERVERINTERFACE_H
|
||||
#pragma once
|
||||
|
||||
#include <ostream>
|
||||
#include "fdbclient/FDBTypes.h"
|
||||
#include "fdbrpc/Locality.h"
|
||||
#include "fdbrpc/QueueModel.h"
|
||||
|
@ -65,6 +66,7 @@ struct StorageServerInterface {
|
|||
// Throws a wrong_shard_server if the keys in the request or result depend on data outside this server OR if a large
|
||||
// selector offset prevents all data from being read in one range read
|
||||
RequestStream<struct GetKeyValuesRequest> getKeyValues;
|
||||
RequestStream<struct GetKeyValuesAndFlatMapRequest> getKeyValuesAndFlatMap;
|
||||
|
||||
RequestStream<struct GetShardStateRequest> getShardState;
|
||||
RequestStream<struct WaitMetricsRequest> waitMetrics;
|
||||
|
@ -124,14 +126,16 @@ struct StorageServerInterface {
|
|||
RequestStream<struct SplitRangeRequest>(getValue.getEndpoint().getAdjustedEndpoint(12));
|
||||
getKeyValuesStream =
|
||||
RequestStream<struct GetKeyValuesStreamRequest>(getValue.getEndpoint().getAdjustedEndpoint(13));
|
||||
getKeyValuesAndFlatMap =
|
||||
RequestStream<struct GetKeyValuesAndFlatMapRequest>(getValue.getEndpoint().getAdjustedEndpoint(14));
|
||||
changeFeedStream =
|
||||
RequestStream<struct ChangeFeedStreamRequest>(getValue.getEndpoint().getAdjustedEndpoint(14));
|
||||
RequestStream<struct ChangeFeedStreamRequest>(getValue.getEndpoint().getAdjustedEndpoint(15));
|
||||
overlappingChangeFeeds =
|
||||
RequestStream<struct OverlappingChangeFeedsRequest>(getValue.getEndpoint().getAdjustedEndpoint(15));
|
||||
RequestStream<struct OverlappingChangeFeedsRequest>(getValue.getEndpoint().getAdjustedEndpoint(16));
|
||||
changeFeedPop =
|
||||
RequestStream<struct ChangeFeedPopRequest>(getValue.getEndpoint().getAdjustedEndpoint(16));
|
||||
RequestStream<struct ChangeFeedPopRequest>(getValue.getEndpoint().getAdjustedEndpoint(17));
|
||||
changeFeedVersionUpdate = RequestStream<struct ChangeFeedVersionUpdateRequest>(
|
||||
getValue.getEndpoint().getAdjustedEndpoint(17));
|
||||
getValue.getEndpoint().getAdjustedEndpoint(18));
|
||||
}
|
||||
} else {
|
||||
ASSERT(Ar::isDeserializing);
|
||||
|
@ -174,6 +178,7 @@ struct StorageServerInterface {
|
|||
streams.push_back(getReadHotRanges.getReceiver());
|
||||
streams.push_back(getRangeSplitPoints.getReceiver());
|
||||
streams.push_back(getKeyValuesStream.getReceiver(TaskPriority::LoadBalancedEndpoint));
|
||||
streams.push_back(getKeyValuesAndFlatMap.getReceiver(TaskPriority::LoadBalancedEndpoint));
|
||||
streams.push_back(changeFeedStream.getReceiver());
|
||||
streams.push_back(overlappingChangeFeeds.getReceiver());
|
||||
streams.push_back(changeFeedPop.getReceiver());
|
||||
|
@ -300,6 +305,9 @@ struct GetKeyValuesRequest : TimedRequest {
|
|||
SpanID spanContext;
|
||||
Arena arena;
|
||||
KeySelectorRef begin, end;
|
||||
// This is a dummy field there has never been used.
|
||||
// TODO: Get rid of this by constexpr or other template magic in getRange
|
||||
KeyRef mapper = KeyRef();
|
||||
Version version; // or latestVersion
|
||||
int limit, limitBytes;
|
||||
bool isFetchKeys;
|
||||
|
@ -314,6 +322,43 @@ struct GetKeyValuesRequest : TimedRequest {
|
|||
}
|
||||
};
|
||||
|
||||
struct GetKeyValuesAndFlatMapReply : public LoadBalancedReply {
|
||||
constexpr static FileIdentifier file_identifier = 1783067;
|
||||
Arena arena;
|
||||
VectorRef<KeyValueRef, VecSerStrategy::String> data;
|
||||
Version version; // useful when latestVersion was requested
|
||||
bool more;
|
||||
bool cached = false;
|
||||
|
||||
GetKeyValuesAndFlatMapReply() : version(invalidVersion), more(false), cached(false) {}
|
||||
|
||||
template <class Ar>
|
||||
void serialize(Ar& ar) {
|
||||
serializer(ar, LoadBalancedReply::penalty, LoadBalancedReply::error, data, version, more, cached, arena);
|
||||
}
|
||||
};
|
||||
|
||||
struct GetKeyValuesAndFlatMapRequest : TimedRequest {
|
||||
constexpr static FileIdentifier file_identifier = 6795747;
|
||||
SpanID spanContext;
|
||||
Arena arena;
|
||||
KeySelectorRef begin, end;
|
||||
KeyRef mapper;
|
||||
Version version; // or latestVersion
|
||||
int limit, limitBytes;
|
||||
bool isFetchKeys;
|
||||
Optional<TagSet> tags;
|
||||
Optional<UID> debugID;
|
||||
ReplyPromise<GetKeyValuesAndFlatMapReply> reply;
|
||||
|
||||
GetKeyValuesAndFlatMapRequest() : isFetchKeys(false) {}
|
||||
template <class Ar>
|
||||
void serialize(Ar& ar) {
|
||||
serializer(
|
||||
ar, begin, end, mapper, version, limit, limitBytes, isFetchKeys, tags, debugID, reply, spanContext, arena);
|
||||
}
|
||||
};
|
||||
|
||||
struct GetKeyValuesStreamReply : public ReplyPromiseStreamReply {
|
||||
constexpr static FileIdentifier file_identifier = 1783066;
|
||||
Arena arena;
|
||||
|
|
|
@ -1033,6 +1033,8 @@ const KeyRangeRef clientLibBinaryKeys(LiteralStringRef("\xff\x02/clientlib/bin/"
|
|||
LiteralStringRef("\xff\x02/clientlib/bin0"));
|
||||
const KeyRef clientLibBinaryPrefix = clientLibBinaryKeys.begin;
|
||||
|
||||
const KeyRef clientLibChangeCounterKey = "\xff\x02/clientlib/changeCounter"_sr;
|
||||
|
||||
const KeyRangeRef testOnlyTxnStateStorePrefixRange(LiteralStringRef("\xff/TESTONLYtxnStateStore/"),
|
||||
LiteralStringRef("\xff/TESTONLYtxnStateStore0"));
|
||||
|
||||
|
|
|
@ -488,6 +488,8 @@ extern const KeyRef clientLibMetadataPrefix;
|
|||
extern const KeyRangeRef clientLibBinaryKeys;
|
||||
extern const KeyRef clientLibBinaryPrefix;
|
||||
|
||||
extern const KeyRef clientLibChangeCounterKey;
|
||||
|
||||
// All mutations done to this range are blindly copied into txnStateStore.
|
||||
// Used to create artifically large txnStateStore instances in testing.
|
||||
extern const KeyRangeRef testOnlyTxnStateStorePrefixRange;
|
||||
|
|
|
@ -257,6 +257,23 @@ ThreadFuture<RangeResult> ThreadSafeTransaction::getRange(const KeySelectorRef&
|
|||
});
|
||||
}
|
||||
|
||||
ThreadFuture<RangeResult> ThreadSafeTransaction::getRangeAndFlatMap(const KeySelectorRef& begin,
|
||||
const KeySelectorRef& end,
|
||||
const StringRef& mapper,
|
||||
GetRangeLimits limits,
|
||||
bool snapshot,
|
||||
bool reverse) {
|
||||
KeySelector b = begin;
|
||||
KeySelector e = end;
|
||||
Key h = mapper;
|
||||
|
||||
ISingleThreadTransaction* tr = this->tr;
|
||||
return onMainThread([tr, b, e, h, limits, snapshot, reverse]() -> Future<RangeResult> {
|
||||
tr->checkDeferredError();
|
||||
return tr->getRangeAndFlatMap(b, e, h, limits, Snapshot{ snapshot }, Reverse{ reverse });
|
||||
});
|
||||
}
|
||||
|
||||
ThreadFuture<Standalone<VectorRef<const char*>>> ThreadSafeTransaction::getAddressesForKey(const KeyRef& key) {
|
||||
Key k = key;
|
||||
|
||||
|
@ -490,5 +507,3 @@ void ThreadSafeApi::addNetworkThreadCompletionHook(void (*hook)(void*), void* ho
|
|||
// upon return that the hook is set.
|
||||
threadCompletionHooks.emplace_back(hook, hookParameter);
|
||||
}
|
||||
|
||||
IClientApi* ThreadSafeApi::api = new ThreadSafeApi();
|
||||
|
|
|
@ -106,6 +106,12 @@ public:
|
|||
bool reverse = false) override {
|
||||
return getRange(firstGreaterOrEqual(keys.begin), firstGreaterOrEqual(keys.end), limits, snapshot, reverse);
|
||||
}
|
||||
ThreadFuture<RangeResult> getRangeAndFlatMap(const KeySelectorRef& begin,
|
||||
const KeySelectorRef& end,
|
||||
const StringRef& mapper,
|
||||
GetRangeLimits limits,
|
||||
bool snapshot,
|
||||
bool reverse) override;
|
||||
ThreadFuture<Standalone<VectorRef<const char*>>> getAddressesForKey(const KeyRef& key) override;
|
||||
ThreadFuture<Standalone<StringRef>> getVersionstamp() override;
|
||||
ThreadFuture<int64_t> getEstimatedRangeSizeBytes(const KeyRangeRef& keys) override;
|
||||
|
@ -166,9 +172,8 @@ public:
|
|||
|
||||
void addNetworkThreadCompletionHook(void (*hook)(void*), void* hookParameter) override;
|
||||
|
||||
static IClientApi* api;
|
||||
|
||||
private:
|
||||
friend IClientApi* getLocalClientAPI();
|
||||
ThreadSafeApi();
|
||||
|
||||
int apiVersion;
|
||||
|
|
|
@ -58,7 +58,7 @@ description is not currently required but encouraged.
|
|||
paramType="String" paramDescription="The identifier that will be part of all trace file names"
|
||||
description="Once provided, this string will be used to replace the port/PID in the log file names." />
|
||||
<Option name="trace_partial_file_suffix" code="39"
|
||||
paramType="String" paramDesciption="Append this suffix to partially written log files. When a log file is complete, it is renamed to remove the suffix. No separator is added between the file and the suffix. If you want to add a file extension, you should include the separator - e.g. '.tmp' instead of 'tmp' to add the 'tmp' extension."
|
||||
paramType="String" paramDescription="Append this suffix to partially written log files. When a log file is complete, it is renamed to remove the suffix. No separator is added between the file and the suffix. If you want to add a file extension, you should include the separator - e.g. '.tmp' instead of 'tmp' to add the 'tmp' extension."
|
||||
description="" />
|
||||
<Option name="knob" code="40"
|
||||
paramType="String" paramDescription="knob_name=knob_value"
|
||||
|
@ -191,10 +191,6 @@ description is not currently required but encouraged.
|
|||
<Option name="transaction_include_port_in_address" code="505"
|
||||
description="Addresses returned by get_addresses_for_key include the port when enabled. As of api version 630, this option is enabled by default and setting this has no effect."
|
||||
defaultFor="23"/>
|
||||
<Option name="distributed_transaction_trace_enable" code="600"
|
||||
description="Enable tracing for all transactions. This is the default." />
|
||||
<Option name="distributed_transaction_trace_disable" code="601"
|
||||
description="Disable tracing for all transactions." />
|
||||
<Option name="transaction_bypass_unreadable" code="700"
|
||||
description="Allows ``get`` operations to read from sections of keyspace that have become unreadable because of versionstamp operations. This sets the ``bypass_unreadable`` option of each transaction created by this database. See the transaction option description for more information."
|
||||
defaultFor="1100"/>
|
||||
|
|
|
@ -809,7 +809,7 @@ TEST_CASE("/flow/flow/chooseTwoActor") {
|
|||
return Void();
|
||||
}
|
||||
|
||||
TEST_CASE("/flow/flow/perf/actor patterns") {
|
||||
TEST_CASE("#flow/flow/perf/actor patterns") {
|
||||
double start;
|
||||
int N = 1000000;
|
||||
|
||||
|
|
|
@ -40,6 +40,7 @@
|
|||
#include "flow/ObjectSerializer.h"
|
||||
#include "flow/ProtocolVersion.h"
|
||||
#include "flow/UnitTest.h"
|
||||
#define XXH_INLINE_ALL
|
||||
#include "flow/xxhash.h"
|
||||
#include "flow/actorcompiler.h" // This must be the last #include.
|
||||
|
||||
|
@ -1589,16 +1590,17 @@ static ReliablePacket* sendPacket(TransportData* self,
|
|||
// Reserve some space for packet length and checksum, write them after serializing data
|
||||
SplitBuffer packetInfoBuffer;
|
||||
uint32_t len;
|
||||
XXH64_hash_t checksum = 0;
|
||||
XXH3_state_t* checksumState = nullptr;
|
||||
|
||||
// This is technically abstraction breaking but avoids XXH3_createState() and XXH3_freeState() which are just
|
||||
// malloc/free
|
||||
XXH3_state_t checksumState;
|
||||
// Checksum will be calculated with buffer API if contiguous, else using stream API. Mode is tracked here.
|
||||
bool checksumStream = false;
|
||||
XXH64_hash_t checksum;
|
||||
|
||||
int packetInfoSize = PACKET_LEN_WIDTH;
|
||||
if (checksumEnabled) {
|
||||
packetInfoSize += sizeof(checksum);
|
||||
checksumState = XXH3_createState();
|
||||
if (XXH3_64bits_reset(checksumState) != XXH_OK) {
|
||||
throw internal_error();
|
||||
}
|
||||
}
|
||||
|
||||
wr.writeAhead(packetInfoSize, &packetInfoBuffer);
|
||||
|
@ -1620,19 +1622,37 @@ static ReliablePacket* sendPacket(TransportData* self,
|
|||
while (checksumUnprocessedLength > 0) {
|
||||
uint32_t processLength =
|
||||
std::min(checksumUnprocessedLength, (uint32_t)(checksumPb->bytes_written - prevBytesWritten));
|
||||
// This won't fail if inputs are non null
|
||||
if (XXH3_64bits_update(checksumState, checksumPb->data() + prevBytesWritten, processLength) != XXH_OK) {
|
||||
throw internal_error();
|
||||
|
||||
// If not in checksum stream mode yet
|
||||
if (!checksumStream) {
|
||||
// If there is nothing left to process then calculate checksum directly
|
||||
if (processLength == checksumUnprocessedLength) {
|
||||
checksum = XXH3_64bits(checksumPb->data() + prevBytesWritten, processLength);
|
||||
} else {
|
||||
// Otherwise, initialize checksum state and switch to stream mode
|
||||
if (XXH3_64bits_reset(&checksumState) != XXH_OK) {
|
||||
throw internal_error();
|
||||
}
|
||||
checksumStream = true;
|
||||
}
|
||||
}
|
||||
|
||||
// If in checksum stream mode, update the checksum state
|
||||
if (checksumStream) {
|
||||
if (XXH3_64bits_update(&checksumState, checksumPb->data() + prevBytesWritten, processLength) !=
|
||||
XXH_OK) {
|
||||
throw internal_error();
|
||||
}
|
||||
}
|
||||
|
||||
checksumUnprocessedLength -= processLength;
|
||||
checksumPb = checksumPb->nextPacketBuffer();
|
||||
prevBytesWritten = 0;
|
||||
}
|
||||
|
||||
checksum = XXH3_64bits_digest(checksumState);
|
||||
// This always returns OK
|
||||
if (XXH3_freeState(checksumState) != XXH_OK) {
|
||||
throw internal_error();
|
||||
// If in checksum stream mode, get the final checksum
|
||||
if (checksumStream) {
|
||||
checksum = XXH3_64bits_digest(&checksumState);
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -67,6 +67,49 @@ public:
|
|||
}
|
||||
};
|
||||
|
||||
bool MockDNS::findMockTCPEndpoint(const std::string& host, const std::string& service) {
|
||||
std::string hostname = host + ":" + service;
|
||||
return hostnameToAddresses.find(hostname) != hostnameToAddresses.end();
|
||||
}
|
||||
|
||||
void MockDNS::addMockTCPEndpoint(const std::string& host,
|
||||
const std::string& service,
|
||||
const std::vector<NetworkAddress>& addresses) {
|
||||
if (findMockTCPEndpoint(host, service)) {
|
||||
throw operation_failed();
|
||||
}
|
||||
hostnameToAddresses[host + ":" + service] = addresses;
|
||||
}
|
||||
|
||||
void MockDNS::updateMockTCPEndpoint(const std::string& host,
|
||||
const std::string& service,
|
||||
const std::vector<NetworkAddress>& addresses) {
|
||||
if (!findMockTCPEndpoint(host, service)) {
|
||||
throw operation_failed();
|
||||
}
|
||||
hostnameToAddresses[host + ":" + service] = addresses;
|
||||
}
|
||||
|
||||
void MockDNS::removeMockTCPEndpoint(const std::string& host, const std::string& service) {
|
||||
if (!findMockTCPEndpoint(host, service)) {
|
||||
throw operation_failed();
|
||||
}
|
||||
hostnameToAddresses.erase(host + ":" + service);
|
||||
}
|
||||
|
||||
std::vector<NetworkAddress> MockDNS::getTCPEndpoint(const std::string& host, const std::string& service) {
|
||||
if (!findMockTCPEndpoint(host, service)) {
|
||||
throw operation_failed();
|
||||
}
|
||||
return hostnameToAddresses[host + ":" + service];
|
||||
}
|
||||
|
||||
void MockDNS::clearMockTCPEndpoints() {
|
||||
hostnameToAddresses.clear();
|
||||
}
|
||||
|
||||
MockDNS SimExternalConnection::mockDNS;
|
||||
|
||||
void SimExternalConnection::close() {
|
||||
socket.close();
|
||||
}
|
||||
|
@ -152,6 +195,9 @@ ACTOR static Future<std::vector<NetworkAddress>> resolveTCPEndpointImpl(std::str
|
|||
|
||||
Future<std::vector<NetworkAddress>> SimExternalConnection::resolveTCPEndpoint(const std::string& host,
|
||||
const std::string& service) {
|
||||
if (mockDNS.findMockTCPEndpoint(host, service)) {
|
||||
return mockDNS.getTCPEndpoint(host, service);
|
||||
}
|
||||
return resolveTCPEndpointImpl(host, service);
|
||||
}
|
||||
|
||||
|
@ -212,4 +258,49 @@ TEST_CASE("fdbrpc/SimExternalClient") {
|
|||
return Void();
|
||||
}
|
||||
|
||||
TEST_CASE("fdbrpc/MockTCPEndpoints") {
|
||||
state MockDNS mockDNS;
|
||||
state std::vector<NetworkAddress> networkAddresses;
|
||||
state NetworkAddress address1(IPAddress(0x13131313), 1);
|
||||
state NetworkAddress address2(IPAddress(0x14141414), 2);
|
||||
networkAddresses.push_back(address1);
|
||||
networkAddresses.push_back(address2);
|
||||
mockDNS.addMockTCPEndpoint("testhost1", "testport1", networkAddresses);
|
||||
ASSERT(mockDNS.findMockTCPEndpoint("testhost1", "testport1"));
|
||||
ASSERT(mockDNS.findMockTCPEndpoint("testhost1", "testport2") == false);
|
||||
std::vector<NetworkAddress> resolvedNetworkAddresses = mockDNS.getTCPEndpoint("testhost1", "testport1");
|
||||
ASSERT(resolvedNetworkAddresses.size() == 2);
|
||||
ASSERT(std::find(resolvedNetworkAddresses.begin(), resolvedNetworkAddresses.end(), address1) !=
|
||||
resolvedNetworkAddresses.end());
|
||||
ASSERT(std::find(resolvedNetworkAddresses.begin(), resolvedNetworkAddresses.end(), address2) !=
|
||||
resolvedNetworkAddresses.end());
|
||||
// Adding a hostname twice should fail.
|
||||
try {
|
||||
mockDNS.addMockTCPEndpoint("testhost1", "testport1", networkAddresses);
|
||||
} catch (Error& e) {
|
||||
ASSERT(e.code() == error_code_operation_failed);
|
||||
}
|
||||
// Updating an unexisted hostname should fail.
|
||||
try {
|
||||
mockDNS.updateMockTCPEndpoint("testhost2", "testport2", networkAddresses);
|
||||
} catch (Error& e) {
|
||||
ASSERT(e.code() == error_code_operation_failed);
|
||||
}
|
||||
// Removing an unexisted hostname should fail.
|
||||
try {
|
||||
mockDNS.removeMockTCPEndpoint("testhost2", "testport2");
|
||||
} catch (Error& e) {
|
||||
ASSERT(e.code() == error_code_operation_failed);
|
||||
}
|
||||
mockDNS.clearMockTCPEndpoints();
|
||||
// Updating any hostname right after clearing endpoints should fail.
|
||||
try {
|
||||
mockDNS.updateMockTCPEndpoint("testhost1", "testport1", networkAddresses);
|
||||
} catch (Error& e) {
|
||||
ASSERT(e.code() == error_code_operation_failed);
|
||||
}
|
||||
|
||||
return Void();
|
||||
}
|
||||
|
||||
void forceLinkSimExternalConnectionTests() {}
|
||||
|
|
|
@ -28,6 +28,24 @@
|
|||
|
||||
#include <boost/asio.hpp>
|
||||
|
||||
// MockDNS is a class maintaining a <hostname, vector<NetworkAddress>> mapping, mocking a DNS in simulation.
|
||||
class MockDNS {
|
||||
public:
|
||||
bool findMockTCPEndpoint(const std::string& host, const std::string& service);
|
||||
void addMockTCPEndpoint(const std::string& host,
|
||||
const std::string& service,
|
||||
const std::vector<NetworkAddress>& addresses);
|
||||
void updateMockTCPEndpoint(const std::string& host,
|
||||
const std::string& service,
|
||||
const std::vector<NetworkAddress>& addresses);
|
||||
void removeMockTCPEndpoint(const std::string& host, const std::string& service);
|
||||
void clearMockTCPEndpoints();
|
||||
std::vector<NetworkAddress> getTCPEndpoint(const std::string& host, const std::string& service);
|
||||
|
||||
private:
|
||||
std::map<std::string, std::vector<NetworkAddress>> hostnameToAddresses;
|
||||
};
|
||||
|
||||
class SimExternalConnection final : public IConnection, public ReferenceCounted<SimExternalConnection> {
|
||||
boost::asio::ip::tcp::socket socket;
|
||||
SimExternalConnection(boost::asio::ip::tcp::socket&& socket);
|
||||
|
@ -50,6 +68,9 @@ public:
|
|||
UID getDebugID() const override;
|
||||
static Future<std::vector<NetworkAddress>> resolveTCPEndpoint(const std::string& host, const std::string& service);
|
||||
static Future<Reference<IConnection>> connect(NetworkAddress toAddr);
|
||||
|
||||
private:
|
||||
static MockDNS mockDNS;
|
||||
};
|
||||
|
||||
#endif
|
||||
|
|
|
@ -51,10 +51,12 @@ struct TSSMetrics : ReferenceCounted<TSSMetrics>, NonCopyable {
|
|||
ContinuousSample<double> SSgetValueLatency;
|
||||
ContinuousSample<double> SSgetKeyLatency;
|
||||
ContinuousSample<double> SSgetKeyValuesLatency;
|
||||
ContinuousSample<double> SSgetKeyValuesAndFlatMapLatency;
|
||||
|
||||
ContinuousSample<double> TSSgetValueLatency;
|
||||
ContinuousSample<double> TSSgetKeyLatency;
|
||||
ContinuousSample<double> TSSgetKeyValuesLatency;
|
||||
ContinuousSample<double> TSSgetKeyValuesAndFlatMapLatency;
|
||||
|
||||
std::unordered_map<int, uint64_t> ssErrorsByCode;
|
||||
std::unordered_map<int, uint64_t> tssErrorsByCode;
|
||||
|
@ -103,7 +105,8 @@ struct TSSMetrics : ReferenceCounted<TSSMetrics>, NonCopyable {
|
|||
: cc("TSSClientMetrics"), requests("Requests", cc), streamComparisons("StreamComparisons", cc),
|
||||
ssErrors("SSErrors", cc), tssErrors("TSSErrors", cc), tssTimeouts("TSSTimeouts", cc),
|
||||
mismatches("Mismatches", cc), SSgetValueLatency(1000), SSgetKeyLatency(1000), SSgetKeyValuesLatency(1000),
|
||||
TSSgetValueLatency(1000), TSSgetKeyLatency(1000), TSSgetKeyValuesLatency(1000) {}
|
||||
SSgetKeyValuesAndFlatMapLatency(1000), TSSgetValueLatency(1000), TSSgetKeyLatency(1000),
|
||||
TSSgetKeyValuesLatency(1000), TSSgetKeyValuesAndFlatMapLatency(1000) {}
|
||||
};
|
||||
|
||||
template <class Rep>
|
||||
|
|
|
@ -25,9 +25,8 @@
|
|||
#include "flow/actorcompiler.h"
|
||||
|
||||
ACTOR Future<Void> disableConnectionFailuresAfter(double time, std::string context) {
|
||||
wait(delay(time));
|
||||
|
||||
if (g_network->isSimulated()) {
|
||||
wait(delayUntil(time));
|
||||
g_simulator.connectionFailuresDisableDuration = 1e6;
|
||||
g_simulator.speedUpSimulation = true;
|
||||
TraceEvent(SevWarnAlways, ("DisableConnectionFailures_" + context).c_str());
|
||||
|
|
|
@ -40,7 +40,7 @@
|
|||
|
||||
// TODO add comments + documentation
|
||||
void handleClientBlobRange(KeyRangeMap<bool>* knownBlobRanges,
|
||||
Arena ar,
|
||||
Arena& ar,
|
||||
VectorRef<KeyRangeRef>* rangesToAdd,
|
||||
VectorRef<KeyRangeRef>* rangesToRemove,
|
||||
KeyRef rangeStart,
|
||||
|
@ -81,7 +81,7 @@ void handleClientBlobRange(KeyRangeMap<bool>* knownBlobRanges,
|
|||
|
||||
void updateClientBlobRanges(KeyRangeMap<bool>* knownBlobRanges,
|
||||
RangeResult dbBlobRanges,
|
||||
Arena ar,
|
||||
Arena& ar,
|
||||
VectorRef<KeyRangeRef>* rangesToAdd,
|
||||
VectorRef<KeyRangeRef>* rangesToRemove) {
|
||||
if (BM_DEBUG) {
|
||||
|
@ -1169,7 +1169,7 @@ ACTOR Future<Void> blobManager(BlobManagerInterface bmInterf,
|
|||
// DB has [A - B) and [C - D). They should show up in knownBlobRanges, and [B - C) should be in removed.
|
||||
// DB has [B - C). It should show up in knownBlobRanges, [B - C) should be in added, and [A - B) and [C - D) should
|
||||
// be in removed.
|
||||
TEST_CASE("/blobmanager/updateranges") {
|
||||
TEST_CASE(":/blobmanager/updateranges") {
|
||||
KeyRangeMap<bool> knownBlobRanges(false, normalKeys.end);
|
||||
Arena ar;
|
||||
|
||||
|
|
|
@ -208,6 +208,7 @@ set(FDBSERVER_SRCS
|
|||
workloads/MemoryLifetime.actor.cpp
|
||||
workloads/MetricLogging.actor.cpp
|
||||
workloads/MutationLogReaderCorrectness.actor.cpp
|
||||
workloads/IndexPrefetchDemo.actor.cpp
|
||||
workloads/ParallelRestore.actor.cpp
|
||||
workloads/Performance.actor.cpp
|
||||
workloads/Ping.actor.cpp
|
||||
|
|
|
@ -4740,6 +4740,48 @@ ACTOR Future<Void> monitorGlobalConfig(ClusterControllerData::DBInfo* db) {
|
|||
}
|
||||
}
|
||||
|
||||
ACTOR Future<Void> monitorClientLibChangeCounter(ClusterControllerData::DBInfo* db) {
|
||||
state ClientDBInfo clientInfo;
|
||||
state ReadYourWritesTransaction tr;
|
||||
state Future<Void> clientLibChangeFuture;
|
||||
|
||||
loop {
|
||||
tr = ReadYourWritesTransaction(db->db);
|
||||
loop {
|
||||
try {
|
||||
tr.setOption(FDBTransactionOptions::READ_SYSTEM_KEYS);
|
||||
tr.setOption(FDBTransactionOptions::READ_LOCK_AWARE);
|
||||
|
||||
Optional<Value> counterVal = wait(tr.get(clientLibChangeCounterKey));
|
||||
if (counterVal.present() && counterVal.get().size() == sizeof(uint64_t)) {
|
||||
uint64_t changeCounter = *reinterpret_cast<const uint64_t*>(counterVal.get().begin());
|
||||
|
||||
clientInfo = db->serverInfo->get().client;
|
||||
if (changeCounter != clientInfo.clientLibChangeCounter) {
|
||||
TraceEvent("ClientLibChangeCounterChanged").detail("Value", changeCounter);
|
||||
clientInfo.id = deterministicRandom()->randomUniqueID();
|
||||
clientInfo.clientLibChangeCounter = changeCounter;
|
||||
db->clientInfo->set(clientInfo);
|
||||
|
||||
ServerDBInfo serverInfo = db->serverInfo->get();
|
||||
serverInfo.id = deterministicRandom()->randomUniqueID();
|
||||
serverInfo.infoGeneration = ++db->dbInfoCount;
|
||||
serverInfo.client = clientInfo;
|
||||
db->serverInfo->set(serverInfo);
|
||||
}
|
||||
}
|
||||
|
||||
clientLibChangeFuture = tr.watch(clientLibChangeCounterKey);
|
||||
wait(tr.commit());
|
||||
wait(clientLibChangeFuture);
|
||||
break;
|
||||
} catch (Error& e) {
|
||||
wait(tr.onError(e));
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
ACTOR Future<Void> updatedChangingDatacenters(ClusterControllerData* self) {
|
||||
// do not change the cluster controller until all the processes have had a chance to register
|
||||
wait(delay(SERVER_KNOBS->WAIT_FOR_GOOD_RECRUITMENT_DELAY));
|
||||
|
@ -5437,6 +5479,7 @@ ACTOR Future<Void> clusterControllerCore(ClusterControllerFullInterface interf,
|
|||
self.addActor.send(monitorProcessClasses(&self));
|
||||
self.addActor.send(monitorServerInfoConfig(&self.db));
|
||||
self.addActor.send(monitorGlobalConfig(&self.db));
|
||||
self.addActor.send(monitorClientLibChangeCounter(&self.db));
|
||||
self.addActor.send(updatedChangingDatacenters(&self));
|
||||
self.addActor.send(updatedChangedDatacenters(&self));
|
||||
self.addActor.send(updateDatacenterVersionDifference(&self));
|
||||
|
|
|
@ -4641,12 +4641,18 @@ ACTOR Future<Void> storageServerFailureTracker(DDTeamCollection* self,
|
|||
self->healthyZone.set(Optional<Key>());
|
||||
}
|
||||
}
|
||||
if (!status->isUnhealthy()) {
|
||||
// On server transistion from unhealthy -> healthy, trigger buildTeam check,
|
||||
// handles scenario when team building failed due to insufficient healthy servers.
|
||||
// Operaton cost is minimal if currentTeamCount == desiredTeamCount/maxTeamCount.
|
||||
self->doBuildTeams = true;
|
||||
}
|
||||
|
||||
// TraceEvent("StatusMapChange", self->distributorId)
|
||||
// .detail("ServerID", interf.id())
|
||||
// .detail("Status", status->toString())
|
||||
// .detail("Available",
|
||||
// IFailureMonitor::failureMonitor().getState(interf.waitFailure.getEndpoint()).isAvailable());
|
||||
TraceEvent(SevDebug, "StatusMapChange", self->distributorId)
|
||||
.detail("ServerID", interf.id())
|
||||
.detail("Status", status->toString())
|
||||
.detail("Available",
|
||||
IFailureMonitor::failureMonitor().getState(interf.waitFailure.getEndpoint()).isAvailable());
|
||||
}
|
||||
when(wait(status->isUnhealthy() ? waitForAllDataRemoved(cx, interf.id(), addedVersion, self) : Never())) {
|
||||
break;
|
||||
|
|
|
@ -21,9 +21,10 @@
|
|||
#ifndef FDBSERVER_DATADISTRIBUTORINTERFACE_H
|
||||
#define FDBSERVER_DATADISTRIBUTORINTERFACE_H
|
||||
|
||||
#include "fdbrpc/fdbrpc.h"
|
||||
#include "fdbrpc/Locality.h"
|
||||
#include "fdbclient/ClusterInterface.h"
|
||||
#include "fdbclient/FDBTypes.h"
|
||||
#include "fdbrpc/Locality.h"
|
||||
#include "fdbrpc/fdbrpc.h"
|
||||
|
||||
struct DataDistributorInterface {
|
||||
constexpr static FileIdentifier file_identifier = 12383874;
|
||||
|
@ -34,6 +35,7 @@ struct DataDistributorInterface {
|
|||
RequestStream<struct DistributorSnapRequest> distributorSnapReq;
|
||||
RequestStream<struct DistributorExclusionSafetyCheckRequest> distributorExclCheckReq;
|
||||
RequestStream<struct GetDataDistributorMetricsRequest> dataDistributorMetrics;
|
||||
RequestStream<struct DistributorSplitRangeRequest> distributorSplitRange;
|
||||
|
||||
DataDistributorInterface() {}
|
||||
explicit DataDistributorInterface(const struct LocalityData& l, UID id) : locality(l), myId(id) {}
|
||||
|
@ -53,7 +55,8 @@ struct DataDistributorInterface {
|
|||
myId,
|
||||
distributorSnapReq,
|
||||
distributorExclCheckReq,
|
||||
dataDistributorMetrics);
|
||||
dataDistributorMetrics,
|
||||
distributorSplitRange);
|
||||
}
|
||||
};
|
||||
|
||||
|
@ -147,4 +150,19 @@ struct DistributorExclusionSafetyCheckRequest {
|
|||
}
|
||||
};
|
||||
|
||||
// Insert split points, and distribute the resulted shards to different teams.
|
||||
struct DistributorSplitRangeRequest {
|
||||
constexpr static FileIdentifier file_identifier = 1384441;
|
||||
std::vector<Key> splitPoints;
|
||||
ReplyPromise<SplitShardReply> reply;
|
||||
|
||||
DistributorSplitRangeRequest() {}
|
||||
explicit DistributorSplitRangeRequest(std::vector<Key> splitPoints) : splitPoints{ std::move(splitPoints) } {}
|
||||
|
||||
template <class Ar>
|
||||
void serialize(Ar& ar) {
|
||||
serializer(ar, splitPoints, reply);
|
||||
}
|
||||
};
|
||||
|
||||
#endif // FDBSERVER_DATADISTRIBUTORINTERFACE_H
|
||||
|
|
|
@ -1226,7 +1226,11 @@ public:
|
|||
return item.get();
|
||||
}
|
||||
|
||||
void switchTree(DeltaTree2* newTree) { tree = newTree; }
|
||||
void switchTree(DeltaTree2* newTree) {
|
||||
tree = newTree;
|
||||
// Reset item because it may point into tree memory
|
||||
item.reset();
|
||||
}
|
||||
|
||||
// If the cursor is valid, return a reference to the cursor's internal T.
|
||||
// Otherwise, returns a reference to the cache's upper boundary.
|
||||
|
|
|
@ -456,14 +456,14 @@ ACTOR Future<Void> queueGetReadVersionRequests(Reference<AsyncVar<ServerDBInfo>
|
|||
stats->txnSystemPriorityStartIn += req.transactionCount;
|
||||
++stats->systemGRVQueueSize;
|
||||
systemQueue->push_back(req);
|
||||
systemQueue->span.addParent(req.spanContext);
|
||||
// systemQueue->span.addParent(req.spanContext);
|
||||
} else if (req.priority >= TransactionPriority::DEFAULT) {
|
||||
++stats->txnRequestIn;
|
||||
stats->txnStartIn += req.transactionCount;
|
||||
stats->txnDefaultPriorityStartIn += req.transactionCount;
|
||||
++stats->defaultGRVQueueSize;
|
||||
defaultQueue->push_back(req);
|
||||
defaultQueue->span.addParent(req.spanContext);
|
||||
// defaultQueue->span.addParent(req.spanContext);
|
||||
} else {
|
||||
// Return error for batch_priority GRV requests
|
||||
int64_t proxiesCount = std::max((int)db->get().client.grvProxies.size(), 1);
|
||||
|
@ -476,7 +476,7 @@ ACTOR Future<Void> queueGetReadVersionRequests(Reference<AsyncVar<ServerDBInfo>
|
|||
stats->txnBatchPriorityStartIn += req.transactionCount;
|
||||
++stats->batchGRVQueueSize;
|
||||
batchQueue->push_back(req);
|
||||
batchQueue->span.addParent(req.spanContext);
|
||||
// batchQueue->span.addParent(req.spanContext);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
@ -722,7 +722,7 @@ ACTOR static Future<Void> transactionStarter(GrvProxyInterface proxy,
|
|||
state PrioritizedTransactionTagMap<ClientTagThrottleLimits> throttledTags;
|
||||
|
||||
state PromiseStream<double> normalGRVLatency;
|
||||
state Span span;
|
||||
// state Span span;
|
||||
|
||||
state int64_t midShardSize = SERVER_KNOBS->MIN_SHARD_BYTES;
|
||||
getCurrentLineage()->modify(&TransactionLineage::operation) =
|
||||
|
@ -805,7 +805,7 @@ ACTOR static Future<Void> transactionStarter(GrvProxyInterface proxy,
|
|||
} else {
|
||||
break;
|
||||
}
|
||||
transactionQueue->span.swap(span);
|
||||
// transactionQueue->span.swap(span);
|
||||
|
||||
auto& req = transactionQueue->front();
|
||||
int tc = req.transactionCount;
|
||||
|
@ -886,7 +886,7 @@ ACTOR static Future<Void> transactionStarter(GrvProxyInterface proxy,
|
|||
int batchGRVProcessed = 0;
|
||||
for (int i = 0; i < start.size(); i++) {
|
||||
if (start[i].size()) {
|
||||
Future<GetReadVersionReply> readVersionReply = getLiveCommittedVersion(span.context,
|
||||
Future<GetReadVersionReply> readVersionReply = getLiveCommittedVersion(UID() /*span.context*/,
|
||||
grvProxyData,
|
||||
i,
|
||||
debugID,
|
||||
|
@ -909,7 +909,7 @@ ACTOR static Future<Void> transactionStarter(GrvProxyInterface proxy,
|
|||
batchGRVProcessed += batchPriTransactionsStarted[i];
|
||||
}
|
||||
}
|
||||
span = Span(span.location);
|
||||
// span = Span(span.location);
|
||||
|
||||
grvProxyData->stats.percentageOfDefaultGRVQueueProcessed =
|
||||
defaultQueueSize ? (double)defaultGRVProcessed / defaultQueueSize : 1;
|
||||
|
|
|
@ -9,6 +9,7 @@
|
|||
#include <rocksdb/table.h>
|
||||
#include <rocksdb/version.h>
|
||||
#include <rocksdb/utilities/table_properties_collectors.h>
|
||||
#include "fdbclient/SystemData.h"
|
||||
#include "fdbserver/CoroFlow.h"
|
||||
#include "flow/flow.h"
|
||||
#include "flow/IThreadPool.h"
|
||||
|
@ -51,19 +52,6 @@ rocksdb::ColumnFamilyOptions getCFOptions() {
|
|||
}
|
||||
// Compact sstables when there's too much deleted stuff.
|
||||
options.table_properties_collector_factories = { rocksdb::NewCompactOnDeletionCollectorFactory(128, 1) };
|
||||
return options;
|
||||
}
|
||||
|
||||
rocksdb::Options getOptions() {
|
||||
rocksdb::Options options({}, getCFOptions());
|
||||
options.avoid_unnecessary_blocking_io = true;
|
||||
options.create_if_missing = true;
|
||||
if (SERVER_KNOBS->ROCKSDB_BACKGROUND_PARALLELISM > 0) {
|
||||
options.IncreaseParallelism(SERVER_KNOBS->ROCKSDB_BACKGROUND_PARALLELISM);
|
||||
}
|
||||
|
||||
options.statistics = rocksdb::CreateDBStatistics();
|
||||
options.statistics->set_stats_level(rocksdb::kExceptHistogramOrTimers);
|
||||
|
||||
rocksdb::BlockBasedTableOptions bbOpts;
|
||||
// TODO: Add a knob for the block cache size. (Default is 8 MB)
|
||||
|
@ -96,6 +84,21 @@ rocksdb::Options getOptions() {
|
|||
}
|
||||
|
||||
options.table_factory.reset(rocksdb::NewBlockBasedTableFactory(bbOpts));
|
||||
|
||||
return options;
|
||||
}
|
||||
|
||||
rocksdb::Options getOptions() {
|
||||
rocksdb::Options options({}, getCFOptions());
|
||||
options.avoid_unnecessary_blocking_io = true;
|
||||
options.create_if_missing = true;
|
||||
if (SERVER_KNOBS->ROCKSDB_BACKGROUND_PARALLELISM > 0) {
|
||||
options.IncreaseParallelism(SERVER_KNOBS->ROCKSDB_BACKGROUND_PARALLELISM);
|
||||
}
|
||||
|
||||
options.statistics = rocksdb::CreateDBStatistics();
|
||||
options.statistics->set_stats_level(rocksdb::kExceptHistogramOrTimers);
|
||||
|
||||
options.db_log_dir = SERVER_KNOBS->LOG_DIRECTORY;
|
||||
return options;
|
||||
}
|
||||
|
@ -107,6 +110,19 @@ rocksdb::ReadOptions getReadOptions() {
|
|||
return options;
|
||||
}
|
||||
|
||||
ACTOR Future<Void> flowLockLogger(const FlowLock* readLock, const FlowLock* fetchLock) {
|
||||
loop {
|
||||
wait(delay(SERVER_KNOBS->ROCKSDB_METRICS_DELAY));
|
||||
TraceEvent e(SevInfo, "RocksDBFlowLock");
|
||||
e.detail("ReadAvailable", readLock->available());
|
||||
e.detail("ReadActivePermits", readLock->activePermits());
|
||||
e.detail("ReadWaiters", readLock->waiters());
|
||||
e.detail("FetchAvailable", fetchLock->available());
|
||||
e.detail("FetchActivePermits", fetchLock->activePermits());
|
||||
e.detail("FetchWaiters", fetchLock->waiters());
|
||||
}
|
||||
}
|
||||
|
||||
ACTOR Future<Void> rocksDBMetricLogger(std::shared_ptr<rocksdb::Statistics> statistics, rocksdb::DB* db) {
|
||||
state std::vector<std::tuple<const char*, uint32_t, uint64_t>> tickerStats = {
|
||||
{ "StallMicros", rocksdb::STALL_MICROS, 0 },
|
||||
|
@ -190,7 +206,8 @@ ACTOR Future<Void> rocksDBMetricLogger(std::shared_ptr<rocksdb::Statistics> stat
|
|||
}
|
||||
|
||||
void logRocksDBError(const rocksdb::Status& status, const std::string& method) {
|
||||
TraceEvent e(SevError, "RocksDBError");
|
||||
auto level = status.IsTimedOut() ? SevWarn : SevError;
|
||||
TraceEvent e(level, "RocksDBError");
|
||||
e.detail("Error", status.ToString()).detail("Method", method).detail("RocksDBSeverity", status.severity());
|
||||
if (status.IsIOError()) {
|
||||
e.detail("SubCode", status.subcode());
|
||||
|
@ -229,7 +246,13 @@ struct RocksDBKeyValueStore : IKeyValueStore {
|
|||
std::string path;
|
||||
ThreadReturnPromise<Void> done;
|
||||
Optional<Future<Void>>& metrics;
|
||||
OpenAction(std::string path, Optional<Future<Void>>& metrics) : path(std::move(path)), metrics(metrics) {}
|
||||
const FlowLock* readLock;
|
||||
const FlowLock* fetchLock;
|
||||
OpenAction(std::string path,
|
||||
Optional<Future<Void>>& metrics,
|
||||
const FlowLock* readLock,
|
||||
const FlowLock* fetchLock)
|
||||
: path(std::move(path)), metrics(metrics), readLock(readLock), fetchLock(fetchLock) {}
|
||||
|
||||
double getTimeEstimate() const override { return SERVER_KNOBS->COMMIT_TIME_ESTIMATE; }
|
||||
};
|
||||
|
@ -249,7 +272,8 @@ struct RocksDBKeyValueStore : IKeyValueStore {
|
|||
// metric logger in simulation.
|
||||
if (!g_network->isSimulated()) {
|
||||
onMainThread([&] {
|
||||
a.metrics = rocksDBMetricLogger(options.statistics, db);
|
||||
a.metrics =
|
||||
rocksDBMetricLogger(options.statistics, db) && flowLockLogger(a.readLock, a.fetchLock);
|
||||
return Future<bool>(true);
|
||||
}).blockUntilReady();
|
||||
}
|
||||
|
@ -391,7 +415,7 @@ struct RocksDBKeyValueStore : IKeyValueStore {
|
|||
} else if (s.IsNotFound()) {
|
||||
a.result.send(Optional<Value>());
|
||||
} else {
|
||||
TraceEvent(SevError, "RocksDBError").detail("Error", s.ToString()).detail("Method", "ReadValue");
|
||||
logRocksDBError(s, "ReadValue");
|
||||
a.result.sendError(statusToError(s));
|
||||
}
|
||||
}
|
||||
|
@ -556,8 +580,27 @@ struct RocksDBKeyValueStore : IKeyValueStore {
|
|||
Future<Void> openFuture;
|
||||
std::unique_ptr<rocksdb::WriteBatch> writeBatch;
|
||||
Optional<Future<Void>> metrics;
|
||||
FlowLock readSemaphore;
|
||||
int numReadWaiters;
|
||||
FlowLock fetchSemaphore;
|
||||
int numFetchWaiters;
|
||||
|
||||
explicit RocksDBKeyValueStore(const std::string& path, UID id) : path(path), id(id) {
|
||||
struct Counters {
|
||||
CounterCollection cc;
|
||||
Counter immediateThrottle;
|
||||
Counter failedToAcquire;
|
||||
|
||||
Counters()
|
||||
: cc("RocksDBThrottle"), immediateThrottle("ImmediateThrottle", cc), failedToAcquire("failedToAcquire", cc) {}
|
||||
};
|
||||
|
||||
Counters counters;
|
||||
|
||||
explicit RocksDBKeyValueStore(const std::string& path, UID id)
|
||||
: path(path), id(id), readSemaphore(SERVER_KNOBS->ROCKSDB_READ_QUEUE_SOFT_MAX),
|
||||
fetchSemaphore(SERVER_KNOBS->ROCKSDB_FETCH_QUEUE_SOFT_MAX),
|
||||
numReadWaiters(SERVER_KNOBS->ROCKSDB_READ_QUEUE_HARD_MAX - SERVER_KNOBS->ROCKSDB_READ_QUEUE_SOFT_MAX),
|
||||
numFetchWaiters(SERVER_KNOBS->ROCKSDB_FETCH_QUEUE_HARD_MAX - SERVER_KNOBS->ROCKSDB_FETCH_QUEUE_SOFT_MAX) {
|
||||
// In simluation, run the reader/writer threads as Coro threads (i.e. in the network thread. The storage engine
|
||||
// is still multi-threaded as background compaction threads are still present. Reads/writes to disk will also
|
||||
// block the network thread in a way that would be unacceptable in production but is a necessary evil here. When
|
||||
|
@ -612,7 +655,7 @@ struct RocksDBKeyValueStore : IKeyValueStore {
|
|||
if (openFuture.isValid()) {
|
||||
return openFuture;
|
||||
}
|
||||
auto a = std::make_unique<Writer::OpenAction>(path, metrics);
|
||||
auto a = std::make_unique<Writer::OpenAction>(path, metrics, &readSemaphore, &fetchSemaphore);
|
||||
openFuture = a->done.getFuture();
|
||||
writeThread->post(a.release());
|
||||
return openFuture;
|
||||
|
@ -645,28 +688,109 @@ struct RocksDBKeyValueStore : IKeyValueStore {
|
|||
return res;
|
||||
}
|
||||
|
||||
Future<Optional<Value>> readValue(KeyRef key, IKeyValueStore::ReadType, Optional<UID> debugID) override {
|
||||
auto a = new Reader::ReadValueAction(key, debugID);
|
||||
auto res = a->result.getFuture();
|
||||
readThreads->post(a);
|
||||
return res;
|
||||
void checkWaiters(const FlowLock& semaphore, int maxWaiters) {
|
||||
if (semaphore.waiters() > maxWaiters) {
|
||||
++counters.immediateThrottle;
|
||||
throw server_overloaded();
|
||||
}
|
||||
}
|
||||
|
||||
// We don't throttle eager reads and reads to the FF keyspace because FDB struggles when those reads fail.
|
||||
// Thus far, they have been low enough volume to not cause an issue.
|
||||
static bool shouldThrottle(IKeyValueStore::ReadType type, KeyRef key) {
|
||||
return type != IKeyValueStore::ReadType::EAGER && !(key.startsWith(systemKeys.begin));
|
||||
}
|
||||
|
||||
ACTOR template <class Action>
|
||||
static Future<Optional<Value>> read(Action* action, FlowLock* semaphore, IThreadPool* pool, Counter* counter) {
|
||||
state std::unique_ptr<Action> a(action);
|
||||
state Optional<Void> slot = wait(timeout(semaphore->take(), SERVER_KNOBS->ROCKSDB_READ_QUEUE_WAIT));
|
||||
if (!slot.present()) {
|
||||
++(*counter);
|
||||
throw server_overloaded();
|
||||
}
|
||||
|
||||
state FlowLock::Releaser release(*semaphore);
|
||||
|
||||
auto fut = a->result.getFuture();
|
||||
pool->post(a.release());
|
||||
Optional<Value> result = wait(fut);
|
||||
|
||||
return result;
|
||||
}
|
||||
|
||||
Future<Optional<Value>> readValue(KeyRef key, IKeyValueStore::ReadType type, Optional<UID> debugID) override {
|
||||
if (!shouldThrottle(type, key)) {
|
||||
auto a = new Reader::ReadValueAction(key, debugID);
|
||||
auto res = a->result.getFuture();
|
||||
readThreads->post(a);
|
||||
return res;
|
||||
}
|
||||
|
||||
auto& semaphore = (type == IKeyValueStore::ReadType::FETCH) ? fetchSemaphore : readSemaphore;
|
||||
int maxWaiters = (type == IKeyValueStore::ReadType::FETCH) ? numFetchWaiters : numReadWaiters;
|
||||
|
||||
checkWaiters(semaphore, maxWaiters);
|
||||
auto a = std::make_unique<Reader::ReadValueAction>(key, debugID);
|
||||
return read(a.release(), &semaphore, readThreads.getPtr(), &counters.failedToAcquire);
|
||||
}
|
||||
|
||||
Future<Optional<Value>> readValuePrefix(KeyRef key,
|
||||
int maxLength,
|
||||
IKeyValueStore::ReadType,
|
||||
IKeyValueStore::ReadType type,
|
||||
Optional<UID> debugID) override {
|
||||
auto a = new Reader::ReadValuePrefixAction(key, maxLength, debugID);
|
||||
auto res = a->result.getFuture();
|
||||
readThreads->post(a);
|
||||
return res;
|
||||
if (!shouldThrottle(type, key)) {
|
||||
auto a = new Reader::ReadValuePrefixAction(key, maxLength, debugID);
|
||||
auto res = a->result.getFuture();
|
||||
readThreads->post(a);
|
||||
return res;
|
||||
}
|
||||
|
||||
auto& semaphore = (type == IKeyValueStore::ReadType::FETCH) ? fetchSemaphore : readSemaphore;
|
||||
int maxWaiters = (type == IKeyValueStore::ReadType::FETCH) ? numFetchWaiters : numReadWaiters;
|
||||
|
||||
checkWaiters(semaphore, maxWaiters);
|
||||
auto a = std::make_unique<Reader::ReadValuePrefixAction>(key, maxLength, debugID);
|
||||
return read(a.release(), &semaphore, readThreads.getPtr(), &counters.failedToAcquire);
|
||||
}
|
||||
|
||||
Future<RangeResult> readRange(KeyRangeRef keys, int rowLimit, int byteLimit, IKeyValueStore::ReadType) override {
|
||||
auto a = new Reader::ReadRangeAction(keys, rowLimit, byteLimit);
|
||||
auto res = a->result.getFuture();
|
||||
readThreads->post(a);
|
||||
return res;
|
||||
ACTOR static Future<Standalone<RangeResultRef>> read(Reader::ReadRangeAction* action,
|
||||
FlowLock* semaphore,
|
||||
IThreadPool* pool,
|
||||
Counter* counter) {
|
||||
state std::unique_ptr<Reader::ReadRangeAction> a(action);
|
||||
state Optional<Void> slot = wait(timeout(semaphore->take(), SERVER_KNOBS->ROCKSDB_READ_QUEUE_WAIT));
|
||||
if (!slot.present()) {
|
||||
++(*counter);
|
||||
throw server_overloaded();
|
||||
}
|
||||
|
||||
state FlowLock::Releaser release(*semaphore);
|
||||
|
||||
auto fut = a->result.getFuture();
|
||||
pool->post(a.release());
|
||||
Standalone<RangeResultRef> result = wait(fut);
|
||||
|
||||
return result;
|
||||
}
|
||||
|
||||
Future<RangeResult> readRange(KeyRangeRef keys,
|
||||
int rowLimit,
|
||||
int byteLimit,
|
||||
IKeyValueStore::ReadType type) override {
|
||||
if (!shouldThrottle(type, keys.begin)) {
|
||||
auto a = new Reader::ReadRangeAction(keys, rowLimit, byteLimit);
|
||||
auto res = a->result.getFuture();
|
||||
readThreads->post(a);
|
||||
return res;
|
||||
}
|
||||
|
||||
auto& semaphore = (type == IKeyValueStore::ReadType::FETCH) ? fetchSemaphore : readSemaphore;
|
||||
int maxWaiters = (type == IKeyValueStore::ReadType::FETCH) ? numFetchWaiters : numReadWaiters;
|
||||
|
||||
checkWaiters(semaphore, maxWaiters);
|
||||
auto a = std::make_unique<Reader::ReadRangeAction>(keys, rowLimit, byteLimit);
|
||||
return read(a.release(), &semaphore, readThreads.getPtr(), &counters.failedToAcquire);
|
||||
}
|
||||
|
||||
StorageBytes getStorageBytes() const override {
|
||||
|
|
|
@ -2886,7 +2886,7 @@ struct DequeAllocator : std::allocator<T> {
|
|||
}
|
||||
};
|
||||
|
||||
TEST_CASE("/fdbserver/tlogserver/VersionMessagesOverheadFactor") {
|
||||
TEST_CASE("Lfdbserver/tlogserver/VersionMessagesOverheadFactor") {
|
||||
|
||||
typedef std::pair<Version, LengthPrefixedStringRef> TestType; // type used by versionMessages
|
||||
|
||||
|
|
|
@ -3374,7 +3374,7 @@ struct DequeAllocator : std::allocator<T> {
|
|||
}
|
||||
};
|
||||
|
||||
TEST_CASE("/fdbserver/tlogserver/VersionMessagesOverheadFactor") {
|
||||
TEST_CASE("Lfdbserver/tlogserver/VersionMessagesOverheadFactor") {
|
||||
|
||||
typedef std::pair<Version, LengthPrefixedStringRef> TestType; // type used by versionMessages
|
||||
|
||||
|
|
|
@ -3308,7 +3308,7 @@ JsonBuilderObject randomDocument(const std::vector<std::string>& strings, int& l
|
|||
return r;
|
||||
}
|
||||
|
||||
TEST_CASE("/status/json/builderPerf") {
|
||||
TEST_CASE("Lstatus/json/builderPerf") {
|
||||
std::vector<std::string> strings;
|
||||
int c = 1000000;
|
||||
printf("Generating random strings\n");
|
||||
|
|
|
@ -3453,7 +3453,7 @@ struct DequeAllocator : std::allocator<T> {
|
|||
}
|
||||
};
|
||||
|
||||
TEST_CASE("/fdbserver/tlogserver/VersionMessagesOverheadFactor") {
|
||||
TEST_CASE("Lfdbserver/tlogserver/VersionMessagesOverheadFactor") {
|
||||
|
||||
typedef std::pair<Version, LengthPrefixedStringRef> TestType; // type used by versionMessages
|
||||
|
||||
|
|
|
@ -128,8 +128,13 @@ public:
|
|||
};
|
||||
|
||||
private:
|
||||
typedef Promise<Lock> Slot;
|
||||
typedef Deque<Slot> Queue;
|
||||
struct Waiter {
|
||||
Waiter() : queuedTime(now()) {}
|
||||
Promise<Lock> lockPromise;
|
||||
double queuedTime;
|
||||
};
|
||||
|
||||
typedef Deque<Waiter> Queue;
|
||||
|
||||
#if PRIORITYMULTILOCK_DEBUG
|
||||
#define prioritylock_printf(...) printf(__VA_ARGS__)
|
||||
|
@ -138,7 +143,8 @@ private:
|
|||
#endif
|
||||
|
||||
public:
|
||||
PriorityMultiLock(int concurrency, int maxPriority) : concurrency(concurrency), available(concurrency), waiting(0) {
|
||||
PriorityMultiLock(int concurrency, int maxPriority, int launchLimit = std::numeric_limits<int>::max())
|
||||
: concurrency(concurrency), available(concurrency), waiting(0), launchLimit(launchLimit) {
|
||||
waiters.resize(maxPriority + 1);
|
||||
fRunner = runner(this);
|
||||
}
|
||||
|
@ -157,11 +163,37 @@ public:
|
|||
return p;
|
||||
}
|
||||
|
||||
Slot s;
|
||||
waiters[priority].push_back(s);
|
||||
Waiter w;
|
||||
waiters[priority].push_back(w);
|
||||
++waiting;
|
||||
prioritylock_printf("lock exit queued %s\n", toString().c_str());
|
||||
return s.getFuture();
|
||||
return w.lockPromise.getFuture();
|
||||
}
|
||||
|
||||
std::string toString() const {
|
||||
int runnersDone = 0;
|
||||
for (int i = 0; i < runners.size(); ++i) {
|
||||
if (runners[i].isReady()) {
|
||||
++runnersDone;
|
||||
}
|
||||
}
|
||||
|
||||
std::string s =
|
||||
format("{ ptr=%p concurrency=%d available=%d running=%d waiting=%d runnersQueue=%d runnersDone=%d ",
|
||||
this,
|
||||
concurrency,
|
||||
available,
|
||||
concurrency - available,
|
||||
waiting,
|
||||
runners.size(),
|
||||
runnersDone);
|
||||
|
||||
for (int i = 0; i < waiters.size(); ++i) {
|
||||
s += format("p%d_waiters=%u ", i, waiters[i].size());
|
||||
}
|
||||
|
||||
s += "}";
|
||||
return s;
|
||||
}
|
||||
|
||||
private:
|
||||
|
@ -181,6 +213,13 @@ private:
|
|||
state Future<Void> error = self->brokenOnDestruct.getFuture();
|
||||
state int maxPriority = self->waiters.size() - 1;
|
||||
|
||||
// Priority to try to run tasks from next
|
||||
state int priority = maxPriority;
|
||||
state Queue* pQueue = &self->waiters[maxPriority];
|
||||
|
||||
// Track the number of waiters unlocked at the same priority in a row
|
||||
state int lastPriorityCount = 0;
|
||||
|
||||
loop {
|
||||
// Cleanup finished runner futures at the front of the runner queue.
|
||||
while (!self->runners.empty() && self->runners.front().isReady()) {
|
||||
|
@ -197,20 +236,22 @@ private:
|
|||
}
|
||||
|
||||
// While there are available slots and there are waiters, launch tasks
|
||||
int priority = maxPriority;
|
||||
|
||||
while (self->available > 0 && self->waiting > 0) {
|
||||
auto& q = self->waiters[priority];
|
||||
prioritylock_printf(
|
||||
"Checking priority=%d prioritySize=%d %s\n", priority, q.size(), self->toString().c_str());
|
||||
prioritylock_printf("Checking priority=%d lastPriorityCount=%d %s\n",
|
||||
priority,
|
||||
lastPriorityCount,
|
||||
self->toString().c_str());
|
||||
|
||||
while (!q.empty()) {
|
||||
Slot s = q.front();
|
||||
q.pop_front();
|
||||
while (!pQueue->empty() && ++lastPriorityCount < self->launchLimit) {
|
||||
Waiter w = pQueue->front();
|
||||
pQueue->pop_front();
|
||||
--self->waiting;
|
||||
Lock lock;
|
||||
prioritylock_printf(" Running waiter priority=%d prioritySize=%d\n", priority, q.size());
|
||||
s.send(lock);
|
||||
prioritylock_printf(" Running waiter priority=%d wait=%f %s\n",
|
||||
priority,
|
||||
now() - w.queuedTime,
|
||||
self->toString().c_str());
|
||||
w.lockPromise.send(lock);
|
||||
|
||||
// Self may have been destructed during the lock callback
|
||||
if (error.isReady()) {
|
||||
|
@ -228,24 +269,28 @@ private:
|
|||
}
|
||||
}
|
||||
|
||||
// Wrap around to highest priority
|
||||
// If there are no more slots available, then don't move to the next priority
|
||||
if (self->available == 0) {
|
||||
break;
|
||||
}
|
||||
|
||||
// Decrease priority, wrapping around to max from 0
|
||||
if (priority == 0) {
|
||||
priority = maxPriority;
|
||||
} else {
|
||||
--priority;
|
||||
}
|
||||
|
||||
pQueue = &self->waiters[priority];
|
||||
lastPriorityCount = 0;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
std::string toString() const {
|
||||
return format(
|
||||
"{ slots=%d/%d waiting=%d runners=%d }", (concurrency - available), concurrency, waiting, runners.size());
|
||||
}
|
||||
|
||||
int concurrency;
|
||||
int available;
|
||||
int waiting;
|
||||
int launchLimit;
|
||||
std::vector<Queue> waiters;
|
||||
Deque<Future<Void>> runners;
|
||||
Future<Void> fRunner;
|
||||
|
@ -371,7 +416,7 @@ std::string toString(const std::pair<F, S>& o) {
|
|||
|
||||
static constexpr int ioMinPriority = 0;
|
||||
static constexpr int ioLeafPriority = 1;
|
||||
static constexpr int ioMaxPriority = 2;
|
||||
static constexpr int ioMaxPriority = 3;
|
||||
|
||||
// A FIFO queue of T stored as a linked list of pages.
|
||||
// Main operations are pop(), pushBack(), pushFront(), and flush().
|
||||
|
@ -2102,10 +2147,10 @@ public:
|
|||
int concurrentExtentReads,
|
||||
bool memoryOnly = false,
|
||||
Promise<Void> errorPromise = {})
|
||||
: ioLock(FLOW_KNOBS->MAX_OUTSTANDING, ioMaxPriority), pageCacheBytes(pageCacheSizeBytes), pHeader(nullptr),
|
||||
desiredPageSize(desiredPageSize), desiredExtentSize(desiredExtentSize), filename(filename),
|
||||
memoryOnly(memoryOnly), errorPromise(errorPromise), remapCleanupWindow(remapCleanupWindow),
|
||||
concurrentExtentReads(new FlowLock(concurrentExtentReads)) {
|
||||
: ioLock(FLOW_KNOBS->MAX_OUTSTANDING, ioMaxPriority, FLOW_KNOBS->MAX_OUTSTANDING / 2),
|
||||
pageCacheBytes(pageCacheSizeBytes), pHeader(nullptr), desiredPageSize(desiredPageSize),
|
||||
desiredExtentSize(desiredExtentSize), filename(filename), memoryOnly(memoryOnly), errorPromise(errorPromise),
|
||||
remapCleanupWindow(remapCleanupWindow), concurrentExtentReads(new FlowLock(concurrentExtentReads)) {
|
||||
|
||||
if (!g_redwoodMetricsActor.isValid()) {
|
||||
g_redwoodMetricsActor = redwoodMetricsLogger();
|
||||
|
@ -2175,6 +2220,8 @@ public:
|
|||
wait(store(fileSize, self->pageFile->size()));
|
||||
}
|
||||
|
||||
self->fileExtension = Void();
|
||||
|
||||
debug_printf(
|
||||
"DWALPager(%s) recover exists=%d fileSize=%" PRId64 "\n", self->filename.c_str(), exists, fileSize);
|
||||
// TODO: If the file exists but appears to never have been successfully committed is this an error or
|
||||
|
@ -2221,6 +2268,9 @@ public:
|
|||
}
|
||||
|
||||
self->setPageSize(self->pHeader->pageSize);
|
||||
self->filePageCount = fileSize / self->physicalPageSize;
|
||||
self->filePageCountPending = self->filePageCount;
|
||||
|
||||
if (self->logicalPageSize != self->desiredPageSize) {
|
||||
TraceEvent(SevWarn, "RedwoodPageSizeNotDesired")
|
||||
.detail("Filename", self->filename)
|
||||
|
@ -2320,6 +2370,8 @@ public:
|
|||
|
||||
// Now that the header page has been allocated, set page size to desired
|
||||
self->setPageSize(self->desiredPageSize);
|
||||
self->filePageCount = 0;
|
||||
self->filePageCountPending = 0;
|
||||
|
||||
// Now set the extent size, do this always after setting the page size as
|
||||
// extent size is a multiple of page size
|
||||
|
@ -2397,7 +2449,10 @@ public:
|
|||
self->recoveryVersion,
|
||||
self->pHeader->oldestVersion,
|
||||
self->logicalPageSize,
|
||||
self->physicalPageSize);
|
||||
self->physicalPageSize,
|
||||
self->pHeader->pageCount,
|
||||
self->filePageCount);
|
||||
|
||||
return Void();
|
||||
}
|
||||
|
||||
|
@ -2486,12 +2541,14 @@ public:
|
|||
// Grow the pager file by one page and return it
|
||||
LogicalPageID newLastPageID() {
|
||||
LogicalPageID id = pHeader->pageCount;
|
||||
++pHeader->pageCount;
|
||||
growPager(1);
|
||||
return id;
|
||||
}
|
||||
|
||||
Future<LogicalPageID> newPageID() override { return newPageID_impl(this); }
|
||||
|
||||
void growPager(int64_t pages) { pHeader->pageCount += pages; }
|
||||
|
||||
// Get a new, previously available extent and it's first page ID. The page will be considered in-use after the next
|
||||
// commit regardless of whether or not it was written to, until it is returned to the pager via freePage()
|
||||
ACTOR static Future<LogicalPageID> newExtentPageID_impl(DWALPager* self, QueueID queueID) {
|
||||
|
@ -2521,7 +2578,7 @@ public:
|
|||
// That translates to extentID being same as the return first pageID
|
||||
LogicalPageID newLastExtentID() {
|
||||
LogicalPageID id = pHeader->pageCount;
|
||||
pHeader->pageCount += pagesPerExtent;
|
||||
growPager(pagesPerExtent);
|
||||
return id;
|
||||
}
|
||||
|
||||
|
@ -2541,11 +2598,44 @@ public:
|
|||
if (self->memoryOnly) {
|
||||
return Void();
|
||||
}
|
||||
|
||||
// If a truncation up to include pageID has not yet been completed
|
||||
if (pageID >= self->filePageCount) {
|
||||
// And no extension pending will include pageID
|
||||
if (pageID >= self->filePageCountPending) {
|
||||
// Update extension to a new one that waits on the old one and extends further
|
||||
self->fileExtension = extendToCover(self, pageID, self->fileExtension);
|
||||
}
|
||||
|
||||
// Wait for extension that covers pageID to complete;
|
||||
wait(self->fileExtension);
|
||||
}
|
||||
|
||||
// Note: Not using forwardError here so a write error won't be discovered until commit time.
|
||||
wait(self->pageFile->write(data, blockSize, (int64_t)pageID * blockSize));
|
||||
return Void();
|
||||
}
|
||||
|
||||
ACTOR static Future<Void> extendToCover(DWALPager* self, uint64_t pageID, Future<Void> previousExtension) {
|
||||
// Calculate new page count, round up to nearest multiple of growth size > pageID
|
||||
state int64_t newPageCount = pageID + SERVER_KNOBS->REDWOOD_PAGEFILE_GROWTH_SIZE_PAGES -
|
||||
(pageID % SERVER_KNOBS->REDWOOD_PAGEFILE_GROWTH_SIZE_PAGES);
|
||||
|
||||
// Indicate that extension to this new count has been started
|
||||
self->filePageCountPending = newPageCount;
|
||||
|
||||
// Wait for any previous extensions to complete
|
||||
wait(previousExtension);
|
||||
|
||||
// Grow the file
|
||||
wait(self->pageFile->truncate(newPageCount * self->physicalPageSize));
|
||||
|
||||
// Indicate that extension to the new count has been completed
|
||||
self->filePageCount = newPageCount;
|
||||
|
||||
return Void();
|
||||
}
|
||||
|
||||
Future<Void> writePhysicalPage(PagerEventReasons reason,
|
||||
unsigned int level,
|
||||
Standalone<VectorRef<PhysicalPageID>> pageIDs,
|
||||
|
@ -3699,6 +3789,13 @@ private:
|
|||
Reference<ArenaPage> headerPage;
|
||||
Header* pHeader;
|
||||
|
||||
// Pages - pages known to be in the file, truncations complete to that size
|
||||
int64_t filePageCount;
|
||||
// Pages that will be in file once fileExtension is ready
|
||||
int64_t filePageCountPending;
|
||||
// Future representing the end of all pending truncations
|
||||
Future<Void> fileExtension;
|
||||
|
||||
int desiredPageSize;
|
||||
int desiredExtentSize;
|
||||
|
||||
|
@ -4941,6 +5038,7 @@ private:
|
|||
bool clearAfterBoundary;
|
||||
|
||||
bool boundaryCleared() const { return boundaryChanged && !boundaryValue.present(); }
|
||||
bool boundarySet() const { return boundaryChanged && boundaryValue.present(); }
|
||||
|
||||
// Returns true if this RangeMutation doesn't actually mutate anything
|
||||
bool noChanges() const { return !boundaryChanged && !clearAfterBoundary; }
|
||||
|
@ -4960,8 +5058,6 @@ private:
|
|||
boundaryValue = v;
|
||||
}
|
||||
|
||||
bool boundarySet() const { return boundaryChanged && boundaryValue.present(); }
|
||||
|
||||
std::string toString() const {
|
||||
return format("boundaryChanged=%d clearAfterBoundary=%d boundaryValue=%s",
|
||||
boundaryChanged,
|
||||
|
@ -6077,22 +6173,22 @@ private:
|
|||
|
||||
// Leaf Page
|
||||
if (btPage->isLeaf()) {
|
||||
bool updating = tryToUpdate;
|
||||
// When true, we are modifying the existing DeltaTree
|
||||
// When false, we are accumulating retained and added records in merged vector to build pages from them.
|
||||
bool updatingDeltaTree = tryToUpdate;
|
||||
bool changesMade = false;
|
||||
|
||||
state Standalone<VectorRef<RedwoodRecordRef>> merged;
|
||||
auto switchToLinearMerge = [&]() {
|
||||
// Couldn't make changes in place, so now do a linear merge and build new pages.
|
||||
updating = false;
|
||||
auto c = cursor;
|
||||
c.moveFirst();
|
||||
while (c != cursor) {
|
||||
debug_printf("%s catch-up adding %s\n", context.c_str(), c.get().toString().c_str());
|
||||
merged.push_back(merged.arena(), c.get());
|
||||
c.moveNext();
|
||||
// Copy page for modification if not already copied
|
||||
auto copyForUpdate = [&]() {
|
||||
if (!pageCopy.isValid()) {
|
||||
pageCopy = clonePageForUpdate(page);
|
||||
btPage = (BTreePage*)pageCopy->begin();
|
||||
cursor.switchTree(btPage->tree());
|
||||
}
|
||||
};
|
||||
|
||||
state Standalone<VectorRef<RedwoodRecordRef>> merged;
|
||||
|
||||
// The first mutation buffer boundary has a key <= the first key in the page.
|
||||
|
||||
cursor.moveFirst();
|
||||
|
@ -6100,48 +6196,59 @@ private:
|
|||
|
||||
// Now, process each mutation range and merge changes with existing data.
|
||||
bool firstMutationBoundary = true;
|
||||
while (mBegin != mEnd) {
|
||||
debug_printf("%s New mutation boundary: '%s': %s\n",
|
||||
context.c_str(),
|
||||
printable(mBegin.key()).c_str(),
|
||||
mBegin.mutation().toString().c_str());
|
||||
constexpr int maxHeightAllowed = 8;
|
||||
|
||||
while (mBegin != mEnd) {
|
||||
// Apply the change to the mutation buffer start boundary key only if
|
||||
// - there actually is a change (whether a set or a clear, old records are to be removed)
|
||||
// - there actually is a change (clear or set to new value)
|
||||
// - either this is not the first boundary or it is but its key matches our lower bound key
|
||||
bool applyBoundaryChange = mBegin.mutation().boundaryChanged &&
|
||||
(!firstMutationBoundary || mBegin.key() == update->subtreeLowerBound.key);
|
||||
bool boundaryExists = cursor.valid() && cursor.get().key == mBegin.key();
|
||||
|
||||
debug_printf("%s New mutation boundary: '%s': %s applyBoundaryChange=%d boundaryExists=%d "
|
||||
"updatingDeltaTree=%d\n",
|
||||
context.c_str(),
|
||||
printable(mBegin.key()).c_str(),
|
||||
mBegin.mutation().toString().c_str(),
|
||||
applyBoundaryChange,
|
||||
boundaryExists,
|
||||
updatingDeltaTree);
|
||||
|
||||
firstMutationBoundary = false;
|
||||
|
||||
// Iterate over records for the mutation boundary key, keep them unless the boundary key was changed or
|
||||
// we are not applying it
|
||||
while (cursor.valid() && cursor.get().key == mBegin.key()) {
|
||||
// If there were no changes to the key or we're not applying it
|
||||
if (!applyBoundaryChange) {
|
||||
// If not updating, add to the output set, otherwise skip ahead past the records for the
|
||||
// mutation boundary
|
||||
if (!updating) {
|
||||
merged.push_back(merged.arena(), cursor.get());
|
||||
debug_printf("%s Added %s [existing, boundary start]\n",
|
||||
context.c_str(),
|
||||
cursor.get().toString().c_str());
|
||||
}
|
||||
cursor.moveNext();
|
||||
} else {
|
||||
if (applyBoundaryChange) {
|
||||
// If the boundary is being set to a value, the new KV record will be inserted
|
||||
bool shouldInsertBoundary = mBegin.mutation().boundarySet();
|
||||
|
||||
// Optimization: In-place value update of new same-sized value
|
||||
// If the boundary exists in the page and we're in update mode and the boundary is being set to a
|
||||
// new value of the same length as the old value then just update the value bytes.
|
||||
if (boundaryExists && updatingDeltaTree && shouldInsertBoundary &&
|
||||
mBegin.mutation().boundaryValue.get().size() == cursor.get().value.get().size()) {
|
||||
changesMade = true;
|
||||
shouldInsertBoundary = false;
|
||||
|
||||
debug_printf("%s In-place value update for %s [existing, boundary start]\n",
|
||||
context.c_str(),
|
||||
cursor.get().toString().c_str());
|
||||
|
||||
copyForUpdate();
|
||||
memcpy((uint8_t*)cursor.get().value.get().begin(),
|
||||
mBegin.mutation().boundaryValue.get().begin(),
|
||||
cursor.get().value.get().size());
|
||||
cursor.moveNext();
|
||||
} else if (boundaryExists) {
|
||||
// An in place update can't be done, so if the boundary exists then erase or skip the record
|
||||
changesMade = true;
|
||||
|
||||
// If updating, erase from the page, otherwise do not add to the output set
|
||||
if (updating) {
|
||||
if (updatingDeltaTree) {
|
||||
debug_printf("%s Erasing %s [existing, boundary start]\n",
|
||||
context.c_str(),
|
||||
cursor.get().toString().c_str());
|
||||
|
||||
// Copy page for modification if not already copied
|
||||
if (!pageCopy.isValid()) {
|
||||
pageCopy = clonePageForUpdate(page);
|
||||
btPage = (BTreePage*)pageCopy->begin();
|
||||
cursor.tree = btPage->tree();
|
||||
}
|
||||
|
||||
copyForUpdate();
|
||||
btPage->kvBytes -= cursor.get().kvBytes();
|
||||
cursor.erase();
|
||||
} else {
|
||||
|
@ -6151,42 +6258,59 @@ private:
|
|||
cursor.moveNext();
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
constexpr int maxHeightAllowed = 8;
|
||||
// If the boundary value is being set and we must insert it, add it to the page or the output set
|
||||
if (shouldInsertBoundary) {
|
||||
RedwoodRecordRef rec(mBegin.key(), mBegin.mutation().boundaryValue.get());
|
||||
changesMade = true;
|
||||
|
||||
// Write the new record(s) for the mutation boundary start key if its value has been set
|
||||
// Clears of this key will have been processed above by not being erased from the updated page or
|
||||
// excluded from the merge output
|
||||
if (applyBoundaryChange && mBegin.mutation().boundarySet()) {
|
||||
RedwoodRecordRef rec(mBegin.key(), mBegin.mutation().boundaryValue.get());
|
||||
changesMade = true;
|
||||
// If updating, first try to add the record to the page
|
||||
if (updatingDeltaTree) {
|
||||
copyForUpdate();
|
||||
if (cursor.insert(rec, update->skipLen, maxHeightAllowed)) {
|
||||
btPage->kvBytes += rec.kvBytes();
|
||||
debug_printf("%s Inserted %s [mutation, boundary start]\n",
|
||||
context.c_str(),
|
||||
rec.toString().c_str());
|
||||
} else {
|
||||
debug_printf("%s Insert failed for %s [mutation, boundary start]\n",
|
||||
context.c_str(),
|
||||
rec.toString().c_str());
|
||||
|
||||
// If updating, add to the page, else add to the output set
|
||||
if (updating) {
|
||||
// Copy page for modification if not already copied
|
||||
if (!pageCopy.isValid()) {
|
||||
pageCopy = clonePageForUpdate(page);
|
||||
btPage = (BTreePage*)pageCopy->begin();
|
||||
cursor.tree = btPage->tree();
|
||||
// Since the insert failed we must switch to a linear merge of existing data and
|
||||
// mutations, accumulating the new record set in the merge vector and build new pages
|
||||
// from it. First, we must populate the merged vector with all the records up to but not
|
||||
// including the current mutation boundary key.
|
||||
auto c = cursor;
|
||||
c.moveFirst();
|
||||
while (c != cursor) {
|
||||
debug_printf(
|
||||
"%s catch-up adding %s\n", context.c_str(), c.get().toString().c_str());
|
||||
merged.push_back(merged.arena(), c.get());
|
||||
c.moveNext();
|
||||
}
|
||||
updatingDeltaTree = false;
|
||||
}
|
||||
}
|
||||
|
||||
if (cursor.insert(rec, update->skipLen, maxHeightAllowed)) {
|
||||
btPage->kvBytes += rec.kvBytes();
|
||||
// If not updating, possibly due to insert failure above, then add record to the output set
|
||||
if (!updatingDeltaTree) {
|
||||
merged.push_back(merged.arena(), rec);
|
||||
debug_printf(
|
||||
"%s Inserted %s [mutation, boundary start]\n", context.c_str(), rec.toString().c_str());
|
||||
} else {
|
||||
debug_printf("%s Insert failed for %s [mutation, boundary start]\n",
|
||||
context.c_str(),
|
||||
rec.toString().c_str());
|
||||
switchToLinearMerge();
|
||||
"%s Added %s [mutation, boundary start]\n", context.c_str(), rec.toString().c_str());
|
||||
}
|
||||
}
|
||||
|
||||
if (!updating) {
|
||||
merged.push_back(merged.arena(), rec);
|
||||
debug_printf(
|
||||
"%s Added %s [mutation, boundary start]\n", context.c_str(), rec.toString().c_str());
|
||||
} else if (boundaryExists) {
|
||||
// If the boundary exists in the page but there is no pending change,
|
||||
// then if updating move past it, otherwise add it to the output set.
|
||||
if (updatingDeltaTree) {
|
||||
cursor.moveNext();
|
||||
} else {
|
||||
merged.push_back(merged.arena(), cursor.get());
|
||||
debug_printf("%s Added %s [existing, boundary start]\n",
|
||||
context.c_str(),
|
||||
cursor.get().toString().c_str());
|
||||
cursor.moveNext();
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -6205,35 +6329,29 @@ private:
|
|||
|
||||
// If the records are being removed and we're not doing an in-place update
|
||||
// OR if we ARE doing an update but the records are NOT being removed, then just skip them.
|
||||
if (remove != updating) {
|
||||
if (remove != updatingDeltaTree) {
|
||||
// If not updating, then the records, if any exist, are being removed. We don't know if there
|
||||
// actually are any but we must assume there are.
|
||||
if (!updating) {
|
||||
if (!updatingDeltaTree) {
|
||||
changesMade = true;
|
||||
}
|
||||
|
||||
debug_printf("%s Seeking forward to next boundary (remove=%d updating=%d) %s\n",
|
||||
context.c_str(),
|
||||
remove,
|
||||
updating,
|
||||
updatingDeltaTree,
|
||||
mBegin.key().toString().c_str());
|
||||
cursor.seekGreaterThanOrEqual(end, update->skipLen);
|
||||
} else {
|
||||
// Otherwise we must visit the records. If updating, the visit is to erase them, and if doing a
|
||||
// linear merge than the visit is to add them to the output set.
|
||||
while (cursor.valid() && cursor.get().compare(end, update->skipLen) < 0) {
|
||||
if (updating) {
|
||||
if (updatingDeltaTree) {
|
||||
debug_printf("%s Erasing %s [existing, boundary start]\n",
|
||||
context.c_str(),
|
||||
cursor.get().toString().c_str());
|
||||
|
||||
// Copy page for modification if not already copied
|
||||
if (!pageCopy.isValid()) {
|
||||
pageCopy = clonePageForUpdate(page);
|
||||
btPage = (BTreePage*)pageCopy->begin();
|
||||
cursor.tree = btPage->tree();
|
||||
}
|
||||
|
||||
copyForUpdate();
|
||||
btPage->kvBytes -= cursor.get().kvBytes();
|
||||
cursor.erase();
|
||||
changesMade = true;
|
||||
|
@ -6257,27 +6375,23 @@ private:
|
|||
|
||||
// If we don't have to remove the records and we are updating, do nothing.
|
||||
// If we do have to remove the records and we are not updating, do nothing.
|
||||
if (remove != updating) {
|
||||
debug_printf(
|
||||
"%s Ignoring remaining records, remove=%d updating=%d\n", context.c_str(), remove, updating);
|
||||
if (remove != updatingDeltaTree) {
|
||||
debug_printf("%s Ignoring remaining records, remove=%d updating=%d\n",
|
||||
context.c_str(),
|
||||
remove,
|
||||
updatingDeltaTree);
|
||||
} else {
|
||||
// If updating and the key is changing, we must visit the records to erase them.
|
||||
// If not updating and the key is not changing, we must visit the records to add them to the output
|
||||
// set.
|
||||
while (cursor.valid()) {
|
||||
if (updating) {
|
||||
if (updatingDeltaTree) {
|
||||
debug_printf(
|
||||
"%s Erasing %s and beyond [existing, matches changed upper mutation boundary]\n",
|
||||
context.c_str(),
|
||||
cursor.get().toString().c_str());
|
||||
|
||||
// Copy page for modification if not already copied
|
||||
if (!pageCopy.isValid()) {
|
||||
pageCopy = clonePageForUpdate(page);
|
||||
btPage = (BTreePage*)pageCopy->begin();
|
||||
cursor.tree = btPage->tree();
|
||||
}
|
||||
|
||||
copyForUpdate();
|
||||
btPage->kvBytes -= cursor.get().kvBytes();
|
||||
cursor.erase();
|
||||
} else {
|
||||
|
@ -6305,7 +6419,7 @@ private:
|
|||
context.c_str());
|
||||
}
|
||||
|
||||
if (updating) {
|
||||
if (updatingDeltaTree) {
|
||||
// If the tree is now empty, delete the page
|
||||
if (cursor.tree->numItems == 0) {
|
||||
update->cleared();
|
||||
|
@ -6581,7 +6695,7 @@ private:
|
|||
if (modifier.clonedPage) {
|
||||
pageCopy = modifier.page;
|
||||
btPage = modifier.btPage();
|
||||
cursor.tree = btPage->tree();
|
||||
cursor.switchTree(btPage->tree());
|
||||
}
|
||||
|
||||
// If page contents have changed
|
||||
|
@ -7137,7 +7251,7 @@ RedwoodRecordRef VersionedBTree::dbEnd(LiteralStringRef("\xff\xff\xff\xff\xff"))
|
|||
class KeyValueStoreRedwood : public IKeyValueStore {
|
||||
public:
|
||||
KeyValueStoreRedwood(std::string filePrefix, UID logID)
|
||||
: m_filePrefix(filePrefix), m_concurrentReads(SERVER_KNOBS->REDWOOD_KVSTORE_CONCURRENT_READS, 0),
|
||||
: m_filename(filePrefix), m_concurrentReads(SERVER_KNOBS->REDWOOD_KVSTORE_CONCURRENT_READS, 0),
|
||||
prefetch(SERVER_KNOBS->REDWOOD_KVSTORE_RANGE_PREFETCH) {
|
||||
|
||||
int pageSize =
|
||||
|
@ -7167,17 +7281,17 @@ public:
|
|||
Future<Void> init() override { return m_init; }
|
||||
|
||||
ACTOR Future<Void> init_impl(KeyValueStoreRedwood* self) {
|
||||
TraceEvent(SevInfo, "RedwoodInit").detail("FilePrefix", self->m_filePrefix);
|
||||
TraceEvent(SevInfo, "RedwoodInit").detail("FilePrefix", self->m_filename);
|
||||
wait(self->m_tree->init());
|
||||
TraceEvent(SevInfo, "RedwoodInitComplete")
|
||||
.detail("FilePrefix", self->m_filePrefix)
|
||||
.detail("Filename", self->m_filename)
|
||||
.detail("Version", self->m_tree->getLastCommittedVersion());
|
||||
self->m_nextCommitVersion = self->m_tree->getLastCommittedVersion() + 1;
|
||||
return Void();
|
||||
}
|
||||
|
||||
ACTOR void shutdown(KeyValueStoreRedwood* self, bool dispose) {
|
||||
TraceEvent(SevInfo, "RedwoodShutdown").detail("FilePrefix", self->m_filePrefix).detail("Dispose", dispose);
|
||||
TraceEvent(SevInfo, "RedwoodShutdown").detail("Filename", self->m_filename).detail("Dispose", dispose);
|
||||
if (self->m_error.canBeSet()) {
|
||||
self->m_error.sendError(actor_cancelled()); // Ideally this should be shutdown_in_progress
|
||||
}
|
||||
|
@ -7189,9 +7303,7 @@ public:
|
|||
self->m_tree->close();
|
||||
wait(closedFuture);
|
||||
self->m_closed.send(Void());
|
||||
TraceEvent(SevInfo, "RedwoodShutdownComplete")
|
||||
.detail("FilePrefix", self->m_filePrefix)
|
||||
.detail("Dispose", dispose);
|
||||
TraceEvent(SevInfo, "RedwoodShutdownComplete").detail("Filename", self->m_filename).detail("Dispose", dispose);
|
||||
delete self;
|
||||
}
|
||||
|
||||
|
@ -7411,7 +7523,7 @@ public:
|
|||
~KeyValueStoreRedwood() override{};
|
||||
|
||||
private:
|
||||
std::string m_filePrefix;
|
||||
std::string m_filename;
|
||||
VersionedBTree* m_tree;
|
||||
Future<Void> m_init;
|
||||
Promise<Void> m_closed;
|
||||
|
@ -8065,7 +8177,7 @@ TEST_CASE("/redwood/correctness/unit/RedwoodRecordRef") {
|
|||
return Void();
|
||||
}
|
||||
|
||||
TEST_CASE("/redwood/correctness/unit/deltaTree/RedwoodRecordRef") {
|
||||
TEST_CASE("Lredwood/correctness/unit/deltaTree/RedwoodRecordRef") {
|
||||
// Sanity check on delta tree node format
|
||||
ASSERT(DeltaTree2<RedwoodRecordRef>::Node::headerSize(false) == 4);
|
||||
ASSERT(DeltaTree2<RedwoodRecordRef>::Node::headerSize(true) == 8);
|
||||
|
@ -8241,7 +8353,7 @@ TEST_CASE("/redwood/correctness/unit/deltaTree/RedwoodRecordRef") {
|
|||
return Void();
|
||||
}
|
||||
|
||||
TEST_CASE("/redwood/correctness/unit/deltaTree/RedwoodRecordRef2") {
|
||||
TEST_CASE("Lredwood/correctness/unit/deltaTree/RedwoodRecordRef2") {
|
||||
// Sanity check on delta tree node format
|
||||
ASSERT(DeltaTree2<RedwoodRecordRef>::Node::headerSize(false) == 4);
|
||||
ASSERT(DeltaTree2<RedwoodRecordRef>::Node::headerSize(true) == 8);
|
||||
|
@ -8420,7 +8532,7 @@ TEST_CASE("/redwood/correctness/unit/deltaTree/RedwoodRecordRef2") {
|
|||
return Void();
|
||||
}
|
||||
|
||||
TEST_CASE("/redwood/correctness/unit/deltaTree/IntIntPair") {
|
||||
TEST_CASE("Lredwood/correctness/unit/deltaTree/IntIntPair") {
|
||||
const int N = 200;
|
||||
IntIntPair lowerBound = { 0, 0 };
|
||||
IntIntPair upperBound = { 1000, 1000 };
|
||||
|
@ -9017,11 +9129,11 @@ TEST_CASE(":/redwood/pager/ArenaPage") {
|
|||
return Void();
|
||||
}
|
||||
|
||||
TEST_CASE("/redwood/correctness/btree") {
|
||||
TEST_CASE("Lredwood/correctness/btree") {
|
||||
g_redwoodMetricsActor = Void(); // Prevent trace event metrics from starting
|
||||
g_redwoodMetrics.clear();
|
||||
|
||||
state std::string fileName = params.get("fileName").orDefault("unittest_pageFile.redwood");
|
||||
state std::string fileName = params.get("Filename").orDefault("unittest_pageFile.redwood");
|
||||
IPager2* pager;
|
||||
|
||||
state bool serialTest = params.getInt("serialTest").orDefault(deterministicRandom()->random01() < 0.25);
|
||||
|
@ -9611,7 +9723,7 @@ TEST_CASE(":/redwood/performance/extentQueue") {
|
|||
TEST_CASE(":/redwood/performance/set") {
|
||||
state SignalableActorCollection actors;
|
||||
|
||||
state std::string fileName = params.get("fileName").orDefault("unittest.redwood");
|
||||
state std::string fileName = params.get("Filename").orDefault("unittest.redwood");
|
||||
state int pageSize = params.getInt("pageSize").orDefault(SERVER_KNOBS->REDWOOD_DEFAULT_PAGE_SIZE);
|
||||
state int extentSize = params.getInt("extentSize").orDefault(SERVER_KNOBS->REDWOOD_DEFAULT_EXTENT_SIZE);
|
||||
state int64_t pageCacheBytes = params.getInt("pageCacheBytes").orDefault(FLOW_KNOBS->PAGE_CACHE_4K);
|
||||
|
@ -10254,7 +10366,7 @@ ACTOR Future<Void> randomRangeScans(IKeyValueStore* kvs,
|
|||
return Void();
|
||||
}
|
||||
|
||||
TEST_CASE("!/redwood/performance/randomRangeScans") {
|
||||
TEST_CASE(":/redwood/performance/randomRangeScans") {
|
||||
state int prefixLen = 30;
|
||||
state int suffixSize = 12;
|
||||
state int valueSize = 100;
|
||||
|
|
|
@ -1094,6 +1094,66 @@ ACTOR Future<Void> tLog(IKeyValueStore* persistentData,
|
|||
|
||||
typedef decltype(&tLog) TLogFn;
|
||||
|
||||
ACTOR template <class T>
|
||||
Future<T> ioTimeoutError(Future<T> what, double time) {
|
||||
// Before simulation is sped up, IO operations can take a very long time so limit timeouts
|
||||
// to not end until at least time after simulation is sped up.
|
||||
if (g_network->isSimulated() && !g_simulator.speedUpSimulation) {
|
||||
time += std::max(0.0, FLOW_KNOBS->SIM_SPEEDUP_AFTER_SECONDS - now());
|
||||
}
|
||||
Future<Void> end = lowPriorityDelay(time);
|
||||
choose {
|
||||
when(T t = wait(what)) { return t; }
|
||||
when(wait(end)) {
|
||||
Error err = io_timeout();
|
||||
if (g_network->isSimulated() && !g_simulator.getCurrentProcess()->isReliable()) {
|
||||
err = err.asInjectedFault();
|
||||
}
|
||||
TraceEvent(SevError, "IoTimeoutError").error(err);
|
||||
throw err;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
ACTOR template <class T>
|
||||
Future<T> ioDegradedOrTimeoutError(Future<T> what,
|
||||
double errTime,
|
||||
Reference<AsyncVar<bool>> degraded,
|
||||
double degradedTime) {
|
||||
// Before simulation is sped up, IO operations can take a very long time so limit timeouts
|
||||
// to not end until at least time after simulation is sped up.
|
||||
if (g_network->isSimulated() && !g_simulator.speedUpSimulation) {
|
||||
double timeShift = std::max(0.0, FLOW_KNOBS->SIM_SPEEDUP_AFTER_SECONDS - now());
|
||||
errTime += timeShift;
|
||||
degradedTime += timeShift;
|
||||
}
|
||||
|
||||
if (degradedTime < errTime) {
|
||||
Future<Void> degradedEnd = lowPriorityDelay(degradedTime);
|
||||
choose {
|
||||
when(T t = wait(what)) { return t; }
|
||||
when(wait(degradedEnd)) {
|
||||
TEST(true); // TLog degraded
|
||||
TraceEvent(SevWarnAlways, "IoDegraded").log();
|
||||
degraded->set(true);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
Future<Void> end = lowPriorityDelay(errTime - degradedTime);
|
||||
choose {
|
||||
when(T t = wait(what)) { return t; }
|
||||
when(wait(end)) {
|
||||
Error err = io_timeout();
|
||||
if (g_network->isSimulated() && !g_simulator.getCurrentProcess()->isReliable()) {
|
||||
err = err.asInjectedFault();
|
||||
}
|
||||
TraceEvent(SevError, "IoTimeoutError").error(err);
|
||||
throw err;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
#include "fdbserver/ServerDBInfo.h"
|
||||
#include "flow/unactorcompiler.h"
|
||||
#endif
|
||||
|
|
|
@ -42,6 +42,7 @@
|
|||
#include "fdbclient/NativeAPI.actor.h"
|
||||
#include "fdbclient/Notified.h"
|
||||
#include "fdbclient/StatusClient.h"
|
||||
#include "fdbclient/Tuple.h"
|
||||
#include "fdbclient/SystemData.h"
|
||||
#include "fdbclient/TransactionLineage.h"
|
||||
#include "fdbclient/VersionedMap.h"
|
||||
|
@ -83,6 +84,14 @@ bool canReplyWith(Error e) {
|
|||
case error_code_process_behind:
|
||||
case error_code_watch_cancelled:
|
||||
case error_code_unknown_change_feed:
|
||||
case error_code_server_overloaded:
|
||||
// getRangeAndMap related exceptions that are not retriable:
|
||||
case error_code_mapper_bad_index:
|
||||
case error_code_mapper_no_such_key:
|
||||
case error_code_mapper_bad_range_decriptor:
|
||||
case error_code_quick_get_key_values_has_more:
|
||||
case error_code_quick_get_value_miss:
|
||||
case error_code_quick_get_key_values_miss:
|
||||
// case error_code_all_alternatives_failed:
|
||||
return true;
|
||||
default:
|
||||
|
@ -780,8 +789,9 @@ public:
|
|||
|
||||
struct Counters {
|
||||
CounterCollection cc;
|
||||
Counter allQueries, getKeyQueries, getValueQueries, getRangeQueries, getRangeStreamQueries, finishedQueries,
|
||||
lowPriorityQueries, rowsQueried, bytesQueried, watchQueries, emptyQueries;
|
||||
Counter allQueries, getKeyQueries, getValueQueries, getRangeQueries, getRangeAndFlatMapQueries,
|
||||
getRangeStreamQueries, finishedQueries, lowPriorityQueries, rowsQueried, bytesQueried, watchQueries,
|
||||
emptyQueries;
|
||||
|
||||
// Bytes of the mutations that have been added to the memory of the storage server. When the data is durable
|
||||
// and cleared from the memory, we do not subtract it but add it to bytesDurable.
|
||||
|
@ -808,6 +818,10 @@ public:
|
|||
Counter wrongShardServer;
|
||||
Counter fetchedVersions;
|
||||
Counter fetchesFromLogs;
|
||||
// The following counters measure how many of lookups in the getRangeAndFlatMapQueries are effective. "Miss"
|
||||
// means fallback if fallback is enabled, otherwise means failure (so that another layer could implement
|
||||
// fallback).
|
||||
Counter quickGetValueHit, quickGetValueMiss, quickGetKeyValuesHit, quickGetKeyValuesMiss;
|
||||
|
||||
LatencySample readLatencySample;
|
||||
LatencyBands readLatencyBands;
|
||||
|
@ -815,22 +829,25 @@ public:
|
|||
Counters(StorageServer* self)
|
||||
: cc("StorageServer", self->thisServerID.toString()), allQueries("QueryQueue", cc),
|
||||
getKeyQueries("GetKeyQueries", cc), getValueQueries("GetValueQueries", cc),
|
||||
getRangeQueries("GetRangeQueries", cc), getRangeStreamQueries("GetRangeStreamQueries", cc),
|
||||
finishedQueries("FinishedQueries", cc), lowPriorityQueries("LowPriorityQueries", cc),
|
||||
rowsQueried("RowsQueried", cc), bytesQueried("BytesQueried", cc), watchQueries("WatchQueries", cc),
|
||||
emptyQueries("EmptyQueries", cc), bytesInput("BytesInput", cc), bytesDurable("BytesDurable", cc),
|
||||
bytesFetched("BytesFetched", cc), mutationBytes("MutationBytes", cc),
|
||||
sampledBytesCleared("SampledBytesCleared", cc), kvFetched("KVFetched", cc), mutations("Mutations", cc),
|
||||
setMutations("SetMutations", cc), clearRangeMutations("ClearRangeMutations", cc),
|
||||
atomicMutations("AtomicMutations", cc), updateBatches("UpdateBatches", cc),
|
||||
updateVersions("UpdateVersions", cc), loops("Loops", cc), fetchWaitingMS("FetchWaitingMS", cc),
|
||||
fetchWaitingCount("FetchWaitingCount", cc), fetchExecutingMS("FetchExecutingMS", cc),
|
||||
fetchExecutingCount("FetchExecutingCount", cc), readsRejected("ReadsRejected", cc),
|
||||
wrongShardServer("WrongShardServer", cc), fetchedVersions("FetchedVersions", cc),
|
||||
fetchesFromLogs("FetchesFromLogs", cc), readLatencySample("ReadLatencyMetrics",
|
||||
self->thisServerID,
|
||||
SERVER_KNOBS->LATENCY_METRICS_LOGGING_INTERVAL,
|
||||
SERVER_KNOBS->LATENCY_SAMPLE_SIZE),
|
||||
getRangeQueries("GetRangeQueries", cc), getRangeAndFlatMapQueries("GetRangeAndFlatMapQueries", cc),
|
||||
getRangeStreamQueries("GetRangeStreamQueries", cc), finishedQueries("FinishedQueries", cc),
|
||||
lowPriorityQueries("LowPriorityQueries", cc), rowsQueried("RowsQueried", cc),
|
||||
bytesQueried("BytesQueried", cc), watchQueries("WatchQueries", cc), emptyQueries("EmptyQueries", cc),
|
||||
bytesInput("BytesInput", cc), bytesDurable("BytesDurable", cc), bytesFetched("BytesFetched", cc),
|
||||
mutationBytes("MutationBytes", cc), sampledBytesCleared("SampledBytesCleared", cc),
|
||||
kvFetched("KVFetched", cc), mutations("Mutations", cc), setMutations("SetMutations", cc),
|
||||
clearRangeMutations("ClearRangeMutations", cc), atomicMutations("AtomicMutations", cc),
|
||||
updateBatches("UpdateBatches", cc), updateVersions("UpdateVersions", cc), loops("Loops", cc),
|
||||
fetchWaitingMS("FetchWaitingMS", cc), fetchWaitingCount("FetchWaitingCount", cc),
|
||||
fetchExecutingMS("FetchExecutingMS", cc), fetchExecutingCount("FetchExecutingCount", cc),
|
||||
readsRejected("ReadsRejected", cc), wrongShardServer("WrongShardServer", cc),
|
||||
fetchedVersions("FetchedVersions", cc), fetchesFromLogs("FetchesFromLogs", cc),
|
||||
quickGetValueHit("QuickGetValueHit", cc), quickGetValueMiss("QuickGetValueMiss", cc),
|
||||
quickGetKeyValuesHit("QuickGetKeyValuesHit", cc), quickGetKeyValuesMiss("QuickGetKeyValuesMiss", cc),
|
||||
readLatencySample("ReadLatencyMetrics",
|
||||
self->thisServerID,
|
||||
SERVER_KNOBS->LATENCY_METRICS_LOGGING_INTERVAL,
|
||||
SERVER_KNOBS->LATENCY_SAMPLE_SIZE),
|
||||
readLatencyBands("ReadLatencyBands", self->thisServerID, SERVER_KNOBS->STORAGE_LOGGING_DELAY) {
|
||||
specialCounter(cc, "LastTLogVersion", [self]() { return self->lastTLogVersion; });
|
||||
specialCounter(cc, "Version", [self]() { return self->version.get(); });
|
||||
|
@ -2037,6 +2054,37 @@ void merge(Arena& arena,
|
|||
}
|
||||
}
|
||||
|
||||
ACTOR Future<Optional<Value>> quickGetValue(StorageServer* data, StringRef key, Version version) {
|
||||
if (data->shards[key]->isReadable()) {
|
||||
try {
|
||||
// TODO: Use a lower level API may be better? Or tweak priorities?
|
||||
GetValueRequest req(Span().context, key, version, Optional<TagSet>(), Optional<UID>());
|
||||
data->actors.add(data->readGuard(req, getValueQ));
|
||||
GetValueReply reply = wait(req.reply.getFuture());
|
||||
++data->counters.quickGetValueHit;
|
||||
return reply.value;
|
||||
} catch (Error& e) {
|
||||
// Fallback.
|
||||
}
|
||||
} else {
|
||||
// Fallback.
|
||||
}
|
||||
|
||||
++data->counters.quickGetValueMiss;
|
||||
if (SERVER_KNOBS->QUICK_GET_VALUE_FALLBACK) {
|
||||
state Transaction tr(data->cx);
|
||||
tr.setVersion(version);
|
||||
// TODO: is DefaultPromiseEndpoint the best priority for this?
|
||||
tr.info.taskID = TaskPriority::DefaultPromiseEndpoint;
|
||||
Future<Optional<Value>> valueFuture = tr.get(key, Snapshot::True);
|
||||
// TODO: async in case it needs to read from other servers.
|
||||
state Optional<Value> valueOption = wait(valueFuture);
|
||||
return valueOption;
|
||||
} else {
|
||||
throw quick_get_value_miss();
|
||||
}
|
||||
};
|
||||
|
||||
// If limit>=0, it returns the first rows in the range (sorted ascending), otherwise the last rows (sorted descending).
|
||||
// readRange has O(|result|) + O(log |data|) cost
|
||||
ACTOR Future<GetKeyValuesReply> readRange(StorageServer* data,
|
||||
|
@ -2522,6 +2570,447 @@ ACTOR Future<Void> getKeyValuesQ(StorageServer* data, GetKeyValuesRequest req)
|
|||
return Void();
|
||||
}
|
||||
|
||||
ACTOR Future<RangeResult> quickGetKeyValues(StorageServer* data, StringRef prefix, Version version) {
|
||||
try {
|
||||
// TODO: Use a lower level API may be better? Or tweak priorities?
|
||||
GetKeyValuesRequest req;
|
||||
req.spanContext = Span().context;
|
||||
req.arena = Arena();
|
||||
req.begin = firstGreaterOrEqual(KeyRef(req.arena, prefix));
|
||||
req.end = firstGreaterOrEqual(strinc(prefix, req.arena));
|
||||
req.version = version;
|
||||
|
||||
data->actors.add(data->readGuard(req, getKeyValuesQ));
|
||||
GetKeyValuesReply reply = wait(req.reply.getFuture());
|
||||
++data->counters.quickGetKeyValuesHit;
|
||||
|
||||
// Convert GetKeyValuesReply to RangeResult.
|
||||
return RangeResult(RangeResultRef(reply.data, reply.more), reply.arena);
|
||||
} catch (Error& e) {
|
||||
// Fallback.
|
||||
}
|
||||
|
||||
++data->counters.quickGetKeyValuesMiss;
|
||||
if (SERVER_KNOBS->QUICK_GET_KEY_VALUES_FALLBACK) {
|
||||
state Transaction tr(data->cx);
|
||||
tr.setVersion(version);
|
||||
// TODO: is DefaultPromiseEndpoint the best priority for this?
|
||||
tr.info.taskID = TaskPriority::DefaultPromiseEndpoint;
|
||||
Future<RangeResult> rangeResultFuture = tr.getRange(prefixRange(prefix), Snapshot::True);
|
||||
// TODO: async in case it needs to read from other servers.
|
||||
RangeResult rangeResult = wait(rangeResultFuture);
|
||||
return rangeResult;
|
||||
} else {
|
||||
throw quick_get_key_values_miss();
|
||||
}
|
||||
};
|
||||
|
||||
Key constructMappedKey(KeyValueRef* keyValue, Tuple& mappedKeyFormatTuple, bool& isRangeQuery) {
|
||||
// Lazily parse key and/or value to tuple because they may not need to be a tuple if not used.
|
||||
Optional<Tuple> keyTuple;
|
||||
Optional<Tuple> valueTuple;
|
||||
|
||||
Tuple mappedKeyTuple;
|
||||
for (int i = 0; i < mappedKeyFormatTuple.size(); i++) {
|
||||
Tuple::ElementType type = mappedKeyFormatTuple.getType(i);
|
||||
if (type == Tuple::BYTES || type == Tuple::UTF8) {
|
||||
std::string s = mappedKeyFormatTuple.getString(i).toString();
|
||||
auto sz = s.size();
|
||||
|
||||
// Handle escape.
|
||||
bool escaped = false;
|
||||
size_t p = 0;
|
||||
while (true) {
|
||||
size_t found = s.find("{{", p);
|
||||
if (found == std::string::npos) {
|
||||
break;
|
||||
}
|
||||
s.replace(found, 2, "{");
|
||||
p += 1;
|
||||
escaped = true;
|
||||
}
|
||||
p = 0;
|
||||
while (true) {
|
||||
size_t found = s.find("}}", p);
|
||||
if (found == std::string::npos) {
|
||||
break;
|
||||
}
|
||||
s.replace(found, 2, "}");
|
||||
p += 1;
|
||||
escaped = true;
|
||||
}
|
||||
if (escaped) {
|
||||
// If the element uses escape, cope the escaped version.
|
||||
mappedKeyTuple.append(s);
|
||||
}
|
||||
// {K[??]} or {V[??]}
|
||||
else if (sz > 5 && s[0] == '{' && (s[1] == 'K' || s[1] == 'V') && s[2] == '[' && s[sz - 2] == ']' &&
|
||||
s[sz - 1] == '}') {
|
||||
int idx;
|
||||
try {
|
||||
idx = std::stoi(s.substr(3, sz - 5));
|
||||
} catch (std::exception& e) {
|
||||
throw mapper_bad_index();
|
||||
}
|
||||
Tuple* referenceTuple;
|
||||
if (s[1] == 'K') {
|
||||
// Use keyTuple as reference.
|
||||
if (!keyTuple.present()) {
|
||||
// May throw exception if the key is not parsable as a tuple.
|
||||
keyTuple = Tuple::unpack(keyValue->key);
|
||||
}
|
||||
referenceTuple = &keyTuple.get();
|
||||
} else if (s[1] == 'V') {
|
||||
// Use valueTuple as reference.
|
||||
if (!valueTuple.present()) {
|
||||
// May throw exception if the value is not parsable as a tuple.
|
||||
valueTuple = Tuple::unpack(keyValue->value);
|
||||
}
|
||||
referenceTuple = &valueTuple.get();
|
||||
} else {
|
||||
ASSERT(false);
|
||||
throw internal_error();
|
||||
}
|
||||
|
||||
if (idx < 0 || idx >= referenceTuple->size()) {
|
||||
throw mapper_bad_index();
|
||||
}
|
||||
mappedKeyTuple.append(referenceTuple->subTuple(idx, idx + 1));
|
||||
} else if (s == "{...}") {
|
||||
// Range query.
|
||||
if (i != mappedKeyFormatTuple.size() - 1) {
|
||||
// It must be the last element of the mapper tuple
|
||||
throw mapper_bad_range_decriptor();
|
||||
}
|
||||
// Every record will try to set it. It's ugly, but not wrong.
|
||||
isRangeQuery = true;
|
||||
// Do not add it to the mapped key.
|
||||
} else {
|
||||
// If the element is a string but neither escaped nor descriptors, just copy it.
|
||||
mappedKeyTuple.append(mappedKeyFormatTuple.subTuple(i, i + 1));
|
||||
}
|
||||
} else {
|
||||
// If the element not a string, just copy it.
|
||||
mappedKeyTuple.append(mappedKeyFormatTuple.subTuple(i, i + 1));
|
||||
}
|
||||
}
|
||||
return mappedKeyTuple.getDataAsStandalone();
|
||||
}
|
||||
|
||||
TEST_CASE("/fdbserver/storageserver/constructMappedKey") {
|
||||
Key key = Tuple().append("key-0"_sr).append("key-1"_sr).append("key-2"_sr).getDataAsStandalone();
|
||||
Value value = Tuple().append("value-0"_sr).append("value-1"_sr).append("value-2"_sr).getDataAsStandalone();
|
||||
state KeyValueRef kvr(key, value);
|
||||
{
|
||||
Tuple mapperTuple = Tuple()
|
||||
.append("normal"_sr)
|
||||
.append("{{escaped}}"_sr)
|
||||
.append("{K[2]}"_sr)
|
||||
.append("{V[0]}"_sr)
|
||||
.append("{...}"_sr);
|
||||
|
||||
bool isRangeQuery = false;
|
||||
Key mappedKey = constructMappedKey(&kvr, mapperTuple, isRangeQuery);
|
||||
|
||||
Key expectedMappedKey = Tuple()
|
||||
.append("normal"_sr)
|
||||
.append("{escaped}"_sr)
|
||||
.append("key-2"_sr)
|
||||
.append("value-0"_sr)
|
||||
.getDataAsStandalone();
|
||||
// std::cout << printable(mappedKey) << " == " << printable(expectedMappedKey) << std::endl;
|
||||
ASSERT(mappedKey.compare(expectedMappedKey) == 0);
|
||||
ASSERT(isRangeQuery == true);
|
||||
}
|
||||
{
|
||||
Tuple mapperTuple = Tuple().append("{{{{}}"_sr).append("}}"_sr);
|
||||
|
||||
bool isRangeQuery = false;
|
||||
Key mappedKey = constructMappedKey(&kvr, mapperTuple, isRangeQuery);
|
||||
|
||||
Key expectedMappedKey = Tuple().append("{{}"_sr).append("}"_sr).getDataAsStandalone();
|
||||
// std::cout << printable(mappedKey) << " == " << printable(expectedMappedKey) << std::endl;
|
||||
ASSERT(mappedKey.compare(expectedMappedKey) == 0);
|
||||
ASSERT(isRangeQuery == false);
|
||||
}
|
||||
{
|
||||
Tuple mapperTuple = Tuple().append("{{{{}}"_sr).append("}}"_sr);
|
||||
|
||||
bool isRangeQuery = false;
|
||||
Key mappedKey = constructMappedKey(&kvr, mapperTuple, isRangeQuery);
|
||||
|
||||
Key expectedMappedKey = Tuple().append("{{}"_sr).append("}"_sr).getDataAsStandalone();
|
||||
// std::cout << printable(mappedKey) << " == " << printable(expectedMappedKey) << std::endl;
|
||||
ASSERT(mappedKey.compare(expectedMappedKey) == 0);
|
||||
ASSERT(isRangeQuery == false);
|
||||
}
|
||||
{
|
||||
Tuple mapperTuple = Tuple().append("{K[100]}"_sr);
|
||||
bool isRangeQuery = false;
|
||||
state bool throwException = false;
|
||||
try {
|
||||
Key mappedKey = constructMappedKey(&kvr, mapperTuple, isRangeQuery);
|
||||
} catch (Error& e) {
|
||||
ASSERT(e.code() == error_code_mapper_bad_index);
|
||||
throwException = true;
|
||||
}
|
||||
ASSERT(throwException);
|
||||
}
|
||||
{
|
||||
Tuple mapperTuple = Tuple().append("{...}"_sr).append("last-element"_sr);
|
||||
bool isRangeQuery = false;
|
||||
state bool throwException2 = false;
|
||||
try {
|
||||
Key mappedKey = constructMappedKey(&kvr, mapperTuple, isRangeQuery);
|
||||
} catch (Error& e) {
|
||||
ASSERT(e.code() == error_code_mapper_bad_range_decriptor);
|
||||
throwException2 = true;
|
||||
}
|
||||
ASSERT(throwException2);
|
||||
}
|
||||
{
|
||||
Tuple mapperTuple = Tuple().append("{K[not-a-number]}"_sr);
|
||||
bool isRangeQuery = false;
|
||||
state bool throwException3 = false;
|
||||
try {
|
||||
Key mappedKey = constructMappedKey(&kvr, mapperTuple, isRangeQuery);
|
||||
} catch (Error& e) {
|
||||
ASSERT(e.code() == error_code_mapper_bad_index);
|
||||
throwException3 = true;
|
||||
}
|
||||
ASSERT(throwException3);
|
||||
}
|
||||
return Void();
|
||||
}
|
||||
|
||||
ACTOR Future<GetKeyValuesAndFlatMapReply> flatMap(StorageServer* data, GetKeyValuesReply input, StringRef mapper) {
|
||||
state GetKeyValuesAndFlatMapReply result;
|
||||
result.version = input.version;
|
||||
result.more = input.more;
|
||||
result.cached = input.cached;
|
||||
result.arena.dependsOn(input.arena);
|
||||
|
||||
result.data.reserve(result.arena, input.data.size());
|
||||
state bool isRangeQuery = false;
|
||||
state Tuple mappedKeyFormatTuple = Tuple::unpack(mapper);
|
||||
state KeyValueRef* it = input.data.begin();
|
||||
for (; it != input.data.end(); it++) {
|
||||
state StringRef key = it->key;
|
||||
|
||||
state Key mappedKey = constructMappedKey(it, mappedKeyFormatTuple, isRangeQuery);
|
||||
// Make sure the mappedKey is always available, so that it's good even we want to get key asynchronously.
|
||||
result.arena.dependsOn(mappedKey.arena());
|
||||
|
||||
if (isRangeQuery) {
|
||||
// Use the mappedKey as the prefix of the range query.
|
||||
RangeResult rangeResult = wait(quickGetKeyValues(data, mappedKey, input.version));
|
||||
|
||||
if (rangeResult.more) {
|
||||
// Probably the fan out is too large. The user should use the old way to query.
|
||||
throw quick_get_key_values_has_more();
|
||||
}
|
||||
result.arena.dependsOn(rangeResult.arena());
|
||||
for (int i = 0; i < rangeResult.size(); i++) {
|
||||
result.data.emplace_back(result.arena, rangeResult[i].key, rangeResult[i].value);
|
||||
}
|
||||
} else {
|
||||
Optional<Value> valueOption = wait(quickGetValue(data, mappedKey, input.version));
|
||||
|
||||
if (valueOption.present()) {
|
||||
Value value = valueOption.get();
|
||||
result.arena.dependsOn(value.arena());
|
||||
result.data.emplace_back(result.arena, mappedKey, value);
|
||||
} else {
|
||||
// TODO: Shall we throw exception if the key doesn't exist or the range is empty?
|
||||
}
|
||||
}
|
||||
}
|
||||
return result;
|
||||
}
|
||||
|
||||
// Most of the actor is copied from getKeyValuesQ. I tried to use templates but things become nearly impossible after
|
||||
// combining actor shenanigans with template shenanigans.
|
||||
ACTOR Future<Void> getKeyValuesAndFlatMapQ(StorageServer* data, GetKeyValuesAndFlatMapRequest req)
|
||||
// Throws a wrong_shard_server if the keys in the request or result depend on data outside this server OR if a large
|
||||
// selector offset prevents all data from being read in one range read
|
||||
{
|
||||
state Span span("SS:getKeyValuesAndFlatMap"_loc, { req.spanContext });
|
||||
state int64_t resultSize = 0;
|
||||
state IKeyValueStore::ReadType type =
|
||||
req.isFetchKeys ? IKeyValueStore::ReadType::FETCH : IKeyValueStore::ReadType::NORMAL;
|
||||
getCurrentLineage()->modify(&TransactionLineage::txID) = req.spanContext.first();
|
||||
|
||||
++data->counters.getRangeAndFlatMapQueries;
|
||||
++data->counters.allQueries;
|
||||
++data->readQueueSizeMetric;
|
||||
data->maxQueryQueue = std::max<int>(
|
||||
data->maxQueryQueue, data->counters.allQueries.getValue() - data->counters.finishedQueries.getValue());
|
||||
|
||||
// Active load balancing runs at a very high priority (to obtain accurate queue lengths)
|
||||
// so we need to downgrade here
|
||||
if (SERVER_KNOBS->FETCH_KEYS_LOWER_PRIORITY && req.isFetchKeys) {
|
||||
wait(delay(0, TaskPriority::FetchKeys));
|
||||
} else {
|
||||
wait(data->getQueryDelay());
|
||||
}
|
||||
|
||||
try {
|
||||
if (req.debugID.present())
|
||||
g_traceBatch.addEvent(
|
||||
"TransactionDebug", req.debugID.get().first(), "storageserver.getKeyValuesAndFlatMap.Before");
|
||||
state Version version = wait(waitForVersion(data, req.version, span.context));
|
||||
|
||||
state uint64_t changeCounter = data->shardChangeCounter;
|
||||
// try {
|
||||
state KeyRange shard = getShardKeyRange(data, req.begin);
|
||||
|
||||
if (req.debugID.present())
|
||||
g_traceBatch.addEvent(
|
||||
"TransactionDebug", req.debugID.get().first(), "storageserver.getKeyValuesAndFlatMap.AfterVersion");
|
||||
//.detail("ShardBegin", shard.begin).detail("ShardEnd", shard.end);
|
||||
//} catch (Error& e) { TraceEvent("WrongShardServer", data->thisServerID).detail("Begin",
|
||||
// req.begin.toString()).detail("End", req.end.toString()).detail("Version", version).detail("Shard",
|
||||
//"None").detail("In", "getKeyValuesAndFlatMap>getShardKeyRange"); throw e; }
|
||||
|
||||
if (!selectorInRange(req.end, shard) && !(req.end.isFirstGreaterOrEqual() && req.end.getKey() == shard.end)) {
|
||||
// TraceEvent("WrongShardServer1", data->thisServerID).detail("Begin",
|
||||
// req.begin.toString()).detail("End", req.end.toString()).detail("Version", version).detail("ShardBegin",
|
||||
// shard.begin).detail("ShardEnd", shard.end).detail("In", "getKeyValuesAndFlatMap>checkShardExtents");
|
||||
throw wrong_shard_server();
|
||||
}
|
||||
|
||||
state int offset1 = 0;
|
||||
state int offset2;
|
||||
state Future<Key> fBegin = req.begin.isFirstGreaterOrEqual()
|
||||
? Future<Key>(req.begin.getKey())
|
||||
: findKey(data, req.begin, version, shard, &offset1, span.context, type);
|
||||
state Future<Key> fEnd = req.end.isFirstGreaterOrEqual()
|
||||
? Future<Key>(req.end.getKey())
|
||||
: findKey(data, req.end, version, shard, &offset2, span.context, type);
|
||||
state Key begin = wait(fBegin);
|
||||
state Key end = wait(fEnd);
|
||||
|
||||
if (req.debugID.present())
|
||||
g_traceBatch.addEvent(
|
||||
"TransactionDebug", req.debugID.get().first(), "storageserver.getKeyValuesAndFlatMap.AfterKeys");
|
||||
//.detail("Off1",offset1).detail("Off2",offset2).detail("ReqBegin",req.begin.getKey()).detail("ReqEnd",req.end.getKey());
|
||||
|
||||
// Offsets of zero indicate begin/end keys in this shard, which obviously means we can answer the query
|
||||
// An end offset of 1 is also OK because the end key is exclusive, so if the first key of the next shard is the
|
||||
// end the last actual key returned must be from this shard. A begin offset of 1 is also OK because then either
|
||||
// begin is past end or equal to end (so the result is definitely empty)
|
||||
if ((offset1 && offset1 != 1) || (offset2 && offset2 != 1)) {
|
||||
TEST(true); // wrong_shard_server due to offset in getKeyValuesAndFlatMapQ
|
||||
// We could detect when offset1 takes us off the beginning of the database or offset2 takes us off the end,
|
||||
// and return a clipped range rather than an error (since that is what the NativeAPI.getRange will do anyway
|
||||
// via its "slow path"), but we would have to add some flags to the response to encode whether we went off
|
||||
// the beginning and the end, since it needs that information.
|
||||
//TraceEvent("WrongShardServer2", data->thisServerID).detail("Begin", req.begin.toString()).detail("End", req.end.toString()).detail("Version", version).detail("ShardBegin", shard.begin).detail("ShardEnd", shard.end).detail("In", "getKeyValuesAndFlatMap>checkOffsets").detail("BeginKey", begin).detail("EndKey", end).detail("BeginOffset", offset1).detail("EndOffset", offset2);
|
||||
throw wrong_shard_server();
|
||||
}
|
||||
|
||||
if (begin >= end) {
|
||||
if (req.debugID.present())
|
||||
g_traceBatch.addEvent(
|
||||
"TransactionDebug", req.debugID.get().first(), "storageserver.getKeyValuesAndFlatMap.Send");
|
||||
//.detail("Begin",begin).detail("End",end);
|
||||
|
||||
GetKeyValuesAndFlatMapReply none;
|
||||
none.version = version;
|
||||
none.more = false;
|
||||
none.penalty = data->getPenalty();
|
||||
|
||||
data->checkChangeCounter(changeCounter,
|
||||
KeyRangeRef(std::min<KeyRef>(req.begin.getKey(), req.end.getKey()),
|
||||
std::max<KeyRef>(req.begin.getKey(), req.end.getKey())));
|
||||
req.reply.send(none);
|
||||
} else {
|
||||
state int remainingLimitBytes = req.limitBytes;
|
||||
|
||||
GetKeyValuesReply getKeyValuesReply = wait(
|
||||
readRange(data, version, KeyRangeRef(begin, end), req.limit, &remainingLimitBytes, span.context, type));
|
||||
|
||||
state GetKeyValuesAndFlatMapReply r;
|
||||
try {
|
||||
// Map the scanned range to another list of keys and look up.
|
||||
GetKeyValuesAndFlatMapReply _r = wait(flatMap(data, getKeyValuesReply, req.mapper));
|
||||
r = _r;
|
||||
} catch (Error& e) {
|
||||
TraceEvent("FlatMapError").error(e);
|
||||
throw;
|
||||
}
|
||||
|
||||
if (req.debugID.present())
|
||||
g_traceBatch.addEvent("TransactionDebug",
|
||||
req.debugID.get().first(),
|
||||
"storageserver.getKeyValuesAndFlatMap.AfterReadRange");
|
||||
//.detail("Begin",begin).detail("End",end).detail("SizeOf",r.data.size());
|
||||
data->checkChangeCounter(
|
||||
changeCounter,
|
||||
KeyRangeRef(std::min<KeyRef>(begin, std::min<KeyRef>(req.begin.getKey(), req.end.getKey())),
|
||||
std::max<KeyRef>(end, std::max<KeyRef>(req.begin.getKey(), req.end.getKey()))));
|
||||
if (EXPENSIVE_VALIDATION) {
|
||||
// TODO: Only mapped keys are returned, which are not supposed to be in the range.
|
||||
// for (int i = 0; i < r.data.size(); i++)
|
||||
// ASSERT(r.data[i].key >= begin && r.data[i].key < end);
|
||||
// TODO: GetKeyValuesWithFlatMapRequest doesn't respect limit yet.
|
||||
// ASSERT(r.data.size() <= std::abs(req.limit));
|
||||
}
|
||||
|
||||
/*for( int i = 0; i < r.data.size(); i++ ) {
|
||||
StorageMetrics m;
|
||||
m.bytesPerKSecond = r.data[i].expectedSize();
|
||||
m.iosPerKSecond = 1; //FIXME: this should be 1/r.data.size(), but we cannot do that because it is an int
|
||||
data->metrics.notify(r.data[i].key, m);
|
||||
}*/
|
||||
|
||||
// For performance concerns, the cost of a range read is billed to the start key and end key of the range.
|
||||
int64_t totalByteSize = 0;
|
||||
for (int i = 0; i < r.data.size(); i++) {
|
||||
totalByteSize += r.data[i].expectedSize();
|
||||
}
|
||||
if (totalByteSize > 0 && SERVER_KNOBS->READ_SAMPLING_ENABLED) {
|
||||
int64_t bytesReadPerKSecond = std::max(totalByteSize, SERVER_KNOBS->EMPTY_READ_PENALTY) / 2;
|
||||
data->metrics.notifyBytesReadPerKSecond(r.data[0].key, bytesReadPerKSecond);
|
||||
data->metrics.notifyBytesReadPerKSecond(r.data[r.data.size() - 1].key, bytesReadPerKSecond);
|
||||
}
|
||||
|
||||
r.penalty = data->getPenalty();
|
||||
req.reply.send(r);
|
||||
|
||||
resultSize = req.limitBytes - remainingLimitBytes;
|
||||
data->counters.bytesQueried += resultSize;
|
||||
data->counters.rowsQueried += r.data.size();
|
||||
if (r.data.size() == 0) {
|
||||
++data->counters.emptyQueries;
|
||||
}
|
||||
}
|
||||
} catch (Error& e) {
|
||||
if (!canReplyWith(e))
|
||||
throw;
|
||||
data->sendErrorWithPenalty(req.reply, e, data->getPenalty());
|
||||
}
|
||||
|
||||
data->transactionTagCounter.addRequest(req.tags, resultSize);
|
||||
++data->counters.finishedQueries;
|
||||
--data->readQueueSizeMetric;
|
||||
|
||||
double duration = g_network->timer() - req.requestTime();
|
||||
data->counters.readLatencySample.addMeasurement(duration);
|
||||
if (data->latencyBandConfig.present()) {
|
||||
int maxReadBytes =
|
||||
data->latencyBandConfig.get().readConfig.maxReadBytes.orDefault(std::numeric_limits<int>::max());
|
||||
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);
|
||||
}
|
||||
|
||||
return Void();
|
||||
}
|
||||
|
||||
ACTOR Future<Void> getKeyValuesStreamQ(StorageServer* data, GetKeyValuesStreamRequest req)
|
||||
// Throws a wrong_shard_server if the keys in the request or result depend on data outside this server OR if a large
|
||||
// selector offset prevents all data from being read in one range read
|
||||
|
@ -3672,7 +4161,7 @@ ACTOR Future<Void> fetchKeys(StorageServer* data, AddingShard* shard) {
|
|||
} catch (Error& e) {
|
||||
if (e.code() != error_code_end_of_stream && e.code() != error_code_connection_failed &&
|
||||
e.code() != error_code_transaction_too_old && e.code() != error_code_future_version &&
|
||||
e.code() != error_code_process_behind) {
|
||||
e.code() != error_code_process_behind && e.code() != error_code_server_overloaded) {
|
||||
throw;
|
||||
}
|
||||
if (nfk == keys.begin) {
|
||||
|
@ -4666,6 +5155,7 @@ ACTOR Future<Void> update(StorageServer* data, bool* pReceivedUpdate) {
|
|||
data->logProtocol = rd.protocolVersion();
|
||||
data->storage.changeLogProtocol(ver, data->logProtocol);
|
||||
cloneCursor2->setProtocolVersion(rd.protocolVersion());
|
||||
spanContext = UID();
|
||||
} else if (rd.protocolVersion().hasSpanContext() && SpanContextMessage::isNextIn(rd)) {
|
||||
SpanContextMessage scm;
|
||||
rd >> scm;
|
||||
|
@ -5743,6 +6233,20 @@ ACTOR Future<Void> serveGetKeyValuesRequests(StorageServer* self, FutureStream<G
|
|||
}
|
||||
}
|
||||
|
||||
ACTOR Future<Void> serveGetKeyValuesAndFlatMapRequests(
|
||||
StorageServer* self,
|
||||
FutureStream<GetKeyValuesAndFlatMapRequest> getKeyValuesAndFlatMap) {
|
||||
// TODO: Is it fine to keep TransactionLineage::Operation::GetKeyValues here?
|
||||
getCurrentLineage()->modify(&TransactionLineage::operation) = TransactionLineage::Operation::GetKeyValues;
|
||||
loop {
|
||||
GetKeyValuesAndFlatMapRequest req = waitNext(getKeyValuesAndFlatMap);
|
||||
|
||||
// Warning: This code is executed at extremely high priority (TaskPriority::LoadBalancedEndpoint), so downgrade
|
||||
// before doing real work
|
||||
self->actors.add(self->readGuard(req, getKeyValuesAndFlatMapQ));
|
||||
}
|
||||
}
|
||||
|
||||
ACTOR Future<Void> serveGetKeyValuesStreamRequests(StorageServer* self,
|
||||
FutureStream<GetKeyValuesStreamRequest> getKeyValuesStream) {
|
||||
loop {
|
||||
|
@ -5951,6 +6455,7 @@ ACTOR Future<Void> storageServerCore(StorageServer* self, StorageServerInterface
|
|||
self->actors.add(checkBehind(self));
|
||||
self->actors.add(serveGetValueRequests(self, ssi.getValue.getFuture()));
|
||||
self->actors.add(serveGetKeyValuesRequests(self, ssi.getKeyValues.getFuture()));
|
||||
self->actors.add(serveGetKeyValuesAndFlatMapRequests(self, ssi.getKeyValuesAndFlatMap.getFuture()));
|
||||
self->actors.add(serveGetKeyValuesStreamRequests(self, ssi.getKeyValuesStream.getFuture()));
|
||||
self->actors.add(serveGetKeyRequests(self, ssi.getKey.getFuture()));
|
||||
self->actors.add(serveWatchValueRequests(self, ssi.watchValue.getFuture()));
|
||||
|
|
|
@ -1439,7 +1439,7 @@ ACTOR Future<Void> runTests(Reference<AsyncVar<Optional<struct ClusterController
|
|||
cx = openDBOnServer(dbInfo);
|
||||
}
|
||||
|
||||
state Future<Void> disabler = disableConnectionFailuresAfter(450, "Tester");
|
||||
state Future<Void> disabler = disableConnectionFailuresAfter(FLOW_KNOBS->SIM_SPEEDUP_AFTER_SECONDS, "Tester");
|
||||
|
||||
// Change the configuration (and/or create the database) if necessary
|
||||
printf("startingConfiguration:%s start\n", startingConfiguration.toString().c_str());
|
||||
|
|
|
@ -1097,6 +1097,7 @@ ACTOR Future<Void> storageServerRollbackRebooter(std::set<std::pair<UID, KeyValu
|
|||
DUMPTOKEN(recruited.getValue);
|
||||
DUMPTOKEN(recruited.getKey);
|
||||
DUMPTOKEN(recruited.getKeyValues);
|
||||
DUMPTOKEN(recruited.getKeyValuesAndFlatMap);
|
||||
DUMPTOKEN(recruited.getShardState);
|
||||
DUMPTOKEN(recruited.waitMetrics);
|
||||
DUMPTOKEN(recruited.splitMetrics);
|
||||
|
@ -1108,6 +1109,7 @@ ACTOR Future<Void> storageServerRollbackRebooter(std::set<std::pair<UID, KeyValu
|
|||
DUMPTOKEN(recruited.getKeyValueStoreType);
|
||||
DUMPTOKEN(recruited.watchValue);
|
||||
DUMPTOKEN(recruited.getKeyValuesStream);
|
||||
DUMPTOKEN(recruited.getKeyValuesAndFlatMap);
|
||||
|
||||
prevStorageServer =
|
||||
storageServer(store, recruited, db, folder, Promise<Void>(), Reference<IClusterConnectionRecord>(nullptr));
|
||||
|
@ -1478,6 +1480,7 @@ ACTOR Future<Void> workerServer(Reference<IClusterConnectionRecord> connRecord,
|
|||
DUMPTOKEN(recruited.getKeyValueStoreType);
|
||||
DUMPTOKEN(recruited.watchValue);
|
||||
DUMPTOKEN(recruited.getKeyValuesStream);
|
||||
DUMPTOKEN(recruited.getKeyValuesAndFlatMap);
|
||||
|
||||
Promise<Void> recovery;
|
||||
Future<Void> f = storageServer(kv, recruited, dbInfo, folder, recovery, connRecord);
|
||||
|
@ -1574,6 +1577,7 @@ ACTOR Future<Void> workerServer(Reference<IClusterConnectionRecord> connRecord,
|
|||
DUMPTOKEN(recruited.getValue);
|
||||
DUMPTOKEN(recruited.getKey);
|
||||
DUMPTOKEN(recruited.getKeyValues);
|
||||
DUMPTOKEN(recruited.getKeyValuesAndFlatMap);
|
||||
DUMPTOKEN(recruited.getShardState);
|
||||
DUMPTOKEN(recruited.waitMetrics);
|
||||
DUMPTOKEN(recruited.splitMetrics);
|
||||
|
@ -1931,6 +1935,7 @@ ACTOR Future<Void> workerServer(Reference<IClusterConnectionRecord> connRecord,
|
|||
DUMPTOKEN(recruited.getKeyValueStoreType);
|
||||
DUMPTOKEN(recruited.watchValue);
|
||||
DUMPTOKEN(recruited.getKeyValuesStream);
|
||||
DUMPTOKEN(recruited.getKeyValuesAndFlatMap);
|
||||
// printf("Recruited as storageServer\n");
|
||||
|
||||
std::string filename =
|
||||
|
|
|
@ -107,6 +107,9 @@ struct ClientLibManagementWorkload : public TestWorkload {
|
|||
wait(testClientLibListAfterUpload(self, cx));
|
||||
wait(testDownloadClientLib(self, cx));
|
||||
wait(testClientLibDownloadNotExisting(self, cx));
|
||||
wait(testChangeClientLibStatusErrors(self, cx));
|
||||
wait(testDisableClientLib(self, cx));
|
||||
wait(testChangeStateToDownload(self, cx));
|
||||
wait(testDeleteClientLib(self, cx));
|
||||
wait(testUploadedClientLibInList(self, cx, ClientLibFilter(), false, "No filter, after delete"));
|
||||
return Void();
|
||||
|
@ -176,10 +179,13 @@ struct ClientLibManagementWorkload : public TestWorkload {
|
|||
ACTOR static Future<Void> testUploadClientLib(ClientLibManagementWorkload* self, Database cx) {
|
||||
state Standalone<StringRef> metadataStr;
|
||||
state std::vector<Future<ErrorOr<Void>>> concurrentUploads;
|
||||
state Future<Void> clientLibChanged = cx->onClientLibStatusChanged();
|
||||
|
||||
validClientLibMetadataSample(self->uploadedMetadataJson);
|
||||
self->uploadedMetadataJson[CLIENTLIB_ATTR_CHECKSUM] = self->generatedChecksum.toString();
|
||||
// avoid clientLibId clashes, when multiple clients try to upload the same file
|
||||
self->uploadedMetadataJson[CLIENTLIB_ATTR_TYPE] = format("devbuild%d", self->clientId);
|
||||
self->uploadedMetadataJson[CLIENTLIB_ATTR_STATUS] = getStatusName(ClientLibStatus::ACTIVE);
|
||||
metadataStr = StringRef(json_spirit::write_string(json_spirit::mValue(self->uploadedMetadataJson)));
|
||||
self->uploadedClientLibId = getClientLibIdFromMetadataJson(metadataStr);
|
||||
|
||||
|
@ -209,6 +215,14 @@ struct ClientLibManagementWorkload : public TestWorkload {
|
|||
TraceEvent(SevError, "ClientLibConflictingUpload").log();
|
||||
self->success = false;
|
||||
}
|
||||
|
||||
// Clients should be notified about upload of a library with the active status
|
||||
Optional<Void> notificationWait = wait(timeout(clientLibChanged, 100.0));
|
||||
if (!notificationWait.present()) {
|
||||
TraceEvent(SevError, "ClientLibChangeNotificationFailed").log();
|
||||
self->success = false;
|
||||
}
|
||||
|
||||
return Void();
|
||||
}
|
||||
|
||||
|
@ -253,7 +267,15 @@ struct ClientLibManagementWorkload : public TestWorkload {
|
|||
}
|
||||
|
||||
ACTOR static Future<Void> testDeleteClientLib(ClientLibManagementWorkload* self, Database cx) {
|
||||
state Future<Void> clientLibChanged = cx->onClientLibStatusChanged();
|
||||
|
||||
wait(deleteClientLibrary(cx, self->uploadedClientLibId));
|
||||
|
||||
// Clients should be notified about deletion of the library, because it has "download" status
|
||||
Optional<Void> notificationWait = wait(timeout(clientLibChanged, 100.0));
|
||||
if (!notificationWait.present()) {
|
||||
TraceEvent(SevError, "ClientLibChangeNotificationFailed").log();
|
||||
}
|
||||
return Void();
|
||||
}
|
||||
|
||||
|
@ -321,6 +343,74 @@ struct ClientLibManagementWorkload : public TestWorkload {
|
|||
return Void();
|
||||
}
|
||||
|
||||
ACTOR static Future<Void> testChangeClientLibStatusErrors(ClientLibManagementWorkload* self, Database cx) {
|
||||
wait(testExpectedError(changeClientLibraryStatus(cx, self->uploadedClientLibId, ClientLibStatus::UPLOADING),
|
||||
"Setting invalid client library status",
|
||||
client_lib_invalid_metadata(),
|
||||
&self->success));
|
||||
|
||||
wait(testExpectedError(changeClientLibraryStatus(cx, "notExistingClientLib"_sr, ClientLibStatus::DOWNLOAD),
|
||||
"Changing not existing client library status",
|
||||
client_lib_not_found(),
|
||||
&self->success));
|
||||
return Void();
|
||||
}
|
||||
|
||||
ACTOR static Future<Void> testDisableClientLib(ClientLibManagementWorkload* self, Database cx) {
|
||||
state std::string destFileName = format("clientLibDownload%d", self->clientId);
|
||||
state Future<Void> clientLibChanged = cx->onClientLibStatusChanged();
|
||||
|
||||
// Set disabled status on the uploaded library
|
||||
wait(changeClientLibraryStatus(cx, self->uploadedClientLibId, ClientLibStatus::DISABLED));
|
||||
state ClientLibStatus newStatus = wait(getClientLibraryStatus(cx, self->uploadedClientLibId));
|
||||
if (newStatus != ClientLibStatus::DISABLED) {
|
||||
TraceEvent(SevError, "ClientLibDisableClientLibFailed")
|
||||
.detail("Reason", "Unexpected status")
|
||||
.detail("Expected", ClientLibStatus::DISABLED)
|
||||
.detail("Actual", newStatus);
|
||||
self->success = false;
|
||||
}
|
||||
|
||||
// Clients should be notified about an active library being disabled
|
||||
Optional<Void> notificationWait = wait(timeout(clientLibChanged, 100.0));
|
||||
if (!notificationWait.present()) {
|
||||
TraceEvent(SevError, "ClientLibChangeNotificationFailed").log();
|
||||
self->success = false;
|
||||
}
|
||||
|
||||
// It should not be possible to download a disabled client library
|
||||
wait(testExpectedError(downloadClientLibrary(cx, self->uploadedClientLibId, StringRef(destFileName)),
|
||||
"Downloading disabled client library",
|
||||
client_lib_not_available(),
|
||||
&self->success));
|
||||
|
||||
return Void();
|
||||
}
|
||||
|
||||
ACTOR static Future<Void> testChangeStateToDownload(ClientLibManagementWorkload* self, Database cx) {
|
||||
state std::string destFileName = format("clientLibDownload%d", self->clientId);
|
||||
state Future<Void> clientLibChanged = cx->onClientLibStatusChanged();
|
||||
|
||||
// Set disabled status on the uploaded library
|
||||
wait(changeClientLibraryStatus(cx, self->uploadedClientLibId, ClientLibStatus::DOWNLOAD));
|
||||
state ClientLibStatus newStatus = wait(getClientLibraryStatus(cx, self->uploadedClientLibId));
|
||||
if (newStatus != ClientLibStatus::DOWNLOAD) {
|
||||
TraceEvent(SevError, "ClientLibChangeStatusFailed")
|
||||
.detail("Reason", "Unexpected status")
|
||||
.detail("Expected", ClientLibStatus::DOWNLOAD)
|
||||
.detail("Actual", newStatus);
|
||||
self->success = false;
|
||||
}
|
||||
|
||||
Optional<Void> notificationWait = wait(timeout(clientLibChanged, 100.0));
|
||||
if (!notificationWait.present()) {
|
||||
TraceEvent(SevError, "ClientLibChangeNotificationFailed").log();
|
||||
self->success = false;
|
||||
}
|
||||
|
||||
return Void();
|
||||
}
|
||||
|
||||
/* ----------------------------------------------------------------
|
||||
* Utility methods
|
||||
*/
|
||||
|
@ -345,7 +435,7 @@ struct ClientLibManagementWorkload : public TestWorkload {
|
|||
metadataJson[CLIENTLIB_ATTR_GIT_HASH] = randomHexadecimalStr(40);
|
||||
metadataJson[CLIENTLIB_ATTR_TYPE] = "debug";
|
||||
metadataJson[CLIENTLIB_ATTR_CHECKSUM] = randomHexadecimalStr(32);
|
||||
metadataJson[CLIENTLIB_ATTR_STATUS] = getStatusName(ClientLibStatus::AVAILABLE);
|
||||
metadataJson[CLIENTLIB_ATTR_STATUS] = getStatusName(ClientLibStatus::DOWNLOAD);
|
||||
metadataJson[CLIENTLIB_ATTR_API_VERSION] = 710;
|
||||
metadataJson[CLIENTLIB_ATTR_PROTOCOL] = "fdb00b07001001";
|
||||
metadataJson[CLIENTLIB_ATTR_CHECKSUM_ALG] = "md5";
|
||||
|
|
|
@ -0,0 +1,160 @@
|
|||
/*
|
||||
* IndexPrefetchDemo.actor.cpp
|
||||
*
|
||||
* This source file is part of the FoundationDB open source project
|
||||
*
|
||||
* Copyright 2013-2018 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 <cstdint>
|
||||
#include <limits>
|
||||
#include "fdbrpc/simulator.h"
|
||||
#include "fdbclient/MutationLogReader.actor.h"
|
||||
#include "fdbclient/Tuple.h"
|
||||
#include "fdbserver/workloads/workloads.actor.h"
|
||||
#include "flow/Error.h"
|
||||
#include "flow/IRandom.h"
|
||||
#include "flow/flow.h"
|
||||
#include "flow/actorcompiler.h" // This must be the last #include.
|
||||
|
||||
const KeyRef prefix = "prefix"_sr;
|
||||
const KeyRef RECORD = "RECORD"_sr;
|
||||
const KeyRef INDEX = "INDEX"_sr;
|
||||
|
||||
struct IndexPrefetchDemoWorkload : TestWorkload {
|
||||
bool enabled;
|
||||
const bool BAD_MAPPER = deterministicRandom()->random01() < 0.1;
|
||||
|
||||
IndexPrefetchDemoWorkload(WorkloadContext const& wcx) : TestWorkload(wcx) {
|
||||
enabled = !clientId; // only do this on the "first" client
|
||||
}
|
||||
|
||||
std::string description() const override { return "IndexPrefetchDemo"; }
|
||||
|
||||
Future<Void> start(Database const& cx) override {
|
||||
if (enabled) {
|
||||
return _start(cx, this);
|
||||
}
|
||||
return Void();
|
||||
}
|
||||
|
||||
static Key primaryKey(int i) { return KeyRef("primary-key-of-record-" + std::to_string(i)); }
|
||||
static Key indexKey(int i) { return KeyRef("index-key-of-record-" + std::to_string(i)); }
|
||||
static Key data(int i) { return KeyRef("data-of-record-" + std::to_string(i)); }
|
||||
|
||||
ACTOR Future<Void> fillInRecords(Database cx, int n) {
|
||||
std::cout << "start fillInRecords n=" << n << std::endl;
|
||||
// TODO: When n is large, split into multiple transactions.
|
||||
state Transaction tr(cx);
|
||||
try {
|
||||
tr.reset();
|
||||
for (int i = 0; i < n; i++) {
|
||||
tr.set(Tuple().append(prefix).append(RECORD).append(primaryKey(i)).pack(),
|
||||
Tuple().append(data(i)).pack());
|
||||
tr.set(Tuple().append(prefix).append(INDEX).append(indexKey(i)).append(primaryKey(i)).pack(),
|
||||
Tuple().pack());
|
||||
}
|
||||
wait(tr.commit());
|
||||
std::cout << "finished fillInRecords" << std::endl;
|
||||
} catch (Error& e) {
|
||||
std::cout << "failed fillInRecords" << std::endl;
|
||||
wait(tr.onError(e));
|
||||
}
|
||||
return Void();
|
||||
}
|
||||
|
||||
static void showResult(const RangeResult& result) {
|
||||
std::cout << "result size: " << result.size() << std::endl;
|
||||
const KeyValueRef* it = result.begin();
|
||||
for (; it != result.end(); it++) {
|
||||
std::cout << "key=" << it->key.printable() << ", value=" << it->value.printable() << std::endl;
|
||||
}
|
||||
}
|
||||
|
||||
ACTOR Future<Void> scanRange(Database cx, KeyRangeRef range) {
|
||||
std::cout << "start scanRange " << range.toString() << std::endl;
|
||||
// TODO: When n is large, split into multiple transactions.
|
||||
state Transaction tr(cx);
|
||||
try {
|
||||
tr.reset();
|
||||
RangeResult result = wait(tr.getRange(range, CLIENT_KNOBS->TOO_MANY));
|
||||
showResult(result);
|
||||
} catch (Error& e) {
|
||||
wait(tr.onError(e));
|
||||
}
|
||||
std::cout << "finished scanRange" << std::endl;
|
||||
return Void();
|
||||
}
|
||||
|
||||
ACTOR Future<Void> scanRangeAndFlatMap(Database cx, KeyRange range, Key mapper, IndexPrefetchDemoWorkload* self) {
|
||||
std::cout << "start scanRangeAndFlatMap " << range.toString() << std::endl;
|
||||
// TODO: When n is large, split into multiple transactions.
|
||||
state Transaction tr(cx);
|
||||
try {
|
||||
tr.reset();
|
||||
RangeResult result =
|
||||
wait(tr.getRangeAndFlatMap(KeySelector(firstGreaterOrEqual(range.begin), range.arena()),
|
||||
KeySelector(firstGreaterOrEqual(range.end), range.arena()),
|
||||
mapper,
|
||||
GetRangeLimits(CLIENT_KNOBS->TOO_MANY),
|
||||
Snapshot::True));
|
||||
showResult(result);
|
||||
if (self->BAD_MAPPER) {
|
||||
TraceEvent("IndexPrefetchDemoWorkloadShouldNotReachable").detail("ResultSize", result.size());
|
||||
}
|
||||
// result size: 2
|
||||
// key=\x01prefix\x00\x01RECORD\x00\x01primary-key-of-record-2\x00, value=\x01data-of-record-2\x00
|
||||
// key=\x01prefix\x00\x01RECORD\x00\x01primary-key-of-record-3\x00, value=\x01data-of-record-3\x00
|
||||
} catch (Error& e) {
|
||||
if (self->BAD_MAPPER && e.code() == error_code_mapper_bad_index) {
|
||||
TraceEvent("IndexPrefetchDemoWorkloadBadMapperDetected").error(e);
|
||||
} else {
|
||||
wait(tr.onError(e));
|
||||
}
|
||||
}
|
||||
std::cout << "finished scanRangeAndFlatMap" << std::endl;
|
||||
return Void();
|
||||
}
|
||||
|
||||
ACTOR Future<Void> _start(Database cx, IndexPrefetchDemoWorkload* self) {
|
||||
TraceEvent("IndexPrefetchDemoWorkloadConfig").detail("BadMapper", self->BAD_MAPPER);
|
||||
|
||||
// TODO: Use toml to config
|
||||
wait(self->fillInRecords(cx, 5));
|
||||
|
||||
wait(self->scanRange(cx, normalKeys));
|
||||
|
||||
Key someIndexesBegin = Tuple().append(prefix).append(INDEX).append(indexKey(2)).getDataAsStandalone();
|
||||
Key someIndexesEnd = Tuple().append(prefix).append(INDEX).append(indexKey(4)).getDataAsStandalone();
|
||||
state KeyRange someIndexes = KeyRangeRef(someIndexesBegin, someIndexesEnd);
|
||||
wait(self->scanRange(cx, someIndexes));
|
||||
|
||||
Tuple mapperTuple;
|
||||
if (self->BAD_MAPPER) {
|
||||
mapperTuple << prefix << RECORD << "{K[xxx]}"_sr;
|
||||
} else {
|
||||
mapperTuple << prefix << RECORD << "{K[3]}"_sr;
|
||||
}
|
||||
Key mapper = mapperTuple.getDataAsStandalone();
|
||||
wait(self->scanRangeAndFlatMap(cx, someIndexes, mapper, self));
|
||||
return Void();
|
||||
}
|
||||
|
||||
Future<bool> check(Database const& cx) override { return true; }
|
||||
|
||||
void getMetrics(std::vector<PerfMetric>& m) override {}
|
||||
};
|
||||
|
||||
WorkloadFactory<IndexPrefetchDemoWorkload> IndexPrefetchDemoWorkloadFactory("IndexPrefetchDemo");
|
|
@ -68,7 +68,7 @@ void FlowKnobs::initialize(Randomize randomize, IsSimulated isSimulated) {
|
|||
init( HUGE_ARENA_LOGGING_INTERVAL, 5.0 );
|
||||
|
||||
init( WRITE_TRACING_ENABLED, true ); if( randomize && BUGGIFY ) WRITE_TRACING_ENABLED = false;
|
||||
init( TRACING_SAMPLE_RATE, 1.0 ); // Fraction of traces (not spans) to sample (0 means ignore all traces)
|
||||
init( TRACING_SAMPLE_RATE, 0.0 ); // Fraction of distributed traces (not spans) to sample (0 means ignore all traces)
|
||||
init( TRACING_UDP_LISTENER_PORT, 8889 ); // Only applicable if TracerType is set to a network option
|
||||
|
||||
//connectionMonitor
|
||||
|
@ -203,6 +203,7 @@ void FlowKnobs::initialize(Randomize randomize, IsSimulated isSimulated) {
|
|||
init( MAX_TRACE_FIELD_LENGTH, 495 ); // If the value of this is changed, the corresponding default in Trace.cpp should be changed as well
|
||||
init( MAX_TRACE_EVENT_LENGTH, 4000 ); // If the value of this is changed, the corresponding default in Trace.cpp should be changed as well
|
||||
init( ALLOCATION_TRACING_ENABLED, true );
|
||||
init( SIM_SPEEDUP_AFTER_SECONDS, 450 );
|
||||
|
||||
//TDMetrics
|
||||
init( MAX_METRICS, 600 );
|
||||
|
|
|
@ -245,6 +245,7 @@ public:
|
|||
double MAX_CLOGGING_LATENCY;
|
||||
double MAX_BUGGIFIED_DELAY;
|
||||
int SIM_CONNECT_ERROR_MODE;
|
||||
double SIM_SPEEDUP_AFTER_SECONDS;
|
||||
|
||||
// Tracefiles
|
||||
int ZERO_LENGTH_FILE_PAD;
|
||||
|
|
|
@ -613,6 +613,7 @@ inline static void flushOutputStreams() {
|
|||
#if defined(_MSC_VER)
|
||||
#define DLLEXPORT __declspec(dllexport)
|
||||
#elif defined(__GNUG__)
|
||||
#undef DLLEXPORT
|
||||
#define DLLEXPORT __attribute__((visibility("default")))
|
||||
#else
|
||||
#error Missing symbol export
|
||||
|
|
|
@ -139,9 +139,9 @@ public: // introduced features
|
|||
PROTOCOL_VERSION_FEATURE(0x0FDB00B070010001LL, TagThrottleValueReason);
|
||||
PROTOCOL_VERSION_FEATURE(0x0FDB00B070010001LL, SpanContext);
|
||||
PROTOCOL_VERSION_FEATURE(0x0FDB00B070010001LL, TSS);
|
||||
PROTOCOL_VERSION_FEATURE(0x0FDB00B070010001LL, ChangeFeed); // FIXME: Change to 7.1 once we cut release
|
||||
PROTOCOL_VERSION_FEATURE(0x0FDB00B070010001LL, BlobGranule); // FIXME: Change to 7.1 once we cut release
|
||||
PROTOCOL_VERSION_FEATURE(0x0FDB00B070010001LL, NetworkAddressHostnameFlag);
|
||||
PROTOCOL_VERSION_FEATURE(0x0FDB00B071010001LL, ChangeFeed);
|
||||
PROTOCOL_VERSION_FEATURE(0x0FDB00B071010001LL, BlobGranule);
|
||||
PROTOCOL_VERSION_FEATURE(0x0FDB00B071010001LL, NetworkAddressHostnameFlag);
|
||||
};
|
||||
|
||||
template <>
|
||||
|
@ -158,10 +158,10 @@ struct Traceable<ProtocolVersion> : std::true_type {
|
|||
//
|
||||
// xyzdev
|
||||
// vvvv
|
||||
constexpr ProtocolVersion currentProtocolVersion(0x0FDB00B070010001LL);
|
||||
constexpr ProtocolVersion currentProtocolVersion(0x0FDB00B071010001LL);
|
||||
// This assert is intended to help prevent incrementing the leftmost digits accidentally. It will probably need to
|
||||
// change when we reach version 10.
|
||||
static_assert(currentProtocolVersion.version() < 0x0FDB00B100000000LL, "Unexpected protocol version");
|
||||
|
||||
// Downgrades are only supported for one minor version
|
||||
constexpr ProtocolVersion minInvalidProtocolVersion(0x0FDB00B072000000LL);
|
||||
constexpr ProtocolVersion minInvalidProtocolVersion(0x0FDB00B073000000LL);
|
||||
|
|
|
@ -38,18 +38,22 @@ struct Span {
|
|||
: context(context), begin(g_network->now()), location(location), parents(arena, parents.begin(), parents.end()) {
|
||||
if (parents.size() > 0) {
|
||||
// If the parents' token is 0 (meaning the trace should not be
|
||||
// recorded), set the child token to 0 as well. Otherwise, use the
|
||||
// existing (likely randomly generated) value.
|
||||
uint64_t traceId = (*parents.begin()).second() > 0 ? context.second() : 0;
|
||||
// recorded), set the child token to 0 as well. Otherwise, generate
|
||||
// a new, random token.
|
||||
uint64_t traceId = 0;
|
||||
if ((*parents.begin()).second() > 0) {
|
||||
traceId = deterministicRandom()->randomUInt64();
|
||||
}
|
||||
this->context = SpanID((*parents.begin()).first(), traceId);
|
||||
}
|
||||
}
|
||||
Span(Location location, std::initializer_list<SpanID> const& parents = {}) {
|
||||
uint64_t tokenId = deterministicRandom()->random01() < FLOW_KNOBS->TRACING_SAMPLE_RATE
|
||||
? deterministicRandom()->randomUInt64()
|
||||
: 0;
|
||||
Span(UID(deterministicRandom()->randomUInt64(), tokenId), location, parents);
|
||||
}
|
||||
Span(Location location, std::initializer_list<SpanID> const& parents = {})
|
||||
: Span(UID(deterministicRandom()->randomUInt64(),
|
||||
deterministicRandom()->random01() < FLOW_KNOBS->TRACING_SAMPLE_RATE
|
||||
? deterministicRandom()->randomUInt64()
|
||||
: 0),
|
||||
location,
|
||||
parents) {}
|
||||
Span(Location location, SpanID context) : Span(location, { context }) {}
|
||||
Span(const Span&) = delete;
|
||||
Span(Span&& o) {
|
||||
|
@ -78,7 +82,10 @@ struct Span {
|
|||
|
||||
void addParent(SpanID span) {
|
||||
if (parents.size() == 0) {
|
||||
uint64_t traceId = (*parents.begin()).second() > 0 ? context.second() : 0;
|
||||
uint64_t traceId = 0;
|
||||
if (span.second() > 0) {
|
||||
traceId = context.second() == 0 ? deterministicRandom()->randomUInt64() : context.second();
|
||||
}
|
||||
// Use first parent to set trace ID. This is non-ideal for spans
|
||||
// with multiple parents, because the trace ID will associate the
|
||||
// span with only one trace. A workaround is to look at the parent
|
||||
|
|
|
@ -160,6 +160,12 @@ ERROR( blocked_from_network_thread, 2026, "Detected a deadlock in a callback cal
|
|||
ERROR( invalid_config_db_range_read, 2027, "Invalid configuration database range read" )
|
||||
ERROR( invalid_config_db_key, 2028, "Invalid configuration database key provided" )
|
||||
ERROR( invalid_config_path, 2029, "Invalid configuration path" )
|
||||
ERROR( mapper_bad_index, 2030, "The index in K[] or V[] is not a valid number or out of range" )
|
||||
ERROR( mapper_no_such_key, 2031, "A mapped key is not set in database" )
|
||||
ERROR( mapper_bad_range_decriptor, 2032, "\"{...}\" must be the last element of the mapper tuple" )
|
||||
ERROR( quick_get_key_values_has_more, 2033, "One of the mapped range queries is too large" )
|
||||
ERROR( quick_get_value_miss, 2034, "Found a mapped key that is not served in the same SS" )
|
||||
ERROR( quick_get_key_values_miss, 2035, "Found a mapped range that is not served in the same SS" )
|
||||
|
||||
ERROR( incompatible_protocol_version, 2100, "Incompatible protocol version" )
|
||||
ERROR( transaction_too_large, 2101, "Transaction exceeds byte limit" )
|
||||
|
|
|
@ -844,53 +844,6 @@ Future<Void> timeoutWarningCollector(FutureStream<Void> const& input,
|
|||
Future<bool> quorumEqualsTrue(std::vector<Future<bool>> const& futures, int const& required);
|
||||
Future<Void> lowPriorityDelay(double const& waitTime);
|
||||
|
||||
ACTOR template <class T>
|
||||
Future<T> ioTimeoutError(Future<T> what, double time) {
|
||||
Future<Void> end = lowPriorityDelay(time);
|
||||
choose {
|
||||
when(T t = wait(what)) { return t; }
|
||||
when(wait(end)) {
|
||||
Error err = io_timeout();
|
||||
if (g_network->isSimulated()) {
|
||||
err = err.asInjectedFault();
|
||||
}
|
||||
TraceEvent(SevError, "IoTimeoutError").error(err);
|
||||
throw err;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
ACTOR template <class T>
|
||||
Future<T> ioDegradedOrTimeoutError(Future<T> what,
|
||||
double errTime,
|
||||
Reference<AsyncVar<bool>> degraded,
|
||||
double degradedTime) {
|
||||
if (degradedTime < errTime) {
|
||||
Future<Void> degradedEnd = lowPriorityDelay(degradedTime);
|
||||
choose {
|
||||
when(T t = wait(what)) { return t; }
|
||||
when(wait(degradedEnd)) {
|
||||
TEST(true); // TLog degraded
|
||||
TraceEvent(SevWarnAlways, "IoDegraded").log();
|
||||
degraded->set(true);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
Future<Void> end = lowPriorityDelay(errTime - degradedTime);
|
||||
choose {
|
||||
when(T t = wait(what)) { return t; }
|
||||
when(wait(end)) {
|
||||
Error err = io_timeout();
|
||||
if (g_network->isSimulated()) {
|
||||
err = err.asInjectedFault();
|
||||
}
|
||||
TraceEvent(SevError, "IoTimeoutError").error(err);
|
||||
throw err;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
ACTOR template <class T>
|
||||
Future<Void> streamHelper(PromiseStream<T> output, PromiseStream<Error> errors, Future<T> input) {
|
||||
try {
|
||||
|
|
|
@ -150,6 +150,7 @@ if(WITH_PYTHON)
|
|||
add_fdb_test(TEST_FILES fast/MemoryLifetime.toml)
|
||||
add_fdb_test(TEST_FILES fast/MoveKeysCycle.toml)
|
||||
add_fdb_test(TEST_FILES fast/MutationLogReaderCorrectness.toml)
|
||||
add_fdb_test(TEST_FILES fast/IndexPrefetchDemo.toml)
|
||||
add_fdb_test(TEST_FILES fast/ProtocolVersion.toml)
|
||||
add_fdb_test(TEST_FILES fast/RandomSelector.toml)
|
||||
add_fdb_test(TEST_FILES fast/RandomUnitTests.toml)
|
||||
|
|
|
@ -0,0 +1,9 @@
|
|||
[[test]]
|
||||
testTitle = 'PerfUnitTests'
|
||||
useDB = false
|
||||
startDelay = 0
|
||||
|
||||
[[test.workload]]
|
||||
testName = 'UnitTests'
|
||||
#maxTestCases = 1
|
||||
testsMatching = '#'
|
|
@ -0,0 +1,6 @@
|
|||
[[test]]
|
||||
testTitle = 'IndexPrefetchDemo'
|
||||
useDB = true
|
||||
|
||||
[[test.workload]]
|
||||
testName = 'IndexPrefetchDemo'
|
|
@ -0,0 +1,9 @@
|
|||
[[test]]
|
||||
testTitle = 'UnitTests'
|
||||
useDB = false
|
||||
startDelay = 0
|
||||
|
||||
[[test.workload]]
|
||||
testName = 'UnitTests'
|
||||
#maxTestCases = 1
|
||||
testsMatching = '/'
|
|
@ -5,5 +5,5 @@ startDelay = 0
|
|||
|
||||
[[test.workload]]
|
||||
testName = 'UnitTests'
|
||||
maxTestCases = 0
|
||||
testsMatching = '/redwood/correctness/btree'
|
||||
maxTestCases = 1
|
||||
testsMatching = 'Lredwood/correctness/btree'
|
||||
|
|
|
@ -0,0 +1,17 @@
|
|||
[[test]]
|
||||
testTitle = 'RedwoodDeltaTree'
|
||||
useDB = false
|
||||
startDelay = 0
|
||||
|
||||
[[test.workload]]
|
||||
testName = 'UnitTests'
|
||||
maxTestCases = 1
|
||||
testsMatching = 'Lredwood/correctness/unit/deltaTree/IntIntPair'
|
||||
|
||||
[[test]]
|
||||
testTitle = "RedwoodRecordRef"
|
||||
|
||||
[[test.workload]]
|
||||
testName = 'UnitTests'
|
||||
maxTestCases = 2 # there are two of those
|
||||
testsMatching = 'Lredwood/correctness/unit/deltaTree/RedwoodRecordRef'
|
|
@ -0,0 +1,9 @@
|
|||
[[test]]
|
||||
testTitle = 'StatusBuilderPerf'
|
||||
useDB = false
|
||||
startDelay = 0
|
||||
|
||||
[[test.workload]]
|
||||
testName = 'UnitTests'
|
||||
maxTestCases = 1
|
||||
testsMatching = 'Lstatus/json/builderPerf'
|
|
@ -0,0 +1,8 @@
|
|||
[[test]]
|
||||
testTitle = 'TLogVersionMessagesOverheadFactor'
|
||||
useDB = false
|
||||
startDelay = 0
|
||||
|
||||
[[test.workload]]
|
||||
testName = 'UnitTests'
|
||||
testsMatching = 'Lfdbserver/tlogserver/VersionMessagesOverheadFactor'
|
Loading…
Reference in New Issue