Merge branch 'main' of github.com:apple/foundationdb into monitorusage
This commit is contained in:
commit
4b52560dbc
|
@ -1,6 +1,6 @@
|
|||
# FindRocksDB
|
||||
|
||||
find_package(RocksDB 6.27.3)
|
||||
find_package(RocksDB 7.7.3)
|
||||
|
||||
include(ExternalProject)
|
||||
|
||||
|
@ -49,8 +49,8 @@ if(ROCKSDB_FOUND)
|
|||
${BINARY_DIR}/librocksdb.a)
|
||||
else()
|
||||
ExternalProject_Add(rocksdb
|
||||
URL https://github.com/facebook/rocksdb/archive/refs/tags/v6.27.3.tar.gz
|
||||
URL_HASH SHA256=ee29901749b9132692b26f0a6c1d693f47d1a9ed8e3771e60556afe80282bf58
|
||||
URL https://github.com/facebook/rocksdb/archive/refs/tags/v7.7.3.tar.gz
|
||||
URL_HASH SHA256=b8ac9784a342b2e314c821f6d701148912215666ac5e9bdbccd93cf3767cb611
|
||||
CMAKE_ARGS ${RocksDB_CMAKE_ARGS}
|
||||
BUILD_BYPRODUCTS <BINARY_DIR>/librocksdb.a
|
||||
INSTALL_COMMAND ""
|
||||
|
|
|
@ -475,7 +475,7 @@ Deletes a tenant from the cluster. The tenant must be empty.
|
|||
list
|
||||
^^^^
|
||||
|
||||
``tenant list [BEGIN] [END] [LIMIT]``
|
||||
``tenant list [BEGIN] [END] [limit=LIMIT] [offset=OFFSET] [state=<STATE1>,<STATE2>,...]``
|
||||
|
||||
Lists the tenants present in the cluster.
|
||||
|
||||
|
@ -485,6 +485,10 @@ Lists the tenants present in the cluster.
|
|||
|
||||
``LIMIT`` - the number of tenants to list. Defaults to 100.
|
||||
|
||||
``OFFSET`` - the number of items to skip over, starting from the beginning of the range. Defaults to 0.
|
||||
|
||||
``STATE``` - TenantState(s) to filter the list with. Defaults to no filters.
|
||||
|
||||
get
|
||||
^^^
|
||||
|
||||
|
|
|
@ -87,6 +87,56 @@ parseTenantConfiguration(std::vector<StringRef> const& tokens, int startIndex, b
|
|||
return configParams;
|
||||
}
|
||||
|
||||
bool parseTenantListOptions(std::vector<StringRef> const& tokens,
|
||||
int startIndex,
|
||||
int& limit,
|
||||
int& offset,
|
||||
std::vector<TenantState>& filters) {
|
||||
for (int tokenNum = startIndex; tokenNum < tokens.size(); ++tokenNum) {
|
||||
Optional<Value> value;
|
||||
StringRef token = tokens[tokenNum];
|
||||
StringRef param;
|
||||
bool foundEquals;
|
||||
param = token.eat("=", &foundEquals);
|
||||
if (!foundEquals) {
|
||||
fmt::print(stderr,
|
||||
"ERROR: invalid option string `{}'. String must specify a value using `='.\n",
|
||||
param.toString().c_str());
|
||||
return false;
|
||||
}
|
||||
value = token;
|
||||
if (tokencmp(param, "limit")) {
|
||||
int n = 0;
|
||||
if (sscanf(value.get().toString().c_str(), "%d%n", &limit, &n) != 1 || n != value.get().size() ||
|
||||
limit <= 0) {
|
||||
fmt::print(stderr, "ERROR: invalid limit `{}'\n", token.toString().c_str());
|
||||
return false;
|
||||
}
|
||||
} else if (tokencmp(param, "offset")) {
|
||||
int n = 0;
|
||||
if (sscanf(value.get().toString().c_str(), "%d%n", &offset, &n) != 1 || n != value.get().size() ||
|
||||
offset < 0) {
|
||||
fmt::print(stderr, "ERROR: invalid offset `{}'\n", token.toString().c_str());
|
||||
return false;
|
||||
}
|
||||
} else if (tokencmp(param, "state")) {
|
||||
auto filterStrings = value.get().splitAny(","_sr);
|
||||
try {
|
||||
for (auto sref : filterStrings) {
|
||||
filters.push_back(TenantMapEntry::stringToTenantState(sref.toString()));
|
||||
}
|
||||
} catch (Error& e) {
|
||||
fmt::print(stderr, "ERROR: unrecognized tenant state(s) `{}'.\n", value.get().toString());
|
||||
return false;
|
||||
}
|
||||
} else {
|
||||
fmt::print(stderr, "ERROR: unrecognized parameter `{}'.\n", param.toString().c_str());
|
||||
return false;
|
||||
}
|
||||
}
|
||||
return true;
|
||||
}
|
||||
|
||||
Key makeConfigKey(TenantNameRef tenantName, StringRef configName) {
|
||||
return tenantConfigSpecialKeyRange.begin.withSuffix(Tuple().append(tenantName).append(configName).pack());
|
||||
}
|
||||
|
@ -225,17 +275,21 @@ ACTOR Future<bool> tenantDeleteCommand(Reference<IDatabase> db, std::vector<Stri
|
|||
|
||||
// tenant list command
|
||||
ACTOR Future<bool> tenantListCommand(Reference<IDatabase> db, std::vector<StringRef> tokens) {
|
||||
if (tokens.size() > 5) {
|
||||
fmt::print("Usage: tenant list [BEGIN] [END] [LIMIT]\n\n");
|
||||
if (tokens.size() > 7) {
|
||||
fmt::print("Usage: tenant list [BEGIN] [END] [limit=LIMIT] [offset=OFFSET] [state=<STATE1>,<STATE2>,...]\n\n");
|
||||
fmt::print("Lists the tenants in a cluster.\n");
|
||||
fmt::print("Only tenants in the range BEGIN - END will be printed.\n");
|
||||
fmt::print("An optional LIMIT can be specified to limit the number of results (default 100).\n");
|
||||
fmt::print("Optionally skip over the first OFFSET results (default 0).\n");
|
||||
fmt::print("Optional comma-separated tenant state(s) can be provided to filter the list.\n");
|
||||
return false;
|
||||
}
|
||||
|
||||
state StringRef beginTenant = ""_sr;
|
||||
state StringRef endTenant = "\xff\xff"_sr;
|
||||
state int limit = 100;
|
||||
state int offset = 0;
|
||||
state std::vector<TenantState> filters;
|
||||
|
||||
if (tokens.size() >= 3) {
|
||||
beginTenant = tokens[2];
|
||||
|
@ -243,14 +297,12 @@ ACTOR Future<bool> tenantListCommand(Reference<IDatabase> db, std::vector<String
|
|||
if (tokens.size() >= 4) {
|
||||
endTenant = tokens[3];
|
||||
if (endTenant <= beginTenant) {
|
||||
fmt::print(stderr, "ERROR: end must be larger than begin");
|
||||
fmt::print(stderr, "ERROR: end must be larger than begin\n");
|
||||
return false;
|
||||
}
|
||||
}
|
||||
if (tokens.size() == 5) {
|
||||
int n = 0;
|
||||
if (sscanf(tokens[4].toString().c_str(), "%d%n", &limit, &n) != 1 || n != tokens[4].size() || limit <= 0) {
|
||||
fmt::print(stderr, "ERROR: invalid limit `{}'\n", tokens[4].toString().c_str());
|
||||
if (tokens.size() >= 5) {
|
||||
if (!parseTenantListOptions(tokens, 4, limit, offset, filters)) {
|
||||
return false;
|
||||
}
|
||||
}
|
||||
|
@ -266,7 +318,7 @@ ACTOR Future<bool> tenantListCommand(Reference<IDatabase> db, std::vector<String
|
|||
state std::vector<TenantName> tenantNames;
|
||||
if (clusterType == ClusterType::METACLUSTER_MANAGEMENT) {
|
||||
std::vector<std::pair<TenantName, TenantMapEntry>> tenants =
|
||||
wait(MetaclusterAPI::listTenantsTransaction(tr, beginTenant, endTenant, limit));
|
||||
wait(MetaclusterAPI::listTenants(db, beginTenant, endTenant, limit, offset, filters));
|
||||
for (auto tenant : tenants) {
|
||||
tenantNames.push_back(tenant.first);
|
||||
}
|
||||
|
@ -613,8 +665,10 @@ std::vector<const char*> tenantHintGenerator(std::vector<StringRef> const& token
|
|||
} else if (tokencmp(tokens[1], "delete") && tokens.size() < 3) {
|
||||
static std::vector<const char*> opts = { "<NAME>" };
|
||||
return std::vector<const char*>(opts.begin() + tokens.size() - 2, opts.end());
|
||||
} else if (tokencmp(tokens[1], "list") && tokens.size() < 5) {
|
||||
static std::vector<const char*> opts = { "[BEGIN]", "[END]", "[LIMIT]" };
|
||||
} else if (tokencmp(tokens[1], "list") && tokens.size() < 7) {
|
||||
static std::vector<const char*> opts = {
|
||||
"[BEGIN]", "[END]", "[limit=LIMIT]", "[offset=OFFSET]", "[state=<STATE1>,<STATE2>,...]"
|
||||
};
|
||||
return std::vector<const char*>(opts.begin() + tokens.size() - 2, opts.end());
|
||||
} else if (tokencmp(tokens[1], "get") && tokens.size() < 4) {
|
||||
static std::vector<const char*> opts = { "<NAME>", "[JSON]" };
|
||||
|
|
|
@ -786,7 +786,7 @@ def tenant_list(logger):
|
|||
output = run_fdbcli_command('tenant list')
|
||||
assert output == '1. tenant\n 2. tenant2'
|
||||
|
||||
output = run_fdbcli_command('tenant list a z 1')
|
||||
output = run_fdbcli_command('tenant list a z limit=1')
|
||||
assert output == '1. tenant'
|
||||
|
||||
output = run_fdbcli_command('tenant list a tenant2')
|
||||
|
@ -801,9 +801,15 @@ def tenant_list(logger):
|
|||
output = run_fdbcli_command_and_get_error('tenant list b a')
|
||||
assert output == 'ERROR: end must be larger than begin'
|
||||
|
||||
output = run_fdbcli_command_and_get_error('tenant list a b 12x')
|
||||
output = run_fdbcli_command_and_get_error('tenant list a b limit=12x')
|
||||
assert output == 'ERROR: invalid limit `12x\''
|
||||
|
||||
output = run_fdbcli_command_and_get_error('tenant list a b offset=13y')
|
||||
assert output == 'ERROR: invalid offset `13y\''
|
||||
|
||||
output = run_fdbcli_command_and_get_error('tenant list a b state=14z')
|
||||
assert output == 'ERROR: unrecognized tenant state(s) `14z\'.'
|
||||
|
||||
@enable_logging()
|
||||
def tenant_get(logger):
|
||||
setup_tenants(['tenant', 'tenant2 tenant_group=tenant_group2'])
|
||||
|
|
|
@ -57,11 +57,11 @@ BlobCipherMetrics::CounterSet::CounterSet(CounterCollection& cc, std::string nam
|
|||
getCipherKeysLatency(name + "GetCipherKeysLatency",
|
||||
UID(),
|
||||
FLOW_KNOBS->ENCRYPT_KEY_CACHE_LOGGING_INTERVAL,
|
||||
FLOW_KNOBS->ENCRYPT_KEY_CACHE_LOGGING_SAMPLE_SIZE),
|
||||
FLOW_KNOBS->ENCRYPT_KEY_CACHE_LOGGING_SKETCH_ACCURACY),
|
||||
getLatestCipherKeysLatency(name + "GetLatestCipherKeysLatency",
|
||||
UID(),
|
||||
FLOW_KNOBS->ENCRYPT_KEY_CACHE_LOGGING_INTERVAL,
|
||||
FLOW_KNOBS->ENCRYPT_KEY_CACHE_LOGGING_SAMPLE_SIZE) {}
|
||||
FLOW_KNOBS->ENCRYPT_KEY_CACHE_LOGGING_SKETCH_ACCURACY) {}
|
||||
|
||||
BlobCipherMetrics::BlobCipherMetrics()
|
||||
: cc("BlobCipher"), cipherKeyCacheHit("CipherKeyCacheHit", cc), cipherKeyCacheMiss("CipherKeyCacheMiss", cc),
|
||||
|
@ -71,15 +71,15 @@ BlobCipherMetrics::BlobCipherMetrics()
|
|||
getCipherKeysLatency("GetCipherKeysLatency",
|
||||
UID(),
|
||||
FLOW_KNOBS->ENCRYPT_KEY_CACHE_LOGGING_INTERVAL,
|
||||
FLOW_KNOBS->ENCRYPT_KEY_CACHE_LOGGING_SAMPLE_SIZE),
|
||||
FLOW_KNOBS->ENCRYPT_KEY_CACHE_LOGGING_SKETCH_ACCURACY),
|
||||
getLatestCipherKeysLatency("GetLatestCipherKeysLatency",
|
||||
UID(),
|
||||
FLOW_KNOBS->ENCRYPT_KEY_CACHE_LOGGING_INTERVAL,
|
||||
FLOW_KNOBS->ENCRYPT_KEY_CACHE_LOGGING_SAMPLE_SIZE),
|
||||
FLOW_KNOBS->ENCRYPT_KEY_CACHE_LOGGING_SKETCH_ACCURACY),
|
||||
getBlobMetadataLatency("GetBlobMetadataLatency",
|
||||
UID(),
|
||||
FLOW_KNOBS->ENCRYPT_KEY_CACHE_LOGGING_INTERVAL,
|
||||
FLOW_KNOBS->ENCRYPT_KEY_CACHE_LOGGING_SAMPLE_SIZE),
|
||||
FLOW_KNOBS->ENCRYPT_KEY_CACHE_LOGGING_SKETCH_ACCURACY),
|
||||
counterSets({ CounterSet(cc, "TLog"),
|
||||
CounterSet(cc, "KVMemory"),
|
||||
CounterSet(cc, "KVRedwood"),
|
||||
|
|
|
@ -50,6 +50,82 @@ KeyRef keyBetween(const KeyRangeRef& keys) {
|
|||
return keys.end;
|
||||
}
|
||||
|
||||
Key randomKeyBetween(const KeyRangeRef& keys) {
|
||||
if (keys.empty() || keys.singleKeyRange()) {
|
||||
return keys.end;
|
||||
}
|
||||
|
||||
KeyRef begin = keys.begin;
|
||||
KeyRef end = keys.end;
|
||||
ASSERT(begin < end);
|
||||
if (begin.size() < end.size()) {
|
||||
// randomly append a char
|
||||
uint8_t maxChar = end[begin.size()] > 0 ? end[begin.size()] : end[begin.size()] + 1;
|
||||
uint8_t newChar = deterministicRandom()->randomInt(0, maxChar);
|
||||
return begin.withSuffix(StringRef(&newChar, 1));
|
||||
}
|
||||
|
||||
int pos = 0; // will be the position of the first difference between keys.begin and keys.end
|
||||
for (; pos < end.size() && pos < CLIENT_KNOBS->KEY_SIZE_LIMIT; pos++) {
|
||||
if (keys.begin[pos] != keys.end[pos]) {
|
||||
break;
|
||||
}
|
||||
}
|
||||
ASSERT_LT(pos, end.size()); // otherwise, begin >= end
|
||||
|
||||
// find the lowest char in range begin[pos+1, begin.size()) that is not \xff (255)
|
||||
int lowest = begin.size() - 1;
|
||||
for (; lowest > pos; lowest--) {
|
||||
if (begin[lowest] < 255) {
|
||||
Key res = begin;
|
||||
uint8_t* ptr = mutateString(res);
|
||||
*(ptr + lowest) = (uint8_t)deterministicRandom()->randomInt(begin[lowest] + 1, 256);
|
||||
return res;
|
||||
}
|
||||
}
|
||||
|
||||
if (begin[pos] + 1 < end[pos]) {
|
||||
Key res = begin;
|
||||
uint8_t* ptr = mutateString(res);
|
||||
*(ptr + pos) = (uint8_t)deterministicRandom()->randomInt(begin[pos] + 1, end[pos]);
|
||||
return res;
|
||||
}
|
||||
|
||||
if (begin.size() + 1 < CLIENT_KNOBS->KEY_SIZE_LIMIT) {
|
||||
// randomly append a char
|
||||
uint8_t newChar = deterministicRandom()->randomInt(1, 255);
|
||||
return begin.withSuffix(StringRef(&newChar, 1));
|
||||
}
|
||||
|
||||
// no possible result
|
||||
return end;
|
||||
}
|
||||
|
||||
TEST_CASE("/KeyRangeUtil/randomKeyBetween") {
|
||||
Key begin = "qwert"_sr;
|
||||
Key end = "qwertyu"_sr;
|
||||
Key res;
|
||||
for (int i = 0; i < 10; ++i) {
|
||||
res = randomKeyBetween(KeyRangeRef(begin, end));
|
||||
ASSERT(res > begin);
|
||||
ASSERT(res < end);
|
||||
}
|
||||
|
||||
begin = "q"_sr;
|
||||
end = "q\x00"_sr;
|
||||
res = randomKeyBetween(KeyRangeRef(begin, end));
|
||||
ASSERT(res == end);
|
||||
|
||||
begin = "aaaaaaa"_sr;
|
||||
end = "b"_sr;
|
||||
for (int i = 0; i < 10; ++i) {
|
||||
res = randomKeyBetween(KeyRangeRef(begin, end));
|
||||
ASSERT(res > begin);
|
||||
ASSERT(res < end);
|
||||
}
|
||||
return Void();
|
||||
}
|
||||
|
||||
void KeySelectorRef::setKey(KeyRef const& key) {
|
||||
// There are no keys in the database with size greater than the max key size, so if this key selector has a key
|
||||
// which is large, then we can translate it to an equivalent key selector with a smaller key
|
||||
|
|
|
@ -578,7 +578,7 @@ void traceTSSErrors(const char* name, UID tssId, const std::unordered_map<int, u
|
|||
Example:
|
||||
GetValueLatencySSMean
|
||||
*/
|
||||
void traceSSOrTSSPercentiles(TraceEvent& ev, const std::string name, ContinuousSample<double>& sample) {
|
||||
void traceSSOrTSSPercentiles(TraceEvent& ev, const std::string name, DDSketch<double>& sample) {
|
||||
ev.detail(name + "Mean", sample.mean());
|
||||
// don't log the larger percentiles unless we actually have enough samples to log the accurate percentile instead of
|
||||
// the largest sample in this window
|
||||
|
@ -595,8 +595,8 @@ void traceSSOrTSSPercentiles(TraceEvent& ev, const std::string name, ContinuousS
|
|||
|
||||
void traceTSSPercentiles(TraceEvent& ev,
|
||||
const std::string name,
|
||||
ContinuousSample<double>& ssSample,
|
||||
ContinuousSample<double>& tssSample) {
|
||||
DDSketch<double>& ssSample,
|
||||
DDSketch<double>& tssSample) {
|
||||
ASSERT(ssSample.getPopulationSize() == tssSample.getPopulationSize());
|
||||
ev.detail(name + "Count", ssSample.getPopulationSize());
|
||||
if (ssSample.getPopulationSize() > 0) {
|
||||
|
@ -1534,17 +1534,16 @@ DatabaseContext::DatabaseContext(Reference<AsyncVar<Reference<IClusterConnection
|
|||
ccBG("BlobGranuleReadMetrics"), bgReadInputBytes("BGReadInputBytes", ccBG),
|
||||
bgReadOutputBytes("BGReadOutputBytes", ccBG), bgReadSnapshotRows("BGReadSnapshotRows", ccBG),
|
||||
bgReadRowsCleared("BGReadRowsCleared", ccBG), bgReadRowsInserted("BGReadRowsInserted", ccBG),
|
||||
bgReadRowsUpdated("BGReadRowsUpdated", ccBG), bgLatencies(1000), bgGranulesPerRequest(1000),
|
||||
usedAnyChangeFeeds(false), ccFeed("ChangeFeedClientMetrics"), feedStreamStarts("FeedStreamStarts", ccFeed),
|
||||
bgReadRowsUpdated("BGReadRowsUpdated", ccBG), bgLatencies(), bgGranulesPerRequest(), usedAnyChangeFeeds(false),
|
||||
ccFeed("ChangeFeedClientMetrics"), feedStreamStarts("FeedStreamStarts", ccFeed),
|
||||
feedMergeStreamStarts("FeedMergeStreamStarts", ccFeed), feedErrors("FeedErrors", ccFeed),
|
||||
feedNonRetriableErrors("FeedNonRetriableErrors", ccFeed), feedPops("FeedPops", ccFeed),
|
||||
feedPopsFallback("FeedPopsFallback", ccFeed), latencies(1000), readLatencies(1000), commitLatencies(1000),
|
||||
GRVLatencies(1000), mutationsPerCommit(1000), bytesPerCommit(1000), outstandingWatches(0), sharedStatePtr(nullptr),
|
||||
lastGrvTime(0.0), cachedReadVersion(0), lastRkBatchThrottleTime(0.0), lastRkDefaultThrottleTime(0.0),
|
||||
lastProxyRequestTime(0.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),
|
||||
feedPopsFallback("FeedPopsFallback", ccFeed), latencies(), readLatencies(), commitLatencies(), GRVLatencies(),
|
||||
mutationsPerCommit(), bytesPerCommit(), outstandingWatches(0), sharedStatePtr(nullptr), lastGrvTime(0.0),
|
||||
cachedReadVersion(0), lastRkBatchThrottleTime(0.0), lastRkDefaultThrottleTime(0.0), lastProxyRequestTime(0.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),
|
||||
specialKeySpace(std::make_unique<SpecialKeySpace>(specialKeys.begin, specialKeys.end, /* test */ false)),
|
||||
connectToDatabaseEventCacheHolder(format("ConnectToDatabase/%s", dbId.toString().c_str())) {
|
||||
|
||||
|
@ -1838,13 +1837,13 @@ DatabaseContext::DatabaseContext(const Error& err)
|
|||
ccBG("BlobGranuleReadMetrics"), bgReadInputBytes("BGReadInputBytes", ccBG),
|
||||
bgReadOutputBytes("BGReadOutputBytes", ccBG), bgReadSnapshotRows("BGReadSnapshotRows", ccBG),
|
||||
bgReadRowsCleared("BGReadRowsCleared", ccBG), bgReadRowsInserted("BGReadRowsInserted", ccBG),
|
||||
bgReadRowsUpdated("BGReadRowsUpdated", ccBG), bgLatencies(1000), bgGranulesPerRequest(1000),
|
||||
usedAnyChangeFeeds(false), ccFeed("ChangeFeedClientMetrics"), feedStreamStarts("FeedStreamStarts", ccFeed),
|
||||
bgReadRowsUpdated("BGReadRowsUpdated", ccBG), bgLatencies(), bgGranulesPerRequest(), usedAnyChangeFeeds(false),
|
||||
ccFeed("ChangeFeedClientMetrics"), feedStreamStarts("FeedStreamStarts", ccFeed),
|
||||
feedMergeStreamStarts("FeedMergeStreamStarts", ccFeed), feedErrors("FeedErrors", ccFeed),
|
||||
feedNonRetriableErrors("FeedNonRetriableErrors", ccFeed), feedPops("FeedPops", ccFeed),
|
||||
feedPopsFallback("FeedPopsFallback", ccFeed), latencies(1000), readLatencies(1000), commitLatencies(1000),
|
||||
GRVLatencies(1000), mutationsPerCommit(1000), bytesPerCommit(1000), sharedStatePtr(nullptr),
|
||||
transactionTracingSample(false), smoothMidShardSize(CLIENT_KNOBS->SHARD_STAT_SMOOTH_AMOUNT),
|
||||
feedPopsFallback("FeedPopsFallback", ccFeed), latencies(), readLatencies(), commitLatencies(), GRVLatencies(),
|
||||
mutationsPerCommit(), bytesPerCommit(), sharedStatePtr(nullptr), transactionTracingSample(false),
|
||||
smoothMidShardSize(CLIENT_KNOBS->SHARD_STAT_SMOOTH_AMOUNT),
|
||||
connectToDatabaseEventCacheHolder(format("ConnectToDatabase/%s", dbId.toString().c_str())) {}
|
||||
|
||||
// Static constructor used by server processes to create a DatabaseContext
|
||||
|
|
|
@ -224,7 +224,7 @@ void ServerKnobs::initialize(Randomize randomize, ClientKnobs* clientKnobs, IsSi
|
|||
shards.
|
||||
|
||||
The bandwidth sample maintained by the storage server needs to be accurate enough to reliably measure this minimum bandwidth. See
|
||||
BANDWIDTH_UNITS_PER_SAMPLE. If this number is too low, the storage server needs to spend more memory and time on sampling.
|
||||
BYTES_WRITTEN_UNITS_PER_SAMPLE. If this number is too low, the storage server needs to spend more memory and time on sampling.
|
||||
*/
|
||||
|
||||
init( SHARD_SPLIT_BYTES_PER_KSEC, 250 * 1000 * 1000 ); if( buggifySmallBandwidthSplit ) SHARD_SPLIT_BYTES_PER_KSEC = 50 * 1000 * 1000;
|
||||
|
@ -391,19 +391,22 @@ void ServerKnobs::initialize(Randomize randomize, ClientKnobs* clientKnobs, IsSi
|
|||
// If true, do not process and store RocksDB logs
|
||||
init( ROCKSDB_MUTE_LOGS, true );
|
||||
// Use a smaller memtable in simulation to avoid OOMs.
|
||||
int64_t memtableBytes = isSimulated ? 32 * 1024 : 512 * 1024 * 1024;
|
||||
int64_t memtableBytes = isSimulated ? 1024 * 1024 : 512 * 1024 * 1024;
|
||||
init( ROCKSDB_MEMTABLE_BYTES, memtableBytes );
|
||||
init( ROCKSDB_LEVEL_STYLE_COMPACTION, true );
|
||||
init( ROCKSDB_UNSAFE_AUTO_FSYNC, false );
|
||||
init( ROCKSDB_PERIODIC_COMPACTION_SECONDS, 0 );
|
||||
init( ROCKSDB_PREFIX_LEN, 0 );
|
||||
// If rocksdb block cache size is 0, the default 8MB is used.
|
||||
int64_t blockCacheSize = isSimulated ? 0 : 1024 * 1024 * 1024 /* 1GB */;
|
||||
int64_t blockCacheSize = isSimulated ? 16 * 1024 * 1024 : 1024 * 1024 * 1024 /* 1GB */;
|
||||
init( ROCKSDB_BLOCK_CACHE_SIZE, blockCacheSize );
|
||||
init( ROCKSDB_METRICS_DELAY, 60.0 );
|
||||
init( ROCKSDB_READ_VALUE_TIMEOUT, isSimulated ? 5.0 : 200.0 );
|
||||
init( ROCKSDB_READ_VALUE_PREFIX_TIMEOUT, isSimulated ? 5.0 : 200.0 );
|
||||
init( ROCKSDB_READ_RANGE_TIMEOUT, isSimulated ? 5.0 : 200.0 );
|
||||
// ROCKSDB_READ_VALUE_TIMEOUT, ROCKSDB_READ_VALUE_PREFIX_TIMEOUT, ROCKSDB_READ_RANGE_TIMEOUT knobs:
|
||||
// In simulation, increasing the read operation timeouts to 5 minutes, as some of the tests have
|
||||
// very high load and single read thread cannot process all the load within the timeouts.
|
||||
init( ROCKSDB_READ_VALUE_TIMEOUT, 200.0 ); if (isSimulated) ROCKSDB_READ_VALUE_TIMEOUT = 5 * 60;
|
||||
init( ROCKSDB_READ_VALUE_PREFIX_TIMEOUT, 200.0 ); if (isSimulated) ROCKSDB_READ_VALUE_PREFIX_TIMEOUT = 5 * 60;
|
||||
init( ROCKSDB_READ_RANGE_TIMEOUT, 200.0 ); if (isSimulated) ROCKSDB_READ_RANGE_TIMEOUT = 5 * 60;
|
||||
init( ROCKSDB_READ_QUEUE_WAIT, 1.0 );
|
||||
init( ROCKSDB_READ_QUEUE_HARD_MAX, 1000 );
|
||||
init( ROCKSDB_READ_QUEUE_SOFT_MAX, 500 );
|
||||
|
@ -759,7 +762,7 @@ void ServerKnobs::initialize(Randomize randomize, ClientKnobs* clientKnobs, IsSi
|
|||
init( STORAGE_METRICS_AVERAGE_INTERVAL_PER_KSECONDS, 1000.0 / STORAGE_METRICS_AVERAGE_INTERVAL ); // milliHz!
|
||||
init( SPLIT_JITTER_AMOUNT, 0.05 ); if( randomize && BUGGIFY ) SPLIT_JITTER_AMOUNT = 0.2;
|
||||
init( IOPS_UNITS_PER_SAMPLE, 10000 * 1000 / STORAGE_METRICS_AVERAGE_INTERVAL_PER_KSECONDS / 100 );
|
||||
init( BANDWIDTH_UNITS_PER_SAMPLE, SHARD_MIN_BYTES_PER_KSEC / STORAGE_METRICS_AVERAGE_INTERVAL_PER_KSECONDS / 25 );
|
||||
init( BYTES_WRITTEN_UNITS_PER_SAMPLE, SHARD_MIN_BYTES_PER_KSEC / STORAGE_METRICS_AVERAGE_INTERVAL_PER_KSECONDS / 25 );
|
||||
init( BYTES_READ_UNITS_PER_SAMPLE, 100000 ); // 100K bytes
|
||||
init( READ_HOT_SUB_RANGE_CHUNK_SIZE, 10000000); // 10MB
|
||||
init( EMPTY_READ_PENALTY, 20 ); // 20 bytes
|
||||
|
@ -956,8 +959,8 @@ void ServerKnobs::initialize(Randomize randomize, ClientKnobs* clientKnobs, IsSi
|
|||
init( REDWOOD_SPLIT_ENCRYPTED_PAGES_BY_TENANT, false );
|
||||
|
||||
// Server request latency measurement
|
||||
init( LATENCY_SAMPLE_SIZE, 100000 );
|
||||
init( FILE_LATENCY_SAMPLE_SIZE, 10000 );
|
||||
init( LATENCY_SKETCH_ACCURACY, 0.01 );
|
||||
init( FILE_LATENCY_SKETCH_ACCURACY, 0.01 );
|
||||
init( LATENCY_METRICS_LOGGING_INTERVAL, 60.0 );
|
||||
|
||||
// Cluster recovery
|
||||
|
|
|
@ -70,6 +70,7 @@ std::string TenantMapEntry::tenantStateToString(TenantState tenantState) {
|
|||
}
|
||||
|
||||
TenantState TenantMapEntry::stringToTenantState(std::string stateStr) {
|
||||
std::transform(stateStr.begin(), stateStr.end(), stateStr.begin(), [](unsigned char c) { return std::tolower(c); });
|
||||
if (stateStr == "registering") {
|
||||
return TenantState::REGISTERING;
|
||||
} else if (stateStr == "ready") {
|
||||
|
@ -86,7 +87,7 @@ TenantState TenantMapEntry::stringToTenantState(std::string stateStr) {
|
|||
return TenantState::ERROR;
|
||||
}
|
||||
|
||||
UNREACHABLE();
|
||||
throw invalid_option();
|
||||
}
|
||||
|
||||
std::string TenantMapEntry::tenantLockStateToString(TenantLockState tenantState) {
|
||||
|
@ -103,6 +104,7 @@ std::string TenantMapEntry::tenantLockStateToString(TenantLockState tenantState)
|
|||
}
|
||||
|
||||
TenantLockState TenantMapEntry::stringToTenantLockState(std::string stateStr) {
|
||||
std::transform(stateStr.begin(), stateStr.end(), stateStr.begin(), [](unsigned char c) { return std::tolower(c); });
|
||||
if (stateStr == "unlocked") {
|
||||
return TenantLockState::UNLOCKED;
|
||||
} else if (stateStr == "read only") {
|
||||
|
|
|
@ -75,8 +75,8 @@ struct BlobWorkerStats {
|
|||
Reference<FlowLock> resnapshotLock,
|
||||
Reference<FlowLock> deltaWritesLock,
|
||||
double sampleLoggingInterval,
|
||||
int fileOpLatencySampleSize,
|
||||
int requestLatencySampleSize)
|
||||
double fileOpLatencySketchAccuracy,
|
||||
double requestLatencySketchAccuracy)
|
||||
: cc("BlobWorkerStats", id.toString()),
|
||||
|
||||
s3PutReqs("S3PutReqs", cc), s3GetReqs("S3GetReqs", cc), s3DeleteReqs("S3DeleteReqs", cc),
|
||||
|
@ -95,10 +95,13 @@ struct BlobWorkerStats {
|
|||
forceFlushCleanups("ForceFlushCleanups", cc), readDrivenCompactions("ReadDrivenCompactions", cc),
|
||||
numRangesAssigned(0), mutationBytesBuffered(0), activeReadRequests(0), granulesPendingSplitCheck(0),
|
||||
minimumCFVersion(0), cfVersionLag(0), notAtLatestChangeFeeds(0), lastResidentMemory(0),
|
||||
snapshotBlobWriteLatencySample("SnapshotBlobWriteMetrics", id, sampleLoggingInterval, fileOpLatencySampleSize),
|
||||
deltaBlobWriteLatencySample("DeltaBlobWriteMetrics", id, sampleLoggingInterval, fileOpLatencySampleSize),
|
||||
reSnapshotLatencySample("GranuleResnapshotMetrics", id, sampleLoggingInterval, fileOpLatencySampleSize),
|
||||
readLatencySample("GranuleReadLatencyMetrics", id, sampleLoggingInterval, requestLatencySampleSize),
|
||||
snapshotBlobWriteLatencySample("SnapshotBlobWriteMetrics",
|
||||
id,
|
||||
sampleLoggingInterval,
|
||||
fileOpLatencySketchAccuracy),
|
||||
deltaBlobWriteLatencySample("DeltaBlobWriteMetrics", id, sampleLoggingInterval, fileOpLatencySketchAccuracy),
|
||||
reSnapshotLatencySample("GranuleResnapshotMetrics", id, sampleLoggingInterval, fileOpLatencySketchAccuracy),
|
||||
readLatencySample("GranuleReadLatencyMetrics", id, sampleLoggingInterval, requestLatencySketchAccuracy),
|
||||
estimatedMaxResidentMemory(0), initialSnapshotLock(initialSnapshotLock), resnapshotLock(resnapshotLock),
|
||||
deltaWritesLock(deltaWritesLock) {
|
||||
specialCounter(cc, "NumRangesAssigned", [this]() { return this->numRangesAssigned; });
|
||||
|
|
|
@ -42,8 +42,8 @@
|
|||
#include "fdbrpc/MultiInterface.h"
|
||||
#include "flow/TDMetric.actor.h"
|
||||
#include "fdbclient/EventTypes.actor.h"
|
||||
#include "fdbrpc/ContinuousSample.h"
|
||||
#include "fdbrpc/Smoother.h"
|
||||
#include "fdbrpc/DDSketch.h"
|
||||
|
||||
class StorageServerInfo : public ReferencedInterface<StorageServerInterface> {
|
||||
public:
|
||||
|
@ -567,7 +567,7 @@ public:
|
|||
Counter bgReadRowsCleared;
|
||||
Counter bgReadRowsInserted;
|
||||
Counter bgReadRowsUpdated;
|
||||
ContinuousSample<double> bgLatencies, bgGranulesPerRequest;
|
||||
DDSketch<double> bgLatencies, bgGranulesPerRequest;
|
||||
|
||||
// Change Feed metrics. Omit change feed metrics from logging if not used
|
||||
bool usedAnyChangeFeeds;
|
||||
|
@ -579,8 +579,7 @@ public:
|
|||
Counter feedPops;
|
||||
Counter feedPopsFallback;
|
||||
|
||||
ContinuousSample<double> latencies, readLatencies, commitLatencies, GRVLatencies, mutationsPerCommit,
|
||||
bytesPerCommit;
|
||||
DDSketch<double> latencies, readLatencies, commitLatencies, GRVLatencies, mutationsPerCommit, bytesPerCommit;
|
||||
|
||||
int outstandingWatches;
|
||||
int maxOutstandingWatches;
|
||||
|
|
|
@ -591,6 +591,9 @@ inline KeyRange prefixRange(KeyRef prefix) {
|
|||
// The returned reference is valid as long as keys is valid.
|
||||
KeyRef keyBetween(const KeyRangeRef& keys);
|
||||
|
||||
// Returns a randomKey between keys. If it's impossible, return keys.end.
|
||||
Key randomKeyBetween(const KeyRangeRef& keys);
|
||||
|
||||
KeyRangeRef toPrefixRelativeRange(KeyRangeRef range, KeyRef prefix);
|
||||
|
||||
struct KeySelectorRef {
|
||||
|
|
|
@ -1561,26 +1561,64 @@ Future<std::vector<std::pair<TenantName, TenantMapEntry>>> listTenantsTransactio
|
|||
int limit) {
|
||||
tr->setOption(FDBTransactionOptions::RAW_ACCESS);
|
||||
|
||||
KeyBackedRangeResult<std::pair<TenantName, TenantMapEntry>> results =
|
||||
state KeyBackedRangeResult<std::pair<TenantName, TenantMapEntry>> results =
|
||||
wait(ManagementClusterMetadata::tenantMetadata().tenantMap.getRange(tr, begin, end, limit));
|
||||
|
||||
return results.results;
|
||||
}
|
||||
|
||||
ACTOR template <class DB>
|
||||
Future<std::vector<std::pair<TenantName, TenantMapEntry>>> listTenants(Reference<DB> db,
|
||||
TenantName begin,
|
||||
TenantName end,
|
||||
int limit) {
|
||||
Future<std::vector<std::pair<TenantName, TenantMapEntry>>> listTenants(
|
||||
Reference<DB> db,
|
||||
TenantName begin,
|
||||
TenantName end,
|
||||
int limit,
|
||||
int offset = 0,
|
||||
std::vector<TenantState> filters = std::vector<TenantState>()) {
|
||||
state Reference<typename DB::TransactionT> tr = db->createTransaction();
|
||||
|
||||
loop {
|
||||
try {
|
||||
tr->setOption(FDBTransactionOptions::READ_SYSTEM_KEYS);
|
||||
tr->setOption(FDBTransactionOptions::READ_LOCK_AWARE);
|
||||
std::vector<std::pair<TenantName, TenantMapEntry>> tenants =
|
||||
wait(listTenantsTransaction(tr, begin, end, limit));
|
||||
return tenants;
|
||||
if (filters.empty()) {
|
||||
state std::vector<std::pair<TenantName, TenantMapEntry>> tenants;
|
||||
wait(store(tenants, listTenantsTransaction(tr, begin, end, limit + offset)));
|
||||
if (offset >= tenants.size()) {
|
||||
tenants.clear();
|
||||
} else if (offset > 0) {
|
||||
tenants.erase(tenants.begin(), tenants.begin() + offset);
|
||||
}
|
||||
return tenants;
|
||||
}
|
||||
tr->setOption(FDBTransactionOptions::RAW_ACCESS);
|
||||
|
||||
state KeyBackedRangeResult<std::pair<TenantName, TenantMapEntry>> results =
|
||||
wait(ManagementClusterMetadata::tenantMetadata().tenantMap.getRange(
|
||||
tr, begin, end, std::max(limit + offset, 100)));
|
||||
state std::vector<std::pair<TenantName, TenantMapEntry>> filterResults;
|
||||
state int count = 0;
|
||||
loop {
|
||||
for (auto pair : results.results) {
|
||||
if (filters.empty() || std::count(filters.begin(), filters.end(), pair.second.tenantState)) {
|
||||
++count;
|
||||
if (count > offset) {
|
||||
filterResults.push_back(pair);
|
||||
if (count - offset == limit) {
|
||||
ASSERT(count - offset == filterResults.size());
|
||||
return filterResults;
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
if (!results.more) {
|
||||
return filterResults;
|
||||
}
|
||||
begin = keyAfter(results.results.back().first);
|
||||
wait(store(results,
|
||||
ManagementClusterMetadata::tenantMetadata().tenantMap.getRange(
|
||||
tr, begin, end, std::max(limit + offset, 100))));
|
||||
}
|
||||
} catch (Error& e) {
|
||||
wait(safeThreadFutureToFuture(tr->onError(e)));
|
||||
}
|
||||
|
|
|
@ -710,7 +710,7 @@ public:
|
|||
double STORAGE_METRICS_AVERAGE_INTERVAL_PER_KSECONDS;
|
||||
double SPLIT_JITTER_AMOUNT;
|
||||
int64_t IOPS_UNITS_PER_SAMPLE;
|
||||
int64_t BANDWIDTH_UNITS_PER_SAMPLE;
|
||||
int64_t BYTES_WRITTEN_UNITS_PER_SAMPLE;
|
||||
int64_t BYTES_READ_UNITS_PER_SAMPLE;
|
||||
int64_t READ_HOT_SUB_RANGE_CHUNK_SIZE;
|
||||
int64_t EMPTY_READ_PENALTY;
|
||||
|
@ -926,8 +926,8 @@ public:
|
|||
std::string REDWOOD_IO_PRIORITIES;
|
||||
|
||||
// Server request latency measurement
|
||||
int LATENCY_SAMPLE_SIZE;
|
||||
int FILE_LATENCY_SAMPLE_SIZE;
|
||||
double LATENCY_SKETCH_ACCURACY;
|
||||
double FILE_LATENCY_SKETCH_ACCURACY;
|
||||
double LATENCY_METRICS_LOGGING_INTERVAL;
|
||||
|
||||
// Cluster recovery
|
||||
|
|
|
@ -634,42 +634,42 @@ struct GetShardStateRequest {
|
|||
struct StorageMetrics {
|
||||
constexpr static FileIdentifier file_identifier = 13622226;
|
||||
int64_t bytes = 0; // total storage
|
||||
// FIXME: currently, neither of bytesPerKSecond or iosPerKSecond are actually used in DataDistribution calculations.
|
||||
// This may change in the future, but this comment is left here to avoid any confusion for the time being.
|
||||
int64_t bytesPerKSecond = 0; // network bandwidth (average over 10s)
|
||||
int64_t bytesWrittenPerKSecond = 0; // bytes write to SQ
|
||||
|
||||
// FIXME: currently, iosPerKSecond is not used in DataDistribution calculations.
|
||||
int64_t iosPerKSecond = 0;
|
||||
int64_t bytesReadPerKSecond = 0;
|
||||
|
||||
static const int64_t infinity = 1LL << 60;
|
||||
|
||||
bool allLessOrEqual(const StorageMetrics& rhs) const {
|
||||
return bytes <= rhs.bytes && bytesPerKSecond <= rhs.bytesPerKSecond && iosPerKSecond <= rhs.iosPerKSecond &&
|
||||
bytesReadPerKSecond <= rhs.bytesReadPerKSecond;
|
||||
return bytes <= rhs.bytes && bytesWrittenPerKSecond <= rhs.bytesWrittenPerKSecond &&
|
||||
iosPerKSecond <= rhs.iosPerKSecond && bytesReadPerKSecond <= rhs.bytesReadPerKSecond;
|
||||
}
|
||||
void operator+=(const StorageMetrics& rhs) {
|
||||
bytes += rhs.bytes;
|
||||
bytesPerKSecond += rhs.bytesPerKSecond;
|
||||
bytesWrittenPerKSecond += rhs.bytesWrittenPerKSecond;
|
||||
iosPerKSecond += rhs.iosPerKSecond;
|
||||
bytesReadPerKSecond += rhs.bytesReadPerKSecond;
|
||||
}
|
||||
void operator-=(const StorageMetrics& rhs) {
|
||||
bytes -= rhs.bytes;
|
||||
bytesPerKSecond -= rhs.bytesPerKSecond;
|
||||
bytesWrittenPerKSecond -= rhs.bytesWrittenPerKSecond;
|
||||
iosPerKSecond -= rhs.iosPerKSecond;
|
||||
bytesReadPerKSecond -= rhs.bytesReadPerKSecond;
|
||||
}
|
||||
template <class F>
|
||||
void operator*=(F f) {
|
||||
bytes *= f;
|
||||
bytesPerKSecond *= f;
|
||||
bytesWrittenPerKSecond *= f;
|
||||
iosPerKSecond *= f;
|
||||
bytesReadPerKSecond *= f;
|
||||
}
|
||||
bool allZero() const { return !bytes && !bytesPerKSecond && !iosPerKSecond && !bytesReadPerKSecond; }
|
||||
bool allZero() const { return !bytes && !bytesWrittenPerKSecond && !iosPerKSecond && !bytesReadPerKSecond; }
|
||||
|
||||
template <class Ar>
|
||||
void serialize(Ar& ar) {
|
||||
serializer(ar, bytes, bytesPerKSecond, iosPerKSecond, bytesReadPerKSecond);
|
||||
serializer(ar, bytes, bytesWrittenPerKSecond, iosPerKSecond, bytesReadPerKSecond);
|
||||
}
|
||||
|
||||
void negate() { operator*=(-1.0); }
|
||||
|
@ -697,14 +697,14 @@ struct StorageMetrics {
|
|||
}
|
||||
|
||||
bool operator==(StorageMetrics const& rhs) const {
|
||||
return bytes == rhs.bytes && bytesPerKSecond == rhs.bytesPerKSecond && iosPerKSecond == rhs.iosPerKSecond &&
|
||||
bytesReadPerKSecond == rhs.bytesReadPerKSecond;
|
||||
return bytes == rhs.bytes && bytesWrittenPerKSecond == rhs.bytesWrittenPerKSecond &&
|
||||
iosPerKSecond == rhs.iosPerKSecond && bytesReadPerKSecond == rhs.bytesReadPerKSecond;
|
||||
}
|
||||
|
||||
std::string toString() const {
|
||||
return format("Bytes: %lld, BPerKSec: %lld, iosPerKSec: %lld, BReadPerKSec: %lld",
|
||||
return format("Bytes: %lld, BWritePerKSec: %lld, iosPerKSec: %lld, BReadPerKSec: %lld",
|
||||
bytes,
|
||||
bytesPerKSecond,
|
||||
bytesWrittenPerKSecond,
|
||||
iosPerKSecond,
|
||||
bytesReadPerKSecond);
|
||||
}
|
||||
|
@ -1180,4 +1180,13 @@ struct StorageQueuingMetricsRequest {
|
|||
}
|
||||
};
|
||||
|
||||
// Memory size for storing mutation in the mutation log and the versioned map.
|
||||
inline int mvccStorageBytes(int mutationBytes) {
|
||||
// Why * 2:
|
||||
// - 1 insertion into version map costs 2 nodes in avg;
|
||||
// - The mutation will be stored in both mutation log and versioned map;
|
||||
return VersionedMap<KeyRef, ValueOrClearToRef>::overheadPerItem * 2 +
|
||||
(mutationBytes + MutationRef::OVERHEAD_BYTES) * 2;
|
||||
}
|
||||
|
||||
#endif
|
||||
|
|
|
@ -878,11 +878,11 @@ Peer::Peer(TransportData* transport, NetworkAddress const& destination)
|
|||
: transport(transport), destination(destination), compatible(true), outgoingConnectionIdle(true),
|
||||
lastConnectTime(0.0), reconnectionDelay(FLOW_KNOBS->INITIAL_RECONNECTION_TIME), peerReferences(-1),
|
||||
bytesReceived(0), bytesSent(0), lastDataPacketSentTime(now()), outstandingReplies(0),
|
||||
pingLatencies(destination.isPublic() ? FLOW_KNOBS->PING_SAMPLE_AMOUNT : 1), lastLoggedTime(0.0),
|
||||
pingLatencies(destination.isPublic() ? FLOW_KNOBS->PING_SKETCH_ACCURACY : 0.1), lastLoggedTime(0.0),
|
||||
lastLoggedBytesReceived(0), lastLoggedBytesSent(0), timeoutCount(0),
|
||||
protocolVersion(Reference<AsyncVar<Optional<ProtocolVersion>>>(new AsyncVar<Optional<ProtocolVersion>>())),
|
||||
connectOutgoingCount(0), connectIncomingCount(0), connectFailedCount(0),
|
||||
connectLatencies(destination.isPublic() ? FLOW_KNOBS->NETWORK_CONNECT_SAMPLE_AMOUNT : 1) {
|
||||
connectLatencies(destination.isPublic() ? FLOW_KNOBS->PING_SKETCH_ACCURACY : 0.1) {
|
||||
IFailureMonitor::failureMonitor().setStatus(destination, FailureStatus(false));
|
||||
}
|
||||
|
||||
|
|
|
@ -62,15 +62,15 @@ public:
|
|||
LatencySample readLatencySample = { "AsyncFileKAIOReadLatency",
|
||||
UID(),
|
||||
FLOW_KNOBS->KAIO_LATENCY_LOGGING_INTERVAL,
|
||||
FLOW_KNOBS->KAIO_LATENCY_SAMPLE_SIZE };
|
||||
FLOW_KNOBS->KAIO_LATENCY_SKETCH_ACCURACY };
|
||||
LatencySample writeLatencySample = { "AsyncFileKAIOWriteLatency",
|
||||
UID(),
|
||||
FLOW_KNOBS->KAIO_LATENCY_LOGGING_INTERVAL,
|
||||
FLOW_KNOBS->KAIO_LATENCY_SAMPLE_SIZE };
|
||||
FLOW_KNOBS->KAIO_LATENCY_SKETCH_ACCURACY };
|
||||
LatencySample syncLatencySample = { "AsyncFileKAIOSyncLatency",
|
||||
UID(),
|
||||
FLOW_KNOBS->KAIO_LATENCY_LOGGING_INTERVAL,
|
||||
FLOW_KNOBS->KAIO_LATENCY_SAMPLE_SIZE };
|
||||
FLOW_KNOBS->KAIO_LATENCY_SKETCH_ACCURACY };
|
||||
};
|
||||
|
||||
static AsyncFileKAIOMetrics& getMetrics() {
|
||||
|
|
|
@ -0,0 +1,326 @@
|
|||
/*
|
||||
* DDSketch.h
|
||||
*
|
||||
* This source file is part of the FoundationDB open source project
|
||||
*
|
||||
* Copyright 2013-2020 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 DDSKETCH_H
|
||||
#define DDSKETCH_H
|
||||
#include <iterator>
|
||||
#include <limits>
|
||||
#include <type_traits>
|
||||
#pragma once
|
||||
|
||||
#include <vector>
|
||||
#include <algorithm>
|
||||
#include <cassert>
|
||||
#include <cmath>
|
||||
#include "flow/Error.h"
|
||||
#include "flow/UnitTest.h"
|
||||
|
||||
// A namespace for fast log() computation.
|
||||
namespace fastLogger {
|
||||
// Basically, the goal is to compute log(x)/log(r).
|
||||
// For double, it is represented as 2^e*(1+s) (0<=s<1), so our goal becomes
|
||||
// e*log(2)/log(r)*log(1+s), and we approximate log(1+s) with a cubic function.
|
||||
// See more details on Datadog's paper, or CubicallyInterpolatedMapping.java in
|
||||
// https://github.com/DataDog/sketches-java/
|
||||
inline const double correctingFactor = 1.00988652862227438516; // = 7 / (10 * log(2));
|
||||
constexpr inline const double A = 6.0 / 35.0, B = -3.0 / 5.0, C = 10.0 / 7.0;
|
||||
|
||||
inline double fastlog(double value) {
|
||||
int e;
|
||||
double s = frexp(value, &e);
|
||||
s = s * 2 - 1;
|
||||
return ((A * s + B) * s + C) * s + e - 1;
|
||||
}
|
||||
|
||||
inline double reverseLog(double index) {
|
||||
long exponent = floor(index);
|
||||
// Derived from Cardano's formula
|
||||
double d0 = B * B - 3 * A * C;
|
||||
double d1 = 2 * B * B * B - 9 * A * B * C - 27 * A * A * (index - exponent);
|
||||
double p = cbrt((d1 - sqrt(d1 * d1 - 4 * d0 * d0 * d0)) / 2);
|
||||
double significandPlusOne = -(B + p + d0 / p) / (3 * A) + 1;
|
||||
return ldexp(significandPlusOne / 2, exponent + 1);
|
||||
}
|
||||
}; // namespace fastLogger
|
||||
|
||||
// DDSketch for non-negative numbers (those < EPS = 10^-18 are
|
||||
// treated as 0, and huge numbers (>1/EPS) fail ASSERT). This is the base
|
||||
// class without a concrete log() implementation.
|
||||
template <class Impl, class T>
|
||||
class DDSketchBase {
|
||||
|
||||
static constexpr T defaultMin() { return std::numeric_limits<T>::max(); }
|
||||
|
||||
static constexpr T defaultMax() {
|
||||
if constexpr (std::is_floating_point_v<T>) {
|
||||
return -std::numeric_limits<T>::max();
|
||||
} else {
|
||||
return std::numeric_limits<T>::min();
|
||||
}
|
||||
}
|
||||
|
||||
public:
|
||||
explicit DDSketchBase(double errorGuarantee)
|
||||
: errorGuarantee(errorGuarantee), populationSize(0), zeroPopulationSize(0), minValue(defaultMin()),
|
||||
maxValue(defaultMax()), sum(T()) {
|
||||
ASSERT(errorGuarantee > 0 && errorGuarantee < 1);
|
||||
}
|
||||
|
||||
DDSketchBase<Impl, T>& addSample(T sample) {
|
||||
// Call it addSample for now, while it is not a sample anymore
|
||||
if (!populationSize)
|
||||
minValue = maxValue = sample;
|
||||
|
||||
if (sample <= EPS) {
|
||||
zeroPopulationSize++;
|
||||
} else {
|
||||
size_t index = static_cast<Impl*>(this)->getIndex(sample);
|
||||
ASSERT(index >= 0 && index < buckets.size());
|
||||
try {
|
||||
buckets.at(index)++;
|
||||
} catch (std::out_of_range const& e) {
|
||||
fmt::print(stderr,
|
||||
"ERROR: Invalid DDSketch bucket index ({}) at {}/{} for sample: {}\n",
|
||||
e.what(),
|
||||
index,
|
||||
buckets.size(),
|
||||
sample);
|
||||
}
|
||||
}
|
||||
|
||||
populationSize++;
|
||||
sum += sample;
|
||||
maxValue = std::max(maxValue, sample);
|
||||
minValue = std::min(minValue, sample);
|
||||
return *this;
|
||||
}
|
||||
|
||||
double mean() const {
|
||||
if (populationSize == 0)
|
||||
return 0;
|
||||
return (double)sum / populationSize;
|
||||
}
|
||||
|
||||
T median() { return percentile(0.5); }
|
||||
|
||||
T percentile(double percentile) {
|
||||
ASSERT(percentile >= 0 && percentile <= 1);
|
||||
|
||||
if (populationSize == 0)
|
||||
return T();
|
||||
uint64_t targetPercentilePopulation = percentile * (populationSize - 1);
|
||||
// Now find the tPP-th (0-indexed) element
|
||||
if (targetPercentilePopulation < zeroPopulationSize)
|
||||
return T(0);
|
||||
|
||||
size_t index = 0;
|
||||
[[maybe_unused]] bool found = false;
|
||||
if (percentile <= 0.5) { // count up
|
||||
uint64_t count = zeroPopulationSize;
|
||||
for (size_t i = 0; i < buckets.size(); i++) {
|
||||
if (targetPercentilePopulation < count + buckets[i]) {
|
||||
// count + buckets[i] = # of numbers so far (from the rightmost to
|
||||
// this bucket, inclusive), so if target is in this bucket, it should
|
||||
// means tPP < cnt + bck[i]
|
||||
found = true;
|
||||
index = i;
|
||||
break;
|
||||
}
|
||||
count += buckets[i];
|
||||
}
|
||||
} else { // and count down
|
||||
uint64_t count = 0;
|
||||
for (auto rit = buckets.rbegin(); rit != buckets.rend(); rit++) {
|
||||
if (targetPercentilePopulation + count + *rit >= populationSize) {
|
||||
// cnt + bkt[i] is # of numbers to the right of this bucket (incl.),
|
||||
// so if target is not in this bucket (i.e., to the left of this
|
||||
// bucket), it would be as right as the left bucket's rightmost
|
||||
// number, so we would have tPP + cnt + bkt[i] < total population (tPP
|
||||
// is 0-indexed), that means target is in this bucket if this
|
||||
// condition is not satisfied.
|
||||
found = true;
|
||||
index = std::distance(rit, buckets.rend()) - 1;
|
||||
break;
|
||||
}
|
||||
count += *rit;
|
||||
}
|
||||
}
|
||||
ASSERT(found);
|
||||
if (!found)
|
||||
return -1;
|
||||
return static_cast<Impl*>(this)->getValue(index);
|
||||
}
|
||||
|
||||
T min() const { return minValue; }
|
||||
T max() const { return maxValue; }
|
||||
|
||||
void clear() {
|
||||
std::fill(buckets.begin(), buckets.end(), 0);
|
||||
populationSize = zeroPopulationSize = 0;
|
||||
sum = 0;
|
||||
minValue = defaultMin();
|
||||
maxValue = defaultMax();
|
||||
}
|
||||
|
||||
uint64_t getPopulationSize() const { return populationSize; }
|
||||
|
||||
double getErrorGuarantee() const { return errorGuarantee; }
|
||||
|
||||
size_t getBucketSize() const { return buckets.size(); }
|
||||
|
||||
DDSketchBase<Impl, T>& mergeWith(const DDSketchBase<Impl, T>& anotherSketch) {
|
||||
// Must have the same guarantee
|
||||
ASSERT(fabs(errorGuarantee - anotherSketch.errorGuarantee) < EPS &&
|
||||
anotherSketch.buckets.size() == buckets.size());
|
||||
for (size_t i = 0; i < anotherSketch.buckets.size(); i++) {
|
||||
buckets[i] += anotherSketch.buckets[i];
|
||||
}
|
||||
populationSize += anotherSketch.populationSize;
|
||||
zeroPopulationSize += anotherSketch.zeroPopulationSize;
|
||||
minValue = std::min(minValue, anotherSketch.minValue);
|
||||
maxValue = std::max(maxValue, anotherSketch.maxValue);
|
||||
sum += anotherSketch.sum;
|
||||
return *this;
|
||||
}
|
||||
|
||||
constexpr static double EPS = 1e-18; // smaller numbers are considered as 0
|
||||
protected:
|
||||
double errorGuarantee; // As defined in the paper
|
||||
|
||||
uint64_t populationSize, zeroPopulationSize; // we need to separately count 0s
|
||||
std::vector<uint64_t> buckets;
|
||||
T minValue, maxValue, sum;
|
||||
void setBucketSize(size_t capacity) { buckets.resize(capacity, 0); }
|
||||
};
|
||||
|
||||
// DDSketch with fast log implementation for float numbers
|
||||
template <class T>
|
||||
class DDSketch : public DDSketchBase<DDSketch<T>, T> {
|
||||
public:
|
||||
explicit DDSketch(double errorGuarantee = 0.01)
|
||||
: DDSketchBase<DDSketch<T>, T>(errorGuarantee), gamma((1.0 + errorGuarantee) / (1.0 - errorGuarantee)),
|
||||
multiplier(fastLogger::correctingFactor * log(2) / log(gamma)) {
|
||||
ASSERT(errorGuarantee > 0);
|
||||
offset = getIndex(1.0 / DDSketchBase<DDSketch<T>, T>::EPS);
|
||||
ASSERT(offset > 0);
|
||||
this->setBucketSize(2 * offset);
|
||||
}
|
||||
|
||||
size_t getIndex(T sample) {
|
||||
static_assert(__BYTE_ORDER__ == __ORDER_LITTLE_ENDIAN__, "Do not support non-little-endian systems");
|
||||
return ceil(fastLogger::fastlog(sample) * multiplier) + offset;
|
||||
}
|
||||
|
||||
T getValue(size_t index) { return fastLogger::reverseLog((index - offset) / multiplier) * 2.0 / (1 + gamma); }
|
||||
|
||||
private:
|
||||
double gamma, multiplier;
|
||||
size_t offset = 0;
|
||||
};
|
||||
|
||||
// DDSketch with <cmath> log. Slow and only use this when others doesn't work.
|
||||
template <class T>
|
||||
class DDSketchSlow : public DDSketchBase<DDSketchSlow<T>, T> {
|
||||
public:
|
||||
DDSketchSlow(double errorGuarantee = 0.1)
|
||||
: DDSketchBase<DDSketchSlow<T>, T>(errorGuarantee), gamma((1.0 + errorGuarantee) / (1.0 - errorGuarantee)),
|
||||
logGamma(log(gamma)) {
|
||||
offset = getIndex(1.0 / DDSketchBase<DDSketch<T>, T>::EPS) + 5;
|
||||
this->setBucketSize(2 * offset);
|
||||
}
|
||||
|
||||
size_t getIndex(T sample) { return ceil(log(sample) / logGamma) + offset; }
|
||||
|
||||
T getValue(size_t index) { return (T)(2.0 * pow(gamma, (index - offset)) / (1 + gamma)); }
|
||||
|
||||
private:
|
||||
double gamma, logGamma;
|
||||
size_t offset = 0;
|
||||
};
|
||||
|
||||
// DDSketch for unsigned int. Faster than the float version. Fixed accuracy.
|
||||
class DDSketchFastUnsigned : public DDSketchBase<DDSketchFastUnsigned, unsigned> {
|
||||
public:
|
||||
DDSketchFastUnsigned() : DDSketchBase<DDSketchFastUnsigned, unsigned>(errorGuarantee) { this->setBucketSize(129); }
|
||||
|
||||
size_t getIndex(unsigned sample) {
|
||||
__uint128_t v = sample;
|
||||
v *= v;
|
||||
v *= v; // sample^4
|
||||
uint64_t low = (uint64_t)v, high = (uint64_t)(v >> 64);
|
||||
|
||||
return 128 - (high == 0 ? ((low == 0 ? 64 : __builtin_clzll(low)) + 64) : __builtin_clzll(high));
|
||||
}
|
||||
|
||||
unsigned getValue(size_t index) {
|
||||
double r = 1, g = gamma;
|
||||
while (index) { // quick power method for power(gamma, index)
|
||||
if (index & 1)
|
||||
r *= g;
|
||||
g *= g;
|
||||
index >>= 1;
|
||||
}
|
||||
// 2.0 * pow(gamma, index) / (1 + gamma) is what we need
|
||||
return (unsigned)(2.0 * r / (1 + gamma) + 0.5); // round to nearest int
|
||||
}
|
||||
|
||||
private:
|
||||
constexpr static double errorGuarantee = 0.08642723372;
|
||||
// getIndex basically calc floor(log_2(x^4)) + 1,
|
||||
// which is almost ceil(log_2(x^4)) as it only matters when x is a power of 2,
|
||||
// and it does not change the error bound. Original sketch asks for
|
||||
// ceil(log_r(x)), so we know r = pow(2, 1/4) = 1.189207115. And r = (1 + eG)
|
||||
// / (1 - eG) so eG = 0.08642723372.
|
||||
constexpr static double gamma = 1.189207115;
|
||||
};
|
||||
|
||||
#endif
|
||||
|
||||
TEST_CASE("/fdbrpc/ddsketch/accuracy") {
|
||||
|
||||
int TRY = 100, SIZE = 1e6;
|
||||
const int totalPercentiles = 7;
|
||||
double targetPercentiles[totalPercentiles] = { .0001, .01, .1, .50, .90, .99, .9999 };
|
||||
double stat[totalPercentiles] = { 0 };
|
||||
for (int t = 0; t < TRY; t++) {
|
||||
DDSketch<double> dd;
|
||||
std::vector<double> nums;
|
||||
for (int i = 0; i < SIZE; i++) {
|
||||
static double a = 1, b = 1; // a skewed distribution
|
||||
auto y = deterministicRandom()->random01();
|
||||
auto num = b / pow(1 - y, 1 / a);
|
||||
nums.push_back(num);
|
||||
dd.addSample(num);
|
||||
}
|
||||
std::sort(nums.begin(), nums.end());
|
||||
for (int percentID = 0; percentID < totalPercentiles; percentID++) {
|
||||
double percentile = targetPercentiles[percentID];
|
||||
double ground = nums[percentile * (SIZE - 1)], ddvalue = dd.percentile(percentile);
|
||||
double relativeError = fabs(ground - ddvalue) / ground;
|
||||
stat[percentID] += relativeError;
|
||||
}
|
||||
}
|
||||
|
||||
for (int percentID = 0; percentID < totalPercentiles; percentID++) {
|
||||
printf("%.4lf per, relative error %.4lf\n", targetPercentiles[percentID], stat[percentID] / TRY);
|
||||
}
|
||||
|
||||
return Void();
|
||||
}
|
|
@ -24,7 +24,7 @@
|
|||
|
||||
#include <algorithm>
|
||||
|
||||
#include "fdbrpc/ContinuousSample.h"
|
||||
#include "fdbrpc/DDSketch.h"
|
||||
#include "fdbrpc/HealthMonitor.h"
|
||||
#include "flow/genericactors.actor.h"
|
||||
#include "flow/network.h"
|
||||
|
@ -159,7 +159,7 @@ struct Peer : public ReferenceCounted<Peer> {
|
|||
int64_t bytesSent;
|
||||
double lastDataPacketSentTime;
|
||||
int outstandingReplies;
|
||||
ContinuousSample<double> pingLatencies;
|
||||
DDSketch<double> pingLatencies;
|
||||
double lastLoggedTime;
|
||||
int64_t lastLoggedBytesReceived;
|
||||
int64_t lastLoggedBytesSent;
|
||||
|
@ -171,7 +171,7 @@ struct Peer : public ReferenceCounted<Peer> {
|
|||
int connectOutgoingCount;
|
||||
int connectIncomingCount;
|
||||
int connectFailedCount;
|
||||
ContinuousSample<double> connectLatencies;
|
||||
DDSketch<double> connectLatencies;
|
||||
Promise<Void> disconnect;
|
||||
|
||||
explicit Peer(TransportData* transport, NetworkAddress const& destination);
|
||||
|
|
|
@ -38,7 +38,7 @@ MyCounters() : foo("foo", cc), bar("bar", cc), baz("baz", cc) {}
|
|||
#include <cstddef>
|
||||
#include "flow/flow.h"
|
||||
#include "flow/TDMetric.actor.h"
|
||||
#include "fdbrpc/ContinuousSample.h"
|
||||
#include "fdbrpc/DDSketch.h"
|
||||
|
||||
struct ICounter {
|
||||
// All counters have a name and value
|
||||
|
@ -216,40 +216,44 @@ public:
|
|||
|
||||
class LatencySample {
|
||||
public:
|
||||
LatencySample(std::string name, UID id, double loggingInterval, int sampleSize)
|
||||
: name(name), id(id), sampleStart(now()), sample(sampleSize),
|
||||
LatencySample(std::string name, UID id, double loggingInterval, double accuracy)
|
||||
: name(name), id(id), sampleStart(now()), sketch(accuracy),
|
||||
latencySampleEventHolder(makeReference<EventCacheHolder>(id.toString() + "/" + name)) {
|
||||
assert(accuracy > 0);
|
||||
if (accuracy <= 0) {
|
||||
fmt::print(stderr, "ERROR: LatencySample {} has invalid accuracy ({})", name, accuracy);
|
||||
}
|
||||
logger = recurring([this]() { logSample(); }, loggingInterval);
|
||||
}
|
||||
|
||||
void addMeasurement(double measurement) { sample.addSample(measurement); }
|
||||
void addMeasurement(double measurement) { sketch.addSample(measurement); }
|
||||
|
||||
private:
|
||||
std::string name;
|
||||
UID id;
|
||||
double sampleStart;
|
||||
|
||||
ContinuousSample<double> sample;
|
||||
DDSketch<double> sketch;
|
||||
Future<Void> logger;
|
||||
|
||||
Reference<EventCacheHolder> latencySampleEventHolder;
|
||||
|
||||
void logSample() {
|
||||
TraceEvent(name.c_str(), id)
|
||||
.detail("Count", sample.getPopulationSize())
|
||||
.detail("Count", sketch.getPopulationSize())
|
||||
.detail("Elapsed", now() - sampleStart)
|
||||
.detail("Min", sample.min())
|
||||
.detail("Max", sample.max())
|
||||
.detail("Mean", sample.mean())
|
||||
.detail("Median", sample.median())
|
||||
.detail("P25", sample.percentile(0.25))
|
||||
.detail("P90", sample.percentile(0.9))
|
||||
.detail("P95", sample.percentile(0.95))
|
||||
.detail("P99", sample.percentile(0.99))
|
||||
.detail("P99.9", sample.percentile(0.999))
|
||||
.detail("Min", sketch.min())
|
||||
.detail("Max", sketch.max())
|
||||
.detail("Mean", sketch.mean())
|
||||
.detail("Median", sketch.median())
|
||||
.detail("P25", sketch.percentile(0.25))
|
||||
.detail("P90", sketch.percentile(0.9))
|
||||
.detail("P95", sketch.percentile(0.95))
|
||||
.detail("P99", sketch.percentile(0.99))
|
||||
.detail("P99.9", sketch.percentile(0.999))
|
||||
.trackLatest(latencySampleEventHolder->trackingKey);
|
||||
|
||||
sample.clear();
|
||||
sketch.clear();
|
||||
sampleStart = now();
|
||||
}
|
||||
};
|
||||
|
|
|
@ -25,7 +25,6 @@
|
|||
#ifndef FDBRPC_TSS_COMPARISON_H
|
||||
#define FDBRPC_TSS_COMPARISON_H
|
||||
|
||||
#include "fdbrpc/ContinuousSample.h"
|
||||
#include "fdbrpc/Stats.h"
|
||||
|
||||
// refcounted + noncopyable because both DatabaseContext and individual endpoints share ownership
|
||||
|
@ -48,15 +47,15 @@ struct TSSMetrics : ReferenceCounted<TSSMetrics>, NonCopyable {
|
|||
Counter mismatches;
|
||||
|
||||
// We could probably just ignore getKey as it's seldom used?
|
||||
ContinuousSample<double> SSgetValueLatency;
|
||||
ContinuousSample<double> SSgetKeyLatency;
|
||||
ContinuousSample<double> SSgetKeyValuesLatency;
|
||||
ContinuousSample<double> SSgetMappedKeyValuesLatency;
|
||||
DDSketch<double> SSgetValueLatency;
|
||||
DDSketch<double> SSgetKeyLatency;
|
||||
DDSketch<double> SSgetKeyValuesLatency;
|
||||
DDSketch<double> SSgetMappedKeyValuesLatency;
|
||||
|
||||
ContinuousSample<double> TSSgetValueLatency;
|
||||
ContinuousSample<double> TSSgetKeyLatency;
|
||||
ContinuousSample<double> TSSgetKeyValuesLatency;
|
||||
ContinuousSample<double> TSSgetMappedKeyValuesLatency;
|
||||
DDSketch<double> TSSgetValueLatency;
|
||||
DDSketch<double> TSSgetKeyLatency;
|
||||
DDSketch<double> TSSgetKeyValuesLatency;
|
||||
DDSketch<double> TSSgetMappedKeyValuesLatency;
|
||||
|
||||
std::unordered_map<int, uint64_t> ssErrorsByCode;
|
||||
std::unordered_map<int, uint64_t> tssErrorsByCode;
|
||||
|
@ -106,9 +105,9 @@ struct TSSMetrics : ReferenceCounted<TSSMetrics>, NonCopyable {
|
|||
TSSMetrics()
|
||||
: 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),
|
||||
SSgetMappedKeyValuesLatency(1000), TSSgetValueLatency(1000), TSSgetKeyLatency(1000),
|
||||
TSSgetKeyValuesLatency(1000), TSSgetMappedKeyValuesLatency(1000) {}
|
||||
mismatches("Mismatches", cc), SSgetValueLatency(), SSgetKeyLatency(), SSgetKeyValuesLatency(),
|
||||
SSgetMappedKeyValuesLatency(), TSSgetValueLatency(), TSSgetKeyLatency(), TSSgetKeyValuesLatency(),
|
||||
TSSgetMappedKeyValuesLatency() {}
|
||||
};
|
||||
|
||||
template <class Rep>
|
||||
|
|
|
@ -638,11 +638,12 @@ ACTOR Future<BlobGranuleSplitPoints> splitRange(Reference<BlobManagerData> bmDat
|
|||
// only split on bytes and write rate
|
||||
state StorageMetrics splitMetrics;
|
||||
splitMetrics.bytes = SERVER_KNOBS->BG_SNAPSHOT_FILE_TARGET_BYTES;
|
||||
splitMetrics.bytesPerKSecond = SERVER_KNOBS->SHARD_SPLIT_BYTES_PER_KSEC;
|
||||
splitMetrics.bytesWrittenPerKSecond = SERVER_KNOBS->SHARD_SPLIT_BYTES_PER_KSEC;
|
||||
if (writeHot) {
|
||||
splitMetrics.bytesPerKSecond = std::min(splitMetrics.bytesPerKSecond, estimated.bytesPerKSecond / 2);
|
||||
splitMetrics.bytesPerKSecond =
|
||||
std::max(splitMetrics.bytesPerKSecond, SERVER_KNOBS->SHARD_MIN_BYTES_PER_KSEC);
|
||||
splitMetrics.bytesWrittenPerKSecond =
|
||||
std::min(splitMetrics.bytesWrittenPerKSecond, estimated.bytesWrittenPerKSecond / 2);
|
||||
splitMetrics.bytesWrittenPerKSecond =
|
||||
std::max(splitMetrics.bytesWrittenPerKSecond, SERVER_KNOBS->SHARD_MIN_BYTES_PER_KSEC);
|
||||
}
|
||||
splitMetrics.iosPerKSecond = splitMetrics.infinity;
|
||||
splitMetrics.bytesReadPerKSecond = splitMetrics.infinity;
|
||||
|
@ -2618,7 +2619,7 @@ ACTOR Future<Void> attemptMerges(Reference<BlobManagerData> bmData,
|
|||
wait(bmData->db->getStorageMetrics(std::get<1>(candidates[i]), CLIENT_KNOBS->TOO_MANY));
|
||||
|
||||
if (metrics.bytes >= SERVER_KNOBS->BG_SNAPSHOT_FILE_TARGET_BYTES ||
|
||||
metrics.bytesPerKSecond >= SERVER_KNOBS->SHARD_MIN_BYTES_PER_KSEC) {
|
||||
metrics.bytesWrittenPerKSecond >= SERVER_KNOBS->SHARD_MIN_BYTES_PER_KSEC) {
|
||||
// This granule cannot be merged with any neighbors.
|
||||
// If current candidates up to here can be merged, merge them and skip over this one
|
||||
attemptStartMerge(bmData, currentCandidates);
|
||||
|
|
|
@ -305,8 +305,8 @@ struct BlobWorkerData : NonCopyable, ReferenceCounted<BlobWorkerData> {
|
|||
resnapshotLock,
|
||||
deltaWritesLock,
|
||||
SERVER_KNOBS->LATENCY_METRICS_LOGGING_INTERVAL,
|
||||
SERVER_KNOBS->FILE_LATENCY_SAMPLE_SIZE,
|
||||
SERVER_KNOBS->LATENCY_SAMPLE_SIZE),
|
||||
SERVER_KNOBS->FILE_LATENCY_SKETCH_ACCURACY,
|
||||
SERVER_KNOBS->LATENCY_SKETCH_ACCURACY),
|
||||
isEncryptionEnabled(isEncryptionOpSupported(EncryptOperationType::BLOB_GRANULE_ENCRYPTION)) {}
|
||||
|
||||
bool managerEpochOk(int64_t epoch) {
|
||||
|
@ -1645,7 +1645,7 @@ ACTOR Future<Void> granuleCheckMergeCandidate(Reference<BlobWorkerData> bwData,
|
|||
|
||||
// FIXME: maybe separate knob and/or value for write rate?
|
||||
if (currentMetrics.bytes >= SERVER_KNOBS->BG_SNAPSHOT_FILE_TARGET_BYTES / 2 ||
|
||||
currentMetrics.bytesPerKSecond >= SERVER_KNOBS->SHARD_MIN_BYTES_PER_KSEC) {
|
||||
currentMetrics.bytesWrittenPerKSecond >= SERVER_KNOBS->SHARD_MIN_BYTES_PER_KSEC) {
|
||||
wait(delayJittered(SERVER_KNOBS->BG_MERGE_CANDIDATE_THRESHOLD_SECONDS / 2.0));
|
||||
CODE_PROBE(true, "wait and check later to see if granule got smaller or colder");
|
||||
continue;
|
||||
|
|
|
@ -1548,14 +1548,20 @@ ACTOR Future<Void> dataDistributionRelocator(DDQueue* self,
|
|||
if (enableShardMove && tciIndex == 1) {
|
||||
ASSERT(physicalShardIDCandidate != UID().first() &&
|
||||
physicalShardIDCandidate != anonymousShardId.first());
|
||||
Optional<ShardsAffectedByTeamFailure::Team> remoteTeamWithPhysicalShard =
|
||||
std::pair<Optional<ShardsAffectedByTeamFailure::Team>, bool> remoteTeamWithPhysicalShard =
|
||||
self->physicalShardCollection->tryGetAvailableRemoteTeamWith(
|
||||
physicalShardIDCandidate, metrics, debugID);
|
||||
// TODO: when we know that `physicalShardIDCandidate` exists, remote team must also exists.
|
||||
if (remoteTeamWithPhysicalShard.present()) {
|
||||
if (!remoteTeamWithPhysicalShard.second) {
|
||||
// Physical shard with `physicalShardIDCandidate` is not available. Retry selecting new
|
||||
// dst physical shard.
|
||||
self->retryFindDstReasonCount[DDQueue::RetryFindDstReason::NoAvailablePhysicalShard]++;
|
||||
foundTeams = false;
|
||||
break;
|
||||
}
|
||||
if (remoteTeamWithPhysicalShard.first.present()) {
|
||||
// Exists a remoteTeam in the mapping that has the physicalShardIDCandidate
|
||||
// use the remoteTeam with the physicalShard as the bestTeam
|
||||
req = GetTeamRequest(remoteTeamWithPhysicalShard.get().servers);
|
||||
req = GetTeamRequest(remoteTeamWithPhysicalShard.first.get().servers);
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -41,9 +41,9 @@ enum BandwidthStatus { BandwidthStatusLow, BandwidthStatusNormal, BandwidthStatu
|
|||
enum ReadBandwidthStatus { ReadBandwidthStatusNormal, ReadBandwidthStatusHigh };
|
||||
|
||||
BandwidthStatus getBandwidthStatus(StorageMetrics const& metrics) {
|
||||
if (metrics.bytesPerKSecond > SERVER_KNOBS->SHARD_MAX_BYTES_PER_KSEC)
|
||||
if (metrics.bytesWrittenPerKSecond > SERVER_KNOBS->SHARD_MAX_BYTES_PER_KSEC)
|
||||
return BandwidthStatusHigh;
|
||||
else if (metrics.bytesPerKSecond < SERVER_KNOBS->SHARD_MIN_BYTES_PER_KSEC)
|
||||
else if (metrics.bytesWrittenPerKSecond < SERVER_KNOBS->SHARD_MIN_BYTES_PER_KSEC)
|
||||
return BandwidthStatusLow;
|
||||
|
||||
return BandwidthStatusNormal;
|
||||
|
@ -176,7 +176,7 @@ ShardSizeBounds getShardSizeBounds(KeyRangeRef shard, int64_t maxShardSize) {
|
|||
bounds.max.bytes = maxShardSize;
|
||||
}
|
||||
|
||||
bounds.max.bytesPerKSecond = bounds.max.infinity;
|
||||
bounds.max.bytesWrittenPerKSecond = bounds.max.infinity;
|
||||
bounds.max.iosPerKSecond = bounds.max.infinity;
|
||||
bounds.max.bytesReadPerKSecond = bounds.max.infinity;
|
||||
|
||||
|
@ -187,14 +187,14 @@ ShardSizeBounds getShardSizeBounds(KeyRangeRef shard, int64_t maxShardSize) {
|
|||
bounds.min.bytes = maxShardSize / SERVER_KNOBS->SHARD_BYTES_RATIO;
|
||||
}
|
||||
|
||||
bounds.min.bytesPerKSecond = 0;
|
||||
bounds.min.bytesWrittenPerKSecond = 0;
|
||||
bounds.min.iosPerKSecond = 0;
|
||||
bounds.min.bytesReadPerKSecond = 0;
|
||||
|
||||
// The permitted error is 1/3 of the general-case minimum bytes (even in the special case where this is the last
|
||||
// shard)
|
||||
bounds.permittedError.bytes = bounds.max.bytes / SERVER_KNOBS->SHARD_BYTES_RATIO / 3;
|
||||
bounds.permittedError.bytesPerKSecond = bounds.permittedError.infinity;
|
||||
bounds.permittedError.bytesWrittenPerKSecond = bounds.permittedError.infinity;
|
||||
bounds.permittedError.iosPerKSecond = bounds.permittedError.infinity;
|
||||
bounds.permittedError.bytesReadPerKSecond = bounds.permittedError.infinity;
|
||||
|
||||
|
@ -222,18 +222,18 @@ ShardSizeBounds calculateShardSizeBounds(const KeyRange& keys,
|
|||
std::max(int64_t(bytes - (SERVER_KNOBS->MIN_SHARD_BYTES * 0.1)), (int64_t)0));
|
||||
bounds.permittedError.bytes = bytes * 0.1;
|
||||
if (bandwidthStatus == BandwidthStatusNormal) { // Not high or low
|
||||
bounds.max.bytesPerKSecond = SERVER_KNOBS->SHARD_MAX_BYTES_PER_KSEC;
|
||||
bounds.min.bytesPerKSecond = SERVER_KNOBS->SHARD_MIN_BYTES_PER_KSEC;
|
||||
bounds.permittedError.bytesPerKSecond = bounds.min.bytesPerKSecond / 4;
|
||||
bounds.max.bytesWrittenPerKSecond = SERVER_KNOBS->SHARD_MAX_BYTES_PER_KSEC;
|
||||
bounds.min.bytesWrittenPerKSecond = SERVER_KNOBS->SHARD_MIN_BYTES_PER_KSEC;
|
||||
bounds.permittedError.bytesWrittenPerKSecond = bounds.min.bytesWrittenPerKSecond / 4;
|
||||
} else if (bandwidthStatus == BandwidthStatusHigh) { // > 10MB/sec for 100MB shard, proportionally lower
|
||||
// for smaller shard, > 200KB/sec no matter what
|
||||
bounds.max.bytesPerKSecond = bounds.max.infinity;
|
||||
bounds.min.bytesPerKSecond = SERVER_KNOBS->SHARD_MAX_BYTES_PER_KSEC;
|
||||
bounds.permittedError.bytesPerKSecond = bounds.min.bytesPerKSecond / 4;
|
||||
bounds.max.bytesWrittenPerKSecond = bounds.max.infinity;
|
||||
bounds.min.bytesWrittenPerKSecond = SERVER_KNOBS->SHARD_MAX_BYTES_PER_KSEC;
|
||||
bounds.permittedError.bytesWrittenPerKSecond = bounds.min.bytesWrittenPerKSecond / 4;
|
||||
} else if (bandwidthStatus == BandwidthStatusLow) { // < 10KB/sec
|
||||
bounds.max.bytesPerKSecond = SERVER_KNOBS->SHARD_MIN_BYTES_PER_KSEC;
|
||||
bounds.min.bytesPerKSecond = 0;
|
||||
bounds.permittedError.bytesPerKSecond = bounds.max.bytesPerKSecond / 4;
|
||||
bounds.max.bytesWrittenPerKSecond = SERVER_KNOBS->SHARD_MIN_BYTES_PER_KSEC;
|
||||
bounds.min.bytesWrittenPerKSecond = 0;
|
||||
bounds.permittedError.bytesWrittenPerKSecond = bounds.max.bytesWrittenPerKSecond / 4;
|
||||
} else {
|
||||
ASSERT(false);
|
||||
}
|
||||
|
@ -306,12 +306,12 @@ ACTOR Future<Void> trackShardMetrics(DataDistributionTracker::SafeAccessor self,
|
|||
/*TraceEvent("ShardSizeUpdate")
|
||||
.detail("Keys", keys)
|
||||
.detail("UpdatedSize", metrics.metrics.bytes)
|
||||
.detail("Bandwidth", metrics.metrics.bytesPerKSecond)
|
||||
.detail("WriteBandwidth", metrics.metrics.bytesWrittenPerKSecond)
|
||||
.detail("BandwidthStatus", getBandwidthStatus(metrics))
|
||||
.detail("BytesLower", bounds.min.bytes)
|
||||
.detail("BytesUpper", bounds.max.bytes)
|
||||
.detail("BandwidthLower", bounds.min.bytesPerKSecond)
|
||||
.detail("BandwidthUpper", bounds.max.bytesPerKSecond)
|
||||
.detail("WriteBandwidthLower", bounds.min.bytesWrittenPerKSecond)
|
||||
.detail("WriteBandwidthUpper", bounds.max.bytesWrittenPerKSecond)
|
||||
.detail("ShardSizePresent", shardSize->get().present())
|
||||
.detail("OldShardSize", shardSize->get().present() ? shardSize->get().get().metrics.bytes : 0)
|
||||
.detail("TrackerID", trackerID);*/
|
||||
|
@ -881,7 +881,7 @@ ACTOR Future<Void> shardSplitter(DataDistributionTracker* self,
|
|||
|
||||
StorageMetrics splitMetrics;
|
||||
splitMetrics.bytes = shardBounds.max.bytes / 2;
|
||||
splitMetrics.bytesPerKSecond =
|
||||
splitMetrics.bytesWrittenPerKSecond =
|
||||
keys.begin >= keyServersKeys.begin ? splitMetrics.infinity : SERVER_KNOBS->SHARD_SPLIT_BYTES_PER_KSEC;
|
||||
splitMetrics.iosPerKSecond = splitMetrics.infinity;
|
||||
splitMetrics.bytesReadPerKSecond = splitMetrics.infinity; // Don't split by readBandwidth
|
||||
|
@ -904,7 +904,7 @@ ACTOR Future<Void> shardSplitter(DataDistributionTracker* self,
|
|||
bandwidthStatus == BandwidthStatusHigh ? "High"
|
||||
: bandwidthStatus == BandwidthStatusNormal ? "Normal"
|
||||
: "Low")
|
||||
.detail("BytesPerKSec", metrics.bytesPerKSecond)
|
||||
.detail("BytesWrittenPerKSec", metrics.bytesWrittenPerKSecond)
|
||||
.detail("NumShards", numShards);
|
||||
|
||||
if (numShards > 1) {
|
||||
|
@ -1205,7 +1205,7 @@ ACTOR Future<Void> shardTracker(DataDistributionTracker::SafeAccessor self,
|
|||
.detail("TrackerID", trackerID)
|
||||
.detail("MaxBytes", self()->maxShardSize->get().get())
|
||||
.detail("ShardSize", shardSize->get().get().bytes)
|
||||
.detail("BytesPerKSec", shardSize->get().get().bytesPerKSecond);*/
|
||||
.detail("BytesPerKSec", shardSize->get().get().bytesWrittenPerKSecond);*/
|
||||
|
||||
try {
|
||||
loop {
|
||||
|
@ -1756,7 +1756,7 @@ InOverSizePhysicalShard PhysicalShardCollection::isInOverSizePhysicalShard(KeyRa
|
|||
}
|
||||
|
||||
// May return a problematic remote team
|
||||
Optional<ShardsAffectedByTeamFailure::Team> PhysicalShardCollection::tryGetAvailableRemoteTeamWith(
|
||||
std::pair<Optional<ShardsAffectedByTeamFailure::Team>, bool> PhysicalShardCollection::tryGetAvailableRemoteTeamWith(
|
||||
uint64_t inputPhysicalShardID,
|
||||
StorageMetrics const& moveInMetrics,
|
||||
uint64_t debugID) {
|
||||
|
@ -1764,10 +1764,10 @@ Optional<ShardsAffectedByTeamFailure::Team> PhysicalShardCollection::tryGetAvail
|
|||
ASSERT(SERVER_KNOBS->ENABLE_DD_PHYSICAL_SHARD);
|
||||
ASSERT(inputPhysicalShardID != anonymousShardId.first() && inputPhysicalShardID != UID().first());
|
||||
if (physicalShardInstances.count(inputPhysicalShardID) == 0) {
|
||||
return Optional<ShardsAffectedByTeamFailure::Team>();
|
||||
return { Optional<ShardsAffectedByTeamFailure::Team>(), true };
|
||||
}
|
||||
if (!checkPhysicalShardAvailable(inputPhysicalShardID, moveInMetrics)) {
|
||||
return Optional<ShardsAffectedByTeamFailure::Team>();
|
||||
return { Optional<ShardsAffectedByTeamFailure::Team>(), false };
|
||||
}
|
||||
for (auto team : physicalShardInstances[inputPhysicalShardID].teams) {
|
||||
if (team.primary == false) {
|
||||
|
@ -1777,10 +1777,12 @@ Optional<ShardsAffectedByTeamFailure::Team> PhysicalShardCollection::tryGetAvail
|
|||
.detail("TeamSize", team.servers.size())
|
||||
.detail("PhysicalShardsOfTeam", convertIDsToString(teamPhysicalShardIDs[team]))
|
||||
.detail("DebugID", debugID);*/
|
||||
return team;
|
||||
return { team, true };
|
||||
}
|
||||
}
|
||||
UNREACHABLE();
|
||||
// In this case, the physical shard may not be populated in the remote region yet, e.g., we are making a
|
||||
// configuration change to turn a single region cluster into HA mode.
|
||||
return { Optional<ShardsAffectedByTeamFailure::Team>(), true };
|
||||
}
|
||||
|
||||
// The update of PhysicalShardToTeams, Collection, keyRangePhysicalShardIDMap should be atomic
|
||||
|
|
|
@ -693,27 +693,48 @@ Future<std::vector<ProcessData>> DDTxnProcessor::getWorkers() const {
|
|||
return ::getWorkers(cx);
|
||||
}
|
||||
|
||||
Future<Void> DDTxnProcessor::rawStartMovement(MoveKeysParams& params,
|
||||
Future<Void> DDTxnProcessor::rawStartMovement(const MoveKeysParams& params,
|
||||
std::map<UID, StorageServerInterface>& tssMapping) {
|
||||
return ::rawStartMovement(cx, params, tssMapping);
|
||||
}
|
||||
|
||||
Future<Void> DDTxnProcessor::rawFinishMovement(MoveKeysParams& params,
|
||||
Future<Void> DDTxnProcessor::rawFinishMovement(const MoveKeysParams& params,
|
||||
const std::map<UID, StorageServerInterface>& tssMapping) {
|
||||
return ::rawFinishMovement(cx, params, tssMapping);
|
||||
}
|
||||
|
||||
struct DDMockTxnProcessorImpl {
|
||||
ACTOR static Future<Void> moveKeys(DDMockTxnProcessor* self, MoveKeysParams params) {
|
||||
state std::map<UID, StorageServerInterface> tssMapping;
|
||||
self->rawStartMovement(params, tssMapping);
|
||||
ASSERT(tssMapping.empty());
|
||||
|
||||
// return when all status become FETCHED
|
||||
ACTOR static Future<Void> checkFetchingState(DDMockTxnProcessor* self, std::vector<UID> ids, KeyRangeRef range) {
|
||||
loop {
|
||||
wait(delayJittered(1.0));
|
||||
DDMockTxnProcessor* selfP = self;
|
||||
KeyRangeRef cloneRef = range;
|
||||
if (std::all_of(ids.begin(), ids.end(), [selfP, cloneRef](const UID& id) {
|
||||
auto& server = selfP->mgs->allServers.at(id);
|
||||
return server.allShardStatusIn(cloneRef, { MockShardStatus::FETCHED, MockShardStatus::COMPLETED });
|
||||
})) {
|
||||
break;
|
||||
}
|
||||
}
|
||||
if (BUGGIFY_WITH_PROB(0.5)) {
|
||||
wait(delayJittered(5.0));
|
||||
}
|
||||
return Void();
|
||||
}
|
||||
|
||||
self->rawFinishMovement(params, tssMapping);
|
||||
ACTOR static Future<Void> moveKeys(DDMockTxnProcessor* self, MoveKeysParams params) {
|
||||
state std::map<UID, StorageServerInterface> tssMapping;
|
||||
// Because SFBTF::Team requires the ID is ordered
|
||||
std::sort(params.destinationTeam.begin(), params.destinationTeam.end());
|
||||
std::sort(params.healthyDestinations.begin(), params.healthyDestinations.end());
|
||||
|
||||
wait(self->rawStartMovement(params, tssMapping));
|
||||
ASSERT(tssMapping.empty());
|
||||
|
||||
wait(checkFetchingState(self, params.destinationTeam, params.keys));
|
||||
|
||||
wait(self->rawFinishMovement(params, tssMapping));
|
||||
if (!params.dataMovementComplete.isSet())
|
||||
params.dataMovementComplete.send(Void());
|
||||
return Void();
|
||||
|
@ -891,32 +912,63 @@ Future<std::vector<ProcessData>> DDMockTxnProcessor::getWorkers() const {
|
|||
return Future<std::vector<ProcessData>>();
|
||||
}
|
||||
|
||||
void DDMockTxnProcessor::rawStartMovement(MoveKeysParams& params, std::map<UID, StorageServerInterface>& tssMapping) {
|
||||
FlowLock::Releaser releaser(*params.startMoveKeysParallelismLock);
|
||||
// Add wait(take) would always return immediately because there won’t be parallel rawStart or rawFinish in mock
|
||||
// world due to the fact the following *mock* transaction code will always finish without coroutine switch.
|
||||
ASSERT(params.startMoveKeysParallelismLock->take().isReady());
|
||||
ACTOR Future<Void> rawStartMovement(std::shared_ptr<MockGlobalState> mgs,
|
||||
MoveKeysParams params,
|
||||
std::map<UID, StorageServerInterface> tssMapping) {
|
||||
// There won’t be parallel rawStart or rawFinish in mock world due to the fact the following *mock* transaction code
|
||||
// will always finish without coroutine switch.
|
||||
ASSERT(params.startMoveKeysParallelismLock->activePermits() == 0);
|
||||
wait(params.startMoveKeysParallelismLock->take(TaskPriority::DataDistributionLaunch));
|
||||
state FlowLock::Releaser releaser(*params.startMoveKeysParallelismLock);
|
||||
|
||||
std::vector<ShardsAffectedByTeamFailure::Team> destTeams;
|
||||
destTeams.emplace_back(params.destinationTeam, true);
|
||||
mgs->shardMapping->moveShard(params.keys, destTeams);
|
||||
|
||||
for (auto& id : params.destinationTeam) {
|
||||
mgs->allServers.at(id).setShardStatus(params.keys, MockShardStatus::INFLIGHT, mgs->restrictSize);
|
||||
// invariant: the splitting and merge operation won't happen at the same moveKeys action. For example, if [a,c) [c,
|
||||
// e) exists, the params.keys won't be [b, d).
|
||||
auto intersectRanges = mgs->shardMapping->intersectingRanges(params.keys);
|
||||
// 1. splitting or just move a range. The new boundary need to be defined in startMovement
|
||||
if (intersectRanges.begin().range().contains(params.keys)) {
|
||||
mgs->shardMapping->defineShard(params.keys);
|
||||
}
|
||||
// 2. merge ops will coalesce the boundary in finishMovement;
|
||||
intersectRanges = mgs->shardMapping->intersectingRanges(params.keys);
|
||||
ASSERT(params.keys.begin == intersectRanges.begin().begin());
|
||||
ASSERT(params.keys.end == intersectRanges.end().begin());
|
||||
|
||||
for (auto it = intersectRanges.begin(); it != intersectRanges.end(); ++it) {
|
||||
auto teamPair = mgs->shardMapping->getTeamsFor(it->begin());
|
||||
auto& srcTeams = teamPair.second.empty() ? teamPair.first : teamPair.second;
|
||||
mgs->shardMapping->rawMoveShard(it->range(), srcTeams, destTeams);
|
||||
}
|
||||
|
||||
auto randomRangeSize =
|
||||
deterministicRandom()->randomInt64(SERVER_KNOBS->MIN_SHARD_BYTES, SERVER_KNOBS->MAX_SHARD_BYTES);
|
||||
for (auto& id : params.destinationTeam) {
|
||||
auto& server = mgs->allServers.at(id);
|
||||
server.setShardStatus(params.keys, MockShardStatus::INFLIGHT, mgs->restrictSize);
|
||||
server.signalFetchKeys(params.keys, randomRangeSize);
|
||||
}
|
||||
return Void();
|
||||
}
|
||||
|
||||
void DDMockTxnProcessor::rawFinishMovement(MoveKeysParams& params,
|
||||
const std::map<UID, StorageServerInterface>& tssMapping) {
|
||||
FlowLock::Releaser releaser(*params.finishMoveKeysParallelismLock);
|
||||
// Add wait(take) would always return immediately because there won’t be parallel rawStart or rawFinish in mock
|
||||
// world due to the fact the following *mock* transaction code will always finish without coroutine switch.
|
||||
ASSERT(params.finishMoveKeysParallelismLock->take().isReady());
|
||||
Future<Void> DDMockTxnProcessor::rawStartMovement(const MoveKeysParams& params,
|
||||
std::map<UID, StorageServerInterface>& tssMapping) {
|
||||
return ::rawStartMovement(mgs, params, tssMapping);
|
||||
}
|
||||
|
||||
ACTOR Future<Void> rawFinishMovement(std::shared_ptr<MockGlobalState> mgs,
|
||||
MoveKeysParams params,
|
||||
std::map<UID, StorageServerInterface> tssMapping) {
|
||||
// There won’t be parallel rawStart or rawFinish in mock world due to the fact the following *mock* transaction code
|
||||
// will always finish without coroutine switch.
|
||||
ASSERT(params.finishMoveKeysParallelismLock->activePermits() == 0);
|
||||
wait(params.finishMoveKeysParallelismLock->take(TaskPriority::DataDistributionLaunch));
|
||||
state FlowLock::Releaser releaser(*params.finishMoveKeysParallelismLock);
|
||||
|
||||
// get source and dest teams
|
||||
auto [destTeams, srcTeams] = mgs->shardMapping->getTeamsForFirstShard(params.keys);
|
||||
|
||||
ASSERT_EQ(destTeams.size(), 0);
|
||||
ASSERT_EQ(destTeams.size(), 1); // Will the multi-region or dynamic replica make destTeam.size() > 1?
|
||||
if (destTeams.front() != ShardsAffectedByTeamFailure::Team{ params.destinationTeam, true }) {
|
||||
TraceEvent(SevError, "MockRawFinishMovementError")
|
||||
.detail("Reason", "InconsistentDestinations")
|
||||
|
@ -929,9 +981,20 @@ void DDMockTxnProcessor::rawFinishMovement(MoveKeysParams& params,
|
|||
mgs->allServers.at(id).setShardStatus(params.keys, MockShardStatus::COMPLETED, mgs->restrictSize);
|
||||
}
|
||||
|
||||
// remove destination servers from source servers
|
||||
ASSERT_EQ(srcTeams.size(), 0);
|
||||
for (auto& id : srcTeams.front().servers) {
|
||||
mgs->allServers.at(id).removeShard(params.keys);
|
||||
// the only caller moveKeys will always make sure the UID are sorted
|
||||
if (!std::binary_search(params.destinationTeam.begin(), params.destinationTeam.end(), id)) {
|
||||
mgs->allServers.at(id).removeShard(params.keys);
|
||||
}
|
||||
}
|
||||
mgs->shardMapping->finishMove(params.keys);
|
||||
mgs->shardMapping->defineShard(params.keys); // coalesce for merge
|
||||
return Void();
|
||||
}
|
||||
|
||||
Future<Void> DDMockTxnProcessor::rawFinishMovement(const MoveKeysParams& params,
|
||||
const std::map<UID, StorageServerInterface>& tssMapping) {
|
||||
return ::rawFinishMovement(mgs, params, tssMapping);
|
||||
}
|
||||
|
|
|
@ -56,12 +56,12 @@
|
|||
ShardSizeBounds ShardSizeBounds::shardSizeBoundsBeforeTrack() {
|
||||
return ShardSizeBounds{
|
||||
.max = StorageMetrics{ .bytes = -1,
|
||||
.bytesPerKSecond = StorageMetrics::infinity,
|
||||
.bytesWrittenPerKSecond = StorageMetrics::infinity,
|
||||
.iosPerKSecond = StorageMetrics::infinity,
|
||||
.bytesReadPerKSecond = StorageMetrics::infinity },
|
||||
.min = StorageMetrics{ .bytes = -1, .bytesPerKSecond = 0, .iosPerKSecond = 0, .bytesReadPerKSecond = 0 },
|
||||
.min = StorageMetrics{ .bytes = -1, .bytesWrittenPerKSecond = 0, .iosPerKSecond = 0, .bytesReadPerKSecond = 0 },
|
||||
.permittedError = StorageMetrics{ .bytes = -1,
|
||||
.bytesPerKSecond = StorageMetrics::infinity,
|
||||
.bytesWrittenPerKSecond = StorageMetrics::infinity,
|
||||
.iosPerKSecond = StorageMetrics::infinity,
|
||||
.bytesReadPerKSecond = StorageMetrics::infinity }
|
||||
};
|
||||
|
|
|
@ -241,15 +241,15 @@ public:
|
|||
kmsLookupByIdsReqLatency("EKPKmsLookupByIdsReqLatency",
|
||||
id,
|
||||
SERVER_KNOBS->LATENCY_METRICS_LOGGING_INTERVAL,
|
||||
SERVER_KNOBS->LATENCY_SAMPLE_SIZE),
|
||||
SERVER_KNOBS->LATENCY_SKETCH_ACCURACY),
|
||||
kmsLookupByDomainIdsReqLatency("EKPKmsLookupByDomainIdsReqLatency",
|
||||
id,
|
||||
SERVER_KNOBS->LATENCY_METRICS_LOGGING_INTERVAL,
|
||||
SERVER_KNOBS->LATENCY_SAMPLE_SIZE),
|
||||
SERVER_KNOBS->LATENCY_SKETCH_ACCURACY),
|
||||
kmsBlobMetadataReqLatency("EKPKmsBlobMetadataReqLatency",
|
||||
id,
|
||||
SERVER_KNOBS->LATENCY_METRICS_LOGGING_INTERVAL,
|
||||
SERVER_KNOBS->LATENCY_SAMPLE_SIZE) {}
|
||||
SERVER_KNOBS->LATENCY_SKETCH_ACCURACY) {}
|
||||
|
||||
EncryptBaseCipherDomainIdKeyIdCacheKey getBaseCipherDomainIdKeyIdCacheKey(
|
||||
const EncryptCipherDomainId domainId,
|
||||
|
|
|
@ -117,20 +117,20 @@ struct GrvProxyStats {
|
|||
defaultTxnGRVTimeInQueue("DefaultTxnGRVTimeInQueue",
|
||||
id,
|
||||
SERVER_KNOBS->LATENCY_METRICS_LOGGING_INTERVAL,
|
||||
SERVER_KNOBS->LATENCY_SAMPLE_SIZE),
|
||||
SERVER_KNOBS->LATENCY_SKETCH_ACCURACY),
|
||||
batchTxnGRVTimeInQueue("BatchTxnGRVTimeInQueue",
|
||||
id,
|
||||
SERVER_KNOBS->LATENCY_METRICS_LOGGING_INTERVAL,
|
||||
SERVER_KNOBS->LATENCY_SAMPLE_SIZE),
|
||||
SERVER_KNOBS->LATENCY_SKETCH_ACCURACY),
|
||||
grvLatencyBands("GRVLatencyBands", id, SERVER_KNOBS->STORAGE_LOGGING_DELAY),
|
||||
grvLatencySample("GRVLatencyMetrics",
|
||||
id,
|
||||
SERVER_KNOBS->LATENCY_METRICS_LOGGING_INTERVAL,
|
||||
SERVER_KNOBS->LATENCY_SAMPLE_SIZE),
|
||||
SERVER_KNOBS->LATENCY_SKETCH_ACCURACY),
|
||||
grvBatchLatencySample("GRVBatchLatencyMetrics",
|
||||
id,
|
||||
SERVER_KNOBS->LATENCY_METRICS_LOGGING_INTERVAL,
|
||||
SERVER_KNOBS->LATENCY_SAMPLE_SIZE),
|
||||
SERVER_KNOBS->LATENCY_SKETCH_ACCURACY),
|
||||
recentRequests(0), lastBucketBegin(now()),
|
||||
bucketInterval(FLOW_KNOBS->BASIC_LOAD_BALANCE_UPDATE_RATE / FLOW_KNOBS->BASIC_LOAD_BALANCE_BUCKETS),
|
||||
grvConfirmEpochLiveDist(
|
||||
|
@ -215,7 +215,7 @@ struct GrvProxyData {
|
|||
versionVectorSizeOnGRVReply("VersionVectorSizeOnGRVReply",
|
||||
dbgid,
|
||||
SERVER_KNOBS->LATENCY_METRICS_LOGGING_INTERVAL,
|
||||
SERVER_KNOBS->LATENCY_SAMPLE_SIZE),
|
||||
SERVER_KNOBS->LATENCY_SKETCH_ACCURACY),
|
||||
updateCommitRequests(0), lastCommitTime(0), version(0), minKnownCommittedVersion(invalidVersion),
|
||||
tagThrottler(SERVER_KNOBS->PROXY_MAX_TAG_THROTTLE_DURATION) {}
|
||||
};
|
||||
|
|
|
@ -68,12 +68,9 @@
|
|||
|
||||
#ifdef SSD_ROCKSDB_EXPERIMENTAL
|
||||
|
||||
// Enforcing rocksdb version to be 6.27.3 or greater.
|
||||
static_assert(ROCKSDB_MAJOR >= 6, "Unsupported rocksdb version. Update the rocksdb to 6.27.3 version");
|
||||
static_assert(ROCKSDB_MAJOR == 6 ? ROCKSDB_MINOR >= 27 : true,
|
||||
"Unsupported rocksdb version. Update the rocksdb to 6.27.3 version");
|
||||
static_assert((ROCKSDB_MAJOR == 6 && ROCKSDB_MINOR == 27) ? ROCKSDB_PATCH >= 3 : true,
|
||||
"Unsupported rocksdb version. Update the rocksdb to 6.27.3 version");
|
||||
// Enforcing rocksdb version to be 7.7.3.
|
||||
static_assert((ROCKSDB_MAJOR == 7 && ROCKSDB_MINOR == 7 && ROCKSDB_PATCH == 3),
|
||||
"Unsupported rocksdb version. Update the rocksdb to 7.7.3 version");
|
||||
|
||||
namespace {
|
||||
using rocksdb::BackgroundErrorReason;
|
||||
|
@ -111,15 +108,15 @@ SharedRocksDBState::SharedRocksDBState(UID id)
|
|||
readOptions(initialReadOptions()), commitLatency(LatencySample("RocksDBCommitLatency",
|
||||
id,
|
||||
SERVER_KNOBS->LATENCY_METRICS_LOGGING_INTERVAL,
|
||||
SERVER_KNOBS->LATENCY_SAMPLE_SIZE)),
|
||||
SERVER_KNOBS->LATENCY_SKETCH_ACCURACY)),
|
||||
commitQueueLatency(LatencySample("RocksDBCommitQueueLatency",
|
||||
id,
|
||||
SERVER_KNOBS->LATENCY_METRICS_LOGGING_INTERVAL,
|
||||
SERVER_KNOBS->LATENCY_SAMPLE_SIZE)),
|
||||
SERVER_KNOBS->LATENCY_SKETCH_ACCURACY)),
|
||||
dbWriteLatency(LatencySample("RocksDBWriteLatency",
|
||||
id,
|
||||
SERVER_KNOBS->LATENCY_METRICS_LOGGING_INTERVAL,
|
||||
SERVER_KNOBS->LATENCY_SAMPLE_SIZE)) {}
|
||||
SERVER_KNOBS->LATENCY_SKETCH_ACCURACY)) {}
|
||||
|
||||
rocksdb::ColumnFamilyOptions SharedRocksDBState::initialCfOptions() {
|
||||
rocksdb::ColumnFamilyOptions options;
|
||||
|
@ -1402,17 +1399,11 @@ struct RocksDBKeyValueStore : IKeyValueStore {
|
|||
ThreadReturnPromiseStream<std::pair<std::string, double>>* metricPromiseStream)
|
||||
: id(id), db(db), cf(cf), sharedState(sharedState), readIterPool(readIterPool),
|
||||
perfContextMetrics(perfContextMetrics), metricPromiseStream(metricPromiseStream), threadIndex(threadIndex) {
|
||||
if (g_network->isSimulated()) {
|
||||
// In simulation, increasing the read operation timeouts to 5 minutes, as some of the tests have
|
||||
// very high load and single read thread cannot process all the load within the timeouts.
|
||||
readValueTimeout = 5 * 60;
|
||||
readValuePrefixTimeout = 5 * 60;
|
||||
readRangeTimeout = 5 * 60;
|
||||
} else {
|
||||
readValueTimeout = SERVER_KNOBS->ROCKSDB_READ_VALUE_TIMEOUT;
|
||||
readValuePrefixTimeout = SERVER_KNOBS->ROCKSDB_READ_VALUE_PREFIX_TIMEOUT;
|
||||
readRangeTimeout = SERVER_KNOBS->ROCKSDB_READ_RANGE_TIMEOUT;
|
||||
}
|
||||
|
||||
readValueTimeout = SERVER_KNOBS->ROCKSDB_READ_VALUE_TIMEOUT;
|
||||
readValuePrefixTimeout = SERVER_KNOBS->ROCKSDB_READ_VALUE_PREFIX_TIMEOUT;
|
||||
readRangeTimeout = SERVER_KNOBS->ROCKSDB_READ_RANGE_TIMEOUT;
|
||||
|
||||
if (SERVER_KNOBS->ROCKSDB_PERFCONTEXT_ENABLE) {
|
||||
// Enable perf context on the same thread with the db thread
|
||||
rocksdb::SetPerfLevel(rocksdb::PerfLevel::kEnableTimeExceptForMutex);
|
||||
|
|
|
@ -41,12 +41,9 @@
|
|||
|
||||
#ifdef SSD_ROCKSDB_EXPERIMENTAL
|
||||
|
||||
// Enforcing rocksdb version to be 6.27.3 or greater.
|
||||
static_assert(ROCKSDB_MAJOR >= 6, "Unsupported rocksdb version. Update the rocksdb to 6.27.3 version");
|
||||
static_assert(ROCKSDB_MAJOR == 6 ? ROCKSDB_MINOR >= 27 : true,
|
||||
"Unsupported rocksdb version. Update the rocksdb to 6.27.3 version");
|
||||
static_assert((ROCKSDB_MAJOR == 6 && ROCKSDB_MINOR == 27) ? ROCKSDB_PATCH >= 3 : true,
|
||||
"Unsupported rocksdb version. Update the rocksdb to 6.27.3 version");
|
||||
// Enforcing rocksdb version to be 7.7.3.
|
||||
static_assert((ROCKSDB_MAJOR == 7 && ROCKSDB_MINOR == 7 && ROCKSDB_PATCH == 3),
|
||||
"Unsupported rocksdb version. Update the rocksdb to 7.7.3 version");
|
||||
|
||||
const std::string rocksDataFolderSuffix = "-data";
|
||||
const std::string METADATA_SHARD_ID = "kvs-metadata";
|
||||
|
|
|
@ -28,6 +28,7 @@
|
|||
#include "flow/ActorCollection.h"
|
||||
#include "flow/Arena.h"
|
||||
#include "flow/Histogram.h"
|
||||
#include "flow/Trace.h"
|
||||
#include "flow/network.h"
|
||||
#include "flow/DebugTrace.h"
|
||||
#include "flow/actorcompiler.h" // This must be the last #include.
|
||||
|
@ -448,6 +449,14 @@ Future<Void> logRouterPeekMessages(PromiseType replyPromise,
|
|||
state int sequence = -1;
|
||||
state UID peekId;
|
||||
|
||||
DebugLogTraceEvent("LogRouterPeek0", self->dbgid)
|
||||
.detail("ReturnIfBlocked", reqReturnIfBlocked)
|
||||
.detail("Tag", reqTag.toString())
|
||||
.detail("Seq", reqSequence.present() ? reqSequence.get().second : -1)
|
||||
.detail("SeqCursor", reqSequence.present() ? reqSequence.get().first : UID())
|
||||
.detail("Ver", self->version.get())
|
||||
.detail("Begin", reqBegin);
|
||||
|
||||
if (reqSequence.present()) {
|
||||
try {
|
||||
peekId = reqSequence.get().first;
|
||||
|
@ -481,6 +490,13 @@ Future<Void> logRouterPeekMessages(PromiseType replyPromise,
|
|||
reqOnlySpilled = prevPeekData.second;
|
||||
wait(yield());
|
||||
} catch (Error& e) {
|
||||
DebugLogTraceEvent("LogRouterPeekError", self->dbgid)
|
||||
.error(e)
|
||||
.detail("Tag", reqTag.toString())
|
||||
.detail("Seq", reqSequence.present() ? reqSequence.get().second : -1)
|
||||
.detail("SeqCursor", reqSequence.present() ? reqSequence.get().first : UID())
|
||||
.detail("Begin", reqBegin);
|
||||
|
||||
if (e.code() == error_code_timed_out || e.code() == error_code_operation_obsolete) {
|
||||
replyPromise.sendError(e);
|
||||
return Void();
|
||||
|
@ -490,12 +506,6 @@ Future<Void> logRouterPeekMessages(PromiseType replyPromise,
|
|||
}
|
||||
}
|
||||
|
||||
DebugLogTraceEvent("LogRouterPeek0", self->dbgid)
|
||||
.detail("ReturnIfBlocked", reqReturnIfBlocked)
|
||||
.detail("Tag", reqTag.toString())
|
||||
.detail("Ver", self->version.get())
|
||||
.detail("Begin", reqBegin);
|
||||
|
||||
if (reqReturnIfBlocked && self->version.get() < reqBegin) {
|
||||
replyPromise.sendError(end_of_stream());
|
||||
if (reqSequence.present()) {
|
||||
|
@ -528,19 +538,22 @@ Future<Void> logRouterPeekMessages(PromiseType replyPromise,
|
|||
TraceEvent(SevWarnAlways, "LogRouterPeekPopped", self->dbgid)
|
||||
.detail("Begin", reqBegin)
|
||||
.detail("Popped", poppedVer)
|
||||
.detail("Tag", reqTag.toString())
|
||||
.detail("Seq", reqSequence.present() ? reqSequence.get().second : -1)
|
||||
.detail("SeqCursor", reqSequence.present() ? reqSequence.get().first : UID())
|
||||
.detail("Start", self->startVersion);
|
||||
if (std::is_same<PromiseType, Promise<TLogPeekReply>>::value) {
|
||||
// kills logRouterPeekStream actor, otherwise that actor becomes stuck
|
||||
throw operation_obsolete();
|
||||
}
|
||||
replyPromise.send(Never());
|
||||
if (reqSequence.present()) {
|
||||
auto& trackerData = self->peekTracker[peekId];
|
||||
auto& sequenceData = trackerData.sequence_version[sequence + 1];
|
||||
if (!sequenceData.isSet()) {
|
||||
sequenceData.send(std::make_pair(reqBegin, reqOnlySpilled));
|
||||
}
|
||||
if (std::is_same<PromiseType, ReplyPromise<TLogPeekReply>>::value) {
|
||||
// Send error to avoid a race condition that the peer is really retrying,
|
||||
// otherwise, the peer could be blocked forever.
|
||||
replyPromise.sendError(operation_obsolete());
|
||||
} else {
|
||||
replyPromise.send(Never());
|
||||
}
|
||||
|
||||
return Void();
|
||||
}
|
||||
|
||||
|
@ -681,6 +694,7 @@ ACTOR Future<Void> logRouterPop(LogRouterData* self, TLogPopRequest req) {
|
|||
if (!tagData) {
|
||||
tagData = self->createTagData(req.tag, req.to, req.durableKnownCommittedVersion);
|
||||
} else if (req.to > tagData->popped) {
|
||||
DebugLogTraceEvent("LogRouterPop", self->dbgid).detail("Tag", req.tag.toString()).detail("PopVersion", req.to);
|
||||
tagData->popped = req.to;
|
||||
tagData->durableKnownCommittedVersion = req.durableKnownCommittedVersion;
|
||||
wait(tagData->eraseMessagesBefore(req.to, self, TaskPriority::TLogPop));
|
||||
|
|
|
@ -62,6 +62,8 @@ ILogSystem::ServerPeekCursor::ServerPeekCursor(Reference<AsyncVar<OptionalInterf
|
|||
this->results.minKnownCommittedVersion = 0;
|
||||
DebugLogTraceEvent(SevDebug, "SPC_Starting", randomID)
|
||||
.detail("Tag", tag.toString())
|
||||
.detail("Parallel", parallelGetMore)
|
||||
.detail("Interf", interf && interf->get().present() ? interf->get().id() : UID())
|
||||
.detail("UsePeekStream", usePeekStream)
|
||||
.detail("Begin", begin)
|
||||
.detail("End", end);
|
||||
|
@ -111,7 +113,9 @@ bool ILogSystem::ServerPeekCursor::hasMessage() const {
|
|||
}
|
||||
|
||||
void ILogSystem::ServerPeekCursor::nextMessage() {
|
||||
//TraceEvent("SPC_NextMessage", randomID).detail("MessageVersion", messageVersion.toString());
|
||||
DebugLogTraceEvent("SPC_NextMessage", randomID)
|
||||
.detail("Tag", tag.toString())
|
||||
.detail("MessageVersion", messageVersion.toString());
|
||||
ASSERT(hasMsg);
|
||||
if (rd.empty()) {
|
||||
messageVersion.reset(std::min(results.end, end.version));
|
||||
|
@ -143,11 +147,13 @@ void ILogSystem::ServerPeekCursor::nextMessage() {
|
|||
rd.rewind();
|
||||
rd.readBytes(messageAndTags.getHeaderSize());
|
||||
hasMsg = true;
|
||||
//TraceEvent("SPC_NextMessageB", randomID).detail("MessageVersion", messageVersion.toString());
|
||||
DebugLogTraceEvent("SPC_NextMessageB", randomID)
|
||||
.detail("Tag", tag.toString())
|
||||
.detail("MessageVersion", messageVersion.toString());
|
||||
}
|
||||
|
||||
StringRef ILogSystem::ServerPeekCursor::getMessage() {
|
||||
//TraceEvent("SPC_GetMessage", randomID);
|
||||
DebugLogTraceEvent("SPC_GetMessage", randomID).detail("Tag", tag.toString());
|
||||
StringRef message = messageAndTags.getMessageWithoutTags();
|
||||
rd.readBytes(message.size()); // Consumes the message.
|
||||
return message;
|
||||
|
@ -260,6 +266,14 @@ ACTOR Future<Void> serverPeekParallelGetMore(ILogSystem::ServerPeekCursor* self,
|
|||
}
|
||||
|
||||
loop {
|
||||
DebugLogTraceEvent("SPC_GetMoreP", self->randomID)
|
||||
.detail("Tag", self->tag.toString())
|
||||
.detail("Has", self->hasMessage())
|
||||
.detail("Begin", self->messageVersion.version)
|
||||
.detail("Parallel", self->parallelGetMore)
|
||||
.detail("Seq", self->sequence)
|
||||
.detail("Sizes", self->futureResults.size())
|
||||
.detail("Interf", self->interf->get().present() ? self->interf->get().id() : UID());
|
||||
state Version expectedBegin = self->messageVersion.version;
|
||||
try {
|
||||
if (self->parallelGetMore || self->onlySpilled) {
|
||||
|
@ -294,7 +308,12 @@ ACTOR Future<Void> serverPeekParallelGetMore(ILogSystem::ServerPeekCursor* self,
|
|||
expectedBegin = res.end;
|
||||
self->futureResults.pop_front();
|
||||
updateCursorWithReply(self, res);
|
||||
//TraceEvent("SPC_GetMoreB", self->randomID).detail("Has", self->hasMessage()).detail("End", res.end).detail("Popped", res.popped.present() ? res.popped.get() : 0);
|
||||
DebugLogTraceEvent("SPC_GetMoreReply", self->randomID)
|
||||
.detail("Has", self->hasMessage())
|
||||
.detail("Tag", self->tag.toString())
|
||||
.detail("End", res.end)
|
||||
.detail("Size", self->futureResults.size())
|
||||
.detail("Popped", res.popped.present() ? res.popped.get() : 0);
|
||||
return Void();
|
||||
}
|
||||
when(wait(self->interfaceChanged)) {
|
||||
|
@ -306,11 +325,17 @@ ACTOR Future<Void> serverPeekParallelGetMore(ILogSystem::ServerPeekCursor* self,
|
|||
}
|
||||
}
|
||||
} catch (Error& e) {
|
||||
DebugLogTraceEvent("PeekCursorError", self->randomID)
|
||||
.error(e)
|
||||
.detail("Tag", self->tag.toString())
|
||||
.detail("Begin", self->messageVersion.version)
|
||||
.detail("Interf", self->interf->get().present() ? self->interf->get().id() : UID());
|
||||
|
||||
if (e.code() == error_code_end_of_stream) {
|
||||
self->end.reset(self->messageVersion.version);
|
||||
return Void();
|
||||
} else if (e.code() == error_code_timed_out || e.code() == error_code_operation_obsolete) {
|
||||
TraceEvent("PeekCursorTimedOut", self->randomID).error(e);
|
||||
TraceEvent ev("PeekCursorTimedOut", self->randomID);
|
||||
// We *should* never get timed_out(), as it means the TLog got stuck while handling a parallel peek,
|
||||
// and thus we've likely just wasted 10min.
|
||||
// timed_out() is sent by cleanupPeekTrackers as value PEEK_TRACKER_EXPIRATION_TIME
|
||||
|
@ -326,6 +351,11 @@ ACTOR Future<Void> serverPeekParallelGetMore(ILogSystem::ServerPeekCursor* self,
|
|||
self->randomID = deterministicRandom()->randomUniqueID();
|
||||
self->sequence = 0;
|
||||
self->futureResults.clear();
|
||||
ev.error(e)
|
||||
.detail("Tag", self->tag.toString())
|
||||
.detail("Begin", self->messageVersion.version)
|
||||
.detail("NewID", self->randomID)
|
||||
.detail("Interf", self->interf->get().present() ? self->interf->get().id() : UID());
|
||||
} else {
|
||||
throw e;
|
||||
}
|
||||
|
@ -415,7 +445,11 @@ ACTOR Future<Void> serverPeekGetMore(ILogSystem::ServerPeekCursor* self, TaskPri
|
|||
taskID))
|
||||
: Never())) {
|
||||
updateCursorWithReply(self, res);
|
||||
//TraceEvent("SPC_GetMoreB", self->randomID).detail("Has", self->hasMessage()).detail("End", res.end).detail("Popped", res.popped.present() ? res.popped.get() : 0);
|
||||
DebugLogTraceEvent("SPC_GetMoreB", self->randomID)
|
||||
.detail("Tag", self->tag.toString())
|
||||
.detail("Has", self->hasMessage())
|
||||
.detail("End", res.end)
|
||||
.detail("Popped", res.popped.present() ? res.popped.get() : 0);
|
||||
return Void();
|
||||
}
|
||||
when(wait(self->interf->onChange())) { self->onlySpilled = false; }
|
||||
|
@ -431,11 +465,13 @@ ACTOR Future<Void> serverPeekGetMore(ILogSystem::ServerPeekCursor* self, TaskPri
|
|||
}
|
||||
|
||||
Future<Void> ILogSystem::ServerPeekCursor::getMore(TaskPriority taskID) {
|
||||
// TraceEvent("SPC_GetMore", randomID)
|
||||
// .detail("HasMessage", hasMessage())
|
||||
// .detail("More", !more.isValid() || more.isReady())
|
||||
// .detail("MessageVersion", messageVersion.toString())
|
||||
// .detail("End", end.toString());
|
||||
DebugLogTraceEvent("SPC_GetMore", randomID)
|
||||
.detail("Tag", tag.toString())
|
||||
.detail("HasMessage", hasMessage())
|
||||
.detail("More", !more.isValid() || more.isReady())
|
||||
.detail("Parallel", parallelGetMore)
|
||||
.detail("MessageVersion", messageVersion.toString())
|
||||
.detail("End", end.toString());
|
||||
if (hasMessage() && !parallelGetMore)
|
||||
return Void();
|
||||
if (!more.isValid() || more.isReady()) {
|
||||
|
|
|
@ -114,9 +114,60 @@ public:
|
|||
}
|
||||
return Void();
|
||||
}
|
||||
|
||||
// Randomly generate keys and kv size between the fetch range, updating the byte sample.
|
||||
// Once the fetchKeys return, the shard status will become FETCHED.
|
||||
ACTOR static Future<Void> waitFetchKeysFinish(MockStorageServer* self, MockStorageServer::FetchKeysParams params) {
|
||||
// between each chunk delay for random time, and finally set the fetchComplete signal.
|
||||
ASSERT(params.totalRangeBytes > 0);
|
||||
state int chunkCount = std::ceil(params.totalRangeBytes * 1.0 / SERVER_KNOBS->FETCH_BLOCK_BYTES);
|
||||
state int64_t currentTotal = 0;
|
||||
state Key lastKey = params.keys.begin;
|
||||
|
||||
state int i = 0;
|
||||
for (; i < chunkCount && currentTotal < params.totalRangeBytes; ++i) {
|
||||
wait(delayJittered(0.01));
|
||||
int remainedBytes = (chunkCount == 1 ? params.totalRangeBytes : SERVER_KNOBS->FETCH_BLOCK_BYTES);
|
||||
|
||||
while (remainedBytes >= lastKey.size()) {
|
||||
Key nextKey;
|
||||
// try 10 times
|
||||
for (int j = 0; j < 10; j++) {
|
||||
nextKey = randomKeyBetween(KeyRangeRef(lastKey, params.keys.end));
|
||||
if (nextKey < params.keys.end)
|
||||
break;
|
||||
}
|
||||
|
||||
// NOTE: in this case, we accumulate the bytes on lastKey on purpose (shall we?)
|
||||
if (nextKey == params.keys.end) {
|
||||
auto bytes = params.totalRangeBytes - currentTotal;
|
||||
self->byteSampleApplySet(lastKey, bytes);
|
||||
self->usedDiskSpace += bytes;
|
||||
currentTotal = params.totalRangeBytes;
|
||||
TraceEvent(SevWarn, "MockFetchKeysInaccurateSample")
|
||||
.detail("Range", params.keys)
|
||||
.detail("LastKey", lastKey)
|
||||
.detail("Size", bytes);
|
||||
break; // break the most outside loop
|
||||
}
|
||||
|
||||
int maxSize = std::min(remainedBytes, 130000) + 1;
|
||||
int randomSize = deterministicRandom()->randomInt(lastKey.size(), maxSize);
|
||||
self->usedDiskSpace += randomSize;
|
||||
currentTotal += randomSize;
|
||||
|
||||
self->byteSampleApplySet(lastKey, randomSize);
|
||||
remainedBytes -= randomSize;
|
||||
lastKey = nextKey;
|
||||
}
|
||||
}
|
||||
|
||||
self->setShardStatus(params.keys, MockShardStatus::FETCHED, true);
|
||||
return Void();
|
||||
}
|
||||
};
|
||||
|
||||
bool MockStorageServer::allShardStatusEqual(KeyRangeRef range, MockShardStatus status) {
|
||||
bool MockStorageServer::allShardStatusEqual(const KeyRangeRef& range, MockShardStatus status) const {
|
||||
auto ranges = serverKeys.intersectingRanges(range);
|
||||
ASSERT(!ranges.empty()); // at least the range is allKeys
|
||||
|
||||
|
@ -127,26 +178,46 @@ bool MockStorageServer::allShardStatusEqual(KeyRangeRef range, MockShardStatus s
|
|||
return true;
|
||||
}
|
||||
|
||||
void MockStorageServer::setShardStatus(KeyRangeRef range, MockShardStatus status, bool restrictSize) {
|
||||
bool MockStorageServer::allShardStatusIn(const KeyRangeRef& range, const std::set<MockShardStatus>& status) const {
|
||||
auto ranges = serverKeys.intersectingRanges(range);
|
||||
ASSERT(!ranges.empty());
|
||||
if (ranges.begin().range().contains(range)) {
|
||||
CODE_PROBE(true, "Implicitly split single shard to 3 pieces", probe::decoration::rare);
|
||||
threeWayShardSplitting(ranges.begin().range(), range, ranges.begin().cvalue().shardSize, restrictSize);
|
||||
ASSERT(!ranges.empty()); // at least the range is allKeys
|
||||
|
||||
for (auto it = ranges.begin(); it != ranges.end(); ++it) {
|
||||
if (!status.count(it->cvalue().status))
|
||||
return false;
|
||||
}
|
||||
return true;
|
||||
}
|
||||
|
||||
void MockStorageServer::setShardStatus(const KeyRangeRef& range, MockShardStatus status, bool restrictSize) {
|
||||
auto ranges = serverKeys.intersectingRanges(range);
|
||||
|
||||
if (ranges.empty()) {
|
||||
CODE_PROBE(true, "new shard is adding to server");
|
||||
serverKeys.insert(range, ShardInfo{ status, 0 });
|
||||
return;
|
||||
}
|
||||
if (ranges.begin().begin() < range.begin) {
|
||||
CODE_PROBE(true, "Implicitly split begin range to 2 pieces", probe::decoration::rare);
|
||||
twoWayShardSplitting(ranges.begin().range(), range.begin, ranges.begin().cvalue().shardSize, restrictSize);
|
||||
}
|
||||
if (ranges.end().end() > range.end) {
|
||||
CODE_PROBE(true, "Implicitly split end range to 2 pieces", probe::decoration::rare);
|
||||
twoWayShardSplitting(ranges.end().range(), range.end, ranges.end().cvalue().shardSize, restrictSize);
|
||||
|
||||
// change the old status
|
||||
if (ranges.begin().begin() < range.begin && ranges.begin().end() > range.end) {
|
||||
CODE_PROBE(true, "Implicitly split single shard to 3 pieces", probe::decoration::rare);
|
||||
threeWayShardSplitting(ranges.begin().range(), range, ranges.begin().cvalue().shardSize, restrictSize);
|
||||
} else {
|
||||
if (ranges.begin().begin() < range.begin) {
|
||||
CODE_PROBE(true, "Implicitly split begin range to 2 pieces", probe::decoration::rare);
|
||||
twoWayShardSplitting(ranges.begin().range(), range.begin, ranges.begin().cvalue().shardSize, restrictSize);
|
||||
}
|
||||
if (ranges.end().begin() > range.end) {
|
||||
CODE_PROBE(true, "Implicitly split end range to 2 pieces", probe::decoration::rare);
|
||||
auto lastRange = ranges.end();
|
||||
--lastRange;
|
||||
twoWayShardSplitting(lastRange.range(), range.end, ranges.end().cvalue().shardSize, restrictSize);
|
||||
}
|
||||
}
|
||||
ranges = serverKeys.containedRanges(range);
|
||||
// now the boundary must be aligned
|
||||
ASSERT(ranges.begin().begin() == range.begin);
|
||||
ASSERT(ranges.end().end() == range.end);
|
||||
ASSERT(ranges.end().begin() == range.end);
|
||||
uint64_t newSize = 0;
|
||||
for (auto it = ranges.begin(); it != ranges.end(); ++it) {
|
||||
newSize += it->cvalue().shardSize;
|
||||
|
@ -155,15 +226,15 @@ void MockStorageServer::setShardStatus(KeyRangeRef range, MockShardStatus status
|
|||
auto oldStatus = it.value().status;
|
||||
if (isStatusTransitionValid(oldStatus, status)) {
|
||||
it.value() = ShardInfo{ status, newSize };
|
||||
} else if (oldStatus == MockShardStatus::COMPLETED && status == MockShardStatus::INFLIGHT) {
|
||||
} else if ((oldStatus == MockShardStatus::COMPLETED || oldStatus == MockShardStatus::FETCHED) &&
|
||||
(status == MockShardStatus::INFLIGHT || status == MockShardStatus::FETCHED)) {
|
||||
CODE_PROBE(true, "Shard already on server", probe::decoration::rare);
|
||||
} else {
|
||||
TraceEvent(SevError, "MockShardStatusTransitionError")
|
||||
TraceEvent(SevError, "MockShardStatusTransitionError", id)
|
||||
.detail("From", oldStatus)
|
||||
.detail("To", status)
|
||||
.detail("ID", id)
|
||||
.detail("KeyBegin", range.begin.toHexString())
|
||||
.detail("KeyEnd", range.begin.toHexString());
|
||||
.detail("KeyBegin", range.begin)
|
||||
.detail("KeyEnd", range.begin);
|
||||
}
|
||||
}
|
||||
serverKeys.coalesce(range);
|
||||
|
@ -171,11 +242,14 @@ void MockStorageServer::setShardStatus(KeyRangeRef range, MockShardStatus status
|
|||
|
||||
// split the out range [a, d) based on the inner range's boundary [b, c). The result would be [a,b), [b,c), [c,d). The
|
||||
// size of the new shards are randomly split from old size of [a, d)
|
||||
void MockStorageServer::threeWayShardSplitting(KeyRangeRef outerRange,
|
||||
KeyRangeRef innerRange,
|
||||
void MockStorageServer::threeWayShardSplitting(const KeyRangeRef& outerRange,
|
||||
const KeyRangeRef& innerRange,
|
||||
uint64_t outerRangeSize,
|
||||
bool restrictSize) {
|
||||
ASSERT(outerRange.contains(innerRange));
|
||||
if (outerRange == innerRange) {
|
||||
return;
|
||||
}
|
||||
|
||||
Key left = outerRange.begin;
|
||||
// random generate 3 shard sizes, the caller guarantee that the min, max parameters are always valid.
|
||||
|
@ -196,10 +270,13 @@ void MockStorageServer::threeWayShardSplitting(KeyRangeRef outerRange,
|
|||
|
||||
// split the range [a,c) with split point b. The result would be [a, b), [b, c). The
|
||||
// size of the new shards are randomly split from old size of [a, c)
|
||||
void MockStorageServer::twoWayShardSplitting(KeyRangeRef range,
|
||||
KeyRef splitPoint,
|
||||
void MockStorageServer::twoWayShardSplitting(const KeyRangeRef& range,
|
||||
const KeyRef& splitPoint,
|
||||
uint64_t rangeSize,
|
||||
bool restrictSize) {
|
||||
if (splitPoint == range.begin || !range.contains(splitPoint)) {
|
||||
return;
|
||||
}
|
||||
Key left = range.begin;
|
||||
// random generate 3 shard sizes, the caller guarantee that the min, max parameters are always valid.
|
||||
int leftSize = deterministicRandom()->randomInt(SERVER_KNOBS->MIN_SHARD_BYTES,
|
||||
|
@ -212,13 +289,17 @@ void MockStorageServer::twoWayShardSplitting(KeyRangeRef range,
|
|||
serverKeys[left].shardSize = leftSize;
|
||||
}
|
||||
|
||||
void MockStorageServer::removeShard(KeyRangeRef range) {
|
||||
void MockStorageServer::removeShard(const KeyRangeRef& range) {
|
||||
auto ranges = serverKeys.containedRanges(range);
|
||||
ASSERT(ranges.begin().range() == range);
|
||||
auto rangeSize = sumRangeSize(range);
|
||||
usedDiskSpace -= rangeSize;
|
||||
serverKeys.rawErase(range);
|
||||
byteSampleApplyClear(range);
|
||||
metrics.notifyNotReadable(range);
|
||||
}
|
||||
|
||||
uint64_t MockStorageServer::sumRangeSize(KeyRangeRef range) const {
|
||||
uint64_t MockStorageServer::sumRangeSize(const KeyRangeRef& range) const {
|
||||
auto ranges = serverKeys.intersectingRanges(range);
|
||||
uint64_t totalSize = 0;
|
||||
for (auto it = ranges.begin(); it != ranges.end(); ++it) {
|
||||
|
@ -246,8 +327,157 @@ Future<Void> MockStorageServer::run() {
|
|||
Optional<Standalone<StringRef>>());
|
||||
ssi.initEndpoints();
|
||||
ssi.startAcceptingRequests();
|
||||
IFailureMonitor::failureMonitor().setStatus(ssi.address(), FailureStatus(false));
|
||||
|
||||
TraceEvent("MockStorageServerStart").detail("Address", ssi.address());
|
||||
return serveStorageMetricsRequests(this, ssi);
|
||||
addActor(serveStorageMetricsRequests(this, ssi));
|
||||
return actors.getResult();
|
||||
}
|
||||
|
||||
void MockStorageServer::set(KeyRef const& key, int64_t bytes, int64_t oldBytes) {
|
||||
notifyWriteMetrics(key, bytes);
|
||||
byteSampleApplySet(key, bytes);
|
||||
auto delta = bytes - oldBytes;
|
||||
usedDiskSpace += delta;
|
||||
serverKeys[key].shardSize += delta;
|
||||
}
|
||||
|
||||
void MockStorageServer::clear(KeyRef const& key, int64_t bytes) {
|
||||
notifyWriteMetrics(key, bytes);
|
||||
KeyRange sr = singleKeyRange(key);
|
||||
byteSampleApplyClear(sr);
|
||||
usedDiskSpace -= bytes;
|
||||
serverKeys[key].shardSize -= bytes;
|
||||
}
|
||||
|
||||
int64_t MockStorageServer::clearRange(KeyRangeRef const& range, int64_t beginShardBytes, int64_t endShardBytes) {
|
||||
notifyWriteMetrics(range.begin, range.begin.size() + range.end.size());
|
||||
byteSampleApplyClear(range);
|
||||
auto totalByteSize = estimateRangeTotalBytes(range, beginShardBytes, endShardBytes);
|
||||
usedDiskSpace -= totalByteSize;
|
||||
clearRangeTotalBytes(range, beginShardBytes, endShardBytes);
|
||||
return totalByteSize;
|
||||
}
|
||||
|
||||
void MockStorageServer::get(KeyRef const& key, int64_t bytes) {
|
||||
// If the read yields no value, randomly sample the empty read.
|
||||
int64_t bytesReadPerKSecond = std::max(bytes, SERVER_KNOBS->EMPTY_READ_PENALTY);
|
||||
metrics.notifyBytesReadPerKSecond(key, bytesReadPerKSecond);
|
||||
}
|
||||
|
||||
int64_t MockStorageServer::getRange(KeyRangeRef const& range, int64_t beginShardBytes, int64_t endShardBytes) {
|
||||
int64_t totalByteSize = estimateRangeTotalBytes(range, beginShardBytes, endShardBytes);
|
||||
// For performance concerns, the cost of a range read is billed to the start key and end key of the
|
||||
// range.
|
||||
if (totalByteSize > 0) {
|
||||
int64_t bytesReadPerKSecond = std::max(totalByteSize, SERVER_KNOBS->EMPTY_READ_PENALTY) / 2;
|
||||
metrics.notifyBytesReadPerKSecond(range.begin, bytesReadPerKSecond);
|
||||
metrics.notifyBytesReadPerKSecond(range.end, bytesReadPerKSecond);
|
||||
}
|
||||
return totalByteSize;
|
||||
}
|
||||
|
||||
int64_t MockStorageServer::estimateRangeTotalBytes(KeyRangeRef const& range,
|
||||
int64_t beginShardBytes,
|
||||
int64_t endShardBytes) {
|
||||
int64_t totalByteSize = 0;
|
||||
auto ranges = serverKeys.intersectingRanges(range);
|
||||
|
||||
// use the beginShardBytes as partial size
|
||||
if (ranges.begin().begin() < range.begin) {
|
||||
ranges.pop_front();
|
||||
totalByteSize += beginShardBytes;
|
||||
}
|
||||
// use the endShardBytes as partial size
|
||||
if (ranges.end().begin() < range.end) {
|
||||
totalByteSize += endShardBytes;
|
||||
}
|
||||
for (auto it = ranges.begin(); it != ranges.end(); ++it) {
|
||||
totalByteSize += it->cvalue().shardSize;
|
||||
}
|
||||
return totalByteSize;
|
||||
}
|
||||
|
||||
void MockStorageServer::clearRangeTotalBytes(KeyRangeRef const& range, int64_t beginShardBytes, int64_t endShardBytes) {
|
||||
auto ranges = serverKeys.intersectingRanges(range);
|
||||
|
||||
// use the beginShardBytes as partial size
|
||||
if (ranges.begin().begin() < range.begin) {
|
||||
auto delta = std::min(ranges.begin().value().shardSize, (uint64_t)beginShardBytes);
|
||||
ranges.begin().value().shardSize -= delta;
|
||||
ranges.pop_front();
|
||||
}
|
||||
// use the endShardBytes as partial size
|
||||
if (ranges.end().begin() < range.end) {
|
||||
auto delta = std::min(ranges.end().value().shardSize, (uint64_t)endShardBytes);
|
||||
ranges.end().value().shardSize -= delta;
|
||||
}
|
||||
for (auto it = ranges.begin(); it != ranges.end(); ++it) {
|
||||
it->value().shardSize = 0;
|
||||
}
|
||||
}
|
||||
|
||||
void MockStorageServer::notifyWriteMetrics(KeyRef const& key, int64_t size) {
|
||||
// update write bandwidth and iops as mock the cost of writing a mutation
|
||||
StorageMetrics s;
|
||||
// FIXME: remove the / 2 and double the related knobs.
|
||||
s.bytesWrittenPerKSecond = mvccStorageBytes(size) / 2;
|
||||
s.iosPerKSecond = 1;
|
||||
metrics.notify(key, s);
|
||||
}
|
||||
|
||||
void MockStorageServer::signalFetchKeys(const KeyRangeRef& range, int64_t rangeTotalBytes) {
|
||||
if (!allShardStatusEqual(range, MockShardStatus::COMPLETED)) {
|
||||
actors.add(MockStorageServerImpl::waitFetchKeysFinish(this, { range, rangeTotalBytes }));
|
||||
}
|
||||
}
|
||||
|
||||
void MockStorageServer::byteSampleApplySet(KeyRef const& key, int64_t kvSize) {
|
||||
// Update byteSample in memory and notify waiting metrics
|
||||
ByteSampleInfo sampleInfo = isKeyValueInSample(key, kvSize);
|
||||
auto& byteSample = metrics.byteSample.sample;
|
||||
|
||||
int64_t delta = 0;
|
||||
auto old = byteSample.find(key);
|
||||
if (old != byteSample.end())
|
||||
delta = -byteSample.getMetric(old);
|
||||
|
||||
if (sampleInfo.inSample) {
|
||||
delta += sampleInfo.sampledSize;
|
||||
byteSample.insert(key, sampleInfo.sampledSize);
|
||||
} else if (old != byteSample.end()) {
|
||||
byteSample.erase(old);
|
||||
}
|
||||
|
||||
if (delta)
|
||||
metrics.notifyBytes(key, delta);
|
||||
}
|
||||
|
||||
void MockStorageServer::byteSampleApplyClear(KeyRangeRef const& range) {
|
||||
// Update byteSample and notify waiting metrics
|
||||
|
||||
auto& byteSample = metrics.byteSample.sample;
|
||||
bool any = false;
|
||||
|
||||
if (range.begin < allKeys.end) {
|
||||
// NotifyBytes should not be called for keys past allKeys.end
|
||||
KeyRangeRef searchRange = KeyRangeRef(range.begin, std::min(range.end, allKeys.end));
|
||||
|
||||
auto r = metrics.waitMetricsMap.intersectingRanges(searchRange);
|
||||
for (auto shard = r.begin(); shard != r.end(); ++shard) {
|
||||
KeyRangeRef intersectingRange = shard.range() & range;
|
||||
int64_t bytes = byteSample.sumRange(intersectingRange.begin, intersectingRange.end);
|
||||
metrics.notifyBytes(shard, -bytes);
|
||||
any = any || bytes > 0;
|
||||
}
|
||||
}
|
||||
|
||||
if (range.end > allKeys.end && byteSample.sumRange(std::max(allKeys.end, range.begin), range.end) > 0)
|
||||
any = true;
|
||||
|
||||
if (any) {
|
||||
byteSample.eraseAsync(range.begin, range.end);
|
||||
}
|
||||
}
|
||||
|
||||
void MockGlobalState::initializeAsEmptyDatabaseMGS(const DatabaseConfiguration& conf, uint64_t defaultDiskSpace) {
|
||||
|
@ -289,12 +519,18 @@ bool MockGlobalState::serverIsSourceForShard(const UID& serverId, KeyRangeRef sh
|
|||
}
|
||||
|
||||
bool MockGlobalState::serverIsDestForShard(const UID& serverId, KeyRangeRef shard) {
|
||||
TraceEvent(SevDebug, "ServerIsDestForShard")
|
||||
.detail("ServerId", serverId)
|
||||
.detail("Keys", shard)
|
||||
.detail("Contains", allServers.count(serverId));
|
||||
|
||||
if (!allServers.count(serverId))
|
||||
return false;
|
||||
|
||||
// check serverKeys
|
||||
auto& mss = allServers.at(serverId);
|
||||
if (!mss.allShardStatusEqual(shard, MockShardStatus::INFLIGHT)) {
|
||||
if (!mss.allShardStatusIn(shard,
|
||||
{ MockShardStatus::INFLIGHT, MockShardStatus::COMPLETED, MockShardStatus::FETCHED })) {
|
||||
return false;
|
||||
}
|
||||
|
||||
|
@ -408,6 +644,105 @@ Future<Standalone<VectorRef<KeyRef>>> MockGlobalState::splitStorageMetrics(const
|
|||
return MockGlobalStateImpl::splitStorageMetrics(this, keys, limit, estimated, minSplitBytes);
|
||||
}
|
||||
|
||||
std::vector<Future<Void>> MockGlobalState::runAllMockServers() {
|
||||
std::vector<Future<Void>> futures;
|
||||
futures.reserve(allServers.size());
|
||||
for (auto& [id, _] : allServers) {
|
||||
futures.emplace_back(runMockServer(id));
|
||||
}
|
||||
return futures;
|
||||
}
|
||||
Future<Void> MockGlobalState::runMockServer(const UID& id) {
|
||||
return allServers.at(id).run();
|
||||
}
|
||||
|
||||
int64_t MockGlobalState::get(KeyRef const& key) {
|
||||
auto ids = shardMapping->getSourceServerIdsFor(key);
|
||||
int64_t randomBytes = 0;
|
||||
if (deterministicRandom()->random01() > emptyProb) {
|
||||
randomBytes = deterministicRandom()->randomInt64(minByteSize, maxByteSize + 1);
|
||||
}
|
||||
// randomly choose 1 server
|
||||
auto id = deterministicRandom()->randomChoice(ids);
|
||||
allServers.at(id).get(key, randomBytes);
|
||||
return randomBytes;
|
||||
}
|
||||
|
||||
int64_t MockGlobalState::getRange(KeyRangeRef const& range) {
|
||||
auto ranges = shardMapping->intersectingRanges(range);
|
||||
int64_t totalSize = 0;
|
||||
KeyRef begin, end;
|
||||
for (auto it = ranges.begin(); it != ranges.end(); ++it) {
|
||||
auto ids = shardMapping->getSourceServerIdsFor(it->begin());
|
||||
if (range.begin > it->begin()) {
|
||||
begin = range.begin;
|
||||
}
|
||||
if (range.end < it->end()) {
|
||||
end = range.end;
|
||||
}
|
||||
|
||||
// randomly choose 1 server
|
||||
auto id = deterministicRandom()->randomChoice(ids);
|
||||
int64_t beginSize = deterministicRandom()->randomInt64(0, SERVER_KNOBS->MIN_SHARD_BYTES),
|
||||
endSize = deterministicRandom()->randomInt64(0, SERVER_KNOBS->MIN_SHARD_BYTES);
|
||||
totalSize += allServers.at(id).getRange(KeyRangeRef(begin, end), beginSize, endSize);
|
||||
}
|
||||
return totalSize;
|
||||
}
|
||||
|
||||
int64_t MockGlobalState::set(KeyRef const& key, int valueSize, bool insert) {
|
||||
auto ids = shardMapping->getSourceServerIdsFor(key);
|
||||
int64_t oldKvBytes = 0;
|
||||
insert |= (deterministicRandom()->random01() < emptyProb);
|
||||
|
||||
if (!insert) {
|
||||
oldKvBytes = key.size() + deterministicRandom()->randomInt64(minByteSize, maxByteSize + 1);
|
||||
}
|
||||
|
||||
for (auto& id : ids) {
|
||||
allServers.at(id).set(key, valueSize + key.size(), oldKvBytes);
|
||||
}
|
||||
return oldKvBytes;
|
||||
}
|
||||
|
||||
int64_t MockGlobalState::clear(KeyRef const& key) {
|
||||
auto ids = shardMapping->getSourceServerIdsFor(key);
|
||||
int64_t randomBytes = 0;
|
||||
if (deterministicRandom()->random01() > emptyProb) {
|
||||
randomBytes = deterministicRandom()->randomInt64(minByteSize, maxByteSize + 1) + key.size();
|
||||
}
|
||||
|
||||
for (auto& id : ids) {
|
||||
allServers.at(id).clear(key, randomBytes);
|
||||
}
|
||||
return randomBytes;
|
||||
}
|
||||
|
||||
int64_t MockGlobalState::clearRange(KeyRangeRef const& range) {
|
||||
auto ranges = shardMapping->intersectingRanges(range);
|
||||
int64_t totalSize = 0;
|
||||
KeyRef begin, end;
|
||||
for (auto it = ranges.begin(); it != ranges.end(); ++it) {
|
||||
auto ids = shardMapping->getSourceServerIdsFor(it->begin());
|
||||
if (range.begin > it->begin()) {
|
||||
begin = range.begin;
|
||||
}
|
||||
if (range.end < it->end()) {
|
||||
end = range.end;
|
||||
}
|
||||
|
||||
int64_t beginSize = deterministicRandom()->randomInt64(0, SERVER_KNOBS->MIN_SHARD_BYTES),
|
||||
endSize = deterministicRandom()->randomInt64(0, SERVER_KNOBS->MIN_SHARD_BYTES);
|
||||
int64_t lastSize = -1;
|
||||
for (auto& id : ids) {
|
||||
int64_t size = allServers.at(id).clearRange(KeyRangeRef(begin, end), beginSize, endSize);
|
||||
ASSERT(lastSize == size || lastSize == -1); // every server should return the same result
|
||||
}
|
||||
totalSize += lastSize;
|
||||
}
|
||||
return totalSize;
|
||||
}
|
||||
|
||||
TEST_CASE("/MockGlobalState/initializeAsEmptyDatabaseMGS/SimpleThree") {
|
||||
BasicTestConfig testConfig;
|
||||
testConfig.simpleConfig = true;
|
||||
|
@ -445,11 +780,13 @@ struct MockGlobalStateTester {
|
|||
mss.threeWayShardSplitting(outerRange, KeyRangeRef(x1, x2), oldSize, false);
|
||||
auto ranges = mss.serverKeys.containedRanges(outerRange);
|
||||
ASSERT(ranges.begin().range() == KeyRangeRef(outerRange.begin, x1));
|
||||
ASSERT(ranges.begin().cvalue().status == oldStatus);
|
||||
ranges.pop_front();
|
||||
ASSERT(ranges.begin().range() == KeyRangeRef(x1, x2));
|
||||
ASSERT(ranges.begin().cvalue().status == oldStatus);
|
||||
ranges.pop_front();
|
||||
ASSERT(ranges.begin().range() == KeyRangeRef(x2, outerRange.end));
|
||||
ASSERT(ranges.begin().cvalue().status == oldStatus);
|
||||
ranges.pop_front();
|
||||
ASSERT(ranges.empty());
|
||||
}
|
||||
|
@ -468,6 +805,7 @@ struct MockGlobalStateTester {
|
|||
mss.twoWayShardSplitting(it->range(), x1, oldSize, false);
|
||||
auto ranges = mss.serverKeys.containedRanges(outerRange);
|
||||
ASSERT(ranges.begin().range() == KeyRangeRef(outerRange.begin, x1));
|
||||
ASSERT(ranges.begin().cvalue().status == oldStatus);
|
||||
ranges.pop_front();
|
||||
ASSERT(ranges.begin().range() == KeyRangeRef(x1, outerRange.end));
|
||||
ASSERT(ranges.begin().cvalue().status == oldStatus);
|
||||
|
@ -520,6 +858,43 @@ TEST_CASE("/MockGlobalState/MockStorageServer/SplittingFunctions") {
|
|||
return Void();
|
||||
}
|
||||
|
||||
TEST_CASE("/MockGlobalState/MockStorageServer/SetShardStatus") {
|
||||
BasicTestConfig testConfig;
|
||||
testConfig.simpleConfig = true;
|
||||
testConfig.minimumReplication = 1;
|
||||
testConfig.logAntiQuorum = 0;
|
||||
DatabaseConfiguration dbConfig = generateNormalDatabaseConfiguration(testConfig);
|
||||
TraceEvent("SetShardStatusUnitTestDbConfig").detail("Config", dbConfig.toString());
|
||||
|
||||
auto mgs = std::make_shared<MockGlobalState>();
|
||||
mgs->initializeAsEmptyDatabaseMGS(dbConfig);
|
||||
|
||||
auto& mss = mgs->allServers.at(MockGlobalState::indexToUID(1));
|
||||
mss.serverKeys.insert(allKeys, { MockShardStatus::UNSET, 0 }); // manually reset status
|
||||
|
||||
// split to 3 shards [allKeys.begin, a, b, allKeys.end]
|
||||
KeyRange testRange(KeyRangeRef("a"_sr, "b"_sr));
|
||||
mss.setShardStatus(testRange, MockShardStatus::INFLIGHT, false);
|
||||
ASSERT(mss.allShardStatusEqual(testRange, MockShardStatus::INFLIGHT));
|
||||
|
||||
// [allKeys.begin, a, ac, b, bc, allKeys.end]
|
||||
testRange = KeyRangeRef("ac"_sr, "bc"_sr);
|
||||
mss.setShardStatus(testRange, MockShardStatus::INFLIGHT, false);
|
||||
ASSERT(mss.allShardStatusEqual(testRange, MockShardStatus::INFLIGHT));
|
||||
|
||||
testRange = KeyRangeRef("b"_sr, "bc"_sr);
|
||||
mss.setShardStatus(testRange, MockShardStatus::FETCHED, false);
|
||||
ASSERT(mss.allShardStatusEqual(testRange, MockShardStatus::FETCHED));
|
||||
mss.setShardStatus(testRange, MockShardStatus::COMPLETED, false);
|
||||
ASSERT(mss.allShardStatusEqual(testRange, MockShardStatus::COMPLETED));
|
||||
mss.setShardStatus(testRange, MockShardStatus::FETCHED, false);
|
||||
ASSERT(mss.allShardStatusEqual(testRange, MockShardStatus::COMPLETED));
|
||||
|
||||
ASSERT(mss.serverKeys.size() == 5);
|
||||
|
||||
return Void();
|
||||
}
|
||||
|
||||
namespace {
|
||||
inline bool locationInfoEqualsToTeam(Reference<LocationInfo> loc, const std::vector<UID>& ids) {
|
||||
return loc->locations()->size() == ids.size() &&
|
||||
|
@ -602,15 +977,12 @@ TEST_CASE("/MockGlobalState/MockStorageServer/WaitStorageMetricsRequest") {
|
|||
|
||||
state std::shared_ptr<MockGlobalState> mgs = std::make_shared<MockGlobalState>();
|
||||
mgs->initializeAsEmptyDatabaseMGS(dbConfig);
|
||||
state ActorCollection actors;
|
||||
|
||||
ActorCollection* ptr = &actors; // get around ACTOR syntax restriction
|
||||
std::for_each(mgs->allServers.begin(), mgs->allServers.end(), [ptr](auto& server) {
|
||||
ptr->add(server.second.run());
|
||||
IFailureMonitor::failureMonitor().setStatus(server.second.ssi.address(), FailureStatus(false));
|
||||
std::for_each(mgs->allServers.begin(), mgs->allServers.end(), [](auto& server) {
|
||||
server.second.metrics.byteSample.sample.insert("something"_sr, 500000);
|
||||
});
|
||||
|
||||
state Future<Void> allServerFutures = waitForAll(mgs->runAllMockServers());
|
||||
|
||||
KeyRange testRange = allKeys;
|
||||
ShardSizeBounds bounds = ShardSizeBounds::shardSizeBoundsBeforeTrack();
|
||||
std::pair<Optional<StorageMetrics>, int> res =
|
||||
|
@ -621,3 +993,37 @@ TEST_CASE("/MockGlobalState/MockStorageServer/WaitStorageMetricsRequest") {
|
|||
ASSERT_EQ(res.first.get().bytes, 500000);
|
||||
return Void();
|
||||
}
|
||||
|
||||
TEST_CASE("/MockGlobalState/MockStorageServer/DataOpsSet") {
|
||||
BasicTestConfig testConfig;
|
||||
testConfig.simpleConfig = true;
|
||||
testConfig.minimumReplication = 1;
|
||||
testConfig.logAntiQuorum = 0;
|
||||
DatabaseConfiguration dbConfig = generateNormalDatabaseConfiguration(testConfig);
|
||||
TraceEvent("DataOpsUnitTestConfig").detail("Config", dbConfig.toString());
|
||||
state std::shared_ptr<MockGlobalState> mgs = std::make_shared<MockGlobalState>();
|
||||
mgs->initializeAsEmptyDatabaseMGS(dbConfig);
|
||||
state Future<Void> allServerFutures = waitForAll(mgs->runAllMockServers());
|
||||
|
||||
// insert
|
||||
{
|
||||
mgs->set("a"_sr, 1 * SERVER_KNOBS->BYTES_WRITTEN_UNITS_PER_SAMPLE, true);
|
||||
mgs->set("b"_sr, 2 * SERVER_KNOBS->BYTES_WRITTEN_UNITS_PER_SAMPLE, true);
|
||||
mgs->set("c"_sr, 3 * SERVER_KNOBS->BYTES_WRITTEN_UNITS_PER_SAMPLE, true);
|
||||
for (auto& server : mgs->allServers) {
|
||||
ASSERT_EQ(server.second.usedDiskSpace, 3 + 6 * SERVER_KNOBS->BYTES_WRITTEN_UNITS_PER_SAMPLE);
|
||||
}
|
||||
ShardSizeBounds bounds = ShardSizeBounds::shardSizeBoundsBeforeTrack();
|
||||
std::pair<Optional<StorageMetrics>, int> res = wait(
|
||||
mgs->waitStorageMetrics(KeyRangeRef("a"_sr, "bc"_sr), bounds.min, bounds.max, bounds.permittedError, 1, 1));
|
||||
|
||||
int64_t testSize = 2 + 3 * SERVER_KNOBS->BYTES_WRITTEN_UNITS_PER_SAMPLE;
|
||||
// SOMEDAY: how to integrate with isKeyValueInSample() better?
|
||||
if (res.first.get().bytes > 0) {
|
||||
// If sampled
|
||||
ASSERT_EQ(res.first.get().bytes, testSize);
|
||||
ASSERT_GT(res.first.get().bytesWrittenPerKSecond, 0);
|
||||
}
|
||||
}
|
||||
return Void();
|
||||
}
|
||||
|
|
|
@ -580,8 +580,8 @@ ACTOR Future<Void> logWarningAfter(const char* context, double duration, std::ve
|
|||
|
||||
// keyServer: map from keys to destination servers
|
||||
// serverKeys: two-dimension map: [servers][keys], value is the servers' state of having the keys: active(not-have),
|
||||
// complete(already has), ""(). Set keyServers[keys].dest = servers Set serverKeys[servers][keys] = active for each
|
||||
// subrange of keys that the server did not already have, complete for each subrange that it already has Set
|
||||
// complete(already has), ""(). Set keyServers[keys].dest = servers. Set serverKeys[servers][keys] = active for each
|
||||
// subrange of keys that the server did not already have, = complete for each subrange that it already has. Set
|
||||
// serverKeys[dest][keys] = "" for the dest servers of each existing shard in keys (unless that destination is a member
|
||||
// of servers OR if the source list is sufficiently degraded)
|
||||
ACTOR static Future<Void> startMoveKeys(Database occ,
|
||||
|
@ -2481,7 +2481,9 @@ ACTOR Future<Void> cleanUpDataMove(Database occ,
|
|||
return Void();
|
||||
}
|
||||
|
||||
Future<Void> rawStartMovement(Database occ, MoveKeysParams& params, std::map<UID, StorageServerInterface>& tssMapping) {
|
||||
Future<Void> rawStartMovement(Database occ,
|
||||
const MoveKeysParams& params,
|
||||
std::map<UID, StorageServerInterface>& tssMapping) {
|
||||
if (SERVER_KNOBS->SHARD_ENCODE_LOCATION_METADATA) {
|
||||
return startMoveShards(std::move(occ),
|
||||
params.dataMoveId,
|
||||
|
@ -2504,7 +2506,7 @@ Future<Void> rawStartMovement(Database occ, MoveKeysParams& params, std::map<UID
|
|||
}
|
||||
|
||||
Future<Void> rawFinishMovement(Database occ,
|
||||
MoveKeysParams& params,
|
||||
const MoveKeysParams& params,
|
||||
const std::map<UID, StorageServerInterface>& tssMapping) {
|
||||
if (SERVER_KNOBS->SHARD_ENCODE_LOCATION_METADATA) {
|
||||
return finishMoveShards(std::move(occ),
|
||||
|
|
|
@ -43,9 +43,9 @@
|
|||
#include "flow/actorcompiler.h" // has to be last include
|
||||
|
||||
#ifdef SSD_ROCKSDB_EXPERIMENTAL
|
||||
// Enforcing rocksdb version to be 6.22.1 or greater.
|
||||
static_assert(ROCKSDB_MAJOR == 6 && ROCKSDB_MINOR >= 22 && ROCKSDB_PATCH >= 1,
|
||||
"Unsupported rocksdb version. Update the rocksdb to at least 6.22.1 version");
|
||||
// Enforcing rocksdb version to be 7.7.3.
|
||||
static_assert((ROCKSDB_MAJOR == 7 && ROCKSDB_MINOR == 7 && ROCKSDB_PATCH == 3),
|
||||
"Unsupported rocksdb version. Update the rocksdb to 7.7.3 version");
|
||||
|
||||
namespace {
|
||||
|
||||
|
|
|
@ -107,7 +107,6 @@ void ShardsAffectedByTeamFailure::defineShard(KeyRangeRef keys) {
|
|||
check();
|
||||
}
|
||||
|
||||
// Move keys to destinationTeams by updating shard_teams
|
||||
void ShardsAffectedByTeamFailure::moveShard(KeyRangeRef keys, std::vector<Team> destinationTeams) {
|
||||
/*TraceEvent("ShardsAffectedByTeamFailureMove")
|
||||
.detail("KeyBegin", keys.begin)
|
||||
|
@ -158,6 +157,25 @@ void ShardsAffectedByTeamFailure::moveShard(KeyRangeRef keys, std::vector<Team>
|
|||
check();
|
||||
}
|
||||
|
||||
void ShardsAffectedByTeamFailure::rawMoveShard(KeyRangeRef keys,
|
||||
const std::vector<Team>& srcTeams,
|
||||
const std::vector<Team>& destinationTeams) {
|
||||
auto it = shard_teams.rangeContaining(keys.begin);
|
||||
std::vector<std::pair<std::pair<std::vector<Team>, std::vector<Team>>, KeyRange>> modifiedShards;
|
||||
ASSERT(it->range() == keys);
|
||||
|
||||
// erase the many teams that were associated with this one shard
|
||||
for (auto t = it->value().first.begin(); t != it->value().first.end(); ++t) {
|
||||
erase(*t, it->range());
|
||||
}
|
||||
it.value() = std::make_pair(destinationTeams, srcTeams);
|
||||
for (auto& team : destinationTeams) {
|
||||
insert(team, keys);
|
||||
}
|
||||
|
||||
check();
|
||||
}
|
||||
|
||||
void ShardsAffectedByTeamFailure::finishMove(KeyRangeRef keys) {
|
||||
auto ranges = shard_teams.containedRanges(keys);
|
||||
for (auto it = ranges.begin(); it != ranges.end(); ++it) {
|
||||
|
@ -246,3 +264,13 @@ void ShardsAffectedByTeamFailure::removeFailedServerForRange(KeyRangeRef keys, c
|
|||
auto ShardsAffectedByTeamFailure::intersectingRanges(KeyRangeRef keyRange) const -> decltype(shard_teams)::ConstRanges {
|
||||
return shard_teams.intersectingRanges(keyRange);
|
||||
}
|
||||
|
||||
std::vector<UID> ShardsAffectedByTeamFailure::getSourceServerIdsFor(KeyRef key) {
|
||||
auto teamPair = getTeamsFor(key);
|
||||
std::set<UID> res;
|
||||
auto& srcTeams = teamPair.second.empty() ? teamPair.first : teamPair.second;
|
||||
for (auto& team : srcTeams) {
|
||||
res.insert(team.servers.begin(), team.servers.end());
|
||||
}
|
||||
return std::vector<UID>(res.begin(), res.end());
|
||||
}
|
||||
|
|
|
@ -75,8 +75,8 @@ KeyRef StorageMetricSample::splitEstimate(KeyRangeRef range, int64_t offset, boo
|
|||
StorageMetrics StorageServerMetrics::getMetrics(KeyRangeRef const& keys) const {
|
||||
StorageMetrics result;
|
||||
result.bytes = byteSample.getEstimate(keys);
|
||||
result.bytesPerKSecond =
|
||||
bandwidthSample.getEstimate(keys) * SERVER_KNOBS->STORAGE_METRICS_AVERAGE_INTERVAL_PER_KSECONDS;
|
||||
result.bytesWrittenPerKSecond =
|
||||
bytesWriteSample.getEstimate(keys) * SERVER_KNOBS->STORAGE_METRICS_AVERAGE_INTERVAL_PER_KSECONDS;
|
||||
result.iosPerKSecond = iopsSample.getEstimate(keys) * SERVER_KNOBS->STORAGE_METRICS_AVERAGE_INTERVAL_PER_KSECONDS;
|
||||
result.bytesReadPerKSecond =
|
||||
bytesReadSample.getEstimate(keys) * SERVER_KNOBS->STORAGE_METRICS_AVERAGE_INTERVAL_PER_KSECONDS;
|
||||
|
@ -88,7 +88,7 @@ StorageMetrics StorageServerMetrics::getMetrics(KeyRangeRef const& keys) const {
|
|||
void StorageServerMetrics::notify(KeyRef key, StorageMetrics& metrics) {
|
||||
ASSERT(metrics.bytes == 0); // ShardNotifyMetrics
|
||||
if (g_network->isSimulated()) {
|
||||
CODE_PROBE(metrics.bytesPerKSecond != 0, "ShardNotifyMetrics bytes");
|
||||
CODE_PROBE(metrics.bytesWrittenPerKSecond != 0, "ShardNotifyMetrics bytes");
|
||||
CODE_PROBE(metrics.iosPerKSecond != 0, "ShardNotifyMetrics ios");
|
||||
CODE_PROBE(metrics.bytesReadPerKSecond != 0, "ShardNotifyMetrics bytesRead", probe::decoration::rare);
|
||||
}
|
||||
|
@ -97,9 +97,10 @@ void StorageServerMetrics::notify(KeyRef key, StorageMetrics& metrics) {
|
|||
|
||||
StorageMetrics notifyMetrics;
|
||||
|
||||
if (metrics.bytesPerKSecond)
|
||||
notifyMetrics.bytesPerKSecond = bandwidthSample.addAndExpire(key, metrics.bytesPerKSecond, expire) *
|
||||
SERVER_KNOBS->STORAGE_METRICS_AVERAGE_INTERVAL_PER_KSECONDS;
|
||||
if (metrics.bytesWrittenPerKSecond)
|
||||
notifyMetrics.bytesWrittenPerKSecond =
|
||||
bytesWriteSample.addAndExpire(key, metrics.bytesWrittenPerKSecond, expire) *
|
||||
SERVER_KNOBS->STORAGE_METRICS_AVERAGE_INTERVAL_PER_KSECONDS;
|
||||
if (metrics.iosPerKSecond)
|
||||
notifyMetrics.iosPerKSecond = iopsSample.addAndExpire(key, metrics.iosPerKSecond, expire) *
|
||||
SERVER_KNOBS->STORAGE_METRICS_AVERAGE_INTERVAL_PER_KSECONDS;
|
||||
|
@ -177,8 +178,8 @@ void StorageServerMetrics::notifyNotReadable(KeyRangeRef keys) {
|
|||
void StorageServerMetrics::poll() {
|
||||
{
|
||||
StorageMetrics m;
|
||||
m.bytesPerKSecond = SERVER_KNOBS->STORAGE_METRICS_AVERAGE_INTERVAL_PER_KSECONDS;
|
||||
bandwidthSample.poll(waitMetricsMap, m);
|
||||
m.bytesWrittenPerKSecond = SERVER_KNOBS->STORAGE_METRICS_AVERAGE_INTERVAL_PER_KSECONDS;
|
||||
bytesWriteSample.poll(waitMetricsMap, m);
|
||||
}
|
||||
{
|
||||
StorageMetrics m;
|
||||
|
@ -250,7 +251,7 @@ void StorageServerMetrics::splitMetrics(SplitMetricsRequest req) const {
|
|||
if (remaining.bytes < 2 * minSplitBytes)
|
||||
break;
|
||||
KeyRef key = req.keys.end;
|
||||
bool hasUsed = used.bytes != 0 || used.bytesPerKSecond != 0 || used.iosPerKSecond != 0;
|
||||
bool hasUsed = used.bytes != 0 || used.bytesWrittenPerKSecond != 0 || used.iosPerKSecond != 0;
|
||||
key = getSplitKey(remaining.bytes,
|
||||
estimated.bytes,
|
||||
req.limits.bytes,
|
||||
|
@ -276,13 +277,13 @@ void StorageServerMetrics::splitMetrics(SplitMetricsRequest req) const {
|
|||
lastKey,
|
||||
key,
|
||||
hasUsed);
|
||||
key = getSplitKey(remaining.bytesPerKSecond,
|
||||
estimated.bytesPerKSecond,
|
||||
req.limits.bytesPerKSecond,
|
||||
used.bytesPerKSecond,
|
||||
key = getSplitKey(remaining.bytesWrittenPerKSecond,
|
||||
estimated.bytesWrittenPerKSecond,
|
||||
req.limits.bytesWrittenPerKSecond,
|
||||
used.bytesWrittenPerKSecond,
|
||||
req.limits.infinity,
|
||||
req.isLastShard,
|
||||
bandwidthSample,
|
||||
bytesWriteSample,
|
||||
SERVER_KNOBS->STORAGE_METRICS_AVERAGE_INTERVAL_PER_KSECONDS,
|
||||
lastKey,
|
||||
key,
|
||||
|
@ -328,12 +329,12 @@ void StorageServerMetrics::getStorageMetrics(GetStorageMetricsRequest req,
|
|||
|
||||
rep.available.bytes = sb.available;
|
||||
rep.available.iosPerKSecond = 10e6;
|
||||
rep.available.bytesPerKSecond = 100e9;
|
||||
rep.available.bytesWrittenPerKSecond = 100e9;
|
||||
rep.available.bytesReadPerKSecond = 100e9;
|
||||
|
||||
rep.capacity.bytes = sb.total;
|
||||
rep.capacity.iosPerKSecond = 10e6;
|
||||
rep.capacity.bytesPerKSecond = 100e9;
|
||||
rep.capacity.bytesWrittenPerKSecond = 100e9;
|
||||
rep.capacity.bytesReadPerKSecond = 100e9;
|
||||
|
||||
rep.bytesInputRate = bytesInputRate;
|
||||
|
|
|
@ -1824,8 +1824,11 @@ Future<Void> tLogPeekMessages(PromiseType replyPromise,
|
|||
if (logData->blockingPeekLatencies.find(reqTag) == logData->blockingPeekLatencies.end()) {
|
||||
UID ssID = nondeterministicRandom()->randomUniqueID();
|
||||
std::string s = "BlockingPeekLatencies-" + reqTag.toString();
|
||||
logData->blockingPeekLatencies.try_emplace(
|
||||
reqTag, s, ssID, SERVER_KNOBS->LATENCY_METRICS_LOGGING_INTERVAL, SERVER_KNOBS->LATENCY_SAMPLE_SIZE);
|
||||
logData->blockingPeekLatencies.try_emplace(reqTag,
|
||||
s,
|
||||
ssID,
|
||||
SERVER_KNOBS->LATENCY_METRICS_LOGGING_INTERVAL,
|
||||
SERVER_KNOBS->LATENCY_SKETCH_ACCURACY);
|
||||
}
|
||||
LatencySample& sample = logData->blockingPeekLatencies.at(reqTag);
|
||||
sample.addMeasurement(latency);
|
||||
|
@ -2184,6 +2187,9 @@ ACTOR Future<Void> doQueueCommit(TLogData* self,
|
|||
if (logData->logSystem->get() &&
|
||||
(!logData->isPrimary || logData->logRouterPoppedVersion < logData->logRouterPopToVersion)) {
|
||||
logData->logRouterPoppedVersion = ver;
|
||||
DebugLogTraceEvent("LogPop", self->dbgid)
|
||||
.detail("Tag", logData->remoteTag.toString())
|
||||
.detail("Version", knownCommittedVersion);
|
||||
logData->logSystem->get()->pop(ver, logData->remoteTag, knownCommittedVersion, logData->locality);
|
||||
}
|
||||
|
||||
|
|
|
@ -21,7 +21,7 @@
|
|||
#include "fdbclient/CommitTransaction.h"
|
||||
#include "fdbclient/FDBTypes.h"
|
||||
#include "fdbclient/Tuple.h"
|
||||
#include "fdbrpc/ContinuousSample.h"
|
||||
#include "fdbrpc/DDSketch.h"
|
||||
#include "fdbrpc/simulator.h"
|
||||
#include "fdbserver/DeltaTree.h"
|
||||
#include "fdbserver/IKeyValueStore.h"
|
||||
|
|
|
@ -225,9 +225,10 @@ public:
|
|||
Future<std::vector<ProcessData>> getWorkers() const override;
|
||||
|
||||
protected:
|
||||
Future<Void> rawStartMovement(MoveKeysParams& params, std::map<UID, StorageServerInterface>& tssMapping);
|
||||
Future<Void> rawStartMovement(const MoveKeysParams& params, std::map<UID, StorageServerInterface>& tssMapping);
|
||||
|
||||
Future<Void> rawFinishMovement(MoveKeysParams& params, const std::map<UID, StorageServerInterface>& tssMapping);
|
||||
Future<Void> rawFinishMovement(const MoveKeysParams& params,
|
||||
const std::map<UID, StorageServerInterface>& tssMapping);
|
||||
};
|
||||
|
||||
struct DDMockTxnProcessorImpl;
|
||||
|
@ -237,6 +238,7 @@ struct DDMockTxnProcessorImpl;
|
|||
class DDMockTxnProcessor : public IDDTxnProcessor {
|
||||
friend struct DDMockTxnProcessorImpl;
|
||||
|
||||
protected:
|
||||
std::shared_ptr<MockGlobalState> mgs;
|
||||
|
||||
std::vector<DDShardInfo> getDDShardInfos() const;
|
||||
|
@ -292,9 +294,10 @@ public:
|
|||
Future<std::vector<ProcessData>> getWorkers() const override;
|
||||
|
||||
protected:
|
||||
void rawStartMovement(MoveKeysParams& params, std::map<UID, StorageServerInterface>& tssMapping);
|
||||
Future<Void> rawStartMovement(const MoveKeysParams& params, std::map<UID, StorageServerInterface>& tssMapping);
|
||||
|
||||
void rawFinishMovement(MoveKeysParams& params, const std::map<UID, StorageServerInterface>& tssMapping);
|
||||
Future<Void> rawFinishMovement(const MoveKeysParams& params,
|
||||
const std::map<UID, StorageServerInterface>& tssMapping);
|
||||
};
|
||||
|
||||
#endif // FOUNDATIONDB_DDTXNPROCESSOR_H
|
||||
|
|
|
@ -84,6 +84,7 @@ public:
|
|||
}
|
||||
operator int() const { return (int)value; }
|
||||
constexpr static int8_t typeCount() { return (int)__COUNT; }
|
||||
bool operator<(const RelocateReason& reason) { return (int)value < (int)reason.value; }
|
||||
|
||||
private:
|
||||
Value value;
|
||||
|
@ -283,12 +284,12 @@ public:
|
|||
const std::unordered_set<uint64_t>& excludedPhysicalShards,
|
||||
uint64_t debugID);
|
||||
|
||||
// Step 2: get a remote team which has the input physical shard
|
||||
// Return empty if no such remote team
|
||||
// May return a problematic remote team, and re-selection is required for this case
|
||||
Optional<ShardsAffectedByTeamFailure::Team> tryGetAvailableRemoteTeamWith(uint64_t inputPhysicalShardID,
|
||||
StorageMetrics const& moveInMetrics,
|
||||
uint64_t debugID);
|
||||
// Step 2: get a remote team which has the input physical shard.
|
||||
// Second field in the returned pair indicates whether this physical shard is available or not.
|
||||
// Return empty if no such remote team.
|
||||
// May return a problematic remote team, and re-selection is required for this case.
|
||||
std::pair<Optional<ShardsAffectedByTeamFailure::Team>, bool>
|
||||
tryGetAvailableRemoteTeamWith(uint64_t inputPhysicalShardID, StorageMetrics const& moveInMetrics, uint64_t debugID);
|
||||
// Invariant:
|
||||
// (1) If forceToUseNewPhysicalShard is set, use the bestTeams selected by getTeam(), and create a new physical
|
||||
// shard for the teams
|
||||
|
|
|
@ -31,19 +31,27 @@
|
|||
|
||||
struct MockGlobalStateTester;
|
||||
|
||||
// the status is roughly order by transition order, except for UNSET and EMPTY
|
||||
enum class MockShardStatus {
|
||||
EMPTY = 0, // data loss
|
||||
COMPLETED,
|
||||
UNSET,
|
||||
INFLIGHT,
|
||||
UNSET
|
||||
FETCHED, // finish fetch but not change the serverKey mapping. Only can be set by MSS itself.
|
||||
COMPLETED
|
||||
};
|
||||
|
||||
inline bool isStatusTransitionValid(MockShardStatus from, MockShardStatus to) {
|
||||
if (from == to)
|
||||
return true;
|
||||
|
||||
switch (from) {
|
||||
case MockShardStatus::UNSET:
|
||||
case MockShardStatus::EMPTY:
|
||||
return to >= MockShardStatus::INFLIGHT;
|
||||
case MockShardStatus::INFLIGHT:
|
||||
return to == MockShardStatus::COMPLETED || to == MockShardStatus::INFLIGHT || to == MockShardStatus::EMPTY;
|
||||
return to == MockShardStatus::FETCHED || to == MockShardStatus::EMPTY;
|
||||
case MockShardStatus::FETCHED:
|
||||
return to == MockShardStatus::COMPLETED;
|
||||
case MockShardStatus::COMPLETED:
|
||||
return to == MockShardStatus::EMPTY;
|
||||
default:
|
||||
|
@ -52,8 +60,10 @@ inline bool isStatusTransitionValid(MockShardStatus from, MockShardStatus to) {
|
|||
return false;
|
||||
}
|
||||
|
||||
class MockStorageServerImpl;
|
||||
class MockStorageServer : public IStorageMetricsService {
|
||||
friend struct MockGlobalStateTester;
|
||||
friend class MockStorageServerImpl;
|
||||
|
||||
ActorCollection actors;
|
||||
|
||||
|
@ -66,10 +76,15 @@ public:
|
|||
bool operator!=(const ShardInfo& a) const { return !(a == *this); }
|
||||
};
|
||||
|
||||
struct FetchKeysParams {
|
||||
KeyRange keys;
|
||||
int64_t totalRangeBytes;
|
||||
};
|
||||
|
||||
static constexpr uint64_t DEFAULT_DISK_SPACE = 1000LL * 1024 * 1024 * 1024;
|
||||
|
||||
// control plane statistics associated with a real storage server
|
||||
uint64_t usedDiskSpace = 0, availableDiskSpace = DEFAULT_DISK_SPACE;
|
||||
uint64_t totalDiskSpace = DEFAULT_DISK_SPACE, usedDiskSpace = DEFAULT_DISK_SPACE;
|
||||
|
||||
// In-memory counterpart of the `serverKeys` in system keyspace
|
||||
// the value ShardStatus is [InFlight, Completed, Empty] and metrics uint64_t is the shard size, the caveat is the
|
||||
|
@ -85,24 +100,26 @@ public:
|
|||
MockStorageServer() = default;
|
||||
|
||||
MockStorageServer(StorageServerInterface ssi, uint64_t availableDiskSpace, uint64_t usedDiskSpace = 0)
|
||||
: usedDiskSpace(usedDiskSpace), availableDiskSpace(availableDiskSpace), ssi(ssi), id(ssi.id()) {}
|
||||
: totalDiskSpace(usedDiskSpace + availableDiskSpace), usedDiskSpace(usedDiskSpace), ssi(ssi), id(ssi.id()) {}
|
||||
|
||||
MockStorageServer(const UID& id, uint64_t availableDiskSpace, uint64_t usedDiskSpace = 0)
|
||||
: MockStorageServer(StorageServerInterface(id), availableDiskSpace, usedDiskSpace) {}
|
||||
|
||||
decltype(serverKeys)::Ranges getAllRanges() { return serverKeys.ranges(); }
|
||||
|
||||
bool allShardStatusEqual(KeyRangeRef range, MockShardStatus status);
|
||||
bool allShardStatusEqual(const KeyRangeRef& range, MockShardStatus status) const;
|
||||
bool allShardStatusIn(const KeyRangeRef& range, const std::set<MockShardStatus>& status) const;
|
||||
|
||||
// change the status of range. This function may result in split to make the shard boundary align with range.begin
|
||||
// and range.end. In this case, if restrictSize==true, the sum of the split shard size is strictly equal to the old
|
||||
// large shard. Otherwise, the size are randomly generated between (min_shard_size, max_shard_size)
|
||||
void setShardStatus(KeyRangeRef range, MockShardStatus status, bool restrictSize);
|
||||
void setShardStatus(const KeyRangeRef& range, MockShardStatus status, bool restrictSize);
|
||||
|
||||
// this function removed an aligned range from server
|
||||
void removeShard(KeyRangeRef range);
|
||||
void removeShard(const KeyRangeRef& range);
|
||||
|
||||
uint64_t sumRangeSize(KeyRangeRef range) const;
|
||||
// intersecting range size
|
||||
uint64_t sumRangeSize(const KeyRangeRef& range) const;
|
||||
|
||||
void addActor(Future<Void> future) override;
|
||||
|
||||
|
@ -133,13 +150,52 @@ public:
|
|||
|
||||
Future<Void> run();
|
||||
|
||||
// data operation APIs - change the metrics sample, disk space and shard size
|
||||
|
||||
// Set key with a new value, the total bytes change from oldBytes to bytes
|
||||
void set(KeyRef const& key, int64_t bytes, int64_t oldBytes);
|
||||
// Clear key and its value of which the size is bytes
|
||||
void clear(KeyRef const& key, int64_t bytes);
|
||||
// Clear range, assuming the first and last shard within the range having size `beginShardBytes` and `endShardBytes`
|
||||
// return the total range size
|
||||
int64_t clearRange(KeyRangeRef const& range, int64_t beginShardBytes, int64_t endShardBytes);
|
||||
|
||||
// modify the metrics as like doing an n-bytes read op
|
||||
// Read key and cause bytes read overhead
|
||||
void get(KeyRef const& key, int64_t bytes);
|
||||
// Read range, assuming the first and last shard within the range having size `beginShardBytes` and `endShardBytes`,
|
||||
// return the total range size;
|
||||
int64_t getRange(KeyRangeRef const& range, int64_t beginShardBytes, int64_t endShardBytes);
|
||||
|
||||
// trigger the asynchronous fetch keys operation
|
||||
void signalFetchKeys(const KeyRangeRef& range, int64_t rangeTotalBytes);
|
||||
|
||||
protected:
|
||||
void threeWayShardSplitting(KeyRangeRef outerRange,
|
||||
KeyRangeRef innerRange,
|
||||
PromiseStream<FetchKeysParams> fetchKeysRequests;
|
||||
|
||||
void threeWayShardSplitting(const KeyRangeRef& outerRange,
|
||||
const KeyRangeRef& innerRange,
|
||||
uint64_t outerRangeSize,
|
||||
bool restrictSize);
|
||||
|
||||
void twoWayShardSplitting(KeyRangeRef range, KeyRef splitPoint, uint64_t rangeSize, bool restrictSize);
|
||||
void twoWayShardSplitting(const KeyRangeRef& range,
|
||||
const KeyRef& splitPoint,
|
||||
uint64_t rangeSize,
|
||||
bool restrictSize);
|
||||
|
||||
// Assuming the first and last shard within the range having size `beginShardBytes` and `endShardBytes`
|
||||
int64_t estimateRangeTotalBytes(KeyRangeRef const& range, int64_t beginShardBytes, int64_t endShardBytes);
|
||||
// Decrease the intersecting shard bytes as if delete the data
|
||||
void clearRangeTotalBytes(KeyRangeRef const& range, int64_t beginShardBytes, int64_t endShardBytes);
|
||||
|
||||
// Update the storage metrics as if we write a k-v pair of `size` bytes.
|
||||
void notifyWriteMetrics(KeyRef const& key, int64_t size);
|
||||
|
||||
// Update byte sample as if set a key value pair of which the size is kvSize
|
||||
void byteSampleApplySet(KeyRef const& key, int64_t kvSize);
|
||||
|
||||
// Update byte sample as if clear a whole range
|
||||
void byteSampleApplyClear(KeyRangeRef const& range);
|
||||
};
|
||||
|
||||
class MockGlobalStateImpl;
|
||||
|
@ -160,7 +216,7 @@ public:
|
|||
|
||||
// user defined parameters for mock workload purpose
|
||||
double emptyProb; // probability of doing an empty read
|
||||
uint32_t minByteSize, maxByteSize; // the size band of a point data operation
|
||||
int minByteSize, maxByteSize; // the size band of a point data operation
|
||||
bool restrictSize = true;
|
||||
|
||||
MockGlobalState() : shardMapping(new ShardsAffectedByTeamFailure) {}
|
||||
|
@ -179,7 +235,7 @@ public:
|
|||
* Shard is in-flight.
|
||||
* * In mgs.shardMapping,the destination teams is non-empty for a given shard;
|
||||
* * For each MSS belonging to the source teams, mss.serverKeys[shard] = Completed
|
||||
* * For each MSS belonging to the destination teams, mss.serverKeys[shard] = InFlight|Completed
|
||||
* * For each MSS belonging to the destination teams, mss.serverKeys[shard] = InFlight | Fetched | Completed
|
||||
* Shard is lost.
|
||||
* * In mgs.shardMapping, the destination teams is empty for the given shard;
|
||||
* * For each MSS belonging to the source teams, mss.serverKeys[shard] = Empty
|
||||
|
@ -228,6 +284,27 @@ public:
|
|||
Optional<UID> debugID,
|
||||
UseProvisionalProxies useProvisionalProxies,
|
||||
Version version) override;
|
||||
|
||||
// data ops - the key is not accurate, only the shard the key locate in matters.
|
||||
|
||||
// MGS finds the shard X contains this key, randomly generates a N-bytes read operation on that shard, which may
|
||||
// change the read sampling stats of shard X. return the random size of value
|
||||
int64_t get(KeyRef const& key);
|
||||
// For the edge shards contains the range boundaries, randomly do N1 byte and N2 byte read operations. For other
|
||||
// shards fully within the range, mock a full shard read op.
|
||||
int64_t getRange(KeyRangeRef const& range);
|
||||
// MGS finds the shard X contains this key, mock an N-bytes write to shard X, where N = valueSize + key.size().
|
||||
// Return a random number representing the old kv size
|
||||
int64_t set(KeyRef const& key, int valueSize, bool insert);
|
||||
// MGS finds the shard X contains this key, randomly generate an N-byte clear operation.
|
||||
// Return a random number representing the old kv size
|
||||
int64_t clear(KeyRef const& key);
|
||||
// Similar as getRange, but need to change shardTotalBytes because this is a clear operation.
|
||||
int64_t clearRange(KeyRangeRef const& range);
|
||||
|
||||
// convenient shortcuts for test
|
||||
std::vector<Future<Void>> runAllMockServers();
|
||||
Future<Void> runMockServer(const UID& id);
|
||||
};
|
||||
|
||||
#endif // FOUNDATIONDB_MOCKGLOBALSTATE_H
|
||||
|
|
|
@ -86,10 +86,12 @@ void seedShardServers(Arena& trArena, CommitTransactionRef& tr, std::vector<Stor
|
|||
// Called by the master server to write the very first transaction to the database
|
||||
// establishing a set of shard servers and all invariants of the systemKeys.
|
||||
|
||||
Future<Void> rawStartMovement(Database occ, MoveKeysParams& params, std::map<UID, StorageServerInterface>& tssMapping);
|
||||
Future<Void> rawStartMovement(Database occ,
|
||||
const MoveKeysParams& params,
|
||||
std::map<UID, StorageServerInterface>& tssMapping);
|
||||
|
||||
Future<Void> rawFinishMovement(Database occ,
|
||||
MoveKeysParams& params,
|
||||
const MoveKeysParams& params,
|
||||
const std::map<UID, StorageServerInterface>& tssMapping);
|
||||
// Eventually moves the given keys to the given destination team
|
||||
// Caller is responsible for cancelling it before issuing an overlapping move,
|
||||
|
|
|
@ -121,20 +121,20 @@ struct ProxyStats {
|
|||
commitLatencySample("CommitLatencyMetrics",
|
||||
id,
|
||||
SERVER_KNOBS->LATENCY_METRICS_LOGGING_INTERVAL,
|
||||
SERVER_KNOBS->LATENCY_SAMPLE_SIZE),
|
||||
SERVER_KNOBS->LATENCY_SKETCH_ACCURACY),
|
||||
commitLatencyBands("CommitLatencyBands", id, SERVER_KNOBS->STORAGE_LOGGING_DELAY),
|
||||
commitBatchingEmptyMessageRatio("CommitBatchingEmptyMessageRatio",
|
||||
id,
|
||||
SERVER_KNOBS->LATENCY_METRICS_LOGGING_INTERVAL,
|
||||
SERVER_KNOBS->LATENCY_SAMPLE_SIZE),
|
||||
SERVER_KNOBS->LATENCY_SKETCH_ACCURACY),
|
||||
commitBatchingWindowSize("CommitBatchingWindowSize",
|
||||
id,
|
||||
SERVER_KNOBS->LATENCY_METRICS_LOGGING_INTERVAL,
|
||||
SERVER_KNOBS->LATENCY_SAMPLE_SIZE),
|
||||
SERVER_KNOBS->LATENCY_SKETCH_ACCURACY),
|
||||
computeLatency("ComputeLatency",
|
||||
id,
|
||||
SERVER_KNOBS->LATENCY_METRICS_LOGGING_INTERVAL,
|
||||
SERVER_KNOBS->LATENCY_SAMPLE_SIZE),
|
||||
SERVER_KNOBS->LATENCY_SKETCH_ACCURACY),
|
||||
maxComputeNS(0), minComputeNS(1e12),
|
||||
commitBatchQueuingDist(
|
||||
Histogram::getHistogram("CommitProxy"_sr, "CommitBatchQueuing"_sr, Histogram::Unit::microseconds)),
|
||||
|
|
|
@ -36,7 +36,9 @@ public:
|
|||
bool primary;
|
||||
|
||||
Team() : primary(true) {}
|
||||
Team(std::vector<UID> const& servers, bool primary) : servers(servers), primary(primary) {}
|
||||
Team(std::vector<UID> const& servers, bool primary) : servers(servers), primary(primary) {
|
||||
ASSERT(std::is_sorted(servers.begin(), servers.end()));
|
||||
}
|
||||
|
||||
bool operator<(const Team& r) const {
|
||||
if (servers == r.servers)
|
||||
|
@ -86,11 +88,17 @@ public:
|
|||
|
||||
std::pair<std::vector<Team>, std::vector<Team>> getTeamsFor(KeyRef key);
|
||||
|
||||
std::vector<UID> getSourceServerIdsFor(KeyRef key);
|
||||
|
||||
// Shard boundaries are modified in defineShard and the content of what servers correspond to each shard is a copy
|
||||
// or union of the shards already there
|
||||
void defineShard(KeyRangeRef keys);
|
||||
// moveShard never change the shard boundary but just change the team value
|
||||
// moveShard never change the shard boundary but just change the team value. Move keys to destinationTeams by
|
||||
// updating shard_teams, the old destination teams will be added to new source teams.
|
||||
void moveShard(KeyRangeRef keys, std::vector<Team> destinationTeam);
|
||||
// This function assume keys is exactly a shard in this mapping, this function set the srcTeam and destination
|
||||
// directly without retaining the old destination team info
|
||||
void rawMoveShard(KeyRangeRef keys, const std::vector<Team>& srcTeams, const std::vector<Team>& destinationTeam);
|
||||
// finishMove never change the shard boundary but just clear the old source team value
|
||||
void finishMove(KeyRangeRef keys);
|
||||
// a convenient function for (defineShard, moveShard, finishMove) pipeline
|
||||
|
|
|
@ -79,14 +79,14 @@ private:
|
|||
struct StorageServerMetrics {
|
||||
KeyRangeMap<std::vector<PromiseStream<StorageMetrics>>> waitMetricsMap;
|
||||
StorageMetricSample byteSample;
|
||||
TransientStorageMetricSample iopsSample,
|
||||
bandwidthSample; // FIXME: iops and bandwidth calculations are not effectively tested, since they aren't
|
||||
// currently used by data distribution
|
||||
|
||||
// FIXME: iops is not effectively tested, and is not used by data distribution
|
||||
TransientStorageMetricSample iopsSample, bytesWriteSample;
|
||||
TransientStorageMetricSample bytesReadSample;
|
||||
|
||||
StorageServerMetrics()
|
||||
: byteSample(0), iopsSample(SERVER_KNOBS->IOPS_UNITS_PER_SAMPLE),
|
||||
bandwidthSample(SERVER_KNOBS->BANDWIDTH_UNITS_PER_SAMPLE),
|
||||
bytesWriteSample(SERVER_KNOBS->BYTES_WRITTEN_UNITS_PER_SAMPLE),
|
||||
bytesReadSample(SERVER_KNOBS->BYTES_READ_UNITS_PER_SAMPLE) {}
|
||||
|
||||
StorageMetrics getMetrics(KeyRangeRef const& keys) const;
|
||||
|
@ -158,7 +158,10 @@ struct ByteSampleInfo {
|
|||
|
||||
// Determines whether a key-value pair should be included in a byte sample
|
||||
// Also returns size information about the sample
|
||||
ByteSampleInfo isKeyValueInSample(KeyValueRef keyValue);
|
||||
ByteSampleInfo isKeyValueInSample(KeyRef key, int64_t totalKvSize);
|
||||
inline ByteSampleInfo isKeyValueInSample(KeyValueRef keyValue) {
|
||||
return isKeyValueInSample(keyValue.key, keyValue.key.size() + keyValue.value.size());
|
||||
}
|
||||
|
||||
class IStorageMetricsService {
|
||||
public:
|
||||
|
@ -229,6 +232,5 @@ Future<Void> serveStorageMetricsRequests(ServiceType* self, StorageServerInterfa
|
|||
}
|
||||
}
|
||||
}
|
||||
|
||||
#include "flow/unactorcompiler.h"
|
||||
#endif // FDBSERVER_STORAGEMETRICS_H
|
||||
|
|
|
@ -0,0 +1,48 @@
|
|||
/*
|
||||
* MockDDTest.h
|
||||
*
|
||||
* This source file is part of the FoundationDB open source project
|
||||
*
|
||||
* Copyright 2013-2022 Apple Inc. and the FoundationDB project authors
|
||||
*
|
||||
* Licensed under the Apache License, Version 2.0 (the "License");
|
||||
* you may not use this file except in compliance with the License.
|
||||
* You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
#ifndef FOUNDATIONDB_MOCKDDTEST_H
|
||||
#define FOUNDATIONDB_MOCKDDTEST_H
|
||||
|
||||
#include "fdbserver/workloads/workloads.actor.h"
|
||||
#include "fdbserver/DDSharedContext.h"
|
||||
#include "fdbserver/DDTxnProcessor.h"
|
||||
#include "fdbserver/MoveKeys.actor.h"
|
||||
#include "fdbclient/StorageServerInterface.h"
|
||||
|
||||
// other Mock DD workload can derive from this class to use the common settings
|
||||
struct MockDDTestWorkload : public TestWorkload {
|
||||
bool enabled;
|
||||
bool simpleConfig;
|
||||
double testDuration;
|
||||
double meanDelay = 0.05;
|
||||
double maxKeyspace = 0.1; // range space
|
||||
int maxByteSize = 1024, minByteSize = 32; // single point value size. The Key size is fixed to 16 bytes
|
||||
|
||||
std::shared_ptr<MockGlobalState> mgs;
|
||||
Reference<DDMockTxnProcessor> mock;
|
||||
|
||||
KeyRange getRandomRange(double offset) const;
|
||||
Future<Void> setup(Database const& cx) override;
|
||||
|
||||
protected:
|
||||
MockDDTestWorkload(WorkloadContext const& wcx);
|
||||
};
|
||||
|
||||
#endif // FOUNDATIONDB_MOCKDDTEST_H
|
|
@ -25,6 +25,7 @@
|
|||
#elif !defined(FDBSERVER_READWRITEWORKLOAD_ACTOR_H)
|
||||
#define FDBSERVER_READWRITEWORKLOAD_ACTOR_H
|
||||
|
||||
#include "fdbrpc/DDSketch.h"
|
||||
#include "fdbserver/workloads/workloads.actor.h"
|
||||
#include "flow/TDMetric.actor.h"
|
||||
#include "flow/actorcompiler.h" // This must be the last #include.
|
||||
|
@ -46,7 +47,7 @@ DESCR struct ReadMetric {
|
|||
|
||||
// Common ReadWrite test settings
|
||||
struct ReadWriteCommon : KVWorkload {
|
||||
static constexpr int sampleSize = 10000;
|
||||
static constexpr double sampleError = 0.01;
|
||||
friend struct ReadWriteCommonImpl;
|
||||
|
||||
// general test setting
|
||||
|
@ -75,7 +76,7 @@ struct ReadWriteCommon : KVWorkload {
|
|||
EventMetricHandle<TransactionFailureMetric> transactionFailureMetric;
|
||||
EventMetricHandle<ReadMetric> readMetric;
|
||||
PerfIntCounter aTransactions, bTransactions, retries;
|
||||
ContinuousSample<double> latencies, readLatencies, commitLatencies, GRVLatencies, fullReadLatencies;
|
||||
DDSketch<double> latencies, readLatencies, commitLatencies, GRVLatencies, fullReadLatencies;
|
||||
double readLatencyTotal;
|
||||
int readLatencyCount;
|
||||
std::vector<PerfMetric> periodicMetrics;
|
||||
|
@ -87,9 +88,9 @@ struct ReadWriteCommon : KVWorkload {
|
|||
|
||||
explicit ReadWriteCommon(WorkloadContext const& wcx)
|
||||
: KVWorkload(wcx), totalReadsMetric("ReadWrite.TotalReads"_sr), totalRetriesMetric("ReadWrite.TotalRetries"_sr),
|
||||
aTransactions("A Transactions"), bTransactions("B Transactions"), retries("Retries"), latencies(sampleSize),
|
||||
readLatencies(sampleSize), commitLatencies(sampleSize), GRVLatencies(sampleSize), fullReadLatencies(sampleSize),
|
||||
readLatencyTotal(0), readLatencyCount(0), loadTime(0.0), clientBegin(0) {
|
||||
aTransactions("A Transactions"), bTransactions("B Transactions"), retries("Retries"), latencies(sampleError),
|
||||
readLatencies(sampleError), commitLatencies(sampleError), GRVLatencies(sampleError),
|
||||
fullReadLatencies(sampleError), readLatencyTotal(0), readLatencyCount(0), loadTime(0.0), clientBegin(0) {
|
||||
|
||||
transactionSuccessMetric.init("ReadWrite.SuccessfulTransaction"_sr);
|
||||
transactionFailureMetric.init("ReadWrite.FailedTransaction"_sr);
|
||||
|
|
|
@ -102,17 +102,17 @@ struct MasterData : NonCopyable, ReferenceCounted<MasterData> {
|
|||
versionVectorTagUpdates("VersionVectorTagUpdates",
|
||||
dbgid,
|
||||
SERVER_KNOBS->LATENCY_METRICS_LOGGING_INTERVAL,
|
||||
SERVER_KNOBS->LATENCY_SAMPLE_SIZE),
|
||||
SERVER_KNOBS->LATENCY_SKETCH_ACCURACY),
|
||||
waitForPrevCommitRequests("WaitForPrevCommitRequests", cc),
|
||||
nonWaitForPrevCommitRequests("NonWaitForPrevCommitRequests", cc),
|
||||
versionVectorSizeOnCVReply("VersionVectorSizeOnCVReply",
|
||||
dbgid,
|
||||
SERVER_KNOBS->LATENCY_METRICS_LOGGING_INTERVAL,
|
||||
SERVER_KNOBS->LATENCY_SAMPLE_SIZE),
|
||||
SERVER_KNOBS->LATENCY_SKETCH_ACCURACY),
|
||||
waitForPrevLatencies("WaitForPrevLatencies",
|
||||
dbgid,
|
||||
SERVER_KNOBS->LATENCY_METRICS_LOGGING_INTERVAL,
|
||||
SERVER_KNOBS->LATENCY_SAMPLE_SIZE),
|
||||
SERVER_KNOBS->LATENCY_SKETCH_ACCURACY),
|
||||
addActor(addActor) {
|
||||
logger = cc.traceCounters("MasterMetrics", dbgid, SERVER_KNOBS->WORKER_LOGGING_INTERVAL, "MasterMetrics");
|
||||
if (forceRecovery && !myInterface.locality.dcId().present()) {
|
||||
|
|
|
@ -534,10 +534,9 @@ const int VERSION_OVERHEAD =
|
|||
sizeof(Reference<VersionedMap<KeyRef, ValueOrClearToRef>::PTreeT>)); // versioned map [ x2 for
|
||||
// createNewVersion(version+1) ], 64b
|
||||
// overhead for map
|
||||
// For both the mutation log and the versioned map.
|
||||
|
||||
static int mvccStorageBytes(MutationRef const& m) {
|
||||
return VersionedMap<KeyRef, ValueOrClearToRef>::overheadPerItem * 2 +
|
||||
(MutationRef::OVERHEAD_BYTES + m.param1.size() + m.param2.size()) * 2;
|
||||
return mvccStorageBytes(m.param1.size() + m.param2.size());
|
||||
}
|
||||
|
||||
struct FetchInjectionInfo {
|
||||
|
@ -1274,48 +1273,48 @@ public:
|
|||
readLatencySample("ReadLatencyMetrics",
|
||||
self->thisServerID,
|
||||
SERVER_KNOBS->LATENCY_METRICS_LOGGING_INTERVAL,
|
||||
SERVER_KNOBS->LATENCY_SAMPLE_SIZE),
|
||||
SERVER_KNOBS->LATENCY_SKETCH_ACCURACY),
|
||||
readKeyLatencySample("GetKeyMetrics",
|
||||
self->thisServerID,
|
||||
SERVER_KNOBS->LATENCY_METRICS_LOGGING_INTERVAL,
|
||||
SERVER_KNOBS->LATENCY_SAMPLE_SIZE),
|
||||
SERVER_KNOBS->LATENCY_SKETCH_ACCURACY),
|
||||
readValueLatencySample("GetValueMetrics",
|
||||
self->thisServerID,
|
||||
SERVER_KNOBS->LATENCY_METRICS_LOGGING_INTERVAL,
|
||||
SERVER_KNOBS->LATENCY_SAMPLE_SIZE),
|
||||
SERVER_KNOBS->LATENCY_SKETCH_ACCURACY),
|
||||
readRangeLatencySample("GetRangeMetrics",
|
||||
self->thisServerID,
|
||||
SERVER_KNOBS->LATENCY_METRICS_LOGGING_INTERVAL,
|
||||
SERVER_KNOBS->LATENCY_SAMPLE_SIZE),
|
||||
SERVER_KNOBS->LATENCY_SKETCH_ACCURACY),
|
||||
readVersionWaitSample("ReadVersionWaitMetrics",
|
||||
self->thisServerID,
|
||||
SERVER_KNOBS->LATENCY_METRICS_LOGGING_INTERVAL,
|
||||
SERVER_KNOBS->LATENCY_SAMPLE_SIZE),
|
||||
SERVER_KNOBS->LATENCY_SKETCH_ACCURACY),
|
||||
readQueueWaitSample("ReadQueueWaitMetrics",
|
||||
self->thisServerID,
|
||||
SERVER_KNOBS->LATENCY_METRICS_LOGGING_INTERVAL,
|
||||
SERVER_KNOBS->LATENCY_SAMPLE_SIZE),
|
||||
SERVER_KNOBS->LATENCY_SKETCH_ACCURACY),
|
||||
readLatencyBands("ReadLatencyBands", self->thisServerID, SERVER_KNOBS->STORAGE_LOGGING_DELAY),
|
||||
mappedRangeSample("GetMappedRangeMetrics",
|
||||
self->thisServerID,
|
||||
SERVER_KNOBS->LATENCY_METRICS_LOGGING_INTERVAL,
|
||||
SERVER_KNOBS->LATENCY_SAMPLE_SIZE),
|
||||
SERVER_KNOBS->LATENCY_SKETCH_ACCURACY),
|
||||
mappedRangeRemoteSample("GetMappedRangeRemoteMetrics",
|
||||
self->thisServerID,
|
||||
SERVER_KNOBS->LATENCY_METRICS_LOGGING_INTERVAL,
|
||||
SERVER_KNOBS->LATENCY_SAMPLE_SIZE),
|
||||
SERVER_KNOBS->LATENCY_SKETCH_ACCURACY),
|
||||
mappedRangeLocalSample("GetMappedRangeLocalMetrics",
|
||||
self->thisServerID,
|
||||
SERVER_KNOBS->LATENCY_METRICS_LOGGING_INTERVAL,
|
||||
SERVER_KNOBS->LATENCY_SAMPLE_SIZE),
|
||||
SERVER_KNOBS->LATENCY_SKETCH_ACCURACY),
|
||||
kvReadRangeLatencySample("KVGetRangeMetrics",
|
||||
self->thisServerID,
|
||||
SERVER_KNOBS->LATENCY_METRICS_LOGGING_INTERVAL,
|
||||
SERVER_KNOBS->LATENCY_SAMPLE_SIZE),
|
||||
SERVER_KNOBS->LATENCY_SKETCH_ACCURACY),
|
||||
updateLatencySample("UpdateLatencyMetrics",
|
||||
self->thisServerID,
|
||||
SERVER_KNOBS->LATENCY_METRICS_LOGGING_INTERVAL,
|
||||
SERVER_KNOBS->LATENCY_SAMPLE_SIZE) {
|
||||
SERVER_KNOBS->LATENCY_SKETCH_ACCURACY) {
|
||||
specialCounter(cc, "LastTLogVersion", [self]() { return self->lastTLogVersion; });
|
||||
specialCounter(cc, "Version", [self]() { return self->version.get(); });
|
||||
specialCounter(cc, "StorageVersion", [self]() { return self->storageVersion(); });
|
||||
|
@ -2126,7 +2125,7 @@ ACTOR Future<Void> getValueQ(StorageServer* data, GetValueRequest req) {
|
|||
|
||||
/*
|
||||
StorageMetrics m;
|
||||
m.bytesPerKSecond = req.key.size() + (v.present() ? v.get().size() : 0);
|
||||
m.bytesWrittenPerKSecond = req.key.size() + (v.present() ? v.get().size() : 0);
|
||||
m.iosPerKSecond = 1;
|
||||
data->metrics.notify(req.key, m);
|
||||
*/
|
||||
|
@ -5828,7 +5827,8 @@ void applyMutation(StorageServer* self,
|
|||
// m is expected to be in arena already
|
||||
// Clear split keys are added to arena
|
||||
StorageMetrics metrics;
|
||||
metrics.bytesPerKSecond = mvccStorageBytes(m) / 2;
|
||||
// FIXME: remove the / 2 and double the related knobs.
|
||||
metrics.bytesWrittenPerKSecond = mvccStorageBytes(m) / 2; // comparable to counter.bytesInput / 2
|
||||
metrics.iosPerKSecond = 1;
|
||||
self->metrics.notify(m.param1, metrics);
|
||||
|
||||
|
@ -10215,11 +10215,11 @@ Future<bool> StorageServerDisk::restoreDurableState() {
|
|||
|
||||
// Determines whether a key-value pair should be included in a byte sample
|
||||
// Also returns size information about the sample
|
||||
ByteSampleInfo isKeyValueInSample(KeyValueRef keyValue) {
|
||||
ByteSampleInfo isKeyValueInSample(const KeyRef key, int64_t totalKvSize) {
|
||||
ASSERT(totalKvSize >= key.size());
|
||||
ByteSampleInfo info;
|
||||
|
||||
const KeyRef key = keyValue.key;
|
||||
info.size = key.size() + keyValue.value.size();
|
||||
info.size = totalKvSize;
|
||||
|
||||
uint32_t a = 0;
|
||||
uint32_t b = 0;
|
||||
|
@ -10354,12 +10354,14 @@ ACTOR Future<Void> waitMetrics(StorageServerMetrics* self, WaitMetricsRequest re
|
|||
// all the messages for one clear or set have been dispatched.
|
||||
|
||||
/*StorageMetrics m = getMetrics( data, req.keys );
|
||||
bool b = ( m.bytes != metrics.bytes || m.bytesPerKSecond != metrics.bytesPerKSecond ||
|
||||
m.iosPerKSecond != metrics.iosPerKSecond ); if (b) { printf("keys: '%s' - '%s' @%p\n",
|
||||
bool b = ( m.bytes != metrics.bytes || m.bytesWrittenPerKSecond !=
|
||||
metrics.bytesWrittenPerKSecond
|
||||
|| m.iosPerKSecond != metrics.iosPerKSecond ); if (b) { printf("keys: '%s' - '%s' @%p\n",
|
||||
printable(req.keys.begin).c_str(), printable(req.keys.end).c_str(), this);
|
||||
printf("waitMetrics: desync %d (%lld %lld %lld) != (%lld %lld %lld); +(%lld %lld %lld)\n",
|
||||
b, m.bytes, m.bytesPerKSecond, m.iosPerKSecond, metrics.bytes, metrics.bytesPerKSecond,
|
||||
metrics.iosPerKSecond, c.bytes, c.bytesPerKSecond, c.iosPerKSecond);
|
||||
b, m.bytes, m.bytesWrittenPerKSecond, m.iosPerKSecond, metrics.bytes,
|
||||
metrics.bytesWrittenPerKSecond, metrics.iosPerKSecond, c.bytes, c.bytesWrittenPerKSecond,
|
||||
c.iosPerKSecond);
|
||||
|
||||
}*/
|
||||
}
|
||||
|
|
|
@ -18,7 +18,6 @@
|
|||
* limitations under the License.
|
||||
*/
|
||||
|
||||
#include "fdbrpc/ContinuousSample.h"
|
||||
#include "fdbclient/NativeAPI.actor.h"
|
||||
#include "fdbserver/TesterInterface.actor.h"
|
||||
#include "fdbserver/workloads/BulkSetup.actor.h"
|
||||
|
|
|
@ -18,7 +18,6 @@
|
|||
* limitations under the License.
|
||||
*/
|
||||
|
||||
#include "fdbrpc/ContinuousSample.h"
|
||||
#include "fdbclient/NativeAPI.actor.h"
|
||||
#include "fdbserver/TesterInterface.actor.h"
|
||||
#include "fdbserver/workloads/workloads.actor.h"
|
||||
|
|
|
@ -18,7 +18,7 @@
|
|||
* limitations under the License.
|
||||
*/
|
||||
|
||||
#include "fdbrpc/ContinuousSample.h"
|
||||
#include "fdbrpc/DDSketch.h"
|
||||
#include "fdbclient/NativeAPI.actor.h"
|
||||
#include "fdbserver/TesterInterface.actor.h"
|
||||
#include "fdbserver/workloads/workloads.actor.h"
|
||||
|
@ -34,11 +34,10 @@ struct BulkLoadWorkload : TestWorkload {
|
|||
|
||||
std::vector<Future<Void>> clients;
|
||||
PerfIntCounter transactions, retries;
|
||||
ContinuousSample<double> latencies;
|
||||
DDSketch<double> latencies;
|
||||
|
||||
BulkLoadWorkload(WorkloadContext const& wcx)
|
||||
: TestWorkload(wcx), clientCount(wcx.clientCount), transactions("Transactions"), retries("Retries"),
|
||||
latencies(2000) {
|
||||
: TestWorkload(wcx), clientCount(wcx.clientCount), transactions("Transactions"), retries("Retries"), latencies() {
|
||||
testDuration = getOption(options, "testDuration"_sr, 10.0);
|
||||
actorCount = getOption(options, "actorCount"_sr, 20);
|
||||
writesPerTransaction = getOption(options, "writesPerTransaction"_sr, 10);
|
||||
|
|
|
@ -18,7 +18,7 @@
|
|||
* limitations under the License.
|
||||
*/
|
||||
|
||||
#include "fdbrpc/ContinuousSample.h"
|
||||
#include "fdbrpc/DDSketch.h"
|
||||
#include "fdbclient/NativeAPI.actor.h"
|
||||
#include "fdbserver/TesterInterface.actor.h"
|
||||
#include "fdbserver/workloads/workloads.actor.h"
|
||||
|
@ -33,10 +33,10 @@ struct DDBalanceWorkload : TestWorkload {
|
|||
|
||||
std::vector<Future<Void>> clients;
|
||||
PerfIntCounter bin_shifts, operations, retries;
|
||||
ContinuousSample<double> latencies;
|
||||
DDSketch<double> latencies;
|
||||
|
||||
DDBalanceWorkload(WorkloadContext const& wcx)
|
||||
: TestWorkload(wcx), bin_shifts("Bin_Shifts"), operations("Operations"), retries("Retries"), latencies(2000) {
|
||||
: TestWorkload(wcx), bin_shifts("Bin_Shifts"), operations("Operations"), retries("Retries"), latencies() {
|
||||
testDuration = getOption(options, "testDuration"_sr, 10.0);
|
||||
binCount = getOption(options, "binCount"_sr, 1000);
|
||||
writesPerTransaction = getOption(options, "writesPerTransaction"_sr, 1);
|
||||
|
|
|
@ -18,7 +18,6 @@
|
|||
* limitations under the License.
|
||||
*/
|
||||
|
||||
#include "fdbrpc/ContinuousSample.h"
|
||||
#include "fdbclient/NativeAPI.actor.h"
|
||||
#include "fdbserver/TesterInterface.actor.h"
|
||||
#include "fdbclient/ReadYourWrites.h"
|
||||
|
|
|
@ -18,7 +18,7 @@
|
|||
* limitations under the License.
|
||||
*/
|
||||
|
||||
#include "fdbrpc/ContinuousSample.h"
|
||||
#include "fdbrpc/DDSketch.h"
|
||||
#include "fdbclient/NativeAPI.actor.h"
|
||||
#include "fdbserver/TesterInterface.actor.h"
|
||||
#include "fdbserver/workloads/workloads.actor.h"
|
||||
|
@ -33,8 +33,8 @@ struct FileSystemWorkload : TestWorkload {
|
|||
|
||||
std::vector<Future<Void>> clients;
|
||||
PerfIntCounter queries, writes;
|
||||
ContinuousSample<double> latencies;
|
||||
ContinuousSample<double> writeLatencies;
|
||||
DDSketch<double> latencies;
|
||||
DDSketch<double> writeLatencies;
|
||||
|
||||
class FileSystemOp {
|
||||
public:
|
||||
|
@ -44,7 +44,7 @@ struct FileSystemWorkload : TestWorkload {
|
|||
};
|
||||
|
||||
FileSystemWorkload(WorkloadContext const& wcx)
|
||||
: TestWorkload(wcx), queries("Queries"), writes("Latency"), latencies(2500), writeLatencies(1000) {
|
||||
: TestWorkload(wcx), queries("Queries"), writes("Latency"), latencies(), writeLatencies() {
|
||||
testDuration = getOption(options, "testDuration"_sr, 10.0);
|
||||
transactionsPerSecond = getOption(options, "transactionsPerSecond"_sr, 5000.0) / clientCount;
|
||||
double allowedLatency = getOption(options, "allowedLatency"_sr, 0.250);
|
||||
|
|
|
@ -28,13 +28,21 @@
|
|||
#include "fdbclient/VersionedMap.h"
|
||||
#include "flow/actorcompiler.h" // This must be the last #include.
|
||||
|
||||
std::string describe(const DDShardInfo& a) {
|
||||
std::string res = "key: " + a.key.toString() + "\n";
|
||||
res += "\tprimarySrc: " + describe(a.primarySrc) + "\n";
|
||||
res += "\tprimaryDest: " + describe(a.primaryDest) + "\n";
|
||||
res += "\tremoteSrc: " + describe(a.remoteSrc) + "\n";
|
||||
res += "\tremoteDest: " + describe(a.remoteDest) + "\n";
|
||||
return res;
|
||||
}
|
||||
bool compareShardInfo(const DDShardInfo& a, const DDShardInfo& other) {
|
||||
// Mock DD just care about the server<->key mapping in DDShardInfo
|
||||
bool result = a.key == other.key && a.hasDest == other.hasDest && a.primaryDest == other.primaryDest &&
|
||||
a.primarySrc == other.primarySrc && a.remoteSrc == other.remoteSrc &&
|
||||
a.remoteDest == other.remoteDest;
|
||||
if (!result) {
|
||||
std::cout << a.key.toHexString() << " | " << other.key.toHexString() << "\n";
|
||||
std::cout << a.key.toStringView() << " | " << other.key.toStringView() << "\n";
|
||||
std::cout << a.hasDest << " | " << other.hasDest << "\n";
|
||||
std::cout << describe(a.primarySrc) << " | " << describe(other.primarySrc) << "\n";
|
||||
std::cout << describe(a.primaryDest) << " | " << describe(other.primaryDest) << "\n";
|
||||
|
@ -46,24 +54,39 @@ bool compareShardInfo(const DDShardInfo& a, const DDShardInfo& other) {
|
|||
|
||||
void verifyInitDataEqual(Reference<InitialDataDistribution> real, Reference<InitialDataDistribution> mock) {
|
||||
// Mock DD just care about the team list and server<->key mapping are consistent with the real cluster
|
||||
if (real->shards.size() != mock->shards.size()) {
|
||||
std::cout << "shardBoundaries: real v.s. mock \n";
|
||||
for (auto& shard : real->shards) {
|
||||
std::cout << describe(shard);
|
||||
}
|
||||
std::cout << " ------- \n";
|
||||
for (auto& shard : mock->shards) {
|
||||
std::cout << describe(shard);
|
||||
}
|
||||
}
|
||||
ASSERT_EQ(real->shards.size(), mock->shards.size());
|
||||
ASSERT(std::equal(
|
||||
real->shards.begin(), real->shards.end(), mock->shards.begin(), mock->shards.end(), compareShardInfo));
|
||||
std::cout << describe(real->primaryTeams) << " | " << describe(mock->primaryTeams) << "\n";
|
||||
ASSERT(real->primaryTeams == mock->primaryTeams);
|
||||
|
||||
if (real->primaryTeams != mock->primaryTeams) {
|
||||
std::cout << describe(real->primaryTeams) << " | " << describe(mock->primaryTeams) << "\n";
|
||||
ASSERT(false);
|
||||
}
|
||||
|
||||
ASSERT(real->remoteTeams == mock->remoteTeams);
|
||||
ASSERT_EQ(real->shards.size(), mock->shards.size());
|
||||
}
|
||||
|
||||
// testers expose protected methods
|
||||
class DDMockTxnProcessorTester : public DDMockTxnProcessor {
|
||||
public:
|
||||
explicit DDMockTxnProcessorTester(std::shared_ptr<MockGlobalState> mgs = nullptr) : DDMockTxnProcessor(mgs) {}
|
||||
void testRawStartMovement(MoveKeysParams& params, std::map<UID, StorageServerInterface>& tssMapping) {
|
||||
rawStartMovement(params, tssMapping);
|
||||
Future<Void> testRawStartMovement(MoveKeysParams& params, std::map<UID, StorageServerInterface>& tssMapping) {
|
||||
return rawStartMovement(params, tssMapping);
|
||||
}
|
||||
|
||||
void testRawFinishMovement(MoveKeysParams& params, const std::map<UID, StorageServerInterface>& tssMapping) {
|
||||
rawFinishMovement(params, tssMapping);
|
||||
Future<Void> testRawFinishMovement(MoveKeysParams& params,
|
||||
const std::map<UID, StorageServerInterface>& tssMapping) {
|
||||
return rawFinishMovement(params, tssMapping);
|
||||
}
|
||||
};
|
||||
|
||||
|
@ -72,12 +95,12 @@ public:
|
|||
explicit DDTxnProcessorTester(Database cx) : DDTxnProcessor(cx) {}
|
||||
|
||||
Future<Void> testRawStartMovement(MoveKeysParams& params, std::map<UID, StorageServerInterface>& tssMapping) {
|
||||
return this->rawStartMovement(params, tssMapping);
|
||||
return rawStartMovement(params, tssMapping);
|
||||
}
|
||||
|
||||
Future<Void> testRawFinishMovement(MoveKeysParams& params,
|
||||
const std::map<UID, StorageServerInterface>& tssMapping) {
|
||||
return this->rawFinishMovement(params, tssMapping);
|
||||
return rawFinishMovement(params, tssMapping);
|
||||
}
|
||||
};
|
||||
|
||||
|
@ -85,6 +108,7 @@ public:
|
|||
struct IDDTxnProcessorApiWorkload : TestWorkload {
|
||||
static constexpr auto NAME = "IDDTxnProcessorApiCorrectness";
|
||||
bool enabled;
|
||||
bool testStartOnly;
|
||||
double testDuration;
|
||||
double meanDelay = 0.05;
|
||||
double maxKeyspace = 0.1;
|
||||
|
@ -95,12 +119,14 @@ struct IDDTxnProcessorApiWorkload : TestWorkload {
|
|||
std::shared_ptr<DDMockTxnProcessorTester> mock;
|
||||
|
||||
Reference<InitialDataDistribution> realInitDD;
|
||||
std::set<Key> boundaries;
|
||||
|
||||
IDDTxnProcessorApiWorkload(WorkloadContext const& wcx) : TestWorkload(wcx), ddContext(UID()) {
|
||||
enabled = !clientId && g_network->isSimulated(); // only do this on the "first" client
|
||||
testDuration = getOption(options, "testDuration"_sr, 10.0);
|
||||
meanDelay = getOption(options, "meanDelay"_sr, meanDelay);
|
||||
maxKeyspace = getOption(options, "maxKeyspace"_sr, maxKeyspace);
|
||||
testStartOnly = getOption(options, "testStartOnly"_sr, false);
|
||||
}
|
||||
|
||||
Future<Void> setup(Database const& cx) override { return enabled ? _setup(cx, this) : Void(); }
|
||||
|
@ -131,13 +157,44 @@ struct IDDTxnProcessorApiWorkload : TestWorkload {
|
|||
throw;
|
||||
}
|
||||
}
|
||||
self->updateBoundaries();
|
||||
return Void();
|
||||
}
|
||||
|
||||
// according to boundaries, generate valid ranges for moveKeys operation
|
||||
KeyRange getRandomKeys() const {
|
||||
double len = deterministicRandom()->random01() * this->maxKeyspace;
|
||||
double pos = deterministicRandom()->random01() * (1.0 - len);
|
||||
return KeyRangeRef(doubleToTestKey(pos), doubleToTestKey(pos + len));
|
||||
// merge or split operations
|
||||
Key begin, end;
|
||||
if (deterministicRandom()->coinflip()) {
|
||||
// pure move
|
||||
if (boundaries.size() == 2) {
|
||||
begin = *boundaries.begin();
|
||||
end = *boundaries.rbegin();
|
||||
} else {
|
||||
// merge shard
|
||||
int a = deterministicRandom()->randomInt(0, boundaries.size() - 1);
|
||||
int b = deterministicRandom()->randomInt(a + 1, boundaries.size());
|
||||
auto it = boundaries.begin();
|
||||
std::advance(it, a);
|
||||
begin = *it;
|
||||
std::advance(it, b - a);
|
||||
end = *it;
|
||||
}
|
||||
} else {
|
||||
// split
|
||||
double start = deterministicRandom()->random01() * this->maxKeyspace;
|
||||
begin = doubleToTestKey(start);
|
||||
auto it = boundaries.upper_bound(begin);
|
||||
ASSERT(it != boundaries.end()); // allKeys.end is larger than any random keys here
|
||||
|
||||
double len = deterministicRandom()->random01() * (1 - maxKeyspace);
|
||||
end = doubleToTestKey(start + len);
|
||||
if (end > *it || deterministicRandom()->coinflip()) {
|
||||
end = *it;
|
||||
}
|
||||
}
|
||||
|
||||
return KeyRangeRef(begin, end);
|
||||
}
|
||||
|
||||
std::vector<UID> getRandomTeam() {
|
||||
|
@ -154,6 +211,13 @@ struct IDDTxnProcessorApiWorkload : TestWorkload {
|
|||
return result;
|
||||
}
|
||||
|
||||
void updateBoundaries() {
|
||||
boundaries.clear();
|
||||
for (auto& shard : realInitDD->shards) {
|
||||
boundaries.insert(boundaries.end(), shard.key);
|
||||
}
|
||||
}
|
||||
|
||||
ACTOR Future<Void> _setup(Database cx, IDDTxnProcessorApiWorkload* self) {
|
||||
int oldMode = wait(setDDMode(cx, 0));
|
||||
TraceEvent("IDDTxnApiTestStartModeSetting").detail("OldValue", oldMode).log();
|
||||
|
@ -165,7 +229,6 @@ struct IDDTxnProcessorApiWorkload : TestWorkload {
|
|||
// FIXME: add support for generating random teams across DCs
|
||||
ASSERT_EQ(self->ddContext.usableRegions(), 1);
|
||||
wait(readRealInitialDataDistribution(self));
|
||||
|
||||
return Void();
|
||||
}
|
||||
|
||||
|
@ -189,18 +252,23 @@ struct IDDTxnProcessorApiWorkload : TestWorkload {
|
|||
|
||||
verifyInitDataEqual(self->realInitDD, mockInitData);
|
||||
|
||||
// wait(timeout(reportErrors(self->worker(cx, self), "IDDTxnProcessorApiWorkload"), self->testDuration,
|
||||
// Void()));
|
||||
wait(timeout(reportErrors(self->worker(cx, self), "IDDTxnProcessorApiWorkload"), self->testDuration, Void()));
|
||||
|
||||
// Always set the DD mode back, even if we die with an error
|
||||
TraceEvent("IDDTxnApiTestDoneMoving").log();
|
||||
wait(success(setDDMode(cx, 1)));
|
||||
TraceEvent("IDDTxnApiTestDoneModeSetting").log();
|
||||
int oldValue = wait(setDDMode(cx, 1));
|
||||
TraceEvent("IDDTxnApiTestDoneModeSetting").detail("OldValue", oldValue);
|
||||
return Void();
|
||||
}
|
||||
|
||||
void verifyServerKeyDest(MoveKeysParams& params) const {
|
||||
// check destination servers
|
||||
for (auto& id : params.destinationTeam) {
|
||||
ASSERT(mgs->serverIsDestForShard(id, params.keys));
|
||||
}
|
||||
}
|
||||
ACTOR static Future<Void> testRawMovementApi(IDDTxnProcessorApiWorkload* self) {
|
||||
state TraceInterval relocateShardInterval("RelocateShard");
|
||||
state TraceInterval relocateShardInterval("RelocateShard_TestRawMovementApi");
|
||||
state FlowLock fl1(1);
|
||||
state FlowLock fl2(1);
|
||||
state std::map<UID, StorageServerInterface> emptyTssMapping;
|
||||
|
@ -209,32 +277,36 @@ struct IDDTxnProcessorApiWorkload : TestWorkload {
|
|||
params.startMoveKeysParallelismLock = &fl1;
|
||||
params.finishMoveKeysParallelismLock = &fl2;
|
||||
params.relocationIntervalId = relocateShardInterval.pairID;
|
||||
TraceEvent(SevDebug, relocateShardInterval.begin(), relocateShardInterval.pairID)
|
||||
.detail("Key", params.keys)
|
||||
.detail("Dest", params.destinationTeam);
|
||||
|
||||
// test start
|
||||
self->mock->testRawStartMovement(params, emptyTssMapping);
|
||||
wait(self->real->testRawStartMovement(params, emptyTssMapping));
|
||||
loop {
|
||||
params.dataMovementComplete.reset();
|
||||
wait(store(params.lock, self->real->takeMoveKeysLock(UID())));
|
||||
try {
|
||||
// test start
|
||||
wait(self->mock->testRawStartMovement(params, emptyTssMapping));
|
||||
wait(self->real->testRawStartMovement(params, emptyTssMapping));
|
||||
|
||||
// read initial data again
|
||||
wait(readRealInitialDataDistribution(self));
|
||||
mockInitData = self->mock
|
||||
->getInitialDataDistribution(self->ddContext.id(),
|
||||
self->ddContext.lock,
|
||||
{},
|
||||
self->ddContext.ddEnabledState.get(),
|
||||
SkipDDModeCheck::True)
|
||||
.get();
|
||||
self->verifyServerKeyDest(params);
|
||||
// test finish or started but cancelled movement
|
||||
if (self->testStartOnly || deterministicRandom()->coinflip()) {
|
||||
CODE_PROBE(true, "RawMovementApi partial started", probe::decoration::rare);
|
||||
break;
|
||||
}
|
||||
|
||||
verifyInitDataEqual(self->realInitDD, mockInitData);
|
||||
|
||||
// test finish or started but cancelled movement
|
||||
if (deterministicRandom()->coinflip()) {
|
||||
CODE_PROBE(true, "RawMovementApi partial started", probe::decoration::rare);
|
||||
return Void();
|
||||
wait(self->mock->testRawFinishMovement(params, emptyTssMapping));
|
||||
wait(self->real->testRawFinishMovement(params, emptyTssMapping));
|
||||
break;
|
||||
} catch (Error& e) {
|
||||
if (e.code() != error_code_movekeys_conflict)
|
||||
throw;
|
||||
wait(delay(FLOW_KNOBS->PREVENT_FAST_SPIN_DELAY));
|
||||
// Keep trying to get the moveKeysLock
|
||||
}
|
||||
}
|
||||
|
||||
self->mock->testRawFinishMovement(params, emptyTssMapping);
|
||||
wait(self->real->testRawFinishMovement(params, emptyTssMapping));
|
||||
|
||||
// read initial data again
|
||||
wait(readRealInitialDataDistribution(self));
|
||||
mockInitData = self->mock
|
||||
|
@ -246,6 +318,11 @@ struct IDDTxnProcessorApiWorkload : TestWorkload {
|
|||
.get();
|
||||
|
||||
verifyInitDataEqual(self->realInitDD, mockInitData);
|
||||
TraceEvent(SevDebug, relocateShardInterval.end(), relocateShardInterval.pairID);
|
||||
// The simulator have chances generating a scenario when after the first setupMockGlobalState call, there is a
|
||||
// new storage server join the cluster, there's no way for mock DD to know the new storage server without
|
||||
// calling setupMockGlobalState again.
|
||||
self->mock->setupMockGlobalState(self->realInitDD);
|
||||
return Void();
|
||||
}
|
||||
|
||||
|
@ -253,11 +330,12 @@ struct IDDTxnProcessorApiWorkload : TestWorkload {
|
|||
state MoveKeysLock lock = wait(takeMoveKeysLock(self->real->context(), UID()));
|
||||
|
||||
KeyRange keys = self->getRandomKeys();
|
||||
std::vector<UID> destTeams = self->getRandomTeam();
|
||||
std::vector<UID> destTeam = self->getRandomTeam();
|
||||
std::sort(destTeam.begin(), destTeam.end());
|
||||
return MoveKeysParams{ deterministicRandom()->randomUniqueID(),
|
||||
keys,
|
||||
destTeams,
|
||||
destTeams,
|
||||
destTeam,
|
||||
destTeam,
|
||||
lock,
|
||||
Promise<Void>(),
|
||||
nullptr,
|
||||
|
@ -269,7 +347,7 @@ struct IDDTxnProcessorApiWorkload : TestWorkload {
|
|||
}
|
||||
|
||||
ACTOR static Future<Void> testMoveKeys(IDDTxnProcessorApiWorkload* self) {
|
||||
state TraceInterval relocateShardInterval("RelocateShard");
|
||||
state TraceInterval relocateShardInterval("RelocateShard_TestMoveKeys");
|
||||
state FlowLock fl1(1);
|
||||
state FlowLock fl2(1);
|
||||
state std::map<UID, StorageServerInterface> emptyTssMapping;
|
||||
|
@ -278,9 +356,24 @@ struct IDDTxnProcessorApiWorkload : TestWorkload {
|
|||
params.startMoveKeysParallelismLock = &fl1;
|
||||
params.finishMoveKeysParallelismLock = &fl2;
|
||||
params.relocationIntervalId = relocateShardInterval.pairID;
|
||||
TraceEvent(SevDebug, relocateShardInterval.begin(), relocateShardInterval.pairID)
|
||||
.detail("Key", params.keys)
|
||||
.detail("Dest", params.destinationTeam);
|
||||
|
||||
self->mock->moveKeys(params);
|
||||
wait(self->real->moveKeys(params));
|
||||
loop {
|
||||
params.dataMovementComplete.reset();
|
||||
wait(store(params.lock, self->real->takeMoveKeysLock(UID())));
|
||||
try {
|
||||
wait(self->mock->moveKeys(params));
|
||||
wait(self->real->moveKeys(params));
|
||||
break;
|
||||
} catch (Error& e) {
|
||||
if (e.code() != error_code_movekeys_conflict)
|
||||
throw;
|
||||
wait(delay(FLOW_KNOBS->PREVENT_FAST_SPIN_DELAY));
|
||||
// Keep trying to get the moveKeysLock
|
||||
}
|
||||
}
|
||||
|
||||
// read initial data again
|
||||
wait(readRealInitialDataDistribution(self));
|
||||
|
@ -293,14 +386,17 @@ struct IDDTxnProcessorApiWorkload : TestWorkload {
|
|||
.get();
|
||||
|
||||
verifyInitDataEqual(self->realInitDD, mockInitData);
|
||||
|
||||
TraceEvent(SevDebug, relocateShardInterval.end(), relocateShardInterval.pairID);
|
||||
self->mock->setupMockGlobalState(self->realInitDD); // in case SS remove or recruit
|
||||
return Void();
|
||||
}
|
||||
|
||||
ACTOR Future<Void> worker(Database cx, IDDTxnProcessorApiWorkload* self) {
|
||||
state double lastTime = now();
|
||||
state int choice = 0;
|
||||
state int maxChoice = self->testStartOnly ? 1 : 2;
|
||||
loop {
|
||||
choice = deterministicRandom()->randomInt(0, 2);
|
||||
choice = deterministicRandom()->randomInt(0, maxChoice);
|
||||
if (choice == 0) { // test rawStartMovement and rawFinishMovement separately
|
||||
wait(testRawMovementApi(self));
|
||||
} else if (choice == 1) { // test moveKeys
|
||||
|
@ -309,7 +405,6 @@ struct IDDTxnProcessorApiWorkload : TestWorkload {
|
|||
ASSERT(false);
|
||||
}
|
||||
wait(delay(FLOW_KNOBS->PREVENT_FAST_SPIN_DELAY));
|
||||
// Keep trying to get the moveKeysLock
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -18,7 +18,6 @@
|
|||
* limitations under the License.
|
||||
*/
|
||||
|
||||
#include "fdbrpc/ContinuousSample.h"
|
||||
#include "fdbclient/NativeAPI.actor.h"
|
||||
#include "fdbserver/TesterInterface.actor.h"
|
||||
#include "fdbserver/workloads/workloads.actor.h"
|
||||
|
|
|
@ -18,7 +18,6 @@
|
|||
* limitations under the License.
|
||||
*/
|
||||
|
||||
#include "fdbrpc/ContinuousSample.h"
|
||||
#include "fdbclient/IKnobCollection.h"
|
||||
#include "fdbclient/NativeAPI.actor.h"
|
||||
#include "fdbserver/TesterInterface.actor.h"
|
||||
|
|
|
@ -63,7 +63,7 @@ struct MakoWorkload : TestWorkload {
|
|||
// used for periodically tracing
|
||||
std::vector<PerfMetric> periodicMetrics;
|
||||
// store latency of each operation with sampling
|
||||
std::vector<ContinuousSample<double>> opLatencies;
|
||||
std::vector<DDSketch<double>> opLatencies;
|
||||
// key used to store checkSum for given key range
|
||||
std::vector<Key> csKeys;
|
||||
// key prefix of for all generated keys
|
||||
|
@ -142,7 +142,7 @@ struct MakoWorkload : TestWorkload {
|
|||
parseOperationsSpec();
|
||||
for (int i = 0; i < MAX_OP; ++i) {
|
||||
// initilize per-operation latency record
|
||||
opLatencies.push_back(ContinuousSample<double>(rowCount / sampleSize));
|
||||
opLatencies.push_back(DDSketch<double>());
|
||||
// initialize per-operation counter
|
||||
opCounters.push_back(PerfIntCounter(opNames[i]));
|
||||
}
|
||||
|
@ -658,7 +658,7 @@ struct MakoWorkload : TestWorkload {
|
|||
return Void();
|
||||
}
|
||||
ACTOR template <class T>
|
||||
static Future<Void> logLatency(Future<T> f, ContinuousSample<double>* opLatencies) {
|
||||
static Future<Void> logLatency(Future<T> f, DDSketch<double>* opLatencies) {
|
||||
state double opBegin = timer();
|
||||
wait(success(f));
|
||||
opLatencies->addSample(timer() - opBegin);
|
||||
|
|
|
@ -18,7 +18,6 @@
|
|||
* limitations under the License.
|
||||
*/
|
||||
|
||||
#include "fdbrpc/ContinuousSample.h"
|
||||
#include "fdbclient/NativeAPI.actor.h"
|
||||
#include "fdbserver/TesterInterface.actor.h"
|
||||
#include "flow/DeterministicRandom.h"
|
||||
|
|
|
@ -392,6 +392,35 @@ struct MetaclusterManagementWorkload : TestWorkload {
|
|||
return Void();
|
||||
}
|
||||
|
||||
ACTOR static Future<Void> verifyListFilter(MetaclusterManagementWorkload* self, TenantName tenant) {
|
||||
try {
|
||||
state TenantMapEntry checkEntry = wait(MetaclusterAPI::getTenant(self->managementDb, tenant));
|
||||
state TenantState checkState = checkEntry.tenantState;
|
||||
state std::vector<TenantState> filters;
|
||||
filters.push_back(checkState);
|
||||
state std::vector<std::pair<TenantName, TenantMapEntry>> tenantList;
|
||||
// Possible to have changed state between now and the getTenant call above
|
||||
state TenantMapEntry checkEntry2;
|
||||
wait(store(checkEntry2, MetaclusterAPI::getTenant(self->managementDb, tenant)) &&
|
||||
store(tenantList,
|
||||
MetaclusterAPI::listTenants(self->managementDb, ""_sr, "\xff\xff"_sr, 10e6, 0, filters)));
|
||||
bool found = false;
|
||||
for (auto pair : tenantList) {
|
||||
ASSERT(pair.second.tenantState == checkState);
|
||||
if (pair.first == tenant) {
|
||||
found = true;
|
||||
}
|
||||
}
|
||||
ASSERT(found || checkEntry2.tenantState != checkState);
|
||||
} catch (Error& e) {
|
||||
if (e.code() != error_code_tenant_not_found) {
|
||||
TraceEvent(SevError, "VerifyListFilterFailure").error(e).detail("Tenant", tenant);
|
||||
throw;
|
||||
}
|
||||
}
|
||||
return Void();
|
||||
}
|
||||
|
||||
ACTOR static Future<Void> createTenant(MetaclusterManagementWorkload* self) {
|
||||
state TenantName tenant = self->chooseTenantName();
|
||||
state Optional<TenantGroupName> tenantGroup = self->chooseTenantGroup();
|
||||
|
@ -433,6 +462,7 @@ struct MetaclusterManagementWorkload : TestWorkload {
|
|||
break;
|
||||
} else {
|
||||
retried = true;
|
||||
wait(verifyListFilter(self, tenant));
|
||||
}
|
||||
} catch (Error& e) {
|
||||
if (e.code() == error_code_tenant_already_exists && retried && !exists) {
|
||||
|
@ -533,6 +563,7 @@ struct MetaclusterManagementWorkload : TestWorkload {
|
|||
break;
|
||||
} else {
|
||||
retried = true;
|
||||
wait(verifyListFilter(self, tenant));
|
||||
}
|
||||
} catch (Error& e) {
|
||||
if (e.code() == error_code_tenant_not_found && retried && exists) {
|
||||
|
@ -622,6 +653,7 @@ struct MetaclusterManagementWorkload : TestWorkload {
|
|||
if (result.present()) {
|
||||
break;
|
||||
}
|
||||
wait(verifyListFilter(self, tenant));
|
||||
}
|
||||
|
||||
ASSERT(exists);
|
||||
|
@ -716,6 +748,8 @@ struct MetaclusterManagementWorkload : TestWorkload {
|
|||
}
|
||||
|
||||
retried = true;
|
||||
wait(verifyListFilter(self, tenant));
|
||||
wait(verifyListFilter(self, newTenantName));
|
||||
} catch (Error& e) {
|
||||
// If we retry the rename after it had succeeded, we will get an error that we should ignore
|
||||
if (e.code() == error_code_tenant_not_found && exists && !newTenantExists && retried) {
|
||||
|
|
|
@ -18,7 +18,6 @@
|
|||
* limitations under the License.
|
||||
*/
|
||||
|
||||
#include "fdbrpc/ContinuousSample.h"
|
||||
#include "fdbclient/NativeAPI.actor.h"
|
||||
#include "fdbserver/TesterInterface.actor.h"
|
||||
#include "flow/TDMetric.actor.h"
|
||||
|
|
|
@ -0,0 +1,58 @@
|
|||
/*
|
||||
* MockDDTest.actor.cpp
|
||||
*
|
||||
* This source file is part of the FoundationDB open source project
|
||||
*
|
||||
* Copyright 2013-2022 Apple Inc. and the FoundationDB project authors
|
||||
*
|
||||
* Licensed under the Apache License, Version 2.0 (the "License");
|
||||
* you may not use this file except in compliance with the License.
|
||||
* You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
#include "fdbserver/workloads/MockDDTest.h"
|
||||
#include "flow/actorcompiler.h" // This must be the last #include.
|
||||
|
||||
KeyRange MockDDTestWorkload::getRandomRange(double offset) const {
|
||||
double len = deterministicRandom()->random01() * this->maxKeyspace;
|
||||
double pos = offset + deterministicRandom()->random01() * (1.0 - len);
|
||||
return KeyRangeRef(doubleToTestKey(pos), doubleToTestKey(pos + len));
|
||||
}
|
||||
|
||||
MockDDTestWorkload::MockDDTestWorkload(WorkloadContext const& wcx) : TestWorkload(wcx) {
|
||||
enabled = !clientId && g_network->isSimulated(); // only do this on the "first" client
|
||||
simpleConfig = getOption(options, "simpleConfig"_sr, true);
|
||||
testDuration = getOption(options, "testDuration"_sr, 10.0);
|
||||
meanDelay = getOption(options, "meanDelay"_sr, meanDelay);
|
||||
maxKeyspace = getOption(options, "maxKeyspace"_sr, maxKeyspace);
|
||||
maxByteSize = getOption(options, "maxByteSize"_sr, maxByteSize);
|
||||
minByteSize = getOption(options, "minByteSize"_sr, minByteSize);
|
||||
}
|
||||
|
||||
Future<Void> MockDDTestWorkload::setup(Database const& cx) {
|
||||
if (!enabled)
|
||||
return Void();
|
||||
// initialize configuration
|
||||
BasicTestConfig testConfig;
|
||||
testConfig.simpleConfig = simpleConfig;
|
||||
testConfig.minimumReplication = 1;
|
||||
testConfig.logAntiQuorum = 0;
|
||||
DatabaseConfiguration dbConfig = generateNormalDatabaseConfiguration(testConfig);
|
||||
|
||||
// initialize mgs
|
||||
mgs = std::make_shared<MockGlobalState>();
|
||||
mgs->maxByteSize = maxByteSize;
|
||||
mgs->minByteSize = minByteSize;
|
||||
mgs->initializeAsEmptyDatabaseMGS(dbConfig);
|
||||
mock = makeReference<DDMockTxnProcessor>(mgs);
|
||||
|
||||
return Void();
|
||||
}
|
|
@ -0,0 +1,186 @@
|
|||
/*
|
||||
* MockDDTrackerShardEvaluator.actor.cpp
|
||||
*
|
||||
* This source file is part of the FoundationDB open source project
|
||||
*
|
||||
* Copyright 2013-2022 Apple Inc. and the FoundationDB project authors
|
||||
*
|
||||
* Licensed under the Apache License, Version 2.0 (the "License");
|
||||
* you may not use this file except in compliance with the License.
|
||||
* You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
#include "fdbserver/workloads/MockDDTest.h"
|
||||
#include "flow/actorcompiler.h" // This must be the last #include.
|
||||
|
||||
struct MockDDTrackerShardEvaluatorWorkload : public MockDDTestWorkload {
|
||||
static constexpr auto NAME = "MockDDTrackerShardEvaluator";
|
||||
DDSharedContext ddcx;
|
||||
|
||||
PromiseStream<RelocateShard> output;
|
||||
PromiseStream<GetMetricsRequest> getShardMetrics;
|
||||
PromiseStream<GetTopKMetricsRequest> getTopKMetrics;
|
||||
PromiseStream<GetMetricsListRequest> getShardMetricsList;
|
||||
PromiseStream<Promise<int64_t>> getAverageShardBytes;
|
||||
|
||||
KeyRangeMap<ShardTrackedData> shards;
|
||||
|
||||
ActorCollection actors;
|
||||
uint64_t mockDbSize = 0;
|
||||
const int keySize = 16;
|
||||
|
||||
std::map<RelocateReason, int> rsReasonCounts;
|
||||
|
||||
// --- test configs ---
|
||||
|
||||
// Each key space is convert from an int N. [N, N+1) represent a key space. So at most we have 2G key spaces
|
||||
int keySpaceCount = 0;
|
||||
// 1. fixed -- each key space has fixed size. The size of each key space is calculated as minSpaceKeyCount *
|
||||
// (minByteSize + 16) ;
|
||||
// 2. linear -- from 0 to keySpaceCount the size of key space increase by size linearStride, from
|
||||
// linearStartSize. Each value is fixed to minByteSize;
|
||||
// 3. random -- each key space can has [minSpaceKeyCount,
|
||||
// maxSpaceKeyCount] pairs and the size of value varies from [minByteSize, maxByteSize];
|
||||
Value keySpaceStrategy = "fixed"_sr;
|
||||
int minSpaceKeyCount = 1000, maxSpaceKeyCount = 1000;
|
||||
int linearStride = 10 * (1 << 20), linearStartSize = 10 * (1 << 20);
|
||||
|
||||
MockDDTrackerShardEvaluatorWorkload(WorkloadContext const& wcx)
|
||||
: MockDDTestWorkload(wcx), ddcx(deterministicRandom()->randomUniqueID()) {
|
||||
keySpaceCount = getOption(options, "keySpaceCount"_sr, keySpaceCount);
|
||||
keySpaceStrategy = getOption(options, "keySpaceStrategy"_sr, keySpaceStrategy);
|
||||
minSpaceKeyCount = getOption(options, "minSpaceKeyCount"_sr, minSpaceKeyCount);
|
||||
maxSpaceKeyCount = getOption(options, "maxSpaceKeyCount"_sr, maxSpaceKeyCount);
|
||||
linearStride = getOption(options, "linearStride"_sr, linearStride);
|
||||
linearStartSize = getOption(options, "linearStartSize"_sr, linearStartSize);
|
||||
}
|
||||
|
||||
void populateRandomStrategy() {
|
||||
mockDbSize = 0;
|
||||
for (int i = 0; i < keySpaceCount; ++i) {
|
||||
int kCount = deterministicRandom()->randomInt(minSpaceKeyCount, maxSpaceKeyCount);
|
||||
for (int j = 0; j < kCount; ++j) {
|
||||
Key k = doubleToTestKey(i + deterministicRandom()->random01());
|
||||
auto vSize = deterministicRandom()->randomInt(minByteSize, maxByteSize + 1);
|
||||
mgs->set(k, vSize, true);
|
||||
mockDbSize += vSize + k.size();
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
void populateLinearStrategy() {
|
||||
mockDbSize = 0;
|
||||
auto pSize = minByteSize + keySize;
|
||||
for (int i = 0; i < keySpaceCount; ++i) {
|
||||
int kCount = std::ceil((linearStride * i + linearStartSize) * 1.0 / pSize);
|
||||
for (int j = 0; j < kCount; ++j) {
|
||||
Key k = doubleToTestKey(i + deterministicRandom()->random01());
|
||||
mgs->set(k, minByteSize, true);
|
||||
}
|
||||
mockDbSize += pSize * kCount;
|
||||
}
|
||||
}
|
||||
|
||||
void populateFixedStrategy() {
|
||||
auto pSize = minByteSize + keySize;
|
||||
for (int i = 0; i < keySpaceCount; ++i) {
|
||||
for (int j = 0; j < minSpaceKeyCount; ++j) {
|
||||
Key k = doubleToTestKey(i + deterministicRandom()->random01());
|
||||
mgs->set(k, minByteSize, true);
|
||||
}
|
||||
}
|
||||
mockDbSize = keySpaceCount * minSpaceKeyCount * pSize;
|
||||
}
|
||||
|
||||
void populateMgs() {
|
||||
// Will the sampling structure become too large?
|
||||
std::cout << "MGS Populating ...\n";
|
||||
if (keySpaceStrategy == "linear") {
|
||||
populateLinearStrategy();
|
||||
} else if (keySpaceStrategy == "fixed") {
|
||||
populateFixedStrategy();
|
||||
} else if (keySpaceStrategy == "random") {
|
||||
populateRandomStrategy();
|
||||
}
|
||||
uint64_t totalSize = 0;
|
||||
for (auto& server : mgs->allServers) {
|
||||
totalSize = server.second.sumRangeSize(allKeys);
|
||||
}
|
||||
TraceEvent("PopulateMockGlobalState")
|
||||
.detail("Strategy", keySpaceStrategy)
|
||||
.detail("EstimatedDbSize", mockDbSize)
|
||||
.detail("MGSReportedTotalSize", totalSize);
|
||||
std::cout << "MGS Populated.\n";
|
||||
}
|
||||
|
||||
Future<Void> setup(Database const& cx) override {
|
||||
if (!enabled)
|
||||
return Void();
|
||||
MockDDTestWorkload::setup(cx);
|
||||
// populate mgs before run tracker
|
||||
populateMgs();
|
||||
return Void();
|
||||
}
|
||||
|
||||
ACTOR static Future<Void> relocateShardReporter(MockDDTrackerShardEvaluatorWorkload* self,
|
||||
FutureStream<RelocateShard> input) {
|
||||
loop choose {
|
||||
when(RelocateShard rs = waitNext(input)) { ++self->rsReasonCounts[rs.reason]; }
|
||||
}
|
||||
}
|
||||
|
||||
Future<Void> start(Database const& cx) override {
|
||||
if (!enabled)
|
||||
return Void();
|
||||
|
||||
// start mock servers
|
||||
actors.add(waitForAll(mgs->runAllMockServers()));
|
||||
|
||||
// start tracker
|
||||
Reference<InitialDataDistribution> initData =
|
||||
mock->getInitialDataDistribution(ddcx.id(), ddcx.lock, {}, ddcx.ddEnabledState.get(), SkipDDModeCheck::True)
|
||||
.get();
|
||||
Reference<PhysicalShardCollection> physicalShardCollection = makeReference<PhysicalShardCollection>();
|
||||
Reference<AsyncVar<bool>> zeroHealthyTeams = makeReference<AsyncVar<bool>>(false);
|
||||
actors.add(dataDistributionTracker(initData,
|
||||
mock,
|
||||
output,
|
||||
ddcx.shardsAffectedByTeamFailure,
|
||||
physicalShardCollection,
|
||||
getShardMetrics,
|
||||
getTopKMetrics.getFuture(),
|
||||
getShardMetricsList,
|
||||
getAverageShardBytes.getFuture(),
|
||||
Promise<Void>(),
|
||||
zeroHealthyTeams,
|
||||
ddcx.id(),
|
||||
&shards,
|
||||
&ddcx.trackerCancelled,
|
||||
{}));
|
||||
actors.add(relocateShardReporter(this, output.getFuture()));
|
||||
|
||||
return timeout(reportErrors(actors.getResult(), "MockDDTrackerShardEvaluatorWorkload"), testDuration, Void());
|
||||
}
|
||||
|
||||
Future<bool> check(Database const& cx) override {
|
||||
std::cout << "Check phase shards count: " << shards.size() << "\n";
|
||||
actors.clear(true);
|
||||
return true;
|
||||
}
|
||||
|
||||
void getMetrics(std::vector<PerfMetric>& m) override {
|
||||
for (const auto& [reason, count] : rsReasonCounts) {
|
||||
m.push_back(PerfMetric(RelocateReason(reason).toString(), count, Averaged::False));
|
||||
}
|
||||
}
|
||||
};
|
||||
|
||||
WorkloadFactory<MockDDTrackerShardEvaluatorWorkload> MockDDTrackerShardEvaluatorWorkload;
|
|
@ -19,7 +19,7 @@
|
|||
*/
|
||||
#include <vector>
|
||||
|
||||
#include "fdbrpc/ContinuousSample.h"
|
||||
#include "fdbrpc/DDSketch.h"
|
||||
#include "fdbclient/NativeAPI.actor.h"
|
||||
#include "fdbserver/TesterInterface.actor.h"
|
||||
#include "fdbserver/workloads/workloads.actor.h"
|
||||
|
@ -38,10 +38,10 @@ struct QueuePushWorkload : TestWorkload {
|
|||
|
||||
std::vector<Future<Void>> clients;
|
||||
PerfIntCounter transactions, retries;
|
||||
ContinuousSample<double> commitLatencies, GRVLatencies;
|
||||
DDSketch<double> commitLatencies, GRVLatencies;
|
||||
|
||||
QueuePushWorkload(WorkloadContext const& wcx)
|
||||
: TestWorkload(wcx), transactions("Transactions"), retries("Retries"), commitLatencies(2000), GRVLatencies(2000) {
|
||||
: TestWorkload(wcx), transactions("Transactions"), retries("Retries"), commitLatencies(), GRVLatencies() {
|
||||
testDuration = getOption(options, "testDuration"_sr, 10.0);
|
||||
actorCount = getOption(options, "actorCount"_sr, 50);
|
||||
|
||||
|
|
|
@ -18,7 +18,6 @@
|
|||
* limitations under the License.
|
||||
*/
|
||||
|
||||
#include "fdbrpc/ContinuousSample.h"
|
||||
#include "fdbclient/NativeAPI.actor.h"
|
||||
#include "fdbserver/TesterInterface.actor.h"
|
||||
#include "fdbclient/ReadYourWrites.h"
|
||||
|
|
|
@ -18,7 +18,6 @@
|
|||
* limitations under the License.
|
||||
*/
|
||||
|
||||
#include "fdbrpc/ContinuousSample.h"
|
||||
#include "fdbclient/NativeAPI.actor.h"
|
||||
#include "fdbserver/TesterInterface.actor.h"
|
||||
#include "fdbclient/ReadYourWrites.h"
|
||||
|
|
|
@ -25,8 +25,6 @@
|
|||
#include "flow/genericactors.actor.h"
|
||||
#include "flow/actorcompiler.h" // This must be the last #include.
|
||||
|
||||
static constexpr int SAMPLE_SIZE = 10000;
|
||||
|
||||
// If the log->storage propagation delay is longer than 1 second, then it's likely that our read
|
||||
// will see a `future_version` error from the storage server. We need to retry the read until
|
||||
// a value is returned, or a different error is thrown.
|
||||
|
@ -51,9 +49,9 @@ struct ReadAfterWriteWorkload : KVWorkload {
|
|||
static constexpr auto NAME = "ReadAfterWrite";
|
||||
|
||||
double testDuration;
|
||||
ContinuousSample<double> propagationLatency;
|
||||
DDSketch<double> propagationLatency;
|
||||
|
||||
ReadAfterWriteWorkload(WorkloadContext const& wcx) : KVWorkload(wcx), propagationLatency(SAMPLE_SIZE) {
|
||||
ReadAfterWriteWorkload(WorkloadContext const& wcx) : KVWorkload(wcx), propagationLatency() {
|
||||
testDuration = getOption(options, "testDuration"_sr, 10.0);
|
||||
}
|
||||
|
||||
|
|
|
@ -18,7 +18,7 @@
|
|||
* limitations under the License.
|
||||
*/
|
||||
|
||||
#include "fdbrpc/ContinuousSample.h"
|
||||
#include "fdbrpc/DDSketch.h"
|
||||
#include "fdbclient/NativeAPI.actor.h"
|
||||
#include "fdbserver/TesterInterface.actor.h"
|
||||
#include "fdbserver/workloads/BulkSetup.actor.h"
|
||||
|
|
|
@ -23,7 +23,7 @@
|
|||
#include <vector>
|
||||
|
||||
#include "fdbclient/FDBTypes.h"
|
||||
#include "fdbrpc/ContinuousSample.h"
|
||||
#include "fdbrpc/DDSketch.h"
|
||||
#include "fdbclient/NativeAPI.actor.h"
|
||||
#include "fdbserver/TesterInterface.actor.h"
|
||||
#include "fdbserver/WorkerInterface.actor.h"
|
||||
|
@ -200,7 +200,7 @@ struct ReadWriteCommonImpl {
|
|||
}
|
||||
}
|
||||
ACTOR static Future<Void> logLatency(Future<Optional<Value>> f,
|
||||
ContinuousSample<double>* latencies,
|
||||
DDSketch<double>* latencies,
|
||||
double* totalLatency,
|
||||
int* latencyCount,
|
||||
EventMetricHandle<ReadMetric> readMetric,
|
||||
|
@ -220,7 +220,7 @@ struct ReadWriteCommonImpl {
|
|||
return Void();
|
||||
}
|
||||
ACTOR static Future<Void> logLatency(Future<RangeResult> f,
|
||||
ContinuousSample<double>* latencies,
|
||||
DDSketch<double>* latencies,
|
||||
double* totalLatency,
|
||||
int* latencyCount,
|
||||
EventMetricHandle<ReadMetric> readMetric,
|
||||
|
|
|
@ -22,7 +22,7 @@
|
|||
#include <utility>
|
||||
#include <vector>
|
||||
|
||||
#include "fdbrpc/ContinuousSample.h"
|
||||
#include "fdbrpc/DDSketch.h"
|
||||
#include "fdbclient/NativeAPI.actor.h"
|
||||
#include "fdbserver/TesterInterface.actor.h"
|
||||
#include "fdbserver/WorkerInterface.actor.h"
|
||||
|
@ -389,4 +389,4 @@ TEST_CASE("/KVWorkload/methods/ParseKeyForIndex") {
|
|||
ASSERT(parse == idx);
|
||||
}
|
||||
return Void();
|
||||
}
|
||||
}
|
||||
|
|
|
@ -23,7 +23,6 @@
|
|||
#include "fdbclient/NativeAPI.actor.h"
|
||||
#include "fdbclient/ReadYourWrites.h"
|
||||
#include "fdbclient/SystemData.h"
|
||||
#include "fdbrpc/ContinuousSample.h"
|
||||
#include "fdbclient/SimpleIni.h"
|
||||
#include "fdbserver/Status.actor.h"
|
||||
#include "fdbserver/TesterInterface.actor.h"
|
||||
|
|
|
@ -18,7 +18,7 @@
|
|||
* limitations under the License.
|
||||
*/
|
||||
|
||||
#include "fdbrpc/ContinuousSample.h"
|
||||
#include "fdbrpc/DDSketch.h"
|
||||
#include "fdbclient/NativeAPI.actor.h"
|
||||
#include "fdbserver/TesterInterface.actor.h"
|
||||
#include "fdbserver/workloads/workloads.actor.h"
|
||||
|
@ -37,11 +37,11 @@ struct StreamingReadWorkload : TestWorkload {
|
|||
std::vector<Future<Void>> clients;
|
||||
PerfIntCounter transactions, readKeys;
|
||||
PerfIntCounter readValueBytes;
|
||||
ContinuousSample<double> latencies;
|
||||
DDSketch<double> latencies;
|
||||
|
||||
StreamingReadWorkload(WorkloadContext const& wcx)
|
||||
: TestWorkload(wcx), transactions("Transactions"), readKeys("Keys Read"), readValueBytes("Value Bytes Read"),
|
||||
latencies(2000) {
|
||||
latencies() {
|
||||
testDuration = getOption(options, "testDuration"_sr, 10.0);
|
||||
actorCount = getOption(options, "actorCount"_sr, 20);
|
||||
readsPerTransaction = getOption(options, "readsPerTransaction"_sr, 10);
|
||||
|
|
|
@ -18,7 +18,7 @@
|
|||
* limitations under the License.
|
||||
*/
|
||||
|
||||
#include "fdbrpc/ContinuousSample.h"
|
||||
#include "fdbrpc/DDSketch.h"
|
||||
#include "fdbclient/NativeAPI.actor.h"
|
||||
#include "fdbserver/TesterInterface.actor.h"
|
||||
#include "fdbserver/WorkerInterface.actor.h"
|
||||
|
@ -189,12 +189,11 @@ struct MeasureSinglePeriod : IMeasurer {
|
|||
double delay, duration;
|
||||
double startT;
|
||||
|
||||
ContinuousSample<double> totalLatency, grvLatency, rowReadLatency, commitLatency;
|
||||
DDSketch<double> totalLatency, grvLatency, rowReadLatency, commitLatency;
|
||||
ITransactor::Stats stats; // totalled over the period
|
||||
|
||||
MeasureSinglePeriod(double delay, double duration)
|
||||
: delay(delay), duration(duration), totalLatency(2000), grvLatency(2000), rowReadLatency(2000),
|
||||
commitLatency(2000) {}
|
||||
: delay(delay), duration(duration), totalLatency(), grvLatency(), rowReadLatency(), commitLatency() {}
|
||||
|
||||
Future<Void> start() override {
|
||||
startT = now();
|
||||
|
|
|
@ -18,7 +18,6 @@
|
|||
* limitations under the License.
|
||||
*/
|
||||
|
||||
#include "fdbrpc/ContinuousSample.h"
|
||||
#include "fdbclient/NativeAPI.actor.h"
|
||||
#include "fdbserver/TesterInterface.actor.h"
|
||||
#include "fdbserver/workloads/BulkSetup.actor.h"
|
||||
|
|
|
@ -18,7 +18,6 @@
|
|||
* limitations under the License.
|
||||
*/
|
||||
|
||||
#include "fdbrpc/ContinuousSample.h"
|
||||
#include "fdbclient/ClusterConnectionMemoryRecord.h"
|
||||
#include "fdbclient/NativeAPI.actor.h"
|
||||
#include "fdbserver/TesterInterface.actor.h"
|
||||
|
|
|
@ -18,7 +18,6 @@
|
|||
* limitations under the License.
|
||||
*/
|
||||
|
||||
#include "fdbrpc/ContinuousSample.h"
|
||||
#include "fdbclient/NativeAPI.actor.h"
|
||||
#include "fdbserver/TesterInterface.actor.h"
|
||||
#include "fdbserver/workloads/BulkSetup.actor.h"
|
||||
|
|
|
@ -18,15 +18,13 @@
|
|||
* limitations under the License.
|
||||
*/
|
||||
|
||||
#include "fdbrpc/ContinuousSample.h"
|
||||
#include "fdbrpc/DDSketch.h"
|
||||
#include "fdbclient/NativeAPI.actor.h"
|
||||
#include "fdbserver/TesterInterface.actor.h"
|
||||
#include "flow/DeterministicRandom.h"
|
||||
#include "fdbserver/workloads/workloads.actor.h"
|
||||
#include "flow/actorcompiler.h" // This must be the last #include.
|
||||
|
||||
const int sampleSize = 10000;
|
||||
|
||||
struct WatchesWorkload : TestWorkload {
|
||||
static constexpr auto NAME = "Watches";
|
||||
|
||||
|
@ -34,10 +32,10 @@ struct WatchesWorkload : TestWorkload {
|
|||
double testDuration;
|
||||
std::vector<Future<Void>> clients;
|
||||
PerfIntCounter cycles;
|
||||
ContinuousSample<double> cycleLatencies;
|
||||
DDSketch<double> cycleLatencies;
|
||||
std::vector<int> nodeOrder;
|
||||
|
||||
WatchesWorkload(WorkloadContext const& wcx) : TestWorkload(wcx), cycles("Cycles"), cycleLatencies(sampleSize) {
|
||||
WatchesWorkload(WorkloadContext const& wcx) : TestWorkload(wcx), cycles("Cycles"), cycleLatencies() {
|
||||
testDuration = getOption(options, "testDuration"_sr, 600.0);
|
||||
nodes = getOption(options, "nodeCount"_sr, 100);
|
||||
extraPerNode = getOption(options, "extraPerNode"_sr, 1000);
|
||||
|
|
|
@ -18,7 +18,6 @@
|
|||
* limitations under the License.
|
||||
*/
|
||||
|
||||
#include "fdbrpc/ContinuousSample.h"
|
||||
#include "fdbclient/ReadYourWrites.h"
|
||||
#include "fdbclient/NativeAPI.actor.h"
|
||||
#include "fdbserver/TesterInterface.actor.h"
|
||||
|
|
|
@ -20,7 +20,7 @@
|
|||
|
||||
#include <boost/lexical_cast.hpp>
|
||||
|
||||
#include "fdbrpc/ContinuousSample.h"
|
||||
#include "fdbrpc/DDSketch.h"
|
||||
#include "fdbclient/NativeAPI.actor.h"
|
||||
#include "fdbserver/TesterInterface.actor.h"
|
||||
#include "fdbserver/WorkerInterface.actor.h"
|
||||
|
@ -37,11 +37,11 @@ struct WriteBandwidthWorkload : KVWorkload {
|
|||
|
||||
std::vector<Future<Void>> clients;
|
||||
PerfIntCounter transactions, retries;
|
||||
ContinuousSample<double> commitLatencies, GRVLatencies;
|
||||
DDSketch<double> commitLatencies, GRVLatencies;
|
||||
|
||||
WriteBandwidthWorkload(WorkloadContext const& wcx)
|
||||
: KVWorkload(wcx), loadTime(0.0), transactions("Transactions"), retries("Retries"), commitLatencies(2000),
|
||||
GRVLatencies(2000) {
|
||||
: KVWorkload(wcx), loadTime(0.0), transactions("Transactions"), retries("Retries"), commitLatencies(),
|
||||
GRVLatencies() {
|
||||
testDuration = getOption(options, "testDuration"_sr, 10.0);
|
||||
keysPerTransaction = getOption(options, "keysPerTransaction"_sr, 100);
|
||||
valueString = std::string(maxValueBytes, '.');
|
||||
|
|
|
@ -26,7 +26,6 @@
|
|||
#include "fdbclient/TagThrottle.actor.h"
|
||||
#include "flow/actorcompiler.h" // This must be the last #include.
|
||||
|
||||
constexpr int SAMPLE_SIZE = 10000;
|
||||
// workload description:
|
||||
// This workload aims to test whether we can throttling some bad clients that doing penetrating write on write hot-spot
|
||||
// range. There are several good clientActor just randomly do read and write ops in transaction. Also, some bad
|
||||
|
@ -41,8 +40,8 @@ struct WriteTagThrottlingWorkload : KVWorkload {
|
|||
int badActorTrNum = 0, badActorRetries = 0, badActorTooOldRetries = 0, badActorCommitFailedRetries = 0;
|
||||
int goodActorThrottleRetries = 0, badActorThrottleRetries = 0;
|
||||
double badActorTotalLatency = 0.0, goodActorTotalLatency = 0.0;
|
||||
ContinuousSample<double> badActorReadLatency, goodActorReadLatency;
|
||||
ContinuousSample<double> badActorCommitLatency, goodActorCommitLatency;
|
||||
DDSketch<double> badActorReadLatency, goodActorReadLatency;
|
||||
DDSketch<double> badActorCommitLatency, goodActorCommitLatency;
|
||||
// Test configuration
|
||||
// KVWorkload::actorCount
|
||||
int goodActorPerClient, badActorPerClient;
|
||||
|
@ -64,8 +63,8 @@ struct WriteTagThrottlingWorkload : KVWorkload {
|
|||
static constexpr int MIN_TRANSACTION_TAG_LENGTH = 2;
|
||||
|
||||
WriteTagThrottlingWorkload(WorkloadContext const& wcx)
|
||||
: KVWorkload(wcx), badActorReadLatency(SAMPLE_SIZE), goodActorReadLatency(SAMPLE_SIZE),
|
||||
badActorCommitLatency(SAMPLE_SIZE), goodActorCommitLatency(SAMPLE_SIZE) {
|
||||
: KVWorkload(wcx), badActorReadLatency(), goodActorReadLatency(), badActorCommitLatency(),
|
||||
goodActorCommitLatency() {
|
||||
testDuration = getOption(options, "testDuration"_sr, 120.0);
|
||||
badOpRate = getOption(options, "badOpRate"_sr, 0.9);
|
||||
numWritePerTr = getOption(options, "numWritePerTr"_sr, 1);
|
||||
|
|
|
@ -112,8 +112,7 @@ void FlowKnobs::initialize(Randomize randomize, IsSimulated isSimulated) {
|
|||
init( PEER_UNAVAILABLE_FOR_LONG_TIME_TIMEOUT, 3600.0 );
|
||||
init( INCOMPATIBLE_PEER_DELAY_BEFORE_LOGGING, 5.0 );
|
||||
init( PING_LOGGING_INTERVAL, 3.0 );
|
||||
init( PING_SAMPLE_AMOUNT, 100 );
|
||||
init( NETWORK_CONNECT_SAMPLE_AMOUNT, 100 );
|
||||
init( PING_SKETCH_ACCURACY, 0.1 );
|
||||
|
||||
init( TLS_CERT_REFRESH_DELAY_SECONDS, 12*60*60 );
|
||||
init( TLS_SERVER_CONNECTION_THROTTLE_TIMEOUT, 9.0 );
|
||||
|
@ -168,7 +167,7 @@ void FlowKnobs::initialize(Randomize randomize, IsSimulated isSimulated) {
|
|||
init( MIN_SUBMIT, 10 );
|
||||
init( SQLITE_DISK_METRIC_LOGGING_INTERVAL, 5.0 );
|
||||
init( KAIO_LATENCY_LOGGING_INTERVAL, 30.0 );
|
||||
init( KAIO_LATENCY_SAMPLE_SIZE, 30000 );
|
||||
init( KAIO_LATENCY_SKETCH_ACCURACY, 0.01 );
|
||||
|
||||
init( PAGE_WRITE_CHECKSUM_HISTORY, 0 ); if( randomize && BUGGIFY ) PAGE_WRITE_CHECKSUM_HISTORY = 10000000;
|
||||
init( DISABLE_POSIX_KERNEL_AIO, 0 );
|
||||
|
@ -303,7 +302,7 @@ void FlowKnobs::initialize(Randomize randomize, IsSimulated isSimulated) {
|
|||
if ( randomize && BUGGIFY) { ENCRYPT_KEY_REFRESH_INTERVAL = deterministicRandom()->randomInt(2, 10); }
|
||||
init( TOKEN_CACHE_SIZE, 100 );
|
||||
init( ENCRYPT_KEY_CACHE_LOGGING_INTERVAL, 5.0 );
|
||||
init( ENCRYPT_KEY_CACHE_LOGGING_SAMPLE_SIZE, 1000 );
|
||||
init( ENCRYPT_KEY_CACHE_LOGGING_SKETCH_ACCURACY, 0.01 );
|
||||
// Refer to EncryptUtil::EncryptAuthTokenAlgo for more details
|
||||
init( ENCRYPT_HEADER_AUTH_TOKEN_ENABLED, true ); if ( randomize && BUGGIFY ) { ENCRYPT_HEADER_AUTH_TOKEN_ENABLED = !ENCRYPT_HEADER_AUTH_TOKEN_ENABLED; }
|
||||
init( ENCRYPT_HEADER_AUTH_TOKEN_ALGO, 1 ); if ( randomize && BUGGIFY ) { ENCRYPT_HEADER_AUTH_TOKEN_ALGO = getRandomAuthTokenAlgo(); }
|
||||
|
|
|
@ -176,8 +176,7 @@ public:
|
|||
int ACCEPT_BATCH_SIZE;
|
||||
double INCOMPATIBLE_PEER_DELAY_BEFORE_LOGGING;
|
||||
double PING_LOGGING_INTERVAL;
|
||||
int PING_SAMPLE_AMOUNT;
|
||||
int NETWORK_CONNECT_SAMPLE_AMOUNT;
|
||||
double PING_SKETCH_ACCURACY;
|
||||
|
||||
int TLS_CERT_REFRESH_DELAY_SECONDS;
|
||||
double TLS_SERVER_CONNECTION_THROTTLE_TIMEOUT;
|
||||
|
@ -231,7 +230,7 @@ public:
|
|||
int MIN_SUBMIT;
|
||||
double SQLITE_DISK_METRIC_LOGGING_INTERVAL;
|
||||
double KAIO_LATENCY_LOGGING_INTERVAL;
|
||||
int KAIO_LATENCY_SAMPLE_SIZE;
|
||||
double KAIO_LATENCY_SKETCH_ACCURACY;
|
||||
|
||||
int PAGE_WRITE_CHECKSUM_HISTORY;
|
||||
int DISABLE_POSIX_KERNEL_AIO;
|
||||
|
@ -365,7 +364,7 @@ public:
|
|||
int64_t ENCRYPT_CIPHER_KEY_CACHE_TTL;
|
||||
int64_t ENCRYPT_KEY_REFRESH_INTERVAL;
|
||||
double ENCRYPT_KEY_CACHE_LOGGING_INTERVAL;
|
||||
double ENCRYPT_KEY_CACHE_LOGGING_SAMPLE_SIZE;
|
||||
double ENCRYPT_KEY_CACHE_LOGGING_SKETCH_ACCURACY;
|
||||
bool ENCRYPT_HEADER_AUTH_TOKEN_ENABLED;
|
||||
int ENCRYPT_HEADER_AUTH_TOKEN_ALGO;
|
||||
|
||||
|
|
|
@ -22,8 +22,62 @@
|
|||
#include "flow/IRandom.h"
|
||||
#include "flowbench/GlobalData.h"
|
||||
#include "fdbrpc/Stats.h"
|
||||
#include "fdbrpc/DDSketch.h"
|
||||
#include "fdbrpc/ContinuousSample.h"
|
||||
#include "flow/Histogram.h"
|
||||
|
||||
static void bench_ddsketchUnsigned(benchmark::State& state) {
|
||||
DDSketchFastUnsigned dds;
|
||||
InputGenerator<unsigned> data(1e6, []() { return deterministicRandom()->randomInt64(0, 1e9); });
|
||||
|
||||
for (auto _ : state) {
|
||||
dds.addSample(data.next());
|
||||
}
|
||||
|
||||
state.SetItemsProcessed(state.iterations());
|
||||
}
|
||||
// DDSketchFastUnsigned has a fixed error margin (~8%)
|
||||
BENCHMARK(bench_ddsketchUnsigned)->ReportAggregatesOnly(true);
|
||||
|
||||
static void bench_ddsketchInt(benchmark::State& state) {
|
||||
DDSketch<int64_t> dds((double)state.range(0) / 100);
|
||||
InputGenerator<int64_t> data(1e6, []() { return deterministicRandom()->randomInt64(0, 1e9); });
|
||||
|
||||
for (auto _ : state) {
|
||||
dds.addSample(data.next());
|
||||
}
|
||||
|
||||
state.SetItemsProcessed(state.iterations());
|
||||
}
|
||||
// Try with 10%, 5% and 1% error margins
|
||||
BENCHMARK(bench_ddsketchInt)->Arg(10)->Arg(5)->Arg(1)->ReportAggregatesOnly(true);
|
||||
|
||||
static void bench_ddsketchDouble(benchmark::State& state) {
|
||||
DDSketch<double> dds((double)state.range(0) / 100);
|
||||
InputGenerator<double> data(1e6, []() { return deterministicRandom()->randomInt64(0, 1e9); });
|
||||
|
||||
for (auto _ : state) {
|
||||
dds.addSample(data.next());
|
||||
}
|
||||
|
||||
state.SetItemsProcessed(state.iterations());
|
||||
}
|
||||
// Try with 10%, 5% and 1% error margins
|
||||
BENCHMARK(bench_ddsketchDouble)->Arg(10)->Arg(5)->Arg(1)->ReportAggregatesOnly(true);
|
||||
|
||||
static void bench_ddsketchLatency(benchmark::State& state) {
|
||||
DDSketch<double> dds((double)state.range(0) / 100);
|
||||
InputGenerator<double> data(1e6, []() { return deterministicRandom()->random01() * 2.0; });
|
||||
|
||||
for (auto _ : state) {
|
||||
dds.addSample(data.next());
|
||||
}
|
||||
|
||||
state.SetItemsProcessed(state.iterations());
|
||||
}
|
||||
// Try with 10%, 5% and 1% error margins
|
||||
BENCHMARK(bench_ddsketchLatency)->Arg(10)->Arg(5)->Arg(1)->ReportAggregatesOnly(true);
|
||||
|
||||
static void bench_continuousSampleInt(benchmark::State& state) {
|
||||
ContinuousSample<int64_t> cs(state.range(0));
|
||||
InputGenerator<int64_t> data(1e6, []() { return deterministicRandom()->randomInt64(0, 1e9); });
|
||||
|
|
|
@ -56,6 +56,7 @@ if(WITH_PYTHON)
|
|||
add_fdb_test(TEST_FILES BlobManagerUnit.toml)
|
||||
add_fdb_test(TEST_FILES ConsistencyCheck.txt IGNORE)
|
||||
add_fdb_test(TEST_FILES DDMetricsExclude.txt IGNORE)
|
||||
add_fdb_test(TEST_FILES DDSketch.txt IGNORE)
|
||||
add_fdb_test(TEST_FILES DataDistributionMetrics.txt IGNORE)
|
||||
add_fdb_test(TEST_FILES DiskDurability.txt IGNORE)
|
||||
add_fdb_test(TEST_FILES FileSystem.txt IGNORE)
|
||||
|
@ -170,7 +171,8 @@ if(WITH_PYTHON)
|
|||
add_fdb_test(TEST_FILES fast/MutationLogReaderCorrectness.toml)
|
||||
add_fdb_test(TEST_FILES fast/GetEstimatedRangeSize.toml)
|
||||
add_fdb_test(TEST_FILES fast/GetMappedRange.toml)
|
||||
add_fdb_test(TEST_FILES fast/IDDTxnProcessorApiCorrectness.toml)
|
||||
add_fdb_test(TEST_FILES fast/IDDTxnProcessorRawStartMovement.toml)
|
||||
add_fdb_test(TEST_FILES fast/IDDTxnProcessorMoveKeys.toml IGNORE)
|
||||
add_fdb_test(TEST_FILES fast/PerpetualWiggleStats.toml)
|
||||
add_fdb_test(TEST_FILES fast/PrivateEndpoints.toml)
|
||||
add_fdb_test(TEST_FILES fast/ProtocolVersion.toml)
|
||||
|
|
|
@ -2,8 +2,11 @@
|
|||
generateFearless = false # prevent generating remote dc because in MGS there's no region setting yet
|
||||
disableTss = true # There's no TSS in MGS this prevent the DD operate TSS mapping
|
||||
|
||||
[[knobs]]
|
||||
max_added_sources_multiplier = 0 # set to 0 because it's impossible to make sure SS and mock SS will finish fetch keys at the same time.
|
||||
|
||||
[[test]]
|
||||
testTitle = 'IDDTxnProcessorApiCorrectness'
|
||||
testTitle = 'IDDTxnProcessorMoveKeys'
|
||||
|
||||
[[test.workload]]
|
||||
testName = 'IDDTxnProcessorApiCorrectness'
|
|
@ -0,0 +1,14 @@
|
|||
[configuration]
|
||||
generateFearless = false # prevent generating remote dc because in MGS there's no region setting yet
|
||||
disableTss = true # There's no TSS in MGS this prevent the DD operate TSS mapping
|
||||
|
||||
[[knobs]]
|
||||
max_added_sources_multiplier = 0 # set to 0 because it's impossible to make sure SS and mock SS will finish fetch keys at the same time.
|
||||
|
||||
[[test]]
|
||||
testTitle = 'IDDTxnProcessorRawStartMovement'
|
||||
|
||||
[[test.workload]]
|
||||
testName = 'IDDTxnProcessorApiCorrectness'
|
||||
testDuration = 50.0
|
||||
testStartOnly = true # only test startMovement implementation
|
Loading…
Reference in New Issue