This commit is contained in:
Nim Wijetunga 2022-08-05 18:52:39 -07:00
commit 369494ab19
125 changed files with 8115 additions and 1302 deletions

View File

@ -63,7 +63,9 @@ struct Tuple {
Tuple& appendNull(); Tuple& appendNull();
Tuple& appendVersionstamp(Versionstamp const&); Tuple& appendVersionstamp(Versionstamp const&);
StringRef pack() const { return StringRef(data.begin(), data.size()); } Standalone<StringRef> pack() const {
return Standalone<StringRef>(StringRef(data.begin(), data.size()), data.arena());
}
template <typename T> template <typename T>
Tuple& operator<<(T const& t) { Tuple& operator<<(T const& t) {

View File

@ -66,6 +66,9 @@ def test_size_limit_option(db):
except fdb.FDBError as e: except fdb.FDBError as e:
assert(e.code == 2101) # Transaction exceeds byte limit (2101) assert(e.code == 2101) # Transaction exceeds byte limit (2101)
# Reset the size limit for future tests
db.options.set_transaction_size_limit(10000000)
@fdb.transactional @fdb.transactional
def test_get_approximate_size(tr): def test_get_approximate_size(tr):
tr[b'key1'] = b'value1' tr[b'key1'] = b'value1'

View File

@ -788,6 +788,13 @@ namespace SummarizeTest
string firstRetryableError = ""; string firstRetryableError = "";
int stderrSeverity = (int)Magnesium.Severity.SevError; int stderrSeverity = (int)Magnesium.Severity.SevError;
xout.Add(new XAttribute("DeterminismCheck", expectedUnseed != -1 ? "1" : "0"));
xout.Add(new XAttribute("OldBinary", Path.GetFileName(oldBinaryName)));
if (traceFiles.Length == 0) {
xout.Add(new XElement("NoTraceFilesFound"));
}
Dictionary<KeyValuePair<string, Magnesium.Severity>, Magnesium.Severity> severityMap = new Dictionary<KeyValuePair<string, Magnesium.Severity>, Magnesium.Severity>(); Dictionary<KeyValuePair<string, Magnesium.Severity>, Magnesium.Severity> severityMap = new Dictionary<KeyValuePair<string, Magnesium.Severity>, Magnesium.Severity>();
var codeCoverage = new Dictionary<Tuple<string, string, string>, bool>(); var codeCoverage = new Dictionary<Tuple<string, string, string>, bool>();
@ -824,9 +831,7 @@ namespace SummarizeTest
new XAttribute("RandomSeed", ev.Details.RandomSeed), new XAttribute("RandomSeed", ev.Details.RandomSeed),
new XAttribute("SourceVersion", ev.Details.SourceVersion), new XAttribute("SourceVersion", ev.Details.SourceVersion),
new XAttribute("Time", ev.Details.ActualTime), new XAttribute("Time", ev.Details.ActualTime),
new XAttribute("BuggifyEnabled", ev.Details.BuggifyEnabled), new XAttribute("BuggifyEnabled", ev.Details.BuggifyEnabled));
new XAttribute("DeterminismCheck", expectedUnseed != -1 ? "1" : "0"),
new XAttribute("OldBinary", Path.GetFileName(oldBinaryName)));
testBeginFound = true; testBeginFound = true;
if (ev.DDetails.ContainsKey("FaultInjectionEnabled")) if (ev.DDetails.ContainsKey("FaultInjectionEnabled"))
xout.Add(new XAttribute("FaultInjectionEnabled", ev.Details.FaultInjectionEnabled)); xout.Add(new XAttribute("FaultInjectionEnabled", ev.Details.FaultInjectionEnabled));
@ -959,6 +964,12 @@ namespace SummarizeTest
xout.Add(new XElement(externalError, new XAttribute("Severity", (int)Magnesium.Severity.SevError))); xout.Add(new XElement(externalError, new XAttribute("Severity", (int)Magnesium.Severity.SevError)));
} }
string joshuaSeed = System.Environment.GetEnvironmentVariable("JOSHUA_SEED");
if (joshuaSeed != null) {
xout.Add(new XAttribute("JoshuaSeed", joshuaSeed));
}
foreach(var kv in codeCoverage) foreach(var kv in codeCoverage)
{ {
var element = new XElement("CodeCoverage", new XAttribute("File", kv.Key.Item1), new XAttribute("Line", kv.Key.Item2)); var element = new XElement("CodeCoverage", new XAttribute("File", kv.Key.Item1), new XAttribute("Line", kv.Key.Item2));

View File

@ -49,7 +49,7 @@ PROTOCOL_VERSION_6_1 = 0x0FDB00B061060001
PROTOCOL_VERSION_6_2 = 0x0FDB00B062010001 PROTOCOL_VERSION_6_2 = 0x0FDB00B062010001
PROTOCOL_VERSION_6_3 = 0x0FDB00B063010001 PROTOCOL_VERSION_6_3 = 0x0FDB00B063010001
PROTOCOL_VERSION_7_0 = 0x0FDB00B070010001 PROTOCOL_VERSION_7_0 = 0x0FDB00B070010001
PROTOCOL_VERSION_7_1 = 0x0FDB00B071010001 PROTOCOL_VERSION_7_1 = 0x0FDB00B071010000
PROTOCOL_VERSION_7_2 = 0x0FDB00B072000000 PROTOCOL_VERSION_7_2 = 0x0FDB00B072000000
supported_protocol_versions = frozenset([PROTOCOL_VERSION_5_2, PROTOCOL_VERSION_6_0, PROTOCOL_VERSION_6_1, supported_protocol_versions = frozenset([PROTOCOL_VERSION_5_2, PROTOCOL_VERSION_6_0, PROTOCOL_VERSION_6_1,
PROTOCOL_VERSION_6_2, PROTOCOL_VERSION_6_3, PROTOCOL_VERSION_7_0, PROTOCOL_VERSION_6_2, PROTOCOL_VERSION_6_3, PROTOCOL_VERSION_7_0,
@ -244,6 +244,11 @@ class CommitInfo(BaseInfo):
self.read_snapshot_version = bb.get_long() self.read_snapshot_version = bb.get_long()
if protocol_version >= PROTOCOL_VERSION_6_3: if protocol_version >= PROTOCOL_VERSION_6_3:
self.report_conflicting_keys = bb.get_bool() self.report_conflicting_keys = bb.get_bool()
if protocol_version >= PROTOCOL_VERSION_7_1:
lock_aware = bb.get_bool()
if bb.get_bool():
spanId = bb.get_bytes(16)
class ErrorGetInfo(BaseInfo): class ErrorGetInfo(BaseInfo):

View File

@ -0,0 +1,130 @@
## Global Tag Throttling
When the `GLOBAL_TAG_THROTTLING` knob is enabled, the ratekeeper will use the [transaction tagging feature](https://apple.github.io/foundationdb/transaction-tagging.html) to throttle tags according to the global tag throttling algorithm. This page describes the implementation of this algorithm.
### Tag Quotas
The global tag throttler bases throttling decisions on "quotas" provided by clients through the tag quota API. Each tag quota has four different components:
* Reserved read quota
* Reserved write quota
* Total read quota
* Total write quota
The global tag throttler can not throttle tags to a throughput below the reserved quotas, and it cannot allow throughput to exceed the total quotas.
### Cost
The units for these quotas are computed as follows. The "cost" of a read operation is computed as:
```
readCost = bytesRead / SERVER_KNOBS->READ_COST_BYTE_FACTOR + 1;
```
The "cost" of a write operation is computed as:
```
writeCost = bytesWritten / CLIENT_KNOBS->WRITE_COST_BYTE_FACTOR + 1;
```
Here `bytesWritten` includes cleared bytes. The size of range clears is estimated at commit time.
### Tuple Layer
Tag quotas are stored inside of the system keyspace (with prefix `\xff/tagQuota/`). They are stored using the tuple layer, in a tuple of form: `(reservedReadQuota, totalReadQuota, reservedWriteQuota, totalWriteQuota)`. There is currently no custom code in the bindings for manipulating these system keys. However, in any language for which bindings are available, it is possible to use the tuple layer to manipulate tag quotas.
### fdbcli
The easiest way for an external client to interact with tag quotas is through `fdbcli`. To get the quota of a particular tag, run the following command:
```
fdbcli> get <tag> [reserved|total] [read|write]
```
To set the quota through `fdbcli`, run:
```
fdbcli> set <tag> [reserved|total] [read|write] <value>
```
### Limit Calculation
The transaction budget that ratekeeper calculates and distributes to clients (via GRV proxies) for each tag is calculated based on several intermediate rate calculations, outlined in this section.
* Reserved Rate: Based on reserved quota and the average transaction cost, a reserved TPS rate is computed for each tag.
* Desired Rate: Based on total quota and the average transaction cost, a desired TPS rate is computed for each tag.
* Limiting Rate: When a storage server is near saturation, tags contributing notably to the workload on this storage server will receive a limiting TPS rate, computed to relieve the workload on the storage server.
* Target Rate: The target rate is the cluster-wide rate enforced by the global tag throttler. This rate is computed as:
```
targetTps = max(reservedTps, min(desiredTps, limitingTps));
```
* Per-Client Rate: While the target rate represents the cluster-wide desired throughput according to the global tag throttler, this budget must be shared across potentially many clients. Therefore, based on observed throughput from various clients, each client will receive an equal budget based on a per-client, per-tag rate computed by the global tag throttler. This rate is in the end what will be sent to clients to enforce throttling.
## Implementation
### Stat collection
The transaction rates and costs of all transactions must be visible to the global tag throttler. Whenever a client tags a transaction, sampling is performed to determine whether to attach the tag to messages sent to storage servers and commit proxies.
For read operations that are sampled (with probability `CLIENT_KNOBS->READ_TAG_SAMPLE_RATE`), read costs are aggregated on storage servers using the `TransactionTagCounter` class. This class tracks the busyness of the top-k tags affecting the storage server with read load (here `k` is determined by `SERVER_KNOBS->SS_THROTTLE_TAGS_TRACKED`). Storage servers periodically send per-tag read cost statistics to the ratekeeper through `StorageQueuingMetricsReply` messages.
For write operations that are sampled (with probability `COMMIT_SAMPLE_COST`), write costs are aggregated on commit proxies in the `ProxyCommitData::ssTrTagCommitCost` object. Per-storage, per-tag write cost statistics are periodically sent from commit proxies to the ratekeeper through `ReportCommitCostEstimationRequest` messages.
The ratekeeper tracks per-storage, per-tag cost statistics in the `GlobalTagThrottlerImpl::throughput` object.
The ratekeeper must also track the rate of transactions performed with each tag. Each GRV proxy agreggates a per-tag counter of transactions started (without sampling). These are sent to the ratekeeper through `GetRateInfoRequest` messages. The global tag throttler then tracks per tag transaction rates in the `GlobalTagThrottlerImpl::tagStatistics` object.
### Average Cost Calculation
Quotas are expressed in terms of cost, but because throttling is enforced at the beginning of transactions, budgets need to be calculated in terms of transactions per second. To make this conversion, it is necessary to track the average cost of transactions (per-tag, and per-tag on a particular storage server).
Both cost and transaction counters are smoothed using the `Smoother` class to provide stability over time. The "smoothing interval" can be modified through `SERVER_KNOBS->GLOBAL_TAG_THROTTLING_FOLDING_TIME`.
### Reserved Rate Calculation
The global tag throttler periodically reads reserved quotas from the system keyspace. Using these reserved quotas and the average cost of transactions with the given tag, a reserved TPS rate is computed. Read and write rates are aggregated as follows:
```
reservedTps = max(reservedReadTps, reservedWriteTps);
```
### Desired Rate Calculation
Similar to reserved rate calculation, the total quota is read from the system key space. Then, using the average cost of transactions with the given tag, a desired TPS rate is computed. Read and write rates are aggregated as follows:
```
desiredTps = min(desiredReadTps, desiredWriteTps);
```
### Limiting Rate Calculation
In addition to tag busyness statistics, the `StorageQueuingMetricsReply` messages sent from storage servers to the ratekeeper also contain metrics on the health of storage servers. The ratekeeper uses these metrics as part of its calculation of a global transaction rate (independent of tag throttling).
The global tag throttler also makes use of these metrics to compute a "throttling ratio" for each storage server. This throttling ratio is computed in `StorageQueueInfo::getThrottlingRatio`. The global tag throttler uses the throttling ratio for each tracked storage server to compute a "limiting transaction rate" for each combination of storage server and tag.
In the "healthy" case where no metrics are near saturation, the throttling ratio will be an empty `Optional<double>`, indicating that the storage server is not near saturation. If, on the other hand, the metrics indicate approaching saturation, the throttling ratio will be a number between 0 and 2 indicating the ratio of current throughput the storage server can serve. In this case, the global tag throttler looks at the current cost being served by the storage server, multiplies it by the throttling ratio, and computes a limiting cost for the storage server. Among all tags using significant resources on this storage server, this limiting cost is divided up according to the relative total quotas allocated to these tags. Next, a transaction limit is determined for each tag, based on how much the average transaction for the given tag affects the given storage server.
These per-tag, per-storage limiting transaction rates are aggregated to compute per-tag limiting transaction rates:
```
limitingTps(tag) = min{limitingTps(tag, storage) : all storage servers}
```
If the throttling ratio is empty for all storage servers affected by a tag, then the per-tag, per-storage limiting TPS rate is also empty. In this case the target rate for this tag is simply the desired rate.
If an individual zone is unhealthy, it may cause the throttling ratio for storage servers in that zone to shoot up. This should not be misinterpretted as a workload issue that requires active throttling. Therefore, the zone with the worst throttling ratios is ignored when computing the limiting transaction rate for a tag (similar to the calculation of the global transaction limit in `Ratekeeper::updateRate`).
### Client Rate Calculation
The smoothed per-client rate for each tag is tracked within `GlobalTagThrottlerImpl::PerTagStatistics`. Once a target rate has been computed, this is passed to `GlobalTagThrotterImpl::PerTagStatistics::updateAndGetPerClientRate` which adjusts the per-client rate. The per-client rate is meant to limit the busiest clients, so that at equilibrium, the per-client rate will remain constant and the sum of throughput from all clients will match the target rate.
## Testing
The `GlobalTagThrottling.toml` test provides a simple end-to-end test using the global tag throttler. Quotas are set using the internal tag quota API in the `GlobalTagThrottling` workload. This is run in parallel with the `ReadWrite` workload, which tags transactions. The number of `transaction_tag_throttled` errors is reported, along with the throughput, which should be roughly predictable based on the quota parameters chosen.
In addition to this end-to-end test, there is a suite of unit tests with the `/GlobalTagThrottler/` prefix. These tests run in a mock environment, with mock storage servers providing simulated storage queue statistics and tag busyness reports. Mock clients simulate workload on these mock storage servers, and get throttling feedback directly from a global tag throttler which is monitoring the mock storage servers.
In each test, the `GlobalTagThrottlerTesting::monitor` function is used to periodically check whether or not a desired equilibrium state has been reached. If the desired state is reached and maintained for a sufficient period of time, the test passes. If the unit test is unable to reach this desired equilibrium state before a timeout, the test will fail. Commonly, the desired state is for the global tag throttler to report a client rate sufficiently close to the desired rate specified as an input to the `GlobalTagThrottlerTesting::rateIsNear` function.
## Visibility
### Tracing
On the ratekeeper, every `SERVER_KNOBS->TAG_THROTTLE_PUSH_INTERVAL` seconds, the ratekeeper will call `GlobalTagThrottler::getClientRates`. At the end of the rate calculation for each tag, a trace event of type `GlobalTagThrottler_GotClientRate` is produced. This trace event reports the relevant inputs that went in to the rate calculation, and can be used for debugging.
On storage servers, every `SERVER_KNOBS->TAG_MEASUREMENT_INTERVAL` seconds, there are `BusyReadTag` events for every tag that has sufficient read cost to be reported to the ratekeeper. Both cost and fractional busyness are reported.
### Status
For each storage server, the busiest read tag is reported in the full status output, along with its cost and fractional busyness.

View File

@ -20,6 +20,8 @@ Fixes
----- -----
* In ``fdbcli``, integer options are now expressed as integers rather than byte strings (e.g. ``option on TIMEOUT 1000``). `(PR #7571) <https://github.com/apple/foundationdb/pull/7571>`_ * In ``fdbcli``, integer options are now expressed as integers rather than byte strings (e.g. ``option on TIMEOUT 1000``). `(PR #7571) <https://github.com/apple/foundationdb/pull/7571>`_
* Fixed the bug in ``ConflictingKeysImpl::getRange`` which happens when an underlying conflicting range contains the read range. Added additional test coverage for validating random ``getRange`` results from special keys. `(PR #7597) <https://github.com/apple/foundationdb/pull/7597>`_
* Fixed the bug in ``SpecialKeyRangeAsyncImpl::getRange`` that the local cache is updated incorrectly after a cross-module read range if it touched more than one ``SpecialKeyRangeAsyncImpl`` in resolving key selectors. Extended the ``SpecialKeySpaceCorrectness`` workload to catch the bug. `(PR #7671) <https://github.com/apple/foundationdb/pull/7671>`_
Status Status
------ ------

View File

@ -49,7 +49,7 @@ All operations performed within a tenant transaction will occur within the tenan
Raw access Raw access
---------- ----------
When operating in the tenant mode ``required_experimental``, transactions are not ordinarily permitted to run without using a tenant. In order to access the system keys or perform maintenance operations that span multiple tenants, it is required to use the ``RAW_ACCESS`` transaction option to access the global key-space. It is an error to specify ``RAW_ACCESS`` on a transaction that is configured to use a tenant. When operating in the tenant mode ``required_experimental`` or using a metacluster, transactions are not ordinarily permitted to run without using a tenant. In order to access the system keys or perform maintenance operations that span multiple tenants, it is required to use the ``RAW_ACCESS`` transaction option to access the global key-space. It is an error to specify ``RAW_ACCESS`` on a transaction that is configured to use a tenant.
.. note :: Setting the ``READ_SYSTEM_KEYS`` or ``ACCESS_SYSTEM_KEYS`` options implies ``RAW_ACCESS`` for your transaction. .. note :: Setting the ``READ_SYSTEM_KEYS`` or ``ACCESS_SYSTEM_KEYS`` options implies ``RAW_ACCESS`` for your transaction.

View File

@ -99,20 +99,68 @@ ACTOR Future<Void> doBlobPurge(Database db, Key startKey, Key endKey, Optional<V
return Void(); return Void();
} }
ACTOR Future<Version> checkBlobSubrange(Database db, KeyRange keyRange, Optional<Version> version) {
state Transaction tr(db);
state Version readVersionOut = invalidVersion;
loop {
try {
wait(success(tr.readBlobGranules(keyRange, 0, version, &readVersionOut)));
return readVersionOut;
} catch (Error& e) {
wait(tr.onError(e));
}
}
}
ACTOR Future<Void> doBlobCheck(Database db, Key startKey, Key endKey, Optional<Version> version) { ACTOR Future<Void> doBlobCheck(Database db, Key startKey, Key endKey, Optional<Version> version) {
state Transaction tr(db); state Transaction tr(db);
state Version readVersionOut = invalidVersion; state Version readVersionOut = invalidVersion;
state double elapsed = -timer_monotonic(); state double elapsed = -timer_monotonic();
state KeyRange range = KeyRange(KeyRangeRef(startKey, endKey));
state Standalone<VectorRef<KeyRangeRef>> allRanges;
loop { loop {
try { try {
wait(success(tr.readBlobGranules(KeyRange(KeyRangeRef(startKey, endKey)), 0, version, &readVersionOut))); wait(store(allRanges, tr.getBlobGranuleRanges(range)));
elapsed += timer_monotonic();
break; break;
} catch (Error& e) { } catch (Error& e) {
wait(tr.onError(e)); wait(tr.onError(e));
} }
} }
if (allRanges.empty()) {
fmt::print("ERROR: No blob ranges for [{0} - {1})\n", startKey.printable(), endKey.printable());
return Void();
}
fmt::print("Loaded {0} blob ranges to check\n", allRanges.size());
state std::vector<Future<Version>> checkParts;
// Chunk up to smaller ranges than this limit. Must be smaller than BG_TOO_MANY_GRANULES to not hit the limit
int maxChunkSize = CLIENT_KNOBS->BG_TOO_MANY_GRANULES / 2;
KeyRange currentChunk;
int currentChunkSize = 0;
for (auto& it : allRanges) {
if (currentChunkSize == maxChunkSize) {
checkParts.push_back(checkBlobSubrange(db, currentChunk, version));
currentChunkSize = 0;
}
if (currentChunkSize == 0) {
currentChunk = it;
} else if (it.begin != currentChunk.end) {
fmt::print("ERROR: Blobrange check failed, gap in blob ranges from [{0} - {1})\n",
currentChunk.end.printable(),
it.begin.printable());
return Void();
} else {
currentChunk = KeyRangeRef(currentChunk.begin, it.end);
}
currentChunkSize++;
}
checkParts.push_back(checkBlobSubrange(db, currentChunk, version));
wait(waitForAll(checkParts));
readVersionOut = checkParts.back().get();
elapsed += timer_monotonic();
fmt::print("Blob check complete for [{0} - {1}) @ {2} in {3:.6f} seconds\n", fmt::print("Blob check complete for [{0} - {1}) @ {2} in {3:.6f} seconds\n",
startKey.printable(), startKey.printable(),
endKey.printable(), endKey.printable(),

View File

@ -272,6 +272,10 @@ ACTOR Future<bool> configureCommandActor(Reference<IDatabase> db,
stderr, stderr,
"WARN: Sharded RocksDB storage engine type is still in experimental stage, not yet production tested.\n"); "WARN: Sharded RocksDB storage engine type is still in experimental stage, not yet production tested.\n");
break; break;
case ConfigurationResult::DATABASE_IS_REGISTERED:
fprintf(stderr, "ERROR: A cluster cannot change its tenant mode while part of a metacluster.\n");
ret = false;
break;
default: default:
ASSERT(false); ASSERT(false);
ret = false; ret = false;

View File

@ -0,0 +1,434 @@
/*
* MetaclusterCommands.actor.cpp
*
* This source file is part of the FoundationDB open source project
*
* Copyright 2013-2022 Apple Inc. and the FoundationDB project authors
*
* Licensed under the Apache License, Version 2.0 (the "License");
* you may not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
#include "fdbcli/fdbcli.actor.h"
#include "fdbclient/FDBOptions.g.h"
#include "fdbclient/IClientApi.h"
#include "fdbclient/Knobs.h"
#include "fdbclient/MetaclusterManagement.actor.h"
#include "fdbclient/Schemas.h"
#include "flow/Arena.h"
#include "flow/FastRef.h"
#include "flow/ThreadHelper.actor.h"
#include "flow/actorcompiler.h" // This must be the last #include.
namespace fdb_cli {
Optional<std::pair<Optional<ClusterConnectionString>, Optional<DataClusterEntry>>>
parseClusterConfiguration(std::vector<StringRef> const& tokens, DataClusterEntry const& defaults, int startIndex) {
Optional<DataClusterEntry> entry;
Optional<ClusterConnectionString> connectionString;
std::set<std::string> usedParams;
for (int tokenNum = startIndex; tokenNum < tokens.size(); ++tokenNum) {
StringRef token = tokens[tokenNum];
bool foundEquals;
StringRef param = token.eat("=", &foundEquals);
if (!foundEquals) {
fmt::print(stderr,
"ERROR: invalid configuration string `{}'. String must specify a value using `='.\n",
param.toString().c_str());
return {};
}
std::string value = token.toString();
if (!usedParams.insert(value).second) {
fmt::print(
stderr, "ERROR: configuration parameter `{}' specified more than once.\n", param.toString().c_str());
return {};
}
if (tokencmp(param, "max_tenant_groups")) {
entry = defaults;
int n;
if (sscanf(value.c_str(), "%d%n", &entry.get().capacity.numTenantGroups, &n) != 1 || n != value.size() ||
entry.get().capacity.numTenantGroups < 0) {
fmt::print(stderr, "ERROR: invalid number of tenant groups `{}'.\n", value.c_str());
return {};
}
} else if (tokencmp(param, "connection_string")) {
connectionString = ClusterConnectionString(value);
} else {
fmt::print(stderr, "ERROR: unrecognized configuration parameter `{}'.\n", param.toString().c_str());
return {};
}
}
return std::make_pair(connectionString, entry);
}
void printMetaclusterConfigureOptionsUsage() {
fmt::print("max_tenant_groups sets the maximum number of tenant groups that can be assigned\n"
"to the named data cluster.\n");
fmt::print("connection_string sets the connection string for the named data cluster.\n");
}
// metacluster create command
ACTOR Future<bool> metaclusterCreateCommand(Reference<IDatabase> db, std::vector<StringRef> tokens) {
if (tokens.size() != 3) {
fmt::print("Usage: metacluster create_experimental <NAME>\n\n");
fmt::print("Configures the cluster to be a management cluster in a metacluster.\n");
fmt::print("NAME is an identifier used to distinguish this metacluster from other metaclusters.\n");
return false;
}
Optional<std::string> errorStr = wait(MetaclusterAPI::createMetacluster(db, tokens[2]));
if (errorStr.present()) {
fmt::print("ERROR: {}.\n", errorStr.get());
} else {
fmt::print("The cluster has been configured as a metacluster.\n");
}
return true;
}
// metacluster decommission command
ACTOR Future<bool> metaclusterDecommissionCommand(Reference<IDatabase> db, std::vector<StringRef> tokens) {
if (tokens.size() != 2) {
fmt::print("Usage: metacluster decommission\n\n");
fmt::print("Converts the current cluster from a metacluster management cluster back into an\n");
fmt::print("ordinary cluster. It must be called on a cluster with no registered data clusters.\n");
return false;
}
wait(MetaclusterAPI::decommissionMetacluster(db));
fmt::print("The cluster is no longer a metacluster.\n");
return true;
}
// metacluster register command
ACTOR Future<bool> metaclusterRegisterCommand(Reference<IDatabase> db, std::vector<StringRef> tokens) {
if (tokens.size() < 4) {
fmt::print("Usage: metacluster register <NAME> connection_string=<CONNECTION_STRING>\n"
"[max_tenant_groups=<NUM_GROUPS>]\n\n");
fmt::print("Adds a data cluster to a metacluster.\n");
fmt::print("NAME is used to identify the cluster in future commands.\n");
printMetaclusterConfigureOptionsUsage();
return false;
}
DataClusterEntry defaultEntry;
auto config = parseClusterConfiguration(tokens, defaultEntry, 3);
if (!config.present()) {
return false;
} else if (!config.get().first.present()) {
fmt::print(stderr, "ERROR: connection_string must be configured when registering a cluster.\n");
return false;
}
wait(MetaclusterAPI::registerCluster(
db, tokens[2], config.get().first.get(), config.get().second.orDefault(defaultEntry)));
fmt::print("The cluster `{}' has been added\n", printable(tokens[2]).c_str());
return true;
}
// metacluster remove command
ACTOR Future<bool> metaclusterRemoveCommand(Reference<IDatabase> db, std::vector<StringRef> tokens) {
if (tokens.size() < 3 || tokens.size() > 4 || (tokens.size() == 4 && tokens[2] != "FORCE"_sr)) {
fmt::print("Usage: metacluster remove [FORCE] <NAME> \n\n");
fmt::print("Removes the specified data cluster from a metacluster.\n");
fmt::print("If FORCE is specified, then the cluster will be detached even if it has\n"
"tenants assigned to it.\n");
return false;
}
state ClusterNameRef clusterName = tokens[tokens.size() - 1];
wait(MetaclusterAPI::removeCluster(db, clusterName, tokens.size() == 4));
fmt::print("The cluster `{}' has been removed\n", printable(clusterName).c_str());
return true;
}
// metacluster configure command
ACTOR Future<bool> metaclusterConfigureCommand(Reference<IDatabase> db, std::vector<StringRef> tokens) {
if (tokens.size() < 4) {
fmt::print("Usage: metacluster configure <NAME> <max_tenant_groups=<NUM_GROUPS>|\n"
"connection_string=<CONNECTION_STRING>> ...\n\n");
fmt::print("Updates the configuration of the metacluster.\n");
printMetaclusterConfigureOptionsUsage();
return false;
}
state Reference<ITransaction> tr = db->createTransaction();
loop {
try {
tr->setOption(FDBTransactionOptions::ACCESS_SYSTEM_KEYS);
tr->setOption(FDBTransactionOptions::SPECIAL_KEY_SPACE_ENABLE_WRITES);
Optional<DataClusterMetadata> metadata = wait(MetaclusterAPI::tryGetClusterTransaction(tr, tokens[2]));
if (!metadata.present()) {
throw cluster_not_found();
}
auto config = parseClusterConfiguration(tokens, metadata.get().entry, 3);
if (!config.present()) {
return false;
}
MetaclusterAPI::updateClusterMetadata(
tr, tokens[2], metadata.get(), config.get().first, config.get().second);
wait(safeThreadFutureToFuture(tr->commit()));
break;
} catch (Error& e) {
wait(safeThreadFutureToFuture(tr->onError(e)));
}
}
return true;
}
// metacluster list command
ACTOR Future<bool> metaclusterListCommand(Reference<IDatabase> db, std::vector<StringRef> tokens) {
if (tokens.size() > 5) {
fmt::print("Usage: metacluster list [BEGIN] [END] [LIMIT]\n\n");
fmt::print("Lists the data clusters in a metacluster.\n");
fmt::print("Only cluster names 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");
return false;
}
state ClusterNameRef begin = tokens.size() > 2 ? tokens[2] : ""_sr;
state ClusterNameRef end = tokens.size() > 3 ? tokens[3] : "\xff"_sr;
int limit = 100;
if (tokens.size() > 4) {
int n = 0;
if (sscanf(tokens[3].toString().c_str(), "%d%n", &limit, &n) != 1 || n != tokens[3].size() || limit < 0) {
fmt::print(stderr, "ERROR: invalid limit {}\n", tokens[3].toString().c_str());
return false;
}
}
std::map<ClusterName, DataClusterMetadata> clusters = wait(MetaclusterAPI::listClusters(db, begin, end, limit));
if (clusters.empty()) {
if (tokens.size() == 2) {
fmt::print("The metacluster has no registered data clusters\n");
} else {
fmt::print("The metacluster has no registered data clusters in the specified range\n");
}
}
int index = 0;
for (auto cluster : clusters) {
fmt::print(" {}. {}\n", ++index, printable(cluster.first).c_str());
}
return true;
}
// metacluster get command
ACTOR Future<bool> metaclusterGetCommand(Reference<IDatabase> db, std::vector<StringRef> tokens) {
if (tokens.size() > 4 || (tokens.size() == 4 && tokens[3] != "JSON"_sr)) {
fmt::print("Usage: metacluster get <NAME> [JSON]\n\n");
fmt::print("Prints metadata associated with the given data cluster.\n");
fmt::print("If JSON is specified, then the output will be in JSON format.\n");
return false;
}
state bool useJson = tokens.size() == 4;
try {
DataClusterMetadata metadata = wait(MetaclusterAPI::getCluster(db, tokens[2]));
if (useJson) {
json_spirit::mObject obj;
obj["type"] = "success";
obj["cluster"] = metadata.toJson();
fmt::print("{}\n", json_spirit::write_string(json_spirit::mValue(obj), json_spirit::pretty_print).c_str());
} else {
fmt::print(" connection string: {}\n", metadata.connectionString.toString().c_str());
fmt::print(" tenant group capacity: {}\n", metadata.entry.capacity.numTenantGroups);
fmt::print(" allocated tenant groups: {}\n", metadata.entry.allocated.numTenantGroups);
if (metadata.entry.locked) {
fmt::print(" locked: true\n");
}
}
} catch (Error& e) {
if (useJson) {
json_spirit::mObject obj;
obj["type"] = "error";
obj["error"] = e.what();
fmt::print("{}\n", json_spirit::write_string(json_spirit::mValue(obj), json_spirit::pretty_print).c_str());
return false;
} else {
throw;
}
}
return true;
}
// metacluster status command
ACTOR Future<bool> metaclusterStatusCommand(Reference<IDatabase> db, std::vector<StringRef> tokens) {
if (tokens.size() < 2 || tokens.size() > 3) {
fmt::print("Usage: metacluster status [JSON]\n\n");
fmt::print("Prints metacluster metadata.\n");
fmt::print("If JSON is specified, then the output will be in JSON format.\n");
return false;
}
state bool useJson = tokens.size() == 3;
try {
std::map<ClusterName, DataClusterMetadata> clusters =
wait(MetaclusterAPI::listClusters(db, ""_sr, "\xff"_sr, CLIENT_KNOBS->MAX_DATA_CLUSTERS));
ClusterUsage totalCapacity;
ClusterUsage totalAllocated;
for (auto cluster : clusters) {
totalCapacity.numTenantGroups +=
std::max(cluster.second.entry.capacity.numTenantGroups, cluster.second.entry.allocated.numTenantGroups);
totalAllocated.numTenantGroups += cluster.second.entry.allocated.numTenantGroups;
}
if (useJson) {
json_spirit::mObject obj;
obj["type"] = "success";
json_spirit::mObject metaclusterObj;
metaclusterObj["data_clusters"] = (int)clusters.size();
metaclusterObj["capacity"] = totalCapacity.toJson();
metaclusterObj["allocated"] = totalAllocated.toJson();
obj["metacluster"] = metaclusterObj;
fmt::print("{}\n", json_spirit::write_string(json_spirit::mValue(obj), json_spirit::pretty_print).c_str());
} else {
fmt::print(" number of data clusters: {}\n", clusters.size());
fmt::print(" tenant group capacity: {}\n", totalCapacity.numTenantGroups);
fmt::print(" allocated tenant groups: {}\n", totalAllocated.numTenantGroups);
}
return true;
} catch (Error& e) {
if (useJson) {
json_spirit::mObject obj;
obj["type"] = "error";
obj["error"] = e.what();
fmt::print("{}\n", json_spirit::write_string(json_spirit::mValue(obj), json_spirit::pretty_print).c_str());
return false;
} else {
throw;
}
}
}
// metacluster command
Future<bool> metaclusterCommand(Reference<IDatabase> db, std::vector<StringRef> tokens) {
if (tokens.size() == 1) {
printUsage(tokens[0]);
return true;
} else if (tokencmp(tokens[1], "create_experimental")) {
return metaclusterCreateCommand(db, tokens);
} else if (tokencmp(tokens[1], "decommission")) {
return metaclusterDecommissionCommand(db, tokens);
} else if (tokencmp(tokens[1], "register")) {
return metaclusterRegisterCommand(db, tokens);
} else if (tokencmp(tokens[1], "remove")) {
return metaclusterRemoveCommand(db, tokens);
} else if (tokencmp(tokens[1], "configure")) {
return metaclusterConfigureCommand(db, tokens);
} else if (tokencmp(tokens[1], "list")) {
return metaclusterListCommand(db, tokens);
} else if (tokencmp(tokens[1], "get")) {
return metaclusterGetCommand(db, tokens);
} else if (tokencmp(tokens[1], "status")) {
return metaclusterStatusCommand(db, tokens);
} else {
printUsage(tokens[0]);
return true;
}
}
void metaclusterGenerator(const char* text,
const char* line,
std::vector<std::string>& lc,
std::vector<StringRef> const& tokens) {
if (tokens.size() == 1) {
const char* opts[] = {
"create_experimental", "decommission", "register", "remove", "configure", "list", "get", "status", nullptr
};
arrayGenerator(text, line, opts, lc);
} else if (tokens.size() > 1 && (tokencmp(tokens[1], "register") || tokencmp(tokens[1], "configure"))) {
const char* opts[] = { "max_tenant_groups=", "connection_string=", nullptr };
arrayGenerator(text, line, opts, lc);
} else if ((tokens.size() == 2 && tokencmp(tokens[1], "status")) ||
(tokens.size() == 3 && tokencmp(tokens[1], "get"))) {
const char* opts[] = { "JSON", nullptr };
arrayGenerator(text, line, opts, lc);
}
}
std::vector<const char*> metaclusterHintGenerator(std::vector<StringRef> const& tokens, bool inArgument) {
if (tokens.size() == 1) {
return { "<create_experimental|decommission|register|remove|configure|list|get|status>", "[ARGS]" };
} else if (tokencmp(tokens[1], "create_experimental")) {
return { "<NAME>" };
} else if (tokencmp(tokens[1], "decommission")) {
return {};
} else if (tokencmp(tokens[1], "register") && tokens.size() < 5) {
static std::vector<const char*> opts = { "<NAME>",
"connection_string=<CONNECTION_STRING>",
"[max_tenant_groups=<NUM_GROUPS>]" };
return std::vector<const char*>(opts.begin() + tokens.size() - 2, opts.end());
} else if (tokencmp(tokens[1], "remove") && tokens.size() < 4) {
static std::vector<const char*> opts = { "[FORCE]", "<NAME>" };
if (tokens.size() == 2) {
return opts;
} else if (tokens.size() == 3 && (inArgument || tokens[2].size() == "FORCE"_sr.size()) &&
"FORCE"_sr.startsWith(tokens[2])) {
return std::vector<const char*>(opts.begin() + tokens.size() - 2, opts.end());
} else {
return {};
}
} else if (tokencmp(tokens[1], "configure")) {
static std::vector<const char*> opts = {
"<NAME>", "<max_tenant_groups=<NUM_GROUPS>|connection_string=<CONNECTION_STRING>>"
};
return std::vector<const char*>(opts.begin() + std::min<int>(1, tokens.size() - 2), opts.end());
} else if (tokencmp(tokens[1], "list") && tokens.size() < 5) {
static std::vector<const char*> opts = { "[BEGIN]", "[END]", "[LIMIT]" };
return std::vector<const char*>(opts.begin() + tokens.size() - 2, opts.end());
} else if (tokencmp(tokens[1], "get") && tokens.size() < 4) {
static std::vector<const char*> opts = { "<NAME>", "[JSON]" };
return std::vector<const char*>(opts.begin() + tokens.size() - 2, opts.end());
} else if (tokencmp(tokens[1], "status") && tokens.size() == 2) {
return { "[JSON]" };
} else {
return {};
}
}
CommandFactory metaclusterRegisterFactory(
"metacluster",
CommandHelp("metacluster <create_experimental|decommission|register|remove|configure|list|get|status> [ARGS]",
"view and manage a metacluster",
"`create_experimental' and `decommission' set up or deconfigure a metacluster.\n"
"`register' and `remove' add and remove data clusters from the metacluster.\n"
"`configure' updates the configuration of a data cluster.\n"
"`list' prints a list of data clusters in the metacluster.\n"
"`get' prints the metadata for a particular data cluster.\n"
"`status' prints metacluster metadata.\n"),
&metaclusterGenerator,
&metaclusterHintGenerator);
} // namespace fdb_cli

View File

@ -411,6 +411,7 @@ void printStatus(StatusObjectReader statusObj,
outputString += "\nConfiguration:"; outputString += "\nConfiguration:";
std::string outputStringCache = outputString; std::string outputStringCache = outputString;
bool isOldMemory = false; bool isOldMemory = false;
bool blobGranuleEnabled{ false };
try { try {
// Configuration section // Configuration section
// FIXME: Should we suppress this if there are cluster messages implying that the database has no // FIXME: Should we suppress this if there are cluster messages implying that the database has no
@ -433,7 +434,14 @@ void printStatus(StatusObjectReader statusObj,
} else } else
outputString += "unknown"; outputString += "unknown";
int intVal; int intVal = 0;
if (statusObjConfig.get("blob_granules_enabled", intVal) && intVal) {
blobGranuleEnabled = true;
}
if (blobGranuleEnabled) {
outputString += "\n Blob granules - enabled";
}
outputString += "\n Coordinators - "; outputString += "\n Coordinators - ";
if (statusObjConfig.get("coordinators_count", intVal)) { if (statusObjConfig.get("coordinators_count", intVal)) {
outputString += std::to_string(intVal); outputString += std::to_string(intVal);
@ -1102,6 +1110,15 @@ void printStatus(StatusObjectReader statusObj,
outputString += "\n\nCoordination servers:"; outputString += "\n\nCoordination servers:";
outputString += getCoordinatorsInfoString(statusObj); outputString += getCoordinatorsInfoString(statusObj);
} }
if (blobGranuleEnabled) {
outputString += "\n\nBlob Granules:";
StatusObjectReader statusObjBlobGranules = statusObjCluster["blob_granules"];
auto numWorkers = statusObjBlobGranules["number_of_blob_workers"].get_int();
outputString += "\n Number of Workers - " + format("%d", numWorkers);
auto numKeyRanges = statusObjBlobGranules["number_of_key_ranges"].get_int();
outputString += "\n Number of Key Ranges - " + format("%d", numKeyRanges);
}
} }
// client time // client time

View File

@ -25,6 +25,7 @@
#include "fdbclient/IClientApi.h" #include "fdbclient/IClientApi.h"
#include "fdbclient/Knobs.h" #include "fdbclient/Knobs.h"
#include "fdbclient/ManagementAPI.actor.h" #include "fdbclient/ManagementAPI.actor.h"
#include "fdbclient/MetaclusterManagement.actor.h"
#include "fdbclient/TenantManagement.actor.h" #include "fdbclient/TenantManagement.actor.h"
#include "fdbclient/Schemas.h" #include "fdbclient/Schemas.h"
@ -100,9 +101,9 @@ Key makeConfigKey(TenantNameRef tenantName, StringRef configName) {
return tenantConfigSpecialKeyRange.begin.withSuffix(Tuple().append(tenantName).append(configName).pack()); return tenantConfigSpecialKeyRange.begin.withSuffix(Tuple().append(tenantName).append(configName).pack());
} }
void applyConfiguration(Reference<ITransaction> tr, void applyConfigurationToSpecialKeys(Reference<ITransaction> tr,
TenantNameRef tenantName, TenantNameRef tenantName,
std::map<Standalone<StringRef>, Optional<Value>> configuration) { std::map<Standalone<StringRef>, Optional<Value>> configuration) {
for (auto [configName, value] : configuration) { for (auto [configName, value] : configuration) {
if (value.present()) { if (value.present()) {
tr->set(makeConfigKey(tenantName, configName), value.get()); tr->set(makeConfigKey(tenantName, configName), value.get());
@ -136,21 +137,32 @@ ACTOR Future<bool> createTenantCommandActor(Reference<IDatabase> db, std::vector
} }
loop { loop {
tr->setOption(FDBTransactionOptions::SPECIAL_KEY_SPACE_ENABLE_WRITES);
try { try {
if (!doneExistenceCheck) { tr->setOption(FDBTransactionOptions::SPECIAL_KEY_SPACE_ENABLE_WRITES);
// Hold the reference to the standalone's memory tr->setOption(FDBTransactionOptions::READ_SYSTEM_KEYS);
state ThreadFuture<Optional<Value>> existingTenantFuture = tr->get(tenantNameKey); state ClusterType clusterType = wait(TenantAPI::getClusterType(tr));
Optional<Value> existingTenant = wait(safeThreadFutureToFuture(existingTenantFuture)); if (clusterType == ClusterType::METACLUSTER_MANAGEMENT) {
if (existingTenant.present()) { TenantMapEntry tenantEntry;
throw tenant_already_exists(); for (auto const& [name, value] : configuration.get()) {
tenantEntry.configure(name, value);
} }
doneExistenceCheck = true; wait(MetaclusterAPI::createTenant(db, tokens[1], tenantEntry));
} else {
if (!doneExistenceCheck) {
// Hold the reference to the standalone's memory
state ThreadFuture<Optional<Value>> existingTenantFuture = tr->get(tenantNameKey);
Optional<Value> existingTenant = wait(safeThreadFutureToFuture(existingTenantFuture));
if (existingTenant.present()) {
throw tenant_already_exists();
}
doneExistenceCheck = true;
}
tr->set(tenantNameKey, ValueRef());
applyConfigurationToSpecialKeys(tr, tokens[1], configuration.get());
wait(safeThreadFutureToFuture(tr->commit()));
} }
tr->set(tenantNameKey, ValueRef());
applyConfiguration(tr, tokens[1], configuration.get());
wait(safeThreadFutureToFuture(tr->commit()));
break; break;
} catch (Error& e) { } catch (Error& e) {
state Error err(e); state Error err(e);
@ -167,10 +179,12 @@ ACTOR Future<bool> createTenantCommandActor(Reference<IDatabase> db, std::vector
return true; return true;
} }
CommandFactory createTenantFactory("createtenant", CommandFactory createTenantFactory(
CommandHelp("createtenant <TENANT_NAME> [tenant_group=<TENANT_GROUP>]", "createtenant",
"creates a new tenant in the cluster", CommandHelp("createtenant <TENANT_NAME> [tenant_group=<TENANT_GROUP>]",
"Creates a new tenant in the cluster with the specified name.")); "creates a new tenant in the cluster",
"Creates a new tenant in the cluster with the specified name. An optional group can be specified"
"that will require this tenant to be placed on the same cluster as other tenants in the same group."));
// deletetenant command // deletetenant command
ACTOR Future<bool> deleteTenantCommandActor(Reference<IDatabase> db, std::vector<StringRef> tokens, int apiVersion) { ACTOR Future<bool> deleteTenantCommandActor(Reference<IDatabase> db, std::vector<StringRef> tokens, int apiVersion) {
@ -184,20 +198,27 @@ ACTOR Future<bool> deleteTenantCommandActor(Reference<IDatabase> db, std::vector
state bool doneExistenceCheck = false; state bool doneExistenceCheck = false;
loop { loop {
tr->setOption(FDBTransactionOptions::SPECIAL_KEY_SPACE_ENABLE_WRITES);
try { try {
if (!doneExistenceCheck) { tr->setOption(FDBTransactionOptions::SPECIAL_KEY_SPACE_ENABLE_WRITES);
// Hold the reference to the standalone's memory tr->setOption(FDBTransactionOptions::READ_SYSTEM_KEYS);
state ThreadFuture<Optional<Value>> existingTenantFuture = tr->get(tenantNameKey); state ClusterType clusterType = wait(TenantAPI::getClusterType(tr));
Optional<Value> existingTenant = wait(safeThreadFutureToFuture(existingTenantFuture)); if (clusterType == ClusterType::METACLUSTER_MANAGEMENT) {
if (!existingTenant.present()) { wait(MetaclusterAPI::deleteTenant(db, tokens[1]));
throw tenant_not_found(); } else {
if (!doneExistenceCheck) {
// Hold the reference to the standalone's memory
state ThreadFuture<Optional<Value>> existingTenantFuture = tr->get(tenantNameKey);
Optional<Value> existingTenant = wait(safeThreadFutureToFuture(existingTenantFuture));
if (!existingTenant.present()) {
throw tenant_not_found();
}
doneExistenceCheck = true;
} }
doneExistenceCheck = true;
tr->clear(tenantNameKey);
wait(safeThreadFutureToFuture(tr->commit()));
} }
tr->clear(tenantNameKey);
wait(safeThreadFutureToFuture(tr->commit()));
break; break;
} catch (Error& e) { } catch (Error& e) {
state Error err(e); state Error err(e);
@ -228,8 +249,8 @@ ACTOR Future<bool> listTenantsCommandActor(Reference<IDatabase> db, std::vector<
return false; return false;
} }
StringRef beginTenant = ""_sr; state StringRef beginTenant = ""_sr;
StringRef endTenant = "\xff\xff"_sr; state StringRef endTenant = "\xff\xff"_sr;
state int limit = 100; state int limit = 100;
if (tokens.size() >= 2) { if (tokens.size() >= 2) {
@ -256,12 +277,26 @@ ACTOR Future<bool> listTenantsCommandActor(Reference<IDatabase> db, std::vector<
loop { loop {
try { try {
// Hold the reference to the standalone's memory tr->setOption(FDBTransactionOptions::READ_SYSTEM_KEYS);
state ThreadFuture<RangeResult> kvsFuture = state ClusterType clusterType = wait(TenantAPI::getClusterType(tr));
tr->getRange(firstGreaterOrEqual(beginTenantKey), firstGreaterOrEqual(endTenantKey), limit); state std::vector<TenantNameRef> tenantNames;
RangeResult tenants = wait(safeThreadFutureToFuture(kvsFuture)); if (clusterType == ClusterType::METACLUSTER_MANAGEMENT) {
std::vector<std::pair<TenantName, TenantMapEntry>> tenants =
wait(MetaclusterAPI::listTenantsTransaction(tr, beginTenant, endTenant, limit));
for (auto tenant : tenants) {
tenantNames.push_back(tenant.first);
}
} else {
// Hold the reference to the standalone's memory
state ThreadFuture<RangeResult> kvsFuture =
tr->getRange(firstGreaterOrEqual(beginTenantKey), firstGreaterOrEqual(endTenantKey), limit);
RangeResult tenants = wait(safeThreadFutureToFuture(kvsFuture));
for (auto tenant : tenants) {
tenantNames.push_back(tenant.key.removePrefix(tenantMapSpecialKeyRange(apiVersion).begin));
}
}
if (tenants.empty()) { if (tenantNames.empty()) {
if (tokens.size() == 1) { if (tokens.size() == 1) {
fmt::print("The cluster has no tenants\n"); fmt::print("The cluster has no tenants\n");
} else { } else {
@ -270,10 +305,8 @@ ACTOR Future<bool> listTenantsCommandActor(Reference<IDatabase> db, std::vector<
} }
int index = 0; int index = 0;
for (auto tenant : tenants) { for (auto tenantName : tenantNames) {
fmt::print(" {}. {}\n", fmt::print(" {}. {}\n", ++index, printable(tenantName).c_str());
++index,
printable(tenant.key.removePrefix(tenantMapSpecialKeyRange(apiVersion).begin)).c_str());
} }
return true; return true;
@ -309,15 +342,24 @@ ACTOR Future<bool> getTenantCommandActor(Reference<IDatabase> db, std::vector<St
loop { loop {
try { try {
// Hold the reference to the standalone's memory tr->setOption(FDBTransactionOptions::READ_SYSTEM_KEYS);
state ThreadFuture<Optional<Value>> tenantFuture = tr->get(tenantNameKey); state ClusterType clusterType = wait(TenantAPI::getClusterType(tr));
Optional<Value> tenant = wait(safeThreadFutureToFuture(tenantFuture)); state std::string tenantJson;
if (!tenant.present()) { if (clusterType == ClusterType::METACLUSTER_MANAGEMENT) {
throw tenant_not_found(); TenantMapEntry entry = wait(MetaclusterAPI::getTenantTransaction(tr, tokens[1]));
tenantJson = entry.toJson(apiVersion);
} else {
// Hold the reference to the standalone's memory
state ThreadFuture<Optional<Value>> tenantFuture = tr->get(tenantNameKey);
Optional<Value> tenant = wait(safeThreadFutureToFuture(tenantFuture));
if (!tenant.present()) {
throw tenant_not_found();
}
tenantJson = tenant.get().toString();
} }
json_spirit::mValue jsonObject; json_spirit::mValue jsonObject;
json_spirit::read_string(tenant.get().toString(), jsonObject); json_spirit::read_string(tenantJson, jsonObject);
if (useJson) { if (useJson) {
json_spirit::mObject resultObj; json_spirit::mObject resultObj;
@ -333,6 +375,7 @@ ACTOR Future<bool> getTenantCommandActor(Reference<IDatabase> db, std::vector<St
std::string prefix; std::string prefix;
std::string tenantState; std::string tenantState;
std::string tenantGroup; std::string tenantGroup;
std::string assignedCluster;
doc.get("id", id); doc.get("id", id);
@ -344,6 +387,7 @@ ACTOR Future<bool> getTenantCommandActor(Reference<IDatabase> db, std::vector<St
doc.get("tenant_state", tenantState); doc.get("tenant_state", tenantState);
bool hasTenantGroup = doc.tryGet("tenant_group.printable", tenantGroup); bool hasTenantGroup = doc.tryGet("tenant_group.printable", tenantGroup);
bool hasAssignedCluster = doc.tryGet("assigned_cluster", assignedCluster);
fmt::print(" id: {}\n", id); fmt::print(" id: {}\n", id);
fmt::print(" prefix: {}\n", printable(prefix).c_str()); fmt::print(" prefix: {}\n", printable(prefix).c_str());
@ -351,8 +395,10 @@ ACTOR Future<bool> getTenantCommandActor(Reference<IDatabase> db, std::vector<St
if (hasTenantGroup) { if (hasTenantGroup) {
fmt::print(" tenant group: {}\n", tenantGroup.c_str()); fmt::print(" tenant group: {}\n", tenantGroup.c_str());
} }
if (hasAssignedCluster) {
fmt::print(" assigned cluster: {}\n", printable(assignedCluster).c_str());
}
} }
return true; return true;
} catch (Error& e) { } catch (Error& e) {
try { try {
@ -408,10 +454,17 @@ ACTOR Future<bool> configureTenantCommandActor(Reference<IDatabase> db, std::vec
state Reference<ITransaction> tr = db->createTransaction(); state Reference<ITransaction> tr = db->createTransaction();
loop { loop {
tr->setOption(FDBTransactionOptions::SPECIAL_KEY_SPACE_ENABLE_WRITES);
try { try {
applyConfiguration(tr, tokens[1], configuration.get()); tr->setOption(FDBTransactionOptions::SPECIAL_KEY_SPACE_ENABLE_WRITES);
wait(safeThreadFutureToFuture(tr->commit())); tr->setOption(FDBTransactionOptions::READ_SYSTEM_KEYS);
ClusterType clusterType = wait(TenantAPI::getClusterType(tr));
if (clusterType == ClusterType::METACLUSTER_MANAGEMENT) {
TenantMapEntry tenantEntry;
wait(MetaclusterAPI::configureTenant(db, tokens[1], configuration.get()));
} else {
applyConfigurationToSpecialKeys(tr, tokens[1], configuration.get());
wait(safeThreadFutureToFuture(tr->commit()));
}
break; break;
} catch (Error& e) { } catch (Error& e) {
state Error err(e); state Error err(e);

View File

@ -1967,6 +1967,13 @@ ACTOR Future<int> cli(CLIOptions opt, LineNoise* plinenoise) {
continue; continue;
} }
if (tokencmp(tokens[0], "metacluster")) {
bool _result = wait(makeInterruptable(metaclusterCommand(db, tokens)));
if (!_result)
is_error = true;
continue;
}
fprintf(stderr, "ERROR: Unknown command `%s'. Try `help'?\n", formatStringRef(tokens[0]).c_str()); fprintf(stderr, "ERROR: Unknown command `%s'. Try `help'?\n", formatStringRef(tokens[0]).c_str());
is_error = true; is_error = true;
} }

View File

@ -200,6 +200,10 @@ ACTOR Future<bool> listTenantsCommandActor(Reference<IDatabase> db, std::vector<
// lock/unlock command // lock/unlock command
ACTOR Future<bool> lockCommandActor(Reference<IDatabase> db, std::vector<StringRef> tokens); ACTOR Future<bool> lockCommandActor(Reference<IDatabase> db, std::vector<StringRef> tokens);
ACTOR Future<bool> unlockDatabaseActor(Reference<IDatabase> db, UID uid); ACTOR Future<bool> unlockDatabaseActor(Reference<IDatabase> db, UID uid);
// metacluster command
Future<bool> metaclusterCommand(Reference<IDatabase> db, std::vector<StringRef> tokens);
// changefeed command // changefeed command
ACTOR Future<bool> changeFeedCommandActor(Database localDb, ACTOR Future<bool> changeFeedCommandActor(Database localDb,
Optional<TenantMapEntry> tenantEntry, Optional<TenantMapEntry> tenantEntry,

View File

@ -111,7 +111,6 @@ void ClientKnobs::initialize(Randomize randomize) {
init( RANGESTREAM_BUFFERED_FRAGMENTS_LIMIT, 20 ); init( RANGESTREAM_BUFFERED_FRAGMENTS_LIMIT, 20 );
init( QUARANTINE_TSS_ON_MISMATCH, true ); if( randomize && BUGGIFY ) QUARANTINE_TSS_ON_MISMATCH = false; // if true, a tss mismatch will put the offending tss in quarantine. If false, it will just be killed init( QUARANTINE_TSS_ON_MISMATCH, true ); if( randomize && BUGGIFY ) QUARANTINE_TSS_ON_MISMATCH = false; // if true, a tss mismatch will put the offending tss in quarantine. If false, it will just be killed
init( CHANGE_FEED_EMPTY_BATCH_TIME, 0.005 ); init( CHANGE_FEED_EMPTY_BATCH_TIME, 0.005 );
init( SHARD_ENCODE_LOCATION_METADATA, false ); if( randomize && BUGGIFY ) SHARD_ENCODE_LOCATION_METADATA = true;
//KeyRangeMap //KeyRangeMap
init( KRM_GET_RANGE_LIMIT, 1e5 ); if( randomize && BUGGIFY ) KRM_GET_RANGE_LIMIT = 10; init( KRM_GET_RANGE_LIMIT, 1e5 ); if( randomize && BUGGIFY ) KRM_GET_RANGE_LIMIT = 10;
@ -286,12 +285,19 @@ void ClientKnobs::initialize(Randomize randomize) {
// Blob granules // Blob granules
init( BG_MAX_GRANULE_PARALLELISM, 10 ); init( BG_MAX_GRANULE_PARALLELISM, 10 );
init( BG_TOO_MANY_GRANULES, 1000 );
init( CHANGE_QUORUM_BAD_STATE_RETRY_TIMES, 3 ); init( CHANGE_QUORUM_BAD_STATE_RETRY_TIMES, 3 );
init( CHANGE_QUORUM_BAD_STATE_RETRY_DELAY, 2.0 ); init( CHANGE_QUORUM_BAD_STATE_RETRY_DELAY, 2.0 );
// Tenants and Metacluster // Tenants and Metacluster
init( MAX_TENANTS_PER_CLUSTER, 1e6 ); if ( randomize && BUGGIFY ) MAX_TENANTS_PER_CLUSTER = deterministicRandom()->randomInt(20, 100); init( MAX_TENANTS_PER_CLUSTER, 1e6 ); if ( randomize && BUGGIFY ) MAX_TENANTS_PER_CLUSTER = deterministicRandom()->randomInt(20, 100);
init( TENANT_TOMBSTONE_CLEANUP_INTERVAL, 60 ); if ( randomize && BUGGIFY ) TENANT_TOMBSTONE_CLEANUP_INTERVAL = deterministicRandom()->random01() * 30;
init( MAX_DATA_CLUSTERS, 1e5 );
init( REMOVE_CLUSTER_TENANT_BATCH_SIZE, 1e4 ); if ( randomize && BUGGIFY ) REMOVE_CLUSTER_TENANT_BATCH_SIZE = 1;
init( METACLUSTER_ASSIGNMENT_CLUSTERS_TO_CHECK, 5 ); if ( randomize && BUGGIFY ) METACLUSTER_ASSIGNMENT_CLUSTERS_TO_CHECK = 1;
init( METACLUSTER_ASSIGNMENT_FIRST_CHOICE_DELAY, 1.0 ); if ( randomize && BUGGIFY ) METACLUSTER_ASSIGNMENT_FIRST_CHOICE_DELAY = deterministicRandom()->random01() * 60;
init( METACLUSTER_ASSIGNMENT_AVAILABILITY_TIMEOUT, 10.0 ); if ( randomize && BUGGIFY ) METACLUSTER_ASSIGNMENT_AVAILABILITY_TIMEOUT = 1 + deterministicRandom()->random01() * 59;
// clang-format on // clang-format on
} }

31
fdbclient/Metacluster.cpp Normal file
View File

@ -0,0 +1,31 @@
/*
* Metacluster.cpp
*
* This source file is part of the FoundationDB open source project
*
* Copyright 2013-2022 Apple Inc. and the FoundationDB project authors
*
* Licensed under the Apache License, Version 2.0 (the "License");
* you may not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
#include "fdbclient/Metacluster.h"
#include "fdbclient/MetaclusterManagement.actor.h"
FDB_DEFINE_BOOLEAN_PARAM(AddNewTenants);
FDB_DEFINE_BOOLEAN_PARAM(RemoveMissingTenants);
json_spirit::mObject ClusterUsage::toJson() const {
json_spirit::mObject obj;
obj["num_tenant_groups"] = numTenantGroups;
return obj;
}

View File

@ -0,0 +1,59 @@
/*
* MetaclusterManagement.actor.cpp
*
* This source file is part of the FoundationDB open source project
*
* Copyright 2013-2022 Apple Inc. and the FoundationDB project authors
*
* Licensed under the Apache License, Version 2.0 (the "License");
* you may not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
#include "fdbclient/ClusterConnectionMemoryRecord.h"
#include "fdbclient/DatabaseContext.h"
#include "fdbclient/FDBTypes.h"
#include "fdbclient/MetaclusterManagement.actor.h"
#include "fdbclient/ThreadSafeTransaction.h"
#include "flow/actorcompiler.h" // has to be last include
namespace MetaclusterAPI {
ACTOR Future<Reference<IDatabase>> openDatabase(ClusterConnectionString connectionString) {
if (g_network->isSimulated()) {
Reference<IClusterConnectionRecord> clusterFile =
makeReference<ClusterConnectionMemoryRecord>(connectionString);
Database nativeDb = Database::createDatabase(clusterFile, -1);
Reference<IDatabase> threadSafeDb =
wait(unsafeThreadFutureToFuture(ThreadSafeDatabase::createFromExistingDatabase(nativeDb)));
return MultiVersionDatabase::debugCreateFromExistingDatabase(threadSafeDb);
} else {
return MultiVersionApi::api->createDatabaseFromConnectionString(connectionString.toString().c_str());
}
}
KeyBackedObjectMap<ClusterName, DataClusterEntry, decltype(IncludeVersion())> ManagementClusterMetadata::dataClusters(
"metacluster/dataCluster/metadata/"_sr,
IncludeVersion());
KeyBackedMap<ClusterName,
ClusterConnectionString,
TupleCodec<ClusterName>,
ManagementClusterMetadata::ConnectionStringCodec>
ManagementClusterMetadata::dataClusterConnectionRecords("metacluster/dataCluster/connectionString/"_sr);
KeyBackedSet<Tuple> ManagementClusterMetadata::clusterCapacityIndex("metacluster/clusterCapacityIndex/"_sr);
KeyBackedMap<ClusterName, int64_t, TupleCodec<ClusterName>, BinaryCodec<int64_t>>
ManagementClusterMetadata::clusterTenantCount("metacluster/clusterTenantCount/"_sr);
KeyBackedSet<Tuple> ManagementClusterMetadata::clusterTenantIndex("metacluster/dataCluster/tenantMap/"_sr);
KeyBackedSet<Tuple> ManagementClusterMetadata::clusterTenantGroupIndex("metacluster/dataCluster/tenantGroupMap/"_sr);
}; // namespace MetaclusterAPI

View File

@ -3242,6 +3242,8 @@ TenantInfo TransactionState::getTenantInfo(AllowInvalidTenantID allowInvalidId /
if (options.rawAccess) { if (options.rawAccess) {
return TenantInfo(); return TenantInfo();
} else if (!cx->internal && cx->clientInfo->get().clusterType == ClusterType::METACLUSTER_MANAGEMENT) {
throw management_cluster_invalid_access();
} else if (!cx->internal && cx->clientInfo->get().tenantMode == TenantMode::REQUIRED && !t.present()) { } else if (!cx->internal && cx->clientInfo->get().tenantMode == TenantMode::REQUIRED && !t.present()) {
throw tenant_name_required(); throw tenant_name_required();
} else if (!t.present()) { } else if (!t.present()) {
@ -7761,7 +7763,7 @@ ACTOR Future<Standalone<VectorRef<BlobGranuleChunkRef>>> readBlobGranulesActor(
// basically krmGetRange, but enable it to not use tenant without RAW_ACCESS by doing manual getRange with // basically krmGetRange, but enable it to not use tenant without RAW_ACCESS by doing manual getRange with
// UseTenant::False // UseTenant::False
GetRangeLimits limits(1000); GetRangeLimits limits(CLIENT_KNOBS->BG_TOO_MANY_GRANULES);
limits.minRows = 2; limits.minRows = 2;
RangeResult rawMapping = wait(getRange(self->trState, RangeResult rawMapping = wait(getRange(self->trState,
self->getReadVersion(), self->getReadVersion(),
@ -7776,19 +7778,24 @@ ACTOR Future<Standalone<VectorRef<BlobGranuleChunkRef>>> readBlobGranulesActor(
blobGranuleMapping = krmDecodeRanges(prefix, range, rawMapping); blobGranuleMapping = krmDecodeRanges(prefix, range, rawMapping);
} else { } else {
self->setOption(FDBTransactionOptions::ACCESS_SYSTEM_KEYS); self->setOption(FDBTransactionOptions::ACCESS_SYSTEM_KEYS);
wait(store( wait(store(blobGranuleMapping,
blobGranuleMapping, krmGetRanges(self,
krmGetRanges(self, blobGranuleMappingKeys.begin, keyRange, 1000, GetRangeLimits::BYTE_LIMIT_UNLIMITED))); blobGranuleMappingKeys.begin,
keyRange,
CLIENT_KNOBS->BG_TOO_MANY_GRANULES,
GetRangeLimits::BYTE_LIMIT_UNLIMITED)));
} }
if (blobGranuleMapping.more) { if (blobGranuleMapping.more) {
if (BG_REQUEST_DEBUG) { if (BG_REQUEST_DEBUG) {
fmt::print( fmt::print(
"BG Mapping for [{0} - %{1}) too large!\n", keyRange.begin.printable(), keyRange.end.printable()); "BG Mapping for [{0} - %{1}) too large!\n", keyRange.begin.printable(), keyRange.end.printable());
} }
TraceEvent(SevWarn, "BGMappingTooLarge").detail("Range", range).detail("Max", 1000); TraceEvent(SevWarn, "BGMappingTooLarge")
.detail("Range", range)
.detail("Max", CLIENT_KNOBS->BG_TOO_MANY_GRANULES);
throw unsupported_operation(); throw unsupported_operation();
} }
ASSERT(!blobGranuleMapping.more && blobGranuleMapping.size() < CLIENT_KNOBS->TOO_MANY); ASSERT(!blobGranuleMapping.more && blobGranuleMapping.size() <= CLIENT_KNOBS->BG_TOO_MANY_GRANULES);
if (blobGranuleMapping.size() < 2) { if (blobGranuleMapping.size() < 2) {
throw blob_granule_transaction_too_old(); throw blob_granule_transaction_too_old();

View File

@ -167,6 +167,7 @@ void ServerKnobs::initialize(Randomize randomize, ClientKnobs* clientKnobs, IsSi
init( PRIORITY_SPLIT_SHARD, 950 ); if( randomize && BUGGIFY ) PRIORITY_SPLIT_SHARD = 350; init( PRIORITY_SPLIT_SHARD, 950 ); if( randomize && BUGGIFY ) PRIORITY_SPLIT_SHARD = 350;
// Data distribution // Data distribution
init( SHARD_ENCODE_LOCATION_METADATA, false ); if( randomize && BUGGIFY ) SHARD_ENCODE_LOCATION_METADATA = true;
init( READ_REBALANCE_CPU_THRESHOLD, 15.0 ); init( READ_REBALANCE_CPU_THRESHOLD, 15.0 );
init( READ_REBALANCE_SRC_PARALLELISM, 20 ); init( READ_REBALANCE_SRC_PARALLELISM, 20 );
init( READ_REBALANCE_SHARD_TOPK, READ_REBALANCE_SRC_PARALLELISM * 2 ); init( READ_REBALANCE_SHARD_TOPK, READ_REBALANCE_SRC_PARALLELISM * 2 );
@ -677,7 +678,6 @@ void ServerKnobs::initialize(Randomize randomize, ClientKnobs* clientKnobs, IsSi
init( GLOBAL_TAG_THROTTLING, false ); init( GLOBAL_TAG_THROTTLING, false );
init( GLOBAL_TAG_THROTTLING_MIN_RATE, 1.0 ); init( GLOBAL_TAG_THROTTLING_MIN_RATE, 1.0 );
init( GLOBAL_TAG_THROTTLING_FOLDING_TIME, 10.0 ); init( GLOBAL_TAG_THROTTLING_FOLDING_TIME, 10.0 );
init( GLOBAL_TAG_THROTTLING_TRACE_INTERVAL, 5.0 );
//Storage Metrics //Storage Metrics
init( STORAGE_METRICS_AVERAGE_INTERVAL, 120.0 ); init( STORAGE_METRICS_AVERAGE_INTERVAL, 120.0 );
@ -904,11 +904,9 @@ void ServerKnobs::initialize(Randomize randomize, ClientKnobs* clientKnobs, IsSi
init( KMS_CONNECTOR_TYPE, "RESTKmsConnector" ); init( KMS_CONNECTOR_TYPE, "RESTKmsConnector" );
// Blob granlues // Blob granlues
init( BG_URL, isSimulated ? "file://fdbblob/" : "" ); // TODO: store in system key space or something, eventually init( BG_URL, isSimulated ? "file://fdbblob/" : "" ); // TODO: store in system key space or something, eventually
// BlobGranuleVerify* simulation tests use "blobRangeKeys", BlobGranuleCorrectness* use "tenant", default in real clusters is "tenant"
init( BG_RANGE_SOURCE, "tenant" );
// BlobGranuleVerify* simulation tests use "knobs", BlobGranuleCorrectness* use "tenant", default in real clusters is "knobs"
bool buggifyMediumGranules = simulationMediumShards || (randomize && BUGGIFY); bool buggifyMediumGranules = simulationMediumShards || (randomize && BUGGIFY);
// BlobGranuleVerify* simulation tests use "knobs", BlobGranuleCorrectness* use "tenant", default in real clusters is "knobs"
init( BG_METADATA_SOURCE, "knobs" ); init( BG_METADATA_SOURCE, "knobs" );
init( BG_SNAPSHOT_FILE_TARGET_BYTES, 10000000 ); if( buggifySmallShards ) BG_SNAPSHOT_FILE_TARGET_BYTES = 100000; else if (buggifyMediumGranules) BG_SNAPSHOT_FILE_TARGET_BYTES = 1000000; init( BG_SNAPSHOT_FILE_TARGET_BYTES, 10000000 ); if( buggifySmallShards ) BG_SNAPSHOT_FILE_TARGET_BYTES = 100000; else if (buggifyMediumGranules) BG_SNAPSHOT_FILE_TARGET_BYTES = 1000000;
init( BG_SNAPSHOT_FILE_TARGET_CHUNK_BYTES, 64*1024 ); if ( randomize && BUGGIFY ) BG_SNAPSHOT_FILE_TARGET_CHUNK_BYTES = BG_SNAPSHOT_FILE_TARGET_BYTES / (1 << deterministicRandom()->randomInt(0, 8)); init( BG_SNAPSHOT_FILE_TARGET_CHUNK_BYTES, 64*1024 ); if ( randomize && BUGGIFY ) BG_SNAPSHOT_FILE_TARGET_CHUNK_BYTES = BG_SNAPSHOT_FILE_TARGET_BYTES / (1 << deterministicRandom()->randomInt(0, 8));
@ -921,6 +919,7 @@ void ServerKnobs::initialize(Randomize randomize, ClientKnobs* clientKnobs, IsSi
init( BG_CONSISTENCY_CHECK_ENABLED, true ); if (randomize && BUGGIFY) BG_CONSISTENCY_CHECK_ENABLED = false; init( BG_CONSISTENCY_CHECK_ENABLED, true ); if (randomize && BUGGIFY) BG_CONSISTENCY_CHECK_ENABLED = false;
init( BG_CONSISTENCY_CHECK_TARGET_SPEED_KB, 1000 ); if (randomize && BUGGIFY) BG_CONSISTENCY_CHECK_TARGET_SPEED_KB *= (deterministicRandom()->randomInt(2, 50) / 10); init( BG_CONSISTENCY_CHECK_TARGET_SPEED_KB, 1000 ); if (randomize && BUGGIFY) BG_CONSISTENCY_CHECK_TARGET_SPEED_KB *= (deterministicRandom()->randomInt(2, 50) / 10);
init( BG_KEY_TUPLE_TRUNCATE_OFFSET, 0 );
init( BG_ENABLE_MERGING, true ); if (randomize && BUGGIFY) BG_ENABLE_MERGING = false; init( BG_ENABLE_MERGING, true ); if (randomize && BUGGIFY) BG_ENABLE_MERGING = false;
init( BG_MERGE_CANDIDATE_THRESHOLD_SECONDS, isSimulated ? 20.0 : 30 * 60 ); if (randomize && BUGGIFY) BG_MERGE_CANDIDATE_THRESHOLD_SECONDS = 5.0; init( BG_MERGE_CANDIDATE_THRESHOLD_SECONDS, isSimulated ? 20.0 : 30 * 60 ); if (randomize && BUGGIFY) BG_MERGE_CANDIDATE_THRESHOLD_SECONDS = 5.0;

View File

@ -148,7 +148,7 @@ RangeResult rywGetRange(ReadYourWritesTransaction* ryw, const KeyRangeRef& kr, c
ACTOR Future<Void> moveKeySelectorOverRangeActor(const SpecialKeyRangeReadImpl* skrImpl, ACTOR Future<Void> moveKeySelectorOverRangeActor(const SpecialKeyRangeReadImpl* skrImpl,
ReadYourWritesTransaction* ryw, ReadYourWritesTransaction* ryw,
KeySelector* ks, KeySelector* ks,
Optional<RangeResult>* cache) { KeyRangeMap<Optional<RangeResult>>* cache) {
// should be removed before calling // should be removed before calling
ASSERT(!ks->orEqual); ASSERT(!ks->orEqual);
@ -234,7 +234,7 @@ ACTOR Future<Void> normalizeKeySelectorActor(SpecialKeySpace* sks,
KeyRangeRef boundary, KeyRangeRef boundary,
int* actualOffset, int* actualOffset,
RangeResult* result, RangeResult* result,
Optional<RangeResult>* cache) { KeyRangeMap<Optional<RangeResult>>* cache) {
// If offset < 1, where we need to move left, iter points to the range containing at least one smaller key // If offset < 1, where we need to move left, iter points to the range containing at least one smaller key
// (It's a wasting of time to walk through the range whose begin key is same as ks->key) // (It's a wasting of time to walk through the range whose begin key is same as ks->key)
// (rangeContainingKeyBefore itself handles the case where ks->key == Key()) // (rangeContainingKeyBefore itself handles the case where ks->key == Key())
@ -337,8 +337,9 @@ ACTOR Future<RangeResult> SpecialKeySpace::getRangeAggregationActor(SpecialKeySp
state int actualBeginOffset; state int actualBeginOffset;
state int actualEndOffset; state int actualEndOffset;
state KeyRangeRef moduleBoundary; state KeyRangeRef moduleBoundary;
// used to cache result from potential first read // used to cache results from potential first async read
state Optional<RangeResult> cache; // the current implementation will read the whole range result to save in the cache
state KeyRangeMap<Optional<RangeResult>> cache(Optional<RangeResult>(), specialKeys.end);
if (ryw->specialKeySpaceRelaxed()) { if (ryw->specialKeySpaceRelaxed()) {
moduleBoundary = sks->range; moduleBoundary = sks->range;
@ -385,7 +386,7 @@ ACTOR Future<RangeResult> SpecialKeySpace::getRangeAggregationActor(SpecialKeySp
KeyRangeRef kr = iter->range(); KeyRangeRef kr = iter->range();
KeyRef keyStart = kr.contains(begin.getKey()) ? begin.getKey() : kr.begin; KeyRef keyStart = kr.contains(begin.getKey()) ? begin.getKey() : kr.begin;
KeyRef keyEnd = kr.contains(end.getKey()) ? end.getKey() : kr.end; KeyRef keyEnd = kr.contains(end.getKey()) ? end.getKey() : kr.end;
if (iter->value()->isAsync() && cache.present()) { if (iter->value()->isAsync() && cache.rangeContaining(keyStart).value().present()) {
const SpecialKeyRangeAsyncImpl* ptr = dynamic_cast<const SpecialKeyRangeAsyncImpl*>(iter->value()); const SpecialKeyRangeAsyncImpl* ptr = dynamic_cast<const SpecialKeyRangeAsyncImpl*>(iter->value());
RangeResult pairs_ = wait(ptr->getRange(ryw, KeyRangeRef(keyStart, keyEnd), limits, &cache)); RangeResult pairs_ = wait(ptr->getRange(ryw, KeyRangeRef(keyStart, keyEnd), limits, &cache));
pairs = pairs_; pairs = pairs_;
@ -416,7 +417,7 @@ ACTOR Future<RangeResult> SpecialKeySpace::getRangeAggregationActor(SpecialKeySp
KeyRangeRef kr = iter->range(); KeyRangeRef kr = iter->range();
KeyRef keyStart = kr.contains(begin.getKey()) ? begin.getKey() : kr.begin; KeyRef keyStart = kr.contains(begin.getKey()) ? begin.getKey() : kr.begin;
KeyRef keyEnd = kr.contains(end.getKey()) ? end.getKey() : kr.end; KeyRef keyEnd = kr.contains(end.getKey()) ? end.getKey() : kr.end;
if (iter->value()->isAsync() && cache.present()) { if (iter->value()->isAsync() && cache.rangeContaining(keyStart).value().present()) {
const SpecialKeyRangeAsyncImpl* ptr = dynamic_cast<const SpecialKeyRangeAsyncImpl*>(iter->value()); const SpecialKeyRangeAsyncImpl* ptr = dynamic_cast<const SpecialKeyRangeAsyncImpl*>(iter->value());
RangeResult pairs_ = wait(ptr->getRange(ryw, KeyRangeRef(keyStart, keyEnd), limits, &cache)); RangeResult pairs_ = wait(ptr->getRange(ryw, KeyRangeRef(keyStart, keyEnd), limits, &cache));
pairs = pairs_; pairs = pairs_;
@ -629,12 +630,8 @@ Future<Void> SpecialKeySpace::commit(ReadYourWritesTransaction* ryw) {
return commitActor(this, ryw); return commitActor(this, ryw);
} }
SKSCTestImpl::SKSCTestImpl(KeyRangeRef kr) : SpecialKeyRangeRWImpl(kr) {} // For SKSCTestRWImpl and SKSCTestAsyncReadImpl
Future<RangeResult> SKSCTestGetRangeBase(ReadYourWritesTransaction* ryw, KeyRangeRef kr, GetRangeLimits limitsHint) {
Future<RangeResult> SKSCTestImpl::getRange(ReadYourWritesTransaction* ryw,
KeyRangeRef kr,
GetRangeLimits limitsHint) const {
ASSERT(range.contains(kr));
auto resultFuture = ryw->getRange(kr, CLIENT_KNOBS->TOO_MANY); auto resultFuture = ryw->getRange(kr, CLIENT_KNOBS->TOO_MANY);
// all keys are written to RYW, since GRV is set, the read should happen locally // all keys are written to RYW, since GRV is set, the read should happen locally
ASSERT(resultFuture.isReady()); ASSERT(resultFuture.isReady());
@ -644,11 +641,29 @@ Future<RangeResult> SKSCTestImpl::getRange(ReadYourWritesTransaction* ryw,
return rywGetRange(ryw, kr, kvs); return rywGetRange(ryw, kr, kvs);
} }
Future<Optional<std::string>> SKSCTestImpl::commit(ReadYourWritesTransaction* ryw) { SKSCTestRWImpl::SKSCTestRWImpl(KeyRangeRef kr) : SpecialKeyRangeRWImpl(kr) {}
Future<RangeResult> SKSCTestRWImpl::getRange(ReadYourWritesTransaction* ryw,
KeyRangeRef kr,
GetRangeLimits limitsHint) const {
ASSERT(range.contains(kr));
return SKSCTestGetRangeBase(ryw, kr, limitsHint);
}
Future<Optional<std::string>> SKSCTestRWImpl::commit(ReadYourWritesTransaction* ryw) {
ASSERT(false); ASSERT(false);
return Optional<std::string>(); return Optional<std::string>();
} }
SKSCTestAsyncReadImpl::SKSCTestAsyncReadImpl(KeyRangeRef kr) : SpecialKeyRangeAsyncImpl(kr) {}
Future<RangeResult> SKSCTestAsyncReadImpl::getRange(ReadYourWritesTransaction* ryw,
KeyRangeRef kr,
GetRangeLimits limitsHint) const {
ASSERT(range.contains(kr));
return SKSCTestGetRangeBase(ryw, kr, limitsHint);
}
ReadConflictRangeImpl::ReadConflictRangeImpl(KeyRangeRef kr) : SpecialKeyRangeReadImpl(kr) {} ReadConflictRangeImpl::ReadConflictRangeImpl(KeyRangeRef kr) : SpecialKeyRangeReadImpl(kr) {}
ACTOR static Future<RangeResult> getReadConflictRangeImpl(ReadYourWritesTransaction* ryw, KeyRange kr) { ACTOR static Future<RangeResult> getReadConflictRangeImpl(ReadYourWritesTransaction* ryw, KeyRange kr) {

View File

@ -1336,6 +1336,8 @@ const KeyRangeRef blobGranuleMappingKeys(LiteralStringRef("\xff\x02/bgm/"), Lite
const KeyRangeRef blobGranuleLockKeys(LiteralStringRef("\xff\x02/bgl/"), LiteralStringRef("\xff\x02/bgl0")); const KeyRangeRef blobGranuleLockKeys(LiteralStringRef("\xff\x02/bgl/"), LiteralStringRef("\xff\x02/bgl0"));
const KeyRangeRef blobGranuleSplitKeys(LiteralStringRef("\xff\x02/bgs/"), LiteralStringRef("\xff\x02/bgs0")); const KeyRangeRef blobGranuleSplitKeys(LiteralStringRef("\xff\x02/bgs/"), LiteralStringRef("\xff\x02/bgs0"));
const KeyRangeRef blobGranuleMergeKeys(LiteralStringRef("\xff\x02/bgmerge/"), LiteralStringRef("\xff\x02/bgmerge0")); const KeyRangeRef blobGranuleMergeKeys(LiteralStringRef("\xff\x02/bgmerge/"), LiteralStringRef("\xff\x02/bgmerge0"));
const KeyRangeRef blobGranuleMergeBoundaryKeys(LiteralStringRef("\xff\x02/bgmergebounds/"),
LiteralStringRef("\xff\x02/bgmergebounds0"));
const KeyRangeRef blobGranuleHistoryKeys(LiteralStringRef("\xff\x02/bgh/"), LiteralStringRef("\xff\x02/bgh0")); const KeyRangeRef blobGranuleHistoryKeys(LiteralStringRef("\xff\x02/bgh/"), LiteralStringRef("\xff\x02/bgh0"));
const KeyRangeRef blobGranulePurgeKeys(LiteralStringRef("\xff\x02/bgp/"), LiteralStringRef("\xff\x02/bgp0")); const KeyRangeRef blobGranulePurgeKeys(LiteralStringRef("\xff\x02/bgp/"), LiteralStringRef("\xff\x02/bgp0"));
const KeyRangeRef blobGranuleVersionKeys(LiteralStringRef("\xff\x02/bgv/"), LiteralStringRef("\xff\x02/bgv0")); const KeyRangeRef blobGranuleVersionKeys(LiteralStringRef("\xff\x02/bgv/"), LiteralStringRef("\xff\x02/bgv0"));
@ -1561,6 +1563,23 @@ std::tuple<KeyRange, Version, std::vector<UID>, std::vector<Key>, std::vector<Ve
return std::tuple(range, bigEndian64(v), parentGranuleIDs, parentGranuleRanges, parentGranuleStartVersions); return std::tuple(range, bigEndian64(v), parentGranuleIDs, parentGranuleRanges, parentGranuleStartVersions);
} }
const Key blobGranuleMergeBoundaryKeyFor(const KeyRef& key) {
return key.withPrefix(blobGranuleMergeBoundaryKeys.begin);
}
const Value blobGranuleMergeBoundaryValueFor(BlobGranuleMergeBoundary const& boundary) {
BinaryWriter wr(IncludeVersion(ProtocolVersion::withBlobGranule()));
wr << boundary;
return wr.toValue();
}
Standalone<BlobGranuleMergeBoundary> decodeBlobGranuleMergeBoundaryValue(const ValueRef& value) {
Standalone<BlobGranuleMergeBoundary> boundaryValue;
BinaryReader reader(value, IncludeVersion());
reader >> boundaryValue;
return boundaryValue;
}
const Key blobGranuleHistoryKeyFor(KeyRangeRef const& range, Version version) { const Key blobGranuleHistoryKeyFor(KeyRangeRef const& range, Version version) {
BinaryWriter wr(AssumeVersion(ProtocolVersion::withBlobGranule())); BinaryWriter wr(AssumeVersion(ProtocolVersion::withBlobGranule()));
wr.serializeBytes(blobGranuleHistoryKeys.begin); wr.serializeBytes(blobGranuleHistoryKeys.begin);

View File

@ -109,7 +109,9 @@ std::string TenantMapEntry::toJson(int apiVersion) const {
} }
tenantEntry["tenant_state"] = TenantMapEntry::tenantStateToString(tenantState); tenantEntry["tenant_state"] = TenantMapEntry::tenantStateToString(tenantState);
if (assignedCluster.present()) {
tenantEntry["assigned_cluster"] = assignedCluster.get().toString();
}
if (tenantGroup.present()) { if (tenantGroup.present()) {
json_spirit::mObject tenantGroupObject; json_spirit::mObject tenantGroupObject;
std::string encodedTenantGroup = base64::encoder::from_string(tenantGroup.get().toString()); std::string encodedTenantGroup = base64::encoder::from_string(tenantGroup.get().toString());

View File

@ -0,0 +1,40 @@
/*
* TenantManagement.actor.cpp
*
* This source file is part of the FoundationDB open source project
*
* Copyright 2013-2022 Apple Inc. and the FoundationDB project authors
*
* Licensed under the Apache License, Version 2.0 (the "License");
* you may not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
#include <string>
#include <map>
#include "fdbclient/SystemData.h"
#include "fdbclient/TenantManagement.actor.h"
#include "fdbclient/Tuple.h"
#include "flow/actorcompiler.h" // has to be last include
namespace TenantAPI {
TenantMode tenantModeForClusterType(ClusterType clusterType, TenantMode tenantMode) {
if (clusterType == ClusterType::METACLUSTER_MANAGEMENT) {
return TenantMode::DISABLED;
} else if (clusterType == ClusterType::METACLUSTER_DATA) {
return TenantMode::REQUIRED;
} else {
return tenantMode;
}
}
} // namespace TenantAPI

View File

@ -236,6 +236,19 @@ struct BlobGranuleChunkRef {
enum BlobGranuleSplitState { Unknown = 0, Initialized = 1, Assigned = 2, Done = 3 }; enum BlobGranuleSplitState { Unknown = 0, Initialized = 1, Assigned = 2, Done = 3 };
// Boundary metadata for each range indexed by the beginning of the range.
struct BlobGranuleMergeBoundary {
constexpr static FileIdentifier file_identifier = 557861;
// Hard boundaries represent backing regions we want to keep separate.
bool buddy;
template <class Ar>
void serialize(Ar& ar) {
serializer(ar, buddy);
}
};
struct BlobGranuleHistoryValue { struct BlobGranuleHistoryValue {
constexpr static FileIdentifier file_identifier = 991434; constexpr static FileIdentifier file_identifier = 991434;
UID granuleID; UID granuleID;

View File

@ -41,6 +41,8 @@ struct BlobWorkerStats {
Counter readRequestsWithBegin; Counter readRequestsWithBegin;
Counter readRequestsCollapsed; Counter readRequestsCollapsed;
Counter flushGranuleReqs; Counter flushGranuleReqs;
Counter compressionBytesRaw;
Counter compressionBytesFinal;
int numRangesAssigned; int numRangesAssigned;
int mutationBytesBuffered; int mutationBytesBuffered;
@ -64,8 +66,9 @@ struct BlobWorkerStats {
readReqDeltaBytesReturned("ReadReqDeltaBytesReturned", cc), commitVersionChecks("CommitVersionChecks", cc), readReqDeltaBytesReturned("ReadReqDeltaBytesReturned", cc), commitVersionChecks("CommitVersionChecks", cc),
granuleUpdateErrors("GranuleUpdateErrors", cc), granuleRequestTimeouts("GranuleRequestTimeouts", cc), granuleUpdateErrors("GranuleUpdateErrors", cc), granuleRequestTimeouts("GranuleRequestTimeouts", cc),
readRequestsWithBegin("ReadRequestsWithBegin", cc), readRequestsCollapsed("ReadRequestsCollapsed", cc), readRequestsWithBegin("ReadRequestsWithBegin", cc), readRequestsCollapsed("ReadRequestsCollapsed", cc),
flushGranuleReqs("FlushGranuleReqs", cc), numRangesAssigned(0), mutationBytesBuffered(0), activeReadRequests(0), flushGranuleReqs("FlushGranuleReqs", cc), compressionBytesRaw("CompressionBytesRaw", cc),
granulesPendingSplitCheck(0) { compressionBytesFinal("CompressionBytesFinal", cc), numRangesAssigned(0), mutationBytesBuffered(0),
activeReadRequests(0), granulesPendingSplitCheck(0) {
specialCounter(cc, "NumRangesAssigned", [this]() { return this->numRangesAssigned; }); specialCounter(cc, "NumRangesAssigned", [this]() { return this->numRangesAssigned; });
specialCounter(cc, "MutationBytesBuffered", [this]() { return this->mutationBytesBuffered; }); specialCounter(cc, "MutationBytesBuffered", [this]() { return this->mutationBytesBuffered; });
specialCounter(cc, "ActiveReadRequests", [this]() { return this->activeReadRequests; }); specialCounter(cc, "ActiveReadRequests", [this]() { return this->activeReadRequests; });

View File

@ -172,6 +172,7 @@ struct GranuleStatusReply : public ReplyPromiseStreamReply {
KeyRange granuleRange; KeyRange granuleRange;
bool doSplit; bool doSplit;
bool writeHotSplit; bool writeHotSplit;
bool initialSplitTooBig;
int64_t continueEpoch; int64_t continueEpoch;
int64_t continueSeqno; int64_t continueSeqno;
UID granuleID; UID granuleID;
@ -180,11 +181,13 @@ struct GranuleStatusReply : public ReplyPromiseStreamReply {
bool mergeCandidate; bool mergeCandidate;
int64_t originalEpoch; int64_t originalEpoch;
int64_t originalSeqno; int64_t originalSeqno;
Optional<Key> proposedSplitKey;
GranuleStatusReply() {} GranuleStatusReply() {}
explicit GranuleStatusReply(KeyRange range, explicit GranuleStatusReply(KeyRange range,
bool doSplit, bool doSplit,
bool writeHotSplit, bool writeHotSplit,
bool initialSplitTooBig,
int64_t continueEpoch, int64_t continueEpoch,
int64_t continueSeqno, int64_t continueSeqno,
UID granuleID, UID granuleID,
@ -193,11 +196,15 @@ struct GranuleStatusReply : public ReplyPromiseStreamReply {
bool mergeCandidate, bool mergeCandidate,
int64_t originalEpoch, int64_t originalEpoch,
int64_t originalSeqno) int64_t originalSeqno)
: granuleRange(range), doSplit(doSplit), writeHotSplit(writeHotSplit), continueEpoch(continueEpoch), : granuleRange(range), doSplit(doSplit), writeHotSplit(writeHotSplit), initialSplitTooBig(initialSplitTooBig),
continueSeqno(continueSeqno), granuleID(granuleID), startVersion(startVersion), blockedVersion(blockedVersion), continueEpoch(continueEpoch), continueSeqno(continueSeqno), granuleID(granuleID), startVersion(startVersion),
mergeCandidate(mergeCandidate), originalEpoch(originalEpoch), originalSeqno(originalSeqno) {} blockedVersion(blockedVersion), mergeCandidate(mergeCandidate), originalEpoch(originalEpoch),
originalSeqno(originalSeqno) {}
int expectedSize() const { return sizeof(GranuleStatusReply) + granuleRange.expectedSize(); } int expectedSize() const {
return sizeof(GranuleStatusReply) + granuleRange.expectedSize() +
(proposedSplitKey.present() ? proposedSplitKey.get().expectedSize() : 0);
}
template <class Ar> template <class Ar>
void serialize(Ar& ar) { void serialize(Ar& ar) {
@ -207,6 +214,7 @@ struct GranuleStatusReply : public ReplyPromiseStreamReply {
granuleRange, granuleRange,
doSplit, doSplit,
writeHotSplit, writeHotSplit,
initialSplitTooBig,
continueEpoch, continueEpoch,
continueSeqno, continueSeqno,
granuleID, granuleID,
@ -214,7 +222,8 @@ struct GranuleStatusReply : public ReplyPromiseStreamReply {
blockedVersion, blockedVersion,
mergeCandidate, mergeCandidate,
originalEpoch, originalEpoch,
originalSeqno); originalSeqno,
proposedSplitKey);
} }
}; };

View File

@ -109,7 +109,6 @@ public:
int RANGESTREAM_BUFFERED_FRAGMENTS_LIMIT; int RANGESTREAM_BUFFERED_FRAGMENTS_LIMIT;
bool QUARANTINE_TSS_ON_MISMATCH; bool QUARANTINE_TSS_ON_MISMATCH;
double CHANGE_FEED_EMPTY_BATCH_TIME; double CHANGE_FEED_EMPTY_BATCH_TIME;
bool SHARD_ENCODE_LOCATION_METADATA;
// KeyRangeMap // KeyRangeMap
int KRM_GET_RANGE_LIMIT; int KRM_GET_RANGE_LIMIT;
@ -276,6 +275,7 @@ public:
// Blob Granules // Blob Granules
int BG_MAX_GRANULE_PARALLELISM; int BG_MAX_GRANULE_PARALLELISM;
int BG_TOO_MANY_GRANULES;
// The coordinator key/value in storage server might be inconsistent to the value stored in the cluster file. // 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. // This might happen when a recovery is happening together with a cluster controller coordinator key change.
@ -286,6 +286,12 @@ public:
// Tenants and Metacluster // Tenants and Metacluster
int MAX_TENANTS_PER_CLUSTER; int MAX_TENANTS_PER_CLUSTER;
int TENANT_TOMBSTONE_CLEANUP_INTERVAL;
int MAX_DATA_CLUSTERS;
int REMOVE_CLUSTER_TENANT_BATCH_SIZE;
int METACLUSTER_ASSIGNMENT_CLUSTERS_TO_CHECK;
double METACLUSTER_ASSIGNMENT_FIRST_CHOICE_DELAY;
double METACLUSTER_ASSIGNMENT_AVAILABILITY_TIMEOUT;
ClientKnobs(Randomize randomize); ClientKnobs(Randomize randomize);
void initialize(Randomize randomize); void initialize(Randomize randomize);

View File

@ -120,6 +120,8 @@ struct ClientDBInfo {
bool isEncryptionEnabled = false; bool isEncryptionEnabled = false;
TenantMode tenantMode; TenantMode tenantMode;
ClusterType clusterType = ClusterType::STANDALONE;
Optional<ClusterName> metaclusterName;
ClientDBInfo() {} ClientDBInfo() {}
@ -131,7 +133,17 @@ struct ClientDBInfo {
if constexpr (!is_fb_function<Archive>) { if constexpr (!is_fb_function<Archive>) {
ASSERT(ar.protocolVersion().isValid()); ASSERT(ar.protocolVersion().isValid());
} }
serializer(ar, grvProxies, commitProxies, id, forward, history, tenantMode, clusterId, isEncryptionEnabled); serializer(ar,
grvProxies,
commitProxies,
id,
forward,
history,
tenantMode,
isEncryptionEnabled,
clusterId,
clusterType,
metaclusterName);
} }
}; };

View File

@ -1392,6 +1392,11 @@ struct TenantMode {
uint32_t mode; uint32_t mode;
}; };
typedef StringRef ClusterNameRef;
typedef Standalone<ClusterNameRef> ClusterName;
enum class ClusterType { STANDALONE, METACLUSTER_MANAGEMENT, METACLUSTER_DATA };
struct GRVCacheSpace { struct GRVCacheSpace {
Version cachedReadVersion; Version cachedReadVersion;
double lastGrvTime; double lastGrvTime;

View File

@ -39,6 +39,7 @@ the contents of the system key space.
#include "fdbclient/Status.h" #include "fdbclient/Status.h"
#include "fdbclient/Subspace.h" #include "fdbclient/Subspace.h"
#include "fdbclient/DatabaseConfiguration.h" #include "fdbclient/DatabaseConfiguration.h"
#include "fdbclient/Metacluster.h"
#include "fdbclient/Status.h" #include "fdbclient/Status.h"
#include "fdbclient/SystemData.h" #include "fdbclient/SystemData.h"
#include "flow/actorcompiler.h" // has to be last include #include "flow/actorcompiler.h" // has to be last include
@ -69,6 +70,7 @@ enum class ConfigurationResult {
SUCCESS_WARN_SHARDED_ROCKSDB_EXPERIMENTAL, SUCCESS_WARN_SHARDED_ROCKSDB_EXPERIMENTAL,
DATABASE_CREATED_WARN_ROCKSDB_EXPERIMENTAL, DATABASE_CREATED_WARN_ROCKSDB_EXPERIMENTAL,
DATABASE_CREATED_WARN_SHARDED_ROCKSDB_EXPERIMENTAL, DATABASE_CREATED_WARN_SHARDED_ROCKSDB_EXPERIMENTAL,
DATABASE_IS_REGISTERED
}; };
enum class CoordinatorsResult { enum class CoordinatorsResult {
@ -475,6 +477,14 @@ Future<ConfigurationResult> changeConfig(Reference<DB> db, std::map<std::string,
newConfig.storageServerStoreType == KeyValueStoreType::SSD_SHARDED_ROCKSDB) { newConfig.storageServerStoreType == KeyValueStoreType::SSD_SHARDED_ROCKSDB) {
warnShardedRocksDBIsExperimental = true; warnShardedRocksDBIsExperimental = true;
} }
if (newConfig.tenantMode != oldConfig.tenantMode) {
Optional<MetaclusterRegistrationEntry> metaclusterRegistration =
wait(MetaclusterMetadata::metaclusterRegistration.get(tr));
if (metaclusterRegistration.present()) {
return ConfigurationResult::DATABASE_IS_REGISTERED;
}
}
} }
} }
if (creating) { if (creating) {

View File

@ -156,6 +156,12 @@ struct NullCodec {
static Standalone<StringRef> unpack(Standalone<StringRef> val) { return val; } static Standalone<StringRef> unpack(Standalone<StringRef> val) { return val; }
}; };
template <class T>
struct BinaryCodec {
static Standalone<StringRef> pack(T val) { return BinaryWriter::toValue<T>(val, Unversioned()); }
static T unpack(Standalone<StringRef> val) { return BinaryReader::fromStringRef<T>(val, Unversioned()); }
};
template <typename ResultType> template <typename ResultType>
struct KeyBackedRangeResult { struct KeyBackedRangeResult {
std::vector<ResultType> results; std::vector<ResultType> results;
@ -364,6 +370,16 @@ public:
})); }));
} }
// Get key's value or defaultValue if it doesn't exist
template <class Transaction>
Future<ValueType> getD(Transaction tr,
KeyType const& key,
Snapshot snapshot = Snapshot::False,
ValueType defaultValue = ValueType()) const {
return map(get(tr, key, snapshot),
[=](Optional<ValueType> val) -> ValueType { return val.orDefault(defaultValue); });
}
// Returns a Property that can be get/set that represents key's entry in this this. // Returns a Property that can be get/set that represents key's entry in this this.
KeyBackedProperty<ValueType> getProperty(KeyType const& key) const { KeyBackedProperty<ValueType> getProperty(KeyType const& key) const {
return subspace.begin.withSuffix(KeyCodec::pack(key)); return subspace.begin.withSuffix(KeyCodec::pack(key));
@ -378,6 +394,13 @@ public:
return k.expectedSize() + v.expectedSize(); return k.expectedSize() + v.expectedSize();
} }
template <class Transaction>
void atomicOp(Transaction tr, KeyType const& key, ValueType const& val, MutationRef::Type type) {
Key k = subspace.begin.withSuffix(KeyCodec::pack(key));
Value v = ValueCodec::pack(val);
tr->atomicOp(k, v, type);
}
template <class Transaction> template <class Transaction>
void erase(Transaction tr, KeyType const& key) { void erase(Transaction tr, KeyType const& key) {
tr->clear(subspace.begin.withSuffix(KeyCodec::pack(key))); tr->clear(subspace.begin.withSuffix(KeyCodec::pack(key)));

View File

@ -0,0 +1,185 @@
/*
* Metacluster.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 FDBCLIENT_METACLUSTER_H
#define FDBCLIENT_METACLUSTER_H
#include "CoordinationInterface.h"
#include "json_spirit/json_spirit_value.h"
#pragma once
#include "fdbclient/FDBTypes.h"
#include "fdbclient/KeyBackedTypes.h"
#include "flow/flat_buffers.h"
struct ClusterUsage {
int numTenantGroups = 0;
ClusterUsage() = default;
ClusterUsage(int numTenantGroups) : numTenantGroups(numTenantGroups) {}
json_spirit::mObject toJson() const;
bool operator==(const ClusterUsage& other) const noexcept { return numTenantGroups == other.numTenantGroups; }
bool operator!=(const ClusterUsage& other) const noexcept { return !(*this == other); }
bool operator<(const ClusterUsage& other) const noexcept { return numTenantGroups < other.numTenantGroups; }
template <class Ar>
void serialize(Ar& ar) {
serializer(ar, numTenantGroups);
}
};
template <>
struct Traceable<ClusterUsage> : std::true_type {
static std::string toString(const ClusterUsage& value) {
return format("NumTenantGroups: %d", value.numTenantGroups);
}
};
struct DataClusterEntry {
constexpr static FileIdentifier file_identifier = 929511;
UID id;
ClusterUsage capacity;
ClusterUsage allocated;
// If true, then tenant groups cannot be assigned to this cluster. This is used when a cluster is being forcefully
// removed.
bool locked = false;
DataClusterEntry() = default;
DataClusterEntry(ClusterUsage capacity) : capacity(capacity) {}
DataClusterEntry(UID id, ClusterUsage capacity, ClusterUsage allocated)
: id(id), capacity(capacity), allocated(allocated) {}
// Returns true if all configurable properties match
bool matchesConfiguration(DataClusterEntry const& other) const {
return id == other.id && capacity == other.capacity;
}
bool hasCapacity() const { return allocated < capacity; }
Value encode() const { return ObjectWriter::toValue(*this, IncludeVersion()); }
static DataClusterEntry decode(ValueRef const& value) {
return ObjectReader::fromStringRef<DataClusterEntry>(value, IncludeVersion());
}
json_spirit::mObject toJson() const {
json_spirit::mObject obj;
obj["capacity"] = capacity.toJson();
obj["allocated"] = allocated.toJson();
if (locked) {
obj["locked"] = locked;
}
return obj;
}
template <class Ar>
void serialize(Ar& ar) {
serializer(ar, id, capacity, allocated, locked);
}
};
struct MetaclusterRegistrationEntry {
constexpr static FileIdentifier file_identifier = 13448589;
ClusterType clusterType;
ClusterName metaclusterName;
ClusterName name;
UID metaclusterId;
UID id;
MetaclusterRegistrationEntry() = default;
MetaclusterRegistrationEntry(ClusterName metaclusterName, UID metaclusterId)
: clusterType(ClusterType::METACLUSTER_MANAGEMENT), metaclusterName(metaclusterName), name(metaclusterName),
metaclusterId(metaclusterId), id(metaclusterId) {}
MetaclusterRegistrationEntry(ClusterName metaclusterName, ClusterName name, UID metaclusterId, UID id)
: clusterType(ClusterType::METACLUSTER_DATA), metaclusterName(metaclusterName), name(name),
metaclusterId(metaclusterId), id(id) {
ASSERT(metaclusterName != name && metaclusterId != id);
}
// Returns true if this entry is associated with the same cluster as the passed in entry. If one entry is from the
// management cluster and the other is from a data cluster, this checks whether they are part of the same
// metacluster.
bool matches(MetaclusterRegistrationEntry const& other) const {
if (metaclusterName != other.metaclusterName || metaclusterId != other.metaclusterId) {
return false;
} else if (clusterType == ClusterType::METACLUSTER_DATA && other.clusterType == ClusterType::METACLUSTER_DATA &&
(name != other.name || id != other.id)) {
return false;
}
return true;
}
MetaclusterRegistrationEntry toManagementClusterRegistration() const {
ASSERT(clusterType == ClusterType::METACLUSTER_DATA);
return MetaclusterRegistrationEntry(metaclusterName, metaclusterId);
}
MetaclusterRegistrationEntry toDataClusterRegistration(ClusterName name, UID id) const {
ASSERT(clusterType == ClusterType::METACLUSTER_MANAGEMENT);
return MetaclusterRegistrationEntry(metaclusterName, name, metaclusterId, id);
}
Value encode() const { return ObjectWriter::toValue(*this, IncludeVersion()); }
static MetaclusterRegistrationEntry decode(ValueRef const& value) {
return ObjectReader::fromStringRef<MetaclusterRegistrationEntry>(value, IncludeVersion());
}
static Optional<MetaclusterRegistrationEntry> decode(Optional<Value> value) {
return value.map<MetaclusterRegistrationEntry>(
[](ValueRef const& v) { return MetaclusterRegistrationEntry::decode(v); });
}
std::string toString() const {
if (clusterType == ClusterType::METACLUSTER_MANAGEMENT) {
return fmt::format(
"metacluster name: {}, metacluster id: {}", printable(metaclusterName), metaclusterId.shortString());
} else {
return fmt::format("metacluster name: {}, metacluster id: {}, data cluster name: {}, data cluster id: {}",
printable(metaclusterName),
metaclusterId.shortString(),
printable(name),
id.shortString());
}
}
template <class Ar>
void serialize(Ar& ar) {
serializer(ar, clusterType, metaclusterName, name, metaclusterId, id);
}
};
template <>
struct Traceable<MetaclusterRegistrationEntry> : std::true_type {
static std::string toString(MetaclusterRegistrationEntry const& entry) { return entry.toString(); }
};
struct MetaclusterMetadata {
// Registration information for a metacluster, stored on both management and data clusters
static inline KeyBackedObjectProperty<MetaclusterRegistrationEntry, decltype(IncludeVersion())>
metaclusterRegistration = KeyBackedObjectProperty<MetaclusterRegistrationEntry, decltype(IncludeVersion())>(
"\xff/metacluster/clusterRegistration"_sr,
IncludeVersion());
};
#endif

File diff suppressed because it is too large Load Diff

View File

@ -163,6 +163,8 @@ public:
int PRIORITY_SPLIT_SHARD; int PRIORITY_SPLIT_SHARD;
// Data distribution // Data distribution
bool SHARD_ENCODE_LOCATION_METADATA; // If true, location metadata will contain shard ID.
double READ_REBALANCE_CPU_THRESHOLD; // read rebalance only happens if the source servers' CPU > threshold 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 int READ_REBALANCE_SRC_PARALLELISM; // the max count a server become a source server within a certain interval
int READ_REBALANCE_SHARD_TOPK; // top k shards from which to select randomly for read-rebalance int READ_REBALANCE_SHARD_TOPK; // top k shards from which to select randomly for read-rebalance
@ -882,8 +884,6 @@ public:
// FIXME: configure url with database configuration instead of knob eventually // FIXME: configure url with database configuration instead of knob eventually
std::string BG_URL; std::string BG_URL;
// whether to use blobRangeKeys or tenants for blob granule range sources
std::string BG_RANGE_SOURCE;
// Whether to use knobs or EKP for blob metadata and credentials // Whether to use knobs or EKP for blob metadata and credentials
std::string BG_METADATA_SOURCE; std::string BG_METADATA_SOURCE;
@ -900,6 +900,7 @@ public:
bool BG_ENABLE_MERGING; bool BG_ENABLE_MERGING;
int BG_MERGE_CANDIDATE_THRESHOLD_SECONDS; int BG_MERGE_CANDIDATE_THRESHOLD_SECONDS;
int BG_MERGE_CANDIDATE_DELAY_SECONDS; int BG_MERGE_CANDIDATE_DELAY_SECONDS;
int BG_KEY_TUPLE_TRUNCATE_OFFSET;
int BLOB_WORKER_INITIAL_SNAPSHOT_PARALLELISM; int BLOB_WORKER_INITIAL_SNAPSHOT_PARALLELISM;
double BLOB_WORKER_TIMEOUT; // Blob Manager's reaction time to a blob worker failure double BLOB_WORKER_TIMEOUT; // Blob Manager's reaction time to a blob worker failure

View File

@ -125,7 +125,7 @@ public:
Future<RangeResult> getRange(ReadYourWritesTransaction* ryw, Future<RangeResult> getRange(ReadYourWritesTransaction* ryw,
KeyRangeRef kr, KeyRangeRef kr,
GetRangeLimits limitsHint, GetRangeLimits limitsHint,
Optional<RangeResult>* cache) const { KeyRangeMap<Optional<RangeResult>>* cache) const {
return getRangeAsyncActor(this, ryw, kr, limitsHint, cache); return getRangeAsyncActor(this, ryw, kr, limitsHint, cache);
} }
@ -135,17 +135,18 @@ public:
ReadYourWritesTransaction* ryw, ReadYourWritesTransaction* ryw,
KeyRangeRef kr, KeyRangeRef kr,
GetRangeLimits limits, GetRangeLimits limits,
Optional<RangeResult>* cache) { KeyRangeMap<Optional<RangeResult>>* cache) {
ASSERT(skrAyncImpl->getKeyRange().contains(kr)); ASSERT(skrAyncImpl->getKeyRange().contains(kr));
ASSERT(cache != nullptr); ASSERT(cache != nullptr);
if (!cache->present()) { ASSERT(cache->rangeContaining(kr.begin) == cache->rangeContainingKeyBefore(kr.end));
if (!(*cache)[kr.begin].present()) {
// For simplicity, every time we need to cache, we read the whole range // For simplicity, every time we need to cache, we read the whole range
// Although sometimes the range can be narrowed, // Although sometimes the range can be narrowed,
// there is not a general way to do it in complicated scenarios // there is not a general way to do it in complicated scenarios
RangeResult result_ = wait(skrAyncImpl->getRange(ryw, skrAyncImpl->getKeyRange(), limits)); RangeResult result_ = wait(skrAyncImpl->getRange(ryw, skrAyncImpl->getKeyRange(), limits));
*cache = result_; cache->insert(skrAyncImpl->getKeyRange(), result_);
} }
const auto& allResults = cache->get(); const auto& allResults = (*cache)[kr.begin].get();
int start = 0, end = allResults.size(); int start = 0, end = allResults.size();
while (start < allResults.size() && allResults[start].key < kr.begin) while (start < allResults.size() && allResults[start].key < kr.begin)
++start; ++start;
@ -271,15 +272,23 @@ private:
}; };
// Used for SpecialKeySpaceCorrectnessWorkload // Used for SpecialKeySpaceCorrectnessWorkload
class SKSCTestImpl : public SpecialKeyRangeRWImpl { class SKSCTestRWImpl : public SpecialKeyRangeRWImpl {
public: public:
explicit SKSCTestImpl(KeyRangeRef kr); explicit SKSCTestRWImpl(KeyRangeRef kr);
Future<RangeResult> getRange(ReadYourWritesTransaction* ryw, Future<RangeResult> getRange(ReadYourWritesTransaction* ryw,
KeyRangeRef kr, KeyRangeRef kr,
GetRangeLimits limitsHint) const override; GetRangeLimits limitsHint) const override;
Future<Optional<std::string>> commit(ReadYourWritesTransaction* ryw) override; Future<Optional<std::string>> commit(ReadYourWritesTransaction* ryw) override;
}; };
class SKSCTestAsyncReadImpl : public SpecialKeyRangeAsyncImpl {
public:
explicit SKSCTestAsyncReadImpl(KeyRangeRef kr);
Future<RangeResult> getRange(ReadYourWritesTransaction* ryw,
KeyRangeRef kr,
GetRangeLimits limitsHint) const override;
};
// Use special key prefix "\xff\xff/transaction/conflicting_keys/<some_key>", // Use special key prefix "\xff\xff/transaction/conflicting_keys/<some_key>",
// to retrieve keys which caused latest not_committed(conflicting with another transaction) error. // to retrieve keys which caused latest not_committed(conflicting with another transaction) error.
// The returned key value pairs are interpreted as : // The returned key value pairs are interpreted as :

View File

@ -22,7 +22,6 @@
#define FDBCLIENT_STORAGESERVERINTERFACE_H #define FDBCLIENT_STORAGESERVERINTERFACE_H
#pragma once #pragma once
#include <ostream>
#include "fdbclient/FDBTypes.h" #include "fdbclient/FDBTypes.h"
#include "fdbclient/StorageCheckpoint.h" #include "fdbclient/StorageCheckpoint.h"
#include "fdbclient/StorageServerShard.h" #include "fdbclient/StorageServerShard.h"
@ -55,6 +54,34 @@ struct VersionReply {
} }
}; };
// This struct is used by RK to forward the commit cost to SS, see discussion in #7258
struct UpdateCommitCostRequest {
constexpr static FileIdentifier file_identifier = 4159439;
// Ratekeeper ID, it is only reasonable to compare postTime from the same Ratekeeper
UID ratekeeperID;
// The time the request being posted
double postTime;
double elapsed;
TransactionTag busiestTag;
// Properties that are defined in TransactionCommitCostEstimation
int opsSum;
uint64_t costSum;
uint64_t totalWriteCosts;
bool reported;
ReplyPromise<Void> reply;
template <typename Ar>
void serialize(Ar& ar) {
serializer(ar, ratekeeperID, postTime, elapsed, busiestTag, opsSum, costSum, totalWriteCosts, reported, reply);
}
};
struct StorageServerInterface { struct StorageServerInterface {
constexpr static FileIdentifier file_identifier = 15302073; constexpr static FileIdentifier file_identifier = 15302073;
enum { BUSY_ALLOWED = 0, BUSY_FORCE = 1, BUSY_LOCAL = 2 }; enum { BUSY_ALLOWED = 0, BUSY_FORCE = 1, BUSY_LOCAL = 2 };
@ -94,6 +121,8 @@ struct StorageServerInterface {
RequestStream<struct FetchCheckpointRequest> fetchCheckpoint; RequestStream<struct FetchCheckpointRequest> fetchCheckpoint;
RequestStream<struct FetchCheckpointKeyValuesRequest> fetchCheckpointKeyValues; RequestStream<struct FetchCheckpointKeyValuesRequest> fetchCheckpointKeyValues;
RequestStream<struct UpdateCommitCostRequest> updateCommitCostRequest;
private: private:
bool acceptingRequests; bool acceptingRequests;
@ -163,6 +192,8 @@ public:
RequestStream<struct FetchCheckpointRequest>(getValue.getEndpoint().getAdjustedEndpoint(20)); RequestStream<struct FetchCheckpointRequest>(getValue.getEndpoint().getAdjustedEndpoint(20));
fetchCheckpointKeyValues = RequestStream<struct FetchCheckpointKeyValuesRequest>( fetchCheckpointKeyValues = RequestStream<struct FetchCheckpointKeyValuesRequest>(
getValue.getEndpoint().getAdjustedEndpoint(21)); getValue.getEndpoint().getAdjustedEndpoint(21));
updateCommitCostRequest =
RequestStream<struct UpdateCommitCostRequest>(getValue.getEndpoint().getAdjustedEndpoint(22));
} }
} else { } else {
ASSERT(Ar::isDeserializing); ASSERT(Ar::isDeserializing);
@ -213,6 +244,7 @@ public:
streams.push_back(checkpoint.getReceiver()); streams.push_back(checkpoint.getReceiver());
streams.push_back(fetchCheckpoint.getReceiver()); streams.push_back(fetchCheckpoint.getReceiver());
streams.push_back(fetchCheckpointKeyValues.getReceiver()); streams.push_back(fetchCheckpointKeyValues.getReceiver());
streams.push_back(updateCommitCostRequest.getReceiver());
FlowTransport::transport().addEndpoints(streams); FlowTransport::transport().addEndpoints(streams);
} }
}; };

View File

@ -613,6 +613,9 @@ extern const KeyRangeRef blobGranuleSplitKeys;
// \xff\x02/bgmerge/mergeGranuleId = [[BlobGranuleMergeState]] // \xff\x02/bgmerge/mergeGranuleId = [[BlobGranuleMergeState]]
extern const KeyRangeRef blobGranuleMergeKeys; extern const KeyRangeRef blobGranuleMergeKeys;
// \xff\x02/bgmergebounds/beginkey = [[BlobGranuleMergeBoundary]]
extern const KeyRangeRef blobGranuleMergeBoundaryKeys;
// \xff\x02/bgh/(beginKey,endKey,startVersion) = { granuleUID, [parentGranuleHistoryKeys] } // \xff\x02/bgh/(beginKey,endKey,startVersion) = { granuleUID, [parentGranuleHistoryKeys] }
extern const KeyRangeRef blobGranuleHistoryKeys; extern const KeyRangeRef blobGranuleHistoryKeys;
@ -664,6 +667,11 @@ const Value blobGranuleMergeValueFor(KeyRange mergeKeyRange,
std::tuple<KeyRange, Version, std::vector<UID>, std::vector<Key>, std::vector<Version>> decodeBlobGranuleMergeValue( std::tuple<KeyRange, Version, std::vector<UID>, std::vector<Key>, std::vector<Version>> decodeBlobGranuleMergeValue(
ValueRef const& value); ValueRef const& value);
// BlobGranuleMergeBoundary.
const Key blobGranuleMergeBoundaryKeyFor(const KeyRef& key);
const Value blobGranuleMergeBoundaryValueFor(BlobGranuleMergeBoundary const& boundary);
Standalone<BlobGranuleMergeBoundary> decodeBlobGranuleMergeBoundaryValue(const ValueRef& value);
const Key blobGranuleHistoryKeyFor(KeyRangeRef const& range, Version version); const Key blobGranuleHistoryKeyFor(KeyRangeRef const& range, Version version);
std::pair<KeyRange, Version> decodeBlobGranuleHistoryKey(KeyRef const& key); std::pair<KeyRange, Version> decodeBlobGranuleHistoryKey(KeyRef const& key);
const KeyRange blobGranuleHistoryKeyRangeFor(KeyRangeRef const& range); const KeyRange blobGranuleHistoryKeyRangeFor(KeyRangeRef const& range);

View File

@ -25,6 +25,7 @@
#include "fdbclient/FDBTypes.h" #include "fdbclient/FDBTypes.h"
#include "fdbclient/KeyBackedTypes.h" #include "fdbclient/KeyBackedTypes.h"
#include "fdbclient/VersionedMap.h" #include "fdbclient/VersionedMap.h"
#include "fdbclient/KeyBackedTypes.h"
#include "fdbrpc/TenantInfo.h" #include "fdbrpc/TenantInfo.h"
#include "flow/flat_buffers.h" #include "flow/flat_buffers.h"
@ -33,6 +34,23 @@ typedef Standalone<TenantNameRef> TenantName;
typedef StringRef TenantGroupNameRef; typedef StringRef TenantGroupNameRef;
typedef Standalone<TenantGroupNameRef> TenantGroupName; typedef Standalone<TenantGroupNameRef> TenantGroupName;
// Represents the various states that a tenant could be in.
// In a standalone cluster, a tenant should only ever be in the READY state.
// In a metacluster, a tenant on the management cluster could be in the other states while changes are applied to the
// data cluster.
//
// REGISTERING - the tenant has been created on the management cluster and is being created on the data cluster
// READY - the tenant has been created on both clusters, is active, and is consistent between the two clusters
// REMOVING - the tenant has been marked for removal and is being removed on the data cluster
// UPDATING_CONFIGURATION - the tenant configuration has changed on the management cluster and is being applied to the
// data cluster
// ERROR - currently unused
//
// A tenant in any configuration is allowed to be removed. Only tenants in the READY or UPDATING_CONFIGURATION phases
// can have their configuration updated. A tenant must not exist or be in the REGISTERING phase to be created.
//
// If an operation fails and the tenant is left in a non-ready state, re-running the same operation is legal. If
// successful, the tenant will return to the READY state.
enum class TenantState { REGISTERING, READY, REMOVING, UPDATING_CONFIGURATION, ERROR }; enum class TenantState { REGISTERING, READY, REMOVING, UPDATING_CONFIGURATION, ERROR };
struct TenantMapEntry { struct TenantMapEntry {
@ -49,10 +67,11 @@ struct TenantMapEntry {
TenantState tenantState = TenantState::READY; TenantState tenantState = TenantState::READY;
Optional<TenantGroupName> tenantGroup; Optional<TenantGroupName> tenantGroup;
bool encrypted = false; bool encrypted = false;
Optional<ClusterName> assignedCluster;
int64_t configurationSequenceNum = 0;
constexpr static int PREFIX_SIZE = sizeof(id); constexpr static int PREFIX_SIZE = sizeof(id);
public:
TenantMapEntry(); TenantMapEntry();
TenantMapEntry(int64_t id, TenantState tenantState, bool encrypted); TenantMapEntry(int64_t id, TenantState tenantState, bool encrypted);
TenantMapEntry(int64_t id, TenantState tenantState, Optional<TenantGroupName> tenantGroup, bool encrypted); TenantMapEntry(int64_t id, TenantState tenantState, Optional<TenantGroupName> tenantGroup, bool encrypted);
@ -70,7 +89,7 @@ public:
template <class Ar> template <class Ar>
void serialize(Ar& ar) { void serialize(Ar& ar) {
serializer(ar, id, tenantState, tenantGroup, encrypted); serializer(ar, id, tenantState, tenantGroup, encrypted, assignedCluster, configurationSequenceNum);
if constexpr (Ar::isDeserializing) { if constexpr (Ar::isDeserializing) {
if (id >= 0) { if (id >= 0) {
prefix = idToPrefix(id); prefix = idToPrefix(id);
@ -83,7 +102,10 @@ public:
struct TenantGroupEntry { struct TenantGroupEntry {
constexpr static FileIdentifier file_identifier = 10764222; constexpr static FileIdentifier file_identifier = 10764222;
Optional<ClusterName> assignedCluster;
TenantGroupEntry() = default; TenantGroupEntry() = default;
TenantGroupEntry(Optional<ClusterName> assignedCluster) : assignedCluster(assignedCluster) {}
Value encode() { return ObjectWriter::toValue(*this, IncludeVersion()); } Value encode() { return ObjectWriter::toValue(*this, IncludeVersion()); }
static TenantGroupEntry decode(ValueRef const& value) { static TenantGroupEntry decode(ValueRef const& value) {
@ -92,34 +114,58 @@ struct TenantGroupEntry {
template <class Ar> template <class Ar>
void serialize(Ar& ar) { void serialize(Ar& ar) {
serializer(ar); serializer(ar, assignedCluster);
}
};
struct TenantTombstoneCleanupData {
constexpr static FileIdentifier file_identifier = 3291339;
// All tombstones have been erased up to and including this id.
// We should not generate new tombstones at IDs equal to or older than this.
int64_t tombstonesErasedThrough = -1;
// The version at which we will next erase tombstones.
Version nextTombstoneEraseVersion = invalidVersion;
// When we reach the nextTombstoneEraseVersion, we will erase tombstones up through this ID.
int64_t nextTombstoneEraseId = -1;
template <class Ar>
void serialize(Ar& ar) {
serializer(ar, tombstonesErasedThrough, nextTombstoneEraseVersion, nextTombstoneEraseId);
} }
}; };
struct TenantMetadataSpecification { struct TenantMetadataSpecification {
static KeyRef subspace; Key subspace;
KeyBackedObjectMap<TenantName, TenantMapEntry, decltype(IncludeVersion()), NullCodec> tenantMap; KeyBackedObjectMap<TenantName, TenantMapEntry, decltype(IncludeVersion()), NullCodec> tenantMap;
KeyBackedProperty<int64_t> lastTenantId; KeyBackedProperty<int64_t> lastTenantId;
KeyBackedBinaryValue<int64_t> tenantCount; KeyBackedBinaryValue<int64_t> tenantCount;
KeyBackedSet<int64_t> tenantTombstones;
KeyBackedObjectProperty<TenantTombstoneCleanupData, decltype(IncludeVersion())> tombstoneCleanupData;
KeyBackedSet<Tuple> tenantGroupTenantIndex; KeyBackedSet<Tuple> tenantGroupTenantIndex;
KeyBackedObjectMap<TenantGroupName, TenantGroupEntry, decltype(IncludeVersion()), NullCodec> tenantGroupMap; KeyBackedObjectMap<TenantGroupName, TenantGroupEntry, decltype(IncludeVersion()), NullCodec> tenantGroupMap;
TenantMetadataSpecification(KeyRef subspace) TenantMetadataSpecification(KeyRef prefix)
: tenantMap(subspace.withSuffix("tenant/map/"_sr), IncludeVersion()), : subspace(prefix.withSuffix("tenant/"_sr)), tenantMap(subspace.withSuffix("map/"_sr), IncludeVersion()),
lastTenantId(subspace.withSuffix("tenant/lastId"_sr)), tenantCount(subspace.withSuffix("tenant/count"_sr)), lastTenantId(subspace.withSuffix("lastId"_sr)), tenantCount(subspace.withSuffix("count"_sr)),
tenantGroupTenantIndex(subspace.withSuffix("tenant/tenantGroup/tenantIndex/"_sr)), tenantTombstones(subspace.withSuffix("tombstones/"_sr)),
tenantGroupMap(subspace.withSuffix("tenant/tenantGroup/map/"_sr), IncludeVersion()) {} tombstoneCleanupData(subspace.withSuffix("tombstoneCleanup"_sr), IncludeVersion()),
tenantGroupTenantIndex(subspace.withSuffix("tenantGroup/tenantIndex/"_sr)),
tenantGroupMap(subspace.withSuffix("tenantGroup/map/"_sr), IncludeVersion()) {}
}; };
struct TenantMetadata { struct TenantMetadata {
private:
static inline TenantMetadataSpecification instance = TenantMetadataSpecification("\xff/"_sr); static inline TenantMetadataSpecification instance = TenantMetadataSpecification("\xff/"_sr);
public: static inline auto& subspace = instance.subspace;
static inline auto& tenantMap = instance.tenantMap; static inline auto& tenantMap = instance.tenantMap;
static inline auto& lastTenantId = instance.lastTenantId; static inline auto& lastTenantId = instance.lastTenantId;
static inline auto& tenantCount = instance.tenantCount; static inline auto& tenantCount = instance.tenantCount;
static inline auto& tenantTombstones = instance.tenantTombstones;
static inline auto& tombstoneCleanupData = instance.tombstoneCleanupData;
static inline auto& tenantGroupTenantIndex = instance.tenantGroupTenantIndex; static inline auto& tenantGroupTenantIndex = instance.tenantGroupTenantIndex;
static inline auto& tenantGroupMap = instance.tenantGroupMap; static inline auto& tenantGroupMap = instance.tenantGroupMap;

View File

@ -21,6 +21,7 @@
#pragma once #pragma once
#include "fdbclient/ClientBooleanParams.h" #include "fdbclient/ClientBooleanParams.h"
#include "flow/IRandom.h" #include "flow/IRandom.h"
#include "flow/ThreadHelper.actor.h"
#if defined(NO_INTELLISENSE) && !defined(FDBCLIENT_TENANT_MANAGEMENT_ACTOR_G_H) #if defined(NO_INTELLISENSE) && !defined(FDBCLIENT_TENANT_MANAGEMENT_ACTOR_G_H)
#define FDBCLIENT_TENANT_MANAGEMENT_ACTOR_G_H #define FDBCLIENT_TENANT_MANAGEMENT_ACTOR_G_H
#include "fdbclient/TenantManagement.actor.g.h" #include "fdbclient/TenantManagement.actor.g.h"
@ -30,6 +31,7 @@
#include <string> #include <string>
#include <map> #include <map>
#include "fdbclient/GenericTransactionHelper.h" #include "fdbclient/GenericTransactionHelper.h"
#include "fdbclient/Metacluster.h"
#include "fdbclient/SystemData.h" #include "fdbclient/SystemData.h"
#include "flow/actorcompiler.h" // has to be last include #include "flow/actorcompiler.h" // has to be last include
@ -78,26 +80,60 @@ Future<TenantMapEntry> getTenant(Reference<DB> db, TenantName name) {
} }
ACTOR template <class Transaction> ACTOR template <class Transaction>
Future<Void> checkTenantMode(Transaction tr) { Future<ClusterType> getClusterType(Transaction tr) {
Optional<MetaclusterRegistrationEntry> metaclusterRegistration =
wait(MetaclusterMetadata::metaclusterRegistration.get(tr));
return metaclusterRegistration.present() ? metaclusterRegistration.get().clusterType : ClusterType::STANDALONE;
}
ACTOR template <class Transaction>
Future<Void> checkTenantMode(Transaction tr, ClusterType expectedClusterType) {
state typename transaction_future_type<Transaction, Optional<Value>>::type tenantModeFuture = state typename transaction_future_type<Transaction, Optional<Value>>::type tenantModeFuture =
tr->get(configKeysPrefix.withSuffix("tenant_mode"_sr)); tr->get(configKeysPrefix.withSuffix("tenant_mode"_sr));
state ClusterType actualClusterType = wait(getClusterType(tr));
Optional<Value> tenantModeValue = wait(safeThreadFutureToFuture(tenantModeFuture)); Optional<Value> tenantModeValue = wait(safeThreadFutureToFuture(tenantModeFuture));
TenantMode tenantMode = TenantMode::fromValue(tenantModeValue.castTo<ValueRef>()); TenantMode tenantMode = TenantMode::fromValue(tenantModeValue.castTo<ValueRef>());
if (tenantMode == TenantMode::DISABLED) { if (actualClusterType != expectedClusterType) {
throw invalid_metacluster_operation();
} else if (actualClusterType == ClusterType::STANDALONE && tenantMode == TenantMode::DISABLED) {
throw tenants_disabled(); throw tenants_disabled();
} }
return Void(); return Void();
} }
TenantMode tenantModeForClusterType(ClusterType clusterType, TenantMode tenantMode);
// Returns true if the specified ID has already been deleted and false if not. If the ID is old enough
// that we no longer keep tombstones for it, an error is thrown.
ACTOR template <class Transaction>
Future<bool> checkTombstone(Transaction tr, int64_t id) {
state Future<bool> tombstoneFuture = TenantMetadata::tenantTombstones.exists(tr, id);
// If we are trying to create a tenant older than the oldest tombstones we still maintain, then we fail it
// with an error.
Optional<TenantTombstoneCleanupData> tombstoneCleanupData = wait(TenantMetadata::tombstoneCleanupData.get(tr));
if (tombstoneCleanupData.present() && tombstoneCleanupData.get().tombstonesErasedThrough >= id) {
throw tenant_creation_permanently_failed();
}
state bool hasTombstone = wait(tombstoneFuture);
return hasTombstone;
}
// Creates a tenant with the given name. If the tenant already exists, the boolean return parameter will be false // Creates a tenant with the given name. If the tenant already exists, the boolean return parameter will be false
// and the existing entry will be returned. If the tenant cannot be created, then the optional will be empty. // and the existing entry will be returned. If the tenant cannot be created, then the optional will be empty.
ACTOR template <class Transaction> ACTOR template <class Transaction>
Future<std::pair<Optional<TenantMapEntry>, bool>> createTenantTransaction(Transaction tr, Future<std::pair<Optional<TenantMapEntry>, bool>> createTenantTransaction(
TenantNameRef name, Transaction tr,
TenantMapEntry tenantEntry) { TenantNameRef name,
TenantMapEntry tenantEntry,
ClusterType clusterType = ClusterType::STANDALONE) {
ASSERT(clusterType != ClusterType::METACLUSTER_MANAGEMENT);
ASSERT(tenantEntry.id >= 0); ASSERT(tenantEntry.id >= 0);
if (name.startsWith("\xff"_sr)) { if (name.startsWith("\xff"_sr)) {
@ -110,17 +146,25 @@ Future<std::pair<Optional<TenantMapEntry>, bool>> createTenantTransaction(Transa
tr->setOption(FDBTransactionOptions::RAW_ACCESS); tr->setOption(FDBTransactionOptions::RAW_ACCESS);
state Future<Optional<TenantMapEntry>> existingEntryFuture = tryGetTenantTransaction(tr, name); state Future<Optional<TenantMapEntry>> existingEntryFuture = tryGetTenantTransaction(tr, name);
wait(checkTenantMode(tr)); state Future<Void> tenantModeCheck = checkTenantMode(tr, clusterType);
state Future<bool> tombstoneFuture =
(clusterType == ClusterType::STANDALONE) ? false : checkTombstone(tr, tenantEntry.id);
state Future<Optional<TenantGroupEntry>> existingTenantGroupEntryFuture; state Future<Optional<TenantGroupEntry>> existingTenantGroupEntryFuture;
if (tenantEntry.tenantGroup.present()) { if (tenantEntry.tenantGroup.present()) {
existingTenantGroupEntryFuture = TenantMetadata::tenantGroupMap.get(tr, tenantEntry.tenantGroup.get()); existingTenantGroupEntryFuture = TenantMetadata::tenantGroupMap.get(tr, tenantEntry.tenantGroup.get());
} }
wait(tenantModeCheck);
Optional<TenantMapEntry> existingEntry = wait(existingEntryFuture); Optional<TenantMapEntry> existingEntry = wait(existingEntryFuture);
if (existingEntry.present()) { if (existingEntry.present()) {
return std::make_pair(existingEntry.get(), false); return std::make_pair(existingEntry.get(), false);
} }
state bool hasTombstone = wait(tombstoneFuture);
if (hasTombstone) {
return std::make_pair(Optional<TenantMapEntry>(), false);
}
state typename transaction_future_type<Transaction, RangeResult>::type prefixRangeFuture = state typename transaction_future_type<Transaction, RangeResult>::type prefixRangeFuture =
tr->getRange(prefixRange(tenantEntry.prefix), 1); tr->getRange(prefixRange(tenantEntry.prefix), 1);
@ -130,6 +174,8 @@ Future<std::pair<Optional<TenantMapEntry>, bool>> createTenantTransaction(Transa
} }
tenantEntry.tenantState = TenantState::READY; tenantEntry.tenantState = TenantState::READY;
tenantEntry.assignedCluster = Optional<ClusterName>();
TenantMetadata::tenantMap.set(tr, name, tenantEntry); TenantMetadata::tenantMap.set(tr, name, tenantEntry);
if (tenantEntry.tenantGroup.present()) { if (tenantEntry.tenantGroup.present()) {
TenantMetadata::tenantGroupTenantIndex.insert(tr, Tuple::makeTuple(tenantEntry.tenantGroup.get(), name)); TenantMetadata::tenantGroupTenantIndex.insert(tr, Tuple::makeTuple(tenantEntry.tenantGroup.get(), name));
@ -167,12 +213,15 @@ Future<int64_t> getNextTenantId(Transaction tr) {
ACTOR template <class DB> ACTOR template <class DB>
Future<Optional<TenantMapEntry>> createTenant(Reference<DB> db, Future<Optional<TenantMapEntry>> createTenant(Reference<DB> db,
TenantName name, TenantName name,
TenantMapEntry tenantEntry = TenantMapEntry()) { TenantMapEntry tenantEntry = TenantMapEntry(),
ClusterType clusterType = ClusterType::STANDALONE) {
state Reference<typename DB::TransactionT> tr = db->createTransaction(); state Reference<typename DB::TransactionT> tr = db->createTransaction();
state bool checkExistence = true; state bool checkExistence = clusterType != ClusterType::METACLUSTER_DATA;
state bool generateTenantId = tenantEntry.id < 0; state bool generateTenantId = tenantEntry.id < 0;
ASSERT(clusterType == ClusterType::STANDALONE || !generateTenantId);
loop { loop {
try { try {
tr->setOption(FDBTransactionOptions::ACCESS_SYSTEM_KEYS); tr->setOption(FDBTransactionOptions::ACCESS_SYSTEM_KEYS);
@ -199,7 +248,7 @@ Future<Optional<TenantMapEntry>> createTenant(Reference<DB> db,
} }
state std::pair<Optional<TenantMapEntry>, bool> newTenant = state std::pair<Optional<TenantMapEntry>, bool> newTenant =
wait(createTenantTransaction(tr, name, tenantEntry)); wait(createTenantTransaction(tr, name, tenantEntry, clusterType));
if (newTenant.second) { if (newTenant.second) {
ASSERT(newTenant.first.present()); ASSERT(newTenant.first.present());
@ -227,11 +276,15 @@ Future<Optional<TenantMapEntry>> createTenant(Reference<DB> db,
ACTOR template <class Transaction> ACTOR template <class Transaction>
Future<Void> deleteTenantTransaction(Transaction tr, Future<Void> deleteTenantTransaction(Transaction tr,
TenantNameRef name, TenantNameRef name,
Optional<int64_t> tenantId = Optional<int64_t>()) { Optional<int64_t> tenantId = Optional<int64_t>(),
ClusterType clusterType = ClusterType::STANDALONE) {
ASSERT(clusterType == ClusterType::STANDALONE || tenantId.present());
ASSERT(clusterType != ClusterType::METACLUSTER_MANAGEMENT);
tr->setOption(FDBTransactionOptions::RAW_ACCESS); tr->setOption(FDBTransactionOptions::RAW_ACCESS);
state Future<Optional<TenantMapEntry>> tenantEntryFuture = tryGetTenantTransaction(tr, name); state Future<Optional<TenantMapEntry>> tenantEntryFuture = tryGetTenantTransaction(tr, name);
wait(checkTenantMode(tr)); wait(checkTenantMode(tr, clusterType));
state Optional<TenantMapEntry> tenantEntry = wait(tenantEntryFuture); state Optional<TenantMapEntry> tenantEntry = wait(tenantEntryFuture);
if (tenantEntry.present() && (!tenantId.present() || tenantEntry.get().id == tenantId.get())) { if (tenantEntry.present() && (!tenantId.present() || tenantEntry.get().id == tenantId.get())) {
@ -262,16 +315,61 @@ Future<Void> deleteTenantTransaction(Transaction tr,
} }
} }
if (clusterType == ClusterType::METACLUSTER_DATA) {
// In data clusters, we store a tombstone
state Future<KeyBackedRangeResult<int64_t>> latestTombstoneFuture =
TenantMetadata::tenantTombstones.getRange(tr, {}, {}, 1, Snapshot::False, Reverse::True);
state Optional<TenantTombstoneCleanupData> cleanupData = wait(TenantMetadata::tombstoneCleanupData.get(tr));
state Version transactionReadVersion = wait(safeThreadFutureToFuture(tr->getReadVersion()));
// If it has been long enough since we last cleaned up the tenant tombstones, we do that first
if (!cleanupData.present() || cleanupData.get().nextTombstoneEraseVersion <= transactionReadVersion) {
state int64_t deleteThroughId = cleanupData.present() ? cleanupData.get().nextTombstoneEraseId : -1;
// Delete all tombstones up through the one currently marked in the cleanup data
if (deleteThroughId >= 0) {
TenantMetadata::tenantTombstones.erase(tr, 0, deleteThroughId + 1);
}
KeyBackedRangeResult<int64_t> latestTombstone = wait(latestTombstoneFuture);
int64_t nextDeleteThroughId = std::max(deleteThroughId, tenantId.get());
if (!latestTombstone.results.empty()) {
nextDeleteThroughId = std::max(nextDeleteThroughId, latestTombstone.results[0]);
}
// The next cleanup will happen at or after TENANT_TOMBSTONE_CLEANUP_INTERVAL seconds have elapsed and
// will clean up tombstones through the most recently allocated ID.
TenantTombstoneCleanupData updatedCleanupData;
updatedCleanupData.tombstonesErasedThrough = deleteThroughId;
updatedCleanupData.nextTombstoneEraseId = nextDeleteThroughId;
updatedCleanupData.nextTombstoneEraseVersion =
transactionReadVersion +
CLIENT_KNOBS->TENANT_TOMBSTONE_CLEANUP_INTERVAL * CLIENT_KNOBS->VERSIONS_PER_SECOND;
TenantMetadata::tombstoneCleanupData.set(tr, updatedCleanupData);
// If the tenant being deleted is within the tombstone window, record the tombstone
if (tenantId.get() > updatedCleanupData.tombstonesErasedThrough) {
TenantMetadata::tenantTombstones.insert(tr, tenantId.get());
}
} else if (tenantId.get() > cleanupData.get().tombstonesErasedThrough) {
// If the tenant being deleted is within the tombstone window, record the tombstone
TenantMetadata::tenantTombstones.insert(tr, tenantId.get());
}
}
return Void(); return Void();
} }
// Deletes the tenant with the given name. If tenantId is specified, the tenant being deleted must also have the same // Deletes the tenant with the given name. If tenantId is specified, the tenant being deleted must also have the same
// ID. // ID.
ACTOR template <class DB> ACTOR template <class DB>
Future<Void> deleteTenant(Reference<DB> db, TenantName name, Optional<int64_t> tenantId = Optional<int64_t>()) { Future<Void> deleteTenant(Reference<DB> db,
TenantName name,
Optional<int64_t> tenantId = Optional<int64_t>(),
ClusterType clusterType = ClusterType::STANDALONE) {
state Reference<typename DB::TransactionT> tr = db->createTransaction(); state Reference<typename DB::TransactionT> tr = db->createTransaction();
state bool checkExistence = true; state bool checkExistence = clusterType == ClusterType::STANDALONE;
loop { loop {
try { try {
tr->setOption(FDBTransactionOptions::ACCESS_SYSTEM_KEYS); tr->setOption(FDBTransactionOptions::ACCESS_SYSTEM_KEYS);
@ -289,7 +387,7 @@ Future<Void> deleteTenant(Reference<DB> db, TenantName name, Optional<int64_t> t
checkExistence = false; checkExistence = false;
} }
wait(deleteTenantTransaction(tr, name, tenantId)); wait(deleteTenantTransaction(tr, name, tenantId, clusterType));
wait(buggifiedCommit(tr, BUGGIFY_WITH_PROB(0.1))); wait(buggifiedCommit(tr, BUGGIFY_WITH_PROB(0.1)));
TraceEvent("DeletedTenant").detail("Tenant", name).detail("Version", tr->getCommittedVersion()); TraceEvent("DeletedTenant").detail("Tenant", name).detail("Version", tr->getCommittedVersion());

View File

@ -289,7 +289,7 @@ public:
state std::set<TenantName> renameSet; state std::set<TenantName> renameSet;
state std::vector<std::pair<TenantName, TenantName>> renameMutations; state std::vector<std::pair<TenantName, TenantName>> renameMutations;
tenantManagementFutures.push_back(TenantAPI::checkTenantMode(&ryw->getTransaction())); tenantManagementFutures.push_back(TenantAPI::checkTenantMode(&ryw->getTransaction(), ClusterType::STANDALONE));
for (auto range : ranges) { for (auto range : ranges) {
if (!range.value().first) { if (!range.value().first) {

View File

@ -56,7 +56,9 @@ struct Tuple {
Tuple& appendNull(); Tuple& appendNull();
Tuple& append(Versionstamp const&); Tuple& append(Versionstamp const&);
StringRef pack() const { return StringRef(data.begin(), data.size()); } Standalone<StringRef> pack() const {
return Standalone<StringRef>(StringRef(data.begin(), data.size()), data.arena());
}
template <typename T> template <typename T>
Tuple& operator<<(T const& t) { Tuple& operator<<(T const& t) {

View File

@ -21,8 +21,10 @@ package api
import ( import (
"fmt" "fmt"
"net"
"os" "os"
"strconv" "strconv"
"strings"
) )
// ProcessConfiguration models the configuration for starting a FoundationDB // ProcessConfiguration models the configuration for starting a FoundationDB
@ -63,8 +65,11 @@ type Argument struct {
Multiplier int `json:"multiplier,omitempty"` Multiplier int `json:"multiplier,omitempty"`
// Offset provides an offset to add to the process number for ProcessNumber // Offset provides an offset to add to the process number for ProcessNumber
// type argujments. // type arguments.
Offset int `json:"offset,omitempty"` Offset int `json:"offset,omitempty"`
// IPFamily provides the family to use for IPList type arguments.
IPFamily int `json:"ipFamily,omitempty"`
} }
// ArgumentType defines the types for arguments. // ArgumentType defines the types for arguments.
@ -84,6 +89,10 @@ const (
// ProcessNumberArgumentType defines an argument that is calculated using // ProcessNumberArgumentType defines an argument that is calculated using
// the number of the process in the process list. // the number of the process in the process list.
ProcessNumberArgumentType = "ProcessNumber" ProcessNumberArgumentType = "ProcessNumber"
// IPListArgumentType defines an argument that is a comma-separated list of
// IP addresses, provided through an environment variable.
IPListArgumentType = "IPList"
) )
// GenerateArgument processes an argument and generates its string // GenerateArgument processes an argument and generates its string
@ -111,24 +120,52 @@ func (argument Argument) GenerateArgument(processNumber int, env map[string]stri
} }
number = number + argument.Offset number = number + argument.Offset
return strconv.Itoa(number), nil return strconv.Itoa(number), nil
case EnvironmentArgumentType: case EnvironmentArgumentType, IPListArgumentType:
var value string return argument.LookupEnv(env)
var present bool
if env != nil {
value, present = env[argument.Source]
}
if !present {
value, present = os.LookupEnv(argument.Source)
}
if !present {
return "", fmt.Errorf("Missing environment variable %s", argument.Source)
}
return value, nil
default: default:
return "", fmt.Errorf("unsupported argument type %s", argument.ArgumentType) return "", fmt.Errorf("unsupported argument type %s", argument.ArgumentType)
} }
} }
// LookupEnv looks up the value for an argument from the environment.
func (argument Argument) LookupEnv(env map[string]string) (string, error) {
var value string
var present bool
if env != nil {
value, present = env[argument.Source]
}
if !present {
value, present = os.LookupEnv(argument.Source)
}
if !present {
return "", fmt.Errorf("missing environment variable %s", argument.Source)
}
if argument.ArgumentType == IPListArgumentType {
ips := strings.Split(value, ",")
for _, ipString := range ips {
ip := net.ParseIP(ipString)
if ip == nil {
continue
}
switch argument.IPFamily {
case 4:
if ip.To4() != nil {
return ipString, nil
}
case 6:
if ip.To16() != nil && ip.To4() == nil {
return ipString, nil
}
default:
return "", fmt.Errorf("unsupported IP family %d", argument.IPFamily)
}
}
return "", fmt.Errorf("could not find IP with family %d", argument.IPFamily)
}
return value, nil
}
// GenerateArguments interprets the arguments in the process configuration and // GenerateArguments interprets the arguments in the process configuration and
// generates a command invocation. // generates a command invocation.
func (configuration *ProcessConfiguration) GenerateArguments(processNumber int, env map[string]string) ([]string, error) { func (configuration *ProcessConfiguration) GenerateArguments(processNumber int, env map[string]string) ([]string, error) {

View File

@ -120,10 +120,216 @@ func TestGeneratingArgumentForEnvironmentVariable(t *testing.T) {
t.Fail() t.Fail()
return return
} }
expectedError := "Missing environment variable FDB_ZONE_ID" expectedError := "missing environment variable FDB_ZONE_ID"
if err.Error() != expectedError { if err.Error() != expectedError {
t.Logf("Expected error %s, but got error %s", expectedError, err) t.Logf("Expected error %s, but got error %s", expectedError, err)
t.Fail() t.Fail()
return return
} }
} }
func TestGeneratingArgumentForIPList(t *testing.T) {
argument := Argument{ArgumentType: IPListArgumentType, Source: "FDB_PUBLIC_IP", IPFamily: 4}
result, err := argument.GenerateArgument(1, map[string]string{"FDB_PUBLIC_IP": "127.0.0.1,::1"})
if err != nil {
t.Error(err)
return
}
if result != "127.0.0.1" {
t.Logf("Expected result 127.0.0.1, but got result %v", result)
t.Fail()
return
}
result, err = argument.GenerateArgument(1, map[string]string{"FDB_PUBLIC_IP": "::1,127.0.0.1"})
if err != nil {
t.Error(err)
return
}
if result != "127.0.0.1" {
t.Logf("Expected result 127.0.0.1, but got result %v", result)
t.Fail()
return
}
argument.IPFamily = 6
result, err = argument.GenerateArgument(1, map[string]string{"FDB_PUBLIC_IP": "127.0.0.1,::1"})
if err != nil {
t.Error(err)
return
}
if result != "::1" {
t.Logf("Expected result ::1, but got result %v", result)
t.Fail()
return
}
result, err = argument.GenerateArgument(1, map[string]string{"FDB_PUBLIC_IP": "::1,127.0.0.1"})
if err != nil {
t.Error(err)
return
}
if result != "::1" {
t.Logf("Expected result ::1, but got result %v", result)
t.Fail()
return
}
result, err = argument.GenerateArgument(1, map[string]string{"FDB_PUBLIC_IP": "bad,::1"})
if err != nil {
t.Error(err)
return
}
if result != "::1" {
t.Logf("Expected result ::1, but got result %v", result)
t.Fail()
return
}
_, err = argument.GenerateArgument(1, map[string]string{"FDB_PUBLIC_IP": "127.0.0.1"})
if err == nil {
t.Logf("Expected error, but did not get an error")
t.Fail()
return
}
expectedError := "could not find IP with family 6"
if err.Error() != expectedError {
t.Logf("Expected error %s, but got error %s", expectedError, err.Error())
t.Fail()
return
}
argument.IPFamily = 5
_, err = argument.GenerateArgument(1, map[string]string{"FDB_PUBLIC_IP": "127.0.0.1"})
if err == nil {
t.Logf("Expected error, but did not get an error")
t.Fail()
return
}
expectedError = "unsupported IP family 5"
if err.Error() != expectedError {
t.Logf("Expected error %s, but got error %s", expectedError, err.Error())
t.Fail()
return
}
}
func TestLookupEnvForEnvironmentVariable(t *testing.T) {
argument := Argument{ArgumentType: EnvironmentArgumentType, Source: "FDB_ZONE_ID"}
result, err := argument.LookupEnv(map[string]string{"FDB_ZONE_ID": "zone1", "FDB_MACHINE_ID": "machine1"})
if err != nil {
t.Error(err)
return
}
if result != "zone1" {
t.Logf("Expected result zone1, but got result %v", result)
t.Fail()
return
}
_, err = argument.LookupEnv(map[string]string{"FDB_MACHINE_ID": "machine1"})
if err == nil {
t.Logf("Expected error result, but did not get an error")
t.Fail()
return
}
expectedError := "missing environment variable FDB_ZONE_ID"
if err.Error() != expectedError {
t.Logf("Expected error %s, but got error %s", expectedError, err)
t.Fail()
return
}
}
func TestLookupEnvForIPList(t *testing.T) {
argument := Argument{ArgumentType: IPListArgumentType, Source: "FDB_PUBLIC_IP", IPFamily: 4}
result, err := argument.LookupEnv(map[string]string{"FDB_PUBLIC_IP": "127.0.0.1,::1"})
if err != nil {
t.Error(err)
return
}
if result != "127.0.0.1" {
t.Logf("Expected result 127.0.0.1, but got result %v", result)
t.Fail()
return
}
result, err = argument.LookupEnv(map[string]string{"FDB_PUBLIC_IP": "::1,127.0.0.1"})
if err != nil {
t.Error(err)
return
}
if result != "127.0.0.1" {
t.Logf("Expected result 127.0.0.1, but got result %v", result)
t.Fail()
return
}
argument.IPFamily = 6
result, err = argument.LookupEnv(map[string]string{"FDB_PUBLIC_IP": "127.0.0.1,::1"})
if err != nil {
t.Error(err)
return
}
if result != "::1" {
t.Logf("Expected result ::1, but got result %v", result)
t.Fail()
return
}
result, err = argument.LookupEnv(map[string]string{"FDB_PUBLIC_IP": "::1,127.0.0.1"})
if err != nil {
t.Error(err)
return
}
if result != "::1" {
t.Logf("Expected result ::1, but got result %v", result)
t.Fail()
return
}
result, err = argument.LookupEnv(map[string]string{"FDB_PUBLIC_IP": "bad,::1"})
if err != nil {
t.Error(err)
return
}
if result != "::1" {
t.Logf("Expected result ::1, but got result %v", result)
t.Fail()
return
}
_, err = argument.LookupEnv(map[string]string{"FDB_PUBLIC_IP": "127.0.0.1"})
if err == nil {
t.Logf("Expected error, but did not get an error")
t.Fail()
return
}
expectedError := "could not find IP with family 6"
if err.Error() != expectedError {
t.Logf("Expected error %s, but got error %s", expectedError, err.Error())
t.Fail()
return
}
argument.IPFamily = 5
_, err = argument.LookupEnv(map[string]string{"FDB_PUBLIC_IP": "127.0.0.1"})
if err == nil {
t.Logf("Expected error, but did not get an error")
t.Fail()
return
}
expectedError = "unsupported IP family 5"
if err.Error() != expectedError {
t.Logf("Expected error %s, but got error %s", expectedError, err.Error())
t.Fail()
return
}
}

View File

@ -96,13 +96,16 @@ func CreatePodClient(logger logr.Logger) (*PodClient, error) {
// retrieveEnvironmentVariables extracts the environment variables we have for // retrieveEnvironmentVariables extracts the environment variables we have for
// an argument into a map. // an argument into a map.
func retrieveEnvironmentVariables(argument api.Argument, target map[string]string) { func retrieveEnvironmentVariables(monitor *Monitor, argument api.Argument, target map[string]string) {
if argument.Source != "" { if argument.Source != "" {
target[argument.Source] = os.Getenv(argument.Source) value, err := argument.LookupEnv(monitor.CustomEnvironment)
if err == nil {
target[argument.Source] = value
}
} }
if argument.Values != nil { if argument.Values != nil {
for _, childArgument := range argument.Values { for _, childArgument := range argument.Values {
retrieveEnvironmentVariables(childArgument, target) retrieveEnvironmentVariables(monitor, childArgument, target)
} }
} }
} }
@ -112,7 +115,7 @@ func retrieveEnvironmentVariables(argument api.Argument, target map[string]strin
func (client *PodClient) UpdateAnnotations(monitor *Monitor) error { func (client *PodClient) UpdateAnnotations(monitor *Monitor) error {
environment := make(map[string]string) environment := make(map[string]string)
for _, argument := range monitor.ActiveConfiguration.Arguments { for _, argument := range monitor.ActiveConfiguration.Arguments {
retrieveEnvironmentVariables(argument, environment) retrieveEnvironmentVariables(monitor, argument, environment)
} }
environment["BINARY_DIR"] = path.Dir(monitor.ActiveConfiguration.BinaryPath) environment["BINARY_DIR"] = path.Dir(monitor.ActiveConfiguration.BinaryPath)
jsonEnvironment, err := json.Marshal(environment) jsonEnvironment, err := json.Marshal(environment)

View File

@ -63,6 +63,25 @@ public:
enum TSSMode { Disabled, EnabledNormal, EnabledAddDelay, EnabledDropMutations }; enum TSSMode { Disabled, EnabledNormal, EnabledAddDelay, EnabledDropMutations };
enum class BackupAgentType { NoBackupAgents, WaitForType, BackupToFile, BackupToDB }; enum class BackupAgentType { NoBackupAgents, WaitForType, BackupToFile, BackupToDB };
enum class ExtraDatabaseMode { Disabled, LocalOrSingle, Single, Local, Multiple };
static ExtraDatabaseMode stringToExtraDatabaseMode(std::string databaseMode) {
if (databaseMode == "Disabled") {
return ExtraDatabaseMode::Disabled;
} else if (databaseMode == "LocalOrSingle") {
return ExtraDatabaseMode::LocalOrSingle;
} else if (databaseMode == "Single") {
return ExtraDatabaseMode::Single;
} else if (databaseMode == "Local") {
return ExtraDatabaseMode::Local;
} else if (databaseMode == "Multiple") {
return ExtraDatabaseMode::Multiple;
} else {
TraceEvent(SevError, "UnknownExtraDatabaseMode").detail("DatabaseMode", databaseMode);
ASSERT(false);
throw internal_error();
}
};
// Subclasses may subclass ProcessInfo as well // Subclasses may subclass ProcessInfo as well
struct MachineInfo; struct MachineInfo;
@ -405,7 +424,7 @@ public:
allSwapsDisabled = false; allSwapsDisabled = false;
} }
bool canSwapToMachine(Optional<Standalone<StringRef>> zoneId) const { bool canSwapToMachine(Optional<Standalone<StringRef>> zoneId) const {
return swapsDisabled.count(zoneId) == 0 && !allSwapsDisabled && !extraDB; return swapsDisabled.count(zoneId) == 0 && !allSwapsDisabled && extraDatabases.empty();
} }
void enableSwapsToAll() { void enableSwapsToAll() {
swapsDisabled.clear(); swapsDisabled.clear();
@ -432,7 +451,7 @@ public:
int listenersPerProcess; int listenersPerProcess;
std::set<NetworkAddress> protectedAddresses; std::set<NetworkAddress> protectedAddresses;
std::map<NetworkAddress, ProcessInfo*> currentlyRebootingProcesses; std::map<NetworkAddress, ProcessInfo*> currentlyRebootingProcesses;
class ClusterConnectionString* extraDB = nullptr; std::vector<std::string> extraDatabases;
Reference<IReplicationPolicy> storagePolicy; Reference<IReplicationPolicy> storagePolicy;
Reference<IReplicationPolicy> tLogPolicy; Reference<IReplicationPolicy> tLogPolicy;
int32_t tLogWriteAntiQuorum; int32_t tLogWriteAntiQuorum;

View File

@ -51,13 +51,13 @@ void debug_advanceVersion(UID id, int64_t version, const char* suffix) {
} }
void debug_advanceMinCommittedVersion(UID id, int64_t version) { void debug_advanceMinCommittedVersion(UID id, int64_t version) {
if (!g_network->isSimulated() || g_simulator.extraDB) if (!g_network->isSimulated() || !g_simulator.extraDatabases.empty())
return; return;
debug_advanceVersion(id, version, "min"); debug_advanceVersion(id, version, "min");
} }
void debug_advanceMaxCommittedVersion(UID id, int64_t version) { void debug_advanceMaxCommittedVersion(UID id, int64_t version) {
if (!g_network->isSimulated() || g_simulator.extraDB) if (!g_network->isSimulated() || !g_simulator.extraDatabases.empty())
return; return;
debug_advanceVersion(id, version, "max"); debug_advanceVersion(id, version, "max");
} }
@ -67,7 +67,7 @@ bool debug_checkPartRestoredVersion(UID id,
std::string context, std::string context,
std::string minormax, std::string minormax,
Severity sev = SevError) { Severity sev = SevError) {
if (!g_network->isSimulated() || g_simulator.extraDB) if (!g_network->isSimulated() || !g_simulator.extraDatabases.empty())
return false; return false;
if (disabledMachines.count(id)) if (disabledMachines.count(id))
return false; return false;
@ -88,33 +88,33 @@ bool debug_checkPartRestoredVersion(UID id,
} }
bool debug_checkRestoredVersion(UID id, int64_t version, std::string context, Severity sev) { bool debug_checkRestoredVersion(UID id, int64_t version, std::string context, Severity sev) {
if (!g_network->isSimulated() || g_simulator.extraDB) if (!g_network->isSimulated() || !g_simulator.extraDatabases.empty())
return false; return false;
return debug_checkPartRestoredVersion(id, version, context, "min", sev) || return debug_checkPartRestoredVersion(id, version, context, "min", sev) ||
debug_checkPartRestoredVersion(id, version, context, "max", sev); debug_checkPartRestoredVersion(id, version, context, "max", sev);
} }
void debug_removeVersions(UID id) { void debug_removeVersions(UID id) {
if (!g_network->isSimulated() || g_simulator.extraDB) if (!g_network->isSimulated() || !g_simulator.extraDatabases.empty())
return; return;
validationData.erase(id.toString() + "min"); validationData.erase(id.toString() + "min");
validationData.erase(id.toString() + "max"); validationData.erase(id.toString() + "max");
} }
bool debug_versionsExist(UID id) { bool debug_versionsExist(UID id) {
if (!g_network->isSimulated() || g_simulator.extraDB) if (!g_network->isSimulated() || !g_simulator.extraDatabases.empty())
return false; return false;
return validationData.count(id.toString() + "min") != 0 || validationData.count(id.toString() + "max") != 0; return validationData.count(id.toString() + "min") != 0 || validationData.count(id.toString() + "max") != 0;
} }
bool debug_checkMinRestoredVersion(UID id, int64_t version, std::string context, Severity sev) { bool debug_checkMinRestoredVersion(UID id, int64_t version, std::string context, Severity sev) {
if (!g_network->isSimulated() || g_simulator.extraDB) if (!g_network->isSimulated() || !g_simulator.extraDatabases.empty())
return false; return false;
return debug_checkPartRestoredVersion(id, version, context, "min", sev); return debug_checkPartRestoredVersion(id, version, context, "min", sev);
} }
bool debug_checkMaxRestoredVersion(UID id, int64_t version, std::string context, Severity sev) { bool debug_checkMaxRestoredVersion(UID id, int64_t version, std::string context, Severity sev) {
if (!g_network->isSimulated() || g_simulator.extraDB) if (!g_network->isSimulated() || !g_simulator.extraDatabases.empty())
return false; return false;
return debug_checkPartRestoredVersion(id, version, context, "max", sev); return debug_checkPartRestoredVersion(id, version, context, "max", sev);
} }
@ -129,13 +129,13 @@ void debug_setCheckRelocationDuration(bool check) {
checkRelocationDuration = check; checkRelocationDuration = check;
} }
void debug_advanceVersionTimestamp(int64_t version, double t) { void debug_advanceVersionTimestamp(int64_t version, double t) {
if (!g_network->isSimulated() || g_simulator.extraDB) if (!g_network->isSimulated() || !g_simulator.extraDatabases.empty())
return; return;
timedVersionsValidationData[version] = t; timedVersionsValidationData[version] = t;
} }
bool debug_checkVersionTime(int64_t version, double t, std::string context, Severity sev) { bool debug_checkVersionTime(int64_t version, double t, std::string context, Severity sev) {
if (!g_network->isSimulated() || g_simulator.extraDB) if (!g_network->isSimulated() || !g_simulator.extraDatabases.empty())
return false; return false;
if (!timedVersionsValidationData.count(version)) { if (!timedVersionsValidationData.count(version)) {
TraceEvent(SevWarn, (context + "UnknownTime").c_str()) TraceEvent(SevWarn, (context + "UnknownTime").c_str())

View File

@ -20,6 +20,7 @@
#include "fdbclient/BackupAgent.actor.h" #include "fdbclient/BackupAgent.actor.h"
#include "fdbclient/KeyBackedTypes.h" // for key backed map codecs for tss mapping #include "fdbclient/KeyBackedTypes.h" // for key backed map codecs for tss mapping
#include "fdbclient/Metacluster.h"
#include "fdbclient/MutationList.h" #include "fdbclient/MutationList.h"
#include "fdbclient/Notified.h" #include "fdbclient/Notified.h"
#include "fdbclient/SystemData.h" #include "fdbclient/SystemData.h"
@ -690,6 +691,26 @@ private:
} }
} }
void checkSetMetaclusterRegistration(MutationRef m) {
if (m.param1 == MetaclusterMetadata::metaclusterRegistration.key) {
MetaclusterRegistrationEntry entry = MetaclusterRegistrationEntry::decode(m.param2);
TraceEvent("SetMetaclusterRegistration", dbgid)
.detail("ClusterType", entry.clusterType)
.detail("MetaclusterID", entry.metaclusterId)
.detail("MetaclusterName", entry.metaclusterName)
.detail("ClusterID", entry.id)
.detail("ClusterName", entry.name);
if (!initialCommit) {
txnStateStore->set(KeyValueRef(m.param1, m.param2));
}
confChange = true;
CODE_PROBE(true, "Metacluster registration set");
}
}
void checkClearKeyServerKeys(KeyRangeRef range) { void checkClearKeyServerKeys(KeyRangeRef range) {
if (!keyServersKeys.intersects(range)) { if (!keyServersKeys.intersects(range)) {
return; return;
@ -1079,6 +1100,19 @@ private:
} }
} }
void checkClearMetaclusterRegistration(KeyRangeRef range) {
if (range.contains(MetaclusterMetadata::metaclusterRegistration.key)) {
TraceEvent("ClearMetaclusterRegistration", dbgid);
if (!initialCommit) {
txnStateStore->clear(singleKeyRange(MetaclusterMetadata::metaclusterRegistration.key));
}
confChange = true;
CODE_PROBE(true, "Metacluster registration cleared");
}
}
void checkClearMiscRangeKeys(KeyRangeRef range) { void checkClearMiscRangeKeys(KeyRangeRef range) {
if (initialCommit) { if (initialCommit) {
return; return;
@ -1201,6 +1235,7 @@ public:
checkSetMinRequiredCommitVersionKey(m); checkSetMinRequiredCommitVersionKey(m);
checkSetVersionEpochKey(m); checkSetVersionEpochKey(m);
checkSetTenantMapPrefix(m); checkSetTenantMapPrefix(m);
checkSetMetaclusterRegistration(m);
checkSetOtherKeys(m); checkSetOtherKeys(m);
} else if (m.type == MutationRef::ClearRange && isSystemKey(m.param2)) { } else if (m.type == MutationRef::ClearRange && isSystemKey(m.param2)) {
KeyRangeRef range(m.param1, m.param2); KeyRangeRef range(m.param1, m.param2);
@ -1218,6 +1253,7 @@ public:
checkClearTssQuarantineKeys(m, range); checkClearTssQuarantineKeys(m, range);
checkClearVersionEpochKeys(m, range); checkClearVersionEpochKeys(m, range);
checkClearTenantMapPrefix(range); checkClearTenantMapPrefix(range);
checkClearMetaclusterRegistration(range);
checkClearMiscRangeKeys(range); checkClearMiscRangeKeys(range);
} }
} }

File diff suppressed because it is too large Load Diff

View File

@ -640,6 +640,8 @@ ACTOR Future<BlobFileIndex> writeDeltaFile(Reference<BlobWorkerData> bwData,
compressFilter, compressFilter,
cipherKeysCtx); cipherKeysCtx);
state size_t serializedSize = serialized.size(); state size_t serializedSize = serialized.size();
bwData->stats.compressionBytesRaw += deltasToWrite.expectedSize();
bwData->stats.compressionBytesFinal += serializedSize;
// Free up deltasToWrite here to reduce memory // Free up deltasToWrite here to reduce memory
deltasToWrite = Standalone<GranuleDeltas>(); deltasToWrite = Standalone<GranuleDeltas>();
@ -735,6 +737,13 @@ ACTOR Future<BlobFileIndex> writeDeltaFile(Reference<BlobWorkerData> bwData,
} }
} }
ACTOR Future<Void> reevaluateInitialSplit(Reference<BlobWorkerData> bwData,
UID granuleID,
KeyRange keyRange,
int64_t epoch,
int64_t seqno,
Key proposedSplitKey);
ACTOR Future<BlobFileIndex> writeSnapshot(Reference<BlobWorkerData> bwData, ACTOR Future<BlobFileIndex> writeSnapshot(Reference<BlobWorkerData> bwData,
Reference<BlobConnectionProvider> bstore, Reference<BlobConnectionProvider> bstore,
KeyRange keyRange, KeyRange keyRange,
@ -743,33 +752,59 @@ ACTOR Future<BlobFileIndex> writeSnapshot(Reference<BlobWorkerData> bwData,
int64_t seqno, int64_t seqno,
Version version, Version version,
PromiseStream<RangeResult> rows, PromiseStream<RangeResult> rows,
bool createGranuleHistory) { bool initialSnapshot) {
state std::string fileName = randomBGFilename(bwData->id, granuleID, version, ".snapshot"); state std::string fileName = randomBGFilename(bwData->id, granuleID, version, ".snapshot");
state Standalone<GranuleSnapshot> snapshot; state Standalone<GranuleSnapshot> snapshot;
state int64_t bytesRead = 0;
state bool injectTooBig = initialSnapshot && g_network->isSimulated() && BUGGIFY_WITH_PROB(0.1);
wait(delay(0, TaskPriority::BlobWorkerUpdateStorage)); wait(delay(0, TaskPriority::BlobWorkerUpdateStorage));
loop { loop {
try { try {
if (initialSnapshot && snapshot.size() > 1 &&
(injectTooBig || bytesRead >= 3 * SERVER_KNOBS->BG_SNAPSHOT_FILE_TARGET_BYTES)) {
// throw transaction too old either on injection for simulation, or if snapshot would be too large now
throw transaction_too_old();
}
RangeResult res = waitNext(rows.getFuture()); RangeResult res = waitNext(rows.getFuture());
snapshot.arena().dependsOn(res.arena()); snapshot.arena().dependsOn(res.arena());
snapshot.append(snapshot.arena(), res.begin(), res.size()); snapshot.append(snapshot.arena(), res.begin(), res.size());
bytesRead += res.expectedSize();
wait(yield(TaskPriority::BlobWorkerUpdateStorage)); wait(yield(TaskPriority::BlobWorkerUpdateStorage));
} catch (Error& e) { } catch (Error& e) {
if (e.code() == error_code_end_of_stream) { if (e.code() == error_code_end_of_stream) {
break; break;
} }
throw e; // if we got transaction_too_old naturally, have lower threshold for re-evaluating (2xlimit)
if (initialSnapshot && snapshot.size() > 1 && e.code() == error_code_transaction_too_old &&
(injectTooBig || bytesRead >= 2 * SERVER_KNOBS->BG_SNAPSHOT_FILE_TARGET_BYTES)) {
// idle this actor, while we tell the manager this is too big and to re-evaluate granules and revoke us
if (BW_DEBUG) {
fmt::print("Granule [{0} - {1}) re-evaluating snapshot after {2} bytes ({3} limit) {4}\n",
keyRange.begin.printable(),
keyRange.end.printable(),
bytesRead,
SERVER_KNOBS->BG_SNAPSHOT_FILE_TARGET_BYTES,
injectTooBig ? "(injected)" : "");
}
wait(reevaluateInitialSplit(
bwData, granuleID, keyRange, epoch, seqno, snapshot[snapshot.size() / 2].key));
ASSERT(false);
} else {
throw e;
}
} }
} }
wait(delay(0, TaskPriority::BlobWorkerUpdateStorage)); wait(delay(0, TaskPriority::BlobWorkerUpdateStorage));
if (BW_DEBUG) { if (BW_DEBUG) {
fmt::print("Granule [{0} - {1}) read {2} snapshot rows\n", fmt::print("Granule [{0} - {1}) read {2} snapshot rows ({3} bytes)\n",
keyRange.begin.printable(), keyRange.begin.printable(),
keyRange.end.printable(), keyRange.end.printable(),
snapshot.size()); snapshot.size(),
bytesRead);
} }
if (g_network->isSimulated()) { if (g_network->isSimulated()) {
@ -804,6 +839,8 @@ ACTOR Future<BlobFileIndex> writeSnapshot(Reference<BlobWorkerData> bwData,
compressFilter, compressFilter,
cipherKeysCtx); cipherKeysCtx);
state size_t serializedSize = serialized.size(); state size_t serializedSize = serialized.size();
bwData->stats.compressionBytesRaw += snapshot.expectedSize();
bwData->stats.compressionBytesFinal += serializedSize;
// free snapshot to reduce memory // free snapshot to reduce memory
snapshot = Standalone<GranuleSnapshot>(); snapshot = Standalone<GranuleSnapshot>();
@ -842,7 +879,7 @@ ACTOR Future<BlobFileIndex> writeSnapshot(Reference<BlobWorkerData> bwData,
blobGranuleFileValueFor(fname, 0, serializedSize, serializedSize, cipherKeysMeta); blobGranuleFileValueFor(fname, 0, serializedSize, serializedSize, cipherKeysMeta);
tr->set(snapshotFileKey, snapshotFileValue); tr->set(snapshotFileKey, snapshotFileValue);
// create granule history at version if this is a new granule with the initial dump from FDB // create granule history at version if this is a new granule with the initial dump from FDB
if (createGranuleHistory) { if (initialSnapshot) {
Key historyKey = blobGranuleHistoryKeyFor(keyRange, version); Key historyKey = blobGranuleHistoryKeyFor(keyRange, version);
Standalone<BlobGranuleHistoryValue> historyValue; Standalone<BlobGranuleHistoryValue> historyValue;
historyValue.granuleID = granuleID; historyValue.granuleID = granuleID;
@ -909,7 +946,6 @@ ACTOR Future<BlobFileIndex> dumpInitialSnapshotFromFDB(Reference<BlobWorkerData>
state FlowLock::Releaser holdingDVL(bwData->initialSnapshotLock); state FlowLock::Releaser holdingDVL(bwData->initialSnapshotLock);
state Reference<ReadYourWritesTransaction> tr = makeReference<ReadYourWritesTransaction>(bwData->db); state Reference<ReadYourWritesTransaction> tr = makeReference<ReadYourWritesTransaction>(bwData->db);
state int64_t bytesRead = 0;
state int retries = 0; state int retries = 0;
state Version lastReadVersion = invalidVersion; state Version lastReadVersion = invalidVersion;
state Version readVersion = invalidVersion; state Version readVersion = invalidVersion;
@ -948,12 +984,11 @@ ACTOR Future<BlobFileIndex> dumpInitialSnapshotFromFDB(Reference<BlobWorkerData>
throw e; throw e;
} }
if (BW_DEBUG) { if (BW_DEBUG) {
fmt::print("Dumping snapshot {0} from FDB for [{1} - {2}) got error {3} after {4} bytes\n", fmt::print("Dumping snapshot {0} from FDB for [{1} - {2}) got error {3}\n",
retries + 1, retries + 1,
metadata->keyRange.begin.printable(), metadata->keyRange.begin.printable(),
metadata->keyRange.end.printable(), metadata->keyRange.end.printable(),
e.name(), e.name());
bytesRead);
} }
state Error err = e; state Error err = e;
if (e.code() == error_code_server_overloaded) { if (e.code() == error_code_server_overloaded) {
@ -968,7 +1003,6 @@ ACTOR Future<BlobFileIndex> dumpInitialSnapshotFromFDB(Reference<BlobWorkerData>
.error(err) .error(err)
.detail("Granule", metadata->keyRange) .detail("Granule", metadata->keyRange)
.detail("Count", retries); .detail("Count", retries);
bytesRead = 0;
lastReadVersion = readVersion; lastReadVersion = readVersion;
// Pop change feed up to readVersion, because that data will be before the next snapshot // Pop change feed up to readVersion, because that data will be before the next snapshot
// Do this to prevent a large amount of CF data from accumulating if we have consecutive failures to // Do this to prevent a large amount of CF data from accumulating if we have consecutive failures to
@ -1190,6 +1224,7 @@ ACTOR Future<BlobFileIndex> checkSplitAndReSnapshot(Reference<BlobWorkerData> bw
bwData->currentManagerStatusStream.get().send(GranuleStatusReply(metadata->keyRange, bwData->currentManagerStatusStream.get().send(GranuleStatusReply(metadata->keyRange,
true, true,
writeHot, writeHot,
false,
statusEpoch, statusEpoch,
statusSeqno, statusSeqno,
granuleID, granuleID,
@ -1253,6 +1288,64 @@ ACTOR Future<BlobFileIndex> checkSplitAndReSnapshot(Reference<BlobWorkerData> bw
return reSnapshotIdx; return reSnapshotIdx;
} }
// wait indefinitely to tell manager to re-evaluate this split, until the granule is revoked
ACTOR Future<Void> reevaluateInitialSplit(Reference<BlobWorkerData> bwData,
UID granuleID,
KeyRange keyRange,
int64_t epoch,
int64_t seqno,
Key proposedSplitKey) {
// wait for first stream to be initialized
while (!bwData->statusStreamInitialized) {
wait(bwData->currentManagerStatusStream.onChange());
}
loop {
try {
// wait for manager stream to become ready, and send a message
loop {
choose {
when(wait(bwData->currentManagerStatusStream.get().onReady())) { break; }
when(wait(bwData->currentManagerStatusStream.onChange())) {}
}
}
GranuleStatusReply reply(keyRange,
true,
false,
true,
epoch,
seqno,
granuleID,
invalidVersion,
invalidVersion,
false,
epoch,
seqno);
reply.proposedSplitKey = proposedSplitKey;
bwData->currentManagerStatusStream.get().send(reply);
// if a new manager appears, also tell it about this granule being splittable, or retry after a certain
// amount of time of not hearing back
wait(success(timeout(bwData->currentManagerStatusStream.onChange(), 10.0)));
wait(delay(0));
CODE_PROBE(true, "Blob worker re-sending initialsplit too big");
} catch (Error& e) {
if (e.code() == error_code_operation_cancelled) {
throw e;
}
CODE_PROBE(true, "Blob worker re-sending merge candidate to manager after not error/not hearing back");
// if we got broken promise while waiting, the old stream was killed, so we don't need to wait
// on change, just retry
if (e.code() == error_code_broken_promise) {
wait(delay(FLOW_KNOBS->PREVENT_FAST_SPIN_DELAY));
} else {
wait(bwData->currentManagerStatusStream.onChange());
}
}
}
}
ACTOR Future<Void> granuleCheckMergeCandidate(Reference<BlobWorkerData> bwData, ACTOR Future<Void> granuleCheckMergeCandidate(Reference<BlobWorkerData> bwData,
Reference<GranuleMetadata> metadata, Reference<GranuleMetadata> metadata,
UID granuleID, UID granuleID,
@ -1285,7 +1378,6 @@ ACTOR Future<Void> granuleCheckMergeCandidate(Reference<BlobWorkerData> bwData,
wait(bwData->currentManagerStatusStream.onChange()); wait(bwData->currentManagerStatusStream.onChange());
} }
// FIXME: after a certain amount of retries/time, we may want to re-check anyway
state double sendTimeGiveUp = now() + SERVER_KNOBS->BG_MERGE_CANDIDATE_THRESHOLD_SECONDS / 2.0; state double sendTimeGiveUp = now() + SERVER_KNOBS->BG_MERGE_CANDIDATE_THRESHOLD_SECONDS / 2.0;
loop { loop {
try { try {
@ -1304,6 +1396,7 @@ ACTOR Future<Void> granuleCheckMergeCandidate(Reference<BlobWorkerData> bwData,
} }
bwData->currentManagerStatusStream.get().send(GranuleStatusReply(metadata->keyRange, bwData->currentManagerStatusStream.get().send(GranuleStatusReply(metadata->keyRange,
false,
false, false,
false, false,
metadata->continueEpoch, metadata->continueEpoch,
@ -2487,7 +2580,7 @@ ACTOR Future<Void> blobGranuleLoadHistory(Reference<BlobWorkerData> bwData,
} }
auto prev = bwData->granuleHistory.intersectingRanges(curHistory.range); auto prev = bwData->granuleHistory.intersectingRanges(curHistory.range);
bool allLess = true; bool allLess = true; // if the version is less than all existing granules
for (auto& it : prev) { for (auto& it : prev) {
if (it.cvalue().isValid() && curHistory.version >= it.cvalue()->endVersion) { if (it.cvalue().isValid() && curHistory.version >= it.cvalue()->endVersion) {
allLess = false; allLess = false;
@ -2524,7 +2617,9 @@ ACTOR Future<Void> blobGranuleLoadHistory(Reference<BlobWorkerData> bwData,
} }
state int pIdx = 0; state int pIdx = 0;
state bool noParentsPresent = true; state bool anyParentsMissing = curHistory.value.parentVersions.empty();
state std::vector<GranuleHistory> nexts;
nexts.reserve(curHistory.value.parentVersions.size());
// FIXME: parallelize this for all parents/all entries in queue? // FIXME: parallelize this for all parents/all entries in queue?
loop { loop {
if (pIdx >= curHistory.value.parentVersions.size()) { if (pIdx >= curHistory.value.parentVersions.size()) {
@ -2547,7 +2642,7 @@ ACTOR Future<Void> blobGranuleLoadHistory(Reference<BlobWorkerData> bwData,
// curHistory.version] // curHistory.version]
inserted.first->second.entry = makeReference<GranuleHistoryEntry>( inserted.first->second.entry = makeReference<GranuleHistoryEntry>(
next.range, next.value.granuleID, next.version, curHistory.version); next.range, next.value.granuleID, next.version, curHistory.version);
queue.push_back(next); nexts.push_back(next);
if (BW_HISTORY_DEBUG) { if (BW_HISTORY_DEBUG) {
fmt::print("HL {0} {1}) [{2} - {3}) @ {4}: loaded parent [{5} - {6}) @ {7}\n", fmt::print("HL {0} {1}) [{2} - {3}) @ {4}: loaded parent [{5} - {6}) @ {7}\n",
bwData->id.shortString().substr(0, 5), bwData->id.shortString().substr(0, 5),
@ -2575,7 +2670,8 @@ ACTOR Future<Void> blobGranuleLoadHistory(Reference<BlobWorkerData> bwData,
} }
ASSERT(inserted.first->second.entry->endVersion == curHistory.version); ASSERT(inserted.first->second.entry->endVersion == curHistory.version);
} }
noParentsPresent = false; } else {
anyParentsMissing = true;
} }
pIdx++; pIdx++;
@ -2584,16 +2680,21 @@ ACTOR Future<Void> blobGranuleLoadHistory(Reference<BlobWorkerData> bwData,
} }
} }
if (noParentsPresent) { if (anyParentsMissing) {
if (BW_HISTORY_DEBUG) { if (BW_HISTORY_DEBUG) {
fmt::print("HL {0} {1}) [{2} - {3}) @ {4}: root b/c no parents\n", fmt::print("HL {0} {1}) [{2} - {3}) @ {4}: root b/c parents missing\n",
bwData->id.shortString().substr(0, 5), bwData->id.shortString().substr(0, 5),
loadId, loadId,
curHistory.range.begin.printable(), curHistory.range.begin.printable(),
curHistory.range.end.printable(), curHistory.range.end.printable(),
curHistory.version); curHistory.version);
} }
rootGranules.push(OrderedHistoryKey(curHistory.version, curHistory.value.granuleID)); rootGranules.push(OrderedHistoryKey(curHistory.version, curHistory.value.granuleID));
} else {
for (auto& it : nexts) {
queue.push_back(it);
}
} }
} }
@ -2806,17 +2907,21 @@ std::vector<std::pair<KeyRange, Future<GranuleFiles>>> loadHistoryChunks(Referen
if (!it.cvalue().isValid()) { if (!it.cvalue().isValid()) {
throw blob_granule_transaction_too_old(); throw blob_granule_transaction_too_old();
} }
if (expectedEndVersion != it.cvalue()->endVersion) { if (expectedEndVersion > it.cvalue()->endVersion) {
fmt::print("live granule history version {0} for [{1} - {2}) != history end version {3} for " if (BW_DEBUG) {
"[{4} - {5})\n", // history must have been pruned up to live granule, but BW still has previous history cached.
expectedEndVersion, fmt::print("live granule history version {0} for [{1} - {2}) != history end version {3} for "
keyRange.begin.printable(), "[{4} - {5}) on BW {6}\n",
keyRange.end.printable(), expectedEndVersion,
it.cvalue()->endVersion, keyRange.begin.printable(),
it.begin().printable(), keyRange.end.printable(),
it.end().printable()); it.cvalue()->endVersion,
it.begin().printable(),
it.end().printable(),
bwData->id.toString().substr(0, 5));
}
throw blob_granule_transaction_too_old();
} }
ASSERT(expectedEndVersion == it.cvalue()->endVersion);
visited.insert(it.cvalue()->granuleID); visited.insert(it.cvalue()->granuleID);
queue.push_back(it.cvalue()); queue.push_back(it.cvalue());
@ -4213,6 +4318,10 @@ ACTOR Future<Void> handleFlushGranuleReq(Reference<BlobWorkerData> self, FlushGr
// force granule to flush at this version, and wait // force granule to flush at this version, and wait
if (req.flushVersion > metadata->pendingDeltaVersion) { if (req.flushVersion > metadata->pendingDeltaVersion) {
// first, wait for granule active // first, wait for granule active
if (!metadata->activeCFData.get().isValid()) {
req.reply.sendError(wrong_shard_server());
return Void();
}
// wait for change feed version to catch up to ensure we have all data // wait for change feed version to catch up to ensure we have all data
if (metadata->activeCFData.get()->getVersion() < req.flushVersion) { if (metadata->activeCFData.get()->getVersion() < req.flushVersion) {
@ -4346,7 +4455,7 @@ ACTOR Future<Void> blobWorker(BlobWorkerInterface bwInterf,
} }
try { try {
if (SERVER_KNOBS->BG_RANGE_SOURCE != "tenant") { if (SERVER_KNOBS->BG_METADATA_SOURCE != "tenant") {
if (BW_DEBUG) { if (BW_DEBUG) {
fmt::print("BW constructing backup container from {0}\n", SERVER_KNOBS->BG_URL); fmt::print("BW constructing backup container from {0}\n", SERVER_KNOBS->BG_URL);
} }
@ -4379,9 +4488,7 @@ ACTOR Future<Void> blobWorker(BlobWorkerInterface bwInterf,
self->addActor.send(waitFailureServer(bwInterf.waitFailure.getFuture())); self->addActor.send(waitFailureServer(bwInterf.waitFailure.getFuture()));
self->addActor.send(runGRVChecks(self)); self->addActor.send(runGRVChecks(self));
if (SERVER_KNOBS->BG_RANGE_SOURCE == "tenant") { self->addActor.send(monitorTenants(self));
self->addActor.send(monitorTenants(self));
}
state Future<Void> selfRemoved = monitorRemoval(self); state Future<Void> selfRemoved = monitorRemoval(self);
TraceEvent("BlobWorkerInit", self->id).log(); TraceEvent("BlobWorkerInit", self->id).log();

View File

@ -32,6 +32,7 @@
#include "flow/ActorCollection.h" #include "flow/ActorCollection.h"
#include "fdbclient/ClusterConnectionMemoryRecord.h" #include "fdbclient/ClusterConnectionMemoryRecord.h"
#include "fdbclient/NativeAPI.actor.h" #include "fdbclient/NativeAPI.actor.h"
#include "fdbclient/TenantManagement.actor.h"
#include "fdbserver/ApplyMetadataMutation.h" #include "fdbserver/ApplyMetadataMutation.h"
#include "fdbserver/BackupInterface.h" #include "fdbserver/BackupInterface.h"
#include "fdbserver/BackupProgress.actor.h" #include "fdbserver/BackupProgress.actor.h"
@ -249,8 +250,10 @@ ACTOR Future<Void> clusterWatchDatabase(ClusterControllerData* cluster,
dbInfo.myLocality = db->serverInfo->get().myLocality; dbInfo.myLocality = db->serverInfo->get().myLocality;
dbInfo.client = ClientDBInfo(); dbInfo.client = ClientDBInfo();
dbInfo.client.isEncryptionEnabled = SERVER_KNOBS->ENABLE_ENCRYPTION; dbInfo.client.isEncryptionEnabled = SERVER_KNOBS->ENABLE_ENCRYPTION;
dbInfo.client.tenantMode = db->config.tenantMode; dbInfo.client.tenantMode = TenantAPI::tenantModeForClusterType(db->clusterType, db->config.tenantMode);
dbInfo.client.clusterId = db->serverInfo->get().client.clusterId; dbInfo.client.clusterId = db->serverInfo->get().client.clusterId;
dbInfo.client.clusterType = db->clusterType;
dbInfo.client.metaclusterName = db->metaclusterName;
TraceEvent("CCWDB", cluster->id) TraceEvent("CCWDB", cluster->id)
.detail("NewMaster", dbInfo.master.id().toString()) .detail("NewMaster", dbInfo.master.id().toString())
@ -1013,7 +1016,9 @@ void clusterRegisterMaster(ClusterControllerData* self, RegisterMasterRequest co
if (db->clientInfo->get().commitProxies != req.commitProxies || if (db->clientInfo->get().commitProxies != req.commitProxies ||
db->clientInfo->get().grvProxies != req.grvProxies || db->clientInfo->get().grvProxies != req.grvProxies ||
db->clientInfo->get().tenantMode != db->config.tenantMode || db->clientInfo->get().clusterId != req.clusterId || db->clientInfo->get().tenantMode != db->config.tenantMode || db->clientInfo->get().clusterId != req.clusterId ||
db->clientInfo->get().isEncryptionEnabled != SERVER_KNOBS->ENABLE_ENCRYPTION) { db->clientInfo->get().isEncryptionEnabled != SERVER_KNOBS->ENABLE_ENCRYPTION ||
db->clientInfo->get().clusterType != db->clusterType ||
db->clientInfo->get().metaclusterName != db->metaclusterName) {
TraceEvent("PublishNewClientInfo", self->id) TraceEvent("PublishNewClientInfo", self->id)
.detail("Master", dbInfo.master.id()) .detail("Master", dbInfo.master.id())
.detail("GrvProxies", db->clientInfo->get().grvProxies) .detail("GrvProxies", db->clientInfo->get().grvProxies)
@ -1023,8 +1028,12 @@ void clusterRegisterMaster(ClusterControllerData* self, RegisterMasterRequest co
.detail("TenantMode", db->clientInfo->get().tenantMode.toString()) .detail("TenantMode", db->clientInfo->get().tenantMode.toString())
.detail("ReqTenantMode", db->config.tenantMode.toString()) .detail("ReqTenantMode", db->config.tenantMode.toString())
.detail("ClusterId", db->clientInfo->get().clusterId) .detail("ClusterId", db->clientInfo->get().clusterId)
.detail("ReqClusterId", req.clusterId)
.detail("EncryptionEnabled", SERVER_KNOBS->ENABLE_ENCRYPTION) .detail("EncryptionEnabled", SERVER_KNOBS->ENABLE_ENCRYPTION)
.detail("ReqClusterId", req.clusterId); .detail("ClusterType", db->clientInfo->get().clusterType)
.detail("ReqClusterType", db->clusterType)
.detail("MetaclusterName", db->clientInfo->get().metaclusterName)
.detail("ReqMetaclusterName", db->metaclusterName);
isChanged = true; isChanged = true;
// TODO why construct a new one and not just copy the old one and change proxies + id? // TODO why construct a new one and not just copy the old one and change proxies + id?
ClientDBInfo clientInfo; ClientDBInfo clientInfo;
@ -1032,8 +1041,10 @@ void clusterRegisterMaster(ClusterControllerData* self, RegisterMasterRequest co
clientInfo.isEncryptionEnabled = SERVER_KNOBS->ENABLE_ENCRYPTION; clientInfo.isEncryptionEnabled = SERVER_KNOBS->ENABLE_ENCRYPTION;
clientInfo.commitProxies = req.commitProxies; clientInfo.commitProxies = req.commitProxies;
clientInfo.grvProxies = req.grvProxies; clientInfo.grvProxies = req.grvProxies;
clientInfo.tenantMode = db->config.tenantMode; clientInfo.tenantMode = TenantAPI::tenantModeForClusterType(db->clusterType, db->config.tenantMode);
clientInfo.clusterId = req.clusterId; clientInfo.clusterId = req.clusterId;
clientInfo.clusterType = db->clusterType;
clientInfo.metaclusterName = db->metaclusterName;
db->clientInfo->set(clientInfo); db->clientInfo->set(clientInfo);
dbInfo.client = db->clientInfo->get(); dbInfo.client = db->clientInfo->get();
} }

View File

@ -18,6 +18,7 @@
* limitations under the License. * limitations under the License.
*/ */
#include "fdbclient/Metacluster.h"
#include "fdbrpc/sim_validation.h" #include "fdbrpc/sim_validation.h"
#include "fdbserver/ApplyMetadataMutation.h" #include "fdbserver/ApplyMetadataMutation.h"
#include "fdbserver/BackupProgress.actor.h" #include "fdbserver/BackupProgress.actor.h"
@ -1159,6 +1160,17 @@ ACTOR Future<Void> readTransactionSystemState(Reference<ClusterRecoveryData> sel
self->allTags.push_back(decodeServerTagValue(kv.value)); self->allTags.push_back(decodeServerTagValue(kv.value));
} }
Optional<Value> metaclusterRegistrationVal =
wait(self->txnStateStore->readValue(MetaclusterMetadata::metaclusterRegistration.key));
Optional<MetaclusterRegistrationEntry> metaclusterRegistration =
MetaclusterRegistrationEntry::decode(metaclusterRegistrationVal);
if (metaclusterRegistration.present()) {
self->controllerData->db.metaclusterName = metaclusterRegistration.get().metaclusterName;
self->controllerData->db.clusterType = metaclusterRegistration.get().clusterType;
} else {
self->controllerData->db.clusterType = ClusterType::STANDALONE;
}
uniquify(self->allTags); uniquify(self->allTags);
// auto kvs = self->txnStateStore->readRange( systemKeys ); // auto kvs = self->txnStateStore->readRange( systemKeys );

View File

@ -349,7 +349,7 @@ class DDTxnProcessorImpl {
// a dummy shard at the end with no keys or servers makes life easier for trackInitialShards() // a dummy shard at the end with no keys or servers makes life easier for trackInitialShards()
result->shards.push_back(DDShardInfo(allKeys.end)); result->shards.push_back(DDShardInfo(allKeys.end));
if (CLIENT_KNOBS->SHARD_ENCODE_LOCATION_METADATA && numDataMoves > 0) { if (SERVER_KNOBS->SHARD_ENCODE_LOCATION_METADATA && numDataMoves > 0) {
for (int shard = 0; shard < result->shards.size() - 1; ++shard) { for (int shard = 0; shard < result->shards.size() - 1; ++shard) {
const DDShardInfo& iShard = result->shards[shard]; const DDShardInfo& iShard = result->shards[shard];
KeyRangeRef keys = KeyRangeRef(iShard.key, result->shards[shard + 1].key); KeyRangeRef keys = KeyRangeRef(iShard.key, result->shards[shard + 1].key);

View File

@ -491,7 +491,7 @@ public:
for (; it != self->initData->dataMoveMap.ranges().end(); ++it) { for (; it != self->initData->dataMoveMap.ranges().end(); ++it) {
const DataMoveMetaData& meta = it.value()->meta; const DataMoveMetaData& meta = it.value()->meta;
if (it.value()->isCancelled() || (it.value()->valid && !CLIENT_KNOBS->SHARD_ENCODE_LOCATION_METADATA)) { if (it.value()->isCancelled() || (it.value()->valid && !SERVER_KNOBS->SHARD_ENCODE_LOCATION_METADATA)) {
RelocateShard rs(meta.range, DataMovementReason::RECOVER_MOVE, RelocateReason::OTHER); RelocateShard rs(meta.range, DataMovementReason::RECOVER_MOVE, RelocateReason::OTHER);
rs.dataMoveId = meta.id; rs.dataMoveId = meta.id;
rs.cancelled = true; rs.cancelled = true;
@ -812,24 +812,6 @@ ACTOR Future<ErrorOr<Void>> trySendSnapReq(RequestStream<WorkerSnapRequest> stre
return ErrorOr<Void>(Void()); return ErrorOr<Void>(Void());
} }
ACTOR static Future<Void> waitForMost(std::vector<Future<ErrorOr<Void>>> futures,
int faultTolerance,
Error e,
double waitMultiplierForSlowFutures = 1.0) {
state std::vector<Future<bool>> successFutures;
state double startTime = now();
successFutures.reserve(futures.size());
for (const auto& future : futures) {
successFutures.push_back(fmap([](auto const& result) { return result.present(); }, future));
}
bool success = wait(quorumEqualsTrue(successFutures, successFutures.size() - faultTolerance));
if (!success) {
throw e;
}
wait(delay((now() - startTime) * waitMultiplierForSlowFutures) || waitForAll(successFutures));
return Void();
}
ACTOR Future<std::map<NetworkAddress, std::pair<WorkerInterface, std::string>>> getStatefulWorkers( ACTOR Future<std::map<NetworkAddress, std::pair<WorkerInterface, std::string>>> getStatefulWorkers(
Database cx, Database cx,
Reference<AsyncVar<ServerDBInfo> const> dbInfo, Reference<AsyncVar<ServerDBInfo> const> dbInfo,
@ -1389,14 +1371,6 @@ ACTOR Future<Void> dataDistributor(DataDistributorInterface di, Reference<AsyncV
namespace data_distribution_test { namespace data_distribution_test {
static Future<ErrorOr<Void>> goodTestFuture(double duration) {
return tag(delay(duration), ErrorOr<Void>(Void()));
}
static Future<ErrorOr<Void>> badTestFuture(double duration, Error e) {
return tag(delay(duration), ErrorOr<Void>(e));
}
inline DDShardInfo doubleToNoLocationShardInfo(double d, bool hasDest) { inline DDShardInfo doubleToNoLocationShardInfo(double d, bool hasDest) {
DDShardInfo res(doubleToTestKey(d), anonymousShardId, anonymousShardId); DDShardInfo res(doubleToTestKey(d), anonymousShardId, anonymousShardId);
res.primarySrc.emplace_back((uint64_t)d, 0); res.primarySrc.emplace_back((uint64_t)d, 0);
@ -1409,50 +1383,7 @@ inline DDShardInfo doubleToNoLocationShardInfo(double d, bool hasDest) {
} // namespace data_distribution_test } // namespace data_distribution_test
TEST_CASE("/DataDistribution/WaitForMost") { TEST_CASE("/DataDistribution/StorageWiggler/Order") {
state std::vector<Future<ErrorOr<Void>>> futures;
{
futures = { data_distribution_test::goodTestFuture(1),
data_distribution_test::goodTestFuture(2),
data_distribution_test::goodTestFuture(3) };
wait(waitForMost(futures, 1, operation_failed(), 0.0)); // Don't wait for slowest future
ASSERT(!futures[2].isReady());
}
{
futures = { data_distribution_test::goodTestFuture(1),
data_distribution_test::goodTestFuture(2),
data_distribution_test::goodTestFuture(3) };
wait(waitForMost(futures, 0, operation_failed(), 0.0)); // Wait for all futures
ASSERT(futures[2].isReady());
}
{
futures = { data_distribution_test::goodTestFuture(1),
data_distribution_test::goodTestFuture(2),
data_distribution_test::goodTestFuture(3) };
wait(waitForMost(futures, 1, operation_failed(), 1.0)); // Wait for slowest future
ASSERT(futures[2].isReady());
}
{
futures = { data_distribution_test::goodTestFuture(1),
data_distribution_test::goodTestFuture(2),
data_distribution_test::badTestFuture(1, success()) };
wait(waitForMost(futures, 1, operation_failed(), 1.0)); // Error ignored
}
{
futures = { data_distribution_test::goodTestFuture(1),
data_distribution_test::goodTestFuture(2),
data_distribution_test::badTestFuture(1, success()) };
try {
wait(waitForMost(futures, 0, operation_failed(), 1.0));
ASSERT(false);
} catch (Error& e) {
ASSERT_EQ(e.code(), error_code_operation_failed);
}
}
return Void();
}
TEST_CASE("/DataDistributor/StorageWiggler/Order") {
StorageWiggler wiggler(nullptr); StorageWiggler wiggler(nullptr);
wiggler.addServer(UID(1, 0), StorageMetadataType(1, KeyValueStoreType::SSD_BTREE_V2)); wiggler.addServer(UID(1, 0), StorageMetadataType(1, KeyValueStoreType::SSD_BTREE_V2));
wiggler.addServer(UID(2, 0), StorageMetadataType(2, KeyValueStoreType::MEMORY, true)); wiggler.addServer(UID(2, 0), StorageMetadataType(2, KeyValueStoreType::MEMORY, true));
@ -1469,7 +1400,7 @@ TEST_CASE("/DataDistributor/StorageWiggler/Order") {
return Void(); return Void();
} }
TEST_CASE("/DataDistributor/Initialization/ResumeFromShard") { TEST_CASE("/DataDistribution/Initialization/ResumeFromShard") {
state Reference<AsyncVar<ServerDBInfo> const> dbInfo; state Reference<AsyncVar<ServerDBInfo> const> dbInfo;
state Reference<DataDistributor> self(new DataDistributor(dbInfo, UID())); state Reference<DataDistributor> self(new DataDistributor(dbInfo, UID()));

View File

@ -1099,7 +1099,7 @@ struct DDQueueData {
} }
Future<Void> fCleanup = Future<Void> fCleanup =
CLIENT_KNOBS->SHARD_ENCODE_LOCATION_METADATA ? cancelDataMove(this, rd.keys, ddEnabledState) : Void(); SERVER_KNOBS->SHARD_ENCODE_LOCATION_METADATA ? cancelDataMove(this, rd.keys, ddEnabledState) : Void();
// If there is a job in flight that wants data relocation which we are about to cancel/modify, // If there is a job in flight that wants data relocation which we are about to cancel/modify,
// make sure that we keep the relocation intent for the job that we launch // make sure that we keep the relocation intent for the job that we launch
@ -1121,13 +1121,13 @@ struct DDQueueData {
rrs.keys = ranges[r]; rrs.keys = ranges[r];
if (rd.keys == ranges[r] && rd.isRestore()) { if (rd.keys == ranges[r] && rd.isRestore()) {
ASSERT(rd.dataMove != nullptr); ASSERT(rd.dataMove != nullptr);
ASSERT(CLIENT_KNOBS->SHARD_ENCODE_LOCATION_METADATA); ASSERT(SERVER_KNOBS->SHARD_ENCODE_LOCATION_METADATA);
rrs.dataMoveId = rd.dataMove->meta.id; rrs.dataMoveId = rd.dataMove->meta.id;
} else { } else {
ASSERT_WE_THINK(!rd.isRestore()); // Restored data move should not overlap. ASSERT_WE_THINK(!rd.isRestore()); // Restored data move should not overlap.
// TODO(psm): The shard id is determined by DD. // TODO(psm): The shard id is determined by DD.
rrs.dataMove.reset(); rrs.dataMove.reset();
if (CLIENT_KNOBS->SHARD_ENCODE_LOCATION_METADATA) { if (SERVER_KNOBS->SHARD_ENCODE_LOCATION_METADATA) {
rrs.dataMoveId = deterministicRandom()->randomUniqueID(); rrs.dataMoveId = deterministicRandom()->randomUniqueID();
} else { } else {
rrs.dataMoveId = anonymousShardId; rrs.dataMoveId = anonymousShardId;
@ -1290,7 +1290,7 @@ ACTOR Future<Void> dataDistributionRelocator(DDQueueData* self,
self->suppressIntervals = 0; self->suppressIntervals = 0;
} }
if (CLIENT_KNOBS->SHARD_ENCODE_LOCATION_METADATA) { if (SERVER_KNOBS->SHARD_ENCODE_LOCATION_METADATA) {
auto inFlightRange = self->inFlight.rangeContaining(rd.keys.begin); auto inFlightRange = self->inFlight.rangeContaining(rd.keys.begin);
ASSERT(inFlightRange.range() == rd.keys); ASSERT(inFlightRange.range() == rd.keys);
ASSERT(inFlightRange.value().randomId == rd.randomId); ASSERT(inFlightRange.value().randomId == rd.randomId);
@ -1332,7 +1332,7 @@ ACTOR Future<Void> dataDistributionRelocator(DDQueueData* self,
bestTeams.clear(); bestTeams.clear();
// Get team from teamCollections in different DCs and find the best one // Get team from teamCollections in different DCs and find the best one
while (tciIndex < self->teamCollections.size()) { while (tciIndex < self->teamCollections.size()) {
if (CLIENT_KNOBS->SHARD_ENCODE_LOCATION_METADATA && rd.isRestore()) { if (SERVER_KNOBS->SHARD_ENCODE_LOCATION_METADATA && rd.isRestore()) {
auto req = GetTeamRequest(tciIndex == 0 ? rd.dataMove->primaryDest : rd.dataMove->remoteDest); auto req = GetTeamRequest(tciIndex == 0 ? rd.dataMove->primaryDest : rd.dataMove->remoteDest);
Future<std::pair<Optional<Reference<IDataDistributionTeam>>, bool>> fbestTeam = Future<std::pair<Optional<Reference<IDataDistributionTeam>>, bool>> fbestTeam =
brokenPromiseToNever(self->teamCollections[tciIndex].getTeam.getReply(req)); brokenPromiseToNever(self->teamCollections[tciIndex].getTeam.getReply(req));
@ -1579,7 +1579,7 @@ ACTOR Future<Void> dataDistributionRelocator(DDQueueData* self,
CancelConflictingDataMoves::False); CancelConflictingDataMoves::False);
} else { } else {
self->fetchKeysComplete.insert(rd); self->fetchKeysComplete.insert(rd);
if (CLIENT_KNOBS->SHARD_ENCODE_LOCATION_METADATA) { if (SERVER_KNOBS->SHARD_ENCODE_LOCATION_METADATA) {
auto ranges = self->dataMoves.getAffectedRangesAfterInsertion(rd.keys); auto ranges = self->dataMoves.getAffectedRangesAfterInsertion(rd.keys);
if (ranges.size() == 1 && static_cast<KeyRange>(ranges[0]) == rd.keys && if (ranges.size() == 1 && static_cast<KeyRange>(ranges[0]) == rd.keys &&
ranges[0].value.id == rd.dataMoveId && !ranges[0].value.cancel.isValid()) { ranges[0].value.id == rd.dataMoveId && !ranges[0].value.cancel.isValid()) {

File diff suppressed because it is too large Load Diff

View File

@ -301,11 +301,9 @@ rocksdb::Options getOptions() {
options.IncreaseParallelism(SERVER_KNOBS->ROCKSDB_BACKGROUND_PARALLELISM); options.IncreaseParallelism(SERVER_KNOBS->ROCKSDB_BACKGROUND_PARALLELISM);
} }
// TODO: enable rocksdb metrics. options.statistics = rocksdb::CreateDBStatistics();
options.statistics->set_stats_level(rocksdb::kExceptHistogramOrTimers);
options.db_log_dir = SERVER_KNOBS->LOG_DIRECTORY; options.db_log_dir = SERVER_KNOBS->LOG_DIRECTORY;
if (g_network->isSimulated()) {
options.OptimizeForSmallDb();
}
return options; return options;
} }
@ -603,11 +601,10 @@ public:
return Void(); return Void();
} }
rocksdb::Status init() { rocksdb::Status init(rocksdb::Options options) {
// Open instance. // Open instance.
TraceEvent(SevInfo, "ShardedRocksShardManagerInitBegin", this->logId).detail("DataPath", path); TraceEvent(SevInfo, "ShardedRocksShardManagerInitBegin", this->logId).detail("DataPath", path);
std::vector<std::string> columnFamilies; std::vector<std::string> columnFamilies;
rocksdb::Options options = getOptions();
rocksdb::Status status = rocksdb::DB::ListColumnFamilies(options, path, &columnFamilies); rocksdb::Status status = rocksdb::DB::ListColumnFamilies(options, path, &columnFamilies);
rocksdb::ColumnFamilyOptions cfOptions = getCFOptions(); rocksdb::ColumnFamilyOptions cfOptions = getCFOptions();
@ -1044,9 +1041,7 @@ private:
class RocksDBMetrics { class RocksDBMetrics {
public: public:
RocksDBMetrics(); RocksDBMetrics(UID debugID, std::shared_ptr<rocksdb::Statistics> stats);
// Statistics
std::shared_ptr<rocksdb::Statistics> getStatsObjForRocksDB();
void logStats(rocksdb::DB* db); void logStats(rocksdb::DB* db);
// PerfContext // PerfContext
void resetPerfContext(); void resetPerfContext();
@ -1072,9 +1067,10 @@ public:
Reference<Histogram> getWriteHistogram(); Reference<Histogram> getWriteHistogram();
Reference<Histogram> getDeleteCompactRangeHistogram(); Reference<Histogram> getDeleteCompactRangeHistogram();
// Stat for Memory Usage // Stat for Memory Usage
void logMemUsagePerShard(std::string shardName, rocksdb::DB* db); void logMemUsage(rocksdb::DB* db);
private: private:
const UID debugID;
// Global Statistic Input to RocksDB DB instance // Global Statistic Input to RocksDB DB instance
std::shared_ptr<rocksdb::Statistics> stats; std::shared_ptr<rocksdb::Statistics> stats;
// Statistic Output from RocksDB // Statistic Output from RocksDB
@ -1161,9 +1157,8 @@ Reference<Histogram> RocksDBMetrics::getDeleteCompactRangeHistogram() {
return deleteCompactRangeHistogram; return deleteCompactRangeHistogram;
} }
RocksDBMetrics::RocksDBMetrics() { RocksDBMetrics::RocksDBMetrics(UID debugID, std::shared_ptr<rocksdb::Statistics> stats)
stats = rocksdb::CreateDBStatistics(); : debugID(debugID), stats(stats) {
stats->set_stats_level(rocksdb::kExceptHistogramOrTimers);
tickerStats = { tickerStats = {
{ "StallMicros", rocksdb::STALL_MICROS, 0 }, { "StallMicros", rocksdb::STALL_MICROS, 0 },
{ "BytesRead", rocksdb::BYTES_READ, 0 }, { "BytesRead", rocksdb::BYTES_READ, 0 },
@ -1346,16 +1341,8 @@ RocksDBMetrics::RocksDBMetrics() {
ROCKSDBSTORAGE_HISTOGRAM_GROUP, ROCKSDB_DELETE_COMPACTRANGE_HISTOGRAM, Histogram::Unit::microseconds); ROCKSDBSTORAGE_HISTOGRAM_GROUP, ROCKSDB_DELETE_COMPACTRANGE_HISTOGRAM, Histogram::Unit::microseconds);
} }
std::shared_ptr<rocksdb::Statistics> RocksDBMetrics::getStatsObjForRocksDB() {
// Zhe: reserved for statistic of RocksDBMetrics per shard
// ASSERT(shard != nullptr && shard->stats != nullptr);
// return shard->stats;
ASSERT(stats != nullptr);
return stats;
}
void RocksDBMetrics::logStats(rocksdb::DB* db) { void RocksDBMetrics::logStats(rocksdb::DB* db) {
TraceEvent e("ShardedRocksDBMetrics"); TraceEvent e(SevInfo, "ShardedRocksDBMetrics", debugID);
uint64_t stat; uint64_t stat;
for (auto& [name, ticker, cumulation] : tickerStats) { for (auto& [name, ticker, cumulation] : tickerStats) {
stat = stats->getTickerCount(ticker); stat = stats->getTickerCount(ticker);
@ -1367,18 +1354,10 @@ void RocksDBMetrics::logStats(rocksdb::DB* db) {
ASSERT(db->GetIntProperty(property, &stat)); ASSERT(db->GetIntProperty(property, &stat));
e.detail(name, stat); e.detail(name, stat);
} }
/*
stat = readIterPool->numReadIteratorsCreated();
e.detail("NumReadIteratorsCreated", stat - readIteratorPoolStats["NumReadIteratorsCreated"]);
readIteratorPoolStats["NumReadIteratorsCreated"] = stat;
stat = readIterPool->numTimesReadIteratorsReused();
e.detail("NumTimesReadIteratorsReused", stat - readIteratorPoolStats["NumTimesReadIteratorsReused"]);
readIteratorPoolStats["NumTimesReadIteratorsReused"] = stat;
*/
} }
void RocksDBMetrics::logMemUsagePerShard(std::string shardName, rocksdb::DB* db) { void RocksDBMetrics::logMemUsage(rocksdb::DB* db) {
TraceEvent e("ShardedRocksDBShardMemMetrics"); TraceEvent e(SevInfo, "ShardedRocksDBMemMetrics", debugID);
uint64_t stat; uint64_t stat;
ASSERT(db != nullptr); ASSERT(db != nullptr);
ASSERT(db->GetIntProperty(rocksdb::DB::Properties::kBlockCacheUsage, &stat)); ASSERT(db->GetIntProperty(rocksdb::DB::Properties::kBlockCacheUsage, &stat));
@ -1389,7 +1368,6 @@ void RocksDBMetrics::logMemUsagePerShard(std::string shardName, rocksdb::DB* db)
e.detail("AllMemtablesBytes", stat); e.detail("AllMemtablesBytes", stat);
ASSERT(db->GetIntProperty(rocksdb::DB::Properties::kBlockCachePinnedUsage, &stat)); ASSERT(db->GetIntProperty(rocksdb::DB::Properties::kBlockCachePinnedUsage, &stat));
e.detail("BlockCachePinnedUsage", stat); e.detail("BlockCachePinnedUsage", stat);
e.detail("Name", shardName);
} }
void RocksDBMetrics::resetPerfContext() { void RocksDBMetrics::resetPerfContext() {
@ -1404,7 +1382,7 @@ void RocksDBMetrics::setPerfContext(int index) {
} }
void RocksDBMetrics::logPerfContext(bool ignoreZeroMetric) { void RocksDBMetrics::logPerfContext(bool ignoreZeroMetric) {
TraceEvent e("ShardedRocksDBPerfContextMetrics"); TraceEvent e(SevInfo, "ShardedRocksDBPerfContextMetrics", debugID);
e.setMaxEventLength(20000); e.setMaxEventLength(20000);
for (auto& [name, metric, vals] : perfContextMetrics) { for (auto& [name, metric, vals] : perfContextMetrics) {
uint64_t s = 0; uint64_t s = 0;
@ -1566,17 +1544,30 @@ uint64_t RocksDBMetrics::getRocksdbPerfcontextMetric(int metric) {
return 0; return 0;
} }
ACTOR Future<Void> rocksDBAggregatedMetricsLogger(std::shared_ptr<RocksDBMetrics> rocksDBMetrics, rocksdb::DB* db) { ACTOR Future<Void> rocksDBAggregatedMetricsLogger(std::shared_ptr<ShardedRocksDBState> rState,
loop { Future<Void> openFuture,
wait(delay(SERVER_KNOBS->ROCKSDB_METRICS_DELAY)); std::shared_ptr<RocksDBMetrics> rocksDBMetrics,
/* ShardManager* shardManager) {
if (SERVER_KNOBS->ROCKSDB_ENABLE_STATISTIC) { try {
rocksDBMetrics->logStats(db); wait(openFuture);
state rocksdb::DB* db = shardManager->getDb();
loop {
wait(delay(SERVER_KNOBS->ROCKSDB_METRICS_DELAY));
if (rState->closing) {
break;
}
rocksDBMetrics->logStats(db);
rocksDBMetrics->logMemUsage(db);
if (SERVER_KNOBS->ROCKSDB_PERFCONTEXT_SAMPLE_RATE != 0) {
rocksDBMetrics->logPerfContext(true);
}
}
} catch (Error& e) {
if (e.code() != error_code_actor_cancelled) {
TraceEvent(SevError, "ShardedRocksDBMetricsError").errorUnsuppressed(e);
} }
if (SERVER_KNOBS->ROCKSDB_PERFCONTEXT_SAMPLE_RATE != 0) {
rocksDBMetrics->logPerfContext(true);
}*/
} }
return Void();
} }
struct ShardedRocksDBKeyValueStore : IKeyValueStore { struct ShardedRocksDBKeyValueStore : IKeyValueStore {
@ -1642,6 +1633,7 @@ struct ShardedRocksDBKeyValueStore : IKeyValueStore {
struct OpenAction : TypedAction<Writer, OpenAction> { struct OpenAction : TypedAction<Writer, OpenAction> {
ShardManager* shardManager; ShardManager* shardManager;
rocksdb::Options dbOptions;
ThreadReturnPromise<Void> done; ThreadReturnPromise<Void> done;
Optional<Future<Void>>& metrics; Optional<Future<Void>>& metrics;
const FlowLock* readLock; const FlowLock* readLock;
@ -1649,18 +1641,20 @@ struct ShardedRocksDBKeyValueStore : IKeyValueStore {
std::shared_ptr<RocksDBErrorListener> errorListener; std::shared_ptr<RocksDBErrorListener> errorListener;
OpenAction(ShardManager* shardManager, OpenAction(ShardManager* shardManager,
rocksdb::Options dbOptions,
Optional<Future<Void>>& metrics, Optional<Future<Void>>& metrics,
const FlowLock* readLock, const FlowLock* readLock,
const FlowLock* fetchLock, const FlowLock* fetchLock,
std::shared_ptr<RocksDBErrorListener> errorListener) std::shared_ptr<RocksDBErrorListener> errorListener)
: shardManager(shardManager), metrics(metrics), readLock(readLock), fetchLock(fetchLock), : shardManager(shardManager), dbOptions(dbOptions), metrics(metrics), readLock(readLock),
errorListener(errorListener) {} fetchLock(fetchLock), errorListener(errorListener) {}
double getTimeEstimate() const override { return SERVER_KNOBS->COMMIT_TIME_ESTIMATE; } double getTimeEstimate() const override { return SERVER_KNOBS->COMMIT_TIME_ESTIMATE; }
}; };
void action(OpenAction& a) { void action(OpenAction& a) {
auto status = a.shardManager->init(); auto status = a.shardManager->init(a.dbOptions);
if (!status.ok()) { if (!status.ok()) {
logRocksDBError(status, "Open"); logRocksDBError(status, "Open");
a.done.sendError(statusToError(status)); a.done.sendError(statusToError(status));
@ -2117,10 +2111,10 @@ struct ShardedRocksDBKeyValueStore : IKeyValueStore {
std::reverse(a.shardRanges.begin(), a.shardRanges.end()); std::reverse(a.shardRanges.begin(), a.shardRanges.end());
} }
// TODO: consider multi-thread reads. It's possible to read multiple shards in parallel. However, the number // TODO: consider multi-thread reads. It's possible to read multiple shards in parallel. However, the
// of rows to read needs to be calculated based on the previous read result. We may read more than we // number of rows to read needs to be calculated based on the previous read result. We may read more
// expected when parallel read is used when the previous result is not available. It's unlikely to get to // than we expected when parallel read is used when the previous result is not available. It's unlikely
// performance improvement when the actual number of rows to read is very small. // to get to performance improvement when the actual number of rows to read is very small.
int accumulatedBytes = 0; int accumulatedBytes = 0;
int numShards = 0; int numShards = 0;
for (auto& [shard, range] : a.shardRanges) { for (auto& [shard, range] : a.shardRanges) {
@ -2178,17 +2172,19 @@ struct ShardedRocksDBKeyValueStore : IKeyValueStore {
numReadWaiters(SERVER_KNOBS->ROCKSDB_READ_QUEUE_HARD_MAX - SERVER_KNOBS->ROCKSDB_READ_QUEUE_SOFT_MAX), numReadWaiters(SERVER_KNOBS->ROCKSDB_READ_QUEUE_HARD_MAX - SERVER_KNOBS->ROCKSDB_READ_QUEUE_SOFT_MAX),
numFetchWaiters(SERVER_KNOBS->ROCKSDB_FETCH_QUEUE_HARD_MAX - SERVER_KNOBS->ROCKSDB_FETCH_QUEUE_SOFT_MAX), numFetchWaiters(SERVER_KNOBS->ROCKSDB_FETCH_QUEUE_HARD_MAX - SERVER_KNOBS->ROCKSDB_FETCH_QUEUE_SOFT_MAX),
errorListener(std::make_shared<RocksDBErrorListener>()), errorFuture(errorListener->getFuture()), errorListener(std::make_shared<RocksDBErrorListener>()), errorFuture(errorListener->getFuture()),
shardManager(path, id), rocksDBMetrics(new RocksDBMetrics()) { shardManager(path, id), dbOptions(getOptions()),
// In simluation, run the reader/writer threads as Coro threads (i.e. in the network thread. The storage engine rocksDBMetrics(std::make_shared<RocksDBMetrics>(id, dbOptions.statistics)) {
// is still multi-threaded as background compaction threads are still present. Reads/writes to disk will also // In simluation, run the reader/writer threads as Coro threads (i.e. in the network thread. The storage
// block the network thread in a way that would be unacceptable in production but is a necessary evil here. When // engine is still multi-threaded as background compaction threads are still present. Reads/writes to disk
// performing the reads in background threads in simulation, the event loop thinks there is no work to do and // will also block the network thread in a way that would be unacceptable in production but is a necessary
// advances time faster than 1 sec/sec. By the time the blocking read actually finishes, simulation has advanced // evil here. When performing the reads in background threads in simulation, the event loop thinks there is
// time by more than 5 seconds, so every read fails with a transaction_too_old error. Doing blocking IO on the // no work to do and advances time faster than 1 sec/sec. By the time the blocking read actually finishes,
// main thread solves this issue. There are almost certainly better fixes, but my goal was to get a less // simulation has advanced time by more than 5 seconds, so every read fails with a transaction_too_old
// invasive change merged first and work on a more realistic version if/when we think that would provide // error. Doing blocking IO on the main thread solves this issue. There are almost certainly better fixes,
// substantially more confidence in the correctness. // but my goal was to get a less invasive change merged first and work on a more realistic version if/when
// TODO: Adapt the simulation framework to not advance time quickly when background reads/writes are occurring. // we think that would provide substantially more confidence in the correctness.
// TODO: Adapt the simulation framework to not advance time quickly when background reads/writes are
// occurring.
if (g_network->isSimulated()) { if (g_network->isSimulated()) {
writeThread = CoroThreadPool::createThreadPool(); writeThread = CoroThreadPool::createThreadPool();
readThreads = CoroThreadPool::createThreadPool(); readThreads = CoroThreadPool::createThreadPool();
@ -2236,14 +2232,15 @@ struct ShardedRocksDBKeyValueStore : IKeyValueStore {
Future<Void> init() override { Future<Void> init() override {
if (openFuture.isValid()) { if (openFuture.isValid()) {
return openFuture; return openFuture;
// Restore durable state if KVS is open. KVS will be re-initialized during rollback. To avoid the cost of // Restore durable state if KVS is open. KVS will be re-initialized during rollback. To avoid the cost
// opening and closing multiple rocksdb instances, we reconcile the shard map using persist shard mapping // of opening and closing multiple rocksdb instances, we reconcile the shard map using persist shard
// data. // mapping data.
} else { } else {
auto a = std::make_unique<Writer::OpenAction>( auto a = std::make_unique<Writer::OpenAction>(
&shardManager, metrics, &readSemaphore, &fetchSemaphore, errorListener); &shardManager, dbOptions, metrics, &readSemaphore, &fetchSemaphore, errorListener);
openFuture = a->done.getFuture(); openFuture = a->done.getFuture();
this->metrics = ShardManager::shardMetricsLogger(this->rState, openFuture, &shardManager); this->metrics = ShardManager::shardMetricsLogger(this->rState, openFuture, &shardManager) &&
rocksDBAggregatedMetricsLogger(this->rState, openFuture, rocksDBMetrics, &shardManager);
this->refreshHolder = refreshReadIteratorPools(this->rState, openFuture, shardManager.getAllShards()); this->refreshHolder = refreshReadIteratorPools(this->rState, openFuture, shardManager.getAllShards());
writeThread->post(a.release()); writeThread->post(a.release());
return openFuture; return openFuture;
@ -2436,6 +2433,7 @@ struct ShardedRocksDBKeyValueStore : IKeyValueStore {
std::shared_ptr<ShardedRocksDBState> rState; std::shared_ptr<ShardedRocksDBState> rState;
ShardManager shardManager; ShardManager shardManager;
rocksdb::Options dbOptions;
std::shared_ptr<RocksDBMetrics> rocksDBMetrics; std::shared_ptr<RocksDBMetrics> rocksDBMetrics;
std::string path; std::string path;
UID id; UID id;

View File

@ -296,7 +296,7 @@ ACTOR Future<Void> cleanUpSingleShardDataMove(Database occ,
FlowLock* cleanUpDataMoveParallelismLock, FlowLock* cleanUpDataMoveParallelismLock,
UID dataMoveId, UID dataMoveId,
const DDEnabledState* ddEnabledState) { const DDEnabledState* ddEnabledState) {
ASSERT(CLIENT_KNOBS->SHARD_ENCODE_LOCATION_METADATA); ASSERT(SERVER_KNOBS->SHARD_ENCODE_LOCATION_METADATA);
TraceEvent(SevInfo, "CleanUpSingleShardDataMoveBegin", dataMoveId).detail("Range", keys); TraceEvent(SevInfo, "CleanUpSingleShardDataMoveBegin", dataMoveId).detail("Range", keys);
loop { loop {
@ -1226,7 +1226,7 @@ ACTOR static Future<Void> startMoveShards(Database occ,
UID relocationIntervalId, UID relocationIntervalId,
const DDEnabledState* ddEnabledState, const DDEnabledState* ddEnabledState,
CancelConflictingDataMoves cancelConflictingDataMoves) { CancelConflictingDataMoves cancelConflictingDataMoves) {
ASSERT(CLIENT_KNOBS->SHARD_ENCODE_LOCATION_METADATA); ASSERT(SERVER_KNOBS->SHARD_ENCODE_LOCATION_METADATA);
state Future<Void> warningLogger = logWarningAfter("StartMoveShardsTooLong", 600, servers); state Future<Void> warningLogger = logWarningAfter("StartMoveShardsTooLong", 600, servers);
wait(startMoveKeysLock->take(TaskPriority::DataDistributionLaunch)); wait(startMoveKeysLock->take(TaskPriority::DataDistributionLaunch));
@ -1561,7 +1561,7 @@ ACTOR static Future<Void> finishMoveShards(Database occ,
UID relocationIntervalId, UID relocationIntervalId,
std::map<UID, StorageServerInterface> tssMapping, std::map<UID, StorageServerInterface> tssMapping,
const DDEnabledState* ddEnabledState) { const DDEnabledState* ddEnabledState) {
ASSERT(CLIENT_KNOBS->SHARD_ENCODE_LOCATION_METADATA); ASSERT(SERVER_KNOBS->SHARD_ENCODE_LOCATION_METADATA);
state KeyRange keys = targetKeys; state KeyRange keys = targetKeys;
state Future<Void> warningLogger = logWarningAfter("FinishMoveShardsTooLong", 600, destinationTeam); state Future<Void> warningLogger = logWarningAfter("FinishMoveShardsTooLong", 600, destinationTeam);
state int retries = 0; state int retries = 0;
@ -2225,7 +2225,7 @@ ACTOR Future<Void> removeKeysFromFailedServer(Database cx,
const UID shardId = newShardId(deterministicRandom()->randomUInt64(), AssignEmptyRange::True); const UID shardId = newShardId(deterministicRandom()->randomUInt64(), AssignEmptyRange::True);
// Assign the shard to teamForDroppedRange in keyServer space. // Assign the shard to teamForDroppedRange in keyServer space.
if (CLIENT_KNOBS->SHARD_ENCODE_LOCATION_METADATA) { if (SERVER_KNOBS->SHARD_ENCODE_LOCATION_METADATA) {
tr.set(keyServersKey(it.key), keyServersValue(teamForDroppedRange, {}, shardId, UID())); tr.set(keyServersKey(it.key), keyServersValue(teamForDroppedRange, {}, shardId, UID()));
} else { } else {
tr.set(keyServersKey(it.key), keyServersValue(UIDtoTagMap, teamForDroppedRange)); tr.set(keyServersKey(it.key), keyServersValue(UIDtoTagMap, teamForDroppedRange));
@ -2242,7 +2242,7 @@ ACTOR Future<Void> removeKeysFromFailedServer(Database cx,
// Assign the shard to the new team as an empty range. // Assign the shard to the new team as an empty range.
// Note, there could be data loss. // Note, there could be data loss.
for (const UID& id : teamForDroppedRange) { for (const UID& id : teamForDroppedRange) {
if (CLIENT_KNOBS->SHARD_ENCODE_LOCATION_METADATA) { if (SERVER_KNOBS->SHARD_ENCODE_LOCATION_METADATA) {
actors.push_back(krmSetRangeCoalescing( actors.push_back(krmSetRangeCoalescing(
&tr, serverKeysPrefixFor(id), range, allKeys, serverKeysValue(shardId))); &tr, serverKeysPrefixFor(id), range, allKeys, serverKeysValue(shardId)));
} else { } else {
@ -2462,7 +2462,7 @@ ACTOR Future<Void> moveKeys(Database cx,
state std::map<UID, StorageServerInterface> tssMapping; state std::map<UID, StorageServerInterface> tssMapping;
if (CLIENT_KNOBS->SHARD_ENCODE_LOCATION_METADATA) { if (SERVER_KNOBS->SHARD_ENCODE_LOCATION_METADATA) {
wait(startMoveShards(cx, wait(startMoveShards(cx,
dataMoveId, dataMoveId,
keys, keys,
@ -2487,7 +2487,7 @@ ACTOR Future<Void> moveKeys(Database cx,
state Future<Void> completionSignaller = state Future<Void> completionSignaller =
checkFetchingState(cx, healthyDestinations, keys, dataMovementComplete, relocationIntervalId, tssMapping); checkFetchingState(cx, healthyDestinations, keys, dataMovementComplete, relocationIntervalId, tssMapping);
if (CLIENT_KNOBS->SHARD_ENCODE_LOCATION_METADATA) { if (SERVER_KNOBS->SHARD_ENCODE_LOCATION_METADATA) {
wait(finishMoveShards(cx, wait(finishMoveShards(cx,
dataMoveId, dataMoveId,
keys, keys,
@ -2570,7 +2570,7 @@ void seedShardServers(Arena& arena, CommitTransactionRef& tr, std::vector<Storag
// We have to set this range in two blocks, because the master tracking of "keyServersLocations" depends on a change // We have to set this range in two blocks, because the master tracking of "keyServersLocations" depends on a change
// to a specific // to a specific
// key (keyServersKeyServersKey) // key (keyServersKeyServersKey)
if (CLIENT_KNOBS->SHARD_ENCODE_LOCATION_METADATA) { if (SERVER_KNOBS->SHARD_ENCODE_LOCATION_METADATA) {
const UID teamId = deterministicRandom()->randomUniqueID(); const UID teamId = deterministicRandom()->randomUniqueID();
ksValue = keyServersValue(serverSrcUID, /*dest=*/std::vector<UID>(), teamId, UID()); ksValue = keyServersValue(serverSrcUID, /*dest=*/std::vector<UID>(), teamId, UID());
krmSetPreviouslyEmptyRange(tr, arena, keyServersPrefix, KeyRangeRef(KeyRef(), allKeys.end), ksValue, Value()); krmSetPreviouslyEmptyRange(tr, arena, keyServersPrefix, KeyRangeRef(KeyRef(), allKeys.end), ksValue, Value());

View File

@ -22,6 +22,7 @@
#include "fdbserver/Ratekeeper.h" #include "fdbserver/Ratekeeper.h"
#include "fdbserver/TagThrottler.h" #include "fdbserver/TagThrottler.h"
#include "fdbserver/WaitFailure.h" #include "fdbserver/WaitFailure.h"
#include "flow/OwningResource.h"
#include "flow/actorcompiler.h" // must be last include #include "flow/actorcompiler.h" // must be last include
@ -147,9 +148,9 @@ public:
} }
} }
ACTOR static Future<Void> trackStorageServerQueueInfo(Ratekeeper* self, StorageServerInterface ssi) { ACTOR static Future<Void> trackStorageServerQueueInfo(ActorWeakSelfRef<Ratekeeper> self,
self->storageQueueInfo.insert(mapPair(ssi.id(), StorageQueueInfo(ssi.id(), ssi.locality))); StorageServerInterface ssi) {
state Map<UID, StorageQueueInfo>::iterator myQueueInfo = self->storageQueueInfo.find(ssi.id()); self->storageQueueInfo.insert(mapPair(ssi.id(), StorageQueueInfo(self->id, ssi.id(), ssi.locality)));
TraceEvent("RkTracking", self->id) TraceEvent("RkTracking", self->id)
.detail("StorageServer", ssi.id()) .detail("StorageServer", ssi.id())
.detail("Locality", ssi.locality.toString()); .detail("Locality", ssi.locality.toString());
@ -157,6 +158,7 @@ public:
loop { loop {
ErrorOr<StorageQueuingMetricsReply> reply = wait(ssi.getQueuingMetrics.getReplyUnlessFailedFor( ErrorOr<StorageQueuingMetricsReply> reply = wait(ssi.getQueuingMetrics.getReplyUnlessFailedFor(
StorageQueuingMetricsRequest(), 0, 0)); // SOMEDAY: or tryGetReply? StorageQueuingMetricsRequest(), 0, 0)); // SOMEDAY: or tryGetReply?
Map<UID, StorageQueueInfo>::iterator myQueueInfo = self->storageQueueInfo.find(ssi.id());
if (reply.present()) { if (reply.present()) {
myQueueInfo->value.update(reply.get(), self->smoothTotalDurableBytes); myQueueInfo->value.update(reply.get(), self->smoothTotalDurableBytes);
myQueueInfo->value.acceptingRequests = ssi.isAcceptingRequests(); myQueueInfo->value.acceptingRequests = ssi.isAcceptingRequests();
@ -173,7 +175,8 @@ public:
} }
} catch (...) { } catch (...) {
// including cancellation // including cancellation
self->storageQueueInfo.erase(myQueueInfo); self->storageQueueInfo.erase(ssi.id());
self->storageServerInterfaces.erase(ssi.id());
throw; throw;
} }
} }
@ -207,28 +210,40 @@ public:
} }
ACTOR static Future<Void> trackEachStorageServer( ACTOR static Future<Void> trackEachStorageServer(
Ratekeeper* self, ActorWeakSelfRef<Ratekeeper> self,
FutureStream<std::pair<UID, Optional<StorageServerInterface>>> serverChanges) { FutureStream<std::pair<UID, Optional<StorageServerInterface>>> serverChanges) {
state Map<UID, Future<Void>> actors;
state std::unordered_map<UID, Future<Void>> storageServerTrackers;
state Promise<Void> err; state Promise<Void> err;
loop choose { loop choose {
when(state std::pair<UID, Optional<StorageServerInterface>> change = waitNext(serverChanges)) { when(state std::pair<UID, Optional<StorageServerInterface>> change = waitNext(serverChanges)) {
wait(delay(0)); // prevent storageServerTracker from getting cancelled while on the call stack wait(delay(0)); // prevent storageServerTracker from getting cancelled while on the call stack
const UID& id = change.first;
if (change.second.present()) { if (change.second.present()) {
if (!change.second.get().isTss()) { if (!change.second.get().isTss()) {
auto& a = actors[change.first];
auto& a = storageServerTrackers[change.first];
a = Future<Void>(); a = Future<Void>();
a = splitError(trackStorageServerQueueInfo(self, change.second.get()), err); a = splitError(trackStorageServerQueueInfo(self, change.second.get()), err);
self->storageServerInterfaces[id] = change.second.get();
} }
} else } else {
actors.erase(change.first); storageServerTrackers.erase(id);
self->storageServerInterfaces.erase(id);
}
} }
when(wait(err.getFuture())) {} when(wait(err.getFuture())) {}
} }
} }
ACTOR static Future<Void> run(RatekeeperInterface rkInterf, Reference<AsyncVar<ServerDBInfo> const> dbInfo) { ACTOR static Future<Void> run(RatekeeperInterface rkInterf, Reference<AsyncVar<ServerDBInfo> const> dbInfo) {
state Ratekeeper self(rkInterf.id(), openDBOnServer(dbInfo, TaskPriority::DefaultEndpoint, LockAware::True)); state ActorOwningSelfRef<Ratekeeper> pSelf(
new Ratekeeper(rkInterf.id(), openDBOnServer(dbInfo, TaskPriority::DefaultEndpoint, LockAware::True)));
state Ratekeeper& self = *pSelf;
state Future<Void> timeout = Void(); state Future<Void> timeout = Void();
state std::vector<Future<Void>> tlogTrackers; state std::vector<Future<Void>> tlogTrackers;
state std::vector<TLogInterface> tlogInterfs; state std::vector<TLogInterface> tlogInterfs;
@ -241,7 +256,7 @@ public:
PromiseStream<std::pair<UID, Optional<StorageServerInterface>>> serverChanges; PromiseStream<std::pair<UID, Optional<StorageServerInterface>>> serverChanges;
self.addActor.send(self.monitorServerListChange(serverChanges)); self.addActor.send(self.monitorServerListChange(serverChanges));
self.addActor.send(self.trackEachStorageServer(serverChanges.getFuture())); self.addActor.send(RatekeeperImpl::trackEachStorageServer(pSelf, serverChanges.getFuture()));
self.addActor.send(traceRole(Role::RATEKEEPER, rkInterf.id())); self.addActor.send(traceRole(Role::RATEKEEPER, rkInterf.id()));
self.addActor.send(self.monitorThrottlingChanges()); self.addActor.send(self.monitorThrottlingChanges());
@ -367,17 +382,23 @@ public:
ACTOR static Future<Void> refreshStorageServerCommitCosts(Ratekeeper* self) { ACTOR static Future<Void> refreshStorageServerCommitCosts(Ratekeeper* self) {
state double lastBusiestCommitTagPick; state double lastBusiestCommitTagPick;
state std::vector<Future<Void>> replies;
loop { loop {
lastBusiestCommitTagPick = now(); lastBusiestCommitTagPick = now();
wait(delay(SERVER_KNOBS->TAG_MEASUREMENT_INTERVAL)); wait(delay(SERVER_KNOBS->TAG_MEASUREMENT_INTERVAL));
replies.clear();
double elapsed = now() - lastBusiestCommitTagPick; double elapsed = now() - lastBusiestCommitTagPick;
// for each SS, select the busiest commit tag from ssTrTagCommitCost // for each SS, select the busiest commit tag from ssTrTagCommitCost
for (auto& [ssId, ssQueueInfo] : self->storageQueueInfo) { for (auto& [ssId, ssQueueInfo] : self->storageQueueInfo) {
ssQueueInfo.refreshCommitCost(elapsed); // NOTE: In some cases, for unknown reason SS will not respond to the updateCommitCostRequest. Since the
// information is not time-sensitive, we do not wait for the replies.
replies.push_back(self->storageServerInterfaces[ssId].updateCommitCostRequest.getReply(
ssQueueInfo.refreshCommitCost(elapsed)));
} }
} }
} }
}; // class RatekeeperImpl }; // class RatekeeperImpl
Future<Void> Ratekeeper::configurationMonitor() { Future<Void> Ratekeeper::configurationMonitor() {
@ -389,15 +410,6 @@ Future<Void> Ratekeeper::monitorServerListChange(
return RatekeeperImpl::monitorServerListChange(this, serverChanges); return RatekeeperImpl::monitorServerListChange(this, serverChanges);
} }
Future<Void> Ratekeeper::trackEachStorageServer(
FutureStream<std::pair<UID, Optional<StorageServerInterface>>> serverChanges) {
return RatekeeperImpl::trackEachStorageServer(this, serverChanges);
}
Future<Void> Ratekeeper::trackStorageServerQueueInfo(StorageServerInterface ssi) {
return RatekeeperImpl::trackStorageServerQueueInfo(this, ssi);
}
Future<Void> Ratekeeper::trackTLogQueueInfo(TLogInterface tli) { Future<Void> Ratekeeper::trackTLogQueueInfo(TLogInterface tli) {
return RatekeeperImpl::trackTLogQueueInfo(this, tli); return RatekeeperImpl::trackTLogQueueInfo(this, tli);
} }
@ -940,17 +952,19 @@ ACTOR Future<Void> ratekeeper(RatekeeperInterface rkInterf, Reference<AsyncVar<S
return Void(); return Void();
} }
StorageQueueInfo::StorageQueueInfo(UID id, LocalityData locality) StorageQueueInfo::StorageQueueInfo(const UID& ratekeeperID_, const UID& id_, const LocalityData& locality_)
: busiestWriteTagEventHolder(makeReference<EventCacheHolder>(id.toString() + "/BusiestWriteTag")), valid(false), : valid(false), ratekeeperID(ratekeeperID_), id(id_), locality(locality_), acceptingRequests(false),
id(id), locality(locality), acceptingRequests(false), smoothDurableBytes(SERVER_KNOBS->SMOOTHING_AMOUNT), smoothDurableBytes(SERVER_KNOBS->SMOOTHING_AMOUNT), smoothInputBytes(SERVER_KNOBS->SMOOTHING_AMOUNT),
smoothInputBytes(SERVER_KNOBS->SMOOTHING_AMOUNT), verySmoothDurableBytes(SERVER_KNOBS->SLOW_SMOOTHING_AMOUNT), verySmoothDurableBytes(SERVER_KNOBS->SLOW_SMOOTHING_AMOUNT), smoothDurableVersion(SERVER_KNOBS->SMOOTHING_AMOUNT),
smoothDurableVersion(SERVER_KNOBS->SMOOTHING_AMOUNT), smoothLatestVersion(SERVER_KNOBS->SMOOTHING_AMOUNT), smoothLatestVersion(SERVER_KNOBS->SMOOTHING_AMOUNT), smoothFreeSpace(SERVER_KNOBS->SMOOTHING_AMOUNT),
smoothFreeSpace(SERVER_KNOBS->SMOOTHING_AMOUNT), smoothTotalSpace(SERVER_KNOBS->SMOOTHING_AMOUNT), smoothTotalSpace(SERVER_KNOBS->SMOOTHING_AMOUNT), limitReason(limitReason_t::unlimited) {
limitReason(limitReason_t::unlimited) {
// FIXME: this is a tacky workaround for a potential uninitialized use in trackStorageServerQueueInfo // FIXME: this is a tacky workaround for a potential uninitialized use in trackStorageServerQueueInfo
lastReply.instanceID = -1; lastReply.instanceID = -1;
} }
StorageQueueInfo::StorageQueueInfo(const UID& id_, const LocalityData& locality_)
: StorageQueueInfo(UID(), id_, locality_) {}
void StorageQueueInfo::addCommitCost(TransactionTagRef tagName, TransactionCommitCostEstimation const& cost) { void StorageQueueInfo::addCommitCost(TransactionTagRef tagName, TransactionCommitCostEstimation const& cost) {
tagCostEst[tagName] += cost; tagCostEst[tagName] += cost;
totalWriteCosts += cost.getCostSum(); totalWriteCosts += cost.getCostSum();
@ -983,7 +997,7 @@ void StorageQueueInfo::update(StorageQueuingMetricsReply const& reply, Smoother&
busiestReadTags = reply.busiestTags; busiestReadTags = reply.busiestTags;
} }
void StorageQueueInfo::refreshCommitCost(double elapsed) { UpdateCommitCostRequest StorageQueueInfo::refreshCommitCost(double elapsed) {
busiestWriteTags.clear(); busiestWriteTags.clear();
TransactionTag busiestTag; TransactionTag busiestTag;
TransactionCommitCostEstimation maxCost; TransactionCommitCostEstimation maxCost;
@ -1003,19 +1017,32 @@ void StorageQueueInfo::refreshCommitCost(double elapsed) {
busiestWriteTags.emplace_back(busiestTag, maxRate, maxBusyness); busiestWriteTags.emplace_back(busiestTag, maxRate, maxBusyness);
} }
TraceEvent("BusiestWriteTag", id) UpdateCommitCostRequest updateCommitCostRequest{ ratekeeperID,
.detail("Elapsed", elapsed) now(),
.detail("Tag", printable(busiestTag)) elapsed,
.detail("TagOps", maxCost.getOpsSum()) busiestTag,
.detail("TagCost", maxCost.getCostSum()) maxCost.getOpsSum(),
.detail("TotalCost", totalWriteCosts) maxCost.getCostSum(),
.detail("Reported", !busiestWriteTags.empty()) totalWriteCosts,
.trackLatest(busiestWriteTagEventHolder->trackingKey); !busiestWriteTags.empty(),
ReplyPromise<Void>() };
// reset statistics // reset statistics
tagCostEst.clear(); tagCostEst.clear();
totalWriteOps = 0; totalWriteOps = 0;
totalWriteCosts = 0; totalWriteCosts = 0;
return updateCommitCostRequest;
}
Optional<double> StorageQueueInfo::getThrottlingRatio(int64_t storageTargetBytes, int64_t storageSpringBytes) const {
auto const storageQueue = getStorageQueueBytes();
if (storageQueue < storageTargetBytes - storageSpringBytes) {
return {};
} else {
return std::max(
0.0, static_cast<double>((storageTargetBytes + storageSpringBytes) - storageQueue) / storageSpringBytes);
}
} }
TLogQueueInfo::TLogQueueInfo(UID id) TLogQueueInfo::TLogQueueInfo(UID id)

View File

@ -215,8 +215,12 @@ class TestConfig {
std::string attrib = removeWhitespace(line.substr(0, found)); std::string attrib = removeWhitespace(line.substr(0, found));
std::string value = removeWhitespace(line.substr(found + 1)); std::string value = removeWhitespace(line.substr(found + 1));
if (attrib == "extraDB") { if (attrib == "extraDatabaseMode") {
sscanf(value.c_str(), "%d", &extraDB); extraDatabaseMode = ISimulator::stringToExtraDatabaseMode(value);
}
if (attrib == "extraDatabaseCount") {
sscanf(value.c_str(), "%d", &extraDatabaseCount);
} }
if (attrib == "minimumReplication") { if (attrib == "minimumReplication") {
@ -296,7 +300,9 @@ class TestConfig {
ConfigDBType configDBType{ ConfigDBType::DISABLED }; ConfigDBType configDBType{ ConfigDBType::DISABLED };
public: public:
int extraDB = 0; ISimulator::ExtraDatabaseMode extraDatabaseMode = ISimulator::ExtraDatabaseMode::Disabled;
// The number of extra database used if the database mode is MULTIPLE
int extraDatabaseCount = 1;
int minimumReplication = 0; int minimumReplication = 0;
int minimumRegions = 0; int minimumRegions = 0;
bool configureLocked = false; bool configureLocked = false;
@ -335,6 +341,7 @@ public:
bool allowDefaultTenant = true; bool allowDefaultTenant = true;
bool allowDisablingTenants = true; bool allowDisablingTenants = true;
bool allowCreatingTenants = true;
bool injectTargetedSSRestart = false; bool injectTargetedSSRestart = false;
bool injectSSDelay = false; bool injectSSDelay = false;
@ -362,8 +369,10 @@ public:
loadIniFile(testFile); loadIniFile(testFile);
return; return;
} }
std::string extraDatabaseModeStr;
ConfigBuilder builder; ConfigBuilder builder;
builder.add("extraDB", &extraDB) builder.add("extraDatabaseMode", &extraDatabaseModeStr)
.add("extraDatabaseCount", &extraDatabaseCount)
.add("minimumReplication", &minimumReplication) .add("minimumReplication", &minimumReplication)
.add("minimumRegions", &minimumRegions) .add("minimumRegions", &minimumRegions)
.add("configureLocked", &configureLocked) .add("configureLocked", &configureLocked)
@ -394,6 +403,8 @@ public:
.add("blobGranulesEnabled", &blobGranulesEnabled) .add("blobGranulesEnabled", &blobGranulesEnabled)
.add("allowDefaultTenant", &allowDefaultTenant) .add("allowDefaultTenant", &allowDefaultTenant)
.add("allowDisablingTenants", &allowDisablingTenants) .add("allowDisablingTenants", &allowDisablingTenants)
.add("allowCreatingTenants", &allowCreatingTenants)
.add("randomlyRenameZoneId", &randomlyRenameZoneId)
.add("randomlyRenameZoneId", &randomlyRenameZoneId) .add("randomlyRenameZoneId", &randomlyRenameZoneId)
.add("injectTargetedSSRestart", &injectTargetedSSRestart) .add("injectTargetedSSRestart", &injectTargetedSSRestart)
.add("injectSSDelay", &injectSSDelay); .add("injectSSDelay", &injectSSDelay);
@ -418,6 +429,9 @@ public:
if (!isFirstTestInRestart) { if (!isFirstTestInRestart) {
isFirstTestInRestart = tomlKeyPresent(file, "restartInfoLocation"); isFirstTestInRestart = tomlKeyPresent(file, "restartInfoLocation");
} }
if (!extraDatabaseModeStr.empty()) {
extraDatabaseMode = ISimulator::stringToExtraDatabaseMode(extraDatabaseModeStr);
}
} catch (std::exception& e) { } catch (std::exception& e) {
std::cerr << e.what() << std::endl; std::cerr << e.what() << std::endl;
TraceEvent("TOMLParseError").detail("Error", printable(e.what())); TraceEvent("TOMLParseError").detail("Error", printable(e.what()));
@ -478,22 +492,24 @@ ACTOR Future<Void> runDr(Reference<IClusterConnectionRecord> connRecord) {
} }
if (g_simulator.drAgents == ISimulator::BackupAgentType::BackupToDB) { if (g_simulator.drAgents == ISimulator::BackupAgentType::BackupToDB) {
ASSERT(g_simulator.extraDatabases.size() == 1);
Database cx = Database::createDatabase(connRecord, -1); Database cx = Database::createDatabase(connRecord, -1);
auto extraFile = makeReference<ClusterConnectionMemoryRecord>(*g_simulator.extraDB); auto extraFile =
state Database extraDB = Database::createDatabase(extraFile, -1); makeReference<ClusterConnectionMemoryRecord>(ClusterConnectionString(g_simulator.extraDatabases[0]));
state Database drDatabase = Database::createDatabase(extraFile, -1);
TraceEvent("StartingDrAgents") TraceEvent("StartingDrAgents")
.detail("ConnectionString", connRecord->getConnectionString().toString()) .detail("ConnectionString", connRecord->getConnectionString().toString())
.detail("ExtraString", extraFile->getConnectionString().toString()); .detail("ExtraString", extraFile->getConnectionString().toString());
state DatabaseBackupAgent dbAgent = DatabaseBackupAgent(cx); state DatabaseBackupAgent dbAgent = DatabaseBackupAgent(cx);
state DatabaseBackupAgent extraAgent = DatabaseBackupAgent(extraDB); state DatabaseBackupAgent extraAgent = DatabaseBackupAgent(drDatabase);
auto drPollDelay = 1.0 / CLIENT_KNOBS->BACKUP_AGGREGATE_POLL_RATE; auto drPollDelay = 1.0 / CLIENT_KNOBS->BACKUP_AGGREGATE_POLL_RATE;
agentFutures.push_back(extraAgent.run(cx, drPollDelay, CLIENT_KNOBS->SIM_BACKUP_TASKS_PER_AGENT)); agentFutures.push_back(extraAgent.run(cx, drPollDelay, CLIENT_KNOBS->SIM_BACKUP_TASKS_PER_AGENT));
agentFutures.push_back(dbAgent.run(extraDB, drPollDelay, CLIENT_KNOBS->SIM_BACKUP_TASKS_PER_AGENT)); agentFutures.push_back(dbAgent.run(drDatabase, drPollDelay, CLIENT_KNOBS->SIM_BACKUP_TASKS_PER_AGENT));
while (g_simulator.drAgents == ISimulator::BackupAgentType::BackupToDB) { while (g_simulator.drAgents == ISimulator::BackupAgentType::BackupToDB) {
wait(delay(1.0)); wait(delay(1.0));
@ -1102,10 +1118,10 @@ ACTOR Future<Void> restartSimulatedSystem(std::vector<Future<Void>>* systemActor
if (tssModeStr != nullptr) { if (tssModeStr != nullptr) {
g_simulator.tssMode = (ISimulator::TSSMode)atoi(tssModeStr); g_simulator.tssMode = (ISimulator::TSSMode)atoi(tssModeStr);
} }
bool enableExtraDB = (testConfig.extraDB == 3);
ClusterConnectionString conn(ini.GetValue("META", "connectionString")); ClusterConnectionString conn(ini.GetValue("META", "connectionString"));
if (enableExtraDB) { if (testConfig.extraDatabaseMode == ISimulator::ExtraDatabaseMode::Local) {
g_simulator.extraDB = new ClusterConnectionString(ini.GetValue("META", "connectionString")); g_simulator.extraDatabases.clear();
g_simulator.extraDatabases.push_back(conn.toString());
} }
if (!testConfig.disableHostname) { if (!testConfig.disableHostname) {
auto mockDNSStr = ini.GetValue("META", "mockDNS"); auto mockDNSStr = ini.GetValue("META", "mockDNS");
@ -1271,7 +1287,8 @@ ACTOR Future<Void> restartSimulatedSystem(std::vector<Future<Void>>* systemActor
// Configuration details compiled in a structure used when setting up a simulated cluster // Configuration details compiled in a structure used when setting up a simulated cluster
struct SimulationConfig { struct SimulationConfig {
explicit SimulationConfig(const TestConfig& testConfig); explicit SimulationConfig(const TestConfig& testConfig);
int extraDB; ISimulator::ExtraDatabaseMode extraDatabaseMode;
int extraDatabaseCount;
bool generateFearless; bool generateFearless;
DatabaseConfiguration db; DatabaseConfiguration db;
@ -1300,7 +1317,8 @@ private:
void generateNormalConfig(const TestConfig& testConfig); void generateNormalConfig(const TestConfig& testConfig);
}; };
SimulationConfig::SimulationConfig(const TestConfig& testConfig) : extraDB(testConfig.extraDB) { SimulationConfig::SimulationConfig(const TestConfig& testConfig)
: extraDatabaseMode(testConfig.extraDatabaseMode), extraDatabaseCount(testConfig.extraDatabaseCount) {
generateNormalConfig(testConfig); generateNormalConfig(testConfig);
} }
@ -1746,7 +1764,9 @@ void SimulationConfig::setMachineCount(const TestConfig& testConfig) {
machine_count = std::max(datacenters + 2, machine_count = std::max(datacenters + 2,
((db.minDatacentersRequired() > 0) ? datacenters : 1) * ((db.minDatacentersRequired() > 0) ? datacenters : 1) *
std::max(3, db.minZonesRequiredPerDatacenter())); std::max(3, db.minZonesRequiredPerDatacenter()));
machine_count = deterministicRandom()->randomInt(machine_count, std::max(machine_count + 1, extraDB ? 6 : 10)); machine_count = deterministicRandom()->randomInt(
machine_count,
std::max(machine_count + 1, extraDatabaseMode == ISimulator::ExtraDatabaseMode::Disabled ? 10 : 6));
// generateMachineTeamTestConfig set up the number of servers per machine and the number of machines such that // generateMachineTeamTestConfig set up the number of servers per machine and the number of machines such that
// if we do not remove the surplus server and machine teams, the simulation test will report error. // if we do not remove the surplus server and machine teams, the simulation test will report error.
// This is needed to make sure the number of server (and machine) teams is no larger than the desired number. // This is needed to make sure the number of server (and machine) teams is no larger than the desired number.
@ -1756,7 +1776,9 @@ void SimulationConfig::setMachineCount(const TestConfig& testConfig) {
// while the max possible machine team number is 10. // while the max possible machine team number is 10.
// If machine_count > 5, we can still test the effectivenss of machine teams // If machine_count > 5, we can still test the effectivenss of machine teams
// Note: machine_count may be much larger than 5 because we may have a big replication factor // Note: machine_count may be much larger than 5 because we may have a big replication factor
machine_count = std::max(machine_count, deterministicRandom()->randomInt(5, extraDB ? 6 : 10)); machine_count = std::max(machine_count,
deterministicRandom()->randomInt(
5, extraDatabaseMode == ISimulator::ExtraDatabaseMode::Disabled ? 10 : 6));
} }
} }
machine_count += datacenters * testConfig.extraMachineCountDC; machine_count += datacenters * testConfig.extraMachineCountDC;
@ -1783,7 +1805,8 @@ void SimulationConfig::setProcessesPerMachine(const TestConfig& testConfig) {
} else if (generateFearless) { } else if (generateFearless) {
processes_per_machine = 1; processes_per_machine = 1;
} else { } else {
processes_per_machine = deterministicRandom()->randomInt(1, (extraDB ? 14 : 28) / machine_count + 2); processes_per_machine = deterministicRandom()->randomInt(
1, (extraDatabaseMode == ISimulator::ExtraDatabaseMode::Disabled ? 28 : 14) / machine_count + 2);
} }
} }
@ -1792,7 +1815,7 @@ void SimulationConfig::setProcessesPerMachine(const TestConfig& testConfig) {
void SimulationConfig::setTss(const TestConfig& testConfig) { void SimulationConfig::setTss(const TestConfig& testConfig) {
int tssCount = 0; int tssCount = 0;
// TODO: Support TSS in SHARD_ENCODE_LOCATION_METADATA mode. // TODO: Support TSS in SHARD_ENCODE_LOCATION_METADATA mode.
if (!testConfig.simpleConfig && !testConfig.disableTss && !CLIENT_KNOBS->SHARD_ENCODE_LOCATION_METADATA && if (!testConfig.simpleConfig && !testConfig.disableTss && !SERVER_KNOBS->SHARD_ENCODE_LOCATION_METADATA &&
deterministicRandom()->random01() < 0.25) { deterministicRandom()->random01() < 0.25) {
// 1 or 2 tss // 1 or 2 tss
tssCount = deterministicRandom()->randomInt(1, 3); tssCount = deterministicRandom()->randomInt(1, 3);
@ -2014,10 +2037,23 @@ void setupSimulatedSystem(std::vector<Future<Void>>* systemActors,
NetworkAddressFromHostname fromHostname = NetworkAddressFromHostname fromHostname =
useHostname ? NetworkAddressFromHostname::True : NetworkAddressFromHostname::False; useHostname ? NetworkAddressFromHostname::True : NetworkAddressFromHostname::False;
int extraDatabaseCount = 0;
bool useLocalDatabase = (testConfig.extraDatabaseMode == ISimulator::ExtraDatabaseMode::LocalOrSingle && BUGGIFY) ||
testConfig.extraDatabaseMode == ISimulator::ExtraDatabaseMode::Local;
if (!useLocalDatabase && testConfig.extraDatabaseMode != ISimulator::ExtraDatabaseMode::Disabled) {
extraDatabaseCount =
testConfig.extraDatabaseMode == ISimulator::ExtraDatabaseMode::Multiple && testConfig.extraDatabaseCount > 0
? testConfig.extraDatabaseCount
: 1;
}
std::vector<NetworkAddress> coordinatorAddresses; std::vector<NetworkAddress> coordinatorAddresses;
std::vector<Hostname> coordinatorHostnames; std::vector<Hostname> coordinatorHostnames;
std::vector<NetworkAddress> extraCoordinatorAddresses; // Used by extra DB if the DR db is a new one
std::vector<Hostname> extraCoordinatorHostnames; // A list of coordinators for each extra database being created. The Nth vector in the outer vector
// contains the coordinators for the Nth extra database.
std::vector<std::vector<NetworkAddress>> extraCoordinatorAddresses(extraDatabaseCount);
std::vector<std::vector<Hostname>> extraCoordinatorHostnames(extraDatabaseCount);
if (testConfig.minimumRegions > 1) { if (testConfig.minimumRegions > 1) {
// do not put coordinators in the primary region so that we can kill that region safely // do not put coordinators in the primary region so that we can kill that region safely
@ -2029,21 +2065,29 @@ void setupSimulatedSystem(std::vector<Future<Void>>* systemActors,
uint16_t port = sslEnabled && !sslOnly ? 2 : 1; uint16_t port = sslEnabled && !sslOnly ? 2 : 1;
NetworkAddress coordinator(ip, port, true, isTLS, fromHostname); NetworkAddress coordinator(ip, port, true, isTLS, fromHostname);
coordinatorAddresses.push_back(coordinator); coordinatorAddresses.push_back(coordinator);
auto extraIp = makeIPAddressForSim(useIPv6, { 4, dc, 1, m });
NetworkAddress extraCoordinator(extraIp, port, true, isTLS, fromHostname);
extraCoordinatorAddresses.push_back(extraCoordinator);
if (useHostname) { if (useHostname) {
std::string hostname = "fakeCoordinatorDC" + std::to_string(dc) + "M" + std::to_string(m); std::string hostname = "fakeCoordinatorDC" + std::to_string(dc) + "M" + std::to_string(m);
Hostname coordinatorHostname(hostname, std::to_string(port), isTLS); Hostname coordinatorHostname(hostname, std::to_string(port), isTLS);
coordinatorHostnames.push_back(coordinatorHostname); coordinatorHostnames.push_back(coordinatorHostname);
INetworkConnections::net()->addMockTCPEndpoint(hostname, std::to_string(port), { coordinator }); INetworkConnections::net()->addMockTCPEndpoint(hostname, std::to_string(port), { coordinator });
hostname = "fakeExtraCoordinatorDC" + std::to_string(dc) + "M" + std::to_string(m);
Hostname extraCoordinatorHostname(hostname, std::to_string(port), isTLS);
extraCoordinatorHostnames.push_back(extraCoordinatorHostname);
INetworkConnections::net()->addMockTCPEndpoint(
hostname, std::to_string(port), { extraCoordinator });
} }
for (int edb = 0; edb < extraDatabaseCount; ++edb) {
auto extraIp = makeIPAddressForSim(useIPv6, { 4 + edb, dc, 1, m });
NetworkAddress extraCoordinator(extraIp, port, true, isTLS, fromHostname);
extraCoordinatorAddresses[edb].push_back(extraCoordinator);
if (useHostname) {
std::string hostname = "fakeExtraCoordinatorDC" + std::to_string(dc) + "M" + std::to_string(m) +
"C" + std::to_string(edb);
Hostname extraCoordinatorHostname(hostname, std::to_string(port), isTLS);
extraCoordinatorHostnames[edb].push_back(extraCoordinatorHostname);
INetworkConnections::net()->addMockTCPEndpoint(
hostname, std::to_string(port), { extraCoordinator });
}
}
TraceEvent("SelectedCoordinator") TraceEvent("SelectedCoordinator")
.detail("Hostname", useHostname ? coordinatorHostnames.back().toString().c_str() : "N/A") .detail("Hostname", useHostname ? coordinatorHostnames.back().toString().c_str() : "N/A")
.detail("Address", coordinatorAddresses.back()); .detail("Address", coordinatorAddresses.back());
@ -2075,20 +2119,26 @@ void setupSimulatedSystem(std::vector<Future<Void>>* systemActors,
uint16_t port = sslEnabled && !sslOnly ? 2 : 1; uint16_t port = sslEnabled && !sslOnly ? 2 : 1;
NetworkAddress coordinator(ip, port, true, isTLS, fromHostname); NetworkAddress coordinator(ip, port, true, isTLS, fromHostname);
coordinatorAddresses.push_back(coordinator); coordinatorAddresses.push_back(coordinator);
auto extraIp = makeIPAddressForSim(useIPv6, { 4, dc, 1, m });
NetworkAddress extraCoordinator(extraIp, port, true, isTLS, fromHostname);
extraCoordinatorAddresses.push_back(extraCoordinator);
if (useHostname) { if (useHostname) {
std::string hostname = "fakeCoordinatorDC" + std::to_string(dc) + "M" + std::to_string(m); std::string hostname = "fakeCoordinatorDC" + std::to_string(dc) + "M" + std::to_string(m);
Hostname coordinatorHostname(hostname, std::to_string(port), isTLS); Hostname coordinatorHostname(hostname, std::to_string(port), isTLS);
coordinatorHostnames.push_back(coordinatorHostname); coordinatorHostnames.push_back(coordinatorHostname);
INetworkConnections::net()->addMockTCPEndpoint(hostname, std::to_string(port), { coordinator }); INetworkConnections::net()->addMockTCPEndpoint(hostname, std::to_string(port), { coordinator });
hostname = "fakeExtraCoordinatorDC" + std::to_string(dc) + "M" + std::to_string(m); }
for (int edb = 0; edb < extraDatabaseCount; ++edb) {
auto extraIp = makeIPAddressForSim(useIPv6, { 4 + edb, dc, 1, m });
NetworkAddress extraCoordinator(extraIp, port, true, isTLS, fromHostname);
extraCoordinatorAddresses[edb].push_back(extraCoordinator);
std::string hostname = "fakeExtraCoordinatorDC" + std::to_string(dc) + "M" + std::to_string(m) +
"C" + std::to_string(edb);
Hostname extraCoordinatorHostname(hostname, std::to_string(port), isTLS); Hostname extraCoordinatorHostname(hostname, std::to_string(port), isTLS);
extraCoordinatorHostnames.push_back(extraCoordinatorHostname); extraCoordinatorHostnames[edb].push_back(extraCoordinatorHostname);
INetworkConnections::net()->addMockTCPEndpoint( INetworkConnections::net()->addMockTCPEndpoint(
hostname, std::to_string(port), { extraCoordinator }); hostname, std::to_string(port), { extraCoordinator });
} }
TraceEvent("SelectedCoordinator") TraceEvent("SelectedCoordinator")
.detail("Hostname", useHostname ? coordinatorHostnames.back().toString().c_str() : "N/A") .detail("Hostname", useHostname ? coordinatorHostnames.back().toString().c_str() : "N/A")
.detail("Address", coordinatorAddresses.back()) .detail("Address", coordinatorAddresses.back())
@ -2127,22 +2177,19 @@ void setupSimulatedSystem(std::vector<Future<Void>>* systemActors,
conn = ClusterConnectionString(coordinatorHostnames, "TestCluster:0"_sr); conn = ClusterConnectionString(coordinatorHostnames, "TestCluster:0"_sr);
} }
// If extraDB==0, leave g_simulator.extraDB as null because the test does not use DR. if (useLocalDatabase) {
if (testConfig.extraDB == 1) { g_simulator.extraDatabases.push_back(
// The DR database can be either a new database or itself useHostname ? ClusterConnectionString(coordinatorHostnames, "TestCluster:0"_sr).toString()
g_simulator.extraDB = : ClusterConnectionString(coordinatorAddresses, "TestCluster:0"_sr).toString());
BUGGIFY ? (useHostname ? new ClusterConnectionString(coordinatorHostnames, "TestCluster:0"_sr) } else if (testConfig.extraDatabaseMode != ISimulator::ExtraDatabaseMode::Disabled) {
: new ClusterConnectionString(coordinatorAddresses, "TestCluster:0"_sr)) for (int i = 0; i < extraDatabaseCount; ++i) {
: (useHostname ? new ClusterConnectionString(extraCoordinatorHostnames, "ExtraCluster:0"_sr) g_simulator.extraDatabases.push_back(
: new ClusterConnectionString(extraCoordinatorAddresses, "ExtraCluster:0"_sr)); useHostname
} else if (testConfig.extraDB == 2) { ? ClusterConnectionString(extraCoordinatorHostnames[i], StringRef(format("ExtraCluster%04d:0", i)))
// The DR database is a new database .toString()
g_simulator.extraDB = useHostname ? new ClusterConnectionString(extraCoordinatorHostnames, "ExtraCluster:0"_sr) : ClusterConnectionString(extraCoordinatorAddresses[i], StringRef(format("ExtraCluster%04d:0", i)))
: new ClusterConnectionString(extraCoordinatorAddresses, "ExtraCluster:0"_sr); .toString());
} else if (testConfig.extraDB == 3) { }
// The DR database is the same database
g_simulator.extraDB = useHostname ? new ClusterConnectionString(coordinatorHostnames, "TestCluster:0"_sr)
: new ClusterConnectionString(coordinatorAddresses, "TestCluster:0"_sr);
} }
*pConnString = conn; *pConnString = conn;
@ -2151,7 +2198,7 @@ void setupSimulatedSystem(std::vector<Future<Void>>* systemActors,
.detail("String", conn.toString()) .detail("String", conn.toString())
.detail("ConfigString", startingConfigString); .detail("ConfigString", startingConfigString);
bool requiresExtraDBMachines = testConfig.extraDB && g_simulator.extraDB->toString() != conn.toString(); bool requiresExtraDBMachines = !g_simulator.extraDatabases.empty() && !useLocalDatabase;
int assignedMachines = 0, nonVersatileMachines = 0; int assignedMachines = 0, nonVersatileMachines = 0;
bool gradualMigrationPossible = true; bool gradualMigrationPossible = true;
std::vector<ProcessClass::ClassType> processClassesSubSet = { ProcessClass::UnsetClass, std::vector<ProcessClass::ClassType> processClassesSubSet = { ProcessClass::UnsetClass,
@ -2259,31 +2306,35 @@ void setupSimulatedSystem(std::vector<Future<Void>>* systemActors,
"SimulatedMachine")); "SimulatedMachine"));
if (requiresExtraDBMachines) { if (requiresExtraDBMachines) {
std::vector<IPAddress> extraIps; int cluster = 4;
extraIps.reserve(processesPerMachine); for (auto extraDatabase : g_simulator.extraDatabases) {
for (int i = 0; i < processesPerMachine; i++) { std::vector<IPAddress> extraIps;
extraIps.push_back( extraIps.reserve(processesPerMachine);
makeIPAddressForSim(useIPv6, { 4, dc, deterministicRandom()->randomInt(1, i + 2), machine })); for (int i = 0; i < processesPerMachine; i++) {
extraIps.push_back(makeIPAddressForSim(
useIPv6, { cluster, dc, deterministicRandom()->randomInt(1, i + 2), machine }));
}
Standalone<StringRef> newMachineId(deterministicRandom()->randomUniqueID().toString());
LocalityData localities(Optional<Standalone<StringRef>>(), newZoneId, newMachineId, dcUID);
localities.set("data_hall"_sr, dcUID);
systemActors->push_back(reportErrors(simulatedMachine(ClusterConnectionString(extraDatabase),
extraIps,
sslEnabled,
localities,
processClass,
baseFolder,
false,
machine == useSeedForMachine,
AgentNone,
sslOnly,
whitelistBinPaths,
protocolVersion,
configDBType),
"SimulatedMachine"));
++cluster;
} }
Standalone<StringRef> newMachineId(deterministicRandom()->randomUniqueID().toString());
LocalityData localities(Optional<Standalone<StringRef>>(), newZoneId, newMachineId, dcUID);
localities.set("data_hall"_sr, dcUID);
systemActors->push_back(reportErrors(simulatedMachine(*g_simulator.extraDB,
extraIps,
sslEnabled,
localities,
processClass,
baseFolder,
false,
machine == useSeedForMachine,
AgentNone,
sslOnly,
whitelistBinPaths,
protocolVersion,
configDBType),
"SimulatedMachine"));
} }
assignedMachines++; assignedMachines++;
@ -2392,9 +2443,9 @@ ACTOR void setupAndRun(std::string dataFolder,
allowList.addTrustedSubnet("abcd::/16"sv); allowList.addTrustedSubnet("abcd::/16"sv);
state bool allowDefaultTenant = testConfig.allowDefaultTenant; state bool allowDefaultTenant = testConfig.allowDefaultTenant;
state bool allowDisablingTenants = testConfig.allowDisablingTenants; state bool allowDisablingTenants = testConfig.allowDisablingTenants;
state bool allowCreatingTenants = true; state bool allowCreatingTenants = testConfig.allowCreatingTenants;
if (!CLIENT_KNOBS->SHARD_ENCODE_LOCATION_METADATA) { if (!SERVER_KNOBS->SHARD_ENCODE_LOCATION_METADATA) {
testConfig.storageEngineExcludeTypes.push_back(5); testConfig.storageEngineExcludeTypes.push_back(5);
} }
@ -2422,7 +2473,8 @@ ACTOR void setupAndRun(std::string dataFolder,
// Disable the default tenant in backup and DR tests for now. This is because backup does not currently duplicate // Disable the default tenant in backup and DR tests for now. This is because backup does not currently duplicate
// the tenant map and related state. // the tenant map and related state.
// TODO: reenable when backup/DR or BlobGranule supports tenants. // TODO: reenable when backup/DR or BlobGranule supports tenants.
if (std::string_view(testFile).find("Backup") != std::string_view::npos || testConfig.extraDB != 0) { if (std::string_view(testFile).find("Backup") != std::string_view::npos ||
testConfig.extraDatabaseMode != ISimulator::ExtraDatabaseMode::Disabled) {
allowDefaultTenant = false; allowDefaultTenant = false;
} }

View File

@ -554,23 +554,11 @@ struct RolesInfo {
TraceEventFields const& busiestReadTag = metrics.at("BusiestReadTag"); TraceEventFields const& busiestReadTag = metrics.at("BusiestReadTag");
if (busiestReadTag.size()) { if (busiestReadTag.size()) {
int64_t tagCost = busiestReadTag.getInt64("TagCost"); int64_t tagCost = busiestReadTag.getInt64("TagCost");
if (tagCost > 0) { if (tagCost > 0) {
JsonBuilderObject busiestReadTagObj; JsonBuilderObject busiestReadTagObj;
int64_t totalSampledCost = busiestReadTag.getInt64("TotalSampledCost");
ASSERT(totalSampledCost > 0);
busiestReadTagObj["tag"] = busiestReadTag.getValue("Tag"); busiestReadTagObj["tag"] = busiestReadTag.getValue("Tag");
busiestReadTagObj["fractional_cost"] = (double)tagCost / totalSampledCost; busiestReadTagObj["cost"] = tagCost;
busiestReadTagObj["fractional_cost"] = busiestReadTag.getValue("FractionalBusyness");
double elapsed = busiestReadTag.getDouble("Elapsed");
if (CLIENT_KNOBS->READ_TAG_SAMPLE_RATE > 0 && elapsed > 0) {
JsonBuilderObject estimatedCostObj;
estimatedCostObj["hz"] = tagCost / CLIENT_KNOBS->READ_TAG_SAMPLE_RATE / elapsed;
busiestReadTagObj["estimated_cost"] = estimatedCostObj;
}
obj["busiest_read_tag"] = busiestReadTagObj; obj["busiest_read_tag"] = busiestReadTagObj;
} }
} }
@ -1912,27 +1900,6 @@ static Future<std::vector<std::pair<iface, EventMap>>> getServerMetrics(
return results; return results;
} }
ACTOR template <class iface>
static Future<std::vector<TraceEventFields>> getServerBusiestWriteTags(
std::vector<iface> servers,
std::unordered_map<NetworkAddress, WorkerInterface> address_workers,
WorkerDetails rkWorker) {
state std::vector<Future<Optional<TraceEventFields>>> futures;
futures.reserve(servers.size());
for (const auto& s : servers) {
futures.push_back(latestEventOnWorker(rkWorker.interf, s.id().toString() + "/BusiestWriteTag"));
}
wait(waitForAll(futures));
std::vector<TraceEventFields> result(servers.size());
for (int i = 0; i < servers.size(); ++i) {
if (futures[i].get().present()) {
result[i] = futures[i].get().get();
}
}
return result;
}
ACTOR ACTOR
static Future<std::vector<StorageServerStatusInfo>> readStorageInterfaceAndMetadata(Database cx, static Future<std::vector<StorageServerStatusInfo>> readStorageInterfaceAndMetadata(Database cx,
bool use_system_priority) { bool use_system_priority) {
@ -1970,6 +1937,16 @@ static Future<std::vector<StorageServerStatusInfo>> readStorageInterfaceAndMetad
return servers; return servers;
} }
namespace {
const std::vector<std::string> STORAGE_SERVER_METRICS_LIST{ "StorageMetrics",
"ReadLatencyMetrics",
"ReadLatencyBands",
"BusiestReadTag",
"BusiestWriteTag" };
} // namespace
ACTOR static Future<std::vector<StorageServerStatusInfo>> getStorageServerStatusInfos( ACTOR static Future<std::vector<StorageServerStatusInfo>> getStorageServerStatusInfos(
Database cx, Database cx,
std::unordered_map<NetworkAddress, WorkerInterface> address_workers, std::unordered_map<NetworkAddress, WorkerInterface> address_workers,
@ -1977,18 +1954,9 @@ ACTOR static Future<std::vector<StorageServerStatusInfo>> getStorageServerStatus
state std::vector<StorageServerStatusInfo> servers = state std::vector<StorageServerStatusInfo> servers =
wait(timeoutError(readStorageInterfaceAndMetadata(cx, true), 5.0)); wait(timeoutError(readStorageInterfaceAndMetadata(cx, true), 5.0));
state std::vector<std::pair<StorageServerStatusInfo, EventMap>> results; state std::vector<std::pair<StorageServerStatusInfo, EventMap>> results;
state std::vector<TraceEventFields> busiestWriteTags; wait(store(results, getServerMetrics(servers, address_workers, STORAGE_SERVER_METRICS_LIST)));
wait(store(results,
getServerMetrics(servers,
address_workers,
std::vector<std::string>{
"StorageMetrics", "ReadLatencyMetrics", "ReadLatencyBands", "BusiestReadTag" })) &&
store(busiestWriteTags, getServerBusiestWriteTags(servers, address_workers, rkWorker)));
ASSERT(busiestWriteTags.size() == results.size());
for (int i = 0; i < results.size(); ++i) { for (int i = 0; i < results.size(); ++i) {
servers[i].eventMap = std::move(results[i].second); servers[i].eventMap = std::move(results[i].second);
servers[i].eventMap.emplace("BusiestWriteTag", busiestWriteTags[i]);
} }
return servers; return servers;
} }
@ -2390,6 +2358,40 @@ ACTOR static Future<JsonBuilderObject> clusterSummaryStatisticsFetcher(
return statusObj; return statusObj;
} }
ACTOR static Future<JsonBuilderObject> blobWorkerStatusFetcher(
std::vector<BlobWorkerInterface> servers,
std::unordered_map<NetworkAddress, WorkerInterface> addressWorkersMap,
std::set<std::string>* incompleteReason) {
state JsonBuilderObject statusObj;
state int totalRanges = 0;
state std::vector<Future<Optional<TraceEventFields>>> futures;
statusObj["number_of_blob_workers"] = static_cast<int>(servers.size());
try {
for (auto& intf : servers) {
auto workerIntf = addressWorkersMap[intf.address()];
futures.push_back(latestEventOnWorker(workerIntf, "BlobWorkerMetrics"));
}
wait(waitForAll(futures));
for (auto future : futures) {
if (future.get().present()) {
auto latestTrace = future.get().get();
totalRanges += latestTrace.getInt("NumRangesAssigned");
}
}
statusObj["number_of_key_ranges"] = totalRanges;
} catch (Error& e) {
if (e.code() == error_code_actor_cancelled)
throw;
incompleteReason->insert("Unknown blob worker stats");
}
return statusObj;
}
static JsonBuilderObject tlogFetcher(int* logFaultTolerance, static JsonBuilderObject tlogFetcher(int* logFaultTolerance,
const std::vector<TLogSet>& tLogs, const std::vector<TLogSet>& tLogs,
std::unordered_map<NetworkAddress, WorkerInterface> const& address_workers) { std::unordered_map<NetworkAddress, WorkerInterface> const& address_workers) {
@ -3017,6 +3019,7 @@ ACTOR Future<StatusReply> clusterGetStatus(
state Optional<DatabaseConfiguration> configuration; state Optional<DatabaseConfiguration> configuration;
state Optional<LoadConfigurationResult> loadResult; state Optional<LoadConfigurationResult> loadResult;
state std::unordered_map<NetworkAddress, WorkerInterface> address_workers;
if (statusCode != RecoveryStatus::configuration_missing) { if (statusCode != RecoveryStatus::configuration_missing) {
std::pair<Optional<DatabaseConfiguration>, Optional<LoadConfigurationResult>> loadResults = std::pair<Optional<DatabaseConfiguration>, Optional<LoadConfigurationResult>> loadResults =
@ -3070,7 +3073,6 @@ ACTOR Future<StatusReply> clusterGetStatus(
// Start getting storage servers now (using system priority) concurrently. Using sys priority because // Start getting storage servers now (using system priority) concurrently. Using sys priority because
// having storage servers in status output is important to give context to error messages in status that // having storage servers in status output is important to give context to error messages in status that
// reference a storage server role ID. // reference a storage server role ID.
state std::unordered_map<NetworkAddress, WorkerInterface> address_workers;
for (auto const& worker : workers) { for (auto const& worker : workers) {
address_workers[worker.interf.address()] = worker.interf; address_workers[worker.interf.address()] = worker.interf;
} }
@ -3291,6 +3293,12 @@ ACTOR Future<StatusReply> clusterGetStatus(
statusObj["processes"] = processStatus; statusObj["processes"] = processStatus;
statusObj["clients"] = clientStatusFetcher(clientStatus); statusObj["clients"] = clientStatusFetcher(clientStatus);
if (configuration.present() && configuration.get().blobGranulesEnabled) {
JsonBuilderObject blobGranuelsStatus =
wait(blobWorkerStatusFetcher(blobWorkers, address_workers, &status_incomplete_reasons));
statusObj["blob_granules"] = blobGranuelsStatus;
}
JsonBuilderArray incompatibleConnectionsArray; JsonBuilderArray incompatibleConnectionsArray;
for (auto it : incompatibleConnections) { for (auto it : incompatibleConnections) {
incompatibleConnectionsArray.push_back(it.toString()); incompatibleConnectionsArray.push_back(it.toString());

View File

@ -117,13 +117,29 @@ public:
if (intervalStart > 0 && CLIENT_KNOBS->READ_TAG_SAMPLE_RATE > 0 && elapsed > 0) { if (intervalStart > 0 && CLIENT_KNOBS->READ_TAG_SAMPLE_RATE > 0 && elapsed > 0) {
previousBusiestTags = topTags.getBusiestTags(elapsed, intervalTotalSampledCount); previousBusiestTags = topTags.getBusiestTags(elapsed, intervalTotalSampledCount);
TraceEvent("BusiestReadTag", thisServerID) // For status, report the busiest tag:
.detail("Elapsed", elapsed) if (previousBusiestTags.empty()) {
//.detail("Tag", printable(busiestTag)) TraceEvent("BusiestReadTag", thisServerID).detail("TagCost", 0);
//.detail("TagCost", busiestTagCount) } else {
.detail("TotalSampledCost", intervalTotalSampledCount) auto busiestTagInfo = previousBusiestTags[0];
.detail("Reported", previousBusiestTags.size()) for (int i = 1; i < previousBusiestTags.size(); ++i) {
.trackLatest(busiestReadTagEventHolder->trackingKey); auto const& tagInfo = previousBusiestTags[i];
if (tagInfo.rate > busiestTagInfo.rate) {
busiestTagInfo = tagInfo;
}
}
TraceEvent("BusiestReadTag", thisServerID)
.detail("Tag", printable(busiestTagInfo.tag))
.detail("TagCost", busiestTagInfo.rate)
.detail("FractionalBusyness", busiestTagInfo.fractionalBusyness);
}
for (const auto& tagInfo : previousBusiestTags) {
TraceEvent("BusyReadTag", thisServerID)
.detail("Tag", printable(tagInfo.tag))
.detail("TagCost", tagInfo.rate)
.detail("FractionalBusyness", tagInfo.fractionalBusyness);
}
} }
intervalCounts.clear(); intervalCounts.clear();

View File

@ -139,6 +139,8 @@ public:
Future<Void> clientCounter; Future<Void> clientCounter;
int clientCount; int clientCount;
AsyncVar<bool> blobGranulesEnabled; AsyncVar<bool> blobGranulesEnabled;
ClusterType clusterType = ClusterType::STANDALONE;
Optional<ClusterName> metaclusterName;
DBInfo() DBInfo()
: clientInfo(new AsyncVar<ClientDBInfo>()), serverInfo(new AsyncVar<ServerDBInfo>()), : clientInfo(new AsyncVar<ClientDBInfo>()), serverInfo(new AsyncVar<ServerDBInfo>()),

View File

@ -18,6 +18,9 @@
* limitations under the License. * limitations under the License.
*/ */
#ifndef FDBSERVER_RATEKEEPER_H
#define FDBSERVER_RATEKEEPER_H
#pragma once #pragma once
#include "fdbclient/DatabaseConfiguration.h" #include "fdbclient/DatabaseConfiguration.h"
@ -49,13 +52,13 @@ enum limitReason_t {
class StorageQueueInfo { class StorageQueueInfo {
uint64_t totalWriteCosts{ 0 }; uint64_t totalWriteCosts{ 0 };
int totalWriteOps{ 0 }; int totalWriteOps{ 0 };
Reference<EventCacheHolder> busiestWriteTagEventHolder;
// refresh periodically // refresh periodically
TransactionTagMap<TransactionCommitCostEstimation> tagCostEst; TransactionTagMap<TransactionCommitCostEstimation> tagCostEst;
public: public:
bool valid; bool valid;
UID ratekeeperID;
UID id; UID id;
LocalityData locality; LocalityData locality;
StorageQueuingMetricsReply lastReply; StorageQueuingMetricsReply lastReply;
@ -67,12 +70,17 @@ public:
limitReason_t limitReason; limitReason_t limitReason;
std::vector<StorageQueuingMetricsReply::TagInfo> busiestReadTags, busiestWriteTags; std::vector<StorageQueuingMetricsReply::TagInfo> busiestReadTags, busiestWriteTags;
StorageQueueInfo(UID id, LocalityData locality); StorageQueueInfo(const UID& id, const LocalityData& locality);
void refreshCommitCost(double elapsed); StorageQueueInfo(const UID& rateKeeperID, const UID& id, const LocalityData& locality);
// Summarizes up the commit cost per storage server. Returns the UpdateCommitCostRequest for corresponding SS.
UpdateCommitCostRequest refreshCommitCost(double elapsed);
int64_t getStorageQueueBytes() const { return lastReply.bytesInput - smoothDurableBytes.smoothTotal(); } int64_t getStorageQueueBytes() const { return lastReply.bytesInput - smoothDurableBytes.smoothTotal(); }
int64_t getDurabilityLag() const { return smoothLatestVersion.smoothTotal() - smoothDurableVersion.smoothTotal(); } int64_t getDurabilityLag() const { return smoothLatestVersion.smoothTotal() - smoothDurableVersion.smoothTotal(); }
void update(StorageQueuingMetricsReply const&, Smoother& smoothTotalDurableBytes); void update(StorageQueuingMetricsReply const&, Smoother& smoothTotalDurableBytes);
void addCommitCost(TransactionTagRef tagName, TransactionCommitCostEstimation const& cost); void addCommitCost(TransactionTagRef tagName, TransactionCommitCostEstimation const& cost);
// Determine the ratio (limit / current throughput) for throttling based on write queue size
Optional<double> getThrottlingRatio(int64_t storageTargetBytes, int64_t storageSpringBytes) const;
}; };
struct TLogQueueInfo { struct TLogQueueInfo {
@ -150,6 +158,9 @@ class Ratekeeper {
std::unique_ptr<class ITagThrottler> tagThrottler; std::unique_ptr<class ITagThrottler> tagThrottler;
// Maps storage server ID to storage server interface
std::unordered_map<UID, StorageServerInterface> storageServerInterfaces;
RatekeeperLimits normalLimits; RatekeeperLimits normalLimits;
RatekeeperLimits batchLimits; RatekeeperLimits batchLimits;
@ -163,7 +174,6 @@ class Ratekeeper {
void updateRate(RatekeeperLimits* limits); void updateRate(RatekeeperLimits* limits);
Future<Void> refreshStorageServerCommitCosts(); Future<Void> refreshStorageServerCommitCosts();
Future<Void> monitorServerListChange(PromiseStream<std::pair<UID, Optional<StorageServerInterface>>> serverChanges); Future<Void> monitorServerListChange(PromiseStream<std::pair<UID, Optional<StorageServerInterface>>> serverChanges);
Future<Void> trackEachStorageServer(FutureStream<std::pair<UID, Optional<StorageServerInterface>>> serverChanges);
// SOMEDAY: template trackStorageServerQueueInfo and trackTLogQueueInfo into one function // SOMEDAY: template trackStorageServerQueueInfo and trackTLogQueueInfo into one function
Future<Void> trackStorageServerQueueInfo(StorageServerInterface); Future<Void> trackStorageServerQueueInfo(StorageServerInterface);
@ -176,3 +186,5 @@ class Ratekeeper {
public: public:
static Future<Void> run(RatekeeperInterface rkInterf, Reference<AsyncVar<ServerDBInfo> const> dbInfo); static Future<Void> run(RatekeeperInterface rkInterf, Reference<AsyncVar<ServerDBInfo> const> dbInfo);
}; };
#endif // FDBSERVER_RATEKEEPER_H

View File

@ -51,6 +51,8 @@ struct RatekeeperInterface {
}; };
struct TransactionCommitCostEstimation { struct TransactionCommitCostEstimation {
// NOTE: If class variables are changed, counterparts in StorageServerInterface.h:UpdateCommitCostRequest should be
// updated too.
int opsSum = 0; int opsSum = 0;
uint64_t costSum = 0; uint64_t costSum = 0;

View File

@ -80,15 +80,18 @@ public:
Future<Void> monitorThrottlingChanges() override; Future<Void> monitorThrottlingChanges() override;
void addRequests(TransactionTag tag, int count) override; void addRequests(TransactionTag tag, int count) override;
uint64_t getThrottledTagChangeId() const override; uint64_t getThrottledTagChangeId() const override;
PrioritizedTransactionTagMap<ClientTagThrottleLimits> getClientRates() override;
int64_t autoThrottleCount() const override; int64_t autoThrottleCount() const override;
uint32_t busyReadTagCount() const override; uint32_t busyReadTagCount() const override;
uint32_t busyWriteTagCount() const override; uint32_t busyWriteTagCount() const override;
int64_t manualThrottleCount() const override; int64_t manualThrottleCount() const override;
bool isAutoThrottlingEnabled() const override; bool isAutoThrottlingEnabled() const override;
Future<Void> tryUpdateAutoThrottling(StorageQueueInfo const&) override;
// testing only Future<Void> tryUpdateAutoThrottling(StorageQueueInfo const&) override;
PrioritizedTransactionTagMap<ClientTagThrottleLimits> getClientRates() override;
// Testing only:
public: public:
void setQuota(TransactionTagRef, ThrottleApi::TagQuotaValue const&); void setQuota(TransactionTagRef, ThrottleApi::TagQuotaValue const&);
void removeQuota(TransactionTagRef);
}; };

View File

@ -286,9 +286,11 @@ struct ApiWorkload : TestWorkload {
minValueLength = getOption(options, LiteralStringRef("minValueLength"), 1); minValueLength = getOption(options, LiteralStringRef("minValueLength"), 1);
maxValueLength = getOption(options, LiteralStringRef("maxValueLength"), 10000); maxValueLength = getOption(options, LiteralStringRef("maxValueLength"), 10000);
useExtraDB = g_network->isSimulated() && g_simulator.extraDB != nullptr; useExtraDB = g_network->isSimulated() && !g_simulator.extraDatabases.empty();
if (useExtraDB) { if (useExtraDB) {
auto extraFile = makeReference<ClusterConnectionMemoryRecord>(*g_simulator.extraDB); ASSERT(g_simulator.extraDatabases.size() == 1);
auto extraFile =
makeReference<ClusterConnectionMemoryRecord>(ClusterConnectionString(g_simulator.extraDatabases[0]));
extraDB = Database::createDatabase(extraFile, -1); extraDB = Database::createDatabase(extraFile, -1);
} }
} }

View File

@ -0,0 +1,354 @@
/*
* MetaclusterConsistency.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.
#include "fdbclient/FDBOptions.g.h"
#include "flow/BooleanParam.h"
#if defined(NO_INTELLISENSE) && !defined(WORKLOADS_METACLUSTER_CONSISTENCY_ACTOR_G_H)
#define WORKLOADS_METACLUSTER_CONSISTENCY_ACTOR_G_H
#include "fdbserver/workloads/MetaclusterConsistency.actor.g.h"
#elif !defined(WORKLOADS_METACLUSTER_CONSISTENCY_ACTOR_H)
#define WORKLOADS_METACLUSTER_CONSISTENCY_ACTOR_H
#include "fdbclient/Metacluster.h"
#include "fdbclient/MetaclusterManagement.actor.h"
#include "fdbserver/workloads/TenantConsistency.actor.h"
#include "flow/actorcompiler.h" // This must be the last #include.
FDB_DECLARE_BOOLEAN_PARAM(AllowPartialMetaclusterOperations);
template <class DB>
class MetaclusterConsistencyCheck {
private:
Reference<DB> managementDb;
AllowPartialMetaclusterOperations allowPartialMetaclusterOperations = AllowPartialMetaclusterOperations::True;
struct ManagementClusterData {
Optional<MetaclusterRegistrationEntry> metaclusterRegistration;
std::map<ClusterName, DataClusterMetadata> dataClusters;
KeyBackedRangeResult<Tuple> clusterCapacityTuples;
KeyBackedRangeResult<std::pair<ClusterName, int64_t>> clusterTenantCounts;
KeyBackedRangeResult<Tuple> clusterTenantTuples;
KeyBackedRangeResult<Tuple> clusterTenantGroupTuples;
std::map<TenantName, TenantMapEntry> tenantMap;
KeyBackedRangeResult<std::pair<TenantGroupName, TenantGroupEntry>> tenantGroups;
std::map<ClusterName, std::set<TenantName>> clusterTenantMap;
std::map<ClusterName, std::set<TenantGroupName>> clusterTenantGroupMap;
int64_t tenantCount;
RangeResult systemTenantSubspaceKeys;
};
ManagementClusterData managementMetadata;
// Note: this check can only be run on metaclusters with a reasonable number of tenants, as should be
// the case with the current metacluster simulation workloads
static inline const int metaclusterMaxTenants = 10e6;
ACTOR static Future<Void> loadManagementClusterMetadata(MetaclusterConsistencyCheck* self) {
state Reference<typename DB::TransactionT> managementTr = self->managementDb->createTransaction();
state std::vector<std::pair<TenantName, TenantMapEntry>> tenantList;
loop {
try {
managementTr->setOption(FDBTransactionOptions::READ_SYSTEM_KEYS);
state typename transaction_future_type<typename DB::TransactionT, RangeResult>::type
systemTenantSubspaceKeysFuture = managementTr->getRange(prefixRange(TenantMetadata::subspace), 1);
wait(store(self->managementMetadata.metaclusterRegistration,
MetaclusterMetadata::metaclusterRegistration.get(managementTr)) &&
store(self->managementMetadata.dataClusters,
MetaclusterAPI::listClustersTransaction(
managementTr, ""_sr, "\xff\xff"_sr, CLIENT_KNOBS->MAX_DATA_CLUSTERS + 1)) &&
store(self->managementMetadata.clusterCapacityTuples,
MetaclusterAPI::ManagementClusterMetadata::clusterCapacityIndex.getRange(
managementTr, {}, {}, CLIENT_KNOBS->MAX_DATA_CLUSTERS)) &&
store(self->managementMetadata.clusterTenantCounts,
MetaclusterAPI::ManagementClusterMetadata::clusterTenantCount.getRange(
managementTr, {}, {}, CLIENT_KNOBS->MAX_DATA_CLUSTERS)) &&
store(self->managementMetadata.clusterTenantTuples,
MetaclusterAPI::ManagementClusterMetadata::clusterTenantIndex.getRange(
managementTr, {}, {}, metaclusterMaxTenants)) &&
store(self->managementMetadata.clusterTenantGroupTuples,
MetaclusterAPI::ManagementClusterMetadata::clusterTenantGroupIndex.getRange(
managementTr, {}, {}, metaclusterMaxTenants)) &&
store(self->managementMetadata.tenantCount,
MetaclusterAPI::ManagementClusterMetadata::tenantMetadata.tenantCount.getD(
managementTr, Snapshot::False, 0)) &&
store(tenantList,
MetaclusterAPI::listTenantsTransaction(
managementTr, ""_sr, "\xff\xff"_sr, metaclusterMaxTenants)) &&
store(self->managementMetadata.tenantGroups,
MetaclusterAPI::ManagementClusterMetadata::tenantMetadata.tenantGroupMap.getRange(
managementTr, {}, {}, metaclusterMaxTenants)) &&
store(self->managementMetadata.systemTenantSubspaceKeys,
safeThreadFutureToFuture(systemTenantSubspaceKeysFuture)));
break;
} catch (Error& e) {
wait(safeThreadFutureToFuture(managementTr->onError(e)));
}
}
self->managementMetadata.tenantMap = std::map<TenantName, TenantMapEntry>(tenantList.begin(), tenantList.end());
for (auto t : self->managementMetadata.clusterTenantTuples.results) {
ASSERT(t.size() == 2);
self->managementMetadata.clusterTenantMap[t.getString(0)].insert(t.getString(1));
}
for (auto t : self->managementMetadata.clusterTenantGroupTuples.results) {
ASSERT(t.size() == 2);
TenantGroupName tenantGroupName = t.getString(1);
self->managementMetadata.clusterTenantGroupMap[t.getString(0)].insert(tenantGroupName);
}
return Void();
}
void validateManagementCluster() {
ASSERT(managementMetadata.metaclusterRegistration.present());
ASSERT(managementMetadata.metaclusterRegistration.get().clusterType == ClusterType::METACLUSTER_MANAGEMENT);
ASSERT(managementMetadata.metaclusterRegistration.get().id ==
managementMetadata.metaclusterRegistration.get().metaclusterId &&
managementMetadata.metaclusterRegistration.get().name ==
managementMetadata.metaclusterRegistration.get().metaclusterName);
ASSERT(managementMetadata.dataClusters.size() <= CLIENT_KNOBS->MAX_DATA_CLUSTERS);
ASSERT(managementMetadata.tenantCount <= metaclusterMaxTenants);
ASSERT(managementMetadata.clusterCapacityTuples.results.size() <= managementMetadata.dataClusters.size() &&
!managementMetadata.clusterCapacityTuples.more);
ASSERT(managementMetadata.clusterTenantCounts.results.size() <= managementMetadata.dataClusters.size() &&
!managementMetadata.clusterTenantCounts.more);
ASSERT(managementMetadata.clusterTenantTuples.results.size() == managementMetadata.tenantCount &&
!managementMetadata.clusterTenantTuples.more);
ASSERT(managementMetadata.clusterTenantGroupTuples.results.size() <= managementMetadata.tenantCount &&
!managementMetadata.clusterTenantGroupTuples.more);
ASSERT(managementMetadata.tenantMap.size() == managementMetadata.tenantCount);
ASSERT(managementMetadata.tenantGroups.results.size() <= managementMetadata.tenantCount &&
!managementMetadata.tenantGroups.more);
ASSERT(managementMetadata.clusterTenantGroupTuples.results.size() ==
managementMetadata.tenantGroups.results.size());
// Parse the cluster capacity index. Check that no cluster is represented in the index more than once.
std::map<ClusterName, int64_t> clusterAllocatedMap;
for (auto t : managementMetadata.clusterCapacityTuples.results) {
ASSERT(t.size() == 2);
auto result = clusterAllocatedMap.emplace(t.getString(1), t.getInt(0));
ASSERT(result.second);
}
// Validate various properties for each data cluster
int numFoundInAllocatedMap = 0;
int numFoundInTenantGroupMap = 0;
for (auto [clusterName, clusterMetadata] : managementMetadata.dataClusters) {
// If the cluster has capacity, it should be in the capacity index and have the correct count of
// allocated tenants stored there
auto allocatedItr = clusterAllocatedMap.find(clusterName);
if (!clusterMetadata.entry.hasCapacity()) {
ASSERT(allocatedItr == clusterAllocatedMap.end());
} else {
ASSERT(allocatedItr->second == clusterMetadata.entry.allocated.numTenantGroups);
++numFoundInAllocatedMap;
}
// Check that the number of tenant groups in the cluster is smaller than the allocated number of tenant
// groups.
auto tenantGroupItr = managementMetadata.clusterTenantGroupMap.find(clusterName);
if (tenantGroupItr != managementMetadata.clusterTenantGroupMap.end()) {
ASSERT(tenantGroupItr->second.size() <= clusterMetadata.entry.allocated.numTenantGroups);
++numFoundInTenantGroupMap;
}
}
// Check that we exhausted the cluster capacity index and the cluster tenant group index
ASSERT(numFoundInAllocatedMap == clusterAllocatedMap.size());
ASSERT(numFoundInTenantGroupMap == managementMetadata.clusterTenantGroupMap.size());
// Check that our cluster tenant counters match the number of tenants in the cluster index
std::map<ClusterName, int64_t> countsMap(managementMetadata.clusterTenantCounts.results.begin(),
managementMetadata.clusterTenantCounts.results.end());
for (auto [cluster, clusterTenants] : managementMetadata.clusterTenantMap) {
auto itr = countsMap.find(cluster);
ASSERT((clusterTenants.empty() && itr == countsMap.end()) || itr->second == clusterTenants.size());
}
// Iterate through all tenants and verify related metadata
std::map<ClusterName, int> clusterAllocated;
std::set<TenantGroupName> processedTenantGroups;
for (auto [name, entry] : managementMetadata.tenantMap) {
ASSERT(entry.assignedCluster.present());
// Each tenant should be assigned to the same cluster where it is stored in the cluster tenant index
auto clusterItr = managementMetadata.clusterTenantMap.find(entry.assignedCluster.get());
ASSERT(clusterItr != managementMetadata.clusterTenantMap.end());
ASSERT(clusterItr->second.count(name));
if (entry.tenantGroup.present()) {
// Count the number of tenant groups allocated in each cluster
if (processedTenantGroups.insert(entry.tenantGroup.get()).second) {
++clusterAllocated[entry.assignedCluster.get()];
}
// The tenant group should be stored in the same cluster where it is stored in the cluster tenant
// group index
auto clusterTenantGroupItr = managementMetadata.clusterTenantGroupMap.find(entry.assignedCluster.get());
ASSERT(clusterTenantGroupItr != managementMetadata.clusterTenantMap.end());
ASSERT(clusterTenantGroupItr->second.count(entry.tenantGroup.get()));
} else {
// Track the actual tenant group allocation per cluster (a tenant with no group counts against the
// allocation)
++clusterAllocated[entry.assignedCluster.get()];
}
}
// The actual allocation for each cluster should match what is stored in the cluster metadata
for (auto [name, allocated] : clusterAllocated) {
auto itr = managementMetadata.dataClusters.find(name);
ASSERT(itr != managementMetadata.dataClusters.end());
ASSERT(allocated == itr->second.entry.allocated.numTenantGroups);
}
// Each tenant group in the tenant group map should be present in the cluster tenant group map
// and have the correct cluster assigned to it.
for (auto [name, entry] : managementMetadata.tenantGroups.results) {
ASSERT(entry.assignedCluster.present());
auto clusterItr = managementMetadata.clusterTenantGroupMap.find(entry.assignedCluster.get());
ASSERT(clusterItr->second.count(name));
}
// We should not be storing any data in the `\xff` tenant subspace.
ASSERT(managementMetadata.systemTenantSubspaceKeys.empty());
}
ACTOR static Future<Void> validateDataCluster(MetaclusterConsistencyCheck* self,
ClusterName clusterName,
DataClusterMetadata clusterMetadata) {
state Reference<IDatabase> dataDb = wait(MetaclusterAPI::openDatabase(clusterMetadata.connectionString));
state Reference<ITransaction> dataTr = dataDb->createTransaction();
state Optional<MetaclusterRegistrationEntry> dataClusterRegistration;
state std::vector<std::pair<TenantName, TenantMapEntry>> dataClusterTenantList;
state KeyBackedRangeResult<std::pair<TenantGroupName, TenantGroupEntry>> dataClusterTenantGroups;
state TenantConsistencyCheck<IDatabase> tenantConsistencyCheck(dataDb);
wait(tenantConsistencyCheck.run());
loop {
try {
dataTr->setOption(FDBTransactionOptions::READ_SYSTEM_KEYS);
wait(store(dataClusterRegistration, MetaclusterMetadata::metaclusterRegistration.get(dataTr)) &&
store(dataClusterTenantList,
TenantAPI::listTenantsTransaction(
dataTr, ""_sr, "\xff\xff"_sr, CLIENT_KNOBS->MAX_TENANTS_PER_CLUSTER + 1)) &&
store(dataClusterTenantGroups,
TenantMetadata::tenantGroupMap.getRange(
dataTr, {}, {}, CLIENT_KNOBS->MAX_TENANTS_PER_CLUSTER + 1)));
break;
} catch (Error& e) {
wait(safeThreadFutureToFuture(dataTr->onError(e)));
}
}
state std::map<TenantName, TenantMapEntry> dataClusterTenantMap(dataClusterTenantList.begin(),
dataClusterTenantList.end());
ASSERT(dataClusterRegistration.present());
ASSERT(dataClusterRegistration.get().clusterType == ClusterType::METACLUSTER_DATA);
ASSERT(dataClusterRegistration.get().matches(self->managementMetadata.metaclusterRegistration.get()));
ASSERT(dataClusterRegistration.get().name == clusterName);
ASSERT(dataClusterRegistration.get().id == clusterMetadata.entry.id);
auto& expectedTenants = self->managementMetadata.clusterTenantMap[clusterName];
if (!self->allowPartialMetaclusterOperations) {
ASSERT(dataClusterTenantMap.size() == expectedTenants.size());
} else {
ASSERT(dataClusterTenantMap.size() <= expectedTenants.size());
for (auto tenantName : expectedTenants) {
if (!dataClusterTenantMap.count(tenantName)) {
TenantMapEntry const& metaclusterEntry = self->managementMetadata.tenantMap[tenantName];
ASSERT(metaclusterEntry.tenantState == TenantState::REGISTERING ||
metaclusterEntry.tenantState == TenantState::REMOVING);
}
}
}
for (auto [name, entry] : dataClusterTenantMap) {
ASSERT(expectedTenants.count(name));
TenantMapEntry const& metaclusterEntry = self->managementMetadata.tenantMap[name];
ASSERT(!entry.assignedCluster.present());
ASSERT(entry.id == metaclusterEntry.id);
ASSERT(entry.encrypted == metaclusterEntry.encrypted);
ASSERT(entry.tenantState == TenantState::READY);
ASSERT(self->allowPartialMetaclusterOperations || metaclusterEntry.tenantState == TenantState::READY);
if (metaclusterEntry.tenantState != TenantState::UPDATING_CONFIGURATION &&
metaclusterEntry.tenantState != TenantState::REMOVING) {
ASSERT(entry.configurationSequenceNum == metaclusterEntry.configurationSequenceNum);
} else {
ASSERT(entry.configurationSequenceNum <= metaclusterEntry.configurationSequenceNum);
}
if (entry.configurationSequenceNum == metaclusterEntry.configurationSequenceNum) {
ASSERT(entry.tenantGroup == metaclusterEntry.tenantGroup);
}
}
auto& expectedTenantGroups = self->managementMetadata.clusterTenantGroupMap[clusterName];
ASSERT(dataClusterTenantGroups.results.size() == expectedTenantGroups.size());
for (auto [name, entry] : dataClusterTenantGroups.results) {
ASSERT(expectedTenantGroups.count(name));
ASSERT(!entry.assignedCluster.present());
}
return Void();
}
ACTOR static Future<Void> run(MetaclusterConsistencyCheck* self) {
state TenantConsistencyCheck<DB> managementTenantConsistencyCheck(self->managementDb);
wait(managementTenantConsistencyCheck.run());
wait(loadManagementClusterMetadata(self));
self->validateManagementCluster();
state std::vector<Future<Void>> dataClusterChecks;
state std::map<ClusterName, DataClusterMetadata>::iterator dataClusterItr;
for (auto [clusterName, clusterMetadata] : self->managementMetadata.dataClusters) {
dataClusterChecks.push_back(validateDataCluster(self, clusterName, clusterMetadata));
}
wait(waitForAll(dataClusterChecks));
return Void();
}
public:
MetaclusterConsistencyCheck() {}
MetaclusterConsistencyCheck(Reference<DB> managementDb,
AllowPartialMetaclusterOperations allowPartialMetaclusterOperations)
: managementDb(managementDb), allowPartialMetaclusterOperations(allowPartialMetaclusterOperations) {}
Future<Void> run() { return run(this); }
};
#endif

View File

@ -0,0 +1,196 @@
/*
* TenantConsistency.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.
#include "fdbclient/FDBOptions.g.h"
#include "fdbclient/KeyBackedTypes.h"
#include "flow/BooleanParam.h"
#if defined(NO_INTELLISENSE) && !defined(WORKLOADS_TENANT_CONSISTENCY_ACTOR_G_H)
#define WORKLOADS_TENANT_CONSISTENCY_ACTOR_G_H
#include "fdbserver/workloads/TenantConsistency.actor.g.h"
#elif !defined(WORKLOADS_TENANT_CONSISTENCY_ACTOR_H)
#define WORKLOADS_TENANT_CONSISTENCY_ACTOR_H
#include "fdbclient/Metacluster.h"
#include "fdbclient/MetaclusterManagement.actor.h"
#include "fdbclient/Tenant.h"
#include "fdbclient/TenantManagement.actor.h"
#include "flow/actorcompiler.h" // This must be the last #include.
template <class DB>
class TenantConsistencyCheck {
private:
Reference<DB> db;
struct TenantData {
Optional<MetaclusterRegistrationEntry> metaclusterRegistration;
std::map<TenantName, TenantMapEntry> tenantMap;
int64_t lastTenantId;
int64_t tenantCount;
std::set<int64_t> tenantTombstones;
Optional<TenantTombstoneCleanupData> tombstoneCleanupData;
std::map<TenantGroupName, TenantGroupEntry> tenantGroupMap;
std::map<TenantGroupName, std::set<TenantName>> tenantGroupIndex;
std::set<TenantName> tenantsInTenantGroupIndex;
ClusterType clusterType;
};
TenantData metadata;
// Note: this check can only be run on metaclusters with a reasonable number of tenants, as should be
// the case with the current metacluster simulation workloads
static inline const int metaclusterMaxTenants = 10e6;
ACTOR static Future<Void> loadTenantMetadata(TenantConsistencyCheck* self) {
state Reference<typename DB::TransactionT> tr = self->db->createTransaction();
state KeyBackedRangeResult<std::pair<TenantName, TenantMapEntry>> tenantList;
state KeyBackedRangeResult<int64_t> tenantTombstoneList;
state KeyBackedRangeResult<std::pair<TenantGroupName, TenantGroupEntry>> tenantGroupList;
state KeyBackedRangeResult<Tuple> tenantGroupTenantTuples;
state TenantMetadataSpecification* tenantMetadata;
loop {
try {
tr->setOption(FDBTransactionOptions::READ_SYSTEM_KEYS);
wait(store(self->metadata.metaclusterRegistration,
MetaclusterMetadata::metaclusterRegistration.get(tr)));
self->metadata.clusterType = self->metadata.metaclusterRegistration.present()
? self->metadata.metaclusterRegistration.get().clusterType
: ClusterType::STANDALONE;
if (self->metadata.clusterType == ClusterType::METACLUSTER_MANAGEMENT) {
tenantMetadata = &MetaclusterAPI::ManagementClusterMetadata::tenantMetadata;
} else {
tenantMetadata = &TenantMetadata::instance;
}
wait(
store(tenantList, tenantMetadata->tenantMap.getRange(tr, {}, {}, metaclusterMaxTenants)) &&
store(self->metadata.lastTenantId, tenantMetadata->lastTenantId.getD(tr, Snapshot::False, -1)) &&
store(self->metadata.tenantCount, tenantMetadata->tenantCount.getD(tr, Snapshot::False, 0)) &&
store(tenantTombstoneList,
tenantMetadata->tenantTombstones.getRange(tr, {}, {}, metaclusterMaxTenants)) &&
store(self->metadata.tombstoneCleanupData, tenantMetadata->tombstoneCleanupData.get(tr)) &&
store(tenantGroupTenantTuples,
tenantMetadata->tenantGroupTenantIndex.getRange(tr, {}, {}, metaclusterMaxTenants)) &&
store(tenantGroupList, tenantMetadata->tenantGroupMap.getRange(tr, {}, {}, metaclusterMaxTenants)));
break;
} catch (Error& e) {
wait(safeThreadFutureToFuture(tr->onError(e)));
}
}
ASSERT(!tenantList.more);
self->metadata.tenantMap =
std::map<TenantName, TenantMapEntry>(tenantList.results.begin(), tenantList.results.end());
self->metadata.tenantTombstones =
std::set<int64_t>(tenantTombstoneList.results.begin(), tenantTombstoneList.results.end());
self->metadata.tenantGroupMap =
std::map<TenantGroupName, TenantGroupEntry>(tenantGroupList.results.begin(), tenantGroupList.results.end());
for (auto t : tenantGroupTenantTuples.results) {
ASSERT(t.size() == 2);
TenantGroupName tenantGroupName = t.getString(0);
TenantName tenantName = t.getString(1);
ASSERT(self->metadata.tenantGroupMap.count(tenantGroupName));
ASSERT(self->metadata.tenantMap.count(tenantName));
self->metadata.tenantGroupIndex[tenantGroupName].insert(tenantName);
ASSERT(self->metadata.tenantsInTenantGroupIndex.insert(tenantName).second);
}
ASSERT(self->metadata.tenantGroupIndex.size() == self->metadata.tenantGroupMap.size());
return Void();
}
void validateTenantMetadata() {
if (metadata.clusterType == ClusterType::METACLUSTER_MANAGEMENT) {
ASSERT(metadata.tenantMap.size() <= metaclusterMaxTenants);
} else {
ASSERT(metadata.tenantMap.size() <= CLIENT_KNOBS->MAX_TENANTS_PER_CLUSTER);
}
ASSERT(metadata.tenantMap.size() == metadata.tenantCount);
std::set<int64_t> tenantIds;
for (auto [tenantName, tenantMapEntry] : metadata.tenantMap) {
if (metadata.clusterType != ClusterType::METACLUSTER_DATA) {
ASSERT(tenantMapEntry.id <= metadata.lastTenantId);
}
ASSERT(tenantIds.insert(tenantMapEntry.id).second);
ASSERT(!metadata.tenantTombstones.count(tenantMapEntry.id));
if (tenantMapEntry.tenantGroup.present()) {
auto tenantGroupMapItr = metadata.tenantGroupMap.find(tenantMapEntry.tenantGroup.get());
ASSERT(tenantGroupMapItr != metadata.tenantGroupMap.end());
ASSERT(tenantMapEntry.assignedCluster == tenantGroupMapItr->second.assignedCluster);
ASSERT(metadata.tenantGroupIndex[tenantMapEntry.tenantGroup.get()].count(tenantName));
} else {
ASSERT(!metadata.tenantsInTenantGroupIndex.count(tenantName));
}
if (metadata.clusterType == ClusterType::METACLUSTER_MANAGEMENT) {
ASSERT(tenantMapEntry.assignedCluster.present());
} else {
ASSERT(tenantMapEntry.tenantState == TenantState::READY);
ASSERT(!tenantMapEntry.assignedCluster.present());
}
}
}
// Check that the tenant tombstones are properly cleaned up and only present on a metacluster data cluster
void checkTenantTombstones() {
if (metadata.clusterType == ClusterType::METACLUSTER_DATA) {
if (!metadata.tombstoneCleanupData.present()) {
ASSERT(metadata.tenantTombstones.empty());
}
if (!metadata.tenantTombstones.empty()) {
ASSERT(*metadata.tenantTombstones.begin() >
metadata.tombstoneCleanupData.get().tombstonesErasedThrough);
}
} else {
ASSERT(metadata.tenantTombstones.empty() && !metadata.tombstoneCleanupData.present());
}
}
ACTOR static Future<Void> run(TenantConsistencyCheck* self) {
wait(loadTenantMetadata(self));
self->validateTenantMetadata();
self->checkTenantTombstones();
return Void();
}
public:
TenantConsistencyCheck() {}
TenantConsistencyCheck(Reference<DB> db) : db(db) {}
Future<Void> run() { return run(this); }
};
#endif

View File

@ -69,6 +69,7 @@
#include "fdbserver/MoveKeys.actor.h" #include "fdbserver/MoveKeys.actor.h"
#include "fdbserver/MutationTracking.h" #include "fdbserver/MutationTracking.h"
#include "fdbserver/OTELSpanContextMessage.h" #include "fdbserver/OTELSpanContextMessage.h"
#include "fdbserver/Ratekeeper.h"
#include "fdbserver/RecoveryState.h" #include "fdbserver/RecoveryState.h"
#include "fdbserver/RocksDBCheckpointUtils.actor.h" #include "fdbserver/RocksDBCheckpointUtils.actor.h"
#include "fdbserver/ServerCheckpoint.actor.h" #include "fdbserver/ServerCheckpoint.actor.h"
@ -101,6 +102,22 @@
#define SHORT_CIRCUT_ACTUAL_STORAGE 0 #define SHORT_CIRCUT_ACTUAL_STORAGE 0
namespace { namespace {
enum ChangeServerKeysContext { CSK_UPDATE, CSK_RESTORE, CSK_ASSIGN_EMPTY };
std::string changeServerKeysContextName(const ChangeServerKeysContext& context) {
switch (context) {
case CSK_UPDATE:
return "Update";
case CSK_RESTORE:
return "Restore";
case CSK_ASSIGN_EMPTY:
return "AssignEmpty";
default:
ASSERT(false);
}
return "UnknownContext";
}
bool canReplyWith(Error e) { bool canReplyWith(Error e) {
switch (e.code()) { switch (e.code()) {
case error_code_transaction_too_old: case error_code_transaction_too_old:
@ -619,6 +636,17 @@ public:
: key(key), value(value), version(version), tags(tags), debugID(debugID) {} : key(key), value(value), version(version), tags(tags), debugID(debugID) {}
}; };
struct BusiestWriteTagContext {
const std::string busiestWriteTagTrackingKey;
UID ratekeeperID;
Reference<EventCacheHolder> busiestWriteTagEventHolder;
double lastUpdateTime;
BusiestWriteTagContext(const UID& thisServerID)
: busiestWriteTagTrackingKey(thisServerID.toString() + "/BusiestWriteTag"), ratekeeperID(UID()),
busiestWriteTagEventHolder(makeReference<EventCacheHolder>(busiestWriteTagTrackingKey)), lastUpdateTime(-1) {}
};
struct StorageServer { struct StorageServer {
typedef VersionedMap<KeyRef, ValueOrClearToRef> VersionedData; typedef VersionedMap<KeyRef, ValueOrClearToRef> VersionedData;
@ -1017,6 +1045,7 @@ public:
} }
TransactionTagCounter transactionTagCounter; TransactionTagCounter transactionTagCounter;
BusiestWriteTagContext busiestWriteTagContext;
Optional<LatencyBandConfig> latencyBandConfig; Optional<LatencyBandConfig> latencyBandConfig;
@ -1225,7 +1254,8 @@ public:
serveFetchCheckpointParallelismLock(SERVER_KNOBS->SERVE_FETCH_CHECKPOINT_PARALLELISM), serveFetchCheckpointParallelismLock(SERVER_KNOBS->SERVE_FETCH_CHECKPOINT_PARALLELISM),
instanceID(deterministicRandom()->randomUniqueID().first()), shuttingDown(false), behind(false), instanceID(deterministicRandom()->randomUniqueID().first()), shuttingDown(false), behind(false),
versionBehind(false), debug_inApplyUpdate(false), debug_lastValidateTime(0), lastBytesInputEBrake(0), versionBehind(false), debug_inApplyUpdate(false), debug_lastValidateTime(0), lastBytesInputEBrake(0),
lastDurableVersionEBrake(0), maxQueryQueue(0), transactionTagCounter(ssi.id()), counters(this), lastDurableVersionEBrake(0), maxQueryQueue(0), transactionTagCounter(ssi.id()),
busiestWriteTagContext(ssi.id()), counters(this),
storageServerSourceTLogIDEventHolder( storageServerSourceTLogIDEventHolder(
makeReference<EventCacheHolder>(ssi.id().toString() + "/StorageServerSourceTLogID")) { makeReference<EventCacheHolder>(ssi.id().toString() + "/StorageServerSourceTLogID")) {
version.initMetric(LiteralStringRef("StorageServer.Version"), counters.cc.id); version.initMetric(LiteralStringRef("StorageServer.Version"), counters.cc.id);
@ -1235,7 +1265,7 @@ public:
newestAvailableVersion.insert(allKeys, invalidVersion); newestAvailableVersion.insert(allKeys, invalidVersion);
newestDirtyVersion.insert(allKeys, invalidVersion); newestDirtyVersion.insert(allKeys, invalidVersion);
if (CLIENT_KNOBS->SHARD_ENCODE_LOCATION_METADATA && if (SERVER_KNOBS->SHARD_ENCODE_LOCATION_METADATA &&
(SERVER_KNOBS->STORAGE_SERVER_SHARD_AWARE || storage->shardAware())) { (SERVER_KNOBS->STORAGE_SERVER_SHARD_AWARE || storage->shardAware())) {
addShard(ShardInfo::newShard(this, StorageServerShard::notAssigned(allKeys))); addShard(ShardInfo::newShard(this, StorageServerShard::notAssigned(allKeys)));
} else { } else {
@ -2267,6 +2297,11 @@ ACTOR Future<Void> overlappingChangeFeedsQ(StorageServer* data, OverlappingChang
} }
rangeIds[it->id] = std::tuple(it->range, it->emptyVersion, stopVersion, it->metadataVersion); rangeIds[it->id] = std::tuple(it->range, it->emptyVersion, stopVersion, it->metadataVersion);
} else if (it->destroyed && it->metadataVersion > metadataWaitVersion) {
// if we communicate the lack of a change feed because it's destroying, ensure the feed destroy isn't
// rolled back first
CODE_PROBE(true, "Overlapping Change Feeds ensuring destroy isn't rolled back");
metadataWaitVersion = it->metadataVersion;
} }
} }
} }
@ -2289,8 +2324,8 @@ ACTOR Future<Void> overlappingChangeFeedsQ(StorageServer* data, OverlappingChang
} }
// Make sure all of the metadata we are sending won't get rolled back // Make sure all of the metadata we are sending won't get rolled back
if (metadataWaitVersion != invalidVersion && metadataWaitVersion > data->knownCommittedVersion.get()) { if (metadataWaitVersion != invalidVersion && metadataWaitVersion > data->desiredOldestVersion.get()) {
CODE_PROBE(true, "overlapping change feeds waiting for metadata version to be committed"); CODE_PROBE(true, "overlapping change feeds waiting for metadata version to be safe from rollback");
wait(data->desiredOldestVersion.whenAtLeast(metadataWaitVersion)); wait(data->desiredOldestVersion.whenAtLeast(metadataWaitVersion));
} }
req.reply.send(reply); req.reply.send(reply);
@ -2491,6 +2526,7 @@ ACTOR Future<std::pair<ChangeFeedStreamReply, bool>> getChangeFeedMutations(Stor
state Version dequeVersion = data->version.get(); state Version dequeVersion = data->version.get();
state Version dequeKnownCommit = data->knownCommittedVersion.get(); state Version dequeKnownCommit = data->knownCommittedVersion.get();
state Version emptyVersion = feedInfo->emptyVersion; state Version emptyVersion = feedInfo->emptyVersion;
state Version durableValidationVersion = std::min(data->durableVersion.get(), feedInfo->durableFetchVersion.get());
Version fetchStorageVersion = std::max(feedInfo->fetchVersion, feedInfo->durableFetchVersion.get()); Version fetchStorageVersion = std::max(feedInfo->fetchVersion, feedInfo->durableFetchVersion.get());
if (DEBUG_CF_TRACE) { if (DEBUG_CF_TRACE) {
@ -2507,7 +2543,8 @@ ACTOR Future<std::pair<ChangeFeedStreamReply, bool>> getChangeFeedMutations(Stor
.detail("DurableVersion", feedInfo->durableVersion) .detail("DurableVersion", feedInfo->durableVersion)
.detail("FetchStorageVersion", fetchStorageVersion) .detail("FetchStorageVersion", fetchStorageVersion)
.detail("FetchVersion", feedInfo->fetchVersion) .detail("FetchVersion", feedInfo->fetchVersion)
.detail("DurableFetchVersion", feedInfo->durableFetchVersion.get()); .detail("DurableFetchVersion", feedInfo->durableFetchVersion.get())
.detail("DurableValidationVersion", durableValidationVersion);
} }
if (req.end > emptyVersion + 1) { if (req.end > emptyVersion + 1) {
@ -2622,18 +2659,19 @@ ACTOR Future<std::pair<ChangeFeedStreamReply, bool>> getChangeFeedMutations(Stor
} }
} else if (memoryVerifyIdx < memoryReply.mutations.size() && } else if (memoryVerifyIdx < memoryReply.mutations.size() &&
version == memoryReply.mutations[memoryVerifyIdx].version) { version == memoryReply.mutations[memoryVerifyIdx].version) {
if (version > feedInfo->storageVersion && version > feedInfo->fetchVersion) { if (version > durableValidationVersion) {
// Another validation case - feed was popped, data was fetched, fetched data was persisted but pop // Another validation case - feed was popped, data was fetched, fetched data was persisted but pop
// wasn't yet, then SS restarted. Now SS has the data without the popped version. This looks wrong // wasn't yet, then SS restarted. Now SS has the data without the popped version. This looks wrong
// here but is fine. // here but is fine.
memoryVerifyIdx++; memoryVerifyIdx++;
} else { } else {
fmt::print( fmt::print("ERROR: SS {0} CF {1} SQ {2} has mutation at {3} in memory but all filtered out on "
"ERROR: SS {0} CF {1} SQ {2} has mutation at {3} in memory but all filtered out on disk!\n", "disk! (durable validation = {4})\n",
data->thisServerID.toString().substr(0, 4), data->thisServerID.toString().substr(0, 4),
req.rangeID.printable().substr(0, 6), req.rangeID.printable().substr(0, 6),
streamUID.toString().substr(0, 8), streamUID.toString().substr(0, 8),
version); version,
durableValidationVersion);
fmt::print(" Memory: ({})\n", memoryReply.mutations[memoryVerifyIdx].mutations.size()); fmt::print(" Memory: ({})\n", memoryReply.mutations[memoryVerifyIdx].mutations.size());
for (auto& it : memoryReply.mutations[memoryVerifyIdx].mutations) { for (auto& it : memoryReply.mutations[memoryVerifyIdx].mutations) {
@ -2651,7 +2689,7 @@ ACTOR Future<std::pair<ChangeFeedStreamReply, bool>> getChangeFeedMutations(Stor
fmt::print(" {} - {}\n", it.param1.printable().c_str(), it.param2.printable().c_str()); fmt::print(" {} - {}\n", it.param1.printable().c_str(), it.param2.printable().c_str());
} }
} }
ASSERT(false); ASSERT_WE_THINK(false);
} }
} }
remainingDurableBytes -= remainingDurableBytes -=
@ -6737,9 +6775,6 @@ void ShardInfo::addMutation(Version version, bool fromFetch, MutationRef const&
} }
} }
enum ChangeServerKeysContext { CSK_UPDATE, CSK_RESTORE, CSK_ASSIGN_EMPTY };
const char* changeServerKeysContextName[] = { "Update", "Restore" };
ACTOR Future<Void> restoreShards(StorageServer* data, ACTOR Future<Void> restoreShards(StorageServer* data,
Version version, Version version,
RangeResult storageShards, RangeResult storageShards,
@ -6938,7 +6973,7 @@ void changeServerKeys(StorageServer* data,
// .detail("KeyEnd", keys.end) // .detail("KeyEnd", keys.end)
// .detail("NowAssigned", nowAssigned) // .detail("NowAssigned", nowAssigned)
// .detail("Version", version) // .detail("Version", version)
// .detail("Context", changeServerKeysContextName[(int)context]); // .detail("Context", changeServerKeysContextName(context));
validate(data); validate(data);
// TODO(alexmiller): Figure out how to selectively enable spammy data distribution events. // TODO(alexmiller): Figure out how to selectively enable spammy data distribution events.
@ -7085,7 +7120,7 @@ void changeServerKeysWithPhysicalShards(StorageServer* data,
.detail("Range", keys) .detail("Range", keys)
.detail("NowAssigned", nowAssigned) .detail("NowAssigned", nowAssigned)
.detail("Version", version) .detail("Version", version)
.detail("Context", changeServerKeysContextName[(int)context]); .detail("Context", changeServerKeysContextName(context));
validate(data); validate(data);
@ -8594,10 +8629,19 @@ ACTOR Future<Void> updateStorage(StorageServer* data) {
.detail("OldestRemoveKVSRangesVersion", data->pendingRemoveRanges.begin()->first); .detail("OldestRemoveKVSRangesVersion", data->pendingRemoveRanges.begin()->first);
ASSERT(newOldestVersion <= data->pendingRemoveRanges.begin()->first); ASSERT(newOldestVersion <= data->pendingRemoveRanges.begin()->first);
if (newOldestVersion == data->pendingRemoveRanges.begin()->first) { if (newOldestVersion == data->pendingRemoveRanges.begin()->first) {
state std::vector<std::string> emptyShardIds;
for (const auto& range : data->pendingRemoveRanges.begin()->second) { for (const auto& range : data->pendingRemoveRanges.begin()->second) {
data->storage.removeRange(range); auto ids = data->storage.removeRange(range);
emptyShardIds.insert(emptyShardIds.end(), ids.begin(), ids.end());
TraceEvent(SevVerbose, "RemoveKVSRange", data->thisServerID).detail("Range", range); TraceEvent(SevVerbose, "RemoveKVSRange", data->thisServerID).detail("Range", range);
} }
if (emptyShardIds.size() > 0) {
state double start = now();
wait(data->storage.cleanUpShardsIfNeeded(emptyShardIds));
TraceEvent(SevInfo, "RemoveEmptyPhysicalShards", data->thisServerID)
.detail("NumShards", emptyShardIds.size())
.detail("TimeSpent", now() - start);
}
data->pendingRemoveRanges.erase(data->pendingRemoveRanges.begin()); data->pendingRemoveRanges.erase(data->pendingRemoveRanges.begin());
} }
removeKVSRanges = false; removeKVSRanges = false;
@ -10068,6 +10112,35 @@ ACTOR Future<Void> storageServerCore(StorageServer* self, StorageServerInterface
when(FetchCheckpointRequest req = waitNext(ssi.fetchCheckpoint.getFuture())) { when(FetchCheckpointRequest req = waitNext(ssi.fetchCheckpoint.getFuture())) {
self->actors.add(fetchCheckpointQ(self, req)); self->actors.add(fetchCheckpointQ(self, req));
} }
when(UpdateCommitCostRequest req = waitNext(ssi.updateCommitCostRequest.getFuture())) {
// Ratekeeper might change with a new ID. In this case, always accept the data.
if (req.ratekeeperID != self->busiestWriteTagContext.ratekeeperID) {
TraceEvent("RatekeeperIDChange")
.detail("OldID", self->busiestWriteTagContext.ratekeeperID)
.detail("OldLastUpdateTime", self->busiestWriteTagContext.lastUpdateTime)
.detail("NewID", req.ratekeeperID)
.detail("LastUpdateTime", req.postTime);
self->busiestWriteTagContext.ratekeeperID = req.ratekeeperID;
self->busiestWriteTagContext.lastUpdateTime = -1;
}
// In case we received an old request/duplicate request, due to, e.g. network problem
ASSERT(req.postTime > 0);
if (req.postTime < self->busiestWriteTagContext.lastUpdateTime) {
continue;
}
self->busiestWriteTagContext.lastUpdateTime = req.postTime;
TraceEvent("BusiestWriteTag", self->thisServerID)
.detail("Elapsed", req.elapsed)
.detail("Tag", printable(req.busiestTag))
.detail("TagOps", req.opsSum)
.detail("TagCost", req.costSum)
.detail("TotalCost", req.totalWriteCosts)
.detail("Reported", req.reported)
.trackLatest(self->busiestWriteTagContext.busiestWriteTagTrackingKey);
req.reply.send(Void());
}
when(FetchCheckpointKeyValuesRequest req = waitNext(ssi.fetchCheckpointKeyValues.getFuture())) { when(FetchCheckpointKeyValuesRequest req = waitNext(ssi.fetchCheckpointKeyValues.getFuture())) {
self->actors.add(fetchCheckpointKeyValuesQ(self, req)); self->actors.add(fetchCheckpointKeyValuesQ(self, req));
} }
@ -10345,7 +10418,7 @@ ACTOR Future<Void> storageServer(IKeyValueStore* persistentData,
Reference<AsyncVar<ServerDBInfo> const> db, Reference<AsyncVar<ServerDBInfo> const> db,
std::string folder) { std::string folder) {
state StorageServer self(persistentData, db, ssi); state StorageServer self(persistentData, db, ssi);
self.shardAware = CLIENT_KNOBS->SHARD_ENCODE_LOCATION_METADATA && self.shardAware = SERVER_KNOBS->SHARD_ENCODE_LOCATION_METADATA &&
(SERVER_KNOBS->STORAGE_SERVER_SHARD_AWARE || persistentData->shardAware()); (SERVER_KNOBS->STORAGE_SERVER_SHARD_AWARE || persistentData->shardAware());
state Future<Void> ssCore; state Future<Void> ssCore;
self.clusterId.send(clusterId); self.clusterId.send(clusterId);

View File

@ -1146,7 +1146,10 @@ ACTOR Future<bool> runTest(Database cx,
std::map<std::string, std::function<void(const std::string&)>> testSpecGlobalKeys = { std::map<std::string, std::function<void(const std::string&)>> testSpecGlobalKeys = {
// These are read by SimulatedCluster and used before testers exist. Thus, they must // These are read by SimulatedCluster and used before testers exist. Thus, they must
// be recognized and accepted, but there's no point in placing them into a testSpec. // be recognized and accepted, but there's no point in placing them into a testSpec.
{ "extraDB", [](const std::string& value) { TraceEvent("TestParserTest").detail("ParsedExtraDB", ""); } }, { "extraDatabaseMode",
[](const std::string& value) { TraceEvent("TestParserTest").detail("ParsedExtraDatabaseMode", ""); } },
{ "extraDatabaseCount",
[](const std::string& value) { TraceEvent("TestParserTest").detail("ParsedExtraDatabaseCount", ""); } },
{ "configureLocked", { "configureLocked",
[](const std::string& value) { TraceEvent("TestParserTest").detail("ParsedConfigureLocked", ""); } }, [](const std::string& value) { TraceEvent("TestParserTest").detail("ParsedConfigureLocked", ""); } },
{ "minimumReplication", { "minimumReplication",

View File

@ -39,7 +39,9 @@ struct AtomicSwitchoverWorkload : TestWorkload {
backupRanges.push_back_deep(backupRanges.arena(), normalKeys); backupRanges.push_back_deep(backupRanges.arena(), normalKeys);
auto extraFile = makeReference<ClusterConnectionMemoryRecord>(*g_simulator.extraDB); ASSERT(g_simulator.extraDatabases.size() == 1);
auto extraFile =
makeReference<ClusterConnectionMemoryRecord>(ClusterConnectionString(g_simulator.extraDatabases[0]));
extraDB = Database::createDatabase(extraFile, -1); extraDB = Database::createDatabase(extraFile, -1);
} }

View File

@ -36,7 +36,9 @@ struct BackupToDBAbort : TestWorkload {
backupRanges.push_back_deep(backupRanges.arena(), normalKeys); backupRanges.push_back_deep(backupRanges.arena(), normalKeys);
auto extraFile = makeReference<ClusterConnectionMemoryRecord>(*g_simulator.extraDB); ASSERT(g_simulator.extraDatabases.size() == 1);
auto extraFile =
makeReference<ClusterConnectionMemoryRecord>(ClusterConnectionString(g_simulator.extraDatabases[0]));
extraDB = Database::createDatabase(extraFile, -1); extraDB = Database::createDatabase(extraFile, -1);
lockid = UID(0xbeeffeed, 0xdecaf00d); lockid = UID(0xbeeffeed, 0xdecaf00d);

View File

@ -128,7 +128,9 @@ struct BackupToDBCorrectnessWorkload : TestWorkload {
} }
} }
auto extraFile = makeReference<ClusterConnectionMemoryRecord>(*g_simulator.extraDB); ASSERT(g_simulator.extraDatabases.size() == 1);
auto extraFile =
makeReference<ClusterConnectionMemoryRecord>(ClusterConnectionString(g_simulator.extraDatabases[0]));
extraDB = Database::createDatabase(extraFile, -1); extraDB = Database::createDatabase(extraFile, -1);
TraceEvent("BARW_Start").detail("Locked", locked); TraceEvent("BARW_Start").detail("Locked", locked);

View File

@ -76,7 +76,9 @@ struct BackupToDBUpgradeWorkload : TestWorkload {
} }
} }
auto extraFile = makeReference<ClusterConnectionMemoryRecord>(*g_simulator.extraDB); ASSERT(g_simulator.extraDatabases.size() == 1);
auto extraFile =
makeReference<ClusterConnectionMemoryRecord>(ClusterConnectionString(g_simulator.extraDatabases[0]));
extraDB = Database::createDatabase(extraFile, -1); extraDB = Database::createDatabase(extraFile, -1);
TraceEvent("DRU_Start").log(); TraceEvent("DRU_Start").log();

View File

@ -31,6 +31,7 @@
#include "fdbclient/ReadYourWrites.h" #include "fdbclient/ReadYourWrites.h"
#include "fdbclient/SystemData.h" #include "fdbclient/SystemData.h"
#include "fdbclient/TenantManagement.actor.h" #include "fdbclient/TenantManagement.actor.h"
#include "fdbclient/Tuple.h"
#include "fdbserver/BlobGranuleServerCommon.actor.h" #include "fdbserver/BlobGranuleServerCommon.actor.h"
#include "fdbserver/BlobGranuleValidation.actor.h" #include "fdbserver/BlobGranuleValidation.actor.h"
#include "fdbserver/Knobs.h" #include "fdbserver/Knobs.h"
@ -43,6 +44,7 @@
#include "flow/actorcompiler.h" // This must be the last #include. #include "flow/actorcompiler.h" // This must be the last #include.
#define BGW_DEBUG true #define BGW_DEBUG true
#define BGW_TUPLE_KEY_SIZE 2
struct WriteData { struct WriteData {
Version writeVersion; Version writeVersion;
@ -111,7 +113,7 @@ struct ThreadData : ReferenceCounted<ThreadData>, NonCopyable {
nextKeySequential = deterministicRandom()->random01() < 0.5; nextKeySequential = deterministicRandom()->random01() < 0.5;
reuseKeyProb = 0.1 + (deterministicRandom()->random01() * 0.8); reuseKeyProb = 0.1 + (deterministicRandom()->random01() * 0.8);
targetIDsPerKey = 1 + deterministicRandom()->randomInt(1, 10); targetIDsPerKey = 1 + deterministicRandom()->randomInt(10, 100);
if (BGW_DEBUG) { if (BGW_DEBUG) {
fmt::print("Directory {0} initialized with the following parameters:\n", directoryID); fmt::print("Directory {0} initialized with the following parameters:\n", directoryID);
@ -124,7 +126,28 @@ struct ThreadData : ReferenceCounted<ThreadData>, NonCopyable {
} }
// TODO could make keys variable length? // TODO could make keys variable length?
Key getKey(uint32_t key, uint32_t id) { return StringRef(format("%08x/%08x", key, id)); } Key getKey(uint32_t key, uint32_t id) { return Tuple().append((int64_t)key).append((int64_t)id).pack(); }
void validateGranuleBoundary(Key k, Key e, Key lastKey) {
if (k == allKeys.begin || k == allKeys.end) {
return;
}
// Fully formed tuples are inserted. The expectation is boundaries should be a
// sub-tuple of the inserted key.
Tuple t = Tuple::unpack(k, true);
if (SERVER_KNOBS->BG_KEY_TUPLE_TRUNCATE_OFFSET) {
Tuple t2;
try {
t2 = Tuple::unpack(lastKey);
} catch (Error& e) {
// Ignore being unable to parse lastKey as it may be a dummy key.
}
if (t2.size() > 0 && t.getInt(0) != t2.getInt(0)) {
ASSERT(t.size() <= BGW_TUPLE_KEY_SIZE - SERVER_KNOBS->BG_KEY_TUPLE_TRUNCATE_OFFSET);
}
}
}
}; };
// For debugging mismatches on what data should be and why // For debugging mismatches on what data should be and why
@ -207,6 +230,27 @@ struct BlobGranuleCorrectnessWorkload : TestWorkload {
} }
} }
ACTOR Future<Void> setUpBlobRange(Database cx, KeyRange keyRange) {
state Reference<ReadYourWritesTransaction> tr = makeReference<ReadYourWritesTransaction>(cx);
loop {
try {
tr->setOption(FDBTransactionOptions::ACCESS_SYSTEM_KEYS);
tr->setOption(FDBTransactionOptions::PRIORITY_SYSTEM_IMMEDIATE);
tr->set(blobRangeChangeKey, deterministicRandom()->randomUniqueID().toString());
wait(krmSetRange(tr, blobRangeKeys.begin, keyRange, LiteralStringRef("1")));
wait(tr->commit());
if (BGW_DEBUG) {
fmt::print("Successfully set up blob granule range for tenant range [{0} - {1})\n",
keyRange.begin.printable(),
keyRange.end.printable());
}
return Void();
} catch (Error& e) {
wait(tr->onError(e));
}
}
}
ACTOR Future<TenantMapEntry> setUpTenant(Database cx, TenantName name) { ACTOR Future<TenantMapEntry> setUpTenant(Database cx, TenantName name) {
if (BGW_DEBUG) { if (BGW_DEBUG) {
fmt::print("Setting up blob granule range for tenant {0}\n", name.printable()); fmt::print("Setting up blob granule range for tenant {0}\n", name.printable());
@ -247,6 +291,7 @@ struct BlobGranuleCorrectnessWorkload : TestWorkload {
self->directories[directoryIdx]->directoryRange = self->directories[directoryIdx]->directoryRange =
KeyRangeRef(tenantEntry.prefix, tenantEntry.prefix.withSuffix(normalKeys.end)); KeyRangeRef(tenantEntry.prefix, tenantEntry.prefix.withSuffix(normalKeys.end));
tenants.push_back({ self->directories[directoryIdx]->tenantName, tenantEntry }); tenants.push_back({ self->directories[directoryIdx]->tenantName, tenantEntry });
wait(self->setUpBlobRange(cx, self->directories[directoryIdx]->directoryRange));
} }
tenantData.addTenants(tenants); tenantData.addTenants(tenants);
@ -437,18 +482,20 @@ struct BlobGranuleCorrectnessWorkload : TestWorkload {
beginVersionByChunk.insert(normalKeys, 0); beginVersionByChunk.insert(normalKeys, 0);
int beginCollapsed = 0; int beginCollapsed = 0;
int beginNotCollapsed = 0; int beginNotCollapsed = 0;
Key lastBeginKey = LiteralStringRef("");
for (auto& chunk : blob.second) { for (auto& chunk : blob.second) {
KeyRange beginVersionRange;
if (chunk.tenantPrefix.present()) {
beginVersionRange = KeyRangeRef(chunk.keyRange.begin.removePrefix(chunk.tenantPrefix.get()),
chunk.keyRange.end.removePrefix(chunk.tenantPrefix.get()));
} else {
beginVersionRange = chunk.keyRange;
}
if (!chunk.snapshotFile.present()) { if (!chunk.snapshotFile.present()) {
ASSERT(beginVersion > 0); ASSERT(beginVersion > 0);
ASSERT(chunk.snapshotVersion == invalidVersion); ASSERT(chunk.snapshotVersion == invalidVersion);
beginCollapsed++; beginCollapsed++;
KeyRange beginVersionRange;
if (chunk.tenantPrefix.present()) {
beginVersionRange = KeyRangeRef(chunk.keyRange.begin.removePrefix(chunk.tenantPrefix.get()),
chunk.keyRange.end.removePrefix(chunk.tenantPrefix.get()));
} else {
beginVersionRange = chunk.keyRange;
}
beginVersionByChunk.insert(beginVersionRange, beginVersion); beginVersionByChunk.insert(beginVersionRange, beginVersion);
} else { } else {
@ -457,6 +504,10 @@ struct BlobGranuleCorrectnessWorkload : TestWorkload {
beginNotCollapsed++; beginNotCollapsed++;
} }
} }
// Validate boundary alignment.
threadData->validateGranuleBoundary(beginVersionRange.begin, beginVersionRange.end, lastBeginKey);
lastBeginKey = beginVersionRange.begin;
} }
CODE_PROBE(beginCollapsed > 0, "BGCorrectness got collapsed request with beginVersion > 0"); CODE_PROBE(beginCollapsed > 0, "BGCorrectness got collapsed request with beginVersion > 0");
CODE_PROBE(beginNotCollapsed > 0, "BGCorrectness got un-collapsed request with beginVersion > 0"); CODE_PROBE(beginNotCollapsed > 0, "BGCorrectness got un-collapsed request with beginVersion > 0");

View File

@ -49,8 +49,6 @@
*/ */
struct BlobGranuleVerifierWorkload : TestWorkload { struct BlobGranuleVerifierWorkload : TestWorkload {
bool doSetup; bool doSetup;
double minDelay;
double maxDelay;
double testDuration; double testDuration;
double timeTravelLimit; double timeTravelLimit;
uint64_t timeTravelBufferSize; uint64_t timeTravelBufferSize;
@ -65,6 +63,9 @@ struct BlobGranuleVerifierWorkload : TestWorkload {
int64_t purges = 0; int64_t purges = 0;
std::vector<Future<Void>> clients; std::vector<Future<Void>> clients;
bool enablePurging; bool enablePurging;
bool initAtEnd;
bool strictPurgeChecking;
bool clearAndMergeCheck;
DatabaseConfiguration config; DatabaseConfiguration config;
@ -73,14 +74,25 @@ struct BlobGranuleVerifierWorkload : TestWorkload {
BlobGranuleVerifierWorkload(WorkloadContext const& wcx) : TestWorkload(wcx) { BlobGranuleVerifierWorkload(WorkloadContext const& wcx) : TestWorkload(wcx) {
doSetup = !clientId; // only do this on the "first" client doSetup = !clientId; // only do this on the "first" client
// FIXME: don't do the delay in setup, as that delays the start of all workloads
minDelay = getOption(options, LiteralStringRef("minDelay"), 0.0);
maxDelay = getOption(options, LiteralStringRef("maxDelay"), 0.0);
testDuration = getOption(options, LiteralStringRef("testDuration"), 120.0); testDuration = getOption(options, LiteralStringRef("testDuration"), 120.0);
timeTravelLimit = getOption(options, LiteralStringRef("timeTravelLimit"), testDuration); timeTravelLimit = getOption(options, LiteralStringRef("timeTravelLimit"), testDuration);
timeTravelBufferSize = getOption(options, LiteralStringRef("timeTravelBufferSize"), 100000000); timeTravelBufferSize = getOption(options, LiteralStringRef("timeTravelBufferSize"), 100000000);
threads = getOption(options, LiteralStringRef("threads"), 1); threads = getOption(options, LiteralStringRef("threads"), 1);
enablePurging = getOption(options, LiteralStringRef("enablePurging"), false /*sharedRandomNumber % 2 == 0*/); enablePurging = getOption(options, LiteralStringRef("enablePurging"), false /*sharedRandomNumber % 2 == 0*/);
sharedRandomNumber /= 2;
// FIXME: re-enable this! There exist several bugs with purging active granules where a small amount of state
// won't be cleaned up.
strictPurgeChecking =
getOption(options, LiteralStringRef("strictPurgeChecking"), false /*sharedRandomNumber % 2 == 0*/);
sharedRandomNumber /= 10;
// randomly some tests write data first and then turn on blob granules later, to test conversion of existing DB
initAtEnd = !enablePurging && sharedRandomNumber % 10 == 0;
sharedRandomNumber /= 10;
clearAndMergeCheck = getOption(options, LiteralStringRef("clearAndMergeCheck"), sharedRandomNumber % 10 == 0);
sharedRandomNumber /= 10;
ASSERT(threads >= 1); ASSERT(threads >= 1);
if (BGV_DEBUG) { if (BGV_DEBUG) {
@ -104,9 +116,8 @@ struct BlobGranuleVerifierWorkload : TestWorkload {
// FIXME: run the actual FDBCLI command instead of copy/pasting its implementation // FIXME: run the actual FDBCLI command instead of copy/pasting its implementation
// Sets the whole user keyspace to be blobified // Sets the whole user keyspace to be blobified
ACTOR Future<Void> setUpBlobRange(Database cx, Future<Void> waitForStart) { ACTOR Future<Void> setUpBlobRange(Database cx) {
state Reference<ReadYourWritesTransaction> tr = makeReference<ReadYourWritesTransaction>(cx); state Reference<ReadYourWritesTransaction> tr = makeReference<ReadYourWritesTransaction>(cx);
wait(waitForStart);
loop { loop {
try { try {
tr->setOption(FDBTransactionOptions::ACCESS_SYSTEM_KEYS); tr->setOption(FDBTransactionOptions::ACCESS_SYSTEM_KEYS);
@ -136,11 +147,9 @@ struct BlobGranuleVerifierWorkload : TestWorkload {
wait(success(ManagementAPI::changeConfig(cx.getReference(), "blob_granules_enabled=1", true))); wait(success(ManagementAPI::changeConfig(cx.getReference(), "blob_granules_enabled=1", true)));
double initialDelay = deterministicRandom()->random01() * (self->maxDelay - self->minDelay) + self->minDelay; if (!self->initAtEnd) {
if (BGV_DEBUG) { wait(self->setUpBlobRange(cx));
printf("BGW setup initial delay of %.3f\n", initialDelay);
} }
wait(self->setUpBlobRange(cx, delay(initialDelay)));
return Void(); return Void();
} }
@ -308,12 +317,14 @@ struct BlobGranuleVerifierWorkload : TestWorkload {
// if purged just before read, verify that purge cleaned up data by restarting blob workers and // if purged just before read, verify that purge cleaned up data by restarting blob workers and
// reading older than the purge version // reading older than the purge version
if (doPurging) { if (doPurging) {
wait(self->killBlobWorkers(cx, self)); if (self->strictPurgeChecking) {
if (BGV_DEBUG) { wait(self->killBlobWorkers(cx, self));
fmt::print("BGV Reading post-purge [{0} - {1}) @ {2}\n", if (BGV_DEBUG) {
oldRead.range.begin.printable(), fmt::print("BGV Reading post-purge [{0} - {1}) @ {2}\n",
oldRead.range.end.printable(), oldRead.range.begin.printable(),
prevPurgeVersion); oldRead.range.end.printable(),
prevPurgeVersion);
}
} }
// ensure purge version exactly is still readable // ensure purge version exactly is still readable
std::pair<RangeResult, Standalone<VectorRef<BlobGranuleChunkRef>>> versionRead1 = std::pair<RangeResult, Standalone<VectorRef<BlobGranuleChunkRef>>> versionRead1 =
@ -322,31 +333,33 @@ struct BlobGranuleVerifierWorkload : TestWorkload {
fmt::print("BGV Post-purge first read:\n"); fmt::print("BGV Post-purge first read:\n");
printGranuleChunks(versionRead1.second); printGranuleChunks(versionRead1.second);
} }
try { if (self->strictPurgeChecking) {
// read at purgeVersion - 1, should NOT be readable try {
Version minSnapshotVersion = newPurgeVersion; // read at purgeVersion - 1, should NOT be readable
for (auto& it : versionRead1.second) { Version minSnapshotVersion = newPurgeVersion;
minSnapshotVersion = std::min(minSnapshotVersion, it.snapshotVersion); for (auto& it : versionRead1.second) {
minSnapshotVersion = std::min(minSnapshotVersion, it.snapshotVersion);
}
if (BGV_DEBUG) {
fmt::print("BGV Reading post-purge again [{0} - {1}) @ {2}\n",
oldRead.range.begin.printable(),
oldRead.range.end.printable(),
minSnapshotVersion - 1);
}
std::pair<RangeResult, Standalone<VectorRef<BlobGranuleChunkRef>>> versionRead2 =
wait(readFromBlob(cx, self->bstore, oldRead.range, 0, minSnapshotVersion - 1));
if (BGV_DEBUG) {
fmt::print("BGV ERROR: data not purged! Read successful!!\n");
printGranuleChunks(versionRead2.second);
}
ASSERT(false);
} catch (Error& e) {
if (e.code() == error_code_actor_cancelled) {
throw;
}
ASSERT(e.code() == error_code_blob_granule_transaction_too_old);
CODE_PROBE(true, "BGV verified too old after purge");
} }
if (BGV_DEBUG) {
fmt::print("BGV Reading post-purge again [{0} - {1}) @ {2}\n",
oldRead.range.begin.printable(),
oldRead.range.end.printable(),
minSnapshotVersion - 1);
}
std::pair<RangeResult, Standalone<VectorRef<BlobGranuleChunkRef>>> versionRead2 =
wait(readFromBlob(cx, self->bstore, oldRead.range, 0, minSnapshotVersion - 1));
if (BGV_DEBUG) {
fmt::print("BGV ERROR: data not purged! Read successful!!\n");
printGranuleChunks(versionRead2.second);
}
ASSERT(false);
} catch (Error& e) {
if (e.code() == error_code_actor_cancelled) {
throw;
}
ASSERT(e.code() == error_code_blob_granule_transaction_too_old);
CODE_PROBE(true, "BGV verified too old after purge");
} }
} }
} }
@ -420,6 +433,11 @@ struct BlobGranuleVerifierWorkload : TestWorkload {
ACTOR Future<bool> _check(Database cx, BlobGranuleVerifierWorkload* self) { ACTOR Future<bool> _check(Database cx, BlobGranuleVerifierWorkload* self) {
// check error counts, and do an availability check at the end // check error counts, and do an availability check at the end
if (self->doSetup && self->initAtEnd) {
// FIXME: this doesn't check the data contents post-conversion, just that it finishes successfully
wait(self->setUpBlobRange(cx));
}
state Transaction tr(cx); state Transaction tr(cx);
state Version readVersion = wait(self->doGrv(&tr)); state Version readVersion = wait(self->doGrv(&tr));
state Version startReadVersion = readVersion; state Version startReadVersion = readVersion;
@ -436,8 +454,21 @@ struct BlobGranuleVerifierWorkload : TestWorkload {
state Future<Void> rangeFetcher = self->findGranules(cx, self); state Future<Void> rangeFetcher = self->findGranules(cx, self);
loop { loop {
wait(self->granuleRanges.onChange()); wait(self->granuleRanges.onChange());
// wait until entire keyspace has granules
if (!self->granuleRanges.get().empty()) { if (!self->granuleRanges.get().empty()) {
break; bool haveAll = true;
if (self->granuleRanges.get().front().begin != normalKeys.begin ||
self->granuleRanges.get().back().end != normalKeys.end) {
haveAll = false;
}
for (int i = 0; haveAll && i < self->granuleRanges.get().size() - 1; i++) {
if (self->granuleRanges.get()[i].end != self->granuleRanges.get()[i + 1].begin) {
haveAll = false;
}
}
if (haveAll) {
break;
}
} }
} }
rangeFetcher.cancel(); rangeFetcher.cancel();
@ -525,7 +556,7 @@ struct BlobGranuleVerifierWorkload : TestWorkload {
// FIXME: if doPurging was set, possibly do one last purge here, and verify it succeeds with no errors // FIXME: if doPurging was set, possibly do one last purge here, and verify it succeeds with no errors
if (self->clientId == 0 && SERVER_KNOBS->BG_ENABLE_MERGING && deterministicRandom()->random01() < 0.1) { if (self->clientId == 0 && SERVER_KNOBS->BG_ENABLE_MERGING && self->clearAndMergeCheck) {
CODE_PROBE(true, "BGV clearing database and awaiting merge"); CODE_PROBE(true, "BGV clearing database and awaiting merge");
wait(clearAndAwaitMerge(cx, normalKeys)); wait(clearAndAwaitMerge(cx, normalKeys));
} }

View File

@ -53,37 +53,45 @@ struct ChangeConfigWorkload : TestWorkload {
void getMetrics(std::vector<PerfMetric>& m) override {} void getMetrics(std::vector<PerfMetric>& m) override {}
// When simulated two clusters for DR tests, this actor sets the starting configuration ACTOR Future<Void> configureExtraDatabase(ChangeConfigWorkload* self, Database db) {
// for the extra cluster. wait(delay(5 * deterministicRandom()->random01()));
ACTOR Future<Void> extraDatabaseConfigure(ChangeConfigWorkload* self) { if (self->configMode.size()) {
if (g_network->isSimulated() && g_simulator.extraDB) { if (g_simulator.startingDisabledConfiguration != "") {
auto extraFile = makeReference<ClusterConnectionMemoryRecord>(*g_simulator.extraDB); // It is not safe to allow automatic failover to a region which is not fully replicated,
state Database extraDB = Database::createDatabase(extraFile, -1); // so wait for both regions to be fully replicated before enabling failover
wait(success(
wait(delay(5 * deterministicRandom()->random01())); ManagementAPI::changeConfig(db.getReference(), g_simulator.startingDisabledConfiguration, true)));
if (self->configMode.size()) { TraceEvent("WaitForReplicasExtra").log();
if (g_simulator.startingDisabledConfiguration != "") { wait(waitForFullReplication(db));
// It is not safe to allow automatic failover to a region which is not fully replicated, TraceEvent("WaitForReplicasExtraEnd").log();
// so wait for both regions to be fully replicated before enabling failover
wait(success(ManagementAPI::changeConfig(
extraDB.getReference(), g_simulator.startingDisabledConfiguration, true)));
TraceEvent("WaitForReplicasExtra").log();
wait(waitForFullReplication(extraDB));
TraceEvent("WaitForReplicasExtraEnd").log();
}
wait(success(ManagementAPI::changeConfig(extraDB.getReference(), self->configMode, true)));
} }
if (self->networkAddresses.size()) { wait(success(ManagementAPI::changeConfig(db.getReference(), self->configMode, true)));
if (self->networkAddresses == "auto")
wait(CoordinatorsChangeActor(extraDB, self, true));
else
wait(CoordinatorsChangeActor(extraDB, self));
}
wait(delay(5 * deterministicRandom()->random01()));
} }
if (self->networkAddresses.size()) {
if (self->networkAddresses == "auto")
wait(CoordinatorsChangeActor(db, self, true));
else
wait(CoordinatorsChangeActor(db, self));
}
wait(delay(5 * deterministicRandom()->random01()));
return Void(); return Void();
} }
// When simulating multiple clusters, this actor sets the starting configuration
// for the extra clusters.
Future<Void> configureExtraDatabases(ChangeConfigWorkload* self) {
std::vector<Future<Void>> futures;
if (g_network->isSimulated()) {
for (auto extraDatabase : g_simulator.extraDatabases) {
auto extraFile = makeReference<ClusterConnectionMemoryRecord>(ClusterConnectionString(extraDatabase));
Database db = Database::createDatabase(extraFile, -1);
futures.push_back(configureExtraDatabase(self, db));
}
}
return waitForAll(futures);
}
// Either changes the database configuration, or changes the coordinators based on the parameters // Either changes the database configuration, or changes the coordinators based on the parameters
// of the workload. // of the workload.
ACTOR Future<Void> ChangeConfigClient(Database cx, ChangeConfigWorkload* self) { ACTOR Future<Void> ChangeConfigClient(Database cx, ChangeConfigWorkload* self) {
@ -93,7 +101,7 @@ struct ChangeConfigWorkload : TestWorkload {
state bool extraConfigureBefore = deterministicRandom()->random01() < 0.5; state bool extraConfigureBefore = deterministicRandom()->random01() < 0.5;
if (extraConfigureBefore) { if (extraConfigureBefore) {
wait(self->extraDatabaseConfigure(self)); wait(self->configureExtraDatabases(self));
} }
if (self->configMode.size()) { if (self->configMode.size()) {
@ -116,7 +124,7 @@ struct ChangeConfigWorkload : TestWorkload {
} }
if (!extraConfigureBefore) { if (!extraConfigureBefore) {
wait(self->extraDatabaseConfigure(self)); wait(self->configureExtraDatabases(self));
} }
return Void(); return Void();

View File

@ -2034,8 +2034,9 @@ struct ConsistencyCheckWorkload : TestWorkload {
} }
ACTOR Future<bool> checkWorkerList(Database cx, ConsistencyCheckWorkload* self) { ACTOR Future<bool> checkWorkerList(Database cx, ConsistencyCheckWorkload* self) {
if (g_simulator.extraDB) if (!g_simulator.extraDatabases.empty()) {
return true; return true;
}
std::vector<WorkerDetails> workers = wait(getWorkers(self->dbInfo)); std::vector<WorkerDetails> workers = wait(getWorkers(self->dbInfo));
std::set<NetworkAddress> workerAddresses; std::set<NetworkAddress> workerAddresses;

View File

@ -37,8 +37,9 @@ struct DifferentClustersSameRVWorkload : TestWorkload {
bool switchComplete = false; bool switchComplete = false;
DifferentClustersSameRVWorkload(WorkloadContext const& wcx) : TestWorkload(wcx) { DifferentClustersSameRVWorkload(WorkloadContext const& wcx) : TestWorkload(wcx) {
ASSERT(g_simulator.extraDB != nullptr); ASSERT(g_simulator.extraDatabases.size() == 1);
auto extraFile = makeReference<ClusterConnectionMemoryRecord>(*g_simulator.extraDB); auto extraFile =
makeReference<ClusterConnectionMemoryRecord>(ClusterConnectionString(g_simulator.extraDatabases[0]));
extraDB = Database::createDatabase(extraFile, -1); extraDB = Database::createDatabase(extraFile, -1);
testDuration = getOption(options, LiteralStringRef("testDuration"), 100.0); testDuration = getOption(options, LiteralStringRef("testDuration"), 100.0);
switchAfter = getOption(options, LiteralStringRef("switchAfter"), 50.0); switchAfter = getOption(options, LiteralStringRef("switchAfter"), 50.0);

View File

@ -50,6 +50,7 @@ class GlobalTagThrottlingWorkload : public TestWorkload {
wait(tr->commit()); wait(tr->commit());
return Void(); return Void();
} catch (Error& e) { } catch (Error& e) {
TraceEvent("GlobalTagThrottlingWorkload_SetupError").error(e);
wait(tr->onError(e)); wait(tr->onError(e));
} }
}; };

View File

@ -0,0 +1,602 @@
/*
* MetaclusterManagementWorkload.actor.cpp
*
* This source file is part of the FoundationDB open source project
*
* Copyright 2013-2022 Apple Inc. and the FoundationDB project authors
*
* Licensed under the Apache License, Version 2.0 (the "License");
* you may not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
#include <cstdint>
#include <limits>
#include "fdbclient/ClusterConnectionMemoryRecord.h"
#include "fdbclient/FDBOptions.g.h"
#include "fdbclient/GenericManagementAPI.actor.h"
#include "fdbclient/Metacluster.h"
#include "fdbclient/MetaclusterManagement.actor.h"
#include "fdbclient/ReadYourWrites.h"
#include "fdbclient/RunTransaction.actor.h"
#include "fdbclient/TenantManagement.actor.h"
#include "fdbclient/ThreadSafeTransaction.h"
#include "fdbrpc/simulator.h"
#include "fdbserver/workloads/MetaclusterConsistency.actor.h"
#include "fdbserver/workloads/workloads.actor.h"
#include "fdbserver/Knobs.h"
#include "flow/BooleanParam.h"
#include "flow/Error.h"
#include "flow/IRandom.h"
#include "flow/ThreadHelper.actor.h"
#include "flow/flow.h"
#include "flow/actorcompiler.h" // This must be the last #include.
FDB_DEFINE_BOOLEAN_PARAM(AllowPartialMetaclusterOperations);
struct MetaclusterManagementWorkload : TestWorkload {
struct DataClusterData {
Database db;
bool registered = false;
int tenantGroupCapacity = 0;
std::set<TenantName> tenants;
DataClusterData() {}
DataClusterData(Database db) : db(db) {}
};
struct TenantData {
ClusterName cluster;
TenantData() {}
TenantData(ClusterName cluster) : cluster(cluster) {}
};
Reference<IDatabase> managementDb;
std::map<ClusterName, DataClusterData> dataDbs;
std::vector<ClusterName> dataDbIndex;
int64_t totalTenantGroupCapacity = 0;
std::map<TenantName, TenantData> createdTenants;
int maxTenants;
int maxTenantGroups;
double testDuration;
MetaclusterManagementWorkload(WorkloadContext const& wcx) : TestWorkload(wcx) {
maxTenants = std::min<int>(1e8 - 1, getOption(options, "maxTenants"_sr, 1000));
maxTenantGroups = std::min<int>(2 * maxTenants, getOption(options, "maxTenantGroups"_sr, 20));
testDuration = getOption(options, "testDuration"_sr, 120.0);
}
std::string description() const override { return "MetaclusterManagement"; }
Future<Void> setup(Database const& cx) override {
if (clientId == 0) {
return _setup(cx, this);
} else {
return Void();
}
}
ACTOR static Future<Void> _setup(Database cx, MetaclusterManagementWorkload* self) {
Reference<IDatabase> threadSafeHandle =
wait(unsafeThreadFutureToFuture(ThreadSafeDatabase::createFromExistingDatabase(cx)));
MultiVersionApi::api->selectApiVersion(cx->apiVersion);
self->managementDb = MultiVersionDatabase::debugCreateFromExistingDatabase(threadSafeHandle);
ASSERT(g_simulator.extraDatabases.size() > 0);
for (auto connectionString : g_simulator.extraDatabases) {
ClusterConnectionString ccs(connectionString);
auto extraFile = makeReference<ClusterConnectionMemoryRecord>(ccs);
self->dataDbIndex.push_back(ClusterName(format("cluster_%08d", self->dataDbs.size())));
self->dataDbs[self->dataDbIndex.back()] = DataClusterData(Database::createDatabase(extraFile, -1));
}
wait(success(MetaclusterAPI::createMetacluster(cx.getReference(), "management_cluster"_sr)));
return Void();
}
ClusterName chooseClusterName() { return dataDbIndex[deterministicRandom()->randomInt(0, dataDbIndex.size())]; }
TenantName chooseTenantName() {
TenantName tenant(format("tenant%08d", deterministicRandom()->randomInt(0, maxTenants)));
return tenant;
}
Optional<TenantGroupName> chooseTenantGroup() {
Optional<TenantGroupName> tenantGroup;
if (deterministicRandom()->coinflip()) {
tenantGroup =
TenantGroupNameRef(format("tenantgroup%08d", deterministicRandom()->randomInt(0, maxTenantGroups)));
}
return tenantGroup;
}
ACTOR static Future<Void> registerCluster(MetaclusterManagementWorkload* self) {
state ClusterName clusterName = self->chooseClusterName();
state DataClusterData* dataDb = &self->dataDbs[clusterName];
state bool retried = false;
try {
state DataClusterEntry entry;
entry.capacity.numTenantGroups = deterministicRandom()->randomInt(0, 4);
loop {
try {
Future<Void> registerFuture =
MetaclusterAPI::registerCluster(self->managementDb,
clusterName,
dataDb->db->getConnectionRecord()->getConnectionString(),
entry);
Optional<Void> result = wait(timeout(registerFuture, deterministicRandom()->randomInt(1, 30)));
if (result.present()) {
break;
} else {
retried = true;
}
} catch (Error& e) {
if (e.code() == error_code_cluster_already_exists && retried && !dataDb->registered) {
Optional<DataClusterMetadata> clusterMetadata =
wait(MetaclusterAPI::tryGetCluster(self->managementDb, clusterName));
ASSERT(clusterMetadata.present());
break;
} else {
throw;
}
}
}
ASSERT(!dataDb->registered);
dataDb->tenantGroupCapacity = entry.capacity.numTenantGroups;
self->totalTenantGroupCapacity += entry.capacity.numTenantGroups;
dataDb->registered = true;
// Get a version to know that the cluster has recovered
wait(success(runTransaction(dataDb->db.getReference(),
[](Reference<ReadYourWritesTransaction> tr) { return tr->getReadVersion(); })));
return Void();
} catch (Error& e) {
if (e.code() == error_code_cluster_already_exists) {
ASSERT(dataDb->registered);
return Void();
}
TraceEvent(SevError, "RegisterClusterFailure").error(e).detail("ClusterName", clusterName);
ASSERT(false);
throw internal_error();
}
}
ACTOR static Future<Void> removeCluster(MetaclusterManagementWorkload* self) {
state ClusterName clusterName = self->chooseClusterName();
state DataClusterData* dataDb = &self->dataDbs[clusterName];
state bool retried = false;
try {
loop {
// TODO: check force removal
Future<Void> removeFuture = MetaclusterAPI::removeCluster(self->managementDb, clusterName, false);
try {
Optional<Void> result = wait(timeout(removeFuture, deterministicRandom()->randomInt(1, 30)));
if (result.present()) {
break;
} else {
retried = true;
}
} catch (Error& e) {
if (e.code() == error_code_cluster_not_found && retried && dataDb->registered) {
Optional<DataClusterMetadata> clusterMetadata =
wait(MetaclusterAPI::tryGetCluster(self->managementDb, clusterName));
ASSERT(!clusterMetadata.present());
break;
} else {
throw;
}
}
}
ASSERT(dataDb->registered);
ASSERT(dataDb->tenants.empty());
self->totalTenantGroupCapacity -= dataDb->tenantGroupCapacity;
dataDb->tenantGroupCapacity = 0;
dataDb->registered = false;
// Get a version to know that the cluster has recovered
wait(success(runTransaction(dataDb->db.getReference(),
[](Reference<ReadYourWritesTransaction> tr) { return tr->getReadVersion(); })));
return Void();
} catch (Error& e) {
if (e.code() == error_code_cluster_not_found) {
ASSERT(!dataDb->registered);
return Void();
} else if (e.code() == error_code_cluster_not_empty) {
ASSERT(!dataDb->tenants.empty());
return Void();
}
TraceEvent(SevError, "RemoveClusterFailure").error(e).detail("ClusterName", clusterName);
ASSERT(false);
throw internal_error();
}
}
ACTOR static Future<Void> listClusters(MetaclusterManagementWorkload* self) {
state ClusterName clusterName1 = self->chooseClusterName();
state ClusterName clusterName2 = self->chooseClusterName();
state int limit = deterministicRandom()->randomInt(1, self->dataDbs.size() + 1);
try {
std::map<ClusterName, DataClusterMetadata> clusterList =
wait(MetaclusterAPI::listClusters(self->managementDb, clusterName1, clusterName2, limit));
ASSERT(clusterName1 <= clusterName2);
auto resultItr = clusterList.begin();
int count = 0;
for (auto localItr = self->dataDbs.find(clusterName1);
localItr != self->dataDbs.find(clusterName2) && count < limit;
++localItr) {
fmt::print("Checking cluster {} {}\n", printable(localItr->first), localItr->second.registered);
if (localItr->second.registered) {
ASSERT(resultItr != clusterList.end());
ASSERT(resultItr->first == localItr->first);
ASSERT(resultItr->second.connectionString ==
localItr->second.db->getConnectionRecord()->getConnectionString());
++resultItr;
++count;
}
}
ASSERT(resultItr == clusterList.end());
return Void();
} catch (Error& e) {
if (e.code() == error_code_inverted_range) {
ASSERT(clusterName1 > clusterName2);
return Void();
}
TraceEvent(SevError, "ListClustersFailure")
.error(e)
.detail("BeginClusterName", clusterName1)
.detail("EndClusterName", clusterName2)
.detail("Limit", limit);
ASSERT(false);
throw internal_error();
}
}
ACTOR static Future<Void> getCluster(MetaclusterManagementWorkload* self) {
state ClusterName clusterName = self->chooseClusterName();
state DataClusterData* dataDb = &self->dataDbs[clusterName];
try {
DataClusterMetadata clusterMetadata = wait(MetaclusterAPI::getCluster(self->managementDb, clusterName));
ASSERT(dataDb->registered);
ASSERT(dataDb->db->getConnectionRecord()->getConnectionString() == clusterMetadata.connectionString);
return Void();
} catch (Error& e) {
if (e.code() == error_code_cluster_not_found) {
ASSERT(!dataDb->registered);
return Void();
}
TraceEvent(SevError, "GetClusterFailure").error(e).detail("ClusterName", clusterName);
ASSERT(false);
throw internal_error();
}
}
ACTOR static Future<Optional<DataClusterEntry>> configureImpl(MetaclusterManagementWorkload* self,
ClusterName clusterName,
DataClusterData* dataDb,
Optional<int64_t> numTenantGroups,
Optional<ClusterConnectionString> connectionString) {
state Reference<ITransaction> tr = self->managementDb->createTransaction();
loop {
try {
tr->setOption(FDBTransactionOptions::ACCESS_SYSTEM_KEYS);
Optional<DataClusterMetadata> clusterMetadata =
wait(MetaclusterAPI::tryGetClusterTransaction(tr, clusterName));
state Optional<DataClusterEntry> entry;
if (clusterMetadata.present()) {
if (numTenantGroups.present()) {
entry = clusterMetadata.get().entry;
entry.get().capacity.numTenantGroups = numTenantGroups.get();
}
MetaclusterAPI::updateClusterMetadata(
tr, clusterName, clusterMetadata.get(), connectionString, entry);
wait(buggifiedCommit(tr, BUGGIFY_WITH_PROB(0.1)));
}
return entry;
} catch (Error& e) {
wait(safeThreadFutureToFuture(tr->onError(e)));
}
}
}
ACTOR static Future<Void> configureCluster(MetaclusterManagementWorkload* self) {
state ClusterName clusterName = self->chooseClusterName();
state DataClusterData* dataDb = &self->dataDbs[clusterName];
state Optional<DataClusterEntry> updatedEntry;
state Optional<int64_t> newNumTenantGroups;
state Optional<ClusterConnectionString> connectionString;
if (deterministicRandom()->coinflip()) {
newNumTenantGroups = deterministicRandom()->randomInt(0, 4);
}
if (deterministicRandom()->coinflip()) {
connectionString = dataDb->db->getConnectionRecord()->getConnectionString();
}
try {
loop {
Optional<Optional<DataClusterEntry>> result =
wait(timeout(configureImpl(self, clusterName, dataDb, newNumTenantGroups, connectionString),
deterministicRandom()->randomInt(1, 30)));
if (result.present()) {
updatedEntry = result.get();
break;
}
}
if (updatedEntry.present()) {
int64_t tenantGroupDelta =
std::max<int64_t>(updatedEntry.get().capacity.numTenantGroups, dataDb->tenants.size()) -
std::max<int64_t>(dataDb->tenantGroupCapacity, dataDb->tenants.size());
self->totalTenantGroupCapacity += tenantGroupDelta;
dataDb->tenantGroupCapacity = updatedEntry.get().capacity.numTenantGroups;
}
return Void();
} catch (Error& e) {
TraceEvent(SevError, "ConfigureClusterFailure").error(e).detail("ClusterName", clusterName);
ASSERT(false);
throw internal_error();
}
}
ACTOR static Future<Void> createTenant(MetaclusterManagementWorkload* self) {
state TenantName tenant = self->chooseTenantName();
auto itr = self->createdTenants.find(tenant);
state bool exists = itr != self->createdTenants.end();
state bool hasCapacity = self->createdTenants.size() < self->totalTenantGroupCapacity;
state bool retried = false;
try {
loop {
try {
Future<Void> createFuture =
MetaclusterAPI::createTenant(self->managementDb, tenant, TenantMapEntry());
Optional<Void> result = wait(timeout(createFuture, deterministicRandom()->randomInt(1, 30)));
if (result.present()) {
break;
} else {
retried = true;
}
} catch (Error& e) {
if (e.code() == error_code_tenant_already_exists && retried && !exists) {
Optional<TenantMapEntry> entry = wait(MetaclusterAPI::tryGetTenant(self->managementDb, tenant));
ASSERT(entry.present());
break;
} else {
throw;
}
}
}
TenantMapEntry entry = wait(MetaclusterAPI::getTenant(self->managementDb, tenant));
ASSERT(!exists);
ASSERT(hasCapacity);
ASSERT(entry.assignedCluster.present());
auto assignedCluster = self->dataDbs.find(entry.assignedCluster.get());
ASSERT(assignedCluster != self->dataDbs.end());
ASSERT(assignedCluster->second.tenants.insert(tenant).second);
ASSERT(assignedCluster->second.tenantGroupCapacity >= assignedCluster->second.tenants.size());
self->createdTenants[tenant] = TenantData(entry.assignedCluster.get());
return Void();
} catch (Error& e) {
if (e.code() == error_code_tenant_already_exists) {
ASSERT(exists);
return Void();
} else if (e.code() == error_code_metacluster_no_capacity) {
ASSERT(!hasCapacity && !exists);
return Void();
}
TraceEvent(SevError, "CreateTenantFailure").error(e).detail("TenantName", tenant);
ASSERT(false);
throw internal_error();
}
}
ACTOR static Future<Void> deleteTenant(MetaclusterManagementWorkload* self) {
state TenantName tenant = self->chooseTenantName();
auto itr = self->createdTenants.find(tenant);
state bool exists = itr != self->createdTenants.end();
state bool retried = false;
try {
loop {
try {
Future<Void> deleteFuture = MetaclusterAPI::deleteTenant(self->managementDb, tenant);
Optional<Void> result = wait(timeout(deleteFuture, deterministicRandom()->randomInt(1, 30)));
if (result.present()) {
break;
} else {
retried = true;
}
} catch (Error& e) {
if (e.code() == error_code_tenant_not_found && retried && exists) {
Optional<TenantMapEntry> entry = wait(MetaclusterAPI::tryGetTenant(self->managementDb, tenant));
ASSERT(!entry.present());
break;
} else {
throw;
}
}
}
ASSERT(exists);
auto tenantData = self->createdTenants.find(tenant);
ASSERT(tenantData != self->createdTenants.end());
auto& dataDb = self->dataDbs[tenantData->second.cluster];
ASSERT(dataDb.registered);
auto tenantItr = dataDb.tenants.find(tenant);
ASSERT(tenantItr != dataDb.tenants.end());
if (dataDb.tenants.size() > dataDb.tenantGroupCapacity) {
--self->totalTenantGroupCapacity;
}
dataDb.tenants.erase(tenantItr);
self->createdTenants.erase(tenant);
return Void();
} catch (Error& e) {
if (e.code() == error_code_tenant_not_found) {
ASSERT(!exists);
return Void();
}
TraceEvent(SevError, "DeleteTenantFailure").error(e).detail("TenantName", tenant);
ASSERT(false);
throw internal_error();
}
}
Future<Void> start(Database const& cx) override {
if (clientId == 0) {
return _start(cx, this);
} else {
return Void();
}
}
ACTOR static Future<Void> _start(Database cx, MetaclusterManagementWorkload* self) {
state double start = now();
// Run a random sequence of operations for the duration of the test
while (now() < start + self->testDuration) {
state int operation = deterministicRandom()->randomInt(0, 7);
if (operation == 0) {
wait(self->registerCluster(self));
} else if (operation == 1) {
wait(self->removeCluster(self));
} else if (operation == 2) {
wait(self->listClusters(self));
} else if (operation == 3) {
wait(self->getCluster(self));
} else if (operation == 4) {
wait(self->configureCluster(self));
} else if (operation == 5) {
wait(self->createTenant(self));
} else if (operation == 6) {
wait(self->deleteTenant(self));
}
}
return Void();
}
// Checks that the data cluster state matches our local state
ACTOR static Future<Void> checkDataCluster(MetaclusterManagementWorkload* self,
ClusterName clusterName,
DataClusterData clusterData) {
state Optional<MetaclusterRegistrationEntry> metaclusterRegistration;
state std::vector<std::pair<TenantName, TenantMapEntry>> tenants;
state Reference<ReadYourWritesTransaction> tr = clusterData.db->createTransaction();
loop {
try {
tr->setOption(FDBTransactionOptions::READ_SYSTEM_KEYS);
wait(
store(metaclusterRegistration,
MetaclusterMetadata::metaclusterRegistration.get(clusterData.db.getReference())) &&
store(tenants,
TenantAPI::listTenantsTransaction(tr, ""_sr, "\xff\xff"_sr, clusterData.tenants.size() + 1)));
break;
} catch (Error& e) {
wait(safeThreadFutureToFuture(tr->onError(e)));
}
}
if (clusterData.registered) {
ASSERT(metaclusterRegistration.present() &&
metaclusterRegistration.get().clusterType == ClusterType::METACLUSTER_DATA);
} else {
ASSERT(!metaclusterRegistration.present());
}
ASSERT(tenants.size() == clusterData.tenants.size());
for (auto [tenantName, tenantEntry] : tenants) {
ASSERT(clusterData.tenants.count(tenantName));
ASSERT(self->createdTenants[tenantName].cluster == clusterName);
}
return Void();
}
Future<bool> check(Database const& cx) override {
if (clientId == 0) {
return _check(this);
} else {
return true;
}
}
ACTOR static Future<bool> _check(MetaclusterManagementWorkload* self) {
// The metacluster consistency check runs the tenant consistency check for each cluster
state MetaclusterConsistencyCheck<IDatabase> metaclusterConsistencyCheck(
self->managementDb, AllowPartialMetaclusterOperations::False);
wait(metaclusterConsistencyCheck.run());
std::map<ClusterName, DataClusterMetadata> dataClusters = wait(MetaclusterAPI::listClusters(
self->managementDb, ""_sr, "\xff\xff"_sr, CLIENT_KNOBS->MAX_DATA_CLUSTERS + 1));
std::vector<Future<Void>> dataClusterChecks;
for (auto [clusterName, dataClusterData] : self->dataDbs) {
auto dataClusterItr = dataClusters.find(clusterName);
if (dataClusterData.registered) {
ASSERT(dataClusterItr != dataClusters.end());
ASSERT(dataClusterItr->second.entry.capacity.numTenantGroups == dataClusterData.tenantGroupCapacity);
} else {
ASSERT(dataClusterItr == dataClusters.end());
}
dataClusterChecks.push_back(checkDataCluster(self, clusterName, dataClusterData));
}
wait(waitForAll(dataClusterChecks));
return true;
}
void getMetrics(std::vector<PerfMetric>& m) override {}
};
WorkloadFactory<MetaclusterManagementWorkload> MetaclusterManagementWorkloadFactory("MetaclusterManagement");

View File

@ -690,6 +690,10 @@ struct ReadWriteWorkload : ReadWriteCommon {
break; break;
} catch (Error& e) { } catch (Error& e) {
if (e.code() == error_code_tag_throttled) {
++self->transactionsTagThrottled;
}
self->transactionFailureMetric->errorCode = e.code(); self->transactionFailureMetric->errorCode = e.code();
self->transactionFailureMetric->log(); self->transactionFailureMetric->log();

View File

@ -39,8 +39,10 @@ struct SpecialKeySpaceCorrectnessWorkload : TestWorkload {
double testDuration, absoluteRandomProb, transactionsPerSecond; double testDuration, absoluteRandomProb, transactionsPerSecond;
PerfIntCounter wrongResults, keysCount; PerfIntCounter wrongResults, keysCount;
Reference<ReadYourWritesTransaction> ryw; // used to store all populated data Reference<ReadYourWritesTransaction> ryw; // used to store all populated data
std::vector<std::shared_ptr<SKSCTestImpl>> impls; std::vector<std::shared_ptr<SKSCTestRWImpl>> rwImpls;
std::vector<std::shared_ptr<SKSCTestAsyncReadImpl>> asyncReadImpls;
Standalone<VectorRef<KeyRangeRef>> keys; Standalone<VectorRef<KeyRangeRef>> keys;
Standalone<VectorRef<KeyRangeRef>> rwKeys;
SpecialKeySpaceCorrectnessWorkload(WorkloadContext const& wcx) SpecialKeySpaceCorrectnessWorkload(WorkloadContext const& wcx)
: TestWorkload(wcx), wrongResults("Wrong Results"), keysCount("Number of generated keys") { : TestWorkload(wcx), wrongResults("Wrong Results"), keysCount("Number of generated keys") {
@ -81,12 +83,20 @@ struct SpecialKeySpaceCorrectnessWorkload : TestWorkload {
Key startKey(baseKey + "/"); Key startKey(baseKey + "/");
Key endKey(baseKey + "/\xff"); Key endKey(baseKey + "/\xff");
self->keys.push_back_deep(self->keys.arena(), KeyRangeRef(startKey, endKey)); self->keys.push_back_deep(self->keys.arena(), KeyRangeRef(startKey, endKey));
self->impls.push_back(std::make_shared<SKSCTestImpl>(KeyRangeRef(startKey, endKey))); if (deterministicRandom()->random01() < 0.2) {
// Although there are already ranges registered, the testing range will replace them self->asyncReadImpls.push_back(std::make_shared<SKSCTestAsyncReadImpl>(KeyRangeRef(startKey, endKey)));
cx->specialKeySpace->registerKeyRange(SpecialKeySpace::MODULE::TESTONLY, cx->specialKeySpace->registerKeyRange(SpecialKeySpace::MODULE::TESTONLY,
SpecialKeySpace::IMPLTYPE::READWRITE, SpecialKeySpace::IMPLTYPE::READONLY,
self->keys.back(), self->keys.back(),
self->impls.back().get()); self->asyncReadImpls.back().get());
} else {
self->rwImpls.push_back(std::make_shared<SKSCTestRWImpl>(KeyRangeRef(startKey, endKey)));
// Although there are already ranges registered, the testing range will replace them
cx->specialKeySpace->registerKeyRange(SpecialKeySpace::MODULE::TESTONLY,
SpecialKeySpace::IMPLTYPE::READWRITE,
self->keys.back(),
self->rwImpls.back().get());
}
// generate keys in each key range // generate keys in each key range
int keysInRange = deterministicRandom()->randomInt(self->minKeysPerRange, self->maxKeysPerRange + 1); int keysInRange = deterministicRandom()->randomInt(self->minKeysPerRange, self->maxKeysPerRange + 1);
self->keysCount += keysInRange; self->keysCount += keysInRange;
@ -154,7 +164,7 @@ struct SpecialKeySpaceCorrectnessWorkload : TestWorkload {
} }
// check ryw result consistency // check ryw result consistency
KeyRange rkr = self->randomKeyRange(); KeyRange rkr = self->randomRWKeyRange();
KeyRef rkey1 = rkr.begin; KeyRef rkey1 = rkr.begin;
KeyRef rkey2 = rkr.end; KeyRef rkey2 = rkr.end;
// randomly set/clear two keys or clear a key range // randomly set/clear two keys or clear a key range
@ -238,8 +248,8 @@ struct SpecialKeySpaceCorrectnessWorkload : TestWorkload {
return true; return true;
} }
KeyRange randomKeyRange() { KeyRange randomRWKeyRange() {
Key prefix = keys[deterministicRandom()->randomInt(0, rangeCount)].begin; Key prefix = rwImpls[deterministicRandom()->randomInt(0, rwImpls.size())]->getKeyRange().begin;
Key rkey1 = Key(deterministicRandom()->randomAlphaNumeric(deterministicRandom()->randomInt(0, keyBytes))) Key rkey1 = Key(deterministicRandom()->randomAlphaNumeric(deterministicRandom()->randomInt(0, keyBytes)))
.withPrefix(prefix); .withPrefix(prefix);
Key rkey2 = Key(deterministicRandom()->randomAlphaNumeric(deterministicRandom()->randomInt(0, keyBytes))) Key rkey2 = Key(deterministicRandom()->randomAlphaNumeric(deterministicRandom()->randomInt(0, keyBytes)))

View File

@ -0,0 +1,305 @@
/*
* TenantManagementConcurrencyWorkload.actor.cpp
*
* This source file is part of the FoundationDB open source project
*
* Copyright 2013-2022 Apple Inc. and the FoundationDB project authors
*
* Licensed under the Apache License, Version 2.0 (the "License");
* you may not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
#include <cstdint>
#include <limits>
#include "fdbclient/ClusterConnectionMemoryRecord.h"
#include "fdbclient/FDBOptions.g.h"
#include "fdbclient/GenericManagementAPI.actor.h"
#include "fdbclient/MetaclusterManagement.actor.h"
#include "fdbclient/ReadYourWrites.h"
#include "fdbclient/TenantManagement.actor.h"
#include "fdbclient/ThreadSafeTransaction.h"
#include "fdbrpc/simulator.h"
#include "fdbserver/workloads/MetaclusterConsistency.actor.h"
#include "fdbserver/workloads/TenantConsistency.actor.h"
#include "fdbserver/workloads/workloads.actor.h"
#include "fdbserver/Knobs.h"
#include "flow/Error.h"
#include "flow/IRandom.h"
#include "flow/flow.h"
#include "flow/actorcompiler.h" // This must be the last #include.
struct TenantManagementConcurrencyWorkload : TestWorkload {
const TenantName tenantNamePrefix = "tenant_management_concurrency_workload_"_sr;
const Key testParametersKey = "test_parameters"_sr;
int maxTenants;
int maxTenantGroups;
double testDuration;
bool useMetacluster;
Reference<IDatabase> mvDb;
Database dataDb;
TenantManagementConcurrencyWorkload(WorkloadContext const& wcx) : TestWorkload(wcx) {
maxTenants = std::min<int>(1e8 - 1, getOption(options, "maxTenants"_sr, 100));
maxTenantGroups = std::min<int>(2 * maxTenants, getOption(options, "maxTenantGroups"_sr, 20));
testDuration = getOption(options, "testDuration"_sr, 120.0);
if (clientId == 0) {
useMetacluster = deterministicRandom()->coinflip();
} else {
// Other clients read the metacluster state from the database
useMetacluster = false;
}
}
std::string description() const override { return "TenantManagementConcurrency"; }
struct TestParameters {
constexpr static FileIdentifier file_identifier = 14350843;
bool useMetacluster = false;
TestParameters() {}
TestParameters(bool useMetacluster) : useMetacluster(useMetacluster) {}
template <class Ar>
void serialize(Ar& ar) {
serializer(ar, useMetacluster);
}
Value encode() const { return ObjectWriter::toValue(*this, Unversioned()); }
static TestParameters decode(ValueRef const& value) {
return ObjectReader::fromStringRef<TestParameters>(value, Unversioned());
}
};
Future<Void> setup(Database const& cx) override { return _setup(cx, this); }
ACTOR static Future<Void> _setup(Database cx, TenantManagementConcurrencyWorkload* self) {
state ClusterConnectionString connectionString(g_simulator.extraDatabases[0]);
Reference<IDatabase> threadSafeHandle =
wait(unsafeThreadFutureToFuture(ThreadSafeDatabase::createFromExistingDatabase(cx)));
MultiVersionApi::api->selectApiVersion(cx->apiVersion);
self->mvDb = MultiVersionDatabase::debugCreateFromExistingDatabase(threadSafeHandle);
if (self->useMetacluster && self->clientId == 0) {
wait(success(MetaclusterAPI::createMetacluster(cx.getReference(), "management_cluster"_sr)));
DataClusterEntry entry;
entry.capacity.numTenantGroups = 1e9;
wait(MetaclusterAPI::registerCluster(self->mvDb, "cluster1"_sr, connectionString, entry));
}
state Transaction tr(cx);
if (self->clientId == 0) {
// Send test parameters to the other clients
loop {
try {
tr.setOption(FDBTransactionOptions::RAW_ACCESS);
tr.set(self->testParametersKey, TestParameters(self->useMetacluster).encode());
wait(tr.commit());
break;
} catch (Error& e) {
wait(tr.onError(e));
}
}
} else {
// Read the tenant subspace chosen and saved by client 0
loop {
try {
tr.setOption(FDBTransactionOptions::RAW_ACCESS);
Optional<Value> val = wait(tr.get(self->testParametersKey));
if (val.present()) {
TestParameters params = TestParameters::decode(val.get());
self->useMetacluster = params.useMetacluster;
break;
}
wait(delay(1.0));
tr.reset();
} catch (Error& e) {
wait(tr.onError(e));
}
}
}
if (self->useMetacluster) {
ASSERT(g_simulator.extraDatabases.size() == 1);
auto extraFile = makeReference<ClusterConnectionMemoryRecord>(connectionString);
self->dataDb = Database::createDatabase(extraFile, -1);
} else {
self->dataDb = cx;
}
return Void();
}
TenantName chooseTenantName() {
TenantName tenant(
format("%s%08d", tenantNamePrefix.toString().c_str(), deterministicRandom()->randomInt(0, maxTenants)));
return tenant;
}
Optional<TenantGroupName> chooseTenantGroup() {
Optional<TenantGroupName> tenantGroup;
if (deterministicRandom()->coinflip()) {
tenantGroup =
TenantGroupNameRef(format("tenantgroup%08d", deterministicRandom()->randomInt(0, maxTenantGroups)));
}
return tenantGroup;
}
ACTOR static Future<Void> createTenant(TenantManagementConcurrencyWorkload* self) {
state TenantName tenant = self->chooseTenantName();
state TenantMapEntry entry;
entry.tenantGroup = self->chooseTenantGroup();
try {
loop {
Future<Void> createFuture =
self->useMetacluster ? MetaclusterAPI::createTenant(self->mvDb, tenant, entry)
: success(TenantAPI::createTenant(self->dataDb.getReference(), tenant, entry));
Optional<Void> result = wait(timeout(createFuture, 30));
if (result.present()) {
break;
}
}
return Void();
} catch (Error& e) {
if (e.code() == error_code_tenant_removed) {
ASSERT(self->useMetacluster);
} else if (e.code() != error_code_tenant_already_exists && e.code() != error_code_cluster_no_capacity) {
TraceEvent(SevError, "CreateTenantFailure").error(e).detail("TenantName", tenant);
ASSERT(false);
}
return Void();
}
}
ACTOR static Future<Void> deleteTenant(TenantManagementConcurrencyWorkload* self) {
state TenantName tenant = self->chooseTenantName();
try {
loop {
Future<Void> deleteFuture = self->useMetacluster
? MetaclusterAPI::deleteTenant(self->mvDb, tenant)
: TenantAPI::deleteTenant(self->dataDb.getReference(), tenant);
Optional<Void> result = wait(timeout(deleteFuture, 30));
if (result.present()) {
break;
}
}
return Void();
} catch (Error& e) {
if (e.code() != error_code_tenant_not_found) {
TraceEvent(SevError, "DeleteTenantFailure").error(e).detail("TenantName", tenant);
}
return Void();
}
}
ACTOR static Future<Void> configureImpl(TenantManagementConcurrencyWorkload* self,
TenantName tenant,
std::map<Standalone<StringRef>, Optional<Value>> configParams) {
if (self->useMetacluster) {
wait(MetaclusterAPI::configureTenant(self->mvDb, tenant, configParams));
} else {
state Reference<ReadYourWritesTransaction> tr = self->dataDb->createTransaction();
loop {
try {
tr->setOption(FDBTransactionOptions::ACCESS_SYSTEM_KEYS);
TenantMapEntry entry = wait(TenantAPI::getTenantTransaction(tr, tenant));
TenantMapEntry updatedEntry = entry;
for (auto param : configParams) {
updatedEntry.configure(param.first, param.second);
}
wait(TenantAPI::configureTenantTransaction(tr, tenant, entry, updatedEntry));
wait(buggifiedCommit(tr, BUGGIFY_WITH_PROB(0.1)));
break;
} catch (Error& e) {
wait(tr->onError(e));
}
}
}
return Void();
}
ACTOR static Future<Void> configureTenant(TenantManagementConcurrencyWorkload* self) {
state TenantName tenant = self->chooseTenantName();
state std::map<Standalone<StringRef>, Optional<Value>> configParams;
configParams["tenant_group"_sr] = self->chooseTenantGroup();
try {
loop {
Optional<Void> result = wait(timeout(configureImpl(self, tenant, configParams), 30));
if (result.present()) {
break;
}
}
return Void();
} catch (Error& e) {
if (e.code() != error_code_tenant_not_found && e.code() != error_code_invalid_tenant_state) {
TraceEvent(SevError, "ConfigureTenantFailure").error(e).detail("TenantName", tenant);
}
return Void();
}
}
Future<Void> start(Database const& cx) override { return _start(cx, this); }
ACTOR static Future<Void> _start(Database cx, TenantManagementConcurrencyWorkload* self) {
state double start = now();
// Run a random sequence of tenant management operations for the duration of the test
while (now() < start + self->testDuration) {
state int operation = deterministicRandom()->randomInt(0, 3);
if (operation == 0) {
wait(createTenant(self));
} else if (operation == 1) {
wait(deleteTenant(self));
} else if (operation == 2) {
wait(configureTenant(self));
}
}
return Void();
}
Future<bool> check(Database const& cx) override { return _check(cx, this); }
ACTOR static Future<bool> _check(Database cx, TenantManagementConcurrencyWorkload* self) {
if (self->useMetacluster) {
// The metacluster consistency check runs the tenant consistency check for each cluster
state MetaclusterConsistencyCheck<IDatabase> metaclusterConsistencyCheck(
self->mvDb, AllowPartialMetaclusterOperations::True);
wait(metaclusterConsistencyCheck.run());
} else {
state TenantConsistencyCheck<DatabaseContext> tenantConsistencyCheck(self->dataDb.getReference());
wait(tenantConsistencyCheck.run());
}
return true;
}
void getMetrics(std::vector<PerfMetric>& m) override {}
};
WorkloadFactory<TenantManagementConcurrencyWorkload> TenantManagementConcurrencyWorkloadFactory(
"TenantManagementConcurrency");

File diff suppressed because it is too large Load Diff

View File

@ -155,7 +155,9 @@ struct VersionStampWorkload : TestWorkload {
ACTOR Future<bool> _check(Database cx, VersionStampWorkload* self) { ACTOR Future<bool> _check(Database cx, VersionStampWorkload* self) {
if (self->validateExtraDB) { if (self->validateExtraDB) {
auto extraFile = makeReference<ClusterConnectionMemoryRecord>(*g_simulator.extraDB); ASSERT(g_simulator.extraDatabases.size() == 1);
auto extraFile =
makeReference<ClusterConnectionMemoryRecord>(ClusterConnectionString(g_simulator.extraDatabases[0]));
cx = Database::createDatabase(extraFile, -1); cx = Database::createDatabase(extraFile, -1);
} }
state ReadYourWritesTransaction tr(cx); state ReadYourWritesTransaction tr(cx);
@ -312,8 +314,10 @@ struct VersionStampWorkload : TestWorkload {
state double lastTime = now(); state double lastTime = now();
state Database extraDB; state Database extraDB;
if (g_simulator.extraDB != nullptr) { if (!g_simulator.extraDatabases.empty()) {
auto extraFile = makeReference<ClusterConnectionMemoryRecord>(*g_simulator.extraDB); ASSERT(g_simulator.extraDatabases.size() == 1);
auto extraFile =
makeReference<ClusterConnectionMemoryRecord>(ClusterConnectionString(g_simulator.extraDatabases[0]));
extraDB = Database::createDatabase(extraFile, -1); extraDB = Database::createDatabase(extraFile, -1);
} }
@ -380,7 +384,7 @@ struct VersionStampWorkload : TestWorkload {
} catch (Error& e) { } catch (Error& e) {
err = e; err = e;
if (err.code() == error_code_database_locked && g_simulator.extraDB != nullptr) { if (err.code() == error_code_database_locked && !g_simulator.extraDatabases.empty()) {
//TraceEvent("VST_CommitDatabaseLocked"); //TraceEvent("VST_CommitDatabaseLocked");
cx_is_primary = !cx_is_primary; cx_is_primary = !cx_is_primary;
tr = ReadYourWritesTransaction(cx_is_primary ? cx : extraDB); tr = ReadYourWritesTransaction(cx_is_primary ? cx : extraDB);

View File

@ -88,9 +88,11 @@ struct WriteDuringReadWorkload : TestWorkload {
CODE_PROBE(adjacentKeys && (nodes + minNode) > CLIENT_KNOBS->KEY_SIZE_LIMIT, CODE_PROBE(adjacentKeys && (nodes + minNode) > CLIENT_KNOBS->KEY_SIZE_LIMIT,
"WriteDuringReadWorkload testing large keys"); "WriteDuringReadWorkload testing large keys");
useExtraDB = g_simulator.extraDB != nullptr; useExtraDB = !g_simulator.extraDatabases.empty();
if (useExtraDB) { if (useExtraDB) {
auto extraFile = makeReference<ClusterConnectionMemoryRecord>(*g_simulator.extraDB); ASSERT(g_simulator.extraDatabases.size() == 1);
auto extraFile =
makeReference<ClusterConnectionMemoryRecord>(ClusterConnectionString(g_simulator.extraDatabases[0]));
extraDB = Database::createDatabase(extraFile, -1); extraDB = Database::createDatabase(extraFile, -1);
useSystemKeys = false; useSystemKeys = false;
} }

View File

@ -20,7 +20,6 @@
#include <boost/asio.hpp> #include <boost/asio.hpp>
#include <boost/asio/ssl.hpp> #include <boost/asio/ssl.hpp>
#include <boost/bind.hpp>
#include <fmt/format.h> #include <fmt/format.h>
#include <algorithm> #include <algorithm>
#include <iostream> #include <iostream>

View File

@ -68,6 +68,24 @@ ACTOR Future<Void> timeoutWarningCollector(FutureStream<Void> input, double logD
} }
} }
ACTOR Future<Void> waitForMost(std::vector<Future<ErrorOr<Void>>> futures,
int faultTolerance,
Error e,
double waitMultiplierForSlowFutures) {
state std::vector<Future<bool>> successFutures;
state double startTime = now();
successFutures.reserve(futures.size());
for (const auto& future : futures) {
successFutures.push_back(fmap([](auto const& result) { return result.present(); }, future));
}
bool success = wait(quorumEqualsTrue(successFutures, successFutures.size() - faultTolerance));
if (!success) {
throw e;
}
wait(delay((now() - startTime) * waitMultiplierForSlowFutures) || waitForAll(successFutures));
return Void();
}
ACTOR Future<bool> quorumEqualsTrue(std::vector<Future<bool>> futures, int required) { ACTOR Future<bool> quorumEqualsTrue(std::vector<Future<bool>> futures, int required) {
state std::vector<Future<Void>> true_futures; state std::vector<Future<Void>> true_futures;
state std::vector<Future<Void>> false_futures; state std::vector<Future<Void>> false_futures;
@ -168,6 +186,14 @@ ACTOR Future<Void> testSubscriber(Reference<IAsyncListener<int>> output, Optiona
} }
} }
static Future<ErrorOr<Void>> goodTestFuture(double duration) {
return tag(delay(duration), ErrorOr<Void>(Void()));
}
static Future<ErrorOr<Void>> badTestFuture(double duration, Error e) {
return tag(delay(duration), ErrorOr<Void>(e));
}
} // namespace } // namespace
TEST_CASE("/flow/genericactors/AsyncListener") { TEST_CASE("/flow/genericactors/AsyncListener") {
@ -181,6 +207,39 @@ TEST_CASE("/flow/genericactors/AsyncListener") {
return Void(); return Void();
} }
TEST_CASE("/flow/genericactors/WaitForMost") {
state std::vector<Future<ErrorOr<Void>>> futures;
{
futures = { goodTestFuture(1), goodTestFuture(2), goodTestFuture(3) };
wait(waitForMost(futures, 1, operation_failed(), 0.0)); // Don't wait for slowest future
ASSERT(!futures[2].isReady());
}
{
futures = { goodTestFuture(1), goodTestFuture(2), goodTestFuture(3) };
wait(waitForMost(futures, 0, operation_failed(), 0.0)); // Wait for all futures
ASSERT(futures[2].isReady());
}
{
futures = { goodTestFuture(1), goodTestFuture(2), goodTestFuture(3) };
wait(waitForMost(futures, 1, operation_failed(), 1.0)); // Wait for slowest future
ASSERT(futures[2].isReady());
}
{
futures = { goodTestFuture(1), goodTestFuture(2), badTestFuture(1, success()) };
wait(waitForMost(futures, 1, operation_failed(), 1.0)); // Error ignored
}
{
futures = { goodTestFuture(1), goodTestFuture(2), badTestFuture(1, success()) };
try {
wait(waitForMost(futures, 0, operation_failed(), 1.0));
ASSERT(false);
} catch (Error& e) {
ASSERT_EQ(e.code(), error_code_operation_failed);
}
}
return Void();
}
#if false #if false
TEST_CASE("/flow/genericactors/generic/storeTuple") { TEST_CASE("/flow/genericactors/generic/storeTuple") {
state std::vector<UID> resA; state std::vector<UID> resA;

View File

@ -0,0 +1,155 @@
/*
* OwningResource.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 FLOW_OWNING_REOSURCE_H
#define FLOW_OWNING_REOSURCE_H
#include "flow/FastRef.h"
// Consider the following situation:
//
// 1. An ACTOR A0 allocates an object O
// 2. A0 spawns another ACTOR A1, which depends on O
// 3. A0 triggers A1 and then terminates, destroying O
// 4. Since A1 is triggered by A0 while not knowing A0 is terminated and O is released, it would cause a SEGV error.
//
// In this header file, two classes
//
// * ResourceOwningRef
// * ResourceWeakRef
//
// are provided. The ResourceOwningRef is the reference that "holds" the resource, When it is destructed, the resource
// is also released; while ResourceWeakRef is the reference that "weakly holds" the resource. Before each access, it is
// the user's responsibility to verify if the resource is still available, via the available() method.
//
// With the two classes, the issue above can be solved by:
//
// 1. A0 allocates the object O via ResourceOwningRef
// 2. A0 forwards O to A1, via ResourceWeakRef
// 3. Every time A1 accesses O, it will verify if the resource is still available.
// 4. When A0 terminates, O is released and all ResourceWeakRef available() call will report the resource is not
// available anymore, preventing the SEGV error being raised.
namespace details {
// The class holding the pointer to the resource.
// SOMEDAY: Think using std::unique_ptr
template <typename T>
struct Resource : public ReferenceCounted<Resource<T>>, NonCopyable {
T* resource;
Resource(T* resource_) : resource(resource_) {}
~Resource() { delete resource; }
void reset(T* resource_) {
delete resource;
resource = resource_;
}
};
template <typename T>
class ResourceRef {
protected:
Reference<Resource<T>> resourceRef;
ResourceRef(const Reference<Resource<T>>& ref) : resourceRef(ref) {}
ResourceRef(Reference<Resource<T>>&& ref) : resourceRef(std::move(ref)) {}
ResourceRef& operator=(const Reference<Resource<T>>& ref) {
resourceRef = ref.resourceRef;
return *this;
}
ResourceRef& operator=(Reference<Resource<T>>&& ref) {
resourceRef = std::move(ref);
return *this;
}
virtual ~ResourceRef() {}
public:
// Retrieves the resource as a pointer
T* operator->() const noexcept { return resourceRef->resource; }
// Retrieves the resource as a reference
T& operator*() const {
if (!available()) {
throw internal_error();
} else {
return *(resourceRef->resource);
}
}
// Returns true if the resource is available, i.e. not nullptr
bool available() const noexcept { return resourceRef->resource != nullptr; }
};
} // namespace details
// The class that holds a Reference to the details::Resource which holds the real object. If the instance is destroyed,
// the object is destroyed, too.
template <typename T>
class ResourceOwningRef : public details::ResourceRef<T>, NonCopyable {
template <typename U>
friend class ResourceWeakRef;
template <typename U>
friend class ActorWeakSelfRef;
public:
ResourceOwningRef(T* resource) : details::ResourceRef<T>(makeReference<details::Resource<T>>(resource)) {}
virtual ~ResourceOwningRef() { details::ResourceRef<T>::resourceRef->reset(nullptr); }
};
// The class that weakly holds a Reference to the details::Resource. Destroying the reference will have no impact to the
// real object. On the other hand, each time accessing the object requires a verification that the object is still alive
template <typename T>
class ResourceWeakRef : public details::ResourceRef<T> {
public:
ResourceWeakRef(const ResourceOwningRef<T>& ref) : details::ResourceRef<T>(ref.resourceRef) {}
ResourceWeakRef(const ResourceWeakRef& ref) : details::ResourceRef<T>(ref.resourceRef) {}
};
// A unique reference that takes the ownership of the self object. The self object is widely used as the "global"
// context of each role.
template <typename T>
using ActorOwningSelfRef = ResourceOwningRef<T>;
// A wrapper of ResourceWeakRef, used to forward the widely used `self` pointer from the core ACTOR to other ACTORs. It
// will check the resource before returning it. If the resource is not available, an operation_cancelled error will be
// thrown to terminate the current ACTOR.
template <typename T>
class ActorWeakSelfRef : public ResourceWeakRef<T> {
public:
ActorWeakSelfRef(const ResourceOwningRef<T>& ref) : ResourceWeakRef<T>(ref) {}
ActorWeakSelfRef(const ResourceWeakRef<T>& ref) : ResourceWeakRef<T>(ref) {}
ActorWeakSelfRef(const ActorWeakSelfRef<T>& ref)
: ResourceWeakRef<T>(static_cast<const ResourceWeakRef<T>&>(ref)) {}
// Retrieves the resource as a pointer, throws operation_cancelled if the resource is not available
T* operator->() const {
if (!ResourceWeakRef<T>::available())
throw operation_cancelled();
return ResourceWeakRef<T>::resourceRef->resource;
}
// Gets the reference to the resource, Throws operation_cancelled if the resource is not available
T& operator*() const { return *(this->operator->()); }
};
#endif // FLOW_OWNING_REOSURCE_H

View File

@ -236,6 +236,19 @@ ERROR( illegal_tenant_access, 2138, "Illegal tenant access" )
ERROR( invalid_tenant_group_name, 2139, "Tenant group name cannot begin with \\xff" ) ERROR( invalid_tenant_group_name, 2139, "Tenant group name cannot begin with \\xff" )
ERROR( invalid_tenant_configuration, 2140, "Tenant configuration is invalid" ) ERROR( invalid_tenant_configuration, 2140, "Tenant configuration is invalid" )
ERROR( cluster_no_capacity, 2141, "Cluster does not have capacity to perform the specified operation" ) ERROR( cluster_no_capacity, 2141, "Cluster does not have capacity to perform the specified operation" )
ERROR( tenant_removed, 2142, "The tenant was removed" )
ERROR( invalid_tenant_state, 2143, "Operation cannot be applied to tenant in its current state" )
ERROR( invalid_cluster_name, 2160, "Data cluster name cannot begin with \\xff" )
ERROR( invalid_metacluster_operation, 2161, "Metacluster operation performed on non-metacluster" )
ERROR( cluster_already_exists, 2162, "A data cluster with the given name already exists" )
ERROR( cluster_not_found, 2163, "Data cluster does not exist" )
ERROR( cluster_not_empty, 2164, "Cluster must be empty" )
ERROR( cluster_already_registered, 2165, "Data cluster is already registered with a metacluster" )
ERROR( metacluster_no_capacity, 2166, "Metacluster does not have capacity to create new tenants" )
ERROR( management_cluster_invalid_access, 2167, "Standard transactions cannot be run against the management cluster" )
ERROR( tenant_creation_permanently_failed, 2168, "The tenant creation did not complete in a timely manner and has permanently failed" )
ERROR( cluster_locked, 2169, "The cluster has been locked" )
// 2200 - errors from bindings and official APIs // 2200 - errors from bindings and official APIs
ERROR( api_version_unset, 2200, "API version is not set" ) ERROR( api_version_unset, 2200, "API version is not set" )

View File

@ -858,7 +858,7 @@ Future<Void> timeoutWarningCollector(FutureStream<Void> const& input,
double const& logDelay, double const& logDelay,
const char* const& context, const char* const& context,
UID const& id); UID const& id);
Future<bool> quorumEqualsTrue(std::vector<Future<bool>> const& futures, int const& required); ACTOR Future<bool> quorumEqualsTrue(std::vector<Future<bool>> futures, int required);
Future<Void> lowPriorityDelay(double const& waitTime); Future<Void> lowPriorityDelay(double const& waitTime);
ACTOR template <class T> ACTOR template <class T>
@ -1004,6 +1004,11 @@ Future<Void> waitForAny(std::vector<Future<T>> const& results) {
return quorum(results, 1); return quorum(results, 1);
} }
ACTOR Future<Void> waitForMost(std::vector<Future<ErrorOr<Void>>> futures,
int faultTolerance,
Error e,
double waitMultiplierForSlowFutures = 1.0);
ACTOR Future<bool> shortCircuitAny(std::vector<Future<bool>> f); ACTOR Future<bool> shortCircuitAny(std::vector<Future<bool>> f);
ACTOR template <class T> ACTOR template <class T>
@ -2002,6 +2007,7 @@ Future<U> operator>>(Future<T> const& lhs, Future<U> const& rhs) {
* IAsyncListener is similar to AsyncVar, but it decouples the input and output, so the translation unit * IAsyncListener is similar to AsyncVar, but it decouples the input and output, so the translation unit
* responsible for handling the output does not need to have knowledge of how the output is generated * responsible for handling the output does not need to have knowledge of how the output is generated
*/ */
template <class Output> template <class Output>
class IAsyncListener : public ReferenceCounted<IAsyncListener<Output>> { class IAsyncListener : public ReferenceCounted<IAsyncListener<Output>> {
public: public:

View File

@ -259,12 +259,12 @@ if(WITH_PYTHON)
add_fdb_test( add_fdb_test(
TEST_FILES restarting/from_6.3.13/CycleTestRestart-1.txt TEST_FILES restarting/from_6.3.13/CycleTestRestart-1.txt
restarting/from_6.3.13/CycleTestRestart-2.txt) restarting/from_6.3.13/CycleTestRestart-2.txt)
add_fdb_test(
TEST_FILES restarting/from_6.3.13/DrUpgradeRestart-1.txt
restarting/from_6.3.13/DrUpgradeRestart-2.txt)
add_fdb_test( add_fdb_test(
TEST_FILES restarting/from_6.3.13/StorefrontTestRestart-1.txt TEST_FILES restarting/from_6.3.13/StorefrontTestRestart-1.txt
restarting/from_6.3.13/StorefrontTestRestart-2.txt) restarting/from_6.3.13/StorefrontTestRestart-2.txt)
add_fdb_test(
TEST_FILES restarting/from_6.3.13_until_7.2.0/DrUpgradeRestart-1.txt
restarting/from_6.3.13_until_7.2.0/DrUpgradeRestart-2.txt)
add_fdb_test( add_fdb_test(
TEST_FILES restarting/from_7.0.0/UpgradeAndBackupRestore-1.toml TEST_FILES restarting/from_7.0.0/UpgradeAndBackupRestore-1.toml
restarting/from_7.0.0/UpgradeAndBackupRestore-2.toml) restarting/from_7.0.0/UpgradeAndBackupRestore-2.toml)
@ -295,6 +295,9 @@ if(WITH_PYTHON)
add_fdb_test( add_fdb_test(
TEST_FILES restarting/from_7.1.0/VersionVectorEnableRestart-1.toml TEST_FILES restarting/from_7.1.0/VersionVectorEnableRestart-1.toml
restarting/from_7.1.0/VersionVectorEnableRestart-2.toml) restarting/from_7.1.0/VersionVectorEnableRestart-2.toml)
add_fdb_test(
TEST_FILES restarting/from_7.2.0/DrUpgradeRestart-1.txt
restarting/from_7.2.0/DrUpgradeRestart-2.txt)
add_fdb_test(TEST_FILES slow/ApiCorrectness.toml) add_fdb_test(TEST_FILES slow/ApiCorrectness.toml)
@ -319,6 +322,7 @@ if(WITH_PYTHON)
add_fdb_test(TEST_FILES slow/DiskFailureCycle.toml) add_fdb_test(TEST_FILES slow/DiskFailureCycle.toml)
add_fdb_test(TEST_FILES slow/FastTriggeredWatches.toml) add_fdb_test(TEST_FILES slow/FastTriggeredWatches.toml)
add_fdb_test(TEST_FILES slow/LowLatencyWithFailures.toml) add_fdb_test(TEST_FILES slow/LowLatencyWithFailures.toml)
add_fdb_test(TEST_FILES slow/MetaclusterManagement.toml)
add_fdb_test(TEST_FILES slow/MoveKeysClean.toml) add_fdb_test(TEST_FILES slow/MoveKeysClean.toml)
add_fdb_test(TEST_FILES slow/MoveKeysSideband.toml) add_fdb_test(TEST_FILES slow/MoveKeysSideband.toml)
add_fdb_test(TEST_FILES slow/RyowCorrectness.toml) add_fdb_test(TEST_FILES slow/RyowCorrectness.toml)
@ -332,7 +336,9 @@ if(WITH_PYTHON)
add_fdb_test(TEST_FILES slow/SwizzledRollbackTimeLapse.toml) add_fdb_test(TEST_FILES slow/SwizzledRollbackTimeLapse.toml)
add_fdb_test(TEST_FILES slow/SwizzledRollbackTimeLapseIncrement.toml) add_fdb_test(TEST_FILES slow/SwizzledRollbackTimeLapseIncrement.toml)
add_fdb_test(TEST_FILES slow/SwizzledTenantManagement.toml) add_fdb_test(TEST_FILES slow/SwizzledTenantManagement.toml)
add_fdb_test(TEST_FILES slow/SwizzledTenantManagementMetacluster.toml)
add_fdb_test(TEST_FILES slow/TenantManagement.toml) add_fdb_test(TEST_FILES slow/TenantManagement.toml)
add_fdb_test(TEST_FILES slow/TenantManagementConcurrency.toml)
add_fdb_test(TEST_FILES slow/VersionStampBackupToDB.toml) add_fdb_test(TEST_FILES slow/VersionStampBackupToDB.toml)
add_fdb_test(TEST_FILES slow/VersionStampSwitchover.toml) add_fdb_test(TEST_FILES slow/VersionStampSwitchover.toml)
add_fdb_test(TEST_FILES slow/WriteDuringReadAtomicRestore.toml) add_fdb_test(TEST_FILES slow/WriteDuringReadAtomicRestore.toml)

View File

@ -301,7 +301,8 @@ logdir = {logdir}
db_config += " blob_granules_enabled:=1" db_config += " blob_granules_enabled:=1"
self.fdbcli_exec(db_config) self.fdbcli_exec(db_config)
if self.blob_granules_enabled: # TODO - want to blobbify tenants explicitly. Right now not blobbifying at all technically fixes the tenant test
if self.blob_granules_enabled and not enable_tenants:
self.fdbcli_exec("blobrange start \\x00 \\xff") self.fdbcli_exec("blobrange start \\x00 \\xff")
# Generate and install test certificate chains and keys # Generate and install test certificate chains and keys

View File

@ -1,5 +1,5 @@
[configuration] [configuration]
extraDB = 1 extraDatabaseMode = 'LocalOrSingle'
[[test]] [[test]]
testTitle = 'BackupAndRestore' testTitle = 'BackupAndRestore'

View File

@ -1,5 +1,5 @@
[configuration] [configuration]
extraDB = 1 extraDatabaseMode = 'LocalOrSingle'
[[test]] [[test]]
testTitle = 'BackupAndRestore' testTitle = 'BackupAndRestore'

View File

@ -1,5 +1,5 @@
[configuration] [configuration]
extraDB = 1 extraDatabaseMode = 'LocalOrSingle'
[[test]] [[test]]
testTitle = 'BackupAndRestore' testTitle = 'BackupAndRestore'

View File

@ -4,9 +4,6 @@ allowDefaultTenant = false
# FIXME: re-enable rocks at some point # FIXME: re-enable rocks at some point
storageEngineExcludeTypes = [4] storageEngineExcludeTypes = [4]
[[knobs]]
bg_range_source = "blobRangeKeys"
[[test]] [[test]]
testTitle = 'BlobGranuleMoveVerifyCycle' testTitle = 'BlobGranuleMoveVerifyCycle'
@ -23,6 +20,8 @@ testTitle = 'BlobGranuleMoveVerifyCycle'
[[test.workload]] [[test.workload]]
testName = 'BlobGranuleVerifier' testName = 'BlobGranuleVerifier'
testDuration = 60.0 testDuration = 60.0
# cycle does its own workload checking, don't want clear racing with its checking
clearAndMergeCheck = false
[[test.workload]] [[test.workload]]
testName = 'RandomClogging' testName = 'RandomClogging'

View File

@ -6,9 +6,6 @@ injectSSDelay = true
# FIXME: re-enable rocks at some point # FIXME: re-enable rocks at some point
storageEngineExcludeTypes = [4, 5] storageEngineExcludeTypes = [4, 5]
[[knobs]]
bg_range_source = "blobRangeKeys"
[[test]] [[test]]
testTitle = 'BlobGranuleVerifyAtomicOps' testTitle = 'BlobGranuleVerifyAtomicOps'

View File

@ -6,9 +6,6 @@ injectSSDelay = true
# FIXME: re-enable rocks at some point # FIXME: re-enable rocks at some point
storageEngineExcludeTypes = [4, 5] storageEngineExcludeTypes = [4, 5]
[[knobs]]
bg_range_source = "blobRangeKeys"
[[test]] [[test]]
testTitle = 'BlobGranuleVerifyCycle' testTitle = 'BlobGranuleVerifyCycle'
@ -21,6 +18,8 @@ testTitle = 'BlobGranuleVerifyCycle'
[[test.workload]] [[test.workload]]
testName = 'BlobGranuleVerifier' testName = 'BlobGranuleVerifier'
testDuration = 60.0 testDuration = 60.0
# cycle does its own workload checking, don't want clear racing with its checking
clearAndMergeCheck = false
[[test.workload]] [[test.workload]]
testName = 'RandomClogging' testName = 'RandomClogging'

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