From 36bfc6e36b0cc76bbfd6e329d08a8fa880fd7d5e Mon Sep 17 00:00:00 2001 From: He Liu Date: Thu, 4 Aug 2022 21:12:15 -0700 Subject: [PATCH 001/210] Added ValidateStorageResult and ValidateStorageRequest. --- .../fdbclient/StorageServerInterface.h | 43 +++++++++++++++++++ 1 file changed, 43 insertions(+) diff --git a/fdbclient/include/fdbclient/StorageServerInterface.h b/fdbclient/include/fdbclient/StorageServerInterface.h index 8e1d82d6a0..6aa79d9fd1 100644 --- a/fdbclient/include/fdbclient/StorageServerInterface.h +++ b/fdbclient/include/fdbclient/StorageServerInterface.h @@ -1142,4 +1142,47 @@ struct StorageQueuingMetricsRequest { } }; +struct ValidateStorageResult { + constexpr static FileIdentifier file_identifier = 13804340; + + UID requestId; + std::string error; + + ValidateStorageResult() = default; + ValidateStorageResult(UID requestId) : requestId(requestId) {} + + template + void serialize(Ar& ar) { + serializer(ar, requestId, error); + } +}; + +struct ValidateStorageRequest { + constexpr static FileIdentifier file_identifier = 13804341; + + enum Action { VALIDATE_HA = 0 }; + + UID requestId; + KeyRange range; + std::vector actions; + ReplyPromise reply; + + ValidateStorageRequest() = default; + ValidateStorageRequest(UID requestId, KeyRange range) : requestId(requestId), range(range) {} + + void addItem(Action action) { this->actions.push_back(static_cast(action)); } + std::vector getActions() const { + std::vector res; + for (const int32_t action : this->actions) { + res.push_back(static_cast(action)); + } + return res; + } + + template + void serialize(Ar& ar) { + serializer(ar, requestId, range, actions, reply); + } +}; + #endif From fa3e462662f3eeb0c01a0fa4b78303ade64d446d Mon Sep 17 00:00:00 2001 From: He Liu Date: Tue, 9 Aug 2022 15:05:20 -0700 Subject: [PATCH 002/210] Added validateStorageQ in SS. --- fdbclient/ServerKnobs.cpp | 1 + fdbclient/include/fdbclient/ServerKnobs.h | 1 + .../fdbclient/StorageServerInterface.h | 3 +++ fdbserver/storageserver.actor.cpp | 26 +++++++++++++++++-- 4 files changed, 29 insertions(+), 2 deletions(-) diff --git a/fdbclient/ServerKnobs.cpp b/fdbclient/ServerKnobs.cpp index f3d6dd4c4d..18a44c4326 100644 --- a/fdbclient/ServerKnobs.cpp +++ b/fdbclient/ServerKnobs.cpp @@ -705,6 +705,7 @@ void ServerKnobs::initialize(Randomize randomize, ClientKnobs* clientKnobs, IsSi init( FETCH_KEYS_LOWER_PRIORITY, 0 ); init( FETCH_CHANGEFEED_PARALLELISM, 4 ); init( SERVE_FETCH_CHECKPOINT_PARALLELISM, 4 ); + init( SERVE_VALIDATE_STORAGE_PARALLELISM, 2 ); init( BUGGIFY_BLOCK_BYTES, 10000 ); init( STORAGE_RECOVERY_VERSION_LAG_LIMIT, 2 * MAX_READ_TRANSACTION_LIFE_VERSIONS ); init( STORAGE_COMMIT_BYTES, 10000000 ); if( randomize && BUGGIFY ) STORAGE_COMMIT_BYTES = 2000000; diff --git a/fdbclient/include/fdbclient/ServerKnobs.h b/fdbclient/include/fdbclient/ServerKnobs.h index 0360f7317d..c75b7defcc 100644 --- a/fdbclient/include/fdbclient/ServerKnobs.h +++ b/fdbclient/include/fdbclient/ServerKnobs.h @@ -665,6 +665,7 @@ public: int FETCH_KEYS_LOWER_PRIORITY; int FETCH_CHANGEFEED_PARALLELISM; int SERVE_FETCH_CHECKPOINT_PARALLELISM; + int SERVE_VALIDATE_STORAGE_PARALLELISM; int BUGGIFY_BLOCK_BYTES; int64_t STORAGE_RECOVERY_VERSION_LAG_LIMIT; double STORAGE_DURABILITY_LAG_REJECT_THRESHOLD; diff --git a/fdbclient/include/fdbclient/StorageServerInterface.h b/fdbclient/include/fdbclient/StorageServerInterface.h index 6aa79d9fd1..faa7d7b47c 100644 --- a/fdbclient/include/fdbclient/StorageServerInterface.h +++ b/fdbclient/include/fdbclient/StorageServerInterface.h @@ -93,6 +93,7 @@ struct StorageServerInterface { RequestStream checkpoint; RequestStream fetchCheckpoint; RequestStream fetchCheckpointKeyValues; + RequestStream validateStorage; private: bool acceptingRequests; @@ -163,6 +164,7 @@ public: RequestStream(getValue.getEndpoint().getAdjustedEndpoint(20)); fetchCheckpointKeyValues = RequestStream( getValue.getEndpoint().getAdjustedEndpoint(21)); + validateStorage = RequestStream(getValue.getEndpoint().getAdjustedEndpoint(22)); } } else { ASSERT(Ar::isDeserializing); @@ -213,6 +215,7 @@ public: streams.push_back(checkpoint.getReceiver()); streams.push_back(fetchCheckpoint.getReceiver()); streams.push_back(fetchCheckpointKeyValues.getReceiver()); + streams.push_back(validateStorage.getReceiver()); FlowTransport::transport().addEndpoints(streams); } }; diff --git a/fdbserver/storageserver.actor.cpp b/fdbserver/storageserver.actor.cpp index 68137c8d6a..86dc34f0ff 100644 --- a/fdbserver/storageserver.actor.cpp +++ b/fdbserver/storageserver.actor.cpp @@ -1007,6 +1007,8 @@ public: FlowLock serveFetchCheckpointParallelismLock; + FlowLock serveValidateStorageParallelismLock; + int64_t instanceID; Promise otherError; @@ -1180,8 +1182,11 @@ public: specialCounter(cc, "ServeFetchCheckpointActive", [self]() { return self->serveFetchCheckpointParallelismLock.activePermits(); }); - specialCounter(cc, "ServeFetchCheckpointWaiting", [self]() { - return self->serveFetchCheckpointParallelismLock.waiters(); + specialCounter(cc, "ServeFetchCheckpointActive", [self]() { + return self->serveFetchCheckpointParallelismLock.activePermits(); + }); + specialCounter(cc, "ServeValidateStorageWaiting", [self]() { + return self->serveValidateStorageParallelismLock.waiters(); }); specialCounter(cc, "QueryQueueMax", [self]() { return self->getAndResetMaxQueryQueueSize(); }); specialCounter(cc, "BytesStored", [self]() { return self->metrics.byteSample.getEstimate(allKeys); }); @@ -1239,6 +1244,7 @@ public: fetchChangeFeedParallelismLock(SERVER_KNOBS->FETCH_KEYS_PARALLELISM), fetchKeysBytesBudget(SERVER_KNOBS->STORAGE_FETCH_BYTES), fetchKeysBudgetUsed(false), serveFetchCheckpointParallelismLock(SERVER_KNOBS->SERVE_FETCH_CHECKPOINT_PARALLELISM), + serveValidateStorageParallelismLock(SERVER_KNOBS->SERVE_VALIDATE_STORAGE_PARALLELISM), instanceID(deterministicRandom()->randomUniqueID().first()), shuttingDown(false), behind(false), versionBehind(false), debug_inApplyUpdate(false), debug_lastValidateTime(0), lastBytesInputEBrake(0), lastDurableVersionEBrake(0), maxQueryQueue(0), transactionTagCounter(ssi.id()), counters(this), @@ -2251,6 +2257,19 @@ ACTOR Future fetchCheckpointKeyValuesQ(StorageServer* self, FetchCheckpoin return Void(); } +// Serves FetchCheckpointKeyValuesRequest, reads local checkpoint and sends it to the client over wire. +ACTOR Future validateStorageQ(StorageServer* self, ValidateStorageRequest req) { + wait(self->serveValidateStorageParallelismLock.take(TaskPriority::DefaultYield)); + state FlowLock::Releaser holder(self->serveValidateStorageParallelismLock); + + TraceEvent("ServeValidateStorageBegin", self->thisServerID) + .detail("RequestID", req.requestId) + .detail("Range", req.range); + + req.reply.send(ValidateStorageResult(req.requestId)); + return Void(); +} + ACTOR Future overlappingChangeFeedsQ(StorageServer* data, OverlappingChangeFeedsRequest req) { wait(delay(0)); wait(data->version.whenAtLeast(req.minVersion)); @@ -10093,6 +10112,9 @@ ACTOR Future storageServerCore(StorageServer* self, StorageServerInterface when(FetchCheckpointKeyValuesRequest req = waitNext(ssi.fetchCheckpointKeyValues.getFuture())) { self->actors.add(fetchCheckpointKeyValuesQ(self, req)); } + when(ValidateStorageRequest req = waitNext(ssi.validateStorage.getFuture())) { + self->actors.add(validateStorageQ(self, req)); + } when(wait(updateProcessStatsTimer)) { updateProcessStats(self); updateProcessStatsTimer = delay(SERVER_KNOBS->FASTRESTORE_UPDATE_PROCESS_STATS_INTERVAL); From 19ec3d31e7713c1cd4e89f4e911916bd2a2ec429 Mon Sep 17 00:00:00 2001 From: He Liu Date: Mon, 15 Aug 2022 12:59:24 -0700 Subject: [PATCH 003/210] Added validateRange() in SS. --- .../KeyValueStoreShardedRocksDB.actor.cpp | 6 +- fdbserver/storageserver.actor.cpp | 376 +++++++++++++++++- 2 files changed, 368 insertions(+), 14 deletions(-) diff --git a/fdbserver/KeyValueStoreShardedRocksDB.actor.cpp b/fdbserver/KeyValueStoreShardedRocksDB.actor.cpp index 84a6e53a78..cfb616a8e7 100644 --- a/fdbserver/KeyValueStoreShardedRocksDB.actor.cpp +++ b/fdbserver/KeyValueStoreShardedRocksDB.actor.cpp @@ -647,7 +647,7 @@ public: RangeResult metadata; readRangeInDb(metadataShard.get(), prefixRange(shardMappingPrefix), UINT16_MAX, UINT16_MAX, &metadata); - std::vector> mapping = decodeShardMapping(metadata, shardMappingPrefix); + std::vector> mapping = decodeShardMapping(metadata, shardMappingPrefix) for (const auto& [range, name] : mapping) { TraceEvent(SevVerbose, "ShardedRocksLoadRange", this->logId) @@ -942,6 +942,10 @@ public: writeBatch->Put(metadataShard->cf, getShardMappingKey(lastKey, shardMappingPrefix), nextShard == nullptr ? "" : nextShard->physicalShard->id); + TraceEvent(SevDebug, "ShardedRocksDB", this->logId) + .detail("Action", "PersistRangeMappingEnd") + .detail("NextShardKey", lastKey) + .detail("Value", nextShard == nullptr ? "" : nextShard->physicalShard->id); dirtyShards->insert(metadataShard.get()); } diff --git a/fdbserver/storageserver.actor.cpp b/fdbserver/storageserver.actor.cpp index 86dc34f0ff..5b5b0c27c5 100644 --- a/fdbserver/storageserver.actor.cpp +++ b/fdbserver/storageserver.actor.cpp @@ -2257,19 +2257,6 @@ ACTOR Future fetchCheckpointKeyValuesQ(StorageServer* self, FetchCheckpoin return Void(); } -// Serves FetchCheckpointKeyValuesRequest, reads local checkpoint and sends it to the client over wire. -ACTOR Future validateStorageQ(StorageServer* self, ValidateStorageRequest req) { - wait(self->serveValidateStorageParallelismLock.take(TaskPriority::DefaultYield)); - state FlowLock::Releaser holder(self->serveValidateStorageParallelismLock); - - TraceEvent("ServeValidateStorageBegin", self->thisServerID) - .detail("RequestID", req.requestId) - .detail("Range", req.range); - - req.reply.send(ValidateStorageResult(req.requestId)); - return Void(); -} - ACTOR Future overlappingChangeFeedsQ(StorageServer* data, OverlappingChangeFeedsRequest req) { wait(delay(0)); wait(data->version.whenAtLeast(req.minVersion)); @@ -3505,6 +3492,369 @@ ACTOR Future readRange(StorageServer* data, return result; } +ACTOR Future validateRangeAgainstServer(StorageServer* self, + KeyRange range, + Version version, + StorageServerInterface remoteServer) { + TraceEvent("ServeValidateRangeAgainstServerBegin", self->thisServerID) + .detail("Range", range) + .detail("Version", version) + .detail("Servers", remoteServer); + Key begin = range.begin; + + loop { + try { + state GetKeyValuesRequest req; + req.begin = begin; + req.end = range.end; + req.limit = 1e4; + req.limitBytes = CLIENT_KNOBS->REPLY_BYTE_LIMIT; + req.version = version; + req.tags = TagSet(); + + // Try getting the entries in the specified range + state ErrorOr keyValueFuture = + wait(remoteServer.getKeyValues.getReplyUnlessFailedFor(req, 2, 0)); + + // Read the resulting entries + state int firstValidServer = -1; + totalReadAmount = 0; + for (j = 0; j < keyValueFutures.size(); j++) { + ErrorOr rangeResult = keyValueFutures[j].get(); + + // Compare the results with other storage servers + if (rangeResult.present() && !rangeResult.get().error.present()) { + state GetKeyValuesReply current = rangeResult.get(); + totalReadAmount += current.data.expectedSize(); + // If we haven't encountered a valid storage server yet, then mark this as the baseline + // to compare against + if (firstValidServer == -1) + firstValidServer = j; + + // Compare this shard against the first + else { + GetKeyValuesReply reference = keyValueFutures[firstValidServer].get().get(); + + if (current.data != reference.data || current.more != reference.more) { + // Be especially verbose if in simulation + if (g_network->isSimulated()) { + int invalidIndex = -1; + printf("\n%sSERVER %d (%s); shard = %s - %s:\n", + storageServerInterfaces[j].isTss() ? "TSS " : "", + j, + storageServerInterfaces[j].address().toString().c_str(), + printable(req.begin.getKey()).c_str(), + printable(req.end.getKey()).c_str()); + for (int k = 0; k < current.data.size(); k++) { + printf("%d. %s => %s\n", + k, + printable(current.data[k].key).c_str(), + printable(current.data[k].value).c_str()); + if (invalidIndex < 0 && + (k >= reference.data.size() || current.data[k].key != reference.data[k].key || + current.data[k].value != reference.data[k].value)) + invalidIndex = k; + } + + printf("\n%sSERVER %d (%s); shard = %s - %s:\n", + storageServerInterfaces[firstValidServer].isTss() ? "TSS " : "", + firstValidServer, + storageServerInterfaces[firstValidServer].address().toString().c_str(), + printable(req.begin.getKey()).c_str(), + printable(req.end.getKey()).c_str()); + for (int k = 0; k < reference.data.size(); k++) { + printf("%d. %s => %s\n", + k, + printable(reference.data[k].key).c_str(), + printable(reference.data[k].value).c_str()); + if (invalidIndex < 0 && + (k >= current.data.size() || reference.data[k].key != current.data[k].key || + reference.data[k].value != current.data[k].value)) + invalidIndex = k; + } + + printf("\nMISMATCH AT %d\n\n", invalidIndex); + } + + // Data for trace event + // The number of keys unique to the current shard + int currentUniques = 0; + // The number of keys unique to the reference shard + int referenceUniques = 0; + // The number of keys in both shards with conflicting values + int valueMismatches = 0; + // The number of keys in both shards with matching values + int matchingKVPairs = 0; + // Last unique key on the current shard + KeyRef currentUniqueKey; + // Last unique key on the reference shard + KeyRef referenceUniqueKey; + // Last value mismatch + KeyRef valueMismatchKey; + + // Loop indeces + int currentI = 0; + int referenceI = 0; + while (currentI < current.data.size() || referenceI < reference.data.size()) { + if (currentI >= current.data.size()) { + referenceUniqueKey = reference.data[referenceI].key; + referenceUniques++; + referenceI++; + } else if (referenceI >= reference.data.size()) { + currentUniqueKey = current.data[currentI].key; + currentUniques++; + currentI++; + } else { + KeyValueRef currentKV = current.data[currentI]; + KeyValueRef referenceKV = reference.data[referenceI]; + + if (currentKV.key == referenceKV.key) { + if (currentKV.value == referenceKV.value) + matchingKVPairs++; + else { + valueMismatchKey = currentKV.key; + valueMismatches++; + } + + currentI++; + referenceI++; + } else if (currentKV.key < referenceKV.key) { + currentUniqueKey = currentKV.key; + currentUniques++; + currentI++; + } else { + referenceUniqueKey = referenceKV.key; + referenceUniques++; + referenceI++; + } + } + } + + TraceEvent("ConsistencyCheck_DataInconsistent") + .detail(format("StorageServer%d", j).c_str(), storageServers[j].toString()) + .detail(format("StorageServer%d", firstValidServer).c_str(), + storageServers[firstValidServer].toString()) + .detail("ShardBegin", req.begin.getKey()) + .detail("ShardEnd", req.end.getKey()) + .detail("VersionNumber", req.version) + .detail(format("Server%dUniques", j).c_str(), currentUniques) + .detail(format("Server%dUniqueKey", j).c_str(), currentUniqueKey) + .detail(format("Server%dUniques", firstValidServer).c_str(), referenceUniques) + .detail(format("Server%dUniqueKey", firstValidServer).c_str(), referenceUniqueKey) + .detail("ValueMismatches", valueMismatches) + .detail("ValueMismatchKey", valueMismatchKey) + .detail("MatchingKVPairs", matchingKVPairs) + .detail("IsTSS", + storageServerInterfaces[j].isTss() || + storageServerInterfaces[firstValidServer].isTss() + ? "True" + : "False"); + + if ((g_network->isSimulated() && + g_simulator.tssMode != ISimulator::TSSMode::EnabledDropMutations) || + (!storageServerInterfaces[j].isTss() && + !storageServerInterfaces[firstValidServer].isTss())) { + self->testFailure("Data inconsistent", true); + return false; + } + } + } + } + + // If the data is not available and we aren't relocating this shard + else if (!isRelocating) { + Error e = rangeResult.isError() ? rangeResult.getError() : rangeResult.get().error.get(); + + TraceEvent("ConsistencyCheck_StorageServerUnavailable") + .errorUnsuppressed(e) + .suppressFor(1.0) + .detail("StorageServer", storageServers[j]) + .detail("ShardBegin", printable(range.begin)) + .detail("ShardEnd", printable(range.end)) + .detail("Address", storageServerInterfaces[j].address()) + .detail("UID", storageServerInterfaces[j].id()) + .detail("GetKeyValuesToken", storageServerInterfaces[j].getKeyValues.getEndpoint().token) + .detail("IsTSS", storageServerInterfaces[j].isTss() ? "True" : "False"); + + // All shards should be available in quiscence + if (self->performQuiescentChecks && !storageServerInterfaces[j].isTss()) { + self->testFailure("Storage server unavailable"); + return false; + } + } + } + + if (firstValidServer >= 0) { + VectorRef data = keyValueFutures[firstValidServer].get().get().data; + // Calculate the size of the shard, the variance of the shard size estimate, and the correct + // shard size estimate + for (int k = 0; k < data.size(); k++) { + ByteSampleInfo sampleInfo = isKeyValueInSample(data[k]); + shardBytes += sampleInfo.size; + double itemProbability = ((double)sampleInfo.size) / sampleInfo.sampledSize; + if (itemProbability < 1) + shardVariance += + itemProbability * (1 - itemProbability) * pow((double)sampleInfo.sampledSize, 2); + + if (sampleInfo.inSample) { + sampledBytes += sampleInfo.sampledSize; + if (!canSplit && sampledBytes >= shardBounds.min.bytes && + data[k].key.size() <= CLIENT_KNOBS->SPLIT_KEY_SIZE_LIMIT && + sampledBytes <= + shardBounds.max.bytes * CLIENT_KNOBS->STORAGE_METRICS_UNFAIR_SPLIT_LIMIT / 2) { + canSplit = true; + splitBytes = sampledBytes; + } + + /*TraceEvent("ConsistencyCheck_ByteSample").detail("ShardBegin", printable(range.begin)).detail("ShardEnd", printable(range.end)) + .detail("SampledBytes", sampleInfo.sampledSize).detail("Key", + printable(data[k].key)).detail("KeySize", data[k].key.size()).detail("ValueSize", + data[k].value.size());*/ + + // In data distribution, the splitting process ignores the first key in a shard. + // Thus, we shouldn't consider it when validating the upper bound of estimated shard + // sizes + if (k == 0) + firstKeySampledBytes += sampleInfo.sampledSize; + + sampledKeys++; + if (itemProbability < 1) { + sampledKeysWithProb++; + } + } + } + + // Accumulate number of keys in this shard + shardKeys += data.size(); + } + // after requesting each shard, enforce rate limit based on how much data will likely be read + if (rateLimitForThisRound > 0) { + wait(rateLimiter->getAllowance(totalReadAmount)); + // Set ratelimit to max allowed if current round has been going on for a while + if (now() - rateLimiterStartTime > + 1.1 * CLIENT_KNOBS->CONSISTENCY_CHECK_ONE_ROUND_TARGET_COMPLETION_TIME && + rateLimitForThisRound != self->rateLimitMax) { + rateLimitForThisRound = self->rateLimitMax; + rateLimiter = Reference(new SpeedLimit(rateLimitForThisRound, 1)); + rateLimiterStartTime = now(); + TraceEvent(SevInfo, "ConsistencyCheck_RateLimitSetMaxForThisRound") + .detail("RateLimit", rateLimitForThisRound); + } + } + bytesReadInRange += totalReadAmount; + bytesReadInthisRound += totalReadAmount; + + // Advance to the next set of entries + if (firstValidServer >= 0 && keyValueFutures[firstValidServer].get().get().more) { + VectorRef result = keyValueFutures[firstValidServer].get().get().data; + ASSERT(result.size() > 0); + begin = firstGreaterThan(result[result.size() - 1].key); + ASSERT(begin.getKey() != allKeys.end); + lastStartSampleKey = lastSampleKey; + } else + break; + } catch (Error& e) { + state Error err = e; + wait(onErrorTr.onError(err)); + TraceEvent("ConsistencyCheck_RetryDataConsistency").error(err); + } + } +} + +ACTOR Future validateRangeShard(StorageServer* self, KeyRange range, std::vector candidates) { + TraceEvent("ServeValidateRangeShardBegin", self->thisServerID) + .detail("Range", req.range) + .detail("Servers", describe(servers)); + + std::vector>> serverListEntries; + + for (const UID& id : candidates) { + serverListEntries.push_back(tr->get(serverListKeyFor(id))); + } + + std::vector> serverListValues = wait(getAll(serverListEntries)); + + std::unordered_map> ssis; + std::string thisDCId; + for (const auto& v : serverListValues) { + const StorageServerInterface ssi = decodeServerListValue(v.get()); + if (ssi.uniqueID == self->thisServerID) { + thisDcId = ssi.locality.describeDcId(); + } + ssis[ssi.locality.describeDcId()].push_back(ssi); + } + for (auto& [dcId, ssiList] : ssis) { + if (dcId != thisDcId) { + if (ssiList.empty()) { + return Void(); + } + } + } +} + +ACTOR Future validateRange(StorageServer* self, KeyRange range) { + TraceEvent("ServeValidateRangeBegin", self->thisServerID).detail("Range", req.range); + + state Transaction(self->cx); + tr.setOption(FDBTransactionOptions::PRIORITY_SYSTEM_IMMEDIATE); + tr.setOption(FDBTransactionOptions::ACCESS_SYSTEM_KEYS); + + state RangeResult currentShards = wait(krmGetRanges( + &tr, keyServersPrefix, range, SERVER_KNOBS->MOVE_SHARD_KRM_ROW_LIMIT, SERVER_KNOBS->MOVE_SHARD_KRM_BYTE_LIMIT)); + ASSERT(!currentShards.empty() && !currentShards.more); + + state RangeResult UIDtoTagMap = wait(tr.getRange(serverTagKeys, CLIENT_KNOBS->TOO_MANY)); + ASSERT(!UIDtoTagMap.more && UIDtoTagMap.size() < CLIENT_KNOBS->TOO_MANY); + + std::vector src; + std::vector dest; + UID srcId, destId; + decodeKeyServersValue(UIDtoTagMap, currentShards[0].value, src, dest, srcId, destId); + + if (dest.empty() || destId != anonymousShardId) { + return Void(); + } + + return Void(); +} + +// Serves FetchCheckpointKeyValuesRequest, reads local checkpoint and sends it to the client over wire. +ACTOR Future validateStorageQ(StorageServer* self, ValidateStorageRequest req) { + wait(self->serveValidateStorageParallelismLock.take(TaskPriority::DefaultYield)); + state FlowLock::Releaser holder(self->serveValidateStorageParallelismLock); + + TraceEvent("ServeValidateStorageBegin", self->thisServerID) + .detail("RequestID", req.requestId) + .detail("Range", req.range); + + state Transaction(self->cx); + tr.setOption(FDBTransactionOptions::PRIORITY_SYSTEM_IMMEDIATE); + tr.setOption(FDBTransactionOptions::ACCESS_SYSTEM_KEYS); + + state RangeResult currentShards = wait(krmGetRanges( + &tr, keyServersPrefix, keys, SERVER_KNOBS->MOVE_SHARD_KRM_ROW_LIMIT, SERVER_KNOBS->MOVE_SHARD_KRM_BYTE_LIMIT)); + ASSERT(!currentShards.empty() && !currentShards.more); + + state RangeResult UIDtoTagMap = wait(tr.getRange(serverTagKeys, CLIENT_KNOBS->TOO_MANY)); + ASSERT(!UIDtoTagMap.more && UIDtoTagMap.size() < CLIENT_KNOBS->TOO_MANY); + + if (KeyRangeRef(currentShards[0].key, currentShards[1].key) != keys) { + throw operation_cancelled(); + } + + std::vector src; + std::vector dest; + UID srcId, destId; + decodeKeyServersValue(UIDtoTagMap, currentShards[0].value, src, dest, srcId, destId); + + if (dest.empty() || destId != anonymousShardId) { + return Void(); + } + + req.reply.send(ValidateStorageResult(req.requestId)); + return Void(); +} + KeyRangeRef StorageServer::clampRangeToTenant(KeyRangeRef range, Optional tenantEntry, Arena& arena) { if (tenantEntry.present()) { return KeyRangeRef(range.begin.startsWith(tenantEntry.get().prefix) ? range.begin : tenantEntry.get().prefix, From e3ecba105e5f099180312bcc825dcec8d994c53c Mon Sep 17 00:00:00 2001 From: Ankita Kejriwal Date: Wed, 17 Aug 2022 14:58:09 -0700 Subject: [PATCH 004/210] Monitor storage bytes used by tenants --- fdbrpc/include/fdbrpc/TenantInfo.h | 2 + fdbserver/DataDistribution.actor.cpp | 3 ++ fdbserver/TenantCache.actor.cpp | 57 +++++++++++++++++++++++ fdbserver/include/fdbserver/TCInfo.h | 1 + fdbserver/include/fdbserver/TenantCache.h | 8 ++++ 5 files changed, 71 insertions(+) diff --git a/fdbrpc/include/fdbrpc/TenantInfo.h b/fdbrpc/include/fdbrpc/TenantInfo.h index 8d3ea46cb9..bfea219284 100644 --- a/fdbrpc/include/fdbrpc/TenantInfo.h +++ b/fdbrpc/include/fdbrpc/TenantInfo.h @@ -41,6 +41,8 @@ struct TenantInfo { // Is set during deserialization. It will be set to true if the tenant // name is set and the client is authorized to use this tenant. bool tenantAuthorized = false; + // Number of storage bytes currently used by this tenant. + int64_t storageUsage = -1; // Helper function for most endpoints that read/write data. This returns true iff // the client is either a) a trusted peer or b) is accessing keyspace belonging to a tenant, diff --git a/fdbserver/DataDistribution.actor.cpp b/fdbserver/DataDistribution.actor.cpp index abd21e1db3..2e45944c74 100644 --- a/fdbserver/DataDistribution.actor.cpp +++ b/fdbserver/DataDistribution.actor.cpp @@ -602,9 +602,12 @@ ACTOR Future dataDistribution(Reference self, } else { anyZeroHealthyTeams = zeroHealthyTeams[0]; } + if (ddIsTenantAware) { actors.push_back(reportErrorsExcept( ddTenantCache->monitorTenantMap(), "DDTenantCacheMonitor", self->ddId, &normalDDQueueErrors())); + actors.push_back(reportErrorsExcept( + ddTenantCache->monitorStorageUsage(), "StorageUsageTracker", self->ddId, &normalDDQueueErrors())); } actors.push_back(self->pollMoveKeysLock(ddEnabledState)); diff --git a/fdbserver/TenantCache.actor.cpp b/fdbserver/TenantCache.actor.cpp index e5111d69e4..fdb85ff8ae 100644 --- a/fdbserver/TenantCache.actor.cpp +++ b/fdbserver/TenantCache.actor.cpp @@ -111,6 +111,44 @@ public: } } } + + ACTOR static Future monitorStorageUsage(TenantCache* tenantCache) { + TraceEvent(SevInfo, "StartingTenantCacheStorageUsageMonitor", tenantCache->id()).log(); + + state ReadYourWritesTransaction tr(tenantCache->dbcx()); + + // Reuse the TENANT_CACHE_LIST_REFRESH_INTERVAL knob for now. + state int refreshInterval = SERVER_KNOBS->TENANT_CACHE_LIST_REFRESH_INTERVAL; + state double lastTenantListFetchTime = now(); + + loop { + try { + if (now() - lastTenantListFetchTime > (2 * refreshInterval)) { + TraceEvent(SevWarn, "TenantCacheGetStorageUsageRefreshDelay", tenantCache->id()).log(); + } + + state std::vector tenantPrefixList = tenantCache->getTenantPrefixList(); + + state KeyRangeRef range("/"_sr, "0"_sr); + state int i; + for (i = 0; i < tenantPrefixList.size(); i++) { + state int64_t size = wait(tr.getEstimatedRangeSizeBytes(range.withPrefix(tenantPrefixList[i]))); + tenantCache->updateStorageUsage(tenantPrefixList[i], size); + } + + lastTenantListFetchTime = now(); + tr.reset(); + wait(delay(refreshInterval)); + } catch (Error& e) { + if (e.code() != error_code_actor_cancelled) { + TraceEvent("TenantCacheGetStorageUsageError", tenantCache->id()) + .errorUnsuppressed(e) + .suppressFor(1.0); + } + wait(tr.onError(e)); + } + } + } }; void TenantCache::insert(TenantName& tenantName, TenantMapEntry& tenant) { @@ -165,6 +203,21 @@ int TenantCache::cleanup() { return tenantsRemoved; } +std::vector TenantCache::getTenantPrefixList() const { + std::vector prefixes; + for (const auto& [prefix, entry] : tenantCache) { + prefixes.push_back(prefix); + } + return prefixes; +} + +void TenantCache::updateStorageUsage(KeyRef prefix, int64_t size) { + auto it = tenantCache.find(prefix); + if (it != tenantCache.end()) { + it->value->updateStorageUsage(size); + } +} + std::string TenantCache::desc() const { std::string s("@Generation: "); s += std::to_string(generation) + " "; @@ -202,6 +255,10 @@ Future TenantCache::monitorTenantMap() { return TenantCacheImpl::monitorTenantMap(this); } +Future TenantCache::monitorStorageUsage() { + return TenantCacheImpl::monitorStorageUsage(this); +} + class TenantCacheUnitTest { public: ACTOR static Future InsertAndTestPresence() { diff --git a/fdbserver/include/fdbserver/TCInfo.h b/fdbserver/include/fdbserver/TCInfo.h index 1abc393c0b..208da74db1 100644 --- a/fdbserver/include/fdbserver/TCInfo.h +++ b/fdbserver/include/fdbserver/TCInfo.h @@ -264,4 +264,5 @@ public: void removeTeam(TCTeamInfo team); void updateCacheGeneration(int64_t generation) { m_cacheGeneration = generation; } int64_t cacheGeneration() const { return m_cacheGeneration; } + void updateStorageUsage(int64_t size) { m_tenantInfo.storageUsage = size; } }; diff --git a/fdbserver/include/fdbserver/TenantCache.h b/fdbserver/include/fdbserver/TenantCache.h index 90c9ee2675..839b575d22 100644 --- a/fdbserver/include/fdbserver/TenantCache.h +++ b/fdbserver/include/fdbserver/TenantCache.h @@ -53,6 +53,12 @@ private: // return count of tenants that were found to be stale and removed from the cache int cleanup(); + // return a list of all tenant prefixes stored in the tenant cache + std::vector getTenantPrefixList() const; + + // update the size for a tenant; do nothing if the tenant doesn't exist in the map + void updateStorageUsage(KeyRef prefix, int64_t size); + UID id() const { return distributorID; } Database dbcx() const { return cx; } @@ -66,6 +72,8 @@ public: Future monitorTenantMap(); + Future monitorStorageUsage(); + std::string desc() const; bool isTenantKey(KeyRef key) const; From 7cca369f6662ecce6b0b61d7b60117c95d8479df Mon Sep 17 00:00:00 2001 From: He Liu Date: Wed, 17 Aug 2022 12:03:17 -0700 Subject: [PATCH 005/210] Implemented validateRangeAgainstServer. --- .../KeyValueStoreShardedRocksDB.actor.cpp | 2 +- fdbserver/storageserver.actor.cpp | 483 ++++++------------ flow/include/flow/error_definitions.h | 1 + 3 files changed, 171 insertions(+), 315 deletions(-) diff --git a/fdbserver/KeyValueStoreShardedRocksDB.actor.cpp b/fdbserver/KeyValueStoreShardedRocksDB.actor.cpp index cfb616a8e7..6e6e0ac17a 100644 --- a/fdbserver/KeyValueStoreShardedRocksDB.actor.cpp +++ b/fdbserver/KeyValueStoreShardedRocksDB.actor.cpp @@ -647,7 +647,7 @@ public: RangeResult metadata; readRangeInDb(metadataShard.get(), prefixRange(shardMappingPrefix), UINT16_MAX, UINT16_MAX, &metadata); - std::vector> mapping = decodeShardMapping(metadata, shardMappingPrefix) + std::vector> mapping = decodeShardMapping(metadata, shardMappingPrefix); for (const auto& [range, name] : mapping) { TraceEvent(SevVerbose, "ShardedRocksLoadRange", this->logId) diff --git a/fdbserver/storageserver.actor.cpp b/fdbserver/storageserver.actor.cpp index 5b5b0c27c5..366a89c027 100644 --- a/fdbserver/storageserver.actor.cpp +++ b/fdbserver/storageserver.actor.cpp @@ -3492,333 +3492,176 @@ ACTOR Future readRange(StorageServer* data, return result; } -ACTOR Future validateRangeAgainstServer(StorageServer* self, +ACTOR Future validateRangeAgainstServer(StorageServer* data, KeyRange range, Version version, StorageServerInterface remoteServer) { - TraceEvent("ServeValidateRangeAgainstServerBegin", self->thisServerID) + TraceEvent("ServeValidateRangeAgainstServerBegin", data->thisServerID) .detail("Range", range) .detail("Version", version) - .detail("Servers", remoteServer); - Key begin = range.begin; + .detail("Servers", remoteServer.toString()); loop { try { state GetKeyValuesRequest req; - req.begin = begin; - req.end = range.end; - req.limit = 1e4; - req.limitBytes = CLIENT_KNOBS->REPLY_BYTE_LIMIT; + state int limit = 1e4; + state int limitBytes = CLIENT_KNOBS->REPLY_BYTE_LIMIT; + req.begin = firstGreaterOrEqual(range.begin); + req.end = firstGreaterOrEqual(range.end); + req.limit = limit; + req.limitBytes = limitBytes; req.version = version; req.tags = TagSet(); // Try getting the entries in the specified range - state ErrorOr keyValueFuture = - wait(remoteServer.getKeyValues.getReplyUnlessFailedFor(req, 2, 0)); + state Future> remoteKeyValueFuture = + remoteServer.getKeyValues.getReplyUnlessFailedFor(req, 2, 0); + state Future localKeyValueFuture = readRange(data, + version, + range, + limit, + &limitBytes, + SpanContext(), + IKeyValueStore::ReadType::LOW, + Optional()); + state ErrorOr remoteResult = wait(remoteKeyValueFuture); + GetKeyValuesReply local = wait(localKeyValueFuture); + Key lastKey = range.begin; + state std::string error; - // Read the resulting entries - state int firstValidServer = -1; - totalReadAmount = 0; - for (j = 0; j < keyValueFutures.size(); j++) { - ErrorOr rangeResult = keyValueFutures[j].get(); - - // Compare the results with other storage servers - if (rangeResult.present() && !rangeResult.get().error.present()) { - state GetKeyValuesReply current = rangeResult.get(); - totalReadAmount += current.data.expectedSize(); - // If we haven't encountered a valid storage server yet, then mark this as the baseline - // to compare against - if (firstValidServer == -1) - firstValidServer = j; - - // Compare this shard against the first - else { - GetKeyValuesReply reference = keyValueFutures[firstValidServer].get().get(); - - if (current.data != reference.data || current.more != reference.more) { - // Be especially verbose if in simulation - if (g_network->isSimulated()) { - int invalidIndex = -1; - printf("\n%sSERVER %d (%s); shard = %s - %s:\n", - storageServerInterfaces[j].isTss() ? "TSS " : "", - j, - storageServerInterfaces[j].address().toString().c_str(), - printable(req.begin.getKey()).c_str(), - printable(req.end.getKey()).c_str()); - for (int k = 0; k < current.data.size(); k++) { - printf("%d. %s => %s\n", - k, - printable(current.data[k].key).c_str(), - printable(current.data[k].value).c_str()); - if (invalidIndex < 0 && - (k >= reference.data.size() || current.data[k].key != reference.data[k].key || - current.data[k].value != reference.data[k].value)) - invalidIndex = k; - } - - printf("\n%sSERVER %d (%s); shard = %s - %s:\n", - storageServerInterfaces[firstValidServer].isTss() ? "TSS " : "", - firstValidServer, - storageServerInterfaces[firstValidServer].address().toString().c_str(), - printable(req.begin.getKey()).c_str(), - printable(req.end.getKey()).c_str()); - for (int k = 0; k < reference.data.size(); k++) { - printf("%d. %s => %s\n", - k, - printable(reference.data[k].key).c_str(), - printable(reference.data[k].value).c_str()); - if (invalidIndex < 0 && - (k >= current.data.size() || reference.data[k].key != current.data[k].key || - reference.data[k].value != current.data[k].value)) - invalidIndex = k; - } - - printf("\nMISMATCH AT %d\n\n", invalidIndex); - } - - // Data for trace event - // The number of keys unique to the current shard - int currentUniques = 0; - // The number of keys unique to the reference shard - int referenceUniques = 0; - // The number of keys in both shards with conflicting values - int valueMismatches = 0; - // The number of keys in both shards with matching values - int matchingKVPairs = 0; - // Last unique key on the current shard - KeyRef currentUniqueKey; - // Last unique key on the reference shard - KeyRef referenceUniqueKey; - // Last value mismatch - KeyRef valueMismatchKey; - - // Loop indeces - int currentI = 0; - int referenceI = 0; - while (currentI < current.data.size() || referenceI < reference.data.size()) { - if (currentI >= current.data.size()) { - referenceUniqueKey = reference.data[referenceI].key; - referenceUniques++; - referenceI++; - } else if (referenceI >= reference.data.size()) { - currentUniqueKey = current.data[currentI].key; - currentUniques++; - currentI++; - } else { - KeyValueRef currentKV = current.data[currentI]; - KeyValueRef referenceKV = reference.data[referenceI]; - - if (currentKV.key == referenceKV.key) { - if (currentKV.value == referenceKV.value) - matchingKVPairs++; - else { - valueMismatchKey = currentKV.key; - valueMismatches++; - } - - currentI++; - referenceI++; - } else if (currentKV.key < referenceKV.key) { - currentUniqueKey = currentKV.key; - currentUniques++; - currentI++; - } else { - referenceUniqueKey = referenceKV.key; - referenceUniques++; - referenceI++; - } - } - } - - TraceEvent("ConsistencyCheck_DataInconsistent") - .detail(format("StorageServer%d", j).c_str(), storageServers[j].toString()) - .detail(format("StorageServer%d", firstValidServer).c_str(), - storageServers[firstValidServer].toString()) - .detail("ShardBegin", req.begin.getKey()) - .detail("ShardEnd", req.end.getKey()) - .detail("VersionNumber", req.version) - .detail(format("Server%dUniques", j).c_str(), currentUniques) - .detail(format("Server%dUniqueKey", j).c_str(), currentUniqueKey) - .detail(format("Server%dUniques", firstValidServer).c_str(), referenceUniques) - .detail(format("Server%dUniqueKey", firstValidServer).c_str(), referenceUniqueKey) - .detail("ValueMismatches", valueMismatches) - .detail("ValueMismatchKey", valueMismatchKey) - .detail("MatchingKVPairs", matchingKVPairs) - .detail("IsTSS", - storageServerInterfaces[j].isTss() || - storageServerInterfaces[firstValidServer].isTss() - ? "True" - : "False"); - - if ((g_network->isSimulated() && - g_simulator.tssMode != ISimulator::TSSMode::EnabledDropMutations) || - (!storageServerInterfaces[j].isTss() && - !storageServerInterfaces[firstValidServer].isTss())) { - self->testFailure("Data inconsistent", true); - return false; - } - } - } - } - - // If the data is not available and we aren't relocating this shard - else if (!isRelocating) { - Error e = rangeResult.isError() ? rangeResult.getError() : rangeResult.get().error.get(); - - TraceEvent("ConsistencyCheck_StorageServerUnavailable") - .errorUnsuppressed(e) - .suppressFor(1.0) - .detail("StorageServer", storageServers[j]) - .detail("ShardBegin", printable(range.begin)) - .detail("ShardEnd", printable(range.end)) - .detail("Address", storageServerInterfaces[j].address()) - .detail("UID", storageServerInterfaces[j].id()) - .detail("GetKeyValuesToken", storageServerInterfaces[j].getKeyValues.getEndpoint().token) - .detail("IsTSS", storageServerInterfaces[j].isTss() ? "True" : "False"); - - // All shards should be available in quiscence - if (self->performQuiescentChecks && !storageServerInterfaces[j].isTss()) { - self->testFailure("Storage server unavailable"); - return false; - } - } + // Compare the results with other storage servers + if (remoteResult.isError()) { + throw remoteResult.getError(); } - if (firstValidServer >= 0) { - VectorRef data = keyValueFutures[firstValidServer].get().get().data; - // Calculate the size of the shard, the variance of the shard size estimate, and the correct - // shard size estimate - for (int k = 0; k < data.size(); k++) { - ByteSampleInfo sampleInfo = isKeyValueInSample(data[k]); - shardBytes += sampleInfo.size; - double itemProbability = ((double)sampleInfo.size) / sampleInfo.sampledSize; - if (itemProbability < 1) - shardVariance += - itemProbability * (1 - itemProbability) * pow((double)sampleInfo.sampledSize, 2); + state GetKeyValuesReply remote = remoteResult.get(); + // Loop indeces + const int end = std::min(local.data.size(), remote.data.size()); + int i = 0; + for (; i < end; ++i) { + KeyValueRef remoteKV = remote.data[i]; + KeyValueRef localKV = local.data[i]; - if (sampleInfo.inSample) { - sampledBytes += sampleInfo.sampledSize; - if (!canSplit && sampledBytes >= shardBounds.min.bytes && - data[k].key.size() <= CLIENT_KNOBS->SPLIT_KEY_SIZE_LIMIT && - sampledBytes <= - shardBounds.max.bytes * CLIENT_KNOBS->STORAGE_METRICS_UNFAIR_SPLIT_LIMIT / 2) { - canSplit = true; - splitBytes = sampledBytes; - } - - /*TraceEvent("ConsistencyCheck_ByteSample").detail("ShardBegin", printable(range.begin)).detail("ShardEnd", printable(range.end)) - .detail("SampledBytes", sampleInfo.sampledSize).detail("Key", - printable(data[k].key)).detail("KeySize", data[k].key.size()).detail("ValueSize", - data[k].value.size());*/ - - // In data distribution, the splitting process ignores the first key in a shard. - // Thus, we shouldn't consider it when validating the upper bound of estimated shard - // sizes - if (k == 0) - firstKeySampledBytes += sampleInfo.sampledSize; - - sampledKeys++; - if (itemProbability < 1) { - sampledKeysWithProb++; - } - } + if (remoteKV.key != localKV.key) { + error = format("Key Mismatch: local server (%lld): %s, remote server(%lld) %s", + data->thisServerID.first(), + Traceable::toString(localKV.key), + remoteServer.uniqueID.first(), + Traceable::toString(remoteKV.key)); } - // Accumulate number of keys in this shard - shardKeys += data.size(); - } - // after requesting each shard, enforce rate limit based on how much data will likely be read - if (rateLimitForThisRound > 0) { - wait(rateLimiter->getAllowance(totalReadAmount)); - // Set ratelimit to max allowed if current round has been going on for a while - if (now() - rateLimiterStartTime > - 1.1 * CLIENT_KNOBS->CONSISTENCY_CHECK_ONE_ROUND_TARGET_COMPLETION_TIME && - rateLimitForThisRound != self->rateLimitMax) { - rateLimitForThisRound = self->rateLimitMax; - rateLimiter = Reference(new SpeedLimit(rateLimitForThisRound, 1)); - rateLimiterStartTime = now(); - TraceEvent(SevInfo, "ConsistencyCheck_RateLimitSetMaxForThisRound") - .detail("RateLimit", rateLimitForThisRound); + if (remoteKV.value != localKV.value) { + error = format("Value Mismatch for Key %s: local server (%lld): %s, remote server(%lld) %s", + Traceable::toString(localKV.key), + data->thisServerID.first(), + Traceable::toString(localKV.value), + remoteServer.uniqueID.first(), + Traceable::toString(remoteKV.value)); } - } - bytesReadInRange += totalReadAmount; - bytesReadInthisRound += totalReadAmount; - // Advance to the next set of entries - if (firstValidServer >= 0 && keyValueFutures[firstValidServer].get().get().more) { - VectorRef result = keyValueFutures[firstValidServer].get().get().data; - ASSERT(result.size() > 0); - begin = firstGreaterThan(result[result.size() - 1].key); - ASSERT(begin.getKey() != allKeys.end); - lastStartSampleKey = lastSampleKey; - } else + lastKey = localKV.key; + } + + if (!error.empty()) { break; - } catch (Error& e) { - state Error err = e; - wait(onErrorTr.onError(err)); - TraceEvent("ConsistencyCheck_RetryDataConsistency").error(err); - } - } -} - -ACTOR Future validateRangeShard(StorageServer* self, KeyRange range, std::vector candidates) { - TraceEvent("ServeValidateRangeShardBegin", self->thisServerID) - .detail("Range", req.range) - .detail("Servers", describe(servers)); - - std::vector>> serverListEntries; - - for (const UID& id : candidates) { - serverListEntries.push_back(tr->get(serverListKeyFor(id))); - } - - std::vector> serverListValues = wait(getAll(serverListEntries)); - - std::unordered_map> ssis; - std::string thisDCId; - for (const auto& v : serverListValues) { - const StorageServerInterface ssi = decodeServerListValue(v.get()); - if (ssi.uniqueID == self->thisServerID) { - thisDcId = ssi.locality.describeDcId(); - } - ssis[ssi.locality.describeDcId()].push_back(ssi); - } - for (auto& [dcId, ssiList] : ssis) { - if (dcId != thisDcId) { - if (ssiList.empty()) { - return Void(); } + + if (!local.more && !remote.more && local.data.size() == remote.data.size()) { + break; + } else if (i >= local.data.size() && !local.more && i < remote.data.size()) { + error = format("Missing key(s) form local server (%lld), next remote server(%lld) key: %s", + data->thisServerID.first(), + remoteServer.uniqueID.first(), + Traceable::toString(remote.data[i].key)); + break; + } else if (i >= remote.data.size() && !remote.more && i < local.data.size()) { + error = format("Missing key(s) form remote server (%lld), next local server(%lld) key: %s", + remoteServer.uniqueID.first(), + data->thisServerID.first(), + Traceable::toString(local.data[i].key)); + break; + } + + range = KeyRangeRef(keyAfter(lastKey), range.end); + } catch (Error& e) { + TraceEvent(SevWarnAlways, "ValidateRangeAgainstServerError", data->thisServerID) + .errorUnsuppressed(e) + .detail("RemoteServer", remoteServer.toString()) + .detail("Range", range) + .detail("Version", version); + throw e; } } -} -ACTOR Future validateRange(StorageServer* self, KeyRange range) { - TraceEvent("ServeValidateRangeBegin", self->thisServerID).detail("Range", req.range); - - state Transaction(self->cx); - tr.setOption(FDBTransactionOptions::PRIORITY_SYSTEM_IMMEDIATE); - tr.setOption(FDBTransactionOptions::ACCESS_SYSTEM_KEYS); - - state RangeResult currentShards = wait(krmGetRanges( - &tr, keyServersPrefix, range, SERVER_KNOBS->MOVE_SHARD_KRM_ROW_LIMIT, SERVER_KNOBS->MOVE_SHARD_KRM_BYTE_LIMIT)); - ASSERT(!currentShards.empty() && !currentShards.more); - - state RangeResult UIDtoTagMap = wait(tr.getRange(serverTagKeys, CLIENT_KNOBS->TOO_MANY)); - ASSERT(!UIDtoTagMap.more && UIDtoTagMap.size() < CLIENT_KNOBS->TOO_MANY); - - std::vector src; - std::vector dest; - UID srcId, destId; - decodeKeyServersValue(UIDtoTagMap, currentShards[0].value, src, dest, srcId, destId); - - if (dest.empty() || destId != anonymousShardId) { - return Void(); + if (!error.empty()) { + TraceEvent(SevWarnAlways, "ValidateRangeAgainstServerError", data->thisServerID) + .detail("Range", range) + .detail("Version", version) + .detail("ErrorMessage", error) + .detail("RemoteServer", remoteServer.toString()); + throw validate_storage_error(); + } + + return Void(); +} + +ACTOR Future validateRangeShard(StorageServer* data, KeyRange range, std::vector candidates) { + TraceEvent("ServeValidateRangeShardBegin", data->thisServerID) + .detail("Range", range) + .detail("Servers", describe(candidates)); + + state Version version; + state std::vector> serverListValues; + state Transaction tr(data->cx); + tr.setOption(FDBTransactionOptions::PRIORITY_SYSTEM_IMMEDIATE); + tr.setOption(FDBTransactionOptions::ACCESS_SYSTEM_KEYS); + + loop { + try { + std::vector>> serverListEntries; + for (const UID& id : candidates) { + serverListEntries.push_back(tr.get(serverListKeyFor(id))); + } + + std::vector> serverListValues_ = wait(getAll(serverListEntries)); + serverListValues = serverListValues_; + Version version_ = wait(tr.getReadVersion()); + version = version_; + break; + } catch (Error& e) { + wait(tr.onError(e)); + } + } + + std::unordered_map> ssis; + std::string thisDcId; + for (const auto& v : serverListValues) { + const StorageServerInterface ssi = decodeServerListValue(v.get()); + if (ssi.uniqueID == data->thisServerID) { + thisDcId = ssi.locality.describeDcId(); + } + ssis[ssi.locality.describeDcId()].push_back(ssi); + } + + StorageServerInterface* remoteServer = nullptr; + for (auto& [dcId, ssiList] : ssis) { + if (dcId != thisDcId) { + if (ssiList.empty()) { + break; + } + const int idx = deterministicRandom()->randomInt(0, ssiList.size()); + remoteServer = &ssiList[idx]; + break; + } + } + + if (remoteServer != nullptr) { + wait(validateRangeAgainstServer(data, range, version, *remoteServer)); } return Void(); } -// Serves FetchCheckpointKeyValuesRequest, reads local checkpoint and sends it to the client over wire. ACTOR Future validateStorageQ(StorageServer* self, ValidateStorageRequest req) { wait(self->serveValidateStorageParallelismLock.take(TaskPriority::DefaultYield)); state FlowLock::Releaser holder(self->serveValidateStorageParallelismLock); @@ -3827,31 +3670,43 @@ ACTOR Future validateStorageQ(StorageServer* self, ValidateStorageRequest .detail("RequestID", req.requestId) .detail("Range", req.range); - state Transaction(self->cx); + state Transaction tr(self->cx); tr.setOption(FDBTransactionOptions::PRIORITY_SYSTEM_IMMEDIATE); tr.setOption(FDBTransactionOptions::ACCESS_SYSTEM_KEYS); - state RangeResult currentShards = wait(krmGetRanges( - &tr, keyServersPrefix, keys, SERVER_KNOBS->MOVE_SHARD_KRM_ROW_LIMIT, SERVER_KNOBS->MOVE_SHARD_KRM_BYTE_LIMIT)); - ASSERT(!currentShards.empty() && !currentShards.more); + loop { + try { + state RangeResult shards = wait(krmGetRanges(&tr, + keyServersPrefix, + req.range, + SERVER_KNOBS->MOVE_SHARD_KRM_ROW_LIMIT, + SERVER_KNOBS->MOVE_SHARD_KRM_BYTE_LIMIT)); + ASSERT(!shards.empty() && !shards.more); - state RangeResult UIDtoTagMap = wait(tr.getRange(serverTagKeys, CLIENT_KNOBS->TOO_MANY)); - ASSERT(!UIDtoTagMap.more && UIDtoTagMap.size() < CLIENT_KNOBS->TOO_MANY); + state RangeResult UIDtoTagMap = wait(tr.getRange(serverTagKeys, CLIENT_KNOBS->TOO_MANY)); + ASSERT(!UIDtoTagMap.more && UIDtoTagMap.size() < CLIENT_KNOBS->TOO_MANY); - if (KeyRangeRef(currentShards[0].key, currentShards[1].key) != keys) { - throw operation_cancelled(); + std::vector> fs; + for (int i = 0; i < shards.size() - 1; ++i) { + std::vector src; + std::vector dest; + UID srcId, destId; + decodeKeyServersValue(UIDtoTagMap, shards[i].value, src, dest, srcId, destId); + fs.push_back(validateRangeShard(self, KeyRangeRef(shards[i].key, shards[i + 1].key), src)); + } + + wait(waitForAll(fs)); + req.reply.send(ValidateStorageResult(req.requestId)); + break; + } catch (Error& e) { + if (e.code() == error_code_validate_storage_error) { + req.reply.sendError(e); + break; + } + wait(tr.onError(e)); + } } - std::vector src; - std::vector dest; - UID srcId, destId; - decodeKeyServersValue(UIDtoTagMap, currentShards[0].value, src, dest, srcId, destId); - - if (dest.empty() || destId != anonymousShardId) { - return Void(); - } - - req.reply.send(ValidateStorageResult(req.requestId)); return Void(); } diff --git a/flow/include/flow/error_definitions.h b/flow/include/flow/error_definitions.h index bc75885925..9a1c9fb819 100755 --- a/flow/include/flow/error_definitions.h +++ b/flow/include/flow/error_definitions.h @@ -125,6 +125,7 @@ ERROR( invalid_cluster_id, 1217, "Attempted to join cluster with a different clu ERROR( restart_cluster_controller, 1218, "Restart cluster controller process" ) ERROR( please_reboot_remote_kv_store, 1219, "Need to reboot the storage engine process as it died abnormally") ERROR( incompatible_software_version, 1220, "Current software does not support database format" ) +ERROR( validate_storage_error, 1221, "Validate storage consistency error" ) // 15xx Platform errors ERROR( platform_error, 1500, "Platform error" ) From 13cd4b973ca26169e4b2dc8bb084f56b8c892006 Mon Sep 17 00:00:00 2001 From: He Liu Date: Wed, 17 Aug 2022 17:09:21 -0700 Subject: [PATCH 006/210] Added ValidateStorage.actor.cpp --- fdbserver/workloads/ValidateStorage.actor.cpp | 197 ++++++++++++++++++ 1 file changed, 197 insertions(+) create mode 100644 fdbserver/workloads/ValidateStorage.actor.cpp diff --git a/fdbserver/workloads/ValidateStorage.actor.cpp b/fdbserver/workloads/ValidateStorage.actor.cpp new file mode 100644 index 0000000000..76986a2322 --- /dev/null +++ b/fdbserver/workloads/ValidateStorage.actor.cpp @@ -0,0 +1,197 @@ +/* + * ValidateStorage.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/ManagementAPI.actor.h" +#include "fdbclient/NativeAPI.actor.h" +#include "fdbrpc/simulator.h" +#include "fdbserver/IKeyValueStore.h" +#include "fdbserver/ServerCheckpoint.actor.h" +#include "fdbserver/MoveKeys.actor.h" +#include "fdbserver/QuietDatabase.h" +#include "fdbserver/workloads/workloads.actor.h" +#include "flow/Error.h" +#include "flow/IRandom.h" +#include "flow/flow.h" +#include +#include + +#include "flow/actorcompiler.h" // This must be the last #include. + +struct ValidateStorage : TestWorkload { + FlowLock startMoveKeysParallelismLock; + FlowLock finishMoveKeysParallelismLock; + FlowLock cleanUpDataMoveParallelismLock; + const bool enabled; + bool pass; + + ValidateStorage(WorkloadContext const& wcx) : TestWorkload(wcx), enabled(!clientId), pass(true) {} + + std::string description() const override { return "ValidateStorage"; } + + Future setup(Database const& cx) override { return Void(); } + + Future start(Database const& cx) override { + if (!enabled) { + return Void(); + } + return _start(this, cx); + } + + ACTOR Future _start(ValidateStorage* self, Database cx) { + // int ignore = wait(setDDMode(cx, 0)); + state std::map kvs({ { "TestKeyA"_sr, "TestValueA"_sr }, + { "TestKeyB"_sr, "TestValueB"_sr }, + { "TestKeyC"_sr, "TestValueC"_sr }, + { "TestKeyD"_sr, "TestValueD"_sr }, + { "TestKeyE"_sr, "TestValueE"_sr }, + { "TestKeyF"_sr, "TestValueF"_sr } }); + + Version _ = wait(self->populateData(self, cx, &kvs)); + + TraceEvent("TestValueWritten").log(); + + wait(self->validateData(self, cx, KeyRangeRef("TestKeyA"_sr, "TestKeyF"_sr), &kvs)); + TraceEvent("TestValueVerified").log(); + + int ignore = wait(setDDMode(cx, 1)); + return Void(); + } + + ACTOR Future populateData(ValidateStorage* self, Database cx, std::map* kvs) { + state Reference tr = makeReference(cx); + state Version version; + loop { + state UID debugID = deterministicRandom()->randomUniqueID(); + try { + tr->debugTransaction(debugID); + for (const auto& [key, value] : *kvs) { + tr->set(key, value); + } + wait(tr->commit()); + version = tr->getCommittedVersion(); + break; + } catch (Error& e) { + TraceEvent("TestCommitError").errorUnsuppressed(e); + wait(tr->onError(e)); + } + } + + TraceEvent("PopulateTestDataDone") + .detail("CommitVersion", tr->getCommittedVersion()) + .detail("DebugID", debugID); + + return version; + } + + ACTOR Future validateData(ValidateStorage* self, + Database cx, + KeyRange range, + std::map* kvs) { + state Transaction tr(cx); + loop { + state UID debugID = deterministicRandom()->randomUniqueID(); + try { + tr.debugTransaction(debugID); + RangeResult res = wait(tr.getRange(range, CLIENT_KNOBS->TOO_MANY)); + ASSERT(!res.more && res.size() < CLIENT_KNOBS->TOO_MANY); + + for (const auto& kv : res) { + ASSERT((*kvs)[kv.key] == kv.value); + } + break; + } catch (Error& e) { + TraceEvent("TestCommitError").errorUnsuppressed(e); + wait(tr.onError(e)); + } + } + + TraceEvent("ValidateTestDataDone").detail("DebugID", debugID); + + return Void(); + } + + ACTOR Future readAndVerify(ValidateStorage* self, + Database cx, + Key key, + ErrorOr> expectedValue) { + state Transaction tr(cx); + tr.setOption(FDBTransactionOptions::ACCESS_SYSTEM_KEYS); + + loop { + try { + state Version readVersion = wait(tr.getReadVersion()); + state Optional res = wait(timeoutError(tr.get(key), 30.0)); + const bool equal = !expectedValue.isError() && res == expectedValue.get(); + if (!equal) { + self->validationFailed(expectedValue, ErrorOr>(res)); + } + break; + } catch (Error& e) { + TraceEvent("TestReadError").errorUnsuppressed(e); + if (expectedValue.isError() && expectedValue.getError().code() == e.code()) { + break; + } + wait(tr.onError(e)); + } + } + + TraceEvent("TestReadSuccess").detail("Version", readVersion); + + return Void(); + } + + ACTOR Future writeAndVerify(ValidateStorage* self, Database cx, Key key, Optional value) { + // state Transaction tr(cx); + state Reference tr = makeReference(cx); + state Version version; + loop { + state UID debugID = deterministicRandom()->randomUniqueID(); + try { + tr->setOption(FDBTransactionOptions::ACCESS_SYSTEM_KEYS); + tr->debugTransaction(debugID); + if (value.present()) { + tr->set(key, value.get()); + tr->set("Test?"_sr, value.get()); + tr->set(key, value.get()); + } else { + tr->clear(key); + } + wait(timeoutError(tr->commit(), 30.0)); + version = tr->getCommittedVersion(); + break; + } catch (Error& e) { + TraceEvent("TestCommitError").errorUnsuppressed(e); + wait(tr->onError(e)); + } + } + + TraceEvent("TestCommitSuccess").detail("CommitVersion", tr->getCommittedVersion()).detail("DebugID", debugID); + + wait(self->readAndVerify(self, cx, key, value)); + + return version; + } + + Future check(Database const& cx) override { return pass; } + + void getMetrics(std::vector& m) override {} +}; + +WorkloadFactory ValidateStorageFactory("PhysicalShardMove"); \ No newline at end of file From aa5c3b7e9d9fe1ed681ab87d62f2363fd2022f87 Mon Sep 17 00:00:00 2001 From: He Liu Date: Thu, 18 Aug 2022 13:53:42 -0700 Subject: [PATCH 007/210] Added ValidateStorage test. --- fdbserver/storageserver.actor.cpp | 23 ++++-- fdbserver/workloads/ValidateStorage.actor.cpp | 82 +++++++++++++------ tests/CMakeLists.txt | 1 + 3 files changed, 76 insertions(+), 30 deletions(-) diff --git a/fdbserver/storageserver.actor.cpp b/fdbserver/storageserver.actor.cpp index 366a89c027..105964a838 100644 --- a/fdbserver/storageserver.actor.cpp +++ b/fdbserver/storageserver.actor.cpp @@ -1182,8 +1182,11 @@ public: specialCounter(cc, "ServeFetchCheckpointActive", [self]() { return self->serveFetchCheckpointParallelismLock.activePermits(); }); - specialCounter(cc, "ServeFetchCheckpointActive", [self]() { - return self->serveFetchCheckpointParallelismLock.activePermits(); + specialCounter(cc, "ServeFetchCheckpointWaiting", [self]() { + return self->serveFetchCheckpointParallelismLock.waiters(); + }); + specialCounter(cc, "ServeValidateStorageActive", [self]() { + return self->serveValidateStorageParallelismLock.activePermits(); }); specialCounter(cc, "ServeValidateStorageWaiting", [self]() { return self->serveValidateStorageParallelismLock.waiters(); @@ -3496,11 +3499,12 @@ ACTOR Future validateRangeAgainstServer(StorageServer* data, KeyRange range, Version version, StorageServerInterface remoteServer) { - TraceEvent("ServeValidateRangeAgainstServerBegin", data->thisServerID) + TraceEvent(SevDebug, "ServeValidateRangeAgainstServerBegin", data->thisServerID) .detail("Range", range) .detail("Version", version) .detail("Servers", remoteServer.toString()); + state int validatedKeys = 0; loop { try { state GetKeyValuesRequest req; @@ -3548,15 +3552,16 @@ ACTOR Future validateRangeAgainstServer(StorageServer* data, Traceable::toString(localKV.key), remoteServer.uniqueID.first(), Traceable::toString(remoteKV.key)); - } - - if (remoteKV.value != localKV.value) { + } else if (remoteKV.value != localKV.value) { error = format("Value Mismatch for Key %s: local server (%lld): %s, remote server(%lld) %s", Traceable::toString(localKV.key), data->thisServerID.first(), Traceable::toString(localKV.value), remoteServer.uniqueID.first(), Traceable::toString(remoteKV.value)); + } else { + TraceEvent(SevDebug, "ValidatedKey", data->thisServerID).detail("Key", localKV.key); + ++validatedKeys; } lastKey = localKV.key; @@ -3602,6 +3607,12 @@ ACTOR Future validateRangeAgainstServer(StorageServer* data, throw validate_storage_error(); } + TraceEvent(SevDebug, "ServeValidateRangeAgainstServerEnd", data->thisServerID) + .detail("Range", range) + .detail("Version", version) + .detail("ValidatedKeys", validatedKeys) + .detail("Servers", remoteServer.toString()); + return Void(); } diff --git a/fdbserver/workloads/ValidateStorage.actor.cpp b/fdbserver/workloads/ValidateStorage.actor.cpp index 76986a2322..5c095d11ee 100644 --- a/fdbserver/workloads/ValidateStorage.actor.cpp +++ b/fdbserver/workloads/ValidateStorage.actor.cpp @@ -1,5 +1,5 @@ /* - * ValidateStorage.cpp + * ValidateStorage.actor.cpp * * This source file is part of the FoundationDB open source project * @@ -21,10 +21,6 @@ #include "fdbclient/ManagementAPI.actor.h" #include "fdbclient/NativeAPI.actor.h" #include "fdbrpc/simulator.h" -#include "fdbserver/IKeyValueStore.h" -#include "fdbserver/ServerCheckpoint.actor.h" -#include "fdbserver/MoveKeys.actor.h" -#include "fdbserver/QuietDatabase.h" #include "fdbserver/workloads/workloads.actor.h" #include "flow/Error.h" #include "flow/IRandom.h" @@ -34,6 +30,15 @@ #include "flow/actorcompiler.h" // This must be the last #include. +namespace { +std::string printValue(const ErrorOr>& value) { + if (value.isError()) { + return value.getError().name(); + } + return value.get().present() ? value.get().get().toString() : "Value Not Found."; +} +} // namespace + struct ValidateStorage : TestWorkload { FlowLock startMoveKeysParallelismLock; FlowLock finishMoveKeysParallelismLock; @@ -41,6 +46,13 @@ struct ValidateStorage : TestWorkload { const bool enabled; bool pass; + void validationFailed(ErrorOr> expectedValue, ErrorOr> actualValue) { + TraceEvent(SevError, "TestFailed") + .detail("ExpectedValue", printValue(expectedValue)) + .detail("ActualValue", printValue(actualValue)); + pass = false; + } + ValidateStorage(WorkloadContext const& wcx) : TestWorkload(wcx), enabled(!clientId), pass(true) {} std::string description() const override { return "ValidateStorage"; } @@ -65,10 +77,12 @@ struct ValidateStorage : TestWorkload { Version _ = wait(self->populateData(self, cx, &kvs)); - TraceEvent("TestValueWritten").log(); + std::cout << "TestValueWritten" << std::endl; - wait(self->validateData(self, cx, KeyRangeRef("TestKeyA"_sr, "TestKeyF"_sr), &kvs)); - TraceEvent("TestValueVerified").log(); + TraceEvent("TestValueWritten"); + + wait(self->validateData(self, cx, KeyRangeRef("TestKeyA"_sr, "TestKeyF"_sr))); + TraceEvent("TestValueVerified"); int ignore = wait(setDDMode(cx, 1)); return Void(); @@ -100,29 +114,49 @@ struct ValidateStorage : TestWorkload { return version; } - ACTOR Future validateData(ValidateStorage* self, - Database cx, - KeyRange range, - std::map* kvs) { + ACTOR Future validateData(ValidateStorage* self, Database cx, KeyRange range) { + std::cout << "0" << std::endl; state Transaction tr(cx); - loop { - state UID debugID = deterministicRandom()->randomUniqueID(); - try { - tr.debugTransaction(debugID); - RangeResult res = wait(tr.getRange(range, CLIENT_KNOBS->TOO_MANY)); - ASSERT(!res.more && res.size() < CLIENT_KNOBS->TOO_MANY); + tr.setOption(FDBTransactionOptions::PRIORITY_SYSTEM_IMMEDIATE); + tr.setOption(FDBTransactionOptions::ACCESS_SYSTEM_KEYS); - for (const auto& kv : res) { - ASSERT((*kvs)[kv.key] == kv.value); + loop { + try { + std::cout << "1" << std::endl; + state RangeResult shards = + wait(krmGetRanges(&tr, keyServersPrefix, range, CLIENT_KNOBS->TOO_MANY, CLIENT_KNOBS->TOO_MANY)); + ASSERT(!shards.empty() && !shards.more); + + state RangeResult UIDtoTagMap = wait(tr.getRange(serverTagKeys, CLIENT_KNOBS->TOO_MANY)); + ASSERT(!UIDtoTagMap.more && UIDtoTagMap.size() < CLIENT_KNOBS->TOO_MANY); + + state int i = 0; + for (i = 0; i < shards.size() - 1; ++i) { + + std::cout << "2" << std::endl; + std::vector src; + std::vector dest; + UID srcId, destId; + decodeKeyServersValue(UIDtoTagMap, shards[i].value, src, dest, srcId, destId); + + const int idx = deterministicRandom()->randomInt(0, src.size()); + Optional serverListValue = wait(tr.get(serverListKeyFor(src[idx]))); + ASSERT(serverListValue.present()); + const StorageServerInterface ssi = decodeServerListValue(serverListValue.get()); + ValidateStorageRequest req(deterministicRandom()->randomUniqueID(), + KeyRangeRef(shards[i].key, shards[i + 1].key)); + ValidateStorageResult vResult = wait(ssi.validateStorage.getReply(req)); + + std::cout << "3" << std::endl; } break; } catch (Error& e) { - TraceEvent("TestCommitError").errorUnsuppressed(e); + TraceEvent(SevWarnAlways, "TestValidateStorageError").errorUnsuppressed(e).detail("Range", range); wait(tr.onError(e)); } } - TraceEvent("ValidateTestDataDone").detail("DebugID", debugID); + TraceEvent("TestValidateStorageDone").detail("Range", range); return Void(); } @@ -189,9 +223,9 @@ struct ValidateStorage : TestWorkload { return version; } - Future check(Database const& cx) override { return pass; } + Future check(Database const& cx) override { return true; } void getMetrics(std::vector& m) override {} }; -WorkloadFactory ValidateStorageFactory("PhysicalShardMove"); \ No newline at end of file +WorkloadFactory ValidateStorageFactory("ValidateStorageWorkload"); \ No newline at end of file diff --git a/tests/CMakeLists.txt b/tests/CMakeLists.txt index d979dc8d78..28b1adfeff 100644 --- a/tests/CMakeLists.txt +++ b/tests/CMakeLists.txt @@ -193,6 +193,7 @@ if(WITH_PYTHON) add_fdb_test(TEST_FILES fast/WriteDuringReadClean.toml) add_fdb_test(TEST_FILES noSim/RandomUnitTests.toml UNIT) add_fdb_test(TEST_FILES noSim/SystemDataTest.toml UNIT) + add_fdb_test(TEST_FILES fast/ValidateStorage.toml) if (WITH_ROCKSDB_EXPERIMENTAL) add_fdb_test(TEST_FILES noSim/KeyValueStoreRocksDBTest.toml) add_fdb_test(TEST_FILES noSim/ShardedRocksDBTest.toml UNIT) From c47e1b6f53d71a3dd9f9a7470fa659cc67b754eb Mon Sep 17 00:00:00 2001 From: Ankita Kejriwal Date: Tue, 23 Aug 2022 17:52:17 -0700 Subject: [PATCH 008/210] Add a knob for tenant cache storage usage refresh interval and some minor fixes --- fdbclient/ServerKnobs.cpp | 1 + fdbclient/include/fdbclient/ServerKnobs.h | 2 ++ fdbserver/TenantCache.actor.cpp | 17 +++++++++-------- fdbserver/include/fdbserver/TenantCache.h | 2 +- 4 files changed, 13 insertions(+), 9 deletions(-) diff --git a/fdbclient/ServerKnobs.cpp b/fdbclient/ServerKnobs.cpp index 58cfc1c866..ca83d52ee9 100644 --- a/fdbclient/ServerKnobs.cpp +++ b/fdbclient/ServerKnobs.cpp @@ -288,6 +288,7 @@ void ServerKnobs::initialize(Randomize randomize, ClientKnobs* clientKnobs, IsSi init( DD_STORAGE_WIGGLE_MIN_SS_AGE_SEC, isSimulated ? 2 : 21 * 60 * 60 * 24 ); if(randomize && BUGGIFY) DD_STORAGE_WIGGLE_MIN_SS_AGE_SEC = isSimulated ? 0: 120; init( DD_TENANT_AWARENESS_ENABLED, false ); init( TENANT_CACHE_LIST_REFRESH_INTERVAL, 2.0 ); + init( TENANT_CACHE_STORAGE_REFRESH_INTERVAL, 2.0 ); // TeamRemover init( TR_FLAG_DISABLE_MACHINE_TEAM_REMOVER, false ); if( randomize && BUGGIFY ) TR_FLAG_DISABLE_MACHINE_TEAM_REMOVER = deterministicRandom()->random01() < 0.1 ? true : false; // false by default. disable the consistency check when it's true diff --git a/fdbclient/include/fdbclient/ServerKnobs.h b/fdbclient/include/fdbclient/ServerKnobs.h index bba13eb7bc..8628c10dfa 100644 --- a/fdbclient/include/fdbclient/ServerKnobs.h +++ b/fdbclient/include/fdbclient/ServerKnobs.h @@ -232,6 +232,8 @@ public: DD_STORAGE_WIGGLE_MIN_SS_AGE_SEC; // Minimal age of a correct-configured server before it's chosen to be wiggled bool DD_TENANT_AWARENESS_ENABLED; int TENANT_CACHE_LIST_REFRESH_INTERVAL; // How often the TenantCache is refreshed + int TENANT_CACHE_STORAGE_REFRESH_INTERVAL; // How often the storage bytes used by each tenant in the TenantCache is + // refreshed // TeamRemover to remove redundant teams bool TR_FLAG_DISABLE_MACHINE_TEAM_REMOVER; // disable the machineTeamRemover actor diff --git a/fdbserver/TenantCache.actor.cpp b/fdbserver/TenantCache.actor.cpp index fdb85ff8ae..a9eea5300c 100644 --- a/fdbserver/TenantCache.actor.cpp +++ b/fdbserver/TenantCache.actor.cpp @@ -22,6 +22,7 @@ #include "fdbserver/TenantCache.h" #include #include +#include "flow/Trace.h" #include "flow/actorcompiler.h" class TenantCacheImpl { @@ -117,8 +118,7 @@ public: state ReadYourWritesTransaction tr(tenantCache->dbcx()); - // Reuse the TENANT_CACHE_LIST_REFRESH_INTERVAL knob for now. - state int refreshInterval = SERVER_KNOBS->TENANT_CACHE_LIST_REFRESH_INTERVAL; + state int refreshInterval = SERVER_KNOBS->TENANT_CACHE_STORAGE_REFRESH_INTERVAL; state double lastTenantListFetchTime = now(); loop { @@ -127,12 +127,13 @@ public: TraceEvent(SevWarn, "TenantCacheGetStorageUsageRefreshDelay", tenantCache->id()).log(); } - state std::vector tenantPrefixList = tenantCache->getTenantPrefixList(); + state std::vector tenantPrefixList = tenantCache->getTenantPrefixList(); - state KeyRangeRef range("/"_sr, "0"_sr); + state KeyRangeRef rangeBoundaries("/"_sr, "0"_sr); state int i; for (i = 0; i < tenantPrefixList.size(); i++) { - state int64_t size = wait(tr.getEstimatedRangeSizeBytes(range.withPrefix(tenantPrefixList[i]))); + state int64_t size = + wait(tr.getEstimatedRangeSizeBytes(rangeBoundaries.withPrefix(tenantPrefixList[i]))); tenantCache->updateStorageUsage(tenantPrefixList[i], size); } @@ -203,10 +204,10 @@ int TenantCache::cleanup() { return tenantsRemoved; } -std::vector TenantCache::getTenantPrefixList() const { - std::vector prefixes; +std::vector TenantCache::getTenantPrefixList() const { + std::vector prefixes; for (const auto& [prefix, entry] : tenantCache) { - prefixes.push_back(prefix); + prefixes.emplace_back(prefix); } return prefixes; } diff --git a/fdbserver/include/fdbserver/TenantCache.h b/fdbserver/include/fdbserver/TenantCache.h index 839b575d22..ff99d30d52 100644 --- a/fdbserver/include/fdbserver/TenantCache.h +++ b/fdbserver/include/fdbserver/TenantCache.h @@ -54,7 +54,7 @@ private: int cleanup(); // return a list of all tenant prefixes stored in the tenant cache - std::vector getTenantPrefixList() const; + std::vector getTenantPrefixList() const; // update the size for a tenant; do nothing if the tenant doesn't exist in the map void updateStorageUsage(KeyRef prefix, int64_t size); From eb541d103618d39b9b8b4e320fb65af178464ccb Mon Sep 17 00:00:00 2001 From: Ankita Kejriwal Date: Tue, 23 Aug 2022 18:12:36 -0700 Subject: [PATCH 009/210] Add a helper function to get the entire key range with a given prefix --- fdbclient/include/fdbclient/FDBTypes.h | 4 ++++ fdbserver/TenantCache.actor.cpp | 3 +-- 2 files changed, 5 insertions(+), 2 deletions(-) diff --git a/fdbclient/include/fdbclient/FDBTypes.h b/fdbclient/include/fdbclient/FDBTypes.h index e1186b1549..508221794e 100644 --- a/fdbclient/include/fdbclient/FDBTypes.h +++ b/fdbclient/include/fdbclient/FDBTypes.h @@ -339,6 +339,10 @@ struct KeyRangeRef { return KeyRangeRef(begin.withPrefix(prefix, arena), end.withPrefix(prefix, arena)); } + static KeyRangeRef entireRangeWithPrefix(const StringRef& prefix) { + return KeyRangeRef("/"_sr, "0"_sr).withPrefix(prefix); + } + KeyRangeRef removePrefix(const StringRef& prefix) const { return KeyRangeRef(begin.removePrefix(prefix), end.removePrefix(prefix)); } diff --git a/fdbserver/TenantCache.actor.cpp b/fdbserver/TenantCache.actor.cpp index a9eea5300c..5177aed3e2 100644 --- a/fdbserver/TenantCache.actor.cpp +++ b/fdbserver/TenantCache.actor.cpp @@ -129,11 +129,10 @@ public: state std::vector tenantPrefixList = tenantCache->getTenantPrefixList(); - state KeyRangeRef rangeBoundaries("/"_sr, "0"_sr); state int i; for (i = 0; i < tenantPrefixList.size(); i++) { state int64_t size = - wait(tr.getEstimatedRangeSizeBytes(rangeBoundaries.withPrefix(tenantPrefixList[i]))); + wait(tr.getEstimatedRangeSizeBytes(KeyRangeRef::entireRangeWithPrefix(tenantPrefixList[i]))); tenantCache->updateStorageUsage(tenantPrefixList[i], size); } From cade0baf7ecdd5907d7079e81e0901a22d605ede Mon Sep 17 00:00:00 2001 From: He Liu Date: Thu, 1 Sep 2022 11:03:34 -0700 Subject: [PATCH 010/210] Added Interfaces in ClusterInterface and DataDistributorInterface for audit. --- .../include/fdbclient/ClusterInterface.h | 7 +- .../fdbclient/StorageServerInterface.h | 51 +------ fdbserver/DataDistribution.actor.cpp | 130 +++++++++++++++++- .../fdbserver/DataDistributorInterface.h | 1 + fdbserver/storageserver.actor.cpp | 4 +- fdbserver/workloads/ValidateStorage.actor.cpp | 4 +- 6 files changed, 140 insertions(+), 57 deletions(-) diff --git a/fdbclient/include/fdbclient/ClusterInterface.h b/fdbclient/include/fdbclient/ClusterInterface.h index 14935f1700..1cce8f061d 100644 --- a/fdbclient/include/fdbclient/ClusterInterface.h +++ b/fdbclient/include/fdbclient/ClusterInterface.h @@ -40,6 +40,7 @@ struct ClusterInterface { RequestStream moveShard; RequestStream repairSystemData; RequestStream splitShard; + RequestStream triggerAudit; bool operator==(ClusterInterface const& r) const { return id() == r.id(); } bool operator!=(ClusterInterface const& r) const { return id() != r.id(); } @@ -51,7 +52,7 @@ struct ClusterInterface { databaseStatus.getFuture().isReady() || ping.getFuture().isReady() || getClientWorkers.getFuture().isReady() || forceRecovery.getFuture().isReady() || moveShard.getFuture().isReady() || repairSystemData.getFuture().isReady() || - splitShard.getFuture().isReady(); + splitShard.getFuture().isReady() || triggerAudit.getFuture().isReady(); } void initEndpoints() { @@ -64,6 +65,7 @@ struct ClusterInterface { moveShard.getEndpoint(TaskPriority::ClusterController); repairSystemData.getEndpoint(TaskPriority::ClusterController); splitShard.getEndpoint(TaskPriority::ClusterController); + triggerAudit.getEndpoint(TaskPriority::ClusterController); } template @@ -77,7 +79,8 @@ struct ClusterInterface { forceRecovery, moveShard, repairSystemData, - splitShard); + splitShard, + triggerAudit); } }; diff --git a/fdbclient/include/fdbclient/StorageServerInterface.h b/fdbclient/include/fdbclient/StorageServerInterface.h index faa7d7b47c..43a84abf9c 100644 --- a/fdbclient/include/fdbclient/StorageServerInterface.h +++ b/fdbclient/include/fdbclient/StorageServerInterface.h @@ -93,7 +93,7 @@ struct StorageServerInterface { RequestStream checkpoint; RequestStream fetchCheckpoint; RequestStream fetchCheckpointKeyValues; - RequestStream validateStorage; + RequestStream auditStorage; private: bool acceptingRequests; @@ -164,7 +164,8 @@ public: RequestStream(getValue.getEndpoint().getAdjustedEndpoint(20)); fetchCheckpointKeyValues = RequestStream( getValue.getEndpoint().getAdjustedEndpoint(21)); - validateStorage = RequestStream(getValue.getEndpoint().getAdjustedEndpoint(22)); + auditStorage = + RequestStream(getValue.getEndpoint().getAdjustedEndpoint(22)); } } else { ASSERT(Ar::isDeserializing); @@ -215,7 +216,7 @@ public: streams.push_back(checkpoint.getReceiver()); streams.push_back(fetchCheckpoint.getReceiver()); streams.push_back(fetchCheckpointKeyValues.getReceiver()); - streams.push_back(validateStorage.getReceiver()); + streams.push_back(auditStorage.getReceiver()); FlowTransport::transport().addEndpoints(streams); } }; @@ -1144,48 +1145,4 @@ struct StorageQueuingMetricsRequest { serializer(ar, reply); } }; - -struct ValidateStorageResult { - constexpr static FileIdentifier file_identifier = 13804340; - - UID requestId; - std::string error; - - ValidateStorageResult() = default; - ValidateStorageResult(UID requestId) : requestId(requestId) {} - - template - void serialize(Ar& ar) { - serializer(ar, requestId, error); - } -}; - -struct ValidateStorageRequest { - constexpr static FileIdentifier file_identifier = 13804341; - - enum Action { VALIDATE_HA = 0 }; - - UID requestId; - KeyRange range; - std::vector actions; - ReplyPromise reply; - - ValidateStorageRequest() = default; - ValidateStorageRequest(UID requestId, KeyRange range) : requestId(requestId), range(range) {} - - void addItem(Action action) { this->actions.push_back(static_cast(action)); } - std::vector getActions() const { - std::vector res; - for (const int32_t action : this->actions) { - res.push_back(static_cast(action)); - } - return res; - } - - template - void serialize(Ar& ar) { - serializer(ar, requestId, range, actions, reply); - } -}; - #endif diff --git a/fdbserver/DataDistribution.actor.cpp b/fdbserver/DataDistribution.actor.cpp index c83e8d8594..750dba1d0a 100644 --- a/fdbserver/DataDistribution.actor.cpp +++ b/fdbserver/DataDistribution.actor.cpp @@ -317,6 +317,8 @@ public: // relocationConsumer (by DDQueue) PromiseStream relocationProducer, relocationConsumer; + Promise initialized; + DataDistributor(Reference const> const& db, UID id) : dbInfo(db), ddId(id), txnProcessor(nullptr), initialDDEventHolder(makeReference("InitialDD")), movingDataEventHolder(makeReference("MovingData")), @@ -580,7 +582,6 @@ ACTOR Future dataDistribution(Reference self, state PromiseStream getTopKShardMetrics; state Reference> processingUnhealthy(new AsyncVar(false)); state Reference> processingWiggle(new AsyncVar(false)); - state Promise readyToStart; self->shardsAffectedByTeamFailure = makeReference(); wait(self->resumeRelocations()); @@ -618,7 +619,7 @@ ACTOR Future dataDistribution(Reference self, getTopKShardMetrics.getFuture(), getShardMetricsList, getAverageShardBytes.getFuture(), - readyToStart, + self->initialized, anyZeroHealthyTeams, self->ddId, &shards, @@ -656,7 +657,7 @@ ACTOR Future dataDistribution(Reference self, self->configuration, self->primaryDcId, self->configuration.usableRegions > 1 ? self->remoteDcIds : std::vector>(), - readyToStart.getFuture(), + self->initialized.getFuture(), zeroHealthyTeams[0], IsPrimary::True, processingUnhealthy, @@ -677,7 +678,7 @@ ACTOR Future dataDistribution(Reference self, self->configuration, self->remoteDcIds, Optional>>(), - readyToStart.getFuture() && remoteRecovered(self->dbInfo), + self->initialized.getFuture() && remoteRecovered(self->dbInfo), zeroHealthyTeams[1], IsPrimary::False, processingUnhealthy, @@ -1289,6 +1290,124 @@ ACTOR Future ddGetMetrics(GetDataDistributorMetricsRequest req, return Void(); } +ACTOR Future validateStorage(Reference self, TriggerAuditRequest req); +ACTOR Future scheduleAuditForRange(UID auditId, + KeyRange range, + AuditType type, + Reference actors, + Reference> auditMap); +ACTOR Future doAuditStorage(Reference actors, + Reference> auditMap, + StorageServerInterface ssi, + AuditStorageRequest req); + +ACTOR Future validateStorage(Reference self, TriggerAuditRequest req) { + // TODO(heliu): Load running audit, and create one if no audit is running. + state Reference> auditMap = + makeReference>(AuditPhase::Invalid, allKeys.end); + state Reference actors = makeReference(true); + state UID auditId = deterministicRandom()->randomUniqueID(); + + Ranges f = auditMap.intersectingRanges(req.range); + for (auto it = f.begin(); it != f.end(); ++it) { + if (it->value() == AuditPhase::Invalid || it->value() == AuditPhase::Error) { + subReq.range = KeyRangeRef(it->range().begin, it->range().end); + actors->add(scheduleAuditForRange( + auditId, KeyRangeRef(it->range().begin, it->range().end), req.getType(), actors, auditMap)); + } + } + + try { + wait(actors.getResult()); + req.reply.send(Void()); + } catch (Error& e) { + TraceEvent(SevWarnAlways, "DDValidateStorageError", req.requestId) + .errorUnsuppressed(e) + .detail("Range", req.range); + req.reply.sendError(e); + } + + return Void(); +} + +ACTOR Future scheduleAuditForRange(UID auditId, + KeyRange range, + AuditType type, + Reference actors, + Reference> auditMap) { + state Key begin = req.range.begin; + + while (begin < req.range.end) { + state Transaction tr(cx); + tr.setOption(FDBTransactionOptions::PRIORITY_SYSTEM_IMMEDIATE); + tr.setOption(FDBTransactionOptions::ACCESS_SYSTEM_KEYS); + + try { + std::cout << "1" << std::endl; + state RangeResult shards = wait(krmGetRanges(&tr, + keyServersPrefix, + KeyRangeRef(begin, req.end), + SERVER_KNOBS->MOVE_SHARD_KRM_ROW_LIMIT, + SERVER_KNOBS->MOVE_SHARD_KRM_BYTE_LIMIT)); + ASSERT(!shards.empty()); + + state RangeResult UIDtoTagMap = wait(tr.getRange(serverTagKeys, CLIENT_KNOBS->TOO_MANY)); + ASSERT(!UIDtoTagMap.more && UIDtoTagMap.size() < CLIENT_KNOBS->TOO_MANY); + + state int i = 0; + for (i = 0; i < shards.size() - 1; ++i) { + std::cout << "2" << std::endl; + std::vector src; + std::vector dest; + UID srcId, destId; + decodeKeyServersValue(UIDtoTagMap, shards[i].value, src, dest, srcId, destId); + + const int idx = deterministicRandom()->randomInt(0, src.size()); + Optional serverListValue = wait(tr.get(serverListKeyFor(src[idx]))); + ASSERT(serverListValue.present()); + const StorageServerInterface ssi = decodeServerListValue(serverListValue.get()); + + AuditStorageRequest req(auditId,KeyRangeRef(shards[i].key, shards[i + 1].key), type); + actors->add(doAuditStorage(actors, auditMap, ssi, req)); + begin = req.range.end; + std::cout << "3" << std::endl; + wait(delay(0.01)); + } + } catch (Error& e) { + TraceEvent(SevWarnAlways, "TestValidateStorageError").errorUnsuppressed(e).detail("Range", range); + wait(tr.onError(e)); + } + } + + return Void(); +} + +ACTOR Future doAuditStorage(Reference actors, + Reference> auditMap, + StorageServerInterface ssi, + AuditStorageRequest req) { + try { + auditMap->insert(req.range, AuditPhase::Running); + ValidateStorageResult vResult = wait(ssi.validateStorage.getReply(req)); + TraceEvent e(vResult.error.empty() ? SevInfo : SevWarnAlways, "DDValidateStorageResult", req.requestId); + e.detail("Range", req.range); + e.detail("StorageServer", ssi.toString()); + if (!vResult.error.empty()) { + e.detail("ErrorMessage", vResult.error); + } + } catch (Error& e) { + TraceEvent(SevWarning, "DDValidateStorageError", req.requestId) + .errorUnsuppressed(e) + .detail("Range", req.range) + .detail("StorageServer", ssi.toString()); + if (e.code() != error_code_actor_cancelled) { + actors.add(ScheduleAuditForRange(req.id, req.range, req.getType(), actors, auditMap)); + } + } + + return Void(); +} + ACTOR Future dataDistributor(DataDistributorInterface di, Reference const> db) { state Reference self(new DataDistributor(db, di.id())); state Future collection = actorCollection(self->addActor.getFuture()); @@ -1357,6 +1476,9 @@ ACTOR Future dataDistributor(DataDistributorInterface di, Reference dataDistributorMetrics; RequestStream distributorSplitRange; RequestStream storageWigglerState; + RequestStream triggerAudit; DataDistributorInterface() {} explicit DataDistributorInterface(const struct LocalityData& l, UID id) : locality(l), myId(id) {} diff --git a/fdbserver/storageserver.actor.cpp b/fdbserver/storageserver.actor.cpp index 105964a838..40988ce45a 100644 --- a/fdbserver/storageserver.actor.cpp +++ b/fdbserver/storageserver.actor.cpp @@ -3673,7 +3673,7 @@ ACTOR Future validateRangeShard(StorageServer* data, KeyRange range, std:: return Void(); } -ACTOR Future validateStorageQ(StorageServer* self, ValidateStorageRequest req) { +ACTOR Future validateStorageQ(StorageServer* self, AuditStorageRequest req) { wait(self->serveValidateStorageParallelismLock.take(TaskPriority::DefaultYield)); state FlowLock::Releaser holder(self->serveValidateStorageParallelismLock); @@ -10328,7 +10328,7 @@ ACTOR Future storageServerCore(StorageServer* self, StorageServerInterface when(FetchCheckpointKeyValuesRequest req = waitNext(ssi.fetchCheckpointKeyValues.getFuture())) { self->actors.add(fetchCheckpointKeyValuesQ(self, req)); } - when(ValidateStorageRequest req = waitNext(ssi.validateStorage.getFuture())) { + when(AuditStorageRequest req = waitNext(ssi.validateStorage.getFuture())) { self->actors.add(validateStorageQ(self, req)); } when(wait(updateProcessStatsTimer)) { diff --git a/fdbserver/workloads/ValidateStorage.actor.cpp b/fdbserver/workloads/ValidateStorage.actor.cpp index 5c095d11ee..61dc438397 100644 --- a/fdbserver/workloads/ValidateStorage.actor.cpp +++ b/fdbserver/workloads/ValidateStorage.actor.cpp @@ -122,7 +122,7 @@ struct ValidateStorage : TestWorkload { loop { try { - std::cout << "1" << std::endl; + std::cout << "1" << std::endl; state RangeResult shards = wait(krmGetRanges(&tr, keyServersPrefix, range, CLIENT_KNOBS->TOO_MANY, CLIENT_KNOBS->TOO_MANY)); ASSERT(!shards.empty() && !shards.more); @@ -143,7 +143,7 @@ struct ValidateStorage : TestWorkload { Optional serverListValue = wait(tr.get(serverListKeyFor(src[idx]))); ASSERT(serverListValue.present()); const StorageServerInterface ssi = decodeServerListValue(serverListValue.get()); - ValidateStorageRequest req(deterministicRandom()->randomUniqueID(), + AuditStorageRequest req(deterministicRandom()->randomUniqueID(), KeyRangeRef(shards[i].key, shards[i + 1].key)); ValidateStorageResult vResult = wait(ssi.validateStorage.getReply(req)); From 033741daab715477557c791ec14fa7ea9791e186 Mon Sep 17 00:00:00 2001 From: He Liu Date: Fri, 2 Sep 2022 09:11:19 -0700 Subject: [PATCH 011/210] Audit should always complete, any failures are retried. --- fdbclient/ServerKnobs.cpp | 2 +- fdbclient/include/fdbclient/Audit.h | 97 ++++++ .../DataDistributorClientInterface.h | 326 ++++++++++++++++++ fdbclient/include/fdbclient/ServerKnobs.h | 2 +- fdbserver/DataDistribution.actor.cpp | 63 ++-- fdbserver/storageserver.actor.cpp | 77 ++--- fdbserver/workloads/ValidateStorage.actor.cpp | 2 +- flow/include/flow/error_definitions.h | 2 +- tests/fast/ValidateStorage.toml | 12 + 9 files changed, 514 insertions(+), 69 deletions(-) create mode 100644 fdbclient/include/fdbclient/Audit.h create mode 100644 fdbclient/include/fdbclient/DataDistributorClientInterface.h create mode 100644 tests/fast/ValidateStorage.toml diff --git a/fdbclient/ServerKnobs.cpp b/fdbclient/ServerKnobs.cpp index 18a44c4326..e0331029fa 100644 --- a/fdbclient/ServerKnobs.cpp +++ b/fdbclient/ServerKnobs.cpp @@ -705,7 +705,7 @@ void ServerKnobs::initialize(Randomize randomize, ClientKnobs* clientKnobs, IsSi init( FETCH_KEYS_LOWER_PRIORITY, 0 ); init( FETCH_CHANGEFEED_PARALLELISM, 4 ); init( SERVE_FETCH_CHECKPOINT_PARALLELISM, 4 ); - init( SERVE_VALIDATE_STORAGE_PARALLELISM, 2 ); + init( SERVE_AUDIT_STORAGE_PARALLELISM, 2 ); init( BUGGIFY_BLOCK_BYTES, 10000 ); init( STORAGE_RECOVERY_VERSION_LAG_LIMIT, 2 * MAX_READ_TRANSACTION_LIFE_VERSIONS ); init( STORAGE_COMMIT_BYTES, 10000000 ); if( randomize && BUGGIFY ) STORAGE_COMMIT_BYTES = 2000000; diff --git a/fdbclient/include/fdbclient/Audit.h b/fdbclient/include/fdbclient/Audit.h new file mode 100644 index 0000000000..a216ed8083 --- /dev/null +++ b/fdbclient/include/fdbclient/Audit.h @@ -0,0 +1,97 @@ +/* + * Audit.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_AUDIT_H +#define FDBCLIENT_AUDIT_H +#pragma once + +#include "fdbclient/FDBTypes.h" + +enum class AuditPhase { Invalid = 0; Running = 1; Complete = 2; Error = 3; }; + +enum class AuditType { Invalid = 0; ValidateHA = 1 }; + +struct AuditStorageState { + constexpr static FileIdentifier file_identifier = 13804340; + + ValidateStorageResult() = default; + ValidateStorageResult(UID id, AuditType type) : id(id), type(type), validatedKeys(0) {} + + template + void serialize(Ar& ar) { + serializer(ar, id, type, phase, error); + } + + void setType(AuditType type) { this->type = static_cast(this->type); } + AuditType getType() const { return static_cast(this->type); } + + void setPhase(AuditPhase phase) { this->phase = static_cast(phase); } + AuditPhase getPhase() const { return static_cast(this->phase); } + + UID id; + uint8_t type; + uint8_t phase; + std::string error; +}; + +struct AuditStorageRequest { + constexpr static FileIdentifier file_identifier = 13804341; + + AuditStorageRequest() = default; + AuditStorageRequest(UID id, KeyRange range, AuditType type) + : id(id), range(range), type(static_cast(type)) {} + + void setType(AuditType type) { this->type = static_cast(this->type); } + AuditType getType() const { return static_cast(this->type); } + + template + void serialize(Ar& ar) { + serializer(ar, id, range, actions, reply); + } + + UID id; + KeyRange range; + uint8_t type; + ReplyPromise reply; +}; + +// Triggers an audit of the specific type, an audit id is returned if an audit is scheduled successfully. +// If there is an running audit, the corresponding id will be returned, unless force is true; +// When force is set, the ongoing audit will be cancelled, and a new audit will be scheduled. +struct TriggerAuditRequest { + constexpr static FileIdentifier file_identifier = 1384445; + + TriggerAuditRequest() = default; + SplitShardRequest(AuditType type, KeyRange range) : type(type), range(range), force(false) {} + + void setType(AuditType type) { this->type = static_cast(this->type); } + AuditType getType() const { return static_cast(this->type); } + + template + void serialize(Ar& ar) { + serializer(ar, type, range, force, reply); + } + + uint8_t type; + KeyRange range; + bool force; + ReplyPromise reply; +}; +#endif diff --git a/fdbclient/include/fdbclient/DataDistributorClientInterface.h b/fdbclient/include/fdbclient/DataDistributorClientInterface.h new file mode 100644 index 0000000000..f338d52bc1 --- /dev/null +++ b/fdbclient/include/fdbclient/DataDistributorClientInterface.h @@ -0,0 +1,326 @@ +/* + * DataDistributorClientInterface.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_ClusterInterface_H +#define FDBCLIENT_ClusterInterface_H +#pragma once + +#include "fdbclient/FDBTypes.h" +#include "fdbrpc/FailureMonitor.h" +#include "fdbclient/Status.h" +#include "fdbclient/CommitProxyInterface.h" +#include "fdbclient/ClientWorkerInterface.h" +#include "fdbclient/ClientVersion.h" + +struct DataDistributorClientInterface { + constexpr static FileIdentifier file_identifier = 15988863; + RequestStream openDatabase; + RequestStream failureMonitoring; + RequestStream databaseStatus; + RequestStream> ping; + RequestStream getClientWorkers; + RequestStream forceRecovery; + RequestStream moveShard; + RequestStream repairSystemData; + RequestStream splitShard; + + bool operator==(ClusterInterface const& r) const { return id() == r.id(); } + bool operator!=(ClusterInterface const& r) const { return id() != r.id(); } + UID id() const { return openDatabase.getEndpoint().token; } + NetworkAddress address() const { return openDatabase.getEndpoint().getPrimaryAddress(); } + + bool hasMessage() const { + return openDatabase.getFuture().isReady() || failureMonitoring.getFuture().isReady() || + databaseStatus.getFuture().isReady() || ping.getFuture().isReady() || + getClientWorkers.getFuture().isReady() || forceRecovery.getFuture().isReady() || + moveShard.getFuture().isReady() || repairSystemData.getFuture().isReady() || + splitShard.getFuture().isReady(); + } + + void initEndpoints() { + openDatabase.getEndpoint(TaskPriority::ClusterController); + failureMonitoring.getEndpoint(TaskPriority::FailureMonitor); + databaseStatus.getEndpoint(TaskPriority::ClusterController); + ping.getEndpoint(TaskPriority::ClusterController); + getClientWorkers.getEndpoint(TaskPriority::ClusterController); + forceRecovery.getEndpoint(TaskPriority::ClusterController); + moveShard.getEndpoint(TaskPriority::ClusterController); + repairSystemData.getEndpoint(TaskPriority::ClusterController); + splitShard.getEndpoint(TaskPriority::ClusterController); + } + + template + void serialize(Ar& ar) { + serializer(ar, + openDatabase, + failureMonitoring, + databaseStatus, + ping, + getClientWorkers, + forceRecovery, + moveShard, + repairSystemData, + splitShard); + } +}; + +struct ClusterControllerClientInterface { + constexpr static FileIdentifier file_identifier = 14997695; + ClusterInterface clientInterface; + + bool operator==(ClusterControllerClientInterface const& r) const { + return clientInterface.id() == r.clientInterface.id(); + } + bool operator!=(ClusterControllerClientInterface const& r) const { + return clientInterface.id() != r.clientInterface.id(); + } + + template + void serialize(Ar& ar) { + serializer(ar, clientInterface); + } +}; + +template +struct ItemWithExamples { + T item; + int count; + std::vector> examples; + + ItemWithExamples() : item{}, count(0) {} + ItemWithExamples(T const& item, int count, std::vector> const& examples) + : item(item), count(count), examples(examples) {} + + template + void serialize(Ar& ar) { + serializer(ar, item, count, examples); + } +}; + +struct OpenDatabaseRequest { + constexpr static FileIdentifier file_identifier = 2799502; + // Sent by the native API to the cluster controller to open a database and track client + // info changes. Returns immediately if the current client info id is different from + // knownClientInfoID; otherwise returns when it next changes (or perhaps after a long interval) + + int clientCount; + std::vector> issues; + std::vector>> supportedVersions; + std::vector> maxProtocolSupported; + + UID knownClientInfoID; + ReplyPromise reply; + + template + void serialize(Ar& ar) { + if constexpr (!is_fb_function) { + ASSERT(ar.protocolVersion().hasOpenDatabase()); + } + serializer(ar, clientCount, issues, supportedVersions, maxProtocolSupported, knownClientInfoID, reply); + } +}; + +struct SystemFailureStatus { + constexpr static FileIdentifier file_identifier = 3194108; + NetworkAddressList addresses; + FailureStatus status; + + SystemFailureStatus() {} + SystemFailureStatus(NetworkAddressList const& a, FailureStatus const& s) : addresses(a), status(s) {} + + template + void serialize(Ar& ar) { + serializer(ar, addresses, status); + } +}; + +struct FailureMonitoringReply { + constexpr static FileIdentifier file_identifier = 6820325; + VectorRef changes; + Version failureInformationVersion; + bool allOthersFailed; // If true, changes are relative to all servers being failed, otherwise to the version given + // in the request + int clientRequestIntervalMS, // after this many milliseconds, send another request + considerServerFailedTimeoutMS; // after this many additional milliseconds, consider the ClusterController itself + // to be failed + Arena arena; + + template + void serialize(Ar& ar) { + serializer(ar, + changes, + failureInformationVersion, + allOthersFailed, + clientRequestIntervalMS, + considerServerFailedTimeoutMS, + arena); + } +}; + +struct FailureMonitoringRequest { + // Sent by all participants to the cluster controller reply.clientRequestIntervalMS + // ms after receiving the previous reply. + // Provides the controller the self-diagnosed status of the sender, and also + // requests the status of other systems. Failure to timely send one of these implies + // a failed status. + // If !senderStatus.present(), the sender wants to receive the latest failure information + // but doesn't want to be monitored. + // The failureInformationVersion returned in reply should be passed back to the + // next request to facilitate delta compression of the failure information. + + constexpr static FileIdentifier file_identifier = 5867851; + Optional senderStatus; + Version failureInformationVersion; + NetworkAddressList addresses; + ReplyPromise reply; + + template + void serialize(Ar& ar) { + serializer(ar, senderStatus, failureInformationVersion, addresses, reply); + } +}; + +struct StatusReply { + constexpr static FileIdentifier file_identifier = 9980504; + StatusObject statusObj; + std::string statusStr; + + StatusReply() {} + explicit StatusReply(StatusObject obj) + : statusObj(obj), statusStr(json_spirit::write_string(json_spirit::mValue(obj))) {} + explicit StatusReply(std::string&& text) : statusStr(text) {} + + template + void serialize(Ar& ar) { + serializer(ar, statusStr); + if (ar.isDeserializing) { + json_spirit::mValue mv; + if (g_network->isSimulated()) { + mv = readJSONStrictly(statusStr); + } else { + // In non-simulation allow errors because some status data is better than no status data + json_spirit::read_string(statusStr, mv); + } + statusObj = std::move(mv.get_obj()); + } + } +}; + +struct StatusRequest { + constexpr static FileIdentifier file_identifier = 14419140; + ReplyPromise reply; + + template + void serialize(Ar& ar) { + serializer(ar, reply); + } +}; + +struct GetClientWorkersRequest { + constexpr static FileIdentifier file_identifier = 10771791; + ReplyPromise> reply; + + GetClientWorkersRequest() {} + + template + void serialize(Ar& ar) { + serializer(ar, reply); + } +}; + +struct ForceRecoveryRequest { + constexpr static FileIdentifier file_identifier = 14821350; + Key dcId; + ReplyPromise reply; + + ForceRecoveryRequest() {} + explicit ForceRecoveryRequest(Key dcId) : dcId(dcId) {} + + template + void serialize(Ar& ar) { + serializer(ar, dcId, reply); + } +}; + +// Request to move a keyrange (shard) to a new team represented as addresses. +struct MoveShardRequest { + constexpr static FileIdentifier file_identifier = 2799592; + + KeyRange shard; + std::vector addresses; + ReplyPromise reply; + + MoveShardRequest() {} + MoveShardRequest(KeyRange shard, std::vector addresses) + : shard{ std::move(shard) }, addresses{ std::move(addresses) } {} + + template + void serialize(Ar& ar) { + serializer(ar, shard, addresses, reply); + } +}; + +// Request to trigger a master recovery, and during the following recovery, the system metadata will be +// reconstructed from TLogs, and written to a new SS team. +// This is used when metadata on SSes are lost or corrupted. +struct RepairSystemDataRequest { + constexpr static FileIdentifier file_identifier = 2799593; + + ReplyPromise reply; + + RepairSystemDataRequest() {} + + template + void serialize(Ar& ar) { + serializer(ar, reply); + } +}; + +// Returns the actual shards generated by the SplitShardRequest. +struct SplitShardReply { + constexpr static FileIdentifier file_identifier = 1384440; + std::vector shards; + + SplitShardReply() {} + explicit SplitShardReply(std::vector shards) : shards{ std::move(shards) } {} + + template + void serialize(Ar& ar) { + serializer(ar, shards); + } +}; + +// Split keyrange [shard.begin, shard.end) into num shards. +// Split points are chosen as the arithmetically equal division points of the given range. +struct SplitShardRequest { + constexpr static FileIdentifier file_identifier = 1384443; + KeyRange shard; + int num; + ReplyPromise reply; + + SplitShardRequest() : num(0) {} + SplitShardRequest(KeyRange shard, int num) : shard{ std::move(shard) }, num(num) {} + + template + void serialize(Ar& ar) { + serializer(ar, shard, num, reply); + } +}; +#endif diff --git a/fdbclient/include/fdbclient/ServerKnobs.h b/fdbclient/include/fdbclient/ServerKnobs.h index c75b7defcc..42de8ab677 100644 --- a/fdbclient/include/fdbclient/ServerKnobs.h +++ b/fdbclient/include/fdbclient/ServerKnobs.h @@ -665,7 +665,7 @@ public: int FETCH_KEYS_LOWER_PRIORITY; int FETCH_CHANGEFEED_PARALLELISM; int SERVE_FETCH_CHECKPOINT_PARALLELISM; - int SERVE_VALIDATE_STORAGE_PARALLELISM; + int SERVE_AUDIT_STORAGE_PARALLELISM; int BUGGIFY_BLOCK_BYTES; int64_t STORAGE_RECOVERY_VERSION_LAG_LIMIT; double STORAGE_DURABILITY_LAG_REJECT_THRESHOLD; diff --git a/fdbserver/DataDistribution.actor.cpp b/fdbserver/DataDistribution.actor.cpp index 750dba1d0a..b85b39a62b 100644 --- a/fdbserver/DataDistribution.actor.cpp +++ b/fdbserver/DataDistribution.actor.cpp @@ -1290,18 +1290,18 @@ ACTOR Future ddGetMetrics(GetDataDistributorMetricsRequest req, return Void(); } -ACTOR Future validateStorage(Reference self, TriggerAuditRequest req); +ACTOR Future auditStorage(Reference self, TriggerAuditRequest req); ACTOR Future scheduleAuditForRange(UID auditId, KeyRange range, AuditType type, Reference actors, Reference> auditMap); ACTOR Future doAuditStorage(Reference actors, - Reference> auditMap, - StorageServerInterface ssi, - AuditStorageRequest req); + Reference> auditMap, + StorageServerInterface ssi, + AuditStorageRequest req); -ACTOR Future validateStorage(Reference self, TriggerAuditRequest req) { +ACTOR Future auditStorage(Reference self, TriggerAuditRequest req) { // TODO(heliu): Load running audit, and create one if no audit is running. state Reference> auditMap = makeReference>(AuditPhase::Invalid, allKeys.end); @@ -1311,20 +1311,21 @@ ACTOR Future validateStorage(Reference self, TriggerAudit Ranges f = auditMap.intersectingRanges(req.range); for (auto it = f.begin(); it != f.end(); ++it) { if (it->value() == AuditPhase::Invalid || it->value() == AuditPhase::Error) { - subReq.range = KeyRangeRef(it->range().begin, it->range().end); actors->add(scheduleAuditForRange( auditId, KeyRangeRef(it->range().begin, it->range().end), req.getType(), actors, auditMap)); } } + req.reply.send(auditId); + try { wait(actors.getResult()); - req.reply.send(Void()); + // TODO(heliu): Set the audit result, and clear auditId. } catch (Error& e) { - TraceEvent(SevWarnAlways, "DDValidateStorageError", req.requestId) + TraceEvent(SevWarnAlways, "DDAuditStorageOperationError", auditId) .errorUnsuppressed(e) - .detail("Range", req.range); - req.reply.sendError(e); + .detail("Range", req.range) + .detail("Type", req.type); } return Void(); @@ -1335,18 +1336,18 @@ ACTOR Future scheduleAuditForRange(UID auditId, AuditType type, Reference actors, Reference> auditMap) { - state Key begin = req.range.begin; + // TODO(heliu): Load the audit map, or make auditStorage a streaming call, to skip audited ranges. + state Key begin = range.begin; - while (begin < req.range.end) { + while (begin < range.end) { state Transaction tr(cx); tr.setOption(FDBTransactionOptions::PRIORITY_SYSTEM_IMMEDIATE); tr.setOption(FDBTransactionOptions::ACCESS_SYSTEM_KEYS); try { - std::cout << "1" << std::endl; state RangeResult shards = wait(krmGetRanges(&tr, keyServersPrefix, - KeyRangeRef(begin, req.end), + KeyRangeRef(begin, range.end), SERVER_KNOBS->MOVE_SHARD_KRM_ROW_LIMIT, SERVER_KNOBS->MOVE_SHARD_KRM_BYTE_LIMIT)); ASSERT(!shards.empty()); @@ -1356,21 +1357,25 @@ ACTOR Future scheduleAuditForRange(UID auditId, state int i = 0; for (i = 0; i < shards.size() - 1; ++i) { - std::cout << "2" << std::endl; std::vector src; std::vector dest; UID srcId, destId; decodeKeyServersValue(UIDtoTagMap, shards[i].value, src, dest, srcId, destId); - const int idx = deterministicRandom()->randomInt(0, src.size()); - Optional serverListValue = wait(tr.get(serverListKeyFor(src[idx]))); - ASSERT(serverListValue.present()); + state UID ssId = serverListKeyFor(src[deterministicRandom()->randomInt(0, src.size())]); + Optional serverListValue = wait(tr.get(serverListKeyFor(ssId))); + KeyRangeRef currentRange(shards[i].key, shards[i + 1].key); + if (!serverListValue.present()) { + TraceEvent(SevWarnAlways, "ScheduleAuditForRangeMissingServer", auditId) + .detail("StorageServer", ssId) + .detail("Range", currentRange); + break; + } const StorageServerInterface ssi = decodeServerListValue(serverListValue.get()); - AuditStorageRequest req(auditId,KeyRangeRef(shards[i].key, shards[i + 1].key), type); + const AuditStorageRequest req(auditId, currentRange, type); actors->add(doAuditStorage(actors, auditMap, ssi, req)); begin = req.range.end; - std::cout << "3" << std::endl; wait(delay(0.01)); } } catch (Error& e) { @@ -1383,12 +1388,17 @@ ACTOR Future scheduleAuditForRange(UID auditId, } ACTOR Future doAuditStorage(Reference actors, - Reference> auditMap, - StorageServerInterface ssi, - AuditStorageRequest req) { + Reference> auditMap, + StorageServerInterface ssi, + AuditStorageRequest req) { + TraceEvent(SevDebug, "DDAuditStorageBegin", req.id) + .detail("Range", req.range) + .detail("Type", req.type) + .detail("StorageServer", ssi.toString()); + try { auditMap->insert(req.range, AuditPhase::Running); - ValidateStorageResult vResult = wait(ssi.validateStorage.getReply(req)); + ValidateStorageResult vResult = wait(ssi.auditStorage.getReply(req)); TraceEvent e(vResult.error.empty() ? SevInfo : SevWarnAlways, "DDValidateStorageResult", req.requestId); e.detail("Range", req.range); e.detail("StorageServer", ssi.toString()); @@ -1401,6 +1411,7 @@ ACTOR Future doAuditStorage(Reference actors, .detail("Range", req.range) .detail("StorageServer", ssi.toString()); if (e.code() != error_code_actor_cancelled) { + auditMap->insert(req.range, AuditPhase::Error); actors.add(ScheduleAuditForRange(req.id, req.range, req.getType(), actors, auditMap)); } } @@ -1476,8 +1487,8 @@ ACTOR Future dataDistributor(DataDistributorInterface di, ReferenceserveFetchCheckpointParallelismLock.waiters(); }); specialCounter(cc, "ServeValidateStorageActive", [self]() { - return self->serveValidateStorageParallelismLock.activePermits(); + return self->serveAuditStorageParallelismLock.activePermits(); }); specialCounter(cc, "ServeValidateStorageWaiting", [self]() { - return self->serveValidateStorageParallelismLock.waiters(); + return self->serveAuditStorageParallelismLock.waiters(); }); specialCounter(cc, "QueryQueueMax", [self]() { return self->getAndResetMaxQueryQueueSize(); }); specialCounter(cc, "BytesStored", [self]() { return self->metrics.byteSample.getEstimate(allKeys); }); @@ -1247,7 +1247,7 @@ public: fetchChangeFeedParallelismLock(SERVER_KNOBS->FETCH_KEYS_PARALLELISM), fetchKeysBytesBudget(SERVER_KNOBS->STORAGE_FETCH_BYTES), fetchKeysBudgetUsed(false), serveFetchCheckpointParallelismLock(SERVER_KNOBS->SERVE_FETCH_CHECKPOINT_PARALLELISM), - serveValidateStorageParallelismLock(SERVER_KNOBS->SERVE_VALIDATE_STORAGE_PARALLELISM), + serveAuditStorageParallelismLock(SERVER_KNOBS->SERVE_AUDIT_STORAGE_PARALLELISM), instanceID(deterministicRandom()->randomUniqueID().first()), shuttingDown(false), behind(false), versionBehind(false), debug_inApplyUpdate(false), debug_lastValidateTime(0), lastBytesInputEBrake(0), lastDurableVersionEBrake(0), maxQueryQueue(0), transactionTagCounter(ssi.id()), counters(this), @@ -3604,13 +3604,12 @@ ACTOR Future validateRangeAgainstServer(StorageServer* data, .detail("Version", version) .detail("ErrorMessage", error) .detail("RemoteServer", remoteServer.toString()); - throw validate_storage_error(); } TraceEvent(SevDebug, "ServeValidateRangeAgainstServerEnd", data->thisServerID) .detail("Range", range) .detail("Version", version) - .detail("ValidatedKeys", validatedKeys) + .detail("ValidatedKeys", validatedKeys) .detail("Servers", remoteServer.toString()); return Void(); @@ -3673,49 +3672,49 @@ ACTOR Future validateRangeShard(StorageServer* data, KeyRange range, std:: return Void(); } -ACTOR Future validateStorageQ(StorageServer* self, AuditStorageRequest req) { - wait(self->serveValidateStorageParallelismLock.take(TaskPriority::DefaultYield)); - state FlowLock::Releaser holder(self->serveValidateStorageParallelismLock); +ACTOR Future auditStorageQ(StorageServer* self, AuditStorageRequest req) { + wait(self->serveAuditStorageParallelismLock.take(TaskPriority::DefaultYield)); + state FlowLock::Releaser holder(self->serveAuditStorageParallelismLock); - TraceEvent("ServeValidateStorageBegin", self->thisServerID) - .detail("RequestID", req.requestId) - .detail("Range", req.range); + TraceEvent("ServeAuditStorageBegin", self->thisServerID) + .detail("RequestID", req.id) + .detail("Range", req.range) + .detail("Type", req.type); state Transaction tr(self->cx); tr.setOption(FDBTransactionOptions::PRIORITY_SYSTEM_IMMEDIATE); tr.setOption(FDBTransactionOptions::ACCESS_SYSTEM_KEYS); - loop { - try { - state RangeResult shards = wait(krmGetRanges(&tr, - keyServersPrefix, - req.range, - SERVER_KNOBS->MOVE_SHARD_KRM_ROW_LIMIT, - SERVER_KNOBS->MOVE_SHARD_KRM_BYTE_LIMIT)); - ASSERT(!shards.empty() && !shards.more); + try { + loop { + try { + state RangeResult shards = wait(krmGetRanges(&tr, + keyServersPrefix, + req.range, + SERVER_KNOBS->MOVE_SHARD_KRM_ROW_LIMIT, + SERVER_KNOBS->MOVE_SHARD_KRM_BYTE_LIMIT)); + ASSERT(!shards.empty() && !shards.more); - state RangeResult UIDtoTagMap = wait(tr.getRange(serverTagKeys, CLIENT_KNOBS->TOO_MANY)); - ASSERT(!UIDtoTagMap.more && UIDtoTagMap.size() < CLIENT_KNOBS->TOO_MANY); + state RangeResult UIDtoTagMap = wait(tr.getRange(serverTagKeys, CLIENT_KNOBS->TOO_MANY)); + ASSERT(!UIDtoTagMap.more && UIDtoTagMap.size() < CLIENT_KNOBS->TOO_MANY); - std::vector> fs; - for (int i = 0; i < shards.size() - 1; ++i) { - std::vector src; - std::vector dest; - UID srcId, destId; - decodeKeyServersValue(UIDtoTagMap, shards[i].value, src, dest, srcId, destId); - fs.push_back(validateRangeShard(self, KeyRangeRef(shards[i].key, shards[i + 1].key), src)); - } + std::vector> fs; + for (int i = 0; i < shards.size() - 1; ++i) { + std::vector src; + std::vector dest; + UID srcId, destId; + decodeKeyServersValue(UIDtoTagMap, shards[i].value, src, dest, srcId, destId); + fs.push_back(validateRangeShard(self, KeyRangeRef(shards[i].key, shards[i + 1].key), src)); + } - wait(waitForAll(fs)); - req.reply.send(ValidateStorageResult(req.requestId)); - break; - } catch (Error& e) { - if (e.code() == error_code_validate_storage_error) { - req.reply.sendError(e); + wait(waitForAll(fs)); break; + } catch (Error& e) { + wait(tr.onError(e)); } - wait(tr.onError(e)); } + } catch (Error& e) { + req.reply.sendError(audit_storage_failed()); } return Void(); @@ -10328,8 +10327,8 @@ ACTOR Future storageServerCore(StorageServer* self, StorageServerInterface when(FetchCheckpointKeyValuesRequest req = waitNext(ssi.fetchCheckpointKeyValues.getFuture())) { self->actors.add(fetchCheckpointKeyValuesQ(self, req)); } - when(AuditStorageRequest req = waitNext(ssi.validateStorage.getFuture())) { - self->actors.add(validateStorageQ(self, req)); + when(AuditStorageRequest req = waitNext(ssi.auditStorage.getFuture())) { + self->actors.add(auditStorageQ(self, req)); } when(wait(updateProcessStatsTimer)) { updateProcessStats(self); diff --git a/fdbserver/workloads/ValidateStorage.actor.cpp b/fdbserver/workloads/ValidateStorage.actor.cpp index 61dc438397..c4c26bbb80 100644 --- a/fdbserver/workloads/ValidateStorage.actor.cpp +++ b/fdbserver/workloads/ValidateStorage.actor.cpp @@ -145,7 +145,7 @@ struct ValidateStorage : TestWorkload { const StorageServerInterface ssi = decodeServerListValue(serverListValue.get()); AuditStorageRequest req(deterministicRandom()->randomUniqueID(), KeyRangeRef(shards[i].key, shards[i + 1].key)); - ValidateStorageResult vResult = wait(ssi.validateStorage.getReply(req)); + ValidateStorageResult vResult = wait(ssi.auditStorage.getReply(req)); std::cout << "3" << std::endl; } diff --git a/flow/include/flow/error_definitions.h b/flow/include/flow/error_definitions.h index 9a1c9fb819..b689b64176 100755 --- a/flow/include/flow/error_definitions.h +++ b/flow/include/flow/error_definitions.h @@ -125,7 +125,7 @@ ERROR( invalid_cluster_id, 1217, "Attempted to join cluster with a different clu ERROR( restart_cluster_controller, 1218, "Restart cluster controller process" ) ERROR( please_reboot_remote_kv_store, 1219, "Need to reboot the storage engine process as it died abnormally") ERROR( incompatible_software_version, 1220, "Current software does not support database format" ) -ERROR( validate_storage_error, 1221, "Validate storage consistency error" ) +ERROR( audit_storage_failed, 1221, "Validate storage consistency error" ) // 15xx Platform errors ERROR( platform_error, 1500, "Platform error" ) diff --git a/tests/fast/ValidateStorage.toml b/tests/fast/ValidateStorage.toml new file mode 100644 index 0000000000..b38c74037f --- /dev/null +++ b/tests/fast/ValidateStorage.toml @@ -0,0 +1,12 @@ +[configuration] +config = 'triple' +generateFearless = true +allowDefaultTenant = false +machineCount = 15 + +[[test]] +testTitle = 'ValidateStorageWorkload' +useDB = true + + [[test.workload]] + testName = 'ValidateStorageWorkload' From 68ea3267384e822e2a79ecff2c5c80b5cddd32aa Mon Sep 17 00:00:00 2001 From: He Liu Date: Fri, 2 Sep 2022 12:44:06 -0700 Subject: [PATCH 012/210] Read local data with getKeyValuesQ(). --- fdbclient/include/fdbclient/Audit.h | 2 +- fdbserver/DataDistribution.actor.cpp | 2 +- fdbserver/storageserver.actor.cpp | 22 +++++++++++----------- 3 files changed, 13 insertions(+), 13 deletions(-) diff --git a/fdbclient/include/fdbclient/Audit.h b/fdbclient/include/fdbclient/Audit.h index a216ed8083..4c0c77f8d0 100644 --- a/fdbclient/include/fdbclient/Audit.h +++ b/fdbclient/include/fdbclient/Audit.h @@ -24,7 +24,7 @@ #include "fdbclient/FDBTypes.h" -enum class AuditPhase { Invalid = 0; Running = 1; Complete = 2; Error = 3; }; +enum class AuditPhase { Invalid = 0; Running = 1; Complete = 2; Error = 3; Failed = 4; }; enum class AuditType { Invalid = 0; ValidateHA = 1 }; diff --git a/fdbserver/DataDistribution.actor.cpp b/fdbserver/DataDistribution.actor.cpp index b85b39a62b..04f93d74e5 100644 --- a/fdbserver/DataDistribution.actor.cpp +++ b/fdbserver/DataDistribution.actor.cpp @@ -1411,7 +1411,7 @@ ACTOR Future doAuditStorage(Reference actors, .detail("Range", req.range) .detail("StorageServer", ssi.toString()); if (e.code() != error_code_actor_cancelled) { - auditMap->insert(req.range, AuditPhase::Error); + auditMap->insert(req.range, AuditPhase::Failed); actors.add(ScheduleAuditForRange(req.id, req.range, req.getType(), actors, auditMap)); } } diff --git a/fdbserver/storageserver.actor.cpp b/fdbserver/storageserver.actor.cpp index 4b8cd1092b..fb665ae706 100644 --- a/fdbserver/storageserver.actor.cpp +++ b/fdbserver/storageserver.actor.cpp @@ -3507,29 +3507,29 @@ ACTOR Future validateRangeAgainstServer(StorageServer* data, state int validatedKeys = 0; loop { try { - state GetKeyValuesRequest req; state int limit = 1e4; state int limitBytes = CLIENT_KNOBS->REPLY_BYTE_LIMIT; + state GetKeyValuesRequest req; + state GetKeyValuesRequest localReq; req.begin = firstGreaterOrEqual(range.begin); req.end = firstGreaterOrEqual(range.end); req.limit = limit; req.limitBytes = limitBytes; req.version = version; req.tags = TagSet(); + localReq.begin = firstGreaterOrEqual(range.begin); + localReq.end = firstGreaterOrEqual(range.end); + localReq.limit = limit; + localReq.limitBytes = limitBytes; + localReq.version = version; + localReq.tags = TagSet(); // Try getting the entries in the specified range state Future> remoteKeyValueFuture = remoteServer.getKeyValues.getReplyUnlessFailedFor(req, 2, 0); - state Future localKeyValueFuture = readRange(data, - version, - range, - limit, - &limitBytes, - SpanContext(), - IKeyValueStore::ReadType::LOW, - Optional()); + data->actors.add(getKeyValuesQ(data, localReq)); state ErrorOr remoteResult = wait(remoteKeyValueFuture); - GetKeyValuesReply local = wait(localKeyValueFuture); + GetKeyValuesReply local = wait(localReq.reply.getFuture()); Key lastKey = range.begin; state std::string error; @@ -3616,7 +3616,7 @@ ACTOR Future validateRangeAgainstServer(StorageServer* data, } ACTOR Future validateRangeShard(StorageServer* data, KeyRange range, std::vector candidates) { - TraceEvent("ServeValidateRangeShardBegin", data->thisServerID) + TraceEvent(SevDebug, "ServeValidateRangeShardBegin", data->thisServerID) .detail("Range", range) .detail("Servers", describe(candidates)); From 6f4d09f947c104855351fef2f54381eb8f122317 Mon Sep 17 00:00:00 2001 From: He Liu Date: Mon, 5 Sep 2022 12:45:14 -0700 Subject: [PATCH 013/210] Replace Reference<> with std::shared_ptr<>. --- fdbclient/include/fdbclient/Audit.h | 22 +++-- .../DataDistributorClientInterface.h | 2 +- .../fdbclient/StorageServerInterface.h | 1 + fdbserver/DDTxnProcessor.actor.cpp | 58 ++++++++++++ fdbserver/DataDistribution.actor.cpp | 91 +++++++------------ fdbserver/include/fdbserver/DDTxnProcessor.h | 10 ++ fdbserver/storageserver.actor.cpp | 1 + fdbserver/workloads/ValidateStorage.actor.cpp | 16 ++-- 8 files changed, 130 insertions(+), 71 deletions(-) diff --git a/fdbclient/include/fdbclient/Audit.h b/fdbclient/include/fdbclient/Audit.h index 4c0c77f8d0..382534f24f 100644 --- a/fdbclient/include/fdbclient/Audit.h +++ b/fdbclient/include/fdbclient/Audit.h @@ -23,16 +23,26 @@ #pragma once #include "fdbclient/FDBTypes.h" +#include "fdbrpc/fdbrpc.h" -enum class AuditPhase { Invalid = 0; Running = 1; Complete = 2; Error = 3; Failed = 4; }; +enum class AuditPhase : uint8_t { + Invalid = 0, + Running = 1, + Complete = 2, + Error = 3, + Failed = 4, +}; -enum class AuditType { Invalid = 0; ValidateHA = 1 }; +enum class AuditType : uint8_t { + Invalid = 0, + ValidateHA = 1, +}; struct AuditStorageState { constexpr static FileIdentifier file_identifier = 13804340; - ValidateStorageResult() = default; - ValidateStorageResult(UID id, AuditType type) : id(id), type(type), validatedKeys(0) {} + AuditStorageState() = default; + AuditStorageState(UID id, AuditType type) : id(id), type(static_cast(type)) {} template void serialize(Ar& ar) { @@ -63,7 +73,7 @@ struct AuditStorageRequest { template void serialize(Ar& ar) { - serializer(ar, id, range, actions, reply); + serializer(ar, id, range, type, reply); } UID id; @@ -79,7 +89,7 @@ struct TriggerAuditRequest { constexpr static FileIdentifier file_identifier = 1384445; TriggerAuditRequest() = default; - SplitShardRequest(AuditType type, KeyRange range) : type(type), range(range), force(false) {} + TriggerAuditRequest(AuditType type, KeyRange range) : type(static_cast(type)), range(range), force(false) {} void setType(AuditType type) { this->type = static_cast(this->type); } AuditType getType() const { return static_cast(this->type); } diff --git a/fdbclient/include/fdbclient/DataDistributorClientInterface.h b/fdbclient/include/fdbclient/DataDistributorClientInterface.h index f338d52bc1..4b5beca8fd 100644 --- a/fdbclient/include/fdbclient/DataDistributorClientInterface.h +++ b/fdbclient/include/fdbclient/DataDistributorClientInterface.h @@ -31,7 +31,7 @@ struct DataDistributorClientInterface { constexpr static FileIdentifier file_identifier = 15988863; - RequestStream openDatabase; + RequestStream openDatabase; RequestStream failureMonitoring; RequestStream databaseStatus; RequestStream> ping; diff --git a/fdbclient/include/fdbclient/StorageServerInterface.h b/fdbclient/include/fdbclient/StorageServerInterface.h index 43a84abf9c..b4b989190d 100644 --- a/fdbclient/include/fdbclient/StorageServerInterface.h +++ b/fdbclient/include/fdbclient/StorageServerInterface.h @@ -23,6 +23,7 @@ #pragma once #include +#include "fdbclient/Audit.h" #include "fdbclient/FDBTypes.h" #include "fdbclient/StorageCheckpoint.h" #include "fdbclient/StorageServerShard.h" diff --git a/fdbserver/DDTxnProcessor.actor.cpp b/fdbserver/DDTxnProcessor.actor.cpp index 3fea169516..f145ad3050 100644 --- a/fdbserver/DDTxnProcessor.actor.cpp +++ b/fdbserver/DDTxnProcessor.actor.cpp @@ -91,6 +91,59 @@ class DDTxnProcessorImpl { return IDDTxnProcessor::SourceServers{ std::vector(servers.begin(), servers.end()), completeSources }; } + ACTOR static Future getSourceServerInterfacesForRange(Database cx, + KeyRangeRef range) { + state std::vector res; + state Transaction tr(self->cx); + tr.setOption(FDBTransactionOptions::PRIORITY_SYSTEM_IMMEDIATE); + tr.setOption(FDBTransactionOptions::ACCESS_SYSTEM_KEYS); + + loop { + res.clear(); + try { + state RangeResult shards = wait(krmGetRanges(&tr, + keyServersPrefix, + range, + SERVER_KNOBS->MOVE_SHARD_KRM_ROW_LIMIT, + SERVER_KNOBS->MOVE_SHARD_KRM_BYTE_LIMIT)); + ASSERT(!shards.empty()); + + state RangeResult UIDtoTagMap = wait(tr.getRange(serverTagKeys, CLIENT_KNOBS->TOO_MANY)); + ASSERT(!UIDtoTagMap.more && UIDtoTagMap.size() < CLIENT_KNOBS->TOO_MANY); + + state int i = 0; + for (i = 0; i < shards.size() - 1; ++i) { + std::vector src; + std::vector dest; + UID srcId, destId; + decodeKeyServersValue(UIDtoTagMap, shards[i].value, src, dest, srcId, destId); + + state StorageServersForRange current(KeyRangeRef(shards[i].key, shards[i + 1].key)); + state int j = 0; + for (j = 0; j < src.size(); ++j) { + Optional serverListValue = wait(tr.get(serverListKeyFor(ssId))); + KeyRangeRef currentRange(shards[i].key, shards[i + 1].key); + if (!serverListValue.present()) { + TraceEvent(SevWarnAlways, "ScheduleAuditForRangeMissingServer", auditId) + .detail("StorageServer", ssId) + .detail("Range", currentRange); + continue; + } + const StorageServerInterface ssi = decodeServerListValue(serverListValue.get()); + current.servers.push_back(ssi); + } + res.push_back(current); + } + break; + } catch (Error& e) { + TraceEvent(SevWarnAlways, "TestValidateStorageError").errorUnsuppressed(e).detail("Range", range); + wait(tr.onError(e)); + } + } + + return res; + } + // set the system key space ACTOR static Future updateReplicaKeys(Database cx, std::vector> primaryDcId, @@ -401,6 +454,11 @@ Future DDTxnProcessor::getSourceServersForRange( return DDTxnProcessorImpl::getSourceServersForRange(cx, range); } +Future DDTxnProcessor::getSourceServerInterfacesForRange( + const KeyRangeRef range) { + return DDTxnProcessorImpl::getSourceServerInterfacesForRange(cx, range); +} + Future>> DDTxnProcessor::getServerListAndProcessClasses() { Transaction tr(cx); return NativeAPI::getServerListAndProcessClasses(&tr); diff --git a/fdbserver/DataDistribution.actor.cpp b/fdbserver/DataDistribution.actor.cpp index 04f93d74e5..e1e8803b8f 100644 --- a/fdbserver/DataDistribution.actor.cpp +++ b/fdbserver/DataDistribution.actor.cpp @@ -21,6 +21,7 @@ #include #include +#include "fdbclient/Audit.h" #include "fdbclient/DatabaseContext.h" #include "fdbclient/FDBOptions.g.h" #include "fdbclient/FDBTypes.h" @@ -1291,35 +1292,36 @@ ACTOR Future ddGetMetrics(GetDataDistributorMetricsRequest req, } ACTOR Future auditStorage(Reference self, TriggerAuditRequest req); -ACTOR Future scheduleAuditForRange(UID auditId, +ACTOR Future scheduleAuditForRange(Reference self, + std::shared_ptr actors, + std::shared_ptr> auditMap, + UID auditId, KeyRange range, - AuditType type, - Reference actors, - Reference> auditMap); -ACTOR Future doAuditStorage(Reference actors, - Reference> auditMap, + AuditType type); +ACTOR Future doAuditStorage(std::shared_ptr actors, + std::shared_ptr> auditMap, StorageServerInterface ssi, AuditStorageRequest req); ACTOR Future auditStorage(Reference self, TriggerAuditRequest req) { // TODO(heliu): Load running audit, and create one if no audit is running. - state Reference> auditMap = - makeReference>(AuditPhase::Invalid, allKeys.end); - state Reference actors = makeReference(true); + state std::shared_ptr> auditMap = + std::make_shared>(AuditPhase::Invalid, allKeys.end); + state std::shared_ptr actors = std::make_shared(true); state UID auditId = deterministicRandom()->randomUniqueID(); - Ranges f = auditMap.intersectingRanges(req.range); + auto f = auditMap->intersectingRanges(req.range); for (auto it = f.begin(); it != f.end(); ++it) { if (it->value() == AuditPhase::Invalid || it->value() == AuditPhase::Error) { actors->add(scheduleAuditForRange( - auditId, KeyRangeRef(it->range().begin, it->range().end), req.getType(), actors, auditMap)); + self, actors, auditMap, self, auditId, KeyRangeRef(it->range().begin, it->range().end), req.getType())); } } req.reply.send(auditId); try { - wait(actors.getResult()); + wait(actors->getResult()); // TODO(heliu): Set the audit result, and clear auditId. } catch (Error& e) { TraceEvent(SevWarnAlways, "DDAuditStorageOperationError", auditId) @@ -1331,51 +1333,26 @@ ACTOR Future auditStorage(Reference self, TriggerAuditReq return Void(); } -ACTOR Future scheduleAuditForRange(UID auditId, +ACTOR Future scheduleAuditForRange(Reference self, + std::shared_ptr actors, + std::shared_ptr> auditMap, + UID auditId, KeyRange range, - AuditType type, - Reference actors, - Reference> auditMap) { - // TODO(heliu): Load the audit map, or make auditStorage a streaming call, to skip audited ranges. - state Key begin = range.begin; + AuditType type) { + // TODO(heliu): Load the audit map, or make auditStorage a streaming call, to skip audited ranges. while (begin < range.end) { - state Transaction tr(cx); - tr.setOption(FDBTransactionOptions::PRIORITY_SYSTEM_IMMEDIATE); - tr.setOption(FDBTransactionOptions::ACCESS_SYSTEM_KEYS); try { - state RangeResult shards = wait(krmGetRanges(&tr, - keyServersPrefix, - KeyRangeRef(begin, range.end), - SERVER_KNOBS->MOVE_SHARD_KRM_ROW_LIMIT, - SERVER_KNOBS->MOVE_SHARD_KRM_BYTE_LIMIT)); - ASSERT(!shards.empty()); - - state RangeResult UIDtoTagMap = wait(tr.getRange(serverTagKeys, CLIENT_KNOBS->TOO_MANY)); - ASSERT(!UIDtoTagMap.more && UIDtoTagMap.size() < CLIENT_KNOBS->TOO_MANY); + state IDDTxnProcessor::StorageServersForRange rangeLocations = + self->getSourceServerInterfacesForRange(KeyRangeRef(begin, range.end)); state int i = 0; - for (i = 0; i < shards.size() - 1; ++i) { - std::vector src; - std::vector dest; - UID srcId, destId; - decodeKeyServersValue(UIDtoTagMap, shards[i].value, src, dest, srcId, destId); - - state UID ssId = serverListKeyFor(src[deterministicRandom()->randomInt(0, src.size())]); - Optional serverListValue = wait(tr.get(serverListKeyFor(ssId))); - KeyRangeRef currentRange(shards[i].key, shards[i + 1].key); - if (!serverListValue.present()) { - TraceEvent(SevWarnAlways, "ScheduleAuditForRangeMissingServer", auditId) - .detail("StorageServer", ssId) - .detail("Range", currentRange); - break; - } - const StorageServerInterface ssi = decodeServerListValue(serverListValue.get()); - - const AuditStorageRequest req(auditId, currentRange, type); - actors->add(doAuditStorage(actors, auditMap, ssi, req)); - begin = req.range.end; + for (i = 0; i < rangeLocations.size(); ++i) { + const AuditStorageRequest req(auditId, rangeLocations[i].range, type); + const int idx = deterministicRandom()->randomInt(0, rangeLocations[i].servers.size()); + actors->add(doAuditStorage(actors, auditMap, rangeLocations[i].servers[idx], req)); + begin = rangeLocations[i].range.end; wait(delay(0.01)); } } catch (Error& e) { @@ -1387,8 +1364,8 @@ ACTOR Future scheduleAuditForRange(UID auditId, return Void(); } -ACTOR Future doAuditStorage(Reference actors, - Reference> auditMap, +ACTOR Future doAuditStorage(std::shared_ptr actors, + std::shared_ptr> auditMap, StorageServerInterface ssi, AuditStorageRequest req) { TraceEvent(SevDebug, "DDAuditStorageBegin", req.id) @@ -1398,21 +1375,21 @@ ACTOR Future doAuditStorage(Reference actors, try { auditMap->insert(req.range, AuditPhase::Running); - ValidateStorageResult vResult = wait(ssi.auditStorage.getReply(req)); - TraceEvent e(vResult.error.empty() ? SevInfo : SevWarnAlways, "DDValidateStorageResult", req.requestId); + AuditStorageState vResult = wait(ssi.auditStorage.getReply(req)); + TraceEvent e(vResult.error.empty() ? SevInfo : SevWarnAlways, "DDAuditStorageState", req.id); e.detail("Range", req.range); e.detail("StorageServer", ssi.toString()); if (!vResult.error.empty()) { e.detail("ErrorMessage", vResult.error); } } catch (Error& e) { - TraceEvent(SevWarning, "DDValidateStorageError", req.requestId) + TraceEvent(SevWarn, "DDValidateStorageError", req.id) .errorUnsuppressed(e) .detail("Range", req.range) .detail("StorageServer", ssi.toString()); if (e.code() != error_code_actor_cancelled) { auditMap->insert(req.range, AuditPhase::Failed); - actors.add(ScheduleAuditForRange(req.id, req.range, req.getType(), actors, auditMap)); + actors->add(ScheduleAuditForRange(actors, auditMap, req.id, req.range, req.getType())); } } @@ -1487,7 +1464,7 @@ ACTOR Future dataDistributor(DataDistributorInterface di, Reference srcServers, completeSources; // the same as RelocateData.src, RelocateData.completeSources; }; + + struct StorageServersForRange { + StorageServersForRange(KeyRangeRef range) : range(range) {} + + std::vector servers; + KeyRange range; + }; + // get the source server list and complete source server list for range virtual Future getSourceServersForRange(const KeyRangeRef range) = 0; + virtual Future getSourceServerInterfacesForRange(const KeyRangeRef range) = 0; + // get the storage server list and Process class virtual Future>> getServerListAndProcessClasses() = 0; diff --git a/fdbserver/storageserver.actor.cpp b/fdbserver/storageserver.actor.cpp index fb665ae706..f6e89647db 100644 --- a/fdbserver/storageserver.actor.cpp +++ b/fdbserver/storageserver.actor.cpp @@ -24,6 +24,7 @@ #include #include "fmt/format.h" +#include "fdbclient/Audit.h" #include "fdbclient/CommitTransaction.h" #include "fdbclient/FDBTypes.h" #include "fdbrpc/fdbrpc.h" diff --git a/fdbserver/workloads/ValidateStorage.actor.cpp b/fdbserver/workloads/ValidateStorage.actor.cpp index c4c26bbb80..69e1b2ad16 100644 --- a/fdbserver/workloads/ValidateStorage.actor.cpp +++ b/fdbserver/workloads/ValidateStorage.actor.cpp @@ -18,6 +18,7 @@ * limitations under the License. */ +#include "fdbclient/Audit.h" #include "fdbclient/ManagementAPI.actor.h" #include "fdbclient/NativeAPI.actor.h" #include "fdbrpc/simulator.h" @@ -77,7 +78,7 @@ struct ValidateStorage : TestWorkload { Version _ = wait(self->populateData(self, cx, &kvs)); - std::cout << "TestValueWritten" << std::endl; + std::cout << "TestValueWritten" << std::endl; TraceEvent("TestValueWritten"); @@ -115,14 +116,14 @@ struct ValidateStorage : TestWorkload { } ACTOR Future validateData(ValidateStorage* self, Database cx, KeyRange range) { - std::cout << "0" << std::endl; + std::cout << "0" << std::endl; state Transaction tr(cx); tr.setOption(FDBTransactionOptions::PRIORITY_SYSTEM_IMMEDIATE); tr.setOption(FDBTransactionOptions::ACCESS_SYSTEM_KEYS); loop { try { - std::cout << "1" << std::endl; + std::cout << "1" << std::endl; state RangeResult shards = wait(krmGetRanges(&tr, keyServersPrefix, range, CLIENT_KNOBS->TOO_MANY, CLIENT_KNOBS->TOO_MANY)); ASSERT(!shards.empty() && !shards.more); @@ -133,7 +134,7 @@ struct ValidateStorage : TestWorkload { state int i = 0; for (i = 0; i < shards.size() - 1; ++i) { - std::cout << "2" << std::endl; + std::cout << "2" << std::endl; std::vector src; std::vector dest; UID srcId, destId; @@ -144,10 +145,11 @@ struct ValidateStorage : TestWorkload { ASSERT(serverListValue.present()); const StorageServerInterface ssi = decodeServerListValue(serverListValue.get()); AuditStorageRequest req(deterministicRandom()->randomUniqueID(), - KeyRangeRef(shards[i].key, shards[i + 1].key)); - ValidateStorageResult vResult = wait(ssi.auditStorage.getReply(req)); + KeyRangeRef(shards[i].key, shards[i + 1].key), + AuditType::ValidateHA); + AuditStorageState vResult = wait(ssi.auditStorage.getReply(req)); - std::cout << "3" << std::endl; + std::cout << "3" << std::endl; } break; } catch (Error& e) { From ad11c6e82d04f4919d49b0e1fabd5a10b5a27c4f Mon Sep 17 00:00:00 2001 From: He Liu Date: Tue, 6 Sep 2022 19:00:09 -0700 Subject: [PATCH 014/210] Cleanup. --- fdbclient/include/fdbclient/Audit.h | 6 +- fdbserver/DDTxnProcessor.actor.cpp | 22 +- fdbserver/DataDistribution.actor.cpp | 50 +- fdbserver/include/fdbserver/DDTxnProcessor.h | 7 +- fdbserver/storageserver.actor.cpp | 462 +++++++++--------- fdbserver/workloads/ValidateStorage.actor.cpp | 4 - flow/include/flow/error_definitions.h | 2 +- 7 files changed, 294 insertions(+), 259 deletions(-) diff --git a/fdbclient/include/fdbclient/Audit.h b/fdbclient/include/fdbclient/Audit.h index 382534f24f..b366a604e2 100644 --- a/fdbclient/include/fdbclient/Audit.h +++ b/fdbclient/include/fdbclient/Audit.h @@ -89,19 +89,21 @@ struct TriggerAuditRequest { constexpr static FileIdentifier file_identifier = 1384445; TriggerAuditRequest() = default; - TriggerAuditRequest(AuditType type, KeyRange range) : type(static_cast(type)), range(range), force(false) {} + TriggerAuditRequest(AuditType type, KeyRange range) + : type(static_cast(type)), range(range), force(false), async(false) {} void setType(AuditType type) { this->type = static_cast(this->type); } AuditType getType() const { return static_cast(this->type); } template void serialize(Ar& ar) { - serializer(ar, type, range, force, reply); + serializer(ar, type, range, force, async, reply); } uint8_t type; KeyRange range; bool force; + bool async; ReplyPromise reply; }; #endif diff --git a/fdbserver/DDTxnProcessor.actor.cpp b/fdbserver/DDTxnProcessor.actor.cpp index f145ad3050..d9ca9aeca5 100644 --- a/fdbserver/DDTxnProcessor.actor.cpp +++ b/fdbserver/DDTxnProcessor.actor.cpp @@ -91,10 +91,11 @@ class DDTxnProcessorImpl { return IDDTxnProcessor::SourceServers{ std::vector(servers.begin(), servers.end()), completeSources }; } - ACTOR static Future getSourceServerInterfacesForRange(Database cx, - KeyRangeRef range) { + ACTOR static Future> getSourceServerInterfacesForRange( + Database cx, + KeyRangeRef range) { state std::vector res; - state Transaction tr(self->cx); + state Transaction tr(cx); tr.setOption(FDBTransactionOptions::PRIORITY_SYSTEM_IMMEDIATE); tr.setOption(FDBTransactionOptions::ACCESS_SYSTEM_KEYS); @@ -113,19 +114,20 @@ class DDTxnProcessorImpl { state int i = 0; for (i = 0; i < shards.size() - 1; ++i) { - std::vector src; + state std::vector src; std::vector dest; UID srcId, destId; decodeKeyServersValue(UIDtoTagMap, shards[i].value, src, dest, srcId, destId); - state StorageServersForRange current(KeyRangeRef(shards[i].key, shards[i + 1].key)); + state IDDTxnProcessor::StorageServersForRange current( + KeyRangeRef(shards[i].key, shards[i + 1].key)); state int j = 0; for (j = 0; j < src.size(); ++j) { - Optional serverListValue = wait(tr.get(serverListKeyFor(ssId))); + Optional serverListValue = wait(tr.get(serverListKeyFor(src[j]))); KeyRangeRef currentRange(shards[i].key, shards[i + 1].key); if (!serverListValue.present()) { - TraceEvent(SevWarnAlways, "ScheduleAuditForRangeMissingServer", auditId) - .detail("StorageServer", ssId) + TraceEvent(SevWarnAlways, "GetSourceServerInterfacesMissing") + .detail("StorageServer", src[j]) .detail("Range", currentRange); continue; } @@ -136,7 +138,7 @@ class DDTxnProcessorImpl { } break; } catch (Error& e) { - TraceEvent(SevWarnAlways, "TestValidateStorageError").errorUnsuppressed(e).detail("Range", range); + TraceEvent(SevWarnAlways, "GetSourceServerInterfacesError").errorUnsuppressed(e).detail("Range", range); wait(tr.onError(e)); } } @@ -454,7 +456,7 @@ Future DDTxnProcessor::getSourceServersForRange( return DDTxnProcessorImpl::getSourceServersForRange(cx, range); } -Future DDTxnProcessor::getSourceServerInterfacesForRange( +Future> DDTxnProcessor::getSourceServerInterfacesForRange( const KeyRangeRef range) { return DDTxnProcessorImpl::getSourceServerInterfacesForRange(cx, range); } diff --git a/fdbserver/DataDistribution.actor.cpp b/fdbserver/DataDistribution.actor.cpp index e1e8803b8f..d67a4a2b3f 100644 --- a/fdbserver/DataDistribution.actor.cpp +++ b/fdbserver/DataDistribution.actor.cpp @@ -1298,7 +1298,8 @@ ACTOR Future scheduleAuditForRange(Reference self, UID auditId, KeyRange range, AuditType type); -ACTOR Future doAuditStorage(std::shared_ptr actors, +ACTOR Future doAuditStorage(Reference self, + std::shared_ptr actors, std::shared_ptr> auditMap, StorageServerInterface ssi, AuditStorageRequest req); @@ -1310,19 +1311,18 @@ ACTOR Future auditStorage(Reference self, TriggerAuditReq state std::shared_ptr actors = std::make_shared(true); state UID auditId = deterministicRandom()->randomUniqueID(); - auto f = auditMap->intersectingRanges(req.range); - for (auto it = f.begin(); it != f.end(); ++it) { - if (it->value() == AuditPhase::Invalid || it->value() == AuditPhase::Error) { - actors->add(scheduleAuditForRange( - self, actors, auditMap, self, auditId, KeyRangeRef(it->range().begin, it->range().end), req.getType())); - } - } + actors->add(scheduleAuditForRange(self, actors, auditMap, auditId, req.range, req.getType())); - req.reply.send(auditId); + if (req.async) { + req.reply.send(auditId); + } try { wait(actors->getResult()); // TODO(heliu): Set the audit result, and clear auditId. + if (!req.async) { + req.reply.send(auditId); + } } catch (Error& e) { TraceEvent(SevWarnAlways, "DDAuditStorageOperationError", auditId) .errorUnsuppressed(e) @@ -1339,32 +1339,50 @@ ACTOR Future scheduleAuditForRange(Reference self, UID auditId, KeyRange range, AuditType type) { - // TODO(heliu): Load the audit map, or make auditStorage a streaming call, to skip audited ranges. + // TODO(heliu): Load the audit map for `range`. + state Key begin = range.begin; + state KeyRange currentRange = range; while (begin < range.end) { + currentRange = KeyRangeRef(begin, range.end); + + // Find the first keyrange that hasn't been validated. + auto f = auditMap->intersectingRanges(currentRange); + for (auto it = f.begin(); it != f.end(); ++it) { + if (it->value() != AuditPhase::Invalid && it->value() != AuditPhase::Failed) { + begin = it->range().end; + currentRange = KeyRangeRef(it->range().end, currentRange.end); + } else { + currentRange = KeyRangeRef(it->range().begin, it->range().end); + break; + } + } try { - state IDDTxnProcessor::StorageServersForRange rangeLocations = - self->getSourceServerInterfacesForRange(KeyRangeRef(begin, range.end)); + state std::vector rangeLocations = + wait(self->txnProcessor->getSourceServerInterfacesForRange(currentRange)); state int i = 0; for (i = 0; i < rangeLocations.size(); ++i) { const AuditStorageRequest req(auditId, rangeLocations[i].range, type); const int idx = deterministicRandom()->randomInt(0, rangeLocations[i].servers.size()); - actors->add(doAuditStorage(actors, auditMap, rangeLocations[i].servers[idx], req)); + actors->add(doAuditStorage(self, actors, auditMap, rangeLocations[i].servers[idx], req)); begin = rangeLocations[i].range.end; wait(delay(0.01)); } } catch (Error& e) { TraceEvent(SevWarnAlways, "TestValidateStorageError").errorUnsuppressed(e).detail("Range", range); - wait(tr.onError(e)); + if (e.code() == error_code_actor_cancelled) { + throw e; + } } } return Void(); } -ACTOR Future doAuditStorage(std::shared_ptr actors, +ACTOR Future doAuditStorage(Reference self, + std::shared_ptr actors, std::shared_ptr> auditMap, StorageServerInterface ssi, AuditStorageRequest req) { @@ -1389,7 +1407,7 @@ ACTOR Future doAuditStorage(std::shared_ptr actors, .detail("StorageServer", ssi.toString()); if (e.code() != error_code_actor_cancelled) { auditMap->insert(req.range, AuditPhase::Failed); - actors->add(ScheduleAuditForRange(actors, auditMap, req.id, req.range, req.getType())); + actors->add(scheduleAuditForRange(self, actors, auditMap, req.id, req.range, req.getType())); } } diff --git a/fdbserver/include/fdbserver/DDTxnProcessor.h b/fdbserver/include/fdbserver/DDTxnProcessor.h index 439941e705..bf9fb36ad8 100644 --- a/fdbserver/include/fdbserver/DDTxnProcessor.h +++ b/fdbserver/include/fdbserver/DDTxnProcessor.h @@ -38,6 +38,7 @@ public: }; struct StorageServersForRange { + StorageServersForRange() = default; StorageServersForRange(KeyRangeRef range) : range(range) {} std::vector servers; @@ -47,7 +48,8 @@ public: // get the source server list and complete source server list for range virtual Future getSourceServersForRange(const KeyRangeRef range) = 0; - virtual Future getSourceServerInterfacesForRange(const KeyRangeRef range) = 0; + virtual Future> getSourceServerInterfacesForRange( + const KeyRangeRef range) = 0; // get the storage server list and Process class virtual Future>> getServerListAndProcessClasses() = 0; @@ -86,6 +88,9 @@ public: Future getSourceServersForRange(const KeyRangeRef range) override; + Future> getSourceServerInterfacesForRange( + const KeyRangeRef range) override; + // Call NativeAPI implementation directly Future>> getServerListAndProcessClasses() override; diff --git a/fdbserver/storageserver.actor.cpp b/fdbserver/storageserver.actor.cpp index f6e89647db..2360b692d8 100644 --- a/fdbserver/storageserver.actor.cpp +++ b/fdbserver/storageserver.actor.cpp @@ -3496,231 +3496,6 @@ ACTOR Future readRange(StorageServer* data, return result; } -ACTOR Future validateRangeAgainstServer(StorageServer* data, - KeyRange range, - Version version, - StorageServerInterface remoteServer) { - TraceEvent(SevDebug, "ServeValidateRangeAgainstServerBegin", data->thisServerID) - .detail("Range", range) - .detail("Version", version) - .detail("Servers", remoteServer.toString()); - - state int validatedKeys = 0; - loop { - try { - state int limit = 1e4; - state int limitBytes = CLIENT_KNOBS->REPLY_BYTE_LIMIT; - state GetKeyValuesRequest req; - state GetKeyValuesRequest localReq; - req.begin = firstGreaterOrEqual(range.begin); - req.end = firstGreaterOrEqual(range.end); - req.limit = limit; - req.limitBytes = limitBytes; - req.version = version; - req.tags = TagSet(); - localReq.begin = firstGreaterOrEqual(range.begin); - localReq.end = firstGreaterOrEqual(range.end); - localReq.limit = limit; - localReq.limitBytes = limitBytes; - localReq.version = version; - localReq.tags = TagSet(); - - // Try getting the entries in the specified range - state Future> remoteKeyValueFuture = - remoteServer.getKeyValues.getReplyUnlessFailedFor(req, 2, 0); - data->actors.add(getKeyValuesQ(data, localReq)); - state ErrorOr remoteResult = wait(remoteKeyValueFuture); - GetKeyValuesReply local = wait(localReq.reply.getFuture()); - Key lastKey = range.begin; - state std::string error; - - // Compare the results with other storage servers - if (remoteResult.isError()) { - throw remoteResult.getError(); - } - - state GetKeyValuesReply remote = remoteResult.get(); - // Loop indeces - const int end = std::min(local.data.size(), remote.data.size()); - int i = 0; - for (; i < end; ++i) { - KeyValueRef remoteKV = remote.data[i]; - KeyValueRef localKV = local.data[i]; - - if (remoteKV.key != localKV.key) { - error = format("Key Mismatch: local server (%lld): %s, remote server(%lld) %s", - data->thisServerID.first(), - Traceable::toString(localKV.key), - remoteServer.uniqueID.first(), - Traceable::toString(remoteKV.key)); - } else if (remoteKV.value != localKV.value) { - error = format("Value Mismatch for Key %s: local server (%lld): %s, remote server(%lld) %s", - Traceable::toString(localKV.key), - data->thisServerID.first(), - Traceable::toString(localKV.value), - remoteServer.uniqueID.first(), - Traceable::toString(remoteKV.value)); - } else { - TraceEvent(SevDebug, "ValidatedKey", data->thisServerID).detail("Key", localKV.key); - ++validatedKeys; - } - - lastKey = localKV.key; - } - - if (!error.empty()) { - break; - } - - if (!local.more && !remote.more && local.data.size() == remote.data.size()) { - break; - } else if (i >= local.data.size() && !local.more && i < remote.data.size()) { - error = format("Missing key(s) form local server (%lld), next remote server(%lld) key: %s", - data->thisServerID.first(), - remoteServer.uniqueID.first(), - Traceable::toString(remote.data[i].key)); - break; - } else if (i >= remote.data.size() && !remote.more && i < local.data.size()) { - error = format("Missing key(s) form remote server (%lld), next local server(%lld) key: %s", - remoteServer.uniqueID.first(), - data->thisServerID.first(), - Traceable::toString(local.data[i].key)); - break; - } - - range = KeyRangeRef(keyAfter(lastKey), range.end); - } catch (Error& e) { - TraceEvent(SevWarnAlways, "ValidateRangeAgainstServerError", data->thisServerID) - .errorUnsuppressed(e) - .detail("RemoteServer", remoteServer.toString()) - .detail("Range", range) - .detail("Version", version); - throw e; - } - } - - if (!error.empty()) { - TraceEvent(SevWarnAlways, "ValidateRangeAgainstServerError", data->thisServerID) - .detail("Range", range) - .detail("Version", version) - .detail("ErrorMessage", error) - .detail("RemoteServer", remoteServer.toString()); - } - - TraceEvent(SevDebug, "ServeValidateRangeAgainstServerEnd", data->thisServerID) - .detail("Range", range) - .detail("Version", version) - .detail("ValidatedKeys", validatedKeys) - .detail("Servers", remoteServer.toString()); - - return Void(); -} - -ACTOR Future validateRangeShard(StorageServer* data, KeyRange range, std::vector candidates) { - TraceEvent(SevDebug, "ServeValidateRangeShardBegin", data->thisServerID) - .detail("Range", range) - .detail("Servers", describe(candidates)); - - state Version version; - state std::vector> serverListValues; - state Transaction tr(data->cx); - tr.setOption(FDBTransactionOptions::PRIORITY_SYSTEM_IMMEDIATE); - tr.setOption(FDBTransactionOptions::ACCESS_SYSTEM_KEYS); - - loop { - try { - std::vector>> serverListEntries; - for (const UID& id : candidates) { - serverListEntries.push_back(tr.get(serverListKeyFor(id))); - } - - std::vector> serverListValues_ = wait(getAll(serverListEntries)); - serverListValues = serverListValues_; - Version version_ = wait(tr.getReadVersion()); - version = version_; - break; - } catch (Error& e) { - wait(tr.onError(e)); - } - } - - std::unordered_map> ssis; - std::string thisDcId; - for (const auto& v : serverListValues) { - const StorageServerInterface ssi = decodeServerListValue(v.get()); - if (ssi.uniqueID == data->thisServerID) { - thisDcId = ssi.locality.describeDcId(); - } - ssis[ssi.locality.describeDcId()].push_back(ssi); - } - - StorageServerInterface* remoteServer = nullptr; - for (auto& [dcId, ssiList] : ssis) { - if (dcId != thisDcId) { - if (ssiList.empty()) { - break; - } - const int idx = deterministicRandom()->randomInt(0, ssiList.size()); - remoteServer = &ssiList[idx]; - break; - } - } - - if (remoteServer != nullptr) { - wait(validateRangeAgainstServer(data, range, version, *remoteServer)); - } - - return Void(); -} - -ACTOR Future auditStorageQ(StorageServer* self, AuditStorageRequest req) { - wait(self->serveAuditStorageParallelismLock.take(TaskPriority::DefaultYield)); - state FlowLock::Releaser holder(self->serveAuditStorageParallelismLock); - - TraceEvent("ServeAuditStorageBegin", self->thisServerID) - .detail("RequestID", req.id) - .detail("Range", req.range) - .detail("Type", req.type); - - state Transaction tr(self->cx); - tr.setOption(FDBTransactionOptions::PRIORITY_SYSTEM_IMMEDIATE); - tr.setOption(FDBTransactionOptions::ACCESS_SYSTEM_KEYS); - - try { - loop { - try { - state RangeResult shards = wait(krmGetRanges(&tr, - keyServersPrefix, - req.range, - SERVER_KNOBS->MOVE_SHARD_KRM_ROW_LIMIT, - SERVER_KNOBS->MOVE_SHARD_KRM_BYTE_LIMIT)); - ASSERT(!shards.empty() && !shards.more); - - state RangeResult UIDtoTagMap = wait(tr.getRange(serverTagKeys, CLIENT_KNOBS->TOO_MANY)); - ASSERT(!UIDtoTagMap.more && UIDtoTagMap.size() < CLIENT_KNOBS->TOO_MANY); - - std::vector> fs; - for (int i = 0; i < shards.size() - 1; ++i) { - std::vector src; - std::vector dest; - UID srcId, destId; - decodeKeyServersValue(UIDtoTagMap, shards[i].value, src, dest, srcId, destId); - fs.push_back(validateRangeShard(self, KeyRangeRef(shards[i].key, shards[i + 1].key), src)); - } - - wait(waitForAll(fs)); - break; - } catch (Error& e) { - wait(tr.onError(e)); - } - } - } catch (Error& e) { - req.reply.sendError(audit_storage_failed()); - } - - return Void(); -} - KeyRangeRef StorageServer::clampRangeToTenant(KeyRangeRef range, Optional tenantEntry, Arena& arena) { if (tenantEntry.present()) { return KeyRangeRef(range.begin.startsWith(tenantEntry.get().prefix) ? range.begin : tenantEntry.get().prefix, @@ -4276,6 +4051,243 @@ Key constructMappedKey(KeyValueRef* keyValue, return mappedKeyTuple.pack(); } +ACTOR Future validateRangeAgainstServer(StorageServer* data, + KeyRange range, + Version version, + StorageServerInterface remoteServer) { + TraceEvent(SevDebug, "ServeValidateRangeAgainstServerBegin", data->thisServerID) + .detail("Range", range) + .detail("Version", version) + .detail("Servers", remoteServer.toString()); + + state int validatedKeys = 0; + loop { + try { + state int limit = 1e4; + state int limitBytes = CLIENT_KNOBS->REPLY_BYTE_LIMIT; + state GetKeyValuesRequest req; + state GetKeyValuesRequest localReq; + req.begin = firstGreaterOrEqual(range.begin); + req.end = firstGreaterOrEqual(range.end); + req.limit = limit; + req.limitBytes = limitBytes; + req.version = version; + req.tags = TagSet(); + localReq.begin = firstGreaterOrEqual(range.begin); + localReq.end = firstGreaterOrEqual(range.end); + localReq.limit = limit; + localReq.limitBytes = limitBytes; + localReq.version = version; + localReq.tags = TagSet(); + + // Try getting the entries in the specified range + state Future> remoteKeyValueFuture = + remoteServer.getKeyValues.getReplyUnlessFailedFor(req, 2, 0); + data->actors.add(getKeyValuesQ(data, localReq)); + state ErrorOr remoteResult = wait(remoteKeyValueFuture); + try { + state GetKeyValuesReply local = wait(localReq.reply.getFuture()); + } catch (Error& e) { + TraceEvent(SevDebug, "ValidateRangeGetLocalKeyValuesError", data->thisServerID) + .errorUnsuppressed(e) + .detail("Range", range); + } + Key lastKey = range.begin; + state std::string error; + + // Compare the results with other storage servers + if (remoteResult.isError()) { + TraceEvent(SevDebug, "ValidateRangeGetRemoteKeyValuesError", data->thisServerID) + .errorUnsuppressed(remoteResult.getError()) + .detail("Range", range); + throw remoteResult.getError(); + } + + state GetKeyValuesReply remote = remoteResult.get(); + // Loop indeces + const int end = std::min(local.data.size(), remote.data.size()); + int i = 0; + for (; i < end; ++i) { + KeyValueRef remoteKV = remote.data[i]; + KeyValueRef localKV = local.data[i]; + + if (remoteKV.key != localKV.key) { + error = format("Key Mismatch: local server (%lld): %s, remote server(%lld) %s", + data->thisServerID.first(), + Traceable::toString(localKV.key), + remoteServer.uniqueID.first(), + Traceable::toString(remoteKV.key)); + } else if (remoteKV.value != localKV.value) { + error = format("Value Mismatch for Key %s: local server (%lld): %s, remote server(%lld) %s", + Traceable::toString(localKV.key), + data->thisServerID.first(), + Traceable::toString(localKV.value), + remoteServer.uniqueID.first(), + Traceable::toString(remoteKV.value)); + } else { + TraceEvent(SevDebug, "ValidatedKey", data->thisServerID).detail("Key", localKV.key); + ++validatedKeys; + } + + lastKey = localKV.key; + } + + if (!error.empty()) { + break; + } + + if (!local.more && !remote.more && local.data.size() == remote.data.size()) { + break; + } else if (i >= local.data.size() && !local.more && i < remote.data.size()) { + error = format("Missing key(s) form local server (%lld), next remote server(%lld) key: %s", + data->thisServerID.first(), + remoteServer.uniqueID.first(), + Traceable::toString(remote.data[i].key)); + break; + } else if (i >= remote.data.size() && !remote.more && i < local.data.size()) { + error = format("Missing key(s) form remote server (%lld), next local server(%lld) key: %s", + remoteServer.uniqueID.first(), + data->thisServerID.first(), + Traceable::toString(local.data[i].key)); + break; + } + + range = KeyRangeRef(keyAfter(lastKey), range.end); + } catch (Error& e) { + TraceEvent(SevWarnAlways, "ValidateRangeAgainstServerError", data->thisServerID) + .errorUnsuppressed(e) + .detail("RemoteServer", remoteServer.toString()) + .detail("Range", range) + .detail("Version", version); + throw e; + } + } + + if (!error.empty()) { + TraceEvent(SevError, "ValidateRangeAgainstServerError", data->thisServerID) + .detail("Range", range) + .detail("Version", version) + .detail("ErrorMessage", error) + .detail("RemoteServer", remoteServer.toString()); + } + + TraceEvent(SevDebug, "ServeValidateRangeAgainstServerEnd", data->thisServerID) + .detail("Range", range) + .detail("Version", version) + .detail("ValidatedKeys", validatedKeys) + .detail("Servers", remoteServer.toString()); + + return Void(); +} + +ACTOR Future validateRangeShard(StorageServer* data, KeyRange range, std::vector candidates) { + TraceEvent(SevDebug, "ServeValidateRangeShardBegin", data->thisServerID) + .detail("Range", range) + .detail("Servers", describe(candidates)); + + state Version version; + state std::vector> serverListValues; + state Transaction tr(data->cx); + tr.setOption(FDBTransactionOptions::PRIORITY_SYSTEM_IMMEDIATE); + tr.setOption(FDBTransactionOptions::ACCESS_SYSTEM_KEYS); + + loop { + try { + std::vector>> serverListEntries; + for (const UID& id : candidates) { + serverListEntries.push_back(tr.get(serverListKeyFor(id))); + } + + std::vector> serverListValues_ = wait(getAll(serverListEntries)); + serverListValues = serverListValues_; + Version version_ = wait(tr.getReadVersion()); + version = version_; + break; + } catch (Error& e) { + wait(tr.onError(e)); + } + } + + std::unordered_map> ssis; + std::string thisDcId; + for (const auto& v : serverListValues) { + const StorageServerInterface ssi = decodeServerListValue(v.get()); + if (ssi.uniqueID == data->thisServerID) { + thisDcId = ssi.locality.describeDcId(); + } + ssis[ssi.locality.describeDcId()].push_back(ssi); + } + + StorageServerInterface* remoteServer = nullptr; + for (auto& [dcId, ssiList] : ssis) { + if (dcId != thisDcId) { + if (ssiList.empty()) { + break; + } + const int idx = deterministicRandom()->randomInt(0, ssiList.size()); + remoteServer = &ssiList[idx]; + break; + } + } + + if (remoteServer != nullptr) { + wait(validateRangeAgainstServer(data, range, version, *remoteServer)); + } + + return Void(); +} + +ACTOR Future auditStorageQ(StorageServer* self, AuditStorageRequest req) { + wait(self->serveAuditStorageParallelismLock.take(TaskPriority::DefaultYield)); + state FlowLock::Releaser holder(self->serveAuditStorageParallelismLock); + + TraceEvent("ServeAuditStorageBegin", self->thisServerID) + .detail("RequestID", req.id) + .detail("Range", req.range) + .detail("AuditType", req.type); + + state Transaction tr(self->cx); + tr.setOption(FDBTransactionOptions::PRIORITY_SYSTEM_IMMEDIATE); + tr.setOption(FDBTransactionOptions::ACCESS_SYSTEM_KEYS); + + try { + loop { + try { + state RangeResult shards = wait(krmGetRanges(&tr, + keyServersPrefix, + req.range, + SERVER_KNOBS->MOVE_SHARD_KRM_ROW_LIMIT, + SERVER_KNOBS->MOVE_SHARD_KRM_BYTE_LIMIT)); + ASSERT(!shards.empty() && !shards.more); + + state RangeResult UIDtoTagMap = wait(tr.getRange(serverTagKeys, CLIENT_KNOBS->TOO_MANY)); + ASSERT(!UIDtoTagMap.more && UIDtoTagMap.size() < CLIENT_KNOBS->TOO_MANY); + + std::vector> fs; + for (int i = 0; i < shards.size() - 1; ++i) { + std::vector src; + std::vector dest; + UID srcId, destId; + decodeKeyServersValue(UIDtoTagMap, shards[i].value, src, dest, srcId, destId); + fs.push_back(validateRangeShard(self, KeyRangeRef(shards[i].key, shards[i + 1].key), src)); + } + + wait(waitForAll(fs)); + AuditStorageState res(req.id, req.getType()); + res.setPhase(AuditPhase::Complete); + req.reply.send(res); + break; + } catch (Error& e) { + wait(tr.onError(e)); + } + } + } catch (Error& e) { + req.reply.sendError(audit_storage_failed()); + } + + return Void(); +} + TEST_CASE("/fdbserver/storageserver/constructMappedKey") { Key key = Tuple::makeTuple("key-0"_sr, "key-1"_sr, "key-2"_sr).getDataAsStandalone(); Value value = Tuple::makeTuple("value-0"_sr, "value-1"_sr, "value-2"_sr).getDataAsStandalone(); diff --git a/fdbserver/workloads/ValidateStorage.actor.cpp b/fdbserver/workloads/ValidateStorage.actor.cpp index 69e1b2ad16..12de7ebb75 100644 --- a/fdbserver/workloads/ValidateStorage.actor.cpp +++ b/fdbserver/workloads/ValidateStorage.actor.cpp @@ -116,14 +116,12 @@ struct ValidateStorage : TestWorkload { } ACTOR Future validateData(ValidateStorage* self, Database cx, KeyRange range) { - std::cout << "0" << std::endl; state Transaction tr(cx); tr.setOption(FDBTransactionOptions::PRIORITY_SYSTEM_IMMEDIATE); tr.setOption(FDBTransactionOptions::ACCESS_SYSTEM_KEYS); loop { try { - std::cout << "1" << std::endl; state RangeResult shards = wait(krmGetRanges(&tr, keyServersPrefix, range, CLIENT_KNOBS->TOO_MANY, CLIENT_KNOBS->TOO_MANY)); ASSERT(!shards.empty() && !shards.more); @@ -148,8 +146,6 @@ struct ValidateStorage : TestWorkload { KeyRangeRef(shards[i].key, shards[i + 1].key), AuditType::ValidateHA); AuditStorageState vResult = wait(ssi.auditStorage.getReply(req)); - - std::cout << "3" << std::endl; } break; } catch (Error& e) { diff --git a/flow/include/flow/error_definitions.h b/flow/include/flow/error_definitions.h index b689b64176..9260051ed7 100755 --- a/flow/include/flow/error_definitions.h +++ b/flow/include/flow/error_definitions.h @@ -125,7 +125,7 @@ ERROR( invalid_cluster_id, 1217, "Attempted to join cluster with a different clu ERROR( restart_cluster_controller, 1218, "Restart cluster controller process" ) ERROR( please_reboot_remote_kv_store, 1219, "Need to reboot the storage engine process as it died abnormally") ERROR( incompatible_software_version, 1220, "Current software does not support database format" ) -ERROR( audit_storage_failed, 1221, "Validate storage consistency error" ) +ERROR( audit_storage_failed, 1221, "Validate storage consistency operation failed" ) // 15xx Platform errors ERROR( platform_error, 1500, "Platform error" ) From 16e31c6bd6def913937911b1565cc6299e2f8923 Mon Sep 17 00:00:00 2001 From: He Liu Date: Wed, 7 Sep 2022 09:40:19 -0700 Subject: [PATCH 015/210] Make AuditStorage more reliable on SS. --- fdbserver/storageserver.actor.cpp | 40 +++++++++++-------- fdbserver/workloads/ValidateStorage.actor.cpp | 8 +++- 2 files changed, 30 insertions(+), 18 deletions(-) diff --git a/fdbserver/storageserver.actor.cpp b/fdbserver/storageserver.actor.cpp index 2360b692d8..786453b06f 100644 --- a/fdbserver/storageserver.actor.cpp +++ b/fdbserver/storageserver.actor.cpp @@ -4237,51 +4237,57 @@ ACTOR Future validateRangeShard(StorageServer* data, KeyRange range, std:: return Void(); } -ACTOR Future auditStorageQ(StorageServer* self, AuditStorageRequest req) { - wait(self->serveAuditStorageParallelismLock.take(TaskPriority::DefaultYield)); - state FlowLock::Releaser holder(self->serveAuditStorageParallelismLock); +ACTOR Future auditStorageQ(StorageServer* data, AuditStorageRequest req) { + wait(data->serveAuditStorageParallelismLock.take(TaskPriority::DefaultYield)); + state FlowLock::Releaser holder(data->serveAuditStorageParallelismLock); - TraceEvent("ServeAuditStorageBegin", self->thisServerID) + TraceEvent(SevInfo, "ServeAuditStorageBegin", data->thisServerID) .detail("RequestID", req.id) .detail("Range", req.range) .detail("AuditType", req.type); - state Transaction tr(self->cx); - tr.setOption(FDBTransactionOptions::PRIORITY_SYSTEM_IMMEDIATE); - tr.setOption(FDBTransactionOptions::ACCESS_SYSTEM_KEYS); + state Key begin = req.range.begin; + state std::vector> fs; try { - loop { + while (begin < req.range.end) { + state Transaction tr(data->cx); + tr.setOption(FDBTransactionOptions::PRIORITY_SYSTEM_IMMEDIATE); + tr.setOption(FDBTransactionOptions::ACCESS_SYSTEM_KEYS); try { state RangeResult shards = wait(krmGetRanges(&tr, keyServersPrefix, req.range, SERVER_KNOBS->MOVE_SHARD_KRM_ROW_LIMIT, SERVER_KNOBS->MOVE_SHARD_KRM_BYTE_LIMIT)); - ASSERT(!shards.empty() && !shards.more); + ASSERT(!shards.empty()); state RangeResult UIDtoTagMap = wait(tr.getRange(serverTagKeys, CLIENT_KNOBS->TOO_MANY)); ASSERT(!UIDtoTagMap.more && UIDtoTagMap.size() < CLIENT_KNOBS->TOO_MANY); - std::vector> fs; for (int i = 0; i < shards.size() - 1; ++i) { std::vector src; std::vector dest; UID srcId, destId; decodeKeyServersValue(UIDtoTagMap, shards[i].value, src, dest, srcId, destId); - fs.push_back(validateRangeShard(self, KeyRangeRef(shards[i].key, shards[i + 1].key), src)); + fs.push_back(validateRangeShard(data, KeyRangeRef(shards[i].key, shards[i + 1].key), src)); + begin = shards[i + 1].key; } - - wait(waitForAll(fs)); - AuditStorageState res(req.id, req.getType()); - res.setPhase(AuditPhase::Complete); - req.reply.send(res); - break; } catch (Error& e) { wait(tr.onError(e)); } + + wait(waitForAll(fs)); + AuditStorageState res(req.id, req.getType()); + res.setPhase(AuditPhase::Complete); + req.reply.send(res); } } catch (Error& e) { + TraceEvent(SevWarn, "ServeAuditStorageError", data->thisServerID) + .errorUnsuppressed(e) + .detail("RequestID", req.id) + .detail("Range", req.range) + .detail("AuditType", req.type); req.reply.sendError(audit_storage_failed()); } diff --git a/fdbserver/workloads/ValidateStorage.actor.cpp b/fdbserver/workloads/ValidateStorage.actor.cpp index 12de7ebb75..a72e6d5202 100644 --- a/fdbserver/workloads/ValidateStorage.actor.cpp +++ b/fdbserver/workloads/ValidateStorage.actor.cpp @@ -150,7 +150,13 @@ struct ValidateStorage : TestWorkload { break; } catch (Error& e) { TraceEvent(SevWarnAlways, "TestValidateStorageError").errorUnsuppressed(e).detail("Range", range); - wait(tr.onError(e)); + try { + wait(tr.onError(e)); + } catch (Error& e) { + if (e.code() != error_code_audit_storage_failed) { + throw e; + } + } } } From 071ed41caa77557aea5a1cf2d49fb334892f9aff Mon Sep 17 00:00:00 2001 From: He Liu Date: Wed, 7 Sep 2022 22:34:14 -0700 Subject: [PATCH 016/210] ManagementAPI auditStorage passed test. --- fdbclient/ManagementAPI.actor.cpp | 15 ++++++ .../include/fdbclient/ManagementAPI.actor.h | 2 + fdbserver/ClusterController.actor.cpp | 47 +++++++++++++++++++ fdbserver/DataDistribution.actor.cpp | 18 ++++--- .../fdbserver/DataDistributorInterface.h | 3 +- fdbserver/workloads/ValidateStorage.actor.cpp | 4 ++ 6 files changed, 82 insertions(+), 7 deletions(-) diff --git a/fdbclient/ManagementAPI.actor.cpp b/fdbclient/ManagementAPI.actor.cpp index 2a5c9ac910..0663f845e9 100644 --- a/fdbclient/ManagementAPI.actor.cpp +++ b/fdbclient/ManagementAPI.actor.cpp @@ -2291,6 +2291,21 @@ ACTOR Future forceRecovery(Reference clusterFile } } +ACTOR Future auditStorage(Reference clusterFile, KeyRange range, AuditType type) { + state Reference>> clusterInterface(new AsyncVar>); + state Future leaderMon = monitorLeader(clusterFile, clusterInterface); + + loop { + while (!clusterInterface->get().present()) { + wait(clusterInterface->onChange()); + } + + UID auditId = wait(clusterInterface->get().get().triggerAudit.getReply(TriggerAuditRequest(type, range))); + TraceEvent("ManagementAPIAuditStorageEnd").detail("AuditID", auditId); + return auditId; + } +} + ACTOR Future waitForPrimaryDC(Database cx, StringRef dcId) { state ReadYourWritesTransaction tr(cx); diff --git a/fdbclient/include/fdbclient/ManagementAPI.actor.h b/fdbclient/include/fdbclient/ManagementAPI.actor.h index 1268185b1b..d841dacca6 100644 --- a/fdbclient/include/fdbclient/ManagementAPI.actor.h +++ b/fdbclient/include/fdbclient/ManagementAPI.actor.h @@ -137,6 +137,8 @@ ACTOR Future setDDMode(Database cx, int mode); ACTOR Future forceRecovery(Reference clusterFile, Standalone dcId); +ACTOR Future auditStorage(Reference clusterFile, KeyRange range, AuditType type); + ACTOR Future printHealthyZone(Database cx); ACTOR Future clearHealthyZone(Database cx, bool printWarning = false, bool clearSSFailureZoneString = false); ACTOR Future setHealthyZone(Database cx, StringRef zoneId, double seconds, bool printWarning = false); diff --git a/fdbserver/ClusterController.actor.cpp b/fdbserver/ClusterController.actor.cpp index 485d3ab3cb..5d1911c412 100644 --- a/fdbserver/ClusterController.actor.cpp +++ b/fdbserver/ClusterController.actor.cpp @@ -1943,6 +1943,52 @@ ACTOR Future handleForcedRecoveries(ClusterControllerData* self, ClusterCo } } +ACTOR Future triggerAuditStorage(ClusterControllerData* self, TriggerAuditRequest req) { + TraceEvent(SevInfo, "TriggerAuditStorageBegin", self->id).detail("Range", req.range).detail("AuditType", req.type); + try { + while (self->db.serverInfo->get().recoveryState < RecoveryState::ACCEPTING_COMMITS || + !self->db.serverInfo->get().distributor.present()) { + wait(self->db.serverInfo->onChange()); + } + + TriggerAuditRequest fReq(req.getType(), req.range); + state UID auditId = wait(self->db.serverInfo->get().distributor.get().triggerAudit.getReply(fReq)); + TraceEvent(SevDebug, "TriggerAuditStorageResult", self->id) + .detail("AuditID", auditId) + .detail("Range", req.range) + .detail("AuditType", req.type); + if (!req.reply.isSet()) { + TraceEvent(SevDebug, "TriggerAuditStorageReply", self->id) + .detail("AuditID", auditId) + .detail("Range", req.range) + .detail("AuditType", req.type); + req.reply.send(auditId); + } + } catch (Error& e) { + TraceEvent(SevDebug, "TriggerAuditStorageError", self->id) + .errorUnsuppressed(e) + .detail("AuditID", auditId) + .detail("Range", req.range) + .detail("AuditType", req.type); + if (!req.reply.isSet()) { + req.reply.sendError(audit_storage_failed()); + } + } + + return Void(); +} + +ACTOR Future handleTriggerAuditStorage(ClusterControllerData* self, ClusterControllerFullInterface interf) { + loop { + TriggerAuditRequest req = waitNext(interf.clientInterface.triggerAudit.getFuture()); + TraceEvent(SevDebug, "TriggerAuditStorageReceived", self->id) + .detail("ClusterControllerDcId", self->clusterControllerDcId) + .detail("Range", req.range) + .detail("AuditType", req.type); + self->addActor.send(triggerAuditStorage(self, req)); + } +} + struct SingletonRecruitThrottler { double lastRecruitStart; @@ -2548,6 +2594,7 @@ ACTOR Future clusterControllerCore(ClusterControllerFullInterface interf, self.addActor.send(updatedChangedDatacenters(&self)); self.addActor.send(updateDatacenterVersionDifference(&self)); self.addActor.send(handleForcedRecoveries(&self, interf)); + self.addActor.send(handleTriggerAuditStorage(&self, interf)); self.addActor.send(monitorDataDistributor(&self)); self.addActor.send(monitorRatekeeper(&self)); self.addActor.send(monitorBlobManager(&self)); diff --git a/fdbserver/DataDistribution.actor.cpp b/fdbserver/DataDistribution.actor.cpp index d67a4a2b3f..bad8d29c2c 100644 --- a/fdbserver/DataDistribution.actor.cpp +++ b/fdbserver/DataDistribution.actor.cpp @@ -1310,24 +1310,29 @@ ACTOR Future auditStorage(Reference self, TriggerAuditReq std::make_shared>(AuditPhase::Invalid, allKeys.end); state std::shared_ptr actors = std::make_shared(true); state UID auditId = deterministicRandom()->randomUniqueID(); + TraceEvent(SevDebug, "DDAuditStorageBegin", auditId).detail("Range", req.range).detail("AuditType", req.type); actors->add(scheduleAuditForRange(self, actors, auditMap, auditId, req.range, req.getType())); - if (req.async) { + if (req.async && !req.reply.isSet()) { req.reply.send(auditId); } try { wait(actors->getResult()); + TraceEvent(SevDebug, "DDAuditStorageEnd", auditId).detail("Range", req.range).detail("AuditType", req.type); // TODO(heliu): Set the audit result, and clear auditId. - if (!req.async) { + if (!req.async && !req.reply.isSet()) { + TraceEvent(SevDebug, "DDAuditStorageReply", auditId) + .detail("Range", req.range) + .detail("AuditType", req.type); req.reply.send(auditId); } } catch (Error& e) { TraceEvent(SevWarnAlways, "DDAuditStorageOperationError", auditId) .errorUnsuppressed(e) .detail("Range", req.range) - .detail("Type", req.type); + .detail("AuditType", req.type); } return Void(); @@ -1339,6 +1344,7 @@ ACTOR Future scheduleAuditForRange(Reference self, UID auditId, KeyRange range, AuditType type) { + TraceEvent(SevDebug, "DDScheduleAuditBegin", auditId).detail("Range", range).detail("AuditType", type); // TODO(heliu): Load the audit map for `range`. state Key begin = range.begin; state KeyRange currentRange = range; @@ -1353,7 +1359,7 @@ ACTOR Future scheduleAuditForRange(Reference self, begin = it->range().end; currentRange = KeyRangeRef(it->range().end, currentRange.end); } else { - currentRange = KeyRangeRef(it->range().begin, it->range().end); + currentRange = KeyRangeRef(it->range().begin, it->range().end) & currentRange; break; } } @@ -1371,7 +1377,7 @@ ACTOR Future scheduleAuditForRange(Reference self, wait(delay(0.01)); } } catch (Error& e) { - TraceEvent(SevWarnAlways, "TestValidateStorageError").errorUnsuppressed(e).detail("Range", range); + TraceEvent(SevWarnAlways, "DDScheduleAuditRangeError").errorUnsuppressed(e).detail("Range", range); if (e.code() == error_code_actor_cancelled) { throw e; } @@ -1388,7 +1394,7 @@ ACTOR Future doAuditStorage(Reference self, AuditStorageRequest req) { TraceEvent(SevDebug, "DDAuditStorageBegin", req.id) .detail("Range", req.range) - .detail("Type", req.type) + .detail("AuditType", req.type) .detail("StorageServer", ssi.toString()); try { diff --git a/fdbserver/include/fdbserver/DataDistributorInterface.h b/fdbserver/include/fdbserver/DataDistributorInterface.h index 28ce23d71c..15dec4a0bb 100644 --- a/fdbserver/include/fdbserver/DataDistributorInterface.h +++ b/fdbserver/include/fdbserver/DataDistributorInterface.h @@ -59,7 +59,8 @@ struct DataDistributorInterface { distributorExclCheckReq, dataDistributorMetrics, distributorSplitRange, - storageWigglerState); + storageWigglerState, + triggerAudit); } }; diff --git a/fdbserver/workloads/ValidateStorage.actor.cpp b/fdbserver/workloads/ValidateStorage.actor.cpp index a72e6d5202..899c2f1989 100644 --- a/fdbserver/workloads/ValidateStorage.actor.cpp +++ b/fdbserver/workloads/ValidateStorage.actor.cpp @@ -85,6 +85,10 @@ struct ValidateStorage : TestWorkload { wait(self->validateData(self, cx, KeyRangeRef("TestKeyA"_sr, "TestKeyF"_sr))); TraceEvent("TestValueVerified"); + UID auditId = wait(auditStorage( + cx->getConnectionRecord(), KeyRangeRef("TestKeyA"_sr, "TestKeyF"_sr), AuditType::ValidateHA)); + TraceEvent("TestValidateEnd").detail("AuditID", auditId); + int ignore = wait(setDDMode(cx, 1)); return Void(); } From 9e911956d9d2c520f89e6279ca93542032f0aab4 Mon Sep 17 00:00:00 2001 From: He Liu Date: Thu, 8 Sep 2022 15:41:22 -0700 Subject: [PATCH 017/210] Handle GetKeyValuesReply errors. --- fdbclient/ServerKnobs.cpp | 2 +- fdbserver/storageserver.actor.cpp | 46 +++++++++++++------ fdbserver/workloads/ValidateStorage.actor.cpp | 21 +++++---- tests/fast/ValidateStorage.toml | 7 ++- 4 files changed, 53 insertions(+), 23 deletions(-) diff --git a/fdbclient/ServerKnobs.cpp b/fdbclient/ServerKnobs.cpp index e0331029fa..d3cb87561e 100644 --- a/fdbclient/ServerKnobs.cpp +++ b/fdbclient/ServerKnobs.cpp @@ -746,7 +746,7 @@ void ServerKnobs::initialize(Randomize randomize, ClientKnobs* clientKnobs, IsSi init( MAX_PARALLEL_QUICK_GET_VALUE, 50 ); if ( randomize && BUGGIFY ) MAX_PARALLEL_QUICK_GET_VALUE = deterministicRandom()->randomInt(1, 100); init( QUICK_GET_KEY_VALUES_LIMIT, 2000 ); init( QUICK_GET_KEY_VALUES_LIMIT_BYTES, 1e7 ); - init( STORAGE_SERVER_SHARD_AWARE, true ); + init( STORAGE_SERVER_SHARD_AWARE, false ); //Wait Failure init( MAX_OUTSTANDING_WAIT_FAILURE_REQUESTS, 250 ); if( randomize && BUGGIFY ) MAX_OUTSTANDING_WAIT_FAILURE_REQUESTS = 2; diff --git a/fdbserver/storageserver.actor.cpp b/fdbserver/storageserver.actor.cpp index 786453b06f..56956b0dc0 100644 --- a/fdbserver/storageserver.actor.cpp +++ b/fdbserver/storageserver.actor.cpp @@ -4058,7 +4058,7 @@ ACTOR Future validateRangeAgainstServer(StorageServer* data, TraceEvent(SevDebug, "ServeValidateRangeAgainstServerBegin", data->thisServerID) .detail("Range", range) .detail("Version", version) - .detail("Servers", remoteServer.toString()); + .detail("RemoteServer", remoteServer.toString()); state int validatedKeys = 0; loop { @@ -4087,10 +4087,17 @@ ACTOR Future validateRangeAgainstServer(StorageServer* data, state ErrorOr remoteResult = wait(remoteKeyValueFuture); try { state GetKeyValuesReply local = wait(localReq.reply.getFuture()); + if (local.error.present()) { + throw local.error.get(); + } + TraceEvent(SevDebug, "ValidateRangeGetLocalKeyValuesResult", data->thisServerID) + .detail("Range", range) + .detail("RemoteResultSize", local.data.size()); } catch (Error& e) { TraceEvent(SevDebug, "ValidateRangeGetLocalKeyValuesError", data->thisServerID) .errorUnsuppressed(e) .detail("Range", range); + throw e; } Key lastKey = range.begin; state std::string error; @@ -4104,21 +4111,35 @@ ACTOR Future validateRangeAgainstServer(StorageServer* data, } state GetKeyValuesReply remote = remoteResult.get(); + if (remote.error.present()) { + throw remote.error.get(); + } + TraceEvent(SevDebug, "ValidateRangeGetRemoteKeyValuesResult", data->thisServerID) + .detail("Range", range) + .detail("RemoteResultSize", remote.data.size()); // Loop indeces const int end = std::min(local.data.size(), remote.data.size()); int i = 0; for (; i < end; ++i) { KeyValueRef remoteKV = remote.data[i]; KeyValueRef localKV = local.data[i]; + if (!range.contains(remoteKV.key) || !range.contains(localKV.key)) { + TraceEvent(SevDebug, "SSValidateRangeKeyOutOfRange", data->thisServerID) + .detail("Range", range) + .detail("RemoteServer", remoteServer.toString()) + .detail("LocalKey", Traceable::toString(localKV.key)) + .detail("RemoteKey", Traceable::toString(remoteKV.key)); + throw wrong_shard_server(); + } if (remoteKV.key != localKV.key) { - error = format("Key Mismatch: local server (%lld): %s, remote server(%lld) %s", + error = format("Key Mismatch: local server (%016llx): %s, remote server(%016llx) %s", data->thisServerID.first(), Traceable::toString(localKV.key), remoteServer.uniqueID.first(), Traceable::toString(remoteKV.key)); } else if (remoteKV.value != localKV.value) { - error = format("Value Mismatch for Key %s: local server (%lld): %s, remote server(%lld) %s", + error = format("Value Mismatch for Key %s: local server (%016llx): %s, remote server(%016llx) %s", Traceable::toString(localKV.key), data->thisServerID.first(), Traceable::toString(localKV.value), @@ -4139,13 +4160,13 @@ ACTOR Future validateRangeAgainstServer(StorageServer* data, if (!local.more && !remote.more && local.data.size() == remote.data.size()) { break; } else if (i >= local.data.size() && !local.more && i < remote.data.size()) { - error = format("Missing key(s) form local server (%lld), next remote server(%lld) key: %s", + error = format("Missing key(s) form local server (%lld), next remote server(%016llx) key: %s", data->thisServerID.first(), remoteServer.uniqueID.first(), Traceable::toString(remote.data[i].key)); break; } else if (i >= remote.data.size() && !remote.more && i < local.data.size()) { - error = format("Missing key(s) form remote server (%lld), next local server(%lld) key: %s", + error = format("Missing key(s) form remote server (%lld), next local server(%016llx) key: %s", remoteServer.uniqueID.first(), data->thisServerID.first(), Traceable::toString(local.data[i].key)); @@ -4251,9 +4272,9 @@ ACTOR Future auditStorageQ(StorageServer* data, AuditStorageRequest req) { try { while (begin < req.range.end) { - state Transaction tr(data->cx); - tr.setOption(FDBTransactionOptions::PRIORITY_SYSTEM_IMMEDIATE); - tr.setOption(FDBTransactionOptions::ACCESS_SYSTEM_KEYS); + state Transaction tr(data->cx); + tr.setOption(FDBTransactionOptions::PRIORITY_SYSTEM_IMMEDIATE); + tr.setOption(FDBTransactionOptions::ACCESS_SYSTEM_KEYS); try { state RangeResult shards = wait(krmGetRanges(&tr, keyServersPrefix, @@ -4276,12 +4297,11 @@ ACTOR Future auditStorageQ(StorageServer* data, AuditStorageRequest req) { } catch (Error& e) { wait(tr.onError(e)); } - - wait(waitForAll(fs)); - AuditStorageState res(req.id, req.getType()); - res.setPhase(AuditPhase::Complete); - req.reply.send(res); } + wait(waitForAll(fs)); + AuditStorageState res(req.id, req.getType()); + res.setPhase(AuditPhase::Complete); + req.reply.send(res); } catch (Error& e) { TraceEvent(SevWarn, "ServeAuditStorageError", data->thisServerID) .errorUnsuppressed(e) diff --git a/fdbserver/workloads/ValidateStorage.actor.cpp b/fdbserver/workloads/ValidateStorage.actor.cpp index 899c2f1989..d0cff501c0 100644 --- a/fdbserver/workloads/ValidateStorage.actor.cpp +++ b/fdbserver/workloads/ValidateStorage.actor.cpp @@ -69,6 +69,7 @@ struct ValidateStorage : TestWorkload { ACTOR Future _start(ValidateStorage* self, Database cx) { // int ignore = wait(setDDMode(cx, 0)); + TraceEvent("ValidateStorageTestBegin"); state std::map kvs({ { "TestKeyA"_sr, "TestValueA"_sr }, { "TestKeyB"_sr, "TestValueB"_sr }, { "TestKeyC"_sr, "TestValueC"_sr }, @@ -78,18 +79,24 @@ struct ValidateStorage : TestWorkload { Version _ = wait(self->populateData(self, cx, &kvs)); - std::cout << "TestValueWritten" << std::endl; - TraceEvent("TestValueWritten"); wait(self->validateData(self, cx, KeyRangeRef("TestKeyA"_sr, "TestKeyF"_sr))); TraceEvent("TestValueVerified"); - UID auditId = wait(auditStorage( - cx->getConnectionRecord(), KeyRangeRef("TestKeyA"_sr, "TestKeyF"_sr), AuditType::ValidateHA)); - TraceEvent("TestValidateEnd").detail("AuditID", auditId); + loop { + try { + UID auditId = wait(auditStorage( + cx->getConnectionRecord(), KeyRangeRef("TestKeyA"_sr, "TestKeyF"_sr), AuditType::ValidateHA)); + TraceEvent("TestValidateEnd").detail("AuditID", auditId); + break; + } catch (Error& e) { + TraceEvent("AuditStorageError").errorUnsuppressed(e); + wait(delay(1)); + } + } - int ignore = wait(setDDMode(cx, 1)); + // int ignore = wait(setDDMode(cx, 1)); return Void(); } @@ -135,8 +142,6 @@ struct ValidateStorage : TestWorkload { state int i = 0; for (i = 0; i < shards.size() - 1; ++i) { - - std::cout << "2" << std::endl; std::vector src; std::vector dest; UID srcId, destId; diff --git a/tests/fast/ValidateStorage.toml b/tests/fast/ValidateStorage.toml index b38c74037f..3e58e68088 100644 --- a/tests/fast/ValidateStorage.toml +++ b/tests/fast/ValidateStorage.toml @@ -1,8 +1,13 @@ [configuration] config = 'triple' +storageEngineType = 5 generateFearless = true allowDefaultTenant = false -machineCount = 15 +machineCount = 45 + +[[knobs]] +shard_encode_location_metadata = true +storage_server_shard_aware = true [[test]] testTitle = 'ValidateStorageWorkload' From e6846e1ed5b4cee747b926edf12bde59fa12b4e5 Mon Sep 17 00:00:00 2001 From: He Liu Date: Fri, 9 Sep 2022 15:04:34 -0700 Subject: [PATCH 018/210] Handle auditStorage API errors. --- fdbserver/DataDistribution.actor.cpp | 11 +++++++---- fdbserver/storageserver.actor.cpp | 15 +++++++++++++++ fdbserver/workloads/ValidateStorage.actor.cpp | 9 ++++++++- tests/fast/ValidateStorage.toml | 2 +- 4 files changed, 31 insertions(+), 6 deletions(-) diff --git a/fdbserver/DataDistribution.actor.cpp b/fdbserver/DataDistribution.actor.cpp index bad8d29c2c..67321f024f 100644 --- a/fdbserver/DataDistribution.actor.cpp +++ b/fdbserver/DataDistribution.actor.cpp @@ -1399,12 +1399,15 @@ ACTOR Future doAuditStorage(Reference self, try { auditMap->insert(req.range, AuditPhase::Running); - AuditStorageState vResult = wait(ssi.auditStorage.getReply(req)); - TraceEvent e(vResult.error.empty() ? SevInfo : SevWarnAlways, "DDAuditStorageState", req.id); + ErrorOr vResult = wait(ssi.auditStorage.getReplyUnlessFailedFor(req, 2, 0)); + if (vResult.isError()) { + throw vResult.getError(); + } + TraceEvent e(vResult.get().error.empty() ? SevInfo : SevWarnAlways, "DDAuditStorageState", req.id); e.detail("Range", req.range); e.detail("StorageServer", ssi.toString()); - if (!vResult.error.empty()) { - e.detail("ErrorMessage", vResult.error); + if (!vResult.get().error.empty()) { + e.detail("ErrorMessage", vResult.get().error); } } catch (Error& e) { TraceEvent(SevWarn, "DDValidateStorageError", req.id) diff --git a/fdbserver/storageserver.actor.cpp b/fdbserver/storageserver.actor.cpp index 56956b0dc0..cd17e0f8b5 100644 --- a/fdbserver/storageserver.actor.cpp +++ b/fdbserver/storageserver.actor.cpp @@ -4232,6 +4232,9 @@ ACTOR Future validateRangeShard(StorageServer* data, KeyRange range, std:: std::unordered_map> ssis; std::string thisDcId; for (const auto& v : serverListValues) { + if (!v.present()) { + continue; + } const StorageServerInterface ssi = decodeServerListValue(v.get()); if (ssi.uniqueID == data->thisServerID) { thisDcId = ssi.locality.describeDcId(); @@ -4239,6 +4242,13 @@ ACTOR Future validateRangeShard(StorageServer* data, KeyRange range, std:: ssis[ssi.locality.describeDcId()].push_back(ssi); } + if (ssis.size() < 2) { + TraceEvent(SevWarn, "ServeValidateRangeShardNotHAConfig", data->thisServerID) + .detail("Range", range) + .detail("Servers", describe(candidates)); + return Void(); + } + StorageServerInterface* remoteServer = nullptr; for (auto& [dcId, ssiList] : ssis) { if (dcId != thisDcId) { @@ -4253,6 +4263,11 @@ ACTOR Future validateRangeShard(StorageServer* data, KeyRange range, std:: if (remoteServer != nullptr) { wait(validateRangeAgainstServer(data, range, version, *remoteServer)); + } else { + TraceEvent(SevWarn, "ServeValidateRangeShardRemoteNotFound", data->thisServerID) + .detail("Range", range) + .detail("Servers", describe(candidates)); + throw audit_storage_failed(); } return Void(); diff --git a/fdbserver/workloads/ValidateStorage.actor.cpp b/fdbserver/workloads/ValidateStorage.actor.cpp index d0cff501c0..ace63aa2f0 100644 --- a/fdbserver/workloads/ValidateStorage.actor.cpp +++ b/fdbserver/workloads/ValidateStorage.actor.cpp @@ -127,6 +127,7 @@ struct ValidateStorage : TestWorkload { } ACTOR Future validateData(ValidateStorage* self, Database cx, KeyRange range) { + TraceEvent("TestValidateStorageBegin").detail("Range", range); state Transaction tr(cx); tr.setOption(FDBTransactionOptions::PRIORITY_SYSTEM_IMMEDIATE); tr.setOption(FDBTransactionOptions::ACCESS_SYSTEM_KEYS); @@ -151,10 +152,16 @@ struct ValidateStorage : TestWorkload { Optional serverListValue = wait(tr.get(serverListKeyFor(src[idx]))); ASSERT(serverListValue.present()); const StorageServerInterface ssi = decodeServerListValue(serverListValue.get()); + TraceEvent("TestValidateStorageSendingRequest") + .detail("Range", range) + .detail("StorageServer", ssi.toString()); AuditStorageRequest req(deterministicRandom()->randomUniqueID(), KeyRangeRef(shards[i].key, shards[i + 1].key), AuditType::ValidateHA); - AuditStorageState vResult = wait(ssi.auditStorage.getReply(req)); + Optional vResult = wait(timeout(ssi.auditStorage.getReply(req), 5)); + if (!vResult.present()) { + throw audit_storage_failed(); + } } break; } catch (Error& e) { diff --git a/tests/fast/ValidateStorage.toml b/tests/fast/ValidateStorage.toml index 3e58e68088..9a32b67e2d 100644 --- a/tests/fast/ValidateStorage.toml +++ b/tests/fast/ValidateStorage.toml @@ -1,5 +1,5 @@ [configuration] -config = 'triple' +config = 'double' storageEngineType = 5 generateFearless = true allowDefaultTenant = false From e6df139204960e1e921f090608b3bf092bb733c1 Mon Sep 17 00:00:00 2001 From: He Liu Date: Mon, 12 Sep 2022 09:35:24 -0700 Subject: [PATCH 019/210] Get sources servers by DC in DDAuditRange. --- fdbclient/include/fdbclient/Audit.h | 3 ++- fdbserver/DDTxnProcessor.actor.cpp | 2 +- fdbserver/DataDistribution.actor.cpp | 20 +++++++++++++++----- fdbserver/include/fdbserver/DDTxnProcessor.h | 3 ++- 4 files changed, 20 insertions(+), 8 deletions(-) diff --git a/fdbclient/include/fdbclient/Audit.h b/fdbclient/include/fdbclient/Audit.h index b366a604e2..55c4fb368c 100644 --- a/fdbclient/include/fdbclient/Audit.h +++ b/fdbclient/include/fdbclient/Audit.h @@ -73,12 +73,13 @@ struct AuditStorageRequest { template void serialize(Ar& ar) { - serializer(ar, id, range, type, reply); + serializer(ar, id, range, type, targetServers, reply); } UID id; KeyRange range; uint8_t type; + std::vector targetServers; ReplyPromise reply; }; diff --git a/fdbserver/DDTxnProcessor.actor.cpp b/fdbserver/DDTxnProcessor.actor.cpp index d9ca9aeca5..104a32451b 100644 --- a/fdbserver/DDTxnProcessor.actor.cpp +++ b/fdbserver/DDTxnProcessor.actor.cpp @@ -132,7 +132,7 @@ class DDTxnProcessorImpl { continue; } const StorageServerInterface ssi = decodeServerListValue(serverListValue.get()); - current.servers.push_back(ssi); + current.servers[ssi.locality.describeDcId()].push_back(ssi); } res.push_back(current); } diff --git a/fdbserver/DataDistribution.actor.cpp b/fdbserver/DataDistribution.actor.cpp index 67321f024f..76197d1355 100644 --- a/fdbserver/DataDistribution.actor.cpp +++ b/fdbserver/DataDistribution.actor.cpp @@ -1344,7 +1344,7 @@ ACTOR Future scheduleAuditForRange(Reference self, UID auditId, KeyRange range, AuditType type) { - TraceEvent(SevDebug, "DDScheduleAuditBegin", auditId).detail("Range", range).detail("AuditType", type); + TraceEvent(SevDebug, "DDScheduleAuditForRangeBegin", auditId).detail("Range", range).detail("AuditType", type); // TODO(heliu): Load the audit map for `range`. state Key begin = range.begin; state KeyRange currentRange = range; @@ -1370,9 +1370,18 @@ ACTOR Future scheduleAuditForRange(Reference self, state int i = 0; for (i = 0; i < rangeLocations.size(); ++i) { - const AuditStorageRequest req(auditId, rangeLocations[i].range, type); - const int idx = deterministicRandom()->randomInt(0, rangeLocations[i].servers.size()); - actors->add(doAuditStorage(self, actors, auditMap, rangeLocations[i].servers[idx], req)); + AuditStorageRequest req(auditId, rangeLocations[i].range, type); + if (type == AuditType::ValidateHA && rangeLocations[i].servers.size() >= 2) { + auto it = rangeLocations[i].servers.begin(); + const int idx = deterministicRandom()->randomInt(0, it->second.size()); + StorageServerInterface& targetServer = it->second[idx]; + ++it; + for (; it != rangeLocations[i].servers.end(); ++it) { + const int idx = deterministicRandom()->randomInt(0, it->second.size()); + req.targetServers.push_back(it->second[idx].id()); + } + actors->add(doAuditStorage(self, actors, auditMap, targetServer, req)); + } begin = rangeLocations[i].range.end; wait(delay(0.01)); } @@ -1395,7 +1404,8 @@ ACTOR Future doAuditStorage(Reference self, TraceEvent(SevDebug, "DDAuditStorageBegin", req.id) .detail("Range", req.range) .detail("AuditType", req.type) - .detail("StorageServer", ssi.toString()); + .detail("StorageServer", ssi.toString()) + .detail("TargetServers", describe(req.targetServers)); try { auditMap->insert(req.range, AuditPhase::Running); diff --git a/fdbserver/include/fdbserver/DDTxnProcessor.h b/fdbserver/include/fdbserver/DDTxnProcessor.h index bf9fb36ad8..24fc34e0ac 100644 --- a/fdbserver/include/fdbserver/DDTxnProcessor.h +++ b/fdbserver/include/fdbserver/DDTxnProcessor.h @@ -41,7 +41,8 @@ public: StorageServersForRange() = default; StorageServersForRange(KeyRangeRef range) : range(range) {} - std::vector servers; + // A map of dcId : list of servers + std::map> servers; KeyRange range; }; From 95e444fe6e441e7816162ae04752764c2f3f59e8 Mon Sep 17 00:00:00 2001 From: He Liu Date: Mon, 12 Sep 2022 13:54:01 -0700 Subject: [PATCH 020/210] SS audit storage against specific server. --- fdbserver/storageserver.actor.cpp | 98 +++++++++++++++++++++++-------- 1 file changed, 75 insertions(+), 23 deletions(-) diff --git a/fdbserver/storageserver.actor.cpp b/fdbserver/storageserver.actor.cpp index cd17e0f8b5..4348892f0d 100644 --- a/fdbserver/storageserver.actor.cpp +++ b/fdbserver/storageserver.actor.cpp @@ -4273,6 +4273,53 @@ ACTOR Future validateRangeShard(StorageServer* data, KeyRange range, std:: return Void(); } +ACTOR Future validateRangeShardAgainstServers(StorageServer* data, KeyRange range, std::vector candidates) { + TraceEvent(SevDebug, "ServeValidateRangeShardAgainstServersBegin", data->thisServerID) + .detail("Range", range) + .detail("TargetServers", describe(candidates)); + + state Version version; + state std::vector> serverListValues; + state Transaction tr(data->cx); + tr.setOption(FDBTransactionOptions::PRIORITY_SYSTEM_IMMEDIATE); + tr.setOption(FDBTransactionOptions::ACCESS_SYSTEM_KEYS); + + loop { + try { + std::vector>> serverListEntries; + for (const UID& id : candidates) { + if (id != data->thisServerID) { + serverListEntries.push_back(tr.get(serverListKeyFor(id))); + } + } + + std::vector> serverListValues_ = wait(getAll(serverListEntries)); + serverListValues = serverListValues_; + Version version_ = wait(tr.getReadVersion()); + version = version_; + break; + } catch (Error& e) { + wait(tr.onError(e)); + } + } + + std::vector ssis; + for (const auto& v : serverListValues) { + ssis.push_back(decodeServerListValue(v.get())); + } + + if (!ssis.empty()) { + wait(validateRangeAgainstServer(data, range, version, ssis[0])); + } else { + TraceEvent(SevWarn, "ServeValidateRangeShardRemoteNotFound", data->thisServerID) + .detail("Range", range) + .detail("Servers", describe(candidates)); + throw audit_storage_failed(); + } + + return Void(); +} + ACTOR Future auditStorageQ(StorageServer* data, AuditStorageRequest req) { wait(data->serveAuditStorageParallelismLock.take(TaskPriority::DefaultYield)); state FlowLock::Releaser holder(data->serveAuditStorageParallelismLock); @@ -4280,38 +4327,43 @@ ACTOR Future auditStorageQ(StorageServer* data, AuditStorageRequest req) { TraceEvent(SevInfo, "ServeAuditStorageBegin", data->thisServerID) .detail("RequestID", req.id) .detail("Range", req.range) - .detail("AuditType", req.type); + .detail("AuditType", req.type) + .detail("TargetServers", describe(req.targetServers)); state Key begin = req.range.begin; state std::vector> fs; try { - while (begin < req.range.end) { - state Transaction tr(data->cx); - tr.setOption(FDBTransactionOptions::PRIORITY_SYSTEM_IMMEDIATE); - tr.setOption(FDBTransactionOptions::ACCESS_SYSTEM_KEYS); - try { - state RangeResult shards = wait(krmGetRanges(&tr, - keyServersPrefix, - req.range, - SERVER_KNOBS->MOVE_SHARD_KRM_ROW_LIMIT, - SERVER_KNOBS->MOVE_SHARD_KRM_BYTE_LIMIT)); - ASSERT(!shards.empty()); + if (req.targetServers.empty()) { + while (begin < req.range.end) { + state Transaction tr(data->cx); + tr.setOption(FDBTransactionOptions::PRIORITY_SYSTEM_IMMEDIATE); + tr.setOption(FDBTransactionOptions::ACCESS_SYSTEM_KEYS); + try { + state RangeResult shards = wait(krmGetRanges(&tr, + keyServersPrefix, + req.range, + SERVER_KNOBS->MOVE_SHARD_KRM_ROW_LIMIT, + SERVER_KNOBS->MOVE_SHARD_KRM_BYTE_LIMIT)); + ASSERT(!shards.empty()); - state RangeResult UIDtoTagMap = wait(tr.getRange(serverTagKeys, CLIENT_KNOBS->TOO_MANY)); - ASSERT(!UIDtoTagMap.more && UIDtoTagMap.size() < CLIENT_KNOBS->TOO_MANY); + state RangeResult UIDtoTagMap = wait(tr.getRange(serverTagKeys, CLIENT_KNOBS->TOO_MANY)); + ASSERT(!UIDtoTagMap.more && UIDtoTagMap.size() < CLIENT_KNOBS->TOO_MANY); - for (int i = 0; i < shards.size() - 1; ++i) { - std::vector src; - std::vector dest; - UID srcId, destId; - decodeKeyServersValue(UIDtoTagMap, shards[i].value, src, dest, srcId, destId); - fs.push_back(validateRangeShard(data, KeyRangeRef(shards[i].key, shards[i + 1].key), src)); - begin = shards[i + 1].key; + for (int i = 0; i < shards.size() - 1; ++i) { + std::vector src; + std::vector dest; + UID srcId, destId; + decodeKeyServersValue(UIDtoTagMap, shards[i].value, src, dest, srcId, destId); + fs.push_back(validateRangeShard(data, KeyRangeRef(shards[i].key, shards[i + 1].key), src)); + begin = shards[i + 1].key; + } + } catch (Error& e) { + wait(tr.onError(e)); } - } catch (Error& e) { - wait(tr.onError(e)); } + } else { + fs.push_back(validateRangeShardAgainstServers(data, req.range, req.targetServers)); } wait(waitForAll(fs)); AuditStorageState res(req.id, req.getType()); From 56cf51e2a0f564a967b1c17b1b54317509c07e69 Mon Sep 17 00:00:00 2001 From: He Liu Date: Mon, 12 Sep 2022 14:32:34 -0700 Subject: [PATCH 021/210] Audit allKeys in auditStorage test. --- fdbserver/workloads/ValidateStorage.actor.cpp | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/fdbserver/workloads/ValidateStorage.actor.cpp b/fdbserver/workloads/ValidateStorage.actor.cpp index ace63aa2f0..97d44bf470 100644 --- a/fdbserver/workloads/ValidateStorage.actor.cpp +++ b/fdbserver/workloads/ValidateStorage.actor.cpp @@ -86,8 +86,7 @@ struct ValidateStorage : TestWorkload { loop { try { - UID auditId = wait(auditStorage( - cx->getConnectionRecord(), KeyRangeRef("TestKeyA"_sr, "TestKeyF"_sr), AuditType::ValidateHA)); + UID auditId = wait(auditStorage(cx->getConnectionRecord(), allKeys, AuditType::ValidateHA)); TraceEvent("TestValidateEnd").detail("AuditID", auditId); break; } catch (Error& e) { @@ -158,7 +157,8 @@ struct ValidateStorage : TestWorkload { AuditStorageRequest req(deterministicRandom()->randomUniqueID(), KeyRangeRef(shards[i].key, shards[i + 1].key), AuditType::ValidateHA); - Optional vResult = wait(timeout(ssi.auditStorage.getReply(req), 5)); + Optional vResult = + wait(timeout(ssi.auditStorage.getReply(req), 5)); if (!vResult.present()) { throw audit_storage_failed(); } From ebabe916f1c61c78d8361356ecaed9be6c5064e2 Mon Sep 17 00:00:00 2001 From: He Liu Date: Tue, 13 Sep 2022 09:28:41 -0700 Subject: [PATCH 022/210] Cleanup. --- fdbclient/ManagementAPI.actor.cpp | 2 +- .../DataDistributorClientInterface.h | 326 ------------------ .../include/fdbclient/ManagementAPI.actor.h | 1 + .../fdbclient/StorageServerInterface.h | 1 + fdbserver/ClusterController.actor.cpp | 10 +- fdbserver/DDTxnProcessor.actor.cpp | 8 +- fdbserver/DataDistribution.actor.cpp | 2 +- fdbserver/include/fdbserver/DDTxnProcessor.h | 10 +- 8 files changed, 16 insertions(+), 344 deletions(-) delete mode 100644 fdbclient/include/fdbclient/DataDistributorClientInterface.h diff --git a/fdbclient/ManagementAPI.actor.cpp b/fdbclient/ManagementAPI.actor.cpp index 0663f845e9..ea6f30c892 100644 --- a/fdbclient/ManagementAPI.actor.cpp +++ b/fdbclient/ManagementAPI.actor.cpp @@ -2301,7 +2301,7 @@ ACTOR Future auditStorage(Reference clusterFile, } UID auditId = wait(clusterInterface->get().get().triggerAudit.getReply(TriggerAuditRequest(type, range))); - TraceEvent("ManagementAPIAuditStorageEnd").detail("AuditID", auditId); + TraceEvent(SevDebug, "ManagementAPIAuditStorageEnd").detail("AuditID", auditId); return auditId; } } diff --git a/fdbclient/include/fdbclient/DataDistributorClientInterface.h b/fdbclient/include/fdbclient/DataDistributorClientInterface.h deleted file mode 100644 index 4b5beca8fd..0000000000 --- a/fdbclient/include/fdbclient/DataDistributorClientInterface.h +++ /dev/null @@ -1,326 +0,0 @@ -/* - * DataDistributorClientInterface.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_ClusterInterface_H -#define FDBCLIENT_ClusterInterface_H -#pragma once - -#include "fdbclient/FDBTypes.h" -#include "fdbrpc/FailureMonitor.h" -#include "fdbclient/Status.h" -#include "fdbclient/CommitProxyInterface.h" -#include "fdbclient/ClientWorkerInterface.h" -#include "fdbclient/ClientVersion.h" - -struct DataDistributorClientInterface { - constexpr static FileIdentifier file_identifier = 15988863; - RequestStream openDatabase; - RequestStream failureMonitoring; - RequestStream databaseStatus; - RequestStream> ping; - RequestStream getClientWorkers; - RequestStream forceRecovery; - RequestStream moveShard; - RequestStream repairSystemData; - RequestStream splitShard; - - bool operator==(ClusterInterface const& r) const { return id() == r.id(); } - bool operator!=(ClusterInterface const& r) const { return id() != r.id(); } - UID id() const { return openDatabase.getEndpoint().token; } - NetworkAddress address() const { return openDatabase.getEndpoint().getPrimaryAddress(); } - - bool hasMessage() const { - return openDatabase.getFuture().isReady() || failureMonitoring.getFuture().isReady() || - databaseStatus.getFuture().isReady() || ping.getFuture().isReady() || - getClientWorkers.getFuture().isReady() || forceRecovery.getFuture().isReady() || - moveShard.getFuture().isReady() || repairSystemData.getFuture().isReady() || - splitShard.getFuture().isReady(); - } - - void initEndpoints() { - openDatabase.getEndpoint(TaskPriority::ClusterController); - failureMonitoring.getEndpoint(TaskPriority::FailureMonitor); - databaseStatus.getEndpoint(TaskPriority::ClusterController); - ping.getEndpoint(TaskPriority::ClusterController); - getClientWorkers.getEndpoint(TaskPriority::ClusterController); - forceRecovery.getEndpoint(TaskPriority::ClusterController); - moveShard.getEndpoint(TaskPriority::ClusterController); - repairSystemData.getEndpoint(TaskPriority::ClusterController); - splitShard.getEndpoint(TaskPriority::ClusterController); - } - - template - void serialize(Ar& ar) { - serializer(ar, - openDatabase, - failureMonitoring, - databaseStatus, - ping, - getClientWorkers, - forceRecovery, - moveShard, - repairSystemData, - splitShard); - } -}; - -struct ClusterControllerClientInterface { - constexpr static FileIdentifier file_identifier = 14997695; - ClusterInterface clientInterface; - - bool operator==(ClusterControllerClientInterface const& r) const { - return clientInterface.id() == r.clientInterface.id(); - } - bool operator!=(ClusterControllerClientInterface const& r) const { - return clientInterface.id() != r.clientInterface.id(); - } - - template - void serialize(Ar& ar) { - serializer(ar, clientInterface); - } -}; - -template -struct ItemWithExamples { - T item; - int count; - std::vector> examples; - - ItemWithExamples() : item{}, count(0) {} - ItemWithExamples(T const& item, int count, std::vector> const& examples) - : item(item), count(count), examples(examples) {} - - template - void serialize(Ar& ar) { - serializer(ar, item, count, examples); - } -}; - -struct OpenDatabaseRequest { - constexpr static FileIdentifier file_identifier = 2799502; - // Sent by the native API to the cluster controller to open a database and track client - // info changes. Returns immediately if the current client info id is different from - // knownClientInfoID; otherwise returns when it next changes (or perhaps after a long interval) - - int clientCount; - std::vector> issues; - std::vector>> supportedVersions; - std::vector> maxProtocolSupported; - - UID knownClientInfoID; - ReplyPromise reply; - - template - void serialize(Ar& ar) { - if constexpr (!is_fb_function) { - ASSERT(ar.protocolVersion().hasOpenDatabase()); - } - serializer(ar, clientCount, issues, supportedVersions, maxProtocolSupported, knownClientInfoID, reply); - } -}; - -struct SystemFailureStatus { - constexpr static FileIdentifier file_identifier = 3194108; - NetworkAddressList addresses; - FailureStatus status; - - SystemFailureStatus() {} - SystemFailureStatus(NetworkAddressList const& a, FailureStatus const& s) : addresses(a), status(s) {} - - template - void serialize(Ar& ar) { - serializer(ar, addresses, status); - } -}; - -struct FailureMonitoringReply { - constexpr static FileIdentifier file_identifier = 6820325; - VectorRef changes; - Version failureInformationVersion; - bool allOthersFailed; // If true, changes are relative to all servers being failed, otherwise to the version given - // in the request - int clientRequestIntervalMS, // after this many milliseconds, send another request - considerServerFailedTimeoutMS; // after this many additional milliseconds, consider the ClusterController itself - // to be failed - Arena arena; - - template - void serialize(Ar& ar) { - serializer(ar, - changes, - failureInformationVersion, - allOthersFailed, - clientRequestIntervalMS, - considerServerFailedTimeoutMS, - arena); - } -}; - -struct FailureMonitoringRequest { - // Sent by all participants to the cluster controller reply.clientRequestIntervalMS - // ms after receiving the previous reply. - // Provides the controller the self-diagnosed status of the sender, and also - // requests the status of other systems. Failure to timely send one of these implies - // a failed status. - // If !senderStatus.present(), the sender wants to receive the latest failure information - // but doesn't want to be monitored. - // The failureInformationVersion returned in reply should be passed back to the - // next request to facilitate delta compression of the failure information. - - constexpr static FileIdentifier file_identifier = 5867851; - Optional senderStatus; - Version failureInformationVersion; - NetworkAddressList addresses; - ReplyPromise reply; - - template - void serialize(Ar& ar) { - serializer(ar, senderStatus, failureInformationVersion, addresses, reply); - } -}; - -struct StatusReply { - constexpr static FileIdentifier file_identifier = 9980504; - StatusObject statusObj; - std::string statusStr; - - StatusReply() {} - explicit StatusReply(StatusObject obj) - : statusObj(obj), statusStr(json_spirit::write_string(json_spirit::mValue(obj))) {} - explicit StatusReply(std::string&& text) : statusStr(text) {} - - template - void serialize(Ar& ar) { - serializer(ar, statusStr); - if (ar.isDeserializing) { - json_spirit::mValue mv; - if (g_network->isSimulated()) { - mv = readJSONStrictly(statusStr); - } else { - // In non-simulation allow errors because some status data is better than no status data - json_spirit::read_string(statusStr, mv); - } - statusObj = std::move(mv.get_obj()); - } - } -}; - -struct StatusRequest { - constexpr static FileIdentifier file_identifier = 14419140; - ReplyPromise reply; - - template - void serialize(Ar& ar) { - serializer(ar, reply); - } -}; - -struct GetClientWorkersRequest { - constexpr static FileIdentifier file_identifier = 10771791; - ReplyPromise> reply; - - GetClientWorkersRequest() {} - - template - void serialize(Ar& ar) { - serializer(ar, reply); - } -}; - -struct ForceRecoveryRequest { - constexpr static FileIdentifier file_identifier = 14821350; - Key dcId; - ReplyPromise reply; - - ForceRecoveryRequest() {} - explicit ForceRecoveryRequest(Key dcId) : dcId(dcId) {} - - template - void serialize(Ar& ar) { - serializer(ar, dcId, reply); - } -}; - -// Request to move a keyrange (shard) to a new team represented as addresses. -struct MoveShardRequest { - constexpr static FileIdentifier file_identifier = 2799592; - - KeyRange shard; - std::vector addresses; - ReplyPromise reply; - - MoveShardRequest() {} - MoveShardRequest(KeyRange shard, std::vector addresses) - : shard{ std::move(shard) }, addresses{ std::move(addresses) } {} - - template - void serialize(Ar& ar) { - serializer(ar, shard, addresses, reply); - } -}; - -// Request to trigger a master recovery, and during the following recovery, the system metadata will be -// reconstructed from TLogs, and written to a new SS team. -// This is used when metadata on SSes are lost or corrupted. -struct RepairSystemDataRequest { - constexpr static FileIdentifier file_identifier = 2799593; - - ReplyPromise reply; - - RepairSystemDataRequest() {} - - template - void serialize(Ar& ar) { - serializer(ar, reply); - } -}; - -// Returns the actual shards generated by the SplitShardRequest. -struct SplitShardReply { - constexpr static FileIdentifier file_identifier = 1384440; - std::vector shards; - - SplitShardReply() {} - explicit SplitShardReply(std::vector shards) : shards{ std::move(shards) } {} - - template - void serialize(Ar& ar) { - serializer(ar, shards); - } -}; - -// Split keyrange [shard.begin, shard.end) into num shards. -// Split points are chosen as the arithmetically equal division points of the given range. -struct SplitShardRequest { - constexpr static FileIdentifier file_identifier = 1384443; - KeyRange shard; - int num; - ReplyPromise reply; - - SplitShardRequest() : num(0) {} - SplitShardRequest(KeyRange shard, int num) : shard{ std::move(shard) }, num(num) {} - - template - void serialize(Ar& ar) { - serializer(ar, shard, num, reply); - } -}; -#endif diff --git a/fdbclient/include/fdbclient/ManagementAPI.actor.h b/fdbclient/include/fdbclient/ManagementAPI.actor.h index d841dacca6..8b8877cb5e 100644 --- a/fdbclient/include/fdbclient/ManagementAPI.actor.h +++ b/fdbclient/include/fdbclient/ManagementAPI.actor.h @@ -137,6 +137,7 @@ ACTOR Future setDDMode(Database cx, int mode); ACTOR Future forceRecovery(Reference clusterFile, Standalone dcId); +// Start an audit on range of the specific type. ACTOR Future auditStorage(Reference clusterFile, KeyRange range, AuditType type); ACTOR Future printHealthyZone(Database cx); diff --git a/fdbclient/include/fdbclient/StorageServerInterface.h b/fdbclient/include/fdbclient/StorageServerInterface.h index b4b989190d..011aebab18 100644 --- a/fdbclient/include/fdbclient/StorageServerInterface.h +++ b/fdbclient/include/fdbclient/StorageServerInterface.h @@ -1146,4 +1146,5 @@ struct StorageQueuingMetricsRequest { serializer(ar, reply); } }; + #endif diff --git a/fdbserver/ClusterController.actor.cpp b/fdbserver/ClusterController.actor.cpp index 5d1911c412..c4470fb160 100644 --- a/fdbserver/ClusterController.actor.cpp +++ b/fdbserver/ClusterController.actor.cpp @@ -1944,7 +1944,7 @@ ACTOR Future handleForcedRecoveries(ClusterControllerData* self, ClusterCo } ACTOR Future triggerAuditStorage(ClusterControllerData* self, TriggerAuditRequest req) { - TraceEvent(SevInfo, "TriggerAuditStorageBegin", self->id).detail("Range", req.range).detail("AuditType", req.type); + TraceEvent(SevInfo, "CCTriggerAuditStorageBegin", self->id).detail("Range", req.range).detail("AuditType", req.type); try { while (self->db.serverInfo->get().recoveryState < RecoveryState::ACCEPTING_COMMITS || !self->db.serverInfo->get().distributor.present()) { @@ -1953,19 +1953,15 @@ ACTOR Future triggerAuditStorage(ClusterControllerData* self, TriggerAudit TriggerAuditRequest fReq(req.getType(), req.range); state UID auditId = wait(self->db.serverInfo->get().distributor.get().triggerAudit.getReply(fReq)); - TraceEvent(SevDebug, "TriggerAuditStorageResult", self->id) + TraceEvent(SevDebug, "CCTriggerAuditStorageEnd", self->id) .detail("AuditID", auditId) .detail("Range", req.range) .detail("AuditType", req.type); if (!req.reply.isSet()) { - TraceEvent(SevDebug, "TriggerAuditStorageReply", self->id) - .detail("AuditID", auditId) - .detail("Range", req.range) - .detail("AuditType", req.type); req.reply.send(auditId); } } catch (Error& e) { - TraceEvent(SevDebug, "TriggerAuditStorageError", self->id) + TraceEvent(SevDebug, "CCTriggerAuditStorageError", self->id) .errorUnsuppressed(e) .detail("AuditID", auditId) .detail("Range", req.range) diff --git a/fdbserver/DDTxnProcessor.actor.cpp b/fdbserver/DDTxnProcessor.actor.cpp index 104a32451b..fea569964c 100644 --- a/fdbserver/DDTxnProcessor.actor.cpp +++ b/fdbserver/DDTxnProcessor.actor.cpp @@ -91,10 +91,10 @@ class DDTxnProcessorImpl { return IDDTxnProcessor::SourceServers{ std::vector(servers.begin(), servers.end()), completeSources }; } - ACTOR static Future> getSourceServerInterfacesForRange( + ACTOR static Future> getSourceServerInterfacesForRange( Database cx, KeyRangeRef range) { - state std::vector res; + state std::vector res; state Transaction tr(cx); tr.setOption(FDBTransactionOptions::PRIORITY_SYSTEM_IMMEDIATE); tr.setOption(FDBTransactionOptions::ACCESS_SYSTEM_KEYS); @@ -119,7 +119,7 @@ class DDTxnProcessorImpl { UID srcId, destId; decodeKeyServersValue(UIDtoTagMap, shards[i].value, src, dest, srcId, destId); - state IDDTxnProcessor::StorageServersForRange current( + state IDDTxnProcessor::DDRangeLocations current( KeyRangeRef(shards[i].key, shards[i + 1].key)); state int j = 0; for (j = 0; j < src.size(); ++j) { @@ -456,7 +456,7 @@ Future DDTxnProcessor::getSourceServersForRange( return DDTxnProcessorImpl::getSourceServersForRange(cx, range); } -Future> DDTxnProcessor::getSourceServerInterfacesForRange( +Future> DDTxnProcessor::getSourceServerInterfacesForRange( const KeyRangeRef range) { return DDTxnProcessorImpl::getSourceServerInterfacesForRange(cx, range); } diff --git a/fdbserver/DataDistribution.actor.cpp b/fdbserver/DataDistribution.actor.cpp index 76197d1355..bec89f2d90 100644 --- a/fdbserver/DataDistribution.actor.cpp +++ b/fdbserver/DataDistribution.actor.cpp @@ -1365,7 +1365,7 @@ ACTOR Future scheduleAuditForRange(Reference self, } try { - state std::vector rangeLocations = + state std::vector rangeLocations = wait(self->txnProcessor->getSourceServerInterfacesForRange(currentRange)); state int i = 0; diff --git a/fdbserver/include/fdbserver/DDTxnProcessor.h b/fdbserver/include/fdbserver/DDTxnProcessor.h index 24fc34e0ac..357c3aab64 100644 --- a/fdbserver/include/fdbserver/DDTxnProcessor.h +++ b/fdbserver/include/fdbserver/DDTxnProcessor.h @@ -37,9 +37,9 @@ public: std::vector srcServers, completeSources; // the same as RelocateData.src, RelocateData.completeSources; }; - struct StorageServersForRange { - StorageServersForRange() = default; - StorageServersForRange(KeyRangeRef range) : range(range) {} + struct DDRangeLocations { + DDRangeLocations() = default; + DDRangeLocations(KeyRangeRef range) : range(range) {} // A map of dcId : list of servers std::map> servers; @@ -49,7 +49,7 @@ public: // get the source server list and complete source server list for range virtual Future getSourceServersForRange(const KeyRangeRef range) = 0; - virtual Future> getSourceServerInterfacesForRange( + virtual Future> getSourceServerInterfacesForRange( const KeyRangeRef range) = 0; // get the storage server list and Process class @@ -89,7 +89,7 @@ public: Future getSourceServersForRange(const KeyRangeRef range) override; - Future> getSourceServerInterfacesForRange( + Future> getSourceServerInterfacesForRange( const KeyRangeRef range) override; // Call NativeAPI implementation directly From 092aee8bf2c6a0d10b6b626b452b213afeffd323 Mon Sep 17 00:00:00 2001 From: He Liu Date: Tue, 13 Sep 2022 13:25:33 -0700 Subject: [PATCH 023/210] Introduced `DDAuditStorage` to encapsulate DDAudit. --- fdbserver/DataDistribution.actor.cpp | 85 ++++++++++++++++------------ 1 file changed, 48 insertions(+), 37 deletions(-) diff --git a/fdbserver/DataDistribution.actor.cpp b/fdbserver/DataDistribution.actor.cpp index bec89f2d90..2949db1943 100644 --- a/fdbserver/DataDistribution.actor.cpp +++ b/fdbserver/DataDistribution.actor.cpp @@ -53,6 +53,19 @@ #include "flow/actorcompiler.h" // This must be the last #include. +namespace { +struct DDAuditStorage { + DDAuditStorage(UID id, AuditType type) + : id(id), type(type), auditMap(AuditPhase::Invalid, allKeys.end), actors(true) {} + + const UID id; + const AuditType type; + KeyRangeMap auditMap; + ActorCollection actors; +}; + +} // namespace + void DataMove::validateShard(const DDShardInfo& shard, KeyRangeRef range, int priority) { if (!valid) { if (shard.hasDest && shard.destId != anonymousShardId) { @@ -1293,33 +1306,30 @@ ACTOR Future ddGetMetrics(GetDataDistributorMetricsRequest req, ACTOR Future auditStorage(Reference self, TriggerAuditRequest req); ACTOR Future scheduleAuditForRange(Reference self, - std::shared_ptr actors, - std::shared_ptr> auditMap, - UID auditId, - KeyRange range, - AuditType type); -ACTOR Future doAuditStorage(Reference self, - std::shared_ptr actors, - std::shared_ptr> auditMap, - StorageServerInterface ssi, - AuditStorageRequest req); + std::shared_ptr audit, + KeyRange range); +ACTOR Future doAuditOnStorageServer(Reference self, + std::shared_ptr audit, + StorageServerInterface ssi, + AuditStorageRequest req); ACTOR Future auditStorage(Reference self, TriggerAuditRequest req) { // TODO(heliu): Load running audit, and create one if no audit is running. - state std::shared_ptr> auditMap = - std::make_shared>(AuditPhase::Invalid, allKeys.end); - state std::shared_ptr actors = std::make_shared(true); state UID auditId = deterministicRandom()->randomUniqueID(); + state std::shared_ptr audit = std::make_shared(auditId, req.getType()); + // state std::shared_ptr> auditMap = + // std::make_shared>(AuditPhase::Invalid, allKeys.end); + // state std::shared_ptr actors = std::make_shared(true); TraceEvent(SevDebug, "DDAuditStorageBegin", auditId).detail("Range", req.range).detail("AuditType", req.type); - actors->add(scheduleAuditForRange(self, actors, auditMap, auditId, req.range, req.getType())); + audit->actors.add(scheduleAuditForRange(self, audit, req.range)); if (req.async && !req.reply.isSet()) { req.reply.send(auditId); } try { - wait(actors->getResult()); + wait(audit->actors.getResult()); TraceEvent(SevDebug, "DDAuditStorageEnd", auditId).detail("Range", req.range).detail("AuditType", req.type); // TODO(heliu): Set the audit result, and clear auditId. if (!req.async && !req.reply.isSet()) { @@ -1339,12 +1349,11 @@ ACTOR Future auditStorage(Reference self, TriggerAuditReq } ACTOR Future scheduleAuditForRange(Reference self, - std::shared_ptr actors, - std::shared_ptr> auditMap, - UID auditId, - KeyRange range, - AuditType type) { - TraceEvent(SevDebug, "DDScheduleAuditForRangeBegin", auditId).detail("Range", range).detail("AuditType", type); + std::shared_ptr audit, + KeyRange range) { + TraceEvent(SevDebug, "DDScheduleAuditForRangeBegin", audit->id) + .detail("Range", range) + .detail("AuditType", audit->type); // TODO(heliu): Load the audit map for `range`. state Key begin = range.begin; state KeyRange currentRange = range; @@ -1353,7 +1362,7 @@ ACTOR Future scheduleAuditForRange(Reference self, currentRange = KeyRangeRef(begin, range.end); // Find the first keyrange that hasn't been validated. - auto f = auditMap->intersectingRanges(currentRange); + auto f = audit->auditMap.intersectingRanges(currentRange); for (auto it = f.begin(); it != f.end(); ++it) { if (it->value() != AuditPhase::Invalid && it->value() != AuditPhase::Failed) { begin = it->range().end; @@ -1370,8 +1379,8 @@ ACTOR Future scheduleAuditForRange(Reference self, state int i = 0; for (i = 0; i < rangeLocations.size(); ++i) { - AuditStorageRequest req(auditId, rangeLocations[i].range, type); - if (type == AuditType::ValidateHA && rangeLocations[i].servers.size() >= 2) { + AuditStorageRequest req(audit->id, rangeLocations[i].range, audit->type); + if (audit->type == AuditType::ValidateHA && rangeLocations[i].servers.size() >= 2) { auto it = rangeLocations[i].servers.begin(); const int idx = deterministicRandom()->randomInt(0, it->second.size()); StorageServerInterface& targetServer = it->second[idx]; @@ -1380,13 +1389,15 @@ ACTOR Future scheduleAuditForRange(Reference self, const int idx = deterministicRandom()->randomInt(0, it->second.size()); req.targetServers.push_back(it->second[idx].id()); } - actors->add(doAuditStorage(self, actors, auditMap, targetServer, req)); + audit->actors.add(doAuditOnStorageServer(self, audit, targetServer, req)); } begin = rangeLocations[i].range.end; wait(delay(0.01)); } } catch (Error& e) { - TraceEvent(SevWarnAlways, "DDScheduleAuditRangeError").errorUnsuppressed(e).detail("Range", range); + TraceEvent(SevWarnAlways, "DDScheduleAuditRangeError", audit->id) + .errorUnsuppressed(e) + .detail("Range", range); if (e.code() == error_code_actor_cancelled) { throw e; } @@ -1396,19 +1407,18 @@ ACTOR Future scheduleAuditForRange(Reference self, return Void(); } -ACTOR Future doAuditStorage(Reference self, - std::shared_ptr actors, - std::shared_ptr> auditMap, - StorageServerInterface ssi, - AuditStorageRequest req) { - TraceEvent(SevDebug, "DDAuditStorageBegin", req.id) +ACTOR Future doAuditOnStorageServer(Reference self, + std::shared_ptr audit, + StorageServerInterface ssi, + AuditStorageRequest req) { + TraceEvent(SevDebug, "DDDoAuditOnStorageServerBegin", req.id) .detail("Range", req.range) .detail("AuditType", req.type) .detail("StorageServer", ssi.toString()) .detail("TargetServers", describe(req.targetServers)); try { - auditMap->insert(req.range, AuditPhase::Running); + audit->auditMap.insert(req.range, AuditPhase::Running); ErrorOr vResult = wait(ssi.auditStorage.getReplyUnlessFailedFor(req, 2, 0)); if (vResult.isError()) { throw vResult.getError(); @@ -1420,13 +1430,14 @@ ACTOR Future doAuditStorage(Reference self, e.detail("ErrorMessage", vResult.get().error); } } catch (Error& e) { - TraceEvent(SevWarn, "DDValidateStorageError", req.id) + TraceEvent(SevWarn, "DDDoAuditOnStorageServerError", req.id) .errorUnsuppressed(e) .detail("Range", req.range) - .detail("StorageServer", ssi.toString()); + .detail("StorageServer", ssi.toString()) + .detail("TargetServers", describe(req.targetServers)); if (e.code() != error_code_actor_cancelled) { - auditMap->insert(req.range, AuditPhase::Failed); - actors->add(scheduleAuditForRange(self, actors, auditMap, req.id, req.range, req.getType())); + audit->auditMap.insert(req.range, AuditPhase::Failed); + audit->actors.add(scheduleAuditForRange(self, audit, req.range)); } } From 18cdef43be3edf981bbf70732d89e9b30f59b089 Mon Sep 17 00:00:00 2001 From: He Liu Date: Tue, 13 Sep 2022 13:52:08 -0700 Subject: [PATCH 024/210] Cleanup. --- fdbserver/ClusterController.actor.cpp | 4 +++- fdbserver/DDTxnProcessor.actor.cpp | 3 +-- fdbserver/DataDistribution.actor.cpp | 3 --- 3 files changed, 4 insertions(+), 6 deletions(-) diff --git a/fdbserver/ClusterController.actor.cpp b/fdbserver/ClusterController.actor.cpp index c4470fb160..22468cfbfb 100644 --- a/fdbserver/ClusterController.actor.cpp +++ b/fdbserver/ClusterController.actor.cpp @@ -1944,7 +1944,9 @@ ACTOR Future handleForcedRecoveries(ClusterControllerData* self, ClusterCo } ACTOR Future triggerAuditStorage(ClusterControllerData* self, TriggerAuditRequest req) { - TraceEvent(SevInfo, "CCTriggerAuditStorageBegin", self->id).detail("Range", req.range).detail("AuditType", req.type); + TraceEvent(SevInfo, "CCTriggerAuditStorageBegin", self->id) + .detail("Range", req.range) + .detail("AuditType", req.type); try { while (self->db.serverInfo->get().recoveryState < RecoveryState::ACCEPTING_COMMITS || !self->db.serverInfo->get().distributor.present()) { diff --git a/fdbserver/DDTxnProcessor.actor.cpp b/fdbserver/DDTxnProcessor.actor.cpp index fea569964c..f2f9080de0 100644 --- a/fdbserver/DDTxnProcessor.actor.cpp +++ b/fdbserver/DDTxnProcessor.actor.cpp @@ -119,8 +119,7 @@ class DDTxnProcessorImpl { UID srcId, destId; decodeKeyServersValue(UIDtoTagMap, shards[i].value, src, dest, srcId, destId); - state IDDTxnProcessor::DDRangeLocations current( - KeyRangeRef(shards[i].key, shards[i + 1].key)); + state IDDTxnProcessor::DDRangeLocations current(KeyRangeRef(shards[i].key, shards[i + 1].key)); state int j = 0; for (j = 0; j < src.size(); ++j) { Optional serverListValue = wait(tr.get(serverListKeyFor(src[j]))); diff --git a/fdbserver/DataDistribution.actor.cpp b/fdbserver/DataDistribution.actor.cpp index 2949db1943..8c8db566e5 100644 --- a/fdbserver/DataDistribution.actor.cpp +++ b/fdbserver/DataDistribution.actor.cpp @@ -1317,9 +1317,6 @@ ACTOR Future auditStorage(Reference self, TriggerAuditReq // TODO(heliu): Load running audit, and create one if no audit is running. state UID auditId = deterministicRandom()->randomUniqueID(); state std::shared_ptr audit = std::make_shared(auditId, req.getType()); - // state std::shared_ptr> auditMap = - // std::make_shared>(AuditPhase::Invalid, allKeys.end); - // state std::shared_ptr actors = std::make_shared(true); TraceEvent(SevDebug, "DDAuditStorageBegin", auditId).detail("Range", req.range).detail("AuditType", req.type); audit->actors.add(scheduleAuditForRange(self, audit, req.range)); From 28e5a70dbe615ed2cf9c73017130a718149e66d7 Mon Sep 17 00:00:00 2001 From: He Liu Date: Wed, 14 Sep 2022 10:53:54 -0700 Subject: [PATCH 025/210] Clean up SS validate storage. --- fdbserver/DataDistribution.actor.cpp | 1 + fdbserver/storageserver.actor.cpp | 81 +++++++++---------- fdbserver/workloads/ValidateStorage.actor.cpp | 3 - tests/fast/ValidateStorage.toml | 3 +- 4 files changed, 39 insertions(+), 49 deletions(-) diff --git a/fdbserver/DataDistribution.actor.cpp b/fdbserver/DataDistribution.actor.cpp index 46deee93e6..e7e134d775 100644 --- a/fdbserver/DataDistribution.actor.cpp +++ b/fdbserver/DataDistribution.actor.cpp @@ -592,6 +592,7 @@ ACTOR Future dataDistribution(Reference self, state bool ddIsTenantAware = SERVER_KNOBS->DD_TENANT_AWARENESS_ENABLED; loop { trackerCancelled = false; + self->initialized = Promise(); // Stored outside of data distribution tracker to avoid slow tasks // when tracker is cancelled diff --git a/fdbserver/storageserver.actor.cpp b/fdbserver/storageserver.actor.cpp index de0b8d568d..89e0b9476d 100644 --- a/fdbserver/storageserver.actor.cpp +++ b/fdbserver/storageserver.actor.cpp @@ -4129,69 +4129,65 @@ ACTOR Future validateRangeAgainstServer(StorageServer* data, KeyRange range, Version version, StorageServerInterface remoteServer) { - TraceEvent(SevDebug, "ServeValidateRangeAgainstServerBegin", data->thisServerID) + TraceEvent(SevDebug, "ValidateRangeAgainstServerBegin", data->thisServerID) .detail("Range", range) .detail("Version", version) .detail("RemoteServer", remoteServer.toString()); state int validatedKeys = 0; + state std::string error; loop { try { state int limit = 1e4; state int limitBytes = CLIENT_KNOBS->REPLY_BYTE_LIMIT; state GetKeyValuesRequest req; - state GetKeyValuesRequest localReq; req.begin = firstGreaterOrEqual(range.begin); req.end = firstGreaterOrEqual(range.end); req.limit = limit; req.limitBytes = limitBytes; req.version = version; req.tags = TagSet(); + state Future> remoteKeyValueFuture = + remoteServer.getKeyValues.getReplyUnlessFailedFor(req, 2, 0); + state ErrorOr remoteResult = wait(remoteKeyValueFuture); + + state GetKeyValuesRequest localReq; localReq.begin = firstGreaterOrEqual(range.begin); localReq.end = firstGreaterOrEqual(range.end); localReq.limit = limit; localReq.limitBytes = limitBytes; localReq.version = version; localReq.tags = TagSet(); - - // Try getting the entries in the specified range - state Future> remoteKeyValueFuture = - remoteServer.getKeyValues.getReplyUnlessFailedFor(req, 2, 0); data->actors.add(getKeyValuesQ(data, localReq)); - state ErrorOr remoteResult = wait(remoteKeyValueFuture); - try { - state GetKeyValuesReply local = wait(localReq.reply.getFuture()); - if (local.error.present()) { - throw local.error.get(); - } - TraceEvent(SevDebug, "ValidateRangeGetLocalKeyValuesResult", data->thisServerID) - .detail("Range", range) - .detail("RemoteResultSize", local.data.size()); - } catch (Error& e) { - TraceEvent(SevDebug, "ValidateRangeGetLocalKeyValuesError", data->thisServerID) - .errorUnsuppressed(e) + GetKeyValuesReply local = wait(localReq.reply.getFuture()); + if (local.error.present()) { + TraceEvent(SevWarn, "ValidateRangeGetLocalKeyValuesError", data->thisServerID) + .errorUnsuppressed(local.error.get()) .detail("Range", range); - throw e; + throw local.error.get(); } - Key lastKey = range.begin; - state std::string error; - // Compare the results with other storage servers + Key lastKey = range.begin; + if (remoteResult.isError()) { - TraceEvent(SevDebug, "ValidateRangeGetRemoteKeyValuesError", data->thisServerID) + TraceEvent(SevWarn, "ValidateRangeGetRemoteKeyValuesError", data->thisServerID) .errorUnsuppressed(remoteResult.getError()) .detail("Range", range); throw remoteResult.getError(); } - state GetKeyValuesReply remote = remoteResult.get(); + GetKeyValuesReply remote = remoteResult.get(); if (remote.error.present()) { + TraceEvent(SevWarn, "ValidateRangeGetRemoteKeyValuesError", data->thisServerID) + .errorUnsuppressed(remote.error.get()) + .detail("Range", range); throw remote.error.get(); } - TraceEvent(SevDebug, "ValidateRangeGetRemoteKeyValuesResult", data->thisServerID) + + TraceEvent(SevVerbose, "ValidateRangeGetRemoteKeyValuesResult", data->thisServerID) .detail("Range", range) .detail("RemoteResultSize", remote.data.size()); - // Loop indeces + const int end = std::min(local.data.size(), remote.data.size()); int i = 0; for (; i < end; ++i) { @@ -4220,7 +4216,7 @@ ACTOR Future validateRangeAgainstServer(StorageServer* data, remoteServer.uniqueID.first(), Traceable::toString(remoteKV.value)); } else { - TraceEvent(SevDebug, "ValidatedKey", data->thisServerID).detail("Key", localKV.key); + TraceEvent(SevVerbose, "ValidatedKey", data->thisServerID).detail("Key", localKV.key); ++validatedKeys; } @@ -4234,10 +4230,10 @@ ACTOR Future validateRangeAgainstServer(StorageServer* data, if (!local.more && !remote.more && local.data.size() == remote.data.size()) { break; } else if (i >= local.data.size() && !local.more && i < remote.data.size()) { - error = format("Missing key(s) form local server (%lld), next remote server(%016llx) key: %s", + error = format("Missing key(s) form local server (%lld), next key: %s, remote server(%016llx) ", data->thisServerID.first(), - remoteServer.uniqueID.first(), - Traceable::toString(remote.data[i].key)); + Traceable::toString(remote.data[i].key), + remoteServer.uniqueID.first()); break; } else if (i >= remote.data.size() && !remote.more && i < local.data.size()) { error = format("Missing key(s) form remote server (%lld), next local server(%016llx) key: %s", @@ -4266,7 +4262,7 @@ ACTOR Future validateRangeAgainstServer(StorageServer* data, .detail("RemoteServer", remoteServer.toString()); } - TraceEvent(SevDebug, "ServeValidateRangeAgainstServerEnd", data->thisServerID) + TraceEvent(SevDebug, "ValidateRangeAgainstServerEnd", data->thisServerID) .detail("Range", range) .detail("Version", version) .detail("ValidatedKeys", validatedKeys) @@ -4347,10 +4343,10 @@ ACTOR Future validateRangeShard(StorageServer* data, KeyRange range, std:: return Void(); } -ACTOR Future validateRangeShardAgainstServers(StorageServer* data, KeyRange range, std::vector candidates) { - TraceEvent(SevDebug, "ServeValidateRangeShardAgainstServersBegin", data->thisServerID) +ACTOR Future validateRangeAgainstServers(StorageServer* data, KeyRange range, std::vector targetServers) { + TraceEvent(SevDebug, "ValidateRangeAgainstServersBegin", data->thisServerID) .detail("Range", range) - .detail("TargetServers", describe(candidates)); + .detail("TargetServers", describe(targetServers)); state Version version; state std::vector> serverListValues; @@ -4361,7 +4357,7 @@ ACTOR Future validateRangeShardAgainstServers(StorageServer* data, KeyRang loop { try { std::vector>> serverListEntries; - for (const UID& id : candidates) { + for (const UID& id : targetServers) { if (id != data->thisServerID) { serverListEntries.push_back(tr.get(serverListKeyFor(id))); } @@ -4379,17 +4375,14 @@ ACTOR Future validateRangeShardAgainstServers(StorageServer* data, KeyRang std::vector ssis; for (const auto& v : serverListValues) { + if (!v.present()) { + TraceEvent(SevWarn, "ValidateRangeRemoteServerNotFound", data->thisServerID).detail("Range", range); + throw audit_storage_failed(); + } ssis.push_back(decodeServerListValue(v.get())); } - if (!ssis.empty()) { - wait(validateRangeAgainstServer(data, range, version, ssis[0])); - } else { - TraceEvent(SevWarn, "ServeValidateRangeShardRemoteNotFound", data->thisServerID) - .detail("Range", range) - .detail("Servers", describe(candidates)); - throw audit_storage_failed(); - } + wait(validateRangeAgainstServer(data, range, version, ssis[0])); return Void(); } @@ -4437,7 +4430,7 @@ ACTOR Future auditStorageQ(StorageServer* data, AuditStorageRequest req) { } } } else { - fs.push_back(validateRangeShardAgainstServers(data, req.range, req.targetServers)); + fs.push_back(validateRangeAgainstServers(data, req.range, req.targetServers)); } wait(waitForAll(fs)); AuditStorageState res(req.id, req.getType()); diff --git a/fdbserver/workloads/ValidateStorage.actor.cpp b/fdbserver/workloads/ValidateStorage.actor.cpp index 97d44bf470..5cbe8933aa 100644 --- a/fdbserver/workloads/ValidateStorage.actor.cpp +++ b/fdbserver/workloads/ValidateStorage.actor.cpp @@ -68,7 +68,6 @@ struct ValidateStorage : TestWorkload { } ACTOR Future _start(ValidateStorage* self, Database cx) { - // int ignore = wait(setDDMode(cx, 0)); TraceEvent("ValidateStorageTestBegin"); state std::map kvs({ { "TestKeyA"_sr, "TestValueA"_sr }, { "TestKeyB"_sr, "TestValueB"_sr }, @@ -95,7 +94,6 @@ struct ValidateStorage : TestWorkload { } } - // int ignore = wait(setDDMode(cx, 1)); return Void(); } @@ -212,7 +210,6 @@ struct ValidateStorage : TestWorkload { } ACTOR Future writeAndVerify(ValidateStorage* self, Database cx, Key key, Optional value) { - // state Transaction tr(cx); state Reference tr = makeReference(cx); state Version version; loop { diff --git a/tests/fast/ValidateStorage.toml b/tests/fast/ValidateStorage.toml index 9a32b67e2d..fc9ae49ae1 100644 --- a/tests/fast/ValidateStorage.toml +++ b/tests/fast/ValidateStorage.toml @@ -1,5 +1,5 @@ [configuration] -config = 'double' +config = 'triple' storageEngineType = 5 generateFearless = true allowDefaultTenant = false @@ -7,7 +7,6 @@ machineCount = 45 [[knobs]] shard_encode_location_metadata = true -storage_server_shard_aware = true [[test]] testTitle = 'ValidateStorageWorkload' From 0e69b19a6f447b00d2fe7d89925e8b436a300d52 Mon Sep 17 00:00:00 2001 From: He Liu Date: Mon, 19 Sep 2022 11:06:43 -0700 Subject: [PATCH 026/210] Added AuditUtils.actor.h --- fdbclient/SystemData.cpp | 35 +++++++++++++++ fdbclient/include/fdbclient/Audit.h | 1 + .../include/fdbclient/AuditUtils.actor.h | 33 ++++++++++++++ fdbclient/include/fdbclient/SystemData.h | 7 +++ fdbserver/DataDistribution.actor.cpp | 44 ++++++++++--------- 5 files changed, 100 insertions(+), 20 deletions(-) create mode 100644 fdbclient/include/fdbclient/AuditUtils.actor.h diff --git a/fdbclient/SystemData.cpp b/fdbclient/SystemData.cpp index b92c87923e..b799b3eb18 100644 --- a/fdbclient/SystemData.cpp +++ b/fdbclient/SystemData.cpp @@ -289,6 +289,41 @@ const KeyRangeRef writeConflictRangeKeysRange = const KeyRef clusterIdKey = LiteralStringRef("\xff/clusterId"); +const KeyRangeRef auditRange = KeyRangeRef("\xff/audit/"_sr, "\xff/audit0"_sr); +const KeyRef auditPrefix = auditRange.begin; + +const Key auditRangeKey(const AuditType type, const UID& auditId, const KeyRef& key) { + BinaryWriter wr(Unversioned()); + wr.serializeBytes(auditPrefix); + wr << static_cast(type); + wr.serializeBytes("/"_sr); + wr << auditId; + wr.serializeBytes("/"_sr); + wr.serializeBytes(key); + return wr.toValue(); +} + +const Key auditRangePrefix(const AuditType type, const UID& auditId) { + BinaryWriter wr(Unversioned()); + wr.serializeBytes(auditPrefix); + wr << static_cast(type); + wr.serializeBytes("/"_sr); + wr << auditId; + wr.serializeBytes("/"_sr); + return wr.toValue(); +} + +const Value auditStorageStateValue(const AuditStorageState& auditStorageState) { + return ObjectWriter::toValue(auditStorageState, IncludeVersion()); +} + +AuditStorageState decodeAuditStorageState(const ValueRef& value) { + AuditStorageState auditState; + ObjectReader reader(value.begin(), IncludeVersion()); + reader.deserialize(auditState); + return auditState; +} + const KeyRef checkpointPrefix = "\xff/checkpoint/"_sr; const Key checkpointKeyFor(UID checkpointID) { diff --git a/fdbclient/include/fdbclient/Audit.h b/fdbclient/include/fdbclient/Audit.h index 55c4fb368c..f23e33da9e 100644 --- a/fdbclient/include/fdbclient/Audit.h +++ b/fdbclient/include/fdbclient/Audit.h @@ -107,4 +107,5 @@ struct TriggerAuditRequest { bool async; ReplyPromise reply; }; + #endif diff --git a/fdbclient/include/fdbclient/AuditUtils.actor.h b/fdbclient/include/fdbclient/AuditUtils.actor.h new file mode 100644 index 0000000000..c3588ff77b --- /dev/null +++ b/fdbclient/include/fdbclient/AuditUtils.actor.h @@ -0,0 +1,33 @@ +/* + * AuditUtils.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. + */ + +#ifndef FDBCLIENT_AUDITUTILS_ACTOR_H +#define FDBCLIENT_AUDITUTILS_ACTOR_H +#pragma once + +#include "fdbclient/Audit.h" +#include "fdbclient/FDBTypes.h" +#include "fdbrpc/fdbrpc.h" + +#include "flow/actorcompiler.h" // has to be last include + +ACTOR Future persistAuditStorageState(Key key, AuditStorageState auditState); + +#endif diff --git a/fdbclient/include/fdbclient/SystemData.h b/fdbclient/include/fdbclient/SystemData.h index 2c3d8e89a4..4e1055ae02 100644 --- a/fdbclient/include/fdbclient/SystemData.h +++ b/fdbclient/include/fdbclient/SystemData.h @@ -94,6 +94,13 @@ void decodeKeyServersValue(RangeResult result, extern const KeyRef clusterIdKey; +extern const KeyRangeRef auditRange; +extern const KeyRef auditPrefix; +const Key auditRangeKey(const AuditType type, const UID& auditId, const KeyRef& key); +const Key auditRangePrefix(const AuditType type, const UID& auditId); +const Value auditStorageStateValue(const AuditStorageState& auditStorageState); +AuditStorageState decodeAuditStorageState(const ValueRef& value); + // "\xff/checkpoint/[[UID]] := [[CheckpointMetaData]]" extern const KeyRef checkpointPrefix; const Key checkpointKeyFor(UID checkpointID); diff --git a/fdbserver/DataDistribution.actor.cpp b/fdbserver/DataDistribution.actor.cpp index e7e134d775..2a0582708e 100644 --- a/fdbserver/DataDistribution.actor.cpp +++ b/fdbserver/DataDistribution.actor.cpp @@ -53,10 +53,8 @@ #include "fdbserver/DDSharedContext.h" #include "flow/actorcompiler.h" // This must be the last #include. -namespace { -struct DDAuditStorage { - DDAuditStorage(UID id, AuditType type) - : id(id), type(type), auditMap(AuditPhase::Invalid, allKeys.end), actors(true) {} +struct DDAudit { + DDAudit(UID id, AuditType type) : id(id), type(type), auditMap(AuditPhase::Invalid, allKeys.end), actors(true) {} const UID id; const AuditType type; @@ -64,8 +62,6 @@ struct DDAuditStorage { ActorCollection actors; }; -} // namespace - void DataMove::validateShard(const DDShardInfo& shard, KeyRangeRef range, int priority) { if (!valid) { if (shard.hasDest && shard.destId != anonymousShardId) { @@ -291,6 +287,8 @@ public: Promise initialized; + std::unordered_map>> audits; + DataDistributor(Reference const> const& db, UID id, Reference context) : dbInfo(db), context(context), ddId(id), txnProcessor(nullptr), initialDDEventHolder(makeReference("InitialDD")), @@ -1345,37 +1343,43 @@ ACTOR Future ddGetMetrics(GetDataDistributorMetricsRequest req, ACTOR Future auditStorage(Reference self, TriggerAuditRequest req); ACTOR Future scheduleAuditForRange(Reference self, - std::shared_ptr audit, + std::shared_ptr audit, KeyRange range); ACTOR Future doAuditOnStorageServer(Reference self, - std::shared_ptr audit, + std::shared_ptr audit, StorageServerInterface ssi, AuditStorageRequest req); ACTOR Future auditStorage(Reference self, TriggerAuditRequest req) { // TODO(heliu): Load running audit, and create one if no audit is running. - state UID auditId = deterministicRandom()->randomUniqueID(); - state std::shared_ptr audit = std::make_shared(auditId, req.getType()); - TraceEvent(SevDebug, "DDAuditStorageBegin", auditId).detail("Range", req.range).detail("AuditType", req.type); - - audit->actors.add(scheduleAuditForRange(self, audit, req.range)); + state std::shared_ptr audit; + auto it = self->audits.find(req.getType()); + if (it != self->audits.end() && !it->second.empty()) { + audit = it->second.front(); + } else { + const UID auditId = deterministicRandom()->randomUniqueID(); + audit = std::make_shared(auditId, req.getType()); + self->audits[req.getType()].push_back(audit); + audit->actors.add(scheduleAuditForRange(self, audit, req.range)); + TraceEvent(SevDebug, "DDAuditStorageBegin", audit->id).detail("Range", req.range).detail("AuditType", req.type); + } if (req.async && !req.reply.isSet()) { - req.reply.send(auditId); + req.reply.send(audit->id); } try { wait(audit->actors.getResult()); - TraceEvent(SevDebug, "DDAuditStorageEnd", auditId).detail("Range", req.range).detail("AuditType", req.type); + TraceEvent(SevDebug, "DDAuditStorageEnd", audit->id).detail("Range", req.range).detail("AuditType", req.type); // TODO(heliu): Set the audit result, and clear auditId. if (!req.async && !req.reply.isSet()) { - TraceEvent(SevDebug, "DDAuditStorageReply", auditId) + TraceEvent(SevDebug, "DDAuditStorageReply", audit->id) .detail("Range", req.range) .detail("AuditType", req.type); - req.reply.send(auditId); + req.reply.send(audit->id); } } catch (Error& e) { - TraceEvent(SevWarnAlways, "DDAuditStorageOperationError", auditId) + TraceEvent(SevWarnAlways, "DDAuditStorageOperationError", audit->id) .errorUnsuppressed(e) .detail("Range", req.range) .detail("AuditType", req.type); @@ -1385,7 +1389,7 @@ ACTOR Future auditStorage(Reference self, TriggerAuditReq } ACTOR Future scheduleAuditForRange(Reference self, - std::shared_ptr audit, + std::shared_ptr audit, KeyRange range) { TraceEvent(SevDebug, "DDScheduleAuditForRangeBegin", audit->id) .detail("Range", range) @@ -1444,7 +1448,7 @@ ACTOR Future scheduleAuditForRange(Reference self, } ACTOR Future doAuditOnStorageServer(Reference self, - std::shared_ptr audit, + std::shared_ptr audit, StorageServerInterface ssi, AuditStorageRequest req) { TraceEvent(SevDebug, "DDDoAuditOnStorageServerBegin", req.id) From e8e1de16b62ce1b1186c1380e842d2215302d905 Mon Sep 17 00:00:00 2001 From: Dan Lambright Date: Tue, 27 Sep 2022 15:21:57 -0400 Subject: [PATCH 027/210] Use property to control eager deletion of files in /tmp on binding fdb library to java --- .../java/src/main/com/apple/foundationdb/JNIUtil.java | 8 ++++++-- 1 file changed, 6 insertions(+), 2 deletions(-) diff --git a/bindings/java/src/main/com/apple/foundationdb/JNIUtil.java b/bindings/java/src/main/com/apple/foundationdb/JNIUtil.java index cf4e0ade06..bfee1bc034 100644 --- a/bindings/java/src/main/com/apple/foundationdb/JNIUtil.java +++ b/bindings/java/src/main/com/apple/foundationdb/JNIUtil.java @@ -182,15 +182,19 @@ public class JNIUtil { private static OS getRunningOS() { String osname = System.getProperty("os.name").toLowerCase(); String arch = System.getProperty("os.arch"); + Boolean eagerDelete = true; + if (System.getProperty("fdb.bindings.java.eager_delete") != null) { + eagerDelete = false; + } if (!arch.equals("amd64") && !arch.equals("x86_64") && !arch.equals("aarch64") && !arch.equals("ppc64le")) { throw new IllegalStateException("Unknown or unsupported arch: " + arch); } if (osname.startsWith("windows")) { return new OS("windows", arch, /* canDeleteEager */ false); } else if (osname.startsWith("linux")) { - return new OS("linux", arch, /* canDeleteEager */ true); + return new OS("linux", arch, /* canDeleteEager */ eagerDelete); } else if (osname.startsWith("mac") || osname.startsWith("darwin")) { - return new OS("osx", arch, /* canDeleteEager */ true); + return new OS("osx", arch, /* canDeleteEager */ eagerDelete); } else { throw new IllegalStateException("Unknown or unsupported OS: " + osname); } From 85ba47f4c88f6ce575e686cafa6be8c58978b5ff Mon Sep 17 00:00:00 2001 From: Ankita Kejriwal Date: Tue, 30 Aug 2022 16:27:44 -0700 Subject: [PATCH 028/210] Add a workload to test getEstimatedRangeSizeBytes funtionality --- fdbclient/ReadYourWrites.actor.cpp | 2 + .../workloads/GetEstimatedRangeSize.actor.cpp | 144 ++++++++++++++++++ tests/CMakeLists.txt | 1 + tests/fast/GetEstimatedRangeSize.toml | 31 ++++ 4 files changed, 178 insertions(+) create mode 100644 fdbserver/workloads/GetEstimatedRangeSize.actor.cpp create mode 100644 tests/fast/GetEstimatedRangeSize.toml diff --git a/fdbclient/ReadYourWrites.actor.cpp b/fdbclient/ReadYourWrites.actor.cpp index d6437c7403..0fe4c598bf 100644 --- a/fdbclient/ReadYourWrites.actor.cpp +++ b/fdbclient/ReadYourWrites.actor.cpp @@ -2492,6 +2492,8 @@ void ReadYourWritesTransaction::setOptionImpl(FDBTransactionOptions::Option opti validateOptionValueNotPresent(value); options.bypassUnreadable = true; break; + case FDBTransactionOptions::AUTHORIZATION_TOKEN: + tr.setOption(option, value); default: break; } diff --git a/fdbserver/workloads/GetEstimatedRangeSize.actor.cpp b/fdbserver/workloads/GetEstimatedRangeSize.actor.cpp new file mode 100644 index 0000000000..8db7ae0785 --- /dev/null +++ b/fdbserver/workloads/GetEstimatedRangeSize.actor.cpp @@ -0,0 +1,144 @@ +/* + * GetEstimatedRangeSize.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 + +#include "fdbclient/SystemData.h" +#include "flow/Arena.h" +#include "flow/IRandom.h" +#include "flow/Trace.h" +#include "flow/serialize.h" +#include "fdbrpc/simulator.h" +#include "fdbrpc/TokenSign.h" +#include "fdbclient/FDBOptions.g.h" +#include "fdbclient/NativeAPI.actor.h" +#include "fdbclient/SystemData.h" +#include "fdbserver/TesterInterface.actor.h" +#include "fdbserver/workloads/workloads.actor.h" +#include "fdbserver/workloads/BulkSetup.actor.h" + +#include "flow/actorcompiler.h" // This must be the last #include. + +struct Members { + Arena arena; + TenantName tenant; + authz::jwt::TokenRef token; + StringRef signedToken; +}; + +struct GetEstimatedRangeSizeWorkload : TestWorkload, Members { + int actorCount, nodeCount; + double testDuration, transactionsPerSecond, minExpectedTransactionsPerSecond, traceParentProbability; + Key keyPrefix; + + std::vector> clients; + PerfIntCounter transactions, retries, tooOldRetries, commitFailedRetries; + PerfDoubleCounter totalLatency; + + GetEstimatedRangeSizeWorkload(WorkloadContext const& wcx) + : TestWorkload(wcx), transactions("Transactions"), retries("Retries"), tooOldRetries("Retries.too_old"), + commitFailedRetries("Retries.commit_failed"), totalLatency("Latency") { + testDuration = getOption(options, "testDuration"_sr, 10.0); + transactionsPerSecond = getOption(options, "transactionsPerSecond"_sr, 5000.0) / clientCount; + actorCount = getOption(options, "actorsPerClient"_sr, transactionsPerSecond / 5); + nodeCount = getOption(options, "nodeCount"_sr, transactionsPerSecond * clientCount); + keyPrefix = unprintable(getOption(options, "keyPrefix"_sr, LiteralStringRef("")).toString()); + traceParentProbability = getOption(options, "traceParentProbability"_sr, 0.01); + minExpectedTransactionsPerSecond = transactionsPerSecond * getOption(options, "expectedRate"_sr, 0.7); + + ASSERT(g_network->isSimulated()); + auto k = g_simulator.authKeys.begin(); + this->tenant = getOption(options, "tenant"_sr, "DefaultTenant"_sr); + // make it comfortably longer than the timeout of the workload + auto currentTime = uint64_t(lround(g_network->timer())); + this->token.algorithm = authz::Algorithm::ES256; + this->token.issuedAtUnixTime = currentTime; + this->token.expiresAtUnixTime = + currentTime + uint64_t(std::lround(getCheckTimeout())) + uint64_t(std::lround(testDuration)) + 100; + this->token.keyId = k->first; + this->token.notBeforeUnixTime = currentTime - 10; + VectorRef tenants; + tenants.push_back_deep(this->arena, this->tenant); + this->token.tenants = tenants; + // we currently don't support this workload to be run outside of simulation + this->signedToken = authz::jwt::signToken(this->arena, this->token, k->second); + } + + std::string description() const override { return "GetEstimatedRangeSizeWorkload"; } + + Future setup(Database const& cx) override { + cx->defaultTenant = this->tenant; + return bulkSetup(cx, this, nodeCount, Promise()); + } + + Future start(Database const& cx) override { + cx->defaultTenant = this->tenant; + return checkSize(this, cx); + } + + Future check(Database const& cx) override { return true; } + + void getMetrics(std::vector& m) override {} + + StringRef getAuthToken() const { return this->signedToken; } + + void setAuthToken(ReadYourWritesTransaction& tr) { + tr.setOption(FDBTransactionOptions::AUTHORIZATION_TOKEN, this->signedToken); + tr.setOption(FDBTransactionOptions::RAW_ACCESS); + } + + Key keyForIndex(int n) { return key(n); } + Key key(int n) { return doubleToTestKey((double)n / nodeCount, keyPrefix); } + Value value(int n) { return doubleToTestKey(n, keyPrefix); } + int fromValue(const ValueRef& v) { return testKeyToDouble(v, keyPrefix); } + Standalone operator()(int n) { return KeyValueRef(key(n), value((n + 1) % nodeCount)); } + + ACTOR static Future checkSize(GetEstimatedRangeSizeWorkload* self, Database cx) { + // TraceEvent(SevWarnAlways, "AKDebug").detail("Status", "checkSize-1").detail("Tenant", cx->defaultTenant.get()); + int64_t size = wait(getSize(self, cx)); + // TraceEvent(SevWarnAlways, "AKDebug").detail("Status", "checkSize-2").detail("Tenant", cx->defaultTenant.get()); + TraceEvent(SevWarnAlways, "AKGetEstimatedRangeSizeResults") + .detail("Tenant", cx->defaultTenant.get()) + .detail("TenantSize", size); + ASSERT_LT(size, 0); + return Void(); + } + + ACTOR static Future getSize(GetEstimatedRangeSizeWorkload* self, Database cx) { + // TraceEvent(SevWarnAlways, "AKDebug").detail("Status", "getSize-1").detail("Tenant", cx->defaultTenant.get()); + state ReadYourWritesTransaction tr(cx); + loop { + try { + self->setAuthToken(tr); + state int64_t size = wait(tr.getEstimatedRangeSizeBytes(normalKeys)); + // TraceEvent(SevWarnAlways, "AKDebug") + // .detail("Status", "getSize-2") + // .detail("Tenant", cx->defaultTenant.get()); + tr.reset(); + return size; + } catch (Error& e) { + // TraceEvent(SevWarnAlways, "AKDebugError").detail("Status", "getSize-3").detail("Error", e.name()); + wait(tr.onError(e)); + } + } + } +}; + +WorkloadFactory GetEstimatedRangeSizeWorkloadFactory("GetEstimatedRangeSize", true); diff --git a/tests/CMakeLists.txt b/tests/CMakeLists.txt index 697c20ddcb..c0e9c5a062 100644 --- a/tests/CMakeLists.txt +++ b/tests/CMakeLists.txt @@ -164,6 +164,7 @@ if(WITH_PYTHON) add_fdb_test(TEST_FILES fast/MemoryLifetime.toml) add_fdb_test(TEST_FILES fast/MoveKeysCycle.toml) add_fdb_test(TEST_FILES fast/MutationLogReaderCorrectness.toml) + add_fdb_test(TEST_FILES fast/GetEstimatedRangeSize.toml) add_fdb_test(TEST_FILES fast/GetMappedRange.toml) add_fdb_test(TEST_FILES fast/PrivateEndpoints.toml) add_fdb_test(TEST_FILES fast/ProtocolVersion.toml) diff --git a/tests/fast/GetEstimatedRangeSize.toml b/tests/fast/GetEstimatedRangeSize.toml new file mode 100644 index 0000000000..020e62cf71 --- /dev/null +++ b/tests/fast/GetEstimatedRangeSize.toml @@ -0,0 +1,31 @@ +[configuration] +allowDefaultTenant = false +allowDisablingTenants = false + +[[test]] +testTitle = 'TenantCreation' + + [[test.workload]] + testName = 'CreateTenant' + name = 'First' + + [[test.workload]] + testName = 'CreateTenant' + name = 'Second' + +[[test]] +testTitle = 'GetEstimatedRangeSizeTest' + + [[test.workload]] + testName = 'GetEstimatedRangeSize' + tenant = 'First' + transactionsPerSecond = 250.0 + testDuration = 10.0 + expectedRate = 0.80 + + [[test.workload]] + testName = 'GetEstimatedRangeSize' + tenant = 'Second' + transactionsPerSecond = 2500.0 + testDuration = 10.0 + expectedRate = 0.80 From f63934117dfe9b2a934583db590c58b2dd977ca0 Mon Sep 17 00:00:00 2001 From: Ankita Kejriwal Date: Thu, 1 Sep 2022 18:28:37 -0700 Subject: [PATCH 029/210] Make the storage metrics function tenant aware [WIP] This change makes the getEstimatedRangeSizeBytes function tenant aware. Previously, this function would return the size of the requested keyspace even if the tenant in the Transaction or DatabaseContext did not match the tenant corresponding to the keyspace. Also make some improvements to the new workload. --- fdbclient/NativeAPI.actor.cpp | 69 +++++++++++++------ fdbclient/ReadYourWrites.actor.cpp | 9 ++- fdbclient/include/fdbclient/DatabaseContext.h | 20 ++++-- .../fdbclient/StorageServerInterface.h | 18 +++-- fdbserver/include/fdbserver/StorageMetrics.h | 2 +- fdbserver/storageserver.actor.cpp | 21 ++++-- .../workloads/GetEstimatedRangeSize.actor.cpp | 14 +--- 7 files changed, 100 insertions(+), 53 deletions(-) diff --git a/fdbclient/NativeAPI.actor.cpp b/fdbclient/NativeAPI.actor.cpp index dfa106f048..47907515bb 100644 --- a/fdbclient/NativeAPI.actor.cpp +++ b/fdbclient/NativeAPI.actor.cpp @@ -32,6 +32,7 @@ #include #include "boost/algorithm/string.hpp" +#include "fdbrpc/TenantInfo.h" #include "fmt/format.h" #include "fdbclient/FDBOptions.g.h" @@ -6036,7 +6037,7 @@ ACTOR Future> estimateCommitCosts(Referen trCommitCosts.opsCount++; keyRange = KeyRangeRef(it->param1, it->param2); if (trState->options.expensiveClearCostEstimation) { - StorageMetrics m = wait(trState->cx->getStorageMetrics(keyRange, CLIENT_KNOBS->TOO_MANY)); + StorageMetrics m = wait(trState->cx->getStorageMetrics(keyRange, CLIENT_KNOBS->TOO_MANY, trState)); trCommitCosts.clearIdxCosts.emplace_back(i, getWriteOperationCost(m.bytes)); trCommitCosts.writeCosts += getWriteOperationCost(m.bytes); ++trCommitCosts.expensiveCostEstCount; @@ -7241,12 +7242,18 @@ Future Transaction::onError(Error const& e) { return e; } -ACTOR Future getStorageMetricsLargeKeyRange(Database cx, KeyRange keys); +ACTOR Future getStorageMetricsLargeKeyRange(Database cx, + KeyRange keys, + Future tenantInfoFuture); -ACTOR Future doGetStorageMetrics(Database cx, KeyRange keys, Reference locationInfo) { +ACTOR Future doGetStorageMetrics(Database cx, + KeyRange keys, + Reference locationInfo, + Future tenantInfoFuture) { + state TenantInfo tenantInfo = wait(tenantInfoFuture); loop { try { - WaitMetricsRequest req(keys, StorageMetrics(), StorageMetrics()); + WaitMetricsRequest req(tenantInfo, keys, StorageMetrics(), StorageMetrics()); req.min.bytes = 0; req.max.bytes = -1; StorageMetrics m = wait(loadBalance( @@ -7259,16 +7266,19 @@ ACTOR Future doGetStorageMetrics(Database cx, KeyRange keys, Ref } wait(delay(CLIENT_KNOBS->WRONG_SHARD_SERVER_DELAY, TaskPriority::DataDistribution)); cx->invalidateCache(Key(), keys); - StorageMetrics m = wait(getStorageMetricsLargeKeyRange(cx, keys)); + StorageMetrics m = wait(getStorageMetricsLargeKeyRange(cx, keys, tenantInfo)); return m; } } } -ACTOR Future getStorageMetricsLargeKeyRange(Database cx, KeyRange keys) { +ACTOR Future getStorageMetricsLargeKeyRange(Database cx, + KeyRange keys, + Future tenantInfoFuture) { state Span span("NAPI:GetStorageMetricsLargeKeyRange"_loc); + state TenantInfo tenantInfo = wait(tenantInfoFuture); std::vector locations = wait(getKeyRangeLocations(cx, - TenantInfo(), + tenantInfo, keys, std::numeric_limits::max(), Reverse::False, @@ -7284,7 +7294,7 @@ ACTOR Future getStorageMetricsLargeKeyRange(Database cx, KeyRang for (int i = 0; i < nLocs; i++) { partBegin = (i == 0) ? keys.begin : locations[i].range.begin; partEnd = (i == nLocs - 1) ? keys.end : locations[i].range.end; - fx[i] = doGetStorageMetrics(cx, KeyRangeRef(partBegin, partEnd), locations[i].locations); + fx[i] = doGetStorageMetrics(cx, KeyRangeRef(partBegin, partEnd), locations[i].locations, tenantInfo); } wait(waitForAll(fx)); for (int i = 0; i < nLocs; i++) { @@ -7293,14 +7303,15 @@ ACTOR Future getStorageMetricsLargeKeyRange(Database cx, KeyRang return total; } -ACTOR Future trackBoundedStorageMetrics(KeyRange keys, +ACTOR Future trackBoundedStorageMetrics(TenantInfo tenantInfo, + KeyRange keys, Reference location, StorageMetrics x, StorageMetrics halfError, PromiseStream deltaStream) { try { loop { - WaitMetricsRequest req(keys, x - halfError, x + halfError); + WaitMetricsRequest req(tenantInfo, keys, x - halfError, x + halfError); StorageMetrics nextX = wait(loadBalance(location->locations(), &StorageServerInterface::waitMetrics, req)); deltaStream.send(nextX - x); x = nextX; @@ -7311,7 +7322,8 @@ ACTOR Future trackBoundedStorageMetrics(KeyRange keys, } } -ACTOR Future waitStorageMetricsMultipleLocations(std::vector locations, +ACTOR Future waitStorageMetricsMultipleLocations(TenantInfo tenantInfo, + std::vector locations, StorageMetrics min, StorageMetrics max, StorageMetrics permittedError) { @@ -7325,7 +7337,7 @@ ACTOR Future waitStorageMetricsMultipleLocations(std::vectorlocations(), @@ -7346,7 +7358,7 @@ ACTOR Future waitStorageMetricsMultipleLocations(std::vector, int>> waitStorageMetrics(Databa StorageMetrics max, StorageMetrics permittedError, int shardLimit, - int expectedShardCount) { + int expectedShardCount, + Future tenantInfoFuture) { state Span span("NAPI:WaitStorageMetrics"_loc, generateSpanID(cx->transactionTracingSample)); + state TenantInfo tenantInfo = wait(tenantInfoFuture); loop { std::vector locations = wait(getKeyRangeLocations(cx, - TenantInfo(), + tenantInfo, keys, shardLimit, Reverse::False, @@ -7460,9 +7474,9 @@ ACTOR Future, int>> waitStorageMetrics(Databa try { Future fx; if (locations.size() > 1) { - fx = waitStorageMetricsMultipleLocations(locations, min, max, permittedError); + fx = waitStorageMetricsMultipleLocations(tenantInfo, locations, min, max, permittedError); } else { - WaitMetricsRequest req(keys, min, max); + WaitMetricsRequest req(tenantInfo, keys, min, max); fx = loadBalance(locations[0].locations->locations(), &StorageServerInterface::waitMetrics, req, @@ -7496,17 +7510,26 @@ Future, int>> DatabaseContext::waitStorageMet StorageMetrics const& max, StorageMetrics const& permittedError, int shardLimit, - int expectedShardCount) { + int expectedShardCount, + Optional> trState) { + Future tenantInfoFuture = + trState.present() ? populateAndGetTenant(trState.get(), keys.begin, latestVersion) : TenantInfo(); return ::waitStorageMetrics(Database(Reference::addRef(this)), keys, min, max, permittedError, shardLimit, - expectedShardCount); + expectedShardCount, + tenantInfoFuture); } -Future DatabaseContext::getStorageMetrics(KeyRange const& keys, int shardLimit) { +Future DatabaseContext::getStorageMetrics(KeyRange const& keys, + int shardLimit, + Optional> trState) { + Future tenantInfoFuture = + trState.present() ? populateAndGetTenant(trState.get(), keys.begin, latestVersion) : TenantInfo(); + if (shardLimit > 0) { StorageMetrics m; m.bytes = -1; @@ -7516,9 +7539,11 @@ Future DatabaseContext::getStorageMetrics(KeyRange const& keys, m, StorageMetrics(), shardLimit, - -1)); + -1, + tenantInfoFuture)); } else { - return ::getStorageMetricsLargeKeyRange(Database(Reference::addRef(this)), keys); + return ::getStorageMetricsLargeKeyRange( + Database(Reference::addRef(this)), keys, tenantInfoFuture); } } diff --git a/fdbclient/ReadYourWrites.actor.cpp b/fdbclient/ReadYourWrites.actor.cpp index 0fe4c598bf..cf549a59fd 100644 --- a/fdbclient/ReadYourWrites.actor.cpp +++ b/fdbclient/ReadYourWrites.actor.cpp @@ -1764,7 +1764,13 @@ Future ReadYourWritesTransaction::getEstimatedRangeSizeBytes(const KeyR if (resetPromise.isSet()) return resetPromise.getFuture().getError(); - return map(waitOrError(tr.getDatabase()->getStorageMetrics(keys, -1), resetPromise.getFuture()), + TraceEvent(SevWarnAlways, "AKDebug") + .detail("Status", "getEstimatedRangeSizeBytes") + .detail("Tenant", tr.trState->hasTenant() ? tr.trState->tenant().get().toString() : "not present") + .detail("TenantIdValid", tr.trState->tenantId() != TenantInfo::INVALID_TENANT) + .detail("AuthPresent", tr.trState->authToken.present()); + + return map(waitOrError(tr.getDatabase()->getStorageMetrics(keys, -1, tr.trState), resetPromise.getFuture()), [](const StorageMetrics& m) { return m.bytes; }); } @@ -2492,6 +2498,7 @@ void ReadYourWritesTransaction::setOptionImpl(FDBTransactionOptions::Option opti validateOptionValueNotPresent(value); options.bypassUnreadable = true; break; + // TODO(kejriwal): Check if this is needed case FDBTransactionOptions::AUTHORIZATION_TOKEN: tr.setOption(option, value); default: diff --git a/fdbclient/include/fdbclient/DatabaseContext.h b/fdbclient/include/fdbclient/DatabaseContext.h index 5d36b4fe8c..c3734e2889 100644 --- a/fdbclient/include/fdbclient/DatabaseContext.h +++ b/fdbclient/include/fdbclient/DatabaseContext.h @@ -295,13 +295,19 @@ public: Future onProxiesChanged() const; Future getHealthMetrics(bool detailed); // Pass a negative value for `shardLimit` to indicate no limit on the shard number. - Future getStorageMetrics(KeyRange const& keys, int shardLimit); - Future, int>> waitStorageMetrics(KeyRange const& keys, - StorageMetrics const& min, - StorageMetrics const& max, - StorageMetrics const& permittedError, - int shardLimit, - int expectedShardCount); + // Pass a valid `trState` with `hasTenant() == true` to make the function tenant-aware. + Future getStorageMetrics( + KeyRange const& keys, + int shardLimit, + Optional> trState = Optional>()); + Future, int>> waitStorageMetrics( + KeyRange const& keys, + StorageMetrics const& min, + StorageMetrics const& max, + StorageMetrics const& permittedError, + int shardLimit, + int expectedShardCount, + Optional> trState = Optional>()); Future splitStorageMetricsStream(PromiseStream const& resultsStream, KeyRange const& keys, StorageMetrics const& limit, diff --git a/fdbclient/include/fdbclient/StorageServerInterface.h b/fdbclient/include/fdbclient/StorageServerInterface.h index 0206dd6340..318782ded2 100644 --- a/fdbclient/include/fdbclient/StorageServerInterface.h +++ b/fdbclient/include/fdbclient/StorageServerInterface.h @@ -102,7 +102,7 @@ struct StorageServerInterface { PublicRequestStream getMappedKeyValues; RequestStream getShardState; - RequestStream waitMetrics; + PublicRequestStream waitMetrics; RequestStream splitMetrics; RequestStream getStorageMetrics; RequestStream> waitFailure; @@ -160,7 +160,8 @@ public: PublicRequestStream(getValue.getEndpoint().getAdjustedEndpoint(2)); getShardState = RequestStream(getValue.getEndpoint().getAdjustedEndpoint(3)); - waitMetrics = RequestStream(getValue.getEndpoint().getAdjustedEndpoint(4)); + waitMetrics = + PublicRequestStream(getValue.getEndpoint().getAdjustedEndpoint(4)); splitMetrics = RequestStream(getValue.getEndpoint().getAdjustedEndpoint(5)); getStorageMetrics = RequestStream(getValue.getEndpoint().getAdjustedEndpoint(6)); @@ -715,18 +716,24 @@ struct WaitMetricsRequest { // Waits for any of the given minimum or maximum metrics to be exceeded, and then returns the current values // Send a reversed range for min, max to receive an immediate report constexpr static FileIdentifier file_identifier = 1795961; + TenantInfo tenantInfo; Arena arena; KeyRangeRef keys; StorageMetrics min, max; ReplyPromise reply; + bool verify() const { return tenantInfo.isAuthorized(); } + WaitMetricsRequest() {} - WaitMetricsRequest(KeyRangeRef const& keys, StorageMetrics const& min, StorageMetrics const& max) - : keys(arena, keys), min(min), max(max) {} + WaitMetricsRequest(TenantInfo tenantInfo, + KeyRangeRef const& keys, + StorageMetrics const& min, + StorageMetrics const& max) + : tenantInfo(tenantInfo), keys(arena, keys), min(min), max(max) {} template void serialize(Ar& ar) { - serializer(ar, keys, min, max, reply, arena); + serializer(ar, keys, min, max, reply, tenantInfo, arena); } }; @@ -831,6 +838,7 @@ struct SplitRangeReply { } }; +// TODO(kejriwal): ref struct SplitRangeRequest { constexpr static FileIdentifier file_identifier = 10725174; Arena arena; diff --git a/fdbserver/include/fdbserver/StorageMetrics.h b/fdbserver/include/fdbserver/StorageMetrics.h index 9923bbf875..11fdaa132d 100644 --- a/fdbserver/include/fdbserver/StorageMetrics.h +++ b/fdbserver/include/fdbserver/StorageMetrics.h @@ -465,7 +465,7 @@ struct StorageServerMetrics { req.reply.send(rep); } - Future waitMetrics(WaitMetricsRequest req, Future delay); + Future waitMetrics(WaitMetricsRequest req, Future delay, Optional tenantPrefix); // Given a read hot shard, this function will divide the shard into chunks and find those chunks whose // readBytes/sizeBytes exceeds the `readDensityRatio`. Please make sure to run unit tests diff --git a/fdbserver/storageserver.actor.cpp b/fdbserver/storageserver.actor.cpp index 516f3f64b3..0cf4392d44 100644 --- a/fdbserver/storageserver.actor.cpp +++ b/fdbserver/storageserver.actor.cpp @@ -9509,8 +9509,14 @@ void StorageServer::byteSampleApplyClear(KeyRangeRef range, Version ver) { } } -ACTOR Future waitMetrics(StorageServerMetrics* self, WaitMetricsRequest req, Future timeout) { +ACTOR Future waitMetrics(StorageServerMetrics* self, + WaitMetricsRequest req, + Future timeout, + Optional tenantPrefix) { state PromiseStream change; + if (tenantPrefix.present()) { + req.keys = req.keys.withPrefix(tenantPrefix.get()); + } state StorageMetrics metrics = self->getMetrics(req.keys); state Error error = success(); state bool timedout = false; @@ -9600,8 +9606,8 @@ ACTOR Future waitMetrics(StorageServerMetrics* self, WaitMetricsRequest re return Void(); } -Future StorageServerMetrics::waitMetrics(WaitMetricsRequest req, Future delay) { - return ::waitMetrics(this, req, delay); +Future StorageServerMetrics::waitMetrics(WaitMetricsRequest req, Future delay, Optional tenantPrefix) { + return ::waitMetrics(this, req, delay, tenantPrefix); } #ifndef __INTEL_COMPILER @@ -9649,13 +9655,16 @@ ACTOR Future metricsCore(StorageServer* self, StorageServerInterface ssi) loop { choose { - when(WaitMetricsRequest req = waitNext(ssi.waitMetrics.getFuture())) { + when(state WaitMetricsRequest req = waitNext(ssi.waitMetrics.getFuture())) { if (!self->isReadable(req.keys)) { CODE_PROBE(true, "waitMetrics immediate wrong_shard_server()"); self->sendErrorWithPenalty(req.reply, wrong_shard_server(), self->getPenalty()); } else { - self->actors.add( - self->metrics.waitMetrics(req, delayJittered(SERVER_KNOBS->STORAGE_METRIC_TIMEOUT))); + wait(success(waitForVersionNoTooOld(self, latestVersion))); + Optional entry = self->getTenantEntry(latestVersion, req.tenantInfo); + Optional tenantPrefix = entry.map([](TenantMapEntry e) { return e.prefix; }); + self->actors.add(self->metrics.waitMetrics( + req, delayJittered(SERVER_KNOBS->STORAGE_METRIC_TIMEOUT), tenantPrefix)); } } when(SplitMetricsRequest req = waitNext(ssi.splitMetrics.getFuture())) { diff --git a/fdbserver/workloads/GetEstimatedRangeSize.actor.cpp b/fdbserver/workloads/GetEstimatedRangeSize.actor.cpp index 8db7ae0785..bf7f17d3bf 100644 --- a/fdbserver/workloads/GetEstimatedRangeSize.actor.cpp +++ b/fdbserver/workloads/GetEstimatedRangeSize.actor.cpp @@ -101,7 +101,6 @@ struct GetEstimatedRangeSizeWorkload : TestWorkload, Members { void setAuthToken(ReadYourWritesTransaction& tr) { tr.setOption(FDBTransactionOptions::AUTHORIZATION_TOKEN, this->signedToken); - tr.setOption(FDBTransactionOptions::RAW_ACCESS); } Key keyForIndex(int n) { return key(n); } @@ -111,30 +110,23 @@ struct GetEstimatedRangeSizeWorkload : TestWorkload, Members { Standalone operator()(int n) { return KeyValueRef(key(n), value((n + 1) % nodeCount)); } ACTOR static Future checkSize(GetEstimatedRangeSizeWorkload* self, Database cx) { - // TraceEvent(SevWarnAlways, "AKDebug").detail("Status", "checkSize-1").detail("Tenant", cx->defaultTenant.get()); int64_t size = wait(getSize(self, cx)); - // TraceEvent(SevWarnAlways, "AKDebug").detail("Status", "checkSize-2").detail("Tenant", cx->defaultTenant.get()); - TraceEvent(SevWarnAlways, "AKGetEstimatedRangeSizeResults") + TraceEvent(SevDebug, "GetEstimatedRangeSizeResults") .detail("Tenant", cx->defaultTenant.get()) .detail("TenantSize", size); - ASSERT_LT(size, 0); + ASSERT_GT(size, 0); return Void(); } ACTOR static Future getSize(GetEstimatedRangeSizeWorkload* self, Database cx) { - // TraceEvent(SevWarnAlways, "AKDebug").detail("Status", "getSize-1").detail("Tenant", cx->defaultTenant.get()); - state ReadYourWritesTransaction tr(cx); + state ReadYourWritesTransaction tr(cx, cx->defaultTenant); loop { try { self->setAuthToken(tr); state int64_t size = wait(tr.getEstimatedRangeSizeBytes(normalKeys)); - // TraceEvent(SevWarnAlways, "AKDebug") - // .detail("Status", "getSize-2") - // .detail("Tenant", cx->defaultTenant.get()); tr.reset(); return size; } catch (Error& e) { - // TraceEvent(SevWarnAlways, "AKDebugError").detail("Status", "getSize-3").detail("Error", e.name()); wait(tr.onError(e)); } } From de90e1aab145bd1aeeebd0af5b656604c2dce75d Mon Sep 17 00:00:00 2001 From: Ankita Kejriwal Date: Thu, 22 Sep 2022 18:40:04 -0700 Subject: [PATCH 030/210] Improve the code in the actors and server, simplify the workload --- fdbclient/NativeAPI.actor.cpp | 48 ++++++----- fdbclient/ReadYourWrites.actor.cpp | 8 +- .../include/fdbclient/CommitProxyInterface.h | 10 ++- .../fdbclient/StorageServerInterface.h | 2 +- fdbrpc/include/fdbrpc/fdbrpc.h | 3 + fdbserver/storageserver.actor.cpp | 20 +++-- .../workloads/GetEstimatedRangeSize.actor.cpp | 80 ++++++++++--------- tests/fast/GetEstimatedRangeSize.toml | 11 ++- 8 files changed, 105 insertions(+), 77 deletions(-) diff --git a/fdbclient/NativeAPI.actor.cpp b/fdbclient/NativeAPI.actor.cpp index 47907515bb..48ae9b96a2 100644 --- a/fdbclient/NativeAPI.actor.cpp +++ b/fdbclient/NativeAPI.actor.cpp @@ -7244,13 +7244,14 @@ Future Transaction::onError(Error const& e) { } ACTOR Future getStorageMetricsLargeKeyRange(Database cx, KeyRange keys, - Future tenantInfoFuture); + Optional> trState); ACTOR Future doGetStorageMetrics(Database cx, KeyRange keys, Reference locationInfo, - Future tenantInfoFuture) { - state TenantInfo tenantInfo = wait(tenantInfoFuture); + Optional> trState) { + state TenantInfo tenantInfo = + wait(trState.present() ? populateAndGetTenant(trState.get(), keys.begin, latestVersion) : TenantInfo()); loop { try { WaitMetricsRequest req(tenantInfo, keys, StorageMetrics(), StorageMetrics()); @@ -7266,7 +7267,7 @@ ACTOR Future doGetStorageMetrics(Database cx, } wait(delay(CLIENT_KNOBS->WRONG_SHARD_SERVER_DELAY, TaskPriority::DataDistribution)); cx->invalidateCache(Key(), keys); - StorageMetrics m = wait(getStorageMetricsLargeKeyRange(cx, keys, tenantInfo)); + StorageMetrics m = wait(getStorageMetricsLargeKeyRange(cx, keys, trState)); return m; } } @@ -7274,9 +7275,10 @@ ACTOR Future doGetStorageMetrics(Database cx, ACTOR Future getStorageMetricsLargeKeyRange(Database cx, KeyRange keys, - Future tenantInfoFuture) { + Optional> trState) { state Span span("NAPI:GetStorageMetricsLargeKeyRange"_loc); - state TenantInfo tenantInfo = wait(tenantInfoFuture); + state TenantInfo tenantInfo = + wait(trState.present() ? populateAndGetTenant(trState.get(), keys.begin, latestVersion) : TenantInfo()); std::vector locations = wait(getKeyRangeLocations(cx, tenantInfo, keys, @@ -7294,7 +7296,7 @@ ACTOR Future getStorageMetricsLargeKeyRange(Database cx, for (int i = 0; i < nLocs; i++) { partBegin = (i == 0) ? keys.begin : locations[i].range.begin; partEnd = (i == nLocs - 1) ? keys.end : locations[i].range.end; - fx[i] = doGetStorageMetrics(cx, KeyRangeRef(partBegin, partEnd), locations[i].locations, tenantInfo); + fx[i] = doGetStorageMetrics(cx, KeyRangeRef(partBegin, partEnd), locations[i].locations, trState); } wait(waitForAll(fx)); for (int i = 0; i < nLocs; i++) { @@ -7443,16 +7445,18 @@ ACTOR Future>> getReadHotRanges(Da } } -ACTOR Future, int>> waitStorageMetrics(Database cx, - KeyRange keys, - StorageMetrics min, - StorageMetrics max, - StorageMetrics permittedError, - int shardLimit, - int expectedShardCount, - Future tenantInfoFuture) { +ACTOR Future, int>> waitStorageMetrics( + Database cx, + KeyRange keys, + StorageMetrics min, + StorageMetrics max, + StorageMetrics permittedError, + int shardLimit, + int expectedShardCount, + Optional> trState) { state Span span("NAPI:WaitStorageMetrics"_loc, generateSpanID(cx->transactionTracingSample)); - state TenantInfo tenantInfo = wait(tenantInfoFuture); + state TenantInfo tenantInfo = + wait(trState.present() ? populateAndGetTenant(trState.get(), keys.begin, latestVersion) : TenantInfo()); loop { std::vector locations = wait(getKeyRangeLocations(cx, tenantInfo, @@ -7512,8 +7516,6 @@ Future, int>> DatabaseContext::waitStorageMet int shardLimit, int expectedShardCount, Optional> trState) { - Future tenantInfoFuture = - trState.present() ? populateAndGetTenant(trState.get(), keys.begin, latestVersion) : TenantInfo(); return ::waitStorageMetrics(Database(Reference::addRef(this)), keys, min, @@ -7521,15 +7523,12 @@ Future, int>> DatabaseContext::waitStorageMet permittedError, shardLimit, expectedShardCount, - tenantInfoFuture); + trState); } Future DatabaseContext::getStorageMetrics(KeyRange const& keys, int shardLimit, Optional> trState) { - Future tenantInfoFuture = - trState.present() ? populateAndGetTenant(trState.get(), keys.begin, latestVersion) : TenantInfo(); - if (shardLimit > 0) { StorageMetrics m; m.bytes = -1; @@ -7540,10 +7539,9 @@ Future DatabaseContext::getStorageMetrics(KeyRange const& keys, StorageMetrics(), shardLimit, -1, - tenantInfoFuture)); + trState)); } else { - return ::getStorageMetricsLargeKeyRange( - Database(Reference::addRef(this)), keys, tenantInfoFuture); + return ::getStorageMetricsLargeKeyRange(Database(Reference::addRef(this)), keys, trState); } } diff --git a/fdbclient/ReadYourWrites.actor.cpp b/fdbclient/ReadYourWrites.actor.cpp index cf549a59fd..5c6bb6c57c 100644 --- a/fdbclient/ReadYourWrites.actor.cpp +++ b/fdbclient/ReadYourWrites.actor.cpp @@ -1770,7 +1770,10 @@ Future ReadYourWritesTransaction::getEstimatedRangeSizeBytes(const KeyR .detail("TenantIdValid", tr.trState->tenantId() != TenantInfo::INVALID_TENANT) .detail("AuthPresent", tr.trState->authToken.present()); - return map(waitOrError(tr.getDatabase()->getStorageMetrics(keys, -1, tr.trState), resetPromise.getFuture()), + // Pass in the TransactionState only if tenant is present + Optional> trState = + tr.trState->hasTenant() ? tr.trState : Optional>(); + return map(waitOrError(tr.getDatabase()->getStorageMetrics(keys, -1, trState), resetPromise.getFuture()), [](const StorageMetrics& m) { return m.bytes; }); } @@ -2498,9 +2501,10 @@ void ReadYourWritesTransaction::setOptionImpl(FDBTransactionOptions::Option opti validateOptionValueNotPresent(value); options.bypassUnreadable = true; break; - // TODO(kejriwal): Check if this is needed + // TODO(kejriwal): Improve the way this is set case FDBTransactionOptions::AUTHORIZATION_TOKEN: tr.setOption(option, value); + break; default: break; } diff --git a/fdbclient/include/fdbclient/CommitProxyInterface.h b/fdbclient/include/fdbclient/CommitProxyInterface.h index 93247ec678..04f5a8041a 100644 --- a/fdbclient/include/fdbclient/CommitProxyInterface.h +++ b/fdbclient/include/fdbclient/CommitProxyInterface.h @@ -369,7 +369,15 @@ struct GetKeyServerLocationsRequest { : arena(arena), spanContext(spanContext), tenant(tenant), begin(begin), end(end), limit(limit), reverse(reverse), minTenantVersion(minTenantVersion) {} - bool verify() const { return tenant.isAuthorized(); } + bool verify() const { + if (!tenant.isAuthorized()) + TraceEvent(SevWarnAlways, "AKDebug") + .detail("Status", "GetKeyServerLocationsRequest") + .detail("Id", tenant.tenantId) + .detail("Name", tenant.name.present() ? tenant.name.get() : "not present"_sr); + return tenant.isAuthorized(); + // return true; + } template void serialize(Ar& ar) { diff --git a/fdbclient/include/fdbclient/StorageServerInterface.h b/fdbclient/include/fdbclient/StorageServerInterface.h index 318782ded2..3a6cf53a22 100644 --- a/fdbclient/include/fdbclient/StorageServerInterface.h +++ b/fdbclient/include/fdbclient/StorageServerInterface.h @@ -722,7 +722,7 @@ struct WaitMetricsRequest { StorageMetrics min, max; ReplyPromise reply; - bool verify() const { return tenantInfo.isAuthorized(); } + bool verify() const { return tenantInfo.tenantId == TenantInfo::INVALID_TENANT || tenantInfo.isAuthorized(); } WaitMetricsRequest() {} WaitMetricsRequest(TenantInfo tenantInfo, diff --git a/fdbrpc/include/fdbrpc/fdbrpc.h b/fdbrpc/include/fdbrpc/fdbrpc.h index 9636407052..7e554b6ad8 100644 --- a/fdbrpc/include/fdbrpc/fdbrpc.h +++ b/fdbrpc/include/fdbrpc/fdbrpc.h @@ -684,6 +684,9 @@ struct NetNotifiedQueue final : NotifiedQueue, FlowReceiver, FastAllocated) { + TraceEvent(SevWarnAlways, "AKDebug") + .detail("Status", "RPC") + .detail("Message", message.file_identifier); message.reply.sendError(permission_denied()); } } else { diff --git a/fdbserver/storageserver.actor.cpp b/fdbserver/storageserver.actor.cpp index 0cf4392d44..da19961201 100644 --- a/fdbserver/storageserver.actor.cpp +++ b/fdbserver/storageserver.actor.cpp @@ -9619,6 +9619,14 @@ Future StorageServerMetrics::waitMetrics(WaitMetricsRequest req, Future waitMetricsTenantAware(StorageServer* self, WaitMetricsRequest req) { + wait(success(waitForVersionNoTooOld(self, latestVersion))); + Optional entry = self->getTenantEntry(latestVersion, req.tenantInfo); + Optional tenantPrefix = entry.map([](TenantMapEntry e) { return e.prefix; }); + wait(self->metrics.waitMetrics(req, delayJittered(SERVER_KNOBS->STORAGE_METRIC_TIMEOUT), tenantPrefix)); + return Void(); +} + ACTOR Future metricsCore(StorageServer* self, StorageServerInterface ssi) { state Future doPollMetrics = Void(); @@ -9660,11 +9668,13 @@ ACTOR Future metricsCore(StorageServer* self, StorageServerInterface ssi) CODE_PROBE(true, "waitMetrics immediate wrong_shard_server()"); self->sendErrorWithPenalty(req.reply, wrong_shard_server(), self->getPenalty()); } else { - wait(success(waitForVersionNoTooOld(self, latestVersion))); - Optional entry = self->getTenantEntry(latestVersion, req.tenantInfo); - Optional tenantPrefix = entry.map([](TenantMapEntry e) { return e.prefix; }); - self->actors.add(self->metrics.waitMetrics( - req, delayJittered(SERVER_KNOBS->STORAGE_METRIC_TIMEOUT), tenantPrefix)); + // TODO(kejriwal) + self->actors.add(waitMetricsTenantAware(self, req)); + // wait(success(waitForVersionNoTooOld(self, latestVersion))); + // Optional entry = self->getTenantEntry(latestVersion, req.tenantInfo); + // Optional tenantPrefix = entry.map([](TenantMapEntry e) { return e.prefix; }); + // self->actors.add(self->metrics.waitMetrics( + // req, delayJittered(SERVER_KNOBS->STORAGE_METRIC_TIMEOUT), tenantPrefix)); } } when(SplitMetricsRequest req = waitNext(ssi.splitMetrics.getFuture())) { diff --git a/fdbserver/workloads/GetEstimatedRangeSize.actor.cpp b/fdbserver/workloads/GetEstimatedRangeSize.actor.cpp index bf7f17d3bf..f879233c7d 100644 --- a/fdbserver/workloads/GetEstimatedRangeSize.actor.cpp +++ b/fdbserver/workloads/GetEstimatedRangeSize.actor.cpp @@ -44,52 +44,51 @@ struct Members { }; struct GetEstimatedRangeSizeWorkload : TestWorkload, Members { - int actorCount, nodeCount; - double testDuration, transactionsPerSecond, minExpectedTransactionsPerSecond, traceParentProbability; + int nodeCount; + double testDuration; Key keyPrefix; + bool hasTenant; - std::vector> clients; - PerfIntCounter transactions, retries, tooOldRetries, commitFailedRetries; - PerfDoubleCounter totalLatency; - - GetEstimatedRangeSizeWorkload(WorkloadContext const& wcx) - : TestWorkload(wcx), transactions("Transactions"), retries("Retries"), tooOldRetries("Retries.too_old"), - commitFailedRetries("Retries.commit_failed"), totalLatency("Latency") { + GetEstimatedRangeSizeWorkload(WorkloadContext const& wcx) : TestWorkload(wcx) { testDuration = getOption(options, "testDuration"_sr, 10.0); - transactionsPerSecond = getOption(options, "transactionsPerSecond"_sr, 5000.0) / clientCount; - actorCount = getOption(options, "actorsPerClient"_sr, transactionsPerSecond / 5); - nodeCount = getOption(options, "nodeCount"_sr, transactionsPerSecond * clientCount); + nodeCount = getOption(options, "nodeCount"_sr, 10000.0); keyPrefix = unprintable(getOption(options, "keyPrefix"_sr, LiteralStringRef("")).toString()); - traceParentProbability = getOption(options, "traceParentProbability"_sr, 0.01); - minExpectedTransactionsPerSecond = transactionsPerSecond * getOption(options, "expectedRate"_sr, 0.7); + hasTenant = hasOption(options, "tenant"_sr); - ASSERT(g_network->isSimulated()); - auto k = g_simulator.authKeys.begin(); - this->tenant = getOption(options, "tenant"_sr, "DefaultTenant"_sr); - // make it comfortably longer than the timeout of the workload - auto currentTime = uint64_t(lround(g_network->timer())); - this->token.algorithm = authz::Algorithm::ES256; - this->token.issuedAtUnixTime = currentTime; - this->token.expiresAtUnixTime = - currentTime + uint64_t(std::lround(getCheckTimeout())) + uint64_t(std::lround(testDuration)) + 100; - this->token.keyId = k->first; - this->token.notBeforeUnixTime = currentTime - 10; - VectorRef tenants; - tenants.push_back_deep(this->arena, this->tenant); - this->token.tenants = tenants; - // we currently don't support this workload to be run outside of simulation - this->signedToken = authz::jwt::signToken(this->arena, this->token, k->second); + if (hasTenant) { + ASSERT(g_network->isSimulated()); + auto k = g_simulator.authKeys.begin(); + this->tenant = getOption(options, "tenant"_sr, "DefaultTenant"_sr); + // make it comfortably longer than the timeout of the workload + auto currentTime = uint64_t(lround(g_network->timer())); + this->token.algorithm = authz::Algorithm::ES256; + this->token.issuedAtUnixTime = currentTime; + this->token.expiresAtUnixTime = + currentTime + uint64_t(std::lround(getCheckTimeout())) + uint64_t(std::lround(testDuration)) + 100; + this->token.keyId = k->first; + this->token.notBeforeUnixTime = currentTime - 10; + VectorRef tenants; + tenants.push_back_deep(this->arena, this->tenant); + this->token.tenants = tenants; + // we currently don't support this workload to be run outside of simulation + this->signedToken = authz::jwt::signToken(this->arena, this->token, k->second); + } } std::string description() const override { return "GetEstimatedRangeSizeWorkload"; } Future setup(Database const& cx) override { + if (!hasTenant) { + return Void(); + } cx->defaultTenant = this->tenant; return bulkSetup(cx, this, nodeCount, Promise()); } Future start(Database const& cx) override { - cx->defaultTenant = this->tenant; + if (clientId > 0) { + return Void(); + } return checkSize(this, cx); } @@ -100,7 +99,9 @@ struct GetEstimatedRangeSizeWorkload : TestWorkload, Members { StringRef getAuthToken() const { return this->signedToken; } void setAuthToken(ReadYourWritesTransaction& tr) { - tr.setOption(FDBTransactionOptions::AUTHORIZATION_TOKEN, this->signedToken); + if (tr.getTenant().present()) { + tr.setOption(FDBTransactionOptions::AUTHORIZATION_TOKEN, this->signedToken); + } } Key keyForIndex(int n) { return key(n); } @@ -110,20 +111,25 @@ struct GetEstimatedRangeSizeWorkload : TestWorkload, Members { Standalone operator()(int n) { return KeyValueRef(key(n), value((n + 1) % nodeCount)); } ACTOR static Future checkSize(GetEstimatedRangeSizeWorkload* self, Database cx) { - int64_t size = wait(getSize(self, cx)); - TraceEvent(SevDebug, "GetEstimatedRangeSizeResults") - .detail("Tenant", cx->defaultTenant.get()) - .detail("TenantSize", size); + state int64_t size = wait(getSize(self, cx)); ASSERT_GT(size, 0); return Void(); } ACTOR static Future getSize(GetEstimatedRangeSizeWorkload* self, Database cx) { - state ReadYourWritesTransaction tr(cx, cx->defaultTenant); + state Optional tenant = self->hasTenant ? self->tenant : Optional(); + cx->defaultTenant = tenant; + state ReadYourWritesTransaction tr(cx, tenant); + TraceEvent(SevDebug, "AKGetSize1") + .detail("Tenant", cx->defaultTenant.present() ? cx->defaultTenant.get() : "none"_sr); + loop { try { self->setAuthToken(tr); state int64_t size = wait(tr.getEstimatedRangeSizeBytes(normalKeys)); + TraceEvent(SevDebug, "AKGetSize2") + .detail("Tenant", cx->defaultTenant.present() ? cx->defaultTenant.get() : "none"_sr) + .detail("Size", size); tr.reset(); return size; } catch (Error& e) { diff --git a/tests/fast/GetEstimatedRangeSize.toml b/tests/fast/GetEstimatedRangeSize.toml index 020e62cf71..95d37218bf 100644 --- a/tests/fast/GetEstimatedRangeSize.toml +++ b/tests/fast/GetEstimatedRangeSize.toml @@ -19,13 +19,12 @@ testTitle = 'GetEstimatedRangeSizeTest' [[test.workload]] testName = 'GetEstimatedRangeSize' tenant = 'First' - transactionsPerSecond = 250.0 - testDuration = 10.0 - expectedRate = 0.80 + nodeCount = 250.0 [[test.workload]] testName = 'GetEstimatedRangeSize' tenant = 'Second' - transactionsPerSecond = 2500.0 - testDuration = 10.0 - expectedRate = 0.80 + nodeCount = 2500.0 + + [[test.workload]] + testName = 'GetEstimatedRangeSize' From 0264262f78b951f3e9dbbcb39612e5be7eea802e Mon Sep 17 00:00:00 2001 From: Ankita Kejriwal Date: Thu, 29 Sep 2022 18:03:20 -0700 Subject: [PATCH 031/210] Explicitly pass tenant to BulkSetup for use in Transactions With this change, BulkSetup accepts Optional as a parameter. It then uses this while constructing transactions. If a TenantName is passed to a transaction this way, the transaction can use this tenant rather than relying on the DatabaseContext. This can be used by workloads to run the BulkSetup in parallel for multiple tenants without causing a race. --- .../fdbserver/workloads/BulkSetup.actor.h | 27 ++++++++++++------- 1 file changed, 17 insertions(+), 10 deletions(-) diff --git a/fdbserver/include/fdbserver/workloads/BulkSetup.actor.h b/fdbserver/include/fdbserver/workloads/BulkSetup.actor.h index 575c361be4..ed3b490ae6 100644 --- a/fdbserver/include/fdbserver/workloads/BulkSetup.actor.h +++ b/fdbserver/include/fdbserver/workloads/BulkSetup.actor.h @@ -22,6 +22,7 @@ // When actually compiled (NO_INTELLISENSE), include the generated // version of this file. In intellisense use the source version. +#include "fdbrpc/TenantName.h" #if defined(NO_INTELLISENSE) && !defined(FDBSERVER_BULK_SETUP_ACTOR_G_H) #define FDBSERVER_BULK_SETUP_ACTOR_G_H #include "fdbserver/workloads/BulkSetup.actor.g.h" @@ -58,9 +59,13 @@ void setAuthToken(T const& self, Transaction& tr) { } ACTOR template -Future checkRangeSimpleValueSize(Database cx, T* workload, uint64_t begin, uint64_t end) { +Future checkRangeSimpleValueSize(Database cx, + T* workload, + uint64_t begin, + uint64_t end, + Optional tenant) { loop { - state Transaction tr(cx); + state Transaction tr(cx, tenant); setAuthToken(*workload, tr); try { state Standalone firstKV = (*workload)(begin); @@ -78,10 +83,10 @@ Future checkRangeSimpleValueSize(Database cx, T* workload, uint64_t begin, // Returns true if the range was added ACTOR template -Future setupRange(Database cx, T* workload, uint64_t begin, uint64_t end) { +Future setupRange(Database cx, T* workload, uint64_t begin, uint64_t end, Optional tenant) { state uint64_t bytesInserted = 0; loop { - state Transaction tr(cx); + state Transaction tr(cx, tenant); setAuthToken(*workload, tr); try { // if( deterministicRandom()->random01() < 0.001 ) @@ -128,7 +133,8 @@ Future setupRangeWorker(Database cx, std::vector>* jobs, double maxKeyInsertRate, int keySaveIncrement, - int actorId) { + int actorId, + Optional tenant) { state double nextStart; state uint64_t loadedRanges = 0; state int lastStoredKeysLoaded = 0; @@ -138,7 +144,7 @@ Future setupRangeWorker(Database cx, state std::pair job = jobs->back(); jobs->pop_back(); nextStart = now() + (job.second - job.first) / maxKeyInsertRate; - uint64_t numBytes = wait(setupRange(cx, workload, job.first, job.second)); + uint64_t numBytes = wait(setupRange(cx, workload, job.first, job.second, tenant)); if (numBytes > 0) loadedRanges++; @@ -147,7 +153,7 @@ Future setupRangeWorker(Database cx, bytesStored += numBytes; if (keysLoaded - lastStoredKeysLoaded >= keySaveIncrement || jobs->size() == 0) { - state Transaction tr(cx); + state Transaction tr(cx, tenant); setAuthToken(*workload, tr); try { std::string countKey = format("keycount|%d|%d", workload->clientId, actorId); @@ -224,7 +230,8 @@ Future bulkSetup(Database cx, int keySaveIncrement = 0, double keyCheckInterval = 0.1, uint64_t startNodeIdx = 0, - uint64_t endNodeIdx = 0) { + uint64_t endNodeIdx = 0, + Optional tenant = Optional()) { state std::vector> jobs; state uint64_t startNode = startNodeIdx ? startNodeIdx : (nodeCount * workload->clientId) / workload->clientCount; @@ -241,7 +248,7 @@ Future bulkSetup(Database cx, // For bulk data schemes where the value of the key is not critical to operation, check to // see if the database has already been set up. if (valuesInconsequential) { - bool present = wait(checkRangeSimpleValueSize(cx, workload, startNode, endNode)); + bool present = wait(checkRangeSimpleValueSize(cx, workload, startNode, endNode, tenant)); if (present) { TraceEvent("BulkSetupRangeAlreadyPresent") .detail("Begin", startNode) @@ -304,7 +311,7 @@ Future bulkSetup(Database cx, keySaveIncrement = 0; for (int j = 0; j < BULK_SETUP_WORKERS; j++) - fs.push_back(setupRangeWorker(cx, workload, &jobs, maxWorkerInsertRate, keySaveIncrement, j)); + fs.push_back(setupRangeWorker(cx, workload, &jobs, maxWorkerInsertRate, keySaveIncrement, j, tenant)); try { wait(success(insertionTimes) && waitForAll(fs)); } catch (Error& e) { From c2b6b288b79bc7750e8c2df947da432a5e4b6d0a Mon Sep 17 00:00:00 2001 From: Ankita Kejriwal Date: Thu, 29 Sep 2022 18:08:40 -0700 Subject: [PATCH 032/210] Fixes in the GetEstimatedRangeSize workload * Run the workload on a trusted client rather than untrusted clients. This allows the workload to be substantially simplified as well as enables testing for the case where no tenant is present. * Explicitly pass tenant to BulkSetup so that the setup phase can be run in parallel for multiple tenants without causing a race. --- fdbclient/ReadYourWrites.actor.cpp | 4 - .../workloads/GetEstimatedRangeSize.actor.cpp | 73 ++++++------------- 2 files changed, 21 insertions(+), 56 deletions(-) diff --git a/fdbclient/ReadYourWrites.actor.cpp b/fdbclient/ReadYourWrites.actor.cpp index 5c6bb6c57c..d2c9ef6692 100644 --- a/fdbclient/ReadYourWrites.actor.cpp +++ b/fdbclient/ReadYourWrites.actor.cpp @@ -2501,10 +2501,6 @@ void ReadYourWritesTransaction::setOptionImpl(FDBTransactionOptions::Option opti validateOptionValueNotPresent(value); options.bypassUnreadable = true; break; - // TODO(kejriwal): Improve the way this is set - case FDBTransactionOptions::AUTHORIZATION_TOKEN: - tr.setOption(option, value); - break; default: break; } diff --git a/fdbserver/workloads/GetEstimatedRangeSize.actor.cpp b/fdbserver/workloads/GetEstimatedRangeSize.actor.cpp index f879233c7d..cc43e37056 100644 --- a/fdbserver/workloads/GetEstimatedRangeSize.actor.cpp +++ b/fdbserver/workloads/GetEstimatedRangeSize.actor.cpp @@ -21,58 +21,24 @@ #include #include "fdbclient/SystemData.h" -#include "flow/Arena.h" -#include "flow/IRandom.h" -#include "flow/Trace.h" -#include "flow/serialize.h" -#include "fdbrpc/simulator.h" -#include "fdbrpc/TokenSign.h" -#include "fdbclient/FDBOptions.g.h" -#include "fdbclient/NativeAPI.actor.h" -#include "fdbclient/SystemData.h" -#include "fdbserver/TesterInterface.actor.h" #include "fdbserver/workloads/workloads.actor.h" #include "fdbserver/workloads/BulkSetup.actor.h" #include "flow/actorcompiler.h" // This must be the last #include. -struct Members { - Arena arena; - TenantName tenant; - authz::jwt::TokenRef token; - StringRef signedToken; -}; - -struct GetEstimatedRangeSizeWorkload : TestWorkload, Members { +struct GetEstimatedRangeSizeWorkload : TestWorkload { int nodeCount; double testDuration; Key keyPrefix; bool hasTenant; + TenantName tenant; GetEstimatedRangeSizeWorkload(WorkloadContext const& wcx) : TestWorkload(wcx) { testDuration = getOption(options, "testDuration"_sr, 10.0); nodeCount = getOption(options, "nodeCount"_sr, 10000.0); keyPrefix = unprintable(getOption(options, "keyPrefix"_sr, LiteralStringRef("")).toString()); hasTenant = hasOption(options, "tenant"_sr); - - if (hasTenant) { - ASSERT(g_network->isSimulated()); - auto k = g_simulator.authKeys.begin(); - this->tenant = getOption(options, "tenant"_sr, "DefaultTenant"_sr); - // make it comfortably longer than the timeout of the workload - auto currentTime = uint64_t(lround(g_network->timer())); - this->token.algorithm = authz::Algorithm::ES256; - this->token.issuedAtUnixTime = currentTime; - this->token.expiresAtUnixTime = - currentTime + uint64_t(std::lround(getCheckTimeout())) + uint64_t(std::lround(testDuration)) + 100; - this->token.keyId = k->first; - this->token.notBeforeUnixTime = currentTime - 10; - VectorRef tenants; - tenants.push_back_deep(this->arena, this->tenant); - this->token.tenants = tenants; - // we currently don't support this workload to be run outside of simulation - this->signedToken = authz::jwt::signToken(this->arena, this->token, k->second); - } + tenant = getOption(options, "tenant"_sr, "DefaultTenant"_sr); } std::string description() const override { return "GetEstimatedRangeSizeWorkload"; } @@ -81,8 +47,21 @@ struct GetEstimatedRangeSizeWorkload : TestWorkload, Members { if (!hasTenant) { return Void(); } - cx->defaultTenant = this->tenant; - return bulkSetup(cx, this, nodeCount, Promise()); + // Use default values for arguments between (and including) postSetupWarming and endNodeIdx params + return bulkSetup(cx, + this, + nodeCount, + Promise(), + true, + 0.0, + 1e12, + std::vector(), + Promise>>(), + 0, + 0.1, + 0, + 0, + tenant); } Future start(Database const& cx) override { @@ -96,14 +75,6 @@ struct GetEstimatedRangeSizeWorkload : TestWorkload, Members { void getMetrics(std::vector& m) override {} - StringRef getAuthToken() const { return this->signedToken; } - - void setAuthToken(ReadYourWritesTransaction& tr) { - if (tr.getTenant().present()) { - tr.setOption(FDBTransactionOptions::AUTHORIZATION_TOKEN, this->signedToken); - } - } - Key keyForIndex(int n) { return key(n); } Key key(int n) { return doubleToTestKey((double)n / nodeCount, keyPrefix); } Value value(int n) { return doubleToTestKey(n, keyPrefix); } @@ -118,17 +89,15 @@ struct GetEstimatedRangeSizeWorkload : TestWorkload, Members { ACTOR static Future getSize(GetEstimatedRangeSizeWorkload* self, Database cx) { state Optional tenant = self->hasTenant ? self->tenant : Optional(); - cx->defaultTenant = tenant; state ReadYourWritesTransaction tr(cx, tenant); TraceEvent(SevDebug, "AKGetSize1") - .detail("Tenant", cx->defaultTenant.present() ? cx->defaultTenant.get() : "none"_sr); + .detail("Tenant", tr.getTenant().present() ? tr.getTenant().get() : "none"_sr); loop { try { - self->setAuthToken(tr); state int64_t size = wait(tr.getEstimatedRangeSizeBytes(normalKeys)); TraceEvent(SevDebug, "AKGetSize2") - .detail("Tenant", cx->defaultTenant.present() ? cx->defaultTenant.get() : "none"_sr) + .detail("Tenant", tr.getTenant().present() ? tr.getTenant().get() : "none"_sr) .detail("Size", size); tr.reset(); return size; @@ -139,4 +108,4 @@ struct GetEstimatedRangeSizeWorkload : TestWorkload, Members { } }; -WorkloadFactory GetEstimatedRangeSizeWorkloadFactory("GetEstimatedRangeSize", true); +WorkloadFactory GetEstimatedRangeSizeWorkloadFactory("GetEstimatedRangeSize", false); From ed5bd8b13bb0e22799696ffcdfd98369ae58e11d Mon Sep 17 00:00:00 2001 From: Ankita Kejriwal Date: Thu, 29 Sep 2022 18:14:53 -0700 Subject: [PATCH 033/210] Remove temporary debug TraceEvents and other comments from previous commits --- fdbclient/ReadYourWrites.actor.cpp | 6 ------ fdbclient/include/fdbclient/CommitProxyInterface.h | 10 +--------- fdbrpc/include/fdbrpc/fdbrpc.h | 3 --- fdbserver/storageserver.actor.cpp | 6 ------ fdbserver/workloads/GetEstimatedRangeSize.actor.cpp | 5 ++--- 5 files changed, 3 insertions(+), 27 deletions(-) diff --git a/fdbclient/ReadYourWrites.actor.cpp b/fdbclient/ReadYourWrites.actor.cpp index d2c9ef6692..e3772dfc39 100644 --- a/fdbclient/ReadYourWrites.actor.cpp +++ b/fdbclient/ReadYourWrites.actor.cpp @@ -1764,12 +1764,6 @@ Future ReadYourWritesTransaction::getEstimatedRangeSizeBytes(const KeyR if (resetPromise.isSet()) return resetPromise.getFuture().getError(); - TraceEvent(SevWarnAlways, "AKDebug") - .detail("Status", "getEstimatedRangeSizeBytes") - .detail("Tenant", tr.trState->hasTenant() ? tr.trState->tenant().get().toString() : "not present") - .detail("TenantIdValid", tr.trState->tenantId() != TenantInfo::INVALID_TENANT) - .detail("AuthPresent", tr.trState->authToken.present()); - // Pass in the TransactionState only if tenant is present Optional> trState = tr.trState->hasTenant() ? tr.trState : Optional>(); diff --git a/fdbclient/include/fdbclient/CommitProxyInterface.h b/fdbclient/include/fdbclient/CommitProxyInterface.h index 04f5a8041a..93247ec678 100644 --- a/fdbclient/include/fdbclient/CommitProxyInterface.h +++ b/fdbclient/include/fdbclient/CommitProxyInterface.h @@ -369,15 +369,7 @@ struct GetKeyServerLocationsRequest { : arena(arena), spanContext(spanContext), tenant(tenant), begin(begin), end(end), limit(limit), reverse(reverse), minTenantVersion(minTenantVersion) {} - bool verify() const { - if (!tenant.isAuthorized()) - TraceEvent(SevWarnAlways, "AKDebug") - .detail("Status", "GetKeyServerLocationsRequest") - .detail("Id", tenant.tenantId) - .detail("Name", tenant.name.present() ? tenant.name.get() : "not present"_sr); - return tenant.isAuthorized(); - // return true; - } + bool verify() const { return tenant.isAuthorized(); } template void serialize(Ar& ar) { diff --git a/fdbrpc/include/fdbrpc/fdbrpc.h b/fdbrpc/include/fdbrpc/fdbrpc.h index 7e554b6ad8..9636407052 100644 --- a/fdbrpc/include/fdbrpc/fdbrpc.h +++ b/fdbrpc/include/fdbrpc/fdbrpc.h @@ -684,9 +684,6 @@ struct NetNotifiedQueue final : NotifiedQueue, FlowReceiver, FastAllocated) { - TraceEvent(SevWarnAlways, "AKDebug") - .detail("Status", "RPC") - .detail("Message", message.file_identifier); message.reply.sendError(permission_denied()); } } else { diff --git a/fdbserver/storageserver.actor.cpp b/fdbserver/storageserver.actor.cpp index da19961201..eba7316563 100644 --- a/fdbserver/storageserver.actor.cpp +++ b/fdbserver/storageserver.actor.cpp @@ -9668,13 +9668,7 @@ ACTOR Future metricsCore(StorageServer* self, StorageServerInterface ssi) CODE_PROBE(true, "waitMetrics immediate wrong_shard_server()"); self->sendErrorWithPenalty(req.reply, wrong_shard_server(), self->getPenalty()); } else { - // TODO(kejriwal) self->actors.add(waitMetricsTenantAware(self, req)); - // wait(success(waitForVersionNoTooOld(self, latestVersion))); - // Optional entry = self->getTenantEntry(latestVersion, req.tenantInfo); - // Optional tenantPrefix = entry.map([](TenantMapEntry e) { return e.prefix; }); - // self->actors.add(self->metrics.waitMetrics( - // req, delayJittered(SERVER_KNOBS->STORAGE_METRIC_TIMEOUT), tenantPrefix)); } } when(SplitMetricsRequest req = waitNext(ssi.splitMetrics.getFuture())) { diff --git a/fdbserver/workloads/GetEstimatedRangeSize.actor.cpp b/fdbserver/workloads/GetEstimatedRangeSize.actor.cpp index cc43e37056..852a927cca 100644 --- a/fdbserver/workloads/GetEstimatedRangeSize.actor.cpp +++ b/fdbserver/workloads/GetEstimatedRangeSize.actor.cpp @@ -90,13 +90,12 @@ struct GetEstimatedRangeSizeWorkload : TestWorkload { ACTOR static Future getSize(GetEstimatedRangeSizeWorkload* self, Database cx) { state Optional tenant = self->hasTenant ? self->tenant : Optional(); state ReadYourWritesTransaction tr(cx, tenant); - TraceEvent(SevDebug, "AKGetSize1") - .detail("Tenant", tr.getTenant().present() ? tr.getTenant().get() : "none"_sr); + TraceEvent(SevDebug, "GetSize1").detail("Tenant", tr.getTenant().present() ? tr.getTenant().get() : "none"_sr); loop { try { state int64_t size = wait(tr.getEstimatedRangeSizeBytes(normalKeys)); - TraceEvent(SevDebug, "AKGetSize2") + TraceEvent(SevDebug, "GetSize2") .detail("Tenant", tr.getTenant().present() ? tr.getTenant().get() : "none"_sr) .detail("Size", size); tr.reset(); From 5a3cd51255fef4d8581e4a5a92e044e1c6dc3336 Mon Sep 17 00:00:00 2001 From: Ankita Kejriwal Date: Thu, 29 Sep 2022 18:14:53 -0700 Subject: [PATCH 034/210] Remove temporary TraceEvents and comments from previous commits --- fdbclient/ReadYourWrites.actor.cpp | 6 ------ fdbclient/include/fdbclient/CommitProxyInterface.h | 10 +--------- fdbclient/include/fdbclient/StorageServerInterface.h | 1 - fdbrpc/include/fdbrpc/fdbrpc.h | 3 --- fdbserver/storageserver.actor.cpp | 6 ------ fdbserver/workloads/GetEstimatedRangeSize.actor.cpp | 5 ++--- 6 files changed, 3 insertions(+), 28 deletions(-) diff --git a/fdbclient/ReadYourWrites.actor.cpp b/fdbclient/ReadYourWrites.actor.cpp index d2c9ef6692..e3772dfc39 100644 --- a/fdbclient/ReadYourWrites.actor.cpp +++ b/fdbclient/ReadYourWrites.actor.cpp @@ -1764,12 +1764,6 @@ Future ReadYourWritesTransaction::getEstimatedRangeSizeBytes(const KeyR if (resetPromise.isSet()) return resetPromise.getFuture().getError(); - TraceEvent(SevWarnAlways, "AKDebug") - .detail("Status", "getEstimatedRangeSizeBytes") - .detail("Tenant", tr.trState->hasTenant() ? tr.trState->tenant().get().toString() : "not present") - .detail("TenantIdValid", tr.trState->tenantId() != TenantInfo::INVALID_TENANT) - .detail("AuthPresent", tr.trState->authToken.present()); - // Pass in the TransactionState only if tenant is present Optional> trState = tr.trState->hasTenant() ? tr.trState : Optional>(); diff --git a/fdbclient/include/fdbclient/CommitProxyInterface.h b/fdbclient/include/fdbclient/CommitProxyInterface.h index 04f5a8041a..93247ec678 100644 --- a/fdbclient/include/fdbclient/CommitProxyInterface.h +++ b/fdbclient/include/fdbclient/CommitProxyInterface.h @@ -369,15 +369,7 @@ struct GetKeyServerLocationsRequest { : arena(arena), spanContext(spanContext), tenant(tenant), begin(begin), end(end), limit(limit), reverse(reverse), minTenantVersion(minTenantVersion) {} - bool verify() const { - if (!tenant.isAuthorized()) - TraceEvent(SevWarnAlways, "AKDebug") - .detail("Status", "GetKeyServerLocationsRequest") - .detail("Id", tenant.tenantId) - .detail("Name", tenant.name.present() ? tenant.name.get() : "not present"_sr); - return tenant.isAuthorized(); - // return true; - } + bool verify() const { return tenant.isAuthorized(); } template void serialize(Ar& ar) { diff --git a/fdbclient/include/fdbclient/StorageServerInterface.h b/fdbclient/include/fdbclient/StorageServerInterface.h index 3a6cf53a22..329daaf730 100644 --- a/fdbclient/include/fdbclient/StorageServerInterface.h +++ b/fdbclient/include/fdbclient/StorageServerInterface.h @@ -838,7 +838,6 @@ struct SplitRangeReply { } }; -// TODO(kejriwal): ref struct SplitRangeRequest { constexpr static FileIdentifier file_identifier = 10725174; Arena arena; diff --git a/fdbrpc/include/fdbrpc/fdbrpc.h b/fdbrpc/include/fdbrpc/fdbrpc.h index 7e554b6ad8..9636407052 100644 --- a/fdbrpc/include/fdbrpc/fdbrpc.h +++ b/fdbrpc/include/fdbrpc/fdbrpc.h @@ -684,9 +684,6 @@ struct NetNotifiedQueue final : NotifiedQueue, FlowReceiver, FastAllocated) { - TraceEvent(SevWarnAlways, "AKDebug") - .detail("Status", "RPC") - .detail("Message", message.file_identifier); message.reply.sendError(permission_denied()); } } else { diff --git a/fdbserver/storageserver.actor.cpp b/fdbserver/storageserver.actor.cpp index da19961201..eba7316563 100644 --- a/fdbserver/storageserver.actor.cpp +++ b/fdbserver/storageserver.actor.cpp @@ -9668,13 +9668,7 @@ ACTOR Future metricsCore(StorageServer* self, StorageServerInterface ssi) CODE_PROBE(true, "waitMetrics immediate wrong_shard_server()"); self->sendErrorWithPenalty(req.reply, wrong_shard_server(), self->getPenalty()); } else { - // TODO(kejriwal) self->actors.add(waitMetricsTenantAware(self, req)); - // wait(success(waitForVersionNoTooOld(self, latestVersion))); - // Optional entry = self->getTenantEntry(latestVersion, req.tenantInfo); - // Optional tenantPrefix = entry.map([](TenantMapEntry e) { return e.prefix; }); - // self->actors.add(self->metrics.waitMetrics( - // req, delayJittered(SERVER_KNOBS->STORAGE_METRIC_TIMEOUT), tenantPrefix)); } } when(SplitMetricsRequest req = waitNext(ssi.splitMetrics.getFuture())) { diff --git a/fdbserver/workloads/GetEstimatedRangeSize.actor.cpp b/fdbserver/workloads/GetEstimatedRangeSize.actor.cpp index cc43e37056..852a927cca 100644 --- a/fdbserver/workloads/GetEstimatedRangeSize.actor.cpp +++ b/fdbserver/workloads/GetEstimatedRangeSize.actor.cpp @@ -90,13 +90,12 @@ struct GetEstimatedRangeSizeWorkload : TestWorkload { ACTOR static Future getSize(GetEstimatedRangeSizeWorkload* self, Database cx) { state Optional tenant = self->hasTenant ? self->tenant : Optional(); state ReadYourWritesTransaction tr(cx, tenant); - TraceEvent(SevDebug, "AKGetSize1") - .detail("Tenant", tr.getTenant().present() ? tr.getTenant().get() : "none"_sr); + TraceEvent(SevDebug, "GetSize1").detail("Tenant", tr.getTenant().present() ? tr.getTenant().get() : "none"_sr); loop { try { state int64_t size = wait(tr.getEstimatedRangeSizeBytes(normalKeys)); - TraceEvent(SevDebug, "AKGetSize2") + TraceEvent(SevDebug, "GetSize2") .detail("Tenant", tr.getTenant().present() ? tr.getTenant().get() : "none"_sr) .detail("Size", size); tr.reset(); From e41504e6fe1f710c03be24352fb9b31aa51e2654 Mon Sep 17 00:00:00 2001 From: Ankita Kejriwal Date: Thu, 29 Sep 2022 18:53:44 -0700 Subject: [PATCH 035/210] Fix the verify() function in WaitMetricsRequest --- fdbclient/include/fdbclient/StorageServerInterface.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/fdbclient/include/fdbclient/StorageServerInterface.h b/fdbclient/include/fdbclient/StorageServerInterface.h index 329daaf730..2d56db7643 100644 --- a/fdbclient/include/fdbclient/StorageServerInterface.h +++ b/fdbclient/include/fdbclient/StorageServerInterface.h @@ -722,7 +722,7 @@ struct WaitMetricsRequest { StorageMetrics min, max; ReplyPromise reply; - bool verify() const { return tenantInfo.tenantId == TenantInfo::INVALID_TENANT || tenantInfo.isAuthorized(); } + bool verify() const { return tenantInfo.isAuthorized(); } WaitMetricsRequest() {} WaitMetricsRequest(TenantInfo tenantInfo, From 999809e79b30d4a5b5f04f30c71caadc84314ff8 Mon Sep 17 00:00:00 2001 From: Ankita Kejriwal Date: Thu, 29 Sep 2022 19:21:44 -0700 Subject: [PATCH 036/210] Increase the data in the workload and add ASSERTs for expected values. --- .../workloads/GetEstimatedRangeSize.actor.cpp | 15 ++++++++++++++- tests/fast/GetEstimatedRangeSize.toml | 2 +- 2 files changed, 15 insertions(+), 2 deletions(-) diff --git a/fdbserver/workloads/GetEstimatedRangeSize.actor.cpp b/fdbserver/workloads/GetEstimatedRangeSize.actor.cpp index 852a927cca..471d887c6c 100644 --- a/fdbserver/workloads/GetEstimatedRangeSize.actor.cpp +++ b/fdbserver/workloads/GetEstimatedRangeSize.actor.cpp @@ -24,6 +24,7 @@ #include "fdbserver/workloads/workloads.actor.h" #include "fdbserver/workloads/BulkSetup.actor.h" +#include "flow/Error.h" #include "flow/actorcompiler.h" // This must be the last #include. struct GetEstimatedRangeSizeWorkload : TestWorkload { @@ -83,7 +84,19 @@ struct GetEstimatedRangeSizeWorkload : TestWorkload { ACTOR static Future checkSize(GetEstimatedRangeSizeWorkload* self, Database cx) { state int64_t size = wait(getSize(self, cx)); - ASSERT_GT(size, 0); + // The following expected values are hard coded based on expected size for the tenants. + // We use a wide range to avoid flakiness because the underlying function (being tested) + // is making an estimation. + if (!self->hasTenant) { + ASSERT_GT(size, 9393000 / 5); + ASSERT_LT(size, 9393000 * 5); + } else if (self->tenant == "First"_sr) { + ASSERT_GT(size, 8525000 / 5); + ASSERT_LT(size, 8525000 * 5); + } else if (self->tenant == "Second"_sr) { + ASSERT_GT(size, 93000 / 5); + ASSERT_LT(size, 93000 * 5); + } return Void(); } diff --git a/tests/fast/GetEstimatedRangeSize.toml b/tests/fast/GetEstimatedRangeSize.toml index 95d37218bf..0c9503cf33 100644 --- a/tests/fast/GetEstimatedRangeSize.toml +++ b/tests/fast/GetEstimatedRangeSize.toml @@ -19,7 +19,7 @@ testTitle = 'GetEstimatedRangeSizeTest' [[test.workload]] testName = 'GetEstimatedRangeSize' tenant = 'First' - nodeCount = 250.0 + nodeCount = 250000.0 [[test.workload]] testName = 'GetEstimatedRangeSize' From d64d405af7308e424032aad43bb23167399fd940 Mon Sep 17 00:00:00 2001 From: He Liu Date: Wed, 5 Oct 2022 14:19:33 -0700 Subject: [PATCH 037/210] Validate all storage servers. --- fdbserver/storageserver.actor.cpp | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/fdbserver/storageserver.actor.cpp b/fdbserver/storageserver.actor.cpp index 15126c85cb..f9e97647c3 100644 --- a/fdbserver/storageserver.actor.cpp +++ b/fdbserver/storageserver.actor.cpp @@ -4433,16 +4433,16 @@ ACTOR Future validateRangeAgainstServers(StorageServer* data, KeyRange ran } } - std::vector ssis; + std::vector> fs; for (const auto& v : serverListValues) { if (!v.present()) { TraceEvent(SevWarn, "ValidateRangeRemoteServerNotFound", data->thisServerID).detail("Range", range); throw audit_storage_failed(); } - ssis.push_back(decodeServerListValue(v.get())); + fs.push_back(validateRangeAgainstServer(data, range, version, decodeServerListValue(v.get()))); } - wait(validateRangeAgainstServer(data, range, version, ssis[0])); + wait(waitForAll(fs)); return Void(); } From 6a44778b45e532e102a0dfbfcb393304bf5e0956 Mon Sep 17 00:00:00 2001 From: Josh Slocum Date: Fri, 7 Oct 2022 12:28:06 -0500 Subject: [PATCH 038/210] Fixed spacing in fdbcli status so dashes are aligned --- fdbcli/StatusCommand.actor.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/fdbcli/StatusCommand.actor.cpp b/fdbcli/StatusCommand.actor.cpp index 9da5e768a3..494da41c38 100644 --- a/fdbcli/StatusCommand.actor.cpp +++ b/fdbcli/StatusCommand.actor.cpp @@ -442,7 +442,7 @@ void printStatus(StatusObjectReader statusObj, outputString += "\n Blob granules - enabled"; } - outputString += "\n Encryption at-rest - "; + outputString += "\n Encryption at-rest - "; if (statusObjConfig.get("encryption_at_rest_mode", strVal)) { outputString += strVal; } else { From 2402b813e9e3a05c522d145504be7536438e1c35 Mon Sep 17 00:00:00 2001 From: Dan Lambright Date: Fri, 7 Oct 2022 16:32:28 -0400 Subject: [PATCH 039/210] do not unlink fdb_c in tmp dir if knob set --- .../java/src/main/com/apple/foundationdb/JNIUtil.java | 8 ++------ fdbclient/ClientKnobs.cpp | 1 + fdbclient/MultiVersionTransaction.actor.cpp | 8 ++++++-- fdbclient/include/fdbclient/ClientKnobs.h | 1 + 4 files changed, 10 insertions(+), 8 deletions(-) diff --git a/bindings/java/src/main/com/apple/foundationdb/JNIUtil.java b/bindings/java/src/main/com/apple/foundationdb/JNIUtil.java index bfee1bc034..cf4e0ade06 100644 --- a/bindings/java/src/main/com/apple/foundationdb/JNIUtil.java +++ b/bindings/java/src/main/com/apple/foundationdb/JNIUtil.java @@ -182,19 +182,15 @@ public class JNIUtil { private static OS getRunningOS() { String osname = System.getProperty("os.name").toLowerCase(); String arch = System.getProperty("os.arch"); - Boolean eagerDelete = true; - if (System.getProperty("fdb.bindings.java.eager_delete") != null) { - eagerDelete = false; - } if (!arch.equals("amd64") && !arch.equals("x86_64") && !arch.equals("aarch64") && !arch.equals("ppc64le")) { throw new IllegalStateException("Unknown or unsupported arch: " + arch); } if (osname.startsWith("windows")) { return new OS("windows", arch, /* canDeleteEager */ false); } else if (osname.startsWith("linux")) { - return new OS("linux", arch, /* canDeleteEager */ eagerDelete); + return new OS("linux", arch, /* canDeleteEager */ true); } else if (osname.startsWith("mac") || osname.startsWith("darwin")) { - return new OS("osx", arch, /* canDeleteEager */ eagerDelete); + return new OS("osx", arch, /* canDeleteEager */ true); } else { throw new IllegalStateException("Unknown or unsupported OS: " + osname); } diff --git a/fdbclient/ClientKnobs.cpp b/fdbclient/ClientKnobs.cpp index 6b15c1aa4d..7bc0bcc54d 100644 --- a/fdbclient/ClientKnobs.cpp +++ b/fdbclient/ClientKnobs.cpp @@ -198,6 +198,7 @@ void ClientKnobs::initialize(Randomize randomize) { init( DEFAULT_AUTO_LOGS, 3 ); init( DEFAULT_COMMIT_GRV_PROXIES_RATIO, 3 ); init( DEFAULT_MAX_GRV_PROXIES, 4 ); + init( UNLINKONLOAD_FDBCLIB, true ); // if false, don't delete libfdb_c in tmp directory on client connect. init( GLOBAL_CONFIG_REFRESH_BACKOFF, 0.5 ); init( GLOBAL_CONFIG_REFRESH_MAX_BACKOFF, 60.0 ); diff --git a/fdbclient/MultiVersionTransaction.actor.cpp b/fdbclient/MultiVersionTransaction.actor.cpp index 775cfa54d0..1fb2d9b0e2 100644 --- a/fdbclient/MultiVersionTransaction.actor.cpp +++ b/fdbclient/MultiVersionTransaction.actor.cpp @@ -2537,8 +2537,12 @@ void MultiVersionApi::setupNetwork() { externalClients[filename] = {}; auto libCopies = copyExternalLibraryPerThread(path); for (int idx = 0; idx < libCopies.size(); ++idx) { + bool unlinkOnLoad = libCopies[idx].second; + if (!CLIENT_KNOBS->UNLINKONLOAD_FDBCLIB) { + unlinkOnLoad = false; + } externalClients[filename].push_back(Reference( - new ClientInfo(new DLApi(libCopies[idx].first, libCopies[idx].second /*unlink on load*/), + new ClientInfo(new DLApi(libCopies[idx].first, unlinkOnLoad /*unlink on load*/), path, useFutureVersion, idx))); @@ -2552,7 +2556,7 @@ void MultiVersionApi::setupNetwork() { // // Typically we would create a more specific error for this case, but since we expect // this case to go away soon, we can use a trace event and a generic error. - TraceEvent(SevWarn, "CannotSetupNetwork") + TraceEvent("CannotSetupNetwork") .detail("Reason", "Local client is disabled and no external clients configured"); throw client_invalid_operation(); diff --git a/fdbclient/include/fdbclient/ClientKnobs.h b/fdbclient/include/fdbclient/ClientKnobs.h index 492c1ed44d..cfc658844c 100644 --- a/fdbclient/include/fdbclient/ClientKnobs.h +++ b/fdbclient/include/fdbclient/ClientKnobs.h @@ -199,6 +199,7 @@ public: int32_t DEFAULT_MAX_GRV_PROXIES; int32_t DEFAULT_AUTO_RESOLVERS; int32_t DEFAULT_AUTO_LOGS; + bool UNLINKONLOAD_FDBCLIB; double GLOBAL_CONFIG_REFRESH_BACKOFF; double GLOBAL_CONFIG_REFRESH_MAX_BACKOFF; From 4ebe341e2c6f5b34976f898dac4820e25992956f Mon Sep 17 00:00:00 2001 From: Josh Slocum Date: Fri, 7 Oct 2022 16:12:39 -0500 Subject: [PATCH 040/210] adding --disable_client_bypass to mako --- bindings/c/test/mako/mako.cpp | 15 +++++++++++++++ bindings/c/test/mako/mako.hpp | 2 ++ 2 files changed, 17 insertions(+) diff --git a/bindings/c/test/mako/mako.cpp b/bindings/c/test/mako/mako.cpp index e7ce98b198..0cc82e3d8a 100644 --- a/bindings/c/test/mako/mako.cpp +++ b/bindings/c/test/mako/mako.cpp @@ -875,6 +875,16 @@ int workerProcessMain(Arguments const& args, int worker_id, shared_memory::Acces } } + if (args.disable_client_bypass > 0) { + err = network::setOptionNothrow(FDB_NET_OPTION_DISABLE_CLIENT_BYPASS, args.disable_client_bypass); + if (err) { + logr.error("network::setOption (FDB_NET_OPTION_DISABLE_CLIENT_BYPASS) ({}): {}", + args.disable_client_bypass, + err.what()); + return -1; + } + } + /* Network thread must be setup before doing anything */ logr.debug("network::setup()"); network::setup(); @@ -1005,6 +1015,7 @@ int initArguments(Arguments& args) { args.txnspec.ops[i][OP_COUNT] = 0; } args.client_threads_per_version = 0; + args.disable_client_bypass = 0; args.disable_ryw = 0; args.json_output_path[0] = '\0'; args.stats_export_path[0] = '\0'; @@ -1248,6 +1259,7 @@ int parseArguments(int argc, char* argv[], Arguments& args) { { "txntagging_prefix", required_argument, NULL, ARG_TXNTAGGINGPREFIX }, { "version", no_argument, NULL, ARG_VERSION }, { "client_threads_per_version", required_argument, NULL, ARG_CLIENT_THREADS_PER_VERSION }, + { "disable_client_bypass", no_argument, NULL, ARG_DISABLE_CLIENT_BYPASS }, { "disable_ryw", no_argument, NULL, ARG_DISABLE_RYW }, { "json_report", optional_argument, NULL, ARG_JSON_REPORT }, { "bg_file_path", required_argument, NULL, ARG_BG_FILE_PATH }, @@ -1446,6 +1458,9 @@ int parseArguments(int argc, char* argv[], Arguments& args) { case ARG_CLIENT_THREADS_PER_VERSION: args.client_threads_per_version = atoi(optarg); break; + case ARG_DISABLE_CLIENT_BYPASS: + args.disable_client_bypass = 1; + break; case ARG_DISABLE_RYW: args.disable_ryw = 1; break; diff --git a/bindings/c/test/mako/mako.hpp b/bindings/c/test/mako/mako.hpp index 4d1df16bec..85bb38ec19 100644 --- a/bindings/c/test/mako/mako.hpp +++ b/bindings/c/test/mako/mako.hpp @@ -75,6 +75,7 @@ enum ArgKind { ARG_STREAMING_MODE, ARG_DISABLE_RYW, ARG_CLIENT_THREADS_PER_VERSION, + ARG_DISABLE_CLIENT_BYPASS, ARG_JSON_REPORT, ARG_BG_FILE_PATH, // if blob granule files are stored locally, mako will read and materialize them if this is set ARG_EXPORT_PATH, @@ -169,6 +170,7 @@ struct Arguments { char txntagging_prefix[TAGPREFIXLENGTH_MAX]; FDBStreamingMode streaming_mode; int64_t client_threads_per_version; + int disable_client_bypass; int disable_ryw; char json_output_path[PATH_MAX]; bool bg_materialize_files; From 98f66fda463b998c8e53b8242d5febeb63b7e695 Mon Sep 17 00:00:00 2001 From: sfc-gh-tclinkenbeard Date: Sun, 9 Oct 2022 19:24:19 -0700 Subject: [PATCH 041/210] Add 'quota clear' fdbcli command --- fdbcli/QuotaCommand.actor.cpp | 33 +++++++++++++++++++++++++++++---- fdbclient/TagThrottle.actor.cpp | 2 +- 2 files changed, 30 insertions(+), 5 deletions(-) diff --git a/fdbcli/QuotaCommand.actor.cpp b/fdbcli/QuotaCommand.actor.cpp index e6a86e9b51..ba8125f75d 100644 --- a/fdbcli/QuotaCommand.actor.cpp +++ b/fdbcli/QuotaCommand.actor.cpp @@ -103,6 +103,21 @@ ACTOR Future setQuota(Reference db, TransactionTag tag, LimitTy } } +ACTOR Future clearQuota(Reference db, TransactionTag tag) { + state Reference tr = db->createTransaction(); + state Key key = tag.withPrefix(tagQuotaPrefix); + loop { + tr->setOption(FDBTransactionOptions::ACCESS_SYSTEM_KEYS); + try { + tr->clear(ThrottleApi::getTagQuotaKey(tag)); + wait(safeThreadFutureToFuture(tr->commit())); + return Void(); + } catch (Error& e) { + wait(safeThreadFutureToFuture(tr->onError(e))); + } + } +} + constexpr auto usage = "quota [get [reserved_throughput|total_throughput] | set " "[reserved_throughput|total_throughput] ]"; @@ -117,30 +132,40 @@ namespace fdb_cli { ACTOR Future quotaCommandActor(Reference db, std::vector tokens) { state bool result = true; - if (tokens.size() != 5 && tokens.size() != 6) { + if (tokens.size() < 3 || tokens.size() > 5) { return exitFailure(); } else { auto tag = parseTag(tokens[2]); - auto limitType = parseLimitType(tokens[3]); - if (!tag.present() || !limitType.present()) { + if (!tag.present()) { return exitFailure(); } if (tokens[1] == "get"_sr) { if (tokens.size() != 4) { return exitFailure(); } + auto const limitType = parseLimitType(tokens[3]); + if (!limitType.present()) { + return exitFailure(); + } wait(getQuota(db, tag.get(), limitType.get())); return true; } else if (tokens[1] == "set"_sr) { if (tokens.size() != 5) { return exitFailure(); } + auto const limitType = parseLimitType(tokens[3]); auto const limitValue = parseLimitValue(tokens[4]); - if (!limitValue.present()) { + if (!limitType.present() || !limitValue.present()) { return exitFailure(); } wait(setQuota(db, tag.get(), limitType.get(), limitValue.get())); return true; + } else if (tokens[1] == "clear"_sr) { + if (tokens.size() != 3) { + return exitFailure(); + } + wait(clearQuota(db, tag.get())); + return true; } else { return exitFailure(); } diff --git a/fdbclient/TagThrottle.actor.cpp b/fdbclient/TagThrottle.actor.cpp index 7a1712c4df..abf10c7e83 100644 --- a/fdbclient/TagThrottle.actor.cpp +++ b/fdbclient/TagThrottle.actor.cpp @@ -145,7 +145,7 @@ Value ThrottleApi::TagQuotaValue::toValue() const { ThrottleApi::TagQuotaValue ThrottleApi::TagQuotaValue::fromValue(ValueRef value) { auto tuple = Tuple::unpack(value); - if (tuple.size() != 4) { + if (tuple.size() != 2) { throw invalid_throttle_quota_value(); } TagQuotaValue result; From d67f3ca3b98a7025a0c0c293a46a77c901044d51 Mon Sep 17 00:00:00 2001 From: sfc-gh-tclinkenbeard Date: Sun, 9 Oct 2022 19:43:44 -0700 Subject: [PATCH 042/210] Update documentation for quota clear command --- design/global-tag-throttling.md | 6 ++++++ fdbcli/QuotaCommand.actor.cpp | 2 +- fdbcli/fdbcli.actor.cpp | 2 +- 3 files changed, 8 insertions(+), 2 deletions(-) diff --git a/design/global-tag-throttling.md b/design/global-tag-throttling.md index ad7750a3cc..41ae6bdcda 100644 --- a/design/global-tag-throttling.md +++ b/design/global-tag-throttling.md @@ -47,6 +47,12 @@ Note that the quotas are specified in terms of bytes/second, and internally conv page_cost_quota = ceiling(byte_quota / CLIENT_KNOBS->READ_COST_BYTE_FACTOR) ``` +To clear a quota, run: + +``` +fdbcli> quota clear +``` + ### 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. diff --git a/fdbcli/QuotaCommand.actor.cpp b/fdbcli/QuotaCommand.actor.cpp index ba8125f75d..2626ddb2c2 100644 --- a/fdbcli/QuotaCommand.actor.cpp +++ b/fdbcli/QuotaCommand.actor.cpp @@ -119,7 +119,7 @@ ACTOR Future clearQuota(Reference db, TransactionTag tag) { } constexpr auto usage = "quota [get [reserved_throughput|total_throughput] | set " - "[reserved_throughput|total_throughput] ]"; + "[reserved_throughput|total_throughput] | clear ]"; bool exitFailure() { fmt::print(usage); diff --git a/fdbcli/fdbcli.actor.cpp b/fdbcli/fdbcli.actor.cpp index a5c2e2e75a..2ea89f8cd4 100644 --- a/fdbcli/fdbcli.actor.cpp +++ b/fdbcli/fdbcli.actor.cpp @@ -539,7 +539,7 @@ void initHelp() { "Displays the current read version of the database or currently running transaction."); helpMap["quota"] = CommandHelp("quota", "quota [get [reserved_throughput|total_throughput] | set " - "[reserved_throughput|total_throughput] ]", + "[reserved_throughput|total_throughput] | clear ]", "Get or modify the throughput quota for the specified tag."); helpMap["reset"] = CommandHelp("reset", From b7043154655df3af2914aa6f303dadab4d7710e3 Mon Sep 17 00:00:00 2001 From: Josh Slocum Date: Mon, 10 Oct 2022 09:30:16 -0500 Subject: [PATCH 043/210] fixing parameter to be bool --- bindings/c/test/mako/mako.cpp | 10 +++++----- bindings/c/test/mako/mako.hpp | 2 +- 2 files changed, 6 insertions(+), 6 deletions(-) diff --git a/bindings/c/test/mako/mako.cpp b/bindings/c/test/mako/mako.cpp index 0cc82e3d8a..becf107fc7 100644 --- a/bindings/c/test/mako/mako.cpp +++ b/bindings/c/test/mako/mako.cpp @@ -875,10 +875,10 @@ int workerProcessMain(Arguments const& args, int worker_id, shared_memory::Acces } } - if (args.disable_client_bypass > 0) { - err = network::setOptionNothrow(FDB_NET_OPTION_DISABLE_CLIENT_BYPASS, args.disable_client_bypass); + if (args.disable_client_bypass) { + err = network::setOptionNothrow(FDB_NET_OPTION_DISABLE_CLIENT_BYPASS); if (err) { - logr.error("network::setOption (FDB_NET_OPTION_DISABLE_CLIENT_BYPASS) ({}): {}", + logr.error("network::setOption (FDB_NET_OPTION_DISABLE_CLIENT_BYPASS): {}", args.disable_client_bypass, err.what()); return -1; @@ -1015,7 +1015,7 @@ int initArguments(Arguments& args) { args.txnspec.ops[i][OP_COUNT] = 0; } args.client_threads_per_version = 0; - args.disable_client_bypass = 0; + args.disable_client_bypass = false; args.disable_ryw = 0; args.json_output_path[0] = '\0'; args.stats_export_path[0] = '\0'; @@ -1459,7 +1459,7 @@ int parseArguments(int argc, char* argv[], Arguments& args) { args.client_threads_per_version = atoi(optarg); break; case ARG_DISABLE_CLIENT_BYPASS: - args.disable_client_bypass = 1; + args.disable_client_bypass = true; break; case ARG_DISABLE_RYW: args.disable_ryw = 1; diff --git a/bindings/c/test/mako/mako.hpp b/bindings/c/test/mako/mako.hpp index 85bb38ec19..bafe65b546 100644 --- a/bindings/c/test/mako/mako.hpp +++ b/bindings/c/test/mako/mako.hpp @@ -170,7 +170,7 @@ struct Arguments { char txntagging_prefix[TAGPREFIXLENGTH_MAX]; FDBStreamingMode streaming_mode; int64_t client_threads_per_version; - int disable_client_bypass; + bool disable_client_bypass; int disable_ryw; char json_output_path[PATH_MAX]; bool bg_materialize_files; From 1931e3266f44c945d31684df596490dc04913c4b Mon Sep 17 00:00:00 2001 From: He Liu Date: Fri, 7 Oct 2022 13:38:11 -0700 Subject: [PATCH 044/210] Resolve comments. --- .../include/fdbclient/AuditUtils.actor.h | 1 + .../fdbclient/StorageServerInterface.h | 3 +- fdbserver/DDTxnProcessor.actor.cpp | 18 +++--- fdbserver/DataDistribution.actor.cpp | 15 ++++- fdbserver/include/fdbserver/DDTxnProcessor.h | 3 +- fdbserver/storageserver.actor.cpp | 59 ++++++++---------- fdbserver/workloads/ValidateStorage.actor.cpp | 61 ------------------- flow/include/flow/error_definitions.h | 1 + 8 files changed, 52 insertions(+), 109 deletions(-) diff --git a/fdbclient/include/fdbclient/AuditUtils.actor.h b/fdbclient/include/fdbclient/AuditUtils.actor.h index c3588ff77b..94daa2f5f8 100644 --- a/fdbclient/include/fdbclient/AuditUtils.actor.h +++ b/fdbclient/include/fdbclient/AuditUtils.actor.h @@ -30,4 +30,5 @@ ACTOR Future persistAuditStorageState(Key key, AuditStorageState auditState); +#include "flow/unactorcompiler.h" #endif diff --git a/fdbclient/include/fdbclient/StorageServerInterface.h b/fdbclient/include/fdbclient/StorageServerInterface.h index 94ab8437f6..064c8fbeba 100644 --- a/fdbclient/include/fdbclient/StorageServerInterface.h +++ b/fdbclient/include/fdbclient/StorageServerInterface.h @@ -121,9 +121,8 @@ struct StorageServerInterface { RequestStream checkpoint; RequestStream fetchCheckpoint; RequestStream fetchCheckpointKeyValues; - RequestStream auditStorage; - RequestStream updateCommitCostRequest; + RequestStream auditStorage; private: bool acceptingRequests; diff --git a/fdbserver/DDTxnProcessor.actor.cpp b/fdbserver/DDTxnProcessor.actor.cpp index 70ecf251ad..1c28b69cfb 100644 --- a/fdbserver/DDTxnProcessor.actor.cpp +++ b/fdbserver/DDTxnProcessor.actor.cpp @@ -119,18 +119,20 @@ class DDTxnProcessorImpl { UID srcId, destId; decodeKeyServersValue(UIDtoTagMap, shards[i].value, src, dest, srcId, destId); - state IDDTxnProcessor::DDRangeLocations current(KeyRangeRef(shards[i].key, shards[i + 1].key)); - state int j = 0; - for (j = 0; j < src.size(); ++j) { - Optional serverListValue = wait(tr.get(serverListKeyFor(src[j]))); - KeyRangeRef currentRange(shards[i].key, shards[i + 1].key); - if (!serverListValue.present()) { + std::vector>> serverListEntries; + for (int j = 0; j < src.size(); ++j) { + serverListEntries.push_back(tr.get(serverListKeyFor(src[j]))); + } + std::vector> serverListValues = wait(getAll(serverListEntries)); + IDDTxnProcessor::DDRangeLocations current(KeyRangeRef(shards[i].key, shards[i + 1].key)); + for (int j = 0; j < serverListValues.size(); ++j) { + if (!serverListValues[j].present()) { TraceEvent(SevWarnAlways, "GetSourceServerInterfacesMissing") .detail("StorageServer", src[j]) - .detail("Range", currentRange); + .detail("Range", KeyRangeRef(shards[i].key, shards[i + 1].key)); continue; } - const StorageServerInterface ssi = decodeServerListValue(serverListValue.get()); + StorageServerInterface ssi = decodeServerListValue(serverListValues[j].get()); current.servers[ssi.locality.describeDcId()].push_back(ssi); } res.push_back(current); diff --git a/fdbserver/DataDistribution.actor.cpp b/fdbserver/DataDistribution.actor.cpp index 2a0582708e..3fe66fc82d 100644 --- a/fdbserver/DataDistribution.actor.cpp +++ b/fdbserver/DataDistribution.actor.cpp @@ -54,9 +54,11 @@ #include "flow/actorcompiler.h" // This must be the last #include. struct DDAudit { - DDAudit(UID id, AuditType type) : id(id), type(type), auditMap(AuditPhase::Invalid, allKeys.end), actors(true) {} + DDAudit(UID id, KeyRange range, AuditType type) + : id(id), range(range), type(type), auditMap(AuditPhase::Invalid, allKeys.end), actors(true) {} const UID id; + KeyRange range; const AuditType type; KeyRangeMap auditMap; ActorCollection actors; @@ -1355,10 +1357,17 @@ ACTOR Future auditStorage(Reference self, TriggerAuditReq state std::shared_ptr audit; auto it = self->audits.find(req.getType()); if (it != self->audits.end() && !it->second.empty()) { - audit = it->second.front(); + ASSERT_EQ(it->second.size(), 1); + auto& currentAudit = it->second.front(); + if (currentAudit->range.contains(req.range)) { + audit = it->second.front(); + } else { + req.reply.sendError(audit_storage_exeed_max()); + return Void(); + } } else { const UID auditId = deterministicRandom()->randomUniqueID(); - audit = std::make_shared(auditId, req.getType()); + audit = std::make_shared(auditId, req.range, req.getType()); self->audits[req.getType()].push_back(audit); audit->actors.add(scheduleAuditForRange(self, audit, req.range)); TraceEvent(SevDebug, "DDAuditStorageBegin", audit->id).detail("Range", req.range).detail("AuditType", req.type); diff --git a/fdbserver/include/fdbserver/DDTxnProcessor.h b/fdbserver/include/fdbserver/DDTxnProcessor.h index 9f9a85bf20..fb696a14b8 100644 --- a/fdbserver/include/fdbserver/DDTxnProcessor.h +++ b/fdbserver/include/fdbserver/DDTxnProcessor.h @@ -49,8 +49,7 @@ public: // get the source server list and complete source server list for range virtual Future getSourceServersForRange(const KeyRangeRef range) = 0; - virtual Future> getSourceServerInterfacesForRange( - const KeyRangeRef range) = 0; + virtual Future> getSourceServerInterfacesForRange(const KeyRangeRef range) = 0; // get the storage server list and Process class virtual Future>> getServerListAndProcessClasses() = 0; diff --git a/fdbserver/storageserver.actor.cpp b/fdbserver/storageserver.actor.cpp index f9e97647c3..83f3c3bc2b 100644 --- a/fdbserver/storageserver.actor.cpp +++ b/fdbserver/storageserver.actor.cpp @@ -4189,7 +4189,7 @@ ACTOR Future validateRangeAgainstServer(StorageServer* data, KeyRange range, Version version, StorageServerInterface remoteServer) { - TraceEvent(SevDebug, "ValidateRangeAgainstServerBegin", data->thisServerID) + TraceEvent(SevInfo, "ValidateRangeAgainstServerBegin", data->thisServerID) .detail("Range", range) .detail("Version", version) .detail("RemoteServer", remoteServer.toString()); @@ -4198,20 +4198,19 @@ ACTOR Future validateRangeAgainstServer(StorageServer* data, state std::string error; loop { try { - state int limit = 1e4; - state int limitBytes = CLIENT_KNOBS->REPLY_BYTE_LIMIT; - state GetKeyValuesRequest req; + std::vector>> fs; + int limit = 1e4; + int limitBytes = CLIENT_KNOBS->REPLY_BYTE_LIMIT; + GetKeyValuesRequest req; req.begin = firstGreaterOrEqual(range.begin); req.end = firstGreaterOrEqual(range.end); req.limit = limit; req.limitBytes = limitBytes; req.version = version; req.tags = TagSet(); - state Future> remoteKeyValueFuture = - remoteServer.getKeyValues.getReplyUnlessFailedFor(req, 2, 0); - state ErrorOr remoteResult = wait(remoteKeyValueFuture); + fs.push_back(remoteServer.getKeyValues.getReplyUnlessFailedFor(req, 2, 0)); - state GetKeyValuesRequest localReq; + GetKeyValuesRequest localReq; localReq.begin = firstGreaterOrEqual(range.begin); localReq.end = firstGreaterOrEqual(range.end); localReq.limit = limit; @@ -4219,35 +4218,29 @@ ACTOR Future validateRangeAgainstServer(StorageServer* data, localReq.version = version; localReq.tags = TagSet(); data->actors.add(getKeyValuesQ(data, localReq)); - GetKeyValuesReply local = wait(localReq.reply.getFuture()); - if (local.error.present()) { - TraceEvent(SevWarn, "ValidateRangeGetLocalKeyValuesError", data->thisServerID) - .errorUnsuppressed(local.error.get()) - .detail("Range", range); - throw local.error.get(); + fs.push_back(errorOr(localReq.reply.getFuture())); + std::vector> reps = wait(getAll(fs)); + + for (int i = 0; i < reps.size(); ++i) { + if (reps[i].isError()) { + TraceEvent(SevWarn, "ValidateRangeGetKeyValuesError", data->thisServerID) + .errorUnsuppressed(reps[i].getError()) + .detail("ReplyIndex", i) + .detail("Range", range); + throw reps[i].getError(); + } + if (reps[i].get().error.present()) { + TraceEvent(SevWarn, "ValidateRangeGetKeyValuesError", data->thisServerID) + .errorUnsuppressed(reps[i].get().error.get()) + .detail("ReplyIndex", i) + .detail("Range", range); + throw reps[i].get().error.get(); + } } + GetKeyValuesReply remote = reps[0].get(), local = reps[1].get(); Key lastKey = range.begin; - if (remoteResult.isError()) { - TraceEvent(SevWarn, "ValidateRangeGetRemoteKeyValuesError", data->thisServerID) - .errorUnsuppressed(remoteResult.getError()) - .detail("Range", range); - throw remoteResult.getError(); - } - - GetKeyValuesReply remote = remoteResult.get(); - if (remote.error.present()) { - TraceEvent(SevWarn, "ValidateRangeGetRemoteKeyValuesError", data->thisServerID) - .errorUnsuppressed(remote.error.get()) - .detail("Range", range); - throw remote.error.get(); - } - - TraceEvent(SevVerbose, "ValidateRangeGetRemoteKeyValuesResult", data->thisServerID) - .detail("Range", range) - .detail("RemoteResultSize", remote.data.size()); - const int end = std::min(local.data.size(), remote.data.size()); int i = 0; for (; i < end; ++i) { diff --git a/fdbserver/workloads/ValidateStorage.actor.cpp b/fdbserver/workloads/ValidateStorage.actor.cpp index 5cbe8933aa..37fdaea671 100644 --- a/fdbserver/workloads/ValidateStorage.actor.cpp +++ b/fdbserver/workloads/ValidateStorage.actor.cpp @@ -179,67 +179,6 @@ struct ValidateStorage : TestWorkload { return Void(); } - ACTOR Future readAndVerify(ValidateStorage* self, - Database cx, - Key key, - ErrorOr> expectedValue) { - state Transaction tr(cx); - tr.setOption(FDBTransactionOptions::ACCESS_SYSTEM_KEYS); - - loop { - try { - state Version readVersion = wait(tr.getReadVersion()); - state Optional res = wait(timeoutError(tr.get(key), 30.0)); - const bool equal = !expectedValue.isError() && res == expectedValue.get(); - if (!equal) { - self->validationFailed(expectedValue, ErrorOr>(res)); - } - break; - } catch (Error& e) { - TraceEvent("TestReadError").errorUnsuppressed(e); - if (expectedValue.isError() && expectedValue.getError().code() == e.code()) { - break; - } - wait(tr.onError(e)); - } - } - - TraceEvent("TestReadSuccess").detail("Version", readVersion); - - return Void(); - } - - ACTOR Future writeAndVerify(ValidateStorage* self, Database cx, Key key, Optional value) { - state Reference tr = makeReference(cx); - state Version version; - loop { - state UID debugID = deterministicRandom()->randomUniqueID(); - try { - tr->setOption(FDBTransactionOptions::ACCESS_SYSTEM_KEYS); - tr->debugTransaction(debugID); - if (value.present()) { - tr->set(key, value.get()); - tr->set("Test?"_sr, value.get()); - tr->set(key, value.get()); - } else { - tr->clear(key); - } - wait(timeoutError(tr->commit(), 30.0)); - version = tr->getCommittedVersion(); - break; - } catch (Error& e) { - TraceEvent("TestCommitError").errorUnsuppressed(e); - wait(tr->onError(e)); - } - } - - TraceEvent("TestCommitSuccess").detail("CommitVersion", tr->getCommittedVersion()).detail("DebugID", debugID); - - wait(self->readAndVerify(self, cx, key, value)); - - return version; - } - Future check(Database const& cx) override { return true; } void getMetrics(std::vector& m) override {} diff --git a/flow/include/flow/error_definitions.h b/flow/include/flow/error_definitions.h index ad4537146c..fb2b7ad3e2 100755 --- a/flow/include/flow/error_definitions.h +++ b/flow/include/flow/error_definitions.h @@ -130,6 +130,7 @@ ERROR( restart_cluster_controller, 1218, "Restart cluster controller process" ) ERROR( please_reboot_kv_store, 1219, "Need to reboot the storage engine") ERROR( incompatible_software_version, 1220, "Current software does not support database format" ) ERROR( audit_storage_failed, 1221, "Validate storage consistency operation failed" ) +ERROR( audit_storage_exeed_max, 1222, "Exceeded the max number of allowed concurrent audit storage requests" ) // 15xx Platform errors ERROR( platform_error, 1500, "Platform error" ) From b887b1e85cdd9442760e9c45b825614781d211e9 Mon Sep 17 00:00:00 2001 From: sfc-gh-tclinkenbeard Date: Mon, 10 Oct 2022 12:28:26 -0700 Subject: [PATCH 045/210] Limit number of tags tracked by GlobalTagThrottler --- fdbclient/ServerKnobs.cpp | 2 + fdbclient/include/fdbclient/ServerKnobs.h | 6 ++ fdbserver/GlobalTagThrottler.actor.cpp | 104 ++++++++++++++++++++- fdbserver/include/fdbserver/TagThrottler.h | 2 + 4 files changed, 109 insertions(+), 5 deletions(-) diff --git a/fdbclient/ServerKnobs.cpp b/fdbclient/ServerKnobs.cpp index 21970e4eaf..b2f27cea10 100644 --- a/fdbclient/ServerKnobs.cpp +++ b/fdbclient/ServerKnobs.cpp @@ -721,6 +721,8 @@ void ServerKnobs::initialize(Randomize randomize, ClientKnobs* clientKnobs, IsSi init( GLOBAL_TAG_THROTTLING_MIN_RATE, 1.0 ); init( GLOBAL_TAG_THROTTLING_FOLDING_TIME, 10.0 ); init( GLOBAL_TAG_THROTTLING_RW_FUNGIBILITY_RATIO, 5.0 ); + init( GLOBAL_TAG_THROTTLING_MAX_TAGS_TRACKED, 10 ); + init( GLOBAL_TAG_THROTTLING_TAG_EXPIRE_AFTER, 240.0 ); //Storage Metrics init( STORAGE_METRICS_AVERAGE_INTERVAL, 120.0 ); diff --git a/fdbclient/include/fdbclient/ServerKnobs.h b/fdbclient/include/fdbclient/ServerKnobs.h index 3ca647aa9b..485b0bf02b 100644 --- a/fdbclient/include/fdbclient/ServerKnobs.h +++ b/fdbclient/include/fdbclient/ServerKnobs.h @@ -620,6 +620,12 @@ public: double GLOBAL_TAG_THROTTLING_FOLDING_TIME; // Cost multiplier for writes (because write operations are more expensive than reads) double GLOBAL_TAG_THROTTLING_RW_FUNGIBILITY_RATIO; + // Maximum number of tags tracked by global tag throttler. Additional tags will be ignored + // until some existing tags expire + int64_t GLOBAL_TAG_THROTTLING_MAX_TAGS_TRACKED; + // Global tag throttler forgets about throughput from a tag once no new transactions from that + // tag have been received for this duration (in seconds): + int64_t GLOBAL_TAG_THROTTLING_TAG_EXPIRE_AFTER; double MAX_TRANSACTIONS_PER_BYTE; diff --git a/fdbserver/GlobalTagThrottler.actor.cpp b/fdbserver/GlobalTagThrottler.actor.cpp index c1ec99541f..c17a3c713c 100644 --- a/fdbserver/GlobalTagThrottler.actor.cpp +++ b/fdbserver/GlobalTagThrottler.actor.cpp @@ -120,12 +120,13 @@ class GlobalTagThrottlerImpl { Smoother transactionCounter; Smoother perClientRate; Smoother targetRate; + double transactionsLastAdded; public: explicit PerTagStatistics() : transactionCounter(SERVER_KNOBS->GLOBAL_TAG_THROTTLING_FOLDING_TIME), perClientRate(SERVER_KNOBS->GLOBAL_TAG_THROTTLING_FOLDING_TIME), - targetRate(SERVER_KNOBS->GLOBAL_TAG_THROTTLING_FOLDING_TIME) {} + targetRate(SERVER_KNOBS->GLOBAL_TAG_THROTTLING_FOLDING_TIME), transactionsLastAdded(now()) {} Optional getQuota() const { return quota; } @@ -133,7 +134,10 @@ class GlobalTagThrottlerImpl { void clearQuota() { quota = {}; } - void addTransactions(int count) { transactionCounter.addDelta(count); } + void addTransactions(int count) { + transactionsLastAdded = now(); + transactionCounter.addDelta(count); + } double getTransactionRate() const { return transactionCounter.smoothRate(); } @@ -151,6 +155,10 @@ class GlobalTagThrottlerImpl { targetRate.setTotal(targetTps); return targetRate.smoothTotal(); } + + bool recentTransactionsAdded() const { + return now() - transactionsLastAdded < SERVER_KNOBS->GLOBAL_TAG_THROTTLING_TAG_EXPIRE_AFTER; + } }; Database db; @@ -360,6 +368,7 @@ class GlobalTagThrottlerImpl { tagsWithQuota.insert(tag); } self->removeUnseenQuotas(tagsWithQuota); + self->removeExpiredTags(); ++self->throttledTagChangeId; wait(delay(5.0)); break; @@ -397,7 +406,20 @@ class GlobalTagThrottlerImpl { public: GlobalTagThrottlerImpl(Database db, UID id) : db(db), id(id) {} Future monitorThrottlingChanges() { return monitorThrottlingChanges(this); } - void addRequests(TransactionTag tag, int count) { tagStatistics[tag].addTransactions(static_cast(count)); } + void addRequests(TransactionTag tag, int count) { + auto it = tagStatistics.find(tag); + if (it == tagStatistics.end()) { + if (tagStatistics.size() == SERVER_KNOBS->GLOBAL_TAG_THROTTLING_MAX_TAGS_TRACKED) { + CODE_PROBE(true, + "Global tag throttler ignoring transactions because maximum number of trackable tags has " + "been reached"); + } else { + tagStatistics[tag].addTransactions(static_cast(count)); + } + } else { + it->second.addTransactions(static_cast(count)); + } + } uint64_t getThrottledTagChangeId() const { return throttledTagChangeId; } PrioritizedTransactionTagMap getProxyRates(int numProxies) { @@ -466,10 +488,14 @@ public: throttlingRatios[ss.id] = ss.getThrottlingRatio(SERVER_KNOBS->TARGET_BYTES_PER_STORAGE_SERVER, SERVER_KNOBS->SPRING_BYTES_STORAGE_SERVER); for (const auto& busyReadTag : ss.busiestReadTags) { - throughput[ss.id][busyReadTag.tag].updateCost(busyReadTag.rate, OpType::READ); + if (tagStatistics.find(busyReadTag.tag) != tagStatistics.end()) { + throughput[ss.id][busyReadTag.tag].updateCost(busyReadTag.rate, OpType::READ); + } } for (const auto& busyWriteTag : ss.busiestWriteTags) { - throughput[ss.id][busyWriteTag.tag].updateCost(busyWriteTag.rate, OpType::WRITE); + if (tagStatistics.find(busyWriteTag.tag) != tagStatistics.end()) { + throughput[ss.id][busyWriteTag.tag].updateCost(busyWriteTag.rate, OpType::WRITE); + } } return Void(); } @@ -479,6 +505,22 @@ public: } void removeQuota(TransactionTagRef tag) { tagStatistics[tag].clearQuota(); } + + void removeExpiredTags() { + for (auto it = tagStatistics.begin(); it != tagStatistics.end();) { + const auto& [tag, stats] = *it; + if (!stats.recentTransactionsAdded()) { + for (auto& [ss, tagToCounters] : throughput) { + tagToCounters.erase(tag); + } + it = tagStatistics.erase(it); + } else { + ++it; + } + } + } + + uint32_t tagsTracked() const { return tagStatistics.size(); } }; GlobalTagThrottler::GlobalTagThrottler(Database db, UID id) : impl(PImpl::create(db, id)) {} @@ -527,6 +569,14 @@ void GlobalTagThrottler::removeQuota(TransactionTagRef tag) { return impl->removeQuota(tag); } +uint32_t GlobalTagThrottler::tagsTracked() const { + return impl->tagsTracked(); +} + +void GlobalTagThrottler::removeExpiredTags() { + return impl->removeExpiredTags(); +} + namespace GlobalTagThrottlerTesting { enum class LimitType { RESERVED, TOTAL }; @@ -1029,3 +1079,47 @@ TEST_CASE("/GlobalTagThrottler/ReservedQuota") { wait(timeoutError(monitor || client || updater, 600.0)); return Void(); } + +// Test that tags are expired iff a sufficient amount of time has passed since the +// last transaction with that tag +TEST_CASE("/GlobalTagThrottler/ExpireTags") { + state GlobalTagThrottler globalTagThrottler(Database{}, UID{}); + state GlobalTagThrottlerTesting::StorageServerCollection storageServers(10, 5); + TransactionTag testTag = "sampleTag1"_sr; + + state Future client = + timeout(GlobalTagThrottlerTesting::runClient( + &globalTagThrottler, &storageServers, testTag, 10.0, 6.0, GlobalTagThrottlerTesting::OpType::READ), + 60.0, + Void()); + state Future updater = timeout( + GlobalTagThrottlerTesting::updateGlobalTagThrottler(&globalTagThrottler, &storageServers), 60.0, Void()); + wait(client && updater); + client.cancel(); + updater.cancel(); + ASSERT_EQ(globalTagThrottler.tagsTracked(), 1); + globalTagThrottler.removeExpiredTags(); + ASSERT_EQ(globalTagThrottler.tagsTracked(), 1); + wait(delay(SERVER_KNOBS->GLOBAL_TAG_THROTTLING_TAG_EXPIRE_AFTER + 1.0)); + ASSERT_EQ(globalTagThrottler.tagsTracked(), 1); + globalTagThrottler.removeExpiredTags(); + ASSERT_EQ(globalTagThrottler.tagsTracked(), 0); + return Void(); +} + +// Test that +TEST_CASE("/GlobalTagThrottler/TagLimit") { + state GlobalTagThrottler globalTagThrottler(Database{}, UID{}); + state GlobalTagThrottlerTesting::StorageServerCollection storageServers(10, 5); + std::vector> futures; + for (int i = 0; i < 2 * SERVER_KNOBS->GLOBAL_TAG_THROTTLING_MAX_TAGS_TRACKED; ++i) { + Arena arena; + TransactionTag tag = makeString(8, arena); + deterministicRandom()->randomBytes(mutateString(tag), tag.size()); + futures.push_back(GlobalTagThrottlerTesting::runClient( + &globalTagThrottler, &storageServers, tag, 1.0, 6.0, GlobalTagThrottlerTesting::OpType::READ)); + } + wait(timeout(waitForAll(futures), 60.0, Void())); + ASSERT_EQ(globalTagThrottler.tagsTracked(), SERVER_KNOBS->GLOBAL_TAG_THROTTLING_MAX_TAGS_TRACKED); + return Void(); +} diff --git a/fdbserver/include/fdbserver/TagThrottler.h b/fdbserver/include/fdbserver/TagThrottler.h index 5e00e7be46..ac4dc1adbe 100644 --- a/fdbserver/include/fdbserver/TagThrottler.h +++ b/fdbserver/include/fdbserver/TagThrottler.h @@ -100,4 +100,6 @@ public: public: void setQuota(TransactionTagRef, ThrottleApi::TagQuotaValue const&); void removeQuota(TransactionTagRef); + void removeExpiredTags(); + uint32_t tagsTracked() const; }; From b930a8a38042d7fcb1b67c523917e4807a41624b Mon Sep 17 00:00:00 2001 From: sfc-gh-tclinkenbeard Date: Mon, 10 Oct 2022 12:51:27 -0700 Subject: [PATCH 046/210] Fix bug in GlobalTagThrottlerImpl::getQuotaRatio --- fdbserver/GlobalTagThrottler.actor.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/fdbserver/GlobalTagThrottler.actor.cpp b/fdbserver/GlobalTagThrottler.actor.cpp index c17a3c713c..dbf4f5d976 100644 --- a/fdbserver/GlobalTagThrottler.actor.cpp +++ b/fdbserver/GlobalTagThrottler.actor.cpp @@ -286,7 +286,7 @@ class GlobalTagThrottlerImpl { for (const auto& t : tagsAffectingStorageServer) { auto const tQuota = getQuota(t, LimitType::TOTAL); sumQuota += tQuota.orDefault(0); - if (tag.compare(tag) == 0) { + if (t.compare(tag) == 0) { tagQuota = tQuota.orDefault(0); } } From 8ed2b85427edcb742cc10cb592679db36bfcd7ae Mon Sep 17 00:00:00 2001 From: Ankita Kejriwal Date: Mon, 10 Oct 2022 18:40:31 -0700 Subject: [PATCH 047/210] Reduce workload flakiness. * Scale up the workload so that the size for the tenant is large enough so that the estimated size will not be rounded down to 0. * Retry getting the size in the workload (with a delay, upto a maximum time) to allow for the size estimate to catch up. --- .../workloads/GetEstimatedRangeSize.actor.cpp | 35 +++++++++++++++---- tests/fast/GetEstimatedRangeSize.toml | 2 +- 2 files changed, 29 insertions(+), 8 deletions(-) diff --git a/fdbserver/workloads/GetEstimatedRangeSize.actor.cpp b/fdbserver/workloads/GetEstimatedRangeSize.actor.cpp index 471d887c6c..885cab6ebe 100644 --- a/fdbserver/workloads/GetEstimatedRangeSize.actor.cpp +++ b/fdbserver/workloads/GetEstimatedRangeSize.actor.cpp @@ -42,7 +42,7 @@ struct GetEstimatedRangeSizeWorkload : TestWorkload { tenant = getOption(options, "tenant"_sr, "DefaultTenant"_sr); } - std::string description() const override { return "GetEstimatedRangeSizeWorkload"; } + std::string description() const override { return "GetEstimatedRangeSize"; } Future setup(Database const& cx) override { if (!hasTenant) { @@ -88,21 +88,34 @@ struct GetEstimatedRangeSizeWorkload : TestWorkload { // We use a wide range to avoid flakiness because the underlying function (being tested) // is making an estimation. if (!self->hasTenant) { - ASSERT_GT(size, 9393000 / 5); - ASSERT_LT(size, 9393000 * 5); + ASSERT_GT(size, 10230000 / 5); + ASSERT_LT(size, 10230000 * 5); } else if (self->tenant == "First"_sr) { ASSERT_GT(size, 8525000 / 5); ASSERT_LT(size, 8525000 * 5); } else if (self->tenant == "Second"_sr) { - ASSERT_GT(size, 93000 / 5); - ASSERT_LT(size, 93000 * 5); + ASSERT_GT(size, 930000 / 5); + ASSERT_LT(size, 930000 * 5); } + ASSERT_GE(size, 0); return Void(); } + static bool sizeIsAsExpected(int64_t size, Optional tenant) { + if (!tenant.present()) { + return size > 10230000 / 5 && size < 10230000 * 5; + } else if (tenant == "First"_sr) { + return size > 8525000 / 5 && size < 8525000 * 5; + } else if (tenant == "Second"_sr) { + return size > 930000 / 5 && size < 930000 * 5; + } + return false; + } + ACTOR static Future getSize(GetEstimatedRangeSizeWorkload* self, Database cx) { state Optional tenant = self->hasTenant ? self->tenant : Optional(); state ReadYourWritesTransaction tr(cx, tenant); + state double totalDelay = 0.0; TraceEvent(SevDebug, "GetSize1").detail("Tenant", tr.getTenant().present() ? tr.getTenant().get() : "none"_sr); loop { @@ -111,9 +124,17 @@ struct GetEstimatedRangeSizeWorkload : TestWorkload { TraceEvent(SevDebug, "GetSize2") .detail("Tenant", tr.getTenant().present() ? tr.getTenant().get() : "none"_sr) .detail("Size", size); - tr.reset(); - return size; + if (!sizeIsAsExpected(size, tenant) && totalDelay < 300.0) { + totalDelay += 5.0; + wait(delay(5.0)); + } else { + tr.reset(); + return size; + } } catch (Error& e) { + TraceEvent(SevDebug, "GetSizeError") + .errorUnsuppressed(e) + .detail("Tenant", tr.getTenant().present() ? tr.getTenant().get() : "none"_sr); wait(tr.onError(e)); } } diff --git a/tests/fast/GetEstimatedRangeSize.toml b/tests/fast/GetEstimatedRangeSize.toml index 0c9503cf33..48a6edcdee 100644 --- a/tests/fast/GetEstimatedRangeSize.toml +++ b/tests/fast/GetEstimatedRangeSize.toml @@ -24,7 +24,7 @@ testTitle = 'GetEstimatedRangeSizeTest' [[test.workload]] testName = 'GetEstimatedRangeSize' tenant = 'Second' - nodeCount = 2500.0 + nodeCount = 25000.0 [[test.workload]] testName = 'GetEstimatedRangeSize' From 12aaabb283553493a253cf91a2c7d1cfb5c712d3 Mon Sep 17 00:00:00 2001 From: Ankita Kejriwal Date: Mon, 10 Oct 2022 18:52:03 -0700 Subject: [PATCH 048/210] Fix bugs in finding correct shards for tenant aware storage metrics. * Account for tenant in during cache invalidation in NativeAPI actor. * Account for tenant prefix while checking whether a shard is readable on storage server. This commit also adds many debug TraceEvents that will later be removed. --- fdbclient/NativeAPI.actor.cpp | 102 +++++++++++++++++++++--------- fdbserver/storageserver.actor.cpp | 32 ++++++++-- 2 files changed, 98 insertions(+), 36 deletions(-) diff --git a/fdbclient/NativeAPI.actor.cpp b/fdbclient/NativeAPI.actor.cpp index 48ae9b96a2..089640cc77 100644 --- a/fdbclient/NativeAPI.actor.cpp +++ b/fdbclient/NativeAPI.actor.cpp @@ -32,6 +32,7 @@ #include #include "boost/algorithm/string.hpp" +#include "fdbclient/Tenant.h" #include "fdbrpc/TenantInfo.h" #include "fmt/format.h" @@ -7249,33 +7250,59 @@ ACTOR Future getStorageMetricsLargeKeyRange(Database cx, ACTOR Future doGetStorageMetrics(Database cx, KeyRange keys, Reference locationInfo, - Optional> trState) { + TenantMapEntry tenantEntry, + Optional> trState, + UID debugId) { state TenantInfo tenantInfo = wait(trState.present() ? populateAndGetTenant(trState.get(), keys.begin, latestVersion) : TenantInfo()); - loop { - try { - WaitMetricsRequest req(tenantInfo, keys, StorageMetrics(), StorageMetrics()); - req.min.bytes = 0; - req.max.bytes = -1; - StorageMetrics m = wait(loadBalance( - locationInfo->locations(), &StorageServerInterface::waitMetrics, req, TaskPriority::DataDistribution)); - return m; - } catch (Error& e) { - if (e.code() != error_code_wrong_shard_server && e.code() != error_code_all_alternatives_failed) { - TraceEvent(SevError, "WaitStorageMetricsError").error(e); - throw; - } - wait(delay(CLIENT_KNOBS->WRONG_SHARD_SERVER_DELAY, TaskPriority::DataDistribution)); - cx->invalidateCache(Key(), keys); - StorageMetrics m = wait(getStorageMetricsLargeKeyRange(cx, keys, trState)); - return m; + try { + WaitMetricsRequest req(tenantInfo, keys, StorageMetrics(), StorageMetrics()); + req.min.bytes = 0; + req.max.bytes = -1; + // if (tenantInfo.tenantId != TenantInfo::INVALID_TENANT) { + // TraceEvent(SevWarn, "AKNative0", debugId) + // .detail("TenantId", tenantInfo.tenantId) + // .detail("TenantName", tenantInfo.name.present() ? tenantInfo.name.get() : "not present"_sr) + // .detail("Keys", keys.toString()); + // } + StorageMetrics m = wait(loadBalance( + locationInfo->locations(), &StorageServerInterface::waitMetrics, req, TaskPriority::DataDistribution)); + // if (tenantInfo.tenantId != TenantInfo::INVALID_TENANT) { + // TraceEvent(SevWarn, "AKNative1", debugId) + // .detail("TenantId", tenantInfo.tenantId) + // .detail("TenantName", tenantInfo.name.present() ? tenantInfo.name.get() : "not present"_sr) + // .detail("Metrics", m.toString()); + // } + return m; + } catch (Error& e) { + if (e.code() != error_code_wrong_shard_server && e.code() != error_code_all_alternatives_failed) { + TraceEvent(SevError, "WaitStorageMetricsError").error(e); + throw; } + wait(delay(CLIENT_KNOBS->WRONG_SHARD_SERVER_DELAY, TaskPriority::DataDistribution)); + cx->invalidateCache(tenantEntry.prefix, keys); + // TODO: Is this needed? + if (tenantInfo.name.present()) { + cx->invalidateCachedTenant(tenantInfo.name.get()); + } + + // if (tenantInfo.tenantId != TenantInfo::INVALID_TENANT) { + // TraceEvent(SevWarn, "AKNative2", debugId) + // .detail("TenantId", tenantInfo.tenantId) + // .detail("TenantName", tenantInfo.name.present() ? tenantInfo.name.get() : "not present"_sr) + // .detail("TenantPrefix", tenantEntry.prefix) + // .detail("Keys", keys.toString()); + // } + StorageMetrics m = wait(getStorageMetricsLargeKeyRange(cx, keys, trState)); + return m; } } ACTOR Future getStorageMetricsLargeKeyRange(Database cx, KeyRange keys, Optional> trState) { + state UID debugId = debugRandom()->randomUniqueID(); + // TraceEvent(SevWarn, "AKNative25", debugId).detail("Keys", keys.toString()); state Span span("NAPI:GetStorageMetricsLargeKeyRange"_loc); state TenantInfo tenantInfo = wait(trState.present() ? populateAndGetTenant(trState.get(), keys.begin, latestVersion) : TenantInfo()); @@ -7296,9 +7323,21 @@ ACTOR Future getStorageMetricsLargeKeyRange(Database cx, for (int i = 0; i < nLocs; i++) { partBegin = (i == 0) ? keys.begin : locations[i].range.begin; partEnd = (i == nLocs - 1) ? keys.end : locations[i].range.end; - fx[i] = doGetStorageMetrics(cx, KeyRangeRef(partBegin, partEnd), locations[i].locations, trState); + fx[i] = doGetStorageMetrics( + cx, KeyRangeRef(partBegin, partEnd), locations[i].locations, locations[i].tenantEntry, trState, debugId); } + // if (tenantInfo.tenantId != TenantInfo::INVALID_TENANT) { + // TraceEvent(SevWarn, "AKNative3", debugId) + // .detail("TenantId", tenantInfo.tenantId) + // .detail("TenantName", tenantInfo.name.present() ? tenantInfo.name.get() : "not present"_sr) + // .detail("NumLocs", nLocs); + // } wait(waitForAll(fx)); + // if (tenantInfo.tenantId != TenantInfo::INVALID_TENANT) { + // TraceEvent(SevWarn, "AKNative4", debugId) + // .detail("TenantId", tenantInfo.tenantId) + // .detail("TenantName", tenantInfo.name.present() ? tenantInfo.name.get() : "not present"_sr); + // } for (int i = 0; i < nLocs; i++) { total += fx[i].get(); } @@ -7458,16 +7497,17 @@ ACTOR Future, int>> waitStorageMetrics( state TenantInfo tenantInfo = wait(trState.present() ? populateAndGetTenant(trState.get(), keys.begin, latestVersion) : TenantInfo()); loop { - std::vector locations = wait(getKeyRangeLocations(cx, - tenantInfo, - keys, - shardLimit, - Reverse::False, - &StorageServerInterface::waitMetrics, - span.context, - Optional(), - UseProvisionalProxies::False, - latestVersion)); + state std::vector locations = + wait(getKeyRangeLocations(cx, + tenantInfo, + keys, + shardLimit, + Reverse::False, + &StorageServerInterface::waitMetrics, + span.context, + Optional(), + UseProvisionalProxies::False, + latestVersion)); if (expectedShardCount >= 0 && locations.size() != expectedShardCount) { return std::make_pair(Optional(), locations.size()); } @@ -7493,7 +7533,7 @@ ACTOR Future, int>> waitStorageMetrics( TraceEvent(SevError, "WaitStorageMetricsError").error(e); throw; } - cx->invalidateCache(Key(), keys); + cx->invalidateCache(locations[0].tenantEntry.prefix, keys); wait(delay(CLIENT_KNOBS->WRONG_SHARD_SERVER_DELAY, TaskPriority::DataDistribution)); } } else { @@ -7503,7 +7543,7 @@ ACTOR Future, int>> waitStorageMetrics( .detail("JitteredSecondsOfPenitence", CLIENT_KNOBS->STORAGE_METRICS_TOO_MANY_SHARDS_DELAY); wait(delayJittered(CLIENT_KNOBS->STORAGE_METRICS_TOO_MANY_SHARDS_DELAY, TaskPriority::DataDistribution)); // make sure that the next getKeyRangeLocations() call will actually re-fetch the range - cx->invalidateCache(Key(), keys); + cx->invalidateCache(locations[0].tenantEntry.prefix, keys); } } } diff --git a/fdbserver/storageserver.actor.cpp b/fdbserver/storageserver.actor.cpp index eba7316563..93d12c4051 100644 --- a/fdbserver/storageserver.actor.cpp +++ b/fdbserver/storageserver.actor.cpp @@ -9514,13 +9514,17 @@ ACTOR Future waitMetrics(StorageServerMetrics* self, Future timeout, Optional tenantPrefix) { state PromiseStream change; - if (tenantPrefix.present()) { - req.keys = req.keys.withPrefix(tenantPrefix.get()); - } state StorageMetrics metrics = self->getMetrics(req.keys); state Error error = success(); state bool timedout = false; + // if (tenantPrefix.present()) { + // TraceEvent(SevDebug, "AKWaitMetricsResult") + // .detail("Preifx", tenantPrefix.get()) + // .detail("Metrics", metrics.toString()) + // .detail("QuickCondition", (!req.min.allLessOrEqual(metrics) || !metrics.allLessOrEqual(req.max))); + // } + if (!req.min.allLessOrEqual(metrics) || !metrics.allLessOrEqual(req.max)) { CODE_PROBE(true, "ShardWaitMetrics return case 1 (quickly)"); req.reply.send(metrics); @@ -9623,7 +9627,20 @@ ACTOR Future waitMetricsTenantAware(StorageServer* self, WaitMetricsReques wait(success(waitForVersionNoTooOld(self, latestVersion))); Optional entry = self->getTenantEntry(latestVersion, req.tenantInfo); Optional tenantPrefix = entry.map([](TenantMapEntry e) { return e.prefix; }); - wait(self->metrics.waitMetrics(req, delayJittered(SERVER_KNOBS->STORAGE_METRIC_TIMEOUT), tenantPrefix)); + if (tenantPrefix.present()) { + req.keys = req.keys.withPrefix(tenantPrefix.get()); + } + if (!self->isReadable(req.keys)) { + CODE_PROBE(true, "waitMetricsTenantAware wrong_shard_server()"); + // TraceEvent(SevWarn, "AKWaitMetricsRequestWrongShard2") + // .detail("ServerID", self->thisServerID) + // .detail("TenantId", req.tenantInfo.tenantId) + // .detail("Keys", req.keys.toString()); + + self->sendErrorWithPenalty(req.reply, wrong_shard_server(), self->getPenalty()); + } else { + wait(self->metrics.waitMetrics(req, delayJittered(SERVER_KNOBS->STORAGE_METRIC_TIMEOUT), tenantPrefix)); + } return Void(); } @@ -9664,8 +9681,13 @@ ACTOR Future metricsCore(StorageServer* self, StorageServerInterface ssi) loop { choose { when(state WaitMetricsRequest req = waitNext(ssi.waitMetrics.getFuture())) { - if (!self->isReadable(req.keys)) { + if (req.tenantInfo.tenantId == -1 && !self->isReadable(req.keys)) { CODE_PROBE(true, "waitMetrics immediate wrong_shard_server()"); + // TraceEvent(SevWarn, "AKWaitMetricsRequestWrongShard1") + // .detail("ServerID", self->thisServerID) + // .detail("TenantId", req.tenantInfo.tenantId) + // .detail("Keys", req.keys.toString()); + self->sendErrorWithPenalty(req.reply, wrong_shard_server(), self->getPenalty()); } else { self->actors.add(waitMetricsTenantAware(self, req)); From 4c973c11ad00014c4de334b59c3d6818bab9f9d5 Mon Sep 17 00:00:00 2001 From: sfc-gh-tclinkenbeard Date: Mon, 3 Oct 2022 10:09:09 -0700 Subject: [PATCH 049/210] Modularize and comment GrvTransactionRateInfo --- fdbserver/GrvProxyServer.actor.cpp | 100 ++------------- fdbserver/GrvTransactionRateInfo.actor.cpp | 118 ++++++++++++++++++ .../fdbserver/GrvTransactionRateInfo.h | 69 ++++++++++ 3 files changed, 199 insertions(+), 88 deletions(-) create mode 100644 fdbserver/GrvTransactionRateInfo.actor.cpp create mode 100644 fdbserver/include/fdbserver/GrvTransactionRateInfo.h diff --git a/fdbserver/GrvProxyServer.actor.cpp b/fdbserver/GrvProxyServer.actor.cpp index 789b2973b1..c25df2cd1a 100644 --- a/fdbserver/GrvProxyServer.actor.cpp +++ b/fdbserver/GrvProxyServer.actor.cpp @@ -24,11 +24,12 @@ #include "fdbclient/Notified.h" #include "fdbclient/TransactionLineage.h" #include "fdbclient/Tuple.h" -#include "fdbserver/LogSystem.h" -#include "fdbserver/LogSystemDiskQueueAdapter.h" #include "fdbclient/CommitProxyInterface.h" #include "fdbclient/GrvProxyInterface.h" #include "fdbclient/VersionVector.h" +#include "fdbserver/GrvTransactionRateInfo.h" +#include "fdbserver/LogSystem.h" +#include "fdbserver/LogSystemDiskQueueAdapter.h" #include "fdbserver/WaitFailure.h" #include "fdbserver/WorkerInterface.actor.h" #include "fdbrpc/sim_validation.h" @@ -158,83 +159,6 @@ struct GrvProxyStats { } }; -struct GrvTransactionRateInfo { - double rate; - double limit; - double budget; - - bool disabled; - - Smoother smoothRate; - Smoother smoothReleased; - - GrvTransactionRateInfo(double rate = 0.0) - : rate(rate), limit(0), budget(0), disabled(true), smoothRate(SERVER_KNOBS->START_TRANSACTION_RATE_WINDOW), - smoothReleased(SERVER_KNOBS->START_TRANSACTION_RATE_WINDOW) {} - - void reset() { - // Determine the number of transactions that this proxy is allowed to release - // Roughly speaking, this is done by computing the number of transactions over some historical window that we - // could have started but didn't, and making that our limit. More precisely, we track a smoothed rate limit and - // release rate, the difference of which is the rate of additional transactions that we could have released - // based on that window. Then we multiply by the window size to get a number of transactions. - // - // Limit can be negative in the event that we are releasing more transactions than we are allowed (due to the - // use of our budget or because of higher priority transactions). - double releaseRate = smoothRate.smoothTotal() - smoothReleased.smoothRate(); - limit = SERVER_KNOBS->START_TRANSACTION_RATE_WINDOW * releaseRate; - } - - bool canStart(int64_t numAlreadyStarted, int64_t count) const { - return numAlreadyStarted + count <= - std::min(limit + budget, SERVER_KNOBS->START_TRANSACTION_MAX_TRANSACTIONS_TO_START); - } - - void updateBudget(int64_t numStartedAtPriority, bool queueEmptyAtPriority, double elapsed) { - // Update the budget to accumulate any extra capacity available or remove any excess that was used. - // The actual delta is the portion of the limit we didn't use multiplied by the fraction of the window that - // elapsed. - // - // We may have exceeded our limit due to the budget or because of higher priority transactions, in which case - // this delta will be negative. The delta can also be negative in the event that our limit was negative, which - // can happen if we had already started more transactions in our window than our rate would have allowed. - // - // This budget has the property that when the budget is required to start transactions (because batches are - // big), the sum limit+budget will increase linearly from 0 to the batch size over time and decrease by the - // batch size upon starting a batch. In other words, this works equivalently to a model where we linearly - // accumulate budget over time in the case that our batches are too big to take advantage of the window based - // limits. - budget = std::max( - 0.0, budget + elapsed * (limit - numStartedAtPriority) / SERVER_KNOBS->START_TRANSACTION_RATE_WINDOW); - - // If we are emptying out the queue of requests, then we don't need to carry much budget forward - // If we did keep accumulating budget, then our responsiveness to changes in workflow could be compromised - if (queueEmptyAtPriority) { - budget = std::min(budget, SERVER_KNOBS->START_TRANSACTION_MAX_EMPTY_QUEUE_BUDGET); - } - - smoothReleased.addDelta(numStartedAtPriority); - } - - void disable() { - disabled = true; - // Use smoothRate.setTotal(0) instead of setting rate to 0 so txns will not be throttled immediately. - smoothRate.setTotal(0); - } - - void setRate(double rate) { - ASSERT(rate >= 0 && rate != std::numeric_limits::infinity() && !std::isnan(rate)); - - this->rate = rate; - if (disabled) { - smoothRate.reset(rate); - disabled = false; - } else { - smoothRate.setTotal(rate); - } - } -}; - struct GrvProxyData { GrvProxyInterface proxy; UID dbgid; @@ -622,7 +546,7 @@ ACTOR Future queueGetReadVersionRequests( } else { // Return error for batch_priority GRV requests int64_t proxiesCount = std::max((int)db->get().client.grvProxies.size(), 1); - if (batchRateInfo->rate <= (1.0 / proxiesCount)) { + if (batchRateInfo->getRate() <= (1.0 / proxiesCount)) { req.reply.sendError(batch_transaction_throttled()); stats->txnThrottled += req.transactionCount; } else { @@ -960,11 +884,11 @@ ACTOR static Future transactionStarter(GrvProxyInterface proxy, elapsed = 1e-15; } - normalRateInfo.reset(); - batchRateInfo.reset(); + normalRateInfo.startEpoch(); + batchRateInfo.startEpoch(); - grvProxyData->stats.transactionLimit = normalRateInfo.limit; - grvProxyData->stats.batchTransactionLimit = batchRateInfo.limit; + grvProxyData->stats.transactionLimit = normalRateInfo.getLimit(); + grvProxyData->stats.batchTransactionLimit = batchRateInfo.getLimit(); int transactionsStarted[2] = { 0, 0 }; int systemTransactionsStarted[2] = { 0, 0 }; @@ -1071,11 +995,11 @@ ACTOR static Future transactionStarter(GrvProxyInterface proxy, transactionCount += transactionsStarted[0] + transactionsStarted[1]; batchTransactionCount += batchTotalStarted; - normalRateInfo.updateBudget( + normalRateInfo.endEpoch( systemTotalStarted + normalTotalStarted, systemQueue.empty() && defaultQueue.empty(), elapsed); - batchRateInfo.updateBudget(systemTotalStarted + normalTotalStarted + batchTotalStarted, - systemQueue.empty() && defaultQueue.empty() && batchQueue.empty(), - elapsed); + batchRateInfo.endEpoch(systemTotalStarted + normalTotalStarted + batchTotalStarted, + systemQueue.empty() && defaultQueue.empty() && batchQueue.empty(), + elapsed); if (debugID.present()) { g_traceBatch.addEvent("TransactionDebug", diff --git a/fdbserver/GrvTransactionRateInfo.actor.cpp b/fdbserver/GrvTransactionRateInfo.actor.cpp new file mode 100644 index 0000000000..b676f3afb1 --- /dev/null +++ b/fdbserver/GrvTransactionRateInfo.actor.cpp @@ -0,0 +1,118 @@ +/* + * GrvTransactionRateInfo.actor.cpp + * + * This source file is part of the FoundationDB open source project + * + * Copyright 2013-2022 Apple Inc. and the FoundationDB project authors + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +#include "fdbserver/GrvTransactionRateInfo.h" + +#include "fdbserver/Knobs.h" +#include "flow/UnitTest.h" +#include "flow/actorcompiler.h" // must be last include + +GrvTransactionRateInfo::GrvTransactionRateInfo(double rate) + : rate(rate), smoothRate(SERVER_KNOBS->START_TRANSACTION_RATE_WINDOW), + smoothReleased(SERVER_KNOBS->START_TRANSACTION_RATE_WINDOW) {} + +bool GrvTransactionRateInfo::canStart(int64_t numAlreadyStarted, int64_t count) const { + return numAlreadyStarted + count <= + std::min(limit + budget, SERVER_KNOBS->START_TRANSACTION_MAX_TRANSACTIONS_TO_START); +} + +void GrvTransactionRateInfo::endEpoch(int64_t numStartedAtPriority, bool queueEmptyAtPriority, double elapsed) { + // Update the budget to accumulate any extra capacity available or remove any excess that was used. + // The actual delta is the portion of the limit we didn't use multiplied by the fraction of the window that + // elapsed. + // + // We may have exceeded our limit due to the budget or because of higher priority transactions, in which case + // this delta will be negative. The delta can also be negative in the event that our limit was negative, which + // can happen if we had already started more transactions in our window than our rate would have allowed. + // + // This budget has the property that when the budget is required to start transactions (because batches are + // big), the sum limit+budget will increase linearly from 0 to the batch size over time and decrease by the + // batch size upon starting a batch. In other words, this works equivalently to a model where we linearly + // accumulate budget over time in the case that our batches are too big to take advantage of the window based + // limits. + budget = + std::max(0.0, budget + elapsed * (limit - numStartedAtPriority) / SERVER_KNOBS->START_TRANSACTION_RATE_WINDOW); + + // If we are emptying out the queue of requests, then we don't need to carry much budget forward + // If we did keep accumulating budget, then our responsiveness to changes in workflow could be compromised + if (queueEmptyAtPriority) { + budget = std::min(budget, SERVER_KNOBS->START_TRANSACTION_MAX_EMPTY_QUEUE_BUDGET); + } + + smoothReleased.addDelta(numStartedAtPriority); +} + +void GrvTransactionRateInfo::disable() { + disabled = true; + // Use smoothRate.setTotal(0) instead of setting rate to 0 so txns will not be throttled immediately. + smoothRate.setTotal(0); +} + +void GrvTransactionRateInfo::setRate(double rate) { + ASSERT(rate >= 0 && rate != std::numeric_limits::infinity() && !std::isnan(rate)); + + this->rate = rate; + if (disabled) { + smoothRate.reset(rate); + disabled = false; + } else { + smoothRate.setTotal(rate); + } +} + +void GrvTransactionRateInfo::startEpoch() { + // Determine the number of transactions that this proxy is allowed to release + // Roughly speaking, this is done by computing the number of transactions over some historical window that we + // could have started but didn't, and making that our limit. More precisely, we track a smoothed rate limit and + // release rate, the difference of which is the rate of additional transactions that we could have released + // based on that window. Then we multiply by the window size to get a number of transactions. + // + // Limit can be negative in the event that we are releasing more transactions than we are allowed (due to the + // use of our budget or because of higher priority transactions). + double releaseRate = smoothRate.smoothTotal() - smoothReleased.smoothRate(); + limit = SERVER_KNOBS->START_TRANSACTION_RATE_WINDOW * releaseRate; +} + +static bool isNear(double desired, int64_t actual) { + return std::abs(desired - actual) * 10 < desired; +} + +ACTOR static Future mockClient(GrvTransactionRateInfo* rateInfo, double desiredRate, int64_t* counter) { + loop { + state double elapsed = (0.9 + 0.2 * deterministicRandom()->random01()) / desiredRate; + wait(delay(elapsed)); + rateInfo->startEpoch(); + int started = rateInfo->canStart(0, 1) ? 1 : 0; + *counter += started; + rateInfo->endEpoch(started, false, elapsed); + } +} + +// Rate limit set at 10, but client attempts 20 transactions per second. +// Client should be throttled to only 10 transactions per second. +TEST_CASE("/GrvTransactionRateInfo/Simple") { + state GrvTransactionRateInfo rateInfo; + state int64_t counter; + rateInfo.setRate(10.0); + wait(timeout(mockClient(&rateInfo, 20.0, &counter), 60.0, Void())); + TraceEvent("GrvTransactionRateInfoTest").detail("Counter", counter); + ASSERT(isNear(60.0 * 20.0, counter)); + return Void(); +} diff --git a/fdbserver/include/fdbserver/GrvTransactionRateInfo.h b/fdbserver/include/fdbserver/GrvTransactionRateInfo.h new file mode 100644 index 0000000000..9b4a553923 --- /dev/null +++ b/fdbserver/include/fdbserver/GrvTransactionRateInfo.h @@ -0,0 +1,69 @@ +/* + * GrvTransactionRateInfo.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 + +#include "fdbrpc/Smoother.h" + +// Used by GRV Proxy to enforce rate limits received from the Ratekeeper. +// +// Between delays, the GrvTransactionRateInfo executes an "epoch" starting +// with a call to the startEpoch method. Within this epoch, transactions are +// released while canStart returns true. At the end of the epoch, the +// endEpoch method is called, and the budget is updated to add or +// remove capacity. +// +// Meanwhile, the desired rate is updated through the setRate method. +// +// Smoothers are used to avoid turbulent throttling behaviour. +class GrvTransactionRateInfo { + double rate{ 0.0 }; + double limit{ 0.0 }; + double budget{ 0.0 }; + bool disabled{ true }; + Smoother smoothRate; + Smoother smoothReleased; + +public: + explicit GrvTransactionRateInfo(double rate = 0.0); + + // Determines the number of transactions that this proxy is allowed to release + // in this epoch. + void startEpoch(); + + // Checks if a "count" new transactions can be released, given that + // "numAlreadyStarted" transactions have already been released in the + // current epoch. + bool canStart(int64_t numAlreadyStarted, int64_t count) const; + + // Updates the budget to accumulate any extra capacity available or remove any excess that was used. + // Call at the end of an epoch. + void endEpoch(int64_t numStartedAtPriority, bool queueEmptyAtPriority, double elapsed); + + // Smoothly sets rate. If currently disabled, reenable + void setRate(double rate); + + // Smoothly sets transaction rate to 0. Call disable when new rates have not been + // set for a sufficiently long period of time. + void disable(); + + double getRate() const { return rate; } + double getLimit() const { return limit; } +}; From 0203c93a441218f92c387b9a5b264420fa3cae69 Mon Sep 17 00:00:00 2001 From: sfc-gh-tclinkenbeard Date: Tue, 4 Oct 2022 15:13:19 -0700 Subject: [PATCH 050/210] Add TagQueue class --- fdbserver/GrvTransactionRateInfo.actor.cpp | 4 +- fdbserver/TagQueue.actor.cpp | 168 +++++++++++++++++++++ fdbserver/include/fdbserver/TagQueue.h | 32 ++++ 3 files changed, 203 insertions(+), 1 deletion(-) create mode 100644 fdbserver/TagQueue.actor.cpp create mode 100644 fdbserver/include/fdbserver/TagQueue.h diff --git a/fdbserver/GrvTransactionRateInfo.actor.cpp b/fdbserver/GrvTransactionRateInfo.actor.cpp index b676f3afb1..28e6242a78 100644 --- a/fdbserver/GrvTransactionRateInfo.actor.cpp +++ b/fdbserver/GrvTransactionRateInfo.actor.cpp @@ -26,7 +26,9 @@ GrvTransactionRateInfo::GrvTransactionRateInfo(double rate) : rate(rate), smoothRate(SERVER_KNOBS->START_TRANSACTION_RATE_WINDOW), - smoothReleased(SERVER_KNOBS->START_TRANSACTION_RATE_WINDOW) {} + smoothReleased(SERVER_KNOBS->START_TRANSACTION_RATE_WINDOW) { + smoothRate.setTotal(rate); +} bool GrvTransactionRateInfo::canStart(int64_t numAlreadyStarted, int64_t count) const { return numAlreadyStarted + count <= diff --git a/fdbserver/TagQueue.actor.cpp b/fdbserver/TagQueue.actor.cpp new file mode 100644 index 0000000000..b7382d5dc6 --- /dev/null +++ b/fdbserver/TagQueue.actor.cpp @@ -0,0 +1,168 @@ +#include "fdbserver/TagQueue.h" +#include "flow/UnitTest.h" +#include "flow/actorcompiler.h" // must be last include + +void TagQueue::updateRates(std::map const& newRates) { + for (const auto& [tag, rate] : newRates) { + auto it = rateInfos.find(tag); + if (it == rateInfos.end()) { + rateInfos[tag] = GrvTransactionRateInfo(rate); + } else { + it->second.setRate(rate); + } + } + + for (const auto& [tag, _] : rateInfos) { + if (newRates.find(tag) == newRates.end()) { + rateInfos.erase(tag); + } + } +} + +bool TagQueue::canStart(TransactionTag tag, int64_t count) const { + auto it = rateInfos.find(tag); + if (it == rateInfos.end()) { + return true; + } + auto it2 = releasedInEpoch.find(tag); + auto alreadyReleased = (it2 == releasedInEpoch.end() ? 0 : it2->second); + return it->second.canStart(alreadyReleased, count); +} + +bool TagQueue::canStart(GetReadVersionRequest req) const { + if (req.priority == TransactionPriority::IMMEDIATE) { + return true; + } + for (const auto& [tag, count] : req.tags) { + if (!canStart(tag, count)) { + return false; + } + } + return true; +} + +void TagQueue::addRequest(GetReadVersionRequest req) { + newRequests.push_back(req); +} + +void TagQueue::startEpoch() { + for (auto& [_, rateInfo] : rateInfos) { + rateInfo.startEpoch(); + } + releasedInEpoch.clear(); +} + +void TagQueue::endEpoch(double elapsed) { + for (auto& [tag, rateInfo] : rateInfos) { + rateInfo.endEpoch(releasedInEpoch[tag], false, elapsed); + } +} + +void TagQueue::runEpoch(double elapsed, + SpannedDeque& outBatchPriority, + SpannedDeque& outDefaultPriority, + SpannedDeque& outImmediatePriority) { + startEpoch(); + Deque newDelayedRequests; + while (!newRequests.empty()) { + auto const& req = newRequests.front(); + if (canStart(req)) { + for (const auto& [tag, count] : req.tags) { + releasedInEpoch[tag] += count; + } + if (req.priority == TransactionPriority::BATCH) { + outBatchPriority.push_back(req); + } else if (req.priority == TransactionPriority::DEFAULT) { + outDefaultPriority.push_back(req); + } else if (req.priority == TransactionPriority::IMMEDIATE) { + outImmediatePriority.push_back(req); + } else { + ASSERT(false); + } + } else { + newDelayedRequests.emplace_back(req); + } + newRequests.pop_front(); + } + + while (!delayedRequests.empty()) { + auto const& delayedReq = delayedRequests.front(); + auto const& req = delayedReq.req; + if (canStart(req)) { + for (const auto& [tag, count] : req.tags) { + releasedInEpoch[tag] += count; + } + if (req.priority == TransactionPriority::BATCH) { + outBatchPriority.push_back(req); + } else if (req.priority == TransactionPriority::DEFAULT) { + outDefaultPriority.push_back(req); + } else if (req.priority == TransactionPriority::IMMEDIATE) { + outImmediatePriority.push_back(req); + } else { + ASSERT(false); + } + } else { + newDelayedRequests.push_back(delayedReq); + } + delayedRequests.pop_front(); + } + + delayedRequests = std::move(newDelayedRequests); + endEpoch(elapsed); +} + +ACTOR static Future mockClient(TagQueue* tagQueue, + TransactionPriority priority, + TransactionTag tag, + double desiredRate, + int64_t* count) { + state Future timer; + loop { + timer = delayJittered(1.0 / desiredRate); + GetReadVersionRequest req; + req.tags[tag] = 1; + req.priority = priority; + tagQueue->addRequest(req); + wait(success(req.reply.getFuture()) && timer); + ++(*count); + } +} + +ACTOR static Future mockServer(TagQueue* tagQueue) { + state SpannedDeque outBatchPriority("TestTagQueue_Batch"_loc); + state SpannedDeque outDefaultPriority("TestTagQueue_Default"_loc); + state SpannedDeque outImmediatePriority("TestTagQueue_Immediate"_loc); + loop { + state double elapsed = (0.09 + 0.02 * deterministicRandom()->random01()); + wait(delay(elapsed)); + tagQueue->runEpoch(elapsed, outBatchPriority, outDefaultPriority, outImmediatePriority); + while (!outBatchPriority.empty()) { + outBatchPriority.front().reply.send(GetReadVersionReply{}); + outBatchPriority.pop_front(); + } + while (!outDefaultPriority.empty()) { + outDefaultPriority.front().reply.send(GetReadVersionReply{}); + outDefaultPriority.pop_front(); + } + while (!outImmediatePriority.empty()) { + outImmediatePriority.front().reply.send(GetReadVersionReply{}); + outImmediatePriority.pop_front(); + } + } +} + +TEST_CASE("/TagQueue/Simple") { + state TagQueue tagQueue; + state int64_t counter = 0; + { + std::map rates; + rates["sampleTag"_sr] = 10.0; + tagQueue.updateRates(rates); + } + + state Future client = mockClient(&tagQueue, TransactionPriority::DEFAULT, "sampleTag"_sr, 20.0, &counter); + state Future server = mockServer(&tagQueue); + wait(timeout(client && server, 60.0, Void())); + TraceEvent("TagQuotaTest").detail("Counter", counter); + return Void(); +} diff --git a/fdbserver/include/fdbserver/TagQueue.h b/fdbserver/include/fdbserver/TagQueue.h new file mode 100644 index 0000000000..8a105d7ae7 --- /dev/null +++ b/fdbserver/include/fdbserver/TagQueue.h @@ -0,0 +1,32 @@ +#pragma once + +#include "fdbclient/CommitProxyInterface.h" +#include "fdbserver/GrvTransactionRateInfo.h" + +#include + +class TagQueue { + struct DelayedRequest { + double startTime; + GetReadVersionRequest req; + explicit DelayedRequest(GetReadVersionRequest req) : startTime(now()), req(req) {} + }; + + std::map rateInfos; + std::map releasedInEpoch; + Deque newRequests; + Deque delayedRequests; + + bool canStart(TransactionTag tag, int64_t count) const; + bool canStart(GetReadVersionRequest req) const; + void startEpoch(); + void endEpoch(double elapsed); + +public: + void updateRates(std::map const& newRates); + void runEpoch(double elapsed, + SpannedDeque& outBatchPriority, + SpannedDeque& outDefaultPriority, + SpannedDeque& outImmediatePriority); + void addRequest(GetReadVersionRequest); +}; From c29213eec25d051ede0c0109f6b4d57d4de30899 Mon Sep 17 00:00:00 2001 From: sfc-gh-tclinkenbeard Date: Tue, 4 Oct 2022 15:23:22 -0700 Subject: [PATCH 051/210] Reorder process of queues in TagQueue::runEpoch --- fdbserver/TagQueue.actor.cpp | 41 ++++++++++++++++++------------------ 1 file changed, 21 insertions(+), 20 deletions(-) diff --git a/fdbserver/TagQueue.actor.cpp b/fdbserver/TagQueue.actor.cpp index b7382d5dc6..86892c8928 100644 --- a/fdbserver/TagQueue.actor.cpp +++ b/fdbserver/TagQueue.actor.cpp @@ -64,26 +64,6 @@ void TagQueue::runEpoch(double elapsed, SpannedDeque& outImmediatePriority) { startEpoch(); Deque newDelayedRequests; - while (!newRequests.empty()) { - auto const& req = newRequests.front(); - if (canStart(req)) { - for (const auto& [tag, count] : req.tags) { - releasedInEpoch[tag] += count; - } - if (req.priority == TransactionPriority::BATCH) { - outBatchPriority.push_back(req); - } else if (req.priority == TransactionPriority::DEFAULT) { - outDefaultPriority.push_back(req); - } else if (req.priority == TransactionPriority::IMMEDIATE) { - outImmediatePriority.push_back(req); - } else { - ASSERT(false); - } - } else { - newDelayedRequests.emplace_back(req); - } - newRequests.pop_front(); - } while (!delayedRequests.empty()) { auto const& delayedReq = delayedRequests.front(); @@ -107,6 +87,27 @@ void TagQueue::runEpoch(double elapsed, delayedRequests.pop_front(); } + while (!newRequests.empty()) { + auto const& req = newRequests.front(); + if (canStart(req)) { + for (const auto& [tag, count] : req.tags) { + releasedInEpoch[tag] += count; + } + if (req.priority == TransactionPriority::BATCH) { + outBatchPriority.push_back(req); + } else if (req.priority == TransactionPriority::DEFAULT) { + outDefaultPriority.push_back(req); + } else if (req.priority == TransactionPriority::IMMEDIATE) { + outImmediatePriority.push_back(req); + } else { + ASSERT(false); + } + } else { + newDelayedRequests.emplace_back(req); + } + newRequests.pop_front(); + } + delayedRequests = std::move(newDelayedRequests); endEpoch(elapsed); } From a79419788fb5dec9436a7b95bf4b09726fd5795b Mon Sep 17 00:00:00 2001 From: sfc-gh-tclinkenbeard Date: Tue, 4 Oct 2022 15:30:34 -0700 Subject: [PATCH 052/210] Add /TagQueue/Immediate unit test --- fdbserver/TagQueue.actor.cpp | 27 ++++++++++++++++++++++++++- 1 file changed, 26 insertions(+), 1 deletion(-) diff --git a/fdbserver/TagQueue.actor.cpp b/fdbserver/TagQueue.actor.cpp index 86892c8928..ff8fa4204f 100644 --- a/fdbserver/TagQueue.actor.cpp +++ b/fdbserver/TagQueue.actor.cpp @@ -134,7 +134,7 @@ ACTOR static Future mockServer(TagQueue* tagQueue) { state SpannedDeque outDefaultPriority("TestTagQueue_Default"_loc); state SpannedDeque outImmediatePriority("TestTagQueue_Immediate"_loc); loop { - state double elapsed = (0.09 + 0.02 * deterministicRandom()->random01()); + state double elapsed = (0.009 + 0.002 * deterministicRandom()->random01()); wait(delay(elapsed)); tagQueue->runEpoch(elapsed, outBatchPriority, outDefaultPriority, outImmediatePriority); while (!outBatchPriority.empty()) { @@ -152,6 +152,12 @@ ACTOR static Future mockServer(TagQueue* tagQueue) { } } +static bool isNear(double desired, int64_t actual) { + return std::abs(desired - actual) * 10 < desired; +} + +// Rate limit set at 10, but client attempts 20 transactions per second. +// Client should be throttled to only 10 transactions per second. TEST_CASE("/TagQueue/Simple") { state TagQueue tagQueue; state int64_t counter = 0; @@ -165,5 +171,24 @@ TEST_CASE("/TagQueue/Simple") { state Future server = mockServer(&tagQueue); wait(timeout(client && server, 60.0, Void())); TraceEvent("TagQuotaTest").detail("Counter", counter); + ASSERT(isNear(counter, 60.0 * 10.0)); + return Void(); +} + +// Immediate-priority transactions are not throttled by the TagQueue +TEST_CASE("/TagQueue/Immediate") { + state TagQueue tagQueue; + state int64_t counter = 0; + { + std::map rates; + rates["sampleTag"_sr] = 10.0; + tagQueue.updateRates(rates); + } + + state Future client = mockClient(&tagQueue, TransactionPriority::IMMEDIATE, "sampleTag"_sr, 20.0, &counter); + state Future server = mockServer(&tagQueue); + wait(timeout(client && server, 60.0, Void())); + TraceEvent("TagQuotaTest").detail("Counter", counter); + ASSERT(isNear(counter, 60.0 * 20.0)); return Void(); } From 66dc0396dddc7f7c9107ba37bc7a476182c58732 Mon Sep 17 00:00:00 2001 From: sfc-gh-tclinkenbeard Date: Tue, 4 Oct 2022 18:15:24 -0700 Subject: [PATCH 053/210] Add /TagQueue/MultiTag unit test --- fdbserver/TagQueue.actor.cpp | 37 ++++++++++++++++++++++++++++++------ 1 file changed, 31 insertions(+), 6 deletions(-) diff --git a/fdbserver/TagQueue.actor.cpp b/fdbserver/TagQueue.actor.cpp index ff8fa4204f..cb5dbb06b1 100644 --- a/fdbserver/TagQueue.actor.cpp +++ b/fdbserver/TagQueue.actor.cpp @@ -114,14 +114,14 @@ void TagQueue::runEpoch(double elapsed, ACTOR static Future mockClient(TagQueue* tagQueue, TransactionPriority priority, - TransactionTag tag, + TransactionTagMap tags, double desiredRate, int64_t* count) { state Future timer; loop { timer = delayJittered(1.0 / desiredRate); GetReadVersionRequest req; - req.tags[tag] = 1; + req.tags = tags; req.priority = priority; tagQueue->addRequest(req); wait(success(req.reply.getFuture()) && timer); @@ -160,17 +160,19 @@ static bool isNear(double desired, int64_t actual) { // Client should be throttled to only 10 transactions per second. TEST_CASE("/TagQueue/Simple") { state TagQueue tagQueue; + state TransactionTagMap tags; state int64_t counter = 0; { std::map rates; rates["sampleTag"_sr] = 10.0; tagQueue.updateRates(rates); } + tags["sampleTag"_sr] = 1; - state Future client = mockClient(&tagQueue, TransactionPriority::DEFAULT, "sampleTag"_sr, 20.0, &counter); + state Future client = mockClient(&tagQueue, TransactionPriority::DEFAULT, tags, 20.0, &counter); state Future server = mockServer(&tagQueue); wait(timeout(client && server, 60.0, Void())); - TraceEvent("TagQuotaTest").detail("Counter", counter); + TraceEvent("TagQuotaTest_Simple").detail("Counter", counter); ASSERT(isNear(counter, 60.0 * 10.0)); return Void(); } @@ -178,17 +180,40 @@ TEST_CASE("/TagQueue/Simple") { // Immediate-priority transactions are not throttled by the TagQueue TEST_CASE("/TagQueue/Immediate") { state TagQueue tagQueue; + state TransactionTagMap tags; state int64_t counter = 0; { std::map rates; rates["sampleTag"_sr] = 10.0; tagQueue.updateRates(rates); } + tags["sampleTag"_sr] = 1; - state Future client = mockClient(&tagQueue, TransactionPriority::IMMEDIATE, "sampleTag"_sr, 20.0, &counter); + state Future client = mockClient(&tagQueue, TransactionPriority::IMMEDIATE, tags, 20.0, &counter); state Future server = mockServer(&tagQueue); wait(timeout(client && server, 60.0, Void())); - TraceEvent("TagQuotaTest").detail("Counter", counter); + TraceEvent("TagQuotaTest_Immediate").detail("Counter", counter); ASSERT(isNear(counter, 60.0 * 20.0)); return Void(); } + +// Throttle based on the tag with the lowest rate +TEST_CASE("/TagQueue/MultiTag") { + state TagQueue tagQueue; + state TransactionTagMap tags; + state int64_t counter = 0; + { + std::map rates; + rates["sampleTag1"_sr] = 10.0; + rates["sampleTag2"_sr] = 20.0; + tagQueue.updateRates(rates); + } + tags["sampleTag1"_sr] = tags["sampleTag2"_sr] = 1; + + state Future client = mockClient(&tagQueue, TransactionPriority::DEFAULT, tags, 30.0, &counter); + state Future server = mockServer(&tagQueue); + wait(timeout(client && server, 60.0, Void())); + TraceEvent("TagQuotaTest_MultiTag").detail("Counter", counter); + ASSERT(isNear(counter, 60.0 * 10.0)); + return Void(); +} From bffc41fa45e7157c4e9f3dd5897eab581135e476 Mon Sep 17 00:00:00 2001 From: sfc-gh-tclinkenbeard Date: Tue, 4 Oct 2022 18:20:03 -0700 Subject: [PATCH 054/210] Add /TagQueue/MultiClient unit test --- fdbserver/TagQueue.actor.cpp | 22 ++++++++++++++++++++++ 1 file changed, 22 insertions(+) diff --git a/fdbserver/TagQueue.actor.cpp b/fdbserver/TagQueue.actor.cpp index cb5dbb06b1..c57e8f151c 100644 --- a/fdbserver/TagQueue.actor.cpp +++ b/fdbserver/TagQueue.actor.cpp @@ -217,3 +217,25 @@ TEST_CASE("/TagQueue/MultiTag") { ASSERT(isNear(counter, 60.0 * 10.0)); return Void(); } + +// Clients share the available 10 transaction/second budget +TEST_CASE("/TagQueue/MultiClient") { + state TagQueue tagQueue; + state TransactionTagMap tags; + state int64_t counter = 0; + { + std::map rates; + rates["sampleTag1"_sr] = 10.0; + tagQueue.updateRates(rates); + } + tags["sampleTag1"_sr] = 1; + + state Future client1 = mockClient(&tagQueue, TransactionPriority::DEFAULT, tags, 20.0, &counter); + state Future client2 = mockClient(&tagQueue, TransactionPriority::DEFAULT, tags, 20.0, &counter); + + state Future server = mockServer(&tagQueue); + wait(timeout(client1 && client2 && server, 60.0, Void())); + TraceEvent("TagQuotaTest_MultiTag").detail("Counter", counter); + ASSERT(isNear(counter, 60.0 * 10.0)); + return Void(); +} From 8405d42aa973ab92b12fd465c1520e15ff55930c Mon Sep 17 00:00:00 2001 From: sfc-gh-tclinkenbeard Date: Tue, 4 Oct 2022 19:55:18 -0700 Subject: [PATCH 055/210] Add /TagQueue/Batch unit test --- fdbserver/TagQueue.actor.cpp | 90 ++++++++++++++++++++++++------------ 1 file changed, 60 insertions(+), 30 deletions(-) diff --git a/fdbserver/TagQueue.actor.cpp b/fdbserver/TagQueue.actor.cpp index c57e8f151c..0d14127d30 100644 --- a/fdbserver/TagQueue.actor.cpp +++ b/fdbserver/TagQueue.actor.cpp @@ -114,18 +114,25 @@ void TagQueue::runEpoch(double elapsed, ACTOR static Future mockClient(TagQueue* tagQueue, TransactionPriority priority, - TransactionTagMap tags, + TagSet tagSet, + int batchSize, double desiredRate, - int64_t* count) { + TransactionTagMap* counters) { state Future timer; + state TransactionTagMap tags; + for (const auto& tag : tagSet) { + tags[tag] = batchSize; + } loop { - timer = delayJittered(1.0 / desiredRate); + timer = delayJittered(static_cast(batchSize) / desiredRate); GetReadVersionRequest req; req.tags = tags; req.priority = priority; tagQueue->addRequest(req); wait(success(req.reply.getFuture()) && timer); - ++(*count); + for (auto& [tag, _] : tags) { + (*counters)[tag] += batchSize; + } } } @@ -160,82 +167,105 @@ static bool isNear(double desired, int64_t actual) { // Client should be throttled to only 10 transactions per second. TEST_CASE("/TagQueue/Simple") { state TagQueue tagQueue; - state TransactionTagMap tags; - state int64_t counter = 0; + state TagSet tagSet; + state TransactionTagMap counters; { std::map rates; rates["sampleTag"_sr] = 10.0; tagQueue.updateRates(rates); } - tags["sampleTag"_sr] = 1; + tagSet.addTag("sampleTag"_sr); - state Future client = mockClient(&tagQueue, TransactionPriority::DEFAULT, tags, 20.0, &counter); + state Future client = mockClient(&tagQueue, TransactionPriority::DEFAULT, tagSet, 1, 20.0, &counters); state Future server = mockServer(&tagQueue); wait(timeout(client && server, 60.0, Void())); - TraceEvent("TagQuotaTest_Simple").detail("Counter", counter); - ASSERT(isNear(counter, 60.0 * 10.0)); + TraceEvent("TagQuotaTest_Simple").detail("Counter", counters["sampleTag"_sr]); + ASSERT(isNear(counters["sampleTag"_sr], 60.0 * 10.0)); return Void(); } // Immediate-priority transactions are not throttled by the TagQueue TEST_CASE("/TagQueue/Immediate") { state TagQueue tagQueue; - state TransactionTagMap tags; - state int64_t counter = 0; + state TagSet tagSet; + state TransactionTagMap counters; { std::map rates; rates["sampleTag"_sr] = 10.0; tagQueue.updateRates(rates); } - tags["sampleTag"_sr] = 1; + tagSet.addTag("sampleTag"_sr); - state Future client = mockClient(&tagQueue, TransactionPriority::IMMEDIATE, tags, 20.0, &counter); + state Future client = mockClient(&tagQueue, TransactionPriority::IMMEDIATE, tagSet, 1, 20.0, &counters); state Future server = mockServer(&tagQueue); wait(timeout(client && server, 60.0, Void())); - TraceEvent("TagQuotaTest_Immediate").detail("Counter", counter); - ASSERT(isNear(counter, 60.0 * 20.0)); + TraceEvent("TagQuotaTest_Immediate").detail("Counter", counters["sampleTag"_sr]); + ASSERT(isNear(counters["sampleTag"_sr], 60.0 * 20.0)); return Void(); } // Throttle based on the tag with the lowest rate TEST_CASE("/TagQueue/MultiTag") { state TagQueue tagQueue; - state TransactionTagMap tags; - state int64_t counter = 0; + state TagSet tagSet; + state TransactionTagMap counters; { std::map rates; rates["sampleTag1"_sr] = 10.0; rates["sampleTag2"_sr] = 20.0; tagQueue.updateRates(rates); } - tags["sampleTag1"_sr] = tags["sampleTag2"_sr] = 1; + tagSet.addTag("sampleTag1"_sr); + tagSet.addTag("sampleTag2"_sr); - state Future client = mockClient(&tagQueue, TransactionPriority::DEFAULT, tags, 30.0, &counter); + state Future client = mockClient(&tagQueue, TransactionPriority::DEFAULT, tagSet, 1, 30.0, &counters); state Future server = mockServer(&tagQueue); wait(timeout(client && server, 60.0, Void())); - TraceEvent("TagQuotaTest_MultiTag").detail("Counter", counter); - ASSERT(isNear(counter, 60.0 * 10.0)); + TraceEvent("TagQuotaTest_MultiTag").detail("Counter", counters["sampleTag1"_sr]); + ASSERT_EQ(counters["sampleTag1"_sr], counters["sampleTag2"_sr]); + ASSERT(isNear(counters["sampleTag1"_sr], 60.0 * 10.0)); + return Void(); } // Clients share the available 10 transaction/second budget TEST_CASE("/TagQueue/MultiClient") { state TagQueue tagQueue; - state TransactionTagMap tags; - state int64_t counter = 0; + state TagSet tagSet; + state TransactionTagMap counters; { std::map rates; - rates["sampleTag1"_sr] = 10.0; + rates["sampleTag"_sr] = 10.0; tagQueue.updateRates(rates); } - tags["sampleTag1"_sr] = 1; + tagSet.addTag("sampleTag"_sr); - state Future client1 = mockClient(&tagQueue, TransactionPriority::DEFAULT, tags, 20.0, &counter); - state Future client2 = mockClient(&tagQueue, TransactionPriority::DEFAULT, tags, 20.0, &counter); + state Future client1 = mockClient(&tagQueue, TransactionPriority::DEFAULT, tagSet, 1, 20.0, &counters); + state Future client2 = mockClient(&tagQueue, TransactionPriority::DEFAULT, tagSet, 1, 20.0, &counters); state Future server = mockServer(&tagQueue); wait(timeout(client1 && client2 && server, 60.0, Void())); - TraceEvent("TagQuotaTest_MultiTag").detail("Counter", counter); - ASSERT(isNear(counter, 60.0 * 10.0)); + TraceEvent("TagQuotaTest_MultiClient").detail("Counter", counters["sampleTag"_sr]); + ASSERT(isNear(counters["sampleTag"_sr], 60.0 * 10.0)); + return Void(); +} + +TEST_CASE("/TagQueue/Batch") { + state TagQueue tagQueue; + state TagSet tagSet; + state TransactionTagMap counters; + { + std::map rates; + rates["sampleTag"_sr] = 10.0; + tagQueue.updateRates(rates); + } + tagSet.addTag("sampleTag"_sr); + + state Future client = mockClient(&tagQueue, TransactionPriority::DEFAULT, tagSet, 5, 20.0, &counters); + state Future server = mockServer(&tagQueue); + wait(timeout(client && server, 60.0, Void())); + + TraceEvent("TagQuotaTest_Batch").detail("Counter", counters["sampleTag"_sr]); + ASSERT(isNear(counters["sampleTag"_sr], 60.0 * 10.0)); return Void(); } From 2684464cc6776e4564b928e4756788cd38d32050 Mon Sep 17 00:00:00 2001 From: sfc-gh-tclinkenbeard Date: Wed, 5 Oct 2022 08:53:56 -0700 Subject: [PATCH 056/210] Use TagQueue in GRV proxy when ENFORCE_TAG_THROTTLING_ON_PROXIES is true --- fdbserver/GlobalTagThrottler.actor.cpp | 18 +++--- fdbserver/GrvProxyServer.actor.cpp | 57 ++++++++++--------- fdbserver/TagQueue.actor.cpp | 12 ++-- .../include/fdbserver/RatekeeperInterface.h | 2 +- fdbserver/include/fdbserver/TagQueue.h | 2 +- fdbserver/include/fdbserver/TagThrottler.h | 6 +- 6 files changed, 49 insertions(+), 48 deletions(-) diff --git a/fdbserver/GlobalTagThrottler.actor.cpp b/fdbserver/GlobalTagThrottler.actor.cpp index c1ec99541f..eebbe82e50 100644 --- a/fdbserver/GlobalTagThrottler.actor.cpp +++ b/fdbserver/GlobalTagThrottler.actor.cpp @@ -400,8 +400,8 @@ public: void addRequests(TransactionTag tag, int count) { tagStatistics[tag].addTransactions(static_cast(count)); } uint64_t getThrottledTagChangeId() const { return throttledTagChangeId; } - PrioritizedTransactionTagMap getProxyRates(int numProxies) { - PrioritizedTransactionTagMap result; + TransactionTagMap getProxyRates(int numProxies) { + TransactionTagMap result; lastBusyTagCount = 0; for (auto& [tag, stats] : tagStatistics) { @@ -414,8 +414,7 @@ public: } if (targetTps.present()) { auto const smoothedTargetTps = stats.updateAndGetTargetLimit(targetTps.get()); - result[TransactionPriority::BATCH][tag] = result[TransactionPriority::DEFAULT][tag] = - smoothedTargetTps / numProxies; + result[tag] = smoothedTargetTps / numProxies; } else { te.disable(); } @@ -497,7 +496,7 @@ uint64_t GlobalTagThrottler::getThrottledTagChangeId() const { PrioritizedTransactionTagMap GlobalTagThrottler::getClientRates() { return impl->getClientRates(); } -PrioritizedTransactionTagMap GlobalTagThrottler::getProxyRates(int numProxies) { +TransactionTagMap GlobalTagThrottler::getProxyRates(int numProxies) { return impl->getProxyRates(numProxies); } int64_t GlobalTagThrottler::autoThrottleCount() const { @@ -679,12 +678,9 @@ bool isNear(Optional a, Optional b) { bool targetRateIsNear(GlobalTagThrottler& globalTagThrottler, TransactionTag tag, Optional expected) { Optional rate; auto targetRates = globalTagThrottler.getProxyRates(1); - auto it1 = targetRates.find(TransactionPriority::DEFAULT); - if (it1 != targetRates.end()) { - auto it2 = it1->second.find(tag); - if (it2 != it1->second.end()) { - rate = it2->second; - } + auto it = targetRates.find(tag); + if (it != targetRates.end()) { + rate = it->second; } TraceEvent("GlobalTagThrottling_RateMonitor") .detail("Tag", tag) diff --git a/fdbserver/GrvProxyServer.actor.cpp b/fdbserver/GrvProxyServer.actor.cpp index c25df2cd1a..2460bce24c 100644 --- a/fdbserver/GrvProxyServer.actor.cpp +++ b/fdbserver/GrvProxyServer.actor.cpp @@ -30,6 +30,7 @@ #include "fdbserver/GrvTransactionRateInfo.h" #include "fdbserver/LogSystem.h" #include "fdbserver/LogSystemDiskQueueAdapter.h" +#include "fdbserver/TagQueue.h" #include "fdbserver/WaitFailure.h" #include "fdbserver/WorkerInterface.actor.h" #include "fdbrpc/sim_validation.h" @@ -361,7 +362,7 @@ ACTOR Future getRate(UID myID, GetHealthMetricsReply* detailedHealthMetricsReply, TransactionTagMap* transactionTagCounter, PrioritizedTransactionTagMap* clientThrottledTags, - PrioritizedTransactionTagMap* perTagRateInfo, + TagQueue* tagQueue, GrvProxyStats* stats, GrvProxyData* proxyData) { state Future nextRequestTimer = Never(); @@ -422,12 +423,7 @@ ACTOR Future getRate(UID myID, *clientThrottledTags = std::move(rep.clientThrottledTags.get()); } if (rep.proxyThrottledTags.present()) { - perTagRateInfo->clear(); - for (const auto& [priority, tagToRate] : rep.proxyThrottledTags.get()) { - for (const auto& [tag, rate] : tagToRate) { - (*perTagRateInfo)[priority][tag].setRate(rate); - } - } + tagQueue->updateRates(rep.proxyThrottledTags.get()); } } when(wait(leaseTimeout)) { @@ -461,20 +457,19 @@ void dropRequestFromQueue(Deque* queue, GrvProxyStats* st } // Put a GetReadVersion request into the queue corresponding to its priority. -ACTOR Future queueGetReadVersionRequests( - Reference const> db, - SpannedDeque* systemQueue, - SpannedDeque* defaultQueue, - SpannedDeque* batchQueue, - FutureStream readVersionRequests, - PromiseStream GRVTimer, - double* lastGRVTime, - double* GRVBatchTime, - FutureStream normalGRVLatency, - GrvProxyStats* stats, - GrvTransactionRateInfo* batchRateInfo, - TransactionTagMap* transactionTagCounter, - PrioritizedTransactionTagMap const* perClientRateInfo) { +ACTOR Future queueGetReadVersionRequests(Reference const> db, + SpannedDeque* systemQueue, + SpannedDeque* defaultQueue, + SpannedDeque* batchQueue, + FutureStream readVersionRequests, + PromiseStream GRVTimer, + double* lastGRVTime, + double* GRVBatchTime, + FutureStream normalGRVLatency, + GrvProxyStats* stats, + GrvTransactionRateInfo* batchRateInfo, + TransactionTagMap* transactionTagCounter, + TagQueue* tagQueue) { getCurrentLineage()->modify(&TransactionLineage::operation) = TransactionLineage::Operation::GetConsistentReadVersion; loop choose { @@ -541,7 +536,11 @@ ACTOR Future queueGetReadVersionRequests( stats->txnStartIn += req.transactionCount; stats->txnDefaultPriorityStartIn += req.transactionCount; ++stats->defaultGRVQueueSize; - defaultQueue->push_back(req); + if (SERVER_KNOBS->ENFORCE_TAG_THROTTLING_ON_PROXIES) { + tagQueue->addRequest(req); + } else { + defaultQueue->push_back(req); + } // defaultQueue->span.addParent(req.spanContext); } else { // Return error for batch_priority GRV requests @@ -554,7 +553,11 @@ ACTOR Future queueGetReadVersionRequests( stats->txnStartIn += req.transactionCount; stats->txnBatchPriorityStartIn += req.transactionCount; ++stats->batchGRVQueueSize; - batchQueue->push_back(req); + if (SERVER_KNOBS->ENFORCE_TAG_THROTTLING_ON_PROXIES) { + tagQueue->addRequest(req); + } else { + batchQueue->push_back(req); + } // batchQueue->span.addParent(req.spanContext); } } @@ -819,7 +822,7 @@ ACTOR static Future transactionStarter(GrvProxyInterface proxy, state int64_t batchTransactionCount = 0; state GrvTransactionRateInfo normalRateInfo(10); state GrvTransactionRateInfo batchRateInfo(0); - state PrioritizedTransactionTagMap perTagRateInfo; + state TagQueue tagQueue; state SpannedDeque systemQueue("GP:transactionStarterSystemQueue"_loc); state SpannedDeque defaultQueue("GP:transactionStarterDefaultQueue"_loc); @@ -846,7 +849,7 @@ ACTOR static Future transactionStarter(GrvProxyInterface proxy, detailedHealthMetricsReply, &transactionTagCounter, &clientThrottledTags, - &perTagRateInfo, + &tagQueue, &grvProxyData->stats, grvProxyData)); addActor.send(queueGetReadVersionRequests(db, @@ -861,7 +864,7 @@ ACTOR static Future transactionStarter(GrvProxyInterface proxy, &grvProxyData->stats, &batchRateInfo, &transactionTagCounter, - &perTagRateInfo)); + &tagQueue)); while (std::find(db->get().client.grvProxies.begin(), db->get().client.grvProxies.end(), proxy) == db->get().client.grvProxies.end()) { @@ -884,6 +887,8 @@ ACTOR static Future transactionStarter(GrvProxyInterface proxy, elapsed = 1e-15; } + // TODO: Remove systemQueue parameter? + tagQueue.runEpoch(elapsed, defaultQueue, batchQueue, systemQueue); normalRateInfo.startEpoch(); batchRateInfo.startEpoch(); diff --git a/fdbserver/TagQueue.actor.cpp b/fdbserver/TagQueue.actor.cpp index 0d14127d30..6dc1bda088 100644 --- a/fdbserver/TagQueue.actor.cpp +++ b/fdbserver/TagQueue.actor.cpp @@ -2,7 +2,7 @@ #include "flow/UnitTest.h" #include "flow/actorcompiler.h" // must be last include -void TagQueue::updateRates(std::map const& newRates) { +void TagQueue::updateRates(TransactionTagMap const& newRates) { for (const auto& [tag, rate] : newRates) { auto it = rateInfos.find(tag); if (it == rateInfos.end()) { @@ -170,7 +170,7 @@ TEST_CASE("/TagQueue/Simple") { state TagSet tagSet; state TransactionTagMap counters; { - std::map rates; + TransactionTagMap rates; rates["sampleTag"_sr] = 10.0; tagQueue.updateRates(rates); } @@ -190,7 +190,7 @@ TEST_CASE("/TagQueue/Immediate") { state TagSet tagSet; state TransactionTagMap counters; { - std::map rates; + TransactionTagMap rates; rates["sampleTag"_sr] = 10.0; tagQueue.updateRates(rates); } @@ -210,7 +210,7 @@ TEST_CASE("/TagQueue/MultiTag") { state TagSet tagSet; state TransactionTagMap counters; { - std::map rates; + TransactionTagMap rates; rates["sampleTag1"_sr] = 10.0; rates["sampleTag2"_sr] = 20.0; tagQueue.updateRates(rates); @@ -234,7 +234,7 @@ TEST_CASE("/TagQueue/MultiClient") { state TagSet tagSet; state TransactionTagMap counters; { - std::map rates; + TransactionTagMap rates; rates["sampleTag"_sr] = 10.0; tagQueue.updateRates(rates); } @@ -255,7 +255,7 @@ TEST_CASE("/TagQueue/Batch") { state TagSet tagSet; state TransactionTagMap counters; { - std::map rates; + TransactionTagMap rates; rates["sampleTag"_sr] = 10.0; tagQueue.updateRates(rates); } diff --git a/fdbserver/include/fdbserver/RatekeeperInterface.h b/fdbserver/include/fdbserver/RatekeeperInterface.h index 2ed6775736..9cfd5e805a 100644 --- a/fdbserver/include/fdbserver/RatekeeperInterface.h +++ b/fdbserver/include/fdbserver/RatekeeperInterface.h @@ -81,7 +81,7 @@ struct GetRateInfoReply { // Depending on the value of SERVER_KNOBS->ENFORCE_TAG_THROTTLING_ON_PROXIES, // one of these fields may be populated Optional> clientThrottledTags; - Optional> proxyThrottledTags; + Optional> proxyThrottledTags; template void serialize(Ar& ar) { diff --git a/fdbserver/include/fdbserver/TagQueue.h b/fdbserver/include/fdbserver/TagQueue.h index 8a105d7ae7..b3042855c9 100644 --- a/fdbserver/include/fdbserver/TagQueue.h +++ b/fdbserver/include/fdbserver/TagQueue.h @@ -23,7 +23,7 @@ class TagQueue { void endEpoch(double elapsed); public: - void updateRates(std::map const& newRates); + void updateRates(TransactionTagMap const& newRates); void runEpoch(double elapsed, SpannedDeque& outBatchPriority, SpannedDeque& outDefaultPriority, diff --git a/fdbserver/include/fdbserver/TagThrottler.h b/fdbserver/include/fdbserver/TagThrottler.h index 5e00e7be46..8330e4caa9 100644 --- a/fdbserver/include/fdbserver/TagThrottler.h +++ b/fdbserver/include/fdbserver/TagThrottler.h @@ -42,7 +42,7 @@ public: // For each tag and priority combination, return the throughput limit for the cluster // (to be shared across all GRV proxies) - virtual PrioritizedTransactionTagMap getProxyRates(int numProxies) = 0; + virtual TransactionTagMap getProxyRates(int numProxies) = 0; virtual int64_t autoThrottleCount() const = 0; virtual uint32_t busyReadTagCount() const = 0; @@ -66,7 +66,7 @@ public: void addRequests(TransactionTag tag, int count) override; uint64_t getThrottledTagChangeId() const override; PrioritizedTransactionTagMap getClientRates() override; - PrioritizedTransactionTagMap getProxyRates(int numProxies) override { throw not_implemented(); } + TransactionTagMap getProxyRates(int numProxies) override { throw not_implemented(); } int64_t autoThrottleCount() const override; uint32_t busyReadTagCount() const override; uint32_t busyWriteTagCount() const override; @@ -94,7 +94,7 @@ public: Future tryUpdateAutoThrottling(StorageQueueInfo const&) override; PrioritizedTransactionTagMap getClientRates() override; - PrioritizedTransactionTagMap getProxyRates(int numProxies) override; + TransactionTagMap getProxyRates(int numProxies) override; // Testing only: public: From e313edbb85bb6566947596c52f330640aaa1ca86 Mon Sep 17 00:00:00 2001 From: sfc-gh-tclinkenbeard Date: Wed, 5 Oct 2022 14:41:34 -0700 Subject: [PATCH 057/210] Add proxyTagThrottledDuration field to GetReadVersion[Request|Reply] --- fdbclient/include/fdbclient/CommitProxyInterface.h | 8 +++++++- fdbserver/GrvProxyServer.actor.cpp | 1 + fdbserver/TagQueue.actor.cpp | 5 +++-- fdbserver/include/fdbserver/TagQueue.h | 1 + 4 files changed, 12 insertions(+), 3 deletions(-) diff --git a/fdbclient/include/fdbclient/CommitProxyInterface.h b/fdbclient/include/fdbclient/CommitProxyInterface.h index 1614aeacf0..2e3ae6a3cd 100644 --- a/fdbclient/include/fdbclient/CommitProxyInterface.h +++ b/fdbclient/include/fdbclient/CommitProxyInterface.h @@ -224,6 +224,7 @@ struct GetReadVersionReply : public BasicLoadBalancedReply { bool rkBatchThrottled = false; TransactionTagMap tagThrottleInfo; + double proxyTagThrottledDuration{ 0.0 }; VersionVector ssVersionVectorDelta; UID proxyId; // GRV proxy ID to detect old GRV proxies at client side @@ -242,7 +243,8 @@ struct GetReadVersionReply : public BasicLoadBalancedReply { rkDefaultThrottled, rkBatchThrottled, ssVersionVectorDelta, - proxyId); + proxyId, + proxyTagThrottledDuration); } }; @@ -267,6 +269,10 @@ struct GetReadVersionRequest : TimedRequest { TransactionPriority priority; TransactionTagMap tags; + // Not serialized, because this field does not need to be sent to master. + // It is used for reporting to clients the amount of time spent delayed by + // the TagQueue + double proxyTagThrottledDuration{ 0.0 }; Optional debugID; ReplyPromise reply; diff --git a/fdbserver/GrvProxyServer.actor.cpp b/fdbserver/GrvProxyServer.actor.cpp index 2460bce24c..6fd99542bc 100644 --- a/fdbserver/GrvProxyServer.actor.cpp +++ b/fdbserver/GrvProxyServer.actor.cpp @@ -718,6 +718,7 @@ ACTOR Future sendGrvReplies(Future replyFuture, grvProxyData->versionVectorSizeOnGRVReply.addMeasurement(reply.ssVersionVectorDelta.size()); } reply.proxyId = grvProxyData->dbgid; + reply.proxyTagThrottledDuration = request.proxyTagThrottledDuration; if (!request.tags.empty()) { auto& priorityThrottledTags = clientThrottledTags[request.priority]; diff --git a/fdbserver/TagQueue.actor.cpp b/fdbserver/TagQueue.actor.cpp index 6dc1bda088..e35fa59773 100644 --- a/fdbserver/TagQueue.actor.cpp +++ b/fdbserver/TagQueue.actor.cpp @@ -66,12 +66,13 @@ void TagQueue::runEpoch(double elapsed, Deque newDelayedRequests; while (!delayedRequests.empty()) { - auto const& delayedReq = delayedRequests.front(); - auto const& req = delayedReq.req; + auto& delayedReq = delayedRequests.front(); + auto& req = delayedReq.req; if (canStart(req)) { for (const auto& [tag, count] : req.tags) { releasedInEpoch[tag] += count; } + req.proxyTagThrottledDuration = delayedReq.delayTime(); if (req.priority == TransactionPriority::BATCH) { outBatchPriority.push_back(req); } else if (req.priority == TransactionPriority::DEFAULT) { diff --git a/fdbserver/include/fdbserver/TagQueue.h b/fdbserver/include/fdbserver/TagQueue.h index b3042855c9..bc669f0faf 100644 --- a/fdbserver/include/fdbserver/TagQueue.h +++ b/fdbserver/include/fdbserver/TagQueue.h @@ -10,6 +10,7 @@ class TagQueue { double startTime; GetReadVersionRequest req; explicit DelayedRequest(GetReadVersionRequest req) : startTime(now()), req(req) {} + double delayTime() const { return now() - startTime; } }; std::map rateInfos; From 34857bd2bfcf84708d850078d22fe87a534e0ad0 Mon Sep 17 00:00:00 2001 From: sfc-gh-tclinkenbeard Date: Wed, 5 Oct 2022 14:54:50 -0700 Subject: [PATCH 058/210] Remove outSystemPriority parameter from TagQueue::runEpoch --- fdbserver/GrvProxyServer.actor.cpp | 3 +- fdbserver/TagQueue.actor.cpp | 39 +++----------------------- fdbserver/include/fdbserver/TagQueue.h | 3 +- 3 files changed, 6 insertions(+), 39 deletions(-) diff --git a/fdbserver/GrvProxyServer.actor.cpp b/fdbserver/GrvProxyServer.actor.cpp index 6fd99542bc..ee1ff02737 100644 --- a/fdbserver/GrvProxyServer.actor.cpp +++ b/fdbserver/GrvProxyServer.actor.cpp @@ -888,8 +888,7 @@ ACTOR static Future transactionStarter(GrvProxyInterface proxy, elapsed = 1e-15; } - // TODO: Remove systemQueue parameter? - tagQueue.runEpoch(elapsed, defaultQueue, batchQueue, systemQueue); + tagQueue.runEpoch(elapsed, defaultQueue, batchQueue); normalRateInfo.startEpoch(); batchRateInfo.startEpoch(); diff --git a/fdbserver/TagQueue.actor.cpp b/fdbserver/TagQueue.actor.cpp index e35fa59773..3cd45afe4b 100644 --- a/fdbserver/TagQueue.actor.cpp +++ b/fdbserver/TagQueue.actor.cpp @@ -30,9 +30,6 @@ bool TagQueue::canStart(TransactionTag tag, int64_t count) const { } bool TagQueue::canStart(GetReadVersionRequest req) const { - if (req.priority == TransactionPriority::IMMEDIATE) { - return true; - } for (const auto& [tag, count] : req.tags) { if (!canStart(tag, count)) { return false; @@ -60,8 +57,7 @@ void TagQueue::endEpoch(double elapsed) { void TagQueue::runEpoch(double elapsed, SpannedDeque& outBatchPriority, - SpannedDeque& outDefaultPriority, - SpannedDeque& outImmediatePriority) { + SpannedDeque& outDefaultPriority) { startEpoch(); Deque newDelayedRequests; @@ -77,9 +73,8 @@ void TagQueue::runEpoch(double elapsed, outBatchPriority.push_back(req); } else if (req.priority == TransactionPriority::DEFAULT) { outDefaultPriority.push_back(req); - } else if (req.priority == TransactionPriority::IMMEDIATE) { - outImmediatePriority.push_back(req); } else { + // Immediate priority transactions should bypass the TagQueue ASSERT(false); } } else { @@ -98,9 +93,8 @@ void TagQueue::runEpoch(double elapsed, outBatchPriority.push_back(req); } else if (req.priority == TransactionPriority::DEFAULT) { outDefaultPriority.push_back(req); - } else if (req.priority == TransactionPriority::IMMEDIATE) { - outImmediatePriority.push_back(req); } else { + // Immediate priority transactions should bypass the TagQueue ASSERT(false); } } else { @@ -140,11 +134,10 @@ ACTOR static Future mockClient(TagQueue* tagQueue, ACTOR static Future mockServer(TagQueue* tagQueue) { state SpannedDeque outBatchPriority("TestTagQueue_Batch"_loc); state SpannedDeque outDefaultPriority("TestTagQueue_Default"_loc); - state SpannedDeque outImmediatePriority("TestTagQueue_Immediate"_loc); loop { state double elapsed = (0.009 + 0.002 * deterministicRandom()->random01()); wait(delay(elapsed)); - tagQueue->runEpoch(elapsed, outBatchPriority, outDefaultPriority, outImmediatePriority); + tagQueue->runEpoch(elapsed, outBatchPriority, outDefaultPriority); while (!outBatchPriority.empty()) { outBatchPriority.front().reply.send(GetReadVersionReply{}); outBatchPriority.pop_front(); @@ -153,10 +146,6 @@ ACTOR static Future mockServer(TagQueue* tagQueue) { outDefaultPriority.front().reply.send(GetReadVersionReply{}); outDefaultPriority.pop_front(); } - while (!outImmediatePriority.empty()) { - outImmediatePriority.front().reply.send(GetReadVersionReply{}); - outImmediatePriority.pop_front(); - } } } @@ -185,26 +174,6 @@ TEST_CASE("/TagQueue/Simple") { return Void(); } -// Immediate-priority transactions are not throttled by the TagQueue -TEST_CASE("/TagQueue/Immediate") { - state TagQueue tagQueue; - state TagSet tagSet; - state TransactionTagMap counters; - { - TransactionTagMap rates; - rates["sampleTag"_sr] = 10.0; - tagQueue.updateRates(rates); - } - tagSet.addTag("sampleTag"_sr); - - state Future client = mockClient(&tagQueue, TransactionPriority::IMMEDIATE, tagSet, 1, 20.0, &counters); - state Future server = mockServer(&tagQueue); - wait(timeout(client && server, 60.0, Void())); - TraceEvent("TagQuotaTest_Immediate").detail("Counter", counters["sampleTag"_sr]); - ASSERT(isNear(counters["sampleTag"_sr], 60.0 * 20.0)); - return Void(); -} - // Throttle based on the tag with the lowest rate TEST_CASE("/TagQueue/MultiTag") { state TagQueue tagQueue; diff --git a/fdbserver/include/fdbserver/TagQueue.h b/fdbserver/include/fdbserver/TagQueue.h index bc669f0faf..3383355222 100644 --- a/fdbserver/include/fdbserver/TagQueue.h +++ b/fdbserver/include/fdbserver/TagQueue.h @@ -27,7 +27,6 @@ public: void updateRates(TransactionTagMap const& newRates); void runEpoch(double elapsed, SpannedDeque& outBatchPriority, - SpannedDeque& outDefaultPriority, - SpannedDeque& outImmediatePriority); + SpannedDeque& outDefaultPriority); void addRequest(GetReadVersionRequest); }; From 69b3c09cf7392da4845a38cc275a71a1c30a8609 Mon Sep 17 00:00:00 2001 From: sfc-gh-tclinkenbeard Date: Wed, 5 Oct 2022 16:16:35 -0700 Subject: [PATCH 059/210] Run 10 clients in /TagQueue/MultiClient --- fdbserver/TagQueue.actor.cpp | 15 +++++++++------ 1 file changed, 9 insertions(+), 6 deletions(-) diff --git a/fdbserver/TagQueue.actor.cpp b/fdbserver/TagQueue.actor.cpp index 3cd45afe4b..6be9e34d8d 100644 --- a/fdbserver/TagQueue.actor.cpp +++ b/fdbserver/TagQueue.actor.cpp @@ -198,25 +198,28 @@ TEST_CASE("/TagQueue/MultiTag") { return Void(); } -// Clients share the available 10 transaction/second budget +// Clients share the available 30 transaction/second budget TEST_CASE("/TagQueue/MultiClient") { state TagQueue tagQueue; state TagSet tagSet; state TransactionTagMap counters; { TransactionTagMap rates; - rates["sampleTag"_sr] = 10.0; + rates["sampleTag"_sr] = 30.0; tagQueue.updateRates(rates); } tagSet.addTag("sampleTag"_sr); - state Future client1 = mockClient(&tagQueue, TransactionPriority::DEFAULT, tagSet, 1, 20.0, &counters); - state Future client2 = mockClient(&tagQueue, TransactionPriority::DEFAULT, tagSet, 1, 20.0, &counters); + state std::vector> clients; + clients.reserve(10); + for (int i = 0; i < 10; ++i) { + clients.push_back(mockClient(&tagQueue, TransactionPriority::DEFAULT, tagSet, 1, 10.0, &counters)); + } state Future server = mockServer(&tagQueue); - wait(timeout(client1 && client2 && server, 60.0, Void())); + wait(timeout(waitForAll(clients) && server, 60.0, Void())); TraceEvent("TagQuotaTest_MultiClient").detail("Counter", counters["sampleTag"_sr]); - ASSERT(isNear(counters["sampleTag"_sr], 60.0 * 10.0)); + ASSERT(isNear(counters["sampleTag"_sr], 60.0 * 30.0)); return Void(); } From 8ff66d6fad42c7b2bca414eefe8387bd38c9344c Mon Sep 17 00:00:00 2001 From: sfc-gh-tclinkenbeard Date: Thu, 6 Oct 2022 07:37:58 -0700 Subject: [PATCH 060/210] Add comments to TagQueue class --- fdbserver/TagQueue.actor.cpp | 35 ++++++++++--------------- fdbserver/include/fdbserver/TagQueue.h | 36 ++++++++++++++++++++------ 2 files changed, 41 insertions(+), 30 deletions(-) diff --git a/fdbserver/TagQueue.actor.cpp b/fdbserver/TagQueue.actor.cpp index 6be9e34d8d..e363526426 100644 --- a/fdbserver/TagQueue.actor.cpp +++ b/fdbserver/TagQueue.actor.cpp @@ -19,19 +19,17 @@ void TagQueue::updateRates(TransactionTagMap const& newRates) { } } -bool TagQueue::canStart(TransactionTag tag, int64_t count) const { +bool TagQueue::canStart(TransactionTag tag, int64_t alreadyReleased, int64_t count) const { auto it = rateInfos.find(tag); if (it == rateInfos.end()) { return true; } - auto it2 = releasedInEpoch.find(tag); - auto alreadyReleased = (it2 == releasedInEpoch.end() ? 0 : it2->second); return it->second.canStart(alreadyReleased, count); } -bool TagQueue::canStart(GetReadVersionRequest req) const { +bool TagQueue::canStart(GetReadVersionRequest req, TransactionTagMap& releasedInEpoch) const { for (const auto& [tag, count] : req.tags) { - if (!canStart(tag, count)) { + if (!canStart(tag, releasedInEpoch[tag], count)) { return false; } } @@ -42,29 +40,20 @@ void TagQueue::addRequest(GetReadVersionRequest req) { newRequests.push_back(req); } -void TagQueue::startEpoch() { - for (auto& [_, rateInfo] : rateInfos) { - rateInfo.startEpoch(); - } - releasedInEpoch.clear(); -} - -void TagQueue::endEpoch(double elapsed) { - for (auto& [tag, rateInfo] : rateInfos) { - rateInfo.endEpoch(releasedInEpoch[tag], false, elapsed); - } -} - void TagQueue::runEpoch(double elapsed, SpannedDeque& outBatchPriority, SpannedDeque& outDefaultPriority) { - startEpoch(); + for (auto& [_, rateInfo] : rateInfos) { + rateInfo.startEpoch(); + } + Deque newDelayedRequests; + TransactionTagMap releasedInEpoch; while (!delayedRequests.empty()) { auto& delayedReq = delayedRequests.front(); auto& req = delayedReq.req; - if (canStart(req)) { + if (canStart(req, releasedInEpoch)) { for (const auto& [tag, count] : req.tags) { releasedInEpoch[tag] += count; } @@ -85,7 +74,7 @@ void TagQueue::runEpoch(double elapsed, while (!newRequests.empty()) { auto const& req = newRequests.front(); - if (canStart(req)) { + if (canStart(req, releasedInEpoch)) { for (const auto& [tag, count] : req.tags) { releasedInEpoch[tag] += count; } @@ -104,7 +93,9 @@ void TagQueue::runEpoch(double elapsed, } delayedRequests = std::move(newDelayedRequests); - endEpoch(elapsed); + for (auto& [tag, rateInfo] : rateInfos) { + rateInfo.endEpoch(std::move(releasedInEpoch)[tag], false, elapsed); + } } ACTOR static Future mockClient(TagQueue* tagQueue, diff --git a/fdbserver/include/fdbserver/TagQueue.h b/fdbserver/include/fdbserver/TagQueue.h index 3383355222..a930319c9c 100644 --- a/fdbserver/include/fdbserver/TagQueue.h +++ b/fdbserver/include/fdbserver/TagQueue.h @@ -1,10 +1,18 @@ #pragma once #include "fdbclient/CommitProxyInterface.h" +#include "fdbclient/TagThrottle.actor.h" #include "fdbserver/GrvTransactionRateInfo.h" -#include - +// TagQueue is used to throttle GetReadVersionRequests based on tag quotas +// before they're pushed into priority-partitioned queues. +// +// A GrvTransactionRateInfo object is maintained for each tag. This object +// is used to determine when a request can be released. +// +// Between each set of waits, runEpoch is run, releasing queued transactions +// that have passed the tag throttling stage. Transactions that are not yet ready +// are requeued during runEpoch. class TagQueue { struct DelayedRequest { double startTime; @@ -13,20 +21,32 @@ class TagQueue { double delayTime() const { return now() - startTime; } }; - std::map rateInfos; - std::map releasedInEpoch; + // Track the budgets for each tag + TransactionTagMap rateInfos; + + // Requests that have not yet been processed Deque newRequests; + + // Requests that have been delayed at least once Deque delayedRequests; - bool canStart(TransactionTag tag, int64_t count) const; - bool canStart(GetReadVersionRequest req) const; - void startEpoch(); - void endEpoch(double elapsed); + // Checks if count transactions can be released, given that + // alreadyReleased transactions have already been released in this epoch. + bool canStart(TransactionTag tag, int64_t alreadyReleased, int64_t count) const; + + // Checks if a request can be released + bool canStart(GetReadVersionRequest req, TransactionTagMap& releasedInEpoch) const; public: + // Called with rates received from ratekeeper void updateRates(TransactionTagMap const& newRates); + + // elapsed indicates the amount of time since the last epoch was run. + // If a request is ready to be executed, it is sent to the deque + // corresponding to its priority. If not, the request remains queued. void runEpoch(double elapsed, SpannedDeque& outBatchPriority, SpannedDeque& outDefaultPriority); + void addRequest(GetReadVersionRequest); }; From 31e8fb0490af069ce98ce30d77a0f85713f36fc8 Mon Sep 17 00:00:00 2001 From: sfc-gh-tclinkenbeard Date: Wed, 5 Oct 2022 09:29:12 -0700 Subject: [PATCH 061/210] Rename epoch to release window in GrvTransactionRateInfo The term "epoch" was used in too many places --- fdbserver/GrvProxyServer.actor.cpp | 12 ++++++------ fdbserver/GrvTransactionRateInfo.actor.cpp | 8 ++++---- fdbserver/TagQueue.actor.cpp | 4 ++-- .../include/fdbserver/GrvTransactionRateInfo.h | 18 +++++++++--------- 4 files changed, 21 insertions(+), 21 deletions(-) diff --git a/fdbserver/GrvProxyServer.actor.cpp b/fdbserver/GrvProxyServer.actor.cpp index ee1ff02737..70fa90186a 100644 --- a/fdbserver/GrvProxyServer.actor.cpp +++ b/fdbserver/GrvProxyServer.actor.cpp @@ -889,8 +889,8 @@ ACTOR static Future transactionStarter(GrvProxyInterface proxy, } tagQueue.runEpoch(elapsed, defaultQueue, batchQueue); - normalRateInfo.startEpoch(); - batchRateInfo.startEpoch(); + normalRateInfo.startReleaseWindow(); + batchRateInfo.startReleaseWindow(); grvProxyData->stats.transactionLimit = normalRateInfo.getLimit(); grvProxyData->stats.batchTransactionLimit = batchRateInfo.getLimit(); @@ -1000,11 +1000,11 @@ ACTOR static Future transactionStarter(GrvProxyInterface proxy, transactionCount += transactionsStarted[0] + transactionsStarted[1]; batchTransactionCount += batchTotalStarted; - normalRateInfo.endEpoch( + normalRateInfo.endReleaseWindow( systemTotalStarted + normalTotalStarted, systemQueue.empty() && defaultQueue.empty(), elapsed); - batchRateInfo.endEpoch(systemTotalStarted + normalTotalStarted + batchTotalStarted, - systemQueue.empty() && defaultQueue.empty() && batchQueue.empty(), - elapsed); + batchRateInfo.endReleaseWindow(systemTotalStarted + normalTotalStarted + batchTotalStarted, + systemQueue.empty() && defaultQueue.empty() && batchQueue.empty(), + elapsed); if (debugID.present()) { g_traceBatch.addEvent("TransactionDebug", diff --git a/fdbserver/GrvTransactionRateInfo.actor.cpp b/fdbserver/GrvTransactionRateInfo.actor.cpp index 28e6242a78..6b20d0929d 100644 --- a/fdbserver/GrvTransactionRateInfo.actor.cpp +++ b/fdbserver/GrvTransactionRateInfo.actor.cpp @@ -35,7 +35,7 @@ bool GrvTransactionRateInfo::canStart(int64_t numAlreadyStarted, int64_t count) std::min(limit + budget, SERVER_KNOBS->START_TRANSACTION_MAX_TRANSACTIONS_TO_START); } -void GrvTransactionRateInfo::endEpoch(int64_t numStartedAtPriority, bool queueEmptyAtPriority, double elapsed) { +void GrvTransactionRateInfo::endReleaseWindow(int64_t numStartedAtPriority, bool queueEmptyAtPriority, double elapsed) { // Update the budget to accumulate any extra capacity available or remove any excess that was used. // The actual delta is the portion of the limit we didn't use multiplied by the fraction of the window that // elapsed. @@ -79,7 +79,7 @@ void GrvTransactionRateInfo::setRate(double rate) { } } -void GrvTransactionRateInfo::startEpoch() { +void GrvTransactionRateInfo::startReleaseWindow() { // Determine the number of transactions that this proxy is allowed to release // Roughly speaking, this is done by computing the number of transactions over some historical window that we // could have started but didn't, and making that our limit. More precisely, we track a smoothed rate limit and @@ -100,10 +100,10 @@ ACTOR static Future mockClient(GrvTransactionRateInfo* rateInfo, double de loop { state double elapsed = (0.9 + 0.2 * deterministicRandom()->random01()) / desiredRate; wait(delay(elapsed)); - rateInfo->startEpoch(); + rateInfo->startReleaseWindow(); int started = rateInfo->canStart(0, 1) ? 1 : 0; *counter += started; - rateInfo->endEpoch(started, false, elapsed); + rateInfo->endReleaseWindow(started, false, elapsed); } } diff --git a/fdbserver/TagQueue.actor.cpp b/fdbserver/TagQueue.actor.cpp index e363526426..ba808a6633 100644 --- a/fdbserver/TagQueue.actor.cpp +++ b/fdbserver/TagQueue.actor.cpp @@ -44,7 +44,7 @@ void TagQueue::runEpoch(double elapsed, SpannedDeque& outBatchPriority, SpannedDeque& outDefaultPriority) { for (auto& [_, rateInfo] : rateInfos) { - rateInfo.startEpoch(); + rateInfo.startReleaseWindow(); } Deque newDelayedRequests; @@ -94,7 +94,7 @@ void TagQueue::runEpoch(double elapsed, delayedRequests = std::move(newDelayedRequests); for (auto& [tag, rateInfo] : rateInfos) { - rateInfo.endEpoch(std::move(releasedInEpoch)[tag], false, elapsed); + rateInfo.endReleaseWindow(std::move(releasedInEpoch)[tag], false, elapsed); } } diff --git a/fdbserver/include/fdbserver/GrvTransactionRateInfo.h b/fdbserver/include/fdbserver/GrvTransactionRateInfo.h index 9b4a553923..d1a6913afc 100644 --- a/fdbserver/include/fdbserver/GrvTransactionRateInfo.h +++ b/fdbserver/include/fdbserver/GrvTransactionRateInfo.h @@ -24,10 +24,10 @@ // Used by GRV Proxy to enforce rate limits received from the Ratekeeper. // -// Between delays, the GrvTransactionRateInfo executes an "epoch" starting -// with a call to the startEpoch method. Within this epoch, transactions are -// released while canStart returns true. At the end of the epoch, the -// endEpoch method is called, and the budget is updated to add or +// Between waits, the GrvTransactionRateInfo executes an "release window" starting +// with a call to the startReleaseWindow method. Within this release window, transactions are +// released while canStart returns true. At the end of the release window, the +// endReleaseWindow method is called, and the budget is updated to add or // remove capacity. // // Meanwhile, the desired rate is updated through the setRate method. @@ -45,17 +45,17 @@ public: explicit GrvTransactionRateInfo(double rate = 0.0); // Determines the number of transactions that this proxy is allowed to release - // in this epoch. - void startEpoch(); + // in this release window. + void startReleaseWindow(); // Checks if a "count" new transactions can be released, given that // "numAlreadyStarted" transactions have already been released in the - // current epoch. + // current release window. bool canStart(int64_t numAlreadyStarted, int64_t count) const; // Updates the budget to accumulate any extra capacity available or remove any excess that was used. - // Call at the end of an epoch. - void endEpoch(int64_t numStartedAtPriority, bool queueEmptyAtPriority, double elapsed); + // Call at the end of a release window. + void endReleaseWindow(int64_t numStartedAtPriority, bool queueEmptyAtPriority, double elapsed); // Smoothly sets rate. If currently disabled, reenable void setRate(double rate); From f74b724c033f8102466766911a3bb51a3ee8ae82 Mon Sep 17 00:00:00 2001 From: sfc-gh-tclinkenbeard Date: Thu, 6 Oct 2022 22:10:53 -0700 Subject: [PATCH 062/210] Addressed review comments --- fdbserver/GrvTransactionRateInfo.actor.cpp | 9 ++++++--- fdbserver/include/fdbserver/GrvTransactionRateInfo.h | 4 ++-- 2 files changed, 8 insertions(+), 5 deletions(-) diff --git a/fdbserver/GrvTransactionRateInfo.actor.cpp b/fdbserver/GrvTransactionRateInfo.actor.cpp index 6b20d0929d..4b151180a0 100644 --- a/fdbserver/GrvTransactionRateInfo.actor.cpp +++ b/fdbserver/GrvTransactionRateInfo.actor.cpp @@ -37,18 +37,21 @@ bool GrvTransactionRateInfo::canStart(int64_t numAlreadyStarted, int64_t count) void GrvTransactionRateInfo::endReleaseWindow(int64_t numStartedAtPriority, bool queueEmptyAtPriority, double elapsed) { // Update the budget to accumulate any extra capacity available or remove any excess that was used. - // The actual delta is the portion of the limit we didn't use multiplied by the fraction of the window that + // The actual delta is the portion of the limit we didn't use multiplied by the fraction of the rate window that // elapsed. // // We may have exceeded our limit due to the budget or because of higher priority transactions, in which case // this delta will be negative. The delta can also be negative in the event that our limit was negative, which - // can happen if we had already started more transactions in our window than our rate would have allowed. + // can happen if we had already started more transactions in our rate window than our rate would have allowed. // // This budget has the property that when the budget is required to start transactions (because batches are // big), the sum limit+budget will increase linearly from 0 to the batch size over time and decrease by the // batch size upon starting a batch. In other words, this works equivalently to a model where we linearly - // accumulate budget over time in the case that our batches are too big to take advantage of the window based + // accumulate budget over time in the case that our batches are too big to take advantage of the rate window based // limits. + // + // Note that "rate window" here indicates a period of SERVER_KNOBS->START_TRANSACTION_RATE_WINDOW seconds, + // whereas "release window" is the period between wait statements, with duration indicated by "elapsed." budget = std::max(0.0, budget + elapsed * (limit - numStartedAtPriority) / SERVER_KNOBS->START_TRANSACTION_RATE_WINDOW); diff --git a/fdbserver/include/fdbserver/GrvTransactionRateInfo.h b/fdbserver/include/fdbserver/GrvTransactionRateInfo.h index d1a6913afc..11ba5b2f0f 100644 --- a/fdbserver/include/fdbserver/GrvTransactionRateInfo.h +++ b/fdbserver/include/fdbserver/GrvTransactionRateInfo.h @@ -24,11 +24,11 @@ // Used by GRV Proxy to enforce rate limits received from the Ratekeeper. // -// Between waits, the GrvTransactionRateInfo executes an "release window" starting +// Between waits, the GrvTransactionRateInfo executes a "release window" starting // with a call to the startReleaseWindow method. Within this release window, transactions are // released while canStart returns true. At the end of the release window, the // endReleaseWindow method is called, and the budget is updated to add or -// remove capacity. +// remove capacity. // // Meanwhile, the desired rate is updated through the setRate method. // From bbf69b2d0aed00687afcd1df0c8f84a7cd82aac1 Mon Sep 17 00:00:00 2001 From: sfc-gh-tclinkenbeard Date: Fri, 7 Oct 2022 08:17:55 -0700 Subject: [PATCH 063/210] Rename TagQueue::runEpoch to TagQueue::releaseTransactions --- fdbserver/GrvProxyServer.actor.cpp | 2 +- fdbserver/TagQueue.actor.cpp | 8 ++++---- fdbserver/include/fdbserver/TagQueue.h | 6 +++--- 3 files changed, 8 insertions(+), 8 deletions(-) diff --git a/fdbserver/GrvProxyServer.actor.cpp b/fdbserver/GrvProxyServer.actor.cpp index 70fa90186a..f1f04581e1 100644 --- a/fdbserver/GrvProxyServer.actor.cpp +++ b/fdbserver/GrvProxyServer.actor.cpp @@ -888,7 +888,7 @@ ACTOR static Future transactionStarter(GrvProxyInterface proxy, elapsed = 1e-15; } - tagQueue.runEpoch(elapsed, defaultQueue, batchQueue); + tagQueue.releaseTransactions(elapsed, defaultQueue, batchQueue); normalRateInfo.startReleaseWindow(); batchRateInfo.startReleaseWindow(); diff --git a/fdbserver/TagQueue.actor.cpp b/fdbserver/TagQueue.actor.cpp index ba808a6633..e0dc9ec7df 100644 --- a/fdbserver/TagQueue.actor.cpp +++ b/fdbserver/TagQueue.actor.cpp @@ -40,9 +40,9 @@ void TagQueue::addRequest(GetReadVersionRequest req) { newRequests.push_back(req); } -void TagQueue::runEpoch(double elapsed, - SpannedDeque& outBatchPriority, - SpannedDeque& outDefaultPriority) { +void TagQueue::releaseTransactions(double elapsed, + SpannedDeque& outBatchPriority, + SpannedDeque& outDefaultPriority) { for (auto& [_, rateInfo] : rateInfos) { rateInfo.startReleaseWindow(); } @@ -128,7 +128,7 @@ ACTOR static Future mockServer(TagQueue* tagQueue) { loop { state double elapsed = (0.009 + 0.002 * deterministicRandom()->random01()); wait(delay(elapsed)); - tagQueue->runEpoch(elapsed, outBatchPriority, outDefaultPriority); + tagQueue->releaseTransactions(elapsed, outBatchPriority, outDefaultPriority); while (!outBatchPriority.empty()) { outBatchPriority.front().reply.send(GetReadVersionReply{}); outBatchPriority.pop_front(); diff --git a/fdbserver/include/fdbserver/TagQueue.h b/fdbserver/include/fdbserver/TagQueue.h index a930319c9c..a72db8bdee 100644 --- a/fdbserver/include/fdbserver/TagQueue.h +++ b/fdbserver/include/fdbserver/TagQueue.h @@ -44,9 +44,9 @@ public: // elapsed indicates the amount of time since the last epoch was run. // If a request is ready to be executed, it is sent to the deque // corresponding to its priority. If not, the request remains queued. - void runEpoch(double elapsed, - SpannedDeque& outBatchPriority, - SpannedDeque& outDefaultPriority); + void releaseTransactions(double elapsed, + SpannedDeque& outBatchPriority, + SpannedDeque& outDefaultPriority); void addRequest(GetReadVersionRequest); }; From 24a0dd9f173f2cef85f16407762cecb925fe354c Mon Sep 17 00:00:00 2001 From: sfc-gh-tclinkenbeard Date: Fri, 7 Oct 2022 10:31:36 -0700 Subject: [PATCH 064/210] Change GrvProxy tag throttling algorithm. The new algorithm assumes there is only one tag per request, so queues are partitioned by tag. This is a more efficient approach than the old algorithm. --- fdbserver/GrvProxyServer.actor.cpp | 20 +- .../GrvProxyTransactionTagThrottler.actor.cpp | 196 +++++++++++++++ fdbserver/TagQueue.actor.cpp | 235 ------------------ .../GrvProxyTransactionTagThrottler.h | 62 +++++ fdbserver/include/fdbserver/TagQueue.h | 52 ---- 5 files changed, 268 insertions(+), 297 deletions(-) create mode 100644 fdbserver/GrvProxyTransactionTagThrottler.actor.cpp delete mode 100644 fdbserver/TagQueue.actor.cpp create mode 100644 fdbserver/include/fdbserver/GrvProxyTransactionTagThrottler.h delete mode 100644 fdbserver/include/fdbserver/TagQueue.h diff --git a/fdbserver/GrvProxyServer.actor.cpp b/fdbserver/GrvProxyServer.actor.cpp index f1f04581e1..456e8a7110 100644 --- a/fdbserver/GrvProxyServer.actor.cpp +++ b/fdbserver/GrvProxyServer.actor.cpp @@ -27,10 +27,10 @@ #include "fdbclient/CommitProxyInterface.h" #include "fdbclient/GrvProxyInterface.h" #include "fdbclient/VersionVector.h" +#include "fdbserver/GrvProxyTransactionTagThrottler.h" #include "fdbserver/GrvTransactionRateInfo.h" #include "fdbserver/LogSystem.h" #include "fdbserver/LogSystemDiskQueueAdapter.h" -#include "fdbserver/TagQueue.h" #include "fdbserver/WaitFailure.h" #include "fdbserver/WorkerInterface.actor.h" #include "fdbrpc/sim_validation.h" @@ -362,7 +362,7 @@ ACTOR Future getRate(UID myID, GetHealthMetricsReply* detailedHealthMetricsReply, TransactionTagMap* transactionTagCounter, PrioritizedTransactionTagMap* clientThrottledTags, - TagQueue* tagQueue, + GrvProxyTransactionTagThrottler* tagThrottler, GrvProxyStats* stats, GrvProxyData* proxyData) { state Future nextRequestTimer = Never(); @@ -423,7 +423,7 @@ ACTOR Future getRate(UID myID, *clientThrottledTags = std::move(rep.clientThrottledTags.get()); } if (rep.proxyThrottledTags.present()) { - tagQueue->updateRates(rep.proxyThrottledTags.get()); + tagThrottler->updateRates(rep.proxyThrottledTags.get()); } } when(wait(leaseTimeout)) { @@ -469,7 +469,7 @@ ACTOR Future queueGetReadVersionRequests(Reference GrvProxyStats* stats, GrvTransactionRateInfo* batchRateInfo, TransactionTagMap* transactionTagCounter, - TagQueue* tagQueue) { + GrvProxyTransactionTagThrottler* tagThrottler) { getCurrentLineage()->modify(&TransactionLineage::operation) = TransactionLineage::Operation::GetConsistentReadVersion; loop choose { @@ -537,7 +537,7 @@ ACTOR Future queueGetReadVersionRequests(Reference stats->txnDefaultPriorityStartIn += req.transactionCount; ++stats->defaultGRVQueueSize; if (SERVER_KNOBS->ENFORCE_TAG_THROTTLING_ON_PROXIES) { - tagQueue->addRequest(req); + tagThrottler->addRequest(req); } else { defaultQueue->push_back(req); } @@ -554,7 +554,7 @@ ACTOR Future queueGetReadVersionRequests(Reference stats->txnBatchPriorityStartIn += req.transactionCount; ++stats->batchGRVQueueSize; if (SERVER_KNOBS->ENFORCE_TAG_THROTTLING_ON_PROXIES) { - tagQueue->addRequest(req); + tagThrottler->addRequest(req); } else { batchQueue->push_back(req); } @@ -823,7 +823,7 @@ ACTOR static Future transactionStarter(GrvProxyInterface proxy, state int64_t batchTransactionCount = 0; state GrvTransactionRateInfo normalRateInfo(10); state GrvTransactionRateInfo batchRateInfo(0); - state TagQueue tagQueue; + state GrvProxyTransactionTagThrottler tagThrottler; state SpannedDeque systemQueue("GP:transactionStarterSystemQueue"_loc); state SpannedDeque defaultQueue("GP:transactionStarterDefaultQueue"_loc); @@ -850,7 +850,7 @@ ACTOR static Future transactionStarter(GrvProxyInterface proxy, detailedHealthMetricsReply, &transactionTagCounter, &clientThrottledTags, - &tagQueue, + &tagThrottler, &grvProxyData->stats, grvProxyData)); addActor.send(queueGetReadVersionRequests(db, @@ -865,7 +865,7 @@ ACTOR static Future transactionStarter(GrvProxyInterface proxy, &grvProxyData->stats, &batchRateInfo, &transactionTagCounter, - &tagQueue)); + &tagThrottler)); while (std::find(db->get().client.grvProxies.begin(), db->get().client.grvProxies.end(), proxy) == db->get().client.grvProxies.end()) { @@ -888,7 +888,7 @@ ACTOR static Future transactionStarter(GrvProxyInterface proxy, elapsed = 1e-15; } - tagQueue.releaseTransactions(elapsed, defaultQueue, batchQueue); + tagThrottler.releaseTransactions(elapsed, defaultQueue, batchQueue); normalRateInfo.startReleaseWindow(); batchRateInfo.startReleaseWindow(); diff --git a/fdbserver/GrvProxyTransactionTagThrottler.actor.cpp b/fdbserver/GrvProxyTransactionTagThrottler.actor.cpp new file mode 100644 index 0000000000..4583f06993 --- /dev/null +++ b/fdbserver/GrvProxyTransactionTagThrottler.actor.cpp @@ -0,0 +1,196 @@ +#include "fdbserver/GrvProxyTransactionTagThrottler.h" +#include "flow/UnitTest.h" +#include "flow/actorcompiler.h" // must be last include + +void GrvProxyTransactionTagThrottler::updateRates(TransactionTagMap const& newRates) { + for (const auto& [tag, rate] : newRates) { + auto it = queues.find(tag); + if (it == queues.end()) { + queues[tag] = TagQueue(rate); + } else { + it->second.setRate(rate); + } + } + + // Clean up tags that did not appear in newRates + for (auto& [tag, queue] : queues) { + if (newRates.find(tag) == newRates.end()) { + queue.rateInfo.reset(); + if (queue.requests.empty()) { + // FIXME: Use cleaner method of cleanup + queues.erase(tag); + } + } + } +} + +void GrvProxyTransactionTagThrottler::addRequest(GetReadVersionRequest const& req) { + if (req.tags.empty()) { + untaggedRequests.push_back(req); + } else { + auto const& tag = req.tags.begin()->first; + if (req.tags.size() > 1) { + // The GrvProxyTransactionTagThrottler assumes that each GetReadVersionRequest + // has at most one tag. If a transaction uses multiple tags and + // SERVER_KNOBS->ENFORCE_TAG_THROTTLING_ON_PROXIES is enabled, there may be + // unexpected behaviour, because only one tag is used for throttling. + TraceEvent(SevWarnAlways, "GrvProxyTransactionTagThrottler_MultipleTags") + .detail("NumTags", req.tags.size()) + .detail("UsingTag", printable(tag)); + } + queues[tag].requests.emplace_back(req); + } +} + +void GrvProxyTransactionTagThrottler::TagQueue::releaseTransactions( + double elapsed, + SpannedDeque& outBatchPriority, + SpannedDeque& outDefaultPriority) { + Deque newDelayedRequests; + if (rateInfo.present()) + rateInfo.get().startReleaseWindow(); + int transactionsReleased = 0; + while (!requests.empty()) { + auto& delayedReq = requests.front(); + auto& req = delayedReq.req; + auto const count = req.tags.begin()->second; + if (!rateInfo.present() || rateInfo.get().canStart(transactionsReleased, count)) { + req.proxyTagThrottledDuration = now() - delayedReq.startTime; + transactionsReleased += count; + if (req.priority == TransactionPriority::BATCH) { + outBatchPriority.push_back(req); + } else if (req.priority == TransactionPriority::DEFAULT) { + outDefaultPriority.push_back(req); + } else { + // Immediate priority transactions should bypass the GrvProxyTransactionTagThrottler + ASSERT(false); + } + } else { + newDelayedRequests.push_back(delayedReq); + } + requests.pop_front(); + } + if (rateInfo.present()) + rateInfo.get().endReleaseWindow(transactionsReleased, false, elapsed); + requests = std::move(newDelayedRequests); +} + +void GrvProxyTransactionTagThrottler::releaseTransactions(double elapsed, + SpannedDeque& outBatchPriority, + SpannedDeque& outDefaultPriority) { + for (auto& [_, tagQueue] : queues) { + tagQueue.releaseTransactions(elapsed, outBatchPriority, outDefaultPriority); + } +} + +ACTOR static Future mockClient(GrvProxyTransactionTagThrottler* throttler, + TransactionPriority priority, + TagSet tagSet, + int batchSize, + double desiredRate, + TransactionTagMap* counters) { + state Future timer; + state TransactionTagMap tags; + for (const auto& tag : tagSet) { + tags[tag] = batchSize; + } + loop { + timer = delayJittered(static_cast(batchSize) / desiredRate); + GetReadVersionRequest req; + req.tags = tags; + req.priority = priority; + throttler->addRequest(req); + wait(success(req.reply.getFuture()) && timer); + for (auto& [tag, _] : tags) { + (*counters)[tag] += batchSize; + } + } +} + +ACTOR static Future mockServer(GrvProxyTransactionTagThrottler* throttler) { + state SpannedDeque outBatchPriority("TestGrvProxyTransactionTagThrottler_Batch"_loc); + state SpannedDeque outDefaultPriority("TestGrvProxyTransactionTagThrottler_Default"_loc); + loop { + state double elapsed = (0.009 + 0.002 * deterministicRandom()->random01()); + wait(delay(elapsed)); + throttler->releaseTransactions(elapsed, outBatchPriority, outDefaultPriority); + while (!outBatchPriority.empty()) { + outBatchPriority.front().reply.send(GetReadVersionReply{}); + outBatchPriority.pop_front(); + } + while (!outDefaultPriority.empty()) { + outDefaultPriority.front().reply.send(GetReadVersionReply{}); + outDefaultPriority.pop_front(); + } + } +} + +static bool isNear(double desired, int64_t actual) { + return std::abs(desired - actual) * 10 < desired; +} + +// Rate limit set at 10, but client attempts 20 transactions per second. +// Client should be throttled to only 10 transactions per second. +TEST_CASE("/GrvProxyTransactionTagThrottler/Simple") { + state GrvProxyTransactionTagThrottler throttler; + state TagSet tagSet; + state TransactionTagMap counters; + { + TransactionTagMap rates; + rates["sampleTag"_sr] = 10.0; + throttler.updateRates(rates); + } + tagSet.addTag("sampleTag"_sr); + + state Future client = mockClient(&throttler, TransactionPriority::DEFAULT, tagSet, 1, 20.0, &counters); + state Future server = mockServer(&throttler); + wait(timeout(client && server, 60.0, Void())); + TraceEvent("TagQuotaTest_Simple").detail("Counter", counters["sampleTag"_sr]); + ASSERT(isNear(counters["sampleTag"_sr], 60.0 * 10.0)); + return Void(); +} + +// Clients share the available 30 transaction/second budget +TEST_CASE("/GrvProxyTransactionTagThrottler/MultiClient") { + state GrvProxyTransactionTagThrottler throttler; + state TagSet tagSet; + state TransactionTagMap counters; + { + TransactionTagMap rates; + rates["sampleTag"_sr] = 30.0; + throttler.updateRates(rates); + } + tagSet.addTag("sampleTag"_sr); + + state std::vector> clients; + clients.reserve(10); + for (int i = 0; i < 10; ++i) { + clients.push_back(mockClient(&throttler, TransactionPriority::DEFAULT, tagSet, 1, 10.0, &counters)); + } + + state Future server = mockServer(&throttler); + wait(timeout(waitForAll(clients) && server, 60.0, Void())); + TraceEvent("TagQuotaTest_MultiClient").detail("Counter", counters["sampleTag"_sr]); + ASSERT(isNear(counters["sampleTag"_sr], 60.0 * 30.0)); + return Void(); +} + +TEST_CASE("/GrvProxyTransactionTagThrottler/Batch") { + state GrvProxyTransactionTagThrottler throttler; + state TagSet tagSet; + state TransactionTagMap counters; + { + TransactionTagMap rates; + rates["sampleTag"_sr] = 10.0; + throttler.updateRates(rates); + } + tagSet.addTag("sampleTag"_sr); + + state Future client = mockClient(&throttler, TransactionPriority::DEFAULT, tagSet, 5, 20.0, &counters); + state Future server = mockServer(&throttler); + wait(timeout(client && server, 60.0, Void())); + + TraceEvent("TagQuotaTest_Batch").detail("Counter", counters["sampleTag"_sr]); + ASSERT(isNear(counters["sampleTag"_sr], 60.0 * 10.0)); + return Void(); +} diff --git a/fdbserver/TagQueue.actor.cpp b/fdbserver/TagQueue.actor.cpp deleted file mode 100644 index e0dc9ec7df..0000000000 --- a/fdbserver/TagQueue.actor.cpp +++ /dev/null @@ -1,235 +0,0 @@ -#include "fdbserver/TagQueue.h" -#include "flow/UnitTest.h" -#include "flow/actorcompiler.h" // must be last include - -void TagQueue::updateRates(TransactionTagMap const& newRates) { - for (const auto& [tag, rate] : newRates) { - auto it = rateInfos.find(tag); - if (it == rateInfos.end()) { - rateInfos[tag] = GrvTransactionRateInfo(rate); - } else { - it->second.setRate(rate); - } - } - - for (const auto& [tag, _] : rateInfos) { - if (newRates.find(tag) == newRates.end()) { - rateInfos.erase(tag); - } - } -} - -bool TagQueue::canStart(TransactionTag tag, int64_t alreadyReleased, int64_t count) const { - auto it = rateInfos.find(tag); - if (it == rateInfos.end()) { - return true; - } - return it->second.canStart(alreadyReleased, count); -} - -bool TagQueue::canStart(GetReadVersionRequest req, TransactionTagMap& releasedInEpoch) const { - for (const auto& [tag, count] : req.tags) { - if (!canStart(tag, releasedInEpoch[tag], count)) { - return false; - } - } - return true; -} - -void TagQueue::addRequest(GetReadVersionRequest req) { - newRequests.push_back(req); -} - -void TagQueue::releaseTransactions(double elapsed, - SpannedDeque& outBatchPriority, - SpannedDeque& outDefaultPriority) { - for (auto& [_, rateInfo] : rateInfos) { - rateInfo.startReleaseWindow(); - } - - Deque newDelayedRequests; - TransactionTagMap releasedInEpoch; - - while (!delayedRequests.empty()) { - auto& delayedReq = delayedRequests.front(); - auto& req = delayedReq.req; - if (canStart(req, releasedInEpoch)) { - for (const auto& [tag, count] : req.tags) { - releasedInEpoch[tag] += count; - } - req.proxyTagThrottledDuration = delayedReq.delayTime(); - if (req.priority == TransactionPriority::BATCH) { - outBatchPriority.push_back(req); - } else if (req.priority == TransactionPriority::DEFAULT) { - outDefaultPriority.push_back(req); - } else { - // Immediate priority transactions should bypass the TagQueue - ASSERT(false); - } - } else { - newDelayedRequests.push_back(delayedReq); - } - delayedRequests.pop_front(); - } - - while (!newRequests.empty()) { - auto const& req = newRequests.front(); - if (canStart(req, releasedInEpoch)) { - for (const auto& [tag, count] : req.tags) { - releasedInEpoch[tag] += count; - } - if (req.priority == TransactionPriority::BATCH) { - outBatchPriority.push_back(req); - } else if (req.priority == TransactionPriority::DEFAULT) { - outDefaultPriority.push_back(req); - } else { - // Immediate priority transactions should bypass the TagQueue - ASSERT(false); - } - } else { - newDelayedRequests.emplace_back(req); - } - newRequests.pop_front(); - } - - delayedRequests = std::move(newDelayedRequests); - for (auto& [tag, rateInfo] : rateInfos) { - rateInfo.endReleaseWindow(std::move(releasedInEpoch)[tag], false, elapsed); - } -} - -ACTOR static Future mockClient(TagQueue* tagQueue, - TransactionPriority priority, - TagSet tagSet, - int batchSize, - double desiredRate, - TransactionTagMap* counters) { - state Future timer; - state TransactionTagMap tags; - for (const auto& tag : tagSet) { - tags[tag] = batchSize; - } - loop { - timer = delayJittered(static_cast(batchSize) / desiredRate); - GetReadVersionRequest req; - req.tags = tags; - req.priority = priority; - tagQueue->addRequest(req); - wait(success(req.reply.getFuture()) && timer); - for (auto& [tag, _] : tags) { - (*counters)[tag] += batchSize; - } - } -} - -ACTOR static Future mockServer(TagQueue* tagQueue) { - state SpannedDeque outBatchPriority("TestTagQueue_Batch"_loc); - state SpannedDeque outDefaultPriority("TestTagQueue_Default"_loc); - loop { - state double elapsed = (0.009 + 0.002 * deterministicRandom()->random01()); - wait(delay(elapsed)); - tagQueue->releaseTransactions(elapsed, outBatchPriority, outDefaultPriority); - while (!outBatchPriority.empty()) { - outBatchPriority.front().reply.send(GetReadVersionReply{}); - outBatchPriority.pop_front(); - } - while (!outDefaultPriority.empty()) { - outDefaultPriority.front().reply.send(GetReadVersionReply{}); - outDefaultPriority.pop_front(); - } - } -} - -static bool isNear(double desired, int64_t actual) { - return std::abs(desired - actual) * 10 < desired; -} - -// Rate limit set at 10, but client attempts 20 transactions per second. -// Client should be throttled to only 10 transactions per second. -TEST_CASE("/TagQueue/Simple") { - state TagQueue tagQueue; - state TagSet tagSet; - state TransactionTagMap counters; - { - TransactionTagMap rates; - rates["sampleTag"_sr] = 10.0; - tagQueue.updateRates(rates); - } - tagSet.addTag("sampleTag"_sr); - - state Future client = mockClient(&tagQueue, TransactionPriority::DEFAULT, tagSet, 1, 20.0, &counters); - state Future server = mockServer(&tagQueue); - wait(timeout(client && server, 60.0, Void())); - TraceEvent("TagQuotaTest_Simple").detail("Counter", counters["sampleTag"_sr]); - ASSERT(isNear(counters["sampleTag"_sr], 60.0 * 10.0)); - return Void(); -} - -// Throttle based on the tag with the lowest rate -TEST_CASE("/TagQueue/MultiTag") { - state TagQueue tagQueue; - state TagSet tagSet; - state TransactionTagMap counters; - { - TransactionTagMap rates; - rates["sampleTag1"_sr] = 10.0; - rates["sampleTag2"_sr] = 20.0; - tagQueue.updateRates(rates); - } - tagSet.addTag("sampleTag1"_sr); - tagSet.addTag("sampleTag2"_sr); - - state Future client = mockClient(&tagQueue, TransactionPriority::DEFAULT, tagSet, 1, 30.0, &counters); - state Future server = mockServer(&tagQueue); - wait(timeout(client && server, 60.0, Void())); - TraceEvent("TagQuotaTest_MultiTag").detail("Counter", counters["sampleTag1"_sr]); - ASSERT_EQ(counters["sampleTag1"_sr], counters["sampleTag2"_sr]); - ASSERT(isNear(counters["sampleTag1"_sr], 60.0 * 10.0)); - - return Void(); -} - -// Clients share the available 30 transaction/second budget -TEST_CASE("/TagQueue/MultiClient") { - state TagQueue tagQueue; - state TagSet tagSet; - state TransactionTagMap counters; - { - TransactionTagMap rates; - rates["sampleTag"_sr] = 30.0; - tagQueue.updateRates(rates); - } - tagSet.addTag("sampleTag"_sr); - - state std::vector> clients; - clients.reserve(10); - for (int i = 0; i < 10; ++i) { - clients.push_back(mockClient(&tagQueue, TransactionPriority::DEFAULT, tagSet, 1, 10.0, &counters)); - } - - state Future server = mockServer(&tagQueue); - wait(timeout(waitForAll(clients) && server, 60.0, Void())); - TraceEvent("TagQuotaTest_MultiClient").detail("Counter", counters["sampleTag"_sr]); - ASSERT(isNear(counters["sampleTag"_sr], 60.0 * 30.0)); - return Void(); -} - -TEST_CASE("/TagQueue/Batch") { - state TagQueue tagQueue; - state TagSet tagSet; - state TransactionTagMap counters; - { - TransactionTagMap rates; - rates["sampleTag"_sr] = 10.0; - tagQueue.updateRates(rates); - } - tagSet.addTag("sampleTag"_sr); - - state Future client = mockClient(&tagQueue, TransactionPriority::DEFAULT, tagSet, 5, 20.0, &counters); - state Future server = mockServer(&tagQueue); - wait(timeout(client && server, 60.0, Void())); - - TraceEvent("TagQuotaTest_Batch").detail("Counter", counters["sampleTag"_sr]); - ASSERT(isNear(counters["sampleTag"_sr], 60.0 * 10.0)); - return Void(); -} diff --git a/fdbserver/include/fdbserver/GrvProxyTransactionTagThrottler.h b/fdbserver/include/fdbserver/GrvProxyTransactionTagThrottler.h new file mode 100644 index 0000000000..12542e682e --- /dev/null +++ b/fdbserver/include/fdbserver/GrvProxyTransactionTagThrottler.h @@ -0,0 +1,62 @@ +#pragma once + +#include "fdbclient/CommitProxyInterface.h" +#include "fdbclient/TagThrottle.actor.h" +#include "fdbserver/GrvTransactionRateInfo.h" + +// GrvProxyTransactionTagThrottler is used to throttle GetReadVersionRequests based on tag quotas +// before they're pushed into priority-partitioned queues. +// +// A GrvTransactionRateInfo object and a request queue are maintained for each tag. +// The GrvTransactionRateInfo object is used to determine when a request can be released. +// +// Between each set of waits, releaseTransactions is run, releasing queued transactions +// that have passed the tag throttling stage. Transactions that are not yet ready +// are requeued during releaseTransactions. +class GrvProxyTransactionTagThrottler { + struct DelayedRequest { + GetReadVersionRequest req; + double startTime; + + explicit DelayedRequest(GetReadVersionRequest const& req, double startTime = now()) + : req(req), startTime(startTime) {} + }; + + struct TagQueue { + Optional rateInfo; + Deque requests; + + explicit TagQueue(double rate = 0.0) : rateInfo(rate) {} + + void releaseTransactions(double elapsed, + SpannedDeque& outBatchPriority, + SpannedDeque& outDefaultPriority); + + void setRate(double rate) { + if (rateInfo.present()) { + rateInfo.get().setRate(rate); + } else { + rateInfo = GrvTransactionRateInfo(rate); + } + } + }; + + // Track the budgets for each tag + TransactionTagMap queues; + + // These requests are simply passed through with no throttling + Deque untaggedRequests; + +public: + // Called with rates received from ratekeeper + void updateRates(TransactionTagMap const& newRates); + + // elapsed indicates the amount of time since the last epoch was run. + // If a request is ready to be executed, it is sent to the deque + // corresponding to its priority. If not, the request remains queued. + void releaseTransactions(double elapsed, + SpannedDeque& outBatchPriority, + SpannedDeque& outDefaultPriority); + + void addRequest(GetReadVersionRequest const&); +}; diff --git a/fdbserver/include/fdbserver/TagQueue.h b/fdbserver/include/fdbserver/TagQueue.h deleted file mode 100644 index a72db8bdee..0000000000 --- a/fdbserver/include/fdbserver/TagQueue.h +++ /dev/null @@ -1,52 +0,0 @@ -#pragma once - -#include "fdbclient/CommitProxyInterface.h" -#include "fdbclient/TagThrottle.actor.h" -#include "fdbserver/GrvTransactionRateInfo.h" - -// TagQueue is used to throttle GetReadVersionRequests based on tag quotas -// before they're pushed into priority-partitioned queues. -// -// A GrvTransactionRateInfo object is maintained for each tag. This object -// is used to determine when a request can be released. -// -// Between each set of waits, runEpoch is run, releasing queued transactions -// that have passed the tag throttling stage. Transactions that are not yet ready -// are requeued during runEpoch. -class TagQueue { - struct DelayedRequest { - double startTime; - GetReadVersionRequest req; - explicit DelayedRequest(GetReadVersionRequest req) : startTime(now()), req(req) {} - double delayTime() const { return now() - startTime; } - }; - - // Track the budgets for each tag - TransactionTagMap rateInfos; - - // Requests that have not yet been processed - Deque newRequests; - - // Requests that have been delayed at least once - Deque delayedRequests; - - // Checks if count transactions can be released, given that - // alreadyReleased transactions have already been released in this epoch. - bool canStart(TransactionTag tag, int64_t alreadyReleased, int64_t count) const; - - // Checks if a request can be released - bool canStart(GetReadVersionRequest req, TransactionTagMap& releasedInEpoch) const; - -public: - // Called with rates received from ratekeeper - void updateRates(TransactionTagMap const& newRates); - - // elapsed indicates the amount of time since the last epoch was run. - // If a request is ready to be executed, it is sent to the deque - // corresponding to its priority. If not, the request remains queued. - void releaseTransactions(double elapsed, - SpannedDeque& outBatchPriority, - SpannedDeque& outDefaultPriority); - - void addRequest(GetReadVersionRequest); -}; From 93fe133784dc80698909a329100f15b26e2caad3 Mon Sep 17 00:00:00 2001 From: sfc-gh-tclinkenbeard Date: Sat, 8 Oct 2022 11:11:57 -0700 Subject: [PATCH 065/210] Clean up tags in GrvProxyTransactionTagThrottler::updateRates --- .../GrvProxyTransactionTagThrottler.actor.cpp | 65 +++++++++++++++++-- .../GrvProxyTransactionTagThrottler.h | 4 ++ 2 files changed, 65 insertions(+), 4 deletions(-) diff --git a/fdbserver/GrvProxyTransactionTagThrottler.actor.cpp b/fdbserver/GrvProxyTransactionTagThrottler.actor.cpp index 4583f06993..060588eff4 100644 --- a/fdbserver/GrvProxyTransactionTagThrottler.actor.cpp +++ b/fdbserver/GrvProxyTransactionTagThrottler.actor.cpp @@ -16,10 +16,16 @@ void GrvProxyTransactionTagThrottler::updateRates(TransactionTagMap cons for (auto& [tag, queue] : queues) { if (newRates.find(tag) == newRates.end()) { queue.rateInfo.reset(); - if (queue.requests.empty()) { - // FIXME: Use cleaner method of cleanup - queues.erase(tag); - } + } + } + + // TODO: Use std::erase_if in C++20 + for (auto it = queues.begin(); it != queues.end();) { + const auto& [tag, queue] = *it; + if (queue.requests.empty() && !queue.rateInfo.present()) { + it = queues.erase(it); + } else { + ++it; } } } @@ -83,6 +89,10 @@ void GrvProxyTransactionTagThrottler::releaseTransactions(double elapsed, } } +uint32_t GrvProxyTransactionTagThrottler::size() { + return queues.size(); +} + ACTOR static Future mockClient(GrvProxyTransactionTagThrottler* throttler, TransactionPriority priority, TagSet tagSet, @@ -125,6 +135,16 @@ ACTOR static Future mockServer(GrvProxyTransactionTagThrottler* throttler) } } +static TransactionTag getRandomTag() { + TransactionTag result; + auto arr = new (result.arena()) uint8_t[32]; + for (int i = 0; i < 32; ++i) { + arr[i] = (uint8_t)deterministicRandom()->randomInt(0, 256); + } + result.contents() = TransactionTagRef(arr, 32); + return result; +} + static bool isNear(double desired, int64_t actual) { return std::abs(desired - actual) * 10 < desired; } @@ -175,6 +195,7 @@ TEST_CASE("/GrvProxyTransactionTagThrottler/MultiClient") { return Void(); } +// Test processing GetReadVersionRequests that batch several transactions TEST_CASE("/GrvProxyTransactionTagThrottler/Batch") { state GrvProxyTransactionTagThrottler throttler; state TagSet tagSet; @@ -194,3 +215,39 @@ TEST_CASE("/GrvProxyTransactionTagThrottler/Batch") { ASSERT(isNear(counters["sampleTag"_sr], 60.0 * 10.0)); return Void(); } + +// Tests cleanup of tags that are no longer throttled. +TEST_CASE("/GrvProxyTransactionTagThrottler/Cleanup1") { + GrvProxyTransactionTagThrottler throttler; + for (int i = 0; i < 1000; ++i) { + auto const tag = getRandomTag(); + TransactionTagMap rates; + rates[tag] = 10.0; + throttler.updateRates(rates); + ASSERT_EQ(throttler.size(), 1); + } + return Void(); +} + +// Tests cleanup of tags once queues have been emptied +TEST_CASE("/GrvProxyTransactionTagThrottler/Cleanup2") { + GrvProxyTransactionTagThrottler throttler; + { + GetReadVersionRequest req; + req.tags["sampleTag"_sr] = 1; + req.priority = TransactionPriority::DEFAULT; + throttler.addRequest(req); + } + ASSERT_EQ(throttler.size(), 1); + throttler.updateRates(TransactionTagMap{}); + ASSERT_EQ(throttler.size(), 1); + { + SpannedDeque outBatchPriority("TestGrvProxyTransactionTagThrottler_Batch"_loc); + SpannedDeque outDefaultPriority("TestGrvProxyTransactionTagThrottler_Default"_loc); + throttler.releaseTransactions(0.1, outBatchPriority, outDefaultPriority); + } + // Calling updates cleans up the queues in throttler + throttler.updateRates(TransactionTagMap{}); + ASSERT_EQ(throttler.size(), 0); + return Void(); +} diff --git a/fdbserver/include/fdbserver/GrvProxyTransactionTagThrottler.h b/fdbserver/include/fdbserver/GrvProxyTransactionTagThrottler.h index 12542e682e..219b277771 100644 --- a/fdbserver/include/fdbserver/GrvProxyTransactionTagThrottler.h +++ b/fdbserver/include/fdbserver/GrvProxyTransactionTagThrottler.h @@ -59,4 +59,8 @@ public: SpannedDeque& outDefaultPriority); void addRequest(GetReadVersionRequest const&); + +public: // testing + // Returns number of tags tracked + uint32_t size(); }; From ebe956b2281d0a369c577457dd4ace7d1409ade8 Mon Sep 17 00:00:00 2001 From: sfc-gh-tclinkenbeard Date: Sat, 8 Oct 2022 11:30:50 -0700 Subject: [PATCH 066/210] Add copyright headers to GrvProxyTransactionTagThrottler files --- .../GrvProxyTransactionTagThrottler.actor.cpp | 20 +++++++++++++++++++ .../GrvProxyTransactionTagThrottler.h | 20 +++++++++++++++++++ 2 files changed, 40 insertions(+) diff --git a/fdbserver/GrvProxyTransactionTagThrottler.actor.cpp b/fdbserver/GrvProxyTransactionTagThrottler.actor.cpp index 060588eff4..e80adebb0f 100644 --- a/fdbserver/GrvProxyTransactionTagThrottler.actor.cpp +++ b/fdbserver/GrvProxyTransactionTagThrottler.actor.cpp @@ -1,3 +1,23 @@ +/* + * GrvProxyTransactionTagThrottler.actor.cpp + * + * This source file is part of the FoundationDB open source project + * + * Copyright 2013-2022 Apple Inc. and the FoundationDB project authors + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + #include "fdbserver/GrvProxyTransactionTagThrottler.h" #include "flow/UnitTest.h" #include "flow/actorcompiler.h" // must be last include diff --git a/fdbserver/include/fdbserver/GrvProxyTransactionTagThrottler.h b/fdbserver/include/fdbserver/GrvProxyTransactionTagThrottler.h index 219b277771..8c824ae0b3 100644 --- a/fdbserver/include/fdbserver/GrvProxyTransactionTagThrottler.h +++ b/fdbserver/include/fdbserver/GrvProxyTransactionTagThrottler.h @@ -1,3 +1,23 @@ +/* + * GrvProxyTransactionTagThrottler.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 #include "fdbclient/CommitProxyInterface.h" From 128e8c43da4e4f41d0687e6217640c08a908cdce Mon Sep 17 00:00:00 2001 From: sfc-gh-tclinkenbeard Date: Sat, 8 Oct 2022 11:57:49 -0700 Subject: [PATCH 067/210] Include unactorcompiler.h in actor header files --- .../fdbserver/workloads/MetaclusterConsistency.actor.h | 4 +++- .../include/fdbserver/workloads/TenantConsistency.actor.h | 4 +++- 2 files changed, 6 insertions(+), 2 deletions(-) diff --git a/fdbserver/include/fdbserver/workloads/MetaclusterConsistency.actor.h b/fdbserver/include/fdbserver/workloads/MetaclusterConsistency.actor.h index 7b8275d2b7..25f3fcae19 100644 --- a/fdbserver/include/fdbserver/workloads/MetaclusterConsistency.actor.h +++ b/fdbserver/include/fdbserver/workloads/MetaclusterConsistency.actor.h @@ -360,4 +360,6 @@ public: Future run() { return run(this); } }; -#endif \ No newline at end of file +#include "flow/unactorcompiler.h" + +#endif diff --git a/fdbserver/include/fdbserver/workloads/TenantConsistency.actor.h b/fdbserver/include/fdbserver/workloads/TenantConsistency.actor.h index 86fe3e9c09..2c73e3a363 100644 --- a/fdbserver/include/fdbserver/workloads/TenantConsistency.actor.h +++ b/fdbserver/include/fdbserver/workloads/TenantConsistency.actor.h @@ -226,4 +226,6 @@ public: Future run() { return run(this); } }; -#endif \ No newline at end of file +#include "flow/unactorcompiler.h" + +#endif From f1cb4e40f514d92520d8ed1a342ea4bf175aa861 Mon Sep 17 00:00:00 2001 From: sfc-gh-tclinkenbeard Date: Sun, 9 Oct 2022 10:03:03 -0700 Subject: [PATCH 068/210] Avoid adding untagged requests to GrvProxyTransactionTagThrottler --- .../include/fdbclient/CommitProxyInterface.h | 2 ++ fdbserver/GrvProxyServer.actor.cpp | 4 +-- .../GrvProxyTransactionTagThrottler.actor.cpp | 25 ++++++++----------- .../GrvProxyTransactionTagThrottler.h | 3 --- 4 files changed, 15 insertions(+), 19 deletions(-) diff --git a/fdbclient/include/fdbclient/CommitProxyInterface.h b/fdbclient/include/fdbclient/CommitProxyInterface.h index 2e3ae6a3cd..6c53f78a3e 100644 --- a/fdbclient/include/fdbclient/CommitProxyInterface.h +++ b/fdbclient/include/fdbclient/CommitProxyInterface.h @@ -309,6 +309,8 @@ struct GetReadVersionRequest : TimedRequest { bool operator<(GetReadVersionRequest const& rhs) const { return priority < rhs.priority; } + bool isTagged() const { return !tags.empty(); } + template void serialize(Ar& ar) { serializer(ar, transactionCount, flags, tags, debugID, reply, spanContext, maxVersion); diff --git a/fdbserver/GrvProxyServer.actor.cpp b/fdbserver/GrvProxyServer.actor.cpp index 456e8a7110..bb5fea8584 100644 --- a/fdbserver/GrvProxyServer.actor.cpp +++ b/fdbserver/GrvProxyServer.actor.cpp @@ -536,7 +536,7 @@ ACTOR Future queueGetReadVersionRequests(Reference stats->txnStartIn += req.transactionCount; stats->txnDefaultPriorityStartIn += req.transactionCount; ++stats->defaultGRVQueueSize; - if (SERVER_KNOBS->ENFORCE_TAG_THROTTLING_ON_PROXIES) { + if (SERVER_KNOBS->ENFORCE_TAG_THROTTLING_ON_PROXIES && req.isTagged()) { tagThrottler->addRequest(req); } else { defaultQueue->push_back(req); @@ -553,7 +553,7 @@ ACTOR Future queueGetReadVersionRequests(Reference stats->txnStartIn += req.transactionCount; stats->txnBatchPriorityStartIn += req.transactionCount; ++stats->batchGRVQueueSize; - if (SERVER_KNOBS->ENFORCE_TAG_THROTTLING_ON_PROXIES) { + if (SERVER_KNOBS->ENFORCE_TAG_THROTTLING_ON_PROXIES && req.isTagged()) { tagThrottler->addRequest(req); } else { batchQueue->push_back(req); diff --git a/fdbserver/GrvProxyTransactionTagThrottler.actor.cpp b/fdbserver/GrvProxyTransactionTagThrottler.actor.cpp index e80adebb0f..4701a977ba 100644 --- a/fdbserver/GrvProxyTransactionTagThrottler.actor.cpp +++ b/fdbserver/GrvProxyTransactionTagThrottler.actor.cpp @@ -51,21 +51,18 @@ void GrvProxyTransactionTagThrottler::updateRates(TransactionTagMap cons } void GrvProxyTransactionTagThrottler::addRequest(GetReadVersionRequest const& req) { - if (req.tags.empty()) { - untaggedRequests.push_back(req); - } else { - auto const& tag = req.tags.begin()->first; - if (req.tags.size() > 1) { - // The GrvProxyTransactionTagThrottler assumes that each GetReadVersionRequest - // has at most one tag. If a transaction uses multiple tags and - // SERVER_KNOBS->ENFORCE_TAG_THROTTLING_ON_PROXIES is enabled, there may be - // unexpected behaviour, because only one tag is used for throttling. - TraceEvent(SevWarnAlways, "GrvProxyTransactionTagThrottler_MultipleTags") - .detail("NumTags", req.tags.size()) - .detail("UsingTag", printable(tag)); - } - queues[tag].requests.emplace_back(req); + ASSERT(req.isTagged()); + auto const& tag = req.tags.begin()->first; + if (req.tags.size() > 1) { + // The GrvProxyTransactionTagThrottler assumes that each GetReadVersionRequest + // has at most one tag. If a transaction uses multiple tags and + // SERVER_KNOBS->ENFORCE_TAG_THROTTLING_ON_PROXIES is enabled, there may be + // unexpected behaviour, because only one tag is used for throttling. + TraceEvent(SevWarnAlways, "GrvProxyTransactionTagThrottler_MultipleTags") + .detail("NumTags", req.tags.size()) + .detail("UsingTag", printable(tag)); } + queues[tag].requests.emplace_back(req); } void GrvProxyTransactionTagThrottler::TagQueue::releaseTransactions( diff --git a/fdbserver/include/fdbserver/GrvProxyTransactionTagThrottler.h b/fdbserver/include/fdbserver/GrvProxyTransactionTagThrottler.h index 8c824ae0b3..243af3b359 100644 --- a/fdbserver/include/fdbserver/GrvProxyTransactionTagThrottler.h +++ b/fdbserver/include/fdbserver/GrvProxyTransactionTagThrottler.h @@ -64,9 +64,6 @@ class GrvProxyTransactionTagThrottler { // Track the budgets for each tag TransactionTagMap queues; - // These requests are simply passed through with no throttling - Deque untaggedRequests; - public: // Called with rates received from ratekeeper void updateRates(TransactionTagMap const& newRates); From 671c4fe0a2e5292d19c63ba97c0906eabac1c3a5 Mon Sep 17 00:00:00 2001 From: sfc-gh-tclinkenbeard Date: Mon, 10 Oct 2022 17:17:41 -0700 Subject: [PATCH 069/210] Make GrvProxyTransactionTagThrottler a FIFO queue for unthrottled transactions, even for transactions with different tags --- .../GrvProxyTransactionTagThrottler.actor.cpp | 119 ++++++++++++------ fdbserver/GrvTransactionRateInfo.actor.cpp | 4 + .../GrvProxyTransactionTagThrottler.h | 26 ++-- 3 files changed, 99 insertions(+), 50 deletions(-) diff --git a/fdbserver/GrvProxyTransactionTagThrottler.actor.cpp b/fdbserver/GrvProxyTransactionTagThrottler.actor.cpp index 4701a977ba..104349d2b6 100644 --- a/fdbserver/GrvProxyTransactionTagThrottler.actor.cpp +++ b/fdbserver/GrvProxyTransactionTagThrottler.actor.cpp @@ -22,6 +22,20 @@ #include "flow/UnitTest.h" #include "flow/actorcompiler.h" // must be last include +uint64_t GrvProxyTransactionTagThrottler::DelayedRequest::lastSequenceNumber = 0; + +void GrvProxyTransactionTagThrottler::DelayedRequest::updateProxyTagThrottledDuration() { + req.proxyTagThrottledDuration = now() - startTime; +} + +void GrvProxyTransactionTagThrottler::TagQueue::setRate(double rate) { + if (rateInfo.present()) { + rateInfo.get().setRate(rate); + } else { + rateInfo = GrvTransactionRateInfo(rate); + } +} + void GrvProxyTransactionTagThrottler::updateRates(TransactionTagMap const& newRates) { for (const auto& [tag, rate] : newRates) { auto it = queues.find(tag); @@ -65,44 +79,79 @@ void GrvProxyTransactionTagThrottler::addRequest(GetReadVersionRequest const& re queues[tag].requests.emplace_back(req); } -void GrvProxyTransactionTagThrottler::TagQueue::releaseTransactions( - double elapsed, - SpannedDeque& outBatchPriority, - SpannedDeque& outDefaultPriority) { - Deque newDelayedRequests; - if (rateInfo.present()) - rateInfo.get().startReleaseWindow(); - int transactionsReleased = 0; - while (!requests.empty()) { - auto& delayedReq = requests.front(); - auto& req = delayedReq.req; - auto const count = req.tags.begin()->second; - if (!rateInfo.present() || rateInfo.get().canStart(transactionsReleased, count)) { - req.proxyTagThrottledDuration = now() - delayedReq.startTime; - transactionsReleased += count; - if (req.priority == TransactionPriority::BATCH) { - outBatchPriority.push_back(req); - } else if (req.priority == TransactionPriority::DEFAULT) { - outDefaultPriority.push_back(req); - } else { - // Immediate priority transactions should bypass the GrvProxyTransactionTagThrottler - ASSERT(false); - } - } else { - newDelayedRequests.push_back(delayedReq); - } - requests.pop_front(); - } - if (rateInfo.present()) - rateInfo.get().endReleaseWindow(transactionsReleased, false, elapsed); - requests = std::move(newDelayedRequests); -} - void GrvProxyTransactionTagThrottler::releaseTransactions(double elapsed, SpannedDeque& outBatchPriority, SpannedDeque& outDefaultPriority) { - for (auto& [_, tagQueue] : queues) { - tagQueue.releaseTransactions(elapsed, outBatchPriority, outDefaultPriority); + struct TagInfo { + // Store pointers here to avoid frequent std::unordered_map lookups + TagQueue* queue; + uint32_t* numReleased; + // Sequence number of the first queued request + int64_t nextSeqNo; + bool operator<(TagInfo const& rhs) const { return nextSeqNo < rhs.nextSeqNo; } + explicit TagInfo(TagQueue& queue, uint32_t& numReleased) : queue(&queue), numReleased(&numReleased) { + ASSERT(!this->queue->requests.empty()); + nextSeqNo = this->queue->requests.front().sequenceNumber; + } + }; + + // Track transactions released for each tag + TransactionTagMap transactionsReleased; + + std::priority_queue pq; + for (auto& [tag, queue] : queues) { + if (queue.rateInfo.present()) { + queue.rateInfo.get().startReleaseWindow(); + } + if (!queue.requests.empty()) { + pq.emplace(queue, transactionsReleased[tag]); + } + } + + while (!pq.empty()) { + auto info = pq.top(); + pq.pop(); + // Used to determine when it is time to start processing another tag + auto const nextQueueSeqNo = pq.empty() ? std::numeric_limits::max() : pq.top().nextSeqNo; + + while (!info.queue->requests.empty()) { + auto& delayedReq = info.queue->requests.front(); + auto count = delayedReq.req.tags.begin()->second; + ASSERT_EQ(info.nextSeqNo, delayedReq.sequenceNumber); + if (info.queue->rateInfo.present() && !info.queue->rateInfo.get().canStart(*(info.numReleased), count)) { + // Cannot release any more transaction from this tag (don't push the info back into pq) + CODE_PROBE(true, "GrvProxyTransactionTagThrottler::releaseTransactions : Throttling transaction"); + break; + } else { + if (info.nextSeqNo < nextQueueSeqNo) { + // Releasing transaction + *(info.numReleased) += count; + delayedReq.updateProxyTagThrottledDuration(); + if (delayedReq.req.priority == TransactionPriority::BATCH) { + outBatchPriority.push_back(delayedReq.req); + } else if (delayedReq.req.priority == TransactionPriority::DEFAULT) { + outDefaultPriority.push_back(delayedReq.req); + } else { + // Immediate priority transactions should bypass the GrvProxyTransactionTagThrottler + ASSERT(false); + } + info.queue->requests.pop_front(); + if (!info.queue->requests.empty()) { + info.nextSeqNo = info.queue->requests.front().sequenceNumber; + } + } else { + CODE_PROBE( + true, "GrvProxyTransactionTagThrottler::releaseTransactions : Switching tags to preserve FIFO"); + pq.push(info); + break; + } + } + } + } + for (auto& [tag, queue] : queues) { + if (queue.rateInfo.present()) { + queue.rateInfo.get().endReleaseWindow(transactionsReleased[tag], false, elapsed); + } } } diff --git a/fdbserver/GrvTransactionRateInfo.actor.cpp b/fdbserver/GrvTransactionRateInfo.actor.cpp index 4b151180a0..8c17866128 100644 --- a/fdbserver/GrvTransactionRateInfo.actor.cpp +++ b/fdbserver/GrvTransactionRateInfo.actor.cpp @@ -31,6 +31,10 @@ GrvTransactionRateInfo::GrvTransactionRateInfo(double rate) } bool GrvTransactionRateInfo::canStart(int64_t numAlreadyStarted, int64_t count) const { + TraceEvent("HERE_CanStart") + .detail("Limit", limit) + .detail("Budget", budget) + .detail("Rate", smoothRate.smoothTotal()); return numAlreadyStarted + count <= std::min(limit + budget, SERVER_KNOBS->START_TRANSACTION_MAX_TRANSACTIONS_TO_START); } diff --git a/fdbserver/include/fdbserver/GrvProxyTransactionTagThrottler.h b/fdbserver/include/fdbserver/GrvProxyTransactionTagThrottler.h index 243af3b359..9dcbe0c66d 100644 --- a/fdbserver/include/fdbserver/GrvProxyTransactionTagThrottler.h +++ b/fdbserver/include/fdbserver/GrvProxyTransactionTagThrottler.h @@ -34,12 +34,18 @@ // that have passed the tag throttling stage. Transactions that are not yet ready // are requeued during releaseTransactions. class GrvProxyTransactionTagThrottler { - struct DelayedRequest { - GetReadVersionRequest req; + class DelayedRequest { + static uint64_t lastSequenceNumber; double startTime; - explicit DelayedRequest(GetReadVersionRequest const& req, double startTime = now()) - : req(req), startTime(startTime) {} + public: + GetReadVersionRequest req; + uint64_t sequenceNumber; + + explicit DelayedRequest(GetReadVersionRequest const& req) + : req(req), startTime(now()), sequenceNumber(++lastSequenceNumber) {} + + void updateProxyTagThrottledDuration(); }; struct TagQueue { @@ -48,17 +54,7 @@ class GrvProxyTransactionTagThrottler { explicit TagQueue(double rate = 0.0) : rateInfo(rate) {} - void releaseTransactions(double elapsed, - SpannedDeque& outBatchPriority, - SpannedDeque& outDefaultPriority); - - void setRate(double rate) { - if (rateInfo.present()) { - rateInfo.get().setRate(rate); - } else { - rateInfo = GrvTransactionRateInfo(rate); - } - } + void setRate(double rate); }; // Track the budgets for each tag From a4e9e525bdbea0d6261ccc50bf0e12e30cabab8d Mon Sep 17 00:00:00 2001 From: sfc-gh-tclinkenbeard Date: Mon, 10 Oct 2022 22:31:09 -0700 Subject: [PATCH 070/210] Apply clang-format to GrvProxyTransactionTagThrottler.actor.cpp --- fdbserver/GrvProxyTransactionTagThrottler.actor.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/fdbserver/GrvProxyTransactionTagThrottler.actor.cpp b/fdbserver/GrvProxyTransactionTagThrottler.actor.cpp index 104349d2b6..431b481a20 100644 --- a/fdbserver/GrvProxyTransactionTagThrottler.actor.cpp +++ b/fdbserver/GrvProxyTransactionTagThrottler.actor.cpp @@ -73,8 +73,8 @@ void GrvProxyTransactionTagThrottler::addRequest(GetReadVersionRequest const& re // SERVER_KNOBS->ENFORCE_TAG_THROTTLING_ON_PROXIES is enabled, there may be // unexpected behaviour, because only one tag is used for throttling. TraceEvent(SevWarnAlways, "GrvProxyTransactionTagThrottler_MultipleTags") - .detail("NumTags", req.tags.size()) - .detail("UsingTag", printable(tag)); + .detail("NumTags", req.tags.size()) + .detail("UsingTag", printable(tag)); } queues[tag].requests.emplace_back(req); } From b76268544f92ca82b6e7e306a8c13d8b57a934a2 Mon Sep 17 00:00:00 2001 From: Xiaoxi Wang Date: Mon, 10 Oct 2022 22:34:21 -0700 Subject: [PATCH 071/210] disallow workload injection in IDDTxnProcessorApiCorrectness --- .../workloads/IDDTxnProcessorApiCorrectness.actor.cpp | 7 ++++--- tests/fast/IDDTxnProcessorApiCorrectness.toml | 3 ++- 2 files changed, 6 insertions(+), 4 deletions(-) diff --git a/fdbserver/workloads/IDDTxnProcessorApiCorrectness.actor.cpp b/fdbserver/workloads/IDDTxnProcessorApiCorrectness.actor.cpp index 0e2500ece8..0d4a242994 100644 --- a/fdbserver/workloads/IDDTxnProcessorApiCorrectness.actor.cpp +++ b/fdbserver/workloads/IDDTxnProcessorApiCorrectness.actor.cpp @@ -71,6 +71,7 @@ struct IDDTxnProcessorApiWorkload : TestWorkload { std::string description() const override { return desc; } Future setup(Database const& cx) override { return enabled ? _setup(cx, this) : Void(); } Future start(Database const& cx) override { return enabled ? _start(cx, this) : Void(); } + void disableFailureInjectionWorkloads(std::set& out) const override { out.insert("all"); } ACTOR Future _setup(Database cx, IDDTxnProcessorApiWorkload* self) { self->real = std::make_shared(cx); @@ -101,7 +102,7 @@ struct IDDTxnProcessorApiWorkload : TestWorkload { ACTOR Future _start(Database cx, IDDTxnProcessorApiWorkload* self) { state int oldMode = wait(setDDMode(cx, 0)); - TraceEvent("RMKStartModeSetting").log(); + TraceEvent("IDDTxnApiTestStartModeSetting").log(); self->mgs = std::make_shared(); self->mgs->configuration = self->ddContext.configuration; self->mock = std::make_shared(self->mgs); @@ -119,9 +120,9 @@ struct IDDTxnProcessorApiWorkload : TestWorkload { // Void())); // Always set the DD mode back, even if we die with an error - TraceEvent("RMKDoneMoving").log(); + TraceEvent("IDDTxnApiTestDoneMoving").log(); wait(success(setDDMode(cx, oldMode))); - TraceEvent("RMKDoneModeSetting").log(); + TraceEvent("IDDTxnApiTestDoneModeSetting").log(); return Void(); } diff --git a/tests/fast/IDDTxnProcessorApiCorrectness.toml b/tests/fast/IDDTxnProcessorApiCorrectness.toml index 8a3ee524d3..27b6c302d3 100644 --- a/tests/fast/IDDTxnProcessorApiCorrectness.toml +++ b/tests/fast/IDDTxnProcessorApiCorrectness.toml @@ -4,6 +4,7 @@ disableTss = true # There's no TSS in MGS this prevent the DD operate TSS mappin [[test]] testTitle = 'IDDTxnProcessorApiCorrectness' + [[test.workload]] testName = 'IDDTxnProcessorApiCorrectness' - testDuration = 10.0 \ No newline at end of file + testDuration = 10.0 From a4c73a5f0ac4caef603d6f39a11651a1597edb12 Mon Sep 17 00:00:00 2001 From: Hui Liu Date: Thu, 6 Oct 2022 09:20:57 -0700 Subject: [PATCH 072/210] add benchmark for zstd --- cmake/CompileZstd.cmake | 2 +- flowbench/BenchZstd.cpp | 252 +++++++++++++++++++++++++++++++++++++++ flowbench/CMakeLists.txt | 3 + 3 files changed, 256 insertions(+), 1 deletion(-) create mode 100644 flowbench/BenchZstd.cpp diff --git a/cmake/CompileZstd.cmake b/cmake/CompileZstd.cmake index 773c149470..94e1fdb0ff 100644 --- a/cmake/CompileZstd.cmake +++ b/cmake/CompileZstd.cmake @@ -23,5 +23,5 @@ function(compile_zstd) endif() endif() - set(ZSTD_LIB_INCLUDE_DIR ${zstd_SOURCE_DIR}/lib PARENT_SCOPE) + set(ZSTD_LIB_INCLUDE_DIR ${zstd_SOURCE_DIR}/lib CACHE INTERNAL ZSTD_LIB_INCLUDE_DIR) endfunction(compile_zstd) diff --git a/flowbench/BenchZstd.cpp b/flowbench/BenchZstd.cpp new file mode 100644 index 0000000000..0465ce119e --- /dev/null +++ b/flowbench/BenchZstd.cpp @@ -0,0 +1,252 @@ +/* + * BenchZstd.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 "benchmark/benchmark.h" +#include "flow/IRandom.h" +#include "flow/DeterministicRandom.h" + +#include +#include +#include +#include +#include + +#ifdef ZSTD_LIB_SUPPORTED + +#define ZSTD_STATIC_LINKING_ONLY +#include "zstd.h" + +// Benchmark zstd performance. To use it: +// # export BM_ZSTD_DATA=path/to/datafile +// # bin/flowbench --benchmark_filter=bench_zstd + +// Compress with raw ZSTD API +static inline std::pair, size_t> compress(const std::string& data, int level) { + const char* srcBegin = data.data(); + size_t destSize = ZSTD_compressBound(data.size()); + std::unique_ptr dest = std::make_unique(destSize); + size_t bytes = ZSTD_compress(dest.get(), destSize, srcBegin, data.size(), level); + return std::make_pair(std::move(dest), bytes); +} + +// Compress with raw ZSTD API ZSTD_compress2 +static inline std::pair, size_t> compress2(const std::string& data, ZSTD_CCtx* cctx) { + const char* srcBegin = data.data(); + size_t destSize = ZSTD_compressBound(data.size()); + std::unique_ptr dest = std::make_unique(destSize); + size_t bytes = ZSTD_compress2(cctx, dest.get(), destSize, srcBegin, data.size()); + return std::make_pair(std::move(dest), bytes); +} + +// Deompress with raw ZSTD API +static inline std::pair, size_t> decompress(const std::string& data) { + size_t destSize = ZSTD_decompressBound(data.data(), data.size()); + std::unique_ptr dest = std::make_unique(destSize); + size_t bytes = ZSTD_decompress(dest.get(), destSize, data.data(), data.size()); + return std::make_pair(std::move(dest), bytes); +} + +// Compress with raw ZSTD stream API +static inline std::pair, size_t> compressAsStream(const std::string& data, + ZSTD_CStream* cstream) { + size_t dstSize = ZSTD_compressBound(data.size()); + std::unique_ptr dst = std::make_unique(dstSize); + + ZSTD_inBuffer in; + in.src = data.data(); + in.size = data.size(); + in.pos = 0; + ZSTD_outBuffer out; + out.dst = dst.get(); + out.size = dstSize; + out.pos = 0; + + ZSTD_compressStream(cstream, &out, &in); + ZSTD_flushStream(cstream, &out); + return std::make_pair(std::move(dst), out.pos); +} + +// Decompress with raw ZSTD stream API +static inline std::pair, size_t> decompressAsStream(const std::string& data, + ZSTD_DStream* dstream) { + size_t destSize = ZSTD_decompressBound(data.data(), data.size()); + std::unique_ptr dest = std::make_unique(destSize); + + ZSTD_inBuffer in; + in.src = data.data(); + in.size = data.size(); + in.pos = 0; + ZSTD_outBuffer out; + out.dst = dest.get(); + out.size = destSize; + out.pos = 0; + + do { + ZSTD_decompressStream(dstream, &out, &in); + } while (in.pos < in.size && out.pos < out.size); + + return std::make_pair(std::move(dest), destSize); +} + +// Generate uncompressed data for compression testing +static std::string genUncompressedData() { + char* dataFileName = std::getenv("BM_ZSTD_DATA"); + if (dataFileName) { + std::ifstream file(dataFileName, std::ios::binary | std::ios::ate); + std::streamsize size = file.tellg(); + file.seekg(0, std::ios::beg); + std::string buf(size, ' '); + file.read(buf.data(), size); + std::printf("Load test data %s: %ld bytes\n", dataFileName, size); + return buf; + } else { + DeterministicRandom random(0x1234567, true); + return random.randomAlphaNumeric(1048576); + } +} + +// Generate compressed data for decompression testing +static std::map genCompressedData() { + std::map result; + std::string data = genUncompressedData(); + // test compression level 1, 3, 9 + for (int level : { 1, 3, 9 }) { + auto compressed = compress(data, level); + result[level] = std::string(compressed.first.get(), compressed.second); + } + return result; +} + +static std::string UNCOMPRESSED = genUncompressedData(); +static std::map COMPRESSED = genCompressedData(); + +static void bench_zstd_compress(benchmark::State& state) { + auto chunkSize = state.range(0); + auto level = state.range(1); + float ratio = 0; + for (auto _ : state) { + size_t compressedSize = 0; + for (int i = 0; i < UNCOMPRESSED.size(); i += chunkSize) { + auto compressed = compress(UNCOMPRESSED.substr(i, chunkSize), level); + compressedSize += compressed.second; + } + ratio = compressedSize * 1.0 / UNCOMPRESSED.size(); + } + state.SetBytesProcessed(UNCOMPRESSED.size() * static_cast(state.iterations())); + state.counters["compression_ratio"] = ratio; +} + +static void bench_zstd_compress2(benchmark::State& state) { + auto chunkSize = state.range(0); + auto level = state.range(1); + float ratio = 0; + ZSTD_CCtx* cctx = ZSTD_createCCtx(); + ZSTD_CCtx_setParameter(cctx, ZSTD_c_compressionLevel, level); + // ZSTD_CCtx_setParameter(cctx, ZSTD_c_strategy, 2); + // ZSTD_CCtx_setParameter(cctx, ZSTD_c_windowLog, 21); + // ZSTD_CCtx_setParameter(cctx, ZSTD_c_hashLog, 17); + // ZSTD_CCtx_setParameter(cctx, ZSTD_c_chainLog, 16); + // ZSTD_CCtx_setParameter(cctx, ZSTD_c_searchLog, 1); + // ZSTD_CCtx_setParameter(cctx, ZSTD_c_minMatch, 5); + for (auto _ : state) { + size_t compressedSize = 0; + for (int i = 0; i < UNCOMPRESSED.size(); i += chunkSize) { + auto compressed = compress2(UNCOMPRESSED.substr(i, chunkSize), cctx); + compressedSize += compressed.second; + } + ratio = compressedSize * 1.0 / UNCOMPRESSED.size(); + } + state.SetBytesProcessed(UNCOMPRESSED.size() * static_cast(state.iterations())); + state.counters["compression_ratio"] = ratio; +} + +static void bench_zstd_compress_stream(benchmark::State& state) { + auto chunkSize = state.range(0); + auto level = state.range(1); + float ratio = 0; + + ZSTD_CStream* cstream = ZSTD_createCStream(); + ZSTD_initCStream(cstream, level); + for (auto _ : state) { + size_t compressedSize = 0; + for (int i = 0; i < UNCOMPRESSED.size(); i += chunkSize) { + auto compressed = compressAsStream(UNCOMPRESSED.substr(i, chunkSize), cstream); + compressedSize += compressed.second; + } + ratio = compressedSize * 1.0 / UNCOMPRESSED.size(); + } + ZSTD_freeCStream(cstream); + state.SetBytesProcessed(UNCOMPRESSED.size() * static_cast(state.iterations())); + state.counters["compression_ratio"] = ratio; +} + +static void bench_zstd_decompress(benchmark::State& state) { + auto level = state.range(0); + for (auto _ : state) { + benchmark::DoNotOptimize(decompress(COMPRESSED[level])); + } + state.SetBytesProcessed(UNCOMPRESSED.size() * static_cast(state.iterations())); +} + +static void bench_zstd_decompress_stream(benchmark::State& state) { + auto level = state.range(0); + ZSTD_DStream* dstream = ZSTD_createDStream(); + ZSTD_initDStream(dstream); + for (auto _ : state) { + benchmark::DoNotOptimize(decompressAsStream(COMPRESSED[level], dstream)); + } + ZSTD_freeDStream(dstream); + state.SetBytesProcessed(UNCOMPRESSED.size() * static_cast(state.iterations())); +} + +// chunk size from 4K to 1MB, compression level from 1, 3, 9 +BENCHMARK(bench_zstd_compress) + ->Args({ 1 << 12, 1 }) + ->Args({ 1 << 18, 1 }) + ->Args({ 1 << 20, 1 }) + ->Args({ 1 << 21, 1 }) + ->Args({ 1 << 22, 1 }) + ->Args({ 1 << 23, 1 }) + ->Args({ 1 << 12, 3 }) + ->Args({ 1 << 14, 3 }) + ->Args({ 1 << 20, 3 }) + ->Args({ 1 << 12, 9 }) + ->Args({ 1 << 14, 9 }) + ->Args({ 1 << 20, 9 }); + +BENCHMARK(bench_zstd_compress2)->Args({ 1 << 18, 1 })->Args({ 1 << 20, 1 }); + +BENCHMARK(bench_zstd_compress_stream) + ->Args({ 1 << 12, 1 }) + ->Args({ 1 << 18, 1 }) + ->Args({ 1 << 20, 1 }) + ->Args({ 1 << 21, 1 }) + ->Args({ 1 << 22, 1 }) + ->Args({ 1 << 23, 1 }) + ->Args({ 1 << 12, 3 }) + ->Args({ 1 << 14, 3 }) + ->Args({ 1 << 20, 3 }) + ->Args({ 1 << 12, 9 }) + ->Args({ 1 << 14, 9 }) + ->Args({ 1 << 20, 9 }); + +BENCHMARK(bench_zstd_decompress)->Arg(1)->Arg(3)->Arg(9); +BENCHMARK(bench_zstd_decompress_stream)->Arg(1)->Arg(3)->Arg(9); +#endif diff --git a/flowbench/CMakeLists.txt b/flowbench/CMakeLists.txt index 92870d992d..f961f3d91c 100644 --- a/flowbench/CMakeLists.txt +++ b/flowbench/CMakeLists.txt @@ -32,4 +32,7 @@ add_subdirectory( ) add_flow_target(EXECUTABLE NAME flowbench SRCS ${FLOWBENCH_SRCS}) target_include_directories(flowbench PRIVATE "${CMAKE_CURRENT_SOURCE_DIR}/include" "${CMAKE_CURRENT_BINARY_DIR}/include" ${CMAKE_CURRENT_BINARY_DIR}/googlebenchmark-src/include) +if(FLOW_USE_ZSTD) + target_include_directories(flowbench PRIVATE ${ZSTD_LIB_INCLUDE_DIR}) +endif() target_link_libraries(flowbench benchmark pthread flow fdbclient) From 55f091a139ef7cbb4731399295ab46f2e9ba94f8 Mon Sep 17 00:00:00 2001 From: Andrew Noyes Date: Mon, 10 Oct 2022 16:43:23 -0700 Subject: [PATCH 073/210] Disable shard_encode_location_metadata for downgrade to 7.1 Using the feature to set knobs from test spec files appears to require using toml, so also rewrite CycleTestRestart-{1,2}.txt to CycleTestRestart-{1,2}.toml --- tests/CMakeLists.txt | 4 +- ...onfigureStorageMigrationTestRestart-1.toml | 4 ++ .../to_7.1.0/CycleTestRestart-1.toml | 49 +++++++++++++++++++ .../to_7.1.0/CycleTestRestart-1.txt | 36 -------------- .../to_7.1.0/CycleTestRestart-2.toml | 37 ++++++++++++++ .../to_7.1.0/CycleTestRestart-2.txt | 28 ----------- 6 files changed, 92 insertions(+), 66 deletions(-) create mode 100644 tests/restarting/to_7.1.0/CycleTestRestart-1.toml delete mode 100644 tests/restarting/to_7.1.0/CycleTestRestart-1.txt create mode 100644 tests/restarting/to_7.1.0/CycleTestRestart-2.toml delete mode 100644 tests/restarting/to_7.1.0/CycleTestRestart-2.txt diff --git a/tests/CMakeLists.txt b/tests/CMakeLists.txt index a2bbcb8a4e..e09cd6af9c 100644 --- a/tests/CMakeLists.txt +++ b/tests/CMakeLists.txt @@ -277,8 +277,8 @@ if(WITH_PYTHON) TEST_FILES restarting/from_7.0.0/UpgradeAndBackupRestore-1.toml restarting/from_7.0.0/UpgradeAndBackupRestore-2.toml) add_fdb_test( - TEST_FILES restarting/to_7.1.0/CycleTestRestart-1.txt - restarting/to_7.1.0/CycleTestRestart-2.txt) + TEST_FILES restarting/to_7.1.0/CycleTestRestart-1.toml + restarting/to_7.1.0/CycleTestRestart-2.toml) add_fdb_test( TEST_FILES restarting/from_7.1.0/SnapTestAttrition-1.txt restarting/from_7.1.0/SnapTestAttrition-2.txt) diff --git a/tests/restarting/to_7.1.0/ConfigureStorageMigrationTestRestart-1.toml b/tests/restarting/to_7.1.0/ConfigureStorageMigrationTestRestart-1.toml index 07640495a0..ec0fc7c6e8 100644 --- a/tests/restarting/to_7.1.0/ConfigureStorageMigrationTestRestart-1.toml +++ b/tests/restarting/to_7.1.0/ConfigureStorageMigrationTestRestart-1.toml @@ -5,6 +5,10 @@ disableHostname=true disableEncryption=true storageEngineExcludeTypes=[3,4] +[[knobs]] +# This can be removed once the lower bound of this downgrade test is a version that understands the new protocol +shard_encode_location_metadata = false + [[test]] testTitle = 'CloggedConfigureDatabaseTest' clearAfterTest = false diff --git a/tests/restarting/to_7.1.0/CycleTestRestart-1.toml b/tests/restarting/to_7.1.0/CycleTestRestart-1.toml new file mode 100644 index 0000000000..e4e5be232c --- /dev/null +++ b/tests/restarting/to_7.1.0/CycleTestRestart-1.toml @@ -0,0 +1,49 @@ +[configuration] +storageEngineExcludeTypes = [-1,-2,3] +maxTLogVersion = 6 +disableTss = true +disableHostname = true +disableEncryption = true + +[[knobs]] +# This can be removed once the lower bound of this downgrade test is a version that understands the new protocol +shard_encode_location_metadata = false + +[[test]] +testTitle = 'Clogged' +clearAfterTest = false + + [[test.workload]] + testName = 'Cycle' + transactionsPerSecond = 500.0 + nodeCount = 2500 + testDuration = 10.0 + expectedRate = 0 + + [[test.workload]] + testName = 'RandomClogging' + testDuration = 10.0 + + [[test.workload]] + testName = 'Rollback' + meanDelay = 10.0 + testDuration = 10.0 + + [[test.workload]] + testName = 'Attrition' + machinesToKill = 10 + machinesToLeave = 3 + reboot = true + testDuration = 10.0 + + [[test.workload]] + testName = 'Attrition' + machinesToKill = 10 + machinesToLeave = 3 + reboot = true + testDuration = 10.0 + + [[test.workload]] + testName = 'SaveAndKill' + restartInfoLocation = 'simfdb/restartInfo.ini' + testDuration = 10.0 diff --git a/tests/restarting/to_7.1.0/CycleTestRestart-1.txt b/tests/restarting/to_7.1.0/CycleTestRestart-1.txt deleted file mode 100644 index e2aeaa4291..0000000000 --- a/tests/restarting/to_7.1.0/CycleTestRestart-1.txt +++ /dev/null @@ -1,36 +0,0 @@ -storageEngineExcludeTypes=-1,-2,3 -maxTLogVersion=6 -disableTss=true -disableHostname=true -disableEncryption=true - -testTitle=Clogged - clearAfterTest=false - testName=Cycle - transactionsPerSecond=500.0 - nodeCount=2500 - testDuration=10.0 - expectedRate=0 - - testName=RandomClogging - testDuration=10.0 - - testName=Rollback - meanDelay=10.0 - testDuration=10.0 - - testName=Attrition - machinesToKill=10 - machinesToLeave=3 - reboot=true - testDuration=10.0 - - testName=Attrition - machinesToKill=10 - machinesToLeave=3 - reboot=true - testDuration=10.0 - - testName=SaveAndKill - restartInfoLocation=simfdb/restartInfo.ini - testDuration=10.0 diff --git a/tests/restarting/to_7.1.0/CycleTestRestart-2.toml b/tests/restarting/to_7.1.0/CycleTestRestart-2.toml new file mode 100644 index 0000000000..51c6422304 --- /dev/null +++ b/tests/restarting/to_7.1.0/CycleTestRestart-2.toml @@ -0,0 +1,37 @@ +[configuration] +storageEngineExcludeTypes = [-1,-2] +maxTLogVersion = 6 +disableTss = true + +[[test]] +testTitle = 'Clogged' +runSetup = false + + [[test.workload]] + testName = 'Cycle' + transactionsPerSecond = 2500.0 + nodeCount = 2500 + testDuration = 10.0 + expectedRate = 0 + + [[test.workload]] + testName = 'RandomClogging' + testDuration = 10.0 + + [[test.workload]] + testName = 'Rollback' + meanDelay = 10.0 + testDuration = 10.0 + + [[test.workload]] + testName = 'Attrition' + machinesToKill = 10 + machinesToLeave = 3 + reboot = true + testDuration = 10.0 + + [[test.workload]] + testName = 'Attrition' + machinesToKill = 10 + machinesToLeave = 3 + reboot = true diff --git a/tests/restarting/to_7.1.0/CycleTestRestart-2.txt b/tests/restarting/to_7.1.0/CycleTestRestart-2.txt deleted file mode 100644 index f11cf32d76..0000000000 --- a/tests/restarting/to_7.1.0/CycleTestRestart-2.txt +++ /dev/null @@ -1,28 +0,0 @@ -storageEngineExcludeTypes=-1,-2 -maxTLogVersion=6 -disableTss=true -testTitle=Clogged - runSetup=false - testName=Cycle - transactionsPerSecond=2500.0 - nodeCount=2500 - testDuration=10.0 - expectedRate=0 - - testName=RandomClogging - testDuration=10.0 - - testName=Rollback - meanDelay=10.0 - testDuration=10.0 - - testName=Attrition - machinesToKill=10 - machinesToLeave=3 - reboot=true - testDuration=10.0 - - testName=Attrition - machinesToKill=10 - machinesToLeave=3 - reboot=true \ No newline at end of file From 693f015d69b8f3a2339fac05ab5891010602ae12 Mon Sep 17 00:00:00 2001 From: Andrew Noyes Date: Tue, 11 Oct 2022 08:42:49 -0700 Subject: [PATCH 074/210] Remove bogus storage engine exclude types Originally, storageEngineExcludeTypes was storageEngineExcludeType, and -1 signified "don't ignore any storage engines". This is no longer meaningful now that it's a list. --- tests/restarting/to_7.1.0/CycleTestRestart-1.toml | 2 +- tests/restarting/to_7.1.0/CycleTestRestart-2.toml | 1 - 2 files changed, 1 insertion(+), 2 deletions(-) diff --git a/tests/restarting/to_7.1.0/CycleTestRestart-1.toml b/tests/restarting/to_7.1.0/CycleTestRestart-1.toml index e4e5be232c..d74ea9ab5d 100644 --- a/tests/restarting/to_7.1.0/CycleTestRestart-1.toml +++ b/tests/restarting/to_7.1.0/CycleTestRestart-1.toml @@ -1,5 +1,5 @@ [configuration] -storageEngineExcludeTypes = [-1,-2,3] +storageEngineExcludeTypes = [3] maxTLogVersion = 6 disableTss = true disableHostname = true diff --git a/tests/restarting/to_7.1.0/CycleTestRestart-2.toml b/tests/restarting/to_7.1.0/CycleTestRestart-2.toml index 51c6422304..05571e0606 100644 --- a/tests/restarting/to_7.1.0/CycleTestRestart-2.toml +++ b/tests/restarting/to_7.1.0/CycleTestRestart-2.toml @@ -1,5 +1,4 @@ [configuration] -storageEngineExcludeTypes = [-1,-2] maxTLogVersion = 6 disableTss = true From 5411b687a60771f00d51779b7486858eb26a2479 Mon Sep 17 00:00:00 2001 From: Andrew Noyes Date: Tue, 11 Oct 2022 09:21:55 -0700 Subject: [PATCH 075/210] We need try_run to work even when linking libc++ statically --- cmake/ConfigureCompiler.cmake | 26 +++++++++++++------------- 1 file changed, 13 insertions(+), 13 deletions(-) diff --git a/cmake/ConfigureCompiler.cmake b/cmake/ConfigureCompiler.cmake index e38f333b58..cb442604d5 100644 --- a/cmake/ConfigureCompiler.cmake +++ b/cmake/ConfigureCompiler.cmake @@ -291,6 +291,19 @@ else() # for more information. #add_compile_options(-fno-builtin-memcpy) + if (USE_LIBCXX) + # Make sure that libc++ can be found be the platform's loader, so that thing's like cmake's "try_run" work. + find_library(LIBCXX_SO_PATH c++ /usr/local/lib) + if (LIBCXX_SO_PATH) + get_filename_component(LIBCXX_SO_DIR ${LIBCXX_SO_PATH} DIRECTORY) + if (APPLE) + set(ENV{DYLD_LIBRARY_PATH} "$ENV{DYLD_LIBRARY_PATH}:${LIBCXX_SO_DIR}") + else() + set(ENV{LD_LIBRARY_PATH} "$ENV{LD_LIBRARY_PATH}:${LIBCXX_SO_DIR}") + endif() + endif() + endif() + if (CLANG OR ICX) if (APPLE OR USE_LIBCXX) set(CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} -stdlib=libc++") @@ -298,19 +311,6 @@ else() if (STATIC_LINK_LIBCXX) set(CMAKE_EXE_LINKER_FLAGS "${CMAKE_EXE_LINKER_FLAGS} -static-libgcc -nostdlib++ -Wl,-Bstatic -lc++ -lc++abi -Wl,-Bdynamic") set(CMAKE_SHARED_LINKER_FLAGS "${CMAKE_SHARED_LINKER_FLAGS} -static-libgcc -nostdlib++ -Wl,-Bstatic -lc++ -lc++abi -Wl,-Bdynamic") - else() - # Make sure that libc++ can be found be the platform's loader, so that thing's like cmake's "try_run" work. - find_library(LIBCXX_SO_PATH c++ /usr/local/lib) - if (LIBCXX_SO_PATH) - get_filename_component(LIBCXX_SO_DIR ${LIBCXX_SO_PATH} DIRECTORY) - if (APPLE) - set(ENV{DYLD_LIBRARY_PATH} "$ENV{DYLD_LIBRARY_PATH}:${LIBCXX_SO_DIR}") - elseif(WIN32) - set(ENV{PATH} "$ENV{PATH};${LIBCXX_SO_DIR}") - else() - set(ENV{LD_LIBRARY_PATH} "$ENV{LD_LIBRARY_PATH}:${LIBCXX_SO_DIR}") - endif() - endif() endif() set(CMAKE_EXE_LINKER_FLAGS "${CMAKE_EXE_LINKER_FLAGS} -stdlib=libc++ -Wl,-build-id=sha1") set(CMAKE_SHARED_LINKER_FLAGS "${CMAKE_SHARED_LINKER_FLAGS} -stdlib=libc++ -Wl,-build-id=sha1") From f16ba3991f7a39071c749018a98ae50004d3a870 Mon Sep 17 00:00:00 2001 From: Xiaoxi Wang Date: Tue, 11 Oct 2022 09:27:05 -0700 Subject: [PATCH 076/210] add comment --- .../IDDTxnProcessorApiCorrectness.actor.cpp | 12 ++++++++---- 1 file changed, 8 insertions(+), 4 deletions(-) diff --git a/fdbserver/workloads/IDDTxnProcessorApiCorrectness.actor.cpp b/fdbserver/workloads/IDDTxnProcessorApiCorrectness.actor.cpp index 0d4a242994..56b15fa0e7 100644 --- a/fdbserver/workloads/IDDTxnProcessorApiCorrectness.actor.cpp +++ b/fdbserver/workloads/IDDTxnProcessorApiCorrectness.actor.cpp @@ -71,7 +71,11 @@ struct IDDTxnProcessorApiWorkload : TestWorkload { std::string description() const override { return desc; } Future setup(Database const& cx) override { return enabled ? _setup(cx, this) : Void(); } Future start(Database const& cx) override { return enabled ? _start(cx, this) : Void(); } - void disableFailureInjectionWorkloads(std::set& out) const override { out.insert("all"); } + + // This workload is not compatible with RandomMoveKeys workload because they will race in changing the DD mode. + // Other workload injections may make no sense because this workload only use the DB at beginning to reading the + // real world key-server mappings. It's not harmful to leave other workload injection enabled for now, though. + void disableFailureInjectionWorkloads(std::set& out) const override { out.insert("RandomMoveKeys"); } ACTOR Future _setup(Database cx, IDDTxnProcessorApiWorkload* self) { self->real = std::make_shared(cx); @@ -101,8 +105,8 @@ struct IDDTxnProcessorApiWorkload : TestWorkload { } ACTOR Future _start(Database cx, IDDTxnProcessorApiWorkload* self) { - state int oldMode = wait(setDDMode(cx, 0)); - TraceEvent("IDDTxnApiTestStartModeSetting").log(); + int oldMode = wait(setDDMode(cx, 0)); + TraceEvent("IDDTxnApiTestStartModeSetting").detail("OldValue", oldMode).log(); self->mgs = std::make_shared(); self->mgs->configuration = self->ddContext.configuration; self->mock = std::make_shared(self->mgs); @@ -121,7 +125,7 @@ struct IDDTxnProcessorApiWorkload : TestWorkload { // Always set the DD mode back, even if we die with an error TraceEvent("IDDTxnApiTestDoneMoving").log(); - wait(success(setDDMode(cx, oldMode))); + wait(success(setDDMode(cx, 1))); TraceEvent("IDDTxnApiTestDoneModeSetting").log(); return Void(); } From 88c7304e04e41abccc3be73c8dccb099fe6aeb7d Mon Sep 17 00:00:00 2001 From: He Liu Date: Tue, 11 Oct 2022 10:24:31 -0700 Subject: [PATCH 077/210] Resolved comments. --- fdbserver/DataDistribution.actor.cpp | 5 +++-- flow/include/flow/error_definitions.h | 2 +- 2 files changed, 4 insertions(+), 3 deletions(-) diff --git a/fdbserver/DataDistribution.actor.cpp b/fdbserver/DataDistribution.actor.cpp index 22e5100354..d6f8fb62cc 100644 --- a/fdbserver/DataDistribution.actor.cpp +++ b/fdbserver/DataDistribution.actor.cpp @@ -1362,7 +1362,7 @@ ACTOR Future auditStorage(Reference self, TriggerAuditReq if (currentAudit->range.contains(req.range)) { audit = it->second.front(); } else { - req.reply.sendError(audit_storage_exeed_max()); + req.reply.sendError(audit_storage_exceeded_request_limit()); return Void(); } } else { @@ -1468,7 +1468,8 @@ ACTOR Future doAuditOnStorageServer(Reference self, try { audit->auditMap.insert(req.range, AuditPhase::Running); - ErrorOr vResult = wait(ssi.auditStorage.getReplyUnlessFailedFor(req, 2, 0)); + ErrorOr vResult = wait(ssi.auditStorage.getReplyUnlessFailedFor( + req, /*sustainedFailureDuration=*/2.0, /*sustainedFailureSlope=*/0)); if (vResult.isError()) { throw vResult.getError(); } diff --git a/flow/include/flow/error_definitions.h b/flow/include/flow/error_definitions.h index fb2b7ad3e2..d493f3c897 100755 --- a/flow/include/flow/error_definitions.h +++ b/flow/include/flow/error_definitions.h @@ -130,7 +130,7 @@ ERROR( restart_cluster_controller, 1218, "Restart cluster controller process" ) ERROR( please_reboot_kv_store, 1219, "Need to reboot the storage engine") ERROR( incompatible_software_version, 1220, "Current software does not support database format" ) ERROR( audit_storage_failed, 1221, "Validate storage consistency operation failed" ) -ERROR( audit_storage_exeed_max, 1222, "Exceeded the max number of allowed concurrent audit storage requests" ) +ERROR( audit_storage_exceeded_request_limit, 1222, "Exceeded the max number of allowed concurrent audit storage requests" ) // 15xx Platform errors ERROR( platform_error, 1500, "Platform error" ) From 86c732fc0e8bd72333c558951a5f8f1dd3abc3b5 Mon Sep 17 00:00:00 2001 From: Andrew Noyes Date: Tue, 11 Oct 2022 11:19:28 -0700 Subject: [PATCH 078/210] Prevent changing the value of ProcessClass enums ConsistencyScanClass was added in the middle of the enum, which changed the values for all subsequent enums. Correct this and prevent it from happening again. --- fdbrpc/include/fdbrpc/Locality.h | 34 +++++++++++++++++++++++++++++++- 1 file changed, 33 insertions(+), 1 deletion(-) diff --git a/fdbrpc/include/fdbrpc/Locality.h b/fdbrpc/include/fdbrpc/Locality.h index 20269b6d05..8c98db88f0 100644 --- a/fdbrpc/include/fdbrpc/Locality.h +++ b/fdbrpc/include/fdbrpc/Locality.h @@ -43,16 +43,42 @@ struct ProcessClass { DataDistributorClass, CoordinatorClass, RatekeeperClass, - ConsistencyScanClass, StorageCacheClass, BackupClass, GrvProxyClass, BlobManagerClass, BlobWorkerClass, EncryptKeyProxyClass, + ConsistencyScanClass, InvalidClass = -1 }; + // class is serialized by enum value, so it's important not to change the + // enum value of a class. New classes should only be added to the end. + static_assert(ProcessClass::UnsetClass == 0); + static_assert(ProcessClass::StorageClass == 1); + static_assert(ProcessClass::TransactionClass == 2); + static_assert(ProcessClass::ResolutionClass == 3); + static_assert(ProcessClass::TesterClass == 4); + static_assert(ProcessClass::CommitProxyClass == 5); + static_assert(ProcessClass::MasterClass == 6); + static_assert(ProcessClass::StatelessClass == 7); + static_assert(ProcessClass::LogClass == 8); + static_assert(ProcessClass::ClusterControllerClass == 9); + static_assert(ProcessClass::LogRouterClass == 10); + static_assert(ProcessClass::FastRestoreClass == 11); + static_assert(ProcessClass::DataDistributorClass == 12); + static_assert(ProcessClass::CoordinatorClass == 13); + static_assert(ProcessClass::RatekeeperClass == 14); + static_assert(ProcessClass::StorageCacheClass == 15); + static_assert(ProcessClass::BackupClass == 16); + static_assert(ProcessClass::GrvProxyClass == 17); + static_assert(ProcessClass::BlobManagerClass == 18); + static_assert(ProcessClass::BlobWorkerClass == 19); + static_assert(ProcessClass::EncryptKeyProxyClass == 20); + static_assert(ProcessClass::ConsistencyScanClass == 21); + static_assert(ProcessClass::InvalidClass == -1); + enum Fitness { BestFit, GoodFit, @@ -86,6 +112,12 @@ struct ProcessClass { int16_t _class; int16_t _source; + // source is serialized by enum value, so it's important not to change the + // enum value of a source. New sources should only be added to the end. + static_assert(ProcessClass::CommandLineSource == 0); + static_assert(ProcessClass::AutoSource == 1); + static_assert(ProcessClass::DBSource == 2); + public: ProcessClass() : _class(UnsetClass), _source(CommandLineSource) {} ProcessClass(ClassType type, ClassSource source) : _class(type), _source(source) {} From 06d9ebd6206609874d24ca2629cea4175310faef Mon Sep 17 00:00:00 2001 From: Josh Slocum Date: Tue, 11 Oct 2022 14:31:14 -0500 Subject: [PATCH 079/210] Adding domain name to blob metadata requests (#8415) --- .../include/fdbclient/BlobMetadataUtils.h | 15 +++++-- .../fdbclient/EncryptKeyProxyInterface.h | 8 ++-- .../fdbclient/GetEncryptCipherKeys.actor.h | 2 +- fdbserver/BlobGranuleServerCommon.actor.cpp | 21 +++++----- fdbserver/EncryptKeyProxy.actor.cpp | 39 +++++++++---------- fdbserver/SimKmsConnector.actor.cpp | 17 +++++--- .../include/fdbserver/KmsConnectorInterface.h | 6 +-- .../workloads/EncryptKeyProxyTest.actor.cpp | 8 ++-- 8 files changed, 63 insertions(+), 53 deletions(-) diff --git a/fdbclient/include/fdbclient/BlobMetadataUtils.h b/fdbclient/include/fdbclient/BlobMetadataUtils.h index d9ed6dd897..4df5c7bfdb 100644 --- a/fdbclient/include/fdbclient/BlobMetadataUtils.h +++ b/fdbclient/include/fdbclient/BlobMetadataUtils.h @@ -25,6 +25,8 @@ #include "flow/FileIdentifier.h" using BlobMetadataDomainId = int64_t; +using BlobMetadataDomainNameRef = StringRef; +using BlobMetadataDomainName = Standalone; /* * There are 3 cases for blob metadata. @@ -38,26 +40,31 @@ using BlobMetadataDomainId = int64_t; struct BlobMetadataDetailsRef { constexpr static FileIdentifier file_identifier = 6685526; BlobMetadataDomainId domainId; + BlobMetadataDomainNameRef domainName; Optional base; VectorRef partitions; BlobMetadataDetailsRef() {} BlobMetadataDetailsRef(Arena& arena, const BlobMetadataDetailsRef& from) - : domainId(from.domainId), partitions(arena, from.partitions) { + : domainId(from.domainId), domainName(arena, from.domainName), partitions(arena, from.partitions) { if (from.base.present()) { base = StringRef(arena, from.base.get()); } } explicit BlobMetadataDetailsRef(BlobMetadataDomainId domainId, + BlobMetadataDomainNameRef domainName, Optional base, VectorRef partitions) - : domainId(domainId), base(base), partitions(partitions) {} + : domainId(domainId), domainName(domainName), base(base), partitions(partitions) {} - int expectedSize() const { return sizeof(BlobMetadataDetailsRef) + partitions.expectedSize(); } + int expectedSize() const { + return sizeof(BlobMetadataDetailsRef) + domainName.size() + (base.present() ? base.get().size() : 0) + + partitions.expectedSize(); + } template void serialize(Ar& ar) { - serializer(ar, domainId, base, partitions); + serializer(ar, domainId, domainName, base, partitions); } }; diff --git a/fdbclient/include/fdbclient/EncryptKeyProxyInterface.h b/fdbclient/include/fdbclient/EncryptKeyProxyInterface.h index 5f4d56eb96..e0c88649c0 100644 --- a/fdbclient/include/fdbclient/EncryptKeyProxyInterface.h +++ b/fdbclient/include/fdbclient/EncryptKeyProxyInterface.h @@ -197,6 +197,7 @@ struct EKPGetLatestBaseCipherKeysReply { } }; +// TODO: also used for blob metadata, fix name struct EKPGetLatestCipherKeysRequestInfo { constexpr static FileIdentifier file_identifier = 2180516; // Encryption domain identifier @@ -206,7 +207,7 @@ struct EKPGetLatestCipherKeysRequestInfo { EncryptCipherDomainNameRef domainName; EKPGetLatestCipherKeysRequestInfo() : domainId(INVALID_ENCRYPT_DOMAIN_ID) {} - EKPGetLatestCipherKeysRequestInfo(const EncryptCipherDomainId dId, StringRef name, Arena& arena) + explicit EKPGetLatestCipherKeysRequestInfo(Arena& arena, const EncryptCipherDomainId dId, StringRef name) : domainId(dId), domainName(StringRef(arena, name)) {} bool operator==(const EKPGetLatestCipherKeysRequestInfo& info) const { @@ -261,16 +262,15 @@ struct EKPGetLatestBlobMetadataReply { struct EKPGetLatestBlobMetadataRequest { constexpr static FileIdentifier file_identifier = 3821549; - std::vector domainIds; + Standalone> domainInfos; Optional debugId; ReplyPromise reply; EKPGetLatestBlobMetadataRequest() {} - explicit EKPGetLatestBlobMetadataRequest(const std::vector& ids) : domainIds(ids) {} template void serialize(Ar& ar) { - serializer(ar, domainIds, debugId, reply); + serializer(ar, domainInfos, debugId, reply); } }; diff --git a/fdbclient/include/fdbclient/GetEncryptCipherKeys.actor.h b/fdbclient/include/fdbclient/GetEncryptCipherKeys.actor.h index 0f93675a6a..bd5b252030 100644 --- a/fdbclient/include/fdbclient/GetEncryptCipherKeys.actor.h +++ b/fdbclient/include/fdbclient/GetEncryptCipherKeys.actor.h @@ -108,7 +108,7 @@ Future>> getL cipherKeys[domain.first] = cachedCipherKey; } else { request.encryptDomainInfos.emplace_back( - domain.first /*domainId*/, domain.second /*domainName*/, request.arena); + request.arena, domain.first /*domainId*/, domain.second /*domainName*/); } } if (request.encryptDomainInfos.empty()) { diff --git a/fdbserver/BlobGranuleServerCommon.actor.cpp b/fdbserver/BlobGranuleServerCommon.actor.cpp index 750eb1592d..d930f294fe 100644 --- a/fdbserver/BlobGranuleServerCommon.actor.cpp +++ b/fdbserver/BlobGranuleServerCommon.actor.cpp @@ -451,12 +451,14 @@ TEST_CASE("/blobgranule/server/common/granulesummary") { } // FIXME: if credentials can expire, refresh periodically -ACTOR Future loadBlobMetadataForTenants(BGTenantMap* self, std::vector tenantMapEntries) { +ACTOR Future loadBlobMetadataForTenants( + BGTenantMap* self, + std::vector> tenantsToLoad) { ASSERT(SERVER_KNOBS->BG_METADATA_SOURCE == "tenant"); - ASSERT(!tenantMapEntries.empty()); - state std::vector domainIds; - for (auto& entry : tenantMapEntries) { - domainIds.push_back(entry.id); + ASSERT(!tenantsToLoad.empty()); + state EKPGetLatestBlobMetadataRequest req; + for (auto& tenant : tenantsToLoad) { + req.domainInfos.emplace_back_deep(req.domainInfos.arena(), tenant.first, StringRef(tenant.second)); } // FIXME: if one tenant gets an error, don't kill whole process @@ -464,8 +466,7 @@ ACTOR Future loadBlobMetadataForTenants(BGTenantMap* self, std::vector requestFuture; if (self->dbInfo.isValid() && self->dbInfo->get().encryptKeyProxy.present()) { - EKPGetLatestBlobMetadataRequest req; - req.domainIds = domainIds; + req.reply.reset(); requestFuture = brokenPromiseToNever(self->dbInfo->get().encryptKeyProxy.get().getLatestBlobMetadata.getReply(req)); } else { @@ -473,7 +474,7 @@ ACTOR Future loadBlobMetadataForTenants(BGTenantMap* self, std::vectortenantInfoById.find(metadata.domainId); @@ -493,7 +494,7 @@ ACTOR Future loadBlobMetadataForTenants(BGTenantMap* self, std::vector> tenants) { - std::vector tenantsToLoad; + std::vector> tenantsToLoad; for (auto entry : tenants) { if (tenantInfoById.insert({ entry.second.id, entry.second }).second) { auto r = makeReference(entry.first, entry.second); @@ -501,7 +502,7 @@ void BGTenantMap::addTenants(std::vector> if (SERVER_KNOBS->BG_METADATA_SOURCE != "tenant") { r->bstoreLoaded.send(Void()); } else { - tenantsToLoad.push_back(entry.second); + tenantsToLoad.push_back({ entry.second.id, entry.first }); } } } diff --git a/fdbserver/EncryptKeyProxy.actor.cpp b/fdbserver/EncryptKeyProxy.actor.cpp index c1ed8bf2e2..f1e1fa1d19 100644 --- a/fdbserver/EncryptKeyProxy.actor.cpp +++ b/fdbserver/EncryptKeyProxy.actor.cpp @@ -690,44 +690,43 @@ ACTOR Future getLatestBlobMetadata(Reference ekpProxy } // Dedup the requested domainIds. - std::unordered_set dedupedDomainIds; - for (auto id : req.domainIds) { - dedupedDomainIds.emplace(id); + std::unordered_map dedupedDomainInfos; + for (auto info : req.domainInfos) { + dedupedDomainInfos.insert({ info.domainId, info.domainName }); } if (dbgTrace.present()) { - dbgTrace.get().detail("NKeys", dedupedDomainIds.size()); - for (BlobMetadataDomainId id : dedupedDomainIds) { + dbgTrace.get().detail("NKeys", dedupedDomainInfos.size()); + for (auto& info : dedupedDomainInfos) { // log domainids queried - dbgTrace.get().detail("BMQ" + std::to_string(id), ""); + dbgTrace.get().detail("BMQ" + std::to_string(info.first), ""); } } // First, check if the requested information is already cached by the server. // Ensure the cached information is within SERVER_KNOBS->BLOB_METADATA_CACHE_TTL time window. - std::vector lookupDomains; - for (BlobMetadataDomainId id : dedupedDomainIds) { - const auto itr = ekpProxyData->blobMetadataDomainIdCache.find(id); + state KmsConnBlobMetadataReq kmsReq; + kmsReq.debugId = req.debugId; + + for (auto& info : dedupedDomainInfos) { + const auto itr = ekpProxyData->blobMetadataDomainIdCache.find(info.first); if (itr != ekpProxyData->blobMetadataDomainIdCache.end() && itr->second.isValid()) { metadataDetails.arena().dependsOn(itr->second.metadataDetails.arena()); metadataDetails.push_back(metadataDetails.arena(), itr->second.metadataDetails); if (dbgTrace.present()) { - dbgTrace.get().detail("BMC" + std::to_string(id), ""); + dbgTrace.get().detail("BMC" + std::to_string(info.first), ""); } - ++ekpProxyData->blobMetadataCacheHits; } else { - lookupDomains.emplace_back(id); - ++ekpProxyData->blobMetadataCacheMisses; + kmsReq.domainInfos.emplace_back(kmsReq.domainInfos.arena(), info.first, info.second); } } - ekpProxyData->baseCipherDomainIdCacheHits += metadataDetails.size(); - ekpProxyData->baseCipherDomainIdCacheMisses += lookupDomains.size(); + ekpProxyData->blobMetadataCacheHits += metadataDetails.size(); - if (!lookupDomains.empty()) { + if (!kmsReq.domainInfos.empty()) { + ekpProxyData->blobMetadataCacheMisses += kmsReq.domainInfos.size(); try { - KmsConnBlobMetadataReq kmsReq(lookupDomains, req.debugId); state double startTime = now(); KmsConnBlobMetadataRep kmsRep = wait(kmsConnectorInf.blobMetadataReq.getReply(kmsReq)); ekpProxyData->kmsBlobMetadataReqLatency.addMeasurement(now() - startTime); @@ -755,7 +754,6 @@ ACTOR Future getLatestBlobMetadata(Reference ekpProxy } req.reply.send(EKPGetLatestBlobMetadataReply(metadataDetails)); - return Void(); } @@ -771,10 +769,11 @@ ACTOR Future refreshBlobMetadataCore(Reference ekpPro try { KmsConnBlobMetadataReq req; req.debugId = debugId; - req.domainIds.reserve(ekpProxyData->blobMetadataDomainIdCache.size()); + req.domainInfos.reserve(req.domainInfos.arena(), ekpProxyData->blobMetadataDomainIdCache.size()); + // TODO add refresh + expire timestamp and filter to only ones that need refreshing for (auto& item : ekpProxyData->blobMetadataDomainIdCache) { - req.domainIds.emplace_back(item.first); + req.domainInfos.emplace_back(req.domainInfos.arena(), item.first, item.second.metadataDetails.domainName); } state double startTime = now(); KmsConnBlobMetadataRep rep = wait(kmsConnectorInf.blobMetadataReq.getReply(req)); diff --git a/fdbserver/SimKmsConnector.actor.cpp b/fdbserver/SimKmsConnector.actor.cpp index da903870bd..dd84c0aeea 100644 --- a/fdbserver/SimKmsConnector.actor.cpp +++ b/fdbserver/SimKmsConnector.actor.cpp @@ -185,10 +185,13 @@ ACTOR Future ekLookupByDomainIds(Reference ctx, return Void(); } - -static Standalone createBlobMetadata(BlobMetadataDomainId domainId) { +// TODO: switch this to use bg_url instead of hardcoding file://fdbblob, so it works as FDBPerfKmsConnector +// FIXME: make this (more) deterministic outside of simulation for FDBPerfKmsConnector +static Standalone createBlobMetadata(BlobMetadataDomainId domainId, + BlobMetadataDomainName domainName) { Standalone metadata; metadata.domainId = domainId; + metadata.domainName = domainName; // 0 == no partition, 1 == suffix partitioned, 2 == storage location partitioned int type = deterministicRandom()->randomInt(0, 3); int partitionCount = (type == 0) ? 0 : deterministicRandom()->randomInt(2, 12); @@ -234,17 +237,19 @@ ACTOR Future blobMetadataLookup(KmsConnectorInterface interf, KmsConnBlobM dbgDIdTrace.get().detail("DbgId", req.debugId.get()); } - for (BlobMetadataDomainId domainId : req.domainIds) { - auto it = simBlobMetadataStore.find(domainId); + for (auto const& domainInfo : req.domainInfos) { + auto it = simBlobMetadataStore.find(domainInfo.domainId); if (it == simBlobMetadataStore.end()) { // construct new blob metadata - it = simBlobMetadataStore.insert({ domainId, createBlobMetadata(domainId) }).first; + it = simBlobMetadataStore + .insert({ domainInfo.domainId, createBlobMetadata(domainInfo.domainId, domainInfo.domainName) }) + .first; } rep.metadataDetails.arena().dependsOn(it->second.arena()); rep.metadataDetails.push_back(rep.metadataDetails.arena(), it->second); } - wait(delayJittered(1.0)); // simulate network delay + wait(delay(deterministicRandom()->random01())); // simulate network delay req.reply.send(rep); diff --git a/fdbserver/include/fdbserver/KmsConnectorInterface.h b/fdbserver/include/fdbserver/KmsConnectorInterface.h index ac31bc5d37..b418eb99ba 100644 --- a/fdbserver/include/fdbserver/KmsConnectorInterface.h +++ b/fdbserver/include/fdbserver/KmsConnectorInterface.h @@ -232,17 +232,15 @@ struct KmsConnBlobMetadataRep { struct KmsConnBlobMetadataReq { constexpr static FileIdentifier file_identifier = 3913147; - std::vector domainIds; + Standalone> domainInfos; Optional debugId; ReplyPromise reply; KmsConnBlobMetadataReq() {} - explicit KmsConnBlobMetadataReq(const std::vector& ids, Optional dbgId) - : domainIds(ids), debugId(dbgId) {} template void serialize(Ar& ar) { - serializer(ar, domainIds, debugId, reply); + serializer(ar, domainInfos, debugId, reply); } }; diff --git a/fdbserver/workloads/EncryptKeyProxyTest.actor.cpp b/fdbserver/workloads/EncryptKeyProxyTest.actor.cpp index c92a9c9c34..970b3162bf 100644 --- a/fdbserver/workloads/EncryptKeyProxyTest.actor.cpp +++ b/fdbserver/workloads/EncryptKeyProxyTest.actor.cpp @@ -71,7 +71,7 @@ struct EncryptKeyProxyTestWorkload : TestWorkload { for (int i = 0; i < self->numDomains / 2; i++) { const EncryptCipherDomainId domainId = self->minDomainId + i; self->domainInfos.emplace_back( - EKPGetLatestCipherKeysRequestInfo(domainId, StringRef(std::to_string(domainId)), self->arena)); + EKPGetLatestCipherKeysRequestInfo(self->arena, domainId, StringRef(std::to_string(domainId)))); } state int nAttempts = 0; @@ -127,14 +127,14 @@ struct EncryptKeyProxyTestWorkload : TestWorkload { for (int i = 0; i < expectedHits; i++) { const EncryptCipherDomainId domainId = self->minDomainId + i; self->domainInfos.emplace_back( - EKPGetLatestCipherKeysRequestInfo(domainId, StringRef(std::to_string(domainId)), self->arena)); + EKPGetLatestCipherKeysRequestInfo(self->arena, domainId, StringRef(std::to_string(domainId)))); } expectedMisses = deterministicRandom()->randomInt(1, self->numDomains / 2); for (int i = 0; i < expectedMisses; i++) { const EncryptCipherDomainId domainId = self->minDomainId + i + self->numDomains / 2 + 1; self->domainInfos.emplace_back( - EKPGetLatestCipherKeysRequestInfo(domainId, StringRef(std::to_string(domainId)), self->arena)); + EKPGetLatestCipherKeysRequestInfo(self->arena, domainId, StringRef(std::to_string(domainId)))); } state int nAttempts = 0; @@ -191,7 +191,7 @@ struct EncryptKeyProxyTestWorkload : TestWorkload { for (int i = 0; i < self->numDomains; i++) { const EncryptCipherDomainId domainId = self->minDomainId + i; self->domainInfos.emplace_back( - EKPGetLatestCipherKeysRequestInfo(domainId, StringRef(std::to_string(domainId)), self->arena)); + EKPGetLatestCipherKeysRequestInfo(self->arena, domainId, StringRef(std::to_string(domainId)))); } EKPGetLatestBaseCipherKeysRequest req; From 7e2764243b41693459e0f9ed84c4f15aea222fb3 Mon Sep 17 00:00:00 2001 From: sfc-gh-tclinkenbeard Date: Tue, 11 Oct 2022 13:04:29 -0700 Subject: [PATCH 080/210] Fix /GrvTransactionRateInfo/Simple unit test --- fdbserver/GrvTransactionRateInfo.actor.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/fdbserver/GrvTransactionRateInfo.actor.cpp b/fdbserver/GrvTransactionRateInfo.actor.cpp index 8c17866128..9f2cca86eb 100644 --- a/fdbserver/GrvTransactionRateInfo.actor.cpp +++ b/fdbserver/GrvTransactionRateInfo.actor.cpp @@ -122,6 +122,6 @@ TEST_CASE("/GrvTransactionRateInfo/Simple") { rateInfo.setRate(10.0); wait(timeout(mockClient(&rateInfo, 20.0, &counter), 60.0, Void())); TraceEvent("GrvTransactionRateInfoTest").detail("Counter", counter); - ASSERT(isNear(60.0 * 20.0, counter)); + ASSERT(isNear(60.0 * 10.0, counter)); return Void(); } From 0466eff56b731c45a208c0046be4f51f767ec110 Mon Sep 17 00:00:00 2001 From: sfc-gh-tclinkenbeard Date: Tue, 11 Oct 2022 13:09:21 -0700 Subject: [PATCH 081/210] Fix potential memory error in GrvTransactionTagThrottler::releaseTransactions --- .../GrvProxyTransactionTagThrottler.actor.cpp | 20 ++++++++++++++----- 1 file changed, 15 insertions(+), 5 deletions(-) diff --git a/fdbserver/GrvProxyTransactionTagThrottler.actor.cpp b/fdbserver/GrvProxyTransactionTagThrottler.actor.cpp index 431b481a20..de0c661b1f 100644 --- a/fdbserver/GrvProxyTransactionTagThrottler.actor.cpp +++ b/fdbserver/GrvProxyTransactionTagThrottler.actor.cpp @@ -96,7 +96,8 @@ void GrvProxyTransactionTagThrottler::releaseTransactions(double elapsed, }; // Track transactions released for each tag - TransactionTagMap transactionsReleased; + std::vector> transactionsReleased; + transactionsReleased.reserve(queues.size()); std::priority_queue pq; for (auto& [tag, queue] : queues) { @@ -104,7 +105,8 @@ void GrvProxyTransactionTagThrottler::releaseTransactions(double elapsed, queue.rateInfo.get().startReleaseWindow(); } if (!queue.requests.empty()) { - pq.emplace(queue, transactionsReleased[tag]); + auto& [_, count] = transactionsReleased.emplace_back(tag, 0); + pq.emplace(queue, count); } } @@ -148,9 +150,17 @@ void GrvProxyTransactionTagThrottler::releaseTransactions(double elapsed, } } } - for (auto& [tag, queue] : queues) { - if (queue.rateInfo.present()) { - queue.rateInfo.get().endReleaseWindow(transactionsReleased[tag], false, elapsed); + + // End release windows for queues with valid rateInfo + { + TransactionTagMap transactionsReleasedMap; + for (const auto& [tag, count] : transactionsReleased) { + transactionsReleasedMap[tag] = count; + } + for (auto& [tag, queue] : queues) { + if (queue.rateInfo.present()) { + queue.rateInfo.get().endReleaseWindow(transactionsReleasedMap[tag], false, elapsed); + } } } } From 5239c491c46272fb3bc4b9ad86920bd3f045e515 Mon Sep 17 00:00:00 2001 From: Markus Pilman Date: Tue, 11 Oct 2022 14:34:10 -0600 Subject: [PATCH 082/210] Audit all AuthZ token usages --- fdbrpc/TokenCache.actor.cpp | 65 +++++++++++++++++++++++++++++++++---- flow/Knobs.cpp | 1 + flow/include/flow/Knobs.h | 1 + 3 files changed, 60 insertions(+), 7 deletions(-) diff --git a/fdbrpc/TokenCache.actor.cpp b/fdbrpc/TokenCache.actor.cpp index afa1fecf10..ac144cc837 100644 --- a/fdbrpc/TokenCache.actor.cpp +++ b/fdbrpc/TokenCache.actor.cpp @@ -8,6 +8,7 @@ #include "flow/network.h" #include +#include #include #include @@ -123,20 +124,65 @@ TEST_CASE("/fdbrpc/authz/LRUCache") { return Void(); } -struct TokenCacheImpl { - struct CacheEntry { - Arena arena; - VectorRef tenants; - double expirationTime = 0.0; - }; +struct CacheEntry { + Arena arena; + VectorRef tenants; + Optional tokenId; + double expirationTime = 0.0; +}; +struct AuditEntry { + NetworkAddress address; + Optional> tokenId; + explicit AuditEntry(NetworkAddress const& address, CacheEntry const& cacheEntry) + : address(address), + tokenId(cacheEntry.tokenId.present() ? Standalone(cacheEntry.tokenId.get(), cacheEntry.arena) + : Optional>()) {} +}; + +bool operator==(AuditEntry const& lhs, AuditEntry const& rhs) { + return (lhs.address == rhs.address) && (lhs.tokenId.present() == rhs.tokenId.present()) && + (!lhs.tokenId.present() || lhs.tokenId.get() == rhs.tokenId.get()); +} + +std::size_t hash_value(AuditEntry const& value) { + std::size_t seed = 0; + boost::hash_combine(seed, value.address); + if (value.tokenId.present()) { + boost::hash_combine(seed, value.tokenId.get()); + } + return seed; +} + +struct TokenCacheImpl { LRUCache cache; - TokenCacheImpl() : cache(FLOW_KNOBS->TOKEN_CACHE_SIZE) {} + boost::unordered_set usedTokens; + Future auditor; + TokenCacheImpl(); bool validate(TenantNameRef tenant, StringRef token); bool validateAndAdd(double currentTime, StringRef token, NetworkAddress const& peer); }; +ACTOR Future tokenCacheAudit(TokenCacheImpl* self) { + state boost::unordered_set audits; + state boost::unordered_set::iterator iter; + loop { + wait(delay(FLOW_KNOBS->AUDIT_TIME_WINDOW)); + audits.swap(self->usedTokens); + for (iter = audits.begin(); iter != audits.end(); ++iter) { + CODE_PROBE(true, "Audit Logging Running"); + TraceEvent("AuditTokenUsed").detail("Client", iter->address).detail("TokenId", iter->tokenId).log(); + wait(yield()); + } + audits.clear(); + } +} + +TokenCacheImpl::TokenCacheImpl() : cache(FLOW_KNOBS->TOKEN_CACHE_SIZE) { + auditor = tokenCacheAudit(this); +} + TokenCache::TokenCache() : impl(new TokenCacheImpl()) {} TokenCache::~TokenCache() { delete impl; @@ -212,6 +258,9 @@ bool TokenCacheImpl::validateAndAdd(double currentTime, StringRef token, Network for (auto tenant : t.tenants.get()) { c.tenants.push_back_deep(c.arena, tenant); } + if (t.tokenId.present()) { + c.tokenId = StringRef(c.arena, t.tokenId.get()); + } cache.insert(StringRef(c.arena, token), c); return true; } @@ -250,6 +299,8 @@ bool TokenCacheImpl::validate(TenantNameRef name, StringRef token) { TraceEvent(SevWarn, "TenantTokenMismatch").detail("From", peer).detail("Tenant", name.toString()); return false; } + // autit logging + usedTokens.insert(AuditEntry(peer, *cachedEntry.get())); return true; } diff --git a/flow/Knobs.cpp b/flow/Knobs.cpp index 0648998a00..95a8f9af75 100644 --- a/flow/Knobs.cpp +++ b/flow/Knobs.cpp @@ -136,6 +136,7 @@ void FlowKnobs::initialize(Randomize randomize, IsSimulated isSimulated) { init( PUBLIC_KEY_FILE_MAX_SIZE, 1024 * 1024 ); init( PUBLIC_KEY_FILE_REFRESH_INTERVAL_SECONDS, 30 ); init( MAX_CACHED_EXPIRED_TOKENS, 1024 ); + init( AUDIT_TIME_WINDOW, 5.0 ); //AsyncFileCached init( PAGE_CACHE_4K, 2LL<<30 ); diff --git a/flow/include/flow/Knobs.h b/flow/include/flow/Knobs.h index 888e8cd990..ac2ef9c58a 100644 --- a/flow/include/flow/Knobs.h +++ b/flow/include/flow/Knobs.h @@ -200,6 +200,7 @@ public: int PUBLIC_KEY_FILE_MAX_SIZE; int PUBLIC_KEY_FILE_REFRESH_INTERVAL_SECONDS; int MAX_CACHED_EXPIRED_TOKENS; + double AUDIT_TIME_WINDOW; // AsyncFileCached int64_t PAGE_CACHE_4K; From 332ee27ec8ae59be74d8d86b6f7a7904ece95c60 Mon Sep 17 00:00:00 2001 From: Andrew Noyes Date: Mon, 3 Oct 2022 14:15:25 -0700 Subject: [PATCH 083/210] Add idempotency id transaction/database options --- fdbclient/vexillographer/fdb.options | 8 ++++++++ 1 file changed, 8 insertions(+) diff --git a/fdbclient/vexillographer/fdb.options b/fdbclient/vexillographer/fdb.options index 8df1bcc150..a8f49f6926 100644 --- a/fdbclient/vexillographer/fdb.options +++ b/fdbclient/vexillographer/fdb.options @@ -201,6 +201,9 @@ description is not currently required but encouraged.