mirror of
https://github.com/apple/foundationdb.git
synced 2025-05-14 01:42:37 +08:00
Merge branch 'main' into feature-metacluster
# Conflicts: # bindings/python/tests/fdbcli_tests.py # fdbcli/TenantCommands.actor.cpp # fdbclient/ClientKnobs.cpp # fdbclient/include/fdbclient/ClientKnobs.h # fdbclient/include/fdbclient/Tenant.h # fdbserver/BlobManager.actor.cpp # fdbserver/BlobWorker.actor.cpp # fdbserver/TenantCache.actor.cpp # fdbserver/storageserver.actor.cpp # fdbserver/workloads/TenantManagementWorkload.actor.cpp # flow/include/flow/error_definitions.h
This commit is contained in:
commit
7f53b340f5
@ -673,11 +673,14 @@ def tenants(logger):
|
||||
lines = output.split('\n')
|
||||
assert len(lines) == 3
|
||||
|
||||
output = run_fdbcli_command_and_get_error('configuretenant tenant tenant_group=tenant_group1 tenant_group=tenant_group2')
|
||||
assert output == 'ERROR: configuration parameter `tenant_group\' specified more than once.'
|
||||
|
||||
output = run_fdbcli_command_and_get_error('configuretenant tenant unset')
|
||||
assert output == 'ERROR: `unset\' specified without a configuration parameter.'
|
||||
|
||||
output = run_fdbcli_command_and_get_error('configuretenant tenant unset tenant_group=tenant_group1')
|
||||
assert output == 'ERROR: unrecognized configuration parameter `tenant_group=tenant_group1\''
|
||||
assert output == 'ERROR: unrecognized configuration parameter `tenant_group=tenant_group1\'.'
|
||||
|
||||
output = run_fdbcli_command_and_get_error('configuretenant tenant tenant_group')
|
||||
assert output == 'ERROR: invalid configuration string `tenant_group\'. String must specify a value using `=\'.'
|
||||
|
@ -2,6 +2,22 @@
|
||||
Release Notes
|
||||
#############
|
||||
|
||||
7.1.17
|
||||
======
|
||||
* Same as 7.1.16 release with AVX enabled.
|
||||
|
||||
7.1.16
|
||||
======
|
||||
* Released with AVX disabled.
|
||||
* Fixed a crash bug when cluster controller shuts down. `(PR #7706) <https://github.com/apple/foundationdb/pull/7706>`_
|
||||
* Fixed a storage server failure when getReadVersion returns an error. `(PR #7688) <https://github.com/apple/foundationdb/pull/7688>`_
|
||||
* Fixed unbounded status json generation. `(PR #7680) <https://github.com/apple/foundationdb/pull/7680>`_
|
||||
* Fixed ScopeEventFieldTypeMismatch error for TLogMetrics. `(PR #7640) <https://github.com/apple/foundationdb/pull/7640>`_
|
||||
* Added getMappedRange latency metrics. `(PR #7632) <https://github.com/apple/foundationdb/pull/7632>`_
|
||||
* Fixed a version vector performance bug due to not updating client side tag cache. `(PR #7616) <https://github.com/apple/foundationdb/pull/7616>`_
|
||||
* Fixed DiskReadSeconds and DiskWriteSeconds calculaion in ProcessMetrics. `(PR #7609) <https://github.com/apple/foundationdb/pull/7609>`_
|
||||
* Added Rocksdb compression and data size stats. `(PR #7596) <https://github.com/apple/foundationdb/pull/7596>`_
|
||||
|
||||
7.1.15
|
||||
======
|
||||
* Same as 7.1.14 release with AVX enabled.
|
||||
|
@ -209,6 +209,7 @@ that process, and wait for necessary data to be moved away.
|
||||
#. ``\xff\xff/management/options/excluded_locality/force`` Read/write. Setting this key disables safety checks for writes to ``\xff\xff/management/excluded_locality/<locality>``. Setting this key only has an effect in the current transaction and is not persisted on commit.
|
||||
#. ``\xff\xff/management/options/failed_locality/force`` Read/write. Setting this key disables safety checks for writes to ``\xff\xff/management/failed_locality/<locality>``. Setting this key only has an effect in the current transaction and is not persisted on commit.
|
||||
#. ``\xff\xff/management/tenant/map/<tenant>`` Read/write. Setting a key in this range to any value will result in a tenant being created with name ``<tenant>``. Clearing a key in this range will delete the tenant with name ``<tenant>``. Reading all or a portion of this range will return the list of tenants currently present in the cluster, excluding any changes in this transaction. Values read in this range will be JSON objects containing the metadata for the associated tenants.
|
||||
#. ``\xff\xff/management/tenant/rename/<tenant>`` Read/write. Setting a key in this range to an unused tenant name will result in the tenant with the name ``<tenant>`` to be renamed to the value provided. If the rename operation is a transaction retried in a loop, it is possible for the rename to be applied twice, in which case ``tenant_not_found`` or ``tenant_already_exists`` errors may be returned. This can be avoided by checking for the tenant's existence first.
|
||||
|
||||
An exclusion is syntactically either an ip address (e.g. ``127.0.0.1``), or
|
||||
an ip address and port (e.g. ``127.0.0.1:4500``) or any locality (e.g ``locality_dcid:primary-satellite`` or
|
||||
|
@ -40,6 +40,8 @@ const KeyRangeRef tenantMapSpecialKeyRange720("\xff\xff/management/tenant/map/"_
|
||||
"\xff\xff/management/tenant/map0"_sr);
|
||||
const KeyRangeRef tenantConfigSpecialKeyRange("\xff\xff/management/tenant/configure/"_sr,
|
||||
"\xff\xff/management/tenant/configure0"_sr);
|
||||
const KeyRangeRef tenantRenameSpecialKeyRange("\xff\xff/management/tenant/rename/"_sr,
|
||||
"\xff\xff/management/tenant/rename0"_sr);
|
||||
|
||||
const KeyRangeRef tenantMapSpecialKeyRange710("\xff\xff/management/tenant_map/"_sr,
|
||||
"\xff\xff/management/tenant_map0"_sr);
|
||||
@ -486,13 +488,82 @@ CommandFactory configureTenantFactory(
|
||||
"Updates the configuration for a tenant. Use `tenant_group=<GROUP_NAME>' to change the tenant group "
|
||||
"that a tenant is assigned to or `unset tenant_group' to remove a tenant from its tenant group."));
|
||||
|
||||
// Helper function to extract tenant ID from json metadata string
|
||||
int64_t getTenantId(Value metadata) {
|
||||
json_spirit::mValue jsonObject;
|
||||
json_spirit::read_string(metadata.toString(), jsonObject);
|
||||
JSONDoc doc(jsonObject);
|
||||
int64_t id;
|
||||
doc.get("id", id);
|
||||
return id;
|
||||
}
|
||||
|
||||
// renametenant command
|
||||
ACTOR Future<bool> renameTenantCommandActor(Reference<IDatabase> db, std::vector<StringRef> tokens) {
|
||||
ACTOR Future<bool> renameTenantCommandActor(Reference<IDatabase> db, std::vector<StringRef> tokens, int apiVersion) {
|
||||
if (tokens.size() != 3) {
|
||||
printUsage(tokens[0]);
|
||||
return false;
|
||||
}
|
||||
wait(safeThreadFutureToFuture(TenantAPI::renameTenant(db, tokens[1], tokens[2])));
|
||||
state Reference<ITransaction> tr = db->createTransaction();
|
||||
state Key tenantRenameKey = tenantRenameSpecialKeyRange.begin.withSuffix(tokens[1]);
|
||||
state Key tenantOldNameKey = tenantMapSpecialKeyRange(apiVersion).begin.withSuffix(tokens[1]);
|
||||
state Key tenantNewNameKey = tenantMapSpecialKeyRange(apiVersion).begin.withSuffix(tokens[2]);
|
||||
state bool firstTry = true;
|
||||
state int64_t id;
|
||||
loop {
|
||||
tr->setOption(FDBTransactionOptions::SPECIAL_KEY_SPACE_ENABLE_WRITES);
|
||||
try {
|
||||
// Hold the reference to the standalone's memory
|
||||
state ThreadFuture<Optional<Value>> oldEntryFuture = tr->get(tenantOldNameKey);
|
||||
state ThreadFuture<Optional<Value>> newEntryFuture = tr->get(tenantNewNameKey);
|
||||
state Optional<Value> oldEntry = wait(safeThreadFutureToFuture(oldEntryFuture));
|
||||
state Optional<Value> newEntry = wait(safeThreadFutureToFuture(newEntryFuture));
|
||||
if (firstTry) {
|
||||
if (!oldEntry.present()) {
|
||||
throw tenant_not_found();
|
||||
}
|
||||
if (newEntry.present()) {
|
||||
throw tenant_already_exists();
|
||||
}
|
||||
// Store the id we see when first reading this key
|
||||
id = getTenantId(oldEntry.get());
|
||||
|
||||
firstTry = false;
|
||||
} else {
|
||||
// If we got commit_unknown_result, the rename may have already occurred.
|
||||
if (newEntry.present()) {
|
||||
int64_t checkId = getTenantId(newEntry.get());
|
||||
if (id == checkId) {
|
||||
ASSERT(!oldEntry.present() || getTenantId(oldEntry.get()) != id);
|
||||
return true;
|
||||
}
|
||||
// If the new entry is present but does not match, then
|
||||
// the rename should fail, so we throw an error.
|
||||
throw tenant_already_exists();
|
||||
}
|
||||
if (!oldEntry.present()) {
|
||||
throw tenant_not_found();
|
||||
}
|
||||
int64_t checkId = getTenantId(oldEntry.get());
|
||||
// If the id has changed since we made our first attempt,
|
||||
// then it's possible we've already moved the tenant. Don't move it again.
|
||||
if (id != checkId) {
|
||||
throw tenant_not_found();
|
||||
}
|
||||
}
|
||||
tr->set(tenantRenameKey, tokens[2]);
|
||||
wait(safeThreadFutureToFuture(tr->commit()));
|
||||
break;
|
||||
} catch (Error& e) {
|
||||
state Error err(e);
|
||||
if (e.code() == error_code_special_keys_api_failure) {
|
||||
std::string errorMsgStr = wait(getSpecialKeysFailureErrorMessage(tr));
|
||||
fmt::print(stderr, "ERROR: {}\n", errorMsgStr.c_str());
|
||||
return false;
|
||||
}
|
||||
wait(safeThreadFutureToFuture(tr->onError(err)));
|
||||
}
|
||||
}
|
||||
|
||||
fmt::print(
|
||||
"The tenant `{}' has been renamed to `{}'\n", printable(tokens[1]).c_str(), printable(tokens[2]).c_str());
|
||||
|
@ -1961,7 +1961,7 @@ ACTOR Future<int> cli(CLIOptions opt, LineNoise* plinenoise) {
|
||||
continue;
|
||||
}
|
||||
|
||||
bool _result = wait(makeInterruptable(renameTenantCommandActor(db, tokens)));
|
||||
bool _result = wait(makeInterruptable(renameTenantCommandActor(db, tokens, opt.apiVersion)));
|
||||
if (!_result)
|
||||
is_error = true;
|
||||
continue;
|
||||
|
@ -225,7 +225,7 @@ ACTOR Future<bool> profileCommandActor(Database db,
|
||||
std::vector<StringRef> tokens,
|
||||
bool intrans);
|
||||
// renametenant command
|
||||
ACTOR Future<bool> renameTenantCommandActor(Reference<IDatabase> db, std::vector<StringRef> tokens);
|
||||
ACTOR Future<bool> renameTenantCommandActor(Reference<IDatabase> db, std::vector<StringRef> tokens, int apiVersion);
|
||||
// quota command
|
||||
ACTOR Future<bool> quotaCommandActor(Reference<IDatabase> db, std::vector<StringRef> tokens);
|
||||
// setclass command
|
||||
|
@ -22,6 +22,7 @@
|
||||
#include "fdbclient/FDBTypes.h"
|
||||
#include "fdbclient/SystemData.h"
|
||||
#include "fdbclient/Tenant.h"
|
||||
#include "flow/IRandom.h"
|
||||
#include "flow/UnitTest.h"
|
||||
|
||||
#define init(...) KNOB_FN(__VA_ARGS__, INIT_ATOMIC_KNOB, INIT_KNOB)(__VA_ARGS__)
|
||||
@ -290,7 +291,7 @@ void ClientKnobs::initialize(Randomize randomize) {
|
||||
init( CHANGE_QUORUM_BAD_STATE_RETRY_DELAY, 2.0 );
|
||||
|
||||
// Tenants and Metacluster
|
||||
init( MAX_TENANTS_PER_CLUSTER, 1e6 );
|
||||
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;
|
||||
|
@ -107,10 +107,7 @@ struct TenantGroupEntry {
|
||||
|
||||
Value encode() { return ObjectWriter::toValue(*this, IncludeVersion()); }
|
||||
static TenantGroupEntry decode(ValueRef const& value) {
|
||||
TenantGroupEntry entry;
|
||||
ObjectReader reader(value.begin(), IncludeVersion());
|
||||
reader.deserialize(entry);
|
||||
return entry;
|
||||
return ObjectReader::fromStringRef<TenantGroupEntry>(value, IncludeVersion());
|
||||
}
|
||||
|
||||
template <class Ar>
|
||||
@ -143,6 +140,7 @@ struct TenantMetadataSpecification {
|
||||
|
||||
KeyBackedObjectMap<TenantName, TenantMapEntry, decltype(IncludeVersion()), NullCodec> tenantMap;
|
||||
KeyBackedProperty<int64_t> lastTenantId;
|
||||
KeyBackedBinaryValue<int64_t> tenantCount;
|
||||
KeyBackedSet<int64_t> tenantTombstones;
|
||||
KeyBackedObjectProperty<TenantTombstoneCleanupData, decltype(IncludeVersion())> tombstoneCleanupData;
|
||||
KeyBackedSet<Tuple> tenantGroupTenantIndex;
|
||||
@ -150,7 +148,7 @@ struct TenantMetadataSpecification {
|
||||
|
||||
TenantMetadataSpecification(KeyRef subspace)
|
||||
: tenantMap(subspace.withSuffix("tenant/map/"_sr), IncludeVersion()),
|
||||
lastTenantId(subspace.withSuffix("tenant/lastId"_sr)),
|
||||
lastTenantId(subspace.withSuffix("tenant/lastId"_sr)), tenantCount(subspace.withSuffix("tenant/count"_sr)),
|
||||
tenantTombstones(subspace.withSuffix("tenant/tombstones/"_sr)),
|
||||
tombstoneCleanupData(subspace.withSuffix("tenant/tombstoneCleanup"_sr), IncludeVersion()),
|
||||
tenantGroupTenantIndex(subspace.withSuffix("tenant/tenantGroup/tenantIndex/"_sr)),
|
||||
@ -164,6 +162,7 @@ private:
|
||||
public:
|
||||
static inline auto& tenantMap = instance.tenantMap;
|
||||
static inline auto& lastTenantId = instance.lastTenantId;
|
||||
static inline auto& tenantCount = instance.tenantCount;
|
||||
static inline auto& tenantTombstones = instance.tenantTombstones;
|
||||
static inline auto& tombstoneCleanupData = instance.tombstoneCleanupData;
|
||||
static inline auto& tenantGroupTenantIndex = instance.tenantGroupTenantIndex;
|
||||
|
@ -19,6 +19,7 @@
|
||||
*/
|
||||
|
||||
#pragma once
|
||||
#include "fdbclient/ClientBooleanParams.h"
|
||||
#include "flow/IRandom.h"
|
||||
#include "flow/ThreadHelper.actor.h"
|
||||
#if defined(NO_INTELLISENSE) && !defined(FDBCLIENT_TENANT_MANAGEMENT_ACTOR_G_H)
|
||||
@ -186,6 +187,16 @@ Future<std::pair<Optional<TenantMapEntry>, bool>> createTenantTransaction(
|
||||
}
|
||||
}
|
||||
|
||||
// This is idempotent because we only add an entry to the tenant map if it isn't already there
|
||||
TenantMetadata::tenantCount.atomicOp(tr, 1, MutationRef::AddValue);
|
||||
|
||||
// Read the tenant count after incrementing the counter so that simultaneous attempts to create
|
||||
// tenants in the same transaction are properly reflected.
|
||||
int64_t tenantCount = wait(TenantMetadata::tenantCount.getD(tr, Snapshot::False, 0));
|
||||
if (tenantCount > CLIENT_KNOBS->MAX_TENANTS_PER_CLUSTER) {
|
||||
throw cluster_no_capacity();
|
||||
}
|
||||
|
||||
return std::make_pair(tenantEntry, true);
|
||||
}
|
||||
|
||||
@ -285,7 +296,10 @@ Future<Void> deleteTenantTransaction(Transaction tr,
|
||||
throw tenant_not_empty();
|
||||
}
|
||||
|
||||
// This is idempotent because we only erase an entry from the tenant map if it is present
|
||||
TenantMetadata::tenantMap.erase(tr, name);
|
||||
TenantMetadata::tenantCount.atomicOp(tr, -1, MutationRef::AddValue);
|
||||
|
||||
if (tenantEntry.get().tenantGroup.present()) {
|
||||
TenantMetadata::tenantGroupTenantIndex.erase(tr,
|
||||
Tuple::makeTuple(tenantEntry.get().tenantGroup.get(), name));
|
||||
@ -467,6 +481,31 @@ Future<std::vector<std::pair<TenantName, TenantMapEntry>>> listTenants(Reference
|
||||
}
|
||||
}
|
||||
|
||||
ACTOR template <class Transaction>
|
||||
Future<Void> renameTenantTransaction(Transaction tr, TenantNameRef oldName, TenantNameRef newName) {
|
||||
tr->setOption(FDBTransactionOptions::RAW_ACCESS);
|
||||
state Optional<TenantMapEntry> oldEntry;
|
||||
state Optional<TenantMapEntry> newEntry;
|
||||
wait(store(oldEntry, tryGetTenantTransaction(tr, oldName)) &&
|
||||
store(newEntry, tryGetTenantTransaction(tr, newName)));
|
||||
if (!oldEntry.present()) {
|
||||
throw tenant_not_found();
|
||||
}
|
||||
if (newEntry.present()) {
|
||||
throw tenant_already_exists();
|
||||
}
|
||||
TenantMetadata::tenantMap.erase(tr, oldName);
|
||||
TenantMetadata::tenantMap.set(tr, newName, oldEntry.get());
|
||||
|
||||
// Update the tenant group index to reflect the new tenant name
|
||||
if (oldEntry.get().tenantGroup.present()) {
|
||||
TenantMetadata::tenantGroupTenantIndex.erase(tr, Tuple::makeTuple(oldEntry.get().tenantGroup.get(), oldName));
|
||||
TenantMetadata::tenantGroupTenantIndex.insert(tr, Tuple::makeTuple(oldEntry.get().tenantGroup.get(), newName));
|
||||
}
|
||||
|
||||
return Void();
|
||||
}
|
||||
|
||||
ACTOR template <class DB>
|
||||
Future<Void> renameTenant(Reference<DB> db, TenantName oldName, TenantName newName) {
|
||||
state Reference<typename DB::TransactionT> tr = db->createTransaction();
|
||||
@ -513,19 +552,8 @@ Future<Void> renameTenant(Reference<DB> db, TenantName oldName, TenantName newNa
|
||||
throw tenant_not_found();
|
||||
}
|
||||
}
|
||||
|
||||
TenantMetadata::tenantMap.erase(tr, oldName);
|
||||
TenantMetadata::tenantMap.set(tr, newName, oldEntry.get());
|
||||
|
||||
// Update the tenant group index to reflect the new tenant name
|
||||
if (oldEntry.get().tenantGroup.present()) {
|
||||
TenantMetadata::tenantGroupTenantIndex.erase(
|
||||
tr, Tuple::makeTuple(oldEntry.get().tenantGroup.get(), oldName));
|
||||
TenantMetadata::tenantGroupTenantIndex.insert(
|
||||
tr, Tuple::makeTuple(oldEntry.get().tenantGroup.get(), newName));
|
||||
}
|
||||
|
||||
wait(safeThreadFutureToFuture(tr->commit()));
|
||||
wait(renameTenantTransaction(tr, oldName, newName));
|
||||
wait(buggifiedCommit(tr, BUGGIFY_WITH_PROB(0.1)));
|
||||
TraceEvent("RenameTenantSuccess").detail("OldName", oldName).detail("NewName", newName);
|
||||
return Void();
|
||||
} catch (Error& e) {
|
||||
|
@ -107,7 +107,8 @@ private:
|
||||
return results;
|
||||
}
|
||||
|
||||
ACTOR static Future<Void> createTenant(
|
||||
// Returns true if the tenant was created, false if it already existed
|
||||
ACTOR static Future<bool> createTenant(
|
||||
ReadYourWritesTransaction* ryw,
|
||||
TenantNameRef tenantName,
|
||||
std::vector<std::pair<Standalone<StringRef>, Optional<Value>>> configMutations,
|
||||
@ -127,23 +128,39 @@ private:
|
||||
std::pair<Optional<TenantMapEntry>, bool> entry =
|
||||
wait(TenantAPI::createTenantTransaction(&ryw->getTransaction(), tenantName, tenantEntry));
|
||||
|
||||
return Void();
|
||||
return entry.second;
|
||||
}
|
||||
|
||||
ACTOR static Future<Void> createTenants(
|
||||
ReadYourWritesTransaction* ryw,
|
||||
std::map<TenantName, std::vector<std::pair<Standalone<StringRef>, Optional<Value>>>> tenants,
|
||||
std::map<TenantGroupName, int>* tenantGroupNetTenantDelta) {
|
||||
state Future<int64_t> tenantCountFuture =
|
||||
TenantMetadata::tenantCount.getD(&ryw->getTransaction(), Snapshot::False, 0);
|
||||
int64_t _nextId = wait(TenantAPI::getNextTenantId(&ryw->getTransaction()));
|
||||
int64_t nextId = _nextId;
|
||||
state int64_t nextId = _nextId;
|
||||
|
||||
std::vector<Future<Void>> createFutures;
|
||||
state std::vector<Future<bool>> createFutures;
|
||||
for (auto const& [tenant, config] : tenants) {
|
||||
createFutures.push_back(createTenant(ryw, tenant, config, nextId++, tenantGroupNetTenantDelta));
|
||||
}
|
||||
|
||||
TenantMetadata::lastTenantId.set(&ryw->getTransaction(), nextId - 1);
|
||||
wait(waitForAll(createFutures));
|
||||
|
||||
state int numCreatedTenants = 0;
|
||||
for (auto f : createFutures) {
|
||||
if (f.get()) {
|
||||
++numCreatedTenants;
|
||||
}
|
||||
}
|
||||
|
||||
// Check the tenant count here rather than rely on the createTenantTransaction check because we don't have RYW
|
||||
int64_t tenantCount = wait(tenantCountFuture);
|
||||
if (tenantCount + numCreatedTenants > CLIENT_KNOBS->MAX_TENANTS_PER_CLUSTER) {
|
||||
throw cluster_no_capacity();
|
||||
}
|
||||
|
||||
return Void();
|
||||
}
|
||||
|
||||
@ -242,6 +259,7 @@ public:
|
||||
|
||||
// These sub-ranges should only be used if HasSubRanges=true
|
||||
const inline static KeyRangeRef configureSubRange = KeyRangeRef("configure/"_sr, "configure0"_sr);
|
||||
const inline static KeyRangeRef renameSubRange = KeyRangeRef("rename/"_sr, "rename0"_sr);
|
||||
|
||||
explicit TenantRangeImpl(KeyRangeRef kr) : SpecialKeyRangeRWImpl(kr) {}
|
||||
|
||||
@ -267,6 +285,8 @@ public:
|
||||
|
||||
state std::vector<std::pair<KeyRangeRef, Optional<Value>>> mapMutations;
|
||||
state std::map<TenantName, std::vector<std::pair<Standalone<StringRef>, Optional<Value>>>> configMutations;
|
||||
state std::set<TenantName> renameSet;
|
||||
state std::vector<std::pair<TenantName, TenantName>> renameMutations;
|
||||
|
||||
tenantManagementFutures.push_back(TenantAPI::checkTenantMode(&ryw->getTransaction(), ClusterType::STANDALONE));
|
||||
|
||||
@ -299,12 +319,31 @@ public:
|
||||
false, "configure tenant", "invalid tenant configuration key"));
|
||||
throw special_keys_api_failure();
|
||||
}
|
||||
} else if (subRangeIntersects(renameSubRange, adjustedRange)) {
|
||||
StringRef oldName = adjustedRange.begin.removePrefix(renameSubRange.begin);
|
||||
StringRef newName = range.value().second.get();
|
||||
// Do not allow overlapping renames in the same commit
|
||||
// e.g. A->B + B->C, D->D
|
||||
if (renameSet.count(oldName) || renameSet.count(newName) || oldName == newName) {
|
||||
ryw->setSpecialKeySpaceErrorMsg(
|
||||
ManagementAPIError::toJsonString(false, "rename tenant", "tenant rename conflict"));
|
||||
throw special_keys_api_failure();
|
||||
}
|
||||
renameSet.insert(oldName);
|
||||
renameSet.insert(newName);
|
||||
renameMutations.push_back(std::make_pair(oldName, newName));
|
||||
}
|
||||
}
|
||||
|
||||
std::map<TenantName, std::vector<std::pair<Standalone<StringRef>, Optional<Value>>>> tenantsToCreate;
|
||||
for (auto mapMutation : mapMutations) {
|
||||
TenantNameRef tenantName = mapMutation.first.begin;
|
||||
auto set_iter = renameSet.lower_bound(tenantName);
|
||||
if (set_iter != renameSet.end() && mapMutation.first.contains(*set_iter)) {
|
||||
ryw->setSpecialKeySpaceErrorMsg(
|
||||
ManagementAPIError::toJsonString(false, "rename tenant", "tenant rename conflict"));
|
||||
throw special_keys_api_failure();
|
||||
}
|
||||
if (mapMutation.second.present()) {
|
||||
std::vector<std::pair<Standalone<StringRef>, Optional<Value>>> createMutations;
|
||||
auto itr = configMutations.find(tenantName);
|
||||
@ -335,10 +374,20 @@ public:
|
||||
tenantManagementFutures.push_back(createTenants(ryw, tenantsToCreate, &tenantGroupNetTenantDelta));
|
||||
}
|
||||
for (auto configMutation : configMutations) {
|
||||
if (renameSet.count(configMutation.first)) {
|
||||
ryw->setSpecialKeySpaceErrorMsg(
|
||||
ManagementAPIError::toJsonString(false, "rename tenant", "tenant rename conflict"));
|
||||
throw special_keys_api_failure();
|
||||
}
|
||||
tenantManagementFutures.push_back(
|
||||
changeTenantConfig(ryw, configMutation.first, configMutation.second, &tenantGroupNetTenantDelta));
|
||||
}
|
||||
|
||||
for (auto renameMutation : renameMutations) {
|
||||
tenantManagementFutures.push_back(TenantAPI::renameTenantTransaction(
|
||||
&ryw->getTransaction(), renameMutation.first, renameMutation.second));
|
||||
}
|
||||
|
||||
wait(waitForAll(tenantManagementFutures));
|
||||
|
||||
state std::vector<Future<Void>> tenantGroupUpdateFutures;
|
||||
|
@ -971,18 +971,8 @@ ACTOR Future<Void> monitorClientRanges(Reference<BlobManagerData> bmData) {
|
||||
Optional<TenantName>(),
|
||||
Optional<TenantName>(),
|
||||
CLIENT_KNOBS->MAX_TENANTS_PER_CLUSTER + 1)));
|
||||
ASSERT_WE_THINK(!tenantResults.more &&
|
||||
tenantResults.results.size() <= CLIENT_KNOBS->MAX_TENANTS_PER_CLUSTER);
|
||||
if (tenantResults.more || tenantResults.results.size() > CLIENT_KNOBS->MAX_TENANTS_PER_CLUSTER) {
|
||||
TraceEvent(SevError, "BlobManagerTooManyTenants", bmData->id)
|
||||
.detail("Epoch", bmData->epoch)
|
||||
.detail("TenantCount", tenantResults.results.size());
|
||||
wait(delay(600));
|
||||
if (bmData->iAmReplaced.canBeSet()) {
|
||||
bmData->iAmReplaced.sendError(internal_error());
|
||||
}
|
||||
throw internal_error();
|
||||
}
|
||||
ASSERT(tenantResults.results.size() <= CLIENT_KNOBS->MAX_TENANTS_PER_CLUSTER &&
|
||||
!tenantResults.more);
|
||||
|
||||
std::vector<Key> prefixes;
|
||||
for (auto& it : tenantResults.results) {
|
||||
|
@ -4003,17 +4003,7 @@ ACTOR Future<Void> monitorTenants(Reference<BlobWorkerData> bwData) {
|
||||
Optional<TenantName>(),
|
||||
Optional<TenantName>(),
|
||||
CLIENT_KNOBS->MAX_TENANTS_PER_CLUSTER + 1)));
|
||||
ASSERT_WE_THINK(!tenantResults.more &&
|
||||
tenantResults.results.size() <= CLIENT_KNOBS->MAX_TENANTS_PER_CLUSTER);
|
||||
if (tenantResults.more || tenantResults.results.size() > CLIENT_KNOBS->MAX_TENANTS_PER_CLUSTER) {
|
||||
TraceEvent(SevError, "BlobWorkerTooManyTenants", bwData->id)
|
||||
.detail("TenantCount", tenantResults.results.size());
|
||||
wait(delay(600));
|
||||
if (bwData->fatalError.canBeSet()) {
|
||||
bwData->fatalError.sendError(internal_error());
|
||||
}
|
||||
throw internal_error();
|
||||
}
|
||||
ASSERT(tenantResults.results.size() <= CLIENT_KNOBS->MAX_TENANTS_PER_CLUSTER && !tenantResults.more);
|
||||
|
||||
std::vector<std::pair<TenantName, TenantMapEntry>> tenants;
|
||||
for (auto& it : tenantResults.results) {
|
||||
|
@ -487,6 +487,11 @@ struct RolesInfo {
|
||||
double dataLagSeconds = -1.0;
|
||||
obj["id"] = iface.id().shortString();
|
||||
obj["role"] = role;
|
||||
if (iface.metadata.present()) {
|
||||
obj["storage_metadata"] = iface.metadata.get().toJSON();
|
||||
// printf("%s\n", metadataObj.getJson().c_str());
|
||||
}
|
||||
|
||||
try {
|
||||
TraceEventFields const& storageMetrics = metrics.at("StorageMetrics");
|
||||
|
||||
@ -594,14 +599,12 @@ struct RolesInfo {
|
||||
}
|
||||
}
|
||||
|
||||
if (iface.metadata.present()) {
|
||||
obj["storage_metadata"] = iface.metadata.get().toJSON();
|
||||
// printf("%s\n", metadataObj.getJson().c_str());
|
||||
}
|
||||
|
||||
} catch (Error& e) {
|
||||
if (e.code() != error_code_attribute_not_found)
|
||||
throw e;
|
||||
else {
|
||||
TraceEvent(SevWarnAlways, "StorageServerStatusJson").error(e);
|
||||
}
|
||||
}
|
||||
|
||||
if (pDataLagSeconds) {
|
||||
|
@ -33,8 +33,8 @@ class TenantCacheImpl {
|
||||
|
||||
KeyBackedRangeResult<std::pair<TenantName, TenantMapEntry>> tenantList =
|
||||
wait(TenantMetadata::tenantMap.getRange(
|
||||
tr, Optional<TenantName>(), Optional<TenantName>(), CLIENT_KNOBS->MAX_TENANTS_PER_CLUSTER));
|
||||
ASSERT(!tenantList.more && tenantList.results.size() <= CLIENT_KNOBS->MAX_TENANTS_PER_CLUSTER);
|
||||
tr, Optional<TenantName>(), Optional<TenantName>(), CLIENT_KNOBS->MAX_TENANTS_PER_CLUSTER + 1));
|
||||
ASSERT(tenantList.results.size() <= CLIENT_KNOBS->MAX_TENANTS_PER_CLUSTER && !tenantList.more);
|
||||
|
||||
return tenantList.results;
|
||||
}
|
||||
|
@ -10127,7 +10127,8 @@ ACTOR Future<Void> initTenantMap(StorageServer* self) {
|
||||
// when SSs store only the local tenants
|
||||
KeyBackedRangeResult<std::pair<TenantName, TenantMapEntry>> entries =
|
||||
wait(TenantMetadata::tenantMap.getRange(
|
||||
tr, Optional<TenantName>(), Optional<TenantName>(), CLIENT_KNOBS->MAX_TENANTS_PER_CLUSTER));
|
||||
tr, Optional<TenantName>(), Optional<TenantName>(), CLIENT_KNOBS->MAX_TENANTS_PER_CLUSTER + 1));
|
||||
ASSERT(entries.results.size() <= CLIENT_KNOBS->MAX_TENANTS_PER_CLUSTER && !entries.more);
|
||||
|
||||
TraceEvent("InitTenantMap", self->thisServerID)
|
||||
.detail("Version", version)
|
||||
|
@ -20,6 +20,7 @@
|
||||
|
||||
#include <cstdint>
|
||||
#include <limits>
|
||||
#include "fdbclient/ClientBooleanParams.h"
|
||||
#include "fdbclient/ClusterConnectionMemoryRecord.h"
|
||||
#include "fdbclient/FDBOptions.g.h"
|
||||
#include "fdbclient/GenericManagementAPI.actor.h"
|
||||
@ -72,6 +73,9 @@ struct TenantManagementWorkload : TestWorkload {
|
||||
const Key specialKeysTenantConfigPrefix = SpecialKeySpace::getModuleRange(SpecialKeySpace::MODULE::MANAGEMENT)
|
||||
.begin.withSuffix(TenantRangeImpl<true>::submoduleRange.begin)
|
||||
.withSuffix(TenantRangeImpl<true>::configureSubRange.begin);
|
||||
const Key specialKeysTenantRenamePrefix = SpecialKeySpace::getModuleRange(SpecialKeySpace::MODULE::MANAGEMENT)
|
||||
.begin.withSuffix(TenantRangeImpl<true>::submoduleRange.begin)
|
||||
.withSuffix(TenantRangeImpl<true>::renameSubRange.begin);
|
||||
|
||||
int maxTenants;
|
||||
int maxTenantGroups;
|
||||
@ -316,6 +320,7 @@ struct TenantManagementWorkload : TestWorkload {
|
||||
// True if any tenant group name starts with \xff
|
||||
state bool hasSystemTenantGroup = false;
|
||||
|
||||
state int newTenants = 0;
|
||||
state std::map<TenantName, TenantMapEntry> tenantsToCreate;
|
||||
for (int i = 0; i < numTenants; ++i) {
|
||||
TenantName tenant = self->chooseTenantName(true);
|
||||
@ -325,9 +330,14 @@ struct TenantManagementWorkload : TestWorkload {
|
||||
|
||||
TenantMapEntry entry;
|
||||
entry.tenantGroup = self->chooseTenantGroup(true);
|
||||
tenantsToCreate[tenant] = entry;
|
||||
|
||||
alreadyExists = alreadyExists || self->createdTenants.count(tenant);
|
||||
if (self->createdTenants.count(tenant)) {
|
||||
alreadyExists = true;
|
||||
} else if (!tenantsToCreate.count(tenant)) {
|
||||
++newTenants;
|
||||
}
|
||||
|
||||
tenantsToCreate[tenant] = entry;
|
||||
hasSystemTenant = hasSystemTenant || tenant.startsWith("\xff"_sr);
|
||||
hasSystemTenantGroup = hasSystemTenantGroup || entry.tenantGroup.orDefault(""_sr).startsWith("\xff"_sr);
|
||||
}
|
||||
@ -337,20 +347,37 @@ struct TenantManagementWorkload : TestWorkload {
|
||||
state bool existedAtStart = alreadyExists;
|
||||
|
||||
state Reference<ReadYourWritesTransaction> tr = makeReference<ReadYourWritesTransaction>(self->dataDb);
|
||||
state int64_t minTenantCount = std::numeric_limits<int64_t>::max();
|
||||
state int64_t finalTenantCount = 0;
|
||||
|
||||
loop {
|
||||
try {
|
||||
// First, attempt to create the tenants
|
||||
state bool retried = false;
|
||||
loop {
|
||||
if (operationType == OperationType::MANAGEMENT_TRANSACTION ||
|
||||
operationType == OperationType::SPECIAL_KEYS) {
|
||||
tr->setOption(FDBTransactionOptions::READ_SYSTEM_KEYS);
|
||||
wait(store(finalTenantCount, TenantMetadata::tenantCount.getD(tr, Snapshot::False, 0)));
|
||||
minTenantCount = std::min(finalTenantCount, minTenantCount);
|
||||
}
|
||||
|
||||
try {
|
||||
Optional<Void> result = wait(timeout(createImpl(tr, tenantsToCreate, operationType, self),
|
||||
deterministicRandom()->randomInt(1, 30)));
|
||||
|
||||
if (result.present()) {
|
||||
// Database operations shouldn't get here if the tenant already exists
|
||||
ASSERT(operationType == OperationType::SPECIAL_KEYS ||
|
||||
operationType == OperationType::MANAGEMENT_TRANSACTION || !alreadyExists);
|
||||
// Make sure that we had capacity to create the tenants. This cannot be validated for
|
||||
// database operations because we cannot determine the tenant count in the same transaction
|
||||
// that the tenant is created
|
||||
if (operationType == OperationType::SPECIAL_KEYS ||
|
||||
operationType == OperationType::MANAGEMENT_TRANSACTION) {
|
||||
ASSERT(minTenantCount + newTenants <= CLIENT_KNOBS->MAX_TENANTS_PER_CLUSTER);
|
||||
} else {
|
||||
// Database operations shouldn't get here if the tenant already exists
|
||||
ASSERT(!alreadyExists);
|
||||
}
|
||||
|
||||
break;
|
||||
}
|
||||
|
||||
@ -483,6 +510,14 @@ struct TenantManagementWorkload : TestWorkload {
|
||||
} else if (e.code() == error_code_invalid_metacluster_operation) {
|
||||
ASSERT(operationType == OperationType::METACLUSTER != self->useMetacluster);
|
||||
return Void();
|
||||
} else if (e.code() == error_code_cluster_no_capacity) {
|
||||
// Confirm that we overshot our capacity. This check cannot be done for database operations
|
||||
// because we cannot transactionally get the tenant count with the creation.
|
||||
if (operationType == OperationType::MANAGEMENT_TRANSACTION ||
|
||||
operationType == OperationType::SPECIAL_KEYS) {
|
||||
ASSERT(finalTenantCount + newTenants > CLIENT_KNOBS->MAX_TENANTS_PER_CLUSTER);
|
||||
}
|
||||
return Void();
|
||||
}
|
||||
|
||||
// Database-based operations should not need to be retried
|
||||
@ -1030,20 +1065,115 @@ struct TenantManagementWorkload : TestWorkload {
|
||||
}
|
||||
}
|
||||
|
||||
ACTOR static Future<Void> renameTenant(TenantManagementWorkload* self) {
|
||||
// Currently only supporting MANAGEMENT_DATABASE op, so numTenants should always be 1
|
||||
// state OperationType operationType = TenantManagementWorkload::randomOperationType();
|
||||
int numTenants = 1;
|
||||
// Helper function that checks tenant keyspace and updates internal Tenant Map after a rename operation
|
||||
ACTOR static Future<Void> verifyTenantRename(TenantManagementWorkload* self,
|
||||
TenantName oldTenantName,
|
||||
TenantName newTenantName) {
|
||||
state Optional<TenantMapEntry> oldTenantEntry =
|
||||
wait(TenantAPI::tryGetTenant(self->dataDb.getReference(), oldTenantName));
|
||||
state Optional<TenantMapEntry> newTenantEntry =
|
||||
wait(TenantAPI::tryGetTenant(self->dataDb.getReference(), newTenantName));
|
||||
ASSERT(!oldTenantEntry.present());
|
||||
ASSERT(newTenantEntry.present());
|
||||
TenantData tData = self->createdTenants[oldTenantName];
|
||||
self->createdTenants[newTenantName] = tData;
|
||||
self->createdTenants.erase(oldTenantName);
|
||||
state Transaction insertTr(self->dataDb, newTenantName);
|
||||
if (!tData.empty) {
|
||||
loop {
|
||||
try {
|
||||
insertTr.set(self->keyName, newTenantName);
|
||||
wait(insertTr.commit());
|
||||
break;
|
||||
} catch (Error& e) {
|
||||
wait(insertTr.onError(e));
|
||||
}
|
||||
}
|
||||
}
|
||||
return Void();
|
||||
}
|
||||
|
||||
state std::vector<TenantName> oldTenantNames;
|
||||
state std::vector<TenantName> newTenantNames;
|
||||
ACTOR static Future<Void> verifyTenantRenames(TenantManagementWorkload* self,
|
||||
std::map<TenantName, TenantName> tenantRenames) {
|
||||
state std::map<TenantName, TenantName> tenantRenamesCopy = tenantRenames;
|
||||
state std::map<TenantName, TenantName>::iterator iter = tenantRenamesCopy.begin();
|
||||
for (; iter != tenantRenamesCopy.end(); ++iter) {
|
||||
wait(verifyTenantRename(self, iter->first, iter->second));
|
||||
wait(checkTenantContents(self, iter->second, self->createdTenants[iter->second]));
|
||||
}
|
||||
return Void();
|
||||
}
|
||||
|
||||
ACTOR static Future<Void> renameImpl(Reference<ReadYourWritesTransaction> tr,
|
||||
OperationType operationType,
|
||||
std::map<TenantName, TenantName> tenantRenames,
|
||||
bool tenantNotFound,
|
||||
bool tenantExists,
|
||||
bool tenantOverlap,
|
||||
TenantManagementWorkload* self) {
|
||||
if (operationType == OperationType::SPECIAL_KEYS) {
|
||||
tr->setOption(FDBTransactionOptions::SPECIAL_KEY_SPACE_ENABLE_WRITES);
|
||||
for (auto& iter : tenantRenames) {
|
||||
tr->set(self->specialKeysTenantRenamePrefix.withSuffix(iter.first), iter.second);
|
||||
}
|
||||
wait(tr->commit());
|
||||
} else if (operationType == OperationType::MANAGEMENT_DATABASE) {
|
||||
ASSERT(tenantRenames.size() == 1);
|
||||
auto iter = tenantRenames.begin();
|
||||
wait(TenantAPI::renameTenant(self->dataDb.getReference(), iter->first, iter->second));
|
||||
ASSERT(!tenantNotFound && !tenantExists);
|
||||
} else { // operationType == OperationType::MANAGEMENT_TRANSACTION
|
||||
tr->setOption(FDBTransactionOptions::ACCESS_SYSTEM_KEYS);
|
||||
std::vector<Future<Void>> renameFutures;
|
||||
for (auto& iter : tenantRenames) {
|
||||
renameFutures.push_back(success(TenantAPI::renameTenantTransaction(tr, iter.first, iter.second)));
|
||||
}
|
||||
wait(waitForAll(renameFutures));
|
||||
wait(tr->commit());
|
||||
ASSERT(!tenantNotFound && !tenantExists);
|
||||
}
|
||||
return Void();
|
||||
}
|
||||
|
||||
ACTOR static Future<Void> renameTenant(TenantManagementWorkload* self) {
|
||||
state OperationType operationType = self->randomOperationType();
|
||||
state int numTenants = 1;
|
||||
state Reference<ReadYourWritesTransaction> tr = self->dataDb->createTransaction();
|
||||
|
||||
if (operationType == OperationType::SPECIAL_KEYS || operationType == OperationType::MANAGEMENT_TRANSACTION) {
|
||||
numTenants = deterministicRandom()->randomInt(1, 5);
|
||||
}
|
||||
|
||||
// TODO: remove this when we have metacluster support for renames
|
||||
if (operationType == OperationType::METACLUSTER) {
|
||||
operationType = OperationType::MANAGEMENT_DATABASE;
|
||||
}
|
||||
|
||||
state std::map<TenantName, TenantName> tenantRenames;
|
||||
state std::set<TenantName> allTenantNames;
|
||||
|
||||
// Tenant Error flags
|
||||
state bool tenantExists = false;
|
||||
state bool tenantNotFound = false;
|
||||
state bool tenantOverlap = false;
|
||||
state bool unknownResult = false;
|
||||
|
||||
for (int i = 0; i < numTenants; ++i) {
|
||||
TenantName oldTenant = self->chooseTenantName(false);
|
||||
TenantName newTenant = self->chooseTenantName(false);
|
||||
newTenantNames.push_back(newTenant);
|
||||
oldTenantNames.push_back(oldTenant);
|
||||
bool checkOverlap =
|
||||
oldTenant == newTenant || allTenantNames.count(oldTenant) || allTenantNames.count(newTenant);
|
||||
// renameTenantTransaction does not handle rename collisions:
|
||||
// reject the rename here if it has overlap and we are doing a transaction operation
|
||||
// and then pick another combination
|
||||
if (checkOverlap && operationType == OperationType::MANAGEMENT_TRANSACTION) {
|
||||
--i;
|
||||
continue;
|
||||
}
|
||||
tenantOverlap = tenantOverlap || checkOverlap;
|
||||
tenantRenames[oldTenant] = newTenant;
|
||||
allTenantNames.insert(oldTenant);
|
||||
allTenantNames.insert(newTenant);
|
||||
if (!self->createdTenants.count(oldTenant)) {
|
||||
tenantNotFound = true;
|
||||
}
|
||||
@ -1054,59 +1184,52 @@ struct TenantManagementWorkload : TestWorkload {
|
||||
|
||||
loop {
|
||||
try {
|
||||
ASSERT(oldTenantNames.size() == 1);
|
||||
state int tenantIndex = 0;
|
||||
for (; tenantIndex != oldTenantNames.size(); ++tenantIndex) {
|
||||
state TenantName oldTenantName = oldTenantNames[tenantIndex];
|
||||
state TenantName newTenantName = newTenantNames[tenantIndex];
|
||||
// Perform rename, then check against the DB for the new results
|
||||
wait(TenantAPI::renameTenant(self->dataDb.getReference(), oldTenantName, newTenantName));
|
||||
ASSERT(!tenantNotFound && !tenantExists);
|
||||
state Optional<TenantMapEntry> oldTenantEntry =
|
||||
wait(self->tryGetTenant(oldTenantName, OperationType::SPECIAL_KEYS));
|
||||
state Optional<TenantMapEntry> newTenantEntry =
|
||||
wait(self->tryGetTenant(newTenantName, OperationType::SPECIAL_KEYS));
|
||||
ASSERT(!oldTenantEntry.present());
|
||||
ASSERT(newTenantEntry.present());
|
||||
|
||||
// Update Internal Tenant Map and check for correctness
|
||||
TenantData tData = self->createdTenants[oldTenantName];
|
||||
self->createdTenants[newTenantName] = tData;
|
||||
self->createdTenants.erase(oldTenantName);
|
||||
if (!tData.empty) {
|
||||
state Transaction insertTr(self->dataDb, newTenantName);
|
||||
loop {
|
||||
try {
|
||||
insertTr.set(self->keyName, newTenantName);
|
||||
wait(insertTr.commit());
|
||||
break;
|
||||
} catch (Error& e) {
|
||||
wait(insertTr.onError(e));
|
||||
}
|
||||
}
|
||||
}
|
||||
wait(checkTenantContents(self, newTenantName, self->createdTenants[newTenantName]));
|
||||
}
|
||||
wait(renameImpl(tr, operationType, tenantRenames, tenantNotFound, tenantExists, tenantOverlap, self));
|
||||
wait(verifyTenantRenames(self, tenantRenames));
|
||||
return Void();
|
||||
} catch (Error& e) {
|
||||
ASSERT(oldTenantNames.size() == 1);
|
||||
if (e.code() == error_code_tenant_not_found) {
|
||||
TraceEvent("RenameTenantOldTenantNotFound")
|
||||
.detail("OldTenantName", oldTenantNames[0])
|
||||
.detail("NewTenantName", newTenantNames[0]);
|
||||
ASSERT(tenantNotFound);
|
||||
.detail("TenantRenames", describe(tenantRenames))
|
||||
.detail("CommitUnknownResult", unknownResult);
|
||||
if (unknownResult) {
|
||||
wait(verifyTenantRenames(self, tenantRenames));
|
||||
} else {
|
||||
ASSERT(tenantNotFound);
|
||||
}
|
||||
return Void();
|
||||
} else if (e.code() == error_code_tenant_already_exists) {
|
||||
TraceEvent("RenameTenantNewTenantAlreadyExists")
|
||||
.detail("OldTenantName", oldTenantNames[0])
|
||||
.detail("NewTenantName", newTenantNames[0]);
|
||||
ASSERT(tenantExists);
|
||||
.detail("TenantRenames", describe(tenantRenames))
|
||||
.detail("CommitUnknownResult", unknownResult);
|
||||
if (unknownResult) {
|
||||
wait(verifyTenantRenames(self, tenantRenames));
|
||||
} else {
|
||||
ASSERT(tenantExists);
|
||||
}
|
||||
return Void();
|
||||
} else if (e.code() == error_code_special_keys_api_failure) {
|
||||
TraceEvent("RenameTenantNameConflict").detail("TenantRenames", describe(tenantRenames));
|
||||
ASSERT(tenantOverlap);
|
||||
return Void();
|
||||
} else {
|
||||
TraceEvent(SevError, "RenameTenantFailure")
|
||||
.error(e)
|
||||
.detail("OldTenantName", oldTenantNames[0])
|
||||
.detail("NewTenantName", newTenantNames[0]);
|
||||
try {
|
||||
// In the case of commit_unknown_result, assume we continue retrying
|
||||
// until it's successful. Next loop around may throw error because it's
|
||||
// already been moved, so account for that and update internal map as needed.
|
||||
if (e.code() == error_code_commit_unknown_result) {
|
||||
TraceEvent("RenameTenantCommitUnknownResult").error(e);
|
||||
ASSERT(operationType != OperationType::MANAGEMENT_DATABASE);
|
||||
unknownResult = true;
|
||||
}
|
||||
wait(tr->onError(e));
|
||||
} catch (Error& e) {
|
||||
TraceEvent(SevError, "RenameTenantFailure")
|
||||
.error(e)
|
||||
.detail("TenantRenames", describe(tenantRenames));
|
||||
return Void();
|
||||
}
|
||||
}
|
||||
return Void();
|
||||
}
|
||||
}
|
||||
}
|
||||
@ -1272,6 +1395,26 @@ struct TenantManagementWorkload : TestWorkload {
|
||||
return Void();
|
||||
}
|
||||
|
||||
// Verify that the tenant count matches the actual number of tenants in the cluster and that we haven't created too
|
||||
// many
|
||||
ACTOR static Future<Void> checkTenantCount(Database cx) {
|
||||
state Reference<ReadYourWritesTransaction> tr = cx->createTransaction();
|
||||
loop {
|
||||
try {
|
||||
tr->setOption(FDBTransactionOptions::READ_SYSTEM_KEYS);
|
||||
state int64_t tenantCount = wait(TenantMetadata::tenantCount.getD(tr, Snapshot::False, 0));
|
||||
KeyBackedRangeResult<std::pair<TenantName, TenantMapEntry>> tenants =
|
||||
wait(TenantMetadata::tenantMap.getRange(tr, {}, {}, CLIENT_KNOBS->MAX_TENANTS_PER_CLUSTER + 1));
|
||||
|
||||
ASSERT(tenants.results.size() == tenantCount && !tenants.more);
|
||||
ASSERT(tenantCount <= CLIENT_KNOBS->MAX_TENANTS_PER_CLUSTER);
|
||||
return Void();
|
||||
} catch (Error& e) {
|
||||
wait(tr->onError(e));
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
// Verify that the set of tenants in the database matches our local state
|
||||
ACTOR static Future<Void> compareTenants(TenantManagementWorkload* self) {
|
||||
state std::map<TenantName, TenantData>::iterator localItr = self->createdTenants.begin();
|
||||
@ -1503,6 +1646,10 @@ struct TenantManagementWorkload : TestWorkload {
|
||||
}
|
||||
}
|
||||
|
||||
if (self->clientId == 0) {
|
||||
wait(checkTenantCount(cx));
|
||||
}
|
||||
|
||||
wait(compareTenants(self) && compareTenantGroups(self) && checkTenantTombstones(self));
|
||||
return true;
|
||||
}
|
||||
|
@ -235,8 +235,9 @@ ERROR( unknown_tenant, 2137, "Tenant is not available from this server" )
|
||||
ERROR( illegal_tenant_access, 2138, "Illegal tenant access" )
|
||||
ERROR( invalid_tenant_group_name, 2139, "Tenant group name cannot begin with \\xff" )
|
||||
ERROR( invalid_tenant_configuration, 2140, "Tenant configuration is invalid" )
|
||||
ERROR( tenant_removed, 2141, "The tenant was removed" )
|
||||
ERROR( invalid_tenant_state, 2142, "Operation cannot be applied to tenant in its current state" )
|
||||
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, 2150, "Data cluster name cannot begin with \\xff" )
|
||||
ERROR( invalid_metacluster_operation, 2151, "Metacluster operation performed on non-metacluster" )
|
||||
|
Loading…
x
Reference in New Issue
Block a user