Merge remote-tracking branch 'origin/main' into fix-arg-type

This commit is contained in:
sfc-gh-tclinkenbeard 2023-03-22 10:46:57 -07:00
commit 495a460887
140 changed files with 5905 additions and 2877 deletions

View File

@ -490,7 +490,7 @@ elseif(NOT WIN32 AND NOT APPLE) # Linux Only
${SHIM_LIB_TEST_EXTRA_OPTIONS}
)
endif() # End Linux only, non-sanitizer only
endif() # End Linux only
# TODO: re-enable once the old vcxproj-based build system is removed.
#generate_export_header(fdb_c EXPORT_MACRO_NAME "DLLEXPORT"
@ -536,7 +536,7 @@ fdb_install(
DESTINATION_SUFFIX "/cmake/${targets_export_name}"
COMPONENT clients)
if(NOT WIN32 AND NOT APPLE AND NOT USE_SANITIZER) # Linux Only, non-sanitizer only
if(NOT WIN32 AND NOT APPLE) # Linux Only
fdb_install(
FILES foundationdb/fdb_c_shim.h
@ -550,4 +550,4 @@ if(NOT WIN32 AND NOT APPLE AND NOT USE_SANITIZER) # Linux Only, non-sanitizer on
DESTINATION lib
COMPONENT clients)
endif() # End Linux only, non-ubsan only
endif() # End Linux only

View File

@ -69,8 +69,8 @@ private:
auto val = f.get<fdb::future_var::ValueRef>();
if (expectedVal != val) {
error(fmt::format("cancelAfterFirstResTx mismatch. expected: {:.80} actual: {:.80}",
fdb::toCharsRef(expectedVal.value()),
fdb::toCharsRef(val.value())));
fdb::toCharsRef(expectedVal),
fdb::toCharsRef(val)));
}
ctx->done();
});

View File

@ -88,8 +88,8 @@ private:
error(
fmt::format("randomCommitReadOp mismatch. key: {} expected: {:.80} actual: {:.80}",
fdb::toCharsRef((*kvPairs)[i].key),
fdb::toCharsRef(expected.value()),
fdb::toCharsRef(actual.value())));
fdb::toCharsRef(expected),
fdb::toCharsRef(actual)));
ASSERT(false);
}
}
@ -129,8 +129,8 @@ private:
if ((*results)[i] != expected) {
error(fmt::format("randomGetOp mismatch. key: {} expected: {:.80} actual: {:.80}",
fdb::toCharsRef((*keys)[i]),
fdb::toCharsRef(expected.value()),
fdb::toCharsRef((*results)[i].value())));
fdb::toCharsRef(expected),
fdb::toCharsRef((*results)[i])));
}
}
schedule(cont);

View File

@ -50,7 +50,7 @@ public:
std::optional<fdb::Value> res = copyValueRef(future.get());
if (res != value) {
error(fmt::format(
"expected: {} actual: {}", fdb::toCharsRef(value), fdb::toCharsRef(res.value())));
"expected: {} actual: {}", fdb::toCharsRef(value), fdb::toCharsRef(res)));
}
ctx->done();
});

View File

@ -162,6 +162,17 @@ CharsRef toCharsRef(const StringLike<Char>& s) noexcept {
return CharsRef(reinterpret_cast<char const*>(s.data()), s.size());
}
// get charstring view from optional bytestring: e.g. std::optional<std::basic_string{_view}<uint8_t>>
template <template <class...> class StringLike, class Char>
CharsRef toCharsRef(const std::optional<StringLike<Char>>& s) noexcept {
static_assert(sizeof(Char) == 1);
if (s) {
return CharsRef(reinterpret_cast<char const*>(s.value().data()), s.value().size());
} else {
return CharsRef("[not set]");
}
}
[[maybe_unused]] constexpr const bool OverflowCheck = false;
inline int intSize(BytesRef b) {

View File

@ -168,6 +168,11 @@ class ClientConfigTest:
self.tc.assertTrue("InitializationState" in self.status_json)
self.tc.assertEqual(expected_state, self.status_json["InitializationState"])
def get_initialization_state(self):
self.tc.assertIsNotNone(self.status_json)
self.tc.assertTrue("InitializationState" in self.status_json)
return self.status_json["InitializationState"]
def check_available_clients(self, expected_clients):
self.tc.assertIsNotNone(self.status_json)
self.tc.assertTrue("AvailableClients" in self.status_json)
@ -461,10 +466,15 @@ class ClientConfigTests(unittest.TestCase):
test.transaction_timeout = 100
test.expected_error = 1031 # Timeout
test.exec()
test.check_initialization_state("incompatible")
test.check_healthy_status(False)
test.check_available_clients([PREV_RELEASE_VERSION])
test.check_current_client(None)
init_state = test.get_initialization_state()
if init_state == "incompatible":
test.check_current_client(None)
elif init_state == "initializing":
test.check_protocol_version_not_set()
else:
self.fail("Unexpected initialization state {}".format(init_state))
def test_cannot_connect_to_coordinator(self):
# Testing a cluster file with a valid address, but no server behind it

View File

@ -83,9 +83,13 @@ add_custom_target(python_package DEPENDS ${package_file})
add_dependencies(python_package python_binding)
add_dependencies(packages python_package)
add_fdbclient_test(
NAME python_unit_tests
COMMAND python ${CMAKE_CURRENT_SOURCE_DIR}/tests/unit_tests.py
--cluster-file @CLUSTER_FILE@ --verbose
DISABLE_LOG_DUMP
)
if(NOT OPEN_FOR_IDE AND NOT USE_SANITIZER)
add_fdbclient_test(
NAME python_unit_tests
COMMAND python ${CMAKE_CURRENT_SOURCE_DIR}/tests/unit_tests.py
--cluster-file @CLUSTER_FILE@ --verbose
DISABLE_LOG_DUMP
)
endif() # NOT OPEN_FOR_IDE AND NOT USE_SANITIZER

View File

@ -5,7 +5,7 @@ set(FORCE_ALL_COMPONENTS OFF CACHE BOOL "Fails cmake if not all dependencies are
################################################################################
if(USE_JEMALLOC)
find_package(jemalloc REQUIRED)
find_package(jemalloc 5.3.0 REQUIRED)
endif()
################################################################################

View File

@ -17,7 +17,7 @@ import uuid
from functools import total_ordering
from pathlib import Path
from test_harness.version import Version
from test_harness.config import config
from test_harness.config import config, BuggifyOptionValue
from typing import Dict, List, Pattern, OrderedDict
from test_harness.summarize import Summary, SummaryTree
@ -304,6 +304,10 @@ def is_restarting_test(test_file: Path):
return False
def is_negative(test_file: Path):
return test_file.parts[-2] == "negative"
def is_no_sim(test_file: Path):
return test_file.parts[-2] == "noSim"
@ -449,7 +453,7 @@ class TestRun:
command.append("--restarting")
if self.buggify_enabled:
command += ["-b", "on"]
if config.crash_on_error:
if config.crash_on_error and not is_negative(self.test_file):
command.append("--crash")
if config.long_running:
# disable simulation speedup
@ -488,6 +492,7 @@ class TestRun:
resources.join()
# we're rounding times up, otherwise we will prefer running very short tests (<1s)
self.run_time = math.ceil(resources.time())
self.summary.is_negative_test = is_negative(self.test_file)
self.summary.runtime = resources.time()
self.summary.max_rss = resources.max_rss
self.summary.was_killed = did_kill
@ -495,7 +500,7 @@ class TestRun:
self.summary.error_out = err_out
self.summary.summarize(self.temp_path, " ".join(command))
if not self.summary.ok():
if not self.summary.is_negative_test and not self.summary.ok():
self._run_rocksdb_logtool()
return self.summary.ok()
@ -549,7 +554,12 @@ class TestRunner:
not is_no_sim(file)
and config.random.random() < config.unseed_check_ratio
)
buggify_enabled: bool = config.random.random() < config.buggify_on_ratio
buggify_enabled: bool = False
if config.buggify.value == BuggifyOptionValue.ON:
buggify_enabled = True
elif config.buggify.value == BuggifyOptionValue.RANDOM:
buggify_enabled = config.random.random() < config.buggify_on_ratio
# FIXME: support unseed checks for restarting tests
run = TestRun(
binary,

View File

@ -316,6 +316,10 @@ class Summary:
self.stderr_severity: str = '40'
self.will_restart: bool = will_restart
self.test_dir: Path | None = None
self.is_negative_test = False
self.negative_test_success = False
self.max_trace_time = -1
self.max_trace_time_type = 'None'
if uid is not None:
self.out.attributes['TestUID'] = str(uid)
@ -323,6 +327,7 @@ class Summary:
self.out.attributes['Statistics'] = stats
self.out.attributes['JoshuaSeed'] = str(config.joshua_seed)
self.out.attributes['WillRestart'] = '1' if self.will_restart else '0'
self.out.attributes['NegativeTest'] = '1' if self.is_negative_test else '0'
self.handler = ParseHandler(self.out)
self.register_handlers()
@ -371,7 +376,8 @@ class Summary:
return res
def ok(self):
return not self.error
# logical xor -- a test is successful if there was either no error or we expected errors (negative test)
return (not self.error) != self.is_negative_test
def done(self):
if config.print_coverage:
@ -435,6 +441,8 @@ class Summary:
if not self.test_end_found:
child = SummaryTree('TestUnexpectedlyNotFinished')
child.attributes['Severity'] = '40'
child.attributes['LastTraceTime'] = str(self.max_trace_time)
child.attributes['LastTraceType'] = self.max_trace_time_type
self.out.append(child)
self.error = True
if self.error_out is not None and len(self.error_out) > 0:
@ -513,6 +521,17 @@ class Summary:
self.handler.add_handler(('Severity', None), remap_event_severity)
def get_max_trace_time(attrs):
if 'Type' not in attrs:
return None
time = float(attrs['Time'])
if time >= self.max_trace_time:
self.max_trace_time = time
self.max_trace_time_type = attrs['Type']
return None
self.handler.add_handler(('Time', None), get_max_trace_time)
def program_start(attrs: Dict[str, str]):
if self.test_begin_found:
return
@ -529,6 +548,17 @@ class Summary:
self.handler.add_handler(('Type', 'ProgramStart'), program_start)
def negative_test_success(attrs: Dict[str, str]):
self.negative_test_success = True
child = SummaryTree(attrs['Type'])
for k, v in attrs:
if k != 'Type':
child.attributes[k] = v
self.out.append(child)
pass
self.handler.add_handler(('Type', 'NegativeTestSuccess'), negative_test_success)
def config_string(attrs: Dict[str, str]):
self.out.attributes['ConfigString'] = attrs['ConfigString']

View File

@ -81,6 +81,17 @@ Actors are created to monitor the reasons of key movement:
(3) `serverTeamRemover` and `machineTeamRemover` actors periodically evaluate if the number of server teams and machine teams is larger than the desired number. If so, they respectively pick a server team or a machine team to remove based on predefined criteria;
(4) `teamTracker` actor monitors a teams healthiness. When a server in the team becomes unhealthy, it issues the `RelocateShard` request to repair the replication factor. The less servers a team has, the higher priority the `RelocateShard` request will be.
#### Movement Priority
There are roughly 4 class of movement priorities
* Healthy priority. The movement is for maintain the cluster healthy status, and the priority is depended on the healthy status of the source team.
* Load balance priority. The movement is for balance cluster workload.
* Boundary change priority. The movement will change current shard boundaries.
* Others. Like resuming a in-flight movement.
Each shard movement has a priority associating with the move attempt, The explanation of each priority knob (`PRIORITY_<XXX>`) is in `ServerKnobs.h`.
In `status json` output, please look at field `.data.team_tracker.state` for team priority state.
### How to move keys?
A key range is a shard. A shard is the minimum unit of moving data. The storage servers ownership of a shard -- which SS owns which shard -- is stored in the system keyspace *serverKeys* (`\xff/serverKeys/`) and *keyServers* (`\xff/keyServers/`). To simplify the explanation, we refer to the storage servers ownership of a shard as a shards ownership.
@ -152,11 +163,11 @@ CPU utilization. This metric is in a positive relationship with “FinishedQueri
* Read-aware DD will balance the read workload under the read-skew scenario. Starting from an imbalance `STD(FinishedQueries per minute)=16k`,the best result it can achieve is `STD(FinishedQueries per minute) = 2k`.
* The typical movement size under a read-skew scenario is 100M ~ 600M under default KNOB value `READ_REBALANCE_MAX_SHARD_FRAC=0.2, READ_REBALANCE_SRC_PARALLELISM = 20`. Increasing those knobs may accelerate the converge speed with the risk of data movement churn, which overwhelms the destination and over-cold the source.
* The upper bound of `READ_REBALANCE_MAX_SHARD_FRAC` is 0.5. Any value larger than 0.5 can result in hot server switching.
* When needing a deeper diagnosis of the read aware DD, `BgDDMountainChopper_New`, and `BgDDValleyFiller_New` trace events are where to go.
* When needing a deeper diagnosis of the read aware DD, `BgDDMountainChopper`, and `BgDDValleyFiller` trace events are where to go.
## Data Distribution Diagnosis Q&A
* Why Read-aware DD hasn't been triggered when there's a read imbalance?
* Check `BgDDMountainChopper_New`, `BgDDValleyFiller_New` `SkipReason` field.
* Check `BgDDMountainChopper`, `BgDDValleyFiller` `SkipReason` field.
* The Read-aware DD is triggered, and some data movement happened, but it doesn't help the read balance. Why?
* Need to figure out which server is selected as the source and destination. The information is in `BgDDMountainChopper*`, `BgDDValleyFiller*` `DestTeam` and `SourceTeam` field.
* Also, the `DDQueueServerCounter` event tells how many times a server being a source or destination (defined in

View File

@ -483,8 +483,6 @@ status json
``status json`` will provide the cluster status in its JSON format. For a detailed description of this format, see :doc:`mr-status`.
.. _cli-throttle:
tenant
------
@ -626,6 +624,8 @@ In the event of an error, the JSON output will include an error message::
"type": "error"
}
.. _cli-throttle:
throttle
--------

View File

@ -2,6 +2,20 @@
Release Notes
#############
7.1.29
======
* Same as 7.1.28 release with AVX enabled.
7.1.28
======
* Released with AVX disabled.
* Changed log router to detect slow peeks and to automatically switch DC for peeking. `(PR #9640) <https://github.com/apple/foundationdb/pull/9640>`_
* Added multiple prefix filter support for fdbdecode. `(PR #9483) <https://github.com/apple/foundationdb/pull/9483>`_, `(PR #9489) <https://github.com/apple/foundationdb/pull/9489>`_, `(PR #9511) <https://github.com/apple/foundationdb/pull/9511>`_, and `(PR #9560) <https://github.com/apple/foundationdb/pull/9560>`_
* Enhanced fdbbackup query command to estimate data processing from a specific snapshot to a target version. `(PR #9506) <https://github.com/apple/foundationdb/pull/9506>`_
* Improved PTree insertion and erase performance for storage servers. `(PR #9508) <https://github.com/apple/foundationdb/pull/9508>`_
* Added exclude to fdbcli's configure command to prevent faulty TLogs from affecting recovery. `(PR #9404) <https://github.com/apple/foundationdb/pull/9404>`_
* Fixed getMappedRange metrics. `(PR #9331) <https://github.com/apple/foundationdb/pull/9331>`_
7.1.27
======
* Same as 7.1.26 release with AVX enabled.

View File

@ -1509,6 +1509,7 @@ DBType getDBType(std::string dbType) {
}
ACTOR Future<std::string> getLayerStatus(Reference<ReadYourWritesTransaction> tr,
IPAddress localIP,
std::string name,
std::string id,
ProgramExe exe,
@ -1544,6 +1545,9 @@ ACTOR Future<std::string> getLayerStatus(Reference<ReadYourWritesTransaction> tr
o.create("main_thread_cpu_seconds") = getProcessorTimeThread();
o.create("process_cpu_seconds") = getProcessorTimeProcess();
o.create("configured_workers") = CLIENT_KNOBS->BACKUP_TASKS_PER_AGENT;
o.create("processID") = ::getpid();
o.create("locality") = tr->getDatabase()->clientLocality.toJSON<json_spirit::mObject>();
o.create("networkAddress") = localIP.toString();
if (exe == ProgramExe::AGENT) {
static S3BlobStoreEndpoint::Stats last_stats;
@ -1789,6 +1793,21 @@ ACTOR Future<Void> statusUpdateActor(Database statusUpdateDest,
state Reference<ReadYourWritesTransaction> tr(new ReadYourWritesTransaction(statusUpdateDest));
state Future<Void> pollRateUpdater;
// In order to report a useful networkAddress to the cluster's layer status JSON object, determine which local
// network interface IP will be used to talk to the cluster. This is a blocking call, so it is only done once,
// and in a retry loop because if we can't connect to the cluster we can't do any work anyway.
state IPAddress localIP;
loop {
try {
localIP = statusUpdateDest->getConnectionRecord()->getConnectionString().determineLocalSourceIP();
break;
} catch (Error& e) {
TraceEvent(SevWarn, "AgentCouldNotDetermineLocalIP").error(e);
wait(delay(1.0));
}
}
// Register the existence of this layer in the meta key space
loop {
try {
@ -1811,7 +1830,7 @@ ACTOR Future<Void> statusUpdateActor(Database statusUpdateDest,
tr->setOption(FDBTransactionOptions::ACCESS_SYSTEM_KEYS);
tr->setOption(FDBTransactionOptions::LOCK_AWARE);
state Future<std::string> futureStatusDoc =
getLayerStatus(tr, name, id, exe, taskDest, Snapshot::True);
getLayerStatus(tr, localIP, name, id, exe, taskDest, Snapshot::True);
wait(cleanupStatus(tr, rootKey, name, id));
std::string statusdoc = wait(futureStatusDoc);
tr->set(instanceKey, statusdoc);

View File

@ -209,8 +209,9 @@ ACTOR Future<bool> metaclusterRemoveCommand(Reference<IDatabase> db, std::vector
}
void printRestoreUsage() {
fmt::print("Usage: metacluster restore <NAME> [dryrun] connection_string=<CONNECTION_STRING>\n"
"<restore_known_data_cluster|repopulate_from_data_cluster> [force_join]\n\n");
fmt::print(
"Usage: metacluster restore <NAME> [dryrun] connection_string=<CONNECTION_STRING>\n"
"<restore_known_data_cluster|repopulate_from_data_cluster> [force_join|force_reuse_tenant_id_prefix] ...\n\n");
fmt::print("Add a restored data cluster back to a metacluster.\n\n");
@ -233,30 +234,49 @@ void printRestoreUsage() {
fmt::print("clusters. Any conflicts arising between the added data cluster and existing data\n");
fmt::print("will cause the restore to fail. Before repopulating a metacluster from a data\n");
fmt::print("cluster, that data cluster needs to be detached from its prior metacluster using\n");
fmt::print("the `metacluster remove' command.\n");
fmt::print("the `metacluster remove' command.\n\n");
fmt::print("When repopulating a management cluster, it is expected that the new metacluster\n");
fmt::print("will be configured with a different tenant ID prefix. By default, reusing the same\n");
fmt::print("prefix will result in an error during the restore process. To override this behavior,\n");
fmt::print("use `force_reuse_tenant_id_prefix'.\n");
}
// metacluster restore command
ACTOR Future<bool> metaclusterRestoreCommand(Reference<IDatabase> db, std::vector<StringRef> tokens) {
if (tokens.size() < 5 || tokens.size() > 7) {
if (tokens.size() < 5 || tokens.size() > 8) {
printRestoreUsage();
return false;
}
state bool dryRun = tokens[3] == "dryrun"_sr;
state bool forceJoin = tokens[tokens.size() - 1] == "force_join"_sr;
state bool forceJoin = false;
state bool forceReuseTenantIdPrefix = false;
if (tokens.size() < 5 + (int)dryRun + (int)forceJoin) {
loop {
if (tokens.back() == "force_join"_sr) {
forceJoin = true;
} else if (tokens.back() == "force_reuse_tenant_id_prefix"_sr) {
forceReuseTenantIdPrefix = true;
} else {
break;
}
tokens.pop_back();
}
int expectedTokens = 5 + (dryRun ? 1 : 0);
if (tokens.size() != expectedTokens) {
printRestoreUsage();
return false;
}
state ClusterName clusterName = tokens[2];
state StringRef restoreType = tokens[tokens.size() - 1 - (int)forceJoin];
state StringRef restoreType = tokens.back();
// connection string
DataClusterEntry defaultEntry;
auto config = parseClusterConfiguration(tokens, defaultEntry, 3 + (int)dryRun, 3 + (int)dryRun + 1);
auto config = parseClusterConfiguration(tokens, defaultEntry, expectedTokens - 2, expectedTokens - 1);
if (!config.present()) {
return false;
} else if (!config.get().first.present()) {
@ -275,6 +295,7 @@ ACTOR Future<bool> metaclusterRestoreCommand(Reference<IDatabase> db, std::vecto
ApplyManagementClusterUpdates::True,
RestoreDryRun(dryRun),
ForceJoin(forceJoin),
ForceReuseTenantIdPrefix(forceReuseTenantIdPrefix),
&messages));
} else if (restoreType == "repopulate_from_data_cluster"_sr) {
wait(MetaclusterAPI::restoreCluster(db,
@ -283,6 +304,7 @@ ACTOR Future<bool> metaclusterRestoreCommand(Reference<IDatabase> db, std::vecto
ApplyManagementClusterUpdates::False,
RestoreDryRun(dryRun),
ForceJoin(forceJoin),
ForceReuseTenantIdPrefix(forceReuseTenantIdPrefix),
&messages));
} else {
fmt::print(stderr, "ERROR: unrecognized restore mode `{}'\n", printable(restoreType));
@ -581,15 +603,15 @@ void metaclusterGenerator(const char* text,
const char* opts[] = { "dryrun", "connection_string=", nullptr };
arrayGenerator(text, line, opts, lc);
} else {
bool dryrun = tokens[3] == "dryrun"_sr;
if (tokens.size() == 3 + (int)dryrun) {
int dryrun = tokens[3] == "dryrun"_sr ? 1 : 0;
if (tokens.size() == 3 + dryrun) {
const char* opts[] = { "connection_string=", nullptr };
arrayGenerator(text, line, opts, lc);
} else if (tokens.size() == 4 + (int)dryrun) {
} else if (tokens.size() == 4 + dryrun) {
const char* opts[] = { "restore_known_data_cluster", "repopulate_from_data_cluster", nullptr };
arrayGenerator(text, line, opts, lc);
} else if (tokens.size() == 5 + (int)dryrun) {
const char* opts[] = { "force_join", nullptr };
} else if (tokens.size() >= 5 + dryrun && tokens.size() < 7 + dryrun) {
const char* opts[] = { "force_join", "force_reuse_tenant_id_prefix", nullptr };
arrayGenerator(text, line, opts, lc);
}
}
@ -619,16 +641,16 @@ std::vector<const char*> metaclusterHintGenerator(std::vector<StringRef> const&
} else {
return {};
}
} else if (tokencmp(tokens[1], "restore") && tokens.size() < 7) {
} else if (tokencmp(tokens[1], "restore") && tokens.size() < 8) {
static std::vector<const char*> opts = { "<NAME>",
"[dryrun]",
"connection_string=<CONNECTION_STRING>",
"<restore_known_data_cluster|repopulate_from_data_cluster>",
"[force_join]" };
if (tokens.size() < 4 || (tokens[3].size() <= 6 && "dryrun"_sr.startsWith(tokens[3]))) {
return std::vector<const char*>(opts.begin() + tokens.size() - 2, opts.end());
} else if (tokens.size() < 6) {
return std::vector<const char*>(opts.begin() + tokens.size() - 1, opts.end());
"[force_join|force_reuse_tenant_id_prefix]" };
if (tokens.size() < 4 || (tokens[3].size() <= 7 && "dryrun"_sr.startsWith(tokens[3]))) {
return std::vector<const char*>(opts.begin() + std::min(tokens.size() - 2, opts.size() - 1), opts.end());
} else if (tokens.size() < 7) {
return std::vector<const char*>(opts.begin() + std::min(tokens.size() - 1, opts.size() - 1), opts.end());
} else {
return {};
}

View File

@ -324,7 +324,8 @@ ACTOR Future<bool> tenantDeleteIdCommand(Reference<IDatabase> db, std::vector<St
// tenant list command
ACTOR Future<bool> tenantListCommand(Reference<IDatabase> db, std::vector<StringRef> tokens) {
if (tokens.size() > 7) {
fmt::print("Usage: tenant list [BEGIN] [END] [limit=LIMIT] [offset=OFFSET] [state=<STATE1>,<STATE2>,...]\n\n");
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");
@ -414,6 +415,65 @@ ACTOR Future<bool> tenantListCommand(Reference<IDatabase> db, std::vector<String
}
}
void tenantGetCmdOutput(json_spirit::mValue jsonObject, bool useJson) {
if (useJson) {
json_spirit::mObject resultObj;
resultObj["tenant"] = jsonObject;
resultObj["type"] = "success";
fmt::print("{}\n",
json_spirit::write_string(json_spirit::mValue(resultObj), json_spirit::pretty_print).c_str());
} else {
JSONDoc doc(jsonObject);
int64_t id;
std::string name;
std::string prefix;
std::string tenantState;
std::string tenantLockState;
std::string lockId;
std::string tenantGroup;
std::string assignedCluster;
std::string error;
doc.get("id", id);
doc.get("prefix.printable", prefix);
doc.get("lock_state", tenantLockState);
bool hasName = doc.tryGet("name.printable", name);
bool hasTenantState = doc.tryGet("tenant_state", tenantState);
bool hasLockId = doc.tryGet("lock_id", lockId);
bool hasTenantGroup = doc.tryGet("tenant_group.printable", tenantGroup);
bool hasAssignedCluster = doc.tryGet("assigned_cluster.printable", assignedCluster);
bool hasError = doc.tryGet("error", error);
fmt::print(" id: {}\n", id);
fmt::print(" prefix: {}\n", printable(prefix));
if (hasName) {
fmt::print(" name: {}\n", name);
}
if (hasTenantState) {
fmt::print(" tenant state: {}\n", printable(tenantState));
}
fmt::print(" lock state: {}\n", tenantLockState);
if (hasLockId) {
fmt::print(" lock id: {}\n", lockId);
}
if (hasTenantGroup) {
fmt::print(" tenant group: {}\n", tenantGroup);
}
if (hasAssignedCluster) {
fmt::print(" assigned cluster: {}\n", printable(assignedCluster));
}
if (hasError) {
fmt::print(" error: {}\n", error);
}
}
}
// tenant get command
ACTOR Future<bool> tenantGetCommand(Reference<IDatabase> db, std::vector<StringRef> tokens) {
if (tokens.size() < 3 || tokens.size() > 4 || (tokens.size() == 4 && tokens[3] != "JSON"_sr)) {
@ -444,60 +504,76 @@ ACTOR Future<bool> tenantGetCommand(Reference<IDatabase> db, std::vector<StringR
}
tenantJson = tenant.get().toString();
}
json_spirit::mValue jsonObject;
json_spirit::read_string(tenantJson, jsonObject);
tenantGetCmdOutput(jsonObject, useJson);
return true;
} catch (Error& e) {
try {
wait(safeThreadFutureToFuture(tr->onError(e)));
} catch (Error& finalErr) {
state std::string errorStr;
if (finalErr.code() == error_code_special_keys_api_failure) {
std::string str = wait(getSpecialKeysFailureErrorMessage(tr));
errorStr = str;
} else if (useJson) {
errorStr = finalErr.what();
} else {
throw finalErr;
}
if (useJson) {
json_spirit::mObject resultObj;
resultObj["type"] = "error";
resultObj["error"] = errorStr;
fmt::print(
"{}\n",
json_spirit::write_string(json_spirit::mValue(resultObj), json_spirit::pretty_print).c_str());
} else {
fmt::print(stderr, "ERROR: {}\n", errorStr.c_str());
}
return false;
}
}
}
}
// tenant getId command
ACTOR Future<bool> tenantGetIdCommand(Reference<IDatabase> db, std::vector<StringRef> tokens) {
if (tokens.size() < 3 || tokens.size() > 4 || (tokens.size() == 4 && tokens[3] != "JSON"_sr)) {
fmt::print("Usage: tenant getId <ID> [JSON]\n\n");
fmt::print("Prints metadata associated with the given tenant ID.\n");
fmt::print("If JSON is specified, then the output will be in JSON format.\n");
return false;
}
state bool useJson = tokens.size() == 4;
state int64_t tenantId;
int n = 0;
if (sscanf(tokens[2].toString().c_str(), "%" PRId64 "%n", &tenantId, &n) != 1 || n != tokens[2].size() ||
tenantId < 0) {
fmt::print(stderr, "ERROR: invalid ID `{}'\n", tokens[2].toString().c_str());
return false;
}
state Reference<ITransaction> tr = db->createTransaction();
loop {
try {
tr->setOption(FDBTransactionOptions::READ_SYSTEM_KEYS);
state ClusterType clusterType = wait(TenantAPI::getClusterType(tr));
state std::string tenantJson;
if (clusterType != ClusterType::METACLUSTER_MANAGEMENT) {
TenantMapEntry entry = wait(TenantAPI::getTenantTransaction(tr, tenantId));
tenantJson = entry.toJson();
} else {
MetaclusterTenantMapEntry mEntry = wait(MetaclusterAPI::getTenantTransaction(tr, tenantId));
tenantJson = mEntry.toJson();
}
json_spirit::mValue jsonObject;
json_spirit::read_string(tenantJson, jsonObject);
if (useJson) {
json_spirit::mObject resultObj;
resultObj["tenant"] = jsonObject;
resultObj["type"] = "success";
fmt::print(
"{}\n",
json_spirit::write_string(json_spirit::mValue(resultObj), json_spirit::pretty_print).c_str());
} else {
JSONDoc doc(jsonObject);
int64_t id;
std::string prefix;
std::string tenantState;
std::string tenantLockState;
std::string lockId;
std::string tenantGroup;
std::string assignedCluster;
std::string error;
doc.get("id", id);
doc.get("prefix.printable", prefix);
doc.get("lock_state", tenantLockState);
bool hasTenantState = doc.tryGet("tenant_state", tenantState);
bool hasLockId = doc.tryGet("lock_id", lockId);
bool hasTenantGroup = doc.tryGet("tenant_group.printable", tenantGroup);
bool hasAssignedCluster = doc.tryGet("assigned_cluster.printable", assignedCluster);
bool hasError = doc.tryGet("error", error);
fmt::print(" id: {}\n", id);
fmt::print(" prefix: {}\n", printable(prefix));
if (hasTenantState) {
fmt::print(" tenant state: {}\n", printable(tenantState));
}
fmt::print(" lock state: {}\n", tenantLockState);
if (hasLockId) {
fmt::print(" lock id: {}\n", lockId);
}
if (hasTenantGroup) {
fmt::print(" tenant group: {}\n", tenantGroup);
}
if (hasAssignedCluster) {
fmt::print(" assigned cluster: {}\n", printable(assignedCluster));
}
if (hasError) {
fmt::print(" error: {}\n", error);
}
}
tenantGetCmdOutput(jsonObject, useJson);
return true;
} catch (Error& e) {
try {
@ -739,8 +815,8 @@ ACTOR Future<bool> tenantLockCommand(Reference<IDatabase> db, std::vector<String
wait(MetaclusterAPI::changeTenantLockState(db, name, desiredLockState, uid));
} else {
tr->setOption(FDBTransactionOptions::ACCESS_SYSTEM_KEYS);
auto f = tr->get(nameKey);
Optional<Value> entry = wait(safeThreadFutureToFuture(f));
state ThreadFuture<Optional<Value>> tenantFuture = tr->get(nameKey);
Optional<Value> entry = wait(safeThreadFutureToFuture(tenantFuture));
if (!entry.present()) {
fmt::print(stderr, "ERROR: Tenant `{}' does not exist\n", name);
return false;
@ -784,6 +860,8 @@ Future<bool> tenantCommand(Reference<IDatabase> db, std::vector<StringRef> token
return tenantListCommand(db, tokens);
} else if (tokencmp(tokens[1], "get")) {
return tenantGetCommand(db, tokens);
} else if (tokencmp(tokens[1], "getId")) {
return tenantGetIdCommand(db, tokens);
} else if (tokencmp(tokens[1], "configure")) {
return tenantConfigureCommand(db, tokens);
} else if (tokencmp(tokens[1], "rename")) {
@ -824,6 +902,9 @@ void tenantGenerator(const char* text,
} else if (tokens.size() == 3 && tokencmp(tokens[1], "get")) {
const char* opts[] = { "JSON", nullptr };
arrayGenerator(text, line, opts, lc);
} else if (tokens.size() == 3 && tokencmp(tokens[1], "getId")) {
const char* opts[] = { "JSON", nullptr };
arrayGenerator(text, line, opts, lc);
} else if (tokencmp(tokens[1], "configure")) {
if (tokens.size() == 3) {
const char* opts[] = { "tenant_group=", "unset", nullptr };
@ -845,7 +926,7 @@ void tenantGenerator(const char* text,
std::vector<const char*> tenantHintGenerator(std::vector<StringRef> const& tokens, bool inArgument) {
if (tokens.size() == 1) {
return { "<create|delete|deleteId|list|get|configure|rename>", "[ARGS]" };
return { "<create|delete|deleteId|list|get|getId|configure|rename>", "[ARGS]" };
} else if (tokencmp(tokens[1], "create") && tokens.size() < 5) {
static std::vector<const char*> opts = { "<NAME>",
"[tenant_group=<TENANT_GROUP>]",
@ -865,6 +946,9 @@ std::vector<const char*> tenantHintGenerator(std::vector<StringRef> const& token
} else if (tokencmp(tokens[1], "get") && tokens.size() < 4) {
static std::vector<const char*> opts = { "<NAME>", "[JSON]" };
return std::vector<const char*>(opts.begin() + tokens.size() - 2, opts.end());
} else if (tokencmp(tokens[1], "getId") && tokens.size() < 4) {
static std::vector<const char*> opts = { "<ID>", "[JSON]" };
return std::vector<const char*>(opts.begin() + tokens.size() - 2, opts.end());
} else if (tokencmp(tokens[1], "configure")) {
if (tokens.size() < 4) {
static std::vector<const char*> opts = { "<TENANT_NAME>",
@ -895,18 +979,19 @@ std::vector<const char*> tenantHintGenerator(std::vector<StringRef> const& token
}
}
CommandFactory tenantRegisterFactory("tenant",
CommandHelp("tenant <create|delete|list|get|configure|rename> [ARGS]",
"view and manage tenants in a cluster or metacluster",
"`create' and `delete' add and remove tenants from the cluster.\n"
"`list' prints a list of tenants in the cluster.\n"
"`get' prints the metadata for a particular tenant.\n"
"`configure' modifies the configuration for a tenant.\n"
"`rename' changes the name of a tenant.\n"
"`lock` locks a tenant.\n"
"`unlock` unlocks a tenant.\n"),
&tenantGenerator,
&tenantHintGenerator);
CommandFactory tenantRegisterFactory(
"tenant",
CommandHelp("tenant <create|delete|list|get|getId|configure|rename|lock|unlock> [ARGS]",
"view and manage tenants in a cluster or metacluster",
"`create' and `delete' add and remove tenants from the cluster.\n"
"`list' prints a list of tenants in the cluster.\n"
"`get' prints the metadata for a particular tenant.\n"
"`configure' modifies the configuration for a tenant.\n"
"`rename' changes the name of a tenant.\n"
"`lock` locks a tenant.\n"
"`unlock` unlocks a tenant.\n"),
&tenantGenerator,
&tenantHintGenerator);
// Generate hidden commands for the old versions of the tenant commands
CommandFactory createTenantFactory("createtenant");

View File

@ -754,10 +754,10 @@ def exclude(logger):
assert coordinator_list[0]["address"] == excluded_address
break
elif (
"ERROR: This exclude may cause the total free space in the cluster to drop below 10%."
"ERROR: This exclude may cause the total available space in the cluster to drop below 10%."
in error_message
):
# exclude the process may cause the free space not enough
# exclude the process may cause the available space not enough
# use FORCE option to ignore it and proceed
assert not force
force = True
@ -1069,10 +1069,17 @@ def tenant_get(logger):
output = run_fdbcli_command("tenant get tenant")
lines = output.split("\n")
assert len(lines) == 3
assert len(lines) == 4
assert lines[0].strip().startswith("id: ")
assert lines[1].strip().startswith("prefix: ")
assert lines[2].strip() == "lock state: unlocked"
assert lines[2].strip().startswith("name: ")
assert lines[3].strip() == "lock state: unlocked"
# id = lines[0].strip().removeprefix("id: ")
# Workaround until Python 3.9+ for removeprefix
id = lines[0].strip()[len("id: "):]
id_output = run_fdbcli_command("tenant getId {}".format(id))
assert id_output == output
output = run_fdbcli_command("tenant get tenant JSON")
json_output = json.loads(output, strict=False)
@ -1092,13 +1099,23 @@ def tenant_get(logger):
assert "lock_state" in json_output["tenant"]
assert json_output["tenant"]["lock_state"] == "unlocked"
id_output = run_fdbcli_command("tenant getId {} JSON".format(id))
assert id_output == output
output = run_fdbcli_command("tenant get tenant2")
lines = output.split("\n")
assert len(lines) == 4
assert len(lines) == 5
assert lines[0].strip().startswith("id: ")
assert lines[1].strip().startswith("prefix: ")
assert lines[2].strip() == "lock state: unlocked"
assert lines[3].strip() == "tenant group: tenant_group2"
assert lines[2].strip().startswith("name: ")
assert lines[3].strip() == "lock state: unlocked"
assert lines[4].strip() == "tenant group: tenant_group2"
# id2 = lines[0].strip().removeprefix("id: ")
# Workaround until Python 3.9+ for removeprefix
id2 = lines[0].strip()[len("id: "):]
id_output = run_fdbcli_command("tenant getId {}".format(id2))
assert id_output == output
output = run_fdbcli_command("tenant get tenant2 JSON")
json_output = json.loads(output, strict=False)
@ -1119,6 +1136,8 @@ def tenant_get(logger):
assert "base64" in json_output["tenant"]["tenant_group"]
assert json_output["tenant"]["tenant_group"]["printable"] == "tenant_group2"
id_output = run_fdbcli_command("tenant getId {} JSON".format(id2))
assert id_output == output
@enable_logging()
def tenant_configure(logger):
@ -1134,8 +1153,8 @@ def tenant_configure(logger):
output = run_fdbcli_command("tenant get tenant")
lines = output.split("\n")
assert len(lines) == 4
assert lines[3].strip() == "tenant group: tenant_group1"
assert len(lines) == 5
assert lines[4].strip() == "tenant group: tenant_group1"
output = run_fdbcli_command("tenant configure tenant unset tenant_group")
assert output == "The configuration for tenant `tenant' has been updated"
@ -1147,7 +1166,7 @@ def tenant_configure(logger):
output = run_fdbcli_command("tenant get tenant")
lines = output.split("\n")
assert len(lines) == 3
assert len(lines) == 4
output = run_fdbcli_command_and_get_error(
"tenant configure tenant tenant_group=tenant_group1 tenant_group=tenant_group2"

View File

@ -193,5 +193,46 @@ TEST_CASE("/fdbserver/blobgranule/isRangeCoveredByBlob") {
testAddChunkRange("key_b1"_sr, "key_b9"_sr, continuedChunks);
ASSERT(isRangeFullyCovered(KeyRangeRef("key_a1"_sr, "key_b9"_sr), continuedChunks));
}
// check functionality of isCovered()
{
std::vector<KeyRangeRef> ranges;
ranges.push_back(KeyRangeRef("key_a"_sr, "key_b"_sr));
ranges.push_back(KeyRangeRef("key_x"_sr, "key_y"_sr));
ASSERT(KeyRangeRef("key_x"_sr, "key_y"_sr).isCovered(ranges));
ranges.clear();
ranges.push_back(KeyRangeRef("key_a"_sr, "key_b"_sr));
ranges.push_back(KeyRangeRef("key_v"_sr, "key_y"_sr));
ASSERT(KeyRangeRef("key_x"_sr, "key_y"_sr).isCovered(ranges));
ranges.clear();
ranges.push_back(KeyRangeRef("key_a"_sr, "key_b"_sr));
ranges.push_back(KeyRangeRef("key_x"_sr, "key_xa"_sr));
ranges.push_back(KeyRangeRef("key_xa"_sr, "key_ya"_sr));
ASSERT(KeyRangeRef("key_x"_sr, "key_y"_sr).isCovered(ranges));
ranges.clear();
ranges.push_back(KeyRangeRef("key_a"_sr, "key_b"_sr));
ranges.push_back(KeyRangeRef("key_x"_sr, "key_xa"_sr));
ranges.push_back(KeyRangeRef("key_xa"_sr, "key_xb"_sr));
ASSERT(!KeyRangeRef("key_x"_sr, "key_y"_sr).isCovered(ranges));
ranges.clear();
ranges.push_back(KeyRangeRef("key_a"_sr, "key_b"_sr));
ranges.push_back(KeyRangeRef("key_x"_sr, "key_xa"_sr));
ASSERT(!KeyRangeRef("key_x"_sr, "key_y"_sr).isCovered(ranges));
ranges.clear();
ranges.push_back(KeyRangeRef("key_a"_sr, "key_b"_sr));
ranges.push_back(KeyRangeRef("key_xa"_sr, "key_y"_sr));
ASSERT(!KeyRangeRef("key_x"_sr, "key_y"_sr).isCovered(ranges));
ranges.clear();
ranges.push_back(KeyRangeRef("key_a"_sr, "key_b"_sr));
ranges.push_back(KeyRangeRef("key_x"_sr, "key_y"_sr));
ASSERT(!KeyRangeRef("key_a"_sr, "key_y"_sr).isCovered(ranges));
}
return Void();
}

View File

@ -287,6 +287,7 @@ void ClientKnobs::initialize(Randomize randomize) {
init( BG_MAX_GRANULE_PARALLELISM, 10 );
init( BG_TOO_MANY_GRANULES, 20000 );
init( BLOB_METADATA_REFRESH_INTERVAL, 3600 ); if ( randomize && BUGGIFY ) { BLOB_METADATA_REFRESH_INTERVAL = deterministicRandom()->randomInt(5, 120); }
init( ENABLE_BLOB_GRANULE_FILE_LOGICAL_SIZE, false ); if ( randomize && BUGGIFY ) { ENABLE_BLOB_GRANULE_FILE_LOGICAL_SIZE = true; }
init( CHANGE_QUORUM_BAD_STATE_RETRY_TIMES, 3 );
init( CHANGE_QUORUM_BAD_STATE_RETRY_DELAY, 2.0 );

View File

@ -34,10 +34,8 @@
#include "fdbclient/CoordinationInterface.h"
// Determine public IP address by calling the first available coordinator.
// If fail connecting all coordinators, throw bind_failed().
IPAddress determinePublicIPAutomatically(ClusterConnectionString& ccs) {
int size = ccs.coords.size() + ccs.hostnames.size();
IPAddress ClusterConnectionString::determineLocalSourceIP() const {
int size = coords.size() + hostnames.size();
int index = 0;
loop {
try {
@ -48,10 +46,10 @@ IPAddress determinePublicIPAutomatically(ClusterConnectionString& ccs) {
NetworkAddress coordAddr;
// Try coords first, because they don't need to be resolved.
if (index < ccs.coords.size()) {
coordAddr = ccs.coords[index];
if (index < coords.size()) {
coordAddr = coords[index];
} else {
Hostname& h = ccs.hostnames[index - ccs.coords.size()];
const Hostname& h = hostnames[index - coords.size()];
Optional<NetworkAddress> resolvedAddr = h.resolveBlocking();
if (!resolvedAddr.present()) {
throw lookup_failed();

View File

@ -32,6 +32,7 @@ FDB_DEFINE_BOOLEAN_PARAM(RunOnDisconnectedCluster);
FDB_DEFINE_BOOLEAN_PARAM(RunOnMismatchedCluster);
FDB_DEFINE_BOOLEAN_PARAM(RestoreDryRun);
FDB_DEFINE_BOOLEAN_PARAM(ForceJoin);
FDB_DEFINE_BOOLEAN_PARAM(ForceReuseTenantIdPrefix);
FDB_DEFINE_BOOLEAN_PARAM(ForceRemove);
FDB_DEFINE_BOOLEAN_PARAM(IgnoreCapacityLimit);

View File

@ -2223,8 +2223,6 @@ void DatabaseContext::expireThrottles() {
}
}
extern IPAddress determinePublicIPAutomatically(ClusterConnectionString& ccs);
// Initialize tracing for FDB client
//
// connRecord is necessary for determining the local IP, which is then included in the trace
@ -2256,7 +2254,7 @@ void initializeClientTracing(Reference<IClusterConnectionRecord> connRecord, Opt
Optional<NetworkAddress> localAddress;
if (connRecord) {
auto publicIP = determinePublicIPAutomatically(connRecord->getConnectionString());
auto publicIP = connRecord->getConnectionString().determineLocalSourceIP();
localAddress = NetworkAddress(publicIP, ::getpid());
}
platform::ImageInfo imageInfo = platform::getImageInfo();
@ -8986,7 +8984,7 @@ static Future<Void> createCheckpointImpl(T tr,
for (const auto& [srcId, ranges] : rangeMap) {
// The checkpoint request is sent to all replicas, in case any of them is unhealthy.
// An alternative is to choose a healthy replica.
const UID checkpointID = UID(srcId.first(), deterministicRandom()->randomUInt64());
const UID checkpointID = UID(deterministicRandom()->randomUInt64(), srcId.first());
CheckpointMetaData checkpoint(ranges, format, srcMap[srcId], checkpointID, actionId.get());
checkpoint.setState(CheckpointMetaData::Pending);
tr->set(checkpointKeyFor(checkpointID), checkpointValue(checkpoint));
@ -9072,17 +9070,20 @@ ACTOR static Future<CheckpointMetaData> getCheckpointMetaDataInternal(KeyRange r
throw error.get();
}
ACTOR static Future<std::vector<CheckpointMetaData>> getCheckpointMetaDataForRange(Database cx,
KeyRange range,
Version version,
CheckpointFormat format,
Optional<UID> actionId,
double timeout) {
ACTOR static Future<std::vector<std::pair<KeyRange, CheckpointMetaData>>> getCheckpointMetaDataForRange(
Database cx,
KeyRange range,
Version version,
CheckpointFormat format,
Optional<UID> actionId,
double timeout) {
state Span span("NAPI:GetCheckpointMetaDataForRange"_loc);
state int index = 0;
state std::vector<Future<CheckpointMetaData>> futures;
state std::vector<KeyRangeLocationInfo> locations;
loop {
locations.clear();
TraceEvent(SevDebug, "GetCheckpointMetaDataForRangeBegin")
.detail("Range", range.toString())
.detail("Version", version)
@ -9090,17 +9091,17 @@ ACTOR static Future<std::vector<CheckpointMetaData>> getCheckpointMetaDataForRan
futures.clear();
try {
state std::vector<KeyRangeLocationInfo> locations =
wait(getKeyRangeLocations(cx,
TenantInfo(),
range,
CLIENT_KNOBS->TOO_MANY,
Reverse::False,
&StorageServerInterface::checkpoint,
span.context,
Optional<UID>(),
UseProvisionalProxies::False,
latestVersion));
wait(store(locations,
getKeyRangeLocations(cx,
TenantInfo(),
range,
CLIENT_KNOBS->TOO_MANY,
Reverse::False,
&StorageServerInterface::checkpoint,
span.context,
Optional<UID>(),
UseProvisionalProxies::False,
latestVersion)));
for (index = 0; index < locations.size(); ++index) {
futures.push_back(getCheckpointMetaDataInternal(
@ -9134,36 +9135,38 @@ ACTOR static Future<std::vector<CheckpointMetaData>> getCheckpointMetaDataForRan
}
}
std::vector<CheckpointMetaData> res;
std::vector<std::pair<KeyRange, CheckpointMetaData>> res;
for (index = 0; index < futures.size(); ++index) {
TraceEvent(SevDebug, "GetCheckpointShardEnd").detail("Checkpoint", futures[index].get().toString());
res.push_back(futures[index].get());
TraceEvent(SevDebug, "GetCheckpointShardEnd")
.detail("Range", locations[index].range)
.detail("Checkpoint", futures[index].get().toString());
res.emplace_back(locations[index].range, futures[index].get());
}
return res;
}
ACTOR Future<std::vector<CheckpointMetaData>> getCheckpointMetaData(Database cx,
std::vector<KeyRange> ranges,
Version version,
CheckpointFormat format,
Optional<UID> actionId,
double timeout) {
state std::vector<Future<std::vector<CheckpointMetaData>>> futures;
ACTOR Future<std::vector<std::pair<KeyRange, CheckpointMetaData>>> getCheckpointMetaData(Database cx,
std::vector<KeyRange> ranges,
Version version,
CheckpointFormat format,
Optional<UID> actionId,
double timeout) {
state std::vector<Future<std::vector<std::pair<KeyRange, CheckpointMetaData>>>> futures;
// TODO(heliu): Avoid send requests to the same shard.
for (const auto& range : ranges) {
futures.push_back(getCheckpointMetaDataForRange(cx, range, version, format, actionId, timeout));
}
std::vector<std::vector<CheckpointMetaData>> results = wait(getAll(futures));
std::vector<std::vector<std::pair<KeyRange, CheckpointMetaData>>> results = wait(getAll(futures));
std::unordered_set<CheckpointMetaData> checkpoints;
std::vector<std::pair<KeyRange, CheckpointMetaData>> res;
for (const auto& r : results) {
checkpoints.insert(r.begin(), r.end());
ASSERT(!r.empty());
res.insert(res.end(), r.begin(), r.end());
}
return std::vector<CheckpointMetaData>(checkpoints.begin(), checkpoints.end());
return res;
}
ACTOR Future<bool> checkSafeExclusions(Database cx, std::vector<AddressExclusion> exclusions) {
@ -9361,6 +9364,11 @@ void DatabaseContext::setSharedState(DatabaseSharedState* p) {
sharedStatePtr->refCount++;
}
// FIXME: this has undesired head-of-line-blocking behavior in the case of large version jumps.
// For example, say that The current feed version is 100, and one waiter wants to wait for the feed version >= 1000.
// This will send a request with minVersion=1000. Then say someone wants to wait for feed version >= 200. Because we've
// already blocked this updater on version 1000, even if the feed would already be at version 200+, we won't get an
// empty version response until version 1000.
ACTOR Future<Void> storageFeedVersionUpdater(StorageServerInterface interf, ChangeFeedStorageData* self) {
loop {
if (self->version.get() < self->desired.get()) {

View File

@ -113,6 +113,15 @@ ACTOR Future<RESTConnectionPool::ReusableConnection> connect_impl(Reference<REST
RESTConnectionPoolKey connectKey,
bool isSecure,
int maxConnLife) {
if (FLOW_KNOBS->REST_LOG_LEVEL >= RESTLogSeverity::VERBOSE) {
TraceEvent("RESTUtilConnectStart")
.detail("Host", connectKey.first)
.detail("Service", connectKey.second)
.detail("IsSecure", isSecure)
.detail("ConnectPoolNumKeys", connectionPool->connectionPoolMap.size());
}
auto poolItr = connectionPool->connectionPoolMap.find(connectKey);
while (poolItr != connectionPool->connectionPoolMap.end() && !poolItr->second.empty()) {
RESTConnectionPool::ReusableConnection rconn = poolItr->second.front();
@ -120,32 +129,31 @@ ACTOR Future<RESTConnectionPool::ReusableConnection> connect_impl(Reference<REST
if (rconn.expirationTime > now()) {
if (FLOW_KNOBS->REST_LOG_LEVEL >= RESTLogSeverity::DEBUG) {
TraceEvent("RESTClientReuseConn")
TraceEvent("RESTUtilReuseConn")
.detail("Host", connectKey.first)
.detail("Service", connectKey.second)
.detail("RemoteEndpoint", rconn.conn->getPeerAddress())
.detail("ExpireIn", rconn.expirationTime - now());
.detail("ExpireIn", rconn.expirationTime - now())
.detail("NumConnsInPool", poolItr->second.size());
}
return rconn;
}
}
ASSERT(poolItr == connectionPool->connectionPoolMap.end() || poolItr->second.empty());
// No valid connection exists, create a new one
state Reference<IConnection> conn =
wait(INetworkConnections::net()->connect(connectKey.first, connectKey.second, isSecure));
state Reference<IConnection> conn = wait(INetworkConnections::net()->connect(connectKey.first, connectKey.second));
wait(conn->connectHandshake());
RESTConnectionPool::ReusableConnection reusableConn =
RESTConnectionPool::ReusableConnection({ conn, now() + maxConnLife });
connectionPool->connectionPoolMap.insert(
{ connectKey, std::queue<RESTConnectionPool::ReusableConnection>({ reusableConn }) });
TraceEvent("RESTClientCreateNewConn")
TraceEvent("RESTTUilCreateNewConn")
.suppressFor(60)
.detail("Host", connectKey.first)
.detail("Service", connectKey.second)
.detail("RemoteEndpoint", conn->getPeerAddress());
return reusableConn;
.detail("RemoteEndpoint", conn->getPeerAddress())
.detail("ConnPoolSize", connectionPool->connectionPoolMap.size());
return RESTConnectionPool::ReusableConnection({ conn, now() + maxConnLife });
}
Future<RESTConnectionPool::ReusableConnection> RESTConnectionPool::connect(RESTConnectionPoolKey connectKey,
@ -157,18 +165,35 @@ Future<RESTConnectionPool::ReusableConnection> RESTConnectionPool::connect(RESTC
void RESTConnectionPool::returnConnection(RESTConnectionPoolKey connectKey,
ReusableConnection& rconn,
const int maxConnections) {
if (FLOW_KNOBS->REST_LOG_LEVEL >= RESTLogSeverity::VERBOSE) {
TraceEvent("RESTUtilReturnConnStart")
.detail("Host", connectKey.first)
.detail("Service", connectKey.second)
.detail("ConnectPoolNumKeys", connectionPoolMap.size());
}
auto poolItr = connectionPoolMap.find(connectKey);
if (poolItr == connectionPoolMap.end()) {
throw rest_connectpool_key_not_found();
}
if (FLOW_KNOBS->REST_LOG_LEVEL >= RESTLogSeverity::DEBUG) {
TraceEvent("RESTClientReturnConn").detail("Host", connectKey.first).detail("Service", connectKey.second);
}
// If it expires in the future then add it to the pool in the front iff connection pool size is not maxed
if (rconn.expirationTime > now() && poolItr->second.size() < maxConnections) {
poolItr->second.push(rconn);
if (rconn.expirationTime > now()) {
bool returned = true;
if (poolItr == connectionPoolMap.end()) {
connectionPoolMap.insert({ connectKey, std::queue<RESTConnectionPool::ReusableConnection>({ rconn }) });
} else if (poolItr->second.size() < maxConnections) {
poolItr->second.push(rconn);
} else {
// Connection pool at its capacity; do nothing
returned = false;
}
if (FLOW_KNOBS->REST_LOG_LEVEL >= RESTLogSeverity::DEBUG && returned) {
poolItr = connectionPoolMap.find(connectKey);
TraceEvent("RESTUtilReturnConnToPool")
.detail("Host", connectKey.first)
.detail("Service", connectKey.second)
.detail("ConnPoolSize", connectionPoolMap.size())
.detail("CachedConns", poolItr->second.size())
.detail("TimeToExpire", rconn.expirationTime - now());
}
}
rconn.conn = Reference<IConnection>();
}
@ -206,8 +231,9 @@ void RESTUrl::parseUrl(const std::string& fullUrl) {
// extract 'resource' and optional 'parameter list' if supplied in the URL
uint8_t foundSeparator = 0;
StringRef hostPort = t.eatAny("/?", &foundSeparator);
this->resource = "/";
if (foundSeparator == '/') {
this->resource = t.eat("?").toString();
this->resource += t.eat("?").toString();
this->reqParameters = t.eat().toString();
}
@ -222,7 +248,7 @@ void RESTUrl::parseUrl(const std::string& fullUrl) {
this->service = hRef.eat().toString();
if (FLOW_KNOBS->REST_LOG_LEVEL >= RESTLogSeverity::DEBUG) {
TraceEvent("RESTClientParseURI")
TraceEvent("RESTUtilParseURI")
.detail("URI", fullUrl)
.detail("Host", this->host)
.detail("Service", this->service)
@ -231,7 +257,7 @@ void RESTUrl::parseUrl(const std::string& fullUrl) {
.detail("ConnectionType", this->connType.toString());
}
} catch (std::string& err) {
TraceEvent(SevWarnAlways, "RESTClientParseError").detail("URI", fullUrl).detail("Error", err);
TraceEvent(SevWarnAlways, "RESTUtilParseError").detail("URI", fullUrl).detail("Error", err);
throw rest_invalid_uri();
}
}
@ -271,7 +297,7 @@ TEST_CASE("/RESTUtils/ValidURIWithService") {
ASSERT_EQ(r.connType.secure, RESTConnectionType::SECURE_CONNECTION);
ASSERT_EQ(r.host.compare("host"), 0);
ASSERT_EQ(r.service.compare("80"), 0);
ASSERT_EQ(r.resource.compare("foo/bar"), 0);
ASSERT_EQ(r.resource.compare("/foo/bar"), 0);
return Void();
}
@ -281,7 +307,17 @@ TEST_CASE("/RESTUtils/ValidURIWithoutService") {
ASSERT_EQ(r.connType.secure, RESTConnectionType::SECURE_CONNECTION);
ASSERT_EQ(r.host.compare("host"), 0);
ASSERT(r.service.empty());
ASSERT_EQ(r.resource.compare("foo/bar"), 0);
ASSERT_EQ(r.resource.compare("/foo/bar"), 0);
return Void();
}
TEST_CASE("/RESTUtils/ValidURIWithExtraForwardSlash") {
std::string uri("https://host//foo/bar");
RESTUrl r(uri);
ASSERT_EQ(r.connType.secure, RESTConnectionType::SECURE_CONNECTION);
ASSERT_EQ(r.host.compare("host"), 0);
ASSERT(r.service.empty());
ASSERT_EQ(r.resource.compare("//foo/bar"), 0);
return Void();
}
@ -291,14 +327,14 @@ TEST_CASE("/RESTUtils/ValidURIWithParamsSecure") {
ASSERT_EQ(r.connType.secure, RESTConnectionType::SECURE_CONNECTION);
ASSERT_EQ(r.host.compare("host"), 0);
ASSERT(r.service.empty());
ASSERT_EQ(r.resource.compare("foo/bar"), 0);
ASSERT_EQ(r.resource.compare("/foo/bar"), 0);
ASSERT_EQ(r.reqParameters.compare("param1,param2"), 0);
return Void();
}
TEST_CASE("/RESTUtils/ValidURIWithParamsKnobNotEnabled") {
auto& g_knobs = IKnobCollection::getMutableGlobalKnobCollection();
g_knobs.setKnob("rest_kms_enable_not_secure_connection", KnobValueRef::create(bool{ false }));
g_knobs.setKnob("rest_kms_allow_not_secure_connection", KnobValueRef::create(bool{ false }));
std::string uri("http://host/foo/bar?param1,param2");
try {
RESTUrl r(uri);
@ -311,13 +347,13 @@ TEST_CASE("/RESTUtils/ValidURIWithParamsKnobNotEnabled") {
TEST_CASE("/RESTUtils/ValidURIWithParams") {
auto& g_knobs = IKnobCollection::getMutableGlobalKnobCollection();
g_knobs.setKnob("rest_kms_enable_not_secure_connection", KnobValueRef::create(bool{ true }));
g_knobs.setKnob("rest_kms_allow_not_secure_connection", KnobValueRef::create(bool{ true }));
std::string uri("http://host/foo/bar?param1,param2");
RESTUrl r(uri);
ASSERT_EQ(r.connType.secure, RESTConnectionType::NOT_SECURE_CONNECTION);
ASSERT_EQ(r.host.compare("host"), 0);
ASSERT(r.service.empty());
ASSERT_EQ(r.resource.compare("foo/bar"), 0);
ASSERT_EQ(r.resource.compare("/foo/bar"), 0);
ASSERT_EQ(r.reqParameters.compare("param1,param2"), 0);
return Void();
}

View File

@ -898,6 +898,7 @@ const KeyRef JSONSchemas::statusSchema = R"statusSchema(
"healthy_repartitioning",
"healthy_removing_server",
"healthy_rebalancing",
"healthy_perpetual_wiggle",
"healthy"
]
},
@ -933,6 +934,7 @@ const KeyRef JSONSchemas::statusSchema = R"statusSchema(
"healthy_repartitioning",
"healthy_removing_server",
"healthy_rebalancing",
"healthy_perpetual_wiggle",
"healthy"
]
},

View File

@ -79,6 +79,7 @@ void ServerKnobs::initialize(Randomize randomize, ClientKnobs* clientKnobs, IsSi
init( DESIRED_GET_MORE_DELAY, 0.005 );
init( CONCURRENT_LOG_ROUTER_READS, 5 ); if( randomize && BUGGIFY ) CONCURRENT_LOG_ROUTER_READS = 1;
init( LOG_ROUTER_PEEK_FROM_SATELLITES_PREFERRED, 1 ); if( randomize && BUGGIFY ) LOG_ROUTER_PEEK_FROM_SATELLITES_PREFERRED = 0;
init( LOG_ROUTER_PEEK_SWITCH_DC_TIME, 60.0 );
init( DISK_QUEUE_ADAPTER_MIN_SWITCH_TIME, 1.0 );
init( DISK_QUEUE_ADAPTER_MAX_SWITCH_TIME, 5.0 );
init( TLOG_SPILL_REFERENCE_MAX_PEEK_MEMORY_BYTES, 2e9 ); if ( randomize && BUGGIFY ) TLOG_SPILL_REFERENCE_MAX_PEEK_MEMORY_BYTES = 2e6;
@ -170,9 +171,11 @@ void ServerKnobs::initialize(Randomize randomize, ClientKnobs* clientKnobs, IsSi
// Data distribution
init( SHARD_ENCODE_LOCATION_METADATA, false ); if( randomize && BUGGIFY ) SHARD_ENCODE_LOCATION_METADATA = true;
init( ENABLE_DD_PHYSICAL_SHARD, false ); // EXPERIMENTAL; If true, SHARD_ENCODE_LOCATION_METADATA must be true; When true, optimization of data move between DCs is disabled
init( ENABLE_DD_PHYSICAL_SHARD_MOVE, false );
init( MAX_PHYSICAL_SHARD_BYTES, 10000000 ); // 10 MB; for ENABLE_DD_PHYSICAL_SHARD; smaller leads to larger number of physicalShard per storage server
init( PHYSICAL_SHARD_METRICS_DELAY, 300.0 ); // 300 seconds; for ENABLE_DD_PHYSICAL_SHARD
init( ANONYMOUS_PHYSICAL_SHARD_TRANSITION_TIME, 600.0 ); if( randomize && BUGGIFY ) ANONYMOUS_PHYSICAL_SHARD_TRANSITION_TIME = 0.0; // 600 seconds; for ENABLE_DD_PHYSICAL_SHARD
init( PHYSICAL_SHARD_MOVE_VERBOSE_TRACKING, false );
init( READ_REBALANCE_CPU_THRESHOLD, 15.0 );
init( READ_REBALANCE_SRC_PARALLELISM, 20 );
init( READ_REBALANCE_SHARD_TOPK, READ_REBALANCE_SRC_PARALLELISM * 2 );
@ -316,6 +319,12 @@ void ServerKnobs::initialize(Randomize randomize, ClientKnobs* clientKnobs, IsSi
init( TENANT_CACHE_STORAGE_USAGE_TRACE_INTERVAL, 300 );
init( CP_FETCH_TENANTS_OVER_STORAGE_QUOTA_INTERVAL, 5 ); if( randomize && BUGGIFY ) CP_FETCH_TENANTS_OVER_STORAGE_QUOTA_INTERVAL = deterministicRandom()->randomInt(1, 10);
init( DD_BUILD_EXTRA_TEAMS_OVERRIDE, 10 ); if( randomize && BUGGIFY ) DD_BUILD_EXTRA_TEAMS_OVERRIDE = 2;
// Large teams are disabled when SHARD_ENCODE_LOCATION_METADATA is enabled
init( DD_MAXIMUM_LARGE_TEAMS, 100 ); if( randomize && BUGGIFY ) DD_MAXIMUM_LARGE_TEAMS = 0;
init( DD_MAXIMUM_LARGE_TEAM_CLEANUP, 10000 ); if( randomize && BUGGIFY ) DD_MAXIMUM_LARGE_TEAM_CLEANUP = 10;
init( DD_LARGE_TEAM_DELAY, 60.0 );
init( DD_FIX_WRONG_REPLICAS_DELAY, 60.0 );
// TeamRemover
init( TR_FLAG_DISABLE_MACHINE_TEAM_REMOVER, false ); if( randomize && BUGGIFY ) TR_FLAG_DISABLE_MACHINE_TEAM_REMOVER = deterministicRandom()->random01() < 0.1 ? true : false; // false by default. disable the consistency check when it's true
@ -395,11 +404,11 @@ void ServerKnobs::initialize(Randomize randomize, ClientKnobs* clientKnobs, IsSi
// KeyValueStoreRocksDB
init( ROCKSDB_SET_READ_TIMEOUT, !isSimulated );
init( ROCKSDB_LEVEL_COMPACTION_DYNAMIC_LEVEL_BYTES, true ); if( randomize && BUGGIFY ) ROCKSDB_LEVEL_COMPACTION_DYNAMIC_LEVEL_BYTES = false;
init( ROCKSDB_SUGGEST_COMPACT_CLEAR_RANGE, true ); if( randomize && BUGGIFY ) ROCKSDB_SUGGEST_COMPACT_CLEAR_RANGE = false;
init( ROCKSDB_SUGGEST_COMPACT_CLEAR_RANGE, false );
init( ROCKSDB_READ_RANGE_ROW_LIMIT, 65535 ); if( randomize && BUGGIFY ) ROCKSDB_READ_RANGE_ROW_LIMIT = deterministicRandom()->randomInt(2, 10);
init( ROCKSDB_READER_THREAD_PRIORITY, 0 );
init( ROCKSDB_WRITER_THREAD_PRIORITY, 0 );
init( ROCKSDB_BACKGROUND_PARALLELISM, 4 );
init( ROCKSDB_BACKGROUND_PARALLELISM, 2 );
init( ROCKSDB_READ_PARALLELISM, 4 );
init( ROCKSDB_CHECKPOINT_READER_PARALLELISM, 4 );
// If true, do not process and store RocksDB logs
@ -411,18 +420,22 @@ void ServerKnobs::initialize(Randomize randomize, ClientKnobs* clientKnobs, IsSi
init( ROCKSDB_UNSAFE_AUTO_FSYNC, false );
init( ROCKSDB_PERIODIC_COMPACTION_SECONDS, 0 );
init( ROCKSDB_PREFIX_LEN, 0 );
init( ROCKSDB_MEMTABLE_PREFIX_BLOOM_SIZE_RATIO, 0.1 );
init( ROCKSDB_BLOOM_BITS_PER_KEY, 10 );
init( ROCKSDB_BLOOM_WHOLE_KEY_FILTERING, false );
init( ROCKSDB_MAX_AUTO_READAHEAD_SIZE, 0 );
// If rocksdb block cache size is 0, the default 8MB is used.
int64_t blockCacheSize = isSimulated ? 16 * 1024 * 1024 : 1024 * 1024 * 1024 /* 1GB */;
int64_t blockCacheSize = isSimulated ? 16 * 1024 * 1024 : 2147483648 /* 2GB */;
init( ROCKSDB_BLOCK_CACHE_SIZE, blockCacheSize );
init( ROCKSDB_METRICS_DELAY, 60.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, 5.0 ); if (isSimulated) ROCKSDB_READ_VALUE_TIMEOUT = 5 * 60;
init( ROCKSDB_READ_VALUE_PREFIX_TIMEOUT, 5.0 ); if (isSimulated) ROCKSDB_READ_VALUE_PREFIX_TIMEOUT = 5 * 60;
init( ROCKSDB_READ_RANGE_TIMEOUT, 5.0 ); if (isSimulated) ROCKSDB_READ_RANGE_TIMEOUT = 5 * 60;
init( ROCKSDB_READ_CHECKPOINT_TIMEOUT, 60.0 ); if (isSimulated) ROCKSDB_READ_CHECKPOINT_TIMEOUT = 5 * 60;
init( ROCKSDB_CHECKPOINT_READ_AHEAD_SIZE, 2 << 20 ); // 2M
init( ROCKSDB_READ_VALUE_TIMEOUT, isSimulated ? 300.0 : 5.0 );
init( ROCKSDB_READ_VALUE_PREFIX_TIMEOUT, isSimulated ? 300.0 : 5.0 );
init( ROCKSDB_READ_RANGE_TIMEOUT, isSimulated ? 300.0 : 5.0 );
init( ROCKSDB_READ_CHECKPOINT_TIMEOUT, isSimulated ? 300.0 : 5.0 );
init( ROCKSDB_CHECKPOINT_READ_AHEAD_SIZE, 2 << 20 ); // 2M
init( ROCKSDB_READ_QUEUE_WAIT, 1.0 );
init( ROCKSDB_READ_QUEUE_HARD_MAX, 1000 );
init( ROCKSDB_READ_QUEUE_SOFT_MAX, 500 );
@ -443,7 +456,7 @@ void ServerKnobs::initialize(Randomize randomize, ClientKnobs* clientKnobs, IsSi
init( ROCKSDB_PERFCONTEXT_ENABLE, false ); if( randomize && BUGGIFY ) ROCKSDB_PERFCONTEXT_ENABLE = deterministicRandom()->coinflip();
init( ROCKSDB_PERFCONTEXT_SAMPLE_RATE, 0.0001 );
init( ROCKSDB_METRICS_SAMPLE_INTERVAL, 0.0);
init( ROCKSDB_MAX_SUBCOMPACTIONS, 2 );
init( ROCKSDB_MAX_SUBCOMPACTIONS, 0 );
init( ROCKSDB_SOFT_PENDING_COMPACT_BYTES_LIMIT, 64000000000 ); // 64GB, Rocksdb option, Writes will slow down.
init( ROCKSDB_HARD_PENDING_COMPACT_BYTES_LIMIT, 100000000000 ); // 100GB, Rocksdb option, Writes will stall.
init( ROCKSDB_CAN_COMMIT_COMPACT_BYTES_LIMIT, 50000000000 ); // 50GB, Commit waits.
@ -452,13 +465,13 @@ void ServerKnobs::initialize(Randomize randomize, ClientKnobs* clientKnobs, IsSi
init( ROCKSDB_DISABLE_WAL_EXPERIMENTAL, false );
// If ROCKSDB_SINGLEKEY_DELETES_ON_CLEARRANGE is enabled, disable ROCKSDB_ENABLE_CLEAR_RANGE_EAGER_READS knob.
// These knobs have contrary functionality.
init( ROCKSDB_SINGLEKEY_DELETES_ON_CLEARRANGE, false ); if( randomize && BUGGIFY ) ROCKSDB_SINGLEKEY_DELETES_ON_CLEARRANGE = deterministicRandom()->coinflip();
init( ROCKSDB_SINGLEKEY_DELETES_BYTES_LIMIT, 200000 ); // 200KB
init( ROCKSDB_ENABLE_CLEAR_RANGE_EAGER_READS, true ); if( randomize && BUGGIFY ) ROCKSDB_ENABLE_CLEAR_RANGE_EAGER_READS = deterministicRandom()->coinflip();
init( ROCKSDB_SINGLEKEY_DELETES_ON_CLEARRANGE, true );
init( ROCKSDB_SINGLEKEY_DELETES_BYTES_LIMIT, 10000 ); // 10KB
init( ROCKSDB_ENABLE_CLEAR_RANGE_EAGER_READS, false );
// ROCKSDB_STATS_LEVEL=1 indicates rocksdb::StatsLevel::kExceptHistogramOrTimers
// Refer StatsLevel: https://github.com/facebook/rocksdb/blob/main/include/rocksdb/statistics.h#L594
init( ROCKSDB_STATS_LEVEL, 1 ); if( randomize && BUGGIFY ) ROCKSDB_STATS_LEVEL = deterministicRandom()->randomInt(0, 6);
init( ROCKSDB_ENABLE_COMPACT_ON_DELETION, true );
init( ROCKSDB_ENABLE_COMPACT_ON_DELETION, false );
// CDCF: CompactOnDeletionCollectorFactory. The below 3 are parameters of the CompactOnDeletionCollectorFactory
// which controls the compaction on deleted data.
init( ROCKSDB_CDCF_SLIDING_WINDOW_SIZE, 128 );
@ -471,7 +484,7 @@ void ServerKnobs::initialize(Randomize randomize, ClientKnobs* clientKnobs, IsSi
init( ROCKSDB_CAN_COMMIT_DELAY_TIMES_ON_OVERLOAD, 5 );
init( ROCKSDB_COMPACTION_READAHEAD_SIZE, 32768 ); // 32 KB, performs bigger reads when doing compaction.
init( ROCKSDB_BLOCK_SIZE, 32768 ); // 32 KB, size of the block in rocksdb cache.
init( ENABLE_SHARDED_ROCKSDB, false );
init( ENABLE_SHARDED_ROCKSDB, false );
init( ROCKSDB_WRITE_BUFFER_SIZE, 1 << 30 ); // 1G
init( ROCKSDB_CF_WRITE_BUFFER_SIZE, 64 << 20 ); // 64M, RocksDB default.
init( ROCKSDB_MAX_TOTAL_WAL_SIZE, 0 ); // RocksDB default.
@ -670,7 +683,7 @@ void ServerKnobs::initialize(Randomize randomize, ClientKnobs* clientKnobs, IsSi
init( MOVE_SHARD_KRM_ROW_LIMIT, 20000 );
init( MOVE_SHARD_KRM_BYTE_LIMIT, 1e6 );
init( MAX_SKIP_TAGS, 1 ); //The TLogs require tags to be densely packed to be memory efficient, so be careful increasing this knob
init( MAX_ADDED_SOURCES_MULTIPLIER, 2.0 );
init( MAX_ADDED_SOURCES_MULTIPLIER, 0.0 ); if( randomize && BUGGIFY ) MAX_ADDED_SOURCES_MULTIPLIER = 2.0;
//FdbServer
bool longReboots = randomize && BUGGIFY;
@ -801,6 +814,8 @@ void ServerKnobs::initialize(Randomize randomize, ClientKnobs* clientKnobs, IsSi
init( EMPTY_READ_PENALTY, 20 ); // 20 bytes
init( DD_SHARD_COMPARE_LIMIT, 1000 );
init( READ_SAMPLING_ENABLED, true ); if ( randomize && BUGGIFY ) READ_SAMPLING_ENABLED = false;// enable/disable read sampling
init( DD_TRACE_MOVE_BYTES_AVERAGE_INTERVAL, 120);
init( MOVING_WINDOW_SAMPLE_SIZE, 10000000); // 10MB
//Storage Server
init( STORAGE_LOGGING_DELAY, 5.0 );
@ -839,7 +854,7 @@ void ServerKnobs::initialize(Randomize randomize, ClientKnobs* clientKnobs, IsSi
init( WAIT_METRICS_WRONG_SHARD_CHANCE, isSimulated ? 1.0 : 0.1 );
init( MIN_TAG_READ_PAGES_RATE, 1.0e4 ); if( randomize && BUGGIFY ) MIN_TAG_READ_PAGES_RATE = 0;
init( MIN_TAG_WRITE_PAGES_RATE, 3200 ); if( randomize && BUGGIFY ) MIN_TAG_WRITE_PAGES_RATE = 0;
init( TAG_MEASUREMENT_INTERVAL, 30.0 ); if( randomize && BUGGIFY ) TAG_MEASUREMENT_INTERVAL = 1.0;
init( TAG_MEASUREMENT_INTERVAL, 30.0 ); if( randomize && BUGGIFY ) TAG_MEASUREMENT_INTERVAL = 4.0;
init( PREFIX_COMPRESS_KVS_MEM_SNAPSHOTS, true ); if( randomize && BUGGIFY ) PREFIX_COMPRESS_KVS_MEM_SNAPSHOTS = false;
init( REPORT_DD_METRICS, true );
init( DD_METRICS_REPORT_INTERVAL, 30.0 );

View File

@ -29,6 +29,7 @@
FDB_DEFINE_BOOLEAN_PARAM(AssignEmptyRange);
FDB_DEFINE_BOOLEAN_PARAM(UnassignShard);
FDB_DEFINE_BOOLEAN_PARAM(EnablePhysicalShardMove);
const KeyRef systemKeysPrefix = "\xff"_sr;
const KeyRangeRef normalKeys(KeyRef(), systemKeysPrefix);
@ -47,8 +48,9 @@ const KeyRangeRef keyServersKeyServersKeys("\xff/keyServers/\xff/keyServers/"_sr
const KeyRef keyServersKeyServersKey = keyServersKeyServersKeys.begin;
// These constants are selected to be easily recognized during debugging.
// Note that the last bit of the follwing constants is 0, indicating that physical shard move is disabled.
const UID anonymousShardId = UID(0x666666, 0x88888888);
const uint64_t emptyShardId = 0x7777777;
const uint64_t emptyShardId = 0x2222222;
const Key keyServersKey(const KeyRef& k) {
return k.withPrefix(keyServersPrefix);
@ -457,7 +459,10 @@ const ValueRef serverKeysTrue = "1"_sr, // compatible with what was serverKeysTr
serverKeysTrueEmptyRange = "3"_sr, // the server treats the range as empty.
serverKeysFalse;
const UID newShardId(const uint64_t physicalShardId, AssignEmptyRange assignEmptyRange, UnassignShard unassignShard) {
const UID newDataMoveId(const uint64_t physicalShardId,
AssignEmptyRange assignEmptyRange,
EnablePhysicalShardMove enablePSM,
UnassignShard unassignShard) {
uint64_t split = 0;
if (assignEmptyRange) {
split = emptyShardId;
@ -466,6 +471,11 @@ const UID newShardId(const uint64_t physicalShardId, AssignEmptyRange assignEmpt
} else {
do {
split = deterministicRandom()->randomUInt64();
if (enablePSM) {
split |= 1U;
} else {
split &= ~1U;
}
} while (split == anonymousShardId.second() || split == 0 || split == emptyShardId);
}
return UID(physicalShardId, split);
@ -505,9 +515,10 @@ std::pair<UID, Key> serverKeysDecodeServerBegin(const KeyRef& key) {
}
bool serverHasKey(ValueRef storedValue) {
UID teamId;
UID shardId;
bool assigned, emptyRange;
decodeServerKeysValue(storedValue, assigned, emptyRange, teamId);
EnablePhysicalShardMove enablePSM = EnablePhysicalShardMove::False;
decodeServerKeysValue(storedValue, assigned, emptyRange, enablePSM, shardId);
return assigned;
}
@ -521,7 +532,12 @@ const Value serverKeysValue(const UID& id) {
return wr.toValue();
}
void decodeServerKeysValue(const ValueRef& value, bool& assigned, bool& emptyRange, UID& id) {
void decodeServerKeysValue(const ValueRef& value,
bool& assigned,
bool& emptyRange,
EnablePhysicalShardMove& enablePSM,
UID& id) {
enablePSM = EnablePhysicalShardMove::False;
if (value.size() == 0) {
assigned = false;
emptyRange = false;
@ -544,6 +560,9 @@ void decodeServerKeysValue(const ValueRef& value, bool& assigned, bool& emptyRan
rd >> id;
assigned = id.second() != 0;
emptyRange = id.second() == emptyShardId;
if (id.second() & 1U) {
enablePSM = EnablePhysicalShardMove::True;
}
}
}
@ -1424,22 +1443,30 @@ const Value blobGranuleFileValueFor(StringRef const& filename,
int64_t offset,
int64_t length,
int64_t fullFileLength,
int64_t logicalSize,
Optional<BlobGranuleCipherKeysMeta> cipherKeysMeta) {
BinaryWriter wr(IncludeVersion(ProtocolVersion::withBlobGranule()));
auto protocolVersion = CLIENT_KNOBS->ENABLE_BLOB_GRANULE_FILE_LOGICAL_SIZE
? ProtocolVersion::withBlobGranuleFileLogicalSize()
: ProtocolVersion::withBlobGranule();
BinaryWriter wr(IncludeVersion(protocolVersion));
wr << filename;
wr << offset;
wr << length;
wr << fullFileLength;
wr << cipherKeysMeta;
if (CLIENT_KNOBS->ENABLE_BLOB_GRANULE_FILE_LOGICAL_SIZE) {
wr << logicalSize;
}
return wr.toValue();
}
std::tuple<Standalone<StringRef>, int64_t, int64_t, int64_t, Optional<BlobGranuleCipherKeysMeta>>
std::tuple<Standalone<StringRef>, int64_t, int64_t, int64_t, int64_t, Optional<BlobGranuleCipherKeysMeta>>
decodeBlobGranuleFileValue(ValueRef const& value) {
StringRef filename;
int64_t offset;
int64_t length;
int64_t fullFileLength;
int64_t logicalSize;
Optional<BlobGranuleCipherKeysMeta> cipherKeysMeta;
BinaryReader reader(value, IncludeVersion());
@ -1448,7 +1475,13 @@ decodeBlobGranuleFileValue(ValueRef const& value) {
reader >> length;
reader >> fullFileLength;
reader >> cipherKeysMeta;
return std::tuple(filename, offset, length, fullFileLength, cipherKeysMeta);
if (reader.protocolVersion().hasBlobGranuleFileLogicalSize()) {
reader >> logicalSize;
} else {
// fall back to estimating logical size as physical size
logicalSize = length;
}
return std::tuple(filename, offset, length, fullFileLength, logicalSize, cipherKeysMeta);
}
const Value blobGranulePurgeValueFor(Version version, KeyRange range, bool force) {

View File

@ -77,17 +77,16 @@ bool tenantMapChanging(MutationRef const& mutation, KeyRangeRef const& tenantMap
return false;
}
// validates whether the lastTenantId and the nextTenantId share the same 2 byte prefix
bool nextTenantIdPrefixMatches(int64_t lastTenantId, int64_t nextTenantId) {
if (getTenantIdPrefix(nextTenantId) != getTenantIdPrefix(lastTenantId)) {
TraceEvent(g_network->isSimulated() ? SevWarnAlways : SevError, "TenantIdPrefixMismatch")
.detail("CurrentTenantId", lastTenantId)
.detail("NewTenantId", nextTenantId)
.detail("CurrentTenantIdPrefix", getTenantIdPrefix(lastTenantId))
.detail("NewTenantIdPrefix", getTenantIdPrefix(nextTenantId));
return false;
// validates whether the the ID created by adding delta to baseID is a valid ID in the same tenant prefix
int64_t computeNextTenantId(int64_t baseId, int64_t delta) {
if ((baseId & 0xFFFFFFFFFFFF) + delta > 0xFFFFFFFFFFFF) {
TraceEvent(g_network->isSimulated() ? SevWarnAlways : SevError, "NoMoreTenantIds")
.detail("LastTenantId", baseId)
.detail("TenantIdPrefix", getTenantIdPrefix(baseId));
throw cluster_no_capacity();
}
return true;
return baseId + delta;
}
// returns the maximum allowable tenant id in which the 2 byte prefix is not overriden

View File

@ -56,13 +56,13 @@ struct BlobMetadataDetailsRef {
explicit BlobMetadataDetailsRef(Arena& ar,
BlobMetadataDomainId domainId,
Optional<StringRef> base,
Optional<StringRef> baseLocation,
VectorRef<StringRef> partitions,
double refreshAt,
double expireAt)
: domainId(domainId), partitions(ar, partitions), refreshAt(refreshAt), expireAt(expireAt) {
if (base.present()) {
base = StringRef(ar, base.get());
if (baseLocation.present()) {
base = StringRef(ar, baseLocation.get());
}
}

View File

@ -278,6 +278,7 @@ public:
int BG_MAX_GRANULE_PARALLELISM;
int BG_TOO_MANY_GRANULES;
int64_t BLOB_METADATA_REFRESH_INTERVAL;
bool ENABLE_BLOB_GRANULE_FILE_LOGICAL_SIZE;
// The coordinator key/value in storage server might be inconsistent to the value stored in the cluster file.
// This might happen when a recovery is happening together with a cluster controller coordinator key change.

View File

@ -165,12 +165,15 @@ ACTOR Future<bool> getKeyServers(
Database cx,
Promise<std::vector<std::pair<KeyRange, std::vector<StorageServerInterface>>>> keyServersPromise,
KeyRangeRef kr,
bool performQuiescentChecks);
bool performQuiescentChecks,
bool failureIsError,
bool* success);
ACTOR Future<bool> getKeyLocations(Database cx,
std::vector<std::pair<KeyRange, std::vector<StorageServerInterface>>> shards,
Promise<Standalone<VectorRef<KeyValueRef>>> keyLocationPromise,
bool performQuiescentChecks);
ACTOR Future<bool> checkDataConsistency(Database cx,
bool performQuiescentChecks,
bool* success);
ACTOR Future<Void> checkDataConsistency(Database cx,
VectorRef<KeyValueRef> keyLocations,
DatabaseConfiguration configuration,
std::map<UID, StorageServerInterface> tssMapping,
@ -189,7 +192,8 @@ ACTOR Future<bool> checkDataConsistency(Database cx,
int restart,
int64_t maxRate,
int64_t targetInterval,
KeyRef progressKey);
KeyRef progressKey,
bool* success);
#include "flow/unactorcompiler.h"

View File

@ -88,6 +88,11 @@ public:
size_t getNumberOfCoordinators() const { return coords.size() + hostnames.size(); }
// Determine the local source IP used to connect to the cluster by connecting to the first available coordinator.
// Throw bind_failed() if no connection attempts were successful.
// This function blocks on connection attempts.
IPAddress determineLocalSourceIP() const;
bool operator==(const ClusterConnectionString& other) const noexcept {
return key == other.key && keyDesc == other.keyDesc && coords == other.coords && hostnames == other.hostnames;
}

View File

@ -342,7 +342,11 @@ struct KeyRangeRef {
return false; // uncovered gap between clone.begin and r.begin
if (clone.end <= r.end)
return true; // range is fully covered
if (clone.end > r.begin)
// If a range of ranges is totally at the left of clone,
// clone needs not update
// If a range of ranges is partially at the left of clone,
// clone = clone - the overlap
if (clone.end > r.end && r.end > clone.begin)
// {clone.begin, r.end} is covered. need to check coverage for {r.end, clone.end}
clone = KeyRangeRef(r.end, clone.end);
}

View File

@ -14,8 +14,10 @@ typedef JsonBuilder JsonString;
template <typename T>
class JsonBuilderObjectSetter;
// Class for building JSON string values.
// Default value is null, as in the JSON type
// Class for building JSON strings linearly.
// JSON data structure is only appendable. No key deduplication is done in JSON Objects, and the output is not readable
// other than obtaining a complete JSON string of what has been written to the builder. Default value is null, as in the
// JSON type
class JsonBuilder {
protected:
enum EType { NULLVALUE, OBJECT, ARRAY };

View File

@ -101,6 +101,7 @@ FDB_DECLARE_BOOLEAN_PARAM(RunOnDisconnectedCluster);
FDB_DECLARE_BOOLEAN_PARAM(RunOnMismatchedCluster);
FDB_DECLARE_BOOLEAN_PARAM(RestoreDryRun);
FDB_DECLARE_BOOLEAN_PARAM(ForceJoin);
FDB_DECLARE_BOOLEAN_PARAM(ForceReuseTenantIdPrefix);
FDB_DECLARE_BOOLEAN_PARAM(ForceRemove);
FDB_DECLARE_BOOLEAN_PARAM(IgnoreCapacityLimit);
@ -550,6 +551,10 @@ Future<Optional<std::string>> createMetacluster(Reference<DB> db,
ASSERT(tenantIdPrefix >= TenantAPI::TENANT_ID_PREFIX_MIN_VALUE &&
tenantIdPrefix <= TenantAPI::TENANT_ID_PREFIX_MAX_VALUE);
if (name.startsWith("\xff"_sr)) {
throw invalid_cluster_name();
}
loop {
try {
tr->setOption(FDBTransactionOptions::ACCESS_SYSTEM_KEYS);
@ -677,11 +682,11 @@ void updateClusterMetadata(Transaction tr,
} else if (previousMetadata.entry.clusterState == DataClusterState::REMOVING) {
throw cluster_removed();
} else if (!isRestoring && previousMetadata.entry.clusterState == DataClusterState::RESTORING &&
(!updatedEntry.present() || (updatedEntry.get().clusterState != DataClusterState::READY &&
updatedEntry.get().clusterState != DataClusterState::REMOVING))) {
(updatedEntry.get().clusterState != DataClusterState::READY &&
updatedEntry.get().clusterState != DataClusterState::REMOVING)) {
throw cluster_restoring();
} else if (isRestoring) {
ASSERT(previousMetadata.entry.clusterState == DataClusterState::RESTORING ||
ASSERT(previousMetadata.entry.clusterState == DataClusterState::RESTORING &&
updatedEntry.get().clusterState == DataClusterState::RESTORING);
}
ManagementClusterMetadata::dataClusters().set(tr, name, updatedEntry.get());
@ -786,13 +791,15 @@ struct RemoveClusterImpl {
// cluster.
if (self->lastTenantId.present()) {
Optional<int64_t> lastId = wait(TenantMetadata::lastTenantId().get(tr));
if (!lastId.present() || lastId.get() < self->lastTenantId.get()) {
if (!lastId.present() || (TenantAPI::getTenantIdPrefix(lastId.get()) ==
TenantAPI::getTenantIdPrefix(self->lastTenantId.get()) &&
lastId.get() < self->lastTenantId.get())) {
TenantMetadata::lastTenantId().set(tr, self->lastTenantId.get());
}
}
}
// Insert a tombstone marking this tenant removed even if we aren't registered
// Insert a tombstone marking this cluster removed even if we aren't registered
MetaclusterMetadata::registrationTombstones().insert(tr, clusterId);
TraceEvent("RemovedMetaclusterRegistrationOnDataCluster")
@ -1151,6 +1158,10 @@ struct RegisterClusterImpl {
self->ctx.metaclusterRegistration.get().toDataClusterRegistration(self->clusterName,
self->clusterEntry.id));
// The data cluster will track the last ID it allocated in this metacluster, so erase any prior tenant
// ID state
TenantMetadata::lastTenantId().clear(tr);
// If we happen to have any orphaned restore IDs from a previous time this cluster was in a metacluster,
// erase them now.
MetaclusterMetadata::activeRestoreIds().clear(tr);
@ -1209,6 +1220,10 @@ struct RegisterClusterImpl {
state RemoveClusterImpl<DB> removeCluster(
self->ctx.managementDb, self->clusterName, ClusterType::METACLUSTER_MANAGEMENT, ForceRemove::True, 5.0);
if (self->clusterName.startsWith("\xff"_sr)) {
throw invalid_cluster_name();
}
wait(self->ctx.runManagementTransaction(
[self = self](Reference<typename DB::TransactionT> tr) { return registerInManagementCluster(self, tr); }));
@ -1392,13 +1407,19 @@ struct RestoreClusterImpl {
ApplyManagementClusterUpdates applyManagementClusterUpdates;
RestoreDryRun restoreDryRun;
ForceJoin forceJoin;
ForceReuseTenantIdPrefix forceReuseTenantIdPrefix;
std::vector<std::string>& messages;
// Unique ID generated for this restore. Used to avoid concurrent restores
UID restoreId = deterministicRandom()->randomUniqueID();
// Loaded from the management cluster
Optional<int64_t> lastManagementClusterTenantId;
// Loaded from the data cluster
UID dataClusterId;
Optional<int64_t> lastDataClusterTenantId;
Optional<int64_t> newLastDataClusterTenantId;
// Tenant list from data and management clusters
std::unordered_map<int64_t, TenantMapEntry> dataClusterTenantMap;
@ -1412,10 +1433,12 @@ struct RestoreClusterImpl {
ApplyManagementClusterUpdates applyManagementClusterUpdates,
RestoreDryRun restoreDryRun,
ForceJoin forceJoin,
ForceReuseTenantIdPrefix forceReuseTenantIdPrefix,
std::vector<std::string>& messages)
: ctx(managementDb, {}, { DataClusterState::RESTORING }), clusterName(clusterName),
connectionString(connectionString), applyManagementClusterUpdates(applyManagementClusterUpdates),
restoreDryRun(restoreDryRun), forceJoin(forceJoin), messages(messages) {}
restoreDryRun(restoreDryRun), forceJoin(forceJoin), forceReuseTenantIdPrefix(forceReuseTenantIdPrefix),
messages(messages) {}
ACTOR template <class Transaction>
static Future<Void> checkRestoreId(RestoreClusterImpl* self, Transaction tr) {
@ -1429,11 +1452,12 @@ struct RestoreClusterImpl {
return Void();
}
// Returns true if the restore ID was erased
ACTOR template <class Transaction>
static Future<Void> eraseRestoreId(RestoreClusterImpl* self, Transaction tr) {
static Future<bool> eraseRestoreId(RestoreClusterImpl* self, Transaction tr) {
Optional<UID> transactionId = wait(MetaclusterMetadata::activeRestoreIds().get(tr, self->clusterName));
if (!transactionId.present()) {
return Void();
return false;
} else if (transactionId.get() != self->restoreId) {
throw conflicting_restore();
} else {
@ -1441,7 +1465,7 @@ struct RestoreClusterImpl {
MetaclusterMetadata::activeRestoreIds().erase(tr, self->clusterName);
}
return Void();
return true;
}
template <class Function>
@ -1561,6 +1585,9 @@ struct RestoreClusterImpl {
state Future<bool> tombstoneFuture =
MetaclusterMetadata::registrationTombstones().exists(tr, self->dataClusterId);
state Future<Void> lastTenantIdFuture =
store(self->lastDataClusterTenantId, TenantMetadata::lastTenantId().get(tr));
state Optional<MetaclusterRegistrationEntry> metaclusterRegistration =
wait(MetaclusterMetadata::metaclusterRegistration().get(tr));
@ -1570,6 +1597,8 @@ struct RestoreClusterImpl {
throw cluster_removed();
}
wait(lastTenantIdFuture);
MetaclusterRegistrationEntry dataClusterEntry =
self->ctx.metaclusterRegistration.get().toDataClusterRegistration(self->clusterName,
self->dataClusterId);
@ -1601,23 +1630,28 @@ struct RestoreClusterImpl {
return Void();
}
void markClusterRestoring(Reference<typename DB::TransactionT> tr) {
MetaclusterMetadata::activeRestoreIds().addReadConflictKey(tr, clusterName);
MetaclusterMetadata::activeRestoreIds().set(tr, clusterName, restoreId);
if (ctx.dataClusterMetadata.get().entry.clusterState != DataClusterState::RESTORING) {
DataClusterEntry updatedEntry = ctx.dataClusterMetadata.get().entry;
ACTOR static Future<Void> markClusterRestoring(RestoreClusterImpl* self, Reference<typename DB::TransactionT> tr) {
MetaclusterMetadata::activeRestoreIds().addReadConflictKey(tr, self->clusterName);
MetaclusterMetadata::activeRestoreIds().set(tr, self->clusterName, self->restoreId);
if (self->ctx.dataClusterMetadata.get().entry.clusterState != DataClusterState::RESTORING) {
DataClusterEntry updatedEntry = self->ctx.dataClusterMetadata.get().entry;
updatedEntry.clusterState = DataClusterState::RESTORING;
updateClusterMetadata(tr, clusterName, ctx.dataClusterMetadata.get(), connectionString, updatedEntry);
updateClusterMetadata(
tr, self->clusterName, self->ctx.dataClusterMetadata.get(), self->connectionString, updatedEntry);
// Remove this cluster from the cluster capacity index, but leave its configured capacity intact in the
// cluster entry. This allows us to retain the configured capacity while preventing the cluster from
// being used to allocate new tenant groups.
DataClusterEntry noCapacityEntry = updatedEntry;
noCapacityEntry.capacity.numTenantGroups = 0;
updateClusterCapacityIndex(tr, clusterName, updatedEntry, noCapacityEntry);
updateClusterCapacityIndex(tr, self->clusterName, updatedEntry, noCapacityEntry);
}
TraceEvent("MarkedDataClusterRestoring").detail("Name", clusterName);
wait(store(self->lastManagementClusterTenantId,
ManagementClusterMetadata::tenantMetadata().lastTenantId.get(tr)));
TraceEvent("MarkedDataClusterRestoring").detail("Name", self->clusterName);
return Void();
}
Future<Void> markClusterAsReady(Reference<typename DB::TransactionT> tr) {
@ -1632,7 +1666,7 @@ struct RestoreClusterImpl {
noCapacityEntry.capacity.numTenantGroups = 0;
updateClusterCapacityIndex(tr, clusterName, noCapacityEntry, updatedEntry);
return eraseRestoreId(this, tr);
return success(eraseRestoreId(this, tr));
}
return Void();
@ -2011,6 +2045,13 @@ struct RestoreClusterImpl {
ManagementClusterMetadata::tenantMetadata().tenantGroupMap.get(tr, tenantEntry.tenantGroup.get());
}
if (self->lastDataClusterTenantId.present() &&
TenantAPI::getTenantIdPrefix(tenantEntry.id) ==
TenantAPI::getTenantIdPrefix(self->lastDataClusterTenantId.get()) &&
!self->restoreDryRun) {
ASSERT_LE(tenantEntry.id, self->lastDataClusterTenantId.get());
}
Optional<MetaclusterTenantMapEntry> existingEntry = wait(tryGetTenantTransaction(tr, tenantEntry.tenantName));
if (existingEntry.present()) {
if (existingEntry.get().assignedCluster == self->clusterName) {
@ -2075,15 +2116,14 @@ struct RestoreClusterImpl {
ACTOR static Future<Void> addTenantBatchToManagementCluster(RestoreClusterImpl* self,
Reference<typename DB::TransactionT> tr,
std::vector<TenantMapEntry> tenants) {
Optional<int64_t> tenantIdPrefix = wait(TenantMetadata::tenantIdPrefix().get(tr));
ASSERT(tenantIdPrefix.present());
std::vector<TenantMapEntry> tenants,
int64_t tenantIdPrefix) {
state std::vector<Future<bool>> futures;
state int64_t maxId = tenantIdPrefix.get() << 48;
state int64_t maxId = -1;
for (auto const& t : tenants) {
if (TenantAPI::getTenantIdPrefix(t.id) == tenantIdPrefix.get()) {
if (TenantAPI::getTenantIdPrefix(t.id) == tenantIdPrefix) {
maxId = std::max(maxId, t.id);
self->newLastDataClusterTenantId = std::max(t.id, self->newLastDataClusterTenantId.orDefault(0));
}
futures.push_back(addTenantToManagementCluster(self, tr, t));
}
@ -2106,33 +2146,68 @@ struct RestoreClusterImpl {
if (!self->restoreDryRun) {
if (numGroupsCreated > 0) {
state DataClusterMetadata clusterMetadata = wait(getClusterTransaction(tr, self->clusterName));
DataClusterEntry updatedEntry = clusterMetadata.entry;
DataClusterEntry updatedEntry = self->ctx.dataClusterMetadata.get().entry;
if (updatedEntry.clusterState != DataClusterState::RESTORING) {
throw conflicting_restore();
}
updatedEntry.allocated.numTenantGroups += numGroupsCreated;
updateClusterMetadata(tr,
self->clusterName,
clusterMetadata,
self->ctx.dataClusterMetadata.get(),
Optional<ClusterConnectionString>(),
updatedEntry,
IsRestoring::True);
}
int64_t lastTenantId =
wait(ManagementClusterMetadata::tenantMetadata().lastTenantId.getD(tr, Snapshot::False, 0));
wait(ManagementClusterMetadata::tenantMetadata().lastTenantId.getD(tr, Snapshot::False, -1));
if (maxId > lastTenantId) {
ManagementClusterMetadata::tenantMetadata().lastTenantId.set(tr, maxId);
}
ManagementClusterMetadata::tenantMetadata().lastTenantId.set(tr, std::max(lastTenantId, maxId));
ManagementClusterMetadata::tenantMetadata().lastTenantModification.setVersionstamp(tr, Versionstamp(), 0);
}
return Void();
}
ACTOR static Future<int64_t> updateLastTenantId(RestoreClusterImpl* self, Reference<typename DB::TransactionT> tr) {
state Optional<int64_t> lastTenantId = wait(ManagementClusterMetadata::tenantMetadata().lastTenantId.get(tr));
state int64_t tenantIdPrefix;
if (!lastTenantId.present()) {
Optional<int64_t> prefix = wait(TenantMetadata::tenantIdPrefix().get(tr));
ASSERT(prefix.present());
tenantIdPrefix = prefix.get();
} else {
tenantIdPrefix = TenantAPI::getTenantIdPrefix(lastTenantId.get());
}
if (self->lastDataClusterTenantId.present() &&
tenantIdPrefix == TenantAPI::getTenantIdPrefix(self->lastDataClusterTenantId.get())) {
if (!self->forceReuseTenantIdPrefix) {
self->messages.push_back(fmt::format(
"The data cluster being added is using the same tenant ID prefix {} as the management cluster.",
tenantIdPrefix));
throw invalid_metacluster_configuration();
} else if (!self->restoreDryRun && self->lastDataClusterTenantId.get() > lastTenantId.orDefault(-1)) {
ManagementClusterMetadata::tenantMetadata().lastTenantId.set(tr, self->lastDataClusterTenantId.get());
}
self->newLastDataClusterTenantId = self->lastDataClusterTenantId;
}
return tenantIdPrefix;
}
ACTOR static Future<Void> addTenantsToManagementCluster(RestoreClusterImpl* self) {
state std::unordered_map<int64_t, TenantMapEntry>::iterator itr;
state std::vector<TenantMapEntry> tenantBatch;
state int64_t tenantsToAdd = 0;
state int64_t tenantIdPrefix = wait(self->runRestoreManagementTransaction(
[self = self](Reference<typename DB::TransactionT> tr) { return updateLastTenantId(self, tr); }));
for (itr = self->dataClusterTenantMap.begin(); itr != self->dataClusterTenantMap.end(); ++itr) {
state std::unordered_map<int64_t, MetaclusterTenantMapEntry>::iterator managementEntry =
self->mgmtClusterTenantMap.find(itr->second.id);
@ -2153,9 +2228,10 @@ struct RestoreClusterImpl {
if (tenantBatch.size() == CLIENT_KNOBS->METACLUSTER_RESTORE_BATCH_SIZE) {
wait(self->runRestoreManagementTransaction(
[self = self, tenantBatch = tenantBatch](Reference<typename DB::TransactionT> tr) {
[self = self, tenantBatch = tenantBatch, tenantIdPrefix = tenantIdPrefix](
Reference<typename DB::TransactionT> tr) {
tr->setOption(FDBTransactionOptions::READ_SYSTEM_KEYS);
return addTenantBatchToManagementCluster(self, tr, tenantBatch);
return addTenantBatchToManagementCluster(self, tr, tenantBatch, tenantIdPrefix);
}));
tenantBatch.clear();
}
@ -2163,9 +2239,10 @@ struct RestoreClusterImpl {
if (!tenantBatch.empty()) {
wait(self->runRestoreManagementTransaction(
[self = self, tenantBatch = tenantBatch](Reference<typename DB::TransactionT> tr) {
[self = self, tenantBatch = tenantBatch, tenantIdPrefix = tenantIdPrefix](
Reference<typename DB::TransactionT> tr) {
tr->setOption(FDBTransactionOptions::READ_SYSTEM_KEYS);
return addTenantBatchToManagementCluster(self, tr, tenantBatch);
return addTenantBatchToManagementCluster(self, tr, tenantBatch, tenantIdPrefix);
}));
}
@ -2179,6 +2256,19 @@ struct RestoreClusterImpl {
return Void();
}
ACTOR static Future<Void> finalizeDataClusterAfterRepopulate(RestoreClusterImpl* self, Reference<ITransaction> tr) {
bool erased = wait(eraseRestoreId(self, tr));
if (erased) {
if (self->newLastDataClusterTenantId.present()) {
TenantMetadata::lastTenantId().set(tr, self->newLastDataClusterTenantId.get());
} else {
TenantMetadata::lastTenantId().clear(tr);
}
}
return Void();
}
ACTOR static Future<Void> runDataClusterRestore(RestoreClusterImpl* self) {
// Run a management transaction to populate the data cluster metadata
wait(self->ctx.runManagementTransaction([self = self](Reference<typename DB::TransactionT> tr) {
@ -2192,10 +2282,8 @@ struct RestoreClusterImpl {
// set state to restoring
if (!self->restoreDryRun) {
try {
wait(self->ctx.runManagementTransaction([self = self](Reference<typename DB::TransactionT> tr) {
self->markClusterRestoring(tr);
return Future<Void>(Void());
}));
wait(self->ctx.runManagementTransaction(
[self = self](Reference<typename DB::TransactionT> tr) { return markClusterRestoring(self, tr); }));
} catch (Error& e) {
// If the transaction retries after success or if we are trying a second time to restore the cluster, it
// will throw an error indicating that the restore has already started
@ -2205,11 +2293,16 @@ struct RestoreClusterImpl {
}
}
// Set the restore ID in the data cluster
// Set the restore ID in the data cluster and update the last tenant ID to match the management cluster
if (!self->restoreDryRun) {
wait(self->ctx.runDataClusterTransaction([self = self](Reference<ITransaction> tr) {
MetaclusterMetadata::activeRestoreIds().addReadConflictKey(tr, self->clusterName);
MetaclusterMetadata::activeRestoreIds().set(tr, self->clusterName, self->restoreId);
if (self->lastManagementClusterTenantId.present()) {
TenantMetadata::lastTenantId().set(tr, self->lastManagementClusterTenantId.get());
} else {
TenantMetadata::lastTenantId().clear(tr);
}
return Future<Void>(Void());
}));
}
@ -2232,7 +2325,7 @@ struct RestoreClusterImpl {
if (!self->restoreDryRun) {
// Remove the active restore ID from the data cluster
wait(self->ctx.runDataClusterTransaction(
[self = self](Reference<ITransaction> tr) { return eraseRestoreId(self, tr); }));
[self = self](Reference<ITransaction> tr) { return success(eraseRestoreId(self, tr)); }));
// set restored cluster to ready state
wait(self->ctx.runManagementTransaction(
@ -2276,7 +2369,7 @@ struct RestoreClusterImpl {
if (!self->restoreDryRun) {
// Remove the active restore ID from the data cluster
wait(self->ctx.runDataClusterTransaction(
[self = self](Reference<ITransaction> tr) { return eraseRestoreId(self, tr); }));
[self = self](Reference<ITransaction> tr) { return finalizeDataClusterAfterRepopulate(self, tr); }));
// set restored cluster to ready state
wait(self->ctx.runManagementTransaction(
@ -2303,9 +2396,16 @@ Future<Void> restoreCluster(Reference<DB> db,
ApplyManagementClusterUpdates applyManagementClusterUpdates,
RestoreDryRun restoreDryRun,
ForceJoin forceJoin,
ForceReuseTenantIdPrefix forceReuseTenantIdPrefix,
std::vector<std::string>* messages) {
state RestoreClusterImpl<DB> impl(
db, name, connectionString, applyManagementClusterUpdates, restoreDryRun, forceJoin, *messages);
state RestoreClusterImpl<DB> impl(db,
name,
connectionString,
applyManagementClusterUpdates,
restoreDryRun,
forceJoin,
forceReuseTenantIdPrefix,
*messages);
wait(impl.run());
return Void();
}
@ -2503,10 +2603,7 @@ struct CreateTenantImpl {
ASSERT(tenantIdPrefix.present());
lastId = tenantIdPrefix.get() << 48;
}
if (!TenantAPI::nextTenantIdPrefixMatches(lastId.get(), lastId.get() + 1)) {
throw cluster_no_capacity();
}
self->tenantEntry.setId(lastId.get() + 1);
self->tenantEntry.setId(TenantAPI::computeNextTenantId(lastId.get(), 1));
ManagementClusterMetadata::tenantMetadata().lastTenantId.set(tr, self->tenantEntry.id);
self->tenantEntry.tenantState = MetaclusterAPI::TenantState::REGISTERING;
@ -2550,9 +2647,15 @@ struct CreateTenantImpl {
}
ACTOR static Future<Void> storeTenantInDataCluster(CreateTenantImpl* self, Reference<ITransaction> tr) {
std::pair<Optional<TenantMapEntry>, bool> dataClusterTenant = wait(TenantAPI::createTenantTransaction(
state Future<int64_t> lastTenantIdFuture = TenantMetadata::lastTenantId().getD(tr, Snapshot::False, -1);
state std::pair<Optional<TenantMapEntry>, bool> dataClusterTenant = wait(TenantAPI::createTenantTransaction(
tr, self->tenantEntry.toTenantMapEntry(), ClusterType::METACLUSTER_DATA));
int64_t lastTenantId = wait(lastTenantIdFuture);
if (lastTenantId < self->tenantEntry.id) {
TenantMetadata::lastTenantId().set(tr, self->tenantEntry.id);
}
// If the tenant map entry is empty, then we encountered a tombstone indicating that the tenant was
// simultaneously removed.
if (!dataClusterTenant.first.present()) {
@ -2833,7 +2936,12 @@ Future<std::vector<std::pair<TenantName, MetaclusterTenantMapEntry>>> listTenant
results.reserve(futures.size());
for (int i = 0; i < futures.size(); ++i) {
const MetaclusterTenantMapEntry& entry = futures[i].get().get();
results.emplace_back(entry.tenantName, entry);
// Tenants being renamed show up in tenantIds twice, once under each name. The destination name will be
// different from the tenant entry and is filtered from the list
if (entry.tenantName == tenantIds[i].first) {
results.emplace_back(entry.tenantName, entry);
}
}
return results;

View File

@ -604,14 +604,15 @@ Future<Void> createCheckpoint(Reference<ReadYourWritesTransaction> tr,
Optional<UID> dataMoveId = Optional<UID>());
// Gets checkpoint metadata for `ranges` at the specific version, with the particular format.
// The keyranges of the returned checkpoint is a super-set of `ranges`.
// Returns a list of [range, checkpoint], where the `checkpoint` has data over `range`.
// checkpoint_not_found() error will be returned if the specific checkpoint cannot be found.
ACTOR Future<std::vector<CheckpointMetaData>> getCheckpointMetaData(Database cx,
std::vector<KeyRange> ranges,
Version version,
CheckpointFormat format,
Optional<UID> dataMoveId = Optional<UID>(),
double timeout = 5.0);
ACTOR Future<std::vector<std::pair<KeyRange, CheckpointMetaData>>> getCheckpointMetaData(
Database cx,
std::vector<KeyRange> ranges,
Version version,
CheckpointFormat format,
Optional<UID> dataMoveId = Optional<UID>(),
double timeout = 5.0);
// Checks with Data Distributor that it is safe to mark all servers in exclusions as failed
ACTOR Future<bool> checkSafeExclusions(Database cx, std::vector<AddressExclusion> exclusions);

View File

@ -27,6 +27,7 @@
#include "flow/FastRef.h"
#include "flow/Net2Packet.h"
#include <boost/functional/hash.hpp>
#include <fmt/format.h>
#include <unordered_map>
#include <utility>
@ -52,7 +53,8 @@ public:
// Maximum number of connections cached in the connection-pool.
int maxConnPerConnectKey;
std::map<RESTConnectionPoolKey, std::queue<ReusableConnection>> connectionPoolMap;
std::unordered_map<RESTConnectionPoolKey, std::queue<ReusableConnection>, boost::hash<RESTConnectionPoolKey>>
connectionPoolMap;
RESTConnectionPool(const int maxConnsPerKey) : maxConnPerConnectKey(maxConnsPerKey) {}

View File

@ -81,6 +81,7 @@ public:
double DESIRED_GET_MORE_DELAY;
int CONCURRENT_LOG_ROUTER_READS;
int LOG_ROUTER_PEEK_FROM_SATELLITES_PREFERRED; // 0==peek from primary, non-zero==peek from satellites
double LOG_ROUTER_PEEK_SWITCH_DC_TIME;
double DISK_QUEUE_ADAPTER_MIN_SWITCH_TIME;
double DISK_QUEUE_ADAPTER_MAX_SWITCH_TIME;
int64_t TLOG_SPILL_REFERENCE_MAX_PEEK_MEMORY_BYTES;
@ -141,31 +142,59 @@ public:
// is possible within but not between priority groups; fewer priority groups
// mean better worst case time bounds
// Maximum allowable priority is 999.
// Update the status json .data.team_tracker.state field when necessary
//
// Priority for movement resume from previous unfinished in-flight movement when a new DD
// start
int PRIORITY_RECOVER_MOVE;
// A load-balance priority for disk valley filler
int PRIORITY_REBALANCE_UNDERUTILIZED_TEAM;
// A load-balance priority disk mountain chopper
int PRIORITY_REBALANCE_OVERUTILIZED_TEAM;
// A load-balance priority read valley filler
int PRIORITY_REBALANCE_READ_OVERUTIL_TEAM;
// A load-balance priority read mountain chopper
int PRIORITY_REBALANCE_READ_UNDERUTIL_TEAM;
// A team healthy priority for wiggle a storage server
int PRIORITY_PERPETUAL_STORAGE_WIGGLE;
// A team healthy priority when all servers in a team are healthy. When a team changes from any unhealthy states to
// healthy, the unfinished relocations will be overriden to healthy priority
int PRIORITY_TEAM_HEALTHY;
// A team healthy priority when there's undesired servers in the team. (ex. same ip
// address as other SS process, or SS is lagging too far ...)
int PRIORITY_TEAM_CONTAINS_UNDESIRED_SERVER;
// A team healthy priority for removing redundant team to make the team count within a good range
int PRIORITY_TEAM_REDUNDANT;
// A shard boundary priority for merge small and write cold shard.
int PRIORITY_MERGE_SHARD;
// A team healthy priority for populate remote region
int PRIORITY_POPULATE_REGION;
// A team healthy priority when the replica > 3 and there's at least one unhealthy server in a team.
// Or when the team contains a server with wrong configuration (ex. storage engine,
// locality, excluded ...)
int PRIORITY_TEAM_UNHEALTHY;
// A team healthy priority when there should be >= 3 replicas and there's 2 healthy servers in a team
int PRIORITY_TEAM_2_LEFT;
// A team healthy priority when there should be >= 2 replicas and there's 1 healthy server in a team
int PRIORITY_TEAM_1_LEFT;
int PRIORITY_TEAM_FAILED; // Priority when a server in the team is excluded as failed
// A team healthy priority when a server in the team is excluded as failed
int PRIORITY_TEAM_FAILED;
// A team healthy priority when there's no healthy server in a team
int PRIORITY_TEAM_0_LEFT;
// A shard boundary priority for split large or write hot shard.
int PRIORITY_SPLIT_SHARD;
int PRIORITY_ENFORCE_MOVE_OUT_OF_PHYSICAL_SHARD; // Priority when a physical shard is oversize or anonymous
// Priority when a physical shard is oversize or anonymous. When DD enable physical shard, the shard created before
// it are default to be 'anonymous' for compatibility.
int PRIORITY_ENFORCE_MOVE_OUT_OF_PHYSICAL_SHARD;
// Data distribution
bool SHARD_ENCODE_LOCATION_METADATA; // If true, location metadata will contain shard ID.
bool ENABLE_DD_PHYSICAL_SHARD; // EXPERIMENTAL; If true, SHARD_ENCODE_LOCATION_METADATA must be true.
bool ENABLE_DD_PHYSICAL_SHARD_MOVE; // Enable physical shard move.
int64_t MAX_PHYSICAL_SHARD_BYTES;
double PHYSICAL_SHARD_METRICS_DELAY;
double ANONYMOUS_PHYSICAL_SHARD_TRANSITION_TIME;
bool PHYSICAL_SHARD_MOVE_VERBOSE_TRACKING;
double READ_REBALANCE_CPU_THRESHOLD; // read rebalance only happens if the source servers' CPU > threshold
int READ_REBALANCE_SRC_PARALLELISM; // the max count a server become a source server within a certain interval
@ -276,6 +305,13 @@ public:
double DD_FAILURE_TIME;
double DD_ZERO_HEALTHY_TEAM_DELAY;
int DD_BUILD_EXTRA_TEAMS_OVERRIDE; // build extra teams to allow data movement to progress. must be larger than 0
int DD_MAXIMUM_LARGE_TEAMS; // the maximum number of large teams data distribution will maintain
int DD_MAXIMUM_LARGE_TEAM_CLEANUP; // the maximum number of large teams data distribution will attempt to cleanup
// without yielding
double DD_LARGE_TEAM_DELAY; // the amount of time data distribution will wait before returning less replicas than
// requested
double DD_FIX_WRONG_REPLICAS_DELAY; // the amount of time between attempts to increase the replication factor of
// under replicated shards
// Run storage enginee on a child process on the same machine with storage process
bool REMOTE_KV_STORE;
@ -340,6 +376,10 @@ public:
bool ROCKSDB_MUTE_LOGS;
int64_t ROCKSDB_PERIODIC_COMPACTION_SECONDS;
int ROCKSDB_PREFIX_LEN;
double ROCKSDB_MEMTABLE_PREFIX_BLOOM_SIZE_RATIO;
int ROCKSDB_BLOOM_BITS_PER_KEY;
bool ROCKSDB_BLOOM_WHOLE_KEY_FILTERING;
int ROCKSDB_MAX_AUTO_READAHEAD_SIZE;
int64_t ROCKSDB_BLOCK_CACHE_SIZE;
double ROCKSDB_METRICS_DELAY;
double ROCKSDB_READ_VALUE_TIMEOUT;
@ -748,6 +788,9 @@ public:
int64_t EMPTY_READ_PENALTY;
int DD_SHARD_COMPARE_LIMIT; // when read-aware DD is enabled, at most how many shards are compared together
bool READ_SAMPLING_ENABLED;
// Rolling window duration over which the average bytes moved by DD is calculated for the 'MovingData' trace event.
double DD_TRACE_MOVE_BYTES_AVERAGE_INTERVAL;
int64_t MOVING_WINDOW_SAMPLE_SIZE;
// Storage Server
double STORAGE_LOGGING_DELAY;

View File

@ -25,6 +25,7 @@
#include "fdbclient/FDBTypes.h"
const std::string checkpointBytesSampleFileName = "metadata_bytes.sst";
const std::string emptySstFilePath = "Dummy Empty SST File Path";
// FDB storage checkpoint format.
enum CheckpointFormat {
@ -61,6 +62,8 @@ struct CheckpointMetaData {
Optional<UID> actionId; // Unique ID defined by the application.
std::string dir;
CheckpointMetaData() = default;
CheckpointMetaData(const std::vector<KeyRange>& ranges,
CheckpointFormat format,
@ -103,13 +106,23 @@ struct CheckpointMetaData {
return true;
}
bool containsKey(const KeyRef key) const {
for (const auto& range : ranges) {
if (range.contains(key)) {
return true;
}
}
return false;
}
bool operator==(const CheckpointMetaData& r) const { return checkpointID == r.checkpointID; }
std::string toString() const {
std::string res = "Checkpoint MetaData: [Ranges]: " + describe(ranges) +
" [Version]: " + std::to_string(version) + " [Format]: " + std::to_string(format) +
" [Server]: " + describe(src) + " [ID]: " + checkpointID.toString() +
" [State]: " + std::to_string(static_cast<int>(state)) +
" [Checkpoint Dir:] " + dir + " [Server]: " + describe(src) +
" [ID]: " + checkpointID.toString() + " [State]: " + std::to_string(static_cast<int>(state)) +
(actionId.present() ? (" [Action ID]: " + actionId.get().toString()) : "") +
(bytesSampleFile.present() ? " [bytesSampleFile]: " + bytesSampleFile.get() : "");
;
@ -118,8 +131,17 @@ struct CheckpointMetaData {
template <class Ar>
void serialize(Ar& ar) {
serializer(
ar, version, ranges, format, state, checkpointID, src, serializedCheckpoint, actionId, bytesSampleFile);
serializer(ar,
version,
ranges,
format,
state,
checkpointID,
src,
serializedCheckpoint,
actionId,
bytesSampleFile,
dir);
}
};
@ -167,7 +189,8 @@ struct DataMoveMetaData {
std::string toString() const {
std::string res = "DataMoveMetaData: [ID]: " + id.shortString() + ", [Range]: " + describe(ranges) +
", [Phase]: " + std::to_string(static_cast<int>(phase)) +
", [Source Servers]: " + describe(src) + ", [Destination Servers]: " + describe(dest);
", [Source Servers]: " + describe(src) + ", [Destination Servers]: " + describe(dest) +
", [Checkpoints]: " + describe(checkpoints);
return res;
}

View File

@ -36,6 +36,7 @@
FDB_DECLARE_BOOLEAN_PARAM(AssignEmptyRange);
FDB_DECLARE_BOOLEAN_PARAM(UnassignShard);
FDB_DECLARE_BOOLEAN_PARAM(EnablePhysicalShardMove);
struct RestoreLoaderInterface;
struct RestoreApplierInterface;
@ -145,16 +146,21 @@ void decodeStorageCacheValue(const ValueRef& value, std::vector<uint16_t>& serve
extern const KeyRangeRef serverKeysRange;
extern const KeyRef serverKeysPrefix;
extern const ValueRef serverKeysTrue, serverKeysTrueEmptyRange, serverKeysFalse;
const UID newShardId(const uint64_t physicalShardId,
AssignEmptyRange assignEmptyRange,
UnassignShard unassignShard = UnassignShard::False);
const UID newDataMoveId(const uint64_t physicalShardId,
AssignEmptyRange assignEmptyRange,
EnablePhysicalShardMove enablePSM = EnablePhysicalShardMove::False,
UnassignShard unassignShard = UnassignShard::False);
const Key serverKeysKey(UID serverID, const KeyRef& keys);
const Key serverKeysPrefixFor(UID serverID);
UID serverKeysDecodeServer(const KeyRef& key);
std::pair<UID, Key> serverKeysDecodeServerBegin(const KeyRef& key);
bool serverHasKey(ValueRef storedValue);
const Value serverKeysValue(const UID& id);
void decodeServerKeysValue(const ValueRef& value, bool& assigned, bool& emptyRange, UID& id);
void decodeServerKeysValue(const ValueRef& value,
bool& assigned,
bool& emptyRange,
EnablePhysicalShardMove& enablePSM,
UID& id);
extern const KeyRangeRef conflictingKeysRange;
extern const ValueRef conflictingKeysTrue, conflictingKeysFalse;
@ -671,8 +677,9 @@ const Value blobGranuleFileValueFor(
int64_t offset,
int64_t length,
int64_t fullFileLength,
int64_t logicalSize,
Optional<BlobGranuleCipherKeysMeta> cipherKeysMeta = Optional<BlobGranuleCipherKeysMeta>());
std::tuple<Standalone<StringRef>, int64_t, int64_t, int64_t, Optional<BlobGranuleCipherKeysMeta>>
std::tuple<Standalone<StringRef>, int64_t, int64_t, int64_t, int64_t, Optional<BlobGranuleCipherKeysMeta>>
decodeBlobGranuleFileValue(ValueRef const& value);
const Value blobGranulePurgeValueFor(Version version, KeyRange range, bool force);

View File

@ -127,7 +127,7 @@ TenantMode tenantModeForClusterType(ClusterType clusterType, TenantMode tenantMo
int64_t extractTenantIdFromMutation(MutationRef m);
int64_t extractTenantIdFromKeyRef(StringRef s);
bool tenantMapChanging(MutationRef const& mutation, KeyRangeRef const& tenantMapRange);
bool nextTenantIdPrefixMatches(int64_t lastTenantId, int64_t nextTenantId);
int64_t computeNextTenantId(int64_t tenantId, int64_t delta);
int64_t getMaxAllowableTenantId(int64_t curTenantId);
int64_t getTenantIdPrefix(int64_t tenantId);
@ -230,14 +230,13 @@ Future<int64_t> getNextTenantId(Transaction tr) {
// Shift by 6 bytes to make the prefix the first two bytes of the tenant id
lastId = tenantIdPrefix << 48;
}
int64_t tenantId = lastId.get() + 1;
int64_t delta = 1;
if (BUGGIFY) {
tenantId += deterministicRandom()->randomSkewedUInt32(1, 1e9);
delta += deterministicRandom()->randomSkewedUInt32(1, 1e9);
}
if (!TenantAPI::nextTenantIdPrefixMatches(lastId.get(), tenantId)) {
throw cluster_no_capacity();
}
return tenantId;
return TenantAPI::computeNextTenantId(lastId.get(), delta);
}
ACTOR template <class DB>

View File

@ -134,17 +134,18 @@ private:
TenantMetadata::tenantCount().getD(&ryw->getTransaction(), Snapshot::False, 0);
int64_t _nextId = wait(TenantAPI::getNextTenantId(&ryw->getTransaction()));
state int64_t nextId = _nextId;
ASSERT(nextId > 0);
ASSERT(nextId >= 0);
state std::vector<Future<bool>> createFutures;
int itrCount = 0;
for (auto const& [tenant, config] : tenants) {
if (!TenantAPI::nextTenantIdPrefixMatches(nextId - 1, nextId)) {
throw cluster_no_capacity();
createFutures.push_back(createTenant(ryw, tenant, config, nextId, tenantGroupNetTenantDelta));
if (++itrCount < tenants.size()) {
nextId = TenantAPI::computeNextTenantId(nextId, 1);
}
createFutures.push_back(createTenant(ryw, tenant, config, nextId++, tenantGroupNetTenantDelta));
}
TenantMetadata::lastTenantId().set(&ryw->getTransaction(), nextId - 1);
TenantMetadata::lastTenantId().set(&ryw->getTransaction(), nextId);
wait(waitForAll(createFutures));
state int numCreatedTenants = 0;

View File

@ -322,6 +322,23 @@ public:
return infoString;
}
// Convert locality fields to a JSON object. This is a template because it works with JSONBuilder, StatusObject,
// and json_spirit::mObject, and none of these types are in the fdbrpc/ project.
template <typename JSONType>
JSONType toJSON() const {
JSONType obj;
for (auto it = _data.begin(); it != _data.end(); it++) {
if (it->second.present()) {
obj[it->first.toString()] = it->second.get().toString();
} else {
obj[it->first.toString()] = nullptr;
}
}
return obj;
}
template <class Ar>
void serialize(Ar& ar) {
// Locality is persisted in the database inside StorageServerInterface, so changes here have to be

View File

@ -444,8 +444,6 @@ Future<X> reportEndpointFailure(Future<X> value, Endpoint endpoint) {
}
}
Future<Void> disableConnectionFailuresAfter(double const& time, std::string const& context);
#include "flow/unactorcompiler.h"
#endif

View File

@ -55,6 +55,8 @@ struct ProcessInfo;
struct MachineInfo;
} // namespace simulator
constexpr double DISABLE_CONNECTION_FAILURE_FOREVER = 1e6;
class ISimulator : public INetwork {
public:
@ -323,6 +325,7 @@ public:
double lastConnectionFailure;
double connectionFailuresDisableDuration;
bool speedUpSimulation;
double connectionFailureEnableTime; // Last time connection failure is enabled.
BackupAgentType backupAgents;
BackupAgentType drAgents;
bool restarted = false;
@ -346,6 +349,10 @@ public:
// 'plaintext marker' is present.
Optional<std::string> dataAtRestPlaintextMarker;
// A collection of custom shard boundaries (begin, end, replication factor) that will be removed once this feature
// is integrated with a way to set these boundaries in the database
std::vector<std::tuple<std::string, std::string, int>> customReplicas;
flowGlobalType global(int id) const final;
void setGlobal(size_t id, flowGlobalType v) final;
@ -393,6 +400,12 @@ struct DiskParameters : ReferenceCounted<DiskParameters> {
// Simulates delays for performing operations on disk
extern Future<Void> waitUntilDiskReady(Reference<DiskParameters> parameters, int64_t size, bool sync = false);
// Enables connection failures, i.e., clogging, in simulation
void enableConnectionFailures(std::string const& context);
// Disables connection failures, i.e., clogging, in simulation
void disableConnectionFailures(std::string const& context);
class Sim2FileSystem : public IAsyncFileSystem {
public:
// Opens a file for asynchronous I/O

View File

@ -51,7 +51,6 @@
#include "fdbrpc/AsyncFileChaos.h"
#include "crc32/crc32c.h"
#include "fdbrpc/TraceFileIO.h"
#include "flow/FaultInjection.h"
#include "flow/flow.h"
#include "flow/genericactors.actor.h"
#include "flow/network.h"
@ -60,6 +59,7 @@
#include "fdbrpc/Replication.h"
#include "fdbrpc/ReplicationUtils.h"
#include "fdbrpc/AsyncFileWriteChecker.h"
#include "fdbrpc/genericactors.actor.h"
#include "flow/FaultInjection.h"
#include "flow/TaskQueue.h"
#include "flow/IUDPSocket.h"
@ -73,8 +73,8 @@ ISimulator::ISimulator()
: desiredCoordinators(1), physicalDatacenters(1), processesPerMachine(0), listenersPerProcess(1), usableRegions(1),
allowLogSetKills(true), tssMode(TSSMode::Disabled), configDBType(ConfigDBType::DISABLED), isStopped(false),
lastConnectionFailure(0), connectionFailuresDisableDuration(0), speedUpSimulation(false),
backupAgents(BackupAgentType::WaitForType), drAgents(BackupAgentType::WaitForType), allSwapsDisabled(false),
blobGranulesEnabled(false) {}
connectionFailureEnableTime(0), backupAgents(BackupAgentType::WaitForType), drAgents(BackupAgentType::WaitForType),
allSwapsDisabled(false), blobGranulesEnabled(false) {}
ISimulator::~ISimulator() = default;
bool simulator_should_inject_fault(const char* context, const char* file, int line, int error_code) {
@ -603,9 +603,7 @@ public:
}
if (openCount == 4000) {
TraceEvent(SevWarnAlways, "DisableConnectionFailures_TooManyFiles").log();
g_simulator->speedUpSimulation = true;
g_simulator->connectionFailuresDisableDuration = 1e6;
disableConnectionFailures("TooManyFiles");
}
// Filesystems on average these days seem to start to have limits of around 255 characters for a
@ -2680,7 +2678,8 @@ Future<Reference<IUDPSocket>> Sim2::createUDPSocket(bool isV6) {
void startNewSimulator(bool printSimTime) {
ASSERT(!g_network);
g_network = g_simulator = new Sim2(printSimTime);
g_simulator->connectionFailuresDisableDuration = deterministicRandom()->random01() < 0.5 ? 0 : 1e6;
g_simulator->connectionFailuresDisableDuration =
deterministicRandom()->coinflip() ? 0 : DISABLE_CONNECTION_FAILURE_FOREVER;
}
ACTOR void doReboot(ISimulator::ProcessInfo* p, ISimulator::KillType kt) {
@ -2782,6 +2781,23 @@ Future<Void> waitUntilDiskReady(Reference<DiskParameters> diskParameters, int64_
return delayUntil(diskParameters->nextOperation + randomLatency);
}
void enableConnectionFailures(std::string const& context) {
if (g_network->isSimulated()) {
g_simulator->connectionFailuresDisableDuration = 0;
g_simulator->speedUpSimulation = false;
g_simulator->connectionFailureEnableTime = now();
TraceEvent(SevWarnAlways, ("EnableConnectionFailures_" + context).c_str());
}
}
void disableConnectionFailures(std::string const& context) {
if (g_network->isSimulated()) {
g_simulator->connectionFailuresDisableDuration = DISABLE_CONNECTION_FAILURE_FOREVER;
g_simulator->speedUpSimulation = true;
TraceEvent(SevWarnAlways, ("DisableConnectionFailures_" + context).c_str());
}
}
#if defined(_WIN32)
/* Opening with FILE_SHARE_DELETE lets simulation actually work on windows - previously renames were always failing.

View File

@ -965,7 +965,8 @@ ACTOR Future<Void> pullAsyncData(BackupData* self) {
}
when(wait(logSystemChange)) {
if (self->logSystem.get()) {
r = self->logSystem.get()->peekLogRouter(self->myId, tagAt, self->tag);
r = self->logSystem.get()->peekLogRouter(
self->myId, tagAt, self->tag, SERVER_KNOBS->LOG_ROUTER_PEEK_FROM_SATELLITES_PREFERRED);
} else {
r = Reference<ILogSystem::IPeekCursor>();
}

View File

@ -74,14 +74,17 @@ ACTOR Future<Void> readGranuleFiles(Transaction* tr, Key* startKey, Key endKey,
int64_t offset;
int64_t length;
int64_t fullFileLength;
int64_t logicalSize;
Optional<BlobGranuleCipherKeysMeta> cipherKeysMeta;
std::tie(gid, version, fileType) = decodeBlobGranuleFileKey(it.key);
ASSERT(gid == granuleID);
std::tie(filename, offset, length, fullFileLength, cipherKeysMeta) = decodeBlobGranuleFileValue(it.value);
std::tie(filename, offset, length, fullFileLength, logicalSize, cipherKeysMeta) =
decodeBlobGranuleFileValue(it.value);
BlobFileIndex idx(version, filename.toString(), offset, length, fullFileLength, cipherKeysMeta);
BlobFileIndex idx(
version, filename.toString(), offset, length, fullFileLength, logicalSize, cipherKeysMeta);
if (fileType == 'S') {
ASSERT(files->snapshotFiles.empty() || files->snapshotFiles.back().version < idx.version);
files->snapshotFiles.push_back(idx);
@ -250,7 +253,7 @@ static std::string makeTestFileName(Version v) {
}
static BlobFileIndex makeTestFile(Version v, int64_t len) {
return BlobFileIndex(v, makeTestFileName(v), 0, len, len);
return BlobFileIndex(v, makeTestFileName(v), 0, len, len, len);
}
static void checkFile(int expectedVersion, const BlobFilePointerRef& actualFile) {

View File

@ -50,6 +50,7 @@
#include "fdbserver/Knobs.h"
#include "fdbserver/BlobGranuleValidation.actor.h"
#include "fdbserver/BlobGranuleServerCommon.actor.h"
#include "fdbserver/ExclusionTracker.actor.h"
#include "fdbserver/QuietDatabase.h"
#include "fdbserver/WaitFailure.h"
#include "fdbserver/WorkerInterface.actor.h"
@ -407,6 +408,8 @@ struct BlobManagerData : NonCopyable, ReferenceCounted<BlobManagerData> {
Debouncer restartRecruiting;
std::set<NetworkAddress> recruitingLocalities; // the addrs of the workers being recruited on
AsyncVar<int> recruitingStream;
ExclusionTracker exclusionTracker;
Promise<Void> foundBlobWorkers;
Promise<Void> doneRecovering;
Promise<Void> loadedClientRanges;
@ -430,7 +433,8 @@ struct BlobManagerData : NonCopyable, ReferenceCounted<BlobManagerData> {
mergeCandidates(MergeCandidateInfo(MergeCandidateUnknown), normalKeys.end),
activeGranuleMerges(invalidVersion, normalKeys.end), forcePurgingRanges(false, normalKeys.end),
concurrentMergeChecks(SERVER_KNOBS->BLOB_MANAGER_CONCURRENT_MERGE_CHECKS),
restartRecruiting(SERVER_KNOBS->DEBOUNCE_RECRUITING_DELAY), recruitingStream(0), epoch(epoch) {}
restartRecruiting(SERVER_KNOBS->DEBOUNCE_RECRUITING_DELAY), recruitingStream(0), exclusionTracker(db),
epoch(epoch) {}
// only initialize blob store if actually needed
void initBStore() {
@ -3101,20 +3105,36 @@ ACTOR Future<Void> monitorBlobWorker(Reference<BlobManagerData> bmData, BlobWork
try {
state Future<Void> waitFailure = waitFailureClient(bwInterf.waitFailure, SERVER_KNOBS->BLOB_WORKER_TIMEOUT);
state Future<Void> monitorStatus = monitorBlobWorkerStatus(bmData, bwInterf);
// set to already run future so we check this first loop
state Future<Void> exclusionsChanged = Future<Void>(Void());
choose {
when(wait(waitFailure)) {
if (SERVER_KNOBS->BLOB_WORKER_DISK_ENABLED) {
wait(delay(SERVER_KNOBS->BLOB_WORKER_REJOIN_TIME));
loop {
choose {
when(wait(waitFailure)) {
if (SERVER_KNOBS->BLOB_WORKER_DISK_ENABLED) {
wait(delay(SERVER_KNOBS->BLOB_WORKER_REJOIN_TIME));
}
if (BM_DEBUG) {
fmt::print("BM {0} detected BW {1} is dead\n", bmData->epoch, bwInterf.id().toString());
}
TraceEvent("BlobWorkerFailed", bmData->id).detail("BlobWorkerID", bwInterf.id());
break;
}
if (BM_DEBUG) {
fmt::print("BM {0} detected BW {1} is dead\n", bmData->epoch, bwInterf.id().toString());
when(wait(monitorStatus)) {
// should only return when manager got replaced
ASSERT(!bmData->iAmReplaced.canBeSet());
break;
}
when(wait(exclusionsChanged)) {
// check to see if we were just excluded
if (bmData->exclusionTracker.isFailedOrExcluded(bwInterf.stableAddress())) {
TraceEvent("BlobWorkerExcluded", bmData->id)
.detail("BlobWorkerID", bwInterf.id())
.detail("Addr", bwInterf.stableAddress());
break;
}
exclusionsChanged = bmData->exclusionTracker.changed.onTrigger();
}
TraceEvent("BlobWorkerFailed", bmData->id).detail("BlobWorkerID", bwInterf.id());
}
when(wait(monitorStatus)) {
// should only return when manager got replaced
ASSERT(!bmData->iAmReplaced.canBeSet());
}
}
} catch (Error& e) {
@ -3175,14 +3195,19 @@ ACTOR Future<Void> checkBlobWorkerList(Reference<BlobManagerData> bmData, Promis
bool foundAnyNew = false;
for (auto& worker : blobWorkers) {
if (!bmData->deadWorkers.count(worker.id())) {
bool isFailedOrExcluded = bmData->exclusionTracker.isFailedOrExcluded(worker.stableAddress());
if (!bmData->workerAddresses.count(worker.stableAddress()) &&
worker.locality.dcId() == bmData->dcId) {
worker.locality.dcId() == bmData->dcId && !isFailedOrExcluded) {
bmData->workerAddresses.insert(worker.stableAddress());
bmData->workersById[worker.id()] = worker;
bmData->workerStats[worker.id()] = BlobWorkerInfo();
bmData->addActor.send(monitorBlobWorker(bmData, worker));
foundAnyNew = true;
} else if (!bmData->workersById.count(worker.id())) {
TraceEvent("KillingExtraneousBlobWorker", bmData->id)
.detail("WorkerId", worker.id())
.detail("Addr", worker.stableAddress())
.detail("FailedOrExcluded", isFailedOrExcluded);
bmData->addActor.send(killBlobWorker(bmData, worker, false));
}
}
@ -3545,7 +3570,7 @@ ACTOR Future<Void> recoverBlobManager(Reference<BlobManagerData> bmData) {
// terminate blob restore for non-retryable errors
TraceEvent("ManifestLoadError", bmData->id).error(e).detail("Phase", phase);
std::string errorMessage = fmt::format("Manifest loading error '{}'", e.what());
wait(BlobRestoreController::updateError(restoreController, errorMessage));
wait(BlobRestoreController::updateError(restoreController, StringRef(errorMessage)));
}
}
}
@ -4066,7 +4091,22 @@ ACTOR Future<Void> blobWorkerRecruiter(
recruitReq.excludeAddresses.emplace_back(AddressExclusion(addr.ip, addr.port));
}
TraceEvent("BMRecruiting", self->id).detail("Epoch", self->epoch).detail("State", "Sending request to CC");
// don't recruit on excluded or failed addresses
CODE_PROBE(!self->exclusionTracker.excluded.empty(), "ignoring excluded hosts in BM recruitment");
CODE_PROBE(!self->exclusionTracker.failed.empty(), "ignoring failed hosts in BM recruitment");
for (auto addr : self->exclusionTracker.excluded) {
recruitReq.excludeAddresses.push_back(addr);
}
for (auto addr : self->exclusionTracker.failed) {
recruitReq.excludeAddresses.push_back(addr);
}
TraceEvent("BMRecruiting", self->id)
.detail("Epoch", self->epoch)
.detail("ExcludedCount", recruitReq.excludeAddresses.size())
.detail("State", "Sending request to CC");
if (!fCandidateWorker.isValid() || fCandidateWorker.isReady() ||
recruitReq.excludeAddresses != lastRequest.excludeAddresses) {
@ -4090,6 +4130,9 @@ ACTOR Future<Void> blobWorkerRecruiter(
// signal used to restart the loop and try to recruit the next blob worker
when(wait(self->restartRecruiting.onTrigger())) {}
// signal used to restart the loop and update request to CC with new exclusions
when(wait(self->exclusionTracker.changed.onTrigger())) {}
}
wait(delay(FLOW_KNOBS->PREVENT_FAST_SPIN_DELAY, TaskPriority::BlobManager));
} catch (Error& e) {

View File

@ -163,11 +163,10 @@ public:
const BlobManifestFile& firstFile = *iter;
result.push_back(firstFile);
// search all following files belonging to same manifest
for (auto it = iter + 1; it != allFiles.end(); ++it) {
if (it->belongToSameManifest(firstFile)) {
result.push_back(*it);
for (++iter; iter != allFiles.end(); ++iter) {
if (iter->belongToSameManifest(firstFile)) {
result.push_back(*iter);
} else {
iter = it; // start point for next search
break;
}
}
@ -768,10 +767,11 @@ private:
int64_t offset;
int64_t length;
int64_t fullFileLength;
int64_t logicalSize;
Optional<BlobGranuleCipherKeysMeta> cipherKeysMeta;
std::tie(gid, version, fileType) = decodeBlobGranuleFileKey(row.key);
std::tie(filename, offset, length, fullFileLength, cipherKeysMeta) =
std::tie(filename, offset, length, fullFileLength, logicalSize, cipherKeysMeta) =
decodeBlobGranuleFileValue(row.value);
GranuleFileVersion vs = { version, fileType, filename.toString(), length };
files.push_back(vs);

View File

@ -94,6 +94,16 @@ private:
.detail("Version", granule.version)
.detail("SizeInBytes", granule.sizeInBytes);
}
// Restore version is expected to be greater than max version from blob granule files.
state Version max = maxVersion(self);
Version targetVersion = wait(BlobRestoreController::getTargetVersion(restoreController, max));
if (targetVersion < max) {
TraceEvent("UnsupportedRestoreVersion", self->interf_.id())
.detail("MaxBlobGranulesVersion", max)
.detail("TargetVersion", targetVersion);
throw restore_missing_data();
}
wait(BlobRestoreController::updateState(restoreController, COPYING_DATA, LOADED_MANIFEST));
return Void();
}
@ -327,13 +337,16 @@ private:
// check last version in mutation logs
state std::string mlogsUrl = wait(getMutationLogUrl());
state Reference<IBackupContainer> bc = IBackupContainer::openContainer(mlogsUrl, {}, {});
state double beginTs = now();
BackupDescription desc = wait(bc->describeBackup(true));
TraceEvent("DescribeBackupLatency", self->interf_.id()).detail("Seconds", now() - beginTs);
if (!desc.contiguousLogEnd.present()) {
TraceEvent("InvalidMutationLogs").detail("Url", SERVER_KNOBS->BLOB_RESTORE_MLOGS_URL);
TraceEvent("InvalidMutationLogs", self->interf_.id()).detail("Url", SERVER_KNOBS->BLOB_RESTORE_MLOGS_URL);
throw blob_restore_missing_logs();
}
if (!desc.minLogBegin.present()) {
TraceEvent("InvalidMutationLogs").detail("Url", SERVER_KNOBS->BLOB_RESTORE_MLOGS_URL);
TraceEvent("InvalidMutationLogs", self->interf_.id()).detail("Url", SERVER_KNOBS->BLOB_RESTORE_MLOGS_URL);
throw blob_restore_missing_logs();
}
state Version minLogVersion = desc.minLogBegin.get();
@ -343,20 +356,20 @@ private:
state Version targetVersion = wait(BlobRestoreController::getTargetVersion(restoreController, maxLogVersion));
if (targetVersion < maxLogVersion) {
if (!needApplyLogs(self, targetVersion)) {
TraceEvent("SkipMutationLogs").detail("TargetVersion", targetVersion);
TraceEvent("SkipMutationLogs", self->interf_.id()).detail("TargetVersion", targetVersion);
dprint("Skip mutation logs as all granules are at version {}\n", targetVersion);
return Void();
}
}
if (targetVersion < minLogVersion) {
TraceEvent("MissingMutationLogs")
TraceEvent("MissingMutationLogs", self->interf_.id())
.detail("MinLogVersion", minLogVersion)
.detail("TargetVersion", maxLogVersion);
throw blob_restore_missing_logs();
}
if (targetVersion > maxLogVersion) {
TraceEvent("SkipTargetVersion")
TraceEvent("SkipTargetVersion", self->interf_.id())
.detail("MaxLogVersion", maxLogVersion)
.detail("TargetVersion", targetVersion);
}
@ -366,7 +379,7 @@ private:
state Standalone<VectorRef<Version>> beginVersions;
for (auto& granule : self->blobGranules_) {
if (granule.version < minLogVersion || granule.version > maxLogVersion) {
TraceEvent("InvalidMutationLogs")
TraceEvent("InvalidMutationLogs", self->interf_.id())
.detail("Granule", granule.granuleID)
.detail("GranuleVersion", granule.version)
.detail("MinLogVersion", minLogVersion)
@ -380,13 +393,15 @@ private:
// Blob granule ends at granule.version(inclusive), so we need to apply mutation logs
// after granule.version(exclusive).
beginVersions.push_back(beginVersions.arena(), granule.version);
TraceEvent("ApplyMutationLogVersion").detail("GID", granule.granuleID).detail("Ver", granule.version);
TraceEvent("ApplyMutationLogVersion", self->interf_.id())
.detail("GID", granule.granuleID)
.detail("Ver", granule.version);
}
}
Optional<RestorableFileSet> restoreSet =
wait(bc->getRestoreSet(maxLogVersion, ranges, OnlyApplyMutationLogs::True, minLogVersion));
if (!restoreSet.present()) {
TraceEvent("InvalidMutationLogs")
TraceEvent("InvalidMutationLogs", self->interf_.id())
.detail("MinLogVersion", minLogVersion)
.detail("MaxLogVersion", maxLogVersion);
throw blob_restore_corrupted_logs();
@ -657,7 +672,7 @@ ACTOR Future<Void> blobMigrator(BlobMigratorInterface interf, Reference<AsyncVar
TraceEvent("BlobMigratorError", interf.id()).error(e);
std::string errorMessage = fmt::format("Migrator failure '{}' on {}", e.what(), interf.address().toString());
Reference<BlobRestoreController> restoreController = makeReference<BlobRestoreController>(db, normalKeys);
wait(BlobRestoreController::updateError(restoreController, errorMessage));
wait(BlobRestoreController::updateError(restoreController, StringRef(errorMessage)));
}
return Void();
}

View File

@ -120,7 +120,8 @@ ACTOR Future<Void> BlobRestoreController::updateState(Reference<BlobRestoreContr
return Void();
}
ACTOR Future<Void> BlobRestoreController::updateError(Reference<BlobRestoreController> self, StringRef errorMessage) {
ACTOR Future<Void> BlobRestoreController::updateError(Reference<BlobRestoreController> self,
Standalone<StringRef> errorMessage) {
Standalone<BlobRestoreState> newState;
newState.error = StringRef(newState.arena(), errorMessage);
newState.phase = BlobRestorePhase::ERROR;

View File

@ -22,6 +22,7 @@
#include "fdbclient/BlobCipher.h"
#include "fdbclient/BlobGranuleFiles.h"
#include "fdbclient/FDBTypes.h"
#include "fdbclient/GetEncryptCipherKeys.actor.h"
#include "fdbclient/KeyRangeMap.h"
#include "fdbclient/SystemData.h"
#include "fdbclient/BackupContainerFileSystem.h"
@ -36,6 +37,7 @@
#include "fdbclient/NativeAPI.actor.h"
#include "fdbclient/Notified.h"
#include "fdbserver/BlobWorker.h"
#include "fdbserver/BlobGranuleServerCommon.actor.h"
#include "fdbclient/GetEncryptCipherKeys.actor.h"
#include "fdbserver/Knobs.h"
@ -67,356 +69,180 @@
#define BW_HISTORY_DEBUG false
#define BW_REQUEST_DEBUG false
void GranuleMetadata::resume() {
if (resumeSnapshot.canBeSet()) {
resumeSnapshot.send(Void());
}
}
void GranuleMetadata::resetReadStats() {
rdcCandidate = false;
readStats.reset();
runRDC.reset();
}
double GranuleMetadata::weightRDC() {
// ratio of read amp to write amp that would be incurred by re-snapshotting now
int64_t lastSnapshotSize = (files.snapshotFiles.empty()) ? 0 : files.snapshotFiles.back().length;
int64_t minSnapshotSize = SERVER_KNOBS->BG_SNAPSHOT_FILE_TARGET_BYTES / 2;
lastSnapshotSize = std::max(minSnapshotSize, lastSnapshotSize);
int64_t writeAmp = lastSnapshotSize + bufferedDeltaBytes + bytesInNewDeltaFiles;
// read amp is deltaBytesRead. Read amp must be READ_FACTOR times larger than write amp
return (1.0 * readStats.deltaBytesRead) / (writeAmp * SERVER_KNOBS->BG_RDC_READ_FACTOR);
}
bool GranuleMetadata::isEligibleRDC() const {
// granule should be reasonably read-hot to be eligible
int64_t bytesWritten = bufferedDeltaBytes + bytesInNewDeltaFiles;
return bytesWritten * SERVER_KNOBS->BG_RDC_READ_FACTOR < readStats.deltaBytesRead;
}
bool GranuleMetadata::updateReadStats(Version readVersion, const BlobGranuleChunkRef& chunk) {
// Only update stats for re-compacting for at-latest reads that have to do snapshot + delta merge
if (!SERVER_KNOBS->BG_ENABLE_READ_DRIVEN_COMPACTION || !chunk.snapshotFile.present() ||
pendingSnapshotVersion != durableSnapshotVersion.get() || readVersion <= pendingSnapshotVersion) {
return false;
}
if (chunk.newDeltas.empty() && chunk.deltaFiles.empty()) {
return false;
}
readStats.deltaBytesRead += chunk.newDeltas.expectedSize();
for (auto& it : chunk.deltaFiles) {
readStats.deltaBytesRead += it.length;
}
if (rdcCandidate) {
return false;
}
if (isEligibleRDC() && weightRDC() > 1.0) {
rdcCandidate = true;
CODE_PROBE(true, "Granule read triggering read-driven compaction");
if (BW_DEBUG) {
fmt::print("Triggering read-driven compaction of [{0} - {1})\n",
keyRange.begin.printable(),
keyRange.end.printable());
}
return true;
}
return false;
}
void GranuleRangeMetadata::cancel() {
if (activeMetadata->cancelled.canBeSet()) {
activeMetadata->cancelled.send(Void());
}
activeMetadata.clear();
assignFuture.cancel();
historyLoaderFuture.cancel();
fileUpdaterFuture.cancel();
}
/*
* The Blob Worker is a stateless role assigned a set of granules by the Blob Manager.
* It is responsible for managing the change feeds for those granules, and for consuming the mutations from
* those change feeds and writing them out as files to blob storage.
*/
struct GranuleStartState {
UID granuleID;
Version changeFeedStartVersion;
Version previousDurableVersion;
Optional<std::pair<KeyRange, UID>> splitParentGranule;
bool doSnapshot;
std::vector<GranuleFiles> blobFilesToSnapshot;
Optional<GranuleFiles> existingFiles;
Optional<GranuleHistory> history;
};
// TODO: add more (blob file request cost, in-memory mutations vs blob delta file, etc...)
struct GranuleReadStats {
int64_t deltaBytesRead;
void reset() { deltaBytesRead = 0; }
GranuleReadStats() { reset(); }
};
struct GranuleMetadata : NonCopyable, ReferenceCounted<GranuleMetadata> {
KeyRange keyRange;
GranuleFiles files;
Standalone<GranuleDeltas>
currentDeltas; // only contain deltas in pendingDeltaVersion + 1 through bufferedDeltaVersion
uint64_t bytesInNewDeltaFiles = 0;
uint64_t bufferedDeltaBytes = 0;
// for client to know when it is safe to read a certain version and from where (check waitForVersion)
Version bufferedDeltaVersion; // largest delta version in currentDeltas (including empty versions)
Version pendingDeltaVersion = 0; // largest version in progress writing to s3/fdb
NotifiedVersion durableDeltaVersion; // largest version persisted in s3/fdb
NotifiedVersion durableSnapshotVersion; // same as delta vars, except for snapshots
Version pendingSnapshotVersion = 0;
Version initialSnapshotVersion = invalidVersion;
Version historyVersion = invalidVersion;
Version knownCommittedVersion;
NotifiedVersion forceFlushVersion; // Version to force a flush at, if necessary
Version forceCompactVersion = invalidVersion;
int64_t originalEpoch;
int64_t originalSeqno;
int64_t continueEpoch;
int64_t continueSeqno;
Promise<Void> cancelled;
Promise<Void> readable;
Promise<Void> historyLoaded;
Promise<Void> resumeSnapshot;
AsyncVar<Reference<ChangeFeedData>> activeCFData;
AssignBlobRangeRequest originalReq;
GranuleReadStats readStats;
bool rdcCandidate;
Promise<Void> runRDC;
void resume() {
if (resumeSnapshot.canBeSet()) {
resumeSnapshot.send(Void());
}
}
void resetReadStats() {
rdcCandidate = false;
readStats.reset();
runRDC.reset();
}
// determine eligibility (>1) and priority for re-snapshotting this granule
double weightRDC() {
// ratio of read amp to write amp that would be incurred by re-snapshotting now
int64_t lastSnapshotSize = (files.snapshotFiles.empty()) ? 0 : files.snapshotFiles.back().length;
int64_t minSnapshotSize = SERVER_KNOBS->BG_SNAPSHOT_FILE_TARGET_BYTES / 2;
lastSnapshotSize = std::max(minSnapshotSize, lastSnapshotSize);
int64_t writeAmp = lastSnapshotSize + bufferedDeltaBytes + bytesInNewDeltaFiles;
// read amp is deltaBytesRead. Read amp must be READ_FACTOR times larger than write amp
return (1.0 * readStats.deltaBytesRead) / (writeAmp * SERVER_KNOBS->BG_RDC_READ_FACTOR);
}
bool isEligibleRDC() const {
// granule should be reasonably read-hot to be eligible
int64_t bytesWritten = bufferedDeltaBytes + bytesInNewDeltaFiles;
return bytesWritten * SERVER_KNOBS->BG_RDC_READ_FACTOR < readStats.deltaBytesRead;
}
bool updateReadStats(Version readVersion, const BlobGranuleChunkRef& chunk) {
// Only update stats for re-compacting for at-latest reads that have to do snapshot + delta merge
if (!SERVER_KNOBS->BG_ENABLE_READ_DRIVEN_COMPACTION || !chunk.snapshotFile.present() ||
pendingSnapshotVersion != durableSnapshotVersion.get() || readVersion <= pendingSnapshotVersion) {
return false;
}
if (chunk.newDeltas.empty() && chunk.deltaFiles.empty()) {
return false;
}
readStats.deltaBytesRead += chunk.newDeltas.expectedSize();
for (auto& it : chunk.deltaFiles) {
readStats.deltaBytesRead += it.length;
}
if (rdcCandidate) {
return false;
}
if (isEligibleRDC() && weightRDC() > 1.0) {
rdcCandidate = true;
CODE_PROBE(true, "Granule read triggering read-driven compaction");
if (BW_DEBUG) {
fmt::print("Triggering read-driven compaction of [{0} - {1})\n",
keyRange.begin.printable(),
keyRange.end.printable());
}
return true;
bool BlobWorkerData::managerEpochOk(int64_t epoch) {
if (epoch < currentManagerEpoch) {
if (BW_DEBUG) {
fmt::print(
"BW {0} got request from old epoch {1}, notifying them they are out of date\n", id.toString(), epoch);
}
return false;
}
inline bool doEarlyReSnapshot() {
return runRDC.isSet() ||
(forceCompactVersion <= pendingDeltaVersion && forceCompactVersion > pendingSnapshotVersion);
}
};
struct GranuleRangeMetadata {
int64_t lastEpoch;
int64_t lastSeqno;
Reference<GranuleMetadata> activeMetadata;
Future<GranuleStartState> assignFuture;
Future<Void> fileUpdaterFuture;
Future<Void> historyLoaderFuture;
void cancel() {
if (activeMetadata->cancelled.canBeSet()) {
activeMetadata->cancelled.send(Void());
}
activeMetadata.clear();
assignFuture.cancel();
historyLoaderFuture.cancel();
fileUpdaterFuture.cancel();
}
GranuleRangeMetadata() : lastEpoch(0), lastSeqno(0) {}
GranuleRangeMetadata(int64_t epoch, int64_t seqno, Reference<GranuleMetadata> activeMetadata)
: lastEpoch(epoch), lastSeqno(seqno), activeMetadata(activeMetadata) {}
};
// represents a previous version of a granule, and optionally the files that compose it.
struct GranuleHistoryEntry : NonCopyable, ReferenceCounted<GranuleHistoryEntry> {
KeyRange range;
UID granuleID;
Version startVersion; // version of the first snapshot
Version endVersion; // version of the last delta file
// load files lazily, and allows for clearing old cold-queried files to save memory
// FIXME: add memory limit and evictor for old cached files
Future<GranuleFiles> files;
// FIXME: do skip pointers with single back-pointer and neighbor pointers
std::vector<Reference<GranuleHistoryEntry>> parentGranules;
GranuleHistoryEntry() : startVersion(invalidVersion), endVersion(invalidVersion) {}
GranuleHistoryEntry(KeyRange range, UID granuleID, Version startVersion, Version endVersion)
: range(range), granuleID(granuleID), startVersion(startVersion), endVersion(endVersion) {}
};
struct BlobWorkerData : NonCopyable, ReferenceCounted<BlobWorkerData> {
UID id;
Database db;
IKeyValueStore* storage;
PromiseStream<Future<Void>> addActor;
LocalityData locality;
int64_t currentManagerEpoch = -1;
AsyncVar<ReplyPromiseStream<GranuleStatusReply>> currentManagerStatusStream;
bool statusStreamInitialized = false;
// FIXME: refactor out the parts of this that are just for interacting with blob stores from the backup business
// logic
Reference<BlobConnectionProvider> bstore;
KeyRangeMap<GranuleRangeMetadata> granuleMetadata;
BGTenantMap tenantData;
Reference<AsyncVar<ServerDBInfo> const> dbInfo;
// contains the history of completed granules before the existing ones. Maps to the latest one, and has
// back-pointers to earlier granules
// FIXME: expire from map after a delay when granule is revoked and the history is no longer needed
KeyRangeMap<Reference<GranuleHistoryEntry>> granuleHistory;
PromiseStream<AssignBlobRangeRequest> granuleUpdateErrors;
Promise<Void> doGRVCheck;
NotifiedVersion grvVersion;
std::deque<Version> prevGRVVersions;
Promise<Void> fatalError;
Promise<Void> simInjectFailure;
Promise<Void> doReadDrivenCompaction;
Reference<FlowLock> initialSnapshotLock;
Reference<FlowLock> resnapshotBudget;
Reference<FlowLock> deltaWritesBudget;
BlobWorkerStats stats;
bool shuttingDown = false;
// FIXME: have cap on this independent of delta file size for larger granules
int changeFeedStreamReplyBufferSize = SERVER_KNOBS->BG_DELTA_FILE_TARGET_BYTES / 4;
EncryptionAtRestMode encryptMode;
bool buggifyFull = false;
int64_t memoryFullThreshold =
(int64_t)(SERVER_KNOBS->BLOB_WORKER_REJECT_WHEN_FULL_THRESHOLD * SERVER_KNOBS->SERVER_MEM_LIMIT);
int64_t lastResidentMemory = 0;
double lastResidentMemoryCheckTime = -100.0;
bool isFullRestoreMode = false;
BlobWorkerData(UID id, Reference<AsyncVar<ServerDBInfo> const> dbInfo, Database db, IKeyValueStore* storage)
: id(id), db(db), storage(storage), tenantData(BGTenantMap(dbInfo)), dbInfo(dbInfo),
initialSnapshotLock(new FlowLock(SERVER_KNOBS->BLOB_WORKER_INITIAL_SNAPSHOT_PARALLELISM)),
resnapshotBudget(new FlowLock(SERVER_KNOBS->BLOB_WORKER_RESNAPSHOT_BUDGET_BYTES)),
deltaWritesBudget(new FlowLock(SERVER_KNOBS->BLOB_WORKER_DELTA_WRITE_BUDGET_BYTES)),
stats(id,
SERVER_KNOBS->WORKER_LOGGING_INTERVAL,
initialSnapshotLock,
resnapshotBudget,
deltaWritesBudget,
SERVER_KNOBS->LATENCY_METRICS_LOGGING_INTERVAL,
SERVER_KNOBS->FILE_LATENCY_SKETCH_ACCURACY,
SERVER_KNOBS->LATENCY_SKETCH_ACCURACY),
encryptMode(EncryptionAtRestMode::DISABLED) {}
bool managerEpochOk(int64_t epoch) {
if (epoch < currentManagerEpoch) {
} else {
if (epoch > currentManagerEpoch) {
currentManagerEpoch = epoch;
if (BW_DEBUG) {
fmt::print("BW {0} got request from old epoch {1}, notifying them they are out of date\n",
id.toString(),
epoch);
fmt::print("BW {0} found new manager epoch {1}\n", id.toString(), currentManagerEpoch);
}
return false;
} else {
if (epoch > currentManagerEpoch) {
currentManagerEpoch = epoch;
if (BW_DEBUG) {
fmt::print("BW {0} found new manager epoch {1}\n", id.toString(), currentManagerEpoch);
}
TraceEvent(SevDebug, "BlobWorkerFoundNewManager", id).detail("Epoch", epoch);
}
return true;
TraceEvent(SevDebug, "BlobWorkerFoundNewManager", id).detail("Epoch", epoch);
}
return true;
}
}
bool isFull() {
if (!SERVER_KNOBS->BLOB_WORKER_DO_REJECT_WHEN_FULL) {
return false;
}
if (g_network->isSimulated()) {
if (g_simulator->speedUpSimulation) {
return false;
}
return buggifyFull;
}
// TODO knob?
if (now() >= 1.0 + lastResidentMemoryCheckTime) {
// fdb as of 7.1 limits on resident memory instead of virtual memory
stats.lastResidentMemory = getResidentMemoryUsage();
lastResidentMemoryCheckTime = now();
}
// if we are already over threshold, no need to estimate extra memory
if (stats.lastResidentMemory >= memoryFullThreshold) {
return true;
}
// FIXME: since this isn't tested in simulation, could unit test this
// Try to model how much memory we *could* use given the already existing assignments and workload on this blob
// worker, before agreeing to take on a new assignment, given that several large sources of memory can grow and
// change post-assignment
// FIXME: change these to be byte counts
// FIXME: buggify an extra multiplication factor for short periods of time to hopefully trigger this logic more
// often? estimate slack in bytes buffered as max(0, assignments * (delta file size / 2) - bytesBuffered)
// FIXME: this doesn't take increased delta file size for heavy write amp cases into account
int64_t expectedExtraBytesBuffered = std::max<int64_t>(
0, stats.numRangesAssigned * (SERVER_KNOBS->BG_DELTA_FILE_TARGET_BYTES / 2) - stats.mutationBytesBuffered);
// estimate slack in potential pending resnapshot
int64_t totalExtra =
expectedExtraBytesBuffered + deltaWritesBudget->available() + resnapshotBudget->available();
// assumes initial snapshot parallelism is small enough and uncommon enough to not add it to this computation
stats.estimatedMaxResidentMemory = stats.lastResidentMemory + totalExtra;
return stats.estimatedMaxResidentMemory >= memoryFullThreshold;
}
void triggerReadDrivenCompaction() {
Promise<Void> doRDC = doReadDrivenCompaction;
if (doRDC.canBeSet()) {
doRDC.send(Void());
}
}
void addGRVHistory(Version readVersion) {
if (grvVersion.get() < readVersion) {
// We use GRVs from grv checker loop, plus other common BW transactions. To prevent the deque size from
// exploding or the effective version window from getting too small, only put GRVs in the deque if they are
// at least some small distance apart
if (grvVersion.get() + SERVER_KNOBS->BLOB_WORKER_GRV_HISTORY_MIN_VERSION_GRANULARITY <= readVersion) {
prevGRVVersions.push_back(readVersion);
while (prevGRVVersions.size() > SERVER_KNOBS->BLOB_WORKER_GRV_HISTORY_MAX_SIZE) {
prevGRVVersions.pop_front();
}
}
// set notified version last, so that all triggered waiters have prevGRVVersions populated too
grvVersion.set(readVersion);
}
}
bool maybeInjectTargetedRestart() {
// inject a BW restart at most once per test
if (g_network->isSimulated() && !g_simulator->speedUpSimulation &&
now() > g_simulator->injectTargetedBWRestartTime) {
CODE_PROBE(true, "Injecting BW targeted restart");
TraceEvent("SimBWInjectTargetedRestart", id);
g_simulator->injectTargetedBWRestartTime = std::numeric_limits<double>::max();
simInjectFailure.send(Void());
return true;
}
bool BlobWorkerData::isFull() {
if (!SERVER_KNOBS->BLOB_WORKER_DO_REJECT_WHEN_FULL) {
return false;
}
};
if (g_network->isSimulated()) {
if (g_simulator->speedUpSimulation) {
return false;
}
return buggifyFull;
}
// TODO knob?
if (now() >= 1.0 + lastResidentMemoryCheckTime) {
// fdb as of 7.1 limits on resident memory instead of virtual memory
stats.lastResidentMemory = getResidentMemoryUsage();
lastResidentMemoryCheckTime = now();
}
// if we are already over threshold, no need to estimate extra memory
if (stats.lastResidentMemory >= memoryFullThreshold) {
return true;
}
// FIXME: since this isn't tested in simulation, could unit test this
// Try to model how much memory we *could* use given the already existing assignments and workload on this blob
// worker, before agreeing to take on a new assignment, given that several large sources of memory can grow and
// change post-assignment
// FIXME: change these to be byte counts
// FIXME: buggify an extra multiplication factor for short periods of time to hopefully trigger this logic more
// often? estimate slack in bytes buffered as max(0, assignments * (delta file size / 2) - bytesBuffered)
// FIXME: this doesn't take increased delta file size for heavy write amp cases into account
int64_t expectedExtraBytesBuffered = std::max<int64_t>(
0, stats.numRangesAssigned * (SERVER_KNOBS->BG_DELTA_FILE_TARGET_BYTES / 2) - stats.mutationBytesBuffered);
// estimate slack in potential pending resnapshot
int64_t totalExtra = expectedExtraBytesBuffered + deltaWritesBudget->available() + resnapshotBudget->available();
// assumes initial snapshot parallelism is small enough and uncommon enough to not add it to this computation
stats.estimatedMaxResidentMemory = stats.lastResidentMemory + totalExtra;
return stats.estimatedMaxResidentMemory >= memoryFullThreshold;
}
void BlobWorkerData::triggerReadDrivenCompaction() {
Promise<Void> doRDC = doReadDrivenCompaction;
if (doRDC.canBeSet()) {
doRDC.send(Void());
}
}
void BlobWorkerData::addGRVHistory(Version readVersion) {
if (grvVersion.get() < readVersion) {
// We use GRVs from grv checker loop, plus other common BW transactions. To prevent the deque size from
// exploding or the effective version window from getting too small, only put GRVs in the deque if they are
// at least some small distance apart
if (prevGRVVersions.empty() ||
prevGRVVersions.back() + SERVER_KNOBS->BLOB_WORKER_GRV_HISTORY_MIN_VERSION_GRANULARITY <= readVersion) {
prevGRVVersions.push_back(readVersion);
while (prevGRVVersions.size() > SERVER_KNOBS->BLOB_WORKER_GRV_HISTORY_MAX_SIZE) {
prevGRVVersions.pop_front();
}
}
// set notified version last, so that all triggered waiters have prevGRVVersions populated too
grvVersion.set(readVersion);
}
}
bool BlobWorkerData::maybeInjectTargetedRestart() {
// inject a BW restart at most once per test
if (g_network->isSimulated() && !g_simulator->speedUpSimulation &&
now() > g_simulator->injectTargetedBWRestartTime) {
CODE_PROBE(true, "Injecting BW targeted restart");
TraceEvent("SimBWInjectTargetedRestart", id);
g_simulator->injectTargetedBWRestartTime = std::numeric_limits<double>::max();
simInjectFailure.send(Void());
return true;
}
return false;
}
namespace {
@ -424,6 +250,9 @@ Optional<CompressionFilter> getBlobFileCompressFilter() {
Optional<CompressionFilter> compFilter;
if (SERVER_KNOBS->ENABLE_BLOB_GRANULE_COMPRESSION) {
compFilter = CompressionUtils::fromFilterString(SERVER_KNOBS->BLOB_GRANULE_COMPRESSION_FILTER);
if (BUGGIFY_WITH_PROB(0.1)) {
compFilter = CompressionUtils::getRandomFilter();
}
}
return compFilter;
}
@ -878,8 +707,9 @@ ACTOR Future<BlobFileIndex> writeDeltaFile(Reference<BlobWorkerData> bwData,
SERVER_KNOBS->BG_DELTA_FILE_TARGET_CHUNK_BYTES,
compressFilter,
cipherKeysCtx);
state size_t logicalSize = deltasToWrite.expectedSize();
state size_t serializedSize = serialized.size();
bwData->stats.compressionBytesRaw += deltasToWrite.expectedSize();
bwData->stats.compressionBytesRaw += logicalSize;
bwData->stats.compressionBytesFinal += serializedSize;
// Free up deltasToWrite here to reduce memory
@ -926,7 +756,8 @@ ACTOR Future<BlobFileIndex> writeDeltaFile(Reference<BlobWorkerData> bwData,
Key dfKey = blobGranuleFileKeyFor(granuleID, currentDeltaVersion, 'D');
// TODO change once we support file multiplexing
Value dfValue = blobGranuleFileValueFor(fname, 0, serializedSize, serializedSize, cipherKeysMeta);
Value dfValue =
blobGranuleFileValueFor(fname, 0, serializedSize, serializedSize, logicalSize, cipherKeysMeta);
tr->set(dfKey, dfValue);
if (oldGranuleComplete.present()) {
@ -970,7 +801,8 @@ ACTOR Future<BlobFileIndex> writeDeltaFile(Reference<BlobWorkerData> bwData,
bwData->stats.deltaUpdateSample.addMeasurement(duration);
// FIXME: change when we implement multiplexing
return BlobFileIndex(currentDeltaVersion, fname, 0, serializedSize, serializedSize, cipherKeysMeta);
return BlobFileIndex(
currentDeltaVersion, fname, 0, serializedSize, serializedSize, logicalSize, cipherKeysMeta);
} catch (Error& e) {
wait(tr->onError(e));
}
@ -1028,6 +860,14 @@ ACTOR Future<BlobFileIndex> writeEmptyDeltaFile(Reference<BlobWorkerData> bwData
if (!dfValue.present()) {
// Only check if not seen yet. If we get commit unknown result and then retry, we'd see our own delete
wait(store(dfValue, tr->get(oldDFKey)));
if (!dfValue.present()) {
TraceEvent("MissingFileEmptyWrite", bwData->id)
.detail("Granule", keyRange)
.detail("PrevVersion", previousVersion)
.detail("CurrentVersion", currentDeltaVersion)
.detail("PrevKey", oldDFKey)
.detail("NewKey", newDFKey);
}
ASSERT(dfValue.present());
} else {
tr->addReadConflictRange(singleKeyRange(oldDFKey));
@ -1065,7 +905,7 @@ ACTOR Future<BlobFileIndex> writeEmptyDeltaFile(Reference<BlobWorkerData> bwData
wait(delay(deterministicRandom()->random01()));
}
return BlobFileIndex(currentDeltaVersion, "", 0, 0, 0, {});
return BlobFileIndex(currentDeltaVersion, "", 0, 0, 0, 0, {});
} catch (Error& e) {
wait(tr->onError(e));
}
@ -1174,8 +1014,9 @@ ACTOR Future<BlobFileIndex> writeSnapshot(Reference<BlobWorkerData> bwData,
SERVER_KNOBS->BG_SNAPSHOT_FILE_TARGET_CHUNK_BYTES,
compressFilter,
cipherKeysCtx);
state size_t logicalSize = snapshot.expectedSize();
state size_t serializedSize = serialized.size();
bwData->stats.compressionBytesRaw += snapshot.expectedSize();
bwData->stats.compressionBytesRaw += logicalSize;
bwData->stats.compressionBytesFinal += serializedSize;
// free snapshot to reduce memory
@ -1226,7 +1067,7 @@ ACTOR Future<BlobFileIndex> writeSnapshot(Reference<BlobWorkerData> bwData,
Key snapshotFileKey = blobGranuleFileKeyFor(granuleID, version, 'S');
// TODO change once we support file multiplexing
Key snapshotFileValue =
blobGranuleFileValueFor(fname, 0, serializedSize, serializedSize, cipherKeysMeta);
blobGranuleFileValueFor(fname, 0, serializedSize, serializedSize, logicalSize, cipherKeysMeta);
tr->set(snapshotFileKey, snapshotFileValue);
// create granule history at version if this is a new granule with the initial dump from FDB
if (initialSnapshot) {
@ -1283,7 +1124,7 @@ ACTOR Future<BlobFileIndex> writeSnapshot(Reference<BlobWorkerData> bwData,
}
// FIXME: change when we implement multiplexing
return BlobFileIndex(version, fname, 0, serializedSize, serializedSize, cipherKeysMeta);
return BlobFileIndex(version, fname, 0, serializedSize, serializedSize, logicalSize, cipherKeysMeta);
}
ACTOR Future<BlobFileIndex> dumpInitialSnapshotFromFDB(Reference<BlobWorkerData> bwData,
@ -1962,7 +1803,9 @@ Version doGranuleRollback(Reference<GranuleMetadata> metadata,
cfRollbackVersion = metadata->durableDeltaVersion.get();
metadata->pendingSnapshotVersion = metadata->durableSnapshotVersion.get();
int toPop = 0;
bool pendingSnapshot = false;
// keep bytes in delta files pending here, then add back already durable delta files at end
metadata->bytesInNewDeltaFiles = 0;
for (auto& f : inFlightFiles) {
if (f.snapshot) {
if (f.version > rollbackVersion) {
@ -1978,14 +1821,10 @@ Version doGranuleRollback(Reference<GranuleMetadata> metadata,
} else {
metadata->pendingSnapshotVersion = f.version;
metadata->bytesInNewDeltaFiles = 0;
pendingSnapshot = true;
}
} else {
if (f.version > rollbackVersion) {
f.future.cancel();
if (!pendingSnapshot) {
metadata->bytesInNewDeltaFiles -= f.bytes;
}
toPop++;
CODE_PROBE(true, "Granule rollback cancelling delta file");
if (BW_DEBUG) {
@ -1997,9 +1836,7 @@ Version doGranuleRollback(Reference<GranuleMetadata> metadata,
} else {
ASSERT(f.version > cfRollbackVersion);
cfRollbackVersion = f.version;
if (pendingSnapshot) {
metadata->bytesInNewDeltaFiles += f.bytes;
}
metadata->bytesInNewDeltaFiles += f.bytes;
}
}
}
@ -2026,6 +1863,13 @@ Version doGranuleRollback(Reference<GranuleMetadata> metadata,
metadata->bufferedDeltaBytes = 0;
metadata->bufferedDeltaVersion = cfRollbackVersion;
// calculate number of bytes in durable delta files after last snapshot
for (int i = metadata->files.deltaFiles.size() - 1;
i >= 0 && metadata->files.deltaFiles[i].version > metadata->pendingSnapshotVersion;
i--) {
metadata->bytesInNewDeltaFiles += metadata->files.deltaFiles[i].logicalSize;
}
// Track that this rollback happened, since we have to re-read mutations up to the rollback
// Add this rollback to in progress, and put all completed ones back in progress
rollbacksInProgress.push_back(std::pair(rollbackVersion, mutationVersion));
@ -2179,17 +2023,19 @@ ACTOR Future<Void> waitCommittedGrv(Reference<BlobWorkerData> bwData,
}
ASSERT(grvVersion >= version);
wait(waitOnCFVersion(metadata, grvVersion));
// If GRV is way in the future, we know we can't roll back more than 5 seconds (or whatever this knob is set
// to) worth of versions
Version waitVersion = std::min(grvVersion, version + SERVER_KNOBS->MAX_READ_TRANSACTION_LIFE_VERSIONS);
wait(waitOnCFVersion(metadata, waitVersion));
return Void();
}
ACTOR Future<Void> waitVersionCommitted(Reference<BlobWorkerData> bwData,
Reference<GranuleMetadata> metadata,
Version version) {
// If GRV is way in the future, we know we can't roll back more than 5 seconds (or whatever this knob is set
// to) worth of versions
wait(waitCommittedGrv(bwData, metadata, version) ||
waitOnCFVersion(metadata, version + SERVER_KNOBS->MAX_READ_TRANSACTION_LIFE_VERSIONS));
wait(waitCommittedGrv(bwData, metadata, version));
if (version > metadata->knownCommittedVersion) {
metadata->knownCommittedVersion = version;
}
@ -2359,7 +2205,7 @@ ACTOR Future<Void> blobGranuleUpdateFiles(Reference<BlobWorkerData> bwData,
Version snapshotVersion = files.snapshotFiles.back().version;
for (int i = files.deltaFiles.size() - 1; i >= 0; i--) {
if (files.deltaFiles[i].version > snapshotVersion) {
metadata->bytesInNewDeltaFiles += files.deltaFiles[i].length;
metadata->bytesInNewDeltaFiles += files.deltaFiles[i].logicalSize;
}
}
}

View File

@ -2666,7 +2666,7 @@ ACTOR Future<Void> dbInfoUpdater(ClusterControllerData* self) {
when(wait(dbInfoChange)) {}
}
UpdateServerDBInfoRequest req;
state UpdateServerDBInfoRequest req;
if (dbInfoChange.isReady()) {
for (auto& it : self->id_worker) {
req.broadcastInfo.push_back(it.second.details.interf.updateServerDBInfo.getEndpoint());
@ -2688,7 +2688,9 @@ ACTOR Future<Void> dbInfoUpdater(ClusterControllerData* self) {
req.serializedDbInfo =
BinaryWriter::toValue(self->db.serverInfo->get(), AssumeVersion(g_network->protocolVersion()));
TraceEvent("DBInfoStartBroadcast", self->id).log();
TraceEvent("DBInfoStartBroadcast", self->id)
.detail("MasterLifetime", self->db.serverInfo->get().masterLifetime.toString());
;
choose {
when(std::vector<Endpoint> notUpdated =
wait(broadcastDBInfoRequest(req, SERVER_KNOBS->DBINFO_SEND_AMOUNT, Optional<Endpoint>(), false))) {
@ -2699,6 +2701,11 @@ ACTOR Future<Void> dbInfoUpdater(ClusterControllerData* self) {
}
}
when(wait(dbInfoChange)) {}
when(wait(updateDBInfo)) {
// The current round of broadcast hasn't finished yet. So we need to include all the current broadcast
// endpoints in the new round as well.
self->updateDBInfoEndpoints.insert(req.broadcastInfo.begin(), req.broadcastInfo.end());
}
}
}
}

View File

@ -1533,9 +1533,7 @@ ACTOR Future<Void> clusterRecoveryCore(Reference<ClusterRecoveryData> self) {
(self->cstate.myDBState.oldTLogData.size() - CLIENT_KNOBS->RECOVERY_DELAY_START_GENERATION)));
}
if (g_network->isSimulated() && self->cstate.myDBState.oldTLogData.size() > CLIENT_KNOBS->MAX_GENERATIONS_SIM) {
g_simulator->connectionFailuresDisableDuration = 1e6;
g_simulator->speedUpSimulation = true;
TraceEvent(SevWarnAlways, "DisableConnectionFailures_TooManyGenerations").log();
disableConnectionFailures("TooManyGenerations");
}
}

View File

@ -37,6 +37,7 @@
#include "fdbserver/TesterInterface.actor.h"
#include "flow/DeterministicRandom.h"
#include "flow/Trace.h"
#include "fdbserver/QuietDatabase.h"
#include "flow/actorcompiler.h" // This must be the last #include.
// Core of the data consistency checking (checkDataConsistency) and many of the supporting functions are shared between
@ -59,6 +60,7 @@ struct ConsistencyScanData {
int finishedRounds = 0;
KeyRef progressKey;
AsyncVar<bool> consistencyScanEnabled = false;
bool success = true;
ConsistencyScanData(UID id, Database db) : id(id), db(db) {}
};
@ -77,7 +79,8 @@ ACTOR Future<Version> getVersion(Database cx) {
}
}
void testFailure(std::string message, bool performQuiescentChecks, bool isError) {
void testFailure(std::string message, bool performQuiescentChecks, bool* success, bool isError) {
*success = false;
TraceEvent failEvent(isError ? SevError : SevWarn, "TestFailure");
if (performQuiescentChecks)
failEvent.detail("Workload", "QuiescentCheck");
@ -94,7 +97,9 @@ ACTOR Future<bool> getKeyServers(
Database cx,
Promise<std::vector<std::pair<KeyRange, std::vector<StorageServerInterface>>>> keyServersPromise,
KeyRangeRef kr,
bool performQuiescentChecks) {
bool performQuiescentChecks,
bool failureIsError,
bool* success) {
state std::vector<std::pair<KeyRange, std::vector<StorageServerInterface>>> keyServers;
// Try getting key server locations from the master proxies
@ -131,7 +136,7 @@ ACTOR Future<bool> getKeyServers(
TraceEvent("ConsistencyCheck_CommitProxyUnavailable")
.error(shards.getError())
.detail("CommitProxyID", commitProxyInfo->getId(i));
testFailure("Commit proxy unavailable", performQuiescentChecks, true);
testFailure("Commit proxy unavailable", performQuiescentChecks, success, failureIsError);
return false;
}
@ -164,7 +169,8 @@ ACTOR Future<bool> getKeyServers(
ACTOR Future<bool> getKeyLocations(Database cx,
std::vector<std::pair<KeyRange, std::vector<StorageServerInterface>>> shards,
Promise<Standalone<VectorRef<KeyValueRef>>> keyLocationPromise,
bool performQuiescentChecks) {
bool performQuiescentChecks,
bool* success) {
state Standalone<VectorRef<KeyValueRef>> keyLocations;
state Key beginKey = allKeys.begin.withPrefix(keyServersPrefix);
state Key endKey = allKeys.end.withPrefix(keyServersPrefix);
@ -221,7 +227,7 @@ ACTOR Future<bool> getKeyLocations(Database cx,
TraceEvent("ConsistencyCheck_InconsistentKeyServers")
.detail("StorageServer1", shards[i].second[firstValidStorageServer].id())
.detail("StorageServer2", shards[i].second[j].id());
testFailure("Key servers inconsistent", performQuiescentChecks, true);
testFailure("Key servers inconsistent", performQuiescentChecks, success, true);
return false;
}
}
@ -353,7 +359,7 @@ ACTOR Future<int64_t> getDatabaseSize(Database cx) {
// Checks that the data in each shard is the same on each storage server that it resides on. Also performs some
// sanity checks on the sizes of shards and storage servers. Returns false if there is a failure
// TODO: Future optimization: Use streaming reads
ACTOR Future<bool> checkDataConsistency(Database cx,
ACTOR Future<Void> checkDataConsistency(Database cx,
VectorRef<KeyValueRef> keyLocations,
DatabaseConfiguration configuration,
std::map<UID, StorageServerInterface> tssMapping,
@ -372,7 +378,8 @@ ACTOR Future<bool> checkDataConsistency(Database cx,
int restart,
int64_t maxRate,
int64_t targetInterval,
KeyRef progressKey) {
KeyRef progressKey,
bool* success) {
// Stores the total number of bytes on each storage server
// In a distributed test, this will be an estimated size
state std::map<UID, int64_t> storageServerSizes;
@ -393,13 +400,20 @@ ACTOR Future<bool> checkDataConsistency(Database cx,
state double rateLimiterStartTime = now();
state int64_t bytesReadInthisRound = 0;
state bool resume = !(restart || shuffleShards);
state bool testResult = true;
state double dbSize = 100e12;
state int ssCount = 1e6;
if (g_network->isSimulated()) {
// This call will get all shard ranges in the database, which is too expensive on real clusters.
int64_t _dbSize = wait(getDatabaseSize(cx));
dbSize = _dbSize;
std::vector<StorageServerInterface> storageServers = wait(getStorageServers(cx));
ssCount = 0;
for (auto& it : storageServers) {
if (!it.isTss()) {
++ssCount;
}
}
}
state std::vector<KeyRangeRef> ranges;
@ -443,23 +457,55 @@ ACTOR Future<bool> checkDataConsistency(Database cx,
// If the destStorageServers is non-empty, then this shard is being relocated
state bool isRelocating = destStorageServers.size() > 0;
state int customReplication = configuration.storageTeamSize;
if (g_network->isSimulated() && ddLargeTeamEnabled()) {
for (auto& it : g_simulator->customReplicas) {
KeyRangeRef replicaRange(std::get<0>(it), std::get<1>(it));
if (range.intersects(replicaRange)) {
TraceEvent("ConsistencyCheck_CheckCustomReplica")
.detail("ShardBegin", printable(range.begin))
.detail("ShardEnd", printable(range.end))
.detail("SourceTeamSize", sourceStorageServers.size())
.detail("DestServerSize", destStorageServers.size())
.detail("ConfigStorageTeamSize", configuration.storageTeamSize)
.detail("CustomBegin", std::get<0>(it))
.detail("CustomEnd", std::get<1>(it))
.detail("CustomReplicas", std::get<2>(it))
.detail("UsableRegions", configuration.usableRegions)
.detail("First", firstClient)
.detail("Perform", performQuiescentChecks);
if (!replicaRange.contains(range)) {
testFailure("Custom shard boundary violated", performQuiescentChecks, success, failureIsError);
return Void();
}
customReplication = std::max(customReplication, std::get<2>(it));
}
}
}
// In a quiescent database, check that the team size is the same as the desired team size
// FIXME: when usable_regions=2, we need to determine how many storage servers are alive in each DC
if (firstClient && performQuiescentChecks &&
sourceStorageServers.size() != configuration.usableRegions * configuration.storageTeamSize) {
((configuration.usableRegions == 1 &&
sourceStorageServers.size() != std::min(ssCount, customReplication)) ||
sourceStorageServers.size() < configuration.usableRegions * configuration.storageTeamSize ||
sourceStorageServers.size() > configuration.usableRegions * customReplication)) {
TraceEvent("ConsistencyCheck_InvalidTeamSize")
.detail("ShardBegin", printable(range.begin))
.detail("ShardEnd", printable(range.end))
.detail("SourceTeamSize", sourceStorageServers.size())
.detail("DestServerSize", destStorageServers.size())
.detail("ConfigStorageTeamSize", configuration.storageTeamSize)
.detail("UsableRegions", configuration.usableRegions);
.detail("CustomReplicas", customReplication)
.detail("UsableRegions", configuration.usableRegions)
.detail("SSCount", ssCount);
// Record the server reponsible for the problematic shards
int k = 0;
for (auto& id : sourceStorageServers) {
TraceEvent("IncorrectSizeTeamInfo").detail("ServerUID", id).detail("TeamIndex", k++);
}
testFailure("Invalid team size", performQuiescentChecks, failureIsError);
return false;
testFailure("Invalid team size", performQuiescentChecks, success, failureIsError);
return Void();
}
state std::vector<UID> storageServers = (isRelocating) ? destStorageServers : sourceStorageServers;
@ -476,8 +522,10 @@ ACTOR Future<bool> checkDataConsistency(Database cx,
if (serverListValues[s].present())
storageServerInterfaces.push_back(decodeServerListValue(serverListValues[s].get()));
else if (performQuiescentChecks)
testFailure(
"/FF/serverList changing in a quiescent database", performQuiescentChecks, failureIsError);
testFailure("/FF/serverList changing in a quiescent database",
performQuiescentChecks,
success,
failureIsError);
}
break;
@ -508,7 +556,7 @@ ACTOR Future<bool> checkDataConsistency(Database cx,
if (firstClient) {
// If there was an error retrieving shard estimated size
if (performQuiescentChecks && estimatedBytes.size() == 0)
testFailure("Error fetching storage metrics", performQuiescentChecks, failureIsError);
testFailure("Error fetching storage metrics", performQuiescentChecks, success, failureIsError);
// If running a distributed test, storage server size is an accumulation of shard estimates
else if (distributed && firstClient)
@ -709,8 +757,7 @@ ACTOR Future<bool> checkDataConsistency(Database cx,
g_simulator->tssMode != ISimulator::TSSMode::EnabledDropMutations) ||
(!storageServerInterfaces[j].isTss() &&
!storageServerInterfaces[firstValidServer].isTss())) {
testFailure("Data inconsistent", performQuiescentChecks, true);
testResult = false;
testFailure("Data inconsistent", performQuiescentChecks, success, true);
}
}
}
@ -735,12 +782,14 @@ ACTOR Future<bool> checkDataConsistency(Database cx,
if (e.code() == error_code_request_maybe_delivered) {
// SS in the team may be removed and we get this error.
return false;
*success = false;
return Void();
}
// All shards should be available in quiscence
if (performQuiescentChecks && !storageServerInterfaces[j].isTss()) {
testFailure("Storage server unavailable", performQuiescentChecks, failureIsError);
return false;
testFailure(
"Storage server unavailable", performQuiescentChecks, success, failureIsError);
return Void();
}
}
}
@ -878,6 +927,7 @@ ACTOR Future<bool> checkDataConsistency(Database cx,
if (!storageServerInterfaces[j].isTss()) {
testFailure("Storage servers had incorrect sampled estimate",
performQuiescentChecks,
success,
failureIsError);
}
@ -918,6 +968,7 @@ ACTOR Future<bool> checkDataConsistency(Database cx,
testFailure(format("Shard size is more than %f std dev from estimate", failErrorNumStdDev),
performQuiescentChecks,
success,
failureIsError);
}
@ -940,8 +991,9 @@ ACTOR Future<bool> checkDataConsistency(Database cx,
testFailure(format("Shard size in quiescent database is too %s",
(sampledBytes < shardBounds.min.bytes) ? "small" : "large"),
performQuiescentChecks,
success,
failureIsError);
return false;
return Void();
}
}
@ -954,7 +1006,7 @@ ACTOR Future<bool> checkDataConsistency(Database cx,
}
*bytesReadInPrevRound = bytesReadInthisRound;
return testResult;
return Void();
}
ACTOR Future<Void> runDataValidationCheck(ConsistencyScanData* self) {
@ -969,38 +1021,41 @@ ACTOR Future<Void> runDataValidationCheck(ConsistencyScanData* self) {
// Get a list of key servers; verify that the TLogs and master all agree about who the key servers are
state Promise<std::vector<std::pair<KeyRange, std::vector<StorageServerInterface>>>> keyServerPromise;
state std::map<UID, StorageServerInterface> tssMapping;
bool keyServerResult = wait(getKeyServers(self->db, keyServerPromise, keyServersKeys, false));
bool keyServerResult =
wait(getKeyServers(self->db, keyServerPromise, keyServersKeys, false, false, &self->success));
if (keyServerResult) {
state std::vector<std::pair<KeyRange, std::vector<StorageServerInterface>>> keyServers =
keyServerPromise.getFuture().get();
// Get the locations of all the shards in the database
state Promise<Standalone<VectorRef<KeyValueRef>>> keyLocationPromise;
bool keyLocationResult = wait(getKeyLocations(self->db, keyServers, keyLocationPromise, false));
bool keyLocationResult =
wait(getKeyLocations(self->db, keyServers, keyLocationPromise, false, &self->success));
if (keyLocationResult) {
state Standalone<VectorRef<KeyValueRef>> keyLocations = keyLocationPromise.getFuture().get();
// Check that each shard has the same data on all storage servers that it resides on
wait(::success(checkDataConsistency(self->db,
keyLocations,
self->configuration,
tssMapping,
false /* quiescentCheck */,
false /* tssCheck */,
true /* firstClient */,
false /* failureIsError */,
0 /* clientId */,
1 /* clientCount */,
false /* distributed */,
false /* shuffleShards */,
1 /* shardSampleFactor */,
deterministicRandom()->randomInt64(0, 10000000),
self->finishedRounds /* repetitions */,
&(self->bytesReadInPrevRound),
self->restart,
self->maxRate,
self->targetInterval,
self->progressKey)));
wait(checkDataConsistency(self->db,
keyLocations,
self->configuration,
tssMapping,
false /* quiescentCheck */,
false /* tssCheck */,
true /* firstClient */,
false /* failureIsError */,
0 /* clientId */,
1 /* clientCount */,
false /* distributed */,
false /* shuffleShards */,
1 /* shardSampleFactor */,
deterministicRandom()->randomInt64(0, 10000000),
self->finishedRounds /* repetitions */,
&(self->bytesReadInPrevRound),
self->restart,
self->maxRate,
self->targetInterval,
self->progressKey,
&self->success));
}
}
} catch (Error& e) {
@ -1072,7 +1127,6 @@ ACTOR Future<Void> watchConsistencyScanInfoKey(ConsistencyScanData* self) {
ACTOR Future<Void> consistencyScan(ConsistencyScanInterface csInterf, Reference<AsyncVar<ServerDBInfo> const> dbInfo) {
state ConsistencyScanData self(csInterf.id(),
openDBOnServer(dbInfo, TaskPriority::DefaultEndpoint, LockAware::True));
state Promise<Void> err;
state Future<Void> collection = actorCollection(self.addActor.getFuture());
state ConsistencyScanInfo csInfo = ConsistencyScanInfo();
@ -1115,15 +1169,20 @@ ACTOR Future<Void> consistencyScan(ConsistencyScanInterface csInterf, Reference<
try {
loop choose {
when(wait(runDataValidationCheck(&self))) {
TraceEvent("ConsistencyScan_Done", csInterf.id()).log();
return Void();
if (self.success) {
TraceEvent("ConsistencyScan_Done", csInterf.id()).log();
return Void();
} else {
self.success = true;
TraceEvent("ConsistencyScan_Failed", csInterf.id()).log();
wait(delay(1.0));
}
}
when(HaltConsistencyScanRequest req = waitNext(csInterf.haltConsistencyScan.getFuture())) {
req.reply.send(Void());
TraceEvent("ConsistencyScan_Halted", csInterf.id()).detail("ReqID", req.requesterID);
break;
}
when(wait(err.getFuture())) {}
when(wait(collection)) {
ASSERT(false);
throw internal_error();

File diff suppressed because it is too large Load Diff

View File

@ -26,13 +26,12 @@
#include "fdbserver/DDSharedContext.h"
#include "fdbserver/TenantCache.h"
#include "fdbserver/Knobs.h"
#include "fdbserver/workloads/workloads.actor.h"
#include "fdbclient/DatabaseContext.h"
#include "flow/ActorCollection.h"
#include "flow/Arena.h"
#include "flow/CodeProbe.h"
#include "flow/FastRef.h"
#include "flow/Trace.h"
#include "fdbserver/DDShardTracker.h"
#include "flow/actorcompiler.h" // This must be the last #include.
// The used bandwidth of a shard. The higher the value is, the busier the shard is.
@ -75,91 +74,6 @@ ACTOR Future<Void> updateMaxShardSize(Reference<AsyncVar<int64_t>> dbSizeEstimat
}
}
struct DataDistributionTracker : public IDDShardTracker {
Reference<IDDTxnProcessor> db;
UID distributorId;
// At now, the lifetime of shards is guaranteed longer than DataDistributionTracker.
KeyRangeMap<ShardTrackedData>* shards = nullptr;
ActorCollection sizeChanges;
int64_t systemSizeEstimate = 0;
Reference<AsyncVar<int64_t>> dbSizeEstimate;
Reference<AsyncVar<Optional<int64_t>>> maxShardSize;
Future<Void> maxShardSizeUpdater;
// CapacityTracker
PromiseStream<RelocateShard> output;
Reference<ShardsAffectedByTeamFailure> shardsAffectedByTeamFailure;
// PhysicalShard Tracker
Reference<PhysicalShardCollection> physicalShardCollection;
Promise<Void> readyToStart;
Reference<AsyncVar<bool>> anyZeroHealthyTeams;
// Read hot detection
PromiseStream<KeyRange> readHotShard;
// The reference to trackerCancelled must be extracted by actors,
// because by the time (trackerCancelled == true) this memory cannot
// be accessed
bool* trackerCancelled = nullptr;
// This class extracts the trackerCancelled reference from a DataDistributionTracker object
// Because some actors spawned by the dataDistributionTracker outlive the DataDistributionTracker
// object, we must guard against memory errors by using a GetTracker functor to access
// the DataDistributionTracker object.
class SafeAccessor {
bool const& trackerCancelled;
DataDistributionTracker& tracker;
public:
SafeAccessor(DataDistributionTracker* tracker)
: trackerCancelled(*tracker->trackerCancelled), tracker(*tracker) {
ASSERT(!trackerCancelled);
}
DataDistributionTracker* operator()() {
if (trackerCancelled) {
CODE_PROBE(true, "Trying to access DataDistributionTracker after tracker has been cancelled");
throw dd_tracker_cancelled();
}
return &tracker;
}
};
Optional<Reference<TenantCache>> ddTenantCache;
DataDistributionTracker() = default;
DataDistributionTracker(Reference<IDDTxnProcessor> db,
UID distributorId,
Promise<Void> const& readyToStart,
PromiseStream<RelocateShard> const& output,
Reference<ShardsAffectedByTeamFailure> shardsAffectedByTeamFailure,
Reference<PhysicalShardCollection> physicalShardCollection,
Reference<AsyncVar<bool>> anyZeroHealthyTeams,
KeyRangeMap<ShardTrackedData>* shards,
bool* trackerCancelled,
Optional<Reference<TenantCache>> ddTenantCache)
: IDDShardTracker(), db(db), distributorId(distributorId), shards(shards), sizeChanges(false),
systemSizeEstimate(0), dbSizeEstimate(new AsyncVar<int64_t>()), maxShardSize(new AsyncVar<Optional<int64_t>>()),
output(output), shardsAffectedByTeamFailure(shardsAffectedByTeamFailure),
physicalShardCollection(physicalShardCollection), readyToStart(readyToStart),
anyZeroHealthyTeams(anyZeroHealthyTeams), trackerCancelled(trackerCancelled), ddTenantCache(ddTenantCache) {}
~DataDistributionTracker() override {
if (trackerCancelled) {
*trackerCancelled = true;
}
// Cancel all actors so they aren't waiting on sizeChanged broken promise
sizeChanges.clear(false);
}
double getAverageShardBytes() override { return maxShardSize->get().get() / 2.0; }
};
void restartShardTrackers(DataDistributionTracker* self,
KeyRangeRef keys,
Optional<ShardMetrics> startingMetrics = Optional<ShardMetrics>(),
@ -208,6 +122,10 @@ int64_t getMaxShardSize(double dbSizeEstimate) {
(int64_t)SERVER_KNOBS->MAX_SHARD_BYTES);
}
bool ddLargeTeamEnabled() {
return SERVER_KNOBS->DD_MAXIMUM_LARGE_TEAMS > 0 && !SERVER_KNOBS->SHARD_ENCODE_LOCATION_METADATA;
}
// Returns the shard size bounds as well as whether `keys` a read hot shard.
std::pair<ShardSizeBounds, bool> calculateShardSizeBounds(
const KeyRange& keys,
@ -550,7 +468,7 @@ void executeShardSplit(DataDistributionTracker* self,
}
}
self->sizeChanges.add(changeSizes(self, keys, shardSize->get().get().metrics.bytes));
self->actors.add(changeSizes(self, keys, shardSize->get().get().metrics.bytes));
}
struct RangeToSplit {
@ -937,12 +855,12 @@ ACTOR Future<Void> brokenPromiseToReady(Future<Void> f) {
}
static bool shardMergeFeasible(DataDistributionTracker* self, KeyRange const& keys, KeyRangeRef adjRange) {
bool honorTenantKeyspaceBoundaries = self->ddTenantCache.present();
if (!honorTenantKeyspaceBoundaries) {
if (!SERVER_KNOBS->DD_TENANT_AWARENESS_ENABLED) {
return true;
}
ASSERT(self->ddTenantCache.present());
Optional<Reference<TCTenantInfo>> tenantOwningRange = {};
Optional<Reference<TCTenantInfo>> tenantOwningAdjRange = {};
@ -962,6 +880,10 @@ static bool shardForwardMergeFeasible(DataDistributionTracker* self, KeyRange co
return false;
}
if (self->customReplication->rangeContaining(keys.begin).range().end < nextRange.end) {
return false;
}
return shardMergeFeasible(self, keys, nextRange);
}
@ -970,6 +892,10 @@ static bool shardBackwardMergeFeasible(DataDistributionTracker* self, KeyRange c
return false;
}
if (self->customReplication->rangeContaining(keys.begin).range().begin > prevRange.begin) {
return false;
}
return shardMergeFeasible(self, keys, prevRange);
}
@ -1276,10 +1202,26 @@ ACTOR Future<Void> trackInitialShards(DataDistributionTracker* self, Reference<I
// SOMEDAY: Figure out what this priority should actually be
wait(delay(0.0, TaskPriority::DataDistribution));
state std::vector<Key> customBoundaries;
for (auto& it : self->customReplication->ranges()) {
customBoundaries.push_back(it->range().begin);
}
state int s;
state int customBoundary = 0;
for (s = 0; s < initData->shards.size() - 1; s++) {
restartShardTrackers(
self, KeyRangeRef(initData->shards[s].key, initData->shards[s + 1].key), Optional<ShardMetrics>(), true);
Key beginKey = initData->shards[s].key;
Key endKey = initData->shards[s + 1].key;
while (customBoundary < customBoundaries.size() && customBoundaries[customBoundary] <= beginKey) {
customBoundary++;
}
while (customBoundary < customBoundaries.size() && customBoundaries[customBoundary] < endKey) {
restartShardTrackers(
self, KeyRangeRef(beginKey, customBoundaries[customBoundary]), Optional<ShardMetrics>(), true);
beginKey = customBoundaries[customBoundary];
customBoundary++;
}
restartShardTrackers(self, KeyRangeRef(beginKey, endKey), Optional<ShardMetrics>(), true);
wait(yield(TaskPriority::DataDistribution));
}
@ -1465,86 +1407,88 @@ ACTOR Future<Void> fetchShardMetricsList(DataDistributionTracker* self, GetMetri
return Void();
}
ACTOR Future<Void> dataDistributionTracker(Reference<InitialDataDistribution> initData,
Reference<IDDTxnProcessor> db,
PromiseStream<RelocateShard> output,
Reference<ShardsAffectedByTeamFailure> shardsAffectedByTeamFailure,
Reference<PhysicalShardCollection> physicalShardCollection,
PromiseStream<GetMetricsRequest> getShardMetrics,
FutureStream<GetTopKMetricsRequest> getTopKMetrics,
PromiseStream<GetMetricsListRequest> getShardMetricsList,
FutureStream<Promise<int64_t>> getAverageShardBytes,
Promise<Void> readyToStart,
Reference<AsyncVar<bool>> zeroHealthyTeams,
UID distributorId,
KeyRangeMap<ShardTrackedData>* shards,
bool* trackerCancelled,
Optional<Reference<TenantCache>> ddTenantCache) {
state DataDistributionTracker self(db,
distributorId,
readyToStart,
output,
shardsAffectedByTeamFailure,
physicalShardCollection,
zeroHealthyTeams,
shards,
trackerCancelled,
ddTenantCache);
state Future<Void> loggingTrigger = Void();
state Future<Void> readHotDetect = readHotDetector(&self);
state Reference<EventCacheHolder> ddTrackerStatsEventHolder = makeReference<EventCacheHolder>("DDTrackerStats");
try {
wait(trackInitialShards(&self, initData));
initData.clear(); // Release reference count.
DataDistributionTracker::DataDistributionTracker(DataDistributionTrackerInitParams const& params)
: IDDShardTracker(), db(params.db), distributorId(params.distributorId), shards(params.shards), actors(false),
systemSizeEstimate(0), dbSizeEstimate(new AsyncVar<int64_t>()), maxShardSize(new AsyncVar<Optional<int64_t>>()),
output(params.output), shardsAffectedByTeamFailure(params.shardsAffectedByTeamFailure),
physicalShardCollection(params.physicalShardCollection), readyToStart(params.readyToStart),
anyZeroHealthyTeams(params.anyZeroHealthyTeams), trackerCancelled(params.trackerCancelled),
ddTenantCache(params.ddTenantCache) {}
state PromiseStream<TenantCacheTenantCreated> tenantCreationSignal;
if (self.ddTenantCache.present()) {
tenantCreationSignal = self.ddTenantCache.get()->tenantCreationSignal;
}
loop choose {
when(Promise<int64_t> req = waitNext(getAverageShardBytes)) {
req.send(self.getAverageShardBytes());
}
when(wait(loggingTrigger)) {
TraceEvent("DDTrackerStats", self.distributorId)
.detail("Shards", self.shards->size())
.detail("TotalSizeBytes", self.dbSizeEstimate->get())
.detail("SystemSizeBytes", self.systemSizeEstimate)
.trackLatest(ddTrackerStatsEventHolder->trackingKey);
loggingTrigger = delay(SERVER_KNOBS->DATA_DISTRIBUTION_LOGGING_INTERVAL, TaskPriority::FlushTrace);
}
when(GetMetricsRequest req = waitNext(getShardMetrics.getFuture())) {
self.sizeChanges.add(fetchShardMetrics(&self, req));
}
when(GetTopKMetricsRequest req = waitNext(getTopKMetrics)) {
self.sizeChanges.add(fetchTopKShardMetrics(&self, req));
}
when(GetMetricsListRequest req = waitNext(getShardMetricsList.getFuture())) {
self.sizeChanges.add(fetchShardMetricsList(&self, req));
}
when(wait(self.sizeChanges.getResult())) {}
when(TenantCacheTenantCreated newTenant = waitNext(tenantCreationSignal.getFuture())) {
self.sizeChanges.add(tenantCreationHandling(&self, newTenant));
}
when(KeyRange req = waitNext(self.shardsAffectedByTeamFailure->restartShardTracker.getFuture())) {
restartShardTrackers(&self, req);
}
}
} catch (Error& e) {
TraceEvent(SevError, "DataDistributionTrackerError", self.distributorId).error(e);
throw e;
DataDistributionTracker::~DataDistributionTracker() {
if (trackerCancelled) {
*trackerCancelled = true;
}
// Cancel all actors so they aren't waiting on sizeChanged broken promise
actors.clear(false);
}
// Not used yet
ACTOR Future<Void> dataDistributionTracker(Reference<DDSharedContext> context,
Reference<InitialDataDistribution> initData,
Database cx,
KeyRangeMap<ShardTrackedData>* shards);
struct DataDistributionTrackerImpl {
ACTOR static Future<Void> run(DataDistributionTracker* self, Reference<InitialDataDistribution> initData) {
state Future<Void> loggingTrigger = Void();
state Future<Void> readHotDetect = readHotDetector(self);
state Reference<EventCacheHolder> ddTrackerStatsEventHolder = makeReference<EventCacheHolder>("DDTrackerStats");
try {
wait(trackInitialShards(self, initData));
initData.clear(); // Release reference count.
state PromiseStream<TenantCacheTenantCreated> tenantCreationSignal;
if (SERVER_KNOBS->DD_TENANT_AWARENESS_ENABLED) {
ASSERT(self->ddTenantCache.present());
tenantCreationSignal = self->ddTenantCache.get()->tenantCreationSignal;
}
loop choose {
when(Promise<int64_t> req = waitNext(self->averageShardBytes)) {
req.send(self->getAverageShardBytes());
}
when(wait(loggingTrigger)) {
TraceEvent("DDTrackerStats", self->distributorId)
.detail("Shards", self->shards->size())
.detail("TotalSizeBytes", self->dbSizeEstimate->get())
.detail("SystemSizeBytes", self->systemSizeEstimate)
.trackLatest(ddTrackerStatsEventHolder->trackingKey);
loggingTrigger = delay(SERVER_KNOBS->DATA_DISTRIBUTION_LOGGING_INTERVAL, TaskPriority::FlushTrace);
}
when(GetMetricsRequest req = waitNext(self->getShardMetrics)) {
self->actors.add(fetchShardMetrics(self, req));
}
when(GetTopKMetricsRequest req = waitNext(self->getTopKMetrics)) {
self->actors.add(fetchTopKShardMetrics(self, req));
}
when(GetMetricsListRequest req = waitNext(self->getShardMetricsList)) {
self->actors.add(fetchShardMetricsList(self, req));
}
when(wait(self->actors.getResult())) {}
when(TenantCacheTenantCreated newTenant = waitNext(tenantCreationSignal.getFuture())) {
self->actors.add(tenantCreationHandling(self, newTenant));
}
when(KeyRange req = waitNext(self->shardsAffectedByTeamFailure->restartShardTracker.getFuture())) {
restartShardTrackers(self, req);
}
}
} catch (Error& e) {
TraceEvent(SevError, "DataDistributionTrackerError", self->distributorId).error(e);
throw e;
}
}
};
Future<Void> DataDistributionTracker::run(Reference<DataDistributionTracker> self,
const Reference<InitialDataDistribution>& initData,
const FutureStream<GetMetricsRequest>& getShardMetrics,
const FutureStream<GetTopKMetricsRequest>& getTopKMetrics,
const FutureStream<GetMetricsListRequest>& getShardMetricsList,
const FutureStream<Promise<int64_t>>& getAverageShardBytes) {
self->getShardMetrics = getShardMetrics;
self->getTopKMetrics = getTopKMetrics;
self->getShardMetricsList = getShardMetricsList;
self->averageShardBytes = getAverageShardBytes;
self->customReplication = initData->customReplication;
return holdWhile(self, DataDistributionTrackerImpl::run(self.getPtr(), initData));
}
// Methods for PhysicalShardCollection
FDB_DEFINE_BOOLEAN_PARAM(InAnonymousPhysicalShard);

View File

@ -19,6 +19,7 @@
*/
#include "fdbserver/DDTeamCollection.h"
#include "fdbserver/ExclusionTracker.actor.h"
#include "flow/Trace.h"
#include "flow/actorcompiler.h" // This must be the last #include.
#include <climits>
@ -247,6 +248,51 @@ public:
Optional<Reference<IDataDistributionTeam>> bestOption;
std::vector<Reference<TCTeamInfo>> randomTeams;
if (ddLargeTeamEnabled() && req.keys.present()) {
int customReplicas = self->configuration.storageTeamSize;
for (auto& it : self->customReplication->intersectingRanges(req.keys.get())) {
customReplicas = std::max(customReplicas, it.value());
}
if (customReplicas > self->configuration.storageTeamSize) {
auto newTeam = self->buildLargeTeam(customReplicas);
if (newTeam) {
if (newTeam->size() < customReplicas) {
if (!self->firstLargeTeamFailure.present()) {
self->firstLargeTeamFailure = now();
}
if (now() - self->firstLargeTeamFailure.get() < SERVER_KNOBS->DD_LARGE_TEAM_DELAY) {
req.reply.send(std::make_pair(Optional<Reference<IDataDistributionTeam>>(), foundSrc));
return Void();
}
self->underReplication.insert(req.keys.get(), true);
} else {
self->firstLargeTeamFailure = Optional<double>();
}
TraceEvent("ReplicatingToLargeTeam", self->distributorId)
.detail("Team", newTeam->getDesc())
.detail("Healthy", newTeam->isHealthy())
.detail("DesiredReplicas", customReplicas)
.detail("UnderReplicated", newTeam->size() < customReplicas);
req.reply.send(std::make_pair(newTeam, foundSrc));
return Void();
} else {
if (!self->firstLargeTeamFailure.present()) {
self->firstLargeTeamFailure = now();
}
if (now() - self->firstLargeTeamFailure.get() < SERVER_KNOBS->DD_LARGE_TEAM_DELAY) {
req.reply.send(std::make_pair(Optional<Reference<IDataDistributionTeam>>(), foundSrc));
return Void();
}
TraceEvent(SevWarnAlways, "LargeTeamNotFound", self->distributorId)
.suppressFor(1.0)
.detail("Replicas", customReplicas)
.detail("StorageTeamSize", self->configuration.storageTeamSize)
.detail("LargeTeamDiff", now() - self->firstLargeTeamFailure.get());
self->underReplication.insert(req.keys.get(), true);
}
}
}
// Note: this block does not apply any filters from the request
if (!req.wantsNewServers) {
auto healthyTeam = self->findTeamFromServers(req.completeSources, /* wantHealthy=*/true);
@ -378,18 +424,20 @@ public:
state std::vector<UID> serverIds;
state Reference<LocalitySet> tempSet = Reference<LocalitySet>(new LocalityMap<UID>());
state LocalityMap<UID>* tempMap = (LocalityMap<UID>*)tempSet.getPtr();
state std::vector<Reference<TCTeamInfo>> largeOrBadTeams = self->badTeams;
largeOrBadTeams.insert(largeOrBadTeams.end(), self->largeTeams.begin(), self->largeTeams.end());
for (; idx < self->badTeams.size(); idx++) {
for (; idx < largeOrBadTeams.size(); idx++) {
servers.clear();
for (const auto& server : self->badTeams[idx]->getServers()) {
for (const auto& server : largeOrBadTeams[idx]->getServers()) {
if (server->isInDesiredDC() && !self->server_status.get(server->getId()).isUnhealthy()) {
servers.push_back(server);
}
}
// For the bad team that is too big (too many servers), we will try to find a subset of servers in the team
// to construct a new healthy team, so that moving data to the new healthy team will not
// cause too much data movement overhead
// For the bad team that is too big (too many servers), we will try to find a subset of servers in the
// team to construct a new healthy team, so that moving data to the new healthy team will not cause too
// much data movement overhead
// FIXME: This code logic can be simplified.
if (servers.size() >= self->configuration.storageTeamSize) {
bool foundTeam = false;
@ -464,6 +512,8 @@ public:
ACTOR static Future<Void> init(DDTeamCollection* self,
Reference<InitialDataDistribution> initTeams,
const DDEnabledState* ddEnabledState) {
self->customReplication = initTeams->customReplication;
self->healthyZone.set(initTeams->initHealthyZoneValue);
// SOMEDAY: If some servers have teams and not others (or some servers have more data than others) and there is
// an address/locality collision, should we preferentially mark the least used server as undesirable?
@ -717,7 +767,7 @@ public:
// Failed server should not trigger DD if SS failures are set to be ignored
if (!badTeam && self->healthyZone.get().present() &&
(self->healthyZone.get().get() == ignoreSSFailuresZoneString)) {
ASSERT_WE_THINK(serversLeft == self->configuration.storageTeamSize);
ASSERT_WE_THINK(serversLeft == team->size());
}
if (!self->initialFailureReactionDelay.isReady()) {
@ -725,7 +775,7 @@ public:
}
change.push_back(self->zeroHealthyTeams->onChange());
bool healthy = !badTeam && !anyUndesired && serversLeft == self->configuration.storageTeamSize;
bool healthy = !badTeam && !anyUndesired && serversLeft == team->size();
team->setHealthy(healthy); // Unhealthy teams won't be chosen by bestTeam
bool optimal = team->isOptimal() && healthy;
bool containsFailed = self->teamContainsFailedServer(team);
@ -820,7 +870,7 @@ public:
state int lastPriority = team->getPriority();
if (team->size() == 0) {
team->setPriority(SERVER_KNOBS->PRIORITY_POPULATE_REGION);
} else if (serversLeft < self->configuration.storageTeamSize) {
} else if (serversLeft < team->size()) {
if (serversLeft == 0)
team->setPriority(SERVER_KNOBS->PRIORITY_TEAM_0_LEFT);
else if (serversLeft == 1)
@ -1828,100 +1878,47 @@ public:
}
}
ACTOR static Future<Void> trackExcludedServers(DDTeamCollection* self) {
// Fetch the list of excluded servers
state ReadYourWritesTransaction tr(self->dbContext());
ACTOR static Future<Void> fixUnderReplicationLoop(DDTeamCollection* self) {
loop {
try {
tr.setOption(FDBTransactionOptions::ACCESS_SYSTEM_KEYS);
tr.setOption(FDBTransactionOptions::PRIORITY_SYSTEM_IMMEDIATE);
state Future<RangeResult> fresultsExclude = tr.getRange(excludedServersKeys, CLIENT_KNOBS->TOO_MANY);
state Future<RangeResult> fresultsFailed = tr.getRange(failedServersKeys, CLIENT_KNOBS->TOO_MANY);
state Future<RangeResult> flocalitiesExclude =
tr.getRange(excludedLocalityKeys, CLIENT_KNOBS->TOO_MANY);
state Future<RangeResult> flocalitiesFailed = tr.getRange(failedLocalityKeys, CLIENT_KNOBS->TOO_MANY);
state Future<std::vector<ProcessData>> fworkers = self->db->getWorkers();
wait(success(fresultsExclude) && success(fresultsFailed) && success(flocalitiesExclude) &&
success(flocalitiesFailed));
wait(delay(SERVER_KNOBS->DD_FIX_WRONG_REPLICAS_DELAY));
self->cleanupLargeTeams();
self->fixUnderReplication();
}
}
state RangeResult excludedResults = fresultsExclude.get();
ASSERT(!excludedResults.more && excludedResults.size() < CLIENT_KNOBS->TOO_MANY);
ACTOR static Future<Void> trackExcludedServers(DDTeamCollection* self) {
state ExclusionTracker exclusionTracker(self->dbContext());
loop {
// wait for new set of excluded servers
wait(exclusionTracker.changed.onTrigger());
state RangeResult failedResults = fresultsFailed.get();
ASSERT(!failedResults.more && failedResults.size() < CLIENT_KNOBS->TOO_MANY);
state RangeResult excludedLocalityResults = flocalitiesExclude.get();
ASSERT(!excludedLocalityResults.more && excludedLocalityResults.size() < CLIENT_KNOBS->TOO_MANY);
state RangeResult failedLocalityResults = flocalitiesFailed.get();
ASSERT(!failedLocalityResults.more && failedLocalityResults.size() < CLIENT_KNOBS->TOO_MANY);
state std::set<AddressExclusion> excluded;
state std::set<AddressExclusion> failed;
for (const auto& r : excludedResults) {
AddressExclusion addr = decodeExcludedServersKey(r.key);
if (addr.isValid()) {
excluded.insert(addr);
}
// Reset and reassign self->excludedServers based on excluded, but we only
// want to trigger entries that are different
// Do not retrigger and double-overwrite failed or wiggling servers
auto old = self->excludedServers.getKeys();
for (const auto& o : old) {
if (!exclusionTracker.excluded.count(o) && !exclusionTracker.failed.count(o) &&
!(self->excludedServers.count(o) &&
self->excludedServers.get(o) == DDTeamCollection::Status::WIGGLING)) {
self->excludedServers.set(o, DDTeamCollection::Status::NONE);
}
for (const auto& r : failedResults) {
AddressExclusion addr = decodeFailedServersKey(r.key);
if (addr.isValid()) {
failed.insert(addr);
}
}
wait(success(fworkers));
std::vector<ProcessData> workers = fworkers.get();
for (const auto& r : excludedLocalityResults) {
std::string locality = decodeExcludedLocalityKey(r.key);
std::set<AddressExclusion> localityExcludedAddresses = getAddressesByLocality(workers, locality);
excluded.insert(localityExcludedAddresses.begin(), localityExcludedAddresses.end());
}
for (const auto& r : failedLocalityResults) {
std::string locality = decodeFailedLocalityKey(r.key);
std::set<AddressExclusion> localityFailedAddresses = getAddressesByLocality(workers, locality);
failed.insert(localityFailedAddresses.begin(), localityFailedAddresses.end());
}
// Reset and reassign self->excludedServers based on excluded, but we only
// want to trigger entries that are different
// Do not retrigger and double-overwrite failed or wiggling servers
auto old = self->excludedServers.getKeys();
for (const auto& o : old) {
if (!excluded.count(o) && !failed.count(o) &&
!(self->excludedServers.count(o) &&
self->excludedServers.get(o) == DDTeamCollection::Status::WIGGLING)) {
self->excludedServers.set(o, DDTeamCollection::Status::NONE);
}
}
for (const auto& n : excluded) {
if (!failed.count(n)) {
self->excludedServers.set(n, DDTeamCollection::Status::EXCLUDED);
}
}
for (const auto& f : failed) {
self->excludedServers.set(f, DDTeamCollection::Status::FAILED);
}
TraceEvent("DDExcludedServersChanged", self->distributorId)
.detail("AddressesExcluded", excludedResults.size())
.detail("AddressesFailed", failedResults.size())
.detail("LocalitiesExcluded", excludedLocalityResults.size())
.detail("Primary", self->isPrimary())
.detail("LocalitiesFailed", failedLocalityResults.size());
self->restartRecruiting.trigger();
state Future<Void> watchFuture =
tr.watch(excludedServersVersionKey) || tr.watch(failedServersVersionKey) ||
tr.watch(excludedLocalityVersionKey) || tr.watch(failedLocalityVersionKey);
wait(tr.commit());
wait(watchFuture);
tr.reset();
} catch (Error& e) {
wait(tr.onError(e));
}
for (const auto& n : exclusionTracker.excluded) {
if (!exclusionTracker.failed.count(n)) {
self->excludedServers.set(n, DDTeamCollection::Status::EXCLUDED);
}
}
for (const auto& f : exclusionTracker.failed) {
self->excludedServers.set(f, DDTeamCollection::Status::FAILED);
}
TraceEvent("DDExcludedServersChanged", self->distributorId)
.detail("AddressesExcluded", exclusionTracker.excluded.size())
.detail("AddressesFailed", exclusionTracker.failed.size())
.detail("Primary", self->isPrimary());
self->restartRecruiting.trigger();
}
}
@ -2942,6 +2939,9 @@ public:
self->addActor.send(self->machineTeamRemover());
self->addActor.send(self->serverTeamRemover());
if (ddLargeTeamEnabled()) {
self->addActor.send(self->fixUnderReplicationLoop());
}
if (self->wrongStoreTypeRemover.isReady()) {
self->wrongStoreTypeRemover = self->removeWrongStoreType();
@ -3251,7 +3251,6 @@ public:
// }
}
}
}; // class DDTeamCollectionImpl
int32_t DDTeamCollection::getTargetTSSInDC() const {
@ -3519,6 +3518,65 @@ Future<Void> DDTeamCollection::serverTeamRemover() {
return DDTeamCollectionImpl::serverTeamRemover(this);
}
Future<Void> DDTeamCollection::fixUnderReplicationLoop() {
return DDTeamCollectionImpl::fixUnderReplicationLoop(this);
}
void DDTeamCollection::fixUnderReplication() {
int maxTeamSize = maxLargeTeamSize();
int checkCount = 0;
for (auto& it : underReplication.ranges()) {
if (!it.value()) {
// The key range is not under-replicated
continue;
}
for (auto& r : shardsAffectedByTeamFailure->intersectingRanges(it.range())) {
if (++checkCount > SERVER_KNOBS->DD_MAXIMUM_LARGE_TEAM_CLEANUP) {
return;
}
auto& teams = r.value();
if (!teams.second.empty()) {
// The key range is currently being moved
continue;
}
int customReplicas = configuration.storageTeamSize;
for (auto& c : customReplication->intersectingRanges(r.range())) {
customReplicas = std::max(customReplicas, c.value());
}
int currentSize = 0;
for (auto& c : teams.first) {
if (c.primary == primary) {
currentSize = c.servers.size();
}
}
if (currentSize < customReplicas) {
// check if a larger team exists
if (maxTeamSize > currentSize) {
TraceEvent("FixUnderReplication", distributorId)
.suppressFor(1.0)
.detail("MaxTeamSize", maxTeamSize)
.detail("Current", currentSize)
.detail("Desired", customReplicas);
RelocateShard rs(r.range(),
SERVER_KNOBS->PRIORITY_TEAM_UNHEALTHY,
RelocateReason::OTHER,
deterministicRandom()->randomUniqueID());
output.send(rs);
underReplication.insert(r.range(), false);
return;
}
} else {
underReplication.insert(r.range(), false);
return;
}
}
}
}
Future<Void> DDTeamCollection::trackExcludedServers() {
ASSERT(!db->isMocked());
return DDTeamCollectionImpl::trackExcludedServers(this);
@ -3660,8 +3718,8 @@ DDTeamCollection::DDTeamCollection(DDTeamCollectionInitParams const& params)
teamCollectionInfoEventHolder(makeReference<EventCacheHolder>("TeamCollectionInfo")),
storageServerRecruitmentEventHolder(
makeReference<EventCacheHolder>("StorageServerRecruitment_" + params.distributorId.toString())),
primary(params.primary), distributorId(params.distributorId), configuration(params.configuration),
storageServerSet(new LocalityMap<UID>()) {
primary(params.primary), distributorId(params.distributorId), underReplication(false),
configuration(params.configuration), storageServerSet(new LocalityMap<UID>()) {
if (!primary || configuration.usableRegions == 1) {
TraceEvent("DDTrackerStarting", distributorId)
@ -3706,6 +3764,11 @@ DDTeamCollection::~DDTeamCollection() {
for (auto& badTeam : badTeams) {
badTeam->tracker.cancel();
}
for (auto& largeTeam : largeTeams) {
largeTeam->tracker.cancel();
}
// TraceEvent("DDTeamCollectionDestructed", distributorId)
// .detail("Primary", primary)
// .detail("BadTeamTrackerDestroyed", badTeams.size());
@ -3914,6 +3977,166 @@ int DDTeamCollection::overlappingMachineMembers(std::vector<Standalone<StringRef
return maxMatchingServers;
}
void DDTeamCollection::cleanupLargeTeams() {
for (int t = 0; t < std::min<int>(largeTeams.size(), SERVER_KNOBS->DD_MAXIMUM_LARGE_TEAM_CLEANUP); t++) {
if (!shardsAffectedByTeamFailure->hasShards(
ShardsAffectedByTeamFailure::Team(largeTeams[t]->getServerIDs(), primary))) {
largeTeams[t]->tracker.cancel();
largeTeams[t--] = largeTeams.back();
largeTeams.pop_back();
}
}
}
int DDTeamCollection::maxLargeTeamSize() const {
std::vector<Reference<TCServerInfo>> healthy;
for (auto& [serverID, server] : server_info) {
if (!server_status.get(serverID).isUnhealthy()) {
healthy.push_back(server);
}
}
if (!satisfiesPolicy(healthy)) {
return -1;
}
return healthy.size();
}
struct ServerPriority {
int healthyShards = 0;
int unhealthyShards = 0;
int64_t loadBytes = std::numeric_limits<int64_t>::max();
UID id;
Reference<TCServerInfo> info;
ServerPriority() {}
ServerPriority(int healthyShards, int unhealthyShards, int64_t loadBytes, UID id, Reference<TCServerInfo> info)
: healthyShards(healthyShards), unhealthyShards(unhealthyShards), loadBytes(loadBytes), id(id), info(info) {}
bool operator<(ServerPriority const& r) const {
if (healthyShards != r.healthyShards) {
return healthyShards < r.healthyShards;
} else if (unhealthyShards != r.unhealthyShards) {
return unhealthyShards < r.unhealthyShards;
} else if (loadBytes != r.loadBytes) {
return loadBytes < r.loadBytes;
} else {
return id < r.id;
}
}
};
Reference<TCTeamInfo> DDTeamCollection::buildLargeTeam(int teamSize) {
cleanupLargeTeams();
if (largeTeams.size() >= SERVER_KNOBS->DD_MAXIMUM_LARGE_TEAMS) {
TraceEvent(SevWarnAlways, "TooManyLargeTeams", distributorId)
.suppressFor(1.0)
.detail("TeamCount", largeTeams.size());
return Reference<TCTeamInfo>();
}
std::map<UID, ServerPriority> server_priority;
for (auto& [serverID, server] : server_info) {
if (!server_status.get(serverID).isUnhealthy()) {
server_priority[serverID] =
ServerPriority(0,
0,
server->metricsPresent() ? server->loadBytes() : std::numeric_limits<int64_t>::max(),
serverID,
server);
}
}
for (auto& team : largeTeams) {
const auto servers = team->getServerIDs();
const int shardCount =
shardsAffectedByTeamFailure->getNumberOfShards(ShardsAffectedByTeamFailure::Team(servers, primary));
if (team->isHealthy()) {
for (auto& it : servers) {
auto f = server_priority.find(it);
if (f != server_priority.end()) {
f->second.healthyShards += shardCount;
}
}
} else {
for (auto& it : servers) {
auto f = server_priority.find(it);
if (f != server_priority.end()) {
f->second.unhealthyShards += shardCount;
}
}
}
}
// The set of all healthy servers sorted so the most desirable option is first in the list
std::vector<ServerPriority> sortedServers;
sortedServers.reserve(server_priority.size());
for (auto& it : server_priority) {
sortedServers.emplace_back(it.second);
}
std::sort(sortedServers.begin(), sortedServers.end());
std::set<UID> serverIds;
std::vector<Reference<TCServerInfo>> candidateTeam;
for (int i = 0; i < sortedServers.size(); i++) {
if (candidateTeam.size() >= teamSize && satisfiesPolicy(candidateTeam)) {
break;
}
candidateTeam.push_back(sortedServers[i].info);
}
if (!satisfiesPolicy(candidateTeam)) {
TraceEvent(SevWarnAlways, "TooFewServersForLargeTeam", distributorId)
.suppressFor(1.0)
.detail("TeamSize", candidateTeam.size())
.detail("Desired", teamSize)
.detail("SatisfiesPolicy", satisfiesPolicy(candidateTeam));
return Reference<TCTeamInfo>();
} else if (candidateTeam.size() > teamSize) {
Reference<LocalitySet> tempSet = Reference<LocalitySet>(new LocalityMap<UID>());
LocalityMap<UID>* tempMap = (LocalityMap<UID>*)tempSet.getPtr();
tempSet->clear();
for (auto& it : candidateTeam) {
tempMap->add(it->getLastKnownInterface().locality, &it->getId());
}
std::vector<LocalityEntry> resultEntries, forcedEntries;
bool result = tempSet->selectReplicas(configuration.storagePolicy, forcedEntries, resultEntries);
ASSERT(result && resultEntries.size() == configuration.storageTeamSize);
for (auto& it : resultEntries) {
serverIds.insert(*tempMap->getObject(it));
}
for (int i = 0; i < sortedServers.size(); i++) {
if (serverIds.size() >= teamSize) {
break;
}
serverIds.insert(sortedServers[i].id);
}
} else {
for (auto& it : candidateTeam) {
serverIds.insert(it->getId());
}
}
const std::vector<UID> serverIDVector(serverIds.begin(), serverIds.end());
for (int t = 0; t < largeTeams.size(); t++) {
if (largeTeams[t]->getServerIDs() == serverIDVector) {
return largeTeams[t];
}
}
candidateTeam.clear();
for (auto& it : serverIds) {
candidateTeam.push_back(server_info[it]);
}
Optional<Reference<TCTenantInfo>> no_tenant = {};
auto teamInfo = makeReference<TCTeamInfo>(candidateTeam, no_tenant);
teamInfo->tracker = teamTracker(teamInfo, IsBadTeam::False, IsRedundantTeam::False);
largeTeams.push_back(teamInfo);
return teamInfo;
}
void DDTeamCollection::addTeam(const std::vector<Reference<TCServerInfo>>& newTeamServers,
IsInitialTeam isInitialTeam,
IsRedundantTeam redundantTeam) {
@ -3921,8 +4144,8 @@ void DDTeamCollection::addTeam(const std::vector<Reference<TCServerInfo>>& newTe
auto teamInfo = makeReference<TCTeamInfo>(newTeamServers, no_tenant);
// Move satisfiesPolicy to the end for performance benefit
auto badTeam = IsBadTeam{ redundantTeam || teamInfo->size() != configuration.storageTeamSize ||
!satisfiesPolicy(teamInfo->getServers()) };
auto badTeam = IsBadTeam{ redundantTeam || !satisfiesPolicy(teamInfo->getServers()) ||
(!ddLargeTeamEnabled() && teamInfo->size() != configuration.storageTeamSize) };
teamInfo->tracker = teamTracker(teamInfo, badTeam, redundantTeam);
// ASSERT( teamInfo->serverIDs.size() > 0 ); //team can be empty at DB initialization
@ -3931,6 +4154,11 @@ void DDTeamCollection::addTeam(const std::vector<Reference<TCServerInfo>>& newTe
return;
}
if (teamInfo->size() > configuration.storageTeamSize) {
largeTeams.push_back(teamInfo);
return;
}
// For a good team, we add it to teams and create machine team for it when necessary
teams.push_back(teamInfo);
for (auto& server : newTeamServers) {
@ -5052,6 +5280,14 @@ void DDTeamCollection::removeServer(UID removedServer) {
}
}
for (int t = 0; t < largeTeams.size(); t++) {
if (std::count(largeTeams[t]->getServerIDs().begin(), largeTeams[t]->getServerIDs().end(), removedServer)) {
largeTeams[t]->tracker.cancel();
largeTeams[t--] = largeTeams.back();
largeTeams.pop_back();
}
}
// Step: Remove machine info related to removedServer
// Remove the server from its machine
Reference<TCMachineInfo> removedMachineInfo = removedServerInfo->machine;

View File

@ -261,6 +261,12 @@ class DDTxnProcessorImpl {
tss_servers.clear();
team_cache.clear();
succeeded = false;
result->customReplication->insert(allKeys, -1);
if (g_network->isSimulated() && ddLargeTeamEnabled()) {
for (auto& it : g_simulator->customReplicas) {
result->customReplication->insert(KeyRangeRef(std::get<0>(it), std::get<1>(it)), std::get<2>(it));
}
}
try {
// Read healthyZone value which is later used to determine on/off of failure triggered DD
tr.setOption(FDBTransactionOptions::READ_SYSTEM_KEYS);

View File

@ -403,6 +403,7 @@ public:
.detail("UnhealthyRelocations", 0)
.detail("HighestPriority", 0)
.detail("BytesWritten", 0)
.detail("BytesWrittenAverageRate", 0)
.detail("PriorityRecoverMove", 0)
.detail("PriorityRebalanceUnderutilizedTeam", 0)
.detail("PriorityRebalannceOverutilizedTeam", 0)
@ -451,41 +452,79 @@ public:
}
}
state std::vector<Key> customBoundaries;
for (auto& it : self->initData->customReplication->ranges()) {
customBoundaries.push_back(it->range().begin);
TraceEvent(SevDebug, "DDInitCustomReplicas", self->ddId)
.detail("Range", it->range())
.detail("Replication", it->value());
}
state int shard = 0;
state int customBoundary = 0;
for (; shard < self->initData->shards.size() - 1; shard++) {
const DDShardInfo& iShard = self->initData->shards[shard];
KeyRangeRef keys = KeyRangeRef(iShard.key, self->initData->shards[shard + 1].key);
std::vector<KeyRangeRef> ranges;
Key beginKey = iShard.key;
Key endKey = self->initData->shards[shard + 1].key;
while (customBoundary < customBoundaries.size() && customBoundaries[customBoundary] <= beginKey) {
customBoundary++;
}
while (customBoundary < customBoundaries.size() && customBoundaries[customBoundary] < endKey) {
ranges.push_back(KeyRangeRef(beginKey, customBoundaries[customBoundary]));
beginKey = customBoundaries[customBoundary];
customBoundary++;
}
ranges.push_back(KeyRangeRef(beginKey, endKey));
self->shardsAffectedByTeamFailure->defineShard(keys);
std::vector<ShardsAffectedByTeamFailure::Team> teams;
teams.push_back(ShardsAffectedByTeamFailure::Team(iShard.primarySrc, true));
if (self->configuration.usableRegions > 1) {
teams.push_back(ShardsAffectedByTeamFailure::Team(iShard.remoteSrc, false));
}
if (traceShard) {
TraceEvent(SevDebug, "DDInitShard", self->ddId)
.detail("Keys", keys)
.detail("PrimarySrc", describe(iShard.primarySrc))
.detail("RemoteSrc", describe(iShard.remoteSrc))
.detail("PrimaryDest", describe(iShard.primaryDest))
.detail("RemoteDest", describe(iShard.remoteDest))
.detail("SrcID", iShard.srcId)
.detail("DestID", iShard.destId);
}
self->shardsAffectedByTeamFailure->moveShard(keys, teams);
if (iShard.hasDest && iShard.destId == anonymousShardId) {
// This shard is already in flight. Ideally we should use dest in ShardsAffectedByTeamFailure and
// generate a dataDistributionRelocator directly in DataDistributionQueue to track it, but it's
// easier to just (with low priority) schedule it for movement.
bool unhealthy = iShard.primarySrc.size() != self->configuration.storageTeamSize;
for (int r = 0; r < ranges.size(); r++) {
auto& keys = ranges[r];
self->shardsAffectedByTeamFailure->defineShard(keys);
auto customRange = self->initData->customReplication->rangeContaining(keys.begin);
int customReplicas = std::max(self->configuration.storageTeamSize, customRange.value());
ASSERT_WE_THINK(customRange.range().contains(keys));
bool unhealthy = iShard.primarySrc.size() != customReplicas;
if (!unhealthy && self->configuration.usableRegions > 1) {
unhealthy = iShard.remoteSrc.size() != self->configuration.storageTeamSize;
unhealthy = iShard.remoteSrc.size() != customReplicas;
}
if (traceShard) {
TraceEvent(SevDebug, "DDInitShard", self->ddId)
.detail("Keys", keys)
.detail("PrimarySrc", describe(iShard.primarySrc))
.detail("RemoteSrc", describe(iShard.remoteSrc))
.detail("PrimaryDest", describe(iShard.primaryDest))
.detail("RemoteDest", describe(iShard.remoteDest))
.detail("SrcID", iShard.srcId)
.detail("DestID", iShard.destId)
.detail("CustomReplicas", customReplicas)
.detail("StorageTeamSize", self->configuration.storageTeamSize)
.detail("Unhealthy", unhealthy);
}
self->shardsAffectedByTeamFailure->moveShard(keys, teams);
if ((ddLargeTeamEnabled() && (unhealthy || r > 0)) ||
(iShard.hasDest && iShard.destId == anonymousShardId)) {
// This shard is already in flight. Ideally we should use dest in ShardsAffectedByTeamFailure and
// generate a dataDistributionRelocator directly in DataDistributionQueue to track it, but it's
// easier to just (with low priority) schedule it for movement.
DataMovementReason reason = DataMovementReason::RECOVER_MOVE;
if (unhealthy) {
reason = DataMovementReason::TEAM_UNHEALTHY;
} else if (r > 0) {
reason = DataMovementReason::SPLIT_SHARD;
}
self->relocationProducer.send(RelocateShard(keys, reason, RelocateReason::OTHER));
}
self->relocationProducer.send(
RelocateShard(keys,
unhealthy ? DataMovementReason::TEAM_UNHEALTHY : DataMovementReason::RECOVER_MOVE,
RelocateReason::OTHER));
}
wait(yield(TaskPriority::DataDistribution));
@ -626,8 +665,10 @@ ACTOR Future<Void> dataDistribution(Reference<DataDistributor> self,
state Reference<DDTeamCollection> primaryTeamCollection;
state Reference<DDTeamCollection> remoteTeamCollection;
state bool trackerCancelled;
loop {
trackerCancelled = false;
// whether all initial shard are tracked
self->initialized = Promise<Void>();
self->auditInitialized = Promise<Void>();
@ -680,41 +721,47 @@ ACTOR Future<Void> dataDistribution(Reference<DataDistributor> self,
actors.push_back(resumeStorageAudits(self));
actors.push_back(self->pollMoveKeysLock());
actors.push_back(reportErrorsExcept(dataDistributionTracker(self->initData,
self->txnProcessor,
self->relocationProducer,
self->shardsAffectedByTeamFailure,
self->physicalShardCollection,
getShardMetrics,
getTopKShardMetrics.getFuture(),
getShardMetricsList,
getAverageShardBytes.getFuture(),
self->initialized,
anyZeroHealthyTeams,
self->ddId,
&shards,
&trackerCancelled,
self->ddTenantCache),
auto shardTracker = makeReference<DataDistributionTracker>(
DataDistributionTrackerInitParams{ .db = self->txnProcessor,
.distributorId = self->ddId,
.readyToStart = self->initialized,
.output = self->relocationProducer,
.shardsAffectedByTeamFailure = self->shardsAffectedByTeamFailure,
.physicalShardCollection = self->physicalShardCollection,
.anyZeroHealthyTeams = anyZeroHealthyTeams,
.shards = &shards,
.trackerCancelled = &trackerCancelled,
.ddTenantCache = self->ddTenantCache });
actors.push_back(reportErrorsExcept(DataDistributionTracker::run(shardTracker,
self->initData,
getShardMetrics.getFuture(),
getTopKShardMetrics.getFuture(),
getShardMetricsList.getFuture(),
getAverageShardBytes.getFuture()),
"DDTracker",
self->ddId,
&normalDDQueueErrors()));
actors.push_back(reportErrorsExcept(dataDistributionQueue(self->txnProcessor,
self->relocationProducer,
self->relocationConsumer.getFuture(),
getShardMetrics,
getTopKShardMetrics,
processingUnhealthy,
processingWiggle,
tcis,
self->shardsAffectedByTeamFailure,
self->physicalShardCollection,
self->lock,
getAverageShardBytes,
getUnhealthyRelocationCount.getFuture(),
self->ddId,
replicaSize,
self->configuration.storageTeamSize,
self->context->ddEnabledState.get()),
auto ddQueue = makeReference<DDQueue>(
DDQueueInitParams{ .id = self->ddId,
.lock = self->lock,
.db = self->txnProcessor,
.teamCollections = tcis,
.shardsAffectedByTeamFailure = self->shardsAffectedByTeamFailure,
.physicalShardCollection = self->physicalShardCollection,
.getAverageShardBytes = getAverageShardBytes,
.teamSize = replicaSize,
.singleRegionTeamSize = self->configuration.storageTeamSize,
.relocationProducer = self->relocationProducer,
.relocationConsumer = self->relocationConsumer.getFuture(),
.getShardMetrics = getShardMetrics,
.getTopKMetrics = getTopKShardMetrics });
actors.push_back(reportErrorsExcept(DDQueue::run(ddQueue,
processingUnhealthy,
processingWiggle,
getUnhealthyRelocationCount.getFuture(),
self->context->ddEnabledState.get()),
"DDQueue",
self->ddId,
&normalDDQueueErrors()));

View File

@ -639,7 +639,7 @@ ACTOR Future<Void> refreshEncryptionKeysImpl(Reference<EncryptKeyProxyData> ekpP
itr != ekpProxyData->baseCipherDomainIdCache.end();) {
if (isCipherKeyEligibleForRefresh(itr->second, currTS)) {
TraceEvent("RefreshEKs").detail("Id", itr->first);
req.encryptDomainIds.emplace_back(itr->first);
req.encryptDomainIds.push_back(itr->first);
}
// Garbage collect expired cached CipherKeys
@ -650,6 +650,12 @@ ACTOR Future<Void> refreshEncryptionKeysImpl(Reference<EncryptKeyProxyData> ekpP
}
}
if (req.encryptDomainIds.empty()) {
// Nothing to refresh
TraceEvent(SevDebug, "RefreshEKsEmptyRefresh");
return Void();
}
state double startTime = now();
std::function<Future<KmsConnLookupEKsByDomainIdsRep>()> repF = [&]() {
return kmsConnectorInf.ekLookupByDomainIds.getReply(req);

View File

@ -186,8 +186,7 @@ rocksdb::ColumnFamilyOptions SharedRocksDBState::initialCfOptions() {
options.prefix_extractor.reset(rocksdb::NewFixedPrefixTransform(SERVER_KNOBS->ROCKSDB_PREFIX_LEN));
// Also turn on bloom filters in the memtable.
// TODO: Make a knob for this as well.
options.memtable_prefix_bloom_size_ratio = 0.1;
options.memtable_prefix_bloom_size_ratio = SERVER_KNOBS->ROCKSDB_MEMTABLE_PREFIX_BLOOM_SIZE_RATIO;
// 5 -- Can be read by RocksDB's versions since 6.6.0. Full and partitioned
// filters use a generally faster and more accurate Bloom filter
@ -198,11 +197,11 @@ rocksdb::ColumnFamilyOptions SharedRocksDBState::initialCfOptions() {
// Create and apply a bloom filter using the 10 bits
// which should yield a ~1% false positive rate:
// https://github.com/facebook/rocksdb/wiki/RocksDB-Bloom-Filter#full-filters-new-format
bbOpts.filter_policy.reset(rocksdb::NewBloomFilterPolicy(10));
bbOpts.filter_policy.reset(rocksdb::NewBloomFilterPolicy(SERVER_KNOBS->ROCKSDB_BLOOM_BITS_PER_KEY));
// The whole key blooms are only used for point lookups.
// https://github.com/facebook/rocksdb/wiki/RocksDB-Bloom-Filter#prefix-vs-whole-key
bbOpts.whole_key_filtering = false;
bbOpts.whole_key_filtering = SERVER_KNOBS->ROCKSDB_BLOOM_WHOLE_KEY_FILTERING;
}
if (SERVER_KNOBS->ROCKSDB_BLOCK_CACHE_SIZE > 0) {
@ -213,6 +212,12 @@ rocksdb::ColumnFamilyOptions SharedRocksDBState::initialCfOptions() {
bbOpts.block_size = SERVER_KNOBS->ROCKSDB_BLOCK_SIZE;
}
// The readahead size starts with 8KB and is exponentially increased on each additional sequential IO,
// up to a max of BlockBasedTableOptions.max_auto_readahead_size (default 256 KB)
if (SERVER_KNOBS->ROCKSDB_MAX_AUTO_READAHEAD_SIZE > 0) {
bbOpts.max_auto_readahead_size = SERVER_KNOBS->ROCKSDB_MAX_AUTO_READAHEAD_SIZE;
}
options.table_factory.reset(rocksdb::NewBlockBasedTableFactory(bbOpts));
return options;
@ -429,13 +434,14 @@ struct Counters {
Counter convertedDeleteKeyReqs;
Counter convertedDeleteRangeReqs;
Counter rocksdbReadRangeQueries;
Counter commitDelayed;
Counters()
: cc("RocksDBThrottle"), immediateThrottle("ImmediateThrottle", cc), failedToAcquire("FailedToAcquire", cc),
deleteKeyReqs("DeleteKeyRequests", cc), deleteRangeReqs("DeleteRangeRequests", cc),
convertedDeleteKeyReqs("ConvertedDeleteKeyRequests", cc),
convertedDeleteRangeReqs("ConvertedDeleteRangeRequests", cc),
rocksdbReadRangeQueries("RocksdbReadRangeQueries", cc) {}
rocksdbReadRangeQueries("RocksdbReadRangeQueries", cc), commitDelayed("CommitDelayed", cc) {}
};
struct ReadIterator {
@ -954,6 +960,8 @@ ACTOR Future<Void> rocksDBMetricLogger(UID id,
{ "RowCacheHit", rocksdb::ROW_CACHE_HIT, 0 },
{ "RowCacheMiss", rocksdb::ROW_CACHE_MISS, 0 },
{ "CountIterSkippedKeys", rocksdb::NUMBER_ITER_SKIP, 0 },
{ "NoIteratorCreated", rocksdb::NO_ITERATOR_CREATED, 0 },
{ "NoIteratorDeleted", rocksdb::NO_ITERATOR_DELETED, 0 },
};
// To control the rocksdb::StatsLevel, use ROCKSDB_STATS_LEVEL knob.
@ -2070,9 +2078,12 @@ struct RocksDBKeyValueStore : IKeyValueStore {
&estPendCompactBytes);
while (count && estPendCompactBytes > SERVER_KNOBS->ROCKSDB_CAN_COMMIT_COMPACT_BYTES_LIMIT) {
wait(delay(SERVER_KNOBS->ROCKSDB_CAN_COMMIT_DELAY_ON_OVERLOAD));
++self->counters.commitDelayed;
count--;
self->db->GetAggregatedIntProperty(rocksdb::DB::Properties::kEstimatePendingCompactionBytes,
&estPendCompactBytes);
if (deterministicRandom()->random01() < 0.001)
TraceEvent(SevWarn, "RocksDBCommitsDelayed1000x", self->id);
}
return Void();

View File

@ -153,17 +153,28 @@ struct PageChecksumCodec {
if (!silent) {
auto severity = SevError;
if (g_network->isSimulated()) {
auto firstBlock = pageNumber == 1 ? 0 : ((pageNumber - 1) * pageLen) / 4096,
lastBlock = (pageNumber * pageLen) / 4096;
auto iter = g_simulator->corruptedBlocks.lower_bound(std::make_pair(filename, firstBlock));
if (iter != g_simulator->corruptedBlocks.end() && iter->first == filename && iter->second < lastBlock) {
// Calculate file offsets for the read/write operation space
// Operation starts at a 1-based pageNumber and is of size pageLen
int64_t fileOffsetStart = (pageNumber - 1) * pageLen;
// End refers to the offset after the operation, not the last byte.
int64_t fileOffsetEnd = fileOffsetStart + pageLen;
// Convert the file offsets to potentially corrupt block numbers
// Corrupt block numbers are 0-based and 4096 bytes in length.
int64_t corruptBlockStart = fileOffsetStart / 4096;
// corrupt block end is the block number AFTER the operation
int64_t corruptBlockEnd = (fileOffsetEnd + 4095) / 4096;
auto iter = g_simulator->corruptedBlocks.lower_bound(std::make_pair(filename, corruptBlockStart));
if (iter != g_simulator->corruptedBlocks.end() && iter->first == filename &&
iter->second < corruptBlockEnd) {
severity = SevWarnAlways;
}
TraceEvent("CheckCorruption")
.detail("Filename", filename)
.detail("NextFile", iter->first)
.detail("FirstBlock", firstBlock)
.detail("LastBlock", lastBlock)
.detail("BlockStart", corruptBlockStart)
.detail("BlockEnd", corruptBlockEnd)
.detail("NextBlock", iter->second);
}
TraceEvent trEvent(severity, "SQLitePageChecksumFailure");

View File

@ -2424,6 +2424,7 @@ struct ShardedRocksDBKeyValueStore : IKeyValueStore {
TraceEvent(SevError, "ShardedRocksDBRestoreFailed", logId)
.detail("Reason", "RestoreFilesRangesMismatch")
.detail("Ranges", describe(a.ranges))
.setMaxFieldLength(1000)
.detail("FetchedFiles", describe(rkvs));
a.done.sendError(failed_to_restore_checkpoint());
return;

View File

@ -27,6 +27,7 @@
#include "fdbserver/TLogInterface.h"
#include "flow/ActorCollection.h"
#include "flow/Arena.h"
#include "flow/CodeProbe.h"
#include "flow/Histogram.h"
#include "flow/Trace.h"
#include "flow/network.h"
@ -78,6 +79,7 @@ struct LogRouterData {
const UID dbgid;
Reference<AsyncVar<Reference<ILogSystem>>> logSystem;
Future<Void> logSystemChanged = Void();
Optional<UID> primaryPeekLocation;
NotifiedVersion version; // The largest version at which the log router has peeked mutations
// from satellite tLog or primary tLogs.
@ -295,48 +297,69 @@ ACTOR Future<Void> waitForVersion(LogRouterData* self, Version ver) {
return Void();
}
ACTOR Future<Reference<ILogSystem::IPeekCursor>> getPeekCursorData(LogRouterData* self,
Reference<ILogSystem::IPeekCursor> r,
Version startVersion) {
state Reference<ILogSystem::IPeekCursor> result = r;
state bool useSatellite = SERVER_KNOBS->LOG_ROUTER_PEEK_FROM_SATELLITES_PREFERRED;
loop {
Future<Void> getMoreF = Never();
if (result) {
getMoreF = result->getMore(TaskPriority::TLogCommit);
++self->getMoreCount;
if (!getMoreF.isReady()) {
++self->getMoreBlockedCount;
}
}
state double startTime = now();
choose {
when(wait(getMoreF)) {
double peekTime = now() - startTime;
self->peekLatencyDist->sampleSeconds(peekTime);
self->getMoreTime += peekTime;
self->maxGetMoreTime = std::max(self->maxGetMoreTime, peekTime);
return result;
}
when(wait(self->logSystemChanged)) {
if (self->logSystem->get()) {
result =
self->logSystem->get()->peekLogRouter(self->dbgid, startVersion, self->routerTag, useSatellite);
self->primaryPeekLocation = result->getPrimaryPeekLocation();
TraceEvent("LogRouterPeekLocation", self->dbgid)
.detail("LogID", result->getPrimaryPeekLocation())
.trackLatest(self->eventCacheHolder->trackingKey);
} else {
result = Reference<ILogSystem::IPeekCursor>();
}
self->logSystemChanged = self->logSystem->onChange();
}
when(wait(result ? delay(SERVER_KNOBS->LOG_ROUTER_PEEK_SWITCH_DC_TIME) : Never())) {
// Peek has become stuck for a while, trying switching between primary DC and satellite
CODE_PROBE(true, "Detect log router slow peeks");
TraceEvent(SevWarnAlways, "LogRouterSlowPeek", self->dbgid).detail("NextTrySatellite", !useSatellite);
useSatellite = !useSatellite;
result =
self->logSystem->get()->peekLogRouter(self->dbgid, startVersion, self->routerTag, useSatellite);
self->primaryPeekLocation = result->getPrimaryPeekLocation();
TraceEvent("LogRouterPeekLocation", self->dbgid)
.detail("LogID", result->getPrimaryPeekLocation())
.trackLatest(self->eventCacheHolder->trackingKey);
}
}
}
}
// Log router (LR) asynchronously pull data from satellite tLogs (preferred) or primary tLogs at tag (self->routerTag)
// for the version range from the LR's current version (exclusive) to its epoch's end version or recovery version.
ACTOR Future<Void> pullAsyncData(LogRouterData* self) {
state Future<Void> dbInfoChange = Void();
state Reference<ILogSystem::IPeekCursor> r;
state Version tagAt = self->version.get() + 1;
state Version lastVer = 0;
state std::vector<int> tags; // an optimization to avoid reallocating vector memory in every loop
loop {
loop {
Future<Void> getMoreF = Never();
if (r) {
getMoreF = r->getMore(TaskPriority::TLogCommit);
++self->getMoreCount;
if (!getMoreF.isReady()) {
++self->getMoreBlockedCount;
}
}
state double startTime = now();
choose {
when(wait(getMoreF)) {
double peekTime = now() - startTime;
self->peekLatencyDist->sampleSeconds(peekTime);
self->getMoreTime += peekTime;
self->maxGetMoreTime = std::max(self->maxGetMoreTime, peekTime);
break;
}
when(wait(dbInfoChange)) { // FIXME: does this actually happen?
if (self->logSystem->get()) {
r = self->logSystem->get()->peekLogRouter(self->dbgid, tagAt, self->routerTag);
self->primaryPeekLocation = r->getPrimaryPeekLocation();
TraceEvent("LogRouterPeekLocation", self->dbgid)
.detail("LogID", r->getPrimaryPeekLocation())
.trackLatest(self->eventCacheHolder->trackingKey);
} else {
r = Reference<ILogSystem::IPeekCursor>();
}
dbInfoChange = self->logSystem->onChange();
}
}
}
Reference<ILogSystem::IPeekCursor> _r = wait(getPeekCursorData(self, r, tagAt));
r = _r;
self->minKnownCommittedVersion = std::max(self->minKnownCommittedVersion, r->getMinKnownCommittedVersion());

View File

@ -343,7 +343,9 @@ ACTOR Future<Void> serverPeekParallelGetMore(ILogSystem::ServerPeekCursor* self,
// A cursor for a log router can be delayed indefinitely during a network partition, so only fail
// simulation tests sufficiently far after we finish simulating network partitions.
CODE_PROBE(e.code() == error_code_timed_out, "peek cursor timed out");
if (now() >= FLOW_KNOBS->SIM_SPEEDUP_AFTER_SECONDS + SERVER_KNOBS->PEEK_TRACKER_EXPIRATION_TIME) {
if (g_network->isSimulated() && now() >= g_simulator->connectionFailureEnableTime +
FLOW_KNOBS->SIM_SPEEDUP_AFTER_SECONDS +
SERVER_KNOBS->PEEK_TRACKER_EXPIRATION_TIME) {
ASSERT_WE_THINK(e.code() == error_code_operation_obsolete ||
SERVER_KNOBS->PEEK_TRACKER_EXPIRATION_TIME < 10);
}

View File

@ -155,6 +155,31 @@ ACTOR Future<Void> unassignServerKeys(Transaction* tr, UID ssId, KeyRange range,
return Void();
}
ACTOR Future<Void> deleteCheckpoints(Transaction* tr, std::set<UID> checkpointIds, UID logId) {
TraceEvent(SevDebug, "DataMoveDeleteCheckpoints", logId).detail("Checkpoints", describe(checkpointIds));
std::vector<Future<Optional<Value>>> checkpointEntries;
for (const UID& id : checkpointIds) {
checkpointEntries.push_back(tr->get(checkpointKeyFor(id)));
}
std::vector<Optional<Value>> checkpointValues = wait(getAll(checkpointEntries));
for (int i = 0; i < checkpointIds.size(); ++i) {
const auto& value = checkpointValues[i];
if (!value.present()) {
TraceEvent(SevWarnAlways, "CheckpointNotFound", logId);
continue;
}
CheckpointMetaData checkpoint = decodeCheckpointValue(value.get());
ASSERT(checkpointIds.find(checkpoint.checkpointID) != checkpointIds.end());
const Key key = checkpointKeyFor(checkpoint.checkpointID);
checkpoint.setState(CheckpointMetaData::Deleting);
tr->set(key, checkpointValue(checkpoint));
tr->clear(singleKeyRange(key));
TraceEvent(SevDebug, "DataMoveDeleteCheckpoint", logId).detail("Checkpoint", checkpoint.toString());
}
return Void();
}
} // namespace
bool DDEnabledState::isDDEnabled() const {
@ -221,6 +246,7 @@ ACTOR Future<MoveKeysLock> takeMoveKeysLock(Database cx, UID ddId) {
state UID txnId;
tr.setOption(FDBTransactionOptions::PRIORITY_SYSTEM_IMMEDIATE);
tr.setOption(FDBTransactionOptions::ACCESS_SYSTEM_KEYS);
tr.setOption(FDBTransactionOptions::LOCK_AWARE);
if (!g_network->isSimulated()) {
txnId = deterministicRandom()->randomUniqueID();
tr.debugTransaction(txnId);
@ -1257,8 +1283,9 @@ ACTOR static Future<Void> startMoveShards(Database occ,
wait(startMoveKeysLock->take(TaskPriority::DataDistributionLaunch));
state FlowLock::Releaser releaser(*startMoveKeysLock);
state DataMoveMetaData dataMove;
state Severity sevDm = SERVER_KNOBS->PHYSICAL_SHARD_MOVE_VERBOSE_TRACKING ? SevInfo : SevDebug;
TraceEvent(SevDebug, "StartMoveShardsBegin", relocationIntervalId)
TraceEvent(SevInfo, "StartMoveShardsBegin", relocationIntervalId)
.detail("DataMoveID", dataMoveId)
.detail("TargetRange", describe(ranges));
@ -1292,7 +1319,7 @@ ACTOR static Future<Void> startMoveShards(Database occ,
if (val.present()) {
DataMoveMetaData dmv = decodeDataMoveValue(val.get()); // dmv: Data move value.
dataMove = dmv;
TraceEvent(SevVerbose, "StartMoveShardsFoundDataMove", relocationIntervalId)
TraceEvent(sevDm, "StartMoveShardsFoundDataMove", relocationIntervalId)
.detail("DataMoveID", dataMoveId)
.detail("DataMove", dataMove.toString());
ASSERT(!dataMove.ranges.empty() && dataMove.ranges.front().begin == keys.begin);
@ -1303,12 +1330,12 @@ ACTOR static Future<Void> startMoveShards(Database occ,
throw movekeys_conflict();
}
if (dataMove.getPhase() == DataMoveMetaData::Deleting) {
TraceEvent(SevVerbose, "StartMoveShardsDataMove", relocationIntervalId)
TraceEvent(sevDm, "StartMoveShardsDataMove", relocationIntervalId)
.detail("DataMoveBeingDeleted", dataMoveId);
throw data_move_cancelled();
}
if (dataMove.getPhase() == DataMoveMetaData::Running) {
TraceEvent(SevVerbose, "StartMoveShardsDataMove", relocationIntervalId)
TraceEvent(sevDm, "StartMoveShardsDataMove", relocationIntervalId)
.detail("DataMoveAlreadyCommitted", dataMoveId);
ASSERT(keys == dataMove.ranges.front());
return Void();
@ -1320,7 +1347,7 @@ ACTOR static Future<Void> startMoveShards(Database occ,
}
dataMove = DataMoveMetaData();
dataMove.id = dataMoveId;
TraceEvent(SevVerbose, "StartMoveKeysNewDataMove", relocationIntervalId)
TraceEvent(sevDm, "StartMoveKeysNewDataMove", relocationIntervalId)
.detail("DataMoveRange", keys)
.detail("DataMoveID", dataMoveId);
}
@ -1367,7 +1394,7 @@ ACTOR static Future<Void> startMoveShards(Database occ,
state UID srcId;
state UID destId;
decodeKeyServersValue(UIDtoTagMap, old[oldIndex].value, src, dest, srcId, destId);
TraceEvent(SevVerbose, "StartMoveShardsProcessingShard", relocationIntervalId)
TraceEvent(sevDm, "StartMoveShardsProcessingShard", relocationIntervalId)
.detail("DataMoveID", dataMoveId)
.detail("Range", rangeIntersectKeys)
.detail("OldSrc", describe(src))
@ -1429,6 +1456,7 @@ ACTOR static Future<Void> startMoveShards(Database occ,
// Track old destination servers. They may be removed from serverKeys soon, since they are
// about to be overwritten in keyServers
ASSERT(dest.empty());
for (const UID& ssId : dest) {
oldDests.insert(ssId);
}
@ -1438,10 +1466,23 @@ ACTOR static Future<Void> startMoveShards(Database occ,
physicalShardMap[ssId].emplace_back(rangeIntersectKeys, srcId);
}
for (const UID& ssId : src) {
dataMove.src.insert(ssId);
// TODO(psm): Create checkpoint for the range.
}
const UID checkpointId = UID(deterministicRandom()->randomUInt64(), srcId.first());
CheckpointMetaData checkpoint(std::vector<KeyRange>{ rangeIntersectKeys },
DataMoveRocksCF,
src,
checkpointId,
dataMoveId);
checkpoint.setState(CheckpointMetaData::Pending);
tr.set(checkpointKeyFor(checkpointId), checkpointValue(checkpoint));
TraceEvent(sevDm, "InitiatedCheckpoint")
.detail("CheckpointID", checkpointId.toString())
.detail("Range", rangeIntersectKeys)
.detail("DataMoveID", dataMoveId)
.detail("SrcServers", describe(src))
.detail("ReadVersion", tr.getReadVersion().get());
dataMove.src.insert(src.begin(), src.end());
dataMove.checkpoints.insert(checkpointId);
}
// Remove old dests from serverKeys.
@ -1470,12 +1511,12 @@ ACTOR static Future<Void> startMoveShards(Database occ,
if (currentKeys.end == keys.end) {
dataMove.setPhase(DataMoveMetaData::Running);
complete = true;
TraceEvent(SevVerbose, "StartMoveShardsDataMoveComplete", dataMoveId)
TraceEvent(sevDm, "StartMoveShardsDataMoveComplete", dataMoveId)
.detail("DataMoveID", dataMoveId)
.detail("DataMove", dataMove.toString());
} else {
dataMove.setPhase(DataMoveMetaData::Prepare);
TraceEvent(SevVerbose, "StartMoveShardsDataMovePartial", dataMoveId)
TraceEvent(sevDm, "StartMoveShardsDataMovePartial", dataMoveId)
.detail("DataMoveID", dataMoveId)
.detail("CurrentRange", currentKeys)
.detail("DataMoveRange", keys)
@ -1488,7 +1529,7 @@ ACTOR static Future<Void> startMoveShards(Database occ,
wait(tr.commit());
TraceEvent(SevVerbose, "DataMoveMetaDataCommit", dataMove.id)
TraceEvent(sevDm, "DataMoveMetaDataCommit", dataMove.id)
.detail("DataMoveID", dataMoveId)
.detail("DataMoveKey", dataMoveKeyFor(dataMoveId))
.detail("CommitVersion", tr.getCommittedVersion())
@ -1520,7 +1561,7 @@ ACTOR static Future<Void> startMoveShards(Database occ,
throw;
}
TraceEvent(SevDebug, "StartMoveShardsEnd", relocationIntervalId).detail("DataMoveID", dataMoveId);
TraceEvent(SevInfo, "StartMoveShardsEnd", relocationIntervalId).detail("DataMoveID", dataMoveId);
return Void();
}
@ -1642,7 +1683,7 @@ ACTOR static Future<Void> finishMoveShards(Database occ,
Optional<Value> val = wait(tr.get(dataMoveKeyFor(dataMoveId)));
if (val.present()) {
dataMove = decodeDataMoveValue(val.get());
TraceEvent(SevVerbose, "FinishMoveShardsFoundDataMove", relocationIntervalId)
TraceEvent(SevDebug, "FinishMoveShardsFoundDataMove", relocationIntervalId)
.detail("DataMoveID", dataMoveId)
.detail("DataMove", dataMove.toString());
destServers.insert(destServers.end(), dataMove.dest.begin(), dataMove.dest.end());
@ -1682,7 +1723,7 @@ ACTOR static Future<Void> finishMoveShards(Database occ,
decodeKeyServersValue(UIDtoTagMap, keyServers[currentIndex].value, src, dest, srcId, destId);
const KeyRange currentRange =
KeyRangeRef(keyServers[currentIndex].key, keyServers[currentIndex + 1].key);
TraceEvent(SevVerbose, "FinishMoveShardsProcessingShard", relocationIntervalId)
TraceEvent(SevDebug, "FinishMoveShardsProcessingShard", relocationIntervalId)
.detail("Range", currentRange)
.detail("SrcID", srcId)
.detail("Src", describe(src))
@ -1739,8 +1780,10 @@ ACTOR static Future<Void> finishMoveShards(Database occ,
releaser.release();
for (int s = 0; s < serverListValues.size(); s++) {
ASSERT(serverListValues[s]
.present()); // There should always be server list entries for servers in keyServers
// TODO: if the server is removed,
if (!serverListValues[s].present()) {
throw retry();
}
auto si = decodeServerListValue(serverListValues[s].get());
ASSERT(si.id() == newDestinations[s]);
storageServerInterfaces.push_back(si);
@ -1753,6 +1796,10 @@ ACTOR static Future<Void> finishMoveShards(Database occ,
storageServerInterfaces[s], range, tr.getReadVersion().get(), GetShardStateRequest::READABLE));
}
TraceEvent(SevDebug, "FinishMoveShardsWaitingServers", relocationIntervalId)
.detail("DataMoveID", dataMoveId)
.detail("NewDestinations", describe(newDestinations));
// Wait for all storage server moves, and explicitly swallow errors for tss ones with
// waitForAllReady If this takes too long the transaction will time out and retry, which is ok
wait(timeout(waitForAll(serverReady),
@ -1767,7 +1814,7 @@ ACTOR static Future<Void> finishMoveShards(Database occ,
}
}
TraceEvent(SevVerbose, "FinishMoveShardsWaitedServers", relocationIntervalId)
TraceEvent(SevDebug, "FinishMoveShardsWaitedServers", relocationIntervalId)
.detail("DataMoveID", dataMoveId)
.detail("ReadyServers", describe(readyServers))
.detail("NewDestinations", describe(newDestinations));
@ -1787,7 +1834,7 @@ ACTOR static Future<Void> finishMoveShards(Database occ,
range,
allKeys,
destHasServer ? serverKeysValue(dataMoveId) : serverKeysFalse));
TraceEvent(SevVerbose, "FinishMoveShardsSetServerKeyRange", dataMoveId)
TraceEvent(SevDebug, "FinishMoveShardsSetServerKeyRange", dataMoveId)
.detail("StorageServerID", ssId)
.detail("KeyRange", range)
.detail("ShardID", destHasServer ? dataMoveId : UID());
@ -1796,9 +1843,10 @@ ACTOR static Future<Void> finishMoveShards(Database occ,
wait(waitForAll(actors));
if (range.end == dataMove.ranges.front().end) {
wait(deleteCheckpoints(&tr, dataMove.checkpoints, dataMoveId));
tr.clear(dataMoveKeyFor(dataMoveId));
complete = true;
TraceEvent(SevVerbose, "FinishMoveShardsDeleteMetaData", dataMoveId)
TraceEvent(SevDebug, "FinishMoveShardsDeleteMetaData", dataMoveId)
.detail("DataMove", dataMove.toString());
} else {
TraceEvent(SevInfo, "FinishMoveShardsPartialComplete", dataMoveId)
@ -1821,17 +1869,21 @@ ACTOR static Future<Void> finishMoveShards(Database occ,
TraceEvent(SevWarn, "TryFinishMoveShardsError", relocationIntervalId)
.errorUnsuppressed(error)
.detail("DataMoveID", dataMoveId);
if (error.code() == error_code_actor_cancelled)
throw;
state Error err = error;
wait(tr.onError(error));
retries++;
if (retries % 10 == 0) {
TraceEvent(retries == 20 ? SevWarnAlways : SevWarn,
"RelocateShard_FinishMoveKeysRetrying",
relocationIntervalId)
.error(err)
.detail("DataMoveID", dataMoveId);
if (error.code() == error_code_retry) {
wait(delay(1));
} else {
if (error.code() == error_code_actor_cancelled)
throw;
state Error err = error;
wait(tr.onError(error));
retries++;
if (retries % 10 == 0) {
TraceEvent(retries == 20 ? SevWarnAlways : SevWarn,
"RelocateShard_FinishMoveKeysRetrying",
relocationIntervalId)
.error(err)
.detail("DataMoveID", dataMoveId);
}
}
}
}
@ -2043,9 +2095,10 @@ ACTOR Future<bool> canRemoveStorageServer(Reference<ReadYourWritesTransaction> t
// Return true if the entire range is false. Since these values are coalesced, we can return false if there is more
// than one result
UID teamId;
UID shardId;
bool assigned, emptyRange;
decodeServerKeysValue(keys[0].value, assigned, emptyRange, teamId);
EnablePhysicalShardMove enablePSM = EnablePhysicalShardMove::False;
decodeServerKeysValue(keys[0].value, assigned, emptyRange, enablePSM, shardId);
TraceEvent(SevVerbose, "CanRemoveStorageServer")
.detail("ServerID", serverID)
.detail("Key1", keys[0].key)
@ -2274,7 +2327,8 @@ ACTOR Future<Void> removeKeysFromFailedServer(Database cx,
}
}
const UID shardId = newShardId(deterministicRandom()->randomUInt64(), AssignEmptyRange::True);
const UID shardId =
newDataMoveId(deterministicRandom()->randomUInt64(), AssignEmptyRange::True);
// Assign the shard to teamForDroppedRange in keyServer space.
if (SERVER_KNOBS->SHARD_ENCODE_LOCATION_METADATA) {
@ -2454,6 +2508,7 @@ ACTOR Future<Void> cleanUpDataMove(Database occ,
}
if (range.end == dataMove.ranges.front().end) {
wait(deleteCheckpoints(&tr, dataMove.checkpoints, dataMoveId));
tr.clear(dataMoveKeyFor(dataMoveId));
complete = true;
TraceEvent(SevVerbose, "CleanUpDataMoveDeleteMetaData", dataMoveId)
@ -2653,13 +2708,13 @@ void seedShardServers(Arena& arena, CommitTransactionRef& tr, std::vector<Storag
// to a specific
// key (keyServersKeyServersKey)
if (SERVER_KNOBS->SHARD_ENCODE_LOCATION_METADATA) {
const UID teamId = deterministicRandom()->randomUniqueID();
ksValue = keyServersValue(serverSrcUID, /*dest=*/std::vector<UID>(), teamId, UID());
const UID shardId = deterministicRandom()->randomUniqueID();
ksValue = keyServersValue(serverSrcUID, /*dest=*/std::vector<UID>(), shardId, UID());
krmSetPreviouslyEmptyRange(tr, arena, keyServersPrefix, KeyRangeRef(KeyRef(), allKeys.end), ksValue, Value());
for (auto& s : servers) {
krmSetPreviouslyEmptyRange(
tr, arena, serverKeysPrefixFor(s.id()), allKeys, serverKeysValue(teamId), serverKeysFalse);
tr, arena, serverKeysPrefixFor(s.id()), allKeys, serverKeysValue(shardId), serverKeysFalse);
}
} else {
krmSetPreviouslyEmptyRange(tr, arena, keyServersPrefix, KeyRangeRef(KeyRef(), allKeys.end), ksValue, Value());

View File

@ -788,13 +788,13 @@ ACTOR Future<Void> reconfigureAfter(Database cx,
Reference<AsyncVar<ServerDBInfo> const> dbInfo,
std::string context) {
wait(delay(time));
wait(repairDeadDatacenter(cx, dbInfo, context));
wait(uncancellable(repairDeadDatacenter(cx, dbInfo, context)));
return Void();
}
struct QuietDatabaseChecker {
ProcessEvents::Callback timeoutCallback = [this](StringRef name, StringRef msg, Error const& e) {
logFailure(name, msg, e);
ProcessEvents::Callback timeoutCallback = [this](StringRef name, std::any const& msg, Error const& e) {
logFailure(name, std::any_cast<StringRef>(msg), e);
};
double start = now();
double maxDDRunTime;

View File

@ -172,7 +172,7 @@ std::string getFullRequestUrl(Reference<RESTKmsConnectorCtx> ctx, const std::str
throw encrypt_invalid_kms_config();
}
std::string fullUrl(url);
return fullUrl.append("/").append(suffix);
return (suffix[0] == '/') ? fullUrl.append(suffix) : fullUrl.append("/").append(suffix);
}
void dropCachedKmsUrls(Reference<RESTKmsConnectorCtx> ctx) {
@ -1539,6 +1539,7 @@ void testGetBlobMetadataRequestBody(Reference<RESTKmsConnectorCtx> ctx) {
for (const auto& detail : details) {
auto it = domainIds.find(detail.domainId);
ASSERT(it != domainIds.end());
ASSERT(detail.base.present() || !detail.partitions.empty());
}
if (refreshKmsUrls) {
validateKmsUrls(ctx);

View File

@ -216,6 +216,31 @@ struct Resolver : ReferenceCounted<Resolver> {
};
} // namespace
ACTOR Future<Void> versionReady(Resolver* self, ProxyRequestsInfo* proxyInfo, Version prevVersion) {
loop {
if (self->recentStateTransactionsInfo.size() &&
proxyInfo->lastVersion <= self->recentStateTransactionsInfo.firstVersion()) {
self->neededVersion.set(std::max(self->neededVersion.get(), prevVersion));
}
// Update queue depth metric before waiting. Check if we're going to be one of the waiters or not.
int waiters = self->version.numWaiting();
if (self->version.get() < prevVersion) {
waiters++;
}
self->queueDepthDist->sampleRecordCounter(waiters);
choose {
when(wait(self->version.whenAtLeast(prevVersion))) {
// Update queue depth metric after waiting.
self->queueDepthDist->sampleRecordCounter(self->version.numWaiting());
return Void();
}
when(wait(self->checkNeededVersion.onTrigger())) {}
}
}
}
ACTOR Future<Void> resolveBatch(Reference<Resolver> self,
ResolveTransactionBatchRequest req,
Reference<AsyncVar<ServerDBInfo> const> db) {
@ -266,28 +291,7 @@ ACTOR Future<Void> resolveBatch(Reference<Resolver> self,
g_traceBatch.addEvent("CommitDebug", debugID.get().first(), "Resolver.resolveBatch.AfterQueueSizeCheck");
}
loop {
if (self->recentStateTransactionsInfo.size() &&
proxyInfo.lastVersion <= self->recentStateTransactionsInfo.firstVersion()) {
self->neededVersion.set(std::max(self->neededVersion.get(), req.prevVersion));
}
// Update queue depth metric before waiting. Check if we're going to be one of the waiters or not.
int waiters = self->version.numWaiting();
if (self->version.get() < req.prevVersion) {
waiters++;
}
self->queueDepthDist->sampleRecordCounter(waiters);
choose {
when(wait(self->version.whenAtLeast(req.prevVersion))) {
// Update queue depth metric after waiting.
self->queueDepthDist->sampleRecordCounter(self->version.numWaiting());
break;
}
when(wait(self->checkNeededVersion.onTrigger())) {}
}
}
wait(versionReady(self.getPtr(), &proxyInfo, req.prevVersion));
if (check_yield(TaskPriority::DefaultEndpoint)) {
wait(delay(0, TaskPriority::Low) || delay(SERVER_KNOBS->COMMIT_SLEEP_TIME)); // FIXME: Is this still right?

24
fdbserver/ResolverBug.cpp Normal file
View File

@ -0,0 +1,24 @@
/*
* ResolverBug.cpp
*
* This source file is part of the FoundationDB open source project
*
* Copyright 2013-2023 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/ResolverBug.h"
std::shared_ptr<ISimBug> ResolverBugID::create() const {
return std::make_shared<ResolverBug>();
}

File diff suppressed because it is too large Load Diff

View File

@ -38,9 +38,13 @@ ICheckpointReader* newCheckpointReader(const CheckpointMetaData& checkpoint,
ACTOR Future<Void> deleteCheckpoint(CheckpointMetaData checkpoint) {
wait(delay(0, TaskPriority::FetchKeys));
state CheckpointFormat format = checkpoint.getFormat();
const CheckpointFormat format = checkpoint.getFormat();
if (format == DataMoveRocksCF || format == RocksDB) {
wait(deleteRocksCheckpoint(checkpoint));
if (!checkpoint.dir.empty()) {
platform::eraseDirectoryRecursive(checkpoint.dir);
} else {
TraceEvent(SevWarn, "CheckpointDirNotFound").detail("Checkpoint", checkpoint.toString());
}
} else {
throw not_implemented();
}
@ -84,6 +88,7 @@ ACTOR Future<CheckpointMetaData> fetchCheckpointRanges(Database cx,
throw not_implemented();
}
initialState.setFormat(RocksDBKeyValues);
initialState.ranges = ranges;
initialState.serializedCheckpoint = ObjectWriter::toValue(RocksDBCheckpointKeyValues(ranges), IncludeVersion());
}
@ -93,4 +98,12 @@ ACTOR Future<CheckpointMetaData> fetchCheckpointRanges(Database cx,
.detail("CheckpointMetaData", result.toString())
.detail("Ranges", describe(ranges));
return result;
}
}
std::string serverCheckpointDir(const std::string& baseDir, const UID& checkpointId) {
return joinPath(baseDir, checkpointId.toString());
}
std::string fetchedCheckpointDir(const std::string& baseDir, const UID& checkpointId) {
return joinPath(baseDir, UID(checkpointId.first(), deterministicRandom()->randomUInt64()).toString());
}

View File

@ -39,6 +39,15 @@ int ShardsAffectedByTeamFailure::getNumberOfShards(UID ssID) const {
return it == storageServerShards.end() ? 0 : it->second;
}
int ShardsAffectedByTeamFailure::getNumberOfShards(Team team) const {
int shardCount = 0;
for (auto it = team_shards.lower_bound(std::pair<Team, KeyRange>(team, KeyRangeRef()));
it != team_shards.end() && it->first == team;
++it)
shardCount++;
return shardCount;
}
std::pair<std::vector<ShardsAffectedByTeamFailure::Team>, std::vector<ShardsAffectedByTeamFailure::Team>>
ShardsAffectedByTeamFailure::getTeamsForFirstShard(KeyRangeRef keys) {
return shard_teams[keys.begin];

View File

@ -816,21 +816,46 @@ struct TransactionInfo {
bool reportConflictingKeys;
};
bool ConflictBatch::ignoreTooOld() const {
return bugs && deterministicRandom()->random01() < bugs->ignoreTooOldProbability;
}
bool ConflictBatch::ignoreReadSet() const {
return bugs && deterministicRandom()->random01() < bugs->ignoreReadSetProbability;
}
bool ConflictBatch::ignoreWriteSet() const {
return bugs && deterministicRandom()->random01() < bugs->ignoreWriteSetProbability;
}
void ConflictBatch::addTransaction(const CommitTransactionRef& tr, Version newOldestVersion) {
const int t = transactionCount++;
Arena& arena = transactionInfo.arena();
TransactionInfo* info = new (arena) TransactionInfo;
info->reportConflictingKeys = tr.report_conflicting_keys;
bool tooOld = tr.read_snapshot < newOldestVersion && tr.read_conflict_ranges.size();
if (tooOld && ignoreTooOld()) {
bugs->hit();
tooOld = false;
}
if (tr.read_snapshot < newOldestVersion && tr.read_conflict_ranges.size()) {
if (tooOld) {
info->tooOld = true;
} else {
info->tooOld = false;
info->readRanges.resize(arena, tr.read_conflict_ranges.size());
info->writeRanges.resize(arena, tr.write_conflict_ranges.size());
if (!ignoreReadSet()) {
info->readRanges.resize(arena, tr.read_conflict_ranges.size());
} else {
bugs->hit();
}
if (!ignoreWriteSet()) {
info->writeRanges.resize(arena, tr.write_conflict_ranges.size());
} else {
bugs->hit();
}
for (int r = 0; r < tr.read_conflict_ranges.size(); r++) {
for (int r = 0; r < info->readRanges.size(); r++) {
const KeyRangeRef& range = tr.read_conflict_ranges[r];
points.emplace_back(range.begin, true, false, t, &info->readRanges[r].first);
points.emplace_back(range.end, false, false, t, &info->readRanges[r].second);
@ -843,7 +868,7 @@ void ConflictBatch::addTransaction(const CommitTransactionRef& tr, Version newOl
: nullptr,
tr.report_conflicting_keys ? resolveBatchReplyArena : nullptr);
}
for (int r = 0; r < tr.write_conflict_ranges.size(); r++) {
for (int r = 0; r < info->writeRanges.size(); r++) {
const KeyRangeRef& range = tr.write_conflict_ranges[r];
points.emplace_back(range.begin, true, true, t, &info->writeRanges[r].first);
points.emplace_back(range.end, false, true, t, &info->writeRanges[r].second);

View File

@ -240,20 +240,6 @@ protected:
int64_t counter;
};
static JsonBuilderObject getLocalityInfo(const LocalityData& locality) {
JsonBuilderObject localityObj;
for (auto it = locality._data.begin(); it != locality._data.end(); it++) {
if (it->second.present()) {
localityObj[it->first] = it->second.get();
} else {
localityObj[it->first] = JsonBuilder();
}
}
return localityObj;
}
static JsonBuilderObject getError(const TraceEventFields& errorFields) {
JsonBuilderObject statusObj;
try {
@ -352,7 +338,7 @@ JsonBuilderObject machineStatusFetcher(WorkerEvents mMetrics,
}
if (locality.count(it->first)) {
statusObj["locality"] = getLocalityInfo(locality[it->first]);
statusObj["locality"] = locality[it->first].toJSON<JsonBuilderObject>();
}
statusObj["address"] = address;
@ -949,7 +935,7 @@ ACTOR static Future<JsonBuilderObject> processStatusFetcher(
std::string MachineID = processMetrics.getValue("MachineID");
statusObj["machine_id"] = MachineID;
statusObj["locality"] = getLocalityInfo(workerItr->interf.locality);
statusObj["locality"] = workerItr->interf.locality.toJSON<JsonBuilderObject>();
statusObj.setKeyRawNumber("uptime_seconds", processMetrics.getValue("UptimeSeconds"));
@ -1907,6 +1893,10 @@ ACTOR static Future<JsonBuilderObject> dataStatusFetcher(WorkerDetails ddWorker,
} else if (highestPriority == SERVER_KNOBS->PRIORITY_TEAM_HEALTHY) {
stateSectionObj["healthy"] = true;
stateSectionObj["name"] = "healthy";
} else if (highestPriority == SERVER_KNOBS->PRIORITY_PERPETUAL_STORAGE_WIGGLE) {
stateSectionObj["healthy"] = true;
stateSectionObj["name"] = "healthy_perpetual_wiggle";
stateSectionObj["description"] = "Wiggling storage server";
} else if (highestPriority >= SERVER_KNOBS->PRIORITY_RECOVER_MOVE) {
stateSectionObj["healthy"] = true;
stateSectionObj["name"] = "healthy_rebalancing";

View File

@ -1212,7 +1212,10 @@ Reference<ILogSystem::IPeekCursor> TagPartitionedLogSystem::peekSingle(UID dbgid
}
}
Reference<ILogSystem::IPeekCursor> TagPartitionedLogSystem::peekLogRouter(UID dbgid, Version begin, Tag tag) {
Reference<ILogSystem::IPeekCursor> TagPartitionedLogSystem::peekLogRouter(UID dbgid,
Version begin,
Tag tag,
bool useSatellite) {
bool found = false;
for (const auto& log : tLogs) {
found = log->hasLogRouter(dbgid) || log->hasBackupWorker(dbgid);
@ -1240,8 +1243,7 @@ Reference<ILogSystem::IPeekCursor> TagPartitionedLogSystem::peekLogRouter(UID db
}
}
int bestSet = bestPrimarySet;
if (SERVER_KNOBS->LOG_ROUTER_PEEK_FROM_SATELLITES_PREFERRED && bestSatelliteSet != -1 &&
tLogs[bestSatelliteSet]->tLogVersion >= TLogVersion::V4) {
if (useSatellite && bestSatelliteSet != -1 && tLogs[bestSatelliteSet]->tLogVersion >= TLogVersion::V4) {
bestSet = bestSatelliteSet;
}
@ -1266,8 +1268,7 @@ Reference<ILogSystem::IPeekCursor> TagPartitionedLogSystem::peekLogRouter(UID db
}
}
int bestSet = bestPrimarySet;
if (SERVER_KNOBS->LOG_ROUTER_PEEK_FROM_SATELLITES_PREFERRED && bestSatelliteSet != -1 &&
tLogs[bestSatelliteSet]->tLogVersion >= TLogVersion::V4) {
if (useSatellite && bestSatelliteSet != -1 && tLogs[bestSatelliteSet]->tLogVersion >= TLogVersion::V4) {
bestSet = bestSatelliteSet;
}
const auto& log = tLogs[bestSet];
@ -1307,8 +1308,7 @@ Reference<ILogSystem::IPeekCursor> TagPartitionedLogSystem::peekLogRouter(UID db
}
}
int bestSet = bestPrimarySet;
if (SERVER_KNOBS->LOG_ROUTER_PEEK_FROM_SATELLITES_PREFERRED && bestSatelliteSet != -1 &&
old.tLogs[bestSatelliteSet]->tLogVersion >= TLogVersion::V4) {
if (useSatellite && bestSatelliteSet != -1 && old.tLogs[bestSatelliteSet]->tLogVersion >= TLogVersion::V4) {
bestSet = bestSatelliteSet;
}

View File

@ -385,8 +385,9 @@ public:
uint16_t tenantNumber = deterministicRandom()->randomInt(0, std::numeric_limits<uint16_t>::max());
for (uint16_t i = 0; i < tenantCount; i++) {
TenantName tenantName(format("%s_%08d", "ddtc_test_tenant", tenantNumber + i));
TenantMapEntry tenant(tenantNumber + i, tenantName);
uint16_t tenantOrdinal = tenantNumber + i;
TenantName tenantName(format("%s_%08d", "ddtc_test_tenant", tenantOrdinal));
TenantMapEntry tenant(tenantOrdinal, tenantName);
tenantCache.insert(tenant);
}

View File

@ -3666,7 +3666,10 @@ public:
if (dispose) {
if (!self->memoryOnly) {
debug_printf("DWALPager(%s) shutdown deleting file\n", self->filename.c_str());
wait(IAsyncFileSystem::filesystem()->incrementalDeleteFile(self->filename, true));
// We wrap this with ready() because we don't care if incrementalDeleteFile throws an error because:
// - if the file was unlinked, the file will be cleaned up by the filesystem after the process exits
// - if the file was not unlinked, the worker will restart and pick it up and it'll be removed later
wait(ready(IAsyncFileSystem::filesystem()->incrementalDeleteFile(self->filename, true)));
}
}

View File

@ -234,8 +234,6 @@ extern void pingtest();
extern void copyTest();
extern void versionedMapTest();
extern void createTemplateDatabase();
// FIXME: this really belongs in a header somewhere since it is actually used.
extern IPAddress determinePublicIPAutomatically(ClusterConnectionString& ccs);
extern const char* getSourceVersion();
@ -897,7 +895,7 @@ std::pair<NetworkAddressList, NetworkAddressList> buildNetworkAddresses(
if (autoPublicAddress) {
try {
const NetworkAddress& parsedAddress = NetworkAddress::parse("0.0.0.0:" + publicAddressStr.substr(5));
const IPAddress publicIP = determinePublicIPAutomatically(connectionRecord.getConnectionString());
const IPAddress publicIP = connectionRecord.getConnectionString().determineLocalSourceIP();
currentPublicAddress = NetworkAddress(publicIP, parsedAddress.port, true, parsedAddress.isTLS());
} catch (Error& e) {
fprintf(stderr,
@ -1114,7 +1112,7 @@ struct CLIOptions {
printHelpTeaser(name);
flushAndExit(FDB_EXIT_ERROR);
}
auto publicIP = determinePublicIPAutomatically(connectionFile->getConnectionString());
auto publicIP = connectionFile->getConnectionString().determineLocalSourceIP();
publicAddresses.address = NetworkAddress(publicIP, ::getpid());
}
}
@ -1706,7 +1704,7 @@ private:
}
case OPT_KMS_REST_ALLOW_NOT_SECURE_CONECTION: {
TraceEvent("RESTKmsConnAllowNotSecureConnection");
knobs.emplace_back("rest_kms_enable_not_secure_connection", "true");
knobs.emplace_back("rest_kms_allow_not_secure_connection", "true");
break;
}
case OPT_KMS_CONN_VALIDATION_TOKEN_DETAILS: {

View File

@ -45,21 +45,29 @@ struct BlobFileIndex {
int64_t offset;
int64_t length;
int64_t fullFileLength;
int64_t logicalSize;
Optional<BlobGranuleCipherKeysMeta> cipherKeysMeta;
BlobFileIndex() {}
BlobFileIndex(Version version, std::string filename, int64_t offset, int64_t length, int64_t fullFileLength)
: version(version), filename(filename), offset(offset), length(length), fullFileLength(fullFileLength) {}
BlobFileIndex(Version version,
std::string filename,
int64_t offset,
int64_t length,
int64_t fullFileLength,
int64_t logicalSize)
: version(version), filename(filename), offset(offset), length(length), fullFileLength(fullFileLength),
logicalSize(logicalSize) {}
BlobFileIndex(Version version,
std::string filename,
int64_t offset,
int64_t length,
int64_t fullFileLength,
int64_t logicalSize,
Optional<BlobGranuleCipherKeysMeta> ciphKeysMeta)
: version(version), filename(filename), offset(offset), length(length), fullFileLength(fullFileLength),
cipherKeysMeta(ciphKeysMeta) {}
logicalSize(logicalSize), cipherKeysMeta(ciphKeysMeta) {}
// compare on version
bool operator<(const BlobFileIndex& r) const { return version < r.version; }
@ -183,7 +191,7 @@ public:
ACTOR static Future<Void> updateState(Reference<BlobRestoreController> self,
BlobRestorePhase newPhase,
Optional<BlobRestorePhase> expectedPhase);
ACTOR static Future<Void> updateError(Reference<BlobRestoreController> self, StringRef errorMessage);
ACTOR static Future<Void> updateError(Reference<BlobRestoreController> self, Standalone<StringRef> errorMessage);
private:
Database db_;

View File

@ -0,0 +1,221 @@
/*
* BlobWorker.h
*
* This source file is part of the FoundationDB open source project
*
* Copyright 2013-2022 Apple Inc. and the FoundationDB project authors
*
* Licensed under the Apache License, Version 2.0 (the "License");
* you may not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
#ifndef FDBSERVER_BLOBWORKER_H
#define FDBSERVER_BLOBWORKER_H
#include "fdbclient/BlobWorkerCommon.h"
#include "fdbserver/BlobGranuleServerCommon.actor.h"
#include "fdbserver/Knobs.h"
#include <vector>
#include "flow/actorcompiler.h" // has to be last include
struct GranuleStartState {
UID granuleID;
Version changeFeedStartVersion;
Version previousDurableVersion;
Optional<std::pair<KeyRange, UID>> splitParentGranule;
bool doSnapshot;
std::vector<GranuleFiles> blobFilesToSnapshot;
Optional<GranuleFiles> existingFiles;
Optional<GranuleHistory> history;
};
// TODO: add more (blob file request cost, in-memory mutations vs blob delta file, etc...)
struct GranuleReadStats {
int64_t deltaBytesRead;
void reset() { deltaBytesRead = 0; }
GranuleReadStats() { reset(); }
};
struct GranuleMetadata : NonCopyable, ReferenceCounted<GranuleMetadata> {
KeyRange keyRange;
GranuleFiles files;
Standalone<GranuleDeltas>
currentDeltas; // only contain deltas in pendingDeltaVersion + 1 through bufferedDeltaVersion
uint64_t bytesInNewDeltaFiles = 0;
uint64_t bufferedDeltaBytes = 0;
// for client to know when it is safe to read a certain version and from where (check waitForVersion)
Version bufferedDeltaVersion; // largest delta version in currentDeltas (including empty versions)
Version pendingDeltaVersion = 0; // largest version in progress writing to s3/fdb
NotifiedVersion durableDeltaVersion; // largest version persisted in s3/fdb
NotifiedVersion durableSnapshotVersion; // same as delta vars, except for snapshots
Version pendingSnapshotVersion = 0;
Version initialSnapshotVersion = invalidVersion;
Version historyVersion = invalidVersion;
Version knownCommittedVersion;
NotifiedVersion forceFlushVersion; // Version to force a flush at, if necessary
Version forceCompactVersion = invalidVersion;
int64_t originalEpoch;
int64_t originalSeqno;
int64_t continueEpoch;
int64_t continueSeqno;
Promise<Void> cancelled;
Promise<Void> readable;
Promise<Void> historyLoaded;
Promise<Void> resumeSnapshot;
AsyncVar<Reference<ChangeFeedData>> activeCFData;
AssignBlobRangeRequest originalReq;
GranuleReadStats readStats;
bool rdcCandidate;
Promise<Void> runRDC;
void resume();
void resetReadStats();
// determine eligibility (>1) and priority for re-snapshotting this granule
double weightRDC();
bool isEligibleRDC() const;
bool updateReadStats(Version readVersion, const BlobGranuleChunkRef& chunk);
inline bool doEarlyReSnapshot() {
return runRDC.isSet() ||
(forceCompactVersion <= pendingDeltaVersion && forceCompactVersion > pendingSnapshotVersion);
}
};
struct GranuleRangeMetadata {
int64_t lastEpoch;
int64_t lastSeqno;
Reference<GranuleMetadata> activeMetadata;
Future<GranuleStartState> assignFuture;
Future<Void> fileUpdaterFuture;
Future<Void> historyLoaderFuture;
void cancel();
GranuleRangeMetadata() : lastEpoch(0), lastSeqno(0) {}
GranuleRangeMetadata(int64_t epoch, int64_t seqno, Reference<GranuleMetadata> activeMetadata)
: lastEpoch(epoch), lastSeqno(seqno), activeMetadata(activeMetadata) {}
};
// represents a previous version of a granule, and optionally the files that compose it.
struct GranuleHistoryEntry : NonCopyable, ReferenceCounted<GranuleHistoryEntry> {
KeyRange range;
UID granuleID;
Version startVersion; // version of the first snapshot
Version endVersion; // version of the last delta file
// load files lazily, and allows for clearing old cold-queried files to save memory
// FIXME: add memory limit and evictor for old cached files
Future<GranuleFiles> files;
// FIXME: do skip pointers with single back-pointer and neighbor pointers
std::vector<Reference<GranuleHistoryEntry>> parentGranules;
GranuleHistoryEntry() : startVersion(invalidVersion), endVersion(invalidVersion) {}
GranuleHistoryEntry(KeyRange range, UID granuleID, Version startVersion, Version endVersion)
: range(range), granuleID(granuleID), startVersion(startVersion), endVersion(endVersion) {}
};
struct BlobWorkerData : NonCopyable, ReferenceCounted<BlobWorkerData> {
UID id;
Database db;
IKeyValueStore* storage;
PromiseStream<Future<Void>> addActor;
LocalityData locality;
int64_t currentManagerEpoch = -1;
AsyncVar<ReplyPromiseStream<GranuleStatusReply>> currentManagerStatusStream;
bool statusStreamInitialized = false;
// FIXME: refactor out the parts of this that are just for interacting with blob stores from the backup business
// logic
Reference<BlobConnectionProvider> bstore;
KeyRangeMap<GranuleRangeMetadata> granuleMetadata;
BGTenantMap tenantData;
Reference<AsyncVar<ServerDBInfo> const> dbInfo;
// contains the history of completed granules before the existing ones. Maps to the latest one, and has
// back-pointers to earlier granules
// FIXME: expire from map after a delay when granule is revoked and the history is no longer needed
KeyRangeMap<Reference<GranuleHistoryEntry>> granuleHistory;
PromiseStream<AssignBlobRangeRequest> granuleUpdateErrors;
Promise<Void> doGRVCheck;
NotifiedVersion grvVersion;
std::deque<Version> prevGRVVersions;
Promise<Void> fatalError;
Promise<Void> simInjectFailure;
Promise<Void> doReadDrivenCompaction;
Reference<FlowLock> initialSnapshotLock;
Reference<FlowLock> resnapshotBudget;
Reference<FlowLock> deltaWritesBudget;
BlobWorkerStats stats;
bool shuttingDown = false;
// FIXME: have cap on this independent of delta file size for larger granules
int changeFeedStreamReplyBufferSize = SERVER_KNOBS->BG_DELTA_FILE_TARGET_BYTES / 4;
EncryptionAtRestMode encryptMode;
bool buggifyFull = false;
int64_t memoryFullThreshold =
(int64_t)(SERVER_KNOBS->BLOB_WORKER_REJECT_WHEN_FULL_THRESHOLD * SERVER_KNOBS->SERVER_MEM_LIMIT);
int64_t lastResidentMemory = 0;
double lastResidentMemoryCheckTime = -100.0;
bool isFullRestoreMode = false;
BlobWorkerData(UID id, Reference<AsyncVar<ServerDBInfo> const> dbInfo, Database db, IKeyValueStore* storage)
: id(id), db(db), storage(storage), tenantData(BGTenantMap(dbInfo)), dbInfo(dbInfo),
initialSnapshotLock(new FlowLock(SERVER_KNOBS->BLOB_WORKER_INITIAL_SNAPSHOT_PARALLELISM)),
resnapshotBudget(new FlowLock(SERVER_KNOBS->BLOB_WORKER_RESNAPSHOT_BUDGET_BYTES)),
deltaWritesBudget(new FlowLock(SERVER_KNOBS->BLOB_WORKER_DELTA_WRITE_BUDGET_BYTES)),
stats(id,
SERVER_KNOBS->WORKER_LOGGING_INTERVAL,
initialSnapshotLock,
resnapshotBudget,
deltaWritesBudget,
SERVER_KNOBS->LATENCY_METRICS_LOGGING_INTERVAL,
SERVER_KNOBS->FILE_LATENCY_SKETCH_ACCURACY,
SERVER_KNOBS->LATENCY_SKETCH_ACCURACY),
encryptMode(EncryptionAtRestMode::DISABLED) {}
bool managerEpochOk(int64_t epoch);
bool isFull();
void triggerReadDrivenCompaction();
void addGRVHistory(Version readVersion);
bool maybeInjectTargetedRestart();
};
#endif

View File

@ -26,6 +26,7 @@
#include <vector>
#include "fdbclient/CommitTransaction.h"
#include "fdbserver/ResolverBug.h"
struct ConflictSet;
ConflictSet* newConflictSet();
@ -63,6 +64,12 @@ private:
// Stores the map: a transaction -> conflicted transactions' indices
std::map<int, VectorRef<int>>* conflictingKeyRangeMap;
Arena* resolveBatchReplyArena;
std::shared_ptr<ResolverBug> bugs = SimBugInjector().get<ResolverBug>(ResolverBugID());
// bug injection
bool ignoreTooOld() const;
bool ignoreWriteSet() const;
bool ignoreReadSet() const;
void checkIntraBatchConflicts();
void combineWriteConflictRanges();

View File

@ -21,15 +21,353 @@
#define FOUNDATIONDB_DDRELOCATIONQUEUE_H
#include "fdbserver/DataDistribution.actor.h"
#include "fdbserver/MovingWindow.h"
// send request/signal to DDRelocationQueue through interface
// call synchronous method from components outside DDRelocationQueue
struct IDDRelocationQueue {
class IDDRelocationQueue {
public:
PromiseStream<RelocateShard> relocationProducer, relocationConsumer; // FIXME(xwang): not used yet
// PromiseStream<Promise<int>> getUnhealthyRelocationCount; // FIXME(xwang): change it to a synchronous call
virtual int getUnhealthyRelocationCount() = 0;
virtual int getUnhealthyRelocationCount() const = 0;
virtual ~IDDRelocationQueue() = default;
;
};
// DDQueue use RelocateData to track proposed movements
class RelocateData {
public:
KeyRange keys;
int priority;
int boundaryPriority;
int healthPriority;
RelocateReason reason;
double startTime;
UID randomId; // inherit from RelocateShard.traceId
UID dataMoveId;
int workFactor;
std::vector<UID> src;
std::vector<UID> completeSources;
std::vector<UID> completeDests;
bool wantsNewServers;
bool cancellable;
TraceInterval interval;
std::shared_ptr<DataMove> dataMove;
RelocateData();
explicit RelocateData(RelocateShard const& rs);
static bool isHealthPriority(int priority) {
return priority == SERVER_KNOBS->PRIORITY_POPULATE_REGION ||
priority == SERVER_KNOBS->PRIORITY_TEAM_UNHEALTHY || priority == SERVER_KNOBS->PRIORITY_TEAM_2_LEFT ||
priority == SERVER_KNOBS->PRIORITY_TEAM_1_LEFT || priority == SERVER_KNOBS->PRIORITY_TEAM_0_LEFT ||
priority == SERVER_KNOBS->PRIORITY_TEAM_REDUNDANT || priority == SERVER_KNOBS->PRIORITY_TEAM_HEALTHY ||
priority == SERVER_KNOBS->PRIORITY_TEAM_CONTAINS_UNDESIRED_SERVER ||
priority == SERVER_KNOBS->PRIORITY_PERPETUAL_STORAGE_WIGGLE;
}
static bool isBoundaryPriority(int priority) {
return priority == SERVER_KNOBS->PRIORITY_SPLIT_SHARD || priority == SERVER_KNOBS->PRIORITY_MERGE_SHARD;
}
bool isRestore() const;
bool operator>(const RelocateData& rhs) const;
bool operator==(const RelocateData& rhs) const;
bool operator!=(const RelocateData& rhs) const;
};
// DDQueue uses Busyness to throttle too many movement to/from a same server
struct Busyness {
std::vector<int> ledger;
Busyness() : ledger(10, 0) {}
bool canLaunch(int prio, int work) const;
void addWork(int prio, int work);
void removeWork(int prio, int work);
std::string toString();
};
struct DDQueueInitParams {
UID const& id;
MoveKeysLock const& lock;
Reference<IDDTxnProcessor> db;
std::vector<TeamCollectionInterface> const& teamCollections;
Reference<ShardsAffectedByTeamFailure> shardsAffectedByTeamFailure;
Reference<PhysicalShardCollection> physicalShardCollection;
PromiseStream<Promise<int64_t>> const& getAverageShardBytes;
int const& teamSize;
int const& singleRegionTeamSize;
PromiseStream<RelocateShard> const& relocationProducer;
FutureStream<RelocateShard> const& relocationConsumer;
PromiseStream<GetMetricsRequest> const& getShardMetrics;
PromiseStream<GetTopKMetricsRequest> const& getTopKMetrics;
};
// DDQueue receives RelocateShard from any other DD components and schedules the actual movements
class DDQueue : public IDDRelocationQueue, public ReferenceCounted<DDQueue> {
public:
friend struct DDQueueImpl;
typedef Reference<IDataDistributionTeam> ITeamRef;
typedef std::pair<ITeamRef, ITeamRef> SrcDestTeamPair;
struct DDDataMove {
DDDataMove() = default;
explicit DDDataMove(UID id) : id(id) {}
bool isValid() const { return id.isValid(); }
UID id;
Future<Void> cancel;
};
class ServerCounter {
public:
enum CountType : uint8_t { ProposedSource = 0, QueuedSource, LaunchedSource, LaunchedDest, __COUNT };
private:
typedef std::array<int, (int)__COUNT> Item; // one for each CountType
typedef std::array<Item, RelocateReason::typeCount()> ReasonItem; // one for each RelocateReason
std::unordered_map<UID, ReasonItem> counter;
std::string toString(const Item& item) const {
return format("%d %d %d %d", item[0], item[1], item[2], item[3]);
}
void traceReasonItem(TraceEvent* event, const ReasonItem& item) const {
for (int i = 0; i < item.size(); ++i) {
if (std::accumulate(item[i].cbegin(), item[i].cend(), 0) > 0) {
// "PQSD" corresponding to CounterType
event->detail(RelocateReason(i).toString() + "PQSD", toString(item[i]));
}
}
}
bool countNonZero(const ReasonItem& item, CountType type) const {
return std::any_of(item.cbegin(), item.cend(), [type](const Item& item) { return item[(int)type] > 0; });
}
void increase(const UID& id, RelocateReason reason, CountType type) {
int idx = (int)(reason);
// if (idx < 0 || idx >= RelocateReason::typeCount()) {
// TraceEvent(SevWarnAlways, "ServerCounterDebug").detail("Reason", reason.toString());
// }
ASSERT(idx >= 0 && idx < RelocateReason::typeCount());
counter[id][idx][(int)type] += 1;
}
void summarizeLaunchedServers(decltype(counter.cbegin()) begin,
decltype(counter.cend()) end,
TraceEvent* event) const {
if (begin == end)
return;
std::string execSrc, execDest;
for (; begin != end; ++begin) {
if (countNonZero(begin->second, LaunchedSource)) {
execSrc += begin->first.shortString() + ",";
}
if (countNonZero(begin->second, LaunchedDest)) {
execDest += begin->first.shortString() + ",";
}
}
event->detail("RemainedLaunchedSources", execSrc).detail("RemainedLaunchedDestinations", execDest);
}
public:
void clear() { counter.clear(); }
int get(const UID& id, RelocateReason reason, CountType type) const {
return counter.at(id)[(int)reason][(int)type];
}
void increaseForTeam(const std::vector<UID>& ids, RelocateReason reason, CountType type) {
for (auto& id : ids) {
increase(id, reason, type);
}
}
void traceAll(const UID& debugId = UID()) const {
auto it = counter.cbegin();
int count = 0;
for (; count < SERVER_KNOBS->DD_QUEUE_COUNTER_MAX_LOG && it != counter.cend(); ++count, ++it) {
TraceEvent event("DDQueueServerCounter", debugId);
event.detail("ServerId", it->first);
traceReasonItem(&event, it->second);
}
if (it != counter.cend()) {
TraceEvent e(SevWarn, "DDQueueServerCounterTooMany", debugId);
e.detail("Servers", size());
if (SERVER_KNOBS->DD_QUEUE_COUNTER_SUMMARIZE) {
summarizeLaunchedServers(it, counter.cend(), &e);
return;
}
}
}
size_t size() const { return counter.size(); }
// for random test
static CountType randomCountType() {
int i = deterministicRandom()->randomInt(0, (int)__COUNT);
return (CountType)i;
}
};
ActorCollectionNoErrors noErrorActors; // has to be the last one to be destroyed because other Actors may use it.
UID distributorId;
MoveKeysLock lock;
Database cx;
Reference<IDDTxnProcessor> txnProcessor;
std::vector<TeamCollectionInterface> teamCollections;
Reference<ShardsAffectedByTeamFailure> shardsAffectedByTeamFailure;
Reference<PhysicalShardCollection> physicalShardCollection;
PromiseStream<Promise<int64_t>> getAverageShardBytes;
FlowLock startMoveKeysParallelismLock;
FlowLock finishMoveKeysParallelismLock;
FlowLock cleanUpDataMoveParallelismLock;
Reference<FlowLock> fetchSourceLock;
int activeRelocations;
int queuedRelocations;
int64_t bytesWritten;
int teamSize;
int singleRegionTeamSize;
std::map<UID, Busyness> busymap; // UID is serverID
std::map<UID, Busyness> destBusymap; // UID is serverID
KeyRangeMap<RelocateData> queueMap;
std::set<RelocateData, std::greater<RelocateData>> fetchingSourcesQueue;
std::set<RelocateData, std::greater<RelocateData>> fetchKeysComplete;
KeyRangeActorMap getSourceActors;
std::map<UID, std::set<RelocateData, std::greater<RelocateData>>>
queue; // Key UID is serverID, value is the serverID's set of RelocateData to relocate
// The last time one server was selected as source team for read rebalance reason. We want to throttle read
// rebalance on time bases because the read workload sample update has delay after the previous moving
std::map<UID, double> lastAsSource;
ServerCounter serverCounter;
KeyRangeMap<RelocateData> inFlight;
// Track all actors that relocates specified keys to a good place; Key: keyRange; Value: actor
KeyRangeActorMap inFlightActors;
KeyRangeMap<DDDataMove> dataMoves;
Promise<Void> error;
PromiseStream<RelocateData> dataTransferComplete;
PromiseStream<RelocateData> relocationComplete;
PromiseStream<RelocateData> fetchSourceServersComplete; // find source SSs for a relocate range
PromiseStream<RelocateShard> output;
FutureStream<RelocateShard> input;
PromiseStream<GetMetricsRequest> getShardMetrics;
PromiseStream<GetTopKMetricsRequest> getTopKMetrics;
double lastInterval;
int suppressIntervals;
Reference<AsyncVar<bool>> rawProcessingUnhealthy; // many operations will remove relocations before adding a new
// one, so delay a small time before settling on a new number.
Reference<AsyncVar<bool>> rawProcessingWiggle;
std::map<int, int> priority_relocations;
int unhealthyRelocations;
Reference<EventCacheHolder> movedKeyServersEventHolder;
int moveReusePhysicalShard;
int moveCreateNewPhysicalShard;
enum RetryFindDstReason {
None = 0,
RemoteBestTeamNotReady,
PrimaryNoHealthyTeam,
RemoteNoHealthyTeam,
RemoteTeamIsFull,
RemoteTeamIsNotHealthy,
NoAvailablePhysicalShard,
UnknownForceNew,
NoAnyHealthy,
DstOverloaded,
RetryLimitReached,
NumberOfTypes,
};
std::vector<int> retryFindDstReasonCount;
MovingWindow<int64_t> moveBytesRate;
DDQueue() = default;
void startRelocation(int priority, int healthPriority);
void finishRelocation(int priority, int healthPriority);
void validate();
// This function cannot handle relocation requests which split a shard into three pieces
void queueRelocation(RelocateShard rs, std::set<UID>& serversToLaunchFrom);
void completeSourceFetch(const RelocateData& results);
void logRelocation(const RelocateData& rd, const char* title);
void launchQueuedWork(KeyRange keys, const DDEnabledState* ddEnabledState);
void launchQueuedWork(const std::set<UID>& serversToLaunchFrom, const DDEnabledState* ddEnabledState);
void launchQueuedWork(RelocateData launchData, const DDEnabledState* ddEnabledState);
// For each relocateData rd in the queue, check if there exist inflight relocate data whose keyrange is overlapped
// with rd. If there exist, cancel them by cancelling their actors and reducing the src servers' busyness of those
// canceled inflight relocateData. Launch the relocation for the rd.
void launchQueuedWork(std::set<RelocateData, std::greater<RelocateData>> combined,
const DDEnabledState* ddEnabledState);
int getHighestPriorityRelocation() const;
// return true if the servers are throttled as source for read rebalance
bool timeThrottle(const std::vector<UID>& ids) const;
void updateLastAsSource(const std::vector<UID>& ids, double t = now());
// Schedules cancellation of a data move.
void enqueueCancelledDataMove(UID dataMoveId, KeyRange range, const DDEnabledState* ddEnabledState);
Future<Void> periodicalRefreshCounter();
int getUnhealthyRelocationCount() const override;
Future<SrcDestTeamPair> getSrcDestTeams(const int& teamCollectionIndex,
const GetTeamRequest& srcReq,
const GetTeamRequest& destReq,
const int& priority,
TraceEvent* traceEvent);
Future<bool> rebalanceReadLoad(DataMovementReason moveReason,
Reference<IDataDistributionTeam> sourceTeam,
Reference<IDataDistributionTeam> destTeam,
bool primary,
TraceEvent* traceEvent);
Future<bool> rebalanceTeams(DataMovementReason moveReason,
Reference<IDataDistributionTeam const> sourceTeam,
Reference<IDataDistributionTeam const> destTeam,
bool primary,
TraceEvent* traceEvent);
static Future<Void> run(Reference<DDQueue> self,
Reference<AsyncVar<bool>> processingUnhealthy,
Reference<AsyncVar<bool>> processingWiggle,
FutureStream<Promise<int>> getUnhealthyRelocationCount,
const DDEnabledState* ddEnabledState);
explicit DDQueue(DDQueueInitParams const& params);
};
#endif // FOUNDATIONDB_DDRELOCATIONQUEUE_H

View File

@ -23,18 +23,107 @@
// send request/signal to DDTracker through interface
// call synchronous method from components outside DDShardTracker
struct IDDShardTracker {
// FIXME: the streams are not used yet
class IDDShardTracker {
public:
Promise<Void> readyToStart;
PromiseStream<GetMetricsRequest> getShardMetrics;
PromiseStream<GetTopKMetricsRequest> getTopKMetrics;
PromiseStream<GetMetricsListRequest> getShardMetricsList;
PromiseStream<KeyRange> restartShardTracker;
// PromiseStream<Promise<int64_t>> averageShardBytes; // FIXME(xwang): change it to a synchronous call
FutureStream<GetMetricsRequest> getShardMetrics;
FutureStream<GetTopKMetricsRequest> getTopKMetrics;
FutureStream<GetMetricsListRequest> getShardMetricsList;
FutureStream<Promise<int64_t>> averageShardBytes;
virtual double getAverageShardBytes() = 0;
virtual ~IDDShardTracker() = default;
};
struct DataDistributionTrackerInitParams {
Reference<IDDTxnProcessor> db;
UID const& distributorId;
Promise<Void> const& readyToStart;
PromiseStream<RelocateShard> const& output;
Reference<ShardsAffectedByTeamFailure> shardsAffectedByTeamFailure;
Reference<PhysicalShardCollection> physicalShardCollection;
Reference<AsyncVar<bool>> anyZeroHealthyTeams;
KeyRangeMap<ShardTrackedData>* shards = nullptr;
bool* trackerCancelled = nullptr;
Optional<Reference<TenantCache>> ddTenantCache;
};
// track the status of shards
class DataDistributionTracker : public IDDShardTracker, public ReferenceCounted<DataDistributionTracker> {
public:
friend struct DataDistributionTrackerImpl;
Reference<IDDTxnProcessor> db;
UID distributorId;
// At now, the lifetime of shards is guaranteed longer than DataDistributionTracker.
KeyRangeMap<ShardTrackedData>* shards = nullptr;
ActorCollection actors;
int64_t systemSizeEstimate = 0;
Reference<AsyncVar<int64_t>> dbSizeEstimate;
Reference<AsyncVar<Optional<int64_t>>> maxShardSize;
Future<Void> maxShardSizeUpdater;
// CapacityTracker
PromiseStream<RelocateShard> output;
Reference<ShardsAffectedByTeamFailure> shardsAffectedByTeamFailure;
// PhysicalShard Tracker
Reference<PhysicalShardCollection> physicalShardCollection;
Promise<Void> readyToStart;
Reference<AsyncVar<bool>> anyZeroHealthyTeams;
// Read hot detection
PromiseStream<KeyRange> readHotShard;
// The reference to trackerCancelled must be extracted by actors,
// because by the time (trackerCancelled == true) this memory cannot
// be accessed
bool* trackerCancelled = nullptr;
// This class extracts the trackerCancelled reference from a DataDistributionTracker object
// Because some actors spawned by the dataDistributionTracker outlive the DataDistributionTracker
// object, we must guard against memory errors by using a GetTracker functor to access
// the DataDistributionTracker object.
class SafeAccessor {
bool const& trackerCancelled;
DataDistributionTracker& tracker;
public:
SafeAccessor(DataDistributionTracker* tracker)
: trackerCancelled(*tracker->trackerCancelled), tracker(*tracker) {
ASSERT(!trackerCancelled);
}
DataDistributionTracker* operator()() {
if (trackerCancelled) {
CODE_PROBE(true, "Trying to access DataDistributionTracker after tracker has been cancelled");
throw dd_tracker_cancelled();
}
return &tracker;
}
};
Optional<Reference<TenantCache>> ddTenantCache;
Reference<KeyRangeMap<int>> customReplication;
DataDistributionTracker() = default;
~DataDistributionTracker() override;
double getAverageShardBytes() override { return maxShardSize->get().get() / 2.0; }
static Future<Void> run(Reference<DataDistributionTracker> self,
Reference<InitialDataDistribution> const& initData,
FutureStream<GetMetricsRequest> const& getShardMetrics,
FutureStream<GetTopKMetricsRequest> const& getTopKMetrics,
FutureStream<GetMetricsListRequest> const& getShardMetricsList,
FutureStream<Promise<int64_t>> const& getAverageShardBytes);
explicit DataDistributionTracker(DataDistributionTrackerInitParams const& params);
};
#endif // FOUNDATIONDB_DDSHARDTRACKER_H

View File

@ -64,7 +64,9 @@ public:
void proposeRelocation(const RelocateShard& rs) const { return relocationQueue->relocationProducer.send(rs); }
void requestRestartShardTracker(KeyRange keys) const { return shardTracker->restartShardTracker.send(keys); }
void requestRestartShardTracker(KeyRange keys) const {
return shardsAffectedByTeamFailure->restartShardTracker.send(keys);
}
};
#endif // FOUNDATIONDB_DDSHAREDCONTEXT_H

View File

@ -239,6 +239,7 @@ protected:
PromiseStream<Promise<int64_t>> getAverageShardBytes;
std::vector<Reference<TCTeamInfo>> badTeams;
std::vector<Reference<TCTeamInfo>> largeTeams;
Reference<ShardsAffectedByTeamFailure> shardsAffectedByTeamFailure;
PromiseStream<UID> removedServers;
PromiseStream<UID> removedTSS;
@ -269,6 +270,7 @@ protected:
Reference<AsyncVar<bool>> processingUnhealthy;
Future<Void> readyToStart;
Future<Void> checkTeamDelay;
Optional<double> firstLargeTeamFailure;
Promise<Void> addSubsetComplete;
Future<Void> badTeamRemover;
Future<Void> checkInvalidLocalities;
@ -302,6 +304,9 @@ protected:
LocalityMap<UID> machineLocalityMap; // locality info of machines
Reference<KeyRangeMap<int>> customReplication;
CoalescedKeyRangeMap<bool> underReplication;
// A mechanism to tell actors that reference a DDTeamCollection object through a direct
// pointer (without doing reference counting) that the object is being destroyed.
// (Introduced to solve the problem of "self" getting destroyed from underneath the
@ -532,6 +537,10 @@ protected:
Future<Void> removeWrongStoreType();
Future<Void> fixUnderReplicationLoop();
void fixUnderReplication();
// Check if the number of server (and machine teams) is larger than the maximum allowed number
void traceTeamCollectionInfo() const;
@ -628,6 +637,12 @@ protected:
// build an extra machine team and record the event in trace
int addTeamsBestOf(int teamsToBuild, int desiredTeams, int maxTeams);
void cleanupLargeTeams();
int maxLargeTeamSize() const;
Reference<TCTeamInfo> buildLargeTeam(int size);
public:
Reference<IDDTxnProcessor> db;

View File

@ -478,7 +478,8 @@ struct DDShardInfo {
};
struct InitialDataDistribution : ReferenceCounted<InitialDataDistribution> {
InitialDataDistribution() : dataMoveMap(std::make_shared<DataMove>()) {}
InitialDataDistribution()
: dataMoveMap(std::make_shared<DataMove>()), customReplication(makeReference<KeyRangeMap<int>>(-1)) {}
// Read from dataDistributionModeKey. Whether DD is disabled. DD can be disabled persistently (mode = 0). Set mode
// to 1 will enable all disabled parts
@ -490,6 +491,7 @@ struct InitialDataDistribution : ReferenceCounted<InitialDataDistribution> {
Optional<Key> initHealthyZoneValue; // set for maintenance mode
KeyRangeMap<std::shared_ptr<DataMove>> dataMoveMap;
std::vector<AuditStorageState> auditStates;
Reference<KeyRangeMap<int>> customReplication;
};
// Holds the permitted size and IO Bounds for a shard
@ -511,6 +513,8 @@ ShardSizeBounds getShardSizeBounds(KeyRangeRef shard, int64_t maxShardSize);
// Determines the maximum shard size based on the size of the database
int64_t getMaxShardSize(double dbSizeEstimate);
bool ddLargeTeamEnabled();
#ifndef __INTEL_COMPILER
#pragma endregion
#endif
@ -525,39 +529,6 @@ struct TeamCollectionInterface {
PromiseStream<GetTeamRequest> getTeam;
};
ACTOR Future<Void> dataDistributionTracker(Reference<InitialDataDistribution> initData,
Reference<IDDTxnProcessor> db,
PromiseStream<RelocateShard> output,
Reference<ShardsAffectedByTeamFailure> shardsAffectedByTeamFailure,
Reference<PhysicalShardCollection> physicalShardCollection,
PromiseStream<GetMetricsRequest> getShardMetrics,
FutureStream<GetTopKMetricsRequest> getTopKMetrics,
PromiseStream<GetMetricsListRequest> getShardMetricsList,
FutureStream<Promise<int64_t>> getAverageShardBytes,
Promise<Void> readyToStart,
Reference<AsyncVar<bool>> zeroHealthyTeams,
UID distributorId,
KeyRangeMap<ShardTrackedData>* shards,
bool* trackerCancelled,
Optional<Reference<TenantCache>> ddTenantCache);
ACTOR Future<Void> dataDistributionQueue(Reference<IDDTxnProcessor> db,
PromiseStream<RelocateShard> output,
FutureStream<RelocateShard> input,
PromiseStream<GetMetricsRequest> getShardMetrics,
PromiseStream<GetTopKMetricsRequest> getTopKMetrics,
Reference<AsyncVar<bool>> processingUnhealthy,
Reference<AsyncVar<bool>> processingWiggle,
std::vector<TeamCollectionInterface> teamCollections,
Reference<ShardsAffectedByTeamFailure> shardsAffectedByTeamFailure,
Reference<PhysicalShardCollection> physicalShardCollection,
MoveKeysLock lock,
PromiseStream<Promise<int64_t>> getAverageShardBytes,
FutureStream<Promise<int>> getUnhealthyRelocationCount,
UID distributorId,
int teamSize,
int singleRegionTeamSize,
const DDEnabledState* ddEnabledState);
#ifndef __INTEL_COMPILER
#pragma endregion
#endif

View File

@ -78,6 +78,7 @@ struct GetTeamRequest {
bool preferLowerReadUtil; // only make sense when forReadBalance is true
double inflightPenalty;
bool findTeamByServers;
Optional<KeyRange> keys;
std::vector<UID> completeSources;
std::vector<UID> src;
Promise<std::pair<Optional<Reference<IDataDistributionTeam>>, bool>> reply;
@ -91,11 +92,12 @@ struct GetTeamRequest {
TeamMustHaveShards teamMustHaveShards,
ForReadBalance forReadBalance = ForReadBalance::False,
PreferLowerReadUtil preferLowerReadUtil = PreferLowerReadUtil::False,
double inflightPenalty = 1.0)
double inflightPenalty = 1.0,
Optional<KeyRange> keys = Optional<KeyRange>())
: wantsNewServers(wantsNewServers), wantsTrueBest(wantsTrueBest), preferLowerDiskUtil(preferLowerDiskUtil),
teamMustHaveShards(teamMustHaveShards), forReadBalance(forReadBalance),
preferLowerReadUtil(preferLowerReadUtil), inflightPenalty(inflightPenalty),
findTeamByServers(FindTeamByServers::False) {}
findTeamByServers(FindTeamByServers::False), keys(keys) {}
GetTeamRequest(std::vector<UID> servers)
: wantsNewServers(WantNewServers::False), wantsTrueBest(WantTrueBest::False),
preferLowerDiskUtil(PreferLowerDiskUtil::False), teamMustHaveShards(TeamMustHaveShards::False),

View File

@ -0,0 +1,129 @@
/*
* ExclusionTracker.actor.h
*
* This source file is part of the FoundationDB open source project
*
* Copyright 2013-2022 Apple Inc. and the FoundationDB project authors
*
* Licensed under the Apache License, Version 2.0 (the "License");
* you may not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
#pragma once
// When actually compiled (NO_INTELLISENSE), include the generated version of this file. In intellisense use the source
// version.
#if defined(NO_INTELLISENSE) && !defined(EXCLUSION_TRACKER_ACTOR_G_H)
#define EXCLUSION_TRACKER_ACTOR_G_H
#include "fdbserver/ExclusionTracker.actor.g.h"
#elif !defined(EXCLUSION_TRACKER_ACTOR_H)
#define EXCLUSION_TRACKER_ACTOR_H
#include <set>
#include "flow/flow.h"
#include "fdbclient/DatabaseContext.h"
#include "fdbclient/ManagementAPI.actor.h"
#include "flow/actorcompiler.h" // This must be the last #include.
struct ExclusionTracker {
std::set<AddressExclusion> excluded;
std::set<AddressExclusion> failed;
AsyncTrigger changed;
Database db;
Future<Void> trackerFuture;
ExclusionTracker() {}
ExclusionTracker(Database db) : db(db) { trackerFuture = tracker(this); }
bool isFailedOrExcluded(NetworkAddress addr) {
AddressExclusion addrExclusion(addr.ip, addr.port);
return excluded.count(addrExclusion) || failed.count(addrExclusion);
}
ACTOR static Future<Void> tracker(ExclusionTracker* self) {
// Fetch the list of excluded servers
state ReadYourWritesTransaction tr(self->db);
loop {
try {
tr.setOption(FDBTransactionOptions::ACCESS_SYSTEM_KEYS);
tr.setOption(FDBTransactionOptions::PRIORITY_SYSTEM_IMMEDIATE);
tr.setOption(FDBTransactionOptions::LOCK_AWARE);
state Future<RangeResult> fresultsExclude = tr.getRange(excludedServersKeys, CLIENT_KNOBS->TOO_MANY);
state Future<RangeResult> fresultsFailed = tr.getRange(failedServersKeys, CLIENT_KNOBS->TOO_MANY);
state Future<RangeResult> flocalitiesExclude =
tr.getRange(excludedLocalityKeys, CLIENT_KNOBS->TOO_MANY);
state Future<RangeResult> flocalitiesFailed = tr.getRange(failedLocalityKeys, CLIENT_KNOBS->TOO_MANY);
state Future<std::vector<ProcessData>> fworkers = getWorkers(&tr.getTransaction());
wait(success(fresultsExclude) && success(fresultsFailed) && success(flocalitiesExclude) &&
success(flocalitiesFailed));
state RangeResult excludedResults = fresultsExclude.get();
ASSERT(!excludedResults.more && excludedResults.size() < CLIENT_KNOBS->TOO_MANY);
state RangeResult failedResults = fresultsFailed.get();
ASSERT(!failedResults.more && failedResults.size() < CLIENT_KNOBS->TOO_MANY);
state RangeResult excludedLocalityResults = flocalitiesExclude.get();
ASSERT(!excludedLocalityResults.more && excludedLocalityResults.size() < CLIENT_KNOBS->TOO_MANY);
state RangeResult failedLocalityResults = flocalitiesFailed.get();
ASSERT(!failedLocalityResults.more && failedLocalityResults.size() < CLIENT_KNOBS->TOO_MANY);
state std::set<AddressExclusion> newExcluded;
state std::set<AddressExclusion> newFailed;
for (const auto& r : excludedResults) {
AddressExclusion addr = decodeExcludedServersKey(r.key);
if (addr.isValid()) {
newExcluded.insert(addr);
}
}
for (const auto& r : failedResults) {
AddressExclusion addr = decodeFailedServersKey(r.key);
if (addr.isValid()) {
newFailed.insert(addr);
}
}
wait(success(fworkers));
std::vector<ProcessData> workers = fworkers.get();
for (const auto& r : excludedLocalityResults) {
std::string locality = decodeExcludedLocalityKey(r.key);
std::set<AddressExclusion> localityExcludedAddresses = getAddressesByLocality(workers, locality);
newExcluded.insert(localityExcludedAddresses.begin(), localityExcludedAddresses.end());
}
for (const auto& r : failedLocalityResults) {
std::string locality = decodeFailedLocalityKey(r.key);
std::set<AddressExclusion> localityFailedAddresses = getAddressesByLocality(workers, locality);
newFailed.insert(localityFailedAddresses.begin(), localityFailedAddresses.end());
}
self->excluded = newExcluded;
self->failed = newFailed;
self->changed.trigger();
state Future<Void> watchFuture =
tr.watch(excludedServersVersionKey) || tr.watch(failedServersVersionKey) ||
tr.watch(excludedLocalityVersionKey) || tr.watch(failedLocalityVersionKey);
wait(tr.commit());
wait(watchFuture);
tr.reset();
} catch (Error& e) {
wait(tr.onError(e));
}
}
}
};
#include "flow/unactorcompiler.h"
#endif

View File

@ -555,7 +555,7 @@ struct ILogSystem {
// Same contract as peek(), but blocks until the preferred log server(s) for the given tag are available (and is
// correspondingly less expensive)
virtual Reference<IPeekCursor> peekLogRouter(UID dbgid, Version begin, Tag tag) = 0;
virtual Reference<IPeekCursor> peekLogRouter(UID dbgid, Version begin, Tag tag, bool useSatellite) = 0;
// Same contract as peek(), but can only peek from the logs elected in the same generation.
// If the preferred log server is down, a different log from the same generation will merge results locally before
// sending them to the log router.

View File

@ -0,0 +1,80 @@
/*
* MovingWindow.h
*
* This source file is part of the FoundationDB open source project
*
* Copyright 2013-2023 Apple Inc. and the FoundationDB project authors
*
* Licensed under the Apache License, Version 2.0 (the "License");
* you may not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
#pragma once
#ifndef FOUNDATIONDB_MOVINGWINDOW_H
#define FOUNDATIONDB_MOVINGWINDOW_H
#include <limits.h>
#include "flow/Deque.h"
#include "fdbserver/Knobs.h"
// Perfomed as the rolling window to calculate average change rates in the past <interval>
// e.g., we may use it in "MovingData" Trace to show average moving bytes rate by DD.
template <class T>
class MovingWindow {
private:
T previous;
T total;
// To avoid having a super large Deque which may lead OOM, we set a maxSize for it.
// Actually, Deque has its own Deque::max_size = 1 << 30, We may narrow it down here.
int maxDequeSize;
Deque<std::pair<double, T>> updates; // pair{time, numeric}
double interval;
// Updated when initialization Or pop() due to full Deque
double previousPopTime;
void pop() {
previous += updates.front().second;
updates.pop_front();
}
public:
MovingWindow() = default;
explicit MovingWindow(double timeWindow)
: previous(0), total(0), maxDequeSize(SERVER_KNOBS->MOVING_WINDOW_SAMPLE_SIZE / sizeof(std::pair<double, T>)),
interval(timeWindow), previousPopTime(now()) {}
T getTotal() const { return total; }
double getAverage() {
if (now() - interval <= previousPopTime) { // struct is just initialized Or pop() due to full
return (total - previous) / (now() - previousPopTime);
} else {
while (!updates.empty() && updates.front().first < now() - interval) {
pop();
}
return (total - previous) / interval;
}
}
void addSample(T sample) {
total += sample;
updates.push_back(std::make_pair(now(), sample));
// If so, we would pop the front element from the Deque.
while (updates.size() > maxDequeSize) {
previousPopTime = updates.front().first;
pop();
}
}
};
#endif // FOUNDATIONDB_MOVINGWINDOW_H

View File

@ -0,0 +1,44 @@
/*
* ResolverBug.h
*
* This source file is part of the FoundationDB open source project
*
* Copyright 2013-2023 Apple Inc. and the FoundationDB project authors
*
* Licensed under the Apache License, Version 2.0 (the "License");
* you may not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
#ifndef FDBSERVER_RESOLVER_BUG_H
#define FDBSERVER_RESOLVER_BUG_H
#pragma once
#include "flow/SimBugInjector.h"
#include <vector>
struct ResolverBug : public ISimBug {
double ignoreTooOldProbability = 0.0;
double ignoreWriteSetProbability = 0.0;
double ignoreReadSetProbability = 0.0;
// data used to control lifetime of cycle clients
bool bugFound = false;
unsigned currentPhase = 0;
std::vector<unsigned> cycleState;
};
class ResolverBugID : public IBugIdentifier {
public:
std::shared_ptr<ISimBug> create() const override;
};
#endif // FDBSERVER_RESOLVER_BUG_H

View File

@ -31,6 +31,15 @@
#include "flow/actorcompiler.h" // has to be last include
class ICheckpointByteSampleReader {
public:
virtual ~ICheckpointByteSampleReader() {}
virtual KeyValue next() = 0;
virtual bool hasNext() const = 0;
};
class IRocksDBSstFileWriter {
public:
virtual void open(const std::string localFile) = 0;
@ -296,6 +305,8 @@ ICheckpointReader* newRocksDBCheckpointReader(const CheckpointMetaData& checkpoi
const CheckpointAsKeyValues checkpointAsKeyValues,
UID logID);
std::unique_ptr<ICheckpointByteSampleReader> newCheckpointByteSampleReader(const CheckpointMetaData& checkpoint);
std::unique_ptr<IRocksDBSstFileWriter> newRocksDBSstFileWriter();
RocksDBColumnFamilyCheckpoint getRocksCF(const CheckpointMetaData& checkpoint);

View File

@ -87,6 +87,8 @@ ACTOR Future<CheckpointMetaData> fetchCheckpointRanges(
std::vector<KeyRange> ranges,
std::function<Future<Void>(const CheckpointMetaData&)> cFun = nullptr);
std::string serverCheckpointDir(const std::string& baseDir, const UID& checkpointId);
std::string fetchedCheckpointDir(const std::string& baseDir, const UID& checkpointId);
#include "flow/unactorcompiler.h"
#endif

View File

@ -78,6 +78,7 @@ public:
// intersecting shards.
int getNumberOfShards(UID ssID) const;
int getNumberOfShards(Team team) const;
std::vector<KeyRange> getShardsFor(Team team) const;
bool hasShards(Team team) const;

View File

@ -1,5 +1,5 @@
/*
* genericactors.actor.cpp
* StorageCorruptionBug.h
*
* This source file is part of the FoundationDB open source project
*
@ -17,19 +17,18 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
#pragma once
#ifndef FDBSERVER_STORAGE_CORRUPTION_BUG_H
#define FDBSERVER_STORAGE_CORRUPTION_BUG_H
#include "flow/SimBugInjector.h"
#include "flow/flow.h"
#include "fdbrpc/genericactors.actor.h" // Gets genericactors.actor.g.h indirectly
#include "flow/network.h"
#include "fdbrpc/simulator.h"
#include "flow/actorcompiler.h"
class StorageCorruptionBug : public ISimBug {
public:
double corruptionProbability = 0.001;
};
class StorageCorruptionBugID : public IBugIdentifier {
public:
std::shared_ptr<ISimBug> create() const override { return std::make_shared<StorageCorruptionBug>(); }
};
ACTOR Future<Void> disableConnectionFailuresAfter(double time, std::string context) {
if (g_network->isSimulated()) {
wait(delayUntil(time));
g_simulator->connectionFailuresDisableDuration = 1e6;
g_simulator->speedUpSimulation = true;
TraceEvent(SevWarnAlways, ("DisableConnectionFailures_" + context).c_str());
}
return Void();
}
#endif // FDBSERVER_STORAGE_CORRUPTION_BUG_H

View File

@ -232,7 +232,7 @@ struct TagPartitionedLogSystem final : ILogSystem, ReferenceCounted<TagPartition
// Specifically, the epoch is determined by looking up "dbgid" in tlog sets of generations.
// The returned cursor can peek data at the "tag" from the given "begin" version to that epoch's end version or
// the recovery version for the latest old epoch. For the current epoch, the cursor has no end version.
Reference<IPeekCursor> peekLogRouter(UID dbgid, Version begin, Tag tag) final;
Reference<IPeekCursor> peekLogRouter(UID dbgid, Version begin, Tag tag, bool useSatellite) final;
Version getKnownCommittedVersion() final;

View File

@ -195,6 +195,15 @@ private:
ASSERT(data.metaclusterRegistration.get().name == clusterName);
ASSERT(data.metaclusterRegistration.get().id == clusterMetadata.entry.id);
if (data.tenantData.lastTenantId >= 0) {
ASSERT_EQ(TenantAPI::getTenantIdPrefix(data.tenantData.lastTenantId), managementData.tenantIdPrefix);
ASSERT_LE(data.tenantData.lastTenantId, managementData.tenantData.lastTenantId);
} else {
for (auto const& [id, tenant] : data.tenantData.tenantMap) {
ASSERT_NE(TenantAPI::getTenantIdPrefix(id), managementData.tenantIdPrefix);
}
}
std::set<int64_t> expectedTenants;
auto clusterTenantMapItr = managementData.clusterTenantMap.find(clusterName);
if (clusterTenantMapItr != managementData.clusterTenantMap.end()) {

View File

@ -62,11 +62,8 @@ private:
ASSERT_EQ(tenantId, tenantMapEntry.id);
ASSERT_EQ(tenantData.tenantNameIndex[tenantMapEntry.tenantName], tenantId);
// Data clusters do not keep their last tenant ID up to date while part of a metacluster
if (tenantData.clusterType != ClusterType::METACLUSTER_DATA) {
if (TenantAPI::getTenantIdPrefix(tenantId) == TenantAPI::getTenantIdPrefix(tenantData.lastTenantId)) {
ASSERT_LE(tenantId, tenantData.lastTenantId);
}
if (TenantAPI::getTenantIdPrefix(tenantId) == TenantAPI::getTenantIdPrefix(tenantData.lastTenantId)) {
ASSERT_LE(tenantId, tenantData.lastTenantId);
}
if (tenantMapEntry.tenantGroup.present()) {

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