From 3228afefd3ea9cd7b8f2d298aecee29233aa431b Mon Sep 17 00:00:00 2001 From: Steve Atherton Date: Thu, 6 Oct 2022 23:41:28 -0700 Subject: [PATCH 01/95] Unrevert #7578 - storage server PriorityMultiLock and PML rewrite. --- fdbclient/NativeAPI.actor.cpp | 1 + fdbclient/ServerKnobs.cpp | 6 +- fdbclient/include/fdbclient/FDBTypes.h | 8 +- fdbclient/include/fdbclient/NativeAPI.actor.h | 1 + fdbclient/include/fdbclient/ServerKnobs.h | 6 +- fdbserver/VersionedBTree.actor.cpp | 309 +++++---------- fdbserver/storageserver.actor.cpp | 85 +++-- fdbserver/workloads/ReadWrite.actor.cpp | 10 +- flow/include/flow/PriorityMultiLock.actor.h | 358 ++++++++++++++++++ flow/include/flow/genericactors.actor.h | 15 + flowbench/BenchPriorityMultiLock.actor.cpp | 87 +++++ 11 files changed, 649 insertions(+), 237 deletions(-) create mode 100644 flow/include/flow/PriorityMultiLock.actor.h create mode 100644 flowbench/BenchPriorityMultiLock.actor.cpp diff --git a/fdbclient/NativeAPI.actor.cpp b/fdbclient/NativeAPI.actor.cpp index 57986ca0bd..e3815aa706 100644 --- a/fdbclient/NativeAPI.actor.cpp +++ b/fdbclient/NativeAPI.actor.cpp @@ -3990,6 +3990,7 @@ Future getExactRange(Reference trState, req.version = version; req.begin = firstGreaterOrEqual(range.begin); req.end = firstGreaterOrEqual(range.end); + setMatchIndex(req, matchIndex); req.spanContext = span.context; trState->cx->getLatestCommitVersions( diff --git a/fdbclient/ServerKnobs.cpp b/fdbclient/ServerKnobs.cpp index 21970e4eaf..aca7600366 100644 --- a/fdbclient/ServerKnobs.cpp +++ b/fdbclient/ServerKnobs.cpp @@ -787,6 +787,10 @@ void ServerKnobs::initialize(Randomize randomize, ClientKnobs* clientKnobs, IsSi init( QUICK_GET_KEY_VALUES_LIMIT, 2000 ); init( QUICK_GET_KEY_VALUES_LIMIT_BYTES, 1e7 ); init( STORAGE_FEED_QUERY_HARD_LIMIT, 100000 ); + init( STORAGE_SERVER_READ_CONCURRENCY, 70 ); + // Priorities which each ReadType maps to, in enumeration order + init( STORAGESERVER_READ_RANKS, "0,2,1,1,1" ); + init( STORAGESERVER_READ_PRIORITIES, "48,32,8" ); //Wait Failure init( MAX_OUTSTANDING_WAIT_FAILURE_REQUESTS, 250 ); if( randomize && BUGGIFY ) MAX_OUTSTANDING_WAIT_FAILURE_REQUESTS = 2; @@ -898,7 +902,6 @@ void ServerKnobs::initialize(Randomize randomize, ClientKnobs* clientKnobs, IsSi init( REDWOOD_DEFAULT_EXTENT_SIZE, 32 * 1024 * 1024 ); init( REDWOOD_DEFAULT_EXTENT_READ_SIZE, 1024 * 1024 ); init( REDWOOD_EXTENT_CONCURRENT_READS, 4 ); - init( REDWOOD_KVSTORE_CONCURRENT_READS, 64 ); init( REDWOOD_KVSTORE_RANGE_PREFETCH, true ); init( REDWOOD_PAGE_REBUILD_MAX_SLACK, 0.33 ); init( REDWOOD_LAZY_CLEAR_BATCH_SIZE_PAGES, 10 ); @@ -911,6 +914,7 @@ void ServerKnobs::initialize(Randomize randomize, ClientKnobs* clientKnobs, IsSi init( REDWOOD_HISTOGRAM_INTERVAL, 30.0 ); init( REDWOOD_EVICT_UPDATED_PAGES, true ); if( randomize && BUGGIFY ) { REDWOOD_EVICT_UPDATED_PAGES = false; } init( REDWOOD_DECODECACHE_REUSE_MIN_HEIGHT, 2 ); if( randomize && BUGGIFY ) { REDWOOD_DECODECACHE_REUSE_MIN_HEIGHT = deterministicRandom()->randomInt(1, 7); } + init( REDWOOD_PRIORITY_LAUNCHS, "32,32,32,32" ); // Server request latency measurement init( LATENCY_SAMPLE_SIZE, 100000 ); diff --git a/fdbclient/include/fdbclient/FDBTypes.h b/fdbclient/include/fdbclient/FDBTypes.h index e4c03b62bd..4bf497df18 100644 --- a/fdbclient/include/fdbclient/FDBTypes.h +++ b/fdbclient/include/fdbclient/FDBTypes.h @@ -1607,13 +1607,7 @@ struct StorageWiggleValue { } }; -enum class ReadType { - EAGER, - FETCH, - LOW, - NORMAL, - HIGH, -}; +enum class ReadType { EAGER = 0, FETCH = 1, LOW = 2, NORMAL = 3, HIGH = 4, MIN = EAGER, MAX = HIGH }; FDB_DECLARE_BOOLEAN_PARAM(CacheResult); diff --git a/fdbclient/include/fdbclient/NativeAPI.actor.h b/fdbclient/include/fdbclient/NativeAPI.actor.h index 3931182ab0..e778329713 100644 --- a/fdbclient/include/fdbclient/NativeAPI.actor.h +++ b/fdbclient/include/fdbclient/NativeAPI.actor.h @@ -482,6 +482,7 @@ public: Database getDatabase() const { return trState->cx; } static Reference createTrLogInfoProbabilistically(const Database& cx); + Transaction& getTransaction() { return *this; } void setTransactionID(UID id); void setToken(uint64_t token); diff --git a/fdbclient/include/fdbclient/ServerKnobs.h b/fdbclient/include/fdbclient/ServerKnobs.h index 3ca647aa9b..3b776567ab 100644 --- a/fdbclient/include/fdbclient/ServerKnobs.h +++ b/fdbclient/include/fdbclient/ServerKnobs.h @@ -740,6 +740,9 @@ public: int QUICK_GET_KEY_VALUES_LIMIT; int QUICK_GET_KEY_VALUES_LIMIT_BYTES; int STORAGE_FEED_QUERY_HARD_LIMIT; + int STORAGE_SERVER_READ_CONCURRENCY; + std::string STORAGESERVER_READ_RANKS; + std::string STORAGESERVER_READ_PRIORITIES; // Wait Failure int MAX_OUTSTANDING_WAIT_FAILURE_REQUESTS; @@ -869,7 +872,6 @@ public: int REDWOOD_DEFAULT_EXTENT_SIZE; // Extent size for new Redwood files int REDWOOD_DEFAULT_EXTENT_READ_SIZE; // Extent read size for Redwood files int REDWOOD_EXTENT_CONCURRENT_READS; // Max number of simultaneous extent disk reads in progress. - int REDWOOD_KVSTORE_CONCURRENT_READS; // Max number of simultaneous point or range reads in progress. bool REDWOOD_KVSTORE_RANGE_PREFETCH; // Whether to use range read prefetching double REDWOOD_PAGE_REBUILD_MAX_SLACK; // When rebuilding pages, max slack to allow in page int REDWOOD_LAZY_CLEAR_BATCH_SIZE_PAGES; // Number of pages to try to pop from the lazy delete queue and process at @@ -888,6 +890,8 @@ public: bool REDWOOD_EVICT_UPDATED_PAGES; // Whether to prioritize eviction of updated pages from cache. int REDWOOD_DECODECACHE_REUSE_MIN_HEIGHT; // Minimum height for which to keep and reuse page decode caches + std::string REDWOOD_PRIORITY_LAUNCHS; + // Server request latency measurement int LATENCY_SAMPLE_SIZE; double LATENCY_METRICS_LOGGING_INTERVAL; diff --git a/fdbserver/VersionedBTree.actor.cpp b/fdbserver/VersionedBTree.actor.cpp index cd34ea49d1..b066af97c7 100644 --- a/fdbserver/VersionedBTree.actor.cpp +++ b/fdbserver/VersionedBTree.actor.cpp @@ -38,6 +38,7 @@ #include "flow/IRandom.h" #include "flow/Knobs.h" #include "flow/ObjectSerializer.h" +#include "flow/PriorityMultiLock.actor.h" #include "flow/serialize.h" #include "flow/Trace.h" #include "flow/UnitTest.h" @@ -105,201 +106,6 @@ std::string addPrefix(std::string prefix, std::string lines) { return s; } -#define PRIORITYMULTILOCK_DEBUG 0 - -// A multi user lock with a concurrent holder limit where waiters are granted the lock according to -// an integer priority from 0 to maxPriority, inclusive, where higher integers are given priority. -// -// The interface is similar to FlowMutex except that lock holders can drop the lock to release it. -// -// Usage: -// Lock lock = wait(prioritylock.lock(priorityLevel)); -// lock.release(); // Explicit release, or -// // let lock and all copies of lock go out of scope to release -class PriorityMultiLock { - -public: - // Waiting on the lock returns a Lock, which is really just a Promise - // Calling release() is not necessary, it exists in case the Lock holder wants to explicitly release - // the Lock before it goes out of scope. - struct Lock { - void release() { promise.send(Void()); } - - // This is exposed in case the caller wants to use/copy it directly - Promise promise; - }; - -private: - struct Waiter { - Waiter() : queuedTime(now()) {} - Promise lockPromise; - double queuedTime; - }; - - typedef Deque Queue; - -#if PRIORITYMULTILOCK_DEBUG -#define prioritylock_printf(...) printf(__VA_ARGS__) -#else -#define prioritylock_printf(...) -#endif - -public: - PriorityMultiLock(int concurrency, int maxPriority, int launchLimit = std::numeric_limits::max()) - : concurrency(concurrency), available(concurrency), waiting(0), launchLimit(launchLimit) { - waiters.resize(maxPriority + 1); - fRunner = runner(this); - } - - ~PriorityMultiLock() { prioritylock_printf("destruct"); } - - Future lock(int priority = 0) { - prioritylock_printf("lock begin %s\n", toString().c_str()); - - // This shortcut may enable a waiter to jump the line when the releaser loop yields - if (available > 0) { - --available; - Lock p; - addRunner(p); - prioritylock_printf("lock exit immediate %s\n", toString().c_str()); - return p; - } - - Waiter w; - waiters[priority].push_back(w); - ++waiting; - prioritylock_printf("lock exit queued %s\n", toString().c_str()); - return w.lockPromise.getFuture(); - } - - std::string toString() const { - int runnersDone = 0; - for (int i = 0; i < runners.size(); ++i) { - if (runners[i].isReady()) { - ++runnersDone; - } - } - - std::string s = - format("{ ptr=%p concurrency=%d available=%d running=%d waiting=%d runnersQueue=%d runnersDone=%d ", - this, - concurrency, - available, - concurrency - available, - waiting, - runners.size(), - runnersDone); - - for (int i = 0; i < waiters.size(); ++i) { - s += format("p%d_waiters=%u ", i, waiters[i].size()); - } - - s += "}"; - return s; - } - -private: - void addRunner(Lock& lock) { - runners.push_back(map(ready(lock.promise.getFuture()), [=](Void) { - prioritylock_printf("Lock released\n"); - ++available; - if (waiting > 0 || runners.size() > 100) { - release.trigger(); - } - return Void(); - })); - } - - ACTOR static Future runner(PriorityMultiLock* self) { - state int sinceYield = 0; - state Future error = self->brokenOnDestruct.getFuture(); - state int maxPriority = self->waiters.size() - 1; - - // Priority to try to run tasks from next - state int priority = maxPriority; - state Queue* pQueue = &self->waiters[maxPriority]; - - // Track the number of waiters unlocked at the same priority in a row - state int lastPriorityCount = 0; - - loop { - // Cleanup finished runner futures at the front of the runner queue. - while (!self->runners.empty() && self->runners.front().isReady()) { - self->runners.pop_front(); - } - - // Wait for a runner to release its lock - wait(self->release.onTrigger()); - prioritylock_printf("runner wakeup %s\n", self->toString().c_str()); - - if (++sinceYield == 1000) { - sinceYield = 0; - wait(delay(0)); - } - - // While there are available slots and there are waiters, launch tasks - while (self->available > 0 && self->waiting > 0) { - prioritylock_printf("Checking priority=%d lastPriorityCount=%d %s\n", - priority, - lastPriorityCount, - self->toString().c_str()); - - while (!pQueue->empty() && ++lastPriorityCount < self->launchLimit) { - Waiter w = pQueue->front(); - pQueue->pop_front(); - --self->waiting; - Lock lock; - prioritylock_printf(" Running waiter priority=%d wait=%f %s\n", - priority, - now() - w.queuedTime, - self->toString().c_str()); - w.lockPromise.send(lock); - - // Self may have been destructed during the lock callback - if (error.isReady()) { - throw error.getError(); - } - - // If the lock was not already released, add it to the runners future queue - if (lock.promise.canBeSet()) { - self->addRunner(lock); - - // A slot has been consumed, so stop reading from this queue if there aren't any more - if (--self->available == 0) { - break; - } - } - } - - // If there are no more slots available, then don't move to the next priority - if (self->available == 0) { - break; - } - - // Decrease priority, wrapping around to max from 0 - if (priority == 0) { - priority = maxPriority; - } else { - --priority; - } - - pQueue = &self->waiters[priority]; - lastPriorityCount = 0; - } - } - } - - int concurrency; - int available; - int waiting; - int launchLimit; - std::vector waiters; - Deque> runners; - Future fRunner; - AsyncTrigger release; - Promise brokenOnDestruct; -}; - // Some convenience functions for debugging to stringify various structures // Classes can add compatibility by either specializing toString or implementing // std::string toString() const; @@ -1668,6 +1474,8 @@ struct RedwoodMetrics { kvSizeReadByGetRange = Reference( new Histogram(Reference(), "kvSize", "ReadByGetRange", Histogram::Unit::bytes)); + ioLock = nullptr; + // These histograms are used for Btree events, hence level > 0 unsigned int levelCounter = 0; for (RedwoodMetrics::Level& level : levels) { @@ -1710,6 +1518,8 @@ struct RedwoodMetrics { // btree levels and one extra level for non btree level. Level levels[btreeLevels + 1]; metrics metric; + // pointer to the priority multi lock used in pager + PriorityMultiLock* ioLock; Reference kvSizeWritten; Reference kvSizeReadByGet; @@ -1764,9 +1574,12 @@ struct RedwoodMetrics { // The string is a reasonably well formatted page of information void getFields(TraceEvent* e, std::string* s = nullptr, bool skipZeroes = false); + void getIOLockFields(TraceEvent* e, std::string* s = nullptr); + std::string toString(bool clearAfter) { std::string s; getFields(nullptr, &s); + getIOLockFields(nullptr, &s); if (clearAfter) { clear(); @@ -1801,6 +1614,7 @@ ACTOR Future redwoodMetricsLogger() { double elapsed = now() - g_redwoodMetrics.startTime; e.detail("Elapsed", elapsed); g_redwoodMetrics.getFields(&e); + g_redwoodMetrics.getIOLockFields(&e); g_redwoodMetrics.clear(); } } @@ -2197,7 +2011,7 @@ public: bool memoryOnly, Reference keyProvider, Promise errorPromise = {}) - : keyProvider(keyProvider), ioLock(FLOW_KNOBS->MAX_OUTSTANDING, ioMaxPriority, FLOW_KNOBS->MAX_OUTSTANDING / 2), + : keyProvider(keyProvider), ioLock(FLOW_KNOBS->MAX_OUTSTANDING, SERVER_KNOBS->REDWOOD_PRIORITY_LAUNCHS), pageCacheBytes(pageCacheSizeBytes), desiredPageSize(desiredPageSize), desiredExtentSize(desiredExtentSize), filename(filename), memoryOnly(memoryOnly), errorPromise(errorPromise), remapCleanupWindowBytes(remapCleanupWindowBytes), concurrentExtentReads(new FlowLock(concurrentExtentReads)) { @@ -2209,6 +2023,7 @@ public: // This sets the page cache size for all PageCacheT instances using the same evictor pageCache.evictor().sizeLimit = pageCacheBytes; + g_redwoodMetrics.ioLock = &ioLock; if (!g_redwoodMetricsActor.isValid()) { g_redwoodMetricsActor = redwoodMetricsLogger(); } @@ -8071,8 +7886,7 @@ RedwoodRecordRef VersionedBTree::dbEnd("\xff\xff\xff\xff\xff"_sr); class KeyValueStoreRedwood : public IKeyValueStore { public: KeyValueStoreRedwood(std::string filename, UID logID, Reference encryptionKeyProvider) - : m_filename(filename), m_concurrentReads(SERVER_KNOBS->REDWOOD_KVSTORE_CONCURRENT_READS, 0), - prefetch(SERVER_KNOBS->REDWOOD_KVSTORE_RANGE_PREFETCH) { + : m_filename(filename), prefetch(SERVER_KNOBS->REDWOOD_KVSTORE_RANGE_PREFETCH) { int pageSize = BUGGIFY ? deterministicRandom()->randomInt(1000, 4096 * 4) : SERVER_KNOBS->REDWOOD_DEFAULT_PAGE_SIZE; @@ -8137,6 +7951,8 @@ public: ACTOR void shutdown(KeyValueStoreRedwood* self, bool dispose) { TraceEvent(SevInfo, "RedwoodShutdown").detail("Filename", self->m_filename).detail("Dispose", dispose); + g_redwoodMetrics.ioLock = nullptr; + // In simulation, if the instance is being disposed of then sometimes run destructive sanity check. if (g_network->isSimulated() && dispose && BUGGIFY) { // Only proceed if the last commit is a success, but don't throw if it's not because shutdown @@ -8237,7 +8053,6 @@ public: f.get(); } else { CODE_PROBE(true, "Uncached forward range read seek"); - wait(store(lock, self->m_concurrentReads.lock())); wait(f); } @@ -8293,7 +8108,6 @@ public: f.get(); } else { CODE_PROBE(true, "Uncached reverse range read seek"); - wait(store(lock, self->m_concurrentReads.lock())); wait(f); } @@ -8360,9 +8174,6 @@ public: wait(self->m_tree->initBTreeCursor( &cur, self->m_tree->getLastCommittedVersion(), PagerEventReasons::PointRead, options)); - // Not locking for point reads, instead relying on IO priority lock - // state PriorityMultiLock::Lock lock = wait(self->m_concurrentReads.lock()); - ++g_redwoodMetrics.metric.opGet; wait(cur.seekGTE(key)); if (cur.isValid() && cur.get().key == key) { @@ -8398,7 +8209,6 @@ private: Future m_init; Promise m_closed; Promise m_error; - PriorityMultiLock m_concurrentReads; bool prefetch; Version m_nextCommitVersion; Reference m_keyProvider; @@ -9034,6 +8844,43 @@ void RedwoodMetrics::getFields(TraceEvent* e, std::string* s, bool skipZeroes) { } } +void RedwoodMetrics::getIOLockFields(TraceEvent* e, std::string* s) { + if (ioLock == nullptr) + return; + + int maxPriority = ioLock->maxPriority(); + + if (e != nullptr) { + e->detail("ActiveReads", ioLock->totalRunners()); + e->detail("AwaitReads", ioLock->totalWaiters()); + + for (int priority = 0; priority <= maxPriority; ++priority) { + e->detail(format("ActiveP%d", priority), ioLock->numRunners(priority)); + e->detail(format("AwaitP%d", priority), ioLock->numWaiters(priority)); + } + } + + if (s != nullptr) { + std::string active = "Active"; + std::string await = "Await"; + + *s += "\n"; + *s += format("%-15s %-8u ", "ActiveReads", ioLock->totalRunners()); + *s += format("%-15s %-8u ", "AwaitReads", ioLock->totalWaiters()); + *s += "\n"; + + for (int priority = 0; priority <= maxPriority; ++priority) { + *s += + format("%-15s %-8u ", (active + 'P' + std::to_string(priority)).c_str(), ioLock->numRunners(priority)); + } + *s += "\n"; + for (int priority = 0; priority <= maxPriority; ++priority) { + *s += + format("%-15s %-8u ", (await + 'P' + std::to_string(priority)).c_str(), ioLock->numWaiters(priority)); + } + } +} + TEST_CASE("/redwood/correctness/unit/RedwoodRecordRef") { ASSERT(RedwoodRecordRef::Delta::LengthFormatSizes[0] == 3); ASSERT(RedwoodRecordRef::Delta::LengthFormatSizes[1] == 4); @@ -11517,3 +11364,57 @@ TEST_CASE(":/redwood/performance/histograms") { return Void(); } + +ACTOR Future waitLockIncrement(PriorityMultiLock* pml, int priority, int* pout) { + state PriorityMultiLock::Lock lock = wait(pml->lock(priority)); + wait(delay(deterministicRandom()->random01() * .1)); + ++*pout; + return Void(); +} + +TEST_CASE("/redwood/PriorityMultiLock") { + state std::vector priorities = { 10, 20, 40 }; + state int concurrency = 25; + state PriorityMultiLock* pml = new PriorityMultiLock(concurrency, priorities); + state std::vector counts; + counts.resize(priorities.size(), 0); + + // Clog the lock buy taking concurrency locks at each level + state std::vector> lockFutures; + for (int i = 0; i < priorities.size(); ++i) { + for (int j = 0; j < concurrency; ++j) { + lockFutures.push_back(pml->lock(i)); + } + } + + // Wait for n = concurrency locks to be acquired + wait(quorum(lockFutures, concurrency)); + + state std::vector> futures; + for (int i = 0; i < 10e3; ++i) { + int p = i % priorities.size(); + futures.push_back(waitLockIncrement(pml, p, &counts[p])); + } + + state Future f = waitForAll(futures); + + // Release the locks + lockFutures.clear(); + + // Print stats and wait for all futures to be ready + loop { + choose { + when(wait(delay(1))) { + printf("counts: "); + for (auto c : counts) { + printf("%d ", c); + } + printf(" pml: %s\n", pml->toString().c_str()); + } + when(wait(f)) { break; } + } + } + + delete pml; + return Void(); +} diff --git a/fdbserver/storageserver.actor.cpp b/fdbserver/storageserver.actor.cpp index def3bf9078..bea18bfc26 100644 --- a/fdbserver/storageserver.actor.cpp +++ b/fdbserver/storageserver.actor.cpp @@ -37,6 +37,7 @@ #include "flow/Error.h" #include "flow/Hash3.h" #include "flow/Histogram.h" +#include "flow/PriorityMultiLock.actor.h" #include "flow/IRandom.h" #include "flow/IndexedSet.h" #include "flow/SystemMonitor.h" @@ -1015,6 +1016,9 @@ public: FlowLock serveFetchCheckpointParallelismLock; + PriorityMultiLock ssLock; + std::vector readPriorityRanks; + int64_t instanceID; Promise otherError; @@ -1280,13 +1284,15 @@ public: changeFeedDiskReadsLock(SERVER_KNOBS->CHANGE_FEED_DISK_READS_PARALLELISM), fetchKeysBytesBudget(SERVER_KNOBS->STORAGE_FETCH_BYTES), fetchKeysBudgetUsed(false), serveFetchCheckpointParallelismLock(SERVER_KNOBS->SERVE_FETCH_CHECKPOINT_PARALLELISM), + ssLock(SERVER_KNOBS->STORAGE_SERVER_READ_CONCURRENCY, SERVER_KNOBS->STORAGESERVER_READ_PRIORITIES), 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()), busiestWriteTagContext(ssi.id()), counters(this), storageServerSourceTLogIDEventHolder( makeReference(ssi.id().toString() + "/StorageServerSourceTLogID")) { - + readPriorityRanks = parseStringToVector(SERVER_KNOBS->STORAGESERVER_READ_RANKS, ','); + ASSERT(readPriorityRanks.size() > (int)ReadType::MAX); version.initMetric("StorageServer.Version"_sr, counters.cc.id); oldestVersion.initMetric("StorageServer.OldestVersion"_sr, counters.cc.id); durableVersion.initMetric("StorageServer.DurableVersion"_sr, counters.cc.id); @@ -1843,6 +1849,7 @@ std::vector StorageServer::getStorageServerShards(KeyRangeRe ACTOR Future getValueQ(StorageServer* data, GetValueRequest req) { state int64_t resultSize = 0; + state PriorityMultiLock::Lock lock; Span span("SS:getValue"_loc, req.spanContext); if (req.tenantInfo.name.present()) { span.addAttribute("tenant"_sr, req.tenantInfo.name.get()); @@ -1851,6 +1858,8 @@ ACTOR Future getValueQ(StorageServer* data, GetValueRequest req) { // Temporarily disabled -- this path is hit a lot // getCurrentLineage()->modify(&TransactionLineage::txID) = req.spanContext.first(); + state ReadType type = req.options.present() ? req.options.get().type : ReadType::NORMAL; + try { ++data->counters.getValueQueries; ++data->counters.allQueries; @@ -1861,6 +1870,8 @@ ACTOR Future getValueQ(StorageServer* data, GetValueRequest req) { // so we need to downgrade here wait(data->getQueryDelay()); + wait(store(lock, data->ssLock.lock(data->readPriorityRanks[(int)type]))); + // Track time from requestTime through now as read queueing wait time state double queueWaitEnd = g_network->timer(); data->counters.readQueueWaitSample.addMeasurement(queueWaitEnd - req.requestTime()); @@ -3716,6 +3727,7 @@ ACTOR Future getKeyValuesQ(StorageServer* data, GetKeyValuesRequest req) state Span span("SS:getKeyValues"_loc, req.spanContext); state int64_t resultSize = 0; state Optional options = req.options; + state ReadType type = options.present() ? options.get().type : ReadType::NORMAL; if (req.tenantInfo.name.present()) { span.addAttribute("tenant"_sr, req.tenantInfo.name.get()); @@ -3730,12 +3742,13 @@ ACTOR Future getKeyValuesQ(StorageServer* data, GetKeyValuesRequest req) // Active load balancing runs at a very high priority (to obtain accurate queue lengths) // so we need to downgrade here - if (SERVER_KNOBS->FETCH_KEYS_LOWER_PRIORITY && req.options.present() && req.options.get().type == ReadType::FETCH) { - wait(delay(0, TaskPriority::FetchKeys)); - } else { - wait(data->getQueryDelay()); + wait(data->getQueryDelay()); + if (!SERVER_KNOBS->FETCH_KEYS_LOWER_PRIORITY && type == ReadType::FETCH) { + type = ReadType::NORMAL; } + state PriorityMultiLock::Lock lock = wait(data->ssLock.lock(data->readPriorityRanks[(int)type])); + // Track time from requestTime through now as read queueing wait time state double queueWaitEnd = g_network->timer(); data->counters.readQueueWaitSample.addMeasurement(queueWaitEnd - req.requestTime()); @@ -4460,6 +4473,7 @@ ACTOR Future getMappedKeyValuesQ(StorageServer* data, GetMappedKeyValuesRe state Span span("SS:getMappedKeyValues"_loc, req.spanContext); state int64_t resultSize = 0; state Optional options = req.options; + state ReadType type = options.present() ? options.get().type : ReadType::NORMAL; if (req.tenantInfo.name.present()) { span.addAttribute("tenant"_sr, req.tenantInfo.name.get()); @@ -4474,12 +4488,13 @@ ACTOR Future getMappedKeyValuesQ(StorageServer* data, GetMappedKeyValuesRe // Active load balancing runs at a very high priority (to obtain accurate queue lengths) // so we need to downgrade here - if (SERVER_KNOBS->FETCH_KEYS_LOWER_PRIORITY && req.options.present() && req.options.get().type == ReadType::FETCH) { - wait(delay(0, TaskPriority::FetchKeys)); - } else { - wait(data->getQueryDelay()); + wait(data->getQueryDelay()); + if (!SERVER_KNOBS->FETCH_KEYS_LOWER_PRIORITY && type == ReadType::FETCH) { + type = ReadType::NORMAL; } + state PriorityMultiLock::Lock lock = wait(data->ssLock.lock(data->readPriorityRanks[(int)type])); + // Track time from requestTime through now as read queueing wait time state double queueWaitEnd = g_network->timer(); data->counters.readQueueWaitSample.addMeasurement(queueWaitEnd - req.requestTime()); @@ -4676,6 +4691,7 @@ ACTOR Future getKeyValuesStreamQ(StorageServer* data, GetKeyValuesStreamRe state Span span("SS:getKeyValuesStream"_loc, req.spanContext); state int64_t resultSize = 0; state Optional options = req.options; + state ReadType type = options.present() ? options.get().type : ReadType::NORMAL; if (req.tenantInfo.name.present()) { span.addAttribute("tenant"_sr, req.tenantInfo.name.get()); @@ -4689,12 +4705,14 @@ ACTOR Future getKeyValuesStreamQ(StorageServer* data, GetKeyValuesStreamRe // Active load balancing runs at a very high priority (to obtain accurate queue lengths) // so we need to downgrade here - if (SERVER_KNOBS->FETCH_KEYS_LOWER_PRIORITY && req.options.present() && req.options.get().type == ReadType::FETCH) { - wait(delay(0, TaskPriority::FetchKeys)); - } else { - wait(delay(0, TaskPriority::DefaultEndpoint)); + wait(delay(0, TaskPriority::DefaultEndpoint)); + if (!SERVER_KNOBS->FETCH_KEYS_LOWER_PRIORITY && type == ReadType::FETCH) { + type = ReadType::NORMAL; } + state int readPriority = data->readPriorityRanks[(int)type]; + state PriorityMultiLock::Lock lock = wait(data->ssLock.lock(readPriority)); + try { if (req.options.present() && req.options.get().debugID.present()) g_traceBatch.addEvent( @@ -4866,12 +4884,8 @@ ACTOR Future getKeyValuesStreamQ(StorageServer* data, GetKeyValuesStreamRe end = lastKey; } - if (SERVER_KNOBS->FETCH_KEYS_LOWER_PRIORITY && req.options.present() && - req.options.get().type == ReadType::FETCH) { - wait(delay(0, TaskPriority::FetchKeys)); - } else { - wait(delay(0, TaskPriority::DefaultEndpoint)); - } + lock.release(); + wait(store(lock, data->ssLock.lock(readPriority))); data->transactionTagCounter.addRequest(req.tags, resultSize); } @@ -4892,14 +4906,19 @@ ACTOR Future getKeyValuesStreamQ(StorageServer* data, GetKeyValuesStreamRe ACTOR Future getKeyQ(StorageServer* data, GetKeyRequest req) { state Span span("SS:getKey"_loc, req.spanContext); + state PriorityMultiLock::Lock lock; if (req.tenantInfo.name.present()) { span.addAttribute("tenant"_sr, req.tenantInfo.name.get()); } state int64_t resultSize = 0; state ReadOptions options; + state ReadType type = ReadType::NORMAL; + if (req.options.present()) { options = req.options.get(); + type = options.type; } + getCurrentLineage()->modify(&TransactionLineage::txID) = req.spanContext.traceID; ++data->counters.getKeyQueries; @@ -4911,6 +4930,8 @@ ACTOR Future getKeyQ(StorageServer* data, GetKeyRequest req) { // so we need to downgrade here wait(data->getQueryDelay()); + wait(store(lock, data->ssLock.lock(data->readPriorityRanks[(int)type]))); + // Track time from requestTime through now as read queueing wait time state double queueWaitEnd = g_network->timer(); data->counters.readQueueWaitSample.addMeasurement(queueWaitEnd - req.requestTime()); @@ -6465,11 +6486,8 @@ ACTOR Future fetchKeys(StorageServer* data, AddingShard* shard) { state int debug_nextRetryToLog = 1; state Error lastError; - // TODO: update to FETCH once the priority multi lock is used. - // leaving the readtype off for now to prevent data fetches stall under heavy load // it is used to inform the storage that the rangeRead is for Fetch - // state ReadOptions options = ReadOptions(Optional(), ReadType::FETCH); - state ReadOptions options = ReadOptions(Optional(), ReadType::NORMAL); + state ReadOptions options = ReadOptions(fetchKeysID, ReadType::FETCH); // FIXME: The client cache does not notice when servers are added to a team. To read from a local storage server // we must refresh the cache manually. @@ -9820,6 +9838,21 @@ ACTOR Future metricsCore(StorageServer* self, StorageServerInterface ssi) [self = self](TraceEvent& te) { te.detail("StorageEngine", self->storage.getKeyValueStoreType().toString()); te.detail("Tag", self->tag.toString()); + std::vector rpr = self->readPriorityRanks; + te.detail("ReadsActive", self->ssLock.totalRunners()); + te.detail("ReadsWaiting", self->ssLock.totalWaiters()); + int type = (int)ReadType::FETCH; + te.detail("ReadFetchActive", self->ssLock.numRunners(rpr[type])); + te.detail("ReadFetchWaiting", self->ssLock.numWaiters(rpr[type])); + type = (int)ReadType::LOW; + te.detail("ReadLowActive", self->ssLock.numRunners(rpr[type])); + te.detail("ReadLowWaiting", self->ssLock.numWaiters(rpr[type])); + type = (int)ReadType::NORMAL; + te.detail("ReadNormalActive", self->ssLock.numRunners(rpr[type])); + te.detail("ReadNormalWaiting", self->ssLock.numWaiters(rpr[type])); + type = (int)ReadType::HIGH; + te.detail("ReadHighActive", self->ssLock.numRunners(rpr[type])); + te.detail("ReadHighWaiting", self->ssLock.numWaiters(rpr[type])); StorageBytes sb = self->storage.getStorageBytes(); te.detail("KvstoreBytesUsed", sb.used); te.detail("KvstoreBytesFree", sb.free); @@ -10676,6 +10709,9 @@ ACTOR Future storageServer(IKeyValueStore* persistentData, // If the storage server dies while something that uses self is still on the stack, // we want that actor to complete before we terminate and that memory goes out of scope + + self.ssLock.kill(); + state Error err = e; if (storageServerTerminated(self, persistentData, err)) { ssCore.cancel(); @@ -10796,6 +10832,9 @@ ACTOR Future storageServer(IKeyValueStore* persistentData, throw internal_error(); } catch (Error& e) { + + self.ssLock.kill(); + if (self.byteSampleRecovery.isValid()) { self.byteSampleRecovery.cancel(); } diff --git a/fdbserver/workloads/ReadWrite.actor.cpp b/fdbserver/workloads/ReadWrite.actor.cpp index d7a44cb6e1..e958a8da8f 100644 --- a/fdbserver/workloads/ReadWrite.actor.cpp +++ b/fdbserver/workloads/ReadWrite.actor.cpp @@ -22,6 +22,7 @@ #include #include +#include "fdbclient/FDBTypes.h" #include "fdbrpc/ContinuousSample.h" #include "fdbclient/NativeAPI.actor.h" #include "fdbserver/TesterInterface.actor.h" @@ -377,6 +378,8 @@ struct ReadWriteWorkload : ReadWriteCommon { bool adjacentReads; // keys are adjacent within a transaction bool adjacentWrites; int extraReadConflictRangesPerTransaction, extraWriteConflictRangesPerTransaction; + int readType; + bool cacheResult; Optional transactionTag; int transactionsTagThrottled{ 0 }; @@ -399,6 +402,8 @@ struct ReadWriteWorkload : ReadWriteCommon { rampUpConcurrency = getOption(options, "rampUpConcurrency"_sr, false); batchPriority = getOption(options, "batchPriority"_sr, false); descriptionString = getOption(options, "description"_sr, "ReadWrite"_sr); + readType = getOption(options, "readType"_sr, 3); + cacheResult = getOption(options, "cacheResult"_sr, true); if (hasOption(options, "transactionTag"_sr)) { transactionTag = getOption(options, "transactionTag"_sr, ""_sr); } @@ -428,6 +433,10 @@ struct ReadWriteWorkload : ReadWriteCommon { if (transactionTag.present() && tr.getTags().size() == 0) { tr.setOption(FDBTransactionOptions::AUTO_THROTTLE_TAG, transactionTag.get()); } + ReadOptions options; + options.type = static_cast(readType); + options.cacheResult = cacheResult; + tr.getTransaction().trState->readOptions = options; } std::string description() const override { return descriptionString.toString(); } @@ -503,7 +512,6 @@ struct ReadWriteWorkload : ReadWriteCommon { state double startTime = now(); loop { state Transaction tr(cx); - try { self->setupTransaction(tr); wait(self->readOp(&tr, keys, self, false)); diff --git a/flow/include/flow/PriorityMultiLock.actor.h b/flow/include/flow/PriorityMultiLock.actor.h new file mode 100644 index 0000000000..363b3c4e27 --- /dev/null +++ b/flow/include/flow/PriorityMultiLock.actor.h @@ -0,0 +1,358 @@ +/* + * PriorityMultiLock.actor.h + * + * This source file is part of the FoundationDB open source project + * + * Copyright 2013-2022 Apple Inc. and the FoundationDB project authors + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +#pragma once + +// When actually compiled (NO_INTELLISENSE), include the generated version of this file. In intellisense use the source +// version. +#if defined(NO_INTELLISENSE) && !defined(FLOW_PRIORITYMULTILOCK_ACTOR_G_H) +#define FLOW_PRIORITYMULTILOCK_ACTOR_G_H +#include "flow/PriorityMultiLock.actor.g.h" +#elif !defined(PRIORITYMULTILOCK_ACTOR_H) +#define PRIORITYMULTILOCK_ACTOR_H + +#include "flow/flow.h" +#include "flow/actorcompiler.h" // This must be the last #include. + +#define PRIORITYMULTILOCK_DEBUG 0 + +#if PRIORITYMULTILOCK_DEBUG +#define pml_debug_printf(...) printf(__VA_ARGS__) +#else +#define pml_debug_printf(...) +#endif + +// A multi user lock with a concurrent holder limit where waiters request a lock with a priority +// id and are granted locks based on a total concurrency and relative importants of the priority +// ids defined. +// +// Scheduling logic +// launchLimits[n] = configured amount from the launchLimit vector for priority n +// waiters[n] = the number of waiters for priority n +// runnerCounts[n] = number of runners at priority n +// +// totalActiveLaunchLimits = sum of limits for all priorities with waiters +// When waiters[n] becomes == 0, totalActiveLaunchLimits -= launchLimits[n] +// When waiters[n] becomes > 0, totalActiveLaunchLimits += launchLimits[n] +// +// The total capacity of a priority to be considered when launching tasks is +// ceil(launchLimits[n] / totalLimits * concurrency) +// +// The interface is similar to FlowMutex except that lock holders can just drop the lock to release it. +// +// Usage: +// Lock lock = wait(prioritylock.lock(priorityLevel)); +// lock.release(); // Explicit release, or +// // let lock and all copies of lock go out of scope to release +class PriorityMultiLock { + +public: + // Waiting on the lock returns a Lock, which is really just a Promise + // Calling release() is not necessary, it exists in case the Lock holder wants to explicitly release + // the Lock before it goes out of scope. + struct Lock { + void release() { promise.send(Void()); } + + // This is exposed in case the caller wants to use/copy it directly + Promise promise; + }; + + PriorityMultiLock(int concurrency, std::string launchLimits) + : PriorityMultiLock(concurrency, parseStringToVector(launchLimits, ',')) {} + + PriorityMultiLock(int concurrency, std::vector launchLimitsByPriority) + : concurrency(concurrency), available(concurrency), waiting(0), totalActiveLaunchLimits(0) { + + priorities.resize(launchLimitsByPriority.size()); + for (int i = 0; i < priorities.size(); ++i) { + priorities[i].launchLimit = launchLimitsByPriority[i]; + } + + fRunner = runner(this); + } + + ~PriorityMultiLock() {} + + Future lock(int priority = 0) { + Priority& p = priorities[priority]; + Queue& q = p.queue; + Waiter w; + + // If this priority currently has no waiters + if (q.empty()) { + // Add this priority's launch limit to totalLimits + totalActiveLaunchLimits += p.launchLimit; + + // If there are slots available and the priority has capacity then don't make the caller wait + if (available > 0 && p.runners < currentCapacity(p.launchLimit)) { + // Remove this priority's launch limit from the total since it will remain empty + totalActiveLaunchLimits -= p.launchLimit; + + // Return a Lock to the caller + Lock lock; + addRunner(lock, &p); + + pml_debug_printf("lock nowait line %d priority %d %s\n", __LINE__, priority, toString().c_str()); + return lock; + } + } + q.push_back(w); + ++waiting; + + pml_debug_printf("lock wait line %d priority %d %s\n", __LINE__, priority, toString().c_str()); + return w.lockPromise.getFuture(); + } + + void kill() { + for (int i = 0; i < runners.size(); ++i) { + if (!runners[i].isReady()) { + runners[i].cancel(); + } + } + runners.clear(); + brokenOnDestruct.sendError(broken_promise()); + waiting = 0; + priorities.clear(); + } + + std::string toString() const { + int runnersDone = 0; + for (int i = 0; i < runners.size(); ++i) { + if (runners[i].isReady()) { + ++runnersDone; + } + } + + std::string s = + format("{ ptr=%p concurrency=%d available=%d running=%d waiting=%d runnersQueue=%d runnersDone=%d ", + this, + concurrency, + available, + concurrency - available, + waiting, + runners.size(), + runnersDone); + + for (int i = 0; i < priorities.size(); ++i) { + s += format("p%d:{%s} ", i, priorities[i].toString(this).c_str()); + } + + s += "}"; + return s; + } + int maxPriority() const { return priorities.size() - 1; } + + int totalWaiters() const { return waiting; } + + int numWaiters(const unsigned int priority) const { + ASSERT(priority < priorities.size()); + return priorities[priority].queue.size(); + } + + int totalRunners() const { return concurrency - available; } + + int numRunners(const unsigned int priority) const { + ASSERT(priority < priorities.size()); + return priorities[priority].runners; + } + +private: + struct Waiter { + Waiter() : queuedTime(now()) {} + Promise lockPromise; + double queuedTime; + }; + + // Total execution slots allowed across all priorities + int concurrency; + // Current available execution slots + int available; + // Total waiters across all priorities + int waiting; + // Sum of launch limits for all priorities with 1 or more waiters + int totalActiveLaunchLimits; + + typedef Deque Queue; + + struct Priority { + Priority() : runners(0), launchLimit(0) {} + + // Queue of waiters at this priority + Queue queue; + // Number of runners at this priority + int runners; + // Configured launch limit for this priority + int launchLimit; + + std::string toString(const PriorityMultiLock* pml) const { + return format("limit=%d run=%d wait=%d cap=%d", + launchLimit, + runners, + queue.size(), + queue.empty() ? 0 : pml->currentCapacity(launchLimit)); + } + }; + + std::vector priorities; + + // Current or recent (ended) runners + Deque> runners; + + Future fRunner; + AsyncTrigger wakeRunner; + Promise brokenOnDestruct; + + ACTOR static Future handleRelease(PriorityMultiLock* self, Future f, Priority* priority) { + try { + wait(f); + } catch (Error& e) { + } + + ++self->available; + priority->runners -= 1; + + pml_debug_printf("lock release line %d priority %d %s\n", + __LINE__, + (int)(priority - &self->priorities.front()), + self->toString().c_str()); + + // If there are any waiters or if the runners array is getting large, trigger the runner loop + if (self->waiting > 0 || self->runners.size() > 1000) { + self->wakeRunner.trigger(); + } + return Void(); + } + + void addRunner(Lock& lock, Priority* p) { + p->runners += 1; + --available; + runners.push_back(handleRelease(this, lock.promise.getFuture(), p)); + } + + // Current maximum running tasks for the specified priority, which must have waiters + // or the result is undefined + int currentCapacity(int launchLimit) const { + // The total concurrency allowed for this priority at present is the total concurrency times + // priority's launch limit divided by the total launch limits for all priorities with waiters. + return ceil((float)launchLimit / totalActiveLaunchLimits * concurrency); + } + + ACTOR static Future runner(PriorityMultiLock* self) { + state int sinceYield = 0; + state Future error = self->brokenOnDestruct.getFuture(); + + // Priority to try to run tasks from next + state int priority = 0; + + loop { + pml_debug_printf( + "runner loop start line %d priority=%d %s\n", __LINE__, priority, self->toString().c_str()); + + // Cleanup finished runner futures at the front of the runner queue. + while (!self->runners.empty() && self->runners.front().isReady()) { + self->runners.pop_front(); + } + + // Wait for a runner to release its lock + pml_debug_printf( + "runner loop waitTrigger line %d priority=%d %s\n", __LINE__, priority, self->toString().c_str()); + wait(self->wakeRunner.onTrigger()); + pml_debug_printf( + "runner loop wake line %d priority=%d %s\n", __LINE__, priority, self->toString().c_str()); + + if (++sinceYield == 100) { + sinceYield = 0; + pml_debug_printf( + " runner waitDelay line %d priority=%d %s\n", __LINE__, priority, self->toString().c_str()); + wait(delay(0)); + pml_debug_printf( + " runner afterDelay line %d priority=%d %s\n", __LINE__, priority, self->toString().c_str()); + } + + // While there are available slots and there are waiters, launch tasks + while (self->available > 0 && self->waiting > 0) { + pml_debug_printf( + " launch loop start line %d priority=%d %s\n", __LINE__, priority, self->toString().c_str()); + + Priority* pPriority; + + // Find the next priority with waiters and capacity. There must be at least one. + loop { + // Rotate to next priority + if (++priority == self->priorities.size()) { + priority = 0; + } + + pPriority = &self->priorities[priority]; + + pml_debug_printf(" launch loop scan line %d priority=%d %s\n", + __LINE__, + priority, + self->toString().c_str()); + + if (!pPriority->queue.empty() && + pPriority->runners < self->currentCapacity(pPriority->launchLimit)) { + break; + } + } + + Queue& queue = pPriority->queue; + + Waiter w = queue.front(); + queue.pop_front(); + + // If this priority is now empty, subtract its launch limit from totalLimits + if (queue.empty()) { + self->totalActiveLaunchLimits -= pPriority->launchLimit; + + pml_debug_printf(" emptied priority line %d priority=%d %s\n", + __LINE__, + priority, + self->toString().c_str()); + } + + --self->waiting; + Lock lock; + + w.lockPromise.send(lock); + + // Self may have been destructed during the lock callback + if (error.isReady()) { + throw error.getError(); + } + + // If the lock was not already released, add it to the runners future queue + if (lock.promise.canBeSet()) { + self->addRunner(lock, pPriority); + } + + pml_debug_printf(" launched line %d alreadyDone=%d priority=%d %s\n", + __LINE__, + !lock.promise.canBeSet(), + priority, + self->toString().c_str()); + } + } + } +}; + +#include "flow/unactorcompiler.h" + +#endif diff --git a/flow/include/flow/genericactors.actor.h b/flow/include/flow/genericactors.actor.h index 5aecc04215..8bdbcdbd53 100644 --- a/flow/include/flow/genericactors.actor.h +++ b/flow/include/flow/genericactors.actor.h @@ -100,6 +100,21 @@ T sorted(T range) { return range; } +template +std::vector parseStringToVector(std::string str, char delim) { + std::vector result; + std::stringstream stream(str); + std::string token; + while (stream.good()) { + getline(stream, token, delim); + std::istringstream tokenStream(token); + T item; + tokenStream >> item; + result.push_back(item); + } + return result; +} + template ErrorOr errorOr(T t) { return ErrorOr(t); diff --git a/flowbench/BenchPriorityMultiLock.actor.cpp b/flowbench/BenchPriorityMultiLock.actor.cpp new file mode 100644 index 0000000000..c1b517dedc --- /dev/null +++ b/flowbench/BenchPriorityMultiLock.actor.cpp @@ -0,0 +1,87 @@ +/* + * BenchStream.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 "benchmark/benchmark.h" + +#include "flow/flow.h" +#include "flow/ThreadHelper.actor.h" +#include "flow/PriorityMultiLock.actor.h" +#include +#include "flow/actorcompiler.h" // This must be the last #include. + +ACTOR static Future benchPriorityMultiLock(benchmark::State* benchState) { + state std::vector priorities; + + // Set up priority list with limits 10, 20, 30, ... + while (priorities.size() < benchState->range(0)) { + priorities.push_back(10 * (priorities.size() + 1)); + } + + state int concurrency = priorities.size() * 10; + state PriorityMultiLock* pml = new PriorityMultiLock(concurrency, priorities); + state std::vector counts; + counts.resize(priorities.size(), 0); + + // Clog the lock buy taking concurrency locks + state std::deque> lockFutures; + for (int j = 0; j < concurrency; ++j) { + lockFutures.push_back(pml->lock(j % priorities.size())); + } + + // Wait for all of the initial locks to be taken + // This will work regardless of their priorities as there are only n = concurrency of them + wait(waitForAll(std::vector>(lockFutures.begin(), lockFutures.end()))); + + // For each iteration of the loop, one new lock user is created, for a total of + // concurrency + 1 users. The new user replaces an old one, which is then waited + // on. This will succeed regardless of the lock priorities used because prior to + // new user there were only n = concurrency users so they will all be served before + // the new user. + state int p = 0; + state int i = 0; + while (benchState->KeepRunning()) { + // Get and replace the i'th lock future with a new lock waiter + Future f = lockFutures[i]; + lockFutures[i] = pml->lock(p); + + PriorityMultiLock::Lock lock = wait(f); + + // Rotate to another priority + if (++p == priorities.size()) { + p = 0; + } + + // Rotate to next lock index + if (++i == lockFutures.size()) { + i = 0; + } + } + + benchState->SetItemsProcessed(static_cast(benchState->iterations())); + + delete pml; + return Void(); +} + +static void bench_priorityMultiLock(benchmark::State& benchState) { + onMainThread([&benchState]() { return benchPriorityMultiLock(&benchState); }).blockUntilReady(); +} + +BENCHMARK(bench_priorityMultiLock)->DenseRange(1, 8)->ReportAggregatesOnly(true); From 5c97553493334831ad7cba7724aec86e8c2f0f7c Mon Sep 17 00:00:00 2001 From: Steve Atherton Date: Tue, 11 Oct 2022 00:15:58 -0700 Subject: [PATCH 02/95] In releaseHandler() do not access self anymore when cancelled. Improved comments. --- flow/include/flow/PriorityMultiLock.actor.h | 9 ++++++++- 1 file changed, 8 insertions(+), 1 deletion(-) diff --git a/flow/include/flow/PriorityMultiLock.actor.h b/flow/include/flow/PriorityMultiLock.actor.h index 363b3c4e27..95b1c267c7 100644 --- a/flow/include/flow/PriorityMultiLock.actor.h +++ b/flow/include/flow/PriorityMultiLock.actor.h @@ -44,17 +44,21 @@ // ids defined. // // Scheduling logic +// Let // launchLimits[n] = configured amount from the launchLimit vector for priority n // waiters[n] = the number of waiters for priority n // runnerCounts[n] = number of runners at priority n // -// totalActiveLaunchLimits = sum of limits for all priorities with waiters +// totalActiveLaunchLimits = sum of limits for all priorities with waiters[n] > 0 // When waiters[n] becomes == 0, totalActiveLaunchLimits -= launchLimits[n] // When waiters[n] becomes > 0, totalActiveLaunchLimits += launchLimits[n] // // The total capacity of a priority to be considered when launching tasks is // ceil(launchLimits[n] / totalLimits * concurrency) // +// For improved memory locality the properties mentioned above are stored as priorities[n]. +// in the actual implementation. +// // The interface is similar to FlowMutex except that lock holders can just drop the lock to release it. // // Usage: @@ -223,6 +227,9 @@ private: try { wait(f); } catch (Error& e) { + if (e.code() == error_code_actor_cancelled) { + throw; + } } ++self->available; From dfc11001afa9f2cad01bd4c07e60de818082eb1d Mon Sep 17 00:00:00 2001 From: sfc-gh-tclinkenbeard Date: Fri, 14 Oct 2022 12:33:33 -0700 Subject: [PATCH 03/95] Reject transactions that have been tag throttled for too long --- fdbclient/ClientKnobs.cpp | 1 + fdbclient/include/fdbclient/ClientKnobs.h | 2 ++ .../GrvProxyTransactionTagThrottler.actor.cpp | 21 +++++++++++++++++++ .../GrvProxyTransactionTagThrottler.h | 3 +++ flow/include/flow/error_definitions.h | 1 + 5 files changed, 28 insertions(+) diff --git a/fdbclient/ClientKnobs.cpp b/fdbclient/ClientKnobs.cpp index 41d218f8fe..ead11794fd 100644 --- a/fdbclient/ClientKnobs.cpp +++ b/fdbclient/ClientKnobs.cpp @@ -272,6 +272,7 @@ void ClientKnobs::initialize(Randomize randomize) { init( TAG_THROTTLE_EXPIRATION_INTERVAL, 60.0 ); if( randomize && BUGGIFY ) TAG_THROTTLE_EXPIRATION_INTERVAL = 1.0; init( WRITE_COST_BYTE_FACTOR, 16384 ); if( randomize && BUGGIFY ) WRITE_COST_BYTE_FACTOR = 4096; init( READ_COST_BYTE_FACTOR, 16384 ); if( randomize && BUGGIFY ) READ_COST_BYTE_FACTOR = 4096; + init( PROXY_MAX_TAG_THROTTLE, 5.0 ); if( randomize && BUGGIFY ) PROXY_MAX_TAG_THROTTLE = 0.5; // busyness reporting init( BUSYNESS_SPIKE_START_THRESHOLD, 0.100 ); diff --git a/fdbclient/include/fdbclient/ClientKnobs.h b/fdbclient/include/fdbclient/ClientKnobs.h index 6f89cdb45c..5e34eab002 100644 --- a/fdbclient/include/fdbclient/ClientKnobs.h +++ b/fdbclient/include/fdbclient/ClientKnobs.h @@ -262,6 +262,8 @@ public: double TAG_THROTTLE_EXPIRATION_INTERVAL; int64_t WRITE_COST_BYTE_FACTOR; // Used to round up the cost of write operations int64_t READ_COST_BYTE_FACTOR; // Used to round up the cost of read operations + double PROXY_MAX_TAG_THROTTLE; // Maximum duration that a transaction can be tag throttled by proxy before being + // rejected // busyness reporting double BUSYNESS_SPIKE_START_THRESHOLD; diff --git a/fdbserver/GrvProxyTransactionTagThrottler.actor.cpp b/fdbserver/GrvProxyTransactionTagThrottler.actor.cpp index 0ab7953344..4a83162496 100644 --- a/fdbserver/GrvProxyTransactionTagThrottler.actor.cpp +++ b/fdbserver/GrvProxyTransactionTagThrottler.actor.cpp @@ -19,6 +19,7 @@ */ #include "fdbserver/GrvProxyTransactionTagThrottler.h" +#include "fdbclient/Knobs.h" #include "flow/UnitTest.h" #include "flow/actorcompiler.h" // must be last include @@ -28,6 +29,10 @@ void GrvProxyTransactionTagThrottler::DelayedRequest::updateProxyTagThrottledDur req.proxyTagThrottledDuration = now() - startTime; } +bool GrvProxyTransactionTagThrottler::DelayedRequest::isTooOld() const { + return now() - startTime > CLIENT_KNOBS->PROXY_MAX_TAG_THROTTLE; +} + void GrvProxyTransactionTagThrottler::TagQueue::setRate(double rate) { if (rateInfo.present()) { rateInfo.get().setRate(rate); @@ -36,6 +41,19 @@ void GrvProxyTransactionTagThrottler::TagQueue::setRate(double rate) { } } +bool GrvProxyTransactionTagThrottler::TagQueue::isTooOld() const { + return requests.empty() || requests.front().isTooOld(); +} + +void GrvProxyTransactionTagThrottler::TagQueue::rejectRequests() { + while (!requests.empty()) { + auto& delayedReq = requests.front(); + delayedReq.updateProxyTagThrottledDuration(); + delayedReq.req.reply.sendError(proxy_tag_throttled()); + requests.pop_front(); + } +} + void GrvProxyTransactionTagThrottler::updateRates(TransactionTagMap const& newRates) { for (const auto& [tag, rate] : newRates) { auto it = queues.find(tag); @@ -140,6 +158,9 @@ void GrvProxyTransactionTagThrottler::releaseTransactions(double elapsed, // Cannot release any more transaction from this tag (don't push the tag queue handle back into // pqOfQueues) CODE_PROBE(true, "GrvProxyTransactionTagThrottler throttling transaction"); + if (tagQueueHandle.queue->isTooOld()) { + tagQueueHandle.queue->rejectRequests(); + } break; } else { if (tagQueueHandle.nextSeqNo < nextQueueSeqNo) { diff --git a/fdbserver/include/fdbserver/GrvProxyTransactionTagThrottler.h b/fdbserver/include/fdbserver/GrvProxyTransactionTagThrottler.h index 77a87a97b2..259bb6adc2 100644 --- a/fdbserver/include/fdbserver/GrvProxyTransactionTagThrottler.h +++ b/fdbserver/include/fdbserver/GrvProxyTransactionTagThrottler.h @@ -46,6 +46,7 @@ class GrvProxyTransactionTagThrottler { : req(req), startTime(now()), sequenceNumber(++lastSequenceNumber) {} void updateProxyTagThrottledDuration(); + bool isTooOld() const; }; struct TagQueue { @@ -56,6 +57,8 @@ class GrvProxyTransactionTagThrottler { explicit TagQueue(double rate) : rateInfo(rate) {} void setRate(double rate); + bool isTooOld() const; + void rejectRequests(); }; // Track the budgets for each tag diff --git a/flow/include/flow/error_definitions.h b/flow/include/flow/error_definitions.h index d493f3c897..396723c60c 100755 --- a/flow/include/flow/error_definitions.h +++ b/flow/include/flow/error_definitions.h @@ -131,6 +131,7 @@ 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_exceeded_request_limit, 1222, "Exceeded the max number of allowed concurrent audit storage requests" ) +ERROR( proxy_tag_throttled, 1223, "Exceeded maximum proxy tag throttling duration" ) // 15xx Platform errors ERROR( platform_error, 1500, "Platform error" ) From 300840ea2e48756344a8b149a8478ccfeb46dde0 Mon Sep 17 00:00:00 2001 From: sfc-gh-tclinkenbeard Date: Fri, 14 Oct 2022 17:08:49 -0700 Subject: [PATCH 04/95] Enable GLOBAL_TAG_THROTTLING by default --- fdbclient/ServerKnobs.cpp | 4 +- fdbserver/workloads/TagThrottleApi.actor.cpp | 2 +- ...ng.actor.cpp => ThroughputQuota.actor.cpp} | 24 ++++++----- tests/CMakeLists.txt | 2 +- tests/rare/GlobalTagThrottling.toml | 41 ------------------- tests/rare/ThroughputQuota.toml | 13 ++++++ 6 files changed, 31 insertions(+), 55 deletions(-) rename fdbserver/workloads/{GlobalTagThrottling.actor.cpp => ThroughputQuota.actor.cpp} (70%) delete mode 100644 tests/rare/GlobalTagThrottling.toml create mode 100644 tests/rare/ThroughputQuota.toml diff --git a/fdbclient/ServerKnobs.cpp b/fdbclient/ServerKnobs.cpp index 1ecda0dae1..3d1d93bcb5 100644 --- a/fdbclient/ServerKnobs.cpp +++ b/fdbclient/ServerKnobs.cpp @@ -721,8 +721,8 @@ void ServerKnobs::initialize(Randomize randomize, ClientKnobs* clientKnobs, IsSi init( TAG_THROTTLE_EXPIRED_CLEANUP_INTERVAL, 30.0 ); if(randomize && BUGGIFY) TAG_THROTTLE_EXPIRED_CLEANUP_INTERVAL = 1.0; init( AUTO_TAG_THROTTLING_ENABLED, true ); if(randomize && BUGGIFY) AUTO_TAG_THROTTLING_ENABLED = false; init( SS_THROTTLE_TAGS_TRACKED, 1 ); if(randomize && BUGGIFY) SS_THROTTLE_TAGS_TRACKED = deterministicRandom()->randomInt(1, 10); - init( GLOBAL_TAG_THROTTLING, false ); - init( ENFORCE_TAG_THROTTLING_ON_PROXIES, false ); + init( GLOBAL_TAG_THROTTLING, true ); + init( ENFORCE_TAG_THROTTLING_ON_PROXIES, GLOBAL_TAG_THROTTLING ); 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 ); diff --git a/fdbserver/workloads/TagThrottleApi.actor.cpp b/fdbserver/workloads/TagThrottleApi.actor.cpp index ac47417959..f6b8061de5 100644 --- a/fdbserver/workloads/TagThrottleApi.actor.cpp +++ b/fdbserver/workloads/TagThrottleApi.actor.cpp @@ -43,7 +43,7 @@ struct TagThrottleApiWorkload : TestWorkload { } Future start(Database const& cx) override { - if (this->clientId != 0) + if (SERVER_KNOBS->GLOBAL_TAG_THROTTLING || this->clientId != 0) return Void(); return timeout(runThrottleApi(this, cx), testDuration, Void()); } diff --git a/fdbserver/workloads/GlobalTagThrottling.actor.cpp b/fdbserver/workloads/ThroughputQuota.actor.cpp similarity index 70% rename from fdbserver/workloads/GlobalTagThrottling.actor.cpp rename to fdbserver/workloads/ThroughputQuota.actor.cpp index 2e2ec07a04..7a7e2c7d2b 100644 --- a/fdbserver/workloads/GlobalTagThrottling.actor.cpp +++ b/fdbserver/workloads/ThroughputQuota.actor.cpp @@ -1,5 +1,5 @@ /* - * GlobalTagThrottling.actor.cpp + * ThroughputQuota.actor.cpp * * This source file is part of the FoundationDB open source project * @@ -23,42 +23,46 @@ #include "flow/actorcompiler.h" // This must be the last #include. -class GlobalTagThrottlingWorkload : public TestWorkload { +// This workload sets the throughput quota of a tag during the setup phase +class ThroughputQuotaWorkload : public TestWorkload { TransactionTag transactionTag; double reservedQuota{ 0.0 }; double totalQuota{ 0.0 }; - ACTOR static Future setup(GlobalTagThrottlingWorkload* self, Database cx) { + ACTOR static Future setup(ThroughputQuotaWorkload* self, Database cx) { state Reference tr = makeReference(cx); loop { try { tr->setOption(FDBTransactionOptions::ACCESS_SYSTEM_KEYS); - TraceEvent("GlobalTagThrottlingWorkload_SettingTagQuota") - .detail("Tag", self->transactionTag) + TraceEvent("ThroughputQuotaWorkload_SettingTagQuota") + .detail("Tag", printable(self->transactionTag)) .detail("ReservedQuota", self->reservedQuota) .detail("TotalQuota", self->totalQuota); ThrottleApi::setTagQuota(tr, self->transactionTag, self->reservedQuota, self->totalQuota); wait(tr->commit()); return Void(); } catch (Error& e) { - TraceEvent("GlobalTagThrottlingWorkload_SetupError").error(e); + TraceEvent("ThroughputQuotaWorkload_SetupError").error(e); wait(tr->onError(e)); } }; } public: - static constexpr auto NAME = "GlobalTagThrottling"; - explicit GlobalTagThrottlingWorkload(WorkloadContext const& wcx) : TestWorkload(wcx) { + static constexpr auto NAME = "ThroughputQuota"; + explicit ThroughputQuotaWorkload(WorkloadContext const& wcx) : TestWorkload(wcx) { transactionTag = getOption(options, "transactionTag"_sr, "sampleTag"_sr); reservedQuota = getOption(options, "reservedQuota"_sr, 0.0); totalQuota = getOption(options, "totalQuota"_sr, 0.0); } - Future setup(Database const& cx) override { return clientId ? Void() : setup(this, cx); } + Future setup(Database const& cx) override { + DatabaseContext::debugUseTags = true; + return clientId ? Void() : setup(this, cx); + } Future start(Database const& cx) override { return Void(); } Future check(Database const& cx) override { return true; } void getMetrics(std::vector& m) override {} }; -WorkloadFactory GlobalTagThrottlingWorkloadFactory; +WorkloadFactory ThroughputQuotaWorkloadFactory; diff --git a/tests/CMakeLists.txt b/tests/CMakeLists.txt index e6d0730255..cba8b4169c 100644 --- a/tests/CMakeLists.txt +++ b/tests/CMakeLists.txt @@ -226,7 +226,6 @@ if(WITH_PYTHON) add_fdb_test(TEST_FILES rare/CycleWithDeadHall.toml) add_fdb_test(TEST_FILES rare/DataDistributionMetrics.toml) add_fdb_test(TEST_FILES rare/FuzzTest.toml) - add_fdb_test(TEST_FILES rare/GlobalTagThrottling.toml IGNORE) add_fdb_test(TEST_FILES rare/HighContentionPrefixAllocator.toml) add_fdb_test(TEST_FILES rare/InventoryTestHeavyWrites.toml) add_fdb_test(TEST_FILES rare/LargeApiCorrectness.toml) @@ -240,6 +239,7 @@ if(WITH_PYTHON) add_fdb_test(TEST_FILES rare/RedwoodCorrectnessBTree.toml) add_fdb_test(TEST_FILES rare/RedwoodDeltaTree.toml) add_fdb_test(TEST_FILES rare/Throttling.toml) + add_fdb_test(TEST_FILES rare/ThroughputQuota.toml) add_fdb_test(TEST_FILES rare/TransactionTagApiCorrectness.toml) add_fdb_test(TEST_FILES rare/TransactionTagSwizzledApiCorrectness.toml) add_fdb_test(TEST_FILES rare/WriteTagThrottling.toml) diff --git a/tests/rare/GlobalTagThrottling.toml b/tests/rare/GlobalTagThrottling.toml deleted file mode 100644 index 499f2990f2..0000000000 --- a/tests/rare/GlobalTagThrottling.toml +++ /dev/null @@ -1,41 +0,0 @@ -[[test]] -testTitle='GlobalTagThrottling' - - [[test.knobs]] - min_tag_read_pages_rate=1.0 - global_tag_throttling=true - - [[test.workload]] - testName='GlobalTagThrottling' - transactionTag='sampleTag1' - totalQuota=1.0 - - [[test.workload]] - testName='ReadWrite' - testDuration=600.0 - transactionsPerSecond=100 - writesPerTransactionA=0 - readsPerTransactionA=10 - writesPerTransactionB=0 - readsPerTransactionB=0 - alpha=0.0 - nodeCount=10000 - valueBytes=1000 - minValueBytes=1000 - warmingDelay=60.0 - transactionTag='sampleTag1' - - [[test.workload]] - testName='ReadWrite' - testDuration=600.0 - transactionsPerSecond=100 - writesPerTransactionA=0 - readsPerTransactionA=10 - writesPerTransactionB=0 - readsPerTransactionB=0 - alpha=0.0 - nodeCount=10000 - valueBytes=1000 - minValueBytes=1000 - warmingDelay=60.0 - transactionTag='sampleTag2' diff --git a/tests/rare/ThroughputQuota.toml b/tests/rare/ThroughputQuota.toml new file mode 100644 index 0000000000..1c7d191306 --- /dev/null +++ b/tests/rare/ThroughputQuota.toml @@ -0,0 +1,13 @@ +[[test]] +testTitle='ThroughputQuotaTest' + + [[test.workload]] + testName='ThroughputQuota' + transactionTag='sampleTag1' + totalQuota=1.0 + + [[test.workload]] + testName = 'Cycle' + transactionsPerSecond = 2500.0 + testDuration = 10.0 + expectedRate = 0 From 003986fdb0a60a81f28e130c77b0e4824aefa694 Mon Sep 17 00:00:00 2001 From: sfc-gh-tclinkenbeard Date: Fri, 14 Oct 2022 18:48:12 -0700 Subject: [PATCH 05/95] Randomize GLOBAL_TAG_THROTTLING knob --- fdbclient/ServerKnobs.cpp | 2 +- fdbserver/BlobWorker.actor.cpp | 2 +- fdbserver/GlobalTagThrottler.actor.cpp | 6 +++++- fdbserver/GrvProxyTransactionTagThrottler.actor.cpp | 1 + 4 files changed, 8 insertions(+), 3 deletions(-) diff --git a/fdbclient/ServerKnobs.cpp b/fdbclient/ServerKnobs.cpp index 3d1d93bcb5..bb860a1781 100644 --- a/fdbclient/ServerKnobs.cpp +++ b/fdbclient/ServerKnobs.cpp @@ -721,7 +721,7 @@ void ServerKnobs::initialize(Randomize randomize, ClientKnobs* clientKnobs, IsSi init( TAG_THROTTLE_EXPIRED_CLEANUP_INTERVAL, 30.0 ); if(randomize && BUGGIFY) TAG_THROTTLE_EXPIRED_CLEANUP_INTERVAL = 1.0; init( AUTO_TAG_THROTTLING_ENABLED, true ); if(randomize && BUGGIFY) AUTO_TAG_THROTTLING_ENABLED = false; init( SS_THROTTLE_TAGS_TRACKED, 1 ); if(randomize && BUGGIFY) SS_THROTTLE_TAGS_TRACKED = deterministicRandom()->randomInt(1, 10); - init( GLOBAL_TAG_THROTTLING, true ); + init( GLOBAL_TAG_THROTTLING, true ); if(isSimulated) GLOBAL_TAG_THROTTLING = deterministicRandom()->coinflip(); init( ENFORCE_TAG_THROTTLING_ON_PROXIES, GLOBAL_TAG_THROTTLING ); init( GLOBAL_TAG_THROTTLING_MIN_RATE, 1.0 ); init( GLOBAL_TAG_THROTTLING_FOLDING_TIME, 10.0 ); diff --git a/fdbserver/BlobWorker.actor.cpp b/fdbserver/BlobWorker.actor.cpp index a1c76463ff..5bd42cea81 100644 --- a/fdbserver/BlobWorker.actor.cpp +++ b/fdbserver/BlobWorker.actor.cpp @@ -1107,7 +1107,7 @@ ACTOR Future dumpInitialSnapshotFromFDB(Reference } retries++; CODE_PROBE(true, "Granule initial snapshot failed"); - // FIXME: why can't we supress error event? + // FIXME: why can't we suppress error event? TraceEvent(retries < 10 ? SevDebug : SevWarn, "BlobGranuleInitialSnapshotRetry", bwData->id) .error(err) .detail("Granule", metadata->keyRange) diff --git a/fdbserver/GlobalTagThrottler.actor.cpp b/fdbserver/GlobalTagThrottler.actor.cpp index b468677ae4..0dcb35a603 100644 --- a/fdbserver/GlobalTagThrottler.actor.cpp +++ b/fdbserver/GlobalTagThrottler.actor.cpp @@ -202,7 +202,8 @@ class GlobalTagThrottlerImpl { for (const auto& [id, _] : throughput) { result += getCurrentCost(id, tag).orDefault(0); } - TraceEvent("GlobalTagThrottler_GetCurrentCost").detail("Tag", printable(tag)).detail("Cost", result); + // FIXME: Disabled due to noisy trace events. Fix the noise and reenabled + //TraceEvent("GlobalTagThrottler_GetCurrentCost").detail("Tag", printable(tag)).detail("Cost", result); return result; } @@ -235,10 +236,13 @@ class GlobalTagThrottlerImpl { return 1.0; } auto const transactionRate = stats.get().getTransactionRate(); + // FIXME: Disabled due to noisy trace events. Fix the noise and reenabled + /* TraceEvent("GlobalTagThrottler_GetAverageTransactionCost") .detail("Tag", tag) .detail("TransactionRate", transactionRate) .detail("Cost", cost); + */ if (transactionRate == 0.0) { return 1.0; } else { diff --git a/fdbserver/GrvProxyTransactionTagThrottler.actor.cpp b/fdbserver/GrvProxyTransactionTagThrottler.actor.cpp index 4a83162496..4487b28266 100644 --- a/fdbserver/GrvProxyTransactionTagThrottler.actor.cpp +++ b/fdbserver/GrvProxyTransactionTagThrottler.actor.cpp @@ -91,6 +91,7 @@ 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") + .suppressFor(1.0) .detail("NumTags", req.tags.size()) .detail("UsingTag", printable(tag)); } From a70a007dcfe498ef672a23a476183d3ca52d181d Mon Sep 17 00:00:00 2001 From: sfc-gh-tclinkenbeard Date: Tue, 18 Oct 2022 16:27:55 -0700 Subject: [PATCH 06/95] Remove outdated comment --- fdbserver/BlobWorker.actor.cpp | 1 - 1 file changed, 1 deletion(-) diff --git a/fdbserver/BlobWorker.actor.cpp b/fdbserver/BlobWorker.actor.cpp index 5bd42cea81..ed2dbcf5da 100644 --- a/fdbserver/BlobWorker.actor.cpp +++ b/fdbserver/BlobWorker.actor.cpp @@ -1107,7 +1107,6 @@ ACTOR Future dumpInitialSnapshotFromFDB(Reference } retries++; CODE_PROBE(true, "Granule initial snapshot failed"); - // FIXME: why can't we suppress error event? TraceEvent(retries < 10 ? SevDebug : SevWarn, "BlobGranuleInitialSnapshotRetry", bwData->id) .error(err) .detail("Granule", metadata->keyRange) From a452e1899bfa6be44898804701ae0ee99e46ebaf Mon Sep 17 00:00:00 2001 From: Evan Tschannen Date: Thu, 20 Oct 2022 12:39:34 -0700 Subject: [PATCH 07/95] added a disk queue load generator --- fdbserver/DiskQueue.actor.cpp | 34 ++++++++++++++++++++++++++++++++++ 1 file changed, 34 insertions(+) diff --git a/fdbserver/DiskQueue.actor.cpp b/fdbserver/DiskQueue.actor.cpp index 40342945b6..de78b859d6 100644 --- a/fdbserver/DiskQueue.actor.cpp +++ b/fdbserver/DiskQueue.actor.cpp @@ -1659,3 +1659,37 @@ IDiskQueue* openDiskQueue(std::string basename, int64_t fileSizeWarningLimit) { return new DiskQueue_PopUncommitted(basename, ext, dbgid, dqv, fileSizeWarningLimit); } + +TEST_CASE("performance/fdbserver/DiskQueue") { + state IDiskQueue* queue = + openDiskQueue("test-", "fdq", deterministicRandom()->randomUniqueID(), DiskQueueVersion::V2); + state std::string valueString = std::string(10e6, '.'); + state StringRef valueStr((uint8_t*)valueString.c_str(), 10e6); + state std::deque locations; + state int loopCount = 0; + state Future lastCommit = Void(); + bool fullyRecovered = wait(queue->initializeRecovery(0)); + if (!fullyRecovered) { + loop { + Standalone h = wait(queue->readNext(1e6)); + if (h.size() < 1e6) { + break; + } + } + } + loop { + if (++loopCount % 2 == 0) { + state IDiskQueue::location frontLocation = locations.front(); + locations.pop_front(); + if (locations.size() > 10) { + Standalone r = wait(queue->read(frontLocation, locations.front(), CheckHashes::True)); + } + queue->pop(frontLocation); + } + wait(delay(0.001)); + locations.push_back(queue->push(valueStr)); + Future prevCommit = lastCommit; + lastCommit = queue->commit(); + wait(prevCommit); + } +} \ No newline at end of file From aa99b89d53103d1b3a9a6684cf801ad0daf622fd Mon Sep 17 00:00:00 2001 From: Vishesh Yadav Date: Fri, 21 Oct 2022 17:02:02 -0700 Subject: [PATCH 08/95] Don't fail ConsistencyCheck on first mismatch ConsistencyCheck fails when it sees the first corrupted shard. We may want to keep it running so that we can see all the corrupted data in logs. --- fdbserver/ConsistencyScan.actor.cpp | 5 +++-- fdbserver/workloads/ConsistencyCheck.actor.cpp | 7 ++++--- 2 files changed, 7 insertions(+), 5 deletions(-) diff --git a/fdbserver/ConsistencyScan.actor.cpp b/fdbserver/ConsistencyScan.actor.cpp index f961cb20a0..34a36b4629 100644 --- a/fdbserver/ConsistencyScan.actor.cpp +++ b/fdbserver/ConsistencyScan.actor.cpp @@ -393,6 +393,7 @@ ACTOR Future checkDataConsistency(Database cx, state double rateLimiterStartTime = now(); state int64_t bytesReadInthisRound = 0; state bool resume = !(restart || shuffleShards); + state bool testResult = true; state double dbSize = 100e12; if (g_network->isSimulated()) { @@ -710,7 +711,7 @@ ACTOR Future checkDataConsistency(Database cx, (!storageServerInterfaces[j].isTss() && !storageServerInterfaces[firstValidServer].isTss())) { testFailure("Data inconsistent", performQuiescentChecks, true); - return false; + testResult = false; } } } @@ -949,7 +950,7 @@ ACTOR Future checkDataConsistency(Database cx, } *bytesReadInPrevRound = bytesReadInthisRound; - return true; + return testResult; } ACTOR Future runDataValidationCheck(ConsistencyScanData* self) { diff --git a/fdbserver/workloads/ConsistencyCheck.actor.cpp b/fdbserver/workloads/ConsistencyCheck.actor.cpp index 6abeb66e5f..7c53744b13 100644 --- a/fdbserver/workloads/ConsistencyCheck.actor.cpp +++ b/fdbserver/workloads/ConsistencyCheck.actor.cpp @@ -394,6 +394,7 @@ struct ConsistencyCheckWorkload : TestWorkload { state Standalone> serverList; // "\xff/serverList/[[serverID]]" := "[[StorageServerInterface]]" state Standalone> serverTag; // "\xff/serverTag/[[serverID]]" = "[[Tag]]" + state bool testResult = true; std::vector> cacheResultsPromise; cacheResultsPromise.push_back(self->fetchKeyValuesFromSS(cx, self, storageCacheKeys, cacheKeyPromise, true)); @@ -581,7 +582,7 @@ struct ConsistencyCheckWorkload : TestWorkload { for (j = 0; j < keyValueFutures.size(); j++) { ErrorOr rangeResult = keyValueFutures[j].get(); // if (rangeResult.isError()) { - // throw rangeResult.getError(); + // throw rangeResult.getError(); // } // Compare the results with other storage servers @@ -709,7 +710,7 @@ struct ConsistencyCheckWorkload : TestWorkload { .detail("MatchingKVPairs", matchingKVPairs); self->testFailure("Data inconsistent", true); - return false; + testResult = false; } } } @@ -755,7 +756,7 @@ struct ConsistencyCheckWorkload : TestWorkload { .detail("BytesRead", bytesReadInRange); } } - return true; + return testResult; } // Directly fetch key/values from storage servers through GetKeyValuesRequest From 04ae47b9b9a3ba2c9efc4aadace122d32067a608 Mon Sep 17 00:00:00 2001 From: sfc-gh-tclinkenbeard Date: Fri, 21 Oct 2022 22:17:33 -0700 Subject: [PATCH 09/95] Addressed review comments --- fdbclient/ClientKnobs.cpp | 2 +- fdbclient/include/fdbclient/ClientKnobs.h | 4 ++-- .../GrvProxyTransactionTagThrottler.actor.cpp | 14 ++++++++------ .../fdbserver/GrvProxyTransactionTagThrottler.h | 4 ++-- 4 files changed, 13 insertions(+), 11 deletions(-) diff --git a/fdbclient/ClientKnobs.cpp b/fdbclient/ClientKnobs.cpp index ead11794fd..36fdaea5b0 100644 --- a/fdbclient/ClientKnobs.cpp +++ b/fdbclient/ClientKnobs.cpp @@ -272,7 +272,7 @@ void ClientKnobs::initialize(Randomize randomize) { init( TAG_THROTTLE_EXPIRATION_INTERVAL, 60.0 ); if( randomize && BUGGIFY ) TAG_THROTTLE_EXPIRATION_INTERVAL = 1.0; init( WRITE_COST_BYTE_FACTOR, 16384 ); if( randomize && BUGGIFY ) WRITE_COST_BYTE_FACTOR = 4096; init( READ_COST_BYTE_FACTOR, 16384 ); if( randomize && BUGGIFY ) READ_COST_BYTE_FACTOR = 4096; - init( PROXY_MAX_TAG_THROTTLE, 5.0 ); if( randomize && BUGGIFY ) PROXY_MAX_TAG_THROTTLE = 0.5; + init( PROXY_MAX_TAG_THROTTLE_DURATION, 5.0 ); if( randomize && BUGGIFY ) PROXY_MAX_TAG_THROTTLE_DURATION = 0.5; // busyness reporting init( BUSYNESS_SPIKE_START_THRESHOLD, 0.100 ); diff --git a/fdbclient/include/fdbclient/ClientKnobs.h b/fdbclient/include/fdbclient/ClientKnobs.h index 5e34eab002..b4961ba3f7 100644 --- a/fdbclient/include/fdbclient/ClientKnobs.h +++ b/fdbclient/include/fdbclient/ClientKnobs.h @@ -262,8 +262,8 @@ public: double TAG_THROTTLE_EXPIRATION_INTERVAL; int64_t WRITE_COST_BYTE_FACTOR; // Used to round up the cost of write operations int64_t READ_COST_BYTE_FACTOR; // Used to round up the cost of read operations - double PROXY_MAX_TAG_THROTTLE; // Maximum duration that a transaction can be tag throttled by proxy before being - // rejected + double PROXY_MAX_TAG_THROTTLE_DURATION; // Maximum duration that a transaction can be tag throttled by proxy before + // being rejected // busyness reporting double BUSYNESS_SPIKE_START_THRESHOLD; diff --git a/fdbserver/GrvProxyTransactionTagThrottler.actor.cpp b/fdbserver/GrvProxyTransactionTagThrottler.actor.cpp index 4487b28266..ae71d575a8 100644 --- a/fdbserver/GrvProxyTransactionTagThrottler.actor.cpp +++ b/fdbserver/GrvProxyTransactionTagThrottler.actor.cpp @@ -18,8 +18,8 @@ * limitations under the License. */ -#include "fdbserver/GrvProxyTransactionTagThrottler.h" #include "fdbclient/Knobs.h" +#include "fdbserver/GrvProxyTransactionTagThrottler.h" #include "flow/UnitTest.h" #include "flow/actorcompiler.h" // must be last include @@ -29,8 +29,8 @@ void GrvProxyTransactionTagThrottler::DelayedRequest::updateProxyTagThrottledDur req.proxyTagThrottledDuration = now() - startTime; } -bool GrvProxyTransactionTagThrottler::DelayedRequest::isTooOld() const { - return now() - startTime > CLIENT_KNOBS->PROXY_MAX_TAG_THROTTLE; +bool GrvProxyTransactionTagThrottler::DelayedRequest::isMaxThrottled() const { + return now() - startTime > CLIENT_KNOBS->PROXY_MAX_TAG_THROTTLE_DURATION; } void GrvProxyTransactionTagThrottler::TagQueue::setRate(double rate) { @@ -41,8 +41,8 @@ void GrvProxyTransactionTagThrottler::TagQueue::setRate(double rate) { } } -bool GrvProxyTransactionTagThrottler::TagQueue::isTooOld() const { - return requests.empty() || requests.front().isTooOld(); +bool GrvProxyTransactionTagThrottler::TagQueue::isMaxThrottled() const { + return !requests.empty() && requests.front().isMaxThrottled(); } void GrvProxyTransactionTagThrottler::TagQueue::rejectRequests() { @@ -159,7 +159,9 @@ void GrvProxyTransactionTagThrottler::releaseTransactions(double elapsed, // Cannot release any more transaction from this tag (don't push the tag queue handle back into // pqOfQueues) CODE_PROBE(true, "GrvProxyTransactionTagThrottler throttling transaction"); - if (tagQueueHandle.queue->isTooOld()) { + if (tagQueueHandle.queue->isMaxThrottled()) { + // Requests in this queue have been throttled too long and errors + // should be sent to clients. tagQueueHandle.queue->rejectRequests(); } break; diff --git a/fdbserver/include/fdbserver/GrvProxyTransactionTagThrottler.h b/fdbserver/include/fdbserver/GrvProxyTransactionTagThrottler.h index 259bb6adc2..887ff9ffb0 100644 --- a/fdbserver/include/fdbserver/GrvProxyTransactionTagThrottler.h +++ b/fdbserver/include/fdbserver/GrvProxyTransactionTagThrottler.h @@ -46,7 +46,7 @@ class GrvProxyTransactionTagThrottler { : req(req), startTime(now()), sequenceNumber(++lastSequenceNumber) {} void updateProxyTagThrottledDuration(); - bool isTooOld() const; + bool isMaxThrottled() const; }; struct TagQueue { @@ -57,7 +57,7 @@ class GrvProxyTransactionTagThrottler { explicit TagQueue(double rate) : rateInfo(rate) {} void setRate(double rate); - bool isTooOld() const; + bool isMaxThrottled() const; void rejectRequests(); }; From 1ae98808f997dd3091a141de2ff6fef55cadf015 Mon Sep 17 00:00:00 2001 From: sfc-gh-tclinkenbeard Date: Sat, 22 Oct 2022 07:12:03 -0700 Subject: [PATCH 10/95] Retry on proxy_tag_throttled errors --- fdbclient/NativeAPI.actor.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/fdbclient/NativeAPI.actor.cpp b/fdbclient/NativeAPI.actor.cpp index bf5483f82c..7372e66f6d 100644 --- a/fdbclient/NativeAPI.actor.cpp +++ b/fdbclient/NativeAPI.actor.cpp @@ -6558,7 +6558,7 @@ ACTOR static Future tryCommit(Reference trState, e.code() != error_code_grv_proxy_memory_limit_exceeded && e.code() != error_code_batch_transaction_throttled && e.code() != error_code_tag_throttled && e.code() != error_code_process_behind && e.code() != error_code_future_version && - e.code() != error_code_tenant_not_found) { + e.code() != error_code_tenant_not_found && e.code() != error_code_proxy_tag_throttled) { TraceEvent(SevError, "TryCommitError").error(e); } if (trState->trLogInfo) @@ -7484,7 +7484,7 @@ Future Transaction::onError(Error const& e) { e.code() == error_code_database_locked || e.code() == error_code_commit_proxy_memory_limit_exceeded || e.code() == error_code_grv_proxy_memory_limit_exceeded || e.code() == error_code_process_behind || e.code() == error_code_batch_transaction_throttled || e.code() == error_code_tag_throttled || - e.code() == error_code_blob_granule_request_failed) { + e.code() == error_code_blob_granule_request_failed || e.code() == error_code_proxy_tag_throttled) { if (e.code() == error_code_not_committed) ++trState->cx->transactionsNotCommitted; else if (e.code() == error_code_commit_unknown_result) From da4ceaf73586599792ea2b2decab0b344e740b79 Mon Sep 17 00:00:00 2001 From: sfc-gh-tclinkenbeard Date: Sun, 23 Oct 2022 10:39:59 -0700 Subject: [PATCH 11/95] Improve code coverage for proxy-level transaction tag throttling --- fdbclient/NativeAPI.actor.cpp | 4 +++- fdbserver/GrvProxyTransactionTagThrottler.actor.cpp | 1 + tests/rare/ThroughputQuota.toml | 2 +- 3 files changed, 5 insertions(+), 2 deletions(-) diff --git a/fdbclient/NativeAPI.actor.cpp b/fdbclient/NativeAPI.actor.cpp index 7372e66f6d..2bf00bc4ba 100644 --- a/fdbclient/NativeAPI.actor.cpp +++ b/fdbclient/NativeAPI.actor.cpp @@ -6999,6 +6999,8 @@ ACTOR Future getConsistentReadVersion(SpanContext parentSpa &GrvProxyInterface::getConsistentReadVersion, req, cx->taskID))) { + CODE_PROBE(v.proxyTagThrottledDuration > 0.0, + "getConsistentReadVersion received GetReadVersionReply delayed by proxy tag throttling"); if (tags.size() != 0) { auto& priorityThrottledTags = cx->throttledTags[priority]; for (auto& tag : tags) { @@ -7033,7 +7035,7 @@ ACTOR Future getConsistentReadVersion(SpanContext parentSpa } } catch (Error& e) { if (e.code() != error_code_broken_promise && e.code() != error_code_batch_transaction_throttled && - e.code() != error_code_grv_proxy_memory_limit_exceeded) + e.code() != error_code_grv_proxy_memory_limit_exceeded && e.code() != error_code_proxy_tag_throttled) TraceEvent(SevError, "GetConsistentReadVersionError").error(e); if (e.code() == error_code_batch_transaction_throttled && !cx->apiVersionAtLeast(630)) { wait(delayJittered(5.0)); diff --git a/fdbserver/GrvProxyTransactionTagThrottler.actor.cpp b/fdbserver/GrvProxyTransactionTagThrottler.actor.cpp index ae71d575a8..d6cf76bc63 100644 --- a/fdbserver/GrvProxyTransactionTagThrottler.actor.cpp +++ b/fdbserver/GrvProxyTransactionTagThrottler.actor.cpp @@ -46,6 +46,7 @@ bool GrvProxyTransactionTagThrottler::TagQueue::isMaxThrottled() const { } void GrvProxyTransactionTagThrottler::TagQueue::rejectRequests() { + CODE_PROBE(true, "GrvProxyTransactionTagThrottler rejecting requests"); while (!requests.empty()) { auto& delayedReq = requests.front(); delayedReq.updateProxyTagThrottledDuration(); diff --git a/tests/rare/ThroughputQuota.toml b/tests/rare/ThroughputQuota.toml index 1c7d191306..9ab3c0aa4b 100644 --- a/tests/rare/ThroughputQuota.toml +++ b/tests/rare/ThroughputQuota.toml @@ -3,7 +3,7 @@ testTitle='ThroughputQuotaTest' [[test.workload]] testName='ThroughputQuota' - transactionTag='sampleTag1' + transactionTag='a' totalQuota=1.0 [[test.workload]] From 70eb9aef28b140f4c85d1c57813b3bc948ddfe8d Mon Sep 17 00:00:00 2001 From: Xiaoxi Wang Date: Tue, 18 Oct 2022 13:24:24 -0700 Subject: [PATCH 12/95] simple MGS IKeyLocationService implementation --- fdbclient/FDBTypes.cpp | 10 ++++ fdbclient/NativeAPI.actor.cpp | 21 +++----- fdbclient/include/fdbclient/FDBTypes.h | 2 + .../include/fdbclient/KeyLocationService.h | 48 +++++++++++++++++++ fdbserver/DDTxnProcessor.actor.cpp | 3 +- fdbserver/MockGlobalState.cpp | 46 ++++++++++++++++++ fdbserver/include/fdbserver/DDTxnProcessor.h | 1 + fdbserver/include/fdbserver/MockGlobalState.h | 27 ++++++++++- 8 files changed, 140 insertions(+), 18 deletions(-) create mode 100644 fdbclient/include/fdbclient/KeyLocationService.h diff --git a/fdbclient/FDBTypes.cpp b/fdbclient/FDBTypes.cpp index ff268075bd..edd016d391 100644 --- a/fdbclient/FDBTypes.cpp +++ b/fdbclient/FDBTypes.cpp @@ -22,6 +22,16 @@ #include "fdbclient/Knobs.h" #include "fdbclient/NativeAPI.actor.h" +KeyRangeRef toPrefixRelativeRange(KeyRangeRef range, KeyRef prefix) { + if (prefix.empty()) { + return range; + } else { + KeyRef begin = range.begin.startsWith(prefix) ? range.begin.removePrefix(prefix) : allKeys.begin; + KeyRef end = range.end.startsWith(prefix) ? range.end.removePrefix(prefix) : allKeys.end; + return KeyRangeRef(begin, end); + } +} + KeyRef keyBetween(const KeyRangeRef& keys) { int pos = 0; // will be the position of the first difference between keys.begin and keys.end int minSize = std::min(keys.begin.size(), keys.end.size()); diff --git a/fdbclient/NativeAPI.actor.cpp b/fdbclient/NativeAPI.actor.cpp index bf5483f82c..4c1b75d905 100644 --- a/fdbclient/NativeAPI.actor.cpp +++ b/fdbclient/NativeAPI.actor.cpp @@ -1479,16 +1479,6 @@ Future HealthMetricsRangeImpl::getRange(ReadYourWritesTransaction* return healthMetricsGetRangeActor(ryw, kr); } -KeyRangeRef toRelativeRange(KeyRangeRef range, KeyRef prefix) { - if (prefix.empty()) { - return range; - } else { - KeyRef begin = range.begin.startsWith(prefix) ? range.begin.removePrefix(prefix) : allKeys.begin; - KeyRef end = range.end.startsWith(prefix) ? range.end.removePrefix(prefix) : allKeys.end; - return KeyRangeRef(begin, end); - } -} - ACTOR Future getClusterId(Database db) { while (!db->clientInfo->get().clusterId.isValid()) { wait(db->clientInfo->onChange()); @@ -1925,7 +1915,7 @@ Optional DatabaseContext::getCachedLocation(const Optional auto range = isBackward ? locationCache.rangeContainingKeyBefore(resolvedKey) : locationCache.rangeContaining(resolvedKey); if (range->value()) { - return KeyRangeLocationInfo(tenantEntry, toRelativeRange(range->range(), tenantEntry.prefix), range->value()); + return KeyRangeLocationInfo(tenantEntry, toPrefixRelativeRange(range->range(), tenantEntry.prefix), range->value()); } return Optional(); @@ -1962,7 +1952,7 @@ bool DatabaseContext::getCachedLocations(const Optional& tenantNa result.clear(); return false; } - result.emplace_back(tenantEntry, toRelativeRange(r->range() & resolvedRange, tenantEntry.prefix), r->value()); + result.emplace_back(tenantEntry, toPrefixRelativeRange(r->range() & resolvedRange, tenantEntry.prefix), r->value()); if (result.size() == limit || begin == end) { break; } @@ -2978,7 +2968,7 @@ ACTOR Future getKeyLocation_internal(Database cx, return KeyRangeLocationInfo( rep.tenantEntry, - KeyRange(toRelativeRange(rep.results[0].first, rep.tenantEntry.prefix), rep.arena), + KeyRange(toPrefixRelativeRange(rep.results[0].first, rep.tenantEntry.prefix), rep.arena), locationInfo); } } @@ -3123,7 +3113,7 @@ ACTOR Future> getKeyRangeLocations_internal( // efficient to save the map pairs and insert them all at once. results.emplace_back( rep.tenantEntry, - (toRelativeRange(rep.results[shard].first, rep.tenantEntry.prefix) & keys), + (toPrefixRelativeRange(rep.results[shard].first, rep.tenantEntry.prefix) & keys), cx->setCachedLocation( tenant.name, rep.tenantEntry, rep.results[shard].first, rep.results[shard].second)); wait(yield()); @@ -7779,7 +7769,8 @@ ACTOR Future, int>> waitStorageMetrics( } else { TraceEvent(SevWarn, "WaitStorageMetricsPenalty") .detail("Keys", keys) - .detail("Limit", CLIENT_KNOBS->STORAGE_METRICS_SHARD_LIMIT) + .detail("Limit", shardLimit) + .detail("LocationSize", locations.size()) .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 diff --git a/fdbclient/include/fdbclient/FDBTypes.h b/fdbclient/include/fdbclient/FDBTypes.h index b25c45ee8b..a532d9d59e 100644 --- a/fdbclient/include/fdbclient/FDBTypes.h +++ b/fdbclient/include/fdbclient/FDBTypes.h @@ -589,6 +589,8 @@ inline KeyRange prefixRange(KeyRef prefix) { // The returned reference is valid as long as keys is valid. KeyRef keyBetween(const KeyRangeRef& keys); +KeyRangeRef toPrefixRelativeRange(KeyRangeRef range, KeyRef prefix); + struct KeySelectorRef { private: KeyRef key; // Find the last item less than key diff --git a/fdbclient/include/fdbclient/KeyLocationService.h b/fdbclient/include/fdbclient/KeyLocationService.h new file mode 100644 index 0000000000..50e8e8820e --- /dev/null +++ b/fdbclient/include/fdbclient/KeyLocationService.h @@ -0,0 +1,48 @@ +/* + * KeyLocationService.h + * + * This source file is part of the FoundationDB open source project + * + * Copyright 2013-2022 Apple Inc. and the FoundationDB project authors + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +#ifndef FOUNDATIONDB_KEYLOCATIONSERVICE_H +#define FOUNDATIONDB_KEYLOCATIONSERVICE_H + +#include "fdbclient/NativeAPI.actor.h" +#include "fdbclient/DatabaseContext.h" + +class IKeyLocationService { + + // If isBackward == true, returns the shard containing the key before 'key' (an infinitely long, inexpressible key). + // Otherwise returns the shard containing key. It's possible the returned location is a failed interface. + virtual Future getKeyLocation(TenantInfo tenant, + Key key, + SpanContext spanContext, + Optional debugID, + UseProvisionalProxies useProvisionalProxies, + Reverse isBackward, + Version version) = 0; + + virtual Future> getKeyRangeLocations(TenantInfo tenant, + KeyRange keys, + int limit, + Reverse reverse, + SpanContext spanContext, + Optional debugID, + UseProvisionalProxies useProvisionalProxies, + Version version) = 0; +}; + +#endif // FOUNDATIONDB_KEYLOCATIONSERVICE_H diff --git a/fdbserver/DDTxnProcessor.actor.cpp b/fdbserver/DDTxnProcessor.actor.cpp index e379b8e33f..bcd8acf0d0 100644 --- a/fdbserver/DDTxnProcessor.actor.cpp +++ b/fdbserver/DDTxnProcessor.actor.cpp @@ -871,7 +871,6 @@ Future>> DDMockTxnProcessor::splitStorageMetrics( return Future>>(); } -// FIXME: finish implementation Future, int>> DDMockTxnProcessor::waitStorageMetrics( const KeyRange& keys, const StorageMetrics& min, @@ -879,7 +878,7 @@ Future, int>> DDMockTxnProcessor::waitStorage const StorageMetrics& permittedError, int shardLimit, int expectedShardCount) const { - return Future, int>>(); + return mgs->waitStorageMetrics(keys, min, max, permittedError, shardLimit, expectedShardCount); } // FIXME: finish implementation diff --git a/fdbserver/MockGlobalState.cpp b/fdbserver/MockGlobalState.cpp index 310cd9c854..31162494fe 100644 --- a/fdbserver/MockGlobalState.cpp +++ b/fdbserver/MockGlobalState.cpp @@ -190,6 +190,52 @@ bool MockGlobalState::allShardRemovedFromServer(const UID& serverId) { return allServers.count(serverId) && shardMapping->getNumberOfShards(serverId) == 0; } +Future, int>> MockGlobalState::waitStorageMetrics( + const KeyRange& keys, + const StorageMetrics& min, + const StorageMetrics& max, + const StorageMetrics& permittedError, + int shardLimit, + int expectedShardCount) { + return Future, int>>(); +} + +Future MockGlobalState::getKeyLocation(TenantInfo tenant, + Key key, + SpanContext spanContext, + Optional debugID, + UseProvisionalProxies useProvisionalProxies, + Reverse isBackward, + Version version) { + GetKeyServerLocationsReply rep; + + // construct the location info with the servers + std::vector>> serverRefs; + auto& servers = rep.results[0].second; + serverRefs.reserve(servers.size()); + for (const auto& interf : servers) { + serverRefs.push_back(makeReference>(interf)); + } + + auto locationInfo = makeReference(serverRefs); + + return KeyRangeLocationInfo( + rep.tenantEntry, + KeyRange(toPrefixRelativeRange(rep.results[0].first, rep.tenantEntry.prefix), rep.arena), + locationInfo); +} +Future> MockGlobalState::getKeyRangeLocations( + TenantInfo tenant, + KeyRange keys, + int limit, + Reverse reverse, + SpanContext spanContext, + Optional debugID, + UseProvisionalProxies useProvisionalProxies, + Version version) { + return Future>(); +} + TEST_CASE("/MockGlobalState/initializeAsEmptyDatabaseMGS/SimpleThree") { BasicTestConfig testConfig; testConfig.simpleConfig = true; diff --git a/fdbserver/include/fdbserver/DDTxnProcessor.h b/fdbserver/include/fdbserver/DDTxnProcessor.h index a3dc98e04a..17373439ae 100644 --- a/fdbserver/include/fdbserver/DDTxnProcessor.h +++ b/fdbserver/include/fdbserver/DDTxnProcessor.h @@ -117,6 +117,7 @@ public: virtual Future moveKeys(const MoveKeysParams& params) = 0; + // metrics.second is the number of key-ranges (i.e., shards) in the 'keys' key-range virtual Future, int>> waitStorageMetrics(KeyRange const& keys, StorageMetrics const& min, StorageMetrics const& max, diff --git a/fdbserver/include/fdbserver/MockGlobalState.h b/fdbserver/include/fdbserver/MockGlobalState.h index d814984270..ce5c34d32e 100644 --- a/fdbserver/include/fdbserver/MockGlobalState.h +++ b/fdbserver/include/fdbserver/MockGlobalState.h @@ -25,6 +25,7 @@ #include "fdbclient/KeyRangeMap.h" #include "fdbclient/StorageServerInterface.h" #include "fdbclient/DatabaseConfiguration.h" +#include "fdbclient/KeyLocationService.h" #include "SimulatedCluster.h" #include "ShardsAffectedByTeamFailure.h" @@ -112,7 +113,7 @@ protected: void twoWayShardSplitting(KeyRangeRef range, KeyRef splitPoint, uint64_t rangeSize, bool restrictSize); }; -class MockGlobalState { +class MockGlobalState : public IKeyLocationService { friend struct MockGlobalStateTester; public: @@ -163,6 +164,30 @@ public: * * mgs.allServer[X] is existed */ bool allShardRemovedFromServer(const UID& serverId); + + Future, int>> waitStorageMetrics(KeyRange const& keys, + StorageMetrics const& min, + StorageMetrics const& max, + StorageMetrics const& permittedError, + int shardLimit, + int expectedShardCount); + + Future getKeyLocation(TenantInfo tenant, + Key key, + SpanContext spanContext, + Optional debugID, + UseProvisionalProxies useProvisionalProxies, + Reverse isBackward, + Version version) override; + + Future> getKeyRangeLocations(TenantInfo tenant, + KeyRange keys, + int limit, + Reverse reverse, + SpanContext spanContext, + Optional debugID, + UseProvisionalProxies useProvisionalProxies, + Version version) override; }; #endif // FOUNDATIONDB_MOCKGLOBALSTATE_H From 5d90703dc8b02af81f8a1a0f9e383008983e6f63 Mon Sep 17 00:00:00 2001 From: Xiaoxi Wang Date: Wed, 19 Oct 2022 11:43:29 -0700 Subject: [PATCH 13/95] finish getKeysLocations etc, and unit test pass. --- fdbserver/DDTeamCollection.actor.cpp | 2 +- fdbserver/DDTxnProcessor.actor.cpp | 4 +- fdbserver/MockGlobalState.cpp | 166 ++++++++++++++++-- fdbserver/ShardsAffectedByTeamFailure.cpp | 12 +- fdbserver/include/fdbserver/MockGlobalState.h | 2 + .../fdbserver/ShardsAffectedByTeamFailure.h | 9 +- 6 files changed, 176 insertions(+), 19 deletions(-) diff --git a/fdbserver/DDTeamCollection.actor.cpp b/fdbserver/DDTeamCollection.actor.cpp index 253e7ee558..f0db0d48fa 100644 --- a/fdbserver/DDTeamCollection.actor.cpp +++ b/fdbserver/DDTeamCollection.actor.cpp @@ -895,7 +895,7 @@ public: if (maxPriority < SERVER_KNOBS->PRIORITY_TEAM_FAILED) { std::pair, std::vector> - teams = self->shardsAffectedByTeamFailure->getTeamsFor(shards[i]); + teams = self->shardsAffectedByTeamFailure->getTeamsForFirstShard(shards[i]); for (int j = 0; j < teams.first.size() + teams.second.size(); j++) { // t is the team in primary DC or the remote DC auto& t = diff --git a/fdbserver/DDTxnProcessor.actor.cpp b/fdbserver/DDTxnProcessor.actor.cpp index bcd8acf0d0..5a5b1de276 100644 --- a/fdbserver/DDTxnProcessor.actor.cpp +++ b/fdbserver/DDTxnProcessor.actor.cpp @@ -763,7 +763,7 @@ std::vector DDMockTxnProcessor::getDDShardInfos() const { KeyRangeRef curRange = it->range(); DDShardInfo info(curRange.begin); - auto teams = mgs->shardMapping->getTeamsFor(curRange); + auto teams = mgs->shardMapping->getTeamsForFirstShard(curRange); if (!teams.first.empty() && !teams.second.empty()) { CODE_PROBE(true, "Mock InitialDataDistribution In-Flight shard"); info.hasDest = true; @@ -909,7 +909,7 @@ void DDMockTxnProcessor::rawFinishMovement(MoveKeysParams& params, ASSERT(params.finishMoveKeysParallelismLock->take().isReady()); // get source and dest teams - auto [destTeams, srcTeams] = mgs->shardMapping->getTeamsFor(params.keys); + auto [destTeams, srcTeams] = mgs->shardMapping->getTeamsForFirstShard(params.keys); ASSERT_EQ(destTeams.size(), 0); if (destTeams.front() != ShardsAffectedByTeamFailure::Team{ params.destinationTeam, true }) { diff --git a/fdbserver/MockGlobalState.cpp b/fdbserver/MockGlobalState.cpp index 31162494fe..7596923fc8 100644 --- a/fdbserver/MockGlobalState.cpp +++ b/fdbserver/MockGlobalState.cpp @@ -19,6 +19,7 @@ */ #include "fdbserver/MockGlobalState.h" +#include "fdbserver/workloads/workloads.actor.h" bool MockStorageServer::allShardStatusEqual(KeyRangeRef range, MockShardStatus status) { auto ranges = serverKeys.intersectingRanges(range); @@ -159,7 +160,7 @@ bool MockGlobalState::serverIsSourceForShard(const UID& serverId, KeyRangeRef sh } // check keyServers - auto teams = shardMapping->getTeamsFor(shard); + auto teams = shardMapping->getTeamsForFirstShard(shard); if (inFlightShard) { return std::any_of(teams.second.begin(), teams.second.end(), [&serverId](const Team& team) { return team.hasServer(serverId); @@ -180,7 +181,7 @@ bool MockGlobalState::serverIsDestForShard(const UID& serverId, KeyRangeRef shar } // check keyServers - auto teams = shardMapping->getTeamsFor(shard); + auto teams = shardMapping->getTeamsForFirstShard(shard); return !teams.second.empty() && std::any_of(teams.first.begin(), teams.first.end(), [&serverId](const Team& team) { return team.hasServer(serverId); }); @@ -200,6 +201,17 @@ Future, int>> MockGlobalState::waitStorageMet return Future, int>>(); } +Reference buildLocationInfo(const std::vector& interfaces) { + // construct the location info with the servers + std::vector>> serverRefs; + serverRefs.reserve(interfaces.size()); + for (const auto& interf : interfaces) { + serverRefs.push_back(makeReference>(interf)); + } + + return makeReference(serverRefs); +} + Future MockGlobalState::getKeyLocation(TenantInfo tenant, Key key, SpanContext spanContext, @@ -207,23 +219,25 @@ Future MockGlobalState::getKeyLocation(TenantInfo tenant, UseProvisionalProxies useProvisionalProxies, Reverse isBackward, Version version) { - GetKeyServerLocationsReply rep; - - // construct the location info with the servers - std::vector>> serverRefs; - auto& servers = rep.results[0].second; - serverRefs.reserve(servers.size()); - for (const auto& interf : servers) { - serverRefs.push_back(makeReference>(interf)); + if (isBackward) { + // DD never ask for backward range. + UNREACHABLE(); } + ASSERT(key < allKeys.end); - auto locationInfo = makeReference(serverRefs); + GetKeyServerLocationsReply rep; + KeyRange single = singleKeyRange(key); + auto teamPair = shardMapping->getTeamsForFirstShard(single); + auto& srcTeam = teamPair.second.empty() ? teamPair.first : teamPair.second; + ASSERT_EQ(srcTeam.size(), 1); + rep.results.emplace_back(single, extractStorageServerInterfaces(srcTeam.front().servers)); return KeyRangeLocationInfo( rep.tenantEntry, KeyRange(toPrefixRelativeRange(rep.results[0].first, rep.tenantEntry.prefix), rep.arena), - locationInfo); + buildLocationInfo(rep.results[0].second)); } + Future> MockGlobalState::getKeyRangeLocations( TenantInfo tenant, KeyRange keys, @@ -233,7 +247,39 @@ Future> MockGlobalState::getKeyRangeLocations( Optional debugID, UseProvisionalProxies useProvisionalProxies, Version version) { - return Future>(); + + if (reverse) { + // DD never ask for backward range. + UNREACHABLE(); + } + ASSERT(keys.begin < keys.end); + + GetKeyServerLocationsReply rep; + auto ranges = shardMapping->intersectingRanges(keys); + auto it = ranges.begin(); + for (int count = 0; it != ranges.end() && count < limit; ++it, ++count) { + auto teamPair = shardMapping->getTeamsFor(it->begin()); + auto& srcTeam = teamPair.second.empty() ? teamPair.first : teamPair.second; + ASSERT_EQ(srcTeam.size(), 1); + rep.results.emplace_back(it->range(), extractStorageServerInterfaces(srcTeam.front().servers)); + } + CODE_PROBE(it != ranges.end(), "getKeyRangeLocations is limited", probe::decoration::rare); + + std::vector results; + for (int shard = 0; shard < rep.results.size(); shard++) { + results.emplace_back(rep.tenantEntry, + (toPrefixRelativeRange(rep.results[shard].first, rep.tenantEntry.prefix) & keys), + buildLocationInfo(rep.results[shard].second)); + } + return results; +} + +std::vector MockGlobalState::extractStorageServerInterfaces(const std::vector& ids) const { + std::vector interfaces; + for (auto& id : ids) { + interfaces.emplace_back(allServers.at(id).ssi); + } + return interfaces; } TEST_CASE("/MockGlobalState/initializeAsEmptyDatabaseMGS/SimpleThree") { @@ -302,6 +348,28 @@ struct MockGlobalStateTester { ranges.pop_front(); ASSERT(ranges.empty()); } + + KeyRangeLocationInfo getKeyLocationInfo(KeyRef key, std::shared_ptr mgs) { + return mgs + ->getKeyLocation( + TenantInfo(), key, SpanContext(), Optional(), UseProvisionalProxies::False, Reverse::False, 0) + .get(); + } + + std::vector getKeyRangeLocations(KeyRangeRef keys, + int limit, + std::shared_ptr mgs) { + return mgs + ->getKeyRangeLocations(TenantInfo(), + keys, + limit, + Reverse::False, + SpanContext(), + Optional(), + UseProvisionalProxies::False, + 0) + .get(); + } }; TEST_CASE("/MockGlobalState/MockStorageServer/SplittingFunctions") { @@ -325,3 +393,75 @@ TEST_CASE("/MockGlobalState/MockStorageServer/SplittingFunctions") { return Void(); } + +namespace { +inline bool locationInfoEqualsToTeam(Reference loc, const std::vector& ids) { + return loc->locations()->size() == ids.size() && + std::all_of(ids.begin(), ids.end(), [loc](const UID& id) { return loc->locations()->hasInterface(id); }); +} +}; // namespace +TEST_CASE("/MockGlobalState/MockStorageServer/GetKeyLocations") { + BasicTestConfig testConfig; + testConfig.simpleConfig = true; + testConfig.minimumReplication = 1; + testConfig.logAntiQuorum = 0; + DatabaseConfiguration dbConfig = generateNormalDatabaseConfiguration(testConfig); + TraceEvent("UnitTestDbConfig").detail("Config", dbConfig.toString()); + + auto mgs = std::make_shared(); + mgs->initializeAsEmptyDatabaseMGS(dbConfig); + // add one empty server + mgs->addStorageServer(StorageServerInterface(mgs->indexToUID(mgs->allServers.size() + 1))); + + // define 3 ranges: + // team 1 (UID 1,2,...,n-1):[begin, 1.0), [2.0, end) + // team 2 (UID 2,3,...n-1, n): [1.0, 2.0) + ShardsAffectedByTeamFailure::Team team1, team2; + for (int i = 0; i < mgs->allServers.size() - 1; ++i) { + UID id = mgs->indexToUID(i + 1); + team1.servers.emplace_back(id); + id = mgs->indexToUID(i + 2); + team2.servers.emplace_back(id); + } + Key one = doubleToTestKey(1.0), two = doubleToTestKey(2.0); + std::vector ranges{ KeyRangeRef(allKeys.begin, one), + KeyRangeRef(one, two), + KeyRangeRef(two, allKeys.end) }; + mgs->shardMapping->assignRangeToTeams(ranges[0], { team1 }); + mgs->shardMapping->assignRangeToTeams(ranges[1], { team2 }); + mgs->shardMapping->assignRangeToTeams(ranges[2], { team1 }); + + // query key location + MockGlobalStateTester tester; + // -- team 1 + Key testKey = doubleToTestKey(0.5); + auto locInfo = tester.getKeyLocationInfo(testKey, mgs); + ASSERT(locationInfoEqualsToTeam(locInfo.locations, team1.servers)); + + // -- team 2 + testKey = doubleToTestKey(1.3); + locInfo = tester.getKeyLocationInfo(testKey, mgs); + ASSERT(locationInfoEqualsToTeam(locInfo.locations, team2.servers)); + + // query range location + testKey = doubleToTestKey(3.0); + // team 1,2,1 + auto locInfos = tester.getKeyRangeLocations(KeyRangeRef(allKeys.begin, testKey), 100, mgs); + ASSERT(locInfos.size() == 3); + ASSERT(locInfos[0].range == ranges[0]); + ASSERT(locationInfoEqualsToTeam(locInfos[0].locations, team1.servers)); + ASSERT(locInfos[1].range == ranges[1]); + ASSERT(locationInfoEqualsToTeam(locInfos[1].locations, team2.servers)); + ASSERT(locInfos[2].range == KeyRangeRef(ranges[2].begin, testKey)); + ASSERT(locationInfoEqualsToTeam(locInfos[2].locations, team1.servers)); + + // team 1,2 + locInfos = tester.getKeyRangeLocations(KeyRangeRef(allKeys.begin, testKey), 2, mgs); + ASSERT(locInfos.size() == 2); + ASSERT(locInfos[0].range == ranges[0]); + ASSERT(locationInfoEqualsToTeam(locInfos[0].locations, team1.servers)); + ASSERT(locInfos[1].range == ranges[1]); + ASSERT(locationInfoEqualsToTeam(locInfos[1].locations, team2.servers)); + + return Void(); +} diff --git a/fdbserver/ShardsAffectedByTeamFailure.cpp b/fdbserver/ShardsAffectedByTeamFailure.cpp index d834febaf7..cc634689ec 100644 --- a/fdbserver/ShardsAffectedByTeamFailure.cpp +++ b/fdbserver/ShardsAffectedByTeamFailure.cpp @@ -40,10 +40,16 @@ int ShardsAffectedByTeamFailure::getNumberOfShards(UID ssID) const { } std::pair, std::vector> -ShardsAffectedByTeamFailure::getTeamsFor(KeyRangeRef keys) { +ShardsAffectedByTeamFailure::getTeamsForFirstShard(KeyRangeRef keys) { return shard_teams[keys.begin]; } +std::pair, std::vector> + +ShardsAffectedByTeamFailure::getTeamsFor(KeyRef key) { + return shard_teams[key]; +} + void ShardsAffectedByTeamFailure::erase(Team team, KeyRange const& range) { DisabledTraceEvent(SevDebug, "ShardsAffectedByTeamFailureErase") .detail("Range", range) @@ -236,3 +242,7 @@ void ShardsAffectedByTeamFailure::removeFailedServerForRange(KeyRangeRef keys, c } check(); } + +auto ShardsAffectedByTeamFailure::intersectingRanges(KeyRangeRef keyRange) const -> decltype(shard_teams)::ConstRanges { + return shard_teams.intersectingRanges(keyRange); +} diff --git a/fdbserver/include/fdbserver/MockGlobalState.h b/fdbserver/include/fdbserver/MockGlobalState.h index ce5c34d32e..b091f263ff 100644 --- a/fdbserver/include/fdbserver/MockGlobalState.h +++ b/fdbserver/include/fdbserver/MockGlobalState.h @@ -116,6 +116,8 @@ protected: class MockGlobalState : public IKeyLocationService { friend struct MockGlobalStateTester; + std::vector extractStorageServerInterfaces(const std::vector& ids) const; + public: typedef ShardsAffectedByTeamFailure::Team Team; // In-memory counterpart of the `keyServers` in system keyspace diff --git a/fdbserver/include/fdbserver/ShardsAffectedByTeamFailure.h b/fdbserver/include/fdbserver/ShardsAffectedByTeamFailure.h index 3f85fefb7a..9055098bc7 100644 --- a/fdbserver/include/fdbserver/ShardsAffectedByTeamFailure.h +++ b/fdbserver/include/fdbserver/ShardsAffectedByTeamFailure.h @@ -80,8 +80,12 @@ public: bool hasShards(Team team) const; // The first element of the pair is either the source for non-moving shards or the destination team for in-flight - // shards The second element of the pair is all previous sources for in-flight shards - std::pair, std::vector> getTeamsFor(KeyRangeRef keys); + // shards The second element of the pair is all previous sources for in-flight shards. This function only return the + // teams for the first shard in [keys.begin, keys.end) + std::pair, std::vector> getTeamsForFirstShard(KeyRangeRef keys); + + std::pair, std::vector> getTeamsFor(KeyRef key); + // Shard boundaries are modified in defineShard and the content of what servers correspond to each shard is a copy // or union of the shards already there void defineShard(KeyRangeRef keys); @@ -124,6 +128,7 @@ private: public: // return the iterator that traversing all ranges auto getAllRanges() const -> decltype(shard_teams)::ConstRanges; + auto intersectingRanges(KeyRangeRef keyRange) const -> decltype(shard_teams)::ConstRanges; // get total shards count size_t getNumberOfShards() const; void removeFailedServerForRange(KeyRangeRef keys, const UID& serverID); From 1603926595b0c9972b780b56de6d0974c500b066 Mon Sep 17 00:00:00 2001 From: Xiaoxi Wang Date: Wed, 19 Oct 2022 23:20:31 -0700 Subject: [PATCH 14/95] refactoring old waitStorageMetrics and finish MGS::waitStorageMetrics (no unit test yet) --- fdbclient/NativeAPI.actor.cpp | 69 ++++++++++++------- fdbclient/include/fdbclient/NativeAPI.actor.h | 10 +++ ...balState.cpp => MockGlobalState.actor.cpp} | 41 ++++++++++- fdbserver/include/fdbserver/MockGlobalState.h | 26 ++++--- 4 files changed, 107 insertions(+), 39 deletions(-) rename fdbserver/{MockGlobalState.cpp => MockGlobalState.actor.cpp} (90%) diff --git a/fdbclient/NativeAPI.actor.cpp b/fdbclient/NativeAPI.actor.cpp index 4c1b75d905..41fd90cd52 100644 --- a/fdbclient/NativeAPI.actor.cpp +++ b/fdbclient/NativeAPI.actor.cpp @@ -1915,7 +1915,8 @@ Optional DatabaseContext::getCachedLocation(const Optional auto range = isBackward ? locationCache.rangeContainingKeyBefore(resolvedKey) : locationCache.rangeContaining(resolvedKey); if (range->value()) { - return KeyRangeLocationInfo(tenantEntry, toPrefixRelativeRange(range->range(), tenantEntry.prefix), range->value()); + return KeyRangeLocationInfo( + tenantEntry, toPrefixRelativeRange(range->range(), tenantEntry.prefix), range->value()); } return Optional(); @@ -1952,7 +1953,8 @@ bool DatabaseContext::getCachedLocations(const Optional& tenantNa result.clear(); return false; } - result.emplace_back(tenantEntry, toPrefixRelativeRange(r->range() & resolvedRange, tenantEntry.prefix), r->value()); + result.emplace_back( + tenantEntry, toPrefixRelativeRange(r->range() & resolvedRange, tenantEntry.prefix), r->value()); if (result.size() == limit || begin == end) { break; } @@ -7714,6 +7716,34 @@ ACTOR Future>> getReadHotRanges(Da } } +ACTOR Future> waitStorageMetricsWithLocation(TenantInfo tenantInfo, + KeyRange keys, + std::vector locations, + StorageMetrics min, + StorageMetrics max, + StorageMetrics permittedError) { + try { + Future fx; + if (locations.size() > 1) { + fx = waitStorageMetricsMultipleLocations(tenantInfo, locations, min, max, permittedError); + } else { + WaitMetricsRequest req(tenantInfo, keys, min, max); + fx = loadBalance(locations[0].locations->locations(), + &StorageServerInterface::waitMetrics, + req, + TaskPriority::DataDistribution); + } + StorageMetrics x = wait(fx); + return x; + } catch (Error& e) { + if (e.code() != error_code_wrong_shard_server && e.code() != error_code_all_alternatives_failed) { + TraceEvent(SevError, "WaitStorageMetricsError").error(e); + throw; + } + } + return Optional(); +} + ACTOR Future, int>> waitStorageMetrics( Database cx, KeyRange keys, @@ -7743,30 +7773,8 @@ ACTOR Future, int>> waitStorageMetrics( } // SOMEDAY: Right now, if there are too many shards we delay and check again later. There may be a better - // solution to this. - if (locations.size() < shardLimit) { - try { - Future fx; - if (locations.size() > 1) { - fx = waitStorageMetricsMultipleLocations(tenantInfo, locations, min, max, permittedError); - } else { - WaitMetricsRequest req(tenantInfo, keys, min, max); - fx = loadBalance(locations[0].locations->locations(), - &StorageServerInterface::waitMetrics, - req, - TaskPriority::DataDistribution); - } - StorageMetrics x = wait(fx); - return std::make_pair(x, -1); - } catch (Error& e) { - if (e.code() != error_code_wrong_shard_server && e.code() != error_code_all_alternatives_failed) { - TraceEvent(SevError, "WaitStorageMetricsError").error(e); - throw; - } - cx->invalidateCache(locations[0].tenantEntry.prefix, keys); - wait(delay(CLIENT_KNOBS->WRONG_SHARD_SERVER_DELAY, TaskPriority::DataDistribution)); - } - } else { + // solution to this. How could this happen? + if (locations.size() >= shardLimit) { TraceEvent(SevWarn, "WaitStorageMetricsPenalty") .detail("Keys", keys) .detail("Limit", shardLimit) @@ -7775,7 +7783,16 @@ ACTOR Future, int>> waitStorageMetrics( 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(locations[0].tenantEntry.prefix, keys); + continue; } + + Optional res = + wait(waitStorageMetricsWithLocation(tenantInfo, keys, locations, min, max, permittedError)); + if (res.present()) { + return std::make_pair(res, -1); + } + cx->invalidateCache(locations[0].tenantEntry.prefix, keys); + wait(delay(CLIENT_KNOBS->WRONG_SHARD_SERVER_DELAY, TaskPriority::DataDistribution)); } } diff --git a/fdbclient/include/fdbclient/NativeAPI.actor.h b/fdbclient/include/fdbclient/NativeAPI.actor.h index 3931182ab0..1d94fc14f4 100644 --- a/fdbclient/include/fdbclient/NativeAPI.actor.h +++ b/fdbclient/include/fdbclient/NativeAPI.actor.h @@ -591,6 +591,16 @@ int64_t getMaxWriteKeySize(KeyRef const& key, bool hasRawAccess); // Returns the maximum legal size of a key that can be cleared. Keys larger than this will be assumed not to exist. int64_t getMaxClearKeySize(KeyRef const& key); +struct KeyRangeLocationInfo; +// Return the aggregated StorageMetrics of range keys to the caller. The locations tell which interface should +// serve the request. The final result is within (min-permittedError/2, max + permittedError/2) if valid. +ACTOR Future> waitStorageMetricsWithLocation(TenantInfo tenantInfo, + KeyRange keys, + std::vector locations, + StorageMetrics min, + StorageMetrics max, + StorageMetrics permittedError); + namespace NativeAPI { ACTOR Future>> getServerListAndProcessClasses( Transaction* tr); diff --git a/fdbserver/MockGlobalState.cpp b/fdbserver/MockGlobalState.actor.cpp similarity index 90% rename from fdbserver/MockGlobalState.cpp rename to fdbserver/MockGlobalState.actor.cpp index 7596923fc8..e5e6263c71 100644 --- a/fdbserver/MockGlobalState.cpp +++ b/fdbserver/MockGlobalState.actor.cpp @@ -1,5 +1,5 @@ /* - * MockGlobalState.cpp + * MockGlobalState.actor.cpp * * This source file is part of the FoundationDB open source project * @@ -20,6 +20,42 @@ #include "fdbserver/MockGlobalState.h" #include "fdbserver/workloads/workloads.actor.h" +#include "flow/actorcompiler.h" + +class MockGlobalStateImpl { +public: + ACTOR static Future, int>> waitStorageMetrics(MockGlobalState* mgs, + KeyRange keys, + StorageMetrics min, + StorageMetrics max, + StorageMetrics permittedError, + int shardLimit, + int expectedShardCount) { + state TenantInfo tenantInfo; + loop { + auto locations = mgs->getKeyRangeLocations(tenantInfo, + keys, + shardLimit, + Reverse::False, + SpanContext(), + Optional(), + UseProvisionalProxies::False, + 0) + .get(); + // NOTE(xwang): in native API, there's code handling the non-equal situation, but I think in mock world + // there shouldn't have any delay to update the locations. + ASSERT_EQ(expectedShardCount, locations.size()); + + Optional res = + wait(::waitStorageMetricsWithLocation(tenantInfo, keys, locations, min, max, permittedError)); + + if (res.present()) { + return std::make_pair(res, -1); + } + wait(delay(CLIENT_KNOBS->WRONG_SHARD_SERVER_DELAY, TaskPriority::DataDistribution)); + } + } +}; bool MockStorageServer::allShardStatusEqual(KeyRangeRef range, MockShardStatus status) { auto ranges = serverKeys.intersectingRanges(range); @@ -198,7 +234,8 @@ Future, int>> MockGlobalState::waitStorageMet const StorageMetrics& permittedError, int shardLimit, int expectedShardCount) { - return Future, int>>(); + return MockGlobalStateImpl::waitStorageMetrics( + this, keys, min, max, permittedError, shardLimit, expectedShardCount); } Reference buildLocationInfo(const std::vector& interfaces) { diff --git a/fdbserver/include/fdbserver/MockGlobalState.h b/fdbserver/include/fdbserver/MockGlobalState.h index b091f263ff..70cb93d8c5 100644 --- a/fdbserver/include/fdbserver/MockGlobalState.h +++ b/fdbserver/include/fdbserver/MockGlobalState.h @@ -113,8 +113,11 @@ protected: void twoWayShardSplitting(KeyRangeRef range, KeyRef splitPoint, uint64_t rangeSize, bool restrictSize); }; +class MockGlobalStateImpl; + class MockGlobalState : public IKeyLocationService { friend struct MockGlobalStateTester; + friend class MockGlobalStateImpl; std::vector extractStorageServerInterfaces(const std::vector& ids) const; @@ -167,6 +170,7 @@ public: */ bool allShardRemovedFromServer(const UID& serverId); + // SOMEDAY: NativeAPI::waitStorageMetrics should share the code in the future, this is a simpler version of it Future, int>> waitStorageMetrics(KeyRange const& keys, StorageMetrics const& min, StorageMetrics const& max, @@ -175,21 +179,21 @@ public: int expectedShardCount); Future getKeyLocation(TenantInfo tenant, - Key key, + Key key, + SpanContext spanContext, + Optional debugID, + UseProvisionalProxies useProvisionalProxies, + Reverse isBackward, + Version version) override; + + Future> getKeyRangeLocations(TenantInfo tenant, + KeyRange keys, + int limit, + Reverse reverse, SpanContext spanContext, Optional debugID, UseProvisionalProxies useProvisionalProxies, - Reverse isBackward, Version version) override; - - Future> getKeyRangeLocations(TenantInfo tenant, - KeyRange keys, - int limit, - Reverse reverse, - SpanContext spanContext, - Optional debugID, - UseProvisionalProxies useProvisionalProxies, - Version version) override; }; #endif // FOUNDATIONDB_MOCKGLOBALSTATE_H From e07a50573a394491573ec4978d9605b8593df0fd Mon Sep 17 00:00:00 2001 From: Xiaoxi Wang Date: Thu, 20 Oct 2022 13:10:03 -0700 Subject: [PATCH 15/95] splitStorageMetrics finish implementation (no unit test yet but 100k test pass) --- fdbclient/NativeAPI.actor.cpp | 109 ++++++++++-------- fdbclient/include/fdbclient/NativeAPI.actor.h | 9 ++ fdbserver/DDTxnProcessor.actor.cpp | 3 +- fdbserver/MockGlobalState.actor.cpp | 42 +++++++ fdbserver/include/fdbserver/MockGlobalState.h | 5 + 5 files changed, 118 insertions(+), 50 deletions(-) diff --git a/fdbclient/NativeAPI.actor.cpp b/fdbclient/NativeAPI.actor.cpp index 41fd90cd52..4aece2f05f 100644 --- a/fdbclient/NativeAPI.actor.cpp +++ b/fdbclient/NativeAPI.actor.cpp @@ -8653,6 +8653,56 @@ Future DatabaseContext::splitStorageMetricsStream(const PromiseStream resultStream, Database(Reference::addRef(this)), keys, limit, estimated, minSplitBytes); } +ACTOR Future>>> splitStorageMetricsWithLocations( + std::vector locations, + KeyRange keys, + StorageMetrics limit, + StorageMetrics estimated, + Optional minSplitBytes) { + state StorageMetrics used; + state Standalone> results; + results.push_back_deep(results.arena(), keys.begin); + //TraceEvent("SplitStorageMetrics").detail("Locations", locations.size()); + try { + state int i = 0; + for (; i < locations.size(); i++) { + SplitMetricsRequest req( + locations[i].range, limit, used, estimated, i == locations.size() - 1, minSplitBytes); + SplitMetricsReply res = wait(loadBalance(locations[i].locations->locations(), + &StorageServerInterface::splitMetrics, + req, + TaskPriority::DataDistribution)); + if (res.splits.size() && res.splits[0] <= results.back()) { // split points are out of order, possibly + // because of moving data, throw error to retry + ASSERT_WE_THINK(false); // FIXME: This seems impossible and doesn't seem to be covered by testing + throw all_alternatives_failed(); + } + if (res.splits.size()) { + results.append(results.arena(), res.splits.begin(), res.splits.size()); + results.arena().dependsOn(res.splits.arena()); + } + used = res.used; + + //TraceEvent("SplitStorageMetricsResult").detail("Used", used.bytes).detail("Location", i).detail("Size", res.splits.size()); + } + + if (used.allLessOrEqual(limit * CLIENT_KNOBS->STORAGE_METRICS_UNFAIR_SPLIT_LIMIT) && results.size() > 1) { + results.resize(results.arena(), results.size() - 1); + } + + if (keys.end <= locations.back().range.end) { + results.push_back_deep(results.arena(), keys.end); + } + return results; + } catch (Error& e) { + if (e.code() != error_code_wrong_shard_server && e.code() != error_code_all_alternatives_failed) { + TraceEvent(SevError, "SplitStorageMetricsError").error(e); + throw; + } + } + return Optional>>(); +} + ACTOR Future>> splitStorageMetrics(Database cx, KeyRange keys, StorageMetrics limit, @@ -8671,61 +8721,24 @@ ACTOR Future>> splitStorageMetrics(Database cx, Optional(), UseProvisionalProxies::False, latestVersion)); - state StorageMetrics used; - state Standalone> results; // SOMEDAY: Right now, if there are too many shards we delay and check again later. There may be a better // solution to this. if (locations.size() == CLIENT_KNOBS->STORAGE_METRICS_SHARD_LIMIT) { wait(delay(CLIENT_KNOBS->STORAGE_METRICS_TOO_MANY_SHARDS_DELAY, TaskPriority::DataDistribution)); cx->invalidateCache(Key(), keys); - } else { - results.push_back_deep(results.arena(), keys.begin); - try { - //TraceEvent("SplitStorageMetrics").detail("Locations", locations.size()); - - state int i = 0; - for (; i < locations.size(); i++) { - SplitMetricsRequest req( - locations[i].range, limit, used, estimated, i == locations.size() - 1, minSplitBytes); - SplitMetricsReply res = wait(loadBalance(locations[i].locations->locations(), - &StorageServerInterface::splitMetrics, - req, - TaskPriority::DataDistribution)); - if (res.splits.size() && - res.splits[0] <= results.back()) { // split points are out of order, possibly because of - // moving data, throw error to retry - ASSERT_WE_THINK( - false); // FIXME: This seems impossible and doesn't seem to be covered by testing - throw all_alternatives_failed(); - } - if (res.splits.size()) { - results.append(results.arena(), res.splits.begin(), res.splits.size()); - results.arena().dependsOn(res.splits.arena()); - } - used = res.used; - - //TraceEvent("SplitStorageMetricsResult").detail("Used", used.bytes).detail("Location", i).detail("Size", res.splits.size()); - } - - if (used.allLessOrEqual(limit * CLIENT_KNOBS->STORAGE_METRICS_UNFAIR_SPLIT_LIMIT) && - results.size() > 1) { - results.resize(results.arena(), results.size() - 1); - } - - if (keys.end <= locations.back().range.end) { - results.push_back_deep(results.arena(), keys.end); - } - return results; - } catch (Error& e) { - if (e.code() != error_code_wrong_shard_server && e.code() != error_code_all_alternatives_failed) { - TraceEvent(SevError, "SplitStorageMetricsError").error(e); - throw; - } - cx->invalidateCache(Key(), keys); - wait(delay(CLIENT_KNOBS->WRONG_SHARD_SERVER_DELAY, TaskPriority::DataDistribution)); - } + continue; } + + Optional>> results = + wait(splitStorageMetricsWithLocations(locations, keys, limit, estimated, minSplitBytes)); + + if (results.present()) { + return results.get(); + } + + cx->invalidateCache(Key(), keys); + wait(delay(CLIENT_KNOBS->WRONG_SHARD_SERVER_DELAY, TaskPriority::DataDistribution)); } } diff --git a/fdbclient/include/fdbclient/NativeAPI.actor.h b/fdbclient/include/fdbclient/NativeAPI.actor.h index 1d94fc14f4..d1f4860f23 100644 --- a/fdbclient/include/fdbclient/NativeAPI.actor.h +++ b/fdbclient/include/fdbclient/NativeAPI.actor.h @@ -601,6 +601,15 @@ ACTOR Future> waitStorageMetricsWithLocation(TenantInfo StorageMetrics max, StorageMetrics permittedError); +// Return the suggested split points from storage server.The locations tell which interface should +// serve the request. The +ACTOR Future>>> splitStorageMetricsWithLocations( + std::vector locations, + KeyRange keys, + StorageMetrics limit, + StorageMetrics estimated, + Optional minSplitBytes); + namespace NativeAPI { ACTOR Future>> getServerListAndProcessClasses( Transaction* tr); diff --git a/fdbserver/DDTxnProcessor.actor.cpp b/fdbserver/DDTxnProcessor.actor.cpp index 5a5b1de276..2c4c695c23 100644 --- a/fdbserver/DDTxnProcessor.actor.cpp +++ b/fdbserver/DDTxnProcessor.actor.cpp @@ -862,13 +862,12 @@ Future DDMockTxnProcessor::getHealthMetrics(bool detailed) const return Future(); } -// FIXME: finish implementation Future>> DDMockTxnProcessor::splitStorageMetrics( const KeyRange& keys, const StorageMetrics& limit, const StorageMetrics& estimated, const Optional& minSplitBytes) const { - return Future>>(); + return mgs->splitStorageMetrics(keys, limit, estimated, minSplitBytes); } Future, int>> DDMockTxnProcessor::waitStorageMetrics( diff --git a/fdbserver/MockGlobalState.actor.cpp b/fdbserver/MockGlobalState.actor.cpp index e5e6263c71..bdeed264fd 100644 --- a/fdbserver/MockGlobalState.actor.cpp +++ b/fdbserver/MockGlobalState.actor.cpp @@ -55,6 +55,41 @@ public: wait(delay(CLIENT_KNOBS->WRONG_SHARD_SERVER_DELAY, TaskPriority::DataDistribution)); } } + + // SOMEDAY: reuse the NativeAPI implementation + ACTOR static Future>> splitStorageMetrics(MockGlobalState* mgs, + KeyRange keys, + StorageMetrics limit, + StorageMetrics estimated, + Optional minSplitBytes) { + state TenantInfo tenantInfo; + loop { + state std::vector locations = + mgs->getKeyRangeLocations(tenantInfo, + keys, + CLIENT_KNOBS->STORAGE_METRICS_SHARD_LIMIT, + Reverse::False, + SpanContext(), + Optional(), + UseProvisionalProxies::False, + 0) + .get(); + + // Same solution to NativeAPI::splitStorageMetrics, wait some merge finished + if (locations.size() == CLIENT_KNOBS->STORAGE_METRICS_SHARD_LIMIT) { + wait(delay(CLIENT_KNOBS->STORAGE_METRICS_TOO_MANY_SHARDS_DELAY, TaskPriority::DataDistribution)); + } + + Optional>> results = + wait(splitStorageMetricsWithLocations(locations, keys, limit, estimated, minSplitBytes)); + + if (results.present()) { + return results.get(); + } + + wait(delay(CLIENT_KNOBS->WRONG_SHARD_SERVER_DELAY, TaskPriority::DataDistribution)); + } + } }; bool MockStorageServer::allShardStatusEqual(KeyRangeRef range, MockShardStatus status) { @@ -319,6 +354,13 @@ std::vector MockGlobalState::extractStorageServerInterfa return interfaces; } +Future>> MockGlobalState::splitStorageMetrics(const KeyRange& keys, + const StorageMetrics& limit, + const StorageMetrics& estimated, + const Optional& minSplitBytes) { + return MockGlobalStateImpl::splitStorageMetrics(this, keys, limit, estimated, minSplitBytes); +} + TEST_CASE("/MockGlobalState/initializeAsEmptyDatabaseMGS/SimpleThree") { BasicTestConfig testConfig; testConfig.simpleConfig = true; diff --git a/fdbserver/include/fdbserver/MockGlobalState.h b/fdbserver/include/fdbserver/MockGlobalState.h index 70cb93d8c5..a8433d7ce2 100644 --- a/fdbserver/include/fdbserver/MockGlobalState.h +++ b/fdbserver/include/fdbserver/MockGlobalState.h @@ -178,6 +178,11 @@ public: int shardLimit, int expectedShardCount); + Future>> splitStorageMetrics(const KeyRange& keys, + const StorageMetrics& limit, + const StorageMetrics& estimated, + const Optional& minSplitBytes); + Future getKeyLocation(TenantInfo tenant, Key key, SpanContext spanContext, From c14ee5395f82839ef9dfce07a45a9fc8c5d8e1d3 Mon Sep 17 00:00:00 2001 From: Xiaoxi Wang Date: Thu, 20 Oct 2022 23:29:13 -0700 Subject: [PATCH 16/95] define IStorageMetricsService --- fdbserver/MockGlobalState.actor.cpp | 4 +++ fdbserver/include/fdbserver/StorageMetrics.h | 22 ++++++++++++++ fdbserver/storageserver.actor.cpp | 30 +++++++++++++------- 3 files changed, 46 insertions(+), 10 deletions(-) diff --git a/fdbserver/MockGlobalState.actor.cpp b/fdbserver/MockGlobalState.actor.cpp index bdeed264fd..d40d280e78 100644 --- a/fdbserver/MockGlobalState.actor.cpp +++ b/fdbserver/MockGlobalState.actor.cpp @@ -203,6 +203,10 @@ uint64_t MockStorageServer::sumRangeSize(KeyRangeRef range) const { return totalSize; } +Future MockStorageServer::serverInterface() const { + return Future(); +} + void MockGlobalState::initializeAsEmptyDatabaseMGS(const DatabaseConfiguration& conf, uint64_t defaultDiskSpace) { ASSERT(conf.storageTeamSize > 0); configuration = conf; diff --git a/fdbserver/include/fdbserver/StorageMetrics.h b/fdbserver/include/fdbserver/StorageMetrics.h index f0f7008cc4..98d562d665 100644 --- a/fdbserver/include/fdbserver/StorageMetrics.h +++ b/fdbserver/include/fdbserver/StorageMetrics.h @@ -152,3 +152,25 @@ struct ByteSampleInfo { // Determines whether a key-value pair should be included in a byte sample // Also returns size information about the sample ByteSampleInfo isKeyValueInSample(KeyValueRef keyValue); + +class IStorageMetricsService { +public: + StorageServerMetrics metrics; + + // penalty used by loadBalance() to balance requests among service instances + virtual double getPenalty() { return 1; } + + virtual bool isReadable(KeyRangeRef const& keys) { return true; } + + virtual void addActor(Future future) = 0; + + virtual void getSplitPoints(SplitRangeRequest const& req) = 0; + + virtual Future waitMetricsTenantAware(const WaitMetricsRequest& req) = 0; + + virtual void getStorageMetrics(const GetStorageMetricsRequest& req) = 0; + + // NOTE: also need to have this function but template can't be a virtual so... + // template + // void sendErrorWithPenalty(const ReplyPromise& promise, const Error& err, double penalty); +}; diff --git a/fdbserver/storageserver.actor.cpp b/fdbserver/storageserver.actor.cpp index aebec03a41..396d4296f8 100644 --- a/fdbserver/storageserver.actor.cpp +++ b/fdbserver/storageserver.actor.cpp @@ -641,7 +641,7 @@ struct BusiestWriteTagContext { busiestWriteTagEventHolder(makeReference(busiestWriteTagTrackingKey)), lastUpdateTime(-1) {} }; -struct StorageServer { +struct StorageServer : public IStorageMetricsService { typedef VersionedMap VersionedData; private: @@ -988,7 +988,6 @@ public: Database cx; ActorCollection actors; - StorageServerMetrics metrics; CoalescedKeyRangeMap> byteSampleClears; AsyncVar byteSampleClearsTooLarge; Future byteSampleRecovery; @@ -1380,7 +1379,7 @@ public: // This is the maximum version that might be read from storage (the minimum version is durableVersion) Version storageVersion() const { return oldestVersion.get(); } - bool isReadable(KeyRangeRef const& keys) { + bool isReadable(KeyRangeRef const& keys) override { auto sh = shards.intersectingRanges(keys); for (auto i = sh.begin(); i != sh.end(); ++i) if (!i->value()->isReadable()) @@ -1409,7 +1408,7 @@ public: Counter::Value queueSize() { return counters.bytesInput.getValue() - counters.bytesDurable.getValue(); } // penalty used by loadBalance() to balance requests among SSes. We prefer SS with less write queue size. - double getPenalty() { + double getPenalty() override { return std::max(std::max(1.0, (queueSize() - (SERVER_KNOBS->TARGET_BYTES_PER_STORAGE_SERVER - 2.0 * SERVER_KNOBS->SPRING_BYTES_STORAGE_SERVER)) / @@ -1503,7 +1502,7 @@ public: } } - void getSplitPoints(SplitRangeRequest const& req) { + void getSplitPoints(SplitRangeRequest const& req) override { try { Optional entry = getTenantEntry(version.get(), req.tenantInfo); metrics.getSplitPoints(req, entry.map([](TenantMapEntry e) { return e.prefix; })); @@ -1533,6 +1532,15 @@ public: } return false; } + + Future waitMetricsTenantAware(const WaitMetricsRequest& req) override; + + void addActor(Future future) override { actors.add(future); } + + void getStorageMetrics(const GetStorageMetricsRequest& req) override { + StorageBytes sb = storage.getStorageBytes(); + metrics.getStorageMetrics(req, sb, counters.bytesInput.getRate(), versionLag, lastUpdate); + } }; const StringRef StorageServer::CurrentRunningFetchKeys::emptyString = ""_sr; @@ -10166,7 +10174,7 @@ Future StorageServerMetrics::waitMetrics(WaitMetricsRequest req, Future waitMetricsTenantAware(StorageServer* self, WaitMetricsRequest req) { +ACTOR Future waitMetricsTenantAware_internal(StorageServer* self, WaitMetricsRequest req) { if (req.tenantInfo.present() && req.tenantInfo.get().tenantId != TenantInfo::INVALID_TENANT) { wait(success(waitForVersionNoTooOld(self, latestVersion))); Optional entry = self->getTenantEntry(latestVersion, req.tenantInfo.get()); @@ -10184,6 +10192,10 @@ ACTOR Future waitMetricsTenantAware(StorageServer* self, WaitMetricsReques return Void(); } +Future StorageServer::waitMetricsTenantAware(const WaitMetricsRequest& req) { + return waitMetricsTenantAware_internal(this, req); +} + ACTOR Future metricsCore(StorageServer* self, StorageServerInterface ssi) { state Future doPollMetrics = Void(); @@ -10225,7 +10237,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 { - self->actors.add(waitMetricsTenantAware(self, req)); + self->addActor(self->waitMetricsTenantAware(req)); } } when(SplitMetricsRequest req = waitNext(ssi.splitMetrics.getFuture())) { @@ -10237,9 +10249,7 @@ ACTOR Future metricsCore(StorageServer* self, StorageServerInterface ssi) } } when(GetStorageMetricsRequest req = waitNext(ssi.getStorageMetrics.getFuture())) { - StorageBytes sb = self->storage.getStorageBytes(); - self->metrics.getStorageMetrics( - req, sb, self->counters.bytesInput.getRate(), self->versionLag, self->lastUpdate); + self->getStorageMetrics(req); } when(ReadHotSubRangeRequest req = waitNext(ssi.getReadHotRanges.getFuture())) { if (!self->isReadable(req.keys)) { From 3c67b7df397c10519502cb9fc953e76bd3f9dfa3 Mon Sep 17 00:00:00 2001 From: Xiaoxi Wang Date: Fri, 21 Oct 2022 10:59:22 -0700 Subject: [PATCH 17/95] extract serveStorageMetricsRequests template function --- fdbserver/ConsistencyScan.actor.cpp | 2 +- fdbserver/MockGlobalState.actor.cpp | 4 -- fdbserver/Resolver.actor.cpp | 2 +- fdbserver/RestoreLoader.actor.cpp | 2 +- fdbserver/StorageMetrics.actor.cpp | 2 +- fdbserver/include/fdbserver/MockGlobalState.h | 2 +- ...torageMetrics.h => StorageMetrics.actor.h} | 58 ++++++++++++++++++- fdbserver/storageserver.actor.cpp | 48 +-------------- .../workloads/ConsistencyCheck.actor.cpp | 2 +- 9 files changed, 66 insertions(+), 56 deletions(-) rename fdbserver/include/fdbserver/{StorageMetrics.h => StorageMetrics.actor.h} (75%) diff --git a/fdbserver/ConsistencyScan.actor.cpp b/fdbserver/ConsistencyScan.actor.cpp index f961cb20a0..657da9fda4 100644 --- a/fdbserver/ConsistencyScan.actor.cpp +++ b/fdbserver/ConsistencyScan.actor.cpp @@ -29,7 +29,7 @@ #include "fdbclient/ReadYourWrites.h" #include "fdbclient/TagThrottle.actor.h" #include "fdbserver/Knobs.h" -#include "fdbserver/StorageMetrics.h" +#include "fdbserver/StorageMetrics.actor.h" #include "fdbserver/DataDistribution.actor.h" #include "fdbserver/RatekeeperInterface.h" #include "fdbserver/ServerDBInfo.h" diff --git a/fdbserver/MockGlobalState.actor.cpp b/fdbserver/MockGlobalState.actor.cpp index d40d280e78..bdeed264fd 100644 --- a/fdbserver/MockGlobalState.actor.cpp +++ b/fdbserver/MockGlobalState.actor.cpp @@ -203,10 +203,6 @@ uint64_t MockStorageServer::sumRangeSize(KeyRangeRef range) const { return totalSize; } -Future MockStorageServer::serverInterface() const { - return Future(); -} - void MockGlobalState::initializeAsEmptyDatabaseMGS(const DatabaseConfiguration& conf, uint64_t defaultDiskSpace) { ASSERT(conf.storageTeamSize > 0); configuration = conf; diff --git a/fdbserver/Resolver.actor.cpp b/fdbserver/Resolver.actor.cpp index 61aaed1246..203383cc9f 100644 --- a/fdbserver/Resolver.actor.cpp +++ b/fdbserver/Resolver.actor.cpp @@ -35,7 +35,7 @@ #include "fdbserver/ResolverInterface.h" #include "fdbserver/RestoreUtil.h" #include "fdbserver/ServerDBInfo.h" -#include "fdbserver/StorageMetrics.h" +#include "fdbserver/StorageMetrics.actor.h" #include "fdbserver/WaitFailure.h" #include "fdbserver/WorkerInterface.actor.h" #include "flow/ActorCollection.h" diff --git a/fdbserver/RestoreLoader.actor.cpp b/fdbserver/RestoreLoader.actor.cpp index 964045c153..dfcd329cc8 100644 --- a/fdbserver/RestoreLoader.actor.cpp +++ b/fdbserver/RestoreLoader.actor.cpp @@ -30,7 +30,7 @@ #include "fdbserver/RestoreLoader.actor.h" #include "fdbserver/RestoreRoleCommon.actor.h" #include "fdbserver/MutationTracking.h" -#include "fdbserver/StorageMetrics.h" +#include "fdbserver/StorageMetrics.actor.h" #include "flow/actorcompiler.h" // This must be the last #include. diff --git a/fdbserver/StorageMetrics.actor.cpp b/fdbserver/StorageMetrics.actor.cpp index 88bae0695f..ada2301483 100644 --- a/fdbserver/StorageMetrics.actor.cpp +++ b/fdbserver/StorageMetrics.actor.cpp @@ -19,7 +19,7 @@ */ #include "flow/UnitTest.h" -#include "fdbserver/StorageMetrics.h" +#include "fdbserver/StorageMetrics.actor.h" #include "flow/actorcompiler.h" // This must be the last #include. int64_t StorageMetricSample::getEstimate(KeyRangeRef keys) const { diff --git a/fdbserver/include/fdbserver/MockGlobalState.h b/fdbserver/include/fdbserver/MockGlobalState.h index a8433d7ce2..f3e5213892 100644 --- a/fdbserver/include/fdbserver/MockGlobalState.h +++ b/fdbserver/include/fdbserver/MockGlobalState.h @@ -21,7 +21,7 @@ #ifndef FOUNDATIONDB_MOCKGLOBALSTATE_H #define FOUNDATIONDB_MOCKGLOBALSTATE_H -#include "StorageMetrics.h" +#include "StorageMetrics.actor.h" #include "fdbclient/KeyRangeMap.h" #include "fdbclient/StorageServerInterface.h" #include "fdbclient/DatabaseConfiguration.h" diff --git a/fdbserver/include/fdbserver/StorageMetrics.h b/fdbserver/include/fdbserver/StorageMetrics.actor.h similarity index 75% rename from fdbserver/include/fdbserver/StorageMetrics.h rename to fdbserver/include/fdbserver/StorageMetrics.actor.h index 98d562d665..7111227f27 100644 --- a/fdbserver/include/fdbserver/StorageMetrics.h +++ b/fdbserver/include/fdbserver/StorageMetrics.actor.h @@ -19,13 +19,18 @@ */ #pragma once - +#if defined(NO_INTELLISENSE) && !defined(FDBSERVER_STORAGEMETRICS_G_H) +#define FDBSERVER_STORAGEMETRICS_G_H +#include "fdbserver/StorageMetrics.actor.g.h" +#elif !defined(FDBSERVER_STORAGEMETRICS_H) +#define FDBSERVER_STORAGEMETRICS_H #include "fdbclient/FDBTypes.h" #include "fdbrpc/simulator.h" #include "flow/UnitTest.h" #include "fdbclient/StorageServerInterface.h" #include "fdbclient/KeyRangeMap.h" #include "fdbserver/Knobs.h" +#include "flow/actorcompiler.h" const StringRef STORAGESERVER_HISTOGRAM_GROUP = "StorageServer"_sr; const StringRef FETCH_KEYS_LATENCY_HISTOGRAM = "FetchKeysLatency"_sr; @@ -174,3 +179,54 @@ public: // template // void sendErrorWithPenalty(const ReplyPromise& promise, const Error& err, double penalty); }; + +ACTOR template +Future serveStorageMetricsRequests(ServiceType* self, StorageServerInterface ssi){ + state Future doPollMetrics = Void(); + loop { + choose { + when(state WaitMetricsRequest req = waitNext(ssi.waitMetrics.getFuture())) { + if (!req.tenantInfo.present() && !self->isReadable(req.keys)) { + CODE_PROBE(true, "waitMetrics immediate wrong_shard_server()"); + self->sendErrorWithPenalty(req.reply, wrong_shard_server(), self->getPenalty()); + } else { + self->addActor(self->waitMetricsTenantAware(req)); + } + } + when(SplitMetricsRequest req = waitNext(ssi.splitMetrics.getFuture())) { + if (!self->isReadable(req.keys)) { + CODE_PROBE(true, "splitMetrics immediate wrong_shard_server()"); + self->sendErrorWithPenalty(req.reply, wrong_shard_server(), self->getPenalty()); + } else { + self->metrics.splitMetrics(req); + } + } + when(GetStorageMetricsRequest req = waitNext(ssi.getStorageMetrics.getFuture())) { + self->getStorageMetrics(req); + } + when(ReadHotSubRangeRequest req = waitNext(ssi.getReadHotRanges.getFuture())) { + if (!self->isReadable(req.keys)) { + CODE_PROBE(true, "readHotSubRanges immediate wrong_shard_server()", probe::decoration::rare); + self->sendErrorWithPenalty(req.reply, wrong_shard_server(), self->getPenalty()); + } else { + self->metrics.getReadHotRanges(req); + } + } + when(SplitRangeRequest req = waitNext(ssi.getRangeSplitPoints.getFuture())) { + if (!self->isReadable(req.keys)) { + CODE_PROBE(true, "getSplitPoints immediate wrong_shard_server()"); + self->sendErrorWithPenalty(req.reply, wrong_shard_server(), self->getPenalty()); + } else { + self->getSplitPoints(req); + } + } + when(wait(doPollMetrics)) { + self->metrics.poll(); + doPollMetrics = delay(SERVER_KNOBS->STORAGE_SERVER_POLL_METRICS_DELAY); + } + } + } +} + +#include "flow/unactorcompiler.h" +#endif // FDBSERVER_STORAGEMETRICS_H \ No newline at end of file diff --git a/fdbserver/storageserver.actor.cpp b/fdbserver/storageserver.actor.cpp index 396d4296f8..1d8fb40c8f 100644 --- a/fdbserver/storageserver.actor.cpp +++ b/fdbserver/storageserver.actor.cpp @@ -81,7 +81,7 @@ #include "fdbserver/ServerCheckpoint.actor.h" #include "fdbserver/ServerDBInfo.h" #include "fdbserver/SpanContextMessage.h" -#include "fdbserver/StorageMetrics.h" +#include "fdbserver/StorageMetrics.actor.h" #include "fdbserver/TLogInterface.h" #include "fdbserver/TransactionTagCounter.h" #include "fdbserver/WaitFailure.h" @@ -10197,7 +10197,6 @@ Future StorageServer::waitMetricsTenantAware(const WaitMetricsRequest& req } ACTOR Future metricsCore(StorageServer* self, StorageServerInterface ssi) { - state Future doPollMetrics = Void(); wait(self->byteSampleRecovery); TraceEvent("StorageServerRestoreDurableState", self->thisServerID).detail("RestoredBytes", self->bytesRestored); @@ -10230,49 +10229,8 @@ ACTOR Future metricsCore(StorageServer* self, StorageServerInterface ssi) } })); - loop { - choose { - when(state WaitMetricsRequest req = waitNext(ssi.waitMetrics.getFuture())) { - if (!req.tenantInfo.present() && !self->isReadable(req.keys)) { - CODE_PROBE(true, "waitMetrics immediate wrong_shard_server()"); - self->sendErrorWithPenalty(req.reply, wrong_shard_server(), self->getPenalty()); - } else { - self->addActor(self->waitMetricsTenantAware(req)); - } - } - when(SplitMetricsRequest req = waitNext(ssi.splitMetrics.getFuture())) { - if (!self->isReadable(req.keys)) { - CODE_PROBE(true, "splitMetrics immediate wrong_shard_server()"); - self->sendErrorWithPenalty(req.reply, wrong_shard_server(), self->getPenalty()); - } else { - self->metrics.splitMetrics(req); - } - } - when(GetStorageMetricsRequest req = waitNext(ssi.getStorageMetrics.getFuture())) { - self->getStorageMetrics(req); - } - when(ReadHotSubRangeRequest req = waitNext(ssi.getReadHotRanges.getFuture())) { - if (!self->isReadable(req.keys)) { - CODE_PROBE(true, "readHotSubRanges immediate wrong_shard_server()", probe::decoration::rare); - self->sendErrorWithPenalty(req.reply, wrong_shard_server(), self->getPenalty()); - } else { - self->metrics.getReadHotRanges(req); - } - } - when(SplitRangeRequest req = waitNext(ssi.getRangeSplitPoints.getFuture())) { - if (!self->isReadable(req.keys)) { - CODE_PROBE(true, "getSplitPoints immediate wrong_shard_server()"); - self->sendErrorWithPenalty(req.reply, wrong_shard_server(), self->getPenalty()); - } else { - self->getSplitPoints(req); - } - } - when(wait(doPollMetrics)) { - self->metrics.poll(); - doPollMetrics = delay(SERVER_KNOBS->STORAGE_SERVER_POLL_METRICS_DELAY); - } - } - } + wait(serveStorageMetricsRequests(self, ssi)); + return Void(); } ACTOR Future logLongByteSampleRecovery(Future recovery) { diff --git a/fdbserver/workloads/ConsistencyCheck.actor.cpp b/fdbserver/workloads/ConsistencyCheck.actor.cpp index 6abeb66e5f..ee9c63bb50 100644 --- a/fdbserver/workloads/ConsistencyCheck.actor.cpp +++ b/fdbserver/workloads/ConsistencyCheck.actor.cpp @@ -30,7 +30,7 @@ #include "flow/IRateControl.h" #include "fdbrpc/simulator.h" #include "fdbserver/Knobs.h" -#include "fdbserver/StorageMetrics.h" +#include "fdbserver/StorageMetrics.actor.h" #include "fdbserver/DataDistribution.actor.h" #include "fdbserver/QuietDatabase.h" #include "fdbserver/TSSMappingUtil.actor.h" From 0d4b4d05e20c320004396f5f0114f1539eb1c44b Mon Sep 17 00:00:00 2001 From: Xiaoxi Wang Date: Fri, 21 Oct 2022 16:53:03 -0700 Subject: [PATCH 18/95] implement MSS as IStorageMetricsService and pass the unit test --- fdbserver/DDShardTracker.actor.cpp | 16 +---- fdbserver/MockGlobalState.actor.cpp | 69 +++++++++++++++++++ .../fdbserver/DataDistribution.actor.h | 14 ++++ fdbserver/include/fdbserver/MockGlobalState.h | 34 ++++++++- 4 files changed, 115 insertions(+), 18 deletions(-) diff --git a/fdbserver/DDShardTracker.actor.cpp b/fdbserver/DDShardTracker.actor.cpp index be7343ba4c..7964915217 100644 --- a/fdbserver/DDShardTracker.actor.cpp +++ b/fdbserver/DDShardTracker.actor.cpp @@ -212,7 +212,7 @@ ShardSizeBounds calculateShardSizeBounds(const KeyRange& keys, const Reference>>& shardMetrics, const BandwidthStatus& bandwidthStatus, PromiseStream readHotShard) { - ShardSizeBounds bounds; + ShardSizeBounds bounds = ShardSizeBounds::shardSizeBoundsBeforeTrack(); if (shardMetrics->get().present()) { auto bytes = shardMetrics->get().get().metrics.bytes; auto readBandwidthStatus = getReadBandwidthStatus(shardMetrics->get().get().metrics); @@ -259,21 +259,7 @@ ShardSizeBounds calculateShardSizeBounds(const KeyRange& keys, } else { ASSERT(false); } - } else { - bounds.max.bytes = -1; - bounds.min.bytes = -1; - bounds.permittedError.bytes = -1; - bounds.max.bytesPerKSecond = bounds.max.infinity; - bounds.min.bytesPerKSecond = 0; - bounds.permittedError.bytesPerKSecond = bounds.permittedError.infinity; - bounds.max.bytesReadPerKSecond = bounds.max.infinity; - bounds.min.bytesReadPerKSecond = 0; - bounds.permittedError.bytesReadPerKSecond = bounds.permittedError.infinity; } - - bounds.max.iosPerKSecond = bounds.max.infinity; - bounds.min.iosPerKSecond = 0; - bounds.permittedError.iosPerKSecond = bounds.permittedError.infinity; return bounds; } diff --git a/fdbserver/MockGlobalState.actor.cpp b/fdbserver/MockGlobalState.actor.cpp index bdeed264fd..1e03b71e85 100644 --- a/fdbserver/MockGlobalState.actor.cpp +++ b/fdbserver/MockGlobalState.actor.cpp @@ -20,6 +20,7 @@ #include "fdbserver/MockGlobalState.h" #include "fdbserver/workloads/workloads.actor.h" +#include "fdbserver/DataDistribution.actor.h" #include "flow/actorcompiler.h" class MockGlobalStateImpl { @@ -42,6 +43,7 @@ public: UseProvisionalProxies::False, 0) .get(); + TraceEvent(SevDebug, "MGSWaitStorageMetrics").detail("Phase", "GetLocation"); // NOTE(xwang): in native API, there's code handling the non-equal situation, but I think in mock world // there shouldn't have any delay to update the locations. ASSERT_EQ(expectedShardCount, locations.size()); @@ -92,6 +94,28 @@ public: } }; +class MockStorageServerImpl { +public: + ACTOR static Future waitMetricsTenantAware(MockStorageServer* self, WaitMetricsRequest req) { + if (req.tenantInfo.present() && req.tenantInfo.get().tenantId != TenantInfo::INVALID_TENANT) { + // TODO(xwang) add support for tenant test, search for tenant entry + Optional entry; + Optional tenantPrefix = entry.map([](TenantMapEntry e) { return e.prefix; }); + if (tenantPrefix.present()) { + UNREACHABLE(); + // req.keys = req.keys.withPrefix(tenantPrefix.get(), req.arena); + } + } + + if (!self->isReadable(req.keys)) { + self->sendErrorWithPenalty(req.reply, wrong_shard_server(), self->getPenalty()); + } else { + wait(self->metrics.waitMetrics(req, delayJittered(SERVER_KNOBS->STORAGE_METRIC_TIMEOUT))); + } + return Void(); + } +}; + bool MockStorageServer::allShardStatusEqual(KeyRangeRef range, MockShardStatus status) { auto ranges = serverKeys.intersectingRanges(range); ASSERT(!ranges.empty()); // at least the range is allKeys @@ -203,6 +227,22 @@ uint64_t MockStorageServer::sumRangeSize(KeyRangeRef range) const { return totalSize; } +void MockStorageServer::addActor(Future future) { + actors.add(future); +} + +void MockStorageServer::getSplitPoints(const SplitRangeRequest& req) {} + +Future MockStorageServer::waitMetricsTenantAware(const WaitMetricsRequest& req) { + return MockStorageServerImpl::waitMetricsTenantAware(this, req); +} + +void MockStorageServer::getStorageMetrics(const GetStorageMetricsRequest& req) {} + +Future MockStorageServer::run() { + return serveStorageMetricsRequests(this, ssi); +} + void MockGlobalState::initializeAsEmptyDatabaseMGS(const DatabaseConfiguration& conf, uint64_t defaultDiskSpace) { ASSERT(conf.storageTeamSize > 0); configuration = conf; @@ -544,3 +584,32 @@ TEST_CASE("/MockGlobalState/MockStorageServer/GetKeyLocations") { return Void(); } + +TEST_CASE("/MockGlobalState/MockStorageServer/WaitStorageMetricsRequest") { + BasicTestConfig testConfig; + testConfig.simpleConfig = true; + testConfig.minimumReplication = 1; + testConfig.logAntiQuorum = 0; + DatabaseConfiguration dbConfig = generateNormalDatabaseConfiguration(testConfig); + TraceEvent("UnitTestDbConfig").detail("Config", dbConfig.toString()); + + state std::shared_ptr mgs = std::make_shared(); + mgs->initializeAsEmptyDatabaseMGS(dbConfig); + state ActorCollection actors; + + ActorCollection* ptr = &actors; // get around ACTOR syntax restriction + std::for_each(mgs->allServers.begin(), mgs->allServers.end(), [ptr](auto& server) { + ptr->add(server.second.run()); + server.second.metrics.byteSample.sample.insert("something"_sr, 500000); + }); + + KeyRange testRange = allKeys; + ShardSizeBounds bounds = ShardSizeBounds::shardSizeBoundsBeforeTrack(); + std::pair, int> res = + wait(mgs->waitStorageMetrics(testRange, bounds.min, bounds.max, bounds.permittedError, 1, 1)); + // std::cout << "get result " << res.second << "\n"; + // std::cout << "get byte "<< res.first.get().bytes << "\n"; + ASSERT_EQ(res.second, -1); // the valid result always return -1, strange contraction though. + ASSERT_EQ(res.first.get().bytes, 500000); + return Void(); +} diff --git a/fdbserver/include/fdbserver/DataDistribution.actor.h b/fdbserver/include/fdbserver/DataDistribution.actor.h index 2389dc0ab6..2e77d07459 100644 --- a/fdbserver/include/fdbserver/DataDistribution.actor.h +++ b/fdbserver/include/fdbserver/DataDistribution.actor.h @@ -476,6 +476,20 @@ struct ShardSizeBounds { bool operator==(ShardSizeBounds const& rhs) const { return max == rhs.max && min == rhs.min && permittedError == rhs.permittedError; } + + static ShardSizeBounds shardSizeBoundsBeforeTrack() { + return ShardSizeBounds{ + .max = StorageMetrics{ .bytes = -1, + .bytesPerKSecond = StorageMetrics::infinity, + .iosPerKSecond = StorageMetrics::infinity, + .bytesReadPerKSecond = StorageMetrics::infinity }, + .min = StorageMetrics{ .bytes = -1, .bytesPerKSecond = 0, .iosPerKSecond = 0, .bytesReadPerKSecond = 0 }, + .permittedError = StorageMetrics{ .bytes = -1, + .bytesPerKSecond = StorageMetrics::infinity, + .iosPerKSecond = StorageMetrics::infinity, + .bytesReadPerKSecond = StorageMetrics::infinity } + }; + } }; // Gets the permitted size and IO bounds for a shard diff --git a/fdbserver/include/fdbserver/MockGlobalState.h b/fdbserver/include/fdbserver/MockGlobalState.h index f3e5213892..a404f24027 100644 --- a/fdbserver/include/fdbserver/MockGlobalState.h +++ b/fdbserver/include/fdbserver/MockGlobalState.h @@ -52,9 +52,11 @@ inline bool isStatusTransitionValid(MockShardStatus from, MockShardStatus to) { return false; } -class MockStorageServer { +class MockStorageServer : public IStorageMetricsService { friend struct MockGlobalStateTester; + ActorCollection actors; + public: struct ShardInfo { MockShardStatus status; @@ -74,8 +76,6 @@ public: // size() and nthRange() would use the metrics as index instead KeyRangeMap serverKeys; - // sampled metrics - StorageServerMetrics metrics; CoalescedKeyRangeMap> byteSampleClears; StorageServerInterface ssi; // serve RPC requests @@ -104,6 +104,34 @@ public: uint64_t sumRangeSize(KeyRangeRef range) const; + void addActor(Future future) override; + + void getSplitPoints(SplitRangeRequest const& req) override; + + Future waitMetricsTenantAware(const WaitMetricsRequest& req) override; + + void getStorageMetrics(const GetStorageMetricsRequest& req) override; + + template + using isLoadBalancedReply = std::is_base_of; + + template + typename std::enable_if::value, void>::type + sendErrorWithPenalty(const ReplyPromise& promise, const Error& err, double penalty) { + Reply reply; + reply.error = err; + reply.penalty = penalty; + promise.send(reply); + } + + template + typename std::enable_if::value, void>::type + sendErrorWithPenalty(const ReplyPromise& promise, const Error& err, double) { + promise.sendError(err); + } + + Future run(); + protected: void threeWayShardSplitting(KeyRangeRef outerRange, KeyRangeRef innerRange, From 918018d4928f1563d5c7ed80123485068fd67013 Mon Sep 17 00:00:00 2001 From: Xiaoxi Wang Date: Mon, 24 Oct 2022 10:50:46 -0700 Subject: [PATCH 19/95] format code --- fdbserver/include/fdbserver/StorageMetrics.actor.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/fdbserver/include/fdbserver/StorageMetrics.actor.h b/fdbserver/include/fdbserver/StorageMetrics.actor.h index 7111227f27..552db2c6f7 100644 --- a/fdbserver/include/fdbserver/StorageMetrics.actor.h +++ b/fdbserver/include/fdbserver/StorageMetrics.actor.h @@ -181,7 +181,7 @@ public: }; ACTOR template -Future serveStorageMetricsRequests(ServiceType* self, StorageServerInterface ssi){ +Future serveStorageMetricsRequests(ServiceType* self, StorageServerInterface ssi) { state Future doPollMetrics = Void(); loop { choose { From dd52d997ba33de0327702a6ff94b63882c775482 Mon Sep 17 00:00:00 2001 From: Dennis Zhou Date: Mon, 24 Oct 2022 11:07:36 -0700 Subject: [PATCH 20/95] blob: rename rowLimit variable for listBlobbifiedRanges() --- fdbclient/NativeAPI.actor.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/fdbclient/NativeAPI.actor.cpp b/fdbclient/NativeAPI.actor.cpp index bf5483f82c..0f33aefb95 100644 --- a/fdbclient/NativeAPI.actor.cpp +++ b/fdbclient/NativeAPI.actor.cpp @@ -10802,9 +10802,9 @@ ACTOR Future>> listBlobbifiedRangesActor(Refer } Future>> DatabaseContext::listBlobbifiedRanges(KeyRange range, - int rowLimit, + int rangeLimit, Optional tenantName) { - return listBlobbifiedRangesActor(Reference::addRef(this), range, rowLimit, tenantName); + return listBlobbifiedRangesActor(Reference::addRef(this), range, rangeLimit, tenantName); } int64_t getMaxKeySize(KeyRef const& key) { From 07f12478853b6a6b987c938a35c25ab6ba18b510 Mon Sep 17 00:00:00 2001 From: Dennis Zhou Date: Tue, 18 Oct 2022 13:24:27 -0700 Subject: [PATCH 21/95] blob: getBlobRanges() use transaction instead of RYWTransaction --- fdbclient/NativeAPI.actor.cpp | 14 ++++++-------- 1 file changed, 6 insertions(+), 8 deletions(-) diff --git a/fdbclient/NativeAPI.actor.cpp b/fdbclient/NativeAPI.actor.cpp index 0f33aefb95..7e59f4324d 100644 --- a/fdbclient/NativeAPI.actor.cpp +++ b/fdbclient/NativeAPI.actor.cpp @@ -10662,9 +10662,7 @@ Future DatabaseContext::waitPurgeGranulesComplete(Key purgeKey) { return waitPurgeGranulesCompleteActor(Reference::addRef(this), purgeKey); } -ACTOR Future>> getBlobRanges(Reference tr, - KeyRange range, - int batchLimit) { +ACTOR Future>> getBlobRanges(Transaction* tr, KeyRange range, int batchLimit) { state Standalone> blobRanges; state Key beginKey = range.begin; @@ -10716,7 +10714,7 @@ ACTOR Future setBlobRangeActor(Reference cx, range = range.withPrefix(tenantEntry.prefix); } - Standalone> startBlobRanges = wait(getBlobRanges(tr, range, 1)); + Standalone> startBlobRanges = wait(getBlobRanges(&tr->getTransaction(), range, 1)); if (active) { // Idempotent request. @@ -10765,22 +10763,22 @@ ACTOR Future>> listBlobbifiedRangesActor(Refer int rangeLimit, Optional tenantName) { state Database db(cx); - state Reference tr = makeReference(db); + state Transaction tr(db); state TenantMapEntry tme; loop { try { if (tenantName.present()) { - wait(store(tme, blobGranuleGetTenantEntry(&tr->getTransaction(), range.begin, tenantName))); + wait(store(tme, blobGranuleGetTenantEntry(&tr, range.begin, tenantName))); range = range.withPrefix(tme.prefix); } break; } catch (Error& e) { - wait(tr->onError(e)); + wait(tr.onError(e)); } } - state Standalone> blobRanges = wait(getBlobRanges(tr, range, rangeLimit)); + state Standalone> blobRanges = wait(getBlobRanges(&tr, range, rangeLimit)); if (!tenantName.present()) { return blobRanges; } From 48d6e725c2ddbb99053c6cd818901611e7319f67 Mon Sep 17 00:00:00 2001 From: Dennis Zhou Date: Tue, 18 Oct 2022 14:11:07 -0700 Subject: [PATCH 22/95] blob: convert listBlobbifiedRangesActor() to take a Transaction --- fdbclient/NativeAPI.actor.cpp | 135 ++++++++++++++++++---------------- 1 file changed, 73 insertions(+), 62 deletions(-) diff --git a/fdbclient/NativeAPI.actor.cpp b/fdbclient/NativeAPI.actor.cpp index 7e59f4324d..1c0a224d18 100644 --- a/fdbclient/NativeAPI.actor.cpp +++ b/fdbclient/NativeAPI.actor.cpp @@ -10540,6 +10540,76 @@ Reference DatabaseContext::createTransaction() { } // BlobGranule API. +ACTOR Future>> getBlobRanges(Transaction* tr, KeyRange range, int batchLimit) { + state Standalone> blobRanges; + state Key beginKey = range.begin; + + loop { + try { + tr->setOption(FDBTransactionOptions::ACCESS_SYSTEM_KEYS); + + state RangeResult results = wait( + krmGetRangesUnaligned(tr, blobRangeKeys.begin, KeyRangeRef(beginKey, range.end), 2 * batchLimit + 2)); + + blobRanges.arena().dependsOn(results.arena()); + for (int i = 0; i < results.size() - 1; i++) { + if (results[i].value == blobRangeActive) { + blobRanges.push_back(blobRanges.arena(), KeyRangeRef(results[i].key, results[i + 1].key)); + } + if (blobRanges.size() == batchLimit) { + return blobRanges; + } + } + + if (!results.more) { + return blobRanges; + } + beginKey = results.back().key; + } catch (Error& e) { + wait(tr->onError(e)); + } + } +} + +ACTOR Future>> getBlobbifiedRanges(Transaction* tr, + KeyRange range, + int rangeLimit, + Optional tenantName) { + state TenantMapEntry tme; + + loop { + try { + if (tenantName.present()) { + wait(store(tme, blobGranuleGetTenantEntry(tr, range.begin, tenantName))); + range = range.withPrefix(tme.prefix); + } + break; + } catch (Error& e) { + wait(tr->onError(e)); + } + } + + state Standalone> blobRanges = wait(getBlobRanges(tr, range, rangeLimit)); + if (!tenantName.present()) { + return blobRanges; + } + + // Strip tenant prefix out. + state Standalone> tenantBlobRanges; + for (auto& blobRange : blobRanges) { + // Filter out blob ranges that span tenants for some reason. + if (!blobRange.begin.startsWith(tme.prefix) || !blobRange.end.startsWith(tme.prefix)) { + TraceEvent("ListBlobbifiedRangeSpansTenants") + .suppressFor(/*seconds=*/5) + .detail("Tenant", tenantName.get()) + .detail("Range", blobRange); + continue; + } + tenantBlobRanges.push_back_deep(tenantBlobRanges.arena(), blobRange.removePrefix(tme.prefix)); + } + return tenantBlobRanges; +} + ACTOR Future purgeBlobGranulesActor(Reference db, KeyRange range, Version purgeVersion, @@ -10662,37 +10732,6 @@ Future DatabaseContext::waitPurgeGranulesComplete(Key purgeKey) { return waitPurgeGranulesCompleteActor(Reference::addRef(this), purgeKey); } -ACTOR Future>> getBlobRanges(Transaction* tr, KeyRange range, int batchLimit) { - state Standalone> blobRanges; - state Key beginKey = range.begin; - - loop { - try { - tr->setOption(FDBTransactionOptions::ACCESS_SYSTEM_KEYS); - - state RangeResult results = wait( - krmGetRangesUnaligned(tr, blobRangeKeys.begin, KeyRangeRef(beginKey, range.end), 2 * batchLimit + 2)); - - blobRanges.arena().dependsOn(results.arena()); - for (int i = 0; i < results.size() - 1; i++) { - if (results[i].value == blobRangeActive) { - blobRanges.push_back(blobRanges.arena(), KeyRangeRef(results[i].key, results[i + 1].key)); - } - if (blobRanges.size() == batchLimit) { - return blobRanges; - } - } - - if (!results.more) { - return blobRanges; - } - beginKey = results.back().key; - } catch (Error& e) { - wait(tr->onError(e)); - } - } -} - ACTOR Future setBlobRangeActor(Reference cx, KeyRange range, bool active, @@ -10762,41 +10801,13 @@ ACTOR Future>> listBlobbifiedRangesActor(Refer KeyRange range, int rangeLimit, Optional tenantName) { + state Database db(cx); state Transaction tr(db); - state TenantMapEntry tme; - loop { - try { - if (tenantName.present()) { - wait(store(tme, blobGranuleGetTenantEntry(&tr, range.begin, tenantName))); - range = range.withPrefix(tme.prefix); - } - break; - } catch (Error& e) { - wait(tr.onError(e)); - } - } + Standalone> blobbifiedRanges = wait(getBlobbifiedRanges(&tr, range, rangeLimit, tenantName)); - state Standalone> blobRanges = wait(getBlobRanges(&tr, range, rangeLimit)); - if (!tenantName.present()) { - return blobRanges; - } - - // Strip tenant prefix out. - state Standalone> tenantBlobRanges; - for (auto& blobRange : blobRanges) { - // Filter out blob ranges that span tenants for some reason. - if (!blobRange.begin.startsWith(tme.prefix) || !blobRange.end.startsWith(tme.prefix)) { - TraceEvent("ListBlobbifiedRangeSpansTenants") - .suppressFor(/*seconds=*/5) - .detail("Tenant", tenantName.get()) - .detail("Range", blobRange); - continue; - } - tenantBlobRanges.push_back_deep(tenantBlobRanges.arena(), blobRange.removePrefix(tme.prefix)); - } - return tenantBlobRanges; + return blobbifiedRanges; } Future>> DatabaseContext::listBlobbifiedRanges(KeyRange range, From 474955b10d11644232d05ed151e18ce95ec51610 Mon Sep 17 00:00:00 2001 From: Dennis Zhou Date: Tue, 18 Oct 2022 14:26:36 -0700 Subject: [PATCH 23/95] blob: allow purge as long as it doesn't start in a range --- fdbclient/NativeAPI.actor.cpp | 11 +++++++---- 1 file changed, 7 insertions(+), 4 deletions(-) diff --git a/fdbclient/NativeAPI.actor.cpp b/fdbclient/NativeAPI.actor.cpp index 1c0a224d18..e44f677ee0 100644 --- a/fdbclient/NativeAPI.actor.cpp +++ b/fdbclient/NativeAPI.actor.cpp @@ -10652,10 +10652,13 @@ ACTOR Future purgeBlobGranulesActor(Reference db, } // must be aligned to blob range(s) - state Future> beginPresent = tr.get(purgeRange.begin.withPrefix(blobRangeKeys.begin)); - state Future> endPresent = tr.get(purgeRange.end.withPrefix(blobRangeKeys.begin)); - wait(success(beginPresent) && success(endPresent)); - if (!beginPresent.get().present() || !endPresent.get().present()) { + state Future>> blobbifiedBegin = + getBlobbifiedRanges(&tr, KeyRangeRef(purgeRange.begin, purgeRange.begin), 2, {}); + state Future>> blobbifiedEnd = + getBlobbifiedRanges(&tr, KeyRangeRef(purgeRange.end, purgeRange.end), 2, {}); + wait(success(blobbifiedBegin) && success(blobbifiedEnd)); + if ((!blobbifiedBegin.get().empty() && blobbifiedBegin.get().front().begin < purgeRange.begin) || + (!blobbifiedEnd.get().empty() && blobbifiedEnd.get().back().end > purgeRange.end)) { TraceEvent("UnalignedPurge") .detail("Range", range) .detail("Version", purgeVersion) From 070e4c133e96104e2c5111bc5a8ac6c49d601fea Mon Sep 17 00:00:00 2001 From: Dennis Zhou Date: Thu, 20 Oct 2022 23:01:38 -0700 Subject: [PATCH 24/95] blob/testing: remove setRange() and call (un)blobbifyRange() directly This also fixes a few wrong setRange(true/false). --- .../BlobGranuleRangesWorkload.actor.cpp | 74 ++++++++----------- 1 file changed, 30 insertions(+), 44 deletions(-) diff --git a/fdbserver/workloads/BlobGranuleRangesWorkload.actor.cpp b/fdbserver/workloads/BlobGranuleRangesWorkload.actor.cpp index 8ac26ccf61..e94a134ae7 100644 --- a/fdbserver/workloads/BlobGranuleRangesWorkload.actor.cpp +++ b/fdbserver/workloads/BlobGranuleRangesWorkload.actor.cpp @@ -105,16 +105,6 @@ struct BlobGranuleRangesWorkload : TestWorkload { } } - ACTOR Future setRange(Database cx, KeyRange range, bool active, Optional tenantName) { - if (active) { - bool success = wait(cx->blobbifyRange(range, tenantName)); - return success; - } else { - bool success = wait(cx->unblobbifyRange(range, tenantName)); - return success; - } - } - ACTOR Future registerNewRange(Database cx, BlobGranuleRangesWorkload* self, Optional tenantName) { std::string nextRangeKey = "R_" + self->newKey(); state KeyRange range(KeyRangeRef(StringRef(nextRangeKey), strinc(StringRef(nextRangeKey)))); @@ -124,8 +114,7 @@ struct BlobGranuleRangesWorkload : TestWorkload { // don't put in active ranges until AFTER set range command succeeds, to avoid checking a range that maybe // wasn't initialized - bool success = - wait(self->setRange(cx, range, true, tenantName.present() ? tenantName.get() : self->tenantName)); + bool success = wait(cx->blobbifyRange(range, tenantName.present() ? tenantName.get() : self->tenantName)); ASSERT(success); if (BGRW_DEBUG) { @@ -163,7 +152,7 @@ struct BlobGranuleRangesWorkload : TestWorkload { Key purgeKey = wait(self->versionedForcePurge(cx, range, self->tenantName)); wait(cx->waitPurgeGranulesComplete(purgeKey)); } - bool success = wait(self->setRange(cx, range, false, self->tenantName)); + bool success = wait(cx->unblobbifyRange(range, self->tenantName)); ASSERT(success); if (BGRW_DEBUG) { @@ -356,7 +345,7 @@ struct BlobGranuleRangesWorkload : TestWorkload { // tear down range at end Key purgeKey = wait(self->versionedForcePurge(cx, range, self->tenantName)); wait(cx->waitPurgeGranulesComplete(purgeKey)); - bool success = wait(self->setRange(cx, range, false, self->tenantName)); + bool success = wait(cx->unblobbifyRange(range, self->tenantName)); ASSERT(success); if (BGRW_DEBUG) { @@ -373,7 +362,7 @@ struct BlobGranuleRangesWorkload : TestWorkload { if (BGRW_DEBUG) { fmt::print("VerifyRangeUnit: [{0} - {1})\n", range.begin.printable(), range.end.printable()); } - bool setSuccess = wait(self->setRange(cx, activeRange, true, self->tenantName)); + bool setSuccess = wait(cx->blobbifyRange(activeRange, self->tenantName)); ASSERT(setSuccess); wait(self->checkRange(cx, self, activeRange, true)); @@ -426,7 +415,7 @@ struct BlobGranuleRangesWorkload : TestWorkload { for (i = 0; i < rangeCount; i++) { state KeyRange subRange(KeyRangeRef(boundaries[i], boundaries[i + 1])); if (i != rangeToNotBlobbify) { - bool setSuccess = wait(self->setRange(cx, subRange, true, self->tenantName)); + bool setSuccess = wait(cx->blobbifyRange(subRange, self->tenantName)); ASSERT(setSuccess); wait(self->checkRange(cx, self, subRange, true)); } else { @@ -473,7 +462,7 @@ struct BlobGranuleRangesWorkload : TestWorkload { } ACTOR Future rangesMisalignedUnit(Database cx, BlobGranuleRangesWorkload* self, KeyRange range) { - bool setSuccess = wait(self->setRange(cx, range, true, self->tenantName)); + bool setSuccess = wait(cx->blobbifyRange(range, self->tenantName)); ASSERT(setSuccess); state KeyRange subRange(KeyRangeRef(range.begin.withSuffix("A"_sr), range.begin.withSuffix("B"_sr))); @@ -526,42 +515,42 @@ struct BlobGranuleRangesWorkload : TestWorkload { // unblobbifying range that already doesn't exist should be no-op if (deterministicRandom()->coinflip()) { - bool unblobbifyStartSuccess = wait(self->setRange(cx, activeRange, false, self->tenantName)); + bool unblobbifyStartSuccess = wait(cx->blobbifyRange(activeRange, self->tenantName)); ASSERT(unblobbifyStartSuccess); } - bool success = wait(self->setRange(cx, activeRange, true, self->tenantName)); + bool success = wait(cx->blobbifyRange(activeRange, self->tenantName)); ASSERT(success); wait(self->checkRange(cx, self, activeRange, true)); // check that re-blobbifying same range is successful - bool retrySuccess = wait(self->setRange(cx, activeRange, true, self->tenantName)); + bool retrySuccess = wait(cx->blobbifyRange(activeRange, self->tenantName)); ASSERT(retrySuccess); wait(self->checkRange(cx, self, activeRange, true)); // check that blobbifying range that overlaps but does not match existing blob range fails - bool fail1 = wait(self->setRange(cx, range, true, self->tenantName)); + bool fail1 = wait(cx->blobbifyRange(range, self->tenantName)); ASSERT(!fail1); - bool fail2 = wait(self->setRange(cx, KeyRangeRef(range.begin, activeRange.end), true, self->tenantName)); + bool fail2 = wait(cx->blobbifyRange(KeyRangeRef(range.begin, activeRange.end), self->tenantName)); ASSERT(!fail2); - bool fail3 = wait(self->setRange(cx, KeyRangeRef(activeRange.begin, range.end), true, self->tenantName)); + bool fail3 = wait(cx->blobbifyRange(KeyRangeRef(activeRange.begin, range.end), self->tenantName)); ASSERT(!fail3); - bool fail4 = wait(self->setRange(cx, KeyRangeRef(range.begin, middleKey), true, self->tenantName)); + bool fail4 = wait(cx->blobbifyRange(KeyRangeRef(range.begin, middleKey), self->tenantName)); ASSERT(!fail4); - bool fail5 = wait(self->setRange(cx, KeyRangeRef(middleKey, range.end), true, self->tenantName)); + bool fail5 = wait(cx->blobbifyRange(KeyRangeRef(middleKey, range.end), self->tenantName)); ASSERT(!fail5); - bool fail6 = wait(self->setRange(cx, KeyRangeRef(activeRange.begin, middleKey), true, self->tenantName)); + bool fail6 = wait(cx->blobbifyRange(KeyRangeRef(activeRange.begin, middleKey), self->tenantName)); ASSERT(!fail6); - bool fail7 = wait(self->setRange(cx, KeyRangeRef(middleKey, activeRange.end), true, self->tenantName)); + bool fail7 = wait(cx->blobbifyRange(KeyRangeRef(middleKey, activeRange.end), self->tenantName)); ASSERT(!fail7); - bool fail8 = wait(self->setRange(cx, KeyRangeRef(middleKey, middleKey2), true, self->tenantName)); + bool fail8 = wait(cx->blobbifyRange(KeyRangeRef(middleKey, middleKey2), self->tenantName)); ASSERT(!fail8); { @@ -600,41 +589,38 @@ struct BlobGranuleRangesWorkload : TestWorkload { ASSERT(blobRanges.size() == 1); ASSERT(blobRanges[0] == activeRange); - bool unblobbifyFail1 = wait(self->setRange(cx, range, false, self->tenantName)); + bool unblobbifyFail1 = wait(cx->unblobbifyRange(range, self->tenantName)); ASSERT(!unblobbifyFail1); bool unblobbifyFail2 = - wait(self->setRange(cx, KeyRangeRef(range.begin, activeRange.end), false, self->tenantName)); + wait(cx->unblobbifyRange(KeyRangeRef(range.begin, activeRange.end), self->tenantName)); ASSERT(!unblobbifyFail2); bool unblobbifyFail3 = - wait(self->setRange(cx, KeyRangeRef(activeRange.begin, range.end), false, self->tenantName)); + wait(cx->unblobbifyRange(KeyRangeRef(activeRange.begin, range.end), self->tenantName)); ASSERT(!unblobbifyFail3); bool unblobbifyFail4 = - wait(self->setRange(cx, KeyRangeRef(activeRange.begin, middleKey), false, self->tenantName)); + wait(cx->unblobbifyRange(KeyRangeRef(activeRange.begin, middleKey), self->tenantName)); ASSERT(!unblobbifyFail4); - bool unblobbifyFail5 = - wait(self->setRange(cx, KeyRangeRef(middleKey, activeRange.end), false, self->tenantName)); + bool unblobbifyFail5 = wait(cx->unblobbifyRange(KeyRangeRef(middleKey, activeRange.end), self->tenantName)); ASSERT(!unblobbifyFail5); bool unblobbifyFail6 = - wait(self->setRange(cx, KeyRangeRef(activeRange.begin, middleKey), false, self->tenantName)); + wait(cx->unblobbifyRange(KeyRangeRef(activeRange.begin, middleKey), self->tenantName)); ASSERT(!unblobbifyFail6); - bool unblobbifyFail7 = - wait(self->setRange(cx, KeyRangeRef(middleKey, activeRange.end), false, self->tenantName)); + bool unblobbifyFail7 = wait(cx->unblobbifyRange(KeyRangeRef(middleKey, activeRange.end), self->tenantName)); ASSERT(!unblobbifyFail7); - bool unblobbifyFail8 = - wait(self->setRange(cx, KeyRangeRef(middleKey, middleKey2), false, self->tenantName)); + bool unblobbifyFail8 = wait(cx->unblobbifyRange(KeyRangeRef(middleKey, middleKey2), self->tenantName)); ASSERT(!unblobbifyFail8); - bool unblobbifySuccess = wait(self->setRange(cx, activeRange, true, self->tenantName)); + bool unblobbifySuccess = wait(cx->unblobbifyRange(activeRange, self->tenantName)); ASSERT(unblobbifySuccess); - bool unblobbifySuccessAgain = wait(self->setRange(cx, activeRange, true, self->tenantName)); + bool unblobbifySuccessAgain = wait(cx->unblobbifyRange(activeRange, self->tenantName)); ASSERT(unblobbifySuccessAgain); } @@ -642,7 +628,7 @@ struct BlobGranuleRangesWorkload : TestWorkload { } ACTOR Future reBlobbifyUnit(Database cx, BlobGranuleRangesWorkload* self, KeyRange range) { - bool setSuccess = wait(self->setRange(cx, range, true, self->tenantName)); + bool setSuccess = wait(cx->blobbifyRange(range, self->tenantName)); ASSERT(setSuccess); wait(self->checkRange(cx, self, range, true)); @@ -651,11 +637,11 @@ struct BlobGranuleRangesWorkload : TestWorkload { wait(cx->waitPurgeGranulesComplete(purgeKey)); wait(self->checkRange(cx, self, range, false)); - bool unsetSuccess = wait(self->setRange(cx, range, false, self->tenantName)); + bool unsetSuccess = wait(cx->unblobbifyRange(range, self->tenantName)); ASSERT(unsetSuccess); wait(self->checkRange(cx, self, range, false)); - bool reSetSuccess = wait(self->setRange(cx, range, true, self->tenantName)); + bool reSetSuccess = wait(cx->blobbifyRange(range, self->tenantName)); ASSERT(reSetSuccess); wait(self->checkRange(cx, self, range, true)); From 136a325fdc6c1463a8f075759e2a970eea011c10 Mon Sep 17 00:00:00 2001 From: Dennis Zhou Date: Thu, 20 Oct 2022 23:07:21 -0700 Subject: [PATCH 25/95] blob/testing: randomly purge the whole range instead of just active --- fdbserver/workloads/BlobGranuleRangesWorkload.actor.cpp | 7 ++++--- 1 file changed, 4 insertions(+), 3 deletions(-) diff --git a/fdbserver/workloads/BlobGranuleRangesWorkload.actor.cpp b/fdbserver/workloads/BlobGranuleRangesWorkload.actor.cpp index e94a134ae7..7bb58b77b2 100644 --- a/fdbserver/workloads/BlobGranuleRangesWorkload.actor.cpp +++ b/fdbserver/workloads/BlobGranuleRangesWorkload.actor.cpp @@ -571,13 +571,14 @@ struct BlobGranuleRangesWorkload : TestWorkload { } } - // tear down + check that un-blobbifying at a non-aligned range also doesn't work - Key purgeKey = wait(self->versionedForcePurge(cx, activeRange, self->tenantName)); + state Version purgeVersion = deterministicRandom()->coinflip() ? latestVersion : 1; + state KeyRangeRef purgeRange = deterministicRandom()->coinflip() ? activeRange : range; + Key purgeKey = wait(cx->purgeBlobGranules(purgeRange, purgeVersion, self->tenantName, true)); wait(cx->waitPurgeGranulesComplete(purgeKey)); if (deterministicRandom()->coinflip()) { // force purge again and ensure it is idempotent - Key purgeKeyAgain = wait(cx->purgeBlobGranules(activeRange, 1, self->tenantName, true)); + Key purgeKeyAgain = wait(cx->purgeBlobGranules(purgeRange, purgeVersion, self->tenantName, true)); wait(cx->waitPurgeGranulesComplete(purgeKeyAgain)); } } From db72a29c06c37163018326ffc42fccbcae5726c1 Mon Sep 17 00:00:00 2001 From: Xiaoxi Wang Date: Mon, 24 Oct 2022 11:16:23 -0700 Subject: [PATCH 26/95] fix compile error after rebase --- fdbserver/include/fdbserver/IKeyValueStore.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/fdbserver/include/fdbserver/IKeyValueStore.h b/fdbserver/include/fdbserver/IKeyValueStore.h index 3069527d7a..22fb03ef73 100644 --- a/fdbserver/include/fdbserver/IKeyValueStore.h +++ b/fdbserver/include/fdbserver/IKeyValueStore.h @@ -29,7 +29,7 @@ #include "fdbserver/IClosable.h" #include "fdbserver/IPageEncryptionKeyProvider.actor.h" #include "fdbserver/ServerDBInfo.h" -#include "fdbserver/StorageMetrics.h" +#include "fdbserver/StorageMetrics.actor.h" struct CheckpointRequest { const Version version; // The FDB version at which the checkpoint is created. From e32affc9f09a0308f5c2dc71be3b56b1fe163d38 Mon Sep 17 00:00:00 2001 From: Junhyun Shim Date: Tue, 25 Oct 2022 15:53:56 +0200 Subject: [PATCH 27/95] Add TLS and token file option to mako --- bindings/c/test/mako/mako.cpp | 245 ++++++++++++++++++++++------------ bindings/c/test/mako/mako.hpp | 14 +- 2 files changed, 170 insertions(+), 89 deletions(-) diff --git a/bindings/c/test/mako/mako.cpp b/bindings/c/test/mako/mako.cpp index f9f3827a70..4303e5343a 100644 --- a/bindings/c/test/mako/mako.cpp +++ b/bindings/c/test/mako/mako.cpp @@ -59,6 +59,8 @@ #include "shm.hpp" #include "stats.hpp" #include "time.hpp" +#include "rapidjson/document.h" +#include "rapidjson/error/en.h" namespace mako { @@ -88,14 +90,29 @@ Transaction createNewTransaction(Database db, Arguments const& args, int id = -1 } // Create Tenant Transaction int tenant_id = (id == -1) ? urand(0, args.active_tenants - 1) : id; + Transaction tr; + std::string tenantStr; // If provided tenants array, use it if (tenants) { - return tenants[tenant_id].createTransaction(); + tr = tenants[tenant_id].createTransaction(); + } else { + tenantStr = "tenant" + std::to_string(tenant_id); + BytesRef tenant_name = toBytesRef(tenantStr); + Tenant t = db.openTenant(tenant_name); + tr = t.createTransaction(); } - std::string tenantStr = "tenant" + std::to_string(tenant_id); - BytesRef tenant_name = toBytesRef(tenantStr); - Tenant t = db.openTenant(tenant_name); - return t.createTransaction(); + if (!args.authorization_tokens.empty()) { + // lookup token based on tenant name and, if found, set authz token to transaction + if (tenantStr.empty()) + tenantStr = "tenant" + std::to_string(tenant_id); + auto tokenMapItr = args.authorization_tokens.find(tenantStr); + if (tokenMapItr != args.authorization_tokens.end()) { + tr.setOption(FDB_TR_OPTION_AUTHORIZATION_TOKEN, tokenMapItr->second); + } else { + logr.warn("Authorization token map is not empty, but could not find token for tenant '{}'", tenantStr); + } + } + return tr; } uint64_t byteswapHelper(uint64_t input) { @@ -815,6 +832,18 @@ int workerProcessMain(Arguments const& args, int worker_id, shared_memory::Acces logr.error("network::setOption(FDB_NET_OPTION_DISTRIBUTED_CLIENT_TRACER): {}", err.what()); } + if (args.tls_certificate_file.has_value()) { + network::setOption(FDB_NET_OPTION_TLS_CERT_PATH, args.tls_certificate_file.value()); + } + + if (args.tls_key_file.has_value()) { + network::setOption(FDB_NET_OPTION_TLS_KEY_PATH, args.tls_key_file.value()); + } + + if (args.tls_ca_file.has_value()) { + network::setOption(FDB_NET_OPTION_TLS_CA_PATH, args.tls_ca_file.value()); + } + /* enable flatbuffers if specified */ if (args.flatbuffers) { #ifdef FDB_NET_OPTION_USE_FLATBUFFERS @@ -982,57 +1011,55 @@ int workerProcessMain(Arguments const& args, int worker_id, shared_memory::Acces } /* initialize the parameters with default values */ -int initArguments(Arguments& args) { - memset(&args, 0, sizeof(Arguments)); /* zero-out everything */ - args.num_fdb_clusters = 0; - args.num_databases = 1; - args.api_version = maxApiVersion(); - args.json = 0; - args.num_processes = 1; - args.num_threads = 1; - args.async_xacts = 0; - args.mode = MODE_INVALID; - args.rows = 100000; - args.load_factor = 1.0; - args.row_digits = digits(args.rows); - args.seconds = 30; - args.iteration = 0; - args.tpsmax = 0; - args.tpsmin = -1; - args.tpsinterval = 10; - args.tpschange = TPS_SIN; - args.sampling = 1000; - args.key_length = 32; - args.value_length = 16; - args.active_tenants = 0; - args.total_tenants = 0; - args.tenant_batch_size = 10000; - args.zipf = 0; - args.commit_get = 0; - args.verbose = 1; - args.flatbuffers = 0; /* internal */ - args.knobs[0] = '\0'; - args.log_group[0] = '\0'; - args.prefixpadding = 0; - args.trace = 0; - args.tracepath[0] = '\0'; - args.traceformat = 0; /* default to client's default (XML) */ - args.streaming_mode = FDB_STREAMING_MODE_WANT_ALL; - args.txntrace = 0; - args.txntagging = 0; - memset(args.txntagging_prefix, 0, TAGPREFIXLENGTH_MAX); +Arguments::Arguments() { + num_fdb_clusters = 0; + num_databases = 1; + api_version = maxApiVersion(); + json = 0; + num_processes = 1; + num_threads = 1; + async_xacts = 0; + mode = MODE_INVALID; + rows = 100000; + load_factor = 1.0; + row_digits = digits(rows); + seconds = 30; + iteration = 0; + tpsmax = 0; + tpsmin = -1; + tpsinterval = 10; + tpschange = TPS_SIN; + sampling = 1000; + key_length = 32; + value_length = 16; + active_tenants = 0; + total_tenants = 0; + tenant_batch_size = 10000; + zipf = 0; + commit_get = 0; + verbose = 1; + flatbuffers = 0; /* internal */ + knobs[0] = '\0'; + log_group[0] = '\0'; + prefixpadding = 0; + trace = 0; + tracepath[0] = '\0'; + traceformat = 0; /* default to client's default (XML) */ + streaming_mode = FDB_STREAMING_MODE_WANT_ALL; + txntrace = 0; + txntagging = 0; + memset(txntagging_prefix, 0, TAGPREFIXLENGTH_MAX); for (auto i = 0; i < MAX_OP; i++) { - args.txnspec.ops[i][OP_COUNT] = 0; + txnspec.ops[i][OP_COUNT] = 0; } - args.client_threads_per_version = 0; - args.disable_client_bypass = false; - args.disable_ryw = 0; - args.json_output_path[0] = '\0'; - args.stats_export_path[0] = '\0'; - args.bg_materialize_files = false; - args.bg_file_path[0] = '\0'; - args.distributed_tracer_client = 0; - return 0; + client_threads_per_version = 0; + disable_client_bypass = false; + disable_ryw = 0; + json_output_path[0] = '\0'; + stats_export_path[0] = '\0'; + bg_materialize_files = false; + bg_file_path[0] = '\0'; + distributed_tracer_client = 0; } /* parse transaction specification */ @@ -1279,6 +1306,10 @@ int parseArguments(int argc, char* argv[], Arguments& args) { { "bg_file_path", required_argument, NULL, ARG_BG_FILE_PATH }, { "stats_export_path", optional_argument, NULL, ARG_EXPORT_PATH }, { "distributed_tracer_client", required_argument, NULL, ARG_DISTRIBUTED_TRACER_CLIENT }, + { "tls_certificate_file", required_argument, NULL, ARG_TLS_CERTIFICATE_FILE }, + { "tls_key_file", required_argument, NULL, ARG_TLS_KEY_FILE }, + { "tls_ca_file", required_argument, NULL, ARG_TLS_CA_FILE }, + { "authorization_token_file", required_argument, NULL, ARG_AUTHORIZATION_TOKEN_FILE }, { NULL, 0, NULL, 0 } }; idx = 0; @@ -1515,6 +1546,45 @@ int parseArguments(int argc, char* argv[], Arguments& args) { args.distributed_tracer_client = -1; } break; + case ARG_TLS_CERTIFICATE_FILE: + args.tls_certificate_file = std::string(optarg); + break; + case ARG_TLS_KEY_FILE: + args.tls_key_file = std::string(optarg); + break; + case ARG_TLS_CA_FILE: + args.tls_ca_file = std::string(optarg); + break; + case ARG_AUTHORIZATION_TOKEN_FILE: { + std::string tokenFilename(optarg); + std::ifstream ifs(tokenFilename); + std::ostringstream oss; + oss << ifs.rdbuf(); + rapidjson::Document d; + d.Parse(oss.str().c_str()); + if (d.HasParseError()) { + logr.error("Failed to parse authorization token JSON file '{}': {} at offset {}", + tokenFilename, + GetParseError_En(d.GetParseError()), + d.GetErrorOffset()); + return -1; + } else if (!d.IsObject()) { + logr.error("Authorization token JSON file '{}' must contain a JSON object", tokenFilename); + return -1; + } + for (auto itr = d.MemberBegin(); itr != d.MemberEnd(); ++itr) { + if (!itr->value.IsString()) { + logr.error("Token '{}' is not a string", itr->name.GetString()); + return -1; + } + args.authorization_tokens.insert_or_assign( + std::string(itr->name.GetString(), itr->name.GetStringLength()), + std::string(itr->value.GetString(), itr->value.GetStringLength())); + } + logr.info("Added {} tenant authorization tokens to map from file '{}'", + args.authorization_tokens.size(), + tokenFilename); + } break; } } @@ -1525,93 +1595,97 @@ int parseArguments(int argc, char* argv[], Arguments& args) { return 0; } -int validateArguments(Arguments const& args) { - if (args.mode == MODE_INVALID) { +int Arguments::validate() { + if (mode == MODE_INVALID) { logr.error("--mode has to be set"); return -1; } - if (args.verbose < VERBOSE_NONE || args.verbose > VERBOSE_DEBUG) { + if (verbose < VERBOSE_NONE || verbose > VERBOSE_DEBUG) { logr.error("--verbose must be between 0 and 3"); return -1; } - if (args.rows <= 0) { + if (rows <= 0) { logr.error("--rows must be a positive integer"); return -1; } - if (args.load_factor <= 0 || args.load_factor > 1) { + if (load_factor <= 0 || load_factor > 1) { logr.error("--load_factor must be in range (0, 1]"); return -1; } - if (args.key_length < 0) { + if (key_length < 0) { logr.error("--keylen must be a positive integer"); return -1; } - if (args.value_length < 0) { + if (value_length < 0) { logr.error("--vallen must be a positive integer"); return -1; } - if (args.num_fdb_clusters > NUM_CLUSTERS_MAX) { + if (num_fdb_clusters > NUM_CLUSTERS_MAX) { logr.error("Mako is not supported to do work to more than {} clusters", NUM_CLUSTERS_MAX); return -1; } - if (args.num_databases > NUM_DATABASES_MAX) { + if (num_databases > NUM_DATABASES_MAX) { logr.error("Mako is not supported to do work to more than {} databases", NUM_DATABASES_MAX); return -1; } - if (args.num_databases < args.num_fdb_clusters) { - logr.error("--num_databases ({}) must be >= number of clusters({})", args.num_databases, args.num_fdb_clusters); + if (num_databases < num_fdb_clusters) { + logr.error("--num_databases ({}) must be >= number of clusters({})", num_databases, num_fdb_clusters); return -1; } - if (args.num_threads < args.num_databases) { - logr.error("--threads ({}) must be >= number of databases ({})", args.num_threads, args.num_databases); + if (num_threads < num_databases) { + logr.error("--threads ({}) must be >= number of databases ({})", num_threads, num_databases); return -1; } - if (args.key_length < 4 /* "mako" */ + args.row_digits) { + if (key_length < 4 /* "mako" */ + row_digits) { logr.error("--keylen must be larger than {} to store \"mako\" prefix " "and maximum row number", - 4 + args.row_digits); + 4 + row_digits); return -1; } - if (args.active_tenants > args.total_tenants) { + if (active_tenants > total_tenants) { logr.error("--active_tenants must be less than or equal to --total_tenants"); return -1; } - if (args.tenant_batch_size < 1) { + if (tenant_batch_size < 1) { logr.error("--tenant_batch_size must be at least 1"); return -1; } - if (args.mode == MODE_RUN) { - if ((args.seconds > 0) && (args.iteration > 0)) { + if (mode == MODE_RUN) { + if ((seconds > 0) && (iteration > 0)) { logr.error("Cannot specify seconds and iteration together"); return -1; } - if ((args.seconds == 0) && (args.iteration == 0)) { + if ((seconds == 0) && (iteration == 0)) { logr.error("Must specify either seconds or iteration"); return -1; } - if (args.txntagging < 0) { + if (txntagging < 0) { logr.error("--txntagging must be a non-negative integer"); return -1; } } // ensure that all of the files provided to mako are valid and exist - if (args.mode == MODE_REPORT) { - if (!args.num_report_files) { + if (mode == MODE_REPORT) { + if (!num_report_files) { logr.error("No files to merge"); } - for (int i = 0; i < args.num_report_files; i++) { + for (int i = 0; i < num_report_files; i++) { struct stat buffer; - if (stat(args.report_files[i], &buffer) != 0) { - logr.error("Couldn't open file {}", args.report_files[i]); + if (stat(report_files[i], &buffer) != 0) { + logr.error("Couldn't open file {}", report_files[i]); return -1; } } } - if (args.distributed_tracer_client < 0) { - logr.error("--disibuted_tracer_client must specify either (disabled, network_lossy, log_file)"); + if (distributed_tracer_client < 0) { + logr.error("--distributed_tracer_client must specify either (disabled, network_lossy, log_file)"); return -1; } + + if (!authorization_tokens.empty() && !tls_ca_file.has_value()) { + logr.warn("Authorization tokens are being used without explicit TLS CA file configured"); + } return 0; } @@ -2262,11 +2336,6 @@ int main(int argc, char* argv[]) { auto rc = int{}; auto args = Arguments{}; - rc = initArguments(args); - if (rc < 0) { - logr.error("initArguments failed"); - return -1; - } rc = parseArguments(argc, argv, args); if (rc < 0) { /* usage printed */ @@ -2282,7 +2351,7 @@ int main(int argc, char* argv[]) { args.total_tenants = args.active_tenants; } - rc = validateArguments(args); + rc = args.validate(); if (rc < 0) return -1; logr.setVerbosity(args.verbose); diff --git a/bindings/c/test/mako/mako.hpp b/bindings/c/test/mako/mako.hpp index 952cffc7fa..dee75bad82 100644 --- a/bindings/c/test/mako/mako.hpp +++ b/bindings/c/test/mako/mako.hpp @@ -30,6 +30,7 @@ #include #include #include +#include #include #include #include @@ -79,7 +80,11 @@ enum ArgKind { 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, - ARG_DISTRIBUTED_TRACER_CLIENT + ARG_DISTRIBUTED_TRACER_CLIENT, + ARG_TLS_CERTIFICATE_FILE, + ARG_TLS_KEY_FILE, + ARG_TLS_CA_FILE, + ARG_AUTHORIZATION_TOKEN_FILE, }; constexpr const int OP_COUNT = 0; @@ -131,6 +136,9 @@ constexpr const int MAX_REPORT_FILES = 200; /* benchmark parameters */ struct Arguments { + Arguments(); + int validate(); + int api_version; int json; int num_processes; @@ -180,6 +188,10 @@ struct Arguments { char report_files[MAX_REPORT_FILES][PATH_MAX]; int num_report_files; int distributed_tracer_client; + std::optional tls_certificate_file; + std::optional tls_key_file; + std::optional tls_ca_file; + std::map authorization_tokens; // maps tenant name to token string }; } // namespace mako From 33c8a8061482518149824b4f0a7cc6d48c7c2ea6 Mon Sep 17 00:00:00 2001 From: Junhyun Shim Date: Tue, 25 Oct 2022 16:13:01 +0200 Subject: [PATCH 28/95] Update Mako documentation for authz/TLS enablement --- bindings/c/test/mako/mako.rst | 16 ++++++++++++++-- 1 file changed, 14 insertions(+), 2 deletions(-) diff --git a/bindings/c/test/mako/mako.rst b/bindings/c/test/mako/mako.rst index 4b3c3d7b26..0e307b3d8a 100644 --- a/bindings/c/test/mako/mako.rst +++ b/bindings/c/test/mako/mako.rst @@ -38,7 +38,7 @@ Arguments | - ``build``: Populate data | - ``run``: Run the benchmark -- | ``-c | --cluster `` +- | ``-c | --cluster `` | FDB cluster files (Required, comma-separated) - | ``-d | --num_databases `` @@ -125,9 +125,21 @@ Arguments | Disable snapshot read-your-writes - | ``--json_report`` defaults to ``mako.json`` - | ``--json_report=PATH`` + | ``--json_report `` | Output stats to the specified json file +- | ``--tls_certificate_file `` + | Use TLS certificate located in ```` + +- | ``--tls_key_file `` + | Use TLS key file located in ```` + +- | ``--tls_ca_file `` + | Use TLS CA file located in ```` + +- | ``--authorization_token_file `` + | Use authorization token JSON file located in ```` + | Expected content is a JSON object where each key is a tenant name and the mapped value is a token string Transaction Specification ========================= From f88b8e2351c1bc5978b1a729d8c45a37f9189ed5 Mon Sep 17 00:00:00 2001 From: Markus Pilman Date: Tue, 25 Oct 2022 11:11:42 -0600 Subject: [PATCH 29/95] fix summarize bug --- contrib/TestHarness2/test_harness/summarize.py | 18 +++++++++++++++--- 1 file changed, 15 insertions(+), 3 deletions(-) diff --git a/contrib/TestHarness2/test_harness/summarize.py b/contrib/TestHarness2/test_harness/summarize.py index 39eae8803e..3629fa7d43 100644 --- a/contrib/TestHarness2/test_harness/summarize.py +++ b/contrib/TestHarness2/test_harness/summarize.py @@ -159,13 +159,20 @@ class Parser: pass -class XmlParser(Parser, xml.sax.handler.ContentHandler): +class XmlParser(Parser, xml.sax.handler.ContentHandler, xml.sax.handler.ErrorHandler): def __init__(self): super().__init__() self.handler: ParseHandler | None = None def parse(self, file: TextIO, handler: ParseHandler) -> None: - xml.sax.parse(file, self) + self.handler = handler + xml.sax.parse(file, self, errorHandler=self) + + def error(self, exception): + pass + + def fatalError(self, exception): + pass def startElement(self, name, attrs) -> None: attributes: Dict[str, str] = {} @@ -276,6 +283,7 @@ class TraceFiles: raise StopIteration self.current += 1 return self.trace_files[self.current - 1] + return TraceFilesIterator(self) @@ -426,7 +434,8 @@ class Summary: lines = self.error_out.splitlines() stderr_bytes = 0 for line in lines: - if line.endswith("WARNING: ASan doesn't fully support makecontext/swapcontext functions and may produce false positives in some cases!"): + if line.endswith( + "WARNING: ASan doesn't fully support makecontext/swapcontext functions and may produce false positives in some cases!"): # When running ASAN we expect to see this message. Boost coroutine should be using the correct asan annotations so that it shouldn't produce any false positives. continue if line.endswith("Warning: unimplemented fcntl command: 1036"): @@ -606,6 +615,7 @@ class Summary: child.attributes['File'] = attrs['File'] child.attributes['Line'] = attrs['Line'] self.out.append(child) + self.handler.add_handler(('Type', 'BuggifySection'), buggify_section) self.handler.add_handler(('Type', 'FaultInjected'), buggify_section) @@ -614,9 +624,11 @@ class Summary: child.attributes['Name'] = attrs['Name'] child.attributes['File'] = attrs['File'] child.attributes['Line'] = attrs['Line'] + self.handler.add_handler(('Type', 'RunningUnitTest'), running_unit_test) def stderr_severity(attrs: Dict[str, str]): if 'NewSeverity' in attrs: self.stderr_severity = attrs['NewSeverity'] + self.handler.add_handler(('Type', 'StderrSeverity'), stderr_severity) From 74212eeacf2d5306dc19ee0e52a42a85cb26e0d4 Mon Sep 17 00:00:00 2001 From: sfc-gh-tclinkenbeard Date: Tue, 25 Oct 2022 10:17:15 -0700 Subject: [PATCH 30/95] Encapsulate CounterCollection --- fdbclient/BlobCipher.cpp | 2 +- fdbclient/TaskBucket.actor.cpp | 4 +- .../include/fdbclient/BlobWorkerCommon.h | 4 +- fdbrpc/Stats.actor.cpp | 68 +++++++++++-------- fdbrpc/include/fdbrpc/Stats.h | 50 +++++++++----- fdbserver/BackupWorker.actor.cpp | 4 +- fdbserver/BlobManager.actor.cpp | 2 +- fdbserver/ClusterController.actor.cpp | 9 ++- fdbserver/ConfigBroadcaster.actor.cpp | 4 +- fdbserver/ConfigNode.actor.cpp | 2 +- fdbserver/GrvProxyServer.actor.cpp | 2 +- fdbserver/LocalConfiguration.actor.cpp | 4 +- fdbserver/LogRouter.actor.cpp | 17 +++-- fdbserver/OldTLogServer_4_6.actor.cpp | 46 ++++++------- fdbserver/OldTLogServer_6_0.actor.cpp | 46 ++++++------- fdbserver/OldTLogServer_6_2.actor.cpp | 46 ++++++------- fdbserver/Resolver.actor.cpp | 2 +- fdbserver/SimpleConfigConsumer.actor.cpp | 4 +- fdbserver/StorageCache.actor.cpp | 15 ++-- fdbserver/TLogServer.actor.cpp | 46 ++++++------- .../include/fdbserver/ClusterRecovery.actor.h | 9 ++- .../include/fdbserver/ProxyCommitData.actor.h | 2 +- .../include/fdbserver/RestoreApplier.actor.h | 10 +-- .../include/fdbserver/RestoreLoader.actor.h | 10 +-- fdbserver/masterserver.actor.cpp | 2 +- fdbserver/storageserver.actor.cpp | 60 ++++++++-------- 26 files changed, 246 insertions(+), 224 deletions(-) diff --git a/fdbclient/BlobCipher.cpp b/fdbclient/BlobCipher.cpp index e2b4890b24..9dc2c19798 100644 --- a/fdbclient/BlobCipher.cpp +++ b/fdbclient/BlobCipher.cpp @@ -83,7 +83,7 @@ BlobCipherMetrics::BlobCipherMetrics() CounterSet(cc, "Backup"), CounterSet(cc, "Test") }) { specialCounter(cc, "CacheSize", []() { return BlobCipherKeyCache::getInstance()->getSize(); }); - traceFuture = traceCounters("BlobCipherMetrics", UID(), FLOW_KNOBS->ENCRYPT_KEY_CACHE_LOGGING_INTERVAL, &cc); + traceFuture = cc.traceCounters("BlobCipherMetrics", UID(), FLOW_KNOBS->ENCRYPT_KEY_CACHE_LOGGING_INTERVAL); } std::string toString(BlobCipherMetrics::UsageType type) { diff --git a/fdbclient/TaskBucket.actor.cpp b/fdbclient/TaskBucket.actor.cpp index 2e72b301c0..347395892b 100644 --- a/fdbclient/TaskBucket.actor.cpp +++ b/fdbclient/TaskBucket.actor.cpp @@ -579,8 +579,8 @@ public: int maxConcurrentTasks) { state Reference> paused = makeReference>(true); state Future watchPausedFuture = watchPaused(cx, taskBucket, paused); - taskBucket->metricLogger = traceCounters( - "TaskBucketMetrics", taskBucket->dbgid, CLIENT_KNOBS->TASKBUCKET_LOGGING_DELAY, &taskBucket->cc); + taskBucket->metricLogger = taskBucket->cc.traceCounters( + "TaskBucketMetrics", taskBucket->dbgid, CLIENT_KNOBS->TASKBUCKET_LOGGING_DELAY); loop { while (paused->get()) { wait(paused->onChange() || watchPausedFuture); diff --git a/fdbclient/include/fdbclient/BlobWorkerCommon.h b/fdbclient/include/fdbclient/BlobWorkerCommon.h index 9539db459b..b4cbbac2a7 100644 --- a/fdbclient/include/fdbclient/BlobWorkerCommon.h +++ b/fdbclient/include/fdbclient/BlobWorkerCommon.h @@ -103,8 +103,8 @@ struct BlobWorkerStats { specialCounter(cc, "DeltaFileWritesActive", [this]() { return this->deltaWritesLock->activePermits(); }); specialCounter(cc, "DeltaFileWritesWaiting", [this]() { return this->deltaWritesLock->waiters(); }); - logger = traceCounters("BlobWorkerMetrics", id, interval, &cc, "BlobWorkerMetrics"); + logger = cc.traceCounters("BlobWorkerMetrics", id, interval, "BlobWorkerMetrics"); } }; -#endif \ No newline at end of file +#endif diff --git a/fdbrpc/Stats.actor.cpp b/fdbrpc/Stats.actor.cpp index 274a4ec92e..065e7622e3 100644 --- a/fdbrpc/Stats.actor.cpp +++ b/fdbrpc/Stats.actor.cpp @@ -24,8 +24,8 @@ Counter::Counter(std::string const& name, CounterCollection& collection) : name(name), interval_start(0), last_event(0), interval_sq_time(0), roughness_interval_start(0), interval_delta(0), interval_start_value(0) { - metric.init(collection.name + "." + (char)toupper(name.at(0)) + name.substr(1), collection.id); - collection.counters.push_back(this); + metric.init(collection.getName() + "." + (char)toupper(name.at(0)) + name.substr(1), collection.getId()); + collection.addCounter(this); } void Counter::operator+=(Value delta) { @@ -88,36 +88,48 @@ void CounterCollection::logToTraceEvent(TraceEvent& te) const { } } -ACTOR Future traceCounters(std::string traceEventName, - UID traceEventID, - double interval, - CounterCollection* counters, - std::string trackLatestName, - std::function decorator) { - wait(delay(0)); // Give an opportunity for all members used in special counters to be initialized +class CounterCollectionImpl { +public: + ACTOR static Future traceCounters(CounterCollection* counters, + std::string traceEventName, + UID traceEventID, + double interval, + std::string trackLatestName, + std::function decorator) { + wait(delay(0)); // Give an opportunity for all members used in special counters to be initialized - for (ICounter* c : counters->counters) - c->resetInterval(); - - state Reference traceEventHolder; - if (!trackLatestName.empty()) { - traceEventHolder = makeReference(trackLatestName); - } - - state double last_interval = now(); - - loop { - TraceEvent te(traceEventName.c_str(), traceEventID); - te.detail("Elapsed", now() - last_interval); - - counters->logToTraceEvent(te); - decorator(te); + for (ICounter* c : counters->counters) + c->resetInterval(); + state Reference traceEventHolder; if (!trackLatestName.empty()) { - te.trackLatest(traceEventHolder->trackingKey); + traceEventHolder = makeReference(trackLatestName); } - last_interval = now(); - wait(delay(interval, TaskPriority::FlushTrace)); + state double last_interval = now(); + + loop { + TraceEvent te(traceEventName.c_str(), traceEventID); + te.detail("Elapsed", now() - last_interval); + + counters->logToTraceEvent(te); + decorator(te); + + if (!trackLatestName.empty()) { + te.trackLatest(traceEventHolder->trackingKey); + } + + last_interval = now(); + wait(delay(interval, TaskPriority::FlushTrace)); + } } +}; + +Future CounterCollection::traceCounters(std::string const& traceEventName, + UID traceEventID, + double interval, + std::string const& trackLatestName, + std::function const& decorator) { + return CounterCollectionImpl::traceCounters( + this, traceEventName, traceEventID, interval, trackLatestName, decorator); } diff --git a/fdbrpc/include/fdbrpc/Stats.h b/fdbrpc/include/fdbrpc/Stats.h index f8a15e7c16..10e430c317 100644 --- a/fdbrpc/include/fdbrpc/Stats.h +++ b/fdbrpc/include/fdbrpc/Stats.h @@ -67,17 +67,39 @@ struct Traceable : std::true_type { } }; -struct CounterCollection { - CounterCollection(std::string name, std::string id = std::string()) : name(name), id(id) {} - std::vector counters, counters_to_remove; - ~CounterCollection() { - for (auto c : counters_to_remove) - c->remove(); - } +class CounterCollectionImpl; + +class CounterCollection { + friend class CounterCollectionImpl; + std::string name; std::string id; + std::vector counters, countersToRemove; + +public: + CounterCollection(std::string const& name, std::string const& id = std::string()) : name(name), id(id) {} + ~CounterCollection() { + for (auto c : countersToRemove) + c->remove(); + } + + void addCounter(ICounter* counter) { counters.push_back(counter); } + + // Call remove method on this counter in ~CounterCollection + void markForRemoval(ICounter* counter) { countersToRemove.push_back(counter); } + + std::string const& getName() const { return name; } + + std::string const& getId() const { return id; } void logToTraceEvent(TraceEvent& te) const; + + Future traceCounters( + std::string const& traceEventName, + UID traceEventID, + double interval, + std::string const& trackLatestName = std::string(), + std::function const& decorator = [](auto& te) {}); }; struct Counter final : ICounter, NonCopyable { @@ -131,8 +153,8 @@ struct Traceable : std::true_type { template struct SpecialCounter final : ICounter, FastAllocated>, NonCopyable { SpecialCounter(CounterCollection& collection, std::string const& name, F&& f) : name(name), f(f) { - collection.counters.push_back(this); - collection.counters_to_remove.push_back(this); + collection.addCounter(this); + collection.markForRemoval(this); } void remove() override { delete this; } @@ -162,14 +184,6 @@ static void specialCounter(CounterCollection& collection, std::string const& nam new SpecialCounter(collection, name, std::move(f)); } -Future traceCounters( - std::string const& traceEventName, - UID const& traceEventID, - double const& interval, - CounterCollection* const& counters, - std::string const& trackLatestName = std::string(), - std::function const& decorator = [](TraceEvent& te) {}); - class LatencyBands { public: LatencyBands(std::string name, UID id, double loggingInterval) @@ -180,7 +194,7 @@ public: if (bands.size() == 0) { ASSERT(!cc && !filteredCount); cc = std::make_unique(name, id.toString()); - logger = traceCounters(name, id, loggingInterval, cc.get(), id.toString() + "/" + name); + logger = cc->traceCounters(name, id, loggingInterval, id.toString() + "/" + name); filteredCount = std::make_unique("Filtered", *cc); insertBand(std::numeric_limits::infinity()); } diff --git a/fdbserver/BackupWorker.actor.cpp b/fdbserver/BackupWorker.actor.cpp index 67fc20aff4..488d35b3c3 100644 --- a/fdbserver/BackupWorker.actor.cpp +++ b/fdbserver/BackupWorker.actor.cpp @@ -290,8 +290,8 @@ struct BackupData { specialCounter(cc, "MsgQ", [this]() { return this->messages.size(); }); specialCounter(cc, "BufferedBytes", [this]() { return this->lock->activePermits(); }); specialCounter(cc, "AvailableBytes", [this]() { return this->lock->available(); }); - logger = traceCounters( - "BackupWorkerMetrics", myId, SERVER_KNOBS->WORKER_LOGGING_INTERVAL, &cc, "BackupWorkerMetrics"); + logger = + cc.traceCounters("BackupWorkerMetrics", myId, SERVER_KNOBS->WORKER_LOGGING_INTERVAL, "BackupWorkerMetrics"); } bool pullFinished() const { return endVersion.present() && pulledVersion.get() > endVersion.get(); } diff --git a/fdbserver/BlobManager.actor.cpp b/fdbserver/BlobManager.actor.cpp index ad6051b602..06bc97b4eb 100644 --- a/fdbserver/BlobManager.actor.cpp +++ b/fdbserver/BlobManager.actor.cpp @@ -296,7 +296,7 @@ struct BlobManagerStats { specialCounter(cc, "HardBoundaries", [mergeHardBoundaries]() { return mergeHardBoundaries->size(); }); specialCounter(cc, "SoftBoundaries", [mergeBoundaries]() { return mergeBoundaries->size(); }); specialCounter(cc, "BlockedAssignments", [this]() { return this->blockedAssignments; }); - logger = traceCounters("BlobManagerMetrics", id, interval, &cc, "BlobManagerMetrics"); + logger = cc.traceCounters("BlobManagerMetrics", id, interval, "BlobManagerMetrics"); } }; diff --git a/fdbserver/ClusterController.actor.cpp b/fdbserver/ClusterController.actor.cpp index c962ca891d..f65bf43746 100644 --- a/fdbserver/ClusterController.actor.cpp +++ b/fdbserver/ClusterController.actor.cpp @@ -3006,11 +3006,10 @@ ACTOR Future clusterControllerCore(ClusterControllerFullInterface interf, self.addActor.send(monitorConsistencyScan(&self)); self.addActor.send(metaclusterMetricsUpdater(&self)); self.addActor.send(dbInfoUpdater(&self)); - self.addActor.send(traceCounters("ClusterControllerMetrics", - self.id, - SERVER_KNOBS->STORAGE_LOGGING_DELAY, - &self.clusterControllerMetrics, - self.id.toString() + "/ClusterControllerMetrics")); + self.addActor.send(self.clusterControllerMetrics.traceCounters("ClusterControllerMetrics", + self.id, + SERVER_KNOBS->STORAGE_LOGGING_DELAY, + self.id.toString() + "/ClusterControllerMetrics")); self.addActor.send(traceRole(Role::CLUSTER_CONTROLLER, interf.id())); // printf("%s: I am the cluster controller\n", g_network->getLocalAddress().toString().c_str()); diff --git a/fdbserver/ConfigBroadcaster.actor.cpp b/fdbserver/ConfigBroadcaster.actor.cpp index eafce9c9cb..bf49f8e58a 100644 --- a/fdbserver/ConfigBroadcaster.actor.cpp +++ b/fdbserver/ConfigBroadcaster.actor.cpp @@ -183,8 +183,8 @@ class ConfigBroadcasterImpl { id(deterministicRandom()->randomUniqueID()), cc("ConfigBroadcaster"), compactRequest("CompactRequest", cc), successfulChangeRequest("SuccessfulChangeRequest", cc), failedChangeRequest("FailedChangeRequest", cc), snapshotRequest("SnapshotRequest", cc) { - logger = traceCounters( - "ConfigBroadcasterMetrics", id, SERVER_KNOBS->WORKER_LOGGING_INTERVAL, &cc, "ConfigBroadcasterMetrics"); + logger = cc.traceCounters( + "ConfigBroadcasterMetrics", id, SERVER_KNOBS->WORKER_LOGGING_INTERVAL, "ConfigBroadcasterMetrics"); } void addChanges(Standalone> const& changes, diff --git a/fdbserver/ConfigNode.actor.cpp b/fdbserver/ConfigNode.actor.cpp index 7c8ce60d04..b67e856be0 100644 --- a/fdbserver/ConfigNode.actor.cpp +++ b/fdbserver/ConfigNode.actor.cpp @@ -812,7 +812,7 @@ public: successfulCommits("SuccessfulCommits", cc), failedCommits("FailedCommits", cc), setMutations("SetMutations", cc), clearMutations("ClearMutations", cc), getValueRequests("GetValueRequests", cc), getGenerationRequests("GetGenerationRequests", cc) { - logger = traceCounters("ConfigNodeMetrics", id, SERVER_KNOBS->WORKER_LOGGING_INTERVAL, &cc, "ConfigNode"); + logger = cc.traceCounters("ConfigNodeMetrics", id, SERVER_KNOBS->WORKER_LOGGING_INTERVAL, "ConfigNode"); TraceEvent(SevInfo, "StartingConfigNode", id).detail("KVStoreAlreadyExists", kvStore.exists()); } diff --git a/fdbserver/GrvProxyServer.actor.cpp b/fdbserver/GrvProxyServer.actor.cpp index b865b9407b..a55748505f 100644 --- a/fdbserver/GrvProxyServer.actor.cpp +++ b/fdbserver/GrvProxyServer.actor.cpp @@ -154,7 +154,7 @@ struct GrvProxyStats { return int64_t(100 * this->percentageOfBatchGRVQueueProcessed); }); - logger = traceCounters("GrvProxyMetrics", id, SERVER_KNOBS->WORKER_LOGGING_INTERVAL, &cc, "GrvProxyMetrics"); + logger = cc.traceCounters("GrvProxyMetrics", id, SERVER_KNOBS->WORKER_LOGGING_INTERVAL, "GrvProxyMetrics"); for (int i = 0; i < FLOW_KNOBS->BASIC_LOAD_BALANCE_BUCKETS; i++) { requestBuckets.push_back(0); } diff --git a/fdbserver/LocalConfiguration.actor.cpp b/fdbserver/LocalConfiguration.actor.cpp index 7a8e04e76b..c2cdaf7479 100644 --- a/fdbserver/LocalConfiguration.actor.cpp +++ b/fdbserver/LocalConfiguration.actor.cpp @@ -347,8 +347,8 @@ public: Randomize::False, g_network->isSimulated() ? IsSimulated::True : IsSimulated::False); } - logger = traceCounters( - "LocalConfigurationMetrics", id, SERVER_KNOBS->WORKER_LOGGING_INTERVAL, &cc, "LocalConfigurationMetrics"); + logger = cc.traceCounters( + "LocalConfigurationMetrics", id, SERVER_KNOBS->WORKER_LOGGING_INTERVAL, "LocalConfigurationMetrics"); } Future addChanges(Standalone> changes, diff --git a/fdbserver/LogRouter.actor.cpp b/fdbserver/LogRouter.actor.cpp index 2ee606e6fd..399d820608 100644 --- a/fdbserver/LogRouter.actor.cpp +++ b/fdbserver/LogRouter.actor.cpp @@ -190,15 +190,14 @@ struct LogRouterData { }); specialCounter(cc, "Generation", [this]() { return this->generation; }); specialCounter(cc, "ActivePeekStreams", [this]() { return this->activePeekStreams; }); - logger = traceCounters("LogRouterMetrics", - dbgid, - SERVER_KNOBS->WORKER_LOGGING_INTERVAL, - &cc, - "LogRouterMetrics", - [this](TraceEvent& te) { - te.detail("PrimaryPeekLocation", this->primaryPeekLocation); - te.detail("RouterTag", this->routerTag.toString()); - }); + logger = cc.traceCounters("LogRouterMetrics", + dbgid, + SERVER_KNOBS->WORKER_LOGGING_INTERVAL, + "LogRouterMetrics", + [this](TraceEvent& te) { + te.detail("PrimaryPeekLocation", this->primaryPeekLocation); + te.detail("RouterTag", this->routerTag.toString()); + }); } }; diff --git a/fdbserver/OldTLogServer_4_6.actor.cpp b/fdbserver/OldTLogServer_4_6.actor.cpp index bdf0d06bc3..b122fde854 100644 --- a/fdbserver/OldTLogServer_4_6.actor.cpp +++ b/fdbserver/OldTLogServer_4_6.actor.cpp @@ -447,10 +447,10 @@ struct LogData : NonCopyable, public ReferenceCounted { "Restored"); addActor.send(traceRole(Role::TRANSACTION_LOG, interf.id())); - persistentDataVersion.init("TLog.PersistentDataVersion"_sr, cc.id); - persistentDataDurableVersion.init("TLog.PersistentDataDurableVersion"_sr, cc.id); - version.initMetric("TLog.Version"_sr, cc.id); - queueCommittedVersion.initMetric("TLog.QueueCommittedVersion"_sr, cc.id); + persistentDataVersion.init("TLog.PersistentDataVersion"_sr, cc.getId()); + persistentDataDurableVersion.init("TLog.PersistentDataDurableVersion"_sr, cc.getId()); + version.initMetric("TLog.Version"_sr, cc.getId()); + queueCommittedVersion.initMetric("TLog.QueueCommittedVersion"_sr, cc.getId()); specialCounter(cc, "Version", [this]() { return this->version.get(); }); specialCounter(cc, "SharedBytesInput", [tLogData]() { return tLogData->bytesInput; }); @@ -1399,26 +1399,26 @@ ACTOR Future tLogCore(TLogData* self, Reference logData) { logData->addActor.send(waitFailureServer(logData->tli.waitFailure.getFuture())); logData->addActor.send(logData->removed); // FIXME: update tlogMetrics to include new information, or possibly only have one copy for the shared instance - logData->addActor.send(traceCounters("TLogMetrics", - logData->logId, - SERVER_KNOBS->STORAGE_LOGGING_DELAY, - &logData->cc, - logData->logId.toString() + "/TLogMetrics", - [self = self](TraceEvent& te) { - StorageBytes sbTlog = self->persistentData->getStorageBytes(); - te.detail("KvstoreBytesUsed", sbTlog.used); - te.detail("KvstoreBytesFree", sbTlog.free); - te.detail("KvstoreBytesAvailable", sbTlog.available); - te.detail("KvstoreBytesTotal", sbTlog.total); - te.detail("KvstoreBytesTemp", sbTlog.temp); + logData->addActor.send(logData->cc.traceCounters("TLogMetrics", + logData->logId, + SERVER_KNOBS->STORAGE_LOGGING_DELAY, + logData->logId.toString() + "/TLogMetrics", + [self = self](TraceEvent& te) { + StorageBytes sbTlog = self->persistentData->getStorageBytes(); + te.detail("KvstoreBytesUsed", sbTlog.used); + te.detail("KvstoreBytesFree", sbTlog.free); + te.detail("KvstoreBytesAvailable", sbTlog.available); + te.detail("KvstoreBytesTotal", sbTlog.total); + te.detail("KvstoreBytesTemp", sbTlog.temp); - StorageBytes sbQueue = self->rawPersistentQueue->getStorageBytes(); - te.detail("QueueDiskBytesUsed", sbQueue.used); - te.detail("QueueDiskBytesFree", sbQueue.free); - te.detail("QueueDiskBytesAvailable", sbQueue.available); - te.detail("QueueDiskBytesTotal", sbQueue.total); - te.detail("QueueDiskBytesTemp", sbQueue.temp); - })); + StorageBytes sbQueue = + self->rawPersistentQueue->getStorageBytes(); + te.detail("QueueDiskBytesUsed", sbQueue.used); + te.detail("QueueDiskBytesFree", sbQueue.free); + te.detail("QueueDiskBytesAvailable", sbQueue.available); + te.detail("QueueDiskBytesTotal", sbQueue.total); + te.detail("QueueDiskBytesTemp", sbQueue.temp); + })); logData->addActor.send(serveTLogInterface(self, logData->tli, logData, warningCollectorInput)); diff --git a/fdbserver/OldTLogServer_6_0.actor.cpp b/fdbserver/OldTLogServer_6_0.actor.cpp index fb0d7daacb..a346d214ff 100644 --- a/fdbserver/OldTLogServer_6_0.actor.cpp +++ b/fdbserver/OldTLogServer_6_0.actor.cpp @@ -533,10 +533,10 @@ struct LogData : NonCopyable, public ReferenceCounted { context); addActor.send(traceRole(Role::TRANSACTION_LOG, interf.id())); - persistentDataVersion.init("TLog.PersistentDataVersion"_sr, cc.id); - persistentDataDurableVersion.init("TLog.PersistentDataDurableVersion"_sr, cc.id); - version.initMetric("TLog.Version"_sr, cc.id); - queueCommittedVersion.initMetric("TLog.QueueCommittedVersion"_sr, cc.id); + persistentDataVersion.init("TLog.PersistentDataVersion"_sr, cc.getId()); + persistentDataDurableVersion.init("TLog.PersistentDataDurableVersion"_sr, cc.getId()); + version.initMetric("TLog.Version"_sr, cc.getId()); + queueCommittedVersion.initMetric("TLog.QueueCommittedVersion"_sr, cc.getId()); specialCounter(cc, "Version", [this]() { return this->version.get(); }); specialCounter(cc, "QueueCommittedVersion", [this]() { return this->queueCommittedVersion.get(); }); @@ -2212,26 +2212,26 @@ ACTOR Future tLogCore(TLogData* self, logData->addActor.send(waitFailureServer(tli.waitFailure.getFuture())); logData->addActor.send(logData->removed); // FIXME: update tlogMetrics to include new information, or possibly only have one copy for the shared instance - logData->addActor.send(traceCounters("TLogMetrics", - logData->logId, - SERVER_KNOBS->STORAGE_LOGGING_DELAY, - &logData->cc, - logData->logId.toString() + "/TLogMetrics", - [self = self](TraceEvent& te) { - StorageBytes sbTlog = self->persistentData->getStorageBytes(); - te.detail("KvstoreBytesUsed", sbTlog.used); - te.detail("KvstoreBytesFree", sbTlog.free); - te.detail("KvstoreBytesAvailable", sbTlog.available); - te.detail("KvstoreBytesTotal", sbTlog.total); - te.detail("KvstoreBytesTemp", sbTlog.temp); + logData->addActor.send(logData->cc.traceCounters("TLogMetrics", + logData->logId, + SERVER_KNOBS->STORAGE_LOGGING_DELAY, + logData->logId.toString() + "/TLogMetrics", + [self = self](TraceEvent& te) { + StorageBytes sbTlog = self->persistentData->getStorageBytes(); + te.detail("KvstoreBytesUsed", sbTlog.used); + te.detail("KvstoreBytesFree", sbTlog.free); + te.detail("KvstoreBytesAvailable", sbTlog.available); + te.detail("KvstoreBytesTotal", sbTlog.total); + te.detail("KvstoreBytesTemp", sbTlog.temp); - StorageBytes sbQueue = self->rawPersistentQueue->getStorageBytes(); - te.detail("QueueDiskBytesUsed", sbQueue.used); - te.detail("QueueDiskBytesFree", sbQueue.free); - te.detail("QueueDiskBytesAvailable", sbQueue.available); - te.detail("QueueDiskBytesTotal", sbQueue.total); - te.detail("QueueDiskBytesTemp", sbQueue.temp); - })); + StorageBytes sbQueue = + self->rawPersistentQueue->getStorageBytes(); + te.detail("QueueDiskBytesUsed", sbQueue.used); + te.detail("QueueDiskBytesFree", sbQueue.free); + te.detail("QueueDiskBytesAvailable", sbQueue.available); + te.detail("QueueDiskBytesTotal", sbQueue.total); + te.detail("QueueDiskBytesTemp", sbQueue.temp); + })); logData->addActor.send(serveTLogInterface(self, tli, logData, warningCollectorInput)); logData->addActor.send(cleanupPeekTrackers(logData.getPtr())); diff --git a/fdbserver/OldTLogServer_6_2.actor.cpp b/fdbserver/OldTLogServer_6_2.actor.cpp index e9e52dd21c..c68fb26b26 100644 --- a/fdbserver/OldTLogServer_6_2.actor.cpp +++ b/fdbserver/OldTLogServer_6_2.actor.cpp @@ -616,10 +616,10 @@ struct LogData : NonCopyable, public ReferenceCounted { context); addActor.send(traceRole(Role::TRANSACTION_LOG, interf.id())); - persistentDataVersion.init("TLog.PersistentDataVersion"_sr, cc.id); - persistentDataDurableVersion.init("TLog.PersistentDataDurableVersion"_sr, cc.id); - version.initMetric("TLog.Version"_sr, cc.id); - queueCommittedVersion.initMetric("TLog.QueueCommittedVersion"_sr, cc.id); + persistentDataVersion.init("TLog.PersistentDataVersion"_sr, cc.getId()); + persistentDataDurableVersion.init("TLog.PersistentDataDurableVersion"_sr, cc.getId()); + version.initMetric("TLog.Version"_sr, cc.getId()); + queueCommittedVersion.initMetric("TLog.QueueCommittedVersion"_sr, cc.getId()); specialCounter(cc, "Version", [this]() { return this->version.get(); }); specialCounter(cc, "QueueCommittedVersion", [this]() { return this->queueCommittedVersion.get(); }); @@ -2671,26 +2671,26 @@ ACTOR Future tLogCore(TLogData* self, logData->addActor.send(waitFailureServer(tli.waitFailure.getFuture())); logData->addActor.send(logData->removed); // FIXME: update tlogMetrics to include new information, or possibly only have one copy for the shared instance - logData->addActor.send(traceCounters("TLogMetrics", - logData->logId, - SERVER_KNOBS->STORAGE_LOGGING_DELAY, - &logData->cc, - logData->logId.toString() + "/TLogMetrics", - [self = self](TraceEvent& te) { - StorageBytes sbTlog = self->persistentData->getStorageBytes(); - te.detail("KvstoreBytesUsed", sbTlog.used); - te.detail("KvstoreBytesFree", sbTlog.free); - te.detail("KvstoreBytesAvailable", sbTlog.available); - te.detail("KvstoreBytesTotal", sbTlog.total); - te.detail("KvstoreBytesTemp", sbTlog.temp); + logData->addActor.send(logData->cc.traceCounters("TLogMetrics", + logData->logId, + SERVER_KNOBS->STORAGE_LOGGING_DELAY, + logData->logId.toString() + "/TLogMetrics", + [self = self](TraceEvent& te) { + StorageBytes sbTlog = self->persistentData->getStorageBytes(); + te.detail("KvstoreBytesUsed", sbTlog.used); + te.detail("KvstoreBytesFree", sbTlog.free); + te.detail("KvstoreBytesAvailable", sbTlog.available); + te.detail("KvstoreBytesTotal", sbTlog.total); + te.detail("KvstoreBytesTemp", sbTlog.temp); - StorageBytes sbQueue = self->rawPersistentQueue->getStorageBytes(); - te.detail("QueueDiskBytesUsed", sbQueue.used); - te.detail("QueueDiskBytesFree", sbQueue.free); - te.detail("QueueDiskBytesAvailable", sbQueue.available); - te.detail("QueueDiskBytesTotal", sbQueue.total); - te.detail("QueueDiskBytesTemp", sbQueue.temp); - })); + StorageBytes sbQueue = + self->rawPersistentQueue->getStorageBytes(); + te.detail("QueueDiskBytesUsed", sbQueue.used); + te.detail("QueueDiskBytesFree", sbQueue.free); + te.detail("QueueDiskBytesAvailable", sbQueue.available); + te.detail("QueueDiskBytesTotal", sbQueue.total); + te.detail("QueueDiskBytesTemp", sbQueue.temp); + })); logData->addActor.send(serveTLogInterface(self, tli, logData, warningCollectorInput)); logData->addActor.send(cleanupPeekTrackers(logData.getPtr())); diff --git a/fdbserver/Resolver.actor.cpp b/fdbserver/Resolver.actor.cpp index 61aaed1246..c3a59166a2 100644 --- a/fdbserver/Resolver.actor.cpp +++ b/fdbserver/Resolver.actor.cpp @@ -188,7 +188,7 @@ struct Resolver : ReferenceCounted { specialCounter(cc, "NeededVersion", [this]() { return this->neededVersion.get(); }); specialCounter(cc, "TotalStateBytes", [this]() { return this->totalStateBytes.get(); }); - logger = traceCounters("ResolverMetrics", dbgid, SERVER_KNOBS->WORKER_LOGGING_INTERVAL, &cc, "ResolverMetrics"); + logger = cc.traceCounters("ResolverMetrics", dbgid, SERVER_KNOBS->WORKER_LOGGING_INTERVAL, "ResolverMetrics"); } ~Resolver() { destroyConflictSet(conflictSet); } }; diff --git a/fdbserver/SimpleConfigConsumer.actor.cpp b/fdbserver/SimpleConfigConsumer.actor.cpp index 7241ffe48d..769bdf142f 100644 --- a/fdbserver/SimpleConfigConsumer.actor.cpp +++ b/fdbserver/SimpleConfigConsumer.actor.cpp @@ -166,8 +166,8 @@ public: successfulChangeRequest("SuccessfulChangeRequest", cc), failedChangeRequest("FailedChangeRequest", cc), snapshotRequest("SnapshotRequest", cc) { cfi = getConfigFollowerInterface(configSource); - logger = traceCounters( - "ConfigConsumerMetrics", id, SERVER_KNOBS->WORKER_LOGGING_INTERVAL, &cc, "ConfigConsumerMetrics"); + logger = cc.traceCounters( + "ConfigConsumerMetrics", id, SERVER_KNOBS->WORKER_LOGGING_INTERVAL, "ConfigConsumerMetrics"); } Future consume(ConfigBroadcaster& broadcaster) { diff --git a/fdbserver/StorageCache.actor.cpp b/fdbserver/StorageCache.actor.cpp index 0ac15840d2..6cf1c9b934 100644 --- a/fdbserver/StorageCache.actor.cpp +++ b/fdbserver/StorageCache.actor.cpp @@ -248,9 +248,9 @@ public: lastTLogVersion(0), lastVersionWithData(0), peekVersion(0), compactionInProgress(Void()), fetchKeysParallelismLock(SERVER_KNOBS->FETCH_KEYS_PARALLELISM_BYTES), debug_inApplyUpdate(false), debug_lastValidateTime(0), versionLag(0), behind(false), counters(this) { - version.initMetric("StorageCacheData.Version"_sr, counters.cc.id); - desiredOldestVersion.initMetric("StorageCacheData.DesriedOldestVersion"_sr, counters.cc.id); - oldestVersion.initMetric("StorageCacheData.OldestVersion"_sr, counters.cc.id); + version.initMetric("StorageCacheData.Version"_sr, counters.cc.getId()); + desiredOldestVersion.initMetric("StorageCacheData.DesriedOldestVersion"_sr, counters.cc.getId()); + oldestVersion.initMetric("StorageCacheData.OldestVersion"_sr, counters.cc.getId()); newestAvailableVersion.insert(allKeys, invalidVersion); newestDirtyVersion.insert(allKeys, invalidVersion); @@ -2224,11 +2224,10 @@ ACTOR Future storageCacheServer(StorageServerInterface ssi, self.ck = cacheKeysPrefixFor(id).withPrefix(systemKeys.begin); // FFFF/02cacheKeys/[this server]/ actors.add(waitFailureServer(ssi.waitFailure.getFuture())); - actors.add(traceCounters("CacheMetrics", - self.thisServerID, - SERVER_KNOBS->STORAGE_LOGGING_DELAY, - &self.counters.cc, - self.thisServerID.toString() + "/CacheMetrics")); + actors.add(self.counters.cc.traceCounters("CacheMetrics", + self.thisServerID, + SERVER_KNOBS->STORAGE_LOGGING_DELAY, + self.thisServerID.toString() + "/CacheMetrics")); // fetch already cached ranges from the database and apply them before proceeding wait(storageCacheStartUpWarmup(&self)); diff --git a/fdbserver/TLogServer.actor.cpp b/fdbserver/TLogServer.actor.cpp index 2c8a007fde..1334c33eef 100644 --- a/fdbserver/TLogServer.actor.cpp +++ b/fdbserver/TLogServer.actor.cpp @@ -652,10 +652,10 @@ struct LogData : NonCopyable, public ReferenceCounted { context); addActor.send(traceRole(Role::TRANSACTION_LOG, interf.id())); - persistentDataVersion.init("TLog.PersistentDataVersion"_sr, cc.id); - persistentDataDurableVersion.init("TLog.PersistentDataDurableVersion"_sr, cc.id); - version.initMetric("TLog.Version"_sr, cc.id); - queueCommittedVersion.initMetric("TLog.QueueCommittedVersion"_sr, cc.id); + persistentDataVersion.init("TLog.PersistentDataVersion"_sr, cc.getId()); + persistentDataDurableVersion.init("TLog.PersistentDataDurableVersion"_sr, cc.getId()); + version.initMetric("TLog.Version"_sr, cc.getId()); + queueCommittedVersion.initMetric("TLog.QueueCommittedVersion"_sr, cc.getId()); specialCounter(cc, "Version", [this]() { return this->version.get(); }); specialCounter(cc, "QueueCommittedVersion", [this]() { return this->queueCommittedVersion.get(); }); @@ -2930,26 +2930,26 @@ ACTOR Future tLogCore(TLogData* self, logData->addActor.send(waitFailureServer(tli.waitFailure.getFuture())); logData->addActor.send(logData->removed); // FIXME: update tlogMetrics to include new information, or possibly only have one copy for the shared instance - logData->addActor.send(traceCounters("TLogMetrics", - logData->logId, - SERVER_KNOBS->STORAGE_LOGGING_DELAY, - &logData->cc, - logData->logId.toString() + "/TLogMetrics", - [self = self](TraceEvent& te) { - StorageBytes sbTlog = self->persistentData->getStorageBytes(); - te.detail("KvstoreBytesUsed", sbTlog.used); - te.detail("KvstoreBytesFree", sbTlog.free); - te.detail("KvstoreBytesAvailable", sbTlog.available); - te.detail("KvstoreBytesTotal", sbTlog.total); - te.detail("KvstoreBytesTemp", sbTlog.temp); + logData->addActor.send(logData->cc.traceCounters("TLogMetrics", + logData->logId, + SERVER_KNOBS->STORAGE_LOGGING_DELAY, + logData->logId.toString() + "/TLogMetrics", + [self = self](TraceEvent& te) { + StorageBytes sbTlog = self->persistentData->getStorageBytes(); + te.detail("KvstoreBytesUsed", sbTlog.used); + te.detail("KvstoreBytesFree", sbTlog.free); + te.detail("KvstoreBytesAvailable", sbTlog.available); + te.detail("KvstoreBytesTotal", sbTlog.total); + te.detail("KvstoreBytesTemp", sbTlog.temp); - StorageBytes sbQueue = self->rawPersistentQueue->getStorageBytes(); - te.detail("QueueDiskBytesUsed", sbQueue.used); - te.detail("QueueDiskBytesFree", sbQueue.free); - te.detail("QueueDiskBytesAvailable", sbQueue.available); - te.detail("QueueDiskBytesTotal", sbQueue.total); - te.detail("QueueDiskBytesTemp", sbQueue.temp); - })); + StorageBytes sbQueue = + self->rawPersistentQueue->getStorageBytes(); + te.detail("QueueDiskBytesUsed", sbQueue.used); + te.detail("QueueDiskBytesFree", sbQueue.free); + te.detail("QueueDiskBytesAvailable", sbQueue.available); + te.detail("QueueDiskBytesTotal", sbQueue.total); + te.detail("QueueDiskBytesTemp", sbQueue.temp); + })); logData->addActor.send(serveTLogInterface(self, tli, logData, warningCollectorInput)); logData->addActor.send(cleanupPeekTrackers(logData.getPtr())); diff --git a/fdbserver/include/fdbserver/ClusterRecovery.actor.h b/fdbserver/include/fdbserver/ClusterRecovery.actor.h index aa5e25a46c..eb2c4bf464 100644 --- a/fdbserver/include/fdbserver/ClusterRecovery.actor.h +++ b/fdbserver/include/fdbserver/ClusterRecovery.actor.h @@ -289,11 +289,10 @@ struct ClusterRecoveryData : NonCopyable, ReferenceCounted getRecoveryEventName(ClusterRecoveryEventType::CLUSTER_RECOVERY_DURATION_EVENT_NAME)); clusterRecoveryAvailableEventHolder = makeReference( getRecoveryEventName(ClusterRecoveryEventType::CLUSTER_RECOVERY_AVAILABLE_EVENT_NAME)); - logger = traceCounters(getRecoveryEventName(ClusterRecoveryEventType::CLUSTER_RECOVERY_METRICS_EVENT_NAME), - dbgid, - SERVER_KNOBS->WORKER_LOGGING_INTERVAL, - &cc, - getRecoveryEventName(ClusterRecoveryEventType::CLUSTER_RECOVERY_METRICS_EVENT_NAME)); + logger = cc.traceCounters(getRecoveryEventName(ClusterRecoveryEventType::CLUSTER_RECOVERY_METRICS_EVENT_NAME), + dbgid, + SERVER_KNOBS->WORKER_LOGGING_INTERVAL, + getRecoveryEventName(ClusterRecoveryEventType::CLUSTER_RECOVERY_METRICS_EVENT_NAME)); if (forceRecovery && !controllerData->clusterControllerDcId.present()) { TraceEvent(SevError, "ForcedRecoveryRequiresDcID").log(); forceRecovery = false; diff --git a/fdbserver/include/fdbserver/ProxyCommitData.actor.h b/fdbserver/include/fdbserver/ProxyCommitData.actor.h index f5a8a060f9..7ac83c935a 100644 --- a/fdbserver/include/fdbserver/ProxyCommitData.actor.h +++ b/fdbserver/include/fdbserver/ProxyCommitData.actor.h @@ -156,7 +156,7 @@ struct ProxyStats { specialCounter(cc, "NumTenants", [pTenantMap]() { return pTenantMap ? pTenantMap->size() : 0; }); specialCounter(cc, "MaxCompute", [this]() { return this->getAndResetMaxCompute(); }); specialCounter(cc, "MinCompute", [this]() { return this->getAndResetMinCompute(); }); - logger = traceCounters("ProxyMetrics", id, SERVER_KNOBS->WORKER_LOGGING_INTERVAL, &cc, "ProxyMetrics"); + logger = cc.traceCounters("ProxyMetrics", id, SERVER_KNOBS->WORKER_LOGGING_INTERVAL, "ProxyMetrics"); } }; diff --git a/fdbserver/include/fdbserver/RestoreApplier.actor.h b/fdbserver/include/fdbserver/RestoreApplier.actor.h index da370bc62c..d85af1c317 100644 --- a/fdbserver/include/fdbserver/RestoreApplier.actor.h +++ b/fdbserver/include/fdbserver/RestoreApplier.actor.h @@ -284,11 +284,11 @@ struct ApplierBatchData : public ReferenceCounted { : vbState(ApplierVersionBatchState::NOT_INIT), receiveMutationReqs(0), receivedBytes(0), appliedBytes(0), targetWriteRateMB(SERVER_KNOBS->FASTRESTORE_WRITE_BW_MB / SERVER_KNOBS->FASTRESTORE_NUM_APPLIERS), totalBytesToWrite(-1), applyingDataBytes(0), counters(this, nodeID, batchIndex) { - pollMetrics = traceCounters(format("FastRestoreApplierMetrics%d", batchIndex), - nodeID, - SERVER_KNOBS->FASTRESTORE_ROLE_LOGGING_DELAY, - &counters.cc, - nodeID.toString() + "/RestoreApplierMetrics/" + std::to_string(batchIndex)); + pollMetrics = + counters.cc.traceCounters(format("FastRestoreApplierMetrics%d", batchIndex), + nodeID, + SERVER_KNOBS->FASTRESTORE_ROLE_LOGGING_DELAY, + nodeID.toString() + "/RestoreApplierMetrics/" + std::to_string(batchIndex)); TraceEvent("FastRestoreApplierMetricsCreated").detail("Node", nodeID); } ~ApplierBatchData() { diff --git a/fdbserver/include/fdbserver/RestoreLoader.actor.h b/fdbserver/include/fdbserver/RestoreLoader.actor.h index 92b11a5a1c..bd4d361c28 100644 --- a/fdbserver/include/fdbserver/RestoreLoader.actor.h +++ b/fdbserver/include/fdbserver/RestoreLoader.actor.h @@ -93,11 +93,11 @@ struct LoaderBatchData : public ReferenceCounted { explicit LoaderBatchData(UID nodeID, int batchIndex) : vbState(LoaderVersionBatchState::NOT_INIT), loadFileReqs(0), counters(this, nodeID, batchIndex) { - pollMetrics = traceCounters(format("FastRestoreLoaderMetrics%d", batchIndex), - nodeID, - SERVER_KNOBS->FASTRESTORE_ROLE_LOGGING_DELAY, - &counters.cc, - nodeID.toString() + "/RestoreLoaderMetrics/" + std::to_string(batchIndex)); + pollMetrics = + counters.cc.traceCounters(format("FastRestoreLoaderMetrics%d", batchIndex), + nodeID, + SERVER_KNOBS->FASTRESTORE_ROLE_LOGGING_DELAY, + nodeID.toString() + "/RestoreLoaderMetrics/" + std::to_string(batchIndex)); TraceEvent("FastRestoreLoaderMetricsCreated").detail("Node", nodeID); } diff --git a/fdbserver/masterserver.actor.cpp b/fdbserver/masterserver.actor.cpp index 16486de2e8..69a02e2bc2 100644 --- a/fdbserver/masterserver.actor.cpp +++ b/fdbserver/masterserver.actor.cpp @@ -114,7 +114,7 @@ struct MasterData : NonCopyable, ReferenceCounted { SERVER_KNOBS->LATENCY_METRICS_LOGGING_INTERVAL, SERVER_KNOBS->LATENCY_SAMPLE_SIZE), addActor(addActor) { - logger = traceCounters("MasterMetrics", dbgid, SERVER_KNOBS->WORKER_LOGGING_INTERVAL, &cc, "MasterMetrics"); + logger = cc.traceCounters("MasterMetrics", dbgid, SERVER_KNOBS->WORKER_LOGGING_INTERVAL, "MasterMetrics"); if (forceRecovery && !myInterface.locality.dcId().present()) { TraceEvent(SevError, "ForcedRecoveryRequiresDcID").log(); forceRecovery = false; diff --git a/fdbserver/storageserver.actor.cpp b/fdbserver/storageserver.actor.cpp index 5c4393de87..801cdad98d 100644 --- a/fdbserver/storageserver.actor.cpp +++ b/fdbserver/storageserver.actor.cpp @@ -1308,10 +1308,10 @@ public: storageServerSourceTLogIDEventHolder( makeReference(ssi.id().toString() + "/StorageServerSourceTLogID")) { - version.initMetric("StorageServer.Version"_sr, counters.cc.id); - oldestVersion.initMetric("StorageServer.OldestVersion"_sr, counters.cc.id); - durableVersion.initMetric("StorageServer.DurableVersion"_sr, counters.cc.id); - desiredOldestVersion.initMetric("StorageServer.DesiredOldestVersion"_sr, counters.cc.id); + version.initMetric("StorageServer.Version"_sr, counters.cc.getId()); + oldestVersion.initMetric("StorageServer.OldestVersion"_sr, counters.cc.getId()); + durableVersion.initMetric("StorageServer.DurableVersion"_sr, counters.cc.getId()); + desiredOldestVersion.initMetric("StorageServer.DesiredOldestVersion"_sr, counters.cc.getId()); newestAvailableVersion.insert(allKeys, invalidVersion); newestDirtyVersion.insert(allKeys, invalidVersion); @@ -10188,32 +10188,32 @@ ACTOR Future metricsCore(StorageServer* self, StorageServerInterface ssi) TraceEvent("StorageServerRestoreDurableState", self->thisServerID).detail("RestoredBytes", self->bytesRestored); // Logs all counters in `counters.cc` and reset the interval. - self->actors.add(traceCounters("StorageMetrics", - self->thisServerID, - SERVER_KNOBS->STORAGE_LOGGING_DELAY, - &self->counters.cc, - self->thisServerID.toString() + "/StorageMetrics", - [self = self](TraceEvent& te) { - te.detail("StorageEngine", self->storage.getKeyValueStoreType().toString()); - te.detail("Tag", self->tag.toString()); - StorageBytes sb = self->storage.getStorageBytes(); - te.detail("KvstoreBytesUsed", sb.used); - te.detail("KvstoreBytesFree", sb.free); - te.detail("KvstoreBytesAvailable", sb.available); - te.detail("KvstoreBytesTotal", sb.total); - te.detail("KvstoreBytesTemp", sb.temp); - if (self->isTss()) { - te.detail("TSSPairID", self->tssPairID); - te.detail("TSSJointID", - UID(self->thisServerID.first() ^ self->tssPairID.get().first(), - self->thisServerID.second() ^ self->tssPairID.get().second())); - } else if (self->isSSWithTSSPair()) { - te.detail("SSPairID", self->ssPairID); - te.detail("TSSJointID", - UID(self->thisServerID.first() ^ self->ssPairID.get().first(), - self->thisServerID.second() ^ self->ssPairID.get().second())); - } - })); + self->actors.add(self->counters.cc.traceCounters( + "StorageMetrics", + self->thisServerID, + SERVER_KNOBS->STORAGE_LOGGING_DELAY, + self->thisServerID.toString() + "/StorageMetrics", + [self = self](TraceEvent& te) { + te.detail("StorageEngine", self->storage.getKeyValueStoreType().toString()); + te.detail("Tag", self->tag.toString()); + StorageBytes sb = self->storage.getStorageBytes(); + te.detail("KvstoreBytesUsed", sb.used); + te.detail("KvstoreBytesFree", sb.free); + te.detail("KvstoreBytesAvailable", sb.available); + te.detail("KvstoreBytesTotal", sb.total); + te.detail("KvstoreBytesTemp", sb.temp); + if (self->isTss()) { + te.detail("TSSPairID", self->tssPairID); + te.detail("TSSJointID", + UID(self->thisServerID.first() ^ self->tssPairID.get().first(), + self->thisServerID.second() ^ self->tssPairID.get().second())); + } else if (self->isSSWithTSSPair()) { + te.detail("SSPairID", self->ssPairID); + te.detail("TSSJointID", + UID(self->thisServerID.first() ^ self->ssPairID.get().first(), + self->thisServerID.second() ^ self->ssPairID.get().second())); + } + })); loop { choose { From 5a8adca1f7a7283a4f225f78a3c5b8143c80d535 Mon Sep 17 00:00:00 2001 From: Xiaoxi Wang Date: Tue, 25 Oct 2022 10:45:21 -0700 Subject: [PATCH 31/95] solve review comments: mark const; add comments; template abbreviation --- fdbclient/include/fdbclient/NativeAPI.actor.h | 3 ++- fdbserver/DataDistribution.actor.cpp | 14 ++++++++++++++ .../include/fdbserver/DataDistribution.actor.h | 14 +------------- fdbserver/include/fdbserver/MockGlobalState.h | 9 +++++---- .../fdbserver/ShardsAffectedByTeamFailure.h | 4 ++-- fdbserver/include/fdbserver/StorageMetrics.actor.h | 4 ++-- fdbserver/storageserver.actor.cpp | 10 +++++----- 7 files changed, 31 insertions(+), 27 deletions(-) diff --git a/fdbclient/include/fdbclient/NativeAPI.actor.h b/fdbclient/include/fdbclient/NativeAPI.actor.h index d1f4860f23..642a4e747a 100644 --- a/fdbclient/include/fdbclient/NativeAPI.actor.h +++ b/fdbclient/include/fdbclient/NativeAPI.actor.h @@ -602,7 +602,8 @@ ACTOR Future> waitStorageMetricsWithLocation(TenantInfo StorageMetrics permittedError); // Return the suggested split points from storage server.The locations tell which interface should -// serve the request. The +// serve the request. `limit` is the current estimated storage metrics of `keys`.The returned points, if present, +// guarantee the metrics of split result is within limit. ACTOR Future>>> splitStorageMetricsWithLocations( std::vector locations, KeyRange keys, diff --git a/fdbserver/DataDistribution.actor.cpp b/fdbserver/DataDistribution.actor.cpp index 9f01a1b2be..10b168b7b3 100644 --- a/fdbserver/DataDistribution.actor.cpp +++ b/fdbserver/DataDistribution.actor.cpp @@ -53,6 +53,20 @@ #include "fdbserver/DDSharedContext.h" #include "flow/actorcompiler.h" // This must be the last #include. +ShardSizeBounds ShardSizeBounds::shardSizeBoundsBeforeTrack() { + return ShardSizeBounds{ + .max = StorageMetrics{ .bytes = -1, + .bytesPerKSecond = StorageMetrics::infinity, + .iosPerKSecond = StorageMetrics::infinity, + .bytesReadPerKSecond = StorageMetrics::infinity }, + .min = StorageMetrics{ .bytes = -1, .bytesPerKSecond = 0, .iosPerKSecond = 0, .bytesReadPerKSecond = 0 }, + .permittedError = StorageMetrics{ .bytes = -1, + .bytesPerKSecond = StorageMetrics::infinity, + .iosPerKSecond = StorageMetrics::infinity, + .bytesReadPerKSecond = StorageMetrics::infinity } + }; +} + struct DDAudit { DDAudit(UID id, KeyRange range, AuditType type) : id(id), range(range), type(type), auditMap(AuditPhase::Invalid, allKeys.end), actors(true) {} diff --git a/fdbserver/include/fdbserver/DataDistribution.actor.h b/fdbserver/include/fdbserver/DataDistribution.actor.h index 2e77d07459..ff33386233 100644 --- a/fdbserver/include/fdbserver/DataDistribution.actor.h +++ b/fdbserver/include/fdbserver/DataDistribution.actor.h @@ -477,19 +477,7 @@ struct ShardSizeBounds { return max == rhs.max && min == rhs.min && permittedError == rhs.permittedError; } - static ShardSizeBounds shardSizeBoundsBeforeTrack() { - return ShardSizeBounds{ - .max = StorageMetrics{ .bytes = -1, - .bytesPerKSecond = StorageMetrics::infinity, - .iosPerKSecond = StorageMetrics::infinity, - .bytesReadPerKSecond = StorageMetrics::infinity }, - .min = StorageMetrics{ .bytes = -1, .bytesPerKSecond = 0, .iosPerKSecond = 0, .bytesReadPerKSecond = 0 }, - .permittedError = StorageMetrics{ .bytes = -1, - .bytesPerKSecond = StorageMetrics::infinity, - .iosPerKSecond = StorageMetrics::infinity, - .bytesReadPerKSecond = StorageMetrics::infinity } - }; - } + static ShardSizeBounds shardSizeBoundsBeforeTrack(); }; // Gets the permitted size and IO bounds for a shard diff --git a/fdbserver/include/fdbserver/MockGlobalState.h b/fdbserver/include/fdbserver/MockGlobalState.h index a404f24027..5f6109626d 100644 --- a/fdbserver/include/fdbserver/MockGlobalState.h +++ b/fdbserver/include/fdbserver/MockGlobalState.h @@ -113,11 +113,12 @@ public: void getStorageMetrics(const GetStorageMetricsRequest& req) override; template - using isLoadBalancedReply = std::is_base_of; + static constexpr bool isLoadBalancedReply = std::is_base_of_v; template - typename std::enable_if::value, void>::type - sendErrorWithPenalty(const ReplyPromise& promise, const Error& err, double penalty) { + typename std::enable_if_t, void> sendErrorWithPenalty(const ReplyPromise& promise, + const Error& err, + double penalty) { Reply reply; reply.error = err; reply.penalty = penalty; @@ -125,7 +126,7 @@ public: } template - typename std::enable_if::value, void>::type + typename std::enable_if_t, void> sendErrorWithPenalty(const ReplyPromise& promise, const Error& err, double) { promise.sendError(err); } diff --git a/fdbserver/include/fdbserver/ShardsAffectedByTeamFailure.h b/fdbserver/include/fdbserver/ShardsAffectedByTeamFailure.h index 9055098bc7..7b674510d4 100644 --- a/fdbserver/include/fdbserver/ShardsAffectedByTeamFailure.h +++ b/fdbserver/include/fdbserver/ShardsAffectedByTeamFailure.h @@ -80,8 +80,8 @@ public: bool hasShards(Team team) const; // The first element of the pair is either the source for non-moving shards or the destination team for in-flight - // shards The second element of the pair is all previous sources for in-flight shards. This function only return the - // teams for the first shard in [keys.begin, keys.end) + // shards. The second element of the pair is all previous sources for in-flight shards. This function only returns + // the teams for the first shard in [keys.begin, keys.end) std::pair, std::vector> getTeamsForFirstShard(KeyRangeRef keys); std::pair, std::vector> getTeamsFor(KeyRef key); diff --git a/fdbserver/include/fdbserver/StorageMetrics.actor.h b/fdbserver/include/fdbserver/StorageMetrics.actor.h index 552db2c6f7..dc518cf318 100644 --- a/fdbserver/include/fdbserver/StorageMetrics.actor.h +++ b/fdbserver/include/fdbserver/StorageMetrics.actor.h @@ -163,9 +163,9 @@ public: StorageServerMetrics metrics; // penalty used by loadBalance() to balance requests among service instances - virtual double getPenalty() { return 1; } + virtual double getPenalty() const { return 1; } - virtual bool isReadable(KeyRangeRef const& keys) { return true; } + virtual bool isReadable(KeyRangeRef const& keys) const { return true; } virtual void addActor(Future future) = 0; diff --git a/fdbserver/storageserver.actor.cpp b/fdbserver/storageserver.actor.cpp index 1d8fb40c8f..1472b4dcaa 100644 --- a/fdbserver/storageserver.actor.cpp +++ b/fdbserver/storageserver.actor.cpp @@ -807,8 +807,8 @@ public: VersionedData const& data() const { return versionedData; } VersionedData& mutableData() { return versionedData; } - double old_rate = 1.0; - double currentRate() { + mutable double old_rate = 1.0; + double currentRate() const { auto versionLag = version.get() - durableVersion.get(); double res; if (versionLag >= SERVER_KNOBS->STORAGE_DURABILITY_LAG_HARD_MAX) { @@ -1379,7 +1379,7 @@ public: // This is the maximum version that might be read from storage (the minimum version is durableVersion) Version storageVersion() const { return oldestVersion.get(); } - bool isReadable(KeyRangeRef const& keys) override { + bool isReadable(KeyRangeRef const& keys) const override { auto sh = shards.intersectingRanges(keys); for (auto i = sh.begin(); i != sh.end(); ++i) if (!i->value()->isReadable()) @@ -1405,10 +1405,10 @@ public: } } - Counter::Value queueSize() { return counters.bytesInput.getValue() - counters.bytesDurable.getValue(); } + Counter::Value queueSize() const { return counters.bytesInput.getValue() - counters.bytesDurable.getValue(); } // penalty used by loadBalance() to balance requests among SSes. We prefer SS with less write queue size. - double getPenalty() override { + double getPenalty() const override { return std::max(std::max(1.0, (queueSize() - (SERVER_KNOBS->TARGET_BYTES_PER_STORAGE_SERVER - 2.0 * SERVER_KNOBS->SPRING_BYTES_STORAGE_SERVER)) / From 31a48f404613221e42c9379bc6004377e34eee45 Mon Sep 17 00:00:00 2001 From: sfc-gh-tclinkenbeard Date: Tue, 25 Oct 2022 13:10:15 -0700 Subject: [PATCH 32/95] Disable GLOBAL_TAG_THROTTLING by default --- fdbclient/ServerKnobs.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/fdbclient/ServerKnobs.cpp b/fdbclient/ServerKnobs.cpp index 8dda15b584..009891f664 100644 --- a/fdbclient/ServerKnobs.cpp +++ b/fdbclient/ServerKnobs.cpp @@ -727,7 +727,7 @@ void ServerKnobs::initialize(Randomize randomize, ClientKnobs* clientKnobs, IsSi init( TAG_THROTTLE_EXPIRED_CLEANUP_INTERVAL, 30.0 ); if(randomize && BUGGIFY) TAG_THROTTLE_EXPIRED_CLEANUP_INTERVAL = 1.0; init( AUTO_TAG_THROTTLING_ENABLED, true ); if(randomize && BUGGIFY) AUTO_TAG_THROTTLING_ENABLED = false; init( SS_THROTTLE_TAGS_TRACKED, 1 ); if(randomize && BUGGIFY) SS_THROTTLE_TAGS_TRACKED = deterministicRandom()->randomInt(1, 10); - init( GLOBAL_TAG_THROTTLING, true ); if(isSimulated) GLOBAL_TAG_THROTTLING = deterministicRandom()->coinflip(); + init( GLOBAL_TAG_THROTTLING, false ); if(isSimulated) GLOBAL_TAG_THROTTLING = deterministicRandom()->coinflip(); init( ENFORCE_TAG_THROTTLING_ON_PROXIES, GLOBAL_TAG_THROTTLING ); init( GLOBAL_TAG_THROTTLING_MIN_RATE, 1.0 ); init( GLOBAL_TAG_THROTTLING_FOLDING_TIME, 10.0 ); From ca0b068f2b37bb65fbe866ac2a44e81fca28c148 Mon Sep 17 00:00:00 2001 From: Trevor Clinkenbeard Date: Tue, 25 Oct 2022 13:14:10 -0700 Subject: [PATCH 33/95] Remove unnecessary forward declaration Co-authored-by: Markus Pilman --- fdbrpc/include/fdbrpc/Stats.h | 2 -- 1 file changed, 2 deletions(-) diff --git a/fdbrpc/include/fdbrpc/Stats.h b/fdbrpc/include/fdbrpc/Stats.h index 10e430c317..5e16e1cd4a 100644 --- a/fdbrpc/include/fdbrpc/Stats.h +++ b/fdbrpc/include/fdbrpc/Stats.h @@ -67,8 +67,6 @@ struct Traceable : std::true_type { } }; -class CounterCollectionImpl; - class CounterCollection { friend class CounterCollectionImpl; From ad2888423cae73202892cde9ba464365ce81a198 Mon Sep 17 00:00:00 2001 From: Markus Pilman Date: Tue, 25 Oct 2022 14:48:04 -0600 Subject: [PATCH 34/95] ignore injected errors in old fdb versions --- contrib/TestHarness2/test_harness/summarize.py | 3 +++ 1 file changed, 3 insertions(+) diff --git a/contrib/TestHarness2/test_harness/summarize.py b/contrib/TestHarness2/test_harness/summarize.py index 3629fa7d43..3ecfa2243b 100644 --- a/contrib/TestHarness2/test_harness/summarize.py +++ b/contrib/TestHarness2/test_harness/summarize.py @@ -569,6 +569,9 @@ class Summary: self.handler.add_handler(('Severity', '30'), parse_warning) def parse_error(attrs: Dict[str, str]): + if 'ErrorIsInjectedFault' in attrs and attrs['ErrorIsInjectedFault'].lower() in ['1', 'true']: + # ignore injected errors. In newer fdb versions these will have a lower severity + return self.errors += 1 self.error = True if self.errors > config.max_errors: From 0c67eed28746d66c80973cb3f22e58915fce0289 Mon Sep 17 00:00:00 2001 From: Steve Atherton Date: Tue, 25 Oct 2022 15:38:19 -0700 Subject: [PATCH 35/95] GetMappedRange is not subject to data fetch control knob. --- fdbserver/storageserver.actor.cpp | 3 --- 1 file changed, 3 deletions(-) diff --git a/fdbserver/storageserver.actor.cpp b/fdbserver/storageserver.actor.cpp index b4c8f629a3..0c3397c4f5 100644 --- a/fdbserver/storageserver.actor.cpp +++ b/fdbserver/storageserver.actor.cpp @@ -4834,9 +4834,6 @@ ACTOR Future getMappedKeyValuesQ(StorageServer* data, GetMappedKeyValuesRe // Active load balancing runs at a very high priority (to obtain accurate queue lengths) // so we need to downgrade here wait(data->getQueryDelay()); - if (!SERVER_KNOBS->FETCH_KEYS_LOWER_PRIORITY && type == ReadType::FETCH) { - type = ReadType::NORMAL; - } state PriorityMultiLock::Lock lock = wait(data->ssLock.lock(data->readPriorityRanks[(int)type])); From f4e74811a28aaa932a2864f6853ac3eac3ec14cf Mon Sep 17 00:00:00 2001 From: Steve Atherton Date: Tue, 25 Oct 2022 15:39:06 -0700 Subject: [PATCH 36/95] Rename var for clarity. --- 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 0c3397c4f5..f1f1bb43a7 100644 --- a/fdbserver/storageserver.actor.cpp +++ b/fdbserver/storageserver.actor.cpp @@ -5254,11 +5254,11 @@ ACTOR Future getKeyQ(StorageServer* data, GetKeyRequest req) { } state int64_t resultSize = 0; state ReadOptions options; - state ReadType type = ReadType::NORMAL; + state ReadType readType = ReadType::NORMAL; if (req.options.present()) { options = req.options.get(); - type = options.type; + readType = options.type; } getCurrentLineage()->modify(&TransactionLineage::txID) = req.spanContext.traceID; @@ -5272,7 +5272,7 @@ ACTOR Future getKeyQ(StorageServer* data, GetKeyRequest req) { // so we need to downgrade here wait(data->getQueryDelay()); - wait(store(lock, data->ssLock.lock(data->readPriorityRanks[(int)type]))); + wait(store(lock, data->ssLock.lock(data->readPriorityRanks[(int)readType]))); // Track time from requestTime through now as read queueing wait time state double queueWaitEnd = g_network->timer(); From 36d9de90724b270f73993ffa1f11b7c67c220e95 Mon Sep 17 00:00:00 2001 From: Xiaoxi Wang Date: Tue, 25 Oct 2022 15:43:24 -0700 Subject: [PATCH 37/95] change UNREACHABLE to ASSERT(false); change function name --- fdbclient/NativeAPI.actor.cpp | 1 + fdbserver/DDTxnProcessor.actor.cpp | 2 +- fdbserver/MockGlobalState.actor.cpp | 13 ++++++++++--- fdbserver/include/fdbserver/MockGlobalState.h | 2 +- 4 files changed, 13 insertions(+), 5 deletions(-) diff --git a/fdbclient/NativeAPI.actor.cpp b/fdbclient/NativeAPI.actor.cpp index 4aece2f05f..84111ecd98 100644 --- a/fdbclient/NativeAPI.actor.cpp +++ b/fdbclient/NativeAPI.actor.cpp @@ -7736,6 +7736,7 @@ ACTOR Future> waitStorageMetricsWithLocation(TenantInfo StorageMetrics x = wait(fx); return x; } catch (Error& e) { + TraceEvent(SevDebug, "WaitStorageMetricsError").error(e); if (e.code() != error_code_wrong_shard_server && e.code() != error_code_all_alternatives_failed) { TraceEvent(SevError, "WaitStorageMetricsError").error(e); throw; diff --git a/fdbserver/DDTxnProcessor.actor.cpp b/fdbserver/DDTxnProcessor.actor.cpp index 2c4c695c23..2770345a72 100644 --- a/fdbserver/DDTxnProcessor.actor.cpp +++ b/fdbserver/DDTxnProcessor.actor.cpp @@ -816,7 +816,7 @@ Future DDMockTxnProcessor::removeStorageServer(const UID& serverID, const Optional& tssPairID, const MoveKeysLock& lock, const DDEnabledState* ddEnabledState) const { - ASSERT(mgs->allShardRemovedFromServer(serverID)); + ASSERT(mgs->allShardsRemovedFromServer(serverID)); mgs->allServers.erase(serverID); return Void(); } diff --git a/fdbserver/MockGlobalState.actor.cpp b/fdbserver/MockGlobalState.actor.cpp index 1e03b71e85..84b28e6d18 100644 --- a/fdbserver/MockGlobalState.actor.cpp +++ b/fdbserver/MockGlobalState.actor.cpp @@ -240,6 +240,13 @@ Future MockStorageServer::waitMetricsTenantAware(const WaitMetricsRequest& void MockStorageServer::getStorageMetrics(const GetStorageMetricsRequest& req) {} Future MockStorageServer::run() { + ssi.locality = LocalityData(Optional>(), + Standalone(deterministicRandom()->randomUniqueID().toString()), + Standalone(deterministicRandom()->randomUniqueID().toString()), + Optional>()); + ssi.initEndpoints(); + ssi.startAcceptingRequests(); + TraceEvent("MockStorageServerStart").detail("Address", ssi.address()); return serveStorageMetricsRequests(this, ssi); } @@ -298,7 +305,7 @@ bool MockGlobalState::serverIsDestForShard(const UID& serverId, KeyRangeRef shar }); } -bool MockGlobalState::allShardRemovedFromServer(const UID& serverId) { +bool MockGlobalState::allShardsRemovedFromServer(const UID& serverId) { return allServers.count(serverId) && shardMapping->getNumberOfShards(serverId) == 0; } @@ -362,7 +369,7 @@ Future> MockGlobalState::getKeyRangeLocations( if (reverse) { // DD never ask for backward range. - UNREACHABLE(); + ASSERT(false); } ASSERT(keys.begin < keys.end); @@ -591,7 +598,7 @@ TEST_CASE("/MockGlobalState/MockStorageServer/WaitStorageMetricsRequest") { testConfig.minimumReplication = 1; testConfig.logAntiQuorum = 0; DatabaseConfiguration dbConfig = generateNormalDatabaseConfiguration(testConfig); - TraceEvent("UnitTestDbConfig").detail("Config", dbConfig.toString()); + TraceEvent("WaitStorageMetricsRequestUnitTestConfig").detail("Config", dbConfig.toString()); state std::shared_ptr mgs = std::make_shared(); mgs->initializeAsEmptyDatabaseMGS(dbConfig); diff --git a/fdbserver/include/fdbserver/MockGlobalState.h b/fdbserver/include/fdbserver/MockGlobalState.h index 5f6109626d..ac984e9069 100644 --- a/fdbserver/include/fdbserver/MockGlobalState.h +++ b/fdbserver/include/fdbserver/MockGlobalState.h @@ -197,7 +197,7 @@ public: * * mgs.shardMapping doesn’t have any information about X * * mgs.allServer[X] is existed */ - bool allShardRemovedFromServer(const UID& serverId); + bool allShardsRemovedFromServer(const UID& serverId); // SOMEDAY: NativeAPI::waitStorageMetrics should share the code in the future, this is a simpler version of it Future, int>> waitStorageMetrics(KeyRange const& keys, From 0a5e59675878bff9277bd9c60667de3dd7208630 Mon Sep 17 00:00:00 2001 From: Xiaoxi Wang Date: Tue, 25 Oct 2022 16:43:00 -0700 Subject: [PATCH 38/95] fix network failure check in unit test --- fdbserver/MockGlobalState.actor.cpp | 1 + 1 file changed, 1 insertion(+) diff --git a/fdbserver/MockGlobalState.actor.cpp b/fdbserver/MockGlobalState.actor.cpp index 84b28e6d18..8de995fd2b 100644 --- a/fdbserver/MockGlobalState.actor.cpp +++ b/fdbserver/MockGlobalState.actor.cpp @@ -607,6 +607,7 @@ TEST_CASE("/MockGlobalState/MockStorageServer/WaitStorageMetricsRequest") { ActorCollection* ptr = &actors; // get around ACTOR syntax restriction std::for_each(mgs->allServers.begin(), mgs->allServers.end(), [ptr](auto& server) { ptr->add(server.second.run()); + IFailureMonitor::failureMonitor().setStatus(server.second.ssi.address(), FailureStatus(false)); server.second.metrics.byteSample.sample.insert("something"_sr, 500000); }); From ee269ebfe96e16d51c855811c9918053f61e3a8c Mon Sep 17 00:00:00 2001 From: Steve Atherton Date: Tue, 25 Oct 2022 19:39:36 -0700 Subject: [PATCH 39/95] Rewrote PriorityMultiLock::kill() to be simpler and fix destruction order and be safe to repeat. Destructor calls kill() to avoid lifetime issues. --- flow/include/flow/PriorityMultiLock.actor.h | 67 +++++++++++++-------- 1 file changed, 41 insertions(+), 26 deletions(-) diff --git a/flow/include/flow/PriorityMultiLock.actor.h b/flow/include/flow/PriorityMultiLock.actor.h index 95b1c267c7..07949fa595 100644 --- a/flow/include/flow/PriorityMultiLock.actor.h +++ b/flow/include/flow/PriorityMultiLock.actor.h @@ -33,8 +33,10 @@ #define PRIORITYMULTILOCK_DEBUG 0 -#if PRIORITYMULTILOCK_DEBUG -#define pml_debug_printf(...) printf(__VA_ARGS__) +#if PRIORITYMULTILOCK_DEBUG || !defined(NO_INTELLISENSE) +#define pml_debug_printf(...) \ + if (now() > 0) \ + printf(__VA_ARGS__) #else #define pml_debug_printf(...) #endif @@ -82,7 +84,7 @@ public: : PriorityMultiLock(concurrency, parseStringToVector(launchLimits, ',')) {} PriorityMultiLock(int concurrency, std::vector launchLimitsByPriority) - : concurrency(concurrency), available(concurrency), waiting(0), totalActiveLaunchLimits(0) { + : concurrency(concurrency), available(concurrency), waiting(0), totalActiveLaunchLimits(0), releaseDebugID(0) { priorities.resize(launchLimitsByPriority.size()); for (int i = 0; i < priorities.size(); ++i) { @@ -92,7 +94,7 @@ public: fRunner = runner(this); } - ~PriorityMultiLock() {} + ~PriorityMultiLock() { kill(); } Future lock(int priority = 0) { Priority& p = priorities[priority]; @@ -125,14 +127,11 @@ public: } void kill() { - for (int i = 0; i < runners.size(); ++i) { - if (!runners[i].isReady()) { - runners[i].cancel(); - } - } + brokenOnDestruct.reset(); + // handleRelease will not free up any execution slots when it ends via cancel + fRunner.cancel(); + available = 0; runners.clear(); - brokenOnDestruct.sendError(broken_promise()); - waiting = 0; priorities.clear(); } @@ -144,23 +143,31 @@ public: } } - std::string s = - format("{ ptr=%p concurrency=%d available=%d running=%d waiting=%d runnersQueue=%d runnersDone=%d ", - this, - concurrency, - available, - concurrency - available, - waiting, - runners.size(), - runnersDone); + std::string s = format("{ ptr=%p concurrency=%d available=%d running=%d waiting=%d runnersQueue=%d " + "runnersDone=%d activeLimits=%d ", + this, + concurrency, + available, + concurrency - available, + waiting, + runners.size(), + runnersDone, + totalActiveLaunchLimits); for (int i = 0; i < priorities.size(); ++i) { s += format("p%d:{%s} ", i, priorities[i].toString(this).c_str()); } s += "}"; + + if (concurrency - available != runners.size() - runnersDone) { + pml_debug_printf("%s\n", s.c_str()); + ASSERT_EQ(concurrency - available, runners.size() - runnersDone); + } + return s; } + int maxPriority() const { return priorities.size() - 1; } int totalWaiters() const { return waiting; } @@ -179,9 +186,8 @@ public: private: struct Waiter { - Waiter() : queuedTime(now()) {} + Waiter() {} Promise lockPromise; - double queuedTime; }; // Total execution slots allowed across all priorities @@ -223,23 +229,32 @@ private: AsyncTrigger wakeRunner; Promise brokenOnDestruct; + // Used for debugging, can roll over without issue + unsigned int releaseDebugID; + ACTOR static Future handleRelease(PriorityMultiLock* self, Future f, Priority* priority) { + state [[maybe_unused]] unsigned int id = self->releaseDebugID++; + + pml_debug_printf("%f handleRelease self=%p id=%u start \n", now(), self, id); try { wait(f); + pml_debug_printf("%f handleRelease self=%p id=%u success\n", now(), self, id); } catch (Error& e) { + pml_debug_printf("%f handleRelease self=%p id=%u error %s\n", now(), self, id, e.what()); if (e.code() == error_code_actor_cancelled) { throw; } } - ++self->available; - priority->runners -= 1; - pml_debug_printf("lock release line %d priority %d %s\n", __LINE__, (int)(priority - &self->priorities.front()), self->toString().c_str()); + pml_debug_printf("%f handleRelease self=%p id=%u releasing\n", now(), self, id); + ++self->available; + priority->runners -= 1; + // If there are any waiters or if the runners array is getting large, trigger the runner loop if (self->waiting > 0 || self->runners.size() > 1000) { self->wakeRunner.trigger(); @@ -282,7 +297,7 @@ private: "runner loop waitTrigger line %d priority=%d %s\n", __LINE__, priority, self->toString().c_str()); wait(self->wakeRunner.onTrigger()); pml_debug_printf( - "runner loop wake line %d priority=%d %s\n", __LINE__, priority, self->toString().c_str()); + "%f runner loop wake line %d priority=%d %s\n", now(), __LINE__, priority, self->toString().c_str()); if (++sinceYield == 100) { sinceYield = 0; From 2c07783f9954c82358e93ecb239acec33efccf40 Mon Sep 17 00:00:00 2001 From: Steve Atherton Date: Tue, 25 Oct 2022 19:43:29 -0700 Subject: [PATCH 40/95] Rename type to readType for clarity. --- fdbserver/storageserver.actor.cpp | 22 +++++++++++----------- 1 file changed, 11 insertions(+), 11 deletions(-) diff --git a/fdbserver/storageserver.actor.cpp b/fdbserver/storageserver.actor.cpp index f1f1bb43a7..b2402baf91 100644 --- a/fdbserver/storageserver.actor.cpp +++ b/fdbserver/storageserver.actor.cpp @@ -1879,7 +1879,7 @@ ACTOR Future getValueQ(StorageServer* data, GetValueRequest req) { // Temporarily disabled -- this path is hit a lot // getCurrentLineage()->modify(&TransactionLineage::txID) = req.spanContext.first(); - state ReadType type = req.options.present() ? req.options.get().type : ReadType::NORMAL; + state ReadType readType = req.options.present() ? req.options.get().type : ReadType::NORMAL; try { ++data->counters.getValueQueries; @@ -1891,7 +1891,7 @@ ACTOR Future getValueQ(StorageServer* data, GetValueRequest req) { // so we need to downgrade here wait(data->getQueryDelay()); - wait(store(lock, data->ssLock.lock(data->readPriorityRanks[(int)type]))); + wait(store(lock, data->ssLock.lock(data->readPriorityRanks[(int)readType]))); // Track time from requestTime through now as read queueing wait time state double queueWaitEnd = g_network->timer(); @@ -3753,7 +3753,7 @@ ACTOR Future getKeyValuesQ(StorageServer* data, GetKeyValuesRequest req) state Span span("SS:getKeyValues"_loc, req.spanContext); state int64_t resultSize = 0; state Optional options = req.options; - state ReadType type = options.present() ? options.get().type : ReadType::NORMAL; + state ReadType readType = options.present() ? options.get().type : ReadType::NORMAL; if (req.tenantInfo.name.present()) { span.addAttribute("tenant"_sr, req.tenantInfo.name.get()); @@ -3769,11 +3769,11 @@ ACTOR Future getKeyValuesQ(StorageServer* data, GetKeyValuesRequest req) // Active load balancing runs at a very high priority (to obtain accurate queue lengths) // so we need to downgrade here wait(data->getQueryDelay()); - if (!SERVER_KNOBS->FETCH_KEYS_LOWER_PRIORITY && type == ReadType::FETCH) { - type = ReadType::NORMAL; + if (!SERVER_KNOBS->FETCH_KEYS_LOWER_PRIORITY && readType == ReadType::FETCH) { + readType = ReadType::NORMAL; } - state PriorityMultiLock::Lock lock = wait(data->ssLock.lock(data->readPriorityRanks[(int)type])); + state PriorityMultiLock::Lock lock = wait(data->ssLock.lock(data->readPriorityRanks[(int)readType])); // Track time from requestTime through now as read queueing wait time state double queueWaitEnd = g_network->timer(); @@ -4818,7 +4818,7 @@ ACTOR Future getMappedKeyValuesQ(StorageServer* data, GetMappedKeyValuesRe state Span span("SS:getMappedKeyValues"_loc, req.spanContext); state int64_t resultSize = 0; state Optional options = req.options; - state ReadType type = options.present() ? options.get().type : ReadType::NORMAL; + state ReadType readType = options.present() ? options.get().type : ReadType::NORMAL; if (req.tenantInfo.name.present()) { span.addAttribute("tenant"_sr, req.tenantInfo.name.get()); @@ -5033,7 +5033,7 @@ ACTOR Future getKeyValuesStreamQ(StorageServer* data, GetKeyValuesStreamRe state Span span("SS:getKeyValuesStream"_loc, req.spanContext); state int64_t resultSize = 0; state Optional options = req.options; - state ReadType type = options.present() ? options.get().type : ReadType::NORMAL; + state ReadType readType = options.present() ? options.get().type : ReadType::NORMAL; if (req.tenantInfo.name.present()) { span.addAttribute("tenant"_sr, req.tenantInfo.name.get()); @@ -5048,12 +5048,12 @@ ACTOR Future getKeyValuesStreamQ(StorageServer* data, GetKeyValuesStreamRe // Active load balancing runs at a very high priority (to obtain accurate queue lengths) // so we need to downgrade here wait(delay(0, TaskPriority::DefaultEndpoint)); - if (!SERVER_KNOBS->FETCH_KEYS_LOWER_PRIORITY && type == ReadType::FETCH) { - type = ReadType::NORMAL; + if (!SERVER_KNOBS->FETCH_KEYS_LOWER_PRIORITY && readType == ReadType::FETCH) { + readType = ReadType::NORMAL; } - state int readPriority = data->readPriorityRanks[(int)type]; state PriorityMultiLock::Lock lock = wait(data->ssLock.lock(readPriority)); + state int readPriority = data->readPriorityRanks[(int)readType]; try { if (req.options.present() && req.options.get().debugID.present()) From 6973f8877146ec724b7dc205a5f2c8b8a4320a2a Mon Sep 17 00:00:00 2001 From: Steve Atherton Date: Tue, 25 Oct 2022 20:43:58 -0700 Subject: [PATCH 41/95] Fix scope of read priority lock in getMappedRange() to only cover the getRange() operation and unlock before the subqueries because they will route though Get*Q() which will each require a lock. Pass the original ReadOptions through to the subquery requests. --- fdbserver/storageserver.actor.cpp | 11 +++++++---- 1 file changed, 7 insertions(+), 4 deletions(-) diff --git a/fdbserver/storageserver.actor.cpp b/fdbserver/storageserver.actor.cpp index b2402baf91..6e15971f05 100644 --- a/fdbserver/storageserver.actor.cpp +++ b/fdbserver/storageserver.actor.cpp @@ -3316,7 +3316,6 @@ ACTOR Future quickGetValue(StorageServer* data, state GetValueReqAndResultRef getValue; state double getValueStart = g_network->timer(); getValue.key = key; - state Optional options = pOriginalReq->options; if (data->shards[key]->isReadable()) { try { @@ -3326,7 +3325,7 @@ ACTOR Future quickGetValue(StorageServer* data, key, version, pOriginalReq->tags, - options, + pOriginalReq->options, VersionVector()); // Note that it does not use readGuard to avoid server being overloaded here. Throttling is enforced at the // original request level, rather than individual underlying lookups. The reason is that throttle any @@ -3967,6 +3966,7 @@ ACTOR Future quickGetKeyValues( // TODO: Use a lower level API may be better? GetKeyValuesRequest req; req.spanContext = pOriginalReq->spanContext; + req.options = pOriginalReq->options; req.arena = *a; req.begin = getRange.begin; req.end = getRange.end; @@ -4835,8 +4835,6 @@ ACTOR Future getMappedKeyValuesQ(StorageServer* data, GetMappedKeyValuesRe // so we need to downgrade here wait(data->getQueryDelay()); - state PriorityMultiLock::Lock lock = wait(data->ssLock.lock(data->readPriorityRanks[(int)type])); - // Track time from requestTime through now as read queueing wait time state double queueWaitEnd = g_network->timer(); data->counters.readQueueWaitSample.addMeasurement(queueWaitEnd - req.requestTime()); @@ -4943,6 +4941,10 @@ ACTOR Future getMappedKeyValuesQ(StorageServer* data, GetMappedKeyValuesRe } else { state int remainingLimitBytes = req.limitBytes; + // Only take the ssLock for the readRange operation and unlock before the subqueries because each + // subquery will route back to getValueQ or getKeyValuesQ with a new request having the same + // read options which will each acquire the ssLock. + state PriorityMultiLock::Lock lock = wait(data->ssLock.lock(data->readPriorityRanks[(int)readType])); GetKeyValuesReply getKeyValuesReply = wait(readRange(data, version, KeyRangeRef(begin, end), @@ -4951,6 +4953,7 @@ ACTOR Future getMappedKeyValuesQ(StorageServer* data, GetMappedKeyValuesRe span.context, options, tenantPrefix)); + lock.release(); state GetMappedKeyValuesReply r; try { From e4116f8aee49fe95f4586259b71d04ee584d7576 Mon Sep 17 00:00:00 2001 From: Aaron Molitor Date: Tue, 25 Oct 2022 19:19:54 -0500 Subject: [PATCH 42/95] cleanup shell script, remove set -x, add more detailed logging --- packaging/docker/run_ycsb.sh | 48 ++++++++++++++++++++++++++---------- 1 file changed, 35 insertions(+), 13 deletions(-) diff --git a/packaging/docker/run_ycsb.sh b/packaging/docker/run_ycsb.sh index deb065a728..bfe3e8df6e 100755 --- a/packaging/docker/run_ycsb.sh +++ b/packaging/docker/run_ycsb.sh @@ -1,22 +1,44 @@ #!/usr/bin/env bash -set -Eeuxo pipefail +set -Eeuo pipefail + +function logg () { + printf "##### $(date +'%Y-%m-%dT%H:%M:%SZ') # %-56.55s #####\n" "${1}" +} + +function error_exit () { + echo "################################################################################" + logg "${0} FAILED" + logg "RUN_ID: ${RUN_ID}" + logg "WORKLOAD: ${WORKLOAD}" + logg "ENVIRONMENT IS:" + env + echo "################################################################################" +} + +trap error_exit ERR namespace=$(cat /var/run/secrets/kubernetes.io/serviceaccount/namespace) -POD_NUM=$(echo $POD_NAME | cut -d - -f3) -KEY="ycsb_load_${POD_NUM}_of_${NUM_PODS}_complete" -CLI=$(ls /var/dynamic-conf/bin/*/fdbcli | head -n1) -echo "WAITING FOR ALL PODS TO COME UP" -while [[ $(kubectl get pods -n ${namespace} -l name=ycsb,run=${RUN_ID} --field-selector=status.phase=Running | grep -cv NAME) -lt ${NUM_PODS} ]]; do +logg "WAITING FOR ${NUM_PODS} PODS TO COME UP IN ${namespace}" +while [[ $(kubectl get pods -n "${namespace}" -l name=ycsb,run="${RUN_ID}" --field-selector=status.phase=Running | grep -cv NAME) -lt ${NUM_PODS} ]]; do sleep 1 done -echo "ALL PODS ARE UP" +logg "${NUM_PODS} PODS ARE UP IN ${namespace}" -echo "RUNNING YCSB" -./bin/ycsb.sh ${MODE} foundationdb -s -P workloads/${WORKLOAD} ${YCSB_ARGS} -echo "YCSB FINISHED" +logg "RUNNING YCSB ${WORKLOAD}" +set -x +./bin/ycsb.sh "${MODE}" foundationdb -s -P "workloads/${WORKLOAD}" "${YCSB_ARGS}" +set +x +logg "YCSB ${WORKLOAD} FINISHED" -echo "COPYING HISTOGRAMS TO S3" -aws s3 sync --sse aws:kms --exclude "*" --include "histogram.*" /tmp s3://${BUCKET}/ycsb_histograms/${namespace}/${POD_NAME} -echo "COPYING HISTOGRAMS TO S3 FINISHED" +logg "COPYING HISTOGRAMS TO S3" +set -x +aws s3 sync --sse aws:kms --exclude "*" --include "histogram.*" /tmp "s3://${BUCKET}/ycsb_histograms/${namespace}/${POD_NAME}" +set +x +logg "COPYING HISTOGRAMS TO S3 FINISHED" +echo "################################################################################" +logg "COMPLETED ${0}" +logg "RUN_ID: ${RUN_ID}" +logg "WORKLOAD: ${WORKLOAD}" +echo "################################################################################" From c2cacb4123f4e004145ccc5334d08d56f3d584de Mon Sep 17 00:00:00 2001 From: Steve Atherton Date: Wed, 26 Oct 2022 01:32:45 -0700 Subject: [PATCH 43/95] Added ReadOptions and read lock support support to ChangeFeed requests and used its DebugID to replace the streamUID. Refactored read lock and read options usage in Storage Server to simplify code and handle ReadOptions to Priority conversion in a single place, fixed a few bugs along the way. Set CacheResult to False for requests created by fetchKeys(). --- fdbclient/NativeAPI.actor.cpp | 49 +++-- fdbclient/include/fdbclient/DatabaseContext.h | 3 +- .../fdbclient/StorageServerInterface.h | 13 +- fdbserver/storageserver.actor.cpp | 203 +++++++++--------- 4 files changed, 147 insertions(+), 121 deletions(-) diff --git a/fdbclient/NativeAPI.actor.cpp b/fdbclient/NativeAPI.actor.cpp index e9dcc975ae..9c88419501 100644 --- a/fdbclient/NativeAPI.actor.cpp +++ b/fdbclient/NativeAPI.actor.cpp @@ -9305,7 +9305,7 @@ void handleTSSChangeFeedMismatch(const ChangeFeedStreamRequest& request, mismatchEvent.detail("EndKey", request.range.end); mismatchEvent.detail("CanReadPopped", request.canReadPopped); mismatchEvent.detail("PopVersion", popVersion); - mismatchEvent.detail("DebugUID", request.debugUID); + mismatchEvent.detail("DebugUID", request.streamUID()); // mismatch info mismatchEvent.detail("MatchesFound", matchesFound); @@ -9331,7 +9331,7 @@ void handleTSSChangeFeedMismatch(const ChangeFeedStreamRequest& request, "TSSMismatchChangeFeedStream"); summaryEvent.detail("TSSID", tssData.tssId) .detail("MismatchId", mismatchUID) - .detail("FeedDebugUID", request.debugUID); + .detail("FeedDebugUID", request.streamUID()); } } } @@ -9884,10 +9884,10 @@ ACTOR Future mergeChangeFeedStream(Reference db, if (replyBufferSize != -1 && req.replyBufferSize < CLIENT_KNOBS->CHANGE_FEED_STREAM_MIN_BYTES) { req.replyBufferSize = CLIENT_KNOBS->CHANGE_FEED_STREAM_MIN_BYTES; } - req.debugUID = deterministicRandom()->randomUniqueID(); - debugUIDs.push_back(req.debugUID); + req.options = ReadOptions(deterministicRandom()->randomUniqueID()); + debugUIDs.push_back(req.streamUID()); mergeCursorUID = - UID(mergeCursorUID.first() ^ req.debugUID.first(), mergeCursorUID.second() ^ req.debugUID.second()); + UID(mergeCursorUID.first() ^ req.streamUID().first(), mergeCursorUID.second() ^ req.streamUID().second()); results->streams.push_back(interfs[i].first.changeFeedStream.getReplyStream(req)); maybeDuplicateTSSChangeFeedStream(req, @@ -10090,7 +10090,8 @@ ACTOR Future singleChangeFeedStream(Reference db, Version* begin, Version end, int replyBufferSize, - bool canReadPopped) { + bool canReadPopped, + ReadOptions readOptions) { state Database cx(db); state ChangeFeedStreamRequest req; state Optional tssData; @@ -10100,10 +10101,13 @@ ACTOR Future singleChangeFeedStream(Reference db, req.range = range; req.canReadPopped = canReadPopped; req.replyBufferSize = replyBufferSize; - req.debugUID = deterministicRandom()->randomUniqueID(); + req.options = readOptions; + if (!req.streamUID().isValid()) { + req.options.get().debugID = readOptions.debugID = deterministicRandom()->randomUniqueID(); + } if (DEBUG_CF_CLIENT_TRACE) { - TraceEvent(SevDebug, "TraceChangeFeedClientSingleCursor", req.debugUID) + TraceEvent(SevDebug, "TraceChangeFeedClientSingleCursor", req.streamUID()) .detail("FeedID", rangeID) .detail("Range", range) .detail("Begin", *begin) @@ -10143,7 +10147,8 @@ ACTOR Future getChangeFeedStreamActor(Reference db, Version end, KeyRange range, int replyBufferSize, - bool canReadPopped) { + bool canReadPopped, + ReadOptions readOptions) { state Database cx(db); state Span span("NAPI:GetChangeFeedStream"_loc); db->usedAnyChangeFeeds = true; @@ -10239,8 +10244,16 @@ ACTOR Future getChangeFeedStreamActor(Reference db, } else { CODE_PROBE(true, "Change feed single cursor"); StorageServerInterface interf = locations[0].locations->getInterface(chosenLocations[0]); - wait(singleChangeFeedStream( - db, interf, range, results, rangeID, &begin, end, replyBufferSize, canReadPopped) || + wait(singleChangeFeedStream(db, + interf, + range, + results, + rangeID, + &begin, + end, + replyBufferSize, + canReadPopped, + readOptions) || cx->connectionFileChanged()); } } catch (Error& e) { @@ -10307,9 +10320,17 @@ Future DatabaseContext::getChangeFeedStream(Reference resu Version end, KeyRange range, int replyBufferSize, - bool canReadPopped) { - return getChangeFeedStreamActor( - Reference::addRef(this), results, rangeID, begin, end, range, replyBufferSize, canReadPopped); + bool canReadPopped, + ReadOptions readOptions) { + return getChangeFeedStreamActor(Reference::addRef(this), + results, + rangeID, + begin, + end, + range, + replyBufferSize, + canReadPopped, + readOptions); } Version OverlappingChangeFeedsInfo::getFeedMetadataVersion(const KeyRangeRef& range) const { diff --git a/fdbclient/include/fdbclient/DatabaseContext.h b/fdbclient/include/fdbclient/DatabaseContext.h index 390873e0ef..6ddf90f69d 100644 --- a/fdbclient/include/fdbclient/DatabaseContext.h +++ b/fdbclient/include/fdbclient/DatabaseContext.h @@ -382,7 +382,8 @@ public: Version end = std::numeric_limits::max(), KeyRange range = allKeys, int replyBufferSize = -1, - bool canReadPopped = true); + bool canReadPopped = true, + ReadOptions readOptions = ReadOptions()); Future getOverlappingChangeFeeds(KeyRangeRef ranges, Version minVersion); Future popChangeFeedMutations(Key rangeID, Version version); diff --git a/fdbclient/include/fdbclient/StorageServerInterface.h b/fdbclient/include/fdbclient/StorageServerInterface.h index b519ffa4b5..40a4fa24b4 100644 --- a/fdbclient/include/fdbclient/StorageServerInterface.h +++ b/fdbclient/include/fdbclient/StorageServerInterface.h @@ -890,16 +890,21 @@ struct ChangeFeedStreamRequest { KeyRange range; int replyBufferSize = -1; bool canReadPopped = true; - UID debugUID; // This is only used for debugging and tracing, but being able to link a client + server side stream - // is so useful for testing, and this is such small overhead compared to streaming large amounts of - // change feed data, it is left in the interface + Optional options; + + UID streamUID() const { + if (options.present()) { + return options.get().debugID.orDefault(UID()); + } + return UID(); + } ReplyPromiseStream reply; ChangeFeedStreamRequest() {} template void serialize(Ar& ar) { - serializer(ar, rangeID, begin, end, range, reply, spanContext, replyBufferSize, canReadPopped, debugUID, arena); + serializer(ar, rangeID, begin, end, range, reply, spanContext, replyBufferSize, canReadPopped, options, arena); } }; diff --git a/fdbserver/storageserver.actor.cpp b/fdbserver/storageserver.actor.cpp index 6e15971f05..292b9caa42 100644 --- a/fdbserver/storageserver.actor.cpp +++ b/fdbserver/storageserver.actor.cpp @@ -1021,6 +1021,13 @@ public: PriorityMultiLock ssLock; std::vector readPriorityRanks; + + Future getReadLock(const Optional& options) { + int readType = (int)(options.present() ? options.get().type : ReadType::NORMAL); + readType = std::clamp(readType, 0, readPriorityRanks.size() - 1); + return ssLock.lock(readPriorityRanks[readType]); + } + FlowLock serveAuditStorageParallelismLock; int64_t instanceID; @@ -1870,7 +1877,6 @@ std::vector StorageServer::getStorageServerShards(KeyRangeRe ACTOR Future getValueQ(StorageServer* data, GetValueRequest req) { state int64_t resultSize = 0; - state PriorityMultiLock::Lock lock; Span span("SS:getValue"_loc, req.spanContext); if (req.tenantInfo.name.present()) { span.addAttribute("tenant"_sr, req.tenantInfo.name.get()); @@ -1879,8 +1885,6 @@ ACTOR Future getValueQ(StorageServer* data, GetValueRequest req) { // Temporarily disabled -- this path is hit a lot // getCurrentLineage()->modify(&TransactionLineage::txID) = req.spanContext.first(); - state ReadType readType = req.options.present() ? req.options.get().type : ReadType::NORMAL; - try { ++data->counters.getValueQueries; ++data->counters.allQueries; @@ -1890,8 +1894,7 @@ ACTOR Future getValueQ(StorageServer* data, GetValueRequest req) { // Active load balancing runs at a very high priority (to obtain accurate queue lengths) // so we need to downgrade here wait(data->getQueryDelay()); - - wait(store(lock, data->ssLock.lock(data->readPriorityRanks[(int)readType]))); + state PriorityMultiLock::Lock lock = wait(data->getReadLock(req.options)); // Track time from requestTime through now as read queueing wait time state double queueWaitEnd = g_network->timer(); @@ -2587,21 +2590,17 @@ static std::deque>::const_iterator searchChan ACTOR Future> getChangeFeedMutations(StorageServer* data, ChangeFeedStreamRequest req, bool inverted, - bool atLatest, - UID streamUID /* for debugging */) { + bool atLatest) { state ChangeFeedStreamReply reply; state ChangeFeedStreamReply memoryReply; state int remainingLimitBytes = CLIENT_KNOBS->REPLY_BYTE_LIMIT; state int remainingDurableBytes = CLIENT_KNOBS->REPLY_BYTE_LIMIT; state Version startVersion = data->version.get(); - // TODO: Change feed reads should probably at least set cacheResult to false, possibly set a different ReadType as - // well, perhaps high priority? - state ReadOptions options; if (DEBUG_CF_TRACE) { TraceEvent(SevDebug, "TraceChangeFeedMutationsBegin", data->thisServerID) .detail("FeedID", req.rangeID) - .detail("StreamUID", streamUID) + .detail("StreamUID", req.streamUID()) .detail("Range", req.range) .detail("Begin", req.begin) .detail("End", req.end) @@ -2639,7 +2638,7 @@ ACTOR Future> getChangeFeedMutations(Stor if (DEBUG_CF_TRACE) { TraceEvent(SevDebug, "TraceChangeFeedMutationsDetails", data->thisServerID) .detail("FeedID", req.rangeID) - .detail("StreamUID", streamUID) + .detail("StreamUID", req.streamUID()) .detail("Range", req.range) .detail("Begin", req.begin) .detail("End", req.end) @@ -2690,13 +2689,15 @@ ACTOR Future> getChangeFeedMutations(Stor } wait(data->changeFeedDiskReadsLock.take(TaskPriority::DefaultYield)); + state PriorityMultiLock::Lock ssReadLock = wait(data->getReadLock(req.options)); state FlowLock::Releaser holdingDiskReadsLock(data->changeFeedDiskReadsLock); RangeResult res = wait( data->storage.readRange(KeyRangeRef(changeFeedDurableKey(req.rangeID, std::max(req.begin, emptyVersion)), changeFeedDurableKey(req.rangeID, req.end)), 1 << 30, remainingDurableBytes, - options)); + req.options)); + ssReadLock.release(); holdingDiskReadsLock.release(); data->counters.kvScanBytes += res.logicalSize(); @@ -2740,7 +2741,7 @@ ACTOR Future> getChangeFeedMutations(Stor "is {4}) (emptyVersion={5}, emptyBefore={6})!\n", data->thisServerID.toString().substr(0, 4), req.rangeID.printable().substr(0, 6), - streamUID.toString().substr(0, 8), + req.streamUID().toString().substr(0, 8), memoryReply.mutations[memoryVerifyIdx].version, version, feedInfo->emptyVersion, @@ -2782,7 +2783,7 @@ ACTOR Future> getChangeFeedMutations(Stor "disk! (durable validation = {4})\n", data->thisServerID.toString().substr(0, 4), req.rangeID.printable().substr(0, 6), - streamUID.toString().substr(0, 8), + req.streamUID().toString().substr(0, 8), version, durableValidationVersion); @@ -2863,7 +2864,7 @@ ACTOR Future> getChangeFeedMutations(Stor if (!ok) { TraceEvent("ChangeFeedMutationsPopped", data->thisServerID) .detail("FeedID", req.rangeID) - .detail("StreamUID", streamUID) + .detail("StreamUID", req.streamUID()) .detail("Range", req.range) .detail("Begin", req.begin) .detail("End", req.end) @@ -2880,7 +2881,7 @@ ACTOR Future> getChangeFeedMutations(Stor for (auto& m : mutations.mutations) { DEBUG_MUTATION("ChangeFeedSSRead", mutations.version, m, data->thisServerID) .detail("ChangeFeedID", req.rangeID) - .detail("StreamUID", streamUID) + .detail("StreamUID", req.streamUID()) .detail("ReqBegin", req.begin) .detail("ReqEnd", req.end) .detail("ReqRange", req.range); @@ -2907,7 +2908,7 @@ ACTOR Future> getChangeFeedMutations(Stor fmt::print("ERROR: SS {0} CF {1} SQ {2} missing {3} @ {4} from request for [{5} - {6}) {7} - {8}\n", data->thisServerID.toString().substr(0, 4), req.rangeID.printable().substr(0, 6), - streamUID.toString().substr(0, 8), + req.streamUID().toString().substr(0, 8), foundVersion ? "key" : "version", DEBUG_CF_MISSING_VERSION, req.range.begin.printable(), @@ -2928,7 +2929,7 @@ ACTOR Future> getChangeFeedMutations(Stor fmt::print("DBG: SS {0} CF {1} SQ {2} correct @ {3} from request for [{4} - {5}) {6} - {7}\n", data->thisServerID.toString().substr(0, 4), req.rangeID.printable().substr(0, 6), - streamUID.toString().substr(0, 8), + req.streamUID().toString().substr(0, 8), DEBUG_CF_MISSING_VERSION, req.range.begin.printable(), req.range.end.printable(), @@ -2942,7 +2943,7 @@ ACTOR Future> getChangeFeedMutations(Stor if (DEBUG_CF_TRACE) { TraceEvent(SevDebug, "ChangeFeedMutationsDone", data->thisServerID) .detail("FeedID", req.rangeID) - .detail("StreamUID", streamUID) + .detail("StreamUID", req.streamUID()) .detail("Range", req.range) .detail("Begin", req.begin) .detail("End", req.end) @@ -2960,14 +2961,14 @@ ACTOR Future> getChangeFeedMutations(Stor } // Change feed stream must be sent an error as soon as it is moved away, or change feed can get incorrect results -ACTOR Future stopChangeFeedOnMove(StorageServer* data, ChangeFeedStreamRequest req, UID streamUID) { +ACTOR Future stopChangeFeedOnMove(StorageServer* data, ChangeFeedStreamRequest req) { auto feed = data->uidChangeFeed.find(req.rangeID); if (feed == data->uidChangeFeed.end() || feed->second->removing) { req.reply.sendError(unknown_change_feed()); return Void(); } state Promise moved; - feed->second->triggerOnMove(req.range, streamUID, moved); + feed->second->triggerOnMove(req.range, req.streamUID(), moved); try { wait(moved.getFuture()); } catch (Error& e) { @@ -2976,7 +2977,7 @@ ACTOR Future stopChangeFeedOnMove(StorageServer* data, ChangeFeedStreamReq auto feed = data->uidChangeFeed.find(req.rangeID); if (feed != data->uidChangeFeed.end()) { - feed->second->removeOnMoveTrigger(req.range, streamUID); + feed->second->removeOnMoveTrigger(req.range, req.streamUID()); } return Void(); } @@ -2987,7 +2988,7 @@ ACTOR Future stopChangeFeedOnMove(StorageServer* data, ChangeFeedStreamReq return Void(); } -ACTOR Future changeFeedStreamQ(StorageServer* data, ChangeFeedStreamRequest req, UID streamUID) { +ACTOR Future changeFeedStreamQ(StorageServer* data, ChangeFeedStreamRequest req) { state Span span("SS:getChangeFeedStream"_loc, req.spanContext); state bool atLatest = false; state bool removeUID = false; @@ -3019,7 +3020,7 @@ ACTOR Future changeFeedStreamQ(StorageServer* data, ChangeFeedStreamReques if (DEBUG_CF_TRACE) { TraceEvent(SevDebug, "TraceChangeFeedStreamStart", data->thisServerID) .detail("FeedID", req.rangeID) - .detail("StreamUID", streamUID) + .detail("StreamUID", req.streamUID()) .detail("Range", req.range) .detail("Begin", req.begin) .detail("End", req.end) @@ -3041,7 +3042,7 @@ ACTOR Future changeFeedStreamQ(StorageServer* data, ChangeFeedStreamReques if (DEBUG_CF_TRACE) { TraceEvent(SevDebug, "TraceChangeFeedStreamSentInitialEmpty", data->thisServerID) .detail("FeedID", req.rangeID) - .detail("StreamUID", streamUID) + .detail("StreamUID", req.streamUID()) .detail("Range", req.range) .detail("Begin", req.begin) .detail("End", req.end) @@ -3053,12 +3054,12 @@ ACTOR Future changeFeedStreamQ(StorageServer* data, ChangeFeedStreamReques loop { Future onReady = req.reply.onReady(); if (atLatest && !onReady.isReady() && !removeUID) { - data->changeFeedClientVersions[req.reply.getEndpoint().getPrimaryAddress()][streamUID] = + data->changeFeedClientVersions[req.reply.getEndpoint().getPrimaryAddress()][req.streamUID()] = blockedVersion.present() ? blockedVersion.get() : data->prevVersion; if (DEBUG_CF_TRACE) { TraceEvent(SevDebug, "TraceChangeFeedStreamBlockedOnReady", data->thisServerID) .detail("FeedID", req.rangeID) - .detail("StreamUID", streamUID) + .detail("StreamUID", req.streamUID()) .detail("Range", req.range) .detail("Begin", req.begin) .detail("End", req.end) @@ -3069,17 +3070,18 @@ ACTOR Future changeFeedStreamQ(StorageServer* data, ChangeFeedStreamReques removeUID = true; } wait(onReady); + // keep this as not state variable so it is freed after sending to reduce memory Future> feedReplyFuture = - getChangeFeedMutations(data, req, false, atLatest, streamUID); + getChangeFeedMutations(data, req, false, atLatest); if (atLatest && !removeUID && !feedReplyFuture.isReady()) { - data->changeFeedClientVersions[req.reply.getEndpoint().getPrimaryAddress()][streamUID] = + data->changeFeedClientVersions[req.reply.getEndpoint().getPrimaryAddress()][req.streamUID()] = blockedVersion.present() ? blockedVersion.get() : data->prevVersion; removeUID = true; if (DEBUG_CF_TRACE) { TraceEvent(SevDebug, "TraceChangeFeedStreamBlockedMutations", data->thisServerID) .detail("FeedID", req.rangeID) - .detail("StreamUID", streamUID) + .detail("StreamUID", req.streamUID()) .detail("Range", req.range) .detail("Begin", req.begin) .detail("End", req.end) @@ -3102,10 +3104,10 @@ ACTOR Future changeFeedStreamQ(StorageServer* data, ChangeFeedStreamReques Version minVersion = removeUID ? data->version.get() : data->prevVersion; if (removeUID) { if (gotAll || req.begin == req.end) { - data->changeFeedClientVersions[req.reply.getEndpoint().getPrimaryAddress()].erase(streamUID); + data->changeFeedClientVersions[req.reply.getEndpoint().getPrimaryAddress()].erase(req.streamUID()); removeUID = false; } else { - data->changeFeedClientVersions[req.reply.getEndpoint().getPrimaryAddress()][streamUID] = + data->changeFeedClientVersions[req.reply.getEndpoint().getPrimaryAddress()][req.streamUID()] = feedReply.mutations.back().version; } } @@ -3153,7 +3155,7 @@ ACTOR Future changeFeedStreamQ(StorageServer* data, ChangeFeedStreamReques auto it = data->changeFeedClientVersions.find(req.reply.getEndpoint().getPrimaryAddress()); if (it != data->changeFeedClientVersions.end()) { if (removeUID) { - it->second.erase(streamUID); + it->second.erase(req.streamUID()); } if (it->second.empty()) { data->changeFeedClientVersions.erase(it); @@ -3751,8 +3753,6 @@ ACTOR Future getKeyValuesQ(StorageServer* data, GetKeyValuesRequest req) { state Span span("SS:getKeyValues"_loc, req.spanContext); state int64_t resultSize = 0; - state Optional options = req.options; - state ReadType readType = options.present() ? options.get().type : ReadType::NORMAL; if (req.tenantInfo.name.present()) { span.addAttribute("tenant"_sr, req.tenantInfo.name.get()); @@ -3768,11 +3768,8 @@ ACTOR Future getKeyValuesQ(StorageServer* data, GetKeyValuesRequest req) // Active load balancing runs at a very high priority (to obtain accurate queue lengths) // so we need to downgrade here wait(data->getQueryDelay()); - if (!SERVER_KNOBS->FETCH_KEYS_LOWER_PRIORITY && readType == ReadType::FETCH) { - readType = ReadType::NORMAL; - } - state PriorityMultiLock::Lock lock = wait(data->ssLock.lock(data->readPriorityRanks[(int)readType])); + state PriorityMultiLock::Lock lock = wait(data->getReadLock(req.options)); // Track time from requestTime through now as read queueing wait time state double queueWaitEnd = g_network->timer(); @@ -3820,10 +3817,11 @@ ACTOR Future getKeyValuesQ(StorageServer* data, GetKeyValuesRequest req) state Future fBegin = req.begin.isFirstGreaterOrEqual() ? Future(req.begin.getKey()) - : findKey(data, req.begin, version, searchRange, &offset1, span.context, options); - state Future fEnd = req.end.isFirstGreaterOrEqual() - ? Future(req.end.getKey()) - : findKey(data, req.end, version, searchRange, &offset2, span.context, options); + : findKey(data, req.begin, version, searchRange, &offset1, span.context, req.options); + state Future fEnd = + req.end.isFirstGreaterOrEqual() + ? Future(req.end.getKey()) + : findKey(data, req.end, version, searchRange, &offset2, span.context, req.options); state Key begin = wait(fBegin); state Key end = wait(fEnd); @@ -3871,7 +3869,7 @@ ACTOR Future getKeyValuesQ(StorageServer* data, GetKeyValuesRequest req) req.limit, &remainingLimitBytes, span.context, - options, + req.options, tenantPrefix)); const double duration = g_network->timer() - kvReadRange; data->counters.kvReadRangeLatencySample.addMeasurement(duration); @@ -4817,8 +4815,6 @@ ACTOR Future getMappedKeyValuesQ(StorageServer* data, GetMappedKeyValuesRe { state Span span("SS:getMappedKeyValues"_loc, req.spanContext); state int64_t resultSize = 0; - state Optional options = req.options; - state ReadType readType = options.present() ? options.get().type : ReadType::NORMAL; if (req.tenantInfo.name.present()) { span.addAttribute("tenant"_sr, req.tenantInfo.name.get()); @@ -4882,10 +4878,11 @@ ACTOR Future getMappedKeyValuesQ(StorageServer* data, GetMappedKeyValuesRe state Future fBegin = req.begin.isFirstGreaterOrEqual() ? Future(req.begin.getKey()) - : findKey(data, req.begin, version, searchRange, &offset1, span.context, options); - state Future fEnd = req.end.isFirstGreaterOrEqual() - ? Future(req.end.getKey()) - : findKey(data, req.end, version, searchRange, &offset2, span.context, options); + : findKey(data, req.begin, version, searchRange, &offset1, span.context, req.options); + state Future fEnd = + req.end.isFirstGreaterOrEqual() + ? Future(req.end.getKey()) + : findKey(data, req.end, version, searchRange, &offset2, span.context, req.options); state Key begin = wait(fBegin); state Key end = wait(fEnd); @@ -4944,14 +4941,15 @@ ACTOR Future getMappedKeyValuesQ(StorageServer* data, GetMappedKeyValuesRe // Only take the ssLock for the readRange operation and unlock before the subqueries because each // subquery will route back to getValueQ or getKeyValuesQ with a new request having the same // read options which will each acquire the ssLock. - state PriorityMultiLock::Lock lock = wait(data->ssLock.lock(data->readPriorityRanks[(int)readType])); + state PriorityMultiLock::Lock lock = wait(data->getReadLock(req.options)); + GetKeyValuesReply getKeyValuesReply = wait(readRange(data, version, KeyRangeRef(begin, end), req.limit, &remainingLimitBytes, span.context, - options, + req.options, tenantPrefix)); lock.release(); @@ -5035,8 +5033,6 @@ ACTOR Future getKeyValuesStreamQ(StorageServer* data, GetKeyValuesStreamRe { state Span span("SS:getKeyValuesStream"_loc, req.spanContext); state int64_t resultSize = 0; - state Optional options = req.options; - state ReadType readType = options.present() ? options.get().type : ReadType::NORMAL; if (req.tenantInfo.name.present()) { span.addAttribute("tenant"_sr, req.tenantInfo.name.get()); @@ -5051,12 +5047,6 @@ ACTOR Future getKeyValuesStreamQ(StorageServer* data, GetKeyValuesStreamRe // Active load balancing runs at a very high priority (to obtain accurate queue lengths) // so we need to downgrade here wait(delay(0, TaskPriority::DefaultEndpoint)); - if (!SERVER_KNOBS->FETCH_KEYS_LOWER_PRIORITY && readType == ReadType::FETCH) { - readType = ReadType::NORMAL; - } - - state PriorityMultiLock::Lock lock = wait(data->ssLock.lock(readPriority)); - state int readPriority = data->readPriorityRanks[(int)readType]; try { if (req.options.present() && req.options.get().debugID.present()) @@ -5100,10 +5090,11 @@ ACTOR Future getKeyValuesStreamQ(StorageServer* data, GetKeyValuesStreamRe state Future fBegin = req.begin.isFirstGreaterOrEqual() ? Future(req.begin.getKey()) - : findKey(data, req.begin, version, searchRange, &offset1, span.context, options); - state Future fEnd = req.end.isFirstGreaterOrEqual() - ? Future(req.end.getKey()) - : findKey(data, req.end, version, searchRange, &offset2, span.context, options); + : findKey(data, req.begin, version, searchRange, &offset1, span.context, req.options); + state Future fEnd = + req.end.isFirstGreaterOrEqual() + ? Future(req.end.getKey()) + : findKey(data, req.end, version, searchRange, &offset2, span.context, req.options); state Key begin = wait(fBegin); state Key end = wait(fEnd); if (req.options.present() && req.options.get().debugID.present()) @@ -5162,14 +5153,18 @@ ACTOR Future getKeyValuesStreamQ(StorageServer* data, GetKeyValuesStreamRe .detail("ReqLimit", req.limit) .detail("Begin", begin.printable()) .detail("End", end.printable()); + + state PriorityMultiLock::Lock lock = wait(data->getReadLock(req.options)); + GetKeyValuesReply _r = wait(readRange(data, version, KeyRangeRef(begin, end), req.limit, &byteLimit, span.context, - options, + req.options, tenantPrefix)); + lock.release(); GetKeyValuesStreamReply r(_r); if (req.options.present() && req.options.get().debugID.present()) @@ -5229,10 +5224,8 @@ ACTOR Future getKeyValuesStreamQ(StorageServer* data, GetKeyValuesStreamRe end = lastKey; } - lock.release(); - wait(store(lock, data->ssLock.lock(readPriority))); - data->transactionTagCounter.addRequest(req.tags, resultSize); + // lock.release(); } } } catch (Error& e) { @@ -5251,18 +5244,10 @@ ACTOR Future getKeyValuesStreamQ(StorageServer* data, GetKeyValuesStreamRe ACTOR Future getKeyQ(StorageServer* data, GetKeyRequest req) { state Span span("SS:getKey"_loc, req.spanContext); - state PriorityMultiLock::Lock lock; if (req.tenantInfo.name.present()) { span.addAttribute("tenant"_sr, req.tenantInfo.name.get()); } state int64_t resultSize = 0; - state ReadOptions options; - state ReadType readType = ReadType::NORMAL; - - if (req.options.present()) { - options = req.options.get(); - readType = options.type; - } getCurrentLineage()->modify(&TransactionLineage::txID) = req.spanContext.traceID; @@ -5275,7 +5260,7 @@ ACTOR Future getKeyQ(StorageServer* data, GetKeyRequest req) { // so we need to downgrade here wait(data->getQueryDelay()); - wait(store(lock, data->ssLock.lock(data->readPriorityRanks[(int)readType]))); + state PriorityMultiLock::Lock lock = wait(data->getReadLock(req.options)); // Track time from requestTime through now as read queueing wait time state double queueWaitEnd = g_network->timer(); @@ -5296,13 +5281,7 @@ ACTOR Future getKeyQ(StorageServer* data, GetKeyRequest req) { KeyRangeRef searchRange = data->clampRangeToTenant(shard, tenantEntry, req.arena); state int offset; - Key absoluteKey = wait(findKey(data, - req.sel, - version, - searchRange, - &offset, - req.spanContext, - req.options.present() ? options : Optional())); + Key absoluteKey = wait(findKey(data, req.sel, version, searchRange, &offset, req.spanContext, req.options)); data->checkChangeCounter(changeCounter, KeyRangeRef(std::min(req.sel.getKey(), absoluteKey), @@ -6124,7 +6103,8 @@ ACTOR Future fetchChangeFeedApplier(StorageServer* data, KeyRange range, Version emptyVersion, Version beginVersion, - Version endVersion) { + Version endVersion, + ReadOptions readOptions) { state Version startVersion = beginVersion; startVersion = std::max(startVersion, emptyVersion + 1); @@ -6145,8 +6125,14 @@ ACTOR Future fetchChangeFeedApplier(StorageServer* data, } state Reference feedResults = makeReference(); - state Future feed = data->cx->getChangeFeedStream( - feedResults, rangeId, startVersion, endVersion, range, SERVER_KNOBS->CHANGEFEEDSTREAM_LIMIT_BYTES, true); + state Future feed = data->cx->getChangeFeedStream(feedResults, + rangeId, + startVersion, + endVersion, + range, + SERVER_KNOBS->CHANGEFEEDSTREAM_LIMIT_BYTES, + true, + readOptions); state Version firstVersion = invalidVersion; state Version lastVersion = invalidVersion; @@ -6322,7 +6308,8 @@ ACTOR Future fetchChangeFeedApplier(StorageServer* data, ACTOR Future fetchChangeFeed(StorageServer* data, Reference changeFeedInfo, Version beginVersion, - Version endVersion) { + Version endVersion, + ReadOptions readOptions) { wait(delay(0)); // allow this actor to be cancelled by removals TraceEvent(SevDebug, "FetchChangeFeed", data->thisServerID) @@ -6365,7 +6352,8 @@ ACTOR Future fetchChangeFeed(StorageServer* data, changeFeedInfo->range, changeFeedInfo->emptyVersion, beginVersion, - endVersion)); + endVersion, + readOptions)); data->fetchingChangeFeeds.insert(changeFeedInfo->id); return maxFetched; } catch (Error& e) { @@ -6660,7 +6648,8 @@ ACTOR Future> dispatchChangeFeeds(StorageServer Version endVersion, PromiseStream destroyedFeeds, std::vector* feedIds, - std::unordered_set newFeedIds) { + std::unordered_set newFeedIds, + ReadOptions readOptions) { state std::unordered_map feedMaxFetched; if (feedIds->empty() && newFeedIds.empty()) { return feedMaxFetched; @@ -6674,7 +6663,8 @@ ACTOR Future> dispatchChangeFeeds(StorageServer auto feedIt = data->uidChangeFeed.find(feedId); // feed may have been moved away or deleted after move was scheduled, do nothing in that case if (feedIt != data->uidChangeFeed.end() && !feedIt->second->removing) { - feedFetches[feedIt->second->id] = fetchChangeFeed(data, feedIt->second, beginVersion, endVersion); + feedFetches[feedIt->second->id] = + fetchChangeFeed(data, feedIt->second, beginVersion, endVersion, readOptions); } } for (auto& feedId : newFeedIds) { @@ -6682,7 +6672,7 @@ ACTOR Future> dispatchChangeFeeds(StorageServer // we just read the change feed data map earlier in fetchKeys without yielding, so these feeds must exist ASSERT(feedIt != data->uidChangeFeed.end()); ASSERT(!feedIt->second->removing); - feedFetches[feedIt->second->id] = fetchChangeFeed(data, feedIt->second, 0, endVersion); + feedFetches[feedIt->second->id] = fetchChangeFeed(data, feedIt->second, 0, endVersion, readOptions); } loop { @@ -6745,6 +6735,11 @@ ACTOR Future fetchKeys(StorageServer* data, AddingShard* shard) { data->counters.bytesFetched, data->counters.kvFetched); + // Set read options to use non-caching reads and set Fetch type unless low priority data fetching is disabled by a + // knob + state ReadOptions readOptions = ReadOptions( + fetchKeysID, SERVER_KNOBS->FETCH_KEYS_LOWER_PRIORITY ? ReadType::FETCH : ReadType::NORMAL, CacheResult::False); + // need to set this at the very start of the fetch, to handle any private change feed destroy mutations we get for // this key range, that apply to change feeds we don't know about yet because their metadata hasn't been fetched yet data->changeFeedDestroys[fetchKeysID] = destroyedFeeds; @@ -6835,9 +6830,6 @@ ACTOR Future fetchKeys(StorageServer* data, AddingShard* shard) { state int debug_nextRetryToLog = 1; state Error lastError; - // it is used to inform the storage that the rangeRead is for Fetch - state ReadOptions options = ReadOptions(fetchKeysID, ReadType::FETCH); - // FIXME: The client cache does not notice when servers are added to a team. To read from a local storage server // we must refresh the cache manually. data->cx->invalidateCache(Key(), keys); @@ -6846,6 +6838,8 @@ ACTOR Future fetchKeys(StorageServer* data, AddingShard* shard) { state Transaction tr(data->cx); tr.setOption(FDBTransactionOptions::PRIORITY_SYSTEM_IMMEDIATE); tr.setOption(FDBTransactionOptions::LOCK_AWARE); + tr.trState->readOptions = readOptions; + // fetchVersion = data->version.get(); // A quick fix: // By default, we use data->version as the fetchVersion. @@ -6894,7 +6888,6 @@ ACTOR Future fetchKeys(StorageServer* data, AddingShard* shard) { shard->updates.pop_front(); tr.setVersion(fetchVersion); tr.trState->taskID = TaskPriority::FetchKeys; - tr.trState->readOptions = options; state PromiseStream results; state Future hold; if (SERVER_KNOBS->FETCH_USING_BLOB) { @@ -7056,7 +7049,8 @@ ACTOR Future fetchKeys(StorageServer* data, AddingShard* shard) { fetchVersion + 1, destroyedFeeds, &changeFeedsToFetch, - std::unordered_set()); + std::unordered_set(), + readOptions); state Future fetchDurable = data->durableVersion.whenAtLeast(data->storageVersion() + 1); state Future dataArrive = data->version.whenAtLeast(fetchVersion); @@ -7121,7 +7115,8 @@ ACTOR Future fetchKeys(StorageServer* data, AddingShard* shard) { shard->transferredVersion, destroyedFeeds, &changeFeedsToFetch, - newChangeFeeds); + newChangeFeeds, + readOptions); TraceEvent(SevDebug, "FetchKeysHaveData", data->thisServerID) .detail("FKID", interval.pairID) @@ -9582,9 +9577,13 @@ ACTOR Future applyByteSampleResult(StorageServer* data, state int totalFetches = 0; state int totalKeys = 0; state int totalBytes = 0; + state ReadOptions readOptions(UID(), ReadType::NORMAL, CacheResult::False); + loop { - RangeResult bs = wait(storage->readRange( - KeyRangeRef(begin, end), SERVER_KNOBS->STORAGE_LIMIT_BYTES, SERVER_KNOBS->STORAGE_LIMIT_BYTES)); + RangeResult bs = wait(storage->readRange(KeyRangeRef(begin, end), + SERVER_KNOBS->STORAGE_LIMIT_BYTES, + SERVER_KNOBS->STORAGE_LIMIT_BYTES, + readOptions)); if (results) { results->push_back(bs.castTo>()); data->bytesRestored += bs.logicalSize(); @@ -10528,7 +10527,7 @@ ACTOR Future serveChangeFeedStreamRequests(StorageServer* self, loop { ChangeFeedStreamRequest req = waitNext(changeFeedStream); // must notify change feed that its shard is moved away ASAP - self->actors.add(changeFeedStreamQ(self, req, req.debugUID) || stopChangeFeedOnMove(self, req, req.debugUID)); + self->actors.add(changeFeedStreamQ(self, req) || stopChangeFeedOnMove(self, req)); } } From 3c5d3f7a94d58639ee8aee84c5cf8216a076f0d5 Mon Sep 17 00:00:00 2001 From: Marian Dvorsky Date: Wed, 26 Oct 2022 16:29:28 +0200 Subject: [PATCH 44/95] Fix SpanContext for GP:getLiveCommittedVersion (#8565) * Fix SpanContext for GP:getLiveCommittedVersion --- fdbclient/include/fdbclient/Tracing.h | 15 +------- fdbserver/GrvProxyServer.actor.cpp | 35 ++++++++++--------- .../GrvProxyTransactionTagThrottler.actor.cpp | 12 +++---- .../GrvProxyTransactionTagThrottler.h | 4 +-- 4 files changed, 28 insertions(+), 38 deletions(-) diff --git a/fdbclient/include/fdbclient/Tracing.h b/fdbclient/include/fdbclient/Tracing.h index 789b346dfd..01ffcaa5dd 100644 --- a/fdbclient/include/fdbclient/Tracing.h +++ b/fdbclient/include/fdbclient/Tracing.h @@ -273,17 +273,4 @@ struct ITracer { virtual void trace(Span const& span) = 0; }; -void openTracer(TracerType type); - -template -struct SpannedDeque : Deque { - Span span; - explicit SpannedDeque(Location loc) : span(loc) {} - SpannedDeque(SpannedDeque&& other) : Deque(std::move(other)), span(std::move(other.span)) {} - SpannedDeque(SpannedDeque const&) = delete; - SpannedDeque& operator=(SpannedDeque const&) = delete; - SpannedDeque& operator=(SpannedDeque&& other) { - *static_cast*>(this) = std::move(other); - span = std::move(other.span); - } -}; +void openTracer(TracerType type); \ No newline at end of file diff --git a/fdbserver/GrvProxyServer.actor.cpp b/fdbserver/GrvProxyServer.actor.cpp index a55748505f..e384678f9d 100644 --- a/fdbserver/GrvProxyServer.actor.cpp +++ b/fdbserver/GrvProxyServer.actor.cpp @@ -459,9 +459,9 @@ 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, + Deque* systemQueue, + Deque* defaultQueue, + Deque* batchQueue, FutureStream readVersionRequests, PromiseStream GRVTimer, double* lastGRVTime, @@ -531,7 +531,6 @@ ACTOR Future queueGetReadVersionRequests(Reference stats->txnSystemPriorityStartIn += req.transactionCount; ++stats->systemGRVQueueSize; systemQueue->push_back(req); - // systemQueue->span.addParent(req.spanContext); } else if (req.priority >= TransactionPriority::DEFAULT) { ++stats->txnRequestIn; stats->txnStartIn += req.transactionCount; @@ -542,7 +541,6 @@ ACTOR Future queueGetReadVersionRequests(Reference } else { defaultQueue->push_back(req); } - // defaultQueue->span.addParent(req.spanContext); } else { // Return error for batch_priority GRV requests int64_t proxiesCount = std::max((int)db->get().client.grvProxies.size(), 1); @@ -559,7 +557,6 @@ ACTOR Future queueGetReadVersionRequests(Reference } else { batchQueue->push_back(req); } - // batchQueue->span.addParent(req.spanContext); } } } @@ -607,7 +604,7 @@ ACTOR Future lastCommitUpdater(GrvProxyData* self, PromiseStream getLiveCommittedVersion(SpanContext parentSpan, +ACTOR Future getLiveCommittedVersion(std::vector spanContexts, GrvProxyData* grvProxyData, uint32_t flags, Optional debugID, @@ -620,7 +617,10 @@ ACTOR Future getLiveCommittedVersion(SpanContext parentSpan // before the request returns, so it is committed. (2) No proxy on our list reported committed a higher version // before this request was received, because then its committedVersion would have been higher, // and no other proxy could have already committed anything without first ending the epoch - state Span span("GP:getLiveCommittedVersion"_loc, parentSpan); + state Span span("GP:getLiveCommittedVersion"_loc); + for (const SpanContext& spanContext : spanContexts) { + span.addLink(spanContext); + } ++grvProxyData->stats.txnStartBatch; state double grvStart = now(); @@ -826,15 +826,14 @@ ACTOR static Future transactionStarter(GrvProxyInterface proxy, state GrvTransactionRateInfo batchRateInfo(0); state GrvProxyTransactionTagThrottler tagThrottler; - state SpannedDeque systemQueue("GP:transactionStarterSystemQueue"_loc); - state SpannedDeque defaultQueue("GP:transactionStarterDefaultQueue"_loc); - state SpannedDeque batchQueue("GP:transactionStarterBatchQueue"_loc); + state Deque systemQueue; + state Deque defaultQueue; + state Deque batchQueue; state TransactionTagMap transactionTagCounter; state PrioritizedTransactionTagMap clientThrottledTags; state PromiseStream normalGRVLatency; - // state Span span; state int64_t midShardSize = SERVER_KNOBS->MIN_SHARD_BYTES; getCurrentLineage()->modify(&TransactionLineage::operation) = @@ -911,7 +910,7 @@ ACTOR static Future transactionStarter(GrvProxyInterface proxy, uint32_t defaultQueueSize = defaultQueue.size(); uint32_t batchQueueSize = batchQueue.size(); while (requestsToStart < SERVER_KNOBS->START_TRANSACTION_MAX_REQUESTS_TO_START) { - SpannedDeque* transactionQueue; + Deque* transactionQueue; if (!systemQueue.empty()) { transactionQueue = &systemQueue; } else if (!defaultQueue.empty()) { @@ -921,7 +920,6 @@ ACTOR static Future transactionStarter(GrvProxyInterface proxy, } else { break; } - // transactionQueue->span.swap(span); auto& req = transactionQueue->front(); int tc = req.transactionCount; @@ -1017,7 +1015,13 @@ ACTOR static Future transactionStarter(GrvProxyInterface proxy, int batchGRVProcessed = 0; for (int i = 0; i < start.size(); i++) { if (start[i].size()) { - Future readVersionReply = getLiveCommittedVersion(SpanContext(), + std::vector spanContexts; + spanContexts.reserve(start[i].size()); + for (const GetReadVersionRequest& request : start[i]) { + spanContexts.push_back(request.spanContext); + } + + Future readVersionReply = getLiveCommittedVersion(spanContexts, grvProxyData, i, debugID, @@ -1041,7 +1045,6 @@ ACTOR static Future transactionStarter(GrvProxyInterface proxy, batchGRVProcessed += batchPriTransactionsStarted[i]; } } - // span = Span(span.location); grvProxyData->stats.percentageOfDefaultGRVQueueProcessed = defaultQueueSize ? (double)defaultGRVProcessed / defaultQueueSize : 1; diff --git a/fdbserver/GrvProxyTransactionTagThrottler.actor.cpp b/fdbserver/GrvProxyTransactionTagThrottler.actor.cpp index d6cf76bc63..772bd24ba7 100644 --- a/fdbserver/GrvProxyTransactionTagThrottler.actor.cpp +++ b/fdbserver/GrvProxyTransactionTagThrottler.actor.cpp @@ -100,8 +100,8 @@ void GrvProxyTransactionTagThrottler::addRequest(GetReadVersionRequest const& re } void GrvProxyTransactionTagThrottler::releaseTransactions(double elapsed, - SpannedDeque& outBatchPriority, - SpannedDeque& outDefaultPriority) { + Deque& outBatchPriority, + Deque& outDefaultPriority) { // Pointer to a TagQueue with some extra metadata stored alongside struct TagQueueHandle { // Store pointers here to avoid frequent std::unordered_map lookups @@ -280,8 +280,8 @@ ACTOR static Future mockFifoClient(GrvProxyTransactionTagThrottler* thrott } ACTOR static Future mockServer(GrvProxyTransactionTagThrottler* throttler) { - state SpannedDeque outBatchPriority("TestGrvProxyTransactionTagThrottler_Batch"_loc); - state SpannedDeque outDefaultPriority("TestGrvProxyTransactionTagThrottler_Default"_loc); + state Deque outBatchPriority; + state Deque outDefaultPriority; loop { state double elapsed = (0.009 + 0.002 * deterministicRandom()->random01()); wait(delay(elapsed)); @@ -404,8 +404,8 @@ TEST_CASE("/GrvProxyTransactionTagThrottler/Cleanup2") { throttler.updateRates(TransactionTagMap{}); ASSERT_EQ(throttler.size(), 1); { - SpannedDeque outBatchPriority("TestGrvProxyTransactionTagThrottler_Batch"_loc); - SpannedDeque outDefaultPriority("TestGrvProxyTransactionTagThrottler_Default"_loc); + Deque outBatchPriority; + Deque outDefaultPriority; throttler.releaseTransactions(0.1, outBatchPriority, outDefaultPriority); } // Calling updates cleans up the queues in throttler diff --git a/fdbserver/include/fdbserver/GrvProxyTransactionTagThrottler.h b/fdbserver/include/fdbserver/GrvProxyTransactionTagThrottler.h index 887ff9ffb0..9575280ea4 100644 --- a/fdbserver/include/fdbserver/GrvProxyTransactionTagThrottler.h +++ b/fdbserver/include/fdbserver/GrvProxyTransactionTagThrottler.h @@ -72,8 +72,8 @@ public: // 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); + Deque& outBatchPriority, + Deque& outDefaultPriority); void addRequest(GetReadVersionRequest const&); From b8b7b46d8f11390ca283ba0a252b60f2d2029b3a Mon Sep 17 00:00:00 2001 From: Aaron Molitor Date: Wed, 26 Oct 2022 08:16:27 -0500 Subject: [PATCH 45/95] update kubectl and awscli --- packaging/docker/Dockerfile | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/packaging/docker/Dockerfile b/packaging/docker/Dockerfile index eddc5b488f..70d488b88c 100644 --- a/packaging/docker/Dockerfile +++ b/packaging/docker/Dockerfile @@ -178,13 +178,13 @@ RUN yum -y install \ rm -rf /var/cache/yum WORKDIR /tmp -RUN curl -Ls https://amazon-eks.s3.amazonaws.com/1.19.6/2021-01-05/bin/linux/amd64/kubectl -o kubectl && \ - echo "08ff68159bbcb844455167abb1d0de75bbfe5ae1b051f81ab060a1988027868a kubectl" > kubectl.txt && \ +RUN curl -Ls https://s3.us-west-2.amazonaws.com/amazon-eks/1.22.6/2022-03-09/bin/linux/amd64/kubectl -o kubectl && \ + echo "860c3d37a5979491895767e7332404d28dc0d7797c7673c33df30ca80e215a07 kubectl" > kubectl.txt && \ sha256sum --quiet -c kubectl.txt && \ mv kubectl /usr/local/bin/kubectl && \ chmod 755 /usr/local/bin/kubectl && \ - curl -Ls https://awscli.amazonaws.com/awscli-exe-linux-x86_64-2.2.43.zip -o "awscliv2.zip" && \ - echo "9a8b3c4e7f72bbcc55e341dce3af42479f2730c225d6d265ee6f9162cfdebdfd awscliv2.zip" > awscliv2.txt && \ + curl -Ls https://awscli.amazonaws.com/awscli-exe-linux-$(uname -m)-2.7.34.zip -o "awscliv2.zip" && \ + echo "daf9253f0071b5cfee9532bc5220bedd7a5d29d4e0f92b42b9e3e4c496341e88 awscliv2.zip" > awscliv2.txt && \ sha256sum --quiet -c awscliv2.txt && \ unzip -qq awscliv2.zip && \ ./aws/install && \ From ab6953be7da4dc652a8abddd21fd2686d9bd16c0 Mon Sep 17 00:00:00 2001 From: Josh Slocum Date: Wed, 26 Oct 2022 11:02:50 -0500 Subject: [PATCH 46/95] Blob Granule read-driven compaction (#8572) --- fdbclient/ServerKnobs.cpp | 5 + .../include/fdbclient/BlobWorkerCommon.h | 10 +- fdbclient/include/fdbclient/ServerKnobs.h | 4 + fdbserver/BlobWorker.actor.cpp | 187 ++++++++++++++++-- .../workloads/BlobGranuleVerifier.actor.cpp | 10 +- 5 files changed, 197 insertions(+), 19 deletions(-) diff --git a/fdbclient/ServerKnobs.cpp b/fdbclient/ServerKnobs.cpp index 009891f664..68eed2867f 100644 --- a/fdbclient/ServerKnobs.cpp +++ b/fdbclient/ServerKnobs.cpp @@ -967,6 +967,9 @@ void ServerKnobs::initialize(Randomize randomize, ClientKnobs* clientKnobs, IsSi init( BG_CONSISTENCY_CHECK_ENABLED, true ); if (randomize && BUGGIFY) BG_CONSISTENCY_CHECK_ENABLED = false; init( BG_CONSISTENCY_CHECK_TARGET_SPEED_KB, 1000 ); if (randomize && BUGGIFY) BG_CONSISTENCY_CHECK_TARGET_SPEED_KB *= (deterministicRandom()->randomInt(2, 50) / 10); init( BG_KEY_TUPLE_TRUNCATE_OFFSET, 0 ); + init( BG_ENABLE_READ_DRIVEN_COMPACTION, true ); if (randomize && BUGGIFY) BG_ENABLE_READ_DRIVEN_COMPACTION = false; + init( BG_RDC_BYTES_FACTOR, 2 ); if (randomize && BUGGIFY) BG_RDC_BYTES_FACTOR = deterministicRandom()->randomInt(1, 10); + init( BG_RDC_READ_FACTOR, 3 ); if (randomize && BUGGIFY) BG_RDC_READ_FACTOR = deterministicRandom()->randomInt(1, 10); init( BG_ENABLE_MERGING, true ); if (randomize && BUGGIFY) BG_ENABLE_MERGING = false; init( BG_MERGE_CANDIDATE_THRESHOLD_SECONDS, isSimulated ? 20.0 : 30 * 60 ); if (randomize && BUGGIFY) BG_MERGE_CANDIDATE_THRESHOLD_SECONDS = 5.0; @@ -975,6 +978,8 @@ void ServerKnobs::initialize(Randomize randomize, ClientKnobs* clientKnobs, IsSi init( BLOB_WORKER_INITIAL_SNAPSHOT_PARALLELISM, 8 ); if( randomize && BUGGIFY ) BLOB_WORKER_INITIAL_SNAPSHOT_PARALLELISM = 1; init( BLOB_WORKER_RESNAPSHOT_PARALLELISM, 40 ); if( randomize && BUGGIFY ) BLOB_WORKER_RESNAPSHOT_PARALLELISM = deterministicRandom()->randomInt(1, 10); init( BLOB_WORKER_DELTA_FILE_WRITE_PARALLELISM, 2000 ); if( randomize && BUGGIFY ) BLOB_WORKER_DELTA_FILE_WRITE_PARALLELISM = deterministicRandom()->randomInt(10, 100); + init( BLOB_WORKER_RDC_PARALLELISM, 2 ); if( randomize && BUGGIFY ) BLOB_WORKER_RDC_PARALLELISM = deterministicRandom()->randomInt(1, 6); + init( BLOB_WORKER_TIMEOUT, 10.0 ); if( randomize && BUGGIFY ) BLOB_WORKER_TIMEOUT = 1.0; init( BLOB_WORKER_REQUEST_TIMEOUT, 5.0 ); if( randomize && BUGGIFY ) BLOB_WORKER_REQUEST_TIMEOUT = 1.0; init( BLOB_WORKERLIST_FETCH_INTERVAL, 1.0 ); diff --git a/fdbclient/include/fdbclient/BlobWorkerCommon.h b/fdbclient/include/fdbclient/BlobWorkerCommon.h index b4cbbac2a7..7870bacdb7 100644 --- a/fdbclient/include/fdbclient/BlobWorkerCommon.h +++ b/fdbclient/include/fdbclient/BlobWorkerCommon.h @@ -45,6 +45,7 @@ struct BlobWorkerStats { Counter compressionBytesFinal; Counter fullRejections; Counter forceFlushCleanups; + Counter readDrivenCompactions; int numRangesAssigned; int mutationBytesBuffered; @@ -83,10 +84,11 @@ struct BlobWorkerStats { readRequestsWithBegin("ReadRequestsWithBegin", cc), readRequestsCollapsed("ReadRequestsCollapsed", cc), flushGranuleReqs("FlushGranuleReqs", cc), compressionBytesRaw("CompressionBytesRaw", cc), compressionBytesFinal("CompressionBytesFinal", cc), fullRejections("FullRejections", cc), - forceFlushCleanups("ForceFlushCleanups", cc), numRangesAssigned(0), mutationBytesBuffered(0), - activeReadRequests(0), granulesPendingSplitCheck(0), minimumCFVersion(0), cfVersionLag(0), - notAtLatestChangeFeeds(0), lastResidentMemory(0), estimatedMaxResidentMemory(0), - initialSnapshotLock(initialSnapshotLock), resnapshotLock(resnapshotLock), deltaWritesLock(deltaWritesLock) { + forceFlushCleanups("ForceFlushCleanups", cc), readDrivenCompactions("ReadDrivenCompactions", cc), + numRangesAssigned(0), mutationBytesBuffered(0), activeReadRequests(0), granulesPendingSplitCheck(0), + minimumCFVersion(0), cfVersionLag(0), notAtLatestChangeFeeds(0), lastResidentMemory(0), + estimatedMaxResidentMemory(0), initialSnapshotLock(initialSnapshotLock), resnapshotLock(resnapshotLock), + deltaWritesLock(deltaWritesLock) { specialCounter(cc, "NumRangesAssigned", [this]() { return this->numRangesAssigned; }); specialCounter(cc, "MutationBytesBuffered", [this]() { return this->mutationBytesBuffered; }); specialCounter(cc, "ActiveReadRequests", [this]() { return this->activeReadRequests; }); diff --git a/fdbclient/include/fdbclient/ServerKnobs.h b/fdbclient/include/fdbclient/ServerKnobs.h index bfa48e8b09..5a5df49dc8 100644 --- a/fdbclient/include/fdbclient/ServerKnobs.h +++ b/fdbclient/include/fdbclient/ServerKnobs.h @@ -950,10 +950,14 @@ public: int BG_MERGE_CANDIDATE_THRESHOLD_SECONDS; int BG_MERGE_CANDIDATE_DELAY_SECONDS; int BG_KEY_TUPLE_TRUNCATE_OFFSET; + bool BG_ENABLE_READ_DRIVEN_COMPACTION; + int BG_RDC_BYTES_FACTOR; + int BG_RDC_READ_FACTOR; int BLOB_WORKER_INITIAL_SNAPSHOT_PARALLELISM; int BLOB_WORKER_RESNAPSHOT_PARALLELISM; int BLOB_WORKER_DELTA_FILE_WRITE_PARALLELISM; + int BLOB_WORKER_RDC_PARALLELISM; double BLOB_WORKER_TIMEOUT; // Blob Manager's reaction time to a blob worker failure double BLOB_WORKER_REQUEST_TIMEOUT; // Blob Worker's server-side request timeout diff --git a/fdbserver/BlobWorker.actor.cpp b/fdbserver/BlobWorker.actor.cpp index 5717472890..cf5b7b1340 100644 --- a/fdbserver/BlobWorker.actor.cpp +++ b/fdbserver/BlobWorker.actor.cpp @@ -84,6 +84,15 @@ struct GranuleStartState { Optional history; }; +// TODO: add more (blob file request cost, in-memory mutations vs blob delta file, etc...) +struct GranuleReadStats { + int64_t deltaBytesRead; + + void reset() { deltaBytesRead = 0; } + + GranuleReadStats() { reset(); } +}; + struct GranuleMetadata : NonCopyable, ReferenceCounted { KeyRange keyRange; @@ -120,11 +129,74 @@ struct GranuleMetadata : NonCopyable, ReferenceCounted { AssignBlobRangeRequest originalReq; + GranuleReadStats readStats; + bool rdcCandidate; + Promise runRDC; + void resume() { if (resumeSnapshot.canBeSet()) { resumeSnapshot.send(Void()); } } + + void resetReadStats() { + rdcCandidate = false; + readStats.reset(); + runRDC.reset(); + } + + // determine eligibility (>1) and priority for re-snapshotting this granule + double weightRDC() { + // ratio of read amp to write amp that would be incurred by re-snapshotting now + int64_t lastSnapshotSize = (files.snapshotFiles.empty()) ? 0 : files.snapshotFiles.back().length; + int64_t minSnapshotSize = SERVER_KNOBS->BG_SNAPSHOT_FILE_TARGET_BYTES / 2; + lastSnapshotSize = std::max(minSnapshotSize, lastSnapshotSize); + + int64_t writeAmp = lastSnapshotSize + bufferedDeltaBytes + bytesInNewDeltaFiles; + // read amp is deltaBytesRead. Read amp must be READ_FACTOR times larger than write amp + return (1.0 * readStats.deltaBytesRead) / (writeAmp * SERVER_KNOBS->BG_RDC_READ_FACTOR); + } + + bool isEligibleRDC() { + // granule should be reasonably read-hot to be eligible + int64_t bytesWritten = bufferedDeltaBytes + bytesInNewDeltaFiles; + return bytesWritten * SERVER_KNOBS->BG_RDC_READ_FACTOR < readStats.deltaBytesRead; + } + + bool updateReadStats(Version readVersion, const BlobGranuleChunkRef& chunk) { + // Only update stats for re-compacting for at-latest reads that have to do snapshot + delta merge + if (!SERVER_KNOBS->BG_ENABLE_READ_DRIVEN_COMPACTION || !chunk.snapshotFile.present() || + pendingSnapshotVersion != durableSnapshotVersion.get() || readVersion <= pendingSnapshotVersion) { + return false; + } + + if (chunk.newDeltas.empty() && chunk.deltaFiles.empty()) { + return false; + } + + readStats.deltaBytesRead += chunk.newDeltas.expectedSize(); + for (auto& it : chunk.deltaFiles) { + readStats.deltaBytesRead += it.length; + } + + if (rdcCandidate) { + return false; + } + + if (isEligibleRDC() && weightRDC() > 1.0) { + rdcCandidate = true; + CODE_PROBE(true, "Granule read triggering read-driven compaction"); + if (BW_DEBUG) { + fmt::print("Triggering read-driven compaction of [{0} - {1})\n", + keyRange.begin.printable(), + keyRange.end.printable()); + } + return true; + } + return false; + } + + inline bool doReadDrivenCompaction() { return runRDC.isSet(); } }; struct GranuleRangeMetadata { @@ -200,6 +272,7 @@ struct BlobWorkerData : NonCopyable, ReferenceCounted { NotifiedVersion grvVersion; Promise fatalError; Promise simInjectFailure; + Promise doReadDrivenCompaction; Reference initialSnapshotLock; Reference resnapshotLock; @@ -293,6 +366,13 @@ struct BlobWorkerData : NonCopyable, ReferenceCounted { return stats.estimatedMaxResidentMemory >= memoryFullThreshold; } + void triggerReadDrivenCompaction() { + Promise doRDC = doReadDrivenCompaction; + if (doRDC.canBeSet()) { + doRDC.send(Void()); + } + } + bool maybeInjectTargetedRestart() { // inject a BW restart at most once per test if (g_network->isSimulated() && !g_simulator->speedUpSimulation && @@ -2042,6 +2122,7 @@ ACTOR Future blobGranuleUpdateFiles(Reference bwData, metadata->pendingDeltaVersion = startVersion; metadata->bufferedDeltaVersion = startVersion; metadata->knownCommittedVersion = startVersion; + metadata->resetReadStats(); Reference cfData = makeReference(bwData->db.getPtr()); @@ -2184,6 +2265,10 @@ ACTOR Future blobGranuleUpdateFiles(Reference bwData, } nextForceFlush = metadata->forceFlushVersion.whenAtLeast(lastForceFlushVersion + 1); } + when(wait(metadata->runRDC.getFuture())) { + // return control flow back to the triggering actor before continuing + wait(delay(0)); + } } } catch (Error& e) { // only error we should expect here is when we finish consuming old change feed @@ -2310,6 +2395,7 @@ ACTOR Future blobGranuleUpdateFiles(Reference bwData, startState.granuleID, inFlightFiles.empty() ? Future(Void()) : success(inFlightFiles.back().future)); + metadata->resetReadStats(); } // reset force flush state, requests should retry and add it back once feed is ready forceFlushVersions.clear(); @@ -2418,20 +2504,20 @@ ACTOR Future blobGranuleUpdateFiles(Reference bwData, // The force flush contract is a version cannot be put in forceFlushVersion unless the change feed // is already whenAtLeast that version bool forceFlush = !forceFlushVersions.empty() && forceFlushVersions.back() > metadata->pendingDeltaVersion; + bool doReadDrivenFlush = !metadata->currentDeltas.empty() && metadata->doReadDrivenCompaction(); CODE_PROBE(forceFlush, "Force flushing granule"); - if (metadata->bufferedDeltaBytes >= SERVER_KNOBS->BG_DELTA_FILE_TARGET_BYTES || forceFlush) { + if (metadata->bufferedDeltaBytes >= SERVER_KNOBS->BG_DELTA_FILE_TARGET_BYTES || forceFlush || + doReadDrivenFlush) { TraceEvent(SevDebug, "BlobGranuleDeltaFile", bwData->id) .detail("Granule", metadata->keyRange) .detail("Version", lastDeltaVersion); // sanity check for version order - - if (forceFlush) { + if (forceFlush || doReadDrivenFlush) { if (lastDeltaVersion == invalidVersion) { - lastDeltaVersion = metadata->currentDeltas.empty() ? metadata->pendingDeltaVersion - : metadata->currentDeltas.back().version; + lastDeltaVersion = metadata->bufferedDeltaVersion; } - if (lastDeltaVersion < forceFlushVersions.back()) { + if (!forceFlushVersions.empty() && lastDeltaVersion < forceFlushVersions.back()) { if (BW_DEBUG) { fmt::print("Granule [{0} - {1}) force flushing delta version {2} -> {3}\n", metadata->keyRange.begin.printable(), @@ -2443,13 +2529,6 @@ ACTOR Future blobGranuleUpdateFiles(Reference bwData, } } if (!metadata->currentDeltas.empty()) { - if (lastDeltaVersion < metadata->currentDeltas.back().version) { - fmt::print("Granule [{0} - {1}) LDV {2} < DeltaBack {3}\n", - metadata->keyRange.begin.printable(), - metadata->keyRange.end.printable(), - lastDeltaVersion, - metadata->currentDeltas.back().version); - } ASSERT(lastDeltaVersion >= metadata->currentDeltas.back().version); ASSERT(metadata->pendingDeltaVersion < metadata->currentDeltas.front().version); } else { @@ -2506,6 +2585,7 @@ ACTOR Future blobGranuleUpdateFiles(Reference bwData, // add new pending delta file ASSERT(metadata->pendingDeltaVersion < lastDeltaVersion); metadata->pendingDeltaVersion = lastDeltaVersion; + ASSERT(metadata->bufferedDeltaVersion <= lastDeltaVersion); metadata->bufferedDeltaVersion = lastDeltaVersion; // In case flush was forced at non-mutation version metadata->bytesInNewDeltaFiles += metadata->bufferedDeltaBytes; @@ -2527,6 +2607,9 @@ ACTOR Future blobGranuleUpdateFiles(Reference bwData, // Wait on delta file starting here. If we have too many pending delta file writes, we need to not // continue to consume from the change feed, as that will pile on even more delta files to write wait(startDeltaFileWrite); + } else if (metadata->doReadDrivenCompaction()) { + ASSERT(metadata->currentDeltas.empty()); + snapshotEligible = true; } // FIXME: if we're still reading from old change feed, we should probably compact if we're @@ -2534,7 +2617,8 @@ ACTOR Future blobGranuleUpdateFiles(Reference bwData, // yet // If we have enough delta files, try to re-snapshot - if (snapshotEligible && metadata->bytesInNewDeltaFiles >= SERVER_KNOBS->BG_DELTA_BYTES_BEFORE_COMPACT) { + if (snapshotEligible && (metadata->doReadDrivenCompaction() || + metadata->bytesInNewDeltaFiles >= SERVER_KNOBS->BG_DELTA_BYTES_BEFORE_COMPACT)) { if (BW_DEBUG && !inFlightFiles.empty()) { fmt::print("Granule [{0} - {1}) ready to re-snapshot at {2} after {3} > {4} bytes, " "waiting for " @@ -2582,6 +2666,7 @@ ACTOR Future blobGranuleUpdateFiles(Reference bwData, // reset metadata metadata->bytesInNewDeltaFiles = 0; + metadata->resetReadStats(); // If we have more than one snapshot file and that file is unblocked (committedVersion >= // snapshotVersion), wait for it to finish @@ -3739,6 +3824,11 @@ ACTOR Future doBlobGranuleFileRequest(Reference bwData, Bl } } } + + // don't update read stats on a summarize read + if (metadata->updateReadStats(req.readVersion, chunk)) { + bwData->triggerReadDrivenCompaction(); + } } rep.chunks.push_back(rep.arena, chunk); @@ -4553,6 +4643,74 @@ ACTOR Future runGRVChecks(Reference bwData) { } } +struct RDCEntry { + double weight; + Reference granule; + RDCEntry(double weight, Reference granule) : weight(weight), granule(granule) {} +}; + +// for a top-k algorithm, we actually want a min-heap, so reverse the sort order +struct OrderForTopK { + bool operator()(RDCEntry const& a, RDCEntry const& b) const { return b.weight - a.weight; } +}; + +typedef std::priority_queue, OrderForTopK> TopKPQ; + +ACTOR Future runReadDrivenCompaction(Reference bwData) { + state bool processedAll = true; + loop { + if (processedAll) { + wait(bwData->doReadDrivenCompaction.getFuture()); + bwData->doReadDrivenCompaction.reset(); + wait(delay(0)); + } + + TopKPQ topK; + + // FIXME: possible to scan candidates instead of all granules? + int candidates = 0; + auto allRanges = bwData->granuleMetadata.intersectingRanges(normalKeys); + for (auto& it : allRanges) { + if (it.value().activeMetadata.isValid() && it.value().activeMetadata->cancelled.canBeSet()) { + auto metadata = it.value().activeMetadata; + if (metadata->rdcCandidate && metadata->isEligibleRDC() && metadata->runRDC.canBeSet() && + metadata->pendingSnapshotVersion == metadata->durableSnapshotVersion.get()) { + candidates++; + double weight = metadata->weightRDC(); + if (weight > 1.0 && + (topK.size() < SERVER_KNOBS->BLOB_WORKER_RDC_PARALLELISM || weight > topK.top().weight)) { + if (topK.size() == SERVER_KNOBS->BLOB_WORKER_RDC_PARALLELISM) { + topK.pop(); + } + topK.push(RDCEntry(weight, metadata)); + } + } + } + } + + CODE_PROBE(candidates > topK.size(), "Too many read-driven compaction candidates for one cycle"); + + std::vector> futures; + futures.reserve(topK.size()); + while (!topK.empty()) { + ++bwData->stats.readDrivenCompactions; + Promise runRDC = topK.top().granule->runRDC; + ASSERT(runRDC.canBeSet()); + Future waitForSnapshotComplete = topK.top().granule->durableSnapshotVersion.whenAtLeast( + topK.top().granule->durableSnapshotVersion.get() + 1) || + topK.top().granule->cancelled.getFuture(); + futures.push_back(waitForSnapshotComplete); + topK.pop(); + runRDC.send(Void()); + } + processedAll = futures.empty(); + if (!futures.empty()) { + // wait at least one second to throttle this actor a bit + wait(waitForAll(futures) && delay(1.0)); + } + } +} + // FIXME: better way to do this? // monitor system keyspace for new tenants ACTOR Future monitorTenants(Reference bwData) { @@ -4890,6 +5048,7 @@ ACTOR Future blobWorker(BlobWorkerInterface bwInterf, self->addActor.send(waitFailureServer(bwInterf.waitFailure.getFuture())); self->addActor.send(runGRVChecks(self)); self->addActor.send(monitorTenants(self)); + self->addActor.send(runReadDrivenCompaction(self)); state Future selfRemoved = monitorRemoval(self); if (g_network->isSimulated() && BUGGIFY_WITH_PROB(0.25)) { self->addActor.send(simForceFileWriteContention(self)); diff --git a/fdbserver/workloads/BlobGranuleVerifier.actor.cpp b/fdbserver/workloads/BlobGranuleVerifier.actor.cpp index 3b87852af9..c904649f56 100644 --- a/fdbserver/workloads/BlobGranuleVerifier.actor.cpp +++ b/fdbserver/workloads/BlobGranuleVerifier.actor.cpp @@ -305,6 +305,8 @@ struct BlobGranuleVerifierWorkload : TestWorkload { state Version prevPurgeVersion = -1; state UID dbgId = debugRandom()->randomUniqueID(); state Version newPurgeVersion = 0; + // usually we want randomness to verify maximum data, but sometimes hotspotting a subset is good too + state bool pickGranuleUniform = deterministicRandom()->random01() < 0.1; TraceEvent("BlobGranuleVerifierStart"); if (BGV_DEBUG) { @@ -458,7 +460,13 @@ struct BlobGranuleVerifierWorkload : TestWorkload { } // pick a random range - int rIndex = deterministicRandom()->randomInt(0, self->granuleRanges.get().size()); + size_t granuleCount = self->granuleRanges.get().size(); + size_t rIndex; + if (pickGranuleUniform) { + rIndex = deterministicRandom()->randomInt(0, granuleCount); + } else { + rIndex = deterministicRandom()->randomSkewedUInt32(0, granuleCount); + } state KeyRange range = self->granuleRanges.get()[rIndex]; state std::pair fdb = wait(readFromFDB(cx, range)); From 6f37f55917b463bb028a7d08a63ea497d9bda58d Mon Sep 17 00:00:00 2001 From: Nim Wijetunga Date: Wed, 26 Oct 2022 09:38:27 -0700 Subject: [PATCH 47/95] Restore System Keys First in Backup/Restore Workloads (#8475) * system key restore ordering * restore system keys before regular data * atomic restore backup fix * change testing * fix compile error * fix compile issue * fix compile issues * Trigger Build * only split restore if encryption is enabled * revert knob changes * Update fdbserver/workloads/AtomicSwitchover.actor.cpp Co-authored-by: A.J. Beamon * Update fdbserver/workloads/AtomicSwitchover.actor.cpp Co-authored-by: A.J. Beamon * Update fdbserver/workloads/BackupCorrectness.actor.cpp Co-authored-by: A.J. Beamon * Update fdbserver/workloads/AtomicRestore.actor.cpp Co-authored-by: A.J. Beamon * add todo * strengthen check * seperate system restore for atomic restore * address pr comments * address pr comments Co-authored-by: A.J. Beamon --- fdbbackup/backup.actor.cpp | 1 + fdbclient/FileBackupAgent.actor.cpp | 208 +++++++++++++----- .../include/fdbclient/BackupAgent.actor.h | 1 + .../fdbclient/TenantEntryCache.actor.h | 4 + fdbserver/workloads/AtomicRestore.actor.cpp | 5 +- ...kupAndParallelRestoreCorrectness.actor.cpp | 9 +- .../workloads/BackupCorrectness.actor.cpp | 76 ++++++- fdbserver/workloads/BackupToBlob.actor.cpp | 5 +- .../workloads/BackupToDBCorrectness.actor.cpp | 44 +++- .../workloads/IncrementalBackup.actor.cpp | 44 +++- fdbserver/workloads/RestoreBackup.actor.cpp | 46 +++- fdbserver/workloads/RestoreFromBlob.actor.cpp | 21 +- fdbserver/workloads/SubmitBackup.actor.cpp | 7 +- tests/fast/EncryptedBackupCorrectness.toml | 1 - 14 files changed, 391 insertions(+), 81 deletions(-) diff --git a/fdbbackup/backup.actor.cpp b/fdbbackup/backup.actor.cpp index a55a6f83df..64233b8e74 100644 --- a/fdbbackup/backup.actor.cpp +++ b/fdbbackup/backup.actor.cpp @@ -2365,6 +2365,7 @@ ACTOR Future runRestore(Database db, KeyRef(addPrefix), KeyRef(removePrefix), LockDB::True, + UnlockDB::True, onlyApplyMutationLogs, inconsistentSnapshotOnly, beginVersion, diff --git a/fdbclient/FileBackupAgent.actor.cpp b/fdbclient/FileBackupAgent.actor.cpp index 52bb607d8d..137f401df0 100644 --- a/fdbclient/FileBackupAgent.actor.cpp +++ b/fdbclient/FileBackupAgent.actor.cpp @@ -167,6 +167,7 @@ public: KeyBackedProperty removePrefix() { return configSpace.pack(__FUNCTION__sr); } KeyBackedProperty onlyApplyMutationLogs() { return configSpace.pack(__FUNCTION__sr); } KeyBackedProperty inconsistentSnapshotOnly() { return configSpace.pack(__FUNCTION__sr); } + KeyBackedProperty unlockDBAfterRestore() { return configSpace.pack(__FUNCTION__sr); } // XXX: Remove restoreRange() once it is safe to remove. It has been changed to restoreRanges KeyBackedProperty restoreRange() { return configSpace.pack(__FUNCTION__sr); } KeyBackedProperty> restoreRanges() { return configSpace.pack(__FUNCTION__sr); } @@ -591,12 +592,11 @@ struct EncryptedRangeFileWriter : public IRangeFileWriter { } ACTOR static Future decryptImpl(Database cx, - StringRef headerS, + BlobCipherEncryptHeader header, const uint8_t* dataP, int64_t dataLen, Arena* arena) { Reference const> dbInfo = cx->clientInfo; - state BlobCipherEncryptHeader header = BlobCipherEncryptHeader::fromStringRef(headerS); TextAndHeaderCipherKeys cipherKeys = wait(getEncryptCipherKeys(dbInfo, header, BlobCipherMetrics::BACKUP)); ASSERT(cipherKeys.cipherHeaderKey.isValid() && cipherKeys.cipherTextKey.isValid()); validateEncryptionHeader(cipherKeys.cipherHeaderKey, cipherKeys.cipherTextKey, header); @@ -606,7 +606,7 @@ struct EncryptedRangeFileWriter : public IRangeFileWriter { } static Future decrypt(Database cx, - StringRef headerS, + BlobCipherEncryptHeader headerS, const uint8_t* dataP, int64_t dataLen, Arena* arena) { @@ -651,7 +651,7 @@ struct EncryptedRangeFileWriter : public IRangeFileWriter { } ACTOR static Future updateEncryptionKeysCtx(EncryptedRangeFileWriter* self, KeyRef key) { - state std::pair curTenantInfo = wait(getEncryptionDomainDetails(key, self)); + state std::pair curTenantInfo = wait(getEncryptionDomainDetails(key, self->tenantCache)); state Reference const> dbInfo = self->cx->clientInfo; // Get text and header cipher key @@ -693,12 +693,13 @@ struct EncryptedRangeFileWriter : public IRangeFileWriter { static bool isSystemKey(KeyRef key) { return key.size() && key[0] == systemKeys.begin[0]; } - ACTOR static Future> - getEncryptionDomainDetailsImpl(KeyRef key, Reference> tenantCache, bool useTenantCache) { + ACTOR static Future> getEncryptionDomainDetailsImpl( + KeyRef key, + Reference> tenantCache) { if (isSystemKey(key)) { return std::make_pair(SYSTEM_KEYSPACE_ENCRYPT_DOMAIN_ID, FDB_SYSTEM_KEYSPACE_ENCRYPT_DOMAIN_NAME); } - if (key.size() < TENANT_PREFIX_SIZE || !useTenantCache) { + if (key.size() < TENANT_PREFIX_SIZE) { return std::make_pair(FDB_DEFAULT_ENCRYPT_DOMAIN_ID, FDB_DEFAULT_ENCRYPT_DOMAIN_NAME); } KeyRef tenantPrefix = KeyRef(key.begin(), TENANT_PREFIX_SIZE); @@ -710,21 +711,10 @@ struct EncryptedRangeFileWriter : public IRangeFileWriter { return std::make_pair(FDB_DEFAULT_ENCRYPT_DOMAIN_ID, FDB_DEFAULT_ENCRYPT_DOMAIN_NAME); } - static Future> getEncryptionDomainDetails(KeyRef key, - EncryptedRangeFileWriter* self) { - // If tenants are disabled on a cluster then don't use the TenantEntryCache as it will result in alot of - // unnecessary cache misses. For a cluster configured in TenantMode::Optional, the backup performance may - // degrade if most of the mutations belong to an invalid tenant - TenantMode mode = self->cx->clientInfo->get().tenantMode; - bool useTenantCache = mode != TenantMode::DISABLED; - if (g_network->isSimulated() && mode == TenantMode::OPTIONAL_TENANT) { - // TODO: Currently simulation tests run with optional tenant mode but most data does not belong to any - // tenant. This results in many timeouts so disable using the tenant cache until optional tenant mode - // support with backups is more performant - useTenantCache = false; - } - CODE_PROBE(useTenantCache, "using tenant cache"); - return getEncryptionDomainDetailsImpl(key, self->tenantCache, useTenantCache); + static Future> getEncryptionDomainDetails( + KeyRef key, + Reference> tenantCache) { + return getEncryptionDomainDetailsImpl(key, tenantCache); } // Handles the first block and internal blocks. Ends current block if needed. @@ -816,6 +806,7 @@ struct EncryptedRangeFileWriter : public IRangeFileWriter { curKeyTenantInfo.first != FDB_DEFAULT_ENCRYPT_DOMAIN_ID) { endKey = StringRef(k.begin(), TENANT_PREFIX_SIZE); } + state ValueRef newValue = StringRef(); self->lastKey = k; self->lastValue = v; @@ -834,9 +825,9 @@ struct EncryptedRangeFileWriter : public IRangeFileWriter { if (self->lastKey.size() == 0 || k.size() == 0) { return false; } - state std::pair curKeyTenantInfo = wait(getEncryptionDomainDetails(k, self)); - state std::pair prevKeyTenantInfo = wait(getEncryptionDomainDetails(self->lastKey, self)); - // crossing tenant boundaries so finish the current block using only the tenant prefix of the new key + state std::pair curKeyTenantInfo = wait(getEncryptionDomainDetails(k, self->tenantCache)); + state std::pair prevKeyTenantInfo = + wait(getEncryptionDomainDetails(self->lastKey, self->tenantCache)); if (curKeyTenantInfo.first != prevKeyTenantInfo.first) { CODE_PROBE(true, "crossed tenant boundaries"); wait(handleTenantBondary(self, k, v, writeValue, curKeyTenantInfo)); @@ -1040,11 +1031,18 @@ private: Key lastValue; }; -void decodeKVPairs(StringRefReader* reader, Standalone>* results) { +ACTOR static Future decodeKVPairs(StringRefReader* reader, + Standalone>* results, + bool encryptedBlock, + Optional>> tenantCache, + Optional encryptHeader) { // Read begin key, if this fails then block was invalid. - uint32_t kLen = reader->consumeNetworkUInt32(); - const uint8_t* k = reader->consume(kLen); + state uint32_t kLen = reader->consumeNetworkUInt32(); + state const uint8_t* k = reader->consume(kLen); results->push_back(results->arena(), KeyValueRef(KeyRef(k, kLen), ValueRef())); + state KeyRef prevKey = KeyRef(k, kLen); + state bool done = false; + state Optional> prevTenantInfo; // Read kv pairs and end key while (1) { @@ -1052,6 +1050,35 @@ void decodeKVPairs(StringRefReader* reader, Standalone>* kLen = reader->consumeNetworkUInt32(); k = reader->consume(kLen); + // make sure that all keys in a block belong to exactly one tenant, + // unless its the last key in which case it can be a truncated (different) tenant prefix + if (encryptedBlock && g_network && g_network->isSimulated()) { + ASSERT(tenantCache.present()); + ASSERT(encryptHeader.present()); + state KeyRef curKey = KeyRef(k, kLen); + if (!prevTenantInfo.present()) { + std::pair tenantInfo = + wait(EncryptedRangeFileWriter::getEncryptionDomainDetails(prevKey, tenantCache.get())); + prevTenantInfo = tenantInfo; + } + std::pair curTenantInfo = + wait(EncryptedRangeFileWriter::getEncryptionDomainDetails(curKey, tenantCache.get())); + if (!curKey.empty() && !prevKey.empty() && prevTenantInfo.get().first != curTenantInfo.first) { + ASSERT(!done); + if (curTenantInfo.first != SYSTEM_KEYSPACE_ENCRYPT_DOMAIN_ID && + curTenantInfo.first != FDB_DEFAULT_ENCRYPT_DOMAIN_ID) { + ASSERT(curKey.size() == TENANT_PREFIX_SIZE); + } + done = true; + } + // make sure that all keys (except possibly the last key) in a block are encrypted using the correct key + if (!prevKey.empty()) { + ASSERT(prevTenantInfo.get().first == encryptHeader.get().cipherTextDetails.encryptDomainId); + } + prevKey = curKey; + prevTenantInfo = curTenantInfo; + } + // If eof reached or first value len byte is 0xFF then a valid block end was reached. if (reader->eof() || *reader->rptr == 0xFF) { results->push_back(results->arena(), KeyValueRef(KeyRef(k, kLen), ValueRef())); @@ -1072,6 +1099,8 @@ void decodeKVPairs(StringRefReader* reader, Standalone>* for (auto b : reader->remainder()) if (b != 0xFF) throw restore_corrupted_data_padding(); + + return Void(); } ACTOR Future>> decodeRangeFileBlock(Reference file, @@ -1094,7 +1123,11 @@ ACTOR Future>> decodeRangeFileBlock(Reference< // BACKUP_AGENT_ENCRYPTED_SNAPSHOT_FILE_VERSION int32_t file_version = reader.consume(); if (file_version == BACKUP_AGENT_SNAPSHOT_FILE_VERSION) { - decodeKVPairs(&reader, &results); + wait(decodeKVPairs(&reader, + &results, + false, + Optional>>(), + Optional())); } else if (file_version == BACKUP_AGENT_ENCRYPTED_SNAPSHOT_FILE_VERSION) { CODE_PROBE(true, "decoding encrypted block"); ASSERT(cx.present()); @@ -1108,7 +1141,8 @@ ACTOR Future>> decodeRangeFileBlock(Reference< // read encryption header const uint8_t* headerStart = reader.consume(BlobCipherEncryptHeader::headerSize); - StringRef header = StringRef(headerStart, BlobCipherEncryptHeader::headerSize); + StringRef headerS = StringRef(headerStart, BlobCipherEncryptHeader::headerSize); + state BlobCipherEncryptHeader header = BlobCipherEncryptHeader::fromStringRef(headerS); const uint8_t* dataPayloadStart = headerStart + BlobCipherEncryptHeader::headerSize; // calculate the total bytes read up to (and including) the header int64_t bytesRead = sizeof(int32_t) + sizeof(uint32_t) + optionsLen + BlobCipherEncryptHeader::headerSize; @@ -1117,7 +1151,12 @@ ACTOR Future>> decodeRangeFileBlock(Reference< StringRef decryptedData = wait(EncryptedRangeFileWriter::decrypt(cx.get(), header, dataPayloadStart, dataLen, &results.arena())); reader = StringRefReader(decryptedData, restore_corrupted_data()); - decodeKVPairs(&reader, &results); + state Optional>> tenantCache; + if (g_network && g_simulator->isSimulated()) { + tenantCache = makeReference>(cx.get(), TenantEntryCacheRefreshMode::WATCH); + wait(tenantCache.get()->init()); + } + wait(decodeKVPairs(&reader, &results, true, tenantCache, header)); } else { throw restore_unsupported_file_version(); } @@ -3398,6 +3437,8 @@ struct RestoreCompleteTaskFunc : RestoreTaskFuncBase { state RestoreConfig restore(task); restore.stateEnum().set(tr, ERestoreState::COMPLETED); + state bool unlockDB = wait(restore.unlockDBAfterRestore().getD(tr, Snapshot::False, true)); + tr->atomicOp(metadataVersionKey, metadataVersionRequiredValue, MutationRef::SetVersionstampedValue); // Clear the file map now since it could be huge. restore.fileSet().clear(tr); @@ -3413,7 +3454,9 @@ struct RestoreCompleteTaskFunc : RestoreTaskFuncBase { restore.clearApplyMutationsKeys(tr); wait(taskBucket->finish(tr, task)); - wait(unlockDatabase(tr, restore.getUid())); + if (unlockDB) { + wait(unlockDatabase(tr, restore.getUid())); + } return Void(); } @@ -5172,6 +5215,7 @@ public: Key addPrefix, Key removePrefix, LockDB lockDB, + UnlockDB unlockDB, OnlyApplyMutationLogs onlyApplyMutationLogs, InconsistentSnapshotOnly inconsistentSnapshotOnly, Version beginVersion, @@ -5245,6 +5289,7 @@ public: restore.onlyApplyMutationLogs().set(tr, onlyApplyMutationLogs); restore.inconsistentSnapshotOnly().set(tr, inconsistentSnapshotOnly); restore.beginVersion().set(tr, beginVersion); + restore.unlockDBAfterRestore().set(tr, unlockDB); if (BUGGIFY && restoreRanges.size() == 1) { restore.restoreRange().set(tr, restoreRanges[0]); } else { @@ -5836,6 +5881,7 @@ public: Key addPrefix, Key removePrefix, LockDB lockDB, + UnlockDB unlockDB, OnlyApplyMutationLogs onlyApplyMutationLogs, InconsistentSnapshotOnly inconsistentSnapshotOnly, Version beginVersion, @@ -5892,6 +5938,7 @@ public: addPrefix, removePrefix, lockDB, + unlockDB, onlyApplyMutationLogs, inconsistentSnapshotOnly, beginVersion, @@ -6017,7 +6064,7 @@ public: } } - Reference bc = wait(backupConfig.backupContainer().getOrThrow(cx.getReference())); + state Reference bc = wait(backupConfig.backupContainer().getOrThrow(cx.getReference())); if (fastRestore) { TraceEvent("AtomicParallelRestoreStartRestore").log(); @@ -6043,24 +6090,80 @@ public: return -1; } else { TraceEvent("AS_StartRestore").log(); - Version ver = wait(restore(backupAgent, - cx, - cx, - tagName, - KeyRef(bc->getURL()), - bc->getProxy(), - ranges, - WaitForComplete::True, - ::invalidVersion, - Verbose::True, - addPrefix, - removePrefix, - LockDB::True, - OnlyApplyMutationLogs::False, - InconsistentSnapshotOnly::False, - ::invalidVersion, - {}, - randomUid)); + state Standalone> restoreRange; + state Standalone> systemRestoreRange; + bool encryptionEnabled = cx->clientInfo->get().isEncryptionEnabled; + for (auto r : ranges) { + if (!encryptionEnabled || !r.intersects(getSystemBackupRanges())) { + restoreRange.push_back_deep(restoreRange.arena(), r); + } else { + KeyRangeRef normalKeyRange = r & normalKeys; + KeyRangeRef systemKeyRange = r & systemKeys; + if (!normalKeyRange.empty()) { + restoreRange.push_back_deep(restoreRange.arena(), normalKeyRange); + } + if (!systemKeyRange.empty()) { + systemRestoreRange.push_back_deep(systemRestoreRange.arena(), systemKeyRange); + } + } + } + if (!systemRestoreRange.empty()) { + // restore system keys + wait(success(restore(backupAgent, + cx, + cx, + "system_restore"_sr, + KeyRef(bc->getURL()), + bc->getProxy(), + systemRestoreRange, + WaitForComplete::True, + ::invalidVersion, + Verbose::True, + addPrefix, + removePrefix, + LockDB::True, + UnlockDB::False, + OnlyApplyMutationLogs::False, + InconsistentSnapshotOnly::False, + ::invalidVersion, + {}, + randomUid))); + state Reference rywTransaction = + Reference(new ReadYourWritesTransaction(cx)); + // clear old restore config associated with system keys + loop { + try { + rywTransaction->setOption(FDBTransactionOptions::ACCESS_SYSTEM_KEYS); + rywTransaction->setOption(FDBTransactionOptions::LOCK_AWARE); + state RestoreConfig oldRestore(randomUid); + oldRestore.clear(rywTransaction); + wait(rywTransaction->commit()); + break; + } catch (Error& e) { + wait(rywTransaction->onError(e)); + } + } + } + // restore user data + state Version ver = wait(restore(backupAgent, + cx, + cx, + tagName, + KeyRef(bc->getURL()), + bc->getProxy(), + restoreRange, + WaitForComplete::True, + ::invalidVersion, + Verbose::True, + addPrefix, + removePrefix, + LockDB::True, + UnlockDB::True, + OnlyApplyMutationLogs::False, + InconsistentSnapshotOnly::False, + ::invalidVersion, + {}, + randomUid)); return ver; } } @@ -6120,6 +6223,7 @@ Future FileBackupAgent::restore(Database cx, Key addPrefix, Key removePrefix, LockDB lockDB, + UnlockDB unlockDB, OnlyApplyMutationLogs onlyApplyMutationLogs, InconsistentSnapshotOnly inconsistentSnapshotOnly, Version beginVersion, @@ -6137,6 +6241,7 @@ Future FileBackupAgent::restore(Database cx, addPrefix, removePrefix, lockDB, + unlockDB, onlyApplyMutationLogs, inconsistentSnapshotOnly, beginVersion, @@ -6178,6 +6283,7 @@ Future FileBackupAgent::restore(Database cx, addPrefix, removePrefix, lockDB, + UnlockDB::True, onlyApplyMutationLogs, inconsistentSnapshotOnly, beginVersion, diff --git a/fdbclient/include/fdbclient/BackupAgent.actor.h b/fdbclient/include/fdbclient/BackupAgent.actor.h index 314f151fd0..de1f5a06f4 100644 --- a/fdbclient/include/fdbclient/BackupAgent.actor.h +++ b/fdbclient/include/fdbclient/BackupAgent.actor.h @@ -196,6 +196,7 @@ public: Key addPrefix = Key(), Key removePrefix = Key(), LockDB = LockDB::True, + UnlockDB = UnlockDB::True, OnlyApplyMutationLogs = OnlyApplyMutationLogs::False, InconsistentSnapshotOnly = InconsistentSnapshotOnly::False, Version beginVersion = ::invalidVersion, diff --git a/fdbclient/include/fdbclient/TenantEntryCache.actor.h b/fdbclient/include/fdbclient/TenantEntryCache.actor.h index 4ff5438d5a..0429494714 100644 --- a/fdbclient/include/fdbclient/TenantEntryCache.actor.h +++ b/fdbclient/include/fdbclient/TenantEntryCache.actor.h @@ -68,6 +68,10 @@ using TenantEntryCachePayloadFunc = std::function(con // 1. Lookup by 'TenantId' // 2. Lookup by 'TenantPrefix' // 3. Lookup by 'TenantName' +// TODO: Currently this cache performs poorly if there are tenant access happening to unknown tenants which happens most +// frequently in optional tenant mode but can also happen in required mode if there are alot of tenants created. Further +// as a consequence of the design we cannot be sure that the state of a given tenant is accurate even if its present in +// the cache. template class TenantEntryCache : public ReferenceCounted>, NonCopyable { diff --git a/fdbserver/workloads/AtomicRestore.actor.cpp b/fdbserver/workloads/AtomicRestore.actor.cpp index 2acc8b092d..6f90f900d7 100644 --- a/fdbserver/workloads/AtomicRestore.actor.cpp +++ b/fdbserver/workloads/AtomicRestore.actor.cpp @@ -18,6 +18,7 @@ * limitations under the License. */ +#include "fdbclient/ManagementAPI.actor.h" #include "fdbrpc/simulator.h" #include "fdbclient/BackupAgent.actor.h" #include "fdbserver/Knobs.h" @@ -95,6 +96,7 @@ struct AtomicRestoreWorkload : TestWorkload { TraceEvent("AtomicRestore_Start").detail("UsePartitionedLog", self->usePartitionedLogs); state std::string backupContainer = "file://simfdb/backups/"; + state DatabaseConfiguration conf = wait(getDatabaseConfiguration(cx)); try { wait(backupAgent.submitBackup(cx, StringRef(backupContainer), @@ -103,7 +105,8 @@ struct AtomicRestoreWorkload : TestWorkload { deterministicRandom()->randomInt(0, 100), BackupAgentBase::getDefaultTagName(), self->backupRanges, - SERVER_KNOBS->ENABLE_ENCRYPTION, + SERVER_KNOBS->ENABLE_ENCRYPTION && + conf.tenantMode != TenantMode::OPTIONAL_TENANT, StopWhenDone::False, self->usePartitionedLogs)); } catch (Error& e) { diff --git a/fdbserver/workloads/BackupAndParallelRestoreCorrectness.actor.cpp b/fdbserver/workloads/BackupAndParallelRestoreCorrectness.actor.cpp index 0feee4f55c..4f8dbc23bb 100644 --- a/fdbserver/workloads/BackupAndParallelRestoreCorrectness.actor.cpp +++ b/fdbserver/workloads/BackupAndParallelRestoreCorrectness.actor.cpp @@ -215,7 +215,7 @@ struct BackupAndParallelRestoreCorrectnessWorkload : TestWorkload { state std::string backupContainer = "file://simfdb/backups/"; state Future status = statusLoop(cx, tag.toString()); - + state DatabaseConfiguration configuration = wait(getDatabaseConfiguration(cx)); try { wait(backupAgent->submitBackup(cx, StringRef(backupContainer), @@ -224,7 +224,8 @@ struct BackupAndParallelRestoreCorrectnessWorkload : TestWorkload { deterministicRandom()->randomInt(0, 100), tag.toString(), backupRanges, - SERVER_KNOBS->ENABLE_ENCRYPTION, + SERVER_KNOBS->ENABLE_ENCRYPTION && + configuration.tenantMode != TenantMode::OPTIONAL_TENANT, StopWhenDone{ !stopDifferentialDelay }, self->usePartitionedLogs)); } catch (Error& e) { @@ -474,6 +475,7 @@ struct BackupAndParallelRestoreCorrectnessWorkload : TestWorkload { // Occasionally start yet another backup that might still be running when we restore if (!self->locked && BUGGIFY) { TraceEvent("BARW_SubmitBackup2", randomID).detail("Tag", printable(self->backupTag)); + state DatabaseConfiguration configuration = wait(getDatabaseConfiguration(cx)); try { // Note the "partitionedLog" must be false, because we change // the configuration to disable backup workers before restore. @@ -484,7 +486,8 @@ struct BackupAndParallelRestoreCorrectnessWorkload : TestWorkload { deterministicRandom()->randomInt(0, 100), self->backupTag.toString(), self->backupRanges, - SERVER_KNOBS->ENABLE_ENCRYPTION, + SERVER_KNOBS->ENABLE_ENCRYPTION && + configuration.tenantMode != TenantMode::OPTIONAL_TENANT, StopWhenDone::True, UsePartitionedLog::False); } catch (Error& e) { diff --git a/fdbserver/workloads/BackupCorrectness.actor.cpp b/fdbserver/workloads/BackupCorrectness.actor.cpp index 7bd309457e..7e65e60e10 100644 --- a/fdbserver/workloads/BackupCorrectness.actor.cpp +++ b/fdbserver/workloads/BackupCorrectness.actor.cpp @@ -18,6 +18,7 @@ * limitations under the License. */ +#include "fdbclient/ManagementAPI.actor.h" #include "fdbclient/ReadYourWrites.h" #include "fdbrpc/simulator.h" #include "fdbclient/BackupAgent.actor.h" @@ -331,7 +332,7 @@ struct BackupAndRestoreCorrectnessWorkload : TestWorkload { state std::string backupContainer = "file://simfdb/backups/"; state Future status = statusLoop(cx, tag.toString()); - + state DatabaseConfiguration configuration = wait(getDatabaseConfiguration(cx)); try { wait(backupAgent->submitBackup(cx, StringRef(backupContainer), @@ -340,7 +341,8 @@ struct BackupAndRestoreCorrectnessWorkload : TestWorkload { deterministicRandom()->randomInt(0, 2000), tag.toString(), backupRanges, - SERVER_KNOBS->ENABLE_ENCRYPTION, + SERVER_KNOBS->ENABLE_ENCRYPTION && + configuration.tenantMode != TenantMode::OPTIONAL_TENANT, StopWhenDone{ !stopDifferentialDelay }, UsePartitionedLog::False, IncrementalBackupOnly::False, @@ -515,6 +517,42 @@ struct BackupAndRestoreCorrectnessWorkload : TestWorkload { return Void(); } + ACTOR static Future clearAndRestoreSystemKeys(Database cx, + BackupAndRestoreCorrectnessWorkload* self, + FileBackupAgent* backupAgent, + Version targetVersion, + Reference lastBackupContainer, + Standalone> systemRestoreRanges) { + // restore system keys before restoring any other ranges + wait(runRYWTransaction(cx, [=](Reference tr) -> Future { + tr->setOption(FDBTransactionOptions::ACCESS_SYSTEM_KEYS); + for (auto& range : systemRestoreRanges) + tr->clear(range); + return Void(); + })); + state Standalone restoreTag(self->backupTag.toString() + "_system"); + printf("BackupCorrectness, backupAgent.restore is called for tag:%s\n", restoreTag.toString().c_str()); + wait(success(backupAgent->restore(cx, + cx, + restoreTag, + KeyRef(lastBackupContainer->getURL()), + lastBackupContainer->getProxy(), + systemRestoreRanges, + WaitForComplete::True, + targetVersion, + Verbose::True, + Key(), + Key(), + self->locked, + UnlockDB::True, + OnlyApplyMutationLogs::False, + InconsistentSnapshotOnly::False, + ::invalidVersion, + self->encryptionKeyFileName))); + printf("BackupCorrectness, backupAgent.restore finished for tag:%s\n", restoreTag.toString().c_str()); + return Void(); + } + ACTOR static Future _start(Database cx, BackupAndRestoreCorrectnessWorkload* self) { state FileBackupAgent backupAgent; state Future extraBackup; @@ -593,6 +631,7 @@ struct BackupAndRestoreCorrectnessWorkload : TestWorkload { // Occasionally start yet another backup that might still be running when we restore if (!self->locked && BUGGIFY) { TraceEvent("BARW_SubmitBackup2", randomID).detail("Tag", printable(self->backupTag)); + state DatabaseConfiguration configuration = wait(getDatabaseConfiguration(cx)); try { extraBackup = backupAgent.submitBackup(cx, "file://simfdb/backups/"_sr, @@ -601,7 +640,8 @@ struct BackupAndRestoreCorrectnessWorkload : TestWorkload { deterministicRandom()->randomInt(0, 100), self->backupTag.toString(), self->backupRanges, - SERVER_KNOBS->ENABLE_ENCRYPTION, + SERVER_KNOBS->ENABLE_ENCRYPTION && + configuration.tenantMode != TenantMode::OPTIONAL_TENANT, StopWhenDone::True); } catch (Error& e) { TraceEvent("BARW_SubmitBackup2Exception", randomID) @@ -638,7 +678,7 @@ struct BackupAndRestoreCorrectnessWorkload : TestWorkload { lastBackupContainer->getEncryptionKeyFileName()); BackupDescription desc = wait(container->describeBackup()); - Version targetVersion = -1; + state Version targetVersion = -1; if (desc.maxRestorableVersion.present()) { if (deterministicRandom()->random01() < 0.1) { targetVersion = desc.minRestorableVersion.get(); @@ -656,6 +696,32 @@ struct BackupAndRestoreCorrectnessWorkload : TestWorkload { state std::vector> restoreTags; state bool multipleRangesInOneTag = false; state int restoreIndex = 0; + // make sure system keys are not present in the restoreRanges as they will get restored first separately + // from the rest + Standalone> modifiedRestoreRanges; + Standalone> systemRestoreRanges; + for (int i = 0; i < self->restoreRanges.size(); ++i) { + if (!SERVER_KNOBS->ENABLE_ENCRYPTION || + !self->restoreRanges[i].intersects(getSystemBackupRanges())) { + modifiedRestoreRanges.push_back_deep(modifiedRestoreRanges.arena(), self->restoreRanges[i]); + } else { + KeyRangeRef normalKeyRange = self->restoreRanges[i] & normalKeys; + KeyRangeRef systemKeyRange = self->restoreRanges[i] & systemKeys; + if (!normalKeyRange.empty()) { + modifiedRestoreRanges.push_back_deep(modifiedRestoreRanges.arena(), normalKeyRange); + } + if (!systemKeyRange.empty()) { + systemRestoreRanges.push_back_deep(systemRestoreRanges.arena(), systemKeyRange); + } + } + } + self->restoreRanges = modifiedRestoreRanges; + if (!systemRestoreRanges.empty()) { + // We are able to restore system keys first since we restore an entire cluster at once rather than + // partial key ranges. + wait(clearAndRestoreSystemKeys( + cx, self, &backupAgent, targetVersion, lastBackupContainer, systemRestoreRanges)); + } if (deterministicRandom()->random01() < 0.5) { for (restoreIndex = 0; restoreIndex < self->restoreRanges.size(); restoreIndex++) { auto range = self->restoreRanges[restoreIndex]; @@ -703,6 +769,7 @@ struct BackupAndRestoreCorrectnessWorkload : TestWorkload { Key(), Key(), self->locked, + UnlockDB::True, OnlyApplyMutationLogs::False, InconsistentSnapshotOnly::False, ::invalidVersion, @@ -735,6 +802,7 @@ struct BackupAndRestoreCorrectnessWorkload : TestWorkload { Key(), Key(), self->locked, + UnlockDB::True, OnlyApplyMutationLogs::False, InconsistentSnapshotOnly::False, ::invalidVersion, diff --git a/fdbserver/workloads/BackupToBlob.actor.cpp b/fdbserver/workloads/BackupToBlob.actor.cpp index d1995f388b..fa09409280 100644 --- a/fdbserver/workloads/BackupToBlob.actor.cpp +++ b/fdbserver/workloads/BackupToBlob.actor.cpp @@ -21,6 +21,7 @@ #include "fdbrpc/simulator.h" #include "fdbclient/BackupAgent.actor.h" #include "fdbclient/BackupContainer.h" +#include "fdbclient/ManagementAPI.actor.h" #include "fdbserver/Knobs.h" #include "fdbserver/workloads/BlobStoreWorkload.h" #include "fdbserver/workloads/workloads.actor.h" @@ -57,6 +58,7 @@ struct BackupToBlobWorkload : TestWorkload { addDefaultBackupRanges(backupRanges); wait(delay(self->backupAfter)); + state DatabaseConfiguration configuration = wait(getDatabaseConfiguration(cx)); wait(backupAgent.submitBackup(cx, self->backupURL, {}, @@ -64,7 +66,8 @@ struct BackupToBlobWorkload : TestWorkload { self->snapshotInterval, self->backupTag.toString(), backupRanges, - SERVER_KNOBS->ENABLE_ENCRYPTION)); + SERVER_KNOBS->ENABLE_ENCRYPTION && + configuration.tenantMode != TenantMode::OPTIONAL_TENANT)); EBackupState backupStatus = wait(backupAgent.waitBackup(cx, self->backupTag.toString(), StopWhenDone::True)); TraceEvent("BackupToBlob_BackupStatus").detail("Status", BackupAgentBase::getStateText(backupStatus)); return Void(); diff --git a/fdbserver/workloads/BackupToDBCorrectness.actor.cpp b/fdbserver/workloads/BackupToDBCorrectness.actor.cpp index f56780d4af..02d05ef208 100644 --- a/fdbserver/workloads/BackupToDBCorrectness.actor.cpp +++ b/fdbserver/workloads/BackupToDBCorrectness.actor.cpp @@ -22,6 +22,7 @@ #include "fdbclient/BackupAgent.actor.h" #include "fdbclient/ClusterConnectionMemoryRecord.h" #include "fdbclient/TenantManagement.actor.h" +#include "fdbserver/Knobs.h" #include "fdbserver/workloads/workloads.actor.h" #include "fdbserver/workloads/BulkSetup.actor.h" #include "flow/ApiVersion.h" @@ -667,10 +668,47 @@ struct BackupToDBCorrectnessWorkload : TestWorkload { // wait(diffRanges(self->backupRanges, self->backupPrefix, cx, self->extraDB)); state Standalone> restoreRange; + state Standalone> systemRestoreRange; for (auto r : self->backupRanges) { - restoreRange.push_back_deep( - restoreRange.arena(), - KeyRangeRef(r.begin.withPrefix(self->backupPrefix), r.end.withPrefix(self->backupPrefix))); + if (!SERVER_KNOBS->ENABLE_ENCRYPTION || !r.intersects(getSystemBackupRanges())) { + restoreRange.push_back_deep( + restoreRange.arena(), + KeyRangeRef(r.begin.withPrefix(self->backupPrefix), r.end.withPrefix(self->backupPrefix))); + } else { + KeyRangeRef normalKeyRange = r & normalKeys; + KeyRangeRef systemKeyRange = r & systemKeys; + if (!normalKeyRange.empty()) { + restoreRange.push_back_deep(restoreRange.arena(), + KeyRangeRef(normalKeyRange.begin.withPrefix(self->backupPrefix), + normalKeyRange.end.withPrefix(self->backupPrefix))); + } + if (!systemKeyRange.empty()) { + systemRestoreRange.push_back_deep(systemRestoreRange.arena(), systemKeyRange); + } + } + } + + // restore system keys first before restoring user data + if (!systemRestoreRange.empty()) { + state Key systemRestoreTag = "restore_system"_sr; + try { + wait(restoreTool.submitBackup(cx, + systemRestoreTag, + systemRestoreRange, + StopWhenDone::True, + StringRef(), + self->backupPrefix, + self->locked, + DatabaseBackupAgent::PreBackupAction::CLEAR)); + } catch (Error& e) { + TraceEvent("BARW_DoBackupSubmitBackupException", randomID) + .error(e) + .detail("Tag", printable(systemRestoreTag)); + if (e.code() != error_code_backup_unneeded && e.code() != error_code_backup_duplicate) + throw; + } + wait(success(restoreTool.waitBackup(cx, systemRestoreTag))); + wait(restoreTool.unlockBackup(cx, systemRestoreTag)); } try { diff --git a/fdbserver/workloads/IncrementalBackup.actor.cpp b/fdbserver/workloads/IncrementalBackup.actor.cpp index 23cd2b6ca7..d2b984114f 100644 --- a/fdbserver/workloads/IncrementalBackup.actor.cpp +++ b/fdbserver/workloads/IncrementalBackup.actor.cpp @@ -20,6 +20,7 @@ #include "fdbclient/FDBTypes.h" #include "fdbclient/Knobs.h" +#include "fdbclient/ManagementAPI.actor.h" #include "fdbclient/SystemData.h" #include "fdbclient/ReadYourWrites.h" #include "fdbrpc/simulator.h" @@ -150,6 +151,7 @@ struct IncrementalBackupWorkload : TestWorkload { if (self->submitOnly) { TraceEvent("IBackupSubmitAttempt").log(); + state DatabaseConfiguration configuration = wait(getDatabaseConfiguration(cx)); try { wait(self->backupAgent.submitBackup(cx, self->backupDir, @@ -158,7 +160,8 @@ struct IncrementalBackupWorkload : TestWorkload { 1e8, self->tag.toString(), backupRanges, - SERVER_KNOBS->ENABLE_ENCRYPTION, + SERVER_KNOBS->ENABLE_ENCRYPTION && + configuration.tenantMode != TenantMode::OPTIONAL_TENANT, StopWhenDone::False, UsePartitionedLog::False, IncrementalBackupOnly::True)); @@ -227,19 +230,56 @@ struct IncrementalBackupWorkload : TestWorkload { .detail("Size", containers.size()) .detail("First", containers.front()); state Key backupURL = Key(containers.front()); + + state Standalone> restoreRange; + state Standalone> systemRestoreRange; + for (auto r : backupRanges) { + if (!SERVER_KNOBS->ENABLE_ENCRYPTION || !r.intersects(getSystemBackupRanges())) { + restoreRange.push_back_deep(restoreRange.arena(), r); + } else { + KeyRangeRef normalKeyRange = r & normalKeys; + KeyRangeRef systemKeyRange = r & systemKeys; + if (!normalKeyRange.empty()) { + restoreRange.push_back_deep(restoreRange.arena(), normalKeyRange); + } + if (!systemKeyRange.empty()) { + systemRestoreRange.push_back_deep(systemRestoreRange.arena(), systemKeyRange); + } + } + } + if (!systemRestoreRange.empty()) { + TraceEvent("IBackupSystemRestoreAttempt").detail("BeginVersion", beginVersion); + wait(success(self->backupAgent.restore(cx, + cx, + "system_restore"_sr, + backupURL, + {}, + systemRestoreRange, + WaitForComplete::True, + invalidVersion, + Verbose::True, + Key(), + Key(), + LockDB::True, + UnlockDB::True, + OnlyApplyMutationLogs::True, + InconsistentSnapshotOnly::False, + beginVersion))); + } TraceEvent("IBackupRestoreAttempt").detail("BeginVersion", beginVersion); wait(success(self->backupAgent.restore(cx, cx, Key(self->tag.toString()), backupURL, {}, - backupRanges, + restoreRange, WaitForComplete::True, invalidVersion, Verbose::True, Key(), Key(), LockDB::True, + UnlockDB::True, OnlyApplyMutationLogs::True, InconsistentSnapshotOnly::False, beginVersion))); diff --git a/fdbserver/workloads/RestoreBackup.actor.cpp b/fdbserver/workloads/RestoreBackup.actor.cpp index 7254d4b59d..02191ef1ea 100644 --- a/fdbserver/workloads/RestoreBackup.actor.cpp +++ b/fdbserver/workloads/RestoreBackup.actor.cpp @@ -24,6 +24,7 @@ #include "fdbrpc/simulator.h" #include "fdbclient/BackupAgent.actor.h" #include "fdbclient/BackupContainer.h" +#include "fdbserver/Knobs.h" #include "fdbserver/workloads/workloads.actor.h" #include "flow/actorcompiler.h" // This must be the last #include. @@ -113,14 +114,43 @@ struct RestoreBackupWorkload : TestWorkload { wait(delay(self->delayFor)); wait(waitOnBackup(self, cx)); wait(clearDatabase(cx)); - wait(success(self->backupAgent.restore(cx, - cx, - self->tag, - Key(self->backupContainer->getURL()), - self->backupContainer->getProxy(), - WaitForComplete::True, - ::invalidVersion, - Verbose::True))); + if (SERVER_KNOBS->ENABLE_ENCRYPTION) { + // restore system keys + VectorRef systemBackupRanges = getSystemBackupRanges(); + state std::vector> restores; + for (int i = 0; i < systemBackupRanges.size(); i++) { + restores.push_back((self->backupAgent.restore(cx, + cx, + "system_restore"_sr, + Key(self->backupContainer->getURL()), + self->backupContainer->getProxy(), + WaitForComplete::True, + ::invalidVersion, + Verbose::True, + systemBackupRanges[i]))); + } + waitForAll(restores); + // restore non-system keys + wait(success(self->backupAgent.restore(cx, + cx, + self->tag, + Key(self->backupContainer->getURL()), + self->backupContainer->getProxy(), + WaitForComplete::True, + ::invalidVersion, + Verbose::True, + normalKeys))); + } else { + wait(success(self->backupAgent.restore(cx, + cx, + self->tag, + Key(self->backupContainer->getURL()), + self->backupContainer->getProxy(), + WaitForComplete::True, + ::invalidVersion, + Verbose::True))); + } + return Void(); } diff --git a/fdbserver/workloads/RestoreFromBlob.actor.cpp b/fdbserver/workloads/RestoreFromBlob.actor.cpp index afea1585e8..41d3d0f569 100644 --- a/fdbserver/workloads/RestoreFromBlob.actor.cpp +++ b/fdbserver/workloads/RestoreFromBlob.actor.cpp @@ -18,9 +18,11 @@ * limitations under the License. */ +#include "fdbclient/SystemData.h" #include "fdbrpc/simulator.h" #include "fdbclient/BackupAgent.actor.h" #include "fdbclient/BackupContainer.h" +#include "fdbserver/Knobs.h" #include "fdbserver/workloads/BlobStoreWorkload.h" #include "fdbserver/workloads/workloads.actor.h" #include "fdbserver/workloads/BulkSetup.actor.h" @@ -52,13 +54,22 @@ struct RestoreFromBlobWorkload : TestWorkload { ACTOR static Future _start(Database cx, RestoreFromBlobWorkload* self) { state FileBackupAgent backupAgent; - state Standalone> restoreRanges; - - addDefaultBackupRanges(restoreRanges); wait(delay(self->restoreAfter)); - Version v = wait( - backupAgent.restore(cx, {}, self->backupTag, self->backupURL, {}, restoreRanges, self->waitForComplete)); + if (SERVER_KNOBS->ENABLE_ENCRYPTION) { + // restore system keys followed by user keys + wait(success(backupAgent.restore( + cx, {}, self->backupTag, self->backupURL, {}, getSystemBackupRanges(), self->waitForComplete))); + Standalone> restoreRanges; + restoreRanges.push_back_deep(restoreRanges.arena(), normalKeys); + wait(success(backupAgent.restore( + cx, {}, self->backupTag, self->backupURL, {}, restoreRanges, self->waitForComplete))); + } else { + Standalone> restoreRanges; + addDefaultBackupRanges(restoreRanges); + wait(success(backupAgent.restore( + cx, {}, self->backupTag, self->backupURL, {}, restoreRanges, self->waitForComplete))); + } return Void(); } diff --git a/fdbserver/workloads/SubmitBackup.actor.cpp b/fdbserver/workloads/SubmitBackup.actor.cpp index 8cf59652ed..417edcba4a 100644 --- a/fdbserver/workloads/SubmitBackup.actor.cpp +++ b/fdbserver/workloads/SubmitBackup.actor.cpp @@ -19,6 +19,7 @@ */ #include "fdbclient/FDBTypes.h" +#include "fdbclient/ManagementAPI.actor.h" #include "fdbclient/ReadYourWrites.h" #include "fdbrpc/simulator.h" #include "fdbclient/BackupAgent.actor.h" @@ -52,8 +53,9 @@ struct SubmitBackupWorkload : TestWorkload { ACTOR static Future _start(SubmitBackupWorkload* self, Database cx) { wait(delay(self->delayFor)); - Standalone> backupRanges; + state Standalone> backupRanges; addDefaultBackupRanges(backupRanges); + state DatabaseConfiguration configuration = wait(getDatabaseConfiguration(cx)); try { wait(self->backupAgent.submitBackup(cx, self->backupDir, @@ -62,7 +64,8 @@ struct SubmitBackupWorkload : TestWorkload { self->snapshotInterval, self->tag.toString(), backupRanges, - SERVER_KNOBS->ENABLE_ENCRYPTION, + SERVER_KNOBS->ENABLE_ENCRYPTION && + configuration.tenantMode != TenantMode::OPTIONAL_TENANT, self->stopWhenDone, UsePartitionedLog::False, self->incremental)); diff --git a/tests/fast/EncryptedBackupCorrectness.toml b/tests/fast/EncryptedBackupCorrectness.toml index 597532737e..e674555491 100644 --- a/tests/fast/EncryptedBackupCorrectness.toml +++ b/tests/fast/EncryptedBackupCorrectness.toml @@ -1,5 +1,4 @@ [configuration] -allowDefaultTenant = true tenantModes = ['required'] [[knobs]] From 623e6ef761962034599a13b261a84b8b2366d588 Mon Sep 17 00:00:00 2001 From: Josh Slocum Date: Wed, 26 Oct 2022 12:22:41 -0500 Subject: [PATCH 48/95] adding delay in bw forced shutdown to prevent crash races (#8552) --- fdbserver/BlobWorker.actor.cpp | 11 ++++++++++- 1 file changed, 10 insertions(+), 1 deletion(-) diff --git a/fdbserver/BlobWorker.actor.cpp b/fdbserver/BlobWorker.actor.cpp index cf5b7b1340..fc51c8fae0 100644 --- a/fdbserver/BlobWorker.actor.cpp +++ b/fdbserver/BlobWorker.actor.cpp @@ -5182,13 +5182,22 @@ ACTOR Future blobWorker(BlobWorkerInterface bwInterf, ASSERT(false); throw internal_error(); } - when(wait(selfRemoved || self->simInjectFailure.getFuture())) { + when(wait(selfRemoved)) { if (BW_DEBUG) { printf("Blob worker detected removal. Exiting...\n"); } TraceEvent("BlobWorkerRemoved", self->id); break; } + when(wait(self->simInjectFailure.getFuture())) { + // wait to let triggering actor finish to prevent weird shutdown races + wait(delay(0)); + if (BW_DEBUG) { + printf("Blob worker simulation injected failure. Exiting...\n"); + } + TraceEvent("BlobWorkerSimRemoved", self->id); + break; + } when(wait(self->fatalError.getFuture())) { TraceEvent(SevError, "BlobWorkerActorCollectionFatalErrorNotError", self->id); ASSERT(false); From f620f391f5ef745e681cf6fe8867c77eacad160c Mon Sep 17 00:00:00 2001 From: Aaron Molitor Date: Wed, 26 Oct 2022 11:07:04 -0500 Subject: [PATCH 49/95] make same change to Dockerfile.eks (from #8583) --- packaging/docker/Dockerfile | 2 +- packaging/docker/Dockerfile.eks | 8 ++++---- 2 files changed, 5 insertions(+), 5 deletions(-) diff --git a/packaging/docker/Dockerfile b/packaging/docker/Dockerfile index 70d488b88c..1c19b1d9bd 100644 --- a/packaging/docker/Dockerfile +++ b/packaging/docker/Dockerfile @@ -183,7 +183,7 @@ RUN curl -Ls https://s3.us-west-2.amazonaws.com/amazon-eks/1.22.6/2022-03-09/bin sha256sum --quiet -c kubectl.txt && \ mv kubectl /usr/local/bin/kubectl && \ chmod 755 /usr/local/bin/kubectl && \ - curl -Ls https://awscli.amazonaws.com/awscli-exe-linux-$(uname -m)-2.7.34.zip -o "awscliv2.zip" && \ + curl -Ls https://awscli.amazonaws.com/awscli-exe-linux-x86_64-2.7.34.zip -o "awscliv2.zip" && \ echo "daf9253f0071b5cfee9532bc5220bedd7a5d29d4e0f92b42b9e3e4c496341e88 awscliv2.zip" > awscliv2.txt && \ sha256sum --quiet -c awscliv2.txt && \ unzip -qq awscliv2.zip && \ diff --git a/packaging/docker/Dockerfile.eks b/packaging/docker/Dockerfile.eks index fc037f3bf9..9a3eacb84b 100644 --- a/packaging/docker/Dockerfile.eks +++ b/packaging/docker/Dockerfile.eks @@ -53,13 +53,13 @@ RUN curl -Ls https://github.com/krallin/tini/releases/download/v0.19.0/tini-amd6 mv tini /usr/bin/ && \ rm -rf /tmp/* -RUN curl -Ls https://amazon-eks.s3.amazonaws.com/1.19.6/2021-01-05/bin/linux/amd64/kubectl -o kubectl && \ - echo "08ff68159bbcb844455167abb1d0de75bbfe5ae1b051f81ab060a1988027868a kubectl" > kubectl.txt && \ +RUN curl -Ls https://s3.us-west-2.amazonaws.com/amazon-eks/1.22.6/2022-03-09/bin/linux/amd64/kubectl -o kubectl && \ + echo "860c3d37a5979491895767e7332404d28dc0d7797c7673c33df30ca80e215a07 kubectl" > kubectl.txt && \ sha256sum --quiet -c kubectl.txt && \ mv kubectl /usr/local/bin/kubectl && \ chmod 755 /usr/local/bin/kubectl && \ - curl -Ls https://awscli.amazonaws.com/awscli-exe-linux-x86_64-2.2.43.zip -o "awscliv2.zip" && \ - echo "9a8b3c4e7f72bbcc55e341dce3af42479f2730c225d6d265ee6f9162cfdebdfd awscliv2.zip" > awscliv2.txt && \ + curl -Ls https://awscli.amazonaws.com/awscli-exe-linux-x86_64-2.7.34.zip -o "awscliv2.zip" && \ + echo "daf9253f0071b5cfee9532bc5220bedd7a5d29d4e0f92b42b9e3e4c496341e88 awscliv2.zip" > awscliv2.txt && \ sha256sum --quiet -c awscliv2.txt && \ unzip -qq awscliv2.zip && \ ./aws/install && \ From 989731f7f4d6aa67cd5c40a81ec49958598ea0de Mon Sep 17 00:00:00 2001 From: Markus Pilman Date: Wed, 26 Oct 2022 11:48:10 -0600 Subject: [PATCH 50/95] Fix AWS SDK build and removed check for old build system --- cmake/FlowCommands.cmake | 38 +++------------------ cmake/awssdk.cmake | 69 ++++++++++++++++++++------------------- fdbmonitor/CMakeLists.txt | 15 ++++----- 3 files changed, 47 insertions(+), 75 deletions(-) diff --git a/cmake/FlowCommands.cmake b/cmake/FlowCommands.cmake index 81ea36892d..c12b4dff03 100644 --- a/cmake/FlowCommands.cmake +++ b/cmake/FlowCommands.cmake @@ -76,38 +76,11 @@ function(generate_coverage_xml) add_dependencies(coverage_${target_name} coveragetool) endfunction() -# This function asserts that `versions.h` does not exist in the source -# directory. It does this in the prebuild phase of the target. -# This is an ugly hack that should make sure that cmake isn't used with -# a source directory in which FDB was previously built with `make`. -function(assert_no_version_h target) - - message(STATUS "Check versions.h on ${target}") - set(target_name "${target}_versions_h_check") - - if (DEFINED ENV{VERBOSE}) - add_custom_target("${target_name}" - COMMAND "${CMAKE_COMMAND}" -DFILE="${CMAKE_SOURCE_DIR}/versions.h" - -P "${CMAKE_SOURCE_DIR}/cmake/AssertFileDoesntExist.cmake" - COMMAND echo - "${CMAKE_COMMAND}" -P "${CMAKE_SOURCE_DIR}/cmake/AssertFileDoesntExist.cmake" - -DFILE="${CMAKE_SOURCE_DIR}/versions.h" - COMMENT "Check old build system wasn't used in source dir") - else() - add_custom_target("${target_name}" - COMMAND "${CMAKE_COMMAND}" -DFILE="${CMAKE_SOURCE_DIR}/versions.h" - -P "${CMAKE_SOURCE_DIR}/cmake/AssertFileDoesntExist.cmake" - COMMENT "Check old build system wasn't used in source dir") - endif() - - add_dependencies(${target} ${target_name}) -endfunction() - add_custom_target(strip_targets) add_dependencies(packages strip_targets) function(strip_debug_symbols target) - if (WIN32) + if(WIN32) return() endif() get_target_property(target_type ${target} TYPE) @@ -146,7 +119,7 @@ function(strip_debug_symbols target) COMMAND objcopy --verbose --only-keep-debug $ "${out_file}.debug" COMMAND objcopy --verbose --add-gnu-debuglink="${out_file}.debug" "${out_file}" COMMENT "Copy debug symbols to ${out_name}.debug") - add_custom_target(strip_${target} DEPENDS "${out_file}.debug") + add_custom_target(strip_${target} DEPENDS "${out_file}.debug") else() add_custom_target(strip_${target}) add_dependencies(strip_${target} strip_only_${target}) @@ -171,7 +144,7 @@ function(copy_headers) foreach(f IN LISTS CP_SRCS) is_prefix(bd "${CMAKE_CURRENT_BINARY_DIR}" "${f}") is_prefix(sd "${CMAKE_CURRENT_SOURCE_DIR}" "${f}") - if (bd OR sd) + if(bd OR sd) continue() endif() is_header(hdr "${f}") @@ -180,7 +153,7 @@ function(copy_headers) endif() get_filename_component(fname ${f} NAME) get_filename_component(dname ${f} DIRECTORY) - if (dname) + if(dname) make_directory(${incl_dir}/${dname}) endif() set(fpath "${incl_dir}/${dname}/${fname}") @@ -309,9 +282,6 @@ function(add_flow_target) add_custom_target(${AFT_NAME}_actors DEPENDS ${generated_files}) add_dependencies(${AFT_NAME} ${AFT_NAME}_actors) - if(NOT WIN32) - assert_no_version_h(${AFT_NAME}_actors) - endif() generate_coverage_xml(${AFT_NAME}) if(strip_target) strip_debug_symbols(${AFT_NAME}) diff --git a/cmake/awssdk.cmake b/cmake/awssdk.cmake index ab62f9b6d6..28c81166d3 100644 --- a/cmake/awssdk.cmake +++ b/cmake/awssdk.cmake @@ -8,40 +8,43 @@ endif() include(ExternalProject) ExternalProject_Add(awssdk_project - GIT_REPOSITORY https://github.com/aws/aws-sdk-cpp.git - GIT_TAG e4b4b310d8631bc7e9a797b6ac03a73c6f210bf6 # v1.9.331 - SOURCE_DIR "${CMAKE_CURRENT_BINARY_DIR}/awssdk-src" - BINARY_DIR "${CMAKE_CURRENT_BINARY_DIR}/awssdk-build" - GIT_CONFIG advice.detachedHead=false - CMAKE_ARGS -DBUILD_SHARED_LIBS=OFF # SDK builds shared libs by default, we want static libs - -DENABLE_TESTING=OFF - -DBUILD_ONLY=core # git repo contains SDK for every AWS product, we only want the core auth libraries - -DSIMPLE_INSTALL=ON - -DCMAKE_INSTALL_PREFIX=install # need to specify an install prefix so it doesn't install in /usr/lib - FIXME: use absolute path - -DBYO_CRYPTO=ON # we have our own crypto libraries that conflict if we let aws sdk build and link its own - -DBUILD_CURL=ON - -DBUILD_ZLIB=ON - - -DCMAKE_CXX_COMPILER=${CMAKE_CXX_COMPILER} - -DCMAKE_CXX_FLAGS=${AWSSDK_COMPILER_FLAGS} - TEST_COMMAND "" + GIT_REPOSITORY https://github.com/aws/aws-sdk-cpp.git + GIT_TAG e4b4b310d8631bc7e9a797b6ac03a73c6f210bf6 # v1.9.331 + SOURCE_DIR "${CMAKE_CURRENT_BINARY_DIR}/awssdk-src" + BINARY_DIR "${CMAKE_CURRENT_BINARY_DIR}/awssdk-build" + GIT_CONFIG advice.detachedHead=false + # it seems advice.detachedHead breaks something which causes aws sdk to always be rebuilt. + # This option forces to cmake to build the aws sdk only once and never attempt to update it + UPDATE_DISCONNECTED ON + CMAKE_ARGS -DBUILD_SHARED_LIBS=OFF # SDK builds shared libs by default, we want static libs + -DENABLE_TESTING=OFF + -DBUILD_ONLY=core # git repo contains SDK for every AWS product, we only want the core auth libraries + -DSIMPLE_INSTALL=ON + -DCMAKE_INSTALL_PREFIX=install # need to specify an install prefix so it doesn't install in /usr/lib - FIXME: use absolute path + -DBYO_CRYPTO=ON # we have our own crypto libraries that conflict if we let aws sdk build and link its own + -DBUILD_CURL=ON + -DBUILD_ZLIB=ON + + -DCMAKE_CXX_COMPILER=${CMAKE_CXX_COMPILER} + -DCMAKE_CXX_FLAGS=${AWSSDK_COMPILER_FLAGS} + TEST_COMMAND "" # the sdk build produces a ton of artifacts, with their own dependency tree, so there is a very specific dependency order they must be linked in - BUILD_BYPRODUCTS "${CMAKE_CURRENT_BINARY_DIR}/awssdk-build/install/lib64/libaws-cpp-sdk-core.a" - "${CMAKE_CURRENT_BINARY_DIR}/awssdk-build/install/lib64/libaws-crt-cpp.a" - "${CMAKE_CURRENT_BINARY_DIR}/awssdk-build/install/lib64/libaws-c-s3.a" - "${CMAKE_CURRENT_BINARY_DIR}/awssdk-build/install/lib64/libaws-c-auth.a" - "${CMAKE_CURRENT_BINARY_DIR}/awssdk-build/install/lib64/libaws-c-event-stream.a" - "${CMAKE_CURRENT_BINARY_DIR}/awssdk-build/install/lib64/libaws-c-http.a" - "${CMAKE_CURRENT_BINARY_DIR}/awssdk-build/install/lib64/libaws-c-mqtt.a" - "${CMAKE_CURRENT_BINARY_DIR}/awssdk-build/install/lib64/libaws-c-sdkutils.a" - "${CMAKE_CURRENT_BINARY_DIR}/awssdk-build/install/lib64/libaws-c-io.a" - "${CMAKE_CURRENT_BINARY_DIR}/awssdk-build/install/lib64/libaws-checksums.a" - "${CMAKE_CURRENT_BINARY_DIR}/awssdk-build/install/lib64/libaws-c-compression.a" - "${CMAKE_CURRENT_BINARY_DIR}/awssdk-build/install/lib64/libaws-c-cal.a" - "${CMAKE_CURRENT_BINARY_DIR}/awssdk-build/install/lib64/libaws-c-common.a" - "${CMAKE_CURRENT_BINARY_DIR}/awssdk-build/install/external-install/curl/lib/libcurl.a" - "${CMAKE_CURRENT_BINARY_DIR}/awssdk-build/install/external-install/zlib/lib/libz.a" -) + BUILD_BYPRODUCTS "${CMAKE_CURRENT_BINARY_DIR}/awssdk-build/install/lib64/libaws-cpp-sdk-core.a" + "${CMAKE_CURRENT_BINARY_DIR}/awssdk-build/install/lib64/libaws-crt-cpp.a" + "${CMAKE_CURRENT_BINARY_DIR}/awssdk-build/install/lib64/libaws-c-s3.a" + "${CMAKE_CURRENT_BINARY_DIR}/awssdk-build/install/lib64/libaws-c-auth.a" + "${CMAKE_CURRENT_BINARY_DIR}/awssdk-build/install/lib64/libaws-c-event-stream.a" + "${CMAKE_CURRENT_BINARY_DIR}/awssdk-build/install/lib64/libaws-c-http.a" + "${CMAKE_CURRENT_BINARY_DIR}/awssdk-build/install/lib64/libaws-c-mqtt.a" + "${CMAKE_CURRENT_BINARY_DIR}/awssdk-build/install/lib64/libaws-c-sdkutils.a" + "${CMAKE_CURRENT_BINARY_DIR}/awssdk-build/install/lib64/libaws-c-io.a" + "${CMAKE_CURRENT_BINARY_DIR}/awssdk-build/install/lib64/libaws-checksums.a" + "${CMAKE_CURRENT_BINARY_DIR}/awssdk-build/install/lib64/libaws-c-compression.a" + "${CMAKE_CURRENT_BINARY_DIR}/awssdk-build/install/lib64/libaws-c-cal.a" + "${CMAKE_CURRENT_BINARY_DIR}/awssdk-build/install/lib64/libaws-c-common.a" + "${CMAKE_CURRENT_BINARY_DIR}/awssdk-build/install/external-install/curl/lib/libcurl.a" + "${CMAKE_CURRENT_BINARY_DIR}/awssdk-build/install/external-install/zlib/lib/libz.a" + ) add_library(awssdk_core STATIC IMPORTED) add_dependencies(awssdk_core awssdk_project) diff --git a/fdbmonitor/CMakeLists.txt b/fdbmonitor/CMakeLists.txt index 177dde5170..1eb15ac269 100644 --- a/fdbmonitor/CMakeLists.txt +++ b/fdbmonitor/CMakeLists.txt @@ -5,9 +5,8 @@ get_target_property(fdbclient_target_includes fdbclient INCLUDE_DIRECTORIES) target_link_libraries(fdbmonitor PUBLIC SimpleOpt) target_include_directories(fdbmonitor PUBLIC "${fdbclient_target_includes}") strip_debug_symbols(fdbmonitor) -assert_no_version_h(fdbmonitor) if(UNIX AND NOT APPLE) - target_link_libraries(fdbmonitor PRIVATE rt) + target_link_libraries(fdbmonitor PRIVATE rt) endif() # FIXME: This include directory is an ugly hack. We probably want to fix this. # as soon as we get rid of the old build system @@ -17,17 +16,17 @@ target_link_libraries(fdbmonitor PUBLIC Threads::Threads) # appears to change its behavior (it no longer seems to restart killed # processes). fdbmonitor is single-threaded anyway. get_target_property(fdbmonitor_options fdbmonitor COMPILE_OPTIONS) -if (NOT "${fdbmonitor_options}" STREQUAL "fdbmonitor_options-NOTFOUND") +if(NOT "${fdbmonitor_options}" STREQUAL "fdbmonitor_options-NOTFOUND") list(REMOVE_ITEM fdbmonitor_options "-fsanitize=thread") set_property(TARGET fdbmonitor PROPERTY COMPILE_OPTIONS ${fdbmonitor_options}) -endif () +endif() get_target_property(fdbmonitor_options fdbmonitor LINK_OPTIONS) -if (NOT "${fdbmonitor_options}" STREQUAL "fdbmonitor_options-NOTFOUND") +if(NOT "${fdbmonitor_options}" STREQUAL "fdbmonitor_options-NOTFOUND") list(REMOVE_ITEM fdbmonitor_options "-fsanitize=thread") set_property(TARGET fdbmonitor PROPERTY LINK_OPTIONS ${fdbmonitor_options}) -endif () +endif() if(GENERATE_DEBUG_PACKAGES) fdb_install(TARGETS fdbmonitor DESTINATION fdbmonitor COMPONENT server) @@ -51,7 +50,7 @@ add_custom_target(clean_sandbox add_custom_target(start_sandbox COMMAND ${CMAKE_BINARY_DIR}/bin/fdbmonitor --conffile ${CMAKE_BINARY_DIR}/sandbox/foundationdb.conf - --lockfile ${CMAKE_BINARY_DIR}/sandbox/fdbmonitor.lock) + --lockfile ${CMAKE_BINARY_DIR}/sandbox/fdbmonitor.lock) add_dependencies(start_sandbox fdbmonitor fdbserver) @@ -61,6 +60,6 @@ if(NOT EXISTS ${CMAKE_BINARY_DIR}/contrib/generate_profile.sh) endif() add_custom_target(generate_profile - COMMAND ${CMAKE_BINARY_DIR}/contrib/generate_profile.sh ${CMAKE_BINARY_DIR}) + COMMAND ${CMAKE_BINARY_DIR}/contrib/generate_profile.sh ${CMAKE_BINARY_DIR}) add_dependencies(generate_profile fdbmonitor fdbserver mako fdbcli) From 3cc106b506241463e917be6d2fc7b4adfebe6e82 Mon Sep 17 00:00:00 2001 From: Steve Atherton Date: Wed, 26 Oct 2022 15:10:34 -0700 Subject: [PATCH 51/95] Bug fix, streaming range read lock was taken too late so data version could change before range read begins. --- fdbserver/storageserver.actor.cpp | 5 +---- 1 file changed, 1 insertion(+), 4 deletions(-) diff --git a/fdbserver/storageserver.actor.cpp b/fdbserver/storageserver.actor.cpp index 292b9caa42..7bc967cdae 100644 --- a/fdbserver/storageserver.actor.cpp +++ b/fdbserver/storageserver.actor.cpp @@ -3768,7 +3768,6 @@ ACTOR Future getKeyValuesQ(StorageServer* data, GetKeyValuesRequest req) // Active load balancing runs at a very high priority (to obtain accurate queue lengths) // so we need to downgrade here wait(data->getQueryDelay()); - state PriorityMultiLock::Lock lock = wait(data->getReadLock(req.options)); // Track time from requestTime through now as read queueing wait time @@ -5136,6 +5135,7 @@ ACTOR Future getKeyValuesStreamQ(StorageServer* data, GetKeyValuesStreamRe } else { loop { wait(req.reply.onReady()); + state PriorityMultiLock::Lock lock = wait(data->getReadLock(req.options)); if (version < data->oldestVersion.get()) { throw transaction_too_old(); @@ -5154,8 +5154,6 @@ ACTOR Future getKeyValuesStreamQ(StorageServer* data, GetKeyValuesStreamRe .detail("Begin", begin.printable()) .detail("End", end.printable()); - state PriorityMultiLock::Lock lock = wait(data->getReadLock(req.options)); - GetKeyValuesReply _r = wait(readRange(data, version, KeyRangeRef(begin, end), @@ -5259,7 +5257,6 @@ ACTOR Future getKeyQ(StorageServer* data, GetKeyRequest req) { // Active load balancing runs at a very high priority (to obtain accurate queue lengths) // so we need to downgrade here wait(data->getQueryDelay()); - state PriorityMultiLock::Lock lock = wait(data->getReadLock(req.options)); // Track time from requestTime through now as read queueing wait time From a5d07f065c924fe0073084146ee46d5170e0958e Mon Sep 17 00:00:00 2001 From: Steve Atherton Date: Wed, 26 Oct 2022 16:16:46 -0700 Subject: [PATCH 52/95] Take read lock for getMappedValues earlier to avoid a data version change, unlock before subqueries. Rename variables for clarity. --- fdbserver/storageserver.actor.cpp | 22 +++++++++++----------- 1 file changed, 11 insertions(+), 11 deletions(-) diff --git a/fdbserver/storageserver.actor.cpp b/fdbserver/storageserver.actor.cpp index 7bc967cdae..ba4deafab5 100644 --- a/fdbserver/storageserver.actor.cpp +++ b/fdbserver/storageserver.actor.cpp @@ -1894,7 +1894,7 @@ ACTOR Future getValueQ(StorageServer* data, GetValueRequest req) { // Active load balancing runs at a very high priority (to obtain accurate queue lengths) // so we need to downgrade here wait(data->getQueryDelay()); - state PriorityMultiLock::Lock lock = wait(data->getReadLock(req.options)); + state PriorityMultiLock::Lock readLock = wait(data->getReadLock(req.options)); // Track time from requestTime through now as read queueing wait time state double queueWaitEnd = g_network->timer(); @@ -3768,7 +3768,7 @@ ACTOR Future getKeyValuesQ(StorageServer* data, GetKeyValuesRequest req) // Active load balancing runs at a very high priority (to obtain accurate queue lengths) // so we need to downgrade here wait(data->getQueryDelay()); - state PriorityMultiLock::Lock lock = wait(data->getReadLock(req.options)); + state PriorityMultiLock::Lock readLock = wait(data->getReadLock(req.options)); // Track time from requestTime through now as read queueing wait time state double queueWaitEnd = g_network->timer(); @@ -4829,6 +4829,7 @@ ACTOR Future getMappedKeyValuesQ(StorageServer* data, GetMappedKeyValuesRe // Active load balancing runs at a very high priority (to obtain accurate queue lengths) // so we need to downgrade here wait(data->getQueryDelay()); + state PriorityMultiLock::Lock readLock = wait(data->getReadLock(req.options)); // Track time from requestTime through now as read queueing wait time state double queueWaitEnd = g_network->timer(); @@ -4937,11 +4938,6 @@ ACTOR Future getMappedKeyValuesQ(StorageServer* data, GetMappedKeyValuesRe } else { state int remainingLimitBytes = req.limitBytes; - // Only take the ssLock for the readRange operation and unlock before the subqueries because each - // subquery will route back to getValueQ or getKeyValuesQ with a new request having the same - // read options which will each acquire the ssLock. - state PriorityMultiLock::Lock lock = wait(data->getReadLock(req.options)); - GetKeyValuesReply getKeyValuesReply = wait(readRange(data, version, KeyRangeRef(begin, end), @@ -4950,7 +4946,11 @@ ACTOR Future getMappedKeyValuesQ(StorageServer* data, GetMappedKeyValuesRe span.context, req.options, tenantPrefix)); - lock.release(); + + // Unlock read lock before the subqueries because each + // subquery will route back to getValueQ or getKeyValuesQ with a new request having the same + // read options which will each acquire the ssLock. + readLock.release(); state GetMappedKeyValuesReply r; try { @@ -5135,7 +5135,7 @@ ACTOR Future getKeyValuesStreamQ(StorageServer* data, GetKeyValuesStreamRe } else { loop { wait(req.reply.onReady()); - state PriorityMultiLock::Lock lock = wait(data->getReadLock(req.options)); + state PriorityMultiLock::Lock readLock = wait(data->getReadLock(req.options)); if (version < data->oldestVersion.get()) { throw transaction_too_old(); @@ -5162,7 +5162,7 @@ ACTOR Future getKeyValuesStreamQ(StorageServer* data, GetKeyValuesStreamRe span.context, req.options, tenantPrefix)); - lock.release(); + readLock.release(); GetKeyValuesStreamReply r(_r); if (req.options.present() && req.options.get().debugID.present()) @@ -5257,7 +5257,7 @@ ACTOR Future getKeyQ(StorageServer* data, GetKeyRequest req) { // Active load balancing runs at a very high priority (to obtain accurate queue lengths) // so we need to downgrade here wait(data->getQueryDelay()); - state PriorityMultiLock::Lock lock = wait(data->getReadLock(req.options)); + state PriorityMultiLock::Lock readLock = wait(data->getReadLock(req.options)); // Track time from requestTime through now as read queueing wait time state double queueWaitEnd = g_network->timer(); From fb44945a899be618a31240ac0e152e557291fb03 Mon Sep 17 00:00:00 2001 From: Steve Atherton Date: Wed, 26 Oct 2022 17:35:39 -0700 Subject: [PATCH 53/95] Use id variable to simplify logic a bit. --- fdbclient/NativeAPI.actor.cpp | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/fdbclient/NativeAPI.actor.cpp b/fdbclient/NativeAPI.actor.cpp index 9c88419501..c563c356a5 100644 --- a/fdbclient/NativeAPI.actor.cpp +++ b/fdbclient/NativeAPI.actor.cpp @@ -9884,10 +9884,10 @@ ACTOR Future mergeChangeFeedStream(Reference db, if (replyBufferSize != -1 && req.replyBufferSize < CLIENT_KNOBS->CHANGE_FEED_STREAM_MIN_BYTES) { req.replyBufferSize = CLIENT_KNOBS->CHANGE_FEED_STREAM_MIN_BYTES; } - req.options = ReadOptions(deterministicRandom()->randomUniqueID()); - debugUIDs.push_back(req.streamUID()); - mergeCursorUID = - UID(mergeCursorUID.first() ^ req.streamUID().first(), mergeCursorUID.second() ^ req.streamUID().second()); + UID id = deterministicRandom()->randomUniqueID(); + req.options = ReadOptions(id); + debugUIDs.push_back(id); + mergeCursorUID = UID(mergeCursorUID.first() ^ id.first(), mergeCursorUID.second() ^ id.second()); results->streams.push_back(interfs[i].first.changeFeedStream.getReplyStream(req)); maybeDuplicateTSSChangeFeedStream(req, From 56abec32f1bcdd9f1b161383563d5fe40e5b8b53 Mon Sep 17 00:00:00 2001 From: Steve Atherton Date: Wed, 26 Oct 2022 20:45:39 -0700 Subject: [PATCH 54/95] Bug fix: The change feed request UID is actually not just for debugging and can't be shared across requests, so the debugID in ReadOptions should not be used. Restored the original ChangeFeedRequest member but renamed it from debugUID to just id. --- fdbclient/NativeAPI.actor.cpp | 28 +++++------ fdbclient/include/fdbclient/DatabaseContext.h | 2 +- fdbclient/include/fdbclient/FDBTypes.h | 8 ++-- .../fdbclient/StorageServerInterface.h | 11 ++--- fdbserver/StorageCache.actor.cpp | 2 +- fdbserver/storageserver.actor.cpp | 46 +++++++++---------- 6 files changed, 46 insertions(+), 51 deletions(-) diff --git a/fdbclient/NativeAPI.actor.cpp b/fdbclient/NativeAPI.actor.cpp index c563c356a5..16572766bb 100644 --- a/fdbclient/NativeAPI.actor.cpp +++ b/fdbclient/NativeAPI.actor.cpp @@ -9305,7 +9305,7 @@ void handleTSSChangeFeedMismatch(const ChangeFeedStreamRequest& request, mismatchEvent.detail("EndKey", request.range.end); mismatchEvent.detail("CanReadPopped", request.canReadPopped); mismatchEvent.detail("PopVersion", popVersion); - mismatchEvent.detail("DebugUID", request.streamUID()); + mismatchEvent.detail("DebugUID", request.id); // mismatch info mismatchEvent.detail("MatchesFound", matchesFound); @@ -9331,7 +9331,7 @@ void handleTSSChangeFeedMismatch(const ChangeFeedStreamRequest& request, "TSSMismatchChangeFeedStream"); summaryEvent.detail("TSSID", tssData.tssId) .detail("MismatchId", mismatchUID) - .detail("FeedDebugUID", request.streamUID()); + .detail("FeedDebugUID", request.id); } } } @@ -9856,7 +9856,8 @@ ACTOR Future mergeChangeFeedStream(Reference db, Version* begin, Version end, int replyBufferSize, - bool canReadPopped) { + bool canReadPopped, + ReadOptions readOptions) { state std::vector> fetchers(interfs.size()); state std::vector> onErrors(interfs.size()); state std::vector streams(interfs.size()); @@ -9884,10 +9885,11 @@ ACTOR Future mergeChangeFeedStream(Reference db, if (replyBufferSize != -1 && req.replyBufferSize < CLIENT_KNOBS->CHANGE_FEED_STREAM_MIN_BYTES) { req.replyBufferSize = CLIENT_KNOBS->CHANGE_FEED_STREAM_MIN_BYTES; } - UID id = deterministicRandom()->randomUniqueID(); - req.options = ReadOptions(id); - debugUIDs.push_back(id); - mergeCursorUID = UID(mergeCursorUID.first() ^ id.first(), mergeCursorUID.second() ^ id.second()); + req.options = readOptions; + req.id = deterministicRandom()->randomUniqueID(); + + debugUIDs.push_back(req.id); + mergeCursorUID = UID(mergeCursorUID.first() ^ req.id.first(), mergeCursorUID.second() ^ req.id.second()); results->streams.push_back(interfs[i].first.changeFeedStream.getReplyStream(req)); maybeDuplicateTSSChangeFeedStream(req, @@ -10102,12 +10104,10 @@ ACTOR Future singleChangeFeedStream(Reference db, req.canReadPopped = canReadPopped; req.replyBufferSize = replyBufferSize; req.options = readOptions; - if (!req.streamUID().isValid()) { - req.options.get().debugID = readOptions.debugID = deterministicRandom()->randomUniqueID(); - } + req.id = deterministicRandom()->randomUniqueID(); if (DEBUG_CF_CLIENT_TRACE) { - TraceEvent(SevDebug, "TraceChangeFeedClientSingleCursor", req.streamUID()) + TraceEvent(SevDebug, "TraceChangeFeedClientSingleCursor", req.id) .detail("FeedID", rangeID) .detail("Range", range) .detail("Begin", *begin) @@ -10238,9 +10238,9 @@ ACTOR Future getChangeFeedStreamActor(Reference db, } CODE_PROBE(true, "Change feed merge cursor"); // TODO (jslocum): validate connectionFileChanged behavior - wait( - mergeChangeFeedStream(db, interfs, results, rangeID, &begin, end, replyBufferSize, canReadPopped) || - cx->connectionFileChanged()); + wait(mergeChangeFeedStream( + db, interfs, results, rangeID, &begin, end, replyBufferSize, canReadPopped, readOptions) || + cx->connectionFileChanged()); } else { CODE_PROBE(true, "Change feed single cursor"); StorageServerInterface interf = locations[0].locations->getInterface(chosenLocations[0]); diff --git a/fdbclient/include/fdbclient/DatabaseContext.h b/fdbclient/include/fdbclient/DatabaseContext.h index 6ddf90f69d..f28da0399a 100644 --- a/fdbclient/include/fdbclient/DatabaseContext.h +++ b/fdbclient/include/fdbclient/DatabaseContext.h @@ -383,7 +383,7 @@ public: KeyRange range = allKeys, int replyBufferSize = -1, bool canReadPopped = true, - ReadOptions readOptions = ReadOptions()); + ReadOptions readOptions = { ReadType::NORMAL, CacheResult::False }); Future getOverlappingChangeFeeds(KeyRangeRef ranges, Version minVersion); Future popChangeFeedMutations(Key rangeID, Version version); diff --git a/fdbclient/include/fdbclient/FDBTypes.h b/fdbclient/include/fdbclient/FDBTypes.h index 94fb9f8942..84dfa41602 100644 --- a/fdbclient/include/fdbclient/FDBTypes.h +++ b/fdbclient/include/fdbclient/FDBTypes.h @@ -1649,14 +1649,14 @@ struct ReadOptions { Optional debugID; Optional consistencyCheckStartVersion; - ReadOptions() : type(ReadType::NORMAL), cacheResult(CacheResult::True){}; - - ReadOptions(Optional debugID, + ReadOptions(Optional debugID = Optional(), ReadType type = ReadType::NORMAL, - CacheResult cache = CacheResult::False, + CacheResult cache = CacheResult::True, Optional version = Optional()) : type(type), cacheResult(cache), debugID(debugID), consistencyCheckStartVersion(version){}; + ReadOptions(ReadType type, CacheResult cache = CacheResult::True) : ReadOptions({}, type, cache) {} + template void serialize(Ar& ar) { serializer(ar, type, cacheResult, debugID, consistencyCheckStartVersion); diff --git a/fdbclient/include/fdbclient/StorageServerInterface.h b/fdbclient/include/fdbclient/StorageServerInterface.h index 40a4fa24b4..6a920d07c3 100644 --- a/fdbclient/include/fdbclient/StorageServerInterface.h +++ b/fdbclient/include/fdbclient/StorageServerInterface.h @@ -890,21 +890,16 @@ struct ChangeFeedStreamRequest { KeyRange range; int replyBufferSize = -1; bool canReadPopped = true; + UID id; Optional options; - UID streamUID() const { - if (options.present()) { - return options.get().debugID.orDefault(UID()); - } - return UID(); - } - ReplyPromiseStream reply; ChangeFeedStreamRequest() {} template void serialize(Ar& ar) { - serializer(ar, rangeID, begin, end, range, reply, spanContext, replyBufferSize, canReadPopped, options, arena); + serializer( + ar, rangeID, begin, end, range, reply, spanContext, replyBufferSize, canReadPopped, id, options, arena); } }; diff --git a/fdbserver/StorageCache.actor.cpp b/fdbserver/StorageCache.actor.cpp index 6cf1c9b934..15927a217a 100644 --- a/fdbserver/StorageCache.actor.cpp +++ b/fdbserver/StorageCache.actor.cpp @@ -1188,7 +1188,7 @@ ACTOR Future tryFetchRange(Database cx, state RangeResult output; state KeySelectorRef begin = firstGreaterOrEqual(keys.begin); state KeySelectorRef end = firstGreaterOrEqual(keys.end); - state ReadOptions options = ReadOptions(Optional(), ReadType::FETCH); + state ReadOptions options = ReadOptions(ReadType::FETCH, CacheResult::False); if (*isTooOld) throw transaction_too_old(); diff --git a/fdbserver/storageserver.actor.cpp b/fdbserver/storageserver.actor.cpp index ba4deafab5..b9311e3240 100644 --- a/fdbserver/storageserver.actor.cpp +++ b/fdbserver/storageserver.actor.cpp @@ -2600,7 +2600,7 @@ ACTOR Future> getChangeFeedMutations(Stor if (DEBUG_CF_TRACE) { TraceEvent(SevDebug, "TraceChangeFeedMutationsBegin", data->thisServerID) .detail("FeedID", req.rangeID) - .detail("StreamUID", req.streamUID()) + .detail("StreamUID", req.id) .detail("Range", req.range) .detail("Begin", req.begin) .detail("End", req.end) @@ -2638,7 +2638,7 @@ ACTOR Future> getChangeFeedMutations(Stor if (DEBUG_CF_TRACE) { TraceEvent(SevDebug, "TraceChangeFeedMutationsDetails", data->thisServerID) .detail("FeedID", req.rangeID) - .detail("StreamUID", req.streamUID()) + .detail("StreamUID", req.id) .detail("Range", req.range) .detail("Begin", req.begin) .detail("End", req.end) @@ -2741,7 +2741,7 @@ ACTOR Future> getChangeFeedMutations(Stor "is {4}) (emptyVersion={5}, emptyBefore={6})!\n", data->thisServerID.toString().substr(0, 4), req.rangeID.printable().substr(0, 6), - req.streamUID().toString().substr(0, 8), + req.id.toString().substr(0, 8), memoryReply.mutations[memoryVerifyIdx].version, version, feedInfo->emptyVersion, @@ -2783,7 +2783,7 @@ ACTOR Future> getChangeFeedMutations(Stor "disk! (durable validation = {4})\n", data->thisServerID.toString().substr(0, 4), req.rangeID.printable().substr(0, 6), - req.streamUID().toString().substr(0, 8), + req.id.toString().substr(0, 8), version, durableValidationVersion); @@ -2864,7 +2864,7 @@ ACTOR Future> getChangeFeedMutations(Stor if (!ok) { TraceEvent("ChangeFeedMutationsPopped", data->thisServerID) .detail("FeedID", req.rangeID) - .detail("StreamUID", req.streamUID()) + .detail("StreamUID", req.id) .detail("Range", req.range) .detail("Begin", req.begin) .detail("End", req.end) @@ -2881,7 +2881,7 @@ ACTOR Future> getChangeFeedMutations(Stor for (auto& m : mutations.mutations) { DEBUG_MUTATION("ChangeFeedSSRead", mutations.version, m, data->thisServerID) .detail("ChangeFeedID", req.rangeID) - .detail("StreamUID", req.streamUID()) + .detail("StreamUID", req.id) .detail("ReqBegin", req.begin) .detail("ReqEnd", req.end) .detail("ReqRange", req.range); @@ -2908,7 +2908,7 @@ ACTOR Future> getChangeFeedMutations(Stor fmt::print("ERROR: SS {0} CF {1} SQ {2} missing {3} @ {4} from request for [{5} - {6}) {7} - {8}\n", data->thisServerID.toString().substr(0, 4), req.rangeID.printable().substr(0, 6), - req.streamUID().toString().substr(0, 8), + req.id.toString().substr(0, 8), foundVersion ? "key" : "version", DEBUG_CF_MISSING_VERSION, req.range.begin.printable(), @@ -2929,7 +2929,7 @@ ACTOR Future> getChangeFeedMutations(Stor fmt::print("DBG: SS {0} CF {1} SQ {2} correct @ {3} from request for [{4} - {5}) {6} - {7}\n", data->thisServerID.toString().substr(0, 4), req.rangeID.printable().substr(0, 6), - req.streamUID().toString().substr(0, 8), + req.id.toString().substr(0, 8), DEBUG_CF_MISSING_VERSION, req.range.begin.printable(), req.range.end.printable(), @@ -2943,7 +2943,7 @@ ACTOR Future> getChangeFeedMutations(Stor if (DEBUG_CF_TRACE) { TraceEvent(SevDebug, "ChangeFeedMutationsDone", data->thisServerID) .detail("FeedID", req.rangeID) - .detail("StreamUID", req.streamUID()) + .detail("StreamUID", req.id) .detail("Range", req.range) .detail("Begin", req.begin) .detail("End", req.end) @@ -2968,7 +2968,7 @@ ACTOR Future stopChangeFeedOnMove(StorageServer* data, ChangeFeedStreamReq return Void(); } state Promise moved; - feed->second->triggerOnMove(req.range, req.streamUID(), moved); + feed->second->triggerOnMove(req.range, req.id, moved); try { wait(moved.getFuture()); } catch (Error& e) { @@ -2977,7 +2977,7 @@ ACTOR Future stopChangeFeedOnMove(StorageServer* data, ChangeFeedStreamReq auto feed = data->uidChangeFeed.find(req.rangeID); if (feed != data->uidChangeFeed.end()) { - feed->second->removeOnMoveTrigger(req.range, req.streamUID()); + feed->second->removeOnMoveTrigger(req.range, req.id); } return Void(); } @@ -3020,7 +3020,7 @@ ACTOR Future changeFeedStreamQ(StorageServer* data, ChangeFeedStreamReques if (DEBUG_CF_TRACE) { TraceEvent(SevDebug, "TraceChangeFeedStreamStart", data->thisServerID) .detail("FeedID", req.rangeID) - .detail("StreamUID", req.streamUID()) + .detail("StreamUID", req.id) .detail("Range", req.range) .detail("Begin", req.begin) .detail("End", req.end) @@ -3042,7 +3042,7 @@ ACTOR Future changeFeedStreamQ(StorageServer* data, ChangeFeedStreamReques if (DEBUG_CF_TRACE) { TraceEvent(SevDebug, "TraceChangeFeedStreamSentInitialEmpty", data->thisServerID) .detail("FeedID", req.rangeID) - .detail("StreamUID", req.streamUID()) + .detail("StreamUID", req.id) .detail("Range", req.range) .detail("Begin", req.begin) .detail("End", req.end) @@ -3054,12 +3054,12 @@ ACTOR Future changeFeedStreamQ(StorageServer* data, ChangeFeedStreamReques loop { Future onReady = req.reply.onReady(); if (atLatest && !onReady.isReady() && !removeUID) { - data->changeFeedClientVersions[req.reply.getEndpoint().getPrimaryAddress()][req.streamUID()] = + data->changeFeedClientVersions[req.reply.getEndpoint().getPrimaryAddress()][req.id] = blockedVersion.present() ? blockedVersion.get() : data->prevVersion; if (DEBUG_CF_TRACE) { TraceEvent(SevDebug, "TraceChangeFeedStreamBlockedOnReady", data->thisServerID) .detail("FeedID", req.rangeID) - .detail("StreamUID", req.streamUID()) + .detail("StreamUID", req.id) .detail("Range", req.range) .detail("Begin", req.begin) .detail("End", req.end) @@ -3075,13 +3075,13 @@ ACTOR Future changeFeedStreamQ(StorageServer* data, ChangeFeedStreamReques Future> feedReplyFuture = getChangeFeedMutations(data, req, false, atLatest); if (atLatest && !removeUID && !feedReplyFuture.isReady()) { - data->changeFeedClientVersions[req.reply.getEndpoint().getPrimaryAddress()][req.streamUID()] = + data->changeFeedClientVersions[req.reply.getEndpoint().getPrimaryAddress()][req.id] = blockedVersion.present() ? blockedVersion.get() : data->prevVersion; removeUID = true; if (DEBUG_CF_TRACE) { TraceEvent(SevDebug, "TraceChangeFeedStreamBlockedMutations", data->thisServerID) .detail("FeedID", req.rangeID) - .detail("StreamUID", req.streamUID()) + .detail("StreamUID", req.id) .detail("Range", req.range) .detail("Begin", req.begin) .detail("End", req.end) @@ -3104,10 +3104,10 @@ ACTOR Future changeFeedStreamQ(StorageServer* data, ChangeFeedStreamReques Version minVersion = removeUID ? data->version.get() : data->prevVersion; if (removeUID) { if (gotAll || req.begin == req.end) { - data->changeFeedClientVersions[req.reply.getEndpoint().getPrimaryAddress()].erase(req.streamUID()); + data->changeFeedClientVersions[req.reply.getEndpoint().getPrimaryAddress()].erase(req.id); removeUID = false; } else { - data->changeFeedClientVersions[req.reply.getEndpoint().getPrimaryAddress()][req.streamUID()] = + data->changeFeedClientVersions[req.reply.getEndpoint().getPrimaryAddress()][req.id] = feedReply.mutations.back().version; } } @@ -3155,7 +3155,7 @@ ACTOR Future changeFeedStreamQ(StorageServer* data, ChangeFeedStreamReques auto it = data->changeFeedClientVersions.find(req.reply.getEndpoint().getPrimaryAddress()); if (it != data->changeFeedClientVersions.end()) { if (removeUID) { - it->second.erase(req.streamUID()); + it->second.erase(req.id); } if (it->second.empty()) { data->changeFeedClientVersions.erase(it); @@ -6005,7 +6005,6 @@ ACTOR Future tryGetRangeFromBlob(PromiseStream results, .detail("Error", e.what()); tr->reset(); tr->setVersion(fetchVersion); - tr->trState->taskID = TaskPriority::FetchKeys; throw; } return Void(); @@ -6836,6 +6835,7 @@ ACTOR Future fetchKeys(StorageServer* data, AddingShard* shard) { tr.setOption(FDBTransactionOptions::PRIORITY_SYSTEM_IMMEDIATE); tr.setOption(FDBTransactionOptions::LOCK_AWARE); tr.trState->readOptions = readOptions; + tr.trState->taskID = TaskPriority::FetchKeys; // fetchVersion = data->version.get(); // A quick fix: @@ -6884,7 +6884,7 @@ ACTOR Future fetchKeys(StorageServer* data, AddingShard* shard) { while (!shard->updates.empty() && shard->updates[0].version <= fetchVersion) shard->updates.pop_front(); tr.setVersion(fetchVersion); - tr.trState->taskID = TaskPriority::FetchKeys; + state PromiseStream results; state Future hold; if (SERVER_KNOBS->FETCH_USING_BLOB) { @@ -9574,7 +9574,7 @@ ACTOR Future applyByteSampleResult(StorageServer* data, state int totalFetches = 0; state int totalKeys = 0; state int totalBytes = 0; - state ReadOptions readOptions(UID(), ReadType::NORMAL, CacheResult::False); + state ReadOptions readOptions(ReadType::NORMAL, CacheResult::False); loop { RangeResult bs = wait(storage->readRange(KeyRangeRef(begin, end), From c6adb3a98c649317c0c5b4152804b3ac3af8ed61 Mon Sep 17 00:00:00 2001 From: Vaidas Gasiunas Date: Thu, 27 Oct 2022 12:37:20 +0200 Subject: [PATCH 55/95] Building fdb_c_shim to a shared library (#8586) --- bindings/c/CMakeLists.txt | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/bindings/c/CMakeLists.txt b/bindings/c/CMakeLists.txt index d095353daf..a6641c4113 100644 --- a/bindings/c/CMakeLists.txt +++ b/bindings/c/CMakeLists.txt @@ -442,7 +442,7 @@ elseif(NOT WIN32 AND NOT APPLE AND NOT USE_SANITIZER) # Linux Only, non-santizer DEPENDS ${IMPLIBSO_SRC} fdb_c COMMENT "Generating source code for C shim library") - add_library(fdb_c_shim STATIC ${SHIM_LIB_GEN_SRC} foundationdb/fdb_c_shim.h fdb_c_shim.cpp) + add_library(fdb_c_shim SHARED ${SHIM_LIB_GEN_SRC} foundationdb/fdb_c_shim.h fdb_c_shim.cpp) target_link_options(fdb_c_shim PRIVATE "LINKER:--version-script=${CMAKE_CURRENT_SOURCE_DIR}/fdb_c.map,-z,nodelete,-z,noexecstack") target_link_libraries(fdb_c_shim PUBLIC dl) target_include_directories(fdb_c_shim PUBLIC From 6c0f890f78744b68c15c11ea7f98d2e00fa001bc Mon Sep 17 00:00:00 2001 From: Jingyu Zhou Date: Wed, 26 Oct 2022 23:02:46 -0700 Subject: [PATCH 56/95] Fix restarting restore test failure Old fdbserver may not set the "enableSnapshotBackupEncryption" key, thus we should allow the key to be not present. --- fdbclient/FileBackupAgent.actor.cpp | 9 +++++---- 1 file changed, 5 insertions(+), 4 deletions(-) diff --git a/fdbclient/FileBackupAgent.actor.cpp b/fdbclient/FileBackupAgent.actor.cpp index 137f401df0..b5e10f4a48 100644 --- a/fdbclient/FileBackupAgent.actor.cpp +++ b/fdbclient/FileBackupAgent.actor.cpp @@ -1750,7 +1750,7 @@ struct BackupRangeTaskFunc : BackupTaskFuncBase { state bool done = false; state int64_t nrKeys = 0; - state bool encryptionEnabled = false; + state Optional encryptionEnabled; loop { state RangeResultWithVersion values; @@ -1816,7 +1816,7 @@ struct BackupRangeTaskFunc : BackupTaskFuncBase { wait(taskBucket->keepRunning(tr, task) && storeOrThrow(snapshotBeginVersion, backup.snapshotBeginVersion().get(tr)) && - storeOrThrow(encryptionEnabled, backup.enableSnapshotBackupEncryption().get(tr)) && + store(encryptionEnabled, backup.enableSnapshotBackupEncryption().get(tr)) && store(snapshotRangeFileCount, backup.snapshotRangeFileCount().getD(tr))); break; @@ -1829,9 +1829,10 @@ struct BackupRangeTaskFunc : BackupTaskFuncBase { wait(bc->writeRangeFile(snapshotBeginVersion, snapshotRangeFileCount, outVersion, blockSize)); outFile = f; - encryptionEnabled = encryptionEnabled && cx->clientInfo->get().isEncryptionEnabled; + const bool encrypted = + encryptionEnabled.present() && encryptionEnabled.get() && cx->clientInfo->get().isEncryptionEnabled; // Initialize range file writer and write begin key - if (encryptionEnabled) { + if (encrypted) { CODE_PROBE(true, "using encrypted snapshot file writer"); if (!tenantCache.isValid()) { tenantCache = makeReference>(cx, TenantEntryCacheRefreshMode::WATCH); From 4d3553481fc6a396fa8791bb521bbd9ce9b789d2 Mon Sep 17 00:00:00 2001 From: Josh Slocum Date: Thu, 27 Oct 2022 10:44:06 -0500 Subject: [PATCH 57/95] Blob connection provider test (#8478) * Refactoring test blob metadata creation * Implementing BlobConnectionProviderTest * createRandomTestBlobMetadata supports blobstore and works outside simulation --- fdbclient/BlobMetadataUtils.cpp | 109 ++++++++++ fdbclient/ClientKnobs.cpp | 1 + fdbclient/ServerKnobs.cpp | 2 - .../include/fdbclient/BlobMetadataUtils.h | 4 + fdbclient/include/fdbclient/ClientKnobs.h | 1 + fdbclient/include/fdbclient/ServerKnobs.h | 1 - .../BlobConnectionProviderTest.actor.cpp | 202 ++++++++++++++++++ fdbserver/EncryptKeyProxy.actor.cpp | 4 +- fdbserver/SimKmsConnector.actor.cpp | 59 +---- fdbserver/workloads/UnitTests.actor.cpp | 2 + tests/rare/SpecificUnitTests.toml | 2 +- 11 files changed, 325 insertions(+), 62 deletions(-) create mode 100644 fdbclient/BlobMetadataUtils.cpp create mode 100644 fdbserver/BlobConnectionProviderTest.actor.cpp diff --git a/fdbclient/BlobMetadataUtils.cpp b/fdbclient/BlobMetadataUtils.cpp new file mode 100644 index 0000000000..da0d5241a2 --- /dev/null +++ b/fdbclient/BlobMetadataUtils.cpp @@ -0,0 +1,109 @@ +/* + * BlobMetadataUtils.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/BlobMetadataUtils.h" + +#include "fmt/format.h" +#include "flow/IRandom.h" +#include "flow/flow.h" +#include "fdbclient/Knobs.h" +#include "fdbclient/S3BlobStore.h" + +std::string buildPartitionPath(const std::string& url, const std::string& partition) { + ASSERT(!partition.empty()); + ASSERT(partition.front() != '/'); + ASSERT(partition.back() == '/'); + StringRef u(url); + if (u.startsWith("file://"_sr)) { + ASSERT(u.endsWith("/"_sr)); + return url + partition; + } else if (u.startsWith("blobstore://"_sr)) { + std::string resource; + std::string lastOpenError; + S3BlobStoreEndpoint::ParametersT backupParams; + + std::string urlCopy = url; + + Reference bstore = + S3BlobStoreEndpoint::fromString(url, {}, &resource, &lastOpenError, &backupParams); + + ASSERT(!resource.empty()); + ASSERT(resource.back() != '/'); + size_t resourceStart = url.find(resource); + ASSERT(resourceStart != std::string::npos); + + return urlCopy.insert(resourceStart + resource.size(), "/" + partition); + } else { + // FIXME: support azure + throw backup_invalid_url(); + } +} + +// FIXME: make this (more) deterministic outside of simulation for FDBPerfKmsConnector +Standalone createRandomTestBlobMetadata(const std::string& baseUrl, + BlobMetadataDomainId domainId, + BlobMetadataDomainName domainName) { + Standalone metadata; + metadata.domainId = domainId; + metadata.arena().dependsOn(domainName.arena()); + 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); + TraceEvent ev(SevDebug, "SimBlobMetadata"); + ev.detail("DomainId", domainId).detail("TypeNum", type).detail("PartitionCount", partitionCount); + if (type == 0) { + // single storage location + std::string partition = std::to_string(domainId) + "/"; + metadata.base = StringRef(metadata.arena(), buildPartitionPath(baseUrl, partition)); + ev.detail("Base", metadata.base); + } + if (type == 1) { + // simulate hash prefixing in s3 + metadata.base = StringRef(metadata.arena(), baseUrl); + ev.detail("Base", metadata.base); + for (int i = 0; i < partitionCount; i++) { + metadata.partitions.push_back_deep(metadata.arena(), + deterministicRandom()->randomUniqueID().shortString() + "-" + + std::to_string(domainId) + "/"); + ev.detail("P" + std::to_string(i), metadata.partitions.back()); + } + } + if (type == 2) { + // simulate separate storage location per partition + for (int i = 0; i < partitionCount; i++) { + std::string partition = std::to_string(domainId) + "_" + std::to_string(i) + "/"; + metadata.partitions.push_back_deep(metadata.arena(), buildPartitionPath(baseUrl, partition)); + ev.detail("P" + std::to_string(i), metadata.partitions.back()); + } + } + + // set random refresh + expire time + if (deterministicRandom()->coinflip()) { + metadata.refreshAt = now() + deterministicRandom()->random01() * CLIENT_KNOBS->BLOB_METADATA_REFRESH_INTERVAL; + metadata.expireAt = + metadata.refreshAt + deterministicRandom()->random01() * CLIENT_KNOBS->BLOB_METADATA_REFRESH_INTERVAL; + } else { + metadata.refreshAt = std::numeric_limits::max(); + metadata.expireAt = metadata.refreshAt; + } + + return metadata; +} \ No newline at end of file diff --git a/fdbclient/ClientKnobs.cpp b/fdbclient/ClientKnobs.cpp index 36fdaea5b0..52879bf4d0 100644 --- a/fdbclient/ClientKnobs.cpp +++ b/fdbclient/ClientKnobs.cpp @@ -281,6 +281,7 @@ void ClientKnobs::initialize(Randomize randomize) { // Blob granules init( BG_MAX_GRANULE_PARALLELISM, 10 ); init( BG_TOO_MANY_GRANULES, 10000 ); + init( BLOB_METADATA_REFRESH_INTERVAL, 3600 ); if ( randomize && BUGGIFY ) { BLOB_METADATA_REFRESH_INTERVAL = deterministicRandom()->randomInt(5, 120); } init( CHANGE_QUORUM_BAD_STATE_RETRY_TIMES, 3 ); init( CHANGE_QUORUM_BAD_STATE_RETRY_DELAY, 2.0 ); diff --git a/fdbclient/ServerKnobs.cpp b/fdbclient/ServerKnobs.cpp index 68eed2867f..0275ab7761 100644 --- a/fdbclient/ServerKnobs.cpp +++ b/fdbclient/ServerKnobs.cpp @@ -1002,8 +1002,6 @@ void ServerKnobs::initialize(Randomize randomize, ClientKnobs* clientKnobs, IsSi // Blob Metadata init( BLOB_METADATA_CACHE_TTL, isSimulated ? 120 : 24 * 60 * 60 ); if ( randomize && BUGGIFY) { BLOB_METADATA_CACHE_TTL = deterministicRandom()->randomInt(50, 100); } - init( BLOB_METADATA_REFRESH_INTERVAL, isSimulated ? 60 : 60 * 60 ); - if ( randomize && BUGGIFY) { BLOB_METADATA_REFRESH_INTERVAL = deterministicRandom()->randomInt(5, 120); } // HTTP KMS Connector init( REST_KMS_CONNECTOR_KMS_DISCOVERY_URL_MODE, "file"); diff --git a/fdbclient/include/fdbclient/BlobMetadataUtils.h b/fdbclient/include/fdbclient/BlobMetadataUtils.h index 3f486b0bb6..56f123100e 100644 --- a/fdbclient/include/fdbclient/BlobMetadataUtils.h +++ b/fdbclient/include/fdbclient/BlobMetadataUtils.h @@ -91,4 +91,8 @@ struct BlobMetadataDetailsRef { } }; +Standalone createRandomTestBlobMetadata(const std::string& baseUrl, + BlobMetadataDomainId domainId, + BlobMetadataDomainName domainName); + #endif \ No newline at end of file diff --git a/fdbclient/include/fdbclient/ClientKnobs.h b/fdbclient/include/fdbclient/ClientKnobs.h index b4961ba3f7..70f0069be3 100644 --- a/fdbclient/include/fdbclient/ClientKnobs.h +++ b/fdbclient/include/fdbclient/ClientKnobs.h @@ -272,6 +272,7 @@ public: // Blob Granules int BG_MAX_GRANULE_PARALLELISM; int BG_TOO_MANY_GRANULES; + int64_t BLOB_METADATA_REFRESH_INTERVAL; // The coordinator key/value in storage server might be inconsistent to the value stored in the cluster file. // This might happen when a recovery is happening together with a cluster controller coordinator key change. diff --git a/fdbclient/include/fdbclient/ServerKnobs.h b/fdbclient/include/fdbclient/ServerKnobs.h index 5a5df49dc8..5fdc88347d 100644 --- a/fdbclient/include/fdbclient/ServerKnobs.h +++ b/fdbclient/include/fdbclient/ServerKnobs.h @@ -979,7 +979,6 @@ public: // Blob metadata int64_t BLOB_METADATA_CACHE_TTL; - int64_t BLOB_METADATA_REFRESH_INTERVAL; // HTTP KMS Connector std::string REST_KMS_CONNECTOR_KMS_DISCOVERY_URL_MODE; diff --git a/fdbserver/BlobConnectionProviderTest.actor.cpp b/fdbserver/BlobConnectionProviderTest.actor.cpp new file mode 100644 index 0000000000..4ac6e5884d --- /dev/null +++ b/fdbserver/BlobConnectionProviderTest.actor.cpp @@ -0,0 +1,202 @@ +/* + * BlobConnectionProviderTest.actor.cpp + * + * This source file is part of the FoundationDB open source project + * + * Copyright 2013-2022 Apple Inc. and the FoundationDB project authors + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +#include "fdbclient/BlobConnectionProvider.h" + +#include "flow/UnitTest.h" +#include "fdbserver/Knobs.h" +#include "flow/actorcompiler.h" // has to be last include + +void forceLinkBlobConnectionProviderTests() {} + +struct ConnectionProviderTestSettings { + uint32_t numProviders; + uint32_t filesPerProvider; + uint32_t maxFileMemory; + uint32_t maxFileSize; + uint32_t threads; + bool uniformProviderChoice; + double readWriteSplit; + + double runtime; + + int writeOps; + int readOps; + + ConnectionProviderTestSettings() { + numProviders = deterministicRandom()->randomSkewedUInt32(1, 1000); + filesPerProvider = + 1 + std::min((uint32_t)100, deterministicRandom()->randomSkewedUInt32(10, 10000) / numProviders); + maxFileMemory = 1024 * 1024 * 1024; + maxFileSize = maxFileMemory / (numProviders * filesPerProvider); + maxFileSize = deterministicRandom()->randomSkewedUInt32(8, std::min((uint32_t)(16 * 1024 * 1024), maxFileSize)); + threads = deterministicRandom()->randomInt(16, 128); + + uniformProviderChoice = deterministicRandom()->coinflip(); + readWriteSplit = deterministicRandom()->randomInt(1, 10) / 10.0; + + runtime = 60.0; + + writeOps = 0; + readOps = 0; + } +}; + +struct ProviderTestData { + Reference provider; + std::vector> data; + std::unordered_set usedNames; + + ProviderTestData() {} + explicit ProviderTestData(Reference provider) : provider(provider) {} +}; + +ACTOR Future createObject(ConnectionProviderTestSettings* settings, ProviderTestData* provider) { + // pick object name before wait so no collisions between concurrent writes + std::string objName; + loop { + objName = deterministicRandom()->randomAlphaNumeric(12); + if (provider->usedNames.insert(objName).second) { + break; + } + } + + int randomDataSize = deterministicRandom()->randomInt(1, settings->maxFileSize); + state Value data = makeString(randomDataSize); + deterministicRandom()->randomBytes(mutateString(data), randomDataSize); + + state Reference bstore; + state std::string fullPath; + std::tie(bstore, fullPath) = provider->provider->createForWrite(objName); + + state Reference file = wait(bstore->writeFile(fullPath)); + wait(file->append(data.begin(), data.size())); + wait(file->finish()); + + // after write, put in the readable list + provider->data.push_back({ fullPath, data }); + + return Void(); +} + +ACTOR Future readAndVerifyObject(ProviderTestData* provider, std::string objFullPath, Value expectedData) { + Reference bstore = provider->provider->getForRead(objFullPath); + state Reference reader = wait(bstore->readFile(objFullPath)); + + state Value actualData = makeString(expectedData.size()); + int readSize = wait(reader->read(mutateString(actualData), expectedData.size(), 0)); + ASSERT_EQ(expectedData.size(), readSize); + ASSERT(expectedData == actualData); + + return Void(); +} + +Future deleteObject(ProviderTestData* provider, std::string objFullPath) { + Reference bstore = provider->provider->getForRead(objFullPath); + return bstore->deleteFile(objFullPath); +} + +ACTOR Future workerThread(ConnectionProviderTestSettings* settings, std::vector* providers) { + state double endTime = now() + settings->runtime; + try { + while (now() < endTime) { + // randomly pick provider + int providerIdx; + if (settings->uniformProviderChoice) { + providerIdx = deterministicRandom()->randomInt(0, providers->size()); + } else { + providerIdx = deterministicRandom()->randomSkewedUInt32(0, providers->size()); + } + ProviderTestData* provider = &(*providers)[providerIdx]; + + // randomly pick create or read + bool doWrite = deterministicRandom()->random01() < settings->readWriteSplit; + if (provider->usedNames.size() < settings->filesPerProvider && (provider->data.empty() || doWrite)) { + // create an object + wait(createObject(settings, provider)); + settings->writeOps++; + } else if (!provider->data.empty()) { + // read a random object + auto& readInfo = provider->data[deterministicRandom()->randomInt(0, provider->data.size())]; + wait(readAndVerifyObject(provider, readInfo.first, readInfo.second)); + settings->readOps++; + } else { + // other threads are creating files up to filesPerProvider limit, but none finished yet. Just wait + wait(delay(0.1)); + } + } + + return Void(); + } catch (Error& e) { + fmt::print("WorkerThread Unexpected Error {0}\n", e.name()); + throw e; + } +} + +ACTOR Future checkAndCleanUp(ProviderTestData* provider) { + state int i; + ASSERT(provider->usedNames.size() == provider->data.size()); + + for (i = 0; i < provider->data.size(); i++) { + auto& readInfo = provider->data[i]; + wait(readAndVerifyObject(provider, readInfo.first, readInfo.second)); + wait(deleteObject(provider, provider->data[i].first)); + } + + return Void(); +} + +// maybe this should be a workload instead? +TEST_CASE("/fdbserver/blob/connectionprovider") { + state ConnectionProviderTestSettings settings; + + state std::vector providers; + providers.reserve(settings.numProviders); + for (int i = 0; i < settings.numProviders; i++) { + std::string nameStr = std::to_string(i); + BlobMetadataDomainName name(nameStr); + auto metadata = createRandomTestBlobMetadata(SERVER_KNOBS->BG_URL, i, name); + providers.emplace_back(BlobConnectionProvider::newBlobConnectionProvider(metadata)); + } + fmt::print("BlobConnectionProviderTest\n"); + + state std::vector> futures; + futures.reserve(settings.threads); + for (int i = 0; i < settings.threads; i++) { + futures.push_back(workerThread(&settings, &providers)); + } + + wait(waitForAll(futures)); + + fmt::print("BlobConnectionProviderTest workload phase complete with {0} files and {1} reads\n", + settings.writeOps, + settings.readOps); + + futures.clear(); + futures.reserve(providers.size()); + for (int i = 0; i < providers.size(); i++) { + futures.push_back(checkAndCleanUp(&providers[i])); + } + + wait(waitForAll(futures)); + + fmt::print("BlobConnectionProviderTest check and cleanup phase complete\n"); + return Void(); +} \ No newline at end of file diff --git a/fdbserver/EncryptKeyProxy.actor.cpp b/fdbserver/EncryptKeyProxy.actor.cpp index 095f3fb386..7eb8b264a5 100644 --- a/fdbserver/EncryptKeyProxy.actor.cpp +++ b/fdbserver/EncryptKeyProxy.actor.cpp @@ -625,7 +625,7 @@ bool isBlobMetadataEligibleForRefresh(const BlobMetadataDetailsRef& blobMetadata if (BUGGIFY_WITH_PROB(0.01)) { return true; } - int64_t nextRefreshCycleTS = currTS + SERVER_KNOBS->BLOB_METADATA_REFRESH_INTERVAL; + int64_t nextRefreshCycleTS = currTS + CLIENT_KNOBS->BLOB_METADATA_REFRESH_INTERVAL; return nextRefreshCycleTS > blobMetadata.expireAt || nextRefreshCycleTS > blobMetadata.refreshAt; } @@ -895,7 +895,7 @@ ACTOR Future encryptKeyProxyServer(EncryptKeyProxyInterface ekpInterface, TaskPriority::Worker); self->blobMetadataRefresher = recurring([&]() { refreshBlobMetadata(self, kmsConnectorInf); }, - SERVER_KNOBS->BLOB_METADATA_REFRESH_INTERVAL, + CLIENT_KNOBS->BLOB_METADATA_REFRESH_INTERVAL, TaskPriority::Worker); try { diff --git a/fdbserver/SimKmsConnector.actor.cpp b/fdbserver/SimKmsConnector.actor.cpp index a8701cd166..80031c6372 100644 --- a/fdbserver/SimKmsConnector.actor.cpp +++ b/fdbserver/SimKmsConnector.actor.cpp @@ -192,61 +192,6 @@ ACTOR Future ekLookupByDomainIds(Reference ctx, success ? req.reply.send(rep) : req.reply.sendError(encrypt_key_not_found()); return Void(); } -// 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.arena().dependsOn(domainName.arena()); - 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); - fmt::print("SimBlobMetadata ({})\n", domainId); - TraceEvent ev(SevDebug, "SimBlobMetadata"); - ev.detail("DomainId", domainId).detail("TypeNum", type).detail("PartitionCount", partitionCount); - if (type == 0) { - // single storage location - metadata.base = StringRef(metadata.arena(), "file://fdbblob/" + std::to_string(domainId) + "/"); - fmt::print(" {}\n", metadata.base.get().printable()); - ev.detail("Base", metadata.base); - } - if (type == 1) { - // simulate hash prefixing in s3 - metadata.base = StringRef(metadata.arena(), "file://fdbblob/"_sr); - ev.detail("Base", metadata.base); - fmt::print(" {} ({})\n", metadata.base.get().printable(), partitionCount); - for (int i = 0; i < partitionCount; i++) { - metadata.partitions.push_back_deep(metadata.arena(), - deterministicRandom()->randomUniqueID().shortString() + "-" + - std::to_string(domainId) + "/"); - fmt::print(" {}\n", metadata.partitions.back().printable()); - ev.detail("P" + std::to_string(i), metadata.partitions.back()); - } - } - if (type == 2) { - // simulate separate storage location per partition - for (int i = 0; i < partitionCount; i++) { - metadata.partitions.push_back_deep( - metadata.arena(), "file://fdbblob" + std::to_string(domainId) + "_" + std::to_string(i) + "/"); - fmt::print(" {}\n", metadata.partitions.back().printable()); - ev.detail("P" + std::to_string(i), metadata.partitions.back()); - } - } - - // set random refresh + expire time - if (deterministicRandom()->coinflip()) { - metadata.refreshAt = now() + deterministicRandom()->random01() * SERVER_KNOBS->BLOB_METADATA_REFRESH_INTERVAL; - metadata.expireAt = - metadata.refreshAt + deterministicRandom()->random01() * SERVER_KNOBS->BLOB_METADATA_REFRESH_INTERVAL; - } else { - metadata.refreshAt = std::numeric_limits::max(); - metadata.expireAt = metadata.refreshAt; - } - - return metadata; -} ACTOR Future blobMetadataLookup(KmsConnectorInterface interf, KmsConnBlobMetadataReq req) { state KmsConnBlobMetadataRep rep; @@ -261,7 +206,9 @@ ACTOR Future blobMetadataLookup(KmsConnectorInterface interf, KmsConnBlobM if (it == simBlobMetadataStore.end()) { // construct new blob metadata it = simBlobMetadataStore - .insert({ domainInfo.domainId, createBlobMetadata(domainInfo.domainId, domainInfo.domainName) }) + .insert({ domainInfo.domainId, + createRandomTestBlobMetadata( + SERVER_KNOBS->BG_URL, domainInfo.domainId, domainInfo.domainName) }) .first; } else if (now() >= it->second.expireAt) { // update random refresh and expire time diff --git a/fdbserver/workloads/UnitTests.actor.cpp b/fdbserver/workloads/UnitTests.actor.cpp index eecff51fcb..bf2c4394ee 100644 --- a/fdbserver/workloads/UnitTests.actor.cpp +++ b/fdbserver/workloads/UnitTests.actor.cpp @@ -44,6 +44,7 @@ void forceLinkRESTKmsConnectorTest(); void forceLinkCompressionUtilsTest(); void forceLinkAtomicTests(); void forceLinkIdempotencyIdTests(); +void forceLinkBlobConnectionProviderTests(); struct UnitTestWorkload : TestWorkload { static constexpr auto NAME = "UnitTests"; @@ -104,6 +105,7 @@ struct UnitTestWorkload : TestWorkload { forceLinkCompressionUtilsTest(); forceLinkAtomicTests(); forceLinkIdempotencyIdTests(); + forceLinkBlobConnectionProviderTests(); } Future setup(Database const& cx) override { diff --git a/tests/rare/SpecificUnitTests.toml b/tests/rare/SpecificUnitTests.toml index 87a8d1a248..72c2b1d387 100644 --- a/tests/rare/SpecificUnitTests.toml +++ b/tests/rare/SpecificUnitTests.toml @@ -7,4 +7,4 @@ startDelay = 0 [[test.workload]] testName = 'UnitTests' maxTestCases = 0 - testsMatching = '/GrvProxyTransactionTagThrottler/Fifo' + testsMatching = '/' From bf01d9b879b59b6b930f6c30b4a1e8a840d01c64 Mon Sep 17 00:00:00 2001 From: Nim Wijetunga Date: Thu, 27 Oct 2022 11:10:14 -0700 Subject: [PATCH 58/95] Bulk Setup Workload Improvements (#8573) * bulk setup workload improvements * fix workload * modify --- .../fdbserver/workloads/BulkSetup.actor.h | 3 +- ...ctor.cpp => BulkLoadWithTenants.actor.cpp} | 44 +++++++++++++++---- tests/fast/EncryptedBackupCorrectness.toml | 20 ++------- 3 files changed, 41 insertions(+), 26 deletions(-) rename fdbserver/workloads/{BulkSetup.actor.cpp => BulkLoadWithTenants.actor.cpp} (59%) diff --git a/fdbserver/include/fdbserver/workloads/BulkSetup.actor.h b/fdbserver/include/fdbserver/workloads/BulkSetup.actor.h index 3b2e9802eb..a3a545a003 100644 --- a/fdbserver/include/fdbserver/workloads/BulkSetup.actor.h +++ b/fdbserver/include/fdbserver/workloads/BulkSetup.actor.h @@ -252,7 +252,8 @@ Future bulkSetup(Database cx, .detail("NodeCount", nodeCount) .detail("ValuesInconsequential", valuesInconsequential) .detail("PostSetupWarming", postSetupWarming) - .detail("MaxKeyInsertRate", maxKeyInsertRate); + .detail("MaxKeyInsertRate", maxKeyInsertRate) + .detail("NumTenants", tenants.size()); // 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. diff --git a/fdbserver/workloads/BulkSetup.actor.cpp b/fdbserver/workloads/BulkLoadWithTenants.actor.cpp similarity index 59% rename from fdbserver/workloads/BulkSetup.actor.cpp rename to fdbserver/workloads/BulkLoadWithTenants.actor.cpp index 7bcb3320be..ed6a443eb2 100644 --- a/fdbserver/workloads/BulkSetup.actor.cpp +++ b/fdbserver/workloads/BulkLoadWithTenants.actor.cpp @@ -18,29 +18,33 @@ * limitations under the License. */ +#include "fdbclient/TenantManagement.actor.h" #include "fdbrpc/ContinuousSample.h" -#include "fdbclient/NativeAPI.actor.h" +#include "fdbserver/Knobs.h" #include "fdbserver/TesterInterface.actor.h" #include "fdbserver/workloads/workloads.actor.h" #include "fdbserver/workloads/BulkSetup.actor.h" +#include "flow/genericactors.actor.h" #include "flow/actorcompiler.h" // This must be the last #include. struct BulkSetupWorkload : TestWorkload { - static constexpr auto NAME = "BulkSetup"; + static constexpr auto NAME = "BulkLoadWithTenants"; - std::vector tenantNames; int nodeCount; double transactionsPerSecond; Key keyPrefix; + double maxNumTenantsPerClient; + double minNumTenantsPerClient; + std::vector tenantNames; BulkSetupWorkload(WorkloadContext const& wcx) : TestWorkload(wcx) { transactionsPerSecond = getOption(options, "transactionsPerSecond"_sr, 5000.0) / clientCount; nodeCount = getOption(options, "nodeCount"_sr, transactionsPerSecond * clientCount); keyPrefix = unprintable(getOption(options, "keyPrefix"_sr, ""_sr).toString()); - std::vector tenants = getOption(options, "tenants"_sr, std::vector()); - for (std::string tenant : tenants) { - tenantNames.push_back(TenantName(tenant)); - } + // maximum and minimum number of tenants per client + maxNumTenantsPerClient = getOption(options, "maxNumTenantsPerClient"_sr, 0); + minNumTenantsPerClient = getOption(options, "minNumTenantsPerClient"_sr, 0); + ASSERT(minNumTenantsPerClient <= maxNumTenantsPerClient); } void getMetrics(std::vector& m) override {} @@ -51,6 +55,28 @@ struct BulkSetupWorkload : TestWorkload { Standalone operator()(int n) { return KeyValueRef(key(n), value((n + 1) % nodeCount)); } + ACTOR static Future _setup(BulkSetupWorkload* workload, Database cx) { + // create a bunch of tenants (between min and max tenants) + state int numTenantsToCreate = + deterministicRandom()->randomInt(workload->minNumTenantsPerClient, workload->maxNumTenantsPerClient + 1); + TraceEvent("BulkSetupTenantCreation").detail("NumTenants", numTenantsToCreate); + if (numTenantsToCreate > 0) { + std::vector> tenantFutures; + for (int i = 0; i < numTenantsToCreate; i++) { + TenantMapEntry entry; + entry.encrypted = SERVER_KNOBS->ENABLE_ENCRYPTION; + workload->tenantNames.push_back(TenantName(format("BulkSetupTenant_%04d_%04d", workload->clientId, i))); + TraceEvent("CreatingTenant") + .detail("Tenant", workload->tenantNames.back()) + .detail("TenantGroup", entry.tenantGroup); + tenantFutures.push_back( + success(TenantAPI::createTenant(cx.getReference(), workload->tenantNames.back()))); + } + wait(waitForAll(tenantFutures)); + } + return Void(); + } + Future start(Database const& cx) override { return bulkSetup(cx, this, @@ -65,9 +91,11 @@ struct BulkSetupWorkload : TestWorkload { 0.1, 0, 0, - this->tenantNames); + tenantNames); } + Future setup(Database const& cx) override { return _setup(this, cx); } + Future check(Database const& cx) override { return true; } }; diff --git a/tests/fast/EncryptedBackupCorrectness.toml b/tests/fast/EncryptedBackupCorrectness.toml index e674555491..db3d16140a 100644 --- a/tests/fast/EncryptedBackupCorrectness.toml +++ b/tests/fast/EncryptedBackupCorrectness.toml @@ -7,29 +7,15 @@ enable_tlog_encryption = true enable_storage_server_encryption = false enable_blob_granule_encryption = true -[[test]] -testTitle = 'TenantCreation' - - [[test.workload]] - testName = 'CreateTenant' - name = 'First' - - [[test.workload]] - testName = 'CreateTenant' - name = 'Second' - - [[test.workload]] - testName = 'CreateTenant' - name = 'Third' - [[test]] testTitle = 'EncryptedBackupAndRestore' clearAfterTest = false simBackupAgents = 'BackupToFile' [[test.workload]] - testName = 'BulkSetup' - tenants = 'First,Second,Third' + testName = 'BulkLoadWithTenants' + maxNumTenantsPerClient = 100 + minNumTenantsPerClient = 0 transactionsPerSecond = 2500.0 [[test.workload]] From 67049518b937fcd89f0f0b705fbedb096b628615 Mon Sep 17 00:00:00 2001 From: Alex Moundalexis Date: Thu, 27 Oct 2022 15:05:52 -0400 Subject: [PATCH 59/95] updated copyright year on web site --- documentation/sphinx/conf.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/documentation/sphinx/conf.py b/documentation/sphinx/conf.py index 04ed43d87b..31a88faaf8 100644 --- a/documentation/sphinx/conf.py +++ b/documentation/sphinx/conf.py @@ -49,7 +49,7 @@ master_doc = 'index' # General information about the project. project = u'FoundationDB' -copyright = u'2013-2021 Apple, Inc and the FoundationDB project authors' +copyright = u'2013-2022 Apple, Inc and the FoundationDB project authors' # Load the version information from 'versions.target' import xml.etree.ElementTree as ET From 03b102d86aecbe700aa8402ae31d0431bfb0b2b9 Mon Sep 17 00:00:00 2001 From: Xiaoge Su Date: Fri, 14 Oct 2022 19:11:25 -0700 Subject: [PATCH 60/95] Clean up unused comment in flow.h --- flow/include/flow/flow.h | 11 ++--------- 1 file changed, 2 insertions(+), 9 deletions(-) diff --git a/flow/include/flow/flow.h b/flow/include/flow/flow.h index 3cc7470dbf..cd6704f8b8 100644 --- a/flow/include/flow/flow.h +++ b/flow/include/flow/flow.h @@ -802,12 +802,8 @@ public: Future(const Future& rhs) : sav(rhs.sav) { if (sav) sav->addFutureRef(); - // if (sav->endpoint.isValid()) std::cout << "Future copied for " << sav->endpoint.key << std::endl; - } - Future(Future&& rhs) noexcept : sav(rhs.sav) { - rhs.sav = 0; - // if (sav->endpoint.isValid()) std::cout << "Future moved for " << sav->endpoint.key << std::endl; } + Future(Future&& rhs) noexcept : sav(rhs.sav) { rhs.sav = 0; } Future(const T& presentValue) : sav(new SAV(1, 0)) { sav->send(presentValue); } Future(T&& presentValue) : sav(new SAV(1, 0)) { sav->send(std::move(presentValue)); } Future(Never) : sav(new SAV(1, 0)) { sav->send(Never()); } @@ -819,7 +815,6 @@ public: #endif ~Future() { - // if (sav && sav->endpoint.isValid()) std::cout << "Future destroyed for " << sav->endpoint.key << std::endl; if (sav) sav->delFutureRef(); } @@ -864,9 +859,7 @@ public: int getFutureReferenceCount() const { return sav->getFutureReferenceCount(); } int getPromiseReferenceCount() const { return sav->getPromiseReferenceCount(); } - explicit Future(SAV* sav) : sav(sav) { - // if (sav->endpoint.isValid()) std::cout << "Future created for " << sav->endpoint.key << std::endl; - } + explicit Future(SAV* sav) : sav(sav) {} private: SAV* sav; From 639afbe62cc157a3428261bf8783088becc9ac13 Mon Sep 17 00:00:00 2001 From: Xiaoge Su Date: Mon, 17 Oct 2022 20:45:02 -0700 Subject: [PATCH 61/95] Cancel watch when the key is not being waited Currently, there is a cyclic reference situation in DatabaseContext -> WatchMetadata -> watchStorageServerResp -> DatabaseContext If there is a watch created in the DatabaseContext, even the corresponding wait ACTOR is cancelled, the WatchMetadata will still hold a reference to watchStorageServerResp ACTOR, which holds a reference to DatabaseContext. In this situation, any DatabaseContext who held a watch will not be automatically destructed since its reference count will never reduce to 0 until the watch value is changed. Every time the cluster recoveries, several watches are created, and when the cluster restarts, the DatabaseContext which not being used, will not be able to destructed due to these watches. With this patch, each wait to the watch will be counted. Either the watch is triggered or cancelled, the corresponding count will be reduced. If a watch is not being waited, the watch will be cancelled, effectively reduce the reference count of DatabaseContext. This will hopefully fix the issue mentioned above. The code is tested by 1) Manually change the number of logs of a local cluster, see the cluster recovery and previous DatabaseContext being destructed; 2) 100K joshua run, with 1 failure, the same test will fail on the current git main branch. --- fdbclient/NativeAPI.actor.cpp | 94 ++++++++++++- fdbclient/include/fdbclient/DatabaseContext.h | 38 +++++- fdbserver/ClusterRecovery.actor.cpp | 124 +++++++++++++----- 3 files changed, 213 insertions(+), 43 deletions(-) diff --git a/fdbclient/NativeAPI.actor.cpp b/fdbclient/NativeAPI.actor.cpp index fa301ef15c..fcef80270a 100644 --- a/fdbclient/NativeAPI.actor.cpp +++ b/fdbclient/NativeAPI.actor.cpp @@ -2163,14 +2163,14 @@ void DatabaseContext::setOption(FDBDatabaseOptions::Option option, Optional= maxOutstandingWatches) throw too_many_watches(); ++outstandingWatches; } -void DatabaseContext::removeWatch() { +void DatabaseContext::removeWatchCounter() { --outstandingWatches; ASSERT(outstandingWatches >= 0); } @@ -2390,15 +2390,44 @@ Reference DatabaseContext::getWatchMetadata(int64_t tenantId, Key } void DatabaseContext::setWatchMetadata(Reference metadata) { - watchMap[std::make_pair(metadata->parameters->tenant.tenantId, metadata->parameters->key)] = metadata; + const WatchMapKey key(metadata->parameters->tenant.tenantId, metadata->parameters->key); + watchMap[key] = metadata; + // NOTE Here we do *NOT* update/reset the reference count for the key, see the source code in getWatchFuture +} + +int32_t DatabaseContext::increaseWatchRefCount(const int64_t tenantID, KeyRef key) { + const WatchMapKey mapKey(tenantID, key); + if (watchCounterMap.count(mapKey) == 0) { + watchCounterMap[mapKey] = 0; + } + const auto count = ++watchCounterMap[mapKey]; + return count; +} + +int32_t DatabaseContext::decreaseWatchRefCount(const int64_t tenantID, KeyRef key) { + const WatchMapKey mapKey(tenantID, key); + if (watchCounterMap.count(mapKey) == 0) { + // Key does not exist. The metadata might be removed by deleteWatchMetadata already. + return 0; + } + const auto count = --watchCounterMap[mapKey]; + ASSERT(watchCounterMap[mapKey] >= 0); + if (watchCounterMap[mapKey] == 0) { + getWatchMetadata(tenantID, key)->watchFutureSS.cancel(); + deleteWatchMetadata(tenantID, key); + } + return count; } void DatabaseContext::deleteWatchMetadata(int64_t tenantId, KeyRef key) { - watchMap.erase(std::make_pair(tenantId, key)); + const WatchMapKey mapKey(tenantId, key); + watchMap.erase(mapKey); + watchCounterMap.erase(mapKey); } void DatabaseContext::clearWatchMetadata() { watchMap.clear(); + watchCounterMap.clear(); } const UniqueOrderedOptionList& Database::getTransactionDefaults() const { @@ -3913,6 +3942,56 @@ Future getWatchFuture(Database cx, Reference parameters) return Void(); } +namespace { + +// NOTE: Since an ACTOR could receive multiple exceptions for a single catch clause, e.g. broken promise together with +// operation cancelled, If the decreaseWatchRefCount is placed at the catch clause, it might be triggered for multiple +// times. One could check if the SAV isSet, but seems a more intuitive way is to use RAII-style constructor/destructor +// pair. Yet the object has to be constructed after a wait statement, so it must be trivially-constructible. This +// requires move-assignment operator implemented. +class WatchRefCountUpdater { + Database cx; + int64_t tenantID; + KeyRef key; + + void tryAddRefCount() { + if (cx.getReference()) { + cx->increaseWatchRefCount(tenantID, key); + } + } + + void tryDelRefCount() { + if (cx.getReference()) { + cx->decreaseWatchRefCount(tenantID, key); + } + } + +public: + WatchRefCountUpdater() {} + + WatchRefCountUpdater(const Database& cx_, const int64_t tenantID_, KeyRef key_) + : cx(cx_), tenantID(tenantID_), key(key_) { + tryAddRefCount(); + } + + WatchRefCountUpdater& operator=(WatchRefCountUpdater&& other) { + tryDelRefCount(); + + // NOTE: Do not use move semantic, this copy allows other delete the reference count properly. + cx = other.cx; + tenantID = other.tenantID; + key = other.key; + + tryAddRefCount(); + + return *this; + } + + ~WatchRefCountUpdater() { tryDelRefCount(); } +}; + +} // namespace + ACTOR Future watchValueMap(Future version, TenantInfo tenant, Key key, @@ -3924,6 +4003,7 @@ ACTOR Future watchValueMap(Future version, Optional debugID, UseProvisionalProxies useProvisionalProxies) { state Version ver = wait(version); + state WatchRefCountUpdater watchRefCountUpdater(cx, tenant.tenantId, key); wait(getWatchFuture(cx, makeReference( @@ -5456,11 +5536,11 @@ ACTOR Future watch(Reference watch, } } } catch (Error& e) { - cx->removeWatch(); + cx->removeWatchCounter(); throw; } - cx->removeWatch(); + cx->removeWatchCounter(); return Void(); } @@ -5471,7 +5551,7 @@ Future Transaction::getRawReadVersion() { Future Transaction::watch(Reference watch) { ++trState->cx->transactionWatchRequests; - trState->cx->addWatch(); + trState->cx->addWatchCounter(); watches.push_back(watch); return ::watch( watch, diff --git a/fdbclient/include/fdbclient/DatabaseContext.h b/fdbclient/include/fdbclient/DatabaseContext.h index 390873e0ef..33c31bf76e 100644 --- a/fdbclient/include/fdbclient/DatabaseContext.h +++ b/fdbclient/include/fdbclient/DatabaseContext.h @@ -156,6 +156,8 @@ public: WatchMetadata(Reference parameters) : watchFuture(watchPromise.getFuture()), parameters(parameters) {} + + ~WatchMetadata() { watchFutureSS.cancel(); } }; struct MutationAndVersionStream { @@ -328,14 +330,32 @@ public: // Note: this will never return if the server is running a protocol from FDB 5.0 or older Future getClusterProtocol(Optional expectedVersion = Optional()); - // Update the watch counter for the database - void addWatch(); - void removeWatch(); + // Increases the counter of the number of watches in this DatabaseContext by 1. If the number of watches is too + // many, throws too_many_watches. + void addWatchCounter(); + + // Decrease the counter of the number of watches in this DatabaseContext by 1 + void removeWatchCounter(); // watch map operations + + // Gets the watch metadata per tenant id and key Reference getWatchMetadata(int64_t tenantId, KeyRef key) const; + + // Refreshes the watch metadata. If the same watch is used (this is determined by the tenant id and the key), the + // metadata will be updated. void setWatchMetadata(Reference metadata); + + // Removes the watch metadata void deleteWatchMetadata(int64_t tenant, KeyRef key); + + // Increases reference count to the given watch. Returns the number of references to the watch. + int32_t increaseWatchRefCount(const int64_t tenant, KeyRef key); + + // Decreases reference count to the given watch. If the reference count is dropped to 0, the watch metadata will be + // removed. Returns the number of references to the watch. + int32_t decreaseWatchRefCount(const int64_t tenant, KeyRef key); + void clearWatchMetadata(); void setOption(FDBDatabaseOptions::Option option, Optional value); @@ -703,8 +723,16 @@ public: EventCacheHolder connectToDatabaseEventCacheHolder; private: - std::unordered_map, Reference, boost::hash>> - watchMap; + using WatchMapKey = std::pair; + using WatchMapKeyHasher = boost::hash; + using WatchMapValue = Reference; + using WatchMap_t = std::unordered_map; + + WatchMap_t watchMap; + + using WatchCounterMap_t = std::unordered_map; + // Maps the number of the WatchMapKey being used. + WatchCounterMap_t watchCounterMap; }; // Similar to tr.onError(), but doesn't require a DatabaseContext. diff --git a/fdbserver/ClusterRecovery.actor.cpp b/fdbserver/ClusterRecovery.actor.cpp index afc6d85bfa..77d3261a53 100644 --- a/fdbserver/ClusterRecovery.actor.cpp +++ b/fdbserver/ClusterRecovery.actor.cpp @@ -567,44 +567,106 @@ ACTOR Future changeCoordinators(Reference self) { } } -ACTOR Future configurationMonitor(Reference self, Database cx) { +namespace { + +// NOTE: This vector may not be initialized here as the keys might be initialized *AFTER* this vector, causing all +// keys empty. The reason is that the order of the initialization of keys and this vector might not be ordered as wished +// so the vector might be initialized before the keys receives values; thus, all values inside the vector are copied +// from uninitialized KeyRefs.. +// See C++11 standard 3.6.2 for more info. +std::vector configurationMonitorWatchKeys; + +ACTOR Future configurationMonitorImpl(Reference self, + Database cx, + Optional* pTenantId) { + state ReadYourWritesTransaction tr(cx); loop { - state ReadYourWritesTransaction tr(cx); + try { + tr.setOption(FDBTransactionOptions::ACCESS_SYSTEM_KEYS); + RangeResult results = wait(tr.getRange(configKeys, CLIENT_KNOBS->TOO_MANY)); + ASSERT(!results.more && results.size() < CLIENT_KNOBS->TOO_MANY); - loop { - try { - tr.setOption(FDBTransactionOptions::ACCESS_SYSTEM_KEYS); - RangeResult results = wait(tr.getRange(configKeys, CLIENT_KNOBS->TOO_MANY)); - ASSERT(!results.more && results.size() < CLIENT_KNOBS->TOO_MANY); - - DatabaseConfiguration conf; - conf.fromKeyValues((VectorRef)results); - TraceEvent("ConfigurationMonitor", self->dbgid) - .detail(getRecoveryEventName(ClusterRecoveryEventType::CLUSTER_RECOVERY_STATE_EVENT_NAME).c_str(), - self->recoveryState); - if (conf != self->configuration) { - if (self->recoveryState != RecoveryState::ALL_LOGS_RECRUITED && - self->recoveryState != RecoveryState::FULLY_RECOVERED) { - self->controllerData->shouldCommitSuicide = true; - throw restart_cluster_controller(); - } - - self->configuration = conf; - self->registrationTrigger.trigger(); + DatabaseConfiguration conf; + conf.fromKeyValues((VectorRef)results); + TraceEvent("ConfigurationMonitor", self->dbgid) + .detail(getRecoveryEventName(ClusterRecoveryEventType::CLUSTER_RECOVERY_STATE_EVENT_NAME).c_str(), + self->recoveryState); + if (conf != self->configuration) { + if (self->recoveryState != RecoveryState::ALL_LOGS_RECRUITED && + self->recoveryState != RecoveryState::FULLY_RECOVERED) { + self->controllerData->shouldCommitSuicide = true; + throw restart_cluster_controller(); } - state Future watchFuture = - tr.watch(moveKeysLockOwnerKey) || tr.watch(excludedServersVersionKey) || - tr.watch(failedServersVersionKey) || tr.watch(excludedLocalityVersionKey) || - tr.watch(failedLocalityVersionKey); - wait(tr.commit()); - wait(watchFuture); - break; - } catch (Error& e) { - wait(tr.onError(e)); + self->configuration = conf; + self->registrationTrigger.trigger(); } + + std::vector> watchFutures; + std::transform(std::begin(configurationMonitorWatchKeys), + std::end(configurationMonitorWatchKeys), + std::back_inserter(watchFutures), + [this](KeyRef key) { return tr.watch(key); }); + // Only after this stage, where getKeyLocation is called indirectly, the tenant information is updated and + // set to the transaction state. + (*pTenantId) = tr.getTransactionState()->tenantId(); + state Future watchFuture = waitForAny(watchFutures); + + wait(tr.commit()); + + wait(watchFuture); + break; + } catch (Error& e) { + if (e.code() == error_code_actor_cancelled) { + throw e; + } + wait(tr.onError(e)); } } + + return Void(); +} + +} // anonymous namespace + +ACTOR Future configurationMonitor(Reference self, Database cx) { + state Optional tenantId; + + // The keys cannot be initialized where it is defined. see comments at the definition. + if (configurationMonitorWatchKeys.empty()) { + configurationMonitorWatchKeys = std::vector{ moveKeysLockOwnerKey, + excludedServersVersionKey, + failedServersVersionKey, + excludedLocalityVersionKey, + failedLocalityVersionKey }; + } + + try { + while (true) { + wait(configurationMonitorImpl(self, cx, &tenantId)); + } + } catch (Error& e) { + if (e.code() != error_code_actor_cancelled) { + throw e; + } + + // Cancel all watches created by configurationMonitorImpl. Due to a circular reference issue, if the watches are + // not cancelled manually, the DatabaseContext object in cx will not be able to destructed properly, see + // + // https://github.com/apple/foundationdb/issues/8321 + // + // for more detailed discussion. + + if (!tenantId.present()) { + // Tenant ID not set, no watches are created in this case, no cleanup required. + return Void(); + } + std::for_each(std::begin(configurationMonitorWatchKeys), + std::end(configurationMonitorWatchKeys), + [this](KeyRef key) { cx->decreaseWatchRefCount(tenantId.get(), key); }); + } + + return Void(); } ACTOR static Future> getMinBackupVersion(Reference self, Database cx) { From ab0f827058c21dfab66462c3ce8545c6eec6a6e5 Mon Sep 17 00:00:00 2001 From: Xiaoge Su Date: Tue, 18 Oct 2022 12:28:36 -0700 Subject: [PATCH 62/95] configurationMonitor does not need to check watch reference count --- fdbserver/ClusterRecovery.actor.cpp | 124 +++++++--------------------- 1 file changed, 31 insertions(+), 93 deletions(-) diff --git a/fdbserver/ClusterRecovery.actor.cpp b/fdbserver/ClusterRecovery.actor.cpp index 77d3261a53..afc6d85bfa 100644 --- a/fdbserver/ClusterRecovery.actor.cpp +++ b/fdbserver/ClusterRecovery.actor.cpp @@ -567,106 +567,44 @@ ACTOR Future changeCoordinators(Reference self) { } } -namespace { - -// NOTE: This vector may not be initialized here as the keys might be initialized *AFTER* this vector, causing all -// keys empty. The reason is that the order of the initialization of keys and this vector might not be ordered as wished -// so the vector might be initialized before the keys receives values; thus, all values inside the vector are copied -// from uninitialized KeyRefs.. -// See C++11 standard 3.6.2 for more info. -std::vector configurationMonitorWatchKeys; - -ACTOR Future configurationMonitorImpl(Reference self, - Database cx, - Optional* pTenantId) { - state ReadYourWritesTransaction tr(cx); +ACTOR Future configurationMonitor(Reference self, Database cx) { loop { - try { - tr.setOption(FDBTransactionOptions::ACCESS_SYSTEM_KEYS); - RangeResult results = wait(tr.getRange(configKeys, CLIENT_KNOBS->TOO_MANY)); - ASSERT(!results.more && results.size() < CLIENT_KNOBS->TOO_MANY); + state ReadYourWritesTransaction tr(cx); - DatabaseConfiguration conf; - conf.fromKeyValues((VectorRef)results); - TraceEvent("ConfigurationMonitor", self->dbgid) - .detail(getRecoveryEventName(ClusterRecoveryEventType::CLUSTER_RECOVERY_STATE_EVENT_NAME).c_str(), - self->recoveryState); - if (conf != self->configuration) { - if (self->recoveryState != RecoveryState::ALL_LOGS_RECRUITED && - self->recoveryState != RecoveryState::FULLY_RECOVERED) { - self->controllerData->shouldCommitSuicide = true; - throw restart_cluster_controller(); + loop { + try { + tr.setOption(FDBTransactionOptions::ACCESS_SYSTEM_KEYS); + RangeResult results = wait(tr.getRange(configKeys, CLIENT_KNOBS->TOO_MANY)); + ASSERT(!results.more && results.size() < CLIENT_KNOBS->TOO_MANY); + + DatabaseConfiguration conf; + conf.fromKeyValues((VectorRef)results); + TraceEvent("ConfigurationMonitor", self->dbgid) + .detail(getRecoveryEventName(ClusterRecoveryEventType::CLUSTER_RECOVERY_STATE_EVENT_NAME).c_str(), + self->recoveryState); + if (conf != self->configuration) { + if (self->recoveryState != RecoveryState::ALL_LOGS_RECRUITED && + self->recoveryState != RecoveryState::FULLY_RECOVERED) { + self->controllerData->shouldCommitSuicide = true; + throw restart_cluster_controller(); + } + + self->configuration = conf; + self->registrationTrigger.trigger(); } - self->configuration = conf; - self->registrationTrigger.trigger(); + state Future watchFuture = + tr.watch(moveKeysLockOwnerKey) || tr.watch(excludedServersVersionKey) || + tr.watch(failedServersVersionKey) || tr.watch(excludedLocalityVersionKey) || + tr.watch(failedLocalityVersionKey); + wait(tr.commit()); + wait(watchFuture); + break; + } catch (Error& e) { + wait(tr.onError(e)); } - - std::vector> watchFutures; - std::transform(std::begin(configurationMonitorWatchKeys), - std::end(configurationMonitorWatchKeys), - std::back_inserter(watchFutures), - [this](KeyRef key) { return tr.watch(key); }); - // Only after this stage, where getKeyLocation is called indirectly, the tenant information is updated and - // set to the transaction state. - (*pTenantId) = tr.getTransactionState()->tenantId(); - state Future watchFuture = waitForAny(watchFutures); - - wait(tr.commit()); - - wait(watchFuture); - break; - } catch (Error& e) { - if (e.code() == error_code_actor_cancelled) { - throw e; - } - wait(tr.onError(e)); } } - - return Void(); -} - -} // anonymous namespace - -ACTOR Future configurationMonitor(Reference self, Database cx) { - state Optional tenantId; - - // The keys cannot be initialized where it is defined. see comments at the definition. - if (configurationMonitorWatchKeys.empty()) { - configurationMonitorWatchKeys = std::vector{ moveKeysLockOwnerKey, - excludedServersVersionKey, - failedServersVersionKey, - excludedLocalityVersionKey, - failedLocalityVersionKey }; - } - - try { - while (true) { - wait(configurationMonitorImpl(self, cx, &tenantId)); - } - } catch (Error& e) { - if (e.code() != error_code_actor_cancelled) { - throw e; - } - - // Cancel all watches created by configurationMonitorImpl. Due to a circular reference issue, if the watches are - // not cancelled manually, the DatabaseContext object in cx will not be able to destructed properly, see - // - // https://github.com/apple/foundationdb/issues/8321 - // - // for more detailed discussion. - - if (!tenantId.present()) { - // Tenant ID not set, no watches are created in this case, no cleanup required. - return Void(); - } - std::for_each(std::begin(configurationMonitorWatchKeys), - std::end(configurationMonitorWatchKeys), - [this](KeyRef key) { cx->decreaseWatchRefCount(tenantId.get(), key); }); - } - - return Void(); } ACTOR static Future> getMinBackupVersion(Reference self, Database cx) { From 4bd24e4d6460c5cf38117b89246561bb0d83e3ef Mon Sep 17 00:00:00 2001 From: Xiaoge Su Date: Wed, 19 Oct 2022 14:12:09 -0700 Subject: [PATCH 63/95] Record the version of each watch In the case 1. A watch to key A is set, the watchValueMap ACTOR, noted as X, starts waiting. 2. All watches are cleared due to connection string change. 3. The watch to key A is restarted with watchValueMap ACTOR Y. 4. X receives the cancel exception, and tries to dereference the counter. This causes Y gets cancelled. the reference count will cause watch prematurely terminate. Recording the versions of each watch would help preventing this issue --- fdbclient/DatabaseContext.cpp | 82 ++++++++++++ fdbclient/NativeAPI.actor.cpp | 121 ++++++------------ fdbclient/include/fdbclient/DatabaseContext.h | 18 ++- flow/include/flow/flow.h | 14 +- 4 files changed, 141 insertions(+), 94 deletions(-) create mode 100644 fdbclient/DatabaseContext.cpp diff --git a/fdbclient/DatabaseContext.cpp b/fdbclient/DatabaseContext.cpp new file mode 100644 index 0000000000..2ba912c9de --- /dev/null +++ b/fdbclient/DatabaseContext.cpp @@ -0,0 +1,82 @@ +/* + * DatabaseContext.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/DatabaseContext.h" + +Reference DatabaseContext::getWatchMetadata(int64_t tenantId, KeyRef key) const { + const auto it = watchMap.find(std::make_pair(tenantId, key)); + if (it == watchMap.end()) + return Reference(); + return it->second; +} + +void DatabaseContext::setWatchMetadata(Reference metadata) { + const WatchMapKey key(metadata->parameters->tenant.tenantId, metadata->parameters->key); + watchMap[key] = metadata; + // NOTE Here we do *NOT* update/reset the reference count for the key, see the source code in getWatchFuture. + // Basically the reference count could be increased, or the same watch is refreshed, or the watch might be cancelled +} + +int32_t DatabaseContext::increaseWatchRefCount(const int64_t tenantID, KeyRef key, const Version& version) { + const WatchMapKey mapKey(tenantID, key); + watchCounterMap[mapKey].insert(version); + return watchCounterMap[mapKey].size(); +} + +int32_t DatabaseContext::decreaseWatchRefCount(const int64_t tenantID, KeyRef key, const Version& version) { + const WatchMapKey mapKey(tenantID, key); + auto mapKeyIter = watchCounterMap.find(mapKey); + if (mapKeyIter == std::end(watchCounterMap)) { + // Key does not exist. The metadata might be removed by deleteWatchMetadata already. + return 0; + } + + auto& versionSet = mapKeyIter->second; + auto versionIter = versionSet.find(version); + + if (versionIter == std::end(versionSet)) { + // Version not found, the watch might be cleared before. + return versionSet.size(); + } + versionSet.erase(versionIter); + + const auto count = versionSet.size(); + // The metadata might be deleted somewhere else, before calling this decreaseWatchRefCount + if (auto metadata = getWatchMetadata(tenantID, key); metadata.isValid() && versionSet.size() == 0) { + // It is a *must* to cancel the watchFutureSS manually. watchFutureSS waits for watchStorageServerResp, which + // holds a reference to the metadata. If the ACTOR is not cancelled, it indirectly holds a Future waiting for + // itself. + metadata->watchFutureSS.cancel(); + deleteWatchMetadata(tenantID, key); + } + + return count; +} + +void DatabaseContext::deleteWatchMetadata(int64_t tenantId, KeyRef key) { + const WatchMapKey mapKey(tenantId, key); + watchMap.erase(mapKey); + watchCounterMap.erase(mapKey); +} + +void DatabaseContext::clearWatchMetadata() { + watchMap.clear(); + watchCounterMap.clear(); +} diff --git a/fdbclient/NativeAPI.actor.cpp b/fdbclient/NativeAPI.actor.cpp index fcef80270a..f550c63afb 100644 --- a/fdbclient/NativeAPI.actor.cpp +++ b/fdbclient/NativeAPI.actor.cpp @@ -2382,54 +2382,6 @@ Database Database::createSimulatedExtraDatabase(std::string connectionString, Op return db; } -Reference DatabaseContext::getWatchMetadata(int64_t tenantId, KeyRef key) const { - const auto it = watchMap.find(std::make_pair(tenantId, key)); - if (it == watchMap.end()) - return Reference(); - return it->second; -} - -void DatabaseContext::setWatchMetadata(Reference metadata) { - const WatchMapKey key(metadata->parameters->tenant.tenantId, metadata->parameters->key); - watchMap[key] = metadata; - // NOTE Here we do *NOT* update/reset the reference count for the key, see the source code in getWatchFuture -} - -int32_t DatabaseContext::increaseWatchRefCount(const int64_t tenantID, KeyRef key) { - const WatchMapKey mapKey(tenantID, key); - if (watchCounterMap.count(mapKey) == 0) { - watchCounterMap[mapKey] = 0; - } - const auto count = ++watchCounterMap[mapKey]; - return count; -} - -int32_t DatabaseContext::decreaseWatchRefCount(const int64_t tenantID, KeyRef key) { - const WatchMapKey mapKey(tenantID, key); - if (watchCounterMap.count(mapKey) == 0) { - // Key does not exist. The metadata might be removed by deleteWatchMetadata already. - return 0; - } - const auto count = --watchCounterMap[mapKey]; - ASSERT(watchCounterMap[mapKey] >= 0); - if (watchCounterMap[mapKey] == 0) { - getWatchMetadata(tenantID, key)->watchFutureSS.cancel(); - deleteWatchMetadata(tenantID, key); - } - return count; -} - -void DatabaseContext::deleteWatchMetadata(int64_t tenantId, KeyRef key) { - const WatchMapKey mapKey(tenantId, key); - watchMap.erase(mapKey); - watchCounterMap.erase(mapKey); -} - -void DatabaseContext::clearWatchMetadata() { - watchMap.clear(); - watchCounterMap.clear(); -} - const UniqueOrderedOptionList& Database::getTransactionDefaults() const { ASSERT(db); return db->transactionDefaults; @@ -3953,41 +3905,35 @@ class WatchRefCountUpdater { Database cx; int64_t tenantID; KeyRef key; - - void tryAddRefCount() { - if (cx.getReference()) { - cx->increaseWatchRefCount(tenantID, key); - } - } - - void tryDelRefCount() { - if (cx.getReference()) { - cx->decreaseWatchRefCount(tenantID, key); - } - } + Version version; public: WatchRefCountUpdater() {} - WatchRefCountUpdater(const Database& cx_, const int64_t tenantID_, KeyRef key_) - : cx(cx_), tenantID(tenantID_), key(key_) { - tryAddRefCount(); - } + WatchRefCountUpdater(const Database& cx_, const int64_t tenantID_, KeyRef key_, const Version& ver) + : cx(cx_), tenantID(tenantID_), key(key_), version(ver) {} WatchRefCountUpdater& operator=(WatchRefCountUpdater&& other) { - tryDelRefCount(); + // Since this class is only used by watchValueMap, and it is used *AFTER* a wait statement, this class is first + // initialized by default constructor, then, after the wait, this function is called to assign the actual + // database, key, etc., to re-initialize this object. At this stage, the reference count can be increased. And + // since the database object is moved, the rvalue will have null reference to the DatabaseContext and will not + // reduce the reference count. + cx = std::move(other.cx); + tenantID = std::move(other.tenantID); + key = std::move(other.key); + version = std::move(other.version); - // NOTE: Do not use move semantic, this copy allows other delete the reference count properly. - cx = other.cx; - tenantID = other.tenantID; - key = other.key; - - tryAddRefCount(); + cx->increaseWatchRefCount(tenantID, key, version); return *this; } - ~WatchRefCountUpdater() { tryDelRefCount(); } + ~WatchRefCountUpdater() { + if (cx.getReference()) { + cx->decreaseWatchRefCount(tenantID, key, version); + } + } }; } // namespace @@ -4003,7 +3949,7 @@ ACTOR Future watchValueMap(Future version, Optional debugID, UseProvisionalProxies useProvisionalProxies) { state Version ver = wait(version); - state WatchRefCountUpdater watchRefCountUpdater(cx, tenant.tenantId, key); + state WatchRefCountUpdater watchRefCountUpdater(cx, tenant.tenantId, key, ver); wait(getWatchFuture(cx, makeReference( @@ -5480,17 +5426,26 @@ ACTOR Future restartWatch(Database cx, tenantInfo.tenantId = locationInfo.tenantEntry.id; } - wait(watchValueMap(cx->minAcceptableReadVersion, - tenantInfo, - key, - value, - cx, - tags, - spanContext, - taskID, - debugID, - useProvisionalProxies)); + if (key == "anotherKey"_sr) + std::cout << cx->dbId.toString() << " restartWatch" << std::endl; + try { + wait(watchValueMap(cx->minAcceptableReadVersion, + tenantInfo, + key, + value, + cx, + tags, + spanContext, + taskID, + debugID, + useProvisionalProxies)); + } catch (Error& err) { + std::cout << cx->dbId.toString() << " restartWatch fail " << err.code() << std::endl; + return Void(); + } + + std::cout << cx->dbId.toString() << " restartWatch pass" << std::endl; return Void(); } diff --git a/fdbclient/include/fdbclient/DatabaseContext.h b/fdbclient/include/fdbclient/DatabaseContext.h index 33c31bf76e..a0d065bbfd 100644 --- a/fdbclient/include/fdbclient/DatabaseContext.h +++ b/fdbclient/include/fdbclient/DatabaseContext.h @@ -157,7 +157,7 @@ public: WatchMetadata(Reference parameters) : watchFuture(watchPromise.getFuture()), parameters(parameters) {} - ~WatchMetadata() { watchFutureSS.cancel(); } + ~WatchMetadata() { /*watchFutureSS.cancel();*/ } }; struct MutationAndVersionStream { @@ -350,11 +350,11 @@ public: void deleteWatchMetadata(int64_t tenant, KeyRef key); // Increases reference count to the given watch. Returns the number of references to the watch. - int32_t increaseWatchRefCount(const int64_t tenant, KeyRef key); + int32_t increaseWatchRefCount(const int64_t tenant, KeyRef key, const Version& version); // Decreases reference count to the given watch. If the reference count is dropped to 0, the watch metadata will be // removed. Returns the number of references to the watch. - int32_t decreaseWatchRefCount(const int64_t tenant, KeyRef key); + int32_t decreaseWatchRefCount(const int64_t tenant, KeyRef key, const Version& version); void clearWatchMetadata(); @@ -730,7 +730,17 @@ private: WatchMap_t watchMap; - using WatchCounterMap_t = std::unordered_map; + // The reason of using a multiset of Versions as counter instead of a simpler integer counter is due to the + // possible race condition: + // + // 1. A watch to key A is set, the watchValueMap ACTOR, noted as X, starts waiting. + // 2. All watches are cleared due to connection string change. + // 3. The watch to key A is restarted with watchValueMap ACTOR Y. + // 4. X receives the cancel exception, and tries to dereference the counter. This causes Y gets cancelled. + // + // By introducing versions, this race condition is solved. + using WatchCounterMapValue = std::multiset; + using WatchCounterMap_t = std::unordered_map; // Maps the number of the WatchMapKey being used. WatchCounterMap_t watchCounterMap; }; diff --git a/flow/include/flow/flow.h b/flow/include/flow/flow.h index cd6704f8b8..bc2fadccea 100644 --- a/flow/include/flow/flow.h +++ b/flow/include/flow/flow.h @@ -788,7 +788,7 @@ public: T const& get() const { return sav->get(); } T getValue() const { return get(); } - bool isValid() const { return sav != 0; } + bool isValid() const { return sav != nullptr; } bool isReady() const { return sav->isSet(); } bool isError() const { return sav->isError(); } // returns true if get can be called on this future (counterpart of canBeSet on Promises) @@ -798,12 +798,12 @@ public: return sav->error_state; } - Future() : sav(0) {} + Future() : sav(nullptr) {} Future(const Future& rhs) : sav(rhs.sav) { if (sav) sav->addFutureRef(); } - Future(Future&& rhs) noexcept : sav(rhs.sav) { rhs.sav = 0; } + Future(Future&& rhs) noexcept : sav(rhs.sav) { rhs.sav = nullptr; } Future(const T& presentValue) : sav(new SAV(1, 0)) { sav->send(presentValue); } Future(T&& presentValue) : sav(new SAV(1, 0)) { sav->send(std::move(presentValue)); } Future(Never) : sav(new SAV(1, 0)) { sav->send(Never()); } @@ -830,7 +830,7 @@ public: if (sav) sav->delFutureRef(); sav = rhs.sav; - rhs.sav = 0; + rhs.sav = nullptr; } } bool operator==(const Future& rhs) { return rhs.sav == sav; } @@ -843,17 +843,17 @@ public: void addCallbackAndClear(Callback* cb) { sav->addCallbackAndDelFutureRef(cb); - sav = 0; + sav = nullptr; } void addYieldedCallbackAndClear(Callback* cb) { sav->addYieldedCallbackAndDelFutureRef(cb); - sav = 0; + sav = nullptr; } void addCallbackChainAndClear(Callback* cb) { sav->addCallbackChainAndDelFutureRef(cb); - sav = 0; + sav = nullptr; } int getFutureReferenceCount() const { return sav->getFutureReferenceCount(); } From ec47c261bf743e4ffefbea2e70641afdf8f16491 Mon Sep 17 00:00:00 2001 From: Xiaoge Su Date: Thu, 20 Oct 2022 01:38:56 -0700 Subject: [PATCH 64/95] Reformat source --- fdbclient/include/fdbclient/DatabaseContext.h | 2 -- 1 file changed, 2 deletions(-) diff --git a/fdbclient/include/fdbclient/DatabaseContext.h b/fdbclient/include/fdbclient/DatabaseContext.h index a0d065bbfd..827c5da291 100644 --- a/fdbclient/include/fdbclient/DatabaseContext.h +++ b/fdbclient/include/fdbclient/DatabaseContext.h @@ -156,8 +156,6 @@ public: WatchMetadata(Reference parameters) : watchFuture(watchPromise.getFuture()), parameters(parameters) {} - - ~WatchMetadata() { /*watchFutureSS.cancel();*/ } }; struct MutationAndVersionStream { From 41d1d6404d933f0574d88d1fa2a68c642413bf4b Mon Sep 17 00:00:00 2001 From: Xiaoge Su Date: Thu, 20 Oct 2022 11:46:08 -0700 Subject: [PATCH 65/95] Remove debugging output --- fdbclient/NativeAPI.actor.cpp | 29 ++++++++++------------------- 1 file changed, 10 insertions(+), 19 deletions(-) diff --git a/fdbclient/NativeAPI.actor.cpp b/fdbclient/NativeAPI.actor.cpp index f550c63afb..87fbb10a2c 100644 --- a/fdbclient/NativeAPI.actor.cpp +++ b/fdbclient/NativeAPI.actor.cpp @@ -5426,26 +5426,17 @@ ACTOR Future restartWatch(Database cx, tenantInfo.tenantId = locationInfo.tenantEntry.id; } - if (key == "anotherKey"_sr) - std::cout << cx->dbId.toString() << " restartWatch" << std::endl; + wait(watchValueMap(cx->minAcceptableReadVersion, + tenantInfo, + key, + value, + cx, + tags, + spanContext, + taskID, + debugID, + useProvisionalProxies)); - try { - wait(watchValueMap(cx->minAcceptableReadVersion, - tenantInfo, - key, - value, - cx, - tags, - spanContext, - taskID, - debugID, - useProvisionalProxies)); - } catch (Error& err) { - std::cout << cx->dbId.toString() << " restartWatch fail " << err.code() << std::endl; - return Void(); - } - - std::cout << cx->dbId.toString() << " restartWatch pass" << std::endl; return Void(); } From 812243bafab4b8cb9cad49c7c22f16063f39b37e Mon Sep 17 00:00:00 2001 From: Xiaoge Su Date: Mon, 24 Oct 2022 09:43:54 -0700 Subject: [PATCH 66/95] Update fdbclient/NativeAPI.actor.cpp Co-authored-by: Jingyu Zhou --- fdbclient/NativeAPI.actor.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/fdbclient/NativeAPI.actor.cpp b/fdbclient/NativeAPI.actor.cpp index 87fbb10a2c..ffe8c4b89e 100644 --- a/fdbclient/NativeAPI.actor.cpp +++ b/fdbclient/NativeAPI.actor.cpp @@ -3908,7 +3908,7 @@ class WatchRefCountUpdater { Version version; public: - WatchRefCountUpdater() {} + WatchRefCountUpdater() = default; WatchRefCountUpdater(const Database& cx_, const int64_t tenantID_, KeyRef key_, const Version& ver) : cx(cx_), tenantID(tenantID_), key(key_), version(ver) {} From a72066be334365bc9cf96606272fa45af34f4083 Mon Sep 17 00:00:00 2001 From: Lukas Joswiak Date: Thu, 13 Oct 2022 14:58:53 -0700 Subject: [PATCH 67/95] Add simulation support for changing the cluster file --- fdbrpc/include/fdbrpc/simulator.h | 6 ++ fdbrpc/sim2.actor.cpp | 19 ++++++- fdbserver/SimulatedCluster.actor.cpp | 55 ++++++++++++++----- .../workloads/MachineAttrition.actor.cpp | 5 ++ 4 files changed, 69 insertions(+), 16 deletions(-) diff --git a/fdbrpc/include/fdbrpc/simulator.h b/fdbrpc/include/fdbrpc/simulator.h index 3eeb405785..c15181c90f 100644 --- a/fdbrpc/include/fdbrpc/simulator.h +++ b/fdbrpc/include/fdbrpc/simulator.h @@ -54,6 +54,7 @@ public: FailDisk, RebootAndDelete, RebootProcessAndDelete, + RebootProcessAndSwitch, Reboot, RebootProcess, None @@ -304,6 +305,7 @@ public: KillType kt, bool forceKill = false, KillType* ktFinal = nullptr) = 0; + virtual bool killAll(KillType kt, bool forceKill = false, KillType* ktFinal = nullptr) = 0; // virtual KillType getMachineKillState( UID zoneID ) = 0; virtual bool canKillProcesses(std::vector const& availableProcesses, std::vector const& deadProcesses, @@ -390,6 +392,9 @@ public: return clearedAddresses.find(address) != clearedAddresses.end(); } + void switchCluster(NetworkAddress const& address) { switchedCluster[address] = !switchedCluster[address]; } + bool hasSwitchedCluster(NetworkAddress const& address) const { return switchedCluster.at(address); } + void excludeAddress(NetworkAddress const& address) { excludedAddresses[address]++; TraceEvent("ExcludeAddress").detail("Address", address).detail("Value", excludedAddresses[address]); @@ -540,6 +545,7 @@ private: std::set>> swapsDisabled; std::map excludedAddresses; std::map clearedAddresses; + std::map switchedCluster; std::map> roleAddresses; std::map disabledMap; bool allSwapsDisabled; diff --git a/fdbrpc/sim2.actor.cpp b/fdbrpc/sim2.actor.cpp index a73674b10e..59727e3161 100644 --- a/fdbrpc/sim2.actor.cpp +++ b/fdbrpc/sim2.actor.cpp @@ -1794,6 +1794,15 @@ public: } return result; } + bool killAll(KillType kt, bool forceKill, KillType* ktFinal) override { + bool result = false; + for (auto& machine : machines) { + if (killMachine(machine.second.machineId, kt, forceKill, ktFinal)) { + result = true; + } + } + return result; + } bool killMachine(Optional> machineId, KillType kt, bool forceKill, @@ -2008,7 +2017,7 @@ public: if (process->startingClass != ProcessClass::TesterClass) killProcess_internal(process, kt); } - } else if (kt == Reboot || kt == RebootAndDelete) { + } else if (kt == Reboot || kt == RebootAndDelete || kt == RebootProcessAndSwitch) { for (auto& process : machines[machineId].processes) { TraceEvent("KillMachineProcess") .detail("KillType", kt) @@ -2564,7 +2573,7 @@ ACTOR void doReboot(ISimulator::ProcessInfo* p, ISimulator::KillType kt) { try { ASSERT(kt == ISimulator::RebootProcess || kt == ISimulator::Reboot || kt == ISimulator::RebootAndDelete || - kt == ISimulator::RebootProcessAndDelete); + kt == ISimulator::RebootProcessAndDelete || kt == ISimulator::RebootProcessAndSwitch); CODE_PROBE(kt == ISimulator::RebootProcess, "Simulated process rebooted", @@ -2580,6 +2589,10 @@ ACTOR void doReboot(ISimulator::ProcessInfo* p, ISimulator::KillType kt) { "Simulated process rebooted with data and coordination state deletion", probe::assert::simOnly, probe::context::sim2); + CODE_PROBE(kt == ISimulator::RebootProcessAndSwitch, + "Simulated process rebooted with different cluster file", + probe::assert::simOnly, + probe::context::sim2); if (p->rebooting || !p->isReliable()) { TraceEvent(SevDebug, "DoRebootFailed") @@ -2608,6 +2621,8 @@ ACTOR void doReboot(ISimulator::ProcessInfo* p, ISimulator::KillType kt) { if ((kt == ISimulator::RebootAndDelete) || (kt == ISimulator::RebootProcessAndDelete)) { p->cleared = true; g_simulator->clearAddress(p->address); + } else if (kt == ISimulator::RebootProcessAndSwitch) { + g_simulator->switchCluster(p->address); } p->shutdownSignal.send(kt); } catch (Error& e) { diff --git a/fdbserver/SimulatedCluster.actor.cpp b/fdbserver/SimulatedCluster.actor.cpp index 783478b7b6..c64bf610df 100644 --- a/fdbserver/SimulatedCluster.actor.cpp +++ b/fdbserver/SimulatedCluster.actor.cpp @@ -620,6 +620,7 @@ ACTOR Future simulatedFDBDRebooter(Reference simulatedFDBDRebooter(Reference(joinPath(*dataFolder, "fdb.cluster"), connStr.toString()); } + } else if (onShutdown.get() == ISimulator::RebootProcessAndSwitch) { + TraceEvent("SimulatedFDBDRebootAndSwitch") + .detail("Cycles", cycles) + .detail("RandomId", randomId) + .detail("Address", process->address) + .detail("ZoneId", localities.zoneId()) + .detail("KillType", shutdownResult) + .detail("ConnectionString", connStr.toString()) + .detail("OtherConnectionString", otherConnStr.toString()) + .detail("SwitchingTo", g_simulator->hasSwitchedCluster(process->address)); + + // Handle the case where otherConnStr is '@'. + if (otherConnStr.toString().size() > 1) { + std::string newConnStr = + g_simulator->hasSwitchedCluster(process->address) ? otherConnStr.toString() : connStr.toString(); + connRecord = makeReference(joinPath(*dataFolder, "fdb.cluster"), newConnStr); + } } else { TraceEvent("SimulatedFDBDJustRepeat") .detail("Cycles", cycles) @@ -846,6 +864,7 @@ std::map>, std::vector>> // process count is no longer needed because it is now the length of the vector of ip's, because it was one ip per // process ACTOR Future simulatedMachine(ClusterConnectionString connStr, + ClusterConnectionString otherConnStr, std::vector ips, bool sslEnabled, LocalityData localities, @@ -924,6 +943,7 @@ ACTOR Future simulatedMachine(ClusterConnectionString connStr, &coordFolders[i], baseFolder, connStr, + otherConnStr, useSeedFile, agentMode, whitelistBinPaths, @@ -942,6 +962,7 @@ ACTOR Future simulatedMachine(ClusterConnectionString connStr, &coordFolders[i], baseFolder, connStr, + otherConnStr, useSeedFile, agentMode, whitelistBinPaths, @@ -1311,6 +1332,7 @@ ACTOR Future restartSimulatedSystem(std::vector>* systemActor // SOMEDAY: parse backup agent from test file systemActors->push_back(reportErrors( simulatedMachine(conn, + ClusterConnectionString(), ipAddrs, usingSSL, localities, @@ -2346,20 +2368,23 @@ void setupSimulatedSystem(std::vector>* systemActors, // check the sslEnablementMap using only one ip LocalityData localities(Optional>(), zoneId, machineId, dcUID); localities.set("data_hall"_sr, dcUID); - systemActors->push_back(reportErrors(simulatedMachine(conn, - ips, - sslEnabled, - localities, - processClass, - baseFolder, - false, - machine == useSeedForMachine, - requiresExtraDBMachines ? AgentOnly : AgentAddition, - sslOnly, - whitelistBinPaths, - protocolVersion, - configDBType), - "SimulatedMachine")); + systemActors->push_back(reportErrors( + simulatedMachine(conn, + requiresExtraDBMachines ? ClusterConnectionString(g_simulator->extraDatabases.at(0)) + : ClusterConnectionString(), + ips, + sslEnabled, + localities, + processClass, + baseFolder, + false, + machine == useSeedForMachine, + requiresExtraDBMachines ? AgentOnly : AgentAddition, + sslOnly, + whitelistBinPaths, + protocolVersion, + configDBType), + "SimulatedMachine")); if (requiresExtraDBMachines) { int cluster = 4; @@ -2376,6 +2401,7 @@ void setupSimulatedSystem(std::vector>* systemActors, LocalityData localities(Optional>(), newZoneId, newMachineId, dcUID); localities.set("data_hall"_sr, dcUID); systemActors->push_back(reportErrors(simulatedMachine(ClusterConnectionString(extraDatabase), + conn, extraIps, sslEnabled, localities, @@ -2422,6 +2448,7 @@ void setupSimulatedSystem(std::vector>* systemActors, Optional>(), newZoneId, newZoneId, Optional>()); systemActors->push_back( reportErrors(simulatedMachine(conn, + ClusterConnectionString(), ips, sslEnabled, localities, diff --git a/fdbserver/workloads/MachineAttrition.actor.cpp b/fdbserver/workloads/MachineAttrition.actor.cpp index 9d1dff8348..e46619cd55 100644 --- a/fdbserver/workloads/MachineAttrition.actor.cpp +++ b/fdbserver/workloads/MachineAttrition.actor.cpp @@ -357,6 +357,11 @@ struct MachineAttritionWorkload : FailureInjectionWorkload { TraceEvent("Assassination").detail("TargetDataHall", target).detail("KillType", kt); g_simulator->killDataHall(target, kt); + } else if (!g_simulator->extraDatabases.empty() && deterministicRandom()->random01() < 0.1) { + state ISimulator::KillType kt = ISimulator::RebootProcessAndSwitch; + g_simulator->killAll(kt, true); + wait(delay(self->testDuration / 2)); + g_simulator->killAll(kt, true); } else { state int killedMachines = 0; while (killedMachines < self->machinesToKill && self->machines.size() > self->machinesToLeave) { From 72a97afcd64b938882ff6208d22a33bb59d14e89 Mon Sep 17 00:00:00 2001 From: Lukas Joswiak Date: Thu, 13 Oct 2022 17:30:40 -0700 Subject: [PATCH 68/95] Avoid recruiting workers with different cluster ID --- fdbserver/ClusterController.actor.cpp | 51 +++++--- fdbserver/TLogServer.actor.cpp | 7 +- .../fdbserver/ClusterController.actor.h | 16 +-- .../include/fdbserver/WorkerInterface.actor.h | 12 +- fdbserver/worker.actor.cpp | 109 +++++++++++++++--- 5 files changed, 154 insertions(+), 41 deletions(-) diff --git a/fdbserver/ClusterController.actor.cpp b/fdbserver/ClusterController.actor.cpp index 5513fbd6bb..79d033262c 100644 --- a/fdbserver/ClusterController.actor.cpp +++ b/fdbserver/ClusterController.actor.cpp @@ -1228,6 +1228,15 @@ ACTOR Future registerWorker(RegisterWorkerRequest req, std::vector coordinatorAddresses = wait(cs.tryResolveHostnames()); const WorkerInterface& w = req.wi; + if (req.clusterId.present() && self->clusterId->get().present() && req.clusterId != self->clusterId->get()) { + TraceEvent(g_network->isSimulated() ? SevWarnAlways : SevError, "WorkerBelongsToExistingCluster", self->id) + .detail("WorkerClusterId", req.clusterId) + .detail("ClusterControllerClusterId", self->clusterId->get()) + .detail("WorkerId", w.id()) + .detail("ProcessId", w.locality.processId()); + return Void(); + } + ProcessClass newProcessClass = req.processClass; auto info = self->id_worker.find(w.locality.processId()); ClusterControllerPriorityInfo newPriorityInfo = req.priorityInfo; @@ -2969,8 +2978,9 @@ ACTOR Future clusterControllerCore(ClusterControllerFullInterface interf, ServerCoordinators coordinators, LocalityData locality, ConfigDBType configDBType, - Future recoveredDiskFiles) { - state ClusterControllerData self(interf, locality, coordinators); + Future recoveredDiskFiles, + Reference>> clusterId) { + state ClusterControllerData self(interf, locality, coordinators, clusterId); state Future coordinationPingDelay = delay(SERVER_KNOBS->WORKER_COORDINATION_PING_DELAY); state uint64_t step = 0; state Future> error = errorOr(actorCollection(self.addActor.getFuture())); @@ -3123,7 +3133,8 @@ ACTOR Future clusterController(ServerCoordinators coordinators, Reference> asyncPriorityInfo, LocalityData locality, ConfigDBType configDBType, - Future recoveredDiskFiles) { + Future recoveredDiskFiles, + Reference>> clusterId) { loop { state ClusterControllerFullInterface cci; state bool inRole = false; @@ -3150,7 +3161,8 @@ ACTOR Future clusterController(ServerCoordinators coordinators, startRole(Role::CLUSTER_CONTROLLER, cci.id(), UID()); inRole = true; - wait(clusterControllerCore(cci, leaderFail, coordinators, locality, configDBType, recoveredDiskFiles)); + wait(clusterControllerCore( + cci, leaderFail, coordinators, locality, configDBType, recoveredDiskFiles, clusterId)); } } catch (Error& e) { if (inRole) @@ -3174,7 +3186,8 @@ ACTOR Future clusterController(Reference connRec Reference> asyncPriorityInfo, Future recoveredDiskFiles, LocalityData locality, - ConfigDBType configDBType) { + ConfigDBType configDBType, + Reference>> clusterId) { // Defer this wait optimization of cluster configuration has 'Encryption data at-rest' enabled. // Encryption depends on available of EncryptKeyProxy (EKP) FDB role to enable fetch/refresh of encryption keys @@ -3194,8 +3207,14 @@ ACTOR Future clusterController(Reference connRec loop { try { ServerCoordinators coordinators(connRecord, configDBType); - wait(clusterController( - coordinators, currentCC, hasConnected, asyncPriorityInfo, locality, configDBType, recoveredDiskFiles)); + wait(clusterController(coordinators, + currentCC, + hasConnected, + asyncPriorityInfo, + locality, + configDBType, + recoveredDiskFiles, + clusterId)); hasConnected = true; } catch (Error& e) { if (e.code() != error_code_coordinators_changed) @@ -3213,7 +3232,8 @@ TEST_CASE("/fdbserver/clustercontroller/updateWorkerHealth") { state ClusterControllerData data(ClusterControllerFullInterface(), LocalityData(), ServerCoordinators(Reference( - new ClusterConnectionMemoryRecord(ClusterConnectionString())))); + new ClusterConnectionMemoryRecord(ClusterConnectionString()))), + makeReference>>()); state NetworkAddress workerAddress(IPAddress(0x01010101), 1); state NetworkAddress badPeer1(IPAddress(0x02020202), 1); state NetworkAddress badPeer2(IPAddress(0x03030303), 1); @@ -3308,7 +3328,8 @@ TEST_CASE("/fdbserver/clustercontroller/updateRecoveredWorkers") { ClusterControllerData data(ClusterControllerFullInterface(), LocalityData(), ServerCoordinators(Reference( - new ClusterConnectionMemoryRecord(ClusterConnectionString())))); + new ClusterConnectionMemoryRecord(ClusterConnectionString()))), + makeReference>>()); NetworkAddress worker1(IPAddress(0x01010101), 1); NetworkAddress worker2(IPAddress(0x11111111), 1); NetworkAddress badPeer1(IPAddress(0x02020202), 1); @@ -3357,7 +3378,8 @@ TEST_CASE("/fdbserver/clustercontroller/getDegradationInfo") { ClusterControllerData data(ClusterControllerFullInterface(), LocalityData(), ServerCoordinators(Reference( - new ClusterConnectionMemoryRecord(ClusterConnectionString())))); + new ClusterConnectionMemoryRecord(ClusterConnectionString()))), + makeReference>>()); NetworkAddress worker(IPAddress(0x01010101), 1); NetworkAddress badPeer1(IPAddress(0x02020202), 1); NetworkAddress badPeer2(IPAddress(0x03030303), 1); @@ -3510,7 +3532,8 @@ TEST_CASE("/fdbserver/clustercontroller/recentRecoveryCountDueToHealth") { ClusterControllerData data(ClusterControllerFullInterface(), LocalityData(), ServerCoordinators(Reference( - new ClusterConnectionMemoryRecord(ClusterConnectionString())))); + new ClusterConnectionMemoryRecord(ClusterConnectionString()))), + makeReference>>()); ASSERT_EQ(data.recentRecoveryCountDueToHealth(), 0); @@ -3531,7 +3554,8 @@ TEST_CASE("/fdbserver/clustercontroller/shouldTriggerRecoveryDueToDegradedServer ClusterControllerData data(ClusterControllerFullInterface(), LocalityData(), ServerCoordinators(Reference( - new ClusterConnectionMemoryRecord(ClusterConnectionString())))); + new ClusterConnectionMemoryRecord(ClusterConnectionString()))), + makeReference>>()); NetworkAddress master(IPAddress(0x01010101), 1); NetworkAddress tlog(IPAddress(0x02020202), 1); NetworkAddress satelliteTlog(IPAddress(0x03030303), 1); @@ -3667,7 +3691,8 @@ TEST_CASE("/fdbserver/clustercontroller/shouldTriggerFailoverDueToDegradedServer ClusterControllerData data(ClusterControllerFullInterface(), LocalityData(), ServerCoordinators(Reference( - new ClusterConnectionMemoryRecord(ClusterConnectionString())))); + new ClusterConnectionMemoryRecord(ClusterConnectionString()))), + makeReference>>()); NetworkAddress master(IPAddress(0x01010101), 1); NetworkAddress tlog(IPAddress(0x02020202), 1); NetworkAddress satelliteTlog(IPAddress(0x03030303), 1); diff --git a/fdbserver/TLogServer.actor.cpp b/fdbserver/TLogServer.actor.cpp index 1334c33eef..21c6601232 100644 --- a/fdbserver/TLogServer.actor.cpp +++ b/fdbserver/TLogServer.actor.cpp @@ -2619,6 +2619,7 @@ ACTOR Future tLogEnablePopReq(TLogEnablePopRequest enablePopReq, TLogData* return Void(); } +// TODO: Remove all cluster ID logic from tlog and storage server ACTOR Future updateDurableClusterID(TLogData* self) { loop { // Persist cluster ID once cluster has recovered. @@ -3607,9 +3608,9 @@ ACTOR Future tLog(IKeyValueStore* persistentData, if (recovered.canBeSet()) recovered.send(Void()); - if (!self.durableClusterId.isValid()) { - self.sharedActors.send(updateDurableClusterID(&self)); - } + // if (!self.durableClusterId.isValid()) { + // self.sharedActors.send(updateDurableClusterID(&self)); + // } self.sharedActors.send(commitQueue(&self)); self.sharedActors.send(updateStorageLoop(&self)); self.sharedActors.send(traceRole(Role::SHARED_TRANSACTION_LOG, tlogId)); diff --git a/fdbserver/include/fdbserver/ClusterController.actor.h b/fdbserver/include/fdbserver/ClusterController.actor.h index 944fd25409..f390b4c1ce 100644 --- a/fdbserver/include/fdbserver/ClusterController.actor.h +++ b/fdbserver/include/fdbserver/ClusterController.actor.h @@ -3341,6 +3341,7 @@ public: AsyncVar>>>> changedDcIds; // current DC priorities to change second, and whether the cluster controller has been changed UID id; + Reference>> clusterId; std::vector> outstandingRecruitmentRequests; std::vector> outstandingRemoteRecruitmentRequests; std::vector> outstandingStorageRequests; @@ -3412,15 +3413,16 @@ public: ClusterControllerData(ClusterControllerFullInterface const& ccInterface, LocalityData const& locality, - ServerCoordinators const& coordinators) + ServerCoordinators const& coordinators, + Reference>> clusterId) : gotProcessClasses(false), gotFullyRecoveredConfig(false), shouldCommitSuicide(false), clusterControllerProcessId(locality.processId()), clusterControllerDcId(locality.dcId()), id(ccInterface.id()), - ac(false), outstandingRequestChecker(Void()), outstandingRemoteRequestChecker(Void()), startTime(now()), - goodRecruitmentTime(Never()), goodRemoteRecruitmentTime(Never()), datacenterVersionDifference(0), - versionDifferenceUpdated(false), remoteDCMonitorStarted(false), remoteTransactionSystemDegraded(false), - recruitDistributor(false), recruitRatekeeper(false), recruitBlobManager(false), recruitBlobMigrator(false), - recruitEncryptKeyProxy(false), recruitConsistencyScan(false), - clusterControllerMetrics("ClusterController", id.toString()), + clusterId(clusterId), ac(false), outstandingRequestChecker(Void()), outstandingRemoteRequestChecker(Void()), + startTime(now()), goodRecruitmentTime(Never()), goodRemoteRecruitmentTime(Never()), + datacenterVersionDifference(0), versionDifferenceUpdated(false), remoteDCMonitorStarted(false), + remoteTransactionSystemDegraded(false), recruitDistributor(false), recruitRatekeeper(false), + recruitBlobManager(false), recruitBlobMigrator(false), recruitEncryptKeyProxy(false), + recruitConsistencyScan(false), clusterControllerMetrics("ClusterController", id.toString()), openDatabaseRequests("OpenDatabaseRequests", clusterControllerMetrics), registerWorkerRequests("RegisterWorkerRequests", clusterControllerMetrics), getWorkersRequests("GetWorkersRequests", clusterControllerMetrics), diff --git a/fdbserver/include/fdbserver/WorkerInterface.actor.h b/fdbserver/include/fdbserver/WorkerInterface.actor.h index 74c90cec32..adcff3b0ed 100644 --- a/fdbserver/include/fdbserver/WorkerInterface.actor.h +++ b/fdbserver/include/fdbserver/WorkerInterface.actor.h @@ -445,6 +445,7 @@ struct RegisterWorkerRequest { bool requestDbInfo; bool recoveredDiskFiles; ConfigBroadcastInterface configBroadcastInterface; + Optional clusterId; RegisterWorkerRequest() : priorityInfo(ProcessClass::UnsetFit, false, ClusterControllerPriorityInfo::FitnessUnknown), degraded(false) {} @@ -463,13 +464,14 @@ struct RegisterWorkerRequest { Optional lastSeenKnobVersion, Optional knobConfigClassSet, bool recoveredDiskFiles, - ConfigBroadcastInterface configBroadcastInterface) + ConfigBroadcastInterface configBroadcastInterface, + Optional clusterId) : wi(wi), initialClass(initialClass), processClass(processClass), priorityInfo(priorityInfo), generation(generation), distributorInterf(ddInterf), ratekeeperInterf(rkInterf), blobManagerInterf(bmInterf), blobMigratorInterf(mgInterf), encryptKeyProxyInterf(ekpInterf), consistencyScanInterf(csInterf), degraded(degraded), lastSeenKnobVersion(lastSeenKnobVersion), knobConfigClassSet(knobConfigClassSet), requestDbInfo(false), recoveredDiskFiles(recoveredDiskFiles), - configBroadcastInterface(configBroadcastInterface) {} + configBroadcastInterface(configBroadcastInterface), clusterId(clusterId) {} template void serialize(Ar& ar) { @@ -493,7 +495,8 @@ struct RegisterWorkerRequest { knobConfigClassSet, requestDbInfo, recoveredDiskFiles, - configBroadcastInterface); + configBroadcastInterface, + clusterId); } }; @@ -1129,7 +1132,8 @@ ACTOR Future clusterController(Reference ccr, Reference> asyncPriorityInfo, Future recoveredDiskFiles, LocalityData locality, - ConfigDBType configDBType); + ConfigDBType configDBType, + Reference>> clusterId); ACTOR Future blobWorker(BlobWorkerInterface bwi, ReplyPromise blobWorkerReady, diff --git a/fdbserver/worker.actor.cpp b/fdbserver/worker.actor.cpp index a6019b6cec..e92fa2b9a5 100644 --- a/fdbserver/worker.actor.cpp +++ b/fdbserver/worker.actor.cpp @@ -572,7 +572,8 @@ ACTOR Future registrationClient( Reference localConfig, ConfigBroadcastInterface configBroadcastInterface, Reference> dbInfo, - Promise recoveredDiskFiles) { + Promise recoveredDiskFiles, + Reference>> clusterId) { // Keeps the cluster controller (as it may be re-elected) informed that this worker exists // The cluster controller uses waitFailureClient to find out if we die, and returns from registrationReply // (requiring us to re-register) The registration request piggybacks optional distributor interface if it exists. @@ -611,7 +612,8 @@ ACTOR Future registrationClient( localConfig.isValid() ? localConfig->lastSeenVersion() : Optional(), localConfig.isValid() ? localConfig->configClassSet() : Optional(), recoveredDiskFiles.isSet(), - configBroadcastInterface); + configBroadcastInterface, + clusterId->get()); for (auto const& i : issues->get()) { request.issues.push_back_deep(request.issues.arena(), i); @@ -651,7 +653,8 @@ ACTOR Future registrationClient( TraceEvent("WorkerRegister") .detail("CCID", ccInterface->get().get().id()) .detail("Generation", requestGeneration) - .detail("RecoveredDiskFiles", recoveredDiskFiles.isSet()); + .detail("RecoveredDiskFiles", recoveredDiskFiles.isSet()) + .detail("ClusterId", clusterId->get()); } state Future registrationReply = ccInterfacePresent ? brokenPromiseToNever(ccInterface->get().get().registerWorker.getReply(request)) @@ -1636,6 +1639,50 @@ ACTOR Future resetBlobManagerWhenDoneOrError( return Void(); } +static const std::string clusterIdFilename = "clusterId"; + +ACTOR Future createClusterIdFile(std::string folder, UID clusterId) { + state std::string clusterIdPath = joinPath(folder, clusterIdFilename); + if (fileExists(clusterIdPath)) { + return Void(); + } + loop { + try { + state ErrorOr> clusterIdFile = + wait(errorOr(IAsyncFileSystem::filesystem(g_network)->open( + clusterIdPath, IAsyncFile::OPEN_READWRITE | IAsyncFile::OPEN_LOCK, 0600))); + + if (clusterIdFile.isError() && clusterIdFile.getError().code() == error_code_file_not_found && + !fileExists(clusterIdPath)) { + Reference _clusterIdFile = wait(IAsyncFileSystem::filesystem()->open( + clusterIdPath, + IAsyncFile::OPEN_ATOMIC_WRITE_AND_CREATE | IAsyncFile::OPEN_CREATE | IAsyncFile::OPEN_LOCK | + IAsyncFile::OPEN_READWRITE, + 0600)); + clusterIdFile = _clusterIdFile; + BinaryWriter wr(IncludeVersion()); + wr << clusterId; + wait(clusterIdFile.get()->write(wr.getData(), wr.getLength(), 0)); + wait(clusterIdFile.get()->sync()); + return Void(); + } else { + throw clusterIdFile.getError(); + } + } catch (Error& e) { + if (e.code() == error_code_actor_cancelled) { + throw; + } + if (!e.isInjectedFault()) { + fprintf(stderr, + "ERROR: error creating or opening cluster id file `%s'.\n", + joinPath(folder, clusterIdFilename).c_str()); + } + TraceEvent(SevError, "OpenClusterIdError").error(e); + throw; + } + } +} + ACTOR Future workerServer(Reference connRecord, Reference> const> ccInterface, LocalityData locality, @@ -1652,7 +1699,8 @@ ACTOR Future workerServer(Reference connRecord, Reference> dbInfo, ConfigBroadcastInterface configBroadcastInterface, Reference configNode, - Reference localConfig) { + Reference localConfig, + Reference>> clusterId) { state PromiseStream errors; state Reference>> ddInterf( new AsyncVar>()); @@ -1997,7 +2045,8 @@ ACTOR Future workerServer(Reference connRecord, localConfig, configBroadcastInterface, dbInfo, - recoveredDiskFiles)); + recoveredDiskFiles, + clusterId)); if (configNode.isValid()) { errorForwarders.add(brokenPromiseToNever(localConfig->consume(configBroadcastInterface))); @@ -2008,7 +2057,7 @@ ACTOR Future workerServer(Reference connRecord, } loop choose { - when(UpdateServerDBInfoRequest req = waitNext(interf.updateServerDBInfo.getFuture())) { + when(state UpdateServerDBInfoRequest req = waitNext(interf.updateServerDBInfo.getFuture())) { ServerDBInfo localInfo = BinaryReader::fromStringRef( req.serializedDbInfo, AssumeVersion(g_network->protocolVersion())); localInfo.myLocality = locality; @@ -2044,6 +2093,18 @@ ACTOR Future workerServer(Reference connRecord, } errorForwarders.add( success(broadcastDBInfoRequest(req, SERVER_KNOBS->DBINFO_SEND_AMOUNT, notUpdated, true))); + + if (!clusterId->get().present() && localInfo.recoveryState >= RecoveryState::ACCEPTING_COMMITS && + localInfo.client.clusterId.isValid()) { + // Persist the cluster ID as a file in the data + // directory once recovery has made the transaction + // state store durable. The txnStateStore also stores + // the cluster ID. + // TODO: Does the txnStateStore need to store the cluster ID? + state UID tmpClusterId = localInfo.client.clusterId; + wait(createClusterIdFile(folder, tmpClusterId)); + clusterId->set(tmpClusterId); + } } } when(RebootRequest req = waitNext(interf.clientInterface.reboot.getFuture())) { @@ -3378,7 +3439,8 @@ ACTOR Future monitorLeaderWithDelayedCandidacy( Future recoveredDiskFiles, LocalityData locality, Reference> dbInfo, - ConfigDBType configDBType) { + ConfigDBType configDBType, + Reference>> clusterId) { state Future monitor = monitorLeaderWithDelayedCandidacyImpl(connRecord, currentCC); state Future timeout; @@ -3405,7 +3467,7 @@ ACTOR Future monitorLeaderWithDelayedCandidacy( when(wait(timeout.isValid() ? timeout : Never())) { monitor.cancel(); wait(clusterController( - connRecord, currentCC, asyncPriorityInfo, recoveredDiskFiles, locality, configDBType)); + connRecord, currentCC, asyncPriorityInfo, recoveredDiskFiles, locality, configDBType, clusterId)); return Void(); } } @@ -3455,6 +3517,17 @@ ACTOR Future serveProcess() { } } +Optional readClusterId(std::string filePath) { + if (!fileExists(filePath)) { + return Optional(); + } + std::string contents(readFileBytes(filePath, 10000)); + BinaryReader br(StringRef(contents), IncludeVersion()); + UID clusterId; + br >> clusterId; + return clusterId; +} + ACTOR Future fdbd(Reference connRecord, LocalityData localities, ProcessClass processClass, @@ -3529,6 +3602,8 @@ ACTOR Future fdbd(Reference connRecord, serverDBInfo.client.isEncryptionEnabled = SERVER_KNOBS->ENABLE_ENCRYPTION; serverDBInfo.myLocality = localities; auto dbInfo = makeReference>(serverDBInfo); + Reference>> clusterId( + new AsyncVar>(readClusterId(joinPath(dataFolder, clusterIdFilename)))); TraceEvent("MyLocality").detail("Locality", dbInfo->get().myLocality.toString()); actors.push_back(reportErrors(monitorAndWriteCCPriorityInfo(fitnessFilePath, asyncPriorityInfo), @@ -3543,13 +3618,18 @@ ACTOR Future fdbd(Reference connRecord, recoveredDiskFiles.getFuture(), localities, dbInfo, - configDBType), + configDBType, + clusterId), "ClusterController")); } else { - actors.push_back(reportErrors( - clusterController( - connRecord, cc, asyncPriorityInfo, recoveredDiskFiles.getFuture(), localities, configDBType), - "ClusterController")); + actors.push_back(reportErrors(clusterController(connRecord, + cc, + asyncPriorityInfo, + recoveredDiskFiles.getFuture(), + localities, + configDBType, + clusterId), + "ClusterController")); } actors.push_back(reportErrors(extractClusterInterface(cc, ci), "ExtractClusterInterface")); actors.push_back(reportErrorsExcept(workerServer(connRecord, @@ -3568,7 +3648,8 @@ ACTOR Future fdbd(Reference connRecord, dbInfo, configBroadcastInterface, configNode, - localConfig), + localConfig, + clusterId), "WorkerServer", UID(), &normalWorkerErrors())); From f43011e4b72f8f7bbd55df016808f60add12191b Mon Sep 17 00:00:00 2001 From: Lukas Joswiak Date: Mon, 17 Oct 2022 11:56:19 -0700 Subject: [PATCH 69/95] Notify processes joining the wrong cluster And have these processes enter a "zombie" state where they cancel all their actors and then wait forever, refusing to do any additional work until they are manually handled by the operator. --- fdbrpc/sim2.actor.cpp | 29 +++++++++------------------ fdbserver/ClusterController.actor.cpp | 5 ++++- fdbserver/worker.actor.cpp | 15 ++++++++++++-- 3 files changed, 26 insertions(+), 23 deletions(-) diff --git a/fdbrpc/sim2.actor.cpp b/fdbrpc/sim2.actor.cpp index 59727e3161..f286f56ab8 100644 --- a/fdbrpc/sim2.actor.cpp +++ b/fdbrpc/sim2.actor.cpp @@ -1959,8 +1959,15 @@ public: probe::context::sim2, probe::assert::simOnly); - // Check if any processes on machine are rebooting - if (processesOnMachine != processesPerMachine && kt >= RebootAndDelete) { + if (processesOnMachine == processesPerMachine + 1 && originalKt == KillType::RebootProcessAndSwitch) { + // Simulation runs which test DR add an extra process to each + // machine in the original cluster. When killing processes with the + // RebootProcessAndSwitch kill type, processes in the original + // cluster should be rebooted in order to kill any zombie + // processes. + kt = KillType::Reboot; + } else if (processesOnMachine != processesPerMachine) { + // Check if any processes on machine are rebooting CODE_PROBE(true, "Attempted reboot, but the target did not have all of its processes running", probe::context::sim2, @@ -1977,24 +1984,6 @@ public: return false; } - // Check if any processes on machine are rebooting - if (processesOnMachine != processesPerMachine) { - CODE_PROBE(true, - "Attempted reboot and kill, but the target did not have all of its processes running", - probe::context::sim2, - probe::assert::simOnly); - TraceEvent(SevWarn, "AbortedKill") - .detail("KillType", kt) - .detail("MachineId", machineId) - .detail("Reason", "Machine processes does not match number of processes per machine") - .detail("Processes", processesOnMachine) - .detail("ProcessesPerMachine", processesPerMachine) - .backtrace(); - if (ktFinal) - *ktFinal = None; - return false; - } - TraceEvent("KillMachine") .detail("MachineId", machineId) .detail("Kt", kt) diff --git a/fdbserver/ClusterController.actor.cpp b/fdbserver/ClusterController.actor.cpp index 79d033262c..a731760fe0 100644 --- a/fdbserver/ClusterController.actor.cpp +++ b/fdbserver/ClusterController.actor.cpp @@ -1228,12 +1228,15 @@ ACTOR Future registerWorker(RegisterWorkerRequest req, std::vector coordinatorAddresses = wait(cs.tryResolveHostnames()); const WorkerInterface& w = req.wi; - if (req.clusterId.present() && self->clusterId->get().present() && req.clusterId != self->clusterId->get()) { + if (req.clusterId.present() && self->clusterId->get().present() && req.clusterId != self->clusterId->get() && + req.processClass != ProcessClass::TesterClass) { + // TODO: Track invalid processes separately, report status in status json TraceEvent(g_network->isSimulated() ? SevWarnAlways : SevError, "WorkerBelongsToExistingCluster", self->id) .detail("WorkerClusterId", req.clusterId) .detail("ClusterControllerClusterId", self->clusterId->get()) .detail("WorkerId", w.id()) .detail("ProcessId", w.locality.processId()); + req.reply.sendError(invalid_cluster_id()); return Void(); } diff --git a/fdbserver/worker.actor.cpp b/fdbserver/worker.actor.cpp index e92fa2b9a5..77316ea74d 100644 --- a/fdbserver/worker.actor.cpp +++ b/fdbserver/worker.actor.cpp @@ -686,6 +686,9 @@ ACTOR Future registrationClient( when(wait(FlowTransport::transport().onIncompatibleChanged())) { break; } when(wait(issues->onChange())) { break; } when(wait(recovered)) { break; } + when(wait(clusterId->onChange())) { + break; + } } } } @@ -2845,7 +2848,8 @@ ACTOR Future workerServer(Reference connRecord, f.cancel(); state Error e = err; bool ok = e.code() == error_code_please_reboot || e.code() == error_code_actor_cancelled || - e.code() == error_code_please_reboot_delete || e.code() == error_code_local_config_changed; + e.code() == error_code_please_reboot_delete || e.code() == error_code_local_config_changed || + e.code() == error_code_invalid_cluster_id; endRole(Role::WORKER, interf.id(), "WorkerError", ok, e); errorForwarders.clear(false); sharedLogs.clear(); @@ -2882,6 +2886,7 @@ static std::set const& normalWorkerErrors() { s.insert(error_code_please_reboot); s.insert(error_code_please_reboot_delete); s.insert(error_code_local_config_changed); + s.insert(error_code_invalid_cluster_id); } return s; } @@ -3663,7 +3668,13 @@ ACTOR Future fdbd(Reference connRecord, // Otherwise, these actors may get a broken promise error. for (auto f : actors) f.cancel(); - Error err = checkIOTimeout(e); + state Error err = checkIOTimeout(e); + if (e.code() == error_code_invalid_cluster_id) { + // If this process tried to join an invalid cluster, become a + // zombie and wait for manual action by the operator. + TraceEvent(g_network->isSimulated() ? SevWarnAlways : SevError, "ZombieProcess").error(e); + wait(Never()); + } throw err; } } From 5ca2b89bdf2a337e59a8ea9c1c4d345c0b48728f Mon Sep 17 00:00:00 2001 From: Lukas Joswiak Date: Mon, 17 Oct 2022 22:33:36 -0700 Subject: [PATCH 70/95] Fix simulation issue where process switch was ignored The simulator tracks only active processes. Rebooted or killed processes are removed from the list of processes, and only get added back when the process is rebooted and starts up again. This causes a problem for the `RebootProcessAndSwitch` kill type, which wants to simultaneously reboot all machines in a cluster and change their cluster file. If a machine is currently being rebooted, it will miss the reboot process and switch command. The fix is to add a check when a process is being started in simulation. If the process has had its cluster file changed and the cluster is in a state where all processes should have had their cluster files reverted to the original value, the simulator will now send a `RebootProcessAndSwitch` signal right when the process is started. This will cause an extra reboot, but should correctly switch the process back to its original, correct cluster file, allowing the cluster to fully recover all clusters. Note that the above issue should only affect simulation, due to how the simulator tracks processes and handles kill signals. This commit also adds a field to each process struct to determine whether the process is being run in a DR cluster in the simulation run. This is needed because simulation does not differentiate between processes in different clusters (other than by the IP), and some processes needed to switch clusters and some simply needed to be rebooted. --- fdbclient/ManagementAPI.actor.cpp | 3 +- fdbrpc/include/fdbrpc/simulator.h | 15 ++++-- fdbrpc/sim2.actor.cpp | 23 ++++++---- fdbserver/FDBExecHelper.actor.cpp | 3 +- fdbserver/SimulatedCluster.actor.cpp | 46 ++++++++++++++----- fdbserver/workloads/ClientWorkload.actor.cpp | 3 +- .../workloads/MachineAttrition.actor.cpp | 3 ++ 7 files changed, 68 insertions(+), 28 deletions(-) diff --git a/fdbclient/ManagementAPI.actor.cpp b/fdbclient/ManagementAPI.actor.cpp index d4382ee086..2ec6c419ef 100644 --- a/fdbclient/ManagementAPI.actor.cpp +++ b/fdbclient/ManagementAPI.actor.cpp @@ -2639,7 +2639,8 @@ TEST_CASE("/ManagementAPI/AutoQuorumChange/checkLocality") { ProcessClass(ProcessClass::CoordinatorClass, ProcessClass::CommandLineSource), "", "", - currentProtocolVersion()); + currentProtocolVersion(), + false); } workers.push_back(data); diff --git a/fdbrpc/include/fdbrpc/simulator.h b/fdbrpc/include/fdbrpc/simulator.h index c15181c90f..0ddf8da238 100644 --- a/fdbrpc/include/fdbrpc/simulator.h +++ b/fdbrpc/include/fdbrpc/simulator.h @@ -105,6 +105,7 @@ public: bool excluded; bool cleared; bool rebooting; + bool drProcess; std::vector globals; INetworkConnections* network; @@ -129,8 +130,8 @@ public: const char* coordinationFolder) : name(name), coordinationFolder(coordinationFolder), dataFolder(dataFolder), machine(nullptr), addresses(addresses), address(addresses.address), locality(locality), startingClass(startingClass), - failed(false), excluded(false), cleared(false), rebooting(false), network(net), fault_injection_r(0), - fault_injection_p1(0), fault_injection_p2(0), failedDisk(false) { + failed(false), excluded(false), cleared(false), rebooting(false), drProcess(false), network(net), + fault_injection_r(0), fault_injection_p1(0), fault_injection_p2(0), failedDisk(false) { uid = deterministicRandom()->randomUniqueID(); } @@ -284,7 +285,8 @@ public: ProcessClass startingClass, const char* dataFolder, const char* coordinationFolder, - ProtocolVersion protocol) = 0; + ProtocolVersion protocol, + bool drProcess) = 0; virtual void killProcess(ProcessInfo* machine, KillType) = 0; virtual void rebootProcess(Optional> zoneId, bool allProcesses) = 0; virtual void rebootProcess(ProcessInfo* process, KillType kt) = 0; @@ -393,7 +395,11 @@ public: } void switchCluster(NetworkAddress const& address) { switchedCluster[address] = !switchedCluster[address]; } - bool hasSwitchedCluster(NetworkAddress const& address) const { return switchedCluster.at(address); } + bool hasSwitchedCluster(NetworkAddress const& address) const { + return switchedCluster.find(address) != switchedCluster.end() ? switchedCluster.at(address) : false; + } + void toggleGlobalSwitchCluster() { globalSwitchedCluster = !globalSwitchedCluster; } + bool globalHasSwitchedCluster() const { return globalSwitchedCluster; } void excludeAddress(NetworkAddress const& address) { excludedAddresses[address]++; @@ -546,6 +552,7 @@ private: std::map excludedAddresses; std::map clearedAddresses; std::map switchedCluster; + bool globalSwitchedCluster = false; std::map> roleAddresses; std::map disabledMap; bool allSwapsDisabled; diff --git a/fdbrpc/sim2.actor.cpp b/fdbrpc/sim2.actor.cpp index f286f56ab8..1618016b77 100644 --- a/fdbrpc/sim2.actor.cpp +++ b/fdbrpc/sim2.actor.cpp @@ -1261,7 +1261,8 @@ public: ProcessClass startingClass, const char* dataFolder, const char* coordinationFolder, - ProtocolVersion protocol) override { + ProtocolVersion protocol, + bool drProcess) override { ASSERT(locality.machineId().present()); MachineInfo& machine = machines[locality.machineId().get()]; if (!machine.machineId.present()) @@ -1311,6 +1312,7 @@ public: m->excluded = g_simulator->isExcluded(NetworkAddress(ip, port, true, false)); m->cleared = g_simulator->isCleared(addresses.address); m->protocolVersion = protocol; + m->drProcess = drProcess; m->setGlobal(enTDMetrics, (flowGlobalType)&m->tdmetrics); if (FLOW_KNOBS->ENABLE_CHAOS_FEATURES) { @@ -1324,7 +1326,8 @@ public: .detail("Address", m->address) .detail("MachineId", m->locality.machineId()) .detail("Excluded", m->excluded) - .detail("Cleared", m->cleared); + .detail("Cleared", m->cleared) + .detail("DrProcess", m->drProcess); if (std::string(name) == "remote flow process") { protectedAddresses.insert(m->address); @@ -1825,6 +1828,7 @@ public: } int processesOnMachine = 0; + bool isMainCluster = true; // false for machines running DR processes KillType originalKt = kt; // Reboot if any of the processes are protected and count the number of processes not rebooting @@ -1833,6 +1837,9 @@ public: kt = Reboot; if (!process->rebooting) processesOnMachine++; + if (process->drProcess) { + isMainCluster = false; + } } // Do nothing, if no processes to kill @@ -1959,14 +1966,12 @@ public: probe::context::sim2, probe::assert::simOnly); - if (processesOnMachine == processesPerMachine + 1 && originalKt == KillType::RebootProcessAndSwitch) { - // Simulation runs which test DR add an extra process to each - // machine in the original cluster. When killing processes with the - // RebootProcessAndSwitch kill type, processes in the original - // cluster should be rebooted in order to kill any zombie - // processes. + if (isMainCluster && originalKt == RebootProcessAndSwitch) { + // When killing processes with the RebootProcessAndSwitch kill + // type, processes in the original cluster should be rebooted in + // order to kill any zombie processes. kt = KillType::Reboot; - } else if (processesOnMachine != processesPerMachine) { + } else if (processesOnMachine != processesPerMachine && kt != RebootProcessAndSwitch) { // Check if any processes on machine are rebooting CODE_PROBE(true, "Attempted reboot, but the target did not have all of its processes running", diff --git a/fdbserver/FDBExecHelper.actor.cpp b/fdbserver/FDBExecHelper.actor.cpp index 8f76bf419a..a0e8037970 100644 --- a/fdbserver/FDBExecHelper.actor.cpp +++ b/fdbserver/FDBExecHelper.actor.cpp @@ -170,7 +170,8 @@ ACTOR Future spawnSimulated(std::vector paramList, ProcessClass(ProcessClass::UnsetClass, ProcessClass::AutoSource), self->dataFolder.c_str(), self->coordinationFolder.c_str(), // do we need to customize this coordination folder path? - self->protocolVersion); + self->protocolVersion, + false); wait(g_simulator->onProcess(child)); state Future onShutdown = child->onShutdown(); state Future parentShutdown = self->onShutdown(); diff --git a/fdbserver/SimulatedCluster.actor.cpp b/fdbserver/SimulatedCluster.actor.cpp index c64bf610df..de8f06bf6b 100644 --- a/fdbserver/SimulatedCluster.actor.cpp +++ b/fdbserver/SimulatedCluster.actor.cpp @@ -625,7 +625,8 @@ ACTOR Future simulatedFDBDRebooter(ReferencegetCurrentProcess(); state UID randomId = nondeterministicRandom()->randomUniqueID(); state int cycles = 0; @@ -645,7 +646,8 @@ ACTOR Future simulatedFDBDRebooter(Reference simulatedFDBDRebooter(Referencec_str(), coordFolder->c_str(), - protocolVersion); + protocolVersion, + isDr); wait(g_simulator->onProcess( process, TaskPriority::DefaultYield)); // Now switch execution to the process on which we will run @@ -725,6 +728,16 @@ ACTOR Future simulatedFDBDRebooter(ReferenceglobalHasSwitchedCluster() && g_simulator->hasSwitchedCluster(process->address)) { + // When switching machines between clusters, a simultaneous + // reboot followed by a reboot and switch can cause the + // reboot and switch to be ignored. Handle this case by + // sending the reboot and switch kill type when the process + // comes back online. + TraceEvent("RebootProcessAndSwitchLateReboot").detail("Address", process->address); + g_simulator->switchCluster(process->address); + process->shutdownSignal.send(ISimulator::KillType::RebootProcessAndSwitch); + } wait(waitForAny(futures)); } catch (Error& e) { // If in simulation, if we make it here with an error other than io_timeout but enASIOTimedOut is set @@ -840,7 +853,8 @@ ACTOR Future simulatedFDBDRebooter(ReferencehasSwitchedCluster(process->address)); + .detail("SwitchingTo", g_simulator->hasSwitchedCluster(process->address)) + .detail("MachineId", process->machine->machineId); // Handle the case where otherConnStr is '@'. if (otherConnStr.toString().size() > 1) { @@ -876,7 +890,8 @@ ACTOR Future simulatedMachine(ClusterConnectionString connStr, bool sslOnly, std::string whitelistBinPaths, ProtocolVersion protocolVersion, - ConfigDBType configDBType) { + ConfigDBType configDBType, + bool isDr) { state int bootCount = 0; state std::vector myFolders; state std::vector coordFolders; @@ -948,7 +963,8 @@ ACTOR Future simulatedMachine(ClusterConnectionString connStr, agentMode, whitelistBinPaths, protocolVersion, - configDBType)); + configDBType, + isDr)); g_simulator->setDiffProtocol = true; } else { processes.push_back(simulatedFDBDRebooter(clusterFile, @@ -967,7 +983,8 @@ ACTOR Future simulatedMachine(ClusterConnectionString connStr, agentMode, whitelistBinPaths, g_network->protocolVersion(), - configDBType)); + configDBType, + isDr)); } TraceEvent("SimulatedMachineProcess", randomId) .detail("Address", NetworkAddress(ips[i], listenPort, true, false)) @@ -1344,7 +1361,8 @@ ACTOR Future restartSimulatedSystem(std::vector>* systemActor usingSSL && (listenersPerProcess == 1 || processClass == ProcessClass::TesterClass), whitelistBinPaths, protocolVersion, - configDBType), + configDBType, + false), processClass == ProcessClass::TesterClass ? "SimulatedTesterMachine" : "SimulatedMachine")); } @@ -2383,7 +2401,8 @@ void setupSimulatedSystem(std::vector>* systemActors, sslOnly, whitelistBinPaths, protocolVersion, - configDBType), + configDBType, + false), "SimulatedMachine")); if (requiresExtraDBMachines) { @@ -2413,7 +2432,8 @@ void setupSimulatedSystem(std::vector>* systemActors, sslOnly, whitelistBinPaths, protocolVersion, - configDBType), + configDBType, + true), "SimulatedMachine")); ++cluster; } @@ -2460,7 +2480,8 @@ void setupSimulatedSystem(std::vector>* systemActors, sslOnly, whitelistBinPaths, protocolVersion, - configDBType), + configDBType, + false), "SimulatedTesterMachine")); } @@ -2584,7 +2605,8 @@ ACTOR void setupAndRun(std::string dataFolder, ProcessClass(ProcessClass::TesterClass, ProcessClass::CommandLineSource), "", "", - currentProtocolVersion()); + currentProtocolVersion(), + false); testSystem->excludeFromRestarts = true; wait(g_simulator->onProcess(testSystem, TaskPriority::DefaultYield)); Sim2FileSystem::newFileSystem(); diff --git a/fdbserver/workloads/ClientWorkload.actor.cpp b/fdbserver/workloads/ClientWorkload.actor.cpp index 73dbb0ea75..41ce0d745f 100644 --- a/fdbserver/workloads/ClientWorkload.actor.cpp +++ b/fdbserver/workloads/ClientWorkload.actor.cpp @@ -74,7 +74,8 @@ class WorkloadProcessState { ProcessClass(ProcessClass::TesterClass, ProcessClass::AutoSource), dataFolder.c_str(), parent->coordinationFolder.c_str(), - parent->protocolVersion); + parent->protocolVersion, + false); self->childProcess->excludeFromRestarts = true; wait(g_simulator->onProcess(self->childProcess, TaskPriority::DefaultYield)); try { diff --git a/fdbserver/workloads/MachineAttrition.actor.cpp b/fdbserver/workloads/MachineAttrition.actor.cpp index e46619cd55..9d4980efae 100644 --- a/fdbserver/workloads/MachineAttrition.actor.cpp +++ b/fdbserver/workloads/MachineAttrition.actor.cpp @@ -359,9 +359,12 @@ struct MachineAttritionWorkload : FailureInjectionWorkload { g_simulator->killDataHall(target, kt); } else if (!g_simulator->extraDatabases.empty() && deterministicRandom()->random01() < 0.1) { state ISimulator::KillType kt = ISimulator::RebootProcessAndSwitch; + TraceEvent("Assassination").detail("KillType", kt); g_simulator->killAll(kt, true); + g_simulator->toggleGlobalSwitchCluster(); wait(delay(self->testDuration / 2)); g_simulator->killAll(kt, true); + g_simulator->toggleGlobalSwitchCluster(); } else { state int killedMachines = 0; while (killedMachines < self->machinesToKill && self->machines.size() > self->machinesToLeave) { From bba05b7c9bae25938ff67cccfbe68883f432a0df Mon Sep 17 00:00:00 2001 From: Lukas Joswiak Date: Tue, 18 Oct 2022 17:57:43 -0700 Subject: [PATCH 71/95] Move cluster ID from txnStateStore to the database The cluster ID is now stored in the database instead of in the txnStateStore. The cluster controller will read it on boot and send it to all processes to persist. --- fdbclient/SystemData.cpp | 7 ++-- fdbclient/include/fdbclient/SystemData.h | 5 +-- fdbserver/ClusterController.actor.cpp | 46 ++++++++++++++++++++++++ fdbserver/worker.actor.cpp | 8 +---- 4 files changed, 54 insertions(+), 12 deletions(-) diff --git a/fdbclient/SystemData.cpp b/fdbclient/SystemData.cpp index f629722c6e..19c23b1ad7 100644 --- a/fdbclient/SystemData.cpp +++ b/fdbclient/SystemData.cpp @@ -284,8 +284,6 @@ const KeyRangeRef readConflictRangeKeysRange = const KeyRangeRef writeConflictRangeKeysRange = KeyRangeRef("\xff\xff/transaction/write_conflict_range/"_sr, "\xff\xff/transaction/write_conflict_range/\xff\xff"_sr); -const KeyRef clusterIdKey = "\xff/clusterId"_sr; - const KeyRangeRef auditRange = KeyRangeRef("\xff/audit/"_sr, "\xff/audit0"_sr); const KeyRef auditPrefix = auditRange.begin; @@ -1074,6 +1072,9 @@ const KeyRangeRef timeKeeperPrefixRange("\xff\x02/timeKeeper/map/"_sr, "\xff\x02 const KeyRef timeKeeperVersionKey = "\xff\x02/timeKeeper/version"_sr; const KeyRef timeKeeperDisableKey = "\xff\x02/timeKeeper/disable"_sr; +// Durable cluster ID key +const KeyRef clusterIdKey = "\xff/clusterId"_sr; + // Backup Log Mutation constant variables const KeyRef backupEnabledKey = "\xff/backupEnabled"_sr; const KeyRangeRef backupLogKeys("\xff\x02/blog/"_sr, "\xff\x02/blog0"_sr); @@ -1810,4 +1811,4 @@ TEST_CASE("noSim/SystemData/compat/KeyServers") { printf("ssi serdes test complete\n"); return Void(); -} \ No newline at end of file +} diff --git a/fdbclient/include/fdbclient/SystemData.h b/fdbclient/include/fdbclient/SystemData.h index 3049e41b10..1393dc63b1 100644 --- a/fdbclient/include/fdbclient/SystemData.h +++ b/fdbclient/include/fdbclient/SystemData.h @@ -92,8 +92,6 @@ void decodeKeyServersValue(RangeResult result, UID& destID, bool missingIsError = true); -extern const KeyRef clusterIdKey; - extern const KeyRangeRef auditRange; extern const KeyRef auditPrefix; const Key auditRangeKey(const AuditType type, const UID& auditId, const KeyRef& key); @@ -505,6 +503,9 @@ extern const KeyRangeRef timeKeeperPrefixRange; extern const KeyRef timeKeeperVersionKey; extern const KeyRef timeKeeperDisableKey; +// Durable cluster ID key +extern const KeyRef clusterIdKey; + // Layer status metadata prefix extern const KeyRangeRef layerStatusMetaPrefixRange; diff --git a/fdbserver/ClusterController.actor.cpp b/fdbserver/ClusterController.actor.cpp index a731760fe0..9a3d997db2 100644 --- a/fdbserver/ClusterController.actor.cpp +++ b/fdbserver/ClusterController.actor.cpp @@ -2976,6 +2976,51 @@ ACTOR Future metaclusterMetricsUpdater(ClusterControllerData* self) { } } +// Update the DBInfo state with this processes cluster ID. If this process does +// not have a cluster ID and one does not exist in the database, generate one. +ACTOR Future updateClusterId(ClusterControllerData* self) { + state Reference tr = makeReference(self->cx); + loop { + try { + state Optional durableClusterId = self->clusterId->get(); + tr->setOption(FDBTransactionOptions::ACCESS_SYSTEM_KEYS); + tr->setOption(FDBTransactionOptions::LOCK_AWARE); + + Optional clusterIdVal = wait(tr->get(clusterIdKey)); + + if (clusterIdVal.present()) { + UID clusterId = BinaryReader::fromStringRef(clusterIdVal.get(), IncludeVersion()); + if (durableClusterId.present()) { + // If this process has an on disk file for the cluster ID, + // verify it matches the value in the database. + ASSERT(clusterId == durableClusterId.get()); + } else { + // Otherwise, write the cluster ID in the database to the + // DbInfo object so all clients will learn of the cluster + // ID. + durableClusterId = clusterId; + } + } else if (!durableClusterId.present()) { + // No cluster ID exists in the database or on the machine. Generate and set one. + ASSERT(!durableClusterId.present()); + durableClusterId = deterministicRandom()->randomUniqueID(); + tr->set(clusterIdKey, BinaryWriter::toValue(durableClusterId.get(), IncludeVersion())); + wait(tr->commit()); + } + auto serverInfo = self->db.serverInfo->get(); + if (!serverInfo.client.clusterId.isValid()) { + ASSERT(durableClusterId.present()); + serverInfo.id = deterministicRandom()->randomUniqueID(); + serverInfo.client.clusterId = durableClusterId.get(); + self->db.serverInfo->set(serverInfo); + } + return Void(); + } catch (Error& e) { + wait(tr->onError(e)); + } + } +} + ACTOR Future clusterControllerCore(ClusterControllerFullInterface interf, Future leaderFail, ServerCoordinators coordinators, @@ -3020,6 +3065,7 @@ ACTOR Future clusterControllerCore(ClusterControllerFullInterface interf, self.addActor.send(monitorConsistencyScan(&self)); self.addActor.send(metaclusterMetricsUpdater(&self)); self.addActor.send(dbInfoUpdater(&self)); + self.addActor.send(updateClusterId(&self)); self.addActor.send(self.clusterControllerMetrics.traceCounters("ClusterControllerMetrics", self.id, SERVER_KNOBS->STORAGE_LOGGING_DELAY, diff --git a/fdbserver/worker.actor.cpp b/fdbserver/worker.actor.cpp index 77316ea74d..a98999589f 100644 --- a/fdbserver/worker.actor.cpp +++ b/fdbserver/worker.actor.cpp @@ -2097,13 +2097,7 @@ ACTOR Future workerServer(Reference connRecord, errorForwarders.add( success(broadcastDBInfoRequest(req, SERVER_KNOBS->DBINFO_SEND_AMOUNT, notUpdated, true))); - if (!clusterId->get().present() && localInfo.recoveryState >= RecoveryState::ACCEPTING_COMMITS && - localInfo.client.clusterId.isValid()) { - // Persist the cluster ID as a file in the data - // directory once recovery has made the transaction - // state store durable. The txnStateStore also stores - // the cluster ID. - // TODO: Does the txnStateStore need to store the cluster ID? + if (!clusterId->get().present() && localInfo.client.clusterId.isValid()) { state UID tmpClusterId = localInfo.client.clusterId; wait(createClusterIdFile(folder, tmpClusterId)); clusterId->set(tmpClusterId); From 1fca3b7ddc7791cda56965a002ba4d749a2c7477 Mon Sep 17 00:00:00 2001 From: Lukas Joswiak Date: Tue, 18 Oct 2022 17:59:49 -0700 Subject: [PATCH 72/95] Modify how cluster ID tests are run in simulation --- fdbserver/workloads/MachineAttrition.actor.cpp | 7 ++++++- 1 file changed, 6 insertions(+), 1 deletion(-) diff --git a/fdbserver/workloads/MachineAttrition.actor.cpp b/fdbserver/workloads/MachineAttrition.actor.cpp index 9d4980efae..8653300e69 100644 --- a/fdbserver/workloads/MachineAttrition.actor.cpp +++ b/fdbserver/workloads/MachineAttrition.actor.cpp @@ -74,6 +74,7 @@ struct MachineAttritionWorkload : FailureInjectionWorkload { bool killProcess = false; bool killZone = false; bool killSelf = false; + bool killAll = false; std::vector targetIds; bool replacement = false; bool waitForVersion = false; @@ -107,6 +108,10 @@ struct MachineAttritionWorkload : FailureInjectionWorkload { killProcess = getOption(options, "killProcess"_sr, killProcess); killZone = getOption(options, "killZone"_sr, killZone); killSelf = getOption(options, "killSelf"_sr, killSelf); + killAll = + getOption(options, + "killAll"_sr, + g_network->isSimulated() && !g_simulator->extraDatabases.empty() && BUGGIFY_WITH_PROB(0.01)); targetIds = getOption(options, "targetIds"_sr, std::vector()); replacement = getOption(options, "replacement"_sr, reboot && deterministicRandom()->random01() < 0.5); waitForVersion = getOption(options, "waitForVersion"_sr, waitForVersion); @@ -357,7 +362,7 @@ struct MachineAttritionWorkload : FailureInjectionWorkload { TraceEvent("Assassination").detail("TargetDataHall", target).detail("KillType", kt); g_simulator->killDataHall(target, kt); - } else if (!g_simulator->extraDatabases.empty() && deterministicRandom()->random01() < 0.1) { + } else if (self->killAll) { state ISimulator::KillType kt = ISimulator::RebootProcessAndSwitch; TraceEvent("Assassination").detail("KillType", kt); g_simulator->killAll(kt, true); From 9d3c3b1efea386f665de7e3d6b62719f15bb0971 Mon Sep 17 00:00:00 2001 From: Lukas Joswiak Date: Tue, 18 Oct 2022 18:21:36 -0700 Subject: [PATCH 73/95] Remove cluster ID logic from individual roles The logic to determine the validity of a process joining a cluster now belongs on the worker and the cluster controller. It is no longer restricted to tlogs and storages, but instead applies to all processes (even stateless ones). --- fdbserver/ApplyMetadataMutation.cpp | 2 +- fdbserver/ClusterController.actor.cpp | 9 +- fdbserver/ClusterRecovery.actor.cpp | 35 +--- fdbserver/DDTeamCollection.actor.cpp | 7 - fdbserver/DDTxnProcessor.actor.cpp | 19 -- fdbserver/TLogServer.actor.cpp | 185 ++++-------------- fdbserver/TagPartitionedLogSystem.actor.cpp | 8 - .../include/fdbserver/DDTeamCollection.h | 2 - fdbserver/include/fdbserver/DDTxnProcessor.h | 4 - fdbserver/include/fdbserver/LogSystem.h | 1 - .../fdbserver/TagPartitionedLogSystem.actor.h | 3 - .../include/fdbserver/WorkerInterface.actor.h | 9 +- fdbserver/storageserver.actor.cpp | 91 +-------- fdbserver/worker.actor.cpp | 4 +- 14 files changed, 54 insertions(+), 325 deletions(-) diff --git a/fdbserver/ApplyMetadataMutation.cpp b/fdbserver/ApplyMetadataMutation.cpp index 70facfd481..1b6ba063c6 100644 --- a/fdbserver/ApplyMetadataMutation.cpp +++ b/fdbserver/ApplyMetadataMutation.cpp @@ -613,7 +613,7 @@ private: m.param1.startsWith(applyMutationsAddPrefixRange.begin) || m.param1.startsWith(applyMutationsRemovePrefixRange.begin) || m.param1.startsWith(tagLocalityListPrefix) || m.param1.startsWith(serverTagHistoryPrefix) || - m.param1.startsWith(testOnlyTxnStateStorePrefixRange.begin) || m.param1 == clusterIdKey) { + m.param1.startsWith(testOnlyTxnStateStorePrefixRange.begin)) { txnStateStore->set(KeyValueRef(m.param1, m.param2)); } diff --git a/fdbserver/ClusterController.actor.cpp b/fdbserver/ClusterController.actor.cpp index 9a3d997db2..9780881b7d 100644 --- a/fdbserver/ClusterController.actor.cpp +++ b/fdbserver/ClusterController.actor.cpp @@ -1060,8 +1060,7 @@ void clusterRegisterMaster(ClusterControllerData* self, RegisterMasterRequest co .detail("GrvProxies", req.grvProxies.size()) .detail("RecoveryCount", req.recoveryCount) .detail("Stalled", req.recoveryStalled) - .detail("OldestBackupEpoch", req.logSystemConfig.oldestBackupEpoch) - .detail("ClusterId", req.clusterId); + .detail("OldestBackupEpoch", req.logSystemConfig.oldestBackupEpoch); // make sure the request comes from an active database auto db = &self->db; @@ -1120,7 +1119,7 @@ void clusterRegisterMaster(ClusterControllerData* self, RegisterMasterRequest co // Construct the client information if (db->clientInfo->get().commitProxies != req.commitProxies || db->clientInfo->get().grvProxies != req.grvProxies || - db->clientInfo->get().tenantMode != db->config.tenantMode || db->clientInfo->get().clusterId != req.clusterId || + db->clientInfo->get().tenantMode != db->config.tenantMode || db->clientInfo->get().isEncryptionEnabled != SERVER_KNOBS->ENABLE_ENCRYPTION || db->clientInfo->get().clusterType != db->clusterType || db->clientInfo->get().metaclusterName != db->metaclusterName || @@ -1133,8 +1132,6 @@ void clusterRegisterMaster(ClusterControllerData* self, RegisterMasterRequest co .detail("ReqCPs", req.commitProxies) .detail("TenantMode", db->clientInfo->get().tenantMode.toString()) .detail("ReqTenantMode", db->config.tenantMode.toString()) - .detail("ClusterId", db->clientInfo->get().clusterId) - .detail("ReqClusterId", req.clusterId) .detail("EncryptionEnabled", SERVER_KNOBS->ENABLE_ENCRYPTION) .detail("ClusterType", db->clientInfo->get().clusterType) .detail("ReqClusterType", db->clusterType) @@ -1149,7 +1146,6 @@ void clusterRegisterMaster(ClusterControllerData* self, RegisterMasterRequest co clientInfo.commitProxies = req.commitProxies; clientInfo.grvProxies = req.grvProxies; clientInfo.tenantMode = TenantAPI::tenantModeForClusterType(db->clusterType, db->config.tenantMode); - clientInfo.clusterId = req.clusterId; clientInfo.clusterType = db->clusterType; clientInfo.metaclusterName = db->metaclusterName; db->clientInfo->set(clientInfo); @@ -1230,7 +1226,6 @@ ACTOR Future registerWorker(RegisterWorkerRequest req, const WorkerInterface& w = req.wi; if (req.clusterId.present() && self->clusterId->get().present() && req.clusterId != self->clusterId->get() && req.processClass != ProcessClass::TesterClass) { - // TODO: Track invalid processes separately, report status in status json TraceEvent(g_network->isSimulated() ? SevWarnAlways : SevError, "WorkerBelongsToExistingCluster", self->id) .detail("WorkerClusterId", req.clusterId) .detail("ClusterControllerClusterId", self->clusterId->get()) diff --git a/fdbserver/ClusterRecovery.actor.cpp b/fdbserver/ClusterRecovery.actor.cpp index afc6d85bfa..15702ab500 100644 --- a/fdbserver/ClusterRecovery.actor.cpp +++ b/fdbserver/ClusterRecovery.actor.cpp @@ -297,7 +297,6 @@ ACTOR Future newTLogServers(Reference self, self->logSystem = Reference(); // Cancels the actors in the previous log system. Reference newLogSystem = wait(oldLogSystem->newEpoch(recr, fRemoteWorkers, - self->clusterId, self->configuration, self->cstate.myDBState.recoveryCount + 1, self->recoveryTransactionVersion, @@ -311,7 +310,6 @@ ACTOR Future newTLogServers(Reference self, self->logSystem = Reference(); // Cancels the actors in the previous log system. Reference newLogSystem = wait(oldLogSystem->newEpoch(recr, Never(), - self->clusterId, self->configuration, self->cstate.myDBState.recoveryCount + 1, self->recoveryTransactionVersion, @@ -347,7 +345,6 @@ ACTOR Future newSeedServers(Reference self, isr.storeType = self->configuration.storageServerStoreType; isr.reqId = deterministicRandom()->randomUniqueID(); isr.interfaceId = deterministicRandom()->randomUniqueID(); - isr.clusterId = self->clusterId; isr.initialClusterVersion = self->recoveryTransactionVersion; ErrorOr newServer = wait(recruits.storageServers[idx].storage.tryGetReply(isr)); @@ -477,7 +474,6 @@ ACTOR Future trackTlogRecovery(Reference self, self->dbgid) .detail("StatusCode", RecoveryStatus::fully_recovered) .detail("Status", RecoveryStatus::names[RecoveryStatus::fully_recovered]) - .detail("ClusterId", self->clusterId) .trackLatest(self->clusterRecoveryStateEventHolder->trackingKey); TraceEvent(getRecoveryEventName(ClusterRecoveryEventType::CLUSTER_RECOVERY_GENERATION_EVENT_NAME).c_str(), @@ -786,7 +782,6 @@ Future sendMasterRegistration(ClusterRecoveryData* self, masterReq.priorCommittedLogServers = priorCommittedLogServers; masterReq.recoveryState = self->recoveryState; masterReq.recoveryStalled = self->recruitmentStalled->get(); - masterReq.clusterId = self->clusterId; return brokenPromiseToNever(self->clusterController.registerMaster.getReply(masterReq)); } @@ -1350,8 +1345,7 @@ ACTOR Future recoverFrom(Reference self, Reference oldLogSystem, std::vector* seedServers, std::vector>* initialConfChanges, - Future poppedTxsVersion, - bool* clusterIdExists) { + Future poppedTxsVersion) { TraceEvent(getRecoveryEventName(ClusterRecoveryEventType::CLUSTER_RECOVERY_STATE_EVENT_NAME).c_str(), self->dbgid) .detail("StatusCode", RecoveryStatus::reading_transaction_system_state) .detail("Status", RecoveryStatus::names[RecoveryStatus::reading_transaction_system_state]) @@ -1375,16 +1369,6 @@ ACTOR Future recoverFrom(Reference self, debug_checkMaxRestoredVersion(UID(), self->lastEpochEnd, "DBRecovery"); - // Generate a cluster ID to uniquely identify the cluster if it doesn't - // already exist in the txnStateStore. - Optional clusterId = self->txnStateStore->readValue(clusterIdKey).get(); - *clusterIdExists = clusterId.present(); - if (!clusterId.present()) { - self->clusterId = deterministicRandom()->randomUniqueID(); - } else { - self->clusterId = BinaryReader::fromStringRef(clusterId.get(), Unversioned()); - } - // Ordinarily we pass through this loop once and recover. We go around the loop if recovery stalls for more than a // second, a provisional master is initialized, and an "emergency transaction" is submitted that might change the // configuration so that we can finish recovery. @@ -1540,7 +1524,6 @@ ACTOR Future clusterRecoveryCore(Reference self) { state Future logChanges; state Future minRecoveryDuration; state Future poppedTxsVersion; - state bool clusterIdExists = false; loop { Reference oldLogSystem = oldLogSystems->get(); @@ -1556,13 +1539,9 @@ ACTOR Future clusterRecoveryCore(Reference self) { self->registrationTrigger.trigger(); choose { - when(wait(oldLogSystem ? recoverFrom(self, - oldLogSystem, - &seedServers, - &initialConfChanges, - poppedTxsVersion, - std::addressof(clusterIdExists)) - : Never())) { + when(wait(oldLogSystem + ? recoverFrom(self, oldLogSystem, &seedServers, &initialConfChanges, poppedTxsVersion) + : Never())) { reg.cancel(); break; } @@ -1591,7 +1570,6 @@ ACTOR Future clusterRecoveryCore(Reference self) { .detail("Status", RecoveryStatus::names[RecoveryStatus::recovery_transaction]) .detail("PrimaryLocality", self->primaryLocality) .detail("DcId", self->masterInterface.locality.dcId()) - .detail("ClusterId", self->clusterId) .trackLatest(self->clusterRecoveryStateEventHolder->trackingKey); // Recovery transaction @@ -1680,11 +1658,6 @@ ACTOR Future clusterRecoveryCore(Reference self) { } } - // Write cluster ID into txnStateStore if it is missing. - if (!clusterIdExists) { - tr.set(recoveryCommitRequest.arena, clusterIdKey, BinaryWriter::toValue(self->clusterId, Unversioned())); - } - applyMetadataMutations(SpanContext(), self->dbgid, recoveryCommitRequest.arena, diff --git a/fdbserver/DDTeamCollection.actor.cpp b/fdbserver/DDTeamCollection.actor.cpp index f0db0d48fa..bc578ddc3f 100644 --- a/fdbserver/DDTeamCollection.actor.cpp +++ b/fdbserver/DDTeamCollection.actor.cpp @@ -2284,15 +2284,12 @@ public: self->recruitingIds.insert(interfaceId); self->recruitingLocalities.insert(candidateWorker.worker.stableAddress()); - UID clusterId = wait(self->getClusterId()); - state InitializeStorageRequest isr; isr.storeType = recruitTss ? self->configuration.testingStorageServerStoreType : self->configuration.storageServerStoreType; isr.seedTag = invalidTag; isr.reqId = deterministicRandom()->randomUniqueID(); isr.interfaceId = interfaceId; - isr.clusterId = clusterId; // if tss, wait for pair ss to finish and add its id to isr. If pair fails, don't recruit tss state bool doRecruit = true; @@ -3470,10 +3467,6 @@ Future DDTeamCollection::monitorHealthyTeams() { return DDTeamCollectionImpl::monitorHealthyTeams(this); } -Future DDTeamCollection::getClusterId() { - return db->getClusterId(); -} - Future DDTeamCollection::getNextWigglingServerID() { Optional localityKey; Optional localityValue; diff --git a/fdbserver/DDTxnProcessor.actor.cpp b/fdbserver/DDTxnProcessor.actor.cpp index 2770345a72..ed5fda041f 100644 --- a/fdbserver/DDTxnProcessor.actor.cpp +++ b/fdbserver/DDTxnProcessor.actor.cpp @@ -221,21 +221,6 @@ class DDTxnProcessorImpl { } } - ACTOR static Future getClusterId(Database cx) { - state Transaction tr(cx); - loop { - try { - tr.setOption(FDBTransactionOptions::ACCESS_SYSTEM_KEYS); - tr.setOption(FDBTransactionOptions::LOCK_AWARE); - Optional clusterId = wait(tr.get(clusterIdKey)); - ASSERT(clusterId.present()); - return BinaryReader::fromStringRef(clusterId.get(), Unversioned()); - } catch (Error& e) { - wait(tr.onError(e)); - } - } - } - // Read keyservers, return unique set of teams ACTOR static Future> getInitialDataDistribution( Database cx, @@ -675,10 +660,6 @@ Future DDTxnProcessor::tryUpdateReplicasKeyForDc(const Optional& dcId, return DDTxnProcessorImpl::tryUpdateReplicasKeyForDc(cx, dcId, storageTeamSize); } -Future DDTxnProcessor::getClusterId() const { - return DDTxnProcessorImpl::getClusterId(cx); -} - Future DDTxnProcessor::waitDDTeamInfoPrintSignal() const { return DDTxnProcessorImpl::waitDDTeamInfoPrintSignal(cx); } diff --git a/fdbserver/TLogServer.actor.cpp b/fdbserver/TLogServer.actor.cpp index 21c6601232..56480b2e35 100644 --- a/fdbserver/TLogServer.actor.cpp +++ b/fdbserver/TLogServer.actor.cpp @@ -26,7 +26,6 @@ #include "fdbclient/RunTransaction.actor.h" #include "fdbclient/SystemData.h" #include "fdbclient/FDBTypes.h" -#include "fdbclient/ManagementAPI.actor.h" #include "fdbserver/WorkerInterface.actor.h" #include "fdbserver/SpanContextMessage.h" #include "fdbserver/TLogInterface.h" @@ -217,8 +216,6 @@ static const KeyRange persistTagMessagesKeys = prefixRange("TagMsg/"_sr); static const KeyRange persistTagMessageRefsKeys = prefixRange("TagMsgRef/"_sr); static const KeyRange persistTagPoppedKeys = prefixRange("TagPop/"_sr); -static const KeyRef persistClusterIdKey = "clusterId"_sr; - static Key persistTagMessagesKey(UID id, Tag tag, Version version) { BinaryWriter wr(Unversioned()); wr.serializeBytes(persistTagMessagesKeys.begin); @@ -306,13 +303,6 @@ struct TLogData : NonCopyable { Deque spillOrder; std::map> id_data; - // The durable cluster ID identifies which cluster the tlogs persistent - // data is written from. This value is restored from disk when the tlog - // restarts. - UID durableClusterId; - // The cluster-controller cluster ID stores the cluster ID read from the txnStateStore. - // It is cached in this variable. - UID ccClusterId; UID dbgid; UID workerID; @@ -2401,24 +2391,6 @@ ACTOR Future initPersistentState(TLogData* self, Reference logDat return Void(); } -ACTOR Future getClusterId(TLogData* self) { - state ReadYourWritesTransaction tr(self->cx); - loop { - try { - tr.setOption(FDBTransactionOptions::ACCESS_SYSTEM_KEYS); - tr.setOption(FDBTransactionOptions::LOCK_AWARE); - Optional clusterId = wait(tr.get(clusterIdKey)); - if (clusterId.present()) { - return BinaryReader::fromStringRef(clusterId.get(), Unversioned()); - } else { - return UID(); - } - } catch (Error& e) { - wait(tr.onError(e)); - } - } -} - // send stopped promise instead of LogData* to avoid reference cycles ACTOR Future rejoinClusterController(TLogData* self, TLogInterface tli, @@ -2441,26 +2413,14 @@ ACTOR Future rejoinClusterController(TLogData* self, } isDisplaced = isDisplaced && !inf.logSystemConfig.hasTLog(tli.id()); if (isDisplaced) { - state TraceEvent ev("TLogDisplaced", tli.id()); - ev.detail("Reason", "DBInfoDoesNotContain") + TraceEvent("TLogDisplaced", tli.id()) + .detail("Reason", "DBInfoDoesNotContain") .detail("RecoveryCount", recoveryCount) .detail("InfRecoveryCount", inf.recoveryCount) .detail("RecoveryState", (int)inf.recoveryState) .detail("LogSysConf", describe(inf.logSystemConfig.tLogs)) .detail("PriorLogs", describe(inf.priorCommittedLogServers)) .detail("OldLogGens", inf.logSystemConfig.oldTLogs.size()); - // Read and cache cluster ID before displacing this tlog. We want - // to avoid removing the tlogs data if it has joined a new cluster - // with a different cluster ID. - - // TODO: #5375 - /* - state UID clusterId = wait(getClusterId(self)); - ASSERT(clusterId.isValid()); - self->ccClusterId = clusterId; - ev.detail("ClusterId", clusterId).detail("SelfClusterId", self->durableClusterId); - */ - if (BUGGIFY) wait(delay(SERVER_KNOBS->BUGGIFY_WORKER_REMOVED_MAX_LAG * deterministicRandom()->random01())); throw worker_removed(); @@ -2619,28 +2579,6 @@ ACTOR Future tLogEnablePopReq(TLogEnablePopRequest enablePopReq, TLogData* return Void(); } -// TODO: Remove all cluster ID logic from tlog and storage server -ACTOR Future updateDurableClusterID(TLogData* self) { - loop { - // Persist cluster ID once cluster has recovered. - if (self->dbInfo->get().recoveryState == RecoveryState::FULLY_RECOVERED) { - ASSERT(!self->durableClusterId.isValid()); - state UID ccClusterId = self->dbInfo->get().client.clusterId; - self->durableClusterId = ccClusterId; - ASSERT(ccClusterId.isValid()); - - wait(self->persistentDataCommitLock.take()); - state FlowLock::Releaser commitLockReleaser(self->persistentDataCommitLock); - self->persistentData->set( - KeyValueRef(persistClusterIdKey, BinaryWriter::toValue(ccClusterId, Unversioned()))); - wait(self->persistentData->commit()); - - return Void(); - } - wait(self->dbInfo->onChange()); - } -} - ACTOR Future serveTLogInterface(TLogData* self, TLogInterface tli, Reference logData, @@ -3028,7 +2966,6 @@ ACTOR Future restorePersistentState(TLogData* self, state IKeyValueStore* storage = self->persistentData; state Future> fFormat = storage->readValue(persistFormat.key); state Future> fRecoveryLocation = storage->readValue(persistRecoveryLocationKey); - state Future> fClusterId = storage->readValue(persistClusterIdKey); state Future fVers = storage->readRange(persistCurrentVersionKeys); state Future fKnownCommitted = storage->readRange(persistKnownCommittedVersionKeys); state Future fLocality = storage->readRange(persistLocalityKeys); @@ -3040,7 +2977,7 @@ ACTOR Future restorePersistentState(TLogData* self, // FIXME: metadata in queue? - wait(waitForAll(std::vector{ fFormat, fRecoveryLocation, fClusterId })); + wait(waitForAll(std::vector{ fFormat, fRecoveryLocation })); wait(waitForAll(std::vector{ fVers, fKnownCommitted, fLocality, @@ -3050,10 +2987,6 @@ ACTOR Future restorePersistentState(TLogData* self, fProtocolVersions, fTLogSpillTypes })); - if (fClusterId.get().present()) { - self->durableClusterId = BinaryReader::fromStringRef(fClusterId.get().get(), Unversioned()); - } - if (fFormat.get().present() && !persistFormatReadableRange.contains(fFormat.get().get())) { // FIXME: remove when we no longer need to test upgrades from 4.X releases if (g_network->isSimulated()) { @@ -3316,7 +3249,7 @@ bool tlogTerminated(TLogData* self, IKeyValueStore* persistentData, TLogQueue* p } if (e.code() == error_code_worker_removed || e.code() == error_code_recruitment_failed || - e.code() == error_code_file_not_found || e.code() == error_code_invalid_cluster_id) { + e.code() == error_code_file_not_found) { TraceEvent("TLogTerminated", self->dbgid).errorUnsuppressed(e); return true; } else @@ -3592,86 +3525,52 @@ ACTOR Future tLog(IKeyValueStore* persistentData, TraceEvent("SharedTlog", tlogId); try { - try { - wait(ioTimeoutError(persistentData->init(), SERVER_KNOBS->TLOG_MAX_CREATE_DURATION)); + wait(ioTimeoutError(persistentData->init(), SERVER_KNOBS->TLOG_MAX_CREATE_DURATION)); - if (restoreFromDisk) { - wait(restorePersistentState(&self, locality, oldLog, recovered, tlogRequests)); - } else { - wait(ioTimeoutError(checkEmptyQueue(&self) && initPersistentStorage(&self), - SERVER_KNOBS->TLOG_MAX_CREATE_DURATION)); - } + if (restoreFromDisk) { + wait(restorePersistentState(&self, locality, oldLog, recovered, tlogRequests)); + } else { + wait(ioTimeoutError(checkEmptyQueue(&self) && initPersistentStorage(&self), + SERVER_KNOBS->TLOG_MAX_CREATE_DURATION)); + } - // Disk errors need a chance to kill this actor. - wait(delay(0.000001)); + // Disk errors need a chance to kill this actor. + wait(delay(0.000001)); - if (recovered.canBeSet()) - recovered.send(Void()); + if (recovered.canBeSet()) + recovered.send(Void()); - // if (!self.durableClusterId.isValid()) { - // self.sharedActors.send(updateDurableClusterID(&self)); - // } - self.sharedActors.send(commitQueue(&self)); - self.sharedActors.send(updateStorageLoop(&self)); - self.sharedActors.send(traceRole(Role::SHARED_TRANSACTION_LOG, tlogId)); - state Future activeSharedChange = Void(); + self.sharedActors.send(commitQueue(&self)); + self.sharedActors.send(updateStorageLoop(&self)); + self.sharedActors.send(traceRole(Role::SHARED_TRANSACTION_LOG, tlogId)); + state Future activeSharedChange = Void(); - loop { - choose { - when(state InitializeTLogRequest req = waitNext(tlogRequests.getFuture())) { - if (!self.tlogCache.exists(req.recruitmentID)) { - self.tlogCache.set(req.recruitmentID, req.reply.getFuture()); - self.sharedActors.send( - self.tlogCache.removeOnReady(req.recruitmentID, tLogStart(&self, req, locality))); - } else { - forwardPromise(req.reply, self.tlogCache.get(req.recruitmentID)); - } - } - when(wait(error)) { throw internal_error(); } - when(wait(activeSharedChange)) { - if (activeSharedTLog->get() == tlogId) { - TraceEvent("SharedTLogNowActive", self.dbgid).detail("NowActive", activeSharedTLog->get()); - self.targetVolatileBytes = SERVER_KNOBS->TLOG_SPILL_THRESHOLD; - } else { - stopAllTLogs(&self, tlogId); - TraceEvent("SharedTLogQueueSpilling", self.dbgid) - .detail("NowActive", activeSharedTLog->get()); - self.sharedActors.send(startSpillingInTenSeconds(&self, tlogId, activeSharedTLog)); - } - activeSharedChange = activeSharedTLog->onChange(); + loop { + choose { + when(state InitializeTLogRequest req = waitNext(tlogRequests.getFuture())) { + if (!self.tlogCache.exists(req.recruitmentID)) { + self.tlogCache.set(req.recruitmentID, req.reply.getFuture()); + self.sharedActors.send( + self.tlogCache.removeOnReady(req.recruitmentID, tLogStart(&self, req, locality))); + } else { + forwardPromise(req.reply, self.tlogCache.get(req.recruitmentID)); } } + when(wait(error)) { + throw internal_error(); + } + when(wait(activeSharedChange)) { + if (activeSharedTLog->get() == tlogId) { + TraceEvent("SharedTLogNowActive", self.dbgid).detail("NowActive", activeSharedTLog->get()); + self.targetVolatileBytes = SERVER_KNOBS->TLOG_SPILL_THRESHOLD; + } else { + stopAllTLogs(&self, tlogId); + TraceEvent("SharedTLogQueueSpilling", self.dbgid).detail("NowActive", activeSharedTLog->get()); + self.sharedActors.send(startSpillingInTenSeconds(&self, tlogId, activeSharedTLog)); + } + activeSharedChange = activeSharedTLog->onChange(); + } } - } catch (Error& e) { - throw; - - // TODO: #5375 - /* - if (e.code() != error_code_worker_removed) { - throw; - } - // Don't need to worry about deleting data if there is no durable - // cluster ID. - if (!self.durableClusterId.isValid()) { - throw; - } - // When a tlog joins a new cluster and has data for an old cluster, - // it should automatically exclude itself to avoid being used in - // the new cluster. - auto recoveryState = self.dbInfo->get().recoveryState; - if (recoveryState == RecoveryState::FULLY_RECOVERED && self.ccClusterId.isValid() && - self.durableClusterId.isValid() && self.ccClusterId != self.durableClusterId) { - state NetworkAddress address = g_network->getLocalAddress(); - wait(excludeServers(self.cx, { AddressExclusion{ address.ip, address.port } })); - TraceEvent(SevWarnAlways, "TLogBelongsToExistingCluster") - .detail("ClusterId", self.durableClusterId) - .detail("NewClusterId", self.ccClusterId); - } - // If the tlog has a valid durable cluster ID, we don't want it to - // wipe its data! Throw this error to signal to `tlogTerminated` to - // close the persistent data store instead of deleting it. - throw invalid_cluster_id(); - */ } } catch (Error& e) { self.terminated.send(Void()); diff --git a/fdbserver/TagPartitionedLogSystem.actor.cpp b/fdbserver/TagPartitionedLogSystem.actor.cpp index 5b34c5b68e..87fe0dea3c 100644 --- a/fdbserver/TagPartitionedLogSystem.actor.cpp +++ b/fdbserver/TagPartitionedLogSystem.actor.cpp @@ -1635,7 +1635,6 @@ Future TagPartitionedLogSystem::endEpoch() { Future> TagPartitionedLogSystem::newEpoch( RecruitFromConfigurationReply const& recr, Future const& fRemoteWorkers, - UID clusterId, DatabaseConfiguration const& config, LogEpoch recoveryCount, Version recoveryTransactionVersion, @@ -1646,7 +1645,6 @@ Future> TagPartitionedLogSystem::newEpoch( return newEpoch(Reference::addRef(this), recr, fRemoteWorkers, - clusterId, config, recoveryCount, recoveryTransactionVersion, @@ -2546,7 +2544,6 @@ std::vector TagPartitionedLogSystem::getLocalTags(int8_t locality, const st ACTOR Future TagPartitionedLogSystem::newRemoteEpoch(TagPartitionedLogSystem* self, Reference oldLogSystem, Future fRemoteWorkers, - UID clusterId, DatabaseConfiguration configuration, LogEpoch recoveryCount, Version recoveryTransactionVersion, @@ -2690,7 +2687,6 @@ ACTOR Future TagPartitionedLogSystem::newRemoteEpoch(TagPartitionedLogSyst req.startVersion = logSet->startVersion; req.logRouterTags = 0; req.txsTags = self->txsTags; - req.clusterId = clusterId; req.recoveryTransactionVersion = recoveryTransactionVersion; } @@ -2742,7 +2738,6 @@ ACTOR Future> TagPartitionedLogSystem::newEpoch( Reference oldLogSystem, RecruitFromConfigurationReply recr, Future fRemoteWorkers, - UID clusterId, DatabaseConfiguration configuration, LogEpoch recoveryCount, Version recoveryTransactionVersion, @@ -2965,7 +2960,6 @@ ACTOR Future> TagPartitionedLogSystem::newEpoch( req.startVersion = logSystem->tLogs[0]->startVersion; req.logRouterTags = logSystem->logRouterTags; req.txsTags = logSystem->txsTags; - req.clusterId = clusterId; req.recoveryTransactionVersion = recoveryTransactionVersion; } @@ -3035,7 +3029,6 @@ ACTOR Future> TagPartitionedLogSystem::newEpoch( req.startVersion = oldLogSystem->knownCommittedVersion + 1; req.logRouterTags = logSystem->logRouterTags; req.txsTags = logSystem->txsTags; - req.clusterId = clusterId; req.recoveryTransactionVersion = recoveryTransactionVersion; } @@ -3094,7 +3087,6 @@ ACTOR Future> TagPartitionedLogSystem::newEpoch( logSystem->remoteRecovery = TagPartitionedLogSystem::newRemoteEpoch(logSystem.getPtr(), oldLogSystem, fRemoteWorkers, - clusterId, configuration, recoveryCount, recoveryTransactionVersion, diff --git a/fdbserver/include/fdbserver/DDTeamCollection.h b/fdbserver/include/fdbserver/DDTeamCollection.h index 4c3e88c0a1..975c8eb52d 100644 --- a/fdbserver/include/fdbserver/DDTeamCollection.h +++ b/fdbserver/include/fdbserver/DDTeamCollection.h @@ -468,8 +468,6 @@ class DDTeamCollection : public ReferenceCounted { bool recruitTss, Reference tssState); - Future getClusterId(); - // return the next ServerID in storageWiggler Future getNextWigglingServerID(); diff --git a/fdbserver/include/fdbserver/DDTxnProcessor.h b/fdbserver/include/fdbserver/DDTxnProcessor.h index 17373439ae..0142c95183 100644 --- a/fdbserver/include/fdbserver/DDTxnProcessor.h +++ b/fdbserver/include/fdbserver/DDTxnProcessor.h @@ -137,8 +137,6 @@ public: virtual Future> readRebalanceDDIgnoreKey() const { return {}; } - virtual Future getClusterId() const { return {}; } - virtual Future waitDDTeamInfoPrintSignal() const { return Never(); } virtual Future> getWorkers() const = 0; @@ -222,8 +220,6 @@ public: Future> readRebalanceDDIgnoreKey() const override; - Future getClusterId() const override; - Future waitDDTeamInfoPrintSignal() const override; Future> getWorkers() const override; diff --git a/fdbserver/include/fdbserver/LogSystem.h b/fdbserver/include/fdbserver/LogSystem.h index fd3f31ab6a..54c8041156 100644 --- a/fdbserver/include/fdbserver/LogSystem.h +++ b/fdbserver/include/fdbserver/LogSystem.h @@ -641,7 +641,6 @@ struct ILogSystem { virtual Future> newEpoch( RecruitFromConfigurationReply const& recr, Future const& fRemoteWorkers, - UID clusterId, DatabaseConfiguration const& config, LogEpoch recoveryCount, Version recoveryTransactionVersion, diff --git a/fdbserver/include/fdbserver/TagPartitionedLogSystem.actor.h b/fdbserver/include/fdbserver/TagPartitionedLogSystem.actor.h index c7d961cec7..c95da29445 100644 --- a/fdbserver/include/fdbserver/TagPartitionedLogSystem.actor.h +++ b/fdbserver/include/fdbserver/TagPartitionedLogSystem.actor.h @@ -269,7 +269,6 @@ struct TagPartitionedLogSystem final : ILogSystem, ReferenceCounted> newEpoch(RecruitFromConfigurationReply const& recr, Future const& fRemoteWorkers, - UID clusterId, DatabaseConfiguration const& config, LogEpoch recoveryCount, Version recoveryTransactionVersion, @@ -350,7 +349,6 @@ struct TagPartitionedLogSystem final : ILogSystem, ReferenceCounted newRemoteEpoch(TagPartitionedLogSystem* self, Reference oldLogSystem, Future fRemoteWorkers, - UID clusterId, DatabaseConfiguration configuration, LogEpoch recoveryCount, Version recoveryTransactionVersion, @@ -360,7 +358,6 @@ struct TagPartitionedLogSystem final : ILogSystem, ReferenceCounted> newEpoch(Reference oldLogSystem, RecruitFromConfigurationReply recr, Future fRemoteWorkers, - UID clusterId, DatabaseConfiguration configuration, LogEpoch recoveryCount, Version recoveryTransactionVersion, diff --git a/fdbserver/include/fdbserver/WorkerInterface.actor.h b/fdbserver/include/fdbserver/WorkerInterface.actor.h index adcff3b0ed..d1e3854baf 100644 --- a/fdbserver/include/fdbserver/WorkerInterface.actor.h +++ b/fdbserver/include/fdbserver/WorkerInterface.actor.h @@ -260,7 +260,6 @@ struct RegisterMasterRequest { std::vector priorCommittedLogServers; RecoveryState recoveryState; bool recoveryStalled; - UID clusterId; ReplyPromise reply; @@ -284,7 +283,6 @@ struct RegisterMasterRequest { priorCommittedLogServers, recoveryState, recoveryStalled, - clusterId, reply); } }; @@ -592,7 +590,6 @@ struct InitializeTLogRequest { Version startVersion; int logRouterTags; int txsTags; - UID clusterId; Version recoveryTransactionVersion; ReplyPromise reply; @@ -619,7 +616,6 @@ struct InitializeTLogRequest { logVersion, spillType, txsTags, - clusterId, recoveryTransactionVersion); } }; @@ -819,14 +815,12 @@ struct InitializeStorageRequest { KeyValueStoreType storeType; Optional> tssPairIDAndVersion; // Only set if recruiting a tss. Will be the UID and Version of its SS pair. - UID clusterId; // Unique cluster identifier. Only needed at recruitment, will be read from txnStateStore on recovery Version initialClusterVersion; ReplyPromise reply; template void serialize(Ar& ar) { - serializer( - ar, seedTag, reqId, interfaceId, storeType, reply, tssPairIDAndVersion, clusterId, initialClusterVersion); + serializer(ar, seedTag, reqId, interfaceId, storeType, reply, tssPairIDAndVersion, initialClusterVersion); } }; @@ -1148,7 +1142,6 @@ class IPageEncryptionKeyProvider; ACTOR Future storageServer(IKeyValueStore* persistentData, StorageServerInterface ssi, Tag seedTag, - UID clusterId, Version startVersion, Version tssSeedVersion, ReplyPromise recruitReply, diff --git a/fdbserver/storageserver.actor.cpp b/fdbserver/storageserver.actor.cpp index 463b9cd159..409f2a7b23 100644 --- a/fdbserver/storageserver.actor.cpp +++ b/fdbserver/storageserver.actor.cpp @@ -170,7 +170,6 @@ static const KeyRef persistID = PERSIST_PREFIX "ID"_sr; static const KeyRef persistTssPairID = PERSIST_PREFIX "tssPairID"_sr; static const KeyRef persistSSPairID = PERSIST_PREFIX "ssWithTSSPairID"_sr; static const KeyRef persistTssQuarantine = PERSIST_PREFIX "tssQ"_sr; -static const KeyRef persistClusterIdKey = PERSIST_PREFIX "clusterId"_sr; // (Potentially) change with the durable version or when fetchKeys completes static const KeyRef persistVersion = PERSIST_PREFIX "Version"_sr; @@ -974,7 +973,6 @@ public: Reference logSystem; Reference logCursor; - Promise clusterId; // The version the cluster starts on. This value is not persisted and may // not be valid after a recovery. Version initialClusterVersion = 1; @@ -9367,9 +9365,6 @@ void StorageServerDisk::makeNewStorageServerDurable(const bool shardAware) { if (data->tssPairID.present()) { storage->set(KeyValueRef(persistTssPairID, BinaryWriter::toValue(data->tssPairID.get(), Unversioned()))); } - ASSERT(data->clusterId.getFuture().isReady() && data->clusterId.getFuture().get().isValid()); - storage->set( - KeyValueRef(persistClusterIdKey, BinaryWriter::toValue(data->clusterId.getFuture().get(), Unversioned()))); storage->set(KeyValueRef(persistVersion, BinaryWriter::toValue(data->version.get(), Unversioned()))); if (shardAware) { @@ -9674,54 +9669,9 @@ ACTOR Future restoreByteSample(StorageServer* data, return Void(); } -// Reads the cluster ID from the transaction state store. -ACTOR Future getClusterId(StorageServer* self) { - state ReadYourWritesTransaction tr(self->cx); - loop { - try { - self->cx->invalidateCache(Key(), systemKeys); - tr.setOption(FDBTransactionOptions::ACCESS_SYSTEM_KEYS); - tr.setOption(FDBTransactionOptions::PRIORITY_SYSTEM_IMMEDIATE); - tr.setOption(FDBTransactionOptions::LOCK_AWARE); - Optional clusterId = wait(tr.get(clusterIdKey)); - ASSERT(clusterId.present()); - return BinaryReader::fromStringRef(clusterId.get(), Unversioned()); - } catch (Error& e) { - wait(tr.onError(e)); - } - } -} - -// Read the cluster ID from the transaction state store and persist it to local -// storage. This function should only be necessary during an upgrade when the -// prior FDB version did not support cluster IDs. The normal path for storage -// server recruitment will include the cluster ID in the initial recruitment -// message. -ACTOR Future persistClusterId(StorageServer* self) { - state Transaction tr(self->cx); - loop { - try { - Optional clusterId = wait(tr.get(clusterIdKey)); - if (clusterId.present()) { - auto uid = BinaryReader::fromStringRef(clusterId.get(), Unversioned()); - self->storage.writeKeyValue( - KeyValueRef(persistClusterIdKey, BinaryWriter::toValue(uid, Unversioned()))); - // Purposely not calling commit here, and letting the recurring - // commit handle save this value to disk - self->clusterId.send(uid); - } - break; - } catch (Error& e) { - wait(tr.onError(e)); - } - } - return Void(); -} - ACTOR Future restoreDurableState(StorageServer* data, IKeyValueStore* storage) { state Future> fFormat = storage->readValue(persistFormat.key); state Future> fID = storage->readValue(persistID); - state Future> fClusterID = storage->readValue(persistClusterIdKey); state Future> ftssPairID = storage->readValue(persistTssPairID); state Future> fssPairID = storage->readValue(persistSSPairID); state Future> fTssQuarantine = storage->readValue(persistTssQuarantine); @@ -9742,8 +9692,8 @@ ACTOR Future restoreDurableState(StorageServer* data, IKeyValueStore* stor restoreByteSample(data, storage, byteSampleSampleRecovered, startByteSampleRestore.getFuture()); TraceEvent("ReadingDurableState", data->thisServerID).log(); - wait(waitForAll(std::vector{ - fFormat, fID, fClusterID, ftssPairID, fssPairID, fTssQuarantine, fVersion, fLogProtocol, fPrimaryLocality })); + wait(waitForAll( + std::vector{ fFormat, fID, ftssPairID, fssPairID, fTssQuarantine, fVersion, fLogProtocol, fPrimaryLocality })); wait(waitForAll(std::vector{ fShardAssigned, fShardAvailable, fChangeFeeds, @@ -9784,14 +9734,6 @@ ACTOR Future restoreDurableState(StorageServer* data, IKeyValueStore* stor data->bytesRestored += fssPairID.get().expectedSize(); } - if (fClusterID.get().present()) { - data->clusterId.send(BinaryReader::fromStringRef(fClusterID.get().get(), Unversioned())); - data->bytesRestored += fClusterID.get().expectedSize(); - } else { - CODE_PROBE(true, "storage server upgraded to version supporting cluster IDs"); - data->actors.add(persistClusterId(data)); - } - // It's a bit sketchy to rely on an untrusted storage engine to persist its quarantine state when the quarantine // state means the storage engine already had a durability or correctness error, but it should get // re-quarantined very quickly because of a mismatch if it starts trying to do things again @@ -10952,7 +10894,6 @@ ACTOR Future storageInterfaceRegistration(StorageServer* self, ACTOR Future storageServer(IKeyValueStore* persistentData, StorageServerInterface ssi, Tag seedTag, - UID clusterId, Version startVersion, Version tssSeedVersion, ReplyPromise recruitReply, @@ -10962,7 +10903,6 @@ ACTOR Future storageServer(IKeyValueStore* persistentData, state StorageServer self(persistentData, db, ssi, encryptionKeyProvider); self.shardAware = SERVER_KNOBS->SHARD_ENCODE_LOCATION_METADATA && persistentData->shardAware(); state Future ssCore; - self.clusterId.send(clusterId); self.initialClusterVersion = startVersion; if (ssi.isTss()) { self.setTssPair(ssi.tssPairID.get()); @@ -11109,32 +11049,7 @@ ACTOR Future storageServer(IKeyValueStore* persistentData, wait(delay(0)); ErrorOr e = wait(errorOr(f)); if (e.isError()) { - Error e = f.getError(); - - throw e; - // TODO: #5375 - /* - if (e.code() != error_code_worker_removed) { - throw e; - } - state UID clusterId = wait(getClusterId(&self)); - ASSERT(self.clusterId.isValid()); - UID durableClusterId = wait(self.clusterId.getFuture()); - ASSERT(durableClusterId.isValid()); - if (clusterId == durableClusterId) { - throw worker_removed(); - } - // When a storage server connects to a new cluster, it deletes its - // old data and creates a new, empty data file for the new cluster. - // We want to avoid this and force a manual removal of the storage - // servers' old data when being assigned to a new cluster to avoid - // accidental data loss. - TraceEvent(SevWarn, "StorageServerBelongsToExistingCluster") - .detail("ServerID", ssi.id()) - .detail("ClusterID", durableClusterId) - .detail("NewClusterID", clusterId); - wait(Future(Never())); - */ + throw f.getError(); } self.interfaceRegistered = diff --git a/fdbserver/worker.actor.cpp b/fdbserver/worker.actor.cpp index a98999589f..234081509f 100644 --- a/fdbserver/worker.actor.cpp +++ b/fdbserver/worker.actor.cpp @@ -2060,7 +2060,7 @@ ACTOR Future workerServer(Reference connRecord, } loop choose { - when(state UpdateServerDBInfoRequest req = waitNext(interf.updateServerDBInfo.getFuture())) { + when(UpdateServerDBInfoRequest req = waitNext(interf.updateServerDBInfo.getFuture())) { ServerDBInfo localInfo = BinaryReader::fromStringRef( req.serializedDbInfo, AssumeVersion(g_network->protocolVersion())); localInfo.myLocality = locality; @@ -2482,7 +2482,6 @@ ACTOR Future workerServer(Reference connRecord, runningStorages.end(), [&req](const auto& p) { return p.second != req.storeType; }) || req.seedTag != invalidTag)) { - ASSERT(req.clusterId.isValid()); ASSERT(req.initialClusterVersion >= 0); LocalLineage _; getCurrentLineage()->modify(&RoleLineage::role) = ProcessClass::ClusterRole::Storage; @@ -2549,7 +2548,6 @@ ACTOR Future workerServer(Reference connRecord, Future s = storageServer(data, recruited, req.seedTag, - req.clusterId, req.initialClusterVersion, isTss ? req.tssPairIDAndVersion.get().second : 0, storageReady, From 02bc5edbf8f0789f8abc830cfb1b763a8e84581d Mon Sep 17 00:00:00 2001 From: Lukas Joswiak Date: Tue, 18 Oct 2022 18:39:51 -0700 Subject: [PATCH 74/95] Avoid blocking in choose when --- fdbserver/worker.actor.cpp | 19 +++++++++++++++---- 1 file changed, 15 insertions(+), 4 deletions(-) diff --git a/fdbserver/worker.actor.cpp b/fdbserver/worker.actor.cpp index 234081509f..f805183255 100644 --- a/fdbserver/worker.actor.cpp +++ b/fdbserver/worker.actor.cpp @@ -1686,6 +1686,16 @@ ACTOR Future createClusterIdFile(std::string folder, UID clusterId) { } } +// Updates this processes cluster ID based off the cluster ID received in the +// ServerDBInfo. Persists the cluster ID to disk if it does not already exist. +ACTOR Future updateClusterId(UID ccClusterId, Reference>> clusterId, std::string folder) { + if (!clusterId->get().present() && ccClusterId.isValid()) { + wait(createClusterIdFile(folder, ccClusterId)); + clusterId->set(ccClusterId); + } + return Void(); +} + ACTOR Future workerServer(Reference connRecord, Reference> const> ccInterface, LocalityData locality, @@ -1753,6 +1763,8 @@ ACTOR Future workerServer(Reference connRecord, state Reference>> issues(new AsyncVar>()); + state Future updateClusterIdFuture; + if (FLOW_KNOBS->ENABLE_CHAOS_FEATURES) { TraceEvent(SevInfo, "ChaosFeaturesEnabled"); chaosMetricsActor = chaosMetricsLogger(); @@ -2097,10 +2109,9 @@ ACTOR Future workerServer(Reference connRecord, errorForwarders.add( success(broadcastDBInfoRequest(req, SERVER_KNOBS->DBINFO_SEND_AMOUNT, notUpdated, true))); - if (!clusterId->get().present() && localInfo.client.clusterId.isValid()) { - state UID tmpClusterId = localInfo.client.clusterId; - wait(createClusterIdFile(folder, tmpClusterId)); - clusterId->set(tmpClusterId); + if (!updateClusterIdFuture.isValid() && !clusterId->get().present() && + localInfo.client.clusterId.isValid()) { + updateClusterIdFuture = updateClusterId(localInfo.client.clusterId, clusterId, folder); } } } From a8f8757f77db84850e6936425489a3c63cdc9997 Mon Sep 17 00:00:00 2001 From: Lukas Joswiak Date: Tue, 18 Oct 2022 21:18:17 -0700 Subject: [PATCH 75/95] Rename cluster ID key In FDB 7.1, this key was stored in the txnStateStore. In 7.2, it has been moved to the database. This was causing protocol compatibility issues during upgrades, so we need to rename the key. --- fdbclient/SystemData.cpp | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/fdbclient/SystemData.cpp b/fdbclient/SystemData.cpp index 19c23b1ad7..39aca67c2e 100644 --- a/fdbclient/SystemData.cpp +++ b/fdbclient/SystemData.cpp @@ -1072,8 +1072,10 @@ const KeyRangeRef timeKeeperPrefixRange("\xff\x02/timeKeeper/map/"_sr, "\xff\x02 const KeyRef timeKeeperVersionKey = "\xff\x02/timeKeeper/version"_sr; const KeyRef timeKeeperDisableKey = "\xff\x02/timeKeeper/disable"_sr; -// Durable cluster ID key -const KeyRef clusterIdKey = "\xff/clusterId"_sr; +// Durable cluster ID key. Added "Key" to the end to differentiate from the key +// "\xff/clusterId" which was stored in the txnStateStore in FDB 7.1, whereas +// this key is stored in the database in 7.2+. +const KeyRef clusterIdKey = "\xff/clusterIdKey"_sr; // Backup Log Mutation constant variables const KeyRef backupEnabledKey = "\xff/backupEnabled"_sr; From 28540e5962cfedd939a58ba1297e7d016144f1e7 Mon Sep 17 00:00:00 2001 From: Lukas Joswiak Date: Tue, 18 Oct 2022 21:56:28 -0700 Subject: [PATCH 76/95] Format --- fdbserver/TLogServer.actor.cpp | 4 +--- fdbserver/worker.actor.cpp | 4 +--- 2 files changed, 2 insertions(+), 6 deletions(-) diff --git a/fdbserver/TLogServer.actor.cpp b/fdbserver/TLogServer.actor.cpp index 56480b2e35..5bd4433657 100644 --- a/fdbserver/TLogServer.actor.cpp +++ b/fdbserver/TLogServer.actor.cpp @@ -3556,9 +3556,7 @@ ACTOR Future tLog(IKeyValueStore* persistentData, forwardPromise(req.reply, self.tlogCache.get(req.recruitmentID)); } } - when(wait(error)) { - throw internal_error(); - } + when(wait(error)) { throw internal_error(); } when(wait(activeSharedChange)) { if (activeSharedTLog->get() == tlogId) { TraceEvent("SharedTLogNowActive", self.dbgid).detail("NowActive", activeSharedTLog->get()); diff --git a/fdbserver/worker.actor.cpp b/fdbserver/worker.actor.cpp index f805183255..0bd2bcc507 100644 --- a/fdbserver/worker.actor.cpp +++ b/fdbserver/worker.actor.cpp @@ -686,9 +686,7 @@ ACTOR Future registrationClient( when(wait(FlowTransport::transport().onIncompatibleChanged())) { break; } when(wait(issues->onChange())) { break; } when(wait(recovered)) { break; } - when(wait(clusterId->onChange())) { - break; - } + when(wait(clusterId->onChange())) { break; } } } } From 91146a03f06dbba4c6f0769f70abc3800c2edad3 Mon Sep 17 00:00:00 2001 From: Lukas Joswiak Date: Mon, 24 Oct 2022 14:11:23 -0700 Subject: [PATCH 77/95] Write cluster ID to `ClientDBInfo` This enables clients to receive the cluster ID. --- fdbserver/ClusterController.actor.cpp | 9 +++++++++ 1 file changed, 9 insertions(+) diff --git a/fdbserver/ClusterController.actor.cpp b/fdbserver/ClusterController.actor.cpp index 9780881b7d..8da129af74 100644 --- a/fdbserver/ClusterController.actor.cpp +++ b/fdbserver/ClusterController.actor.cpp @@ -1121,6 +1121,7 @@ void clusterRegisterMaster(ClusterControllerData* self, RegisterMasterRequest co db->clientInfo->get().grvProxies != req.grvProxies || db->clientInfo->get().tenantMode != db->config.tenantMode || db->clientInfo->get().isEncryptionEnabled != SERVER_KNOBS->ENABLE_ENCRYPTION || + db->clientInfo->get().clusterId != db->serverInfo->get().client.clusterId || db->clientInfo->get().clusterType != db->clusterType || db->clientInfo->get().metaclusterName != db->metaclusterName || db->clientInfo->get().encryptKeyProxy != db->serverInfo->get().encryptKeyProxy) { @@ -1133,6 +1134,8 @@ void clusterRegisterMaster(ClusterControllerData* self, RegisterMasterRequest co .detail("TenantMode", db->clientInfo->get().tenantMode.toString()) .detail("ReqTenantMode", db->config.tenantMode.toString()) .detail("EncryptionEnabled", SERVER_KNOBS->ENABLE_ENCRYPTION) + .detail("ClusterId", db->serverInfo->get().client.clusterId) + .detail("ClientClusterId", db->clientInfo->get().clusterId) .detail("ClusterType", db->clientInfo->get().clusterType) .detail("ReqClusterType", db->clusterType) .detail("MetaclusterName", db->clientInfo->get().metaclusterName) @@ -1146,6 +1149,7 @@ void clusterRegisterMaster(ClusterControllerData* self, RegisterMasterRequest co clientInfo.commitProxies = req.commitProxies; clientInfo.grvProxies = req.grvProxies; clientInfo.tenantMode = TenantAPI::tenantModeForClusterType(db->clusterType, db->config.tenantMode); + clientInfo.clusterId = db->serverInfo->get().client.clusterId; clientInfo.clusterType = db->clusterType; clientInfo.metaclusterName = db->metaclusterName; db->clientInfo->set(clientInfo); @@ -3008,6 +3012,11 @@ ACTOR Future updateClusterId(ClusterControllerData* self) { serverInfo.id = deterministicRandom()->randomUniqueID(); serverInfo.client.clusterId = durableClusterId.get(); self->db.serverInfo->set(serverInfo); + + ClientDBInfo clientInfo = self->db.clientInfo->get(); + clientInfo.id = deterministicRandom()->randomUniqueID(); + clientInfo.clusterId = durableClusterId.get(); + self->db.clientInfo->set(clientInfo); } return Void(); } catch (Error& e) { From 8e76621653fb296a8c5732c181031fc02e7cde4d Mon Sep 17 00:00:00 2001 From: Lukas Joswiak Date: Mon, 24 Oct 2022 14:25:10 -0700 Subject: [PATCH 78/95] Disable shared state updates on configuration database --- fdbclient/MultiVersionTransaction.actor.cpp | 7 +++++-- fdbclient/include/fdbclient/MultiVersionTransaction.h | 1 + 2 files changed, 6 insertions(+), 2 deletions(-) diff --git a/fdbclient/MultiVersionTransaction.actor.cpp b/fdbclient/MultiVersionTransaction.actor.cpp index 8521df430c..816b984663 100644 --- a/fdbclient/MultiVersionTransaction.actor.cpp +++ b/fdbclient/MultiVersionTransaction.actor.cpp @@ -1863,6 +1863,9 @@ void MultiVersionDatabase::setOption(FDBDatabaseOptions::Option option, Optional TraceEvent("UnknownDatabaseOption").detail("Option", option); throw invalid_option(); } + if (itr->first == FDBDatabaseOptions::USE_CONFIG_DATABASE) { + dbState->isConfigDB = true; + } int defaultFor = itr->second.defaultFor; if (defaultFor >= 0) { @@ -1969,7 +1972,7 @@ ThreadFuture MultiVersionDatabase::getServerProtocol(Optional

versionMonitorDb) : dbVar(new ThreadSafeAsyncVar>(Reference(nullptr))), - connectionRecord(connectionRecord), versionMonitorDb(versionMonitorDb), closed(false) {} + connectionRecord(connectionRecord), versionMonitorDb(versionMonitorDb), closed(false), isConfigDB(false) {} // Adds a client (local or externally loaded) that can be used to connect to the cluster void MultiVersionDatabase::DatabaseState::addClient(Reference client) { @@ -2168,7 +2171,7 @@ void MultiVersionDatabase::DatabaseState::updateDatabase(Reference ne } } if (db.isValid() && dbProtocolVersion.present() && - MultiVersionApi::api->getApiVersion().hasClusterSharedStateMap()) { + MultiVersionApi::api->getApiVersion().hasClusterSharedStateMap() && !isConfigDB) { Future updateResult = MultiVersionApi::api->updateClusterSharedStateMap(connectionRecord, dbProtocolVersion.get(), db); sharedStateUpdater = map(errorOr(updateResult), [this](ErrorOr result) { diff --git a/fdbclient/include/fdbclient/MultiVersionTransaction.h b/fdbclient/include/fdbclient/MultiVersionTransaction.h index 74eebdf690..cb7e65a4e8 100644 --- a/fdbclient/include/fdbclient/MultiVersionTransaction.h +++ b/fdbclient/include/fdbclient/MultiVersionTransaction.h @@ -1024,6 +1024,7 @@ public: ThreadFuture protocolVersionMonitor; Future sharedStateUpdater; + bool isConfigDB; // Versions older than 6.1 do not benefit from having their database connections closed. Additionally, // there are various issues that result in negative behavior in some cases if the connections are closed. From 9625efd5b9561e0355ca161c3fc76c7b8cf6aed5 Mon Sep 17 00:00:00 2001 From: Lukas Joswiak Date: Thu, 27 Oct 2022 09:11:38 -0700 Subject: [PATCH 79/95] Add comment about configuration database --- fdbclient/MultiVersionTransaction.actor.cpp | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/fdbclient/MultiVersionTransaction.actor.cpp b/fdbclient/MultiVersionTransaction.actor.cpp index 816b984663..ff5639c717 100644 --- a/fdbclient/MultiVersionTransaction.actor.cpp +++ b/fdbclient/MultiVersionTransaction.actor.cpp @@ -2170,6 +2170,10 @@ void MultiVersionDatabase::DatabaseState::updateDatabase(Reference ne .detail("ConnectionRecord", connectionRecord); } } + // Verify the database has the necessary functionality to update the shared + // state. Avoid updating the shared state if the database is a + // configuration database, because a configuration database does not have + // access to typical system keys and does not need to be updated. if (db.isValid() && dbProtocolVersion.present() && MultiVersionApi::api->getApiVersion().hasClusterSharedStateMap() && !isConfigDB) { Future updateResult = From 848e80c08b952e48f245919c252bb7b290d9fece Mon Sep 17 00:00:00 2001 From: Steve Atherton Date: Thu, 27 Oct 2022 17:52:56 -0700 Subject: [PATCH 80/95] Add comment. --- 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 6a920d07c3..2358312a4a 100644 --- a/fdbclient/include/fdbclient/StorageServerInterface.h +++ b/fdbclient/include/fdbclient/StorageServerInterface.h @@ -890,7 +890,7 @@ struct ChangeFeedStreamRequest { KeyRange range; int replyBufferSize = -1; bool canReadPopped = true; - UID id; + UID id; // This must be globally unique among ChangeFeedStreamRequest instances Optional options; ReplyPromiseStream reply; From 67a76db9bd40b37cc5cc3430dc9ab6b786b15166 Mon Sep 17 00:00:00 2001 From: Steve Atherton Date: Thu, 27 Oct 2022 17:54:33 -0700 Subject: [PATCH 81/95] Bug fix, FlowLock releaser created too late. --- fdbserver/storageserver.actor.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/fdbserver/storageserver.actor.cpp b/fdbserver/storageserver.actor.cpp index b9311e3240..8a9f38f125 100644 --- a/fdbserver/storageserver.actor.cpp +++ b/fdbserver/storageserver.actor.cpp @@ -2689,8 +2689,8 @@ ACTOR Future> getChangeFeedMutations(Stor } wait(data->changeFeedDiskReadsLock.take(TaskPriority::DefaultYield)); - state PriorityMultiLock::Lock ssReadLock = wait(data->getReadLock(req.options)); state FlowLock::Releaser holdingDiskReadsLock(data->changeFeedDiskReadsLock); + state PriorityMultiLock::Lock ssReadLock = wait(data->getReadLock(req.options)); RangeResult res = wait( data->storage.readRange(KeyRangeRef(changeFeedDurableKey(req.rangeID, std::max(req.begin, emptyVersion)), changeFeedDurableKey(req.rangeID, req.end)), From 1dad43cb066f92cc7a7dd067750136d6e1956846 Mon Sep 17 00:00:00 2001 From: Steve Atherton Date: Thu, 27 Oct 2022 18:03:14 -0700 Subject: [PATCH 82/95] Remove unnecessary change feed disk read lock as its functionality is obsoleted by the storage server read priority lock. --- fdbclient/ServerKnobs.cpp | 1 - fdbclient/include/fdbclient/ServerKnobs.h | 1 - fdbserver/storageserver.actor.cpp | 9 --------- 3 files changed, 11 deletions(-) diff --git a/fdbclient/ServerKnobs.cpp b/fdbclient/ServerKnobs.cpp index a44fc093d6..d61503d60e 100644 --- a/fdbclient/ServerKnobs.cpp +++ b/fdbclient/ServerKnobs.cpp @@ -762,7 +762,6 @@ void ServerKnobs::initialize(Randomize randomize, ClientKnobs* clientKnobs, IsSi init( FETCH_KEYS_LOWER_PRIORITY, 0 ); init( SERVE_FETCH_CHECKPOINT_PARALLELISM, 4 ); init( SERVE_AUDIT_STORAGE_PARALLELISM, 2 ); - init( CHANGE_FEED_DISK_READS_PARALLELISM, 1000 ); if( randomize && BUGGIFY ) CHANGE_FEED_DISK_READS_PARALLELISM = 20; 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 bf98b9a493..7305dbf820 100644 --- a/fdbclient/include/fdbclient/ServerKnobs.h +++ b/fdbclient/include/fdbclient/ServerKnobs.h @@ -717,7 +717,6 @@ public: int FETCH_KEYS_LOWER_PRIORITY; int SERVE_FETCH_CHECKPOINT_PARALLELISM; int SERVE_AUDIT_STORAGE_PARALLELISM; - int CHANGE_FEED_DISK_READS_PARALLELISM; int BUGGIFY_BLOCK_BYTES; int64_t STORAGE_RECOVERY_VERSION_LAG_LIMIT; double STORAGE_DURABILITY_LAG_REJECT_THRESHOLD; diff --git a/fdbserver/storageserver.actor.cpp b/fdbserver/storageserver.actor.cpp index 9f0bd7a9da..6054430798 100644 --- a/fdbserver/storageserver.actor.cpp +++ b/fdbserver/storageserver.actor.cpp @@ -1009,7 +1009,6 @@ public: // Extra lock that prevents too much post-initial-fetch work from building up, such as mutation applying and change // feed tail fetching FlowLock fetchKeysParallelismFullLock; - FlowLock changeFeedDiskReadsLock; int64_t fetchKeysBytesBudget; AsyncVar fetchKeysBudgetUsed; std::vector> readyFetchKeys; @@ -1242,10 +1241,6 @@ public: specialCounter(cc, "ServeValidateStorageWaiting", [self]() { return self->serveAuditStorageParallelismLock.waiters(); }); - specialCounter( - cc, "ChangeFeedDiskReadsActive", [self]() { return self->changeFeedDiskReadsLock.activePermits(); }); - specialCounter( - cc, "ChangeFeedDiskReadsWaiting", [self]() { return self->changeFeedDiskReadsLock.waiters(); }); specialCounter(cc, "QueryQueueMax", [self]() { return self->getAndResetMaxQueryQueueSize(); }); specialCounter(cc, "BytesStored", [self]() { return self->metrics.byteSample.getEstimate(allKeys); }); specialCounter(cc, "ActiveWatches", [self]() { return self->numWatches; }); @@ -1305,7 +1300,6 @@ public: noRecentUpdates(false), lastUpdate(now()), updateEagerReads(nullptr), fetchKeysParallelismLock(SERVER_KNOBS->FETCH_KEYS_PARALLELISM), fetchKeysParallelismFullLock(SERVER_KNOBS->FETCH_KEYS_PARALLELISM_FULL), - changeFeedDiskReadsLock(SERVER_KNOBS->CHANGE_FEED_DISK_READS_PARALLELISM), fetchKeysBytesBudget(SERVER_KNOBS->STORAGE_FETCH_BYTES), fetchKeysBudgetUsed(false), serveFetchCheckpointParallelismLock(SERVER_KNOBS->SERVE_FETCH_CHECKPOINT_PARALLELISM), ssLock(SERVER_KNOBS->STORAGE_SERVER_READ_CONCURRENCY, SERVER_KNOBS->STORAGESERVER_READ_PRIORITIES), @@ -2694,8 +2688,6 @@ ACTOR Future> getChangeFeedMutations(Stor wait(delay(0)); } - wait(data->changeFeedDiskReadsLock.take(TaskPriority::DefaultYield)); - state FlowLock::Releaser holdingDiskReadsLock(data->changeFeedDiskReadsLock); state PriorityMultiLock::Lock ssReadLock = wait(data->getReadLock(req.options)); RangeResult res = wait( data->storage.readRange(KeyRangeRef(changeFeedDurableKey(req.rangeID, std::max(req.begin, emptyVersion)), @@ -2704,7 +2696,6 @@ ACTOR Future> getChangeFeedMutations(Stor remainingDurableBytes, req.options)); ssReadLock.release(); - holdingDiskReadsLock.release(); data->counters.kvScanBytes += res.logicalSize(); ++data->counters.changeFeedDiskReads; From e7fd3eda00403fa9235cc66051d6e22df8080ed0 Mon Sep 17 00:00:00 2001 From: Jingyu Zhou Date: Thu, 27 Oct 2022 19:46:05 -0700 Subject: [PATCH 83/95] Revert "Update fdbclient/NativeAPI.actor.cpp" This reverts commit 812243bafab4b8cb9cad49c7c22f16063f39b37e. --- fdbclient/NativeAPI.actor.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/fdbclient/NativeAPI.actor.cpp b/fdbclient/NativeAPI.actor.cpp index ffe8c4b89e..87fbb10a2c 100644 --- a/fdbclient/NativeAPI.actor.cpp +++ b/fdbclient/NativeAPI.actor.cpp @@ -3908,7 +3908,7 @@ class WatchRefCountUpdater { Version version; public: - WatchRefCountUpdater() = default; + WatchRefCountUpdater() {} WatchRefCountUpdater(const Database& cx_, const int64_t tenantID_, KeyRef key_, const Version& ver) : cx(cx_), tenantID(tenantID_), key(key_), version(ver) {} From e460933b524e07c4347eba75b0f99726c2d5efcb Mon Sep 17 00:00:00 2001 From: Jingyu Zhou Date: Thu, 27 Oct 2022 19:46:05 -0700 Subject: [PATCH 84/95] Revert "Remove debugging output" This reverts commit 41d1d6404d933f0574d88d1fa2a68c642413bf4b. --- fdbclient/NativeAPI.actor.cpp | 29 +++++++++++++++++++---------- 1 file changed, 19 insertions(+), 10 deletions(-) diff --git a/fdbclient/NativeAPI.actor.cpp b/fdbclient/NativeAPI.actor.cpp index 87fbb10a2c..f550c63afb 100644 --- a/fdbclient/NativeAPI.actor.cpp +++ b/fdbclient/NativeAPI.actor.cpp @@ -5426,17 +5426,26 @@ ACTOR Future restartWatch(Database cx, tenantInfo.tenantId = locationInfo.tenantEntry.id; } - wait(watchValueMap(cx->minAcceptableReadVersion, - tenantInfo, - key, - value, - cx, - tags, - spanContext, - taskID, - debugID, - useProvisionalProxies)); + if (key == "anotherKey"_sr) + std::cout << cx->dbId.toString() << " restartWatch" << std::endl; + try { + wait(watchValueMap(cx->minAcceptableReadVersion, + tenantInfo, + key, + value, + cx, + tags, + spanContext, + taskID, + debugID, + useProvisionalProxies)); + } catch (Error& err) { + std::cout << cx->dbId.toString() << " restartWatch fail " << err.code() << std::endl; + return Void(); + } + + std::cout << cx->dbId.toString() << " restartWatch pass" << std::endl; return Void(); } From 19ae4e7eb74985c278874f719b5357c25b9e5df1 Mon Sep 17 00:00:00 2001 From: Jingyu Zhou Date: Thu, 27 Oct 2022 19:46:05 -0700 Subject: [PATCH 85/95] Revert "Reformat source" This reverts commit ec47c261bf743e4ffefbea2e70641afdf8f16491. --- fdbclient/include/fdbclient/DatabaseContext.h | 2 ++ 1 file changed, 2 insertions(+) diff --git a/fdbclient/include/fdbclient/DatabaseContext.h b/fdbclient/include/fdbclient/DatabaseContext.h index 827c5da291..a0d065bbfd 100644 --- a/fdbclient/include/fdbclient/DatabaseContext.h +++ b/fdbclient/include/fdbclient/DatabaseContext.h @@ -156,6 +156,8 @@ public: WatchMetadata(Reference parameters) : watchFuture(watchPromise.getFuture()), parameters(parameters) {} + + ~WatchMetadata() { /*watchFutureSS.cancel();*/ } }; struct MutationAndVersionStream { From 634bd529e7b6f484c8840f782df63f57752075c4 Mon Sep 17 00:00:00 2001 From: Jingyu Zhou Date: Thu, 27 Oct 2022 19:46:05 -0700 Subject: [PATCH 86/95] Revert "Record the version of each watch" This reverts commit 4bd24e4d6460c5cf38117b89246561bb0d83e3ef. --- fdbclient/DatabaseContext.cpp | 82 ------------ fdbclient/NativeAPI.actor.cpp | 121 ++++++++++++------ fdbclient/include/fdbclient/DatabaseContext.h | 18 +-- flow/include/flow/flow.h | 14 +- 4 files changed, 94 insertions(+), 141 deletions(-) delete mode 100644 fdbclient/DatabaseContext.cpp diff --git a/fdbclient/DatabaseContext.cpp b/fdbclient/DatabaseContext.cpp deleted file mode 100644 index 2ba912c9de..0000000000 --- a/fdbclient/DatabaseContext.cpp +++ /dev/null @@ -1,82 +0,0 @@ -/* - * DatabaseContext.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/DatabaseContext.h" - -Reference DatabaseContext::getWatchMetadata(int64_t tenantId, KeyRef key) const { - const auto it = watchMap.find(std::make_pair(tenantId, key)); - if (it == watchMap.end()) - return Reference(); - return it->second; -} - -void DatabaseContext::setWatchMetadata(Reference metadata) { - const WatchMapKey key(metadata->parameters->tenant.tenantId, metadata->parameters->key); - watchMap[key] = metadata; - // NOTE Here we do *NOT* update/reset the reference count for the key, see the source code in getWatchFuture. - // Basically the reference count could be increased, or the same watch is refreshed, or the watch might be cancelled -} - -int32_t DatabaseContext::increaseWatchRefCount(const int64_t tenantID, KeyRef key, const Version& version) { - const WatchMapKey mapKey(tenantID, key); - watchCounterMap[mapKey].insert(version); - return watchCounterMap[mapKey].size(); -} - -int32_t DatabaseContext::decreaseWatchRefCount(const int64_t tenantID, KeyRef key, const Version& version) { - const WatchMapKey mapKey(tenantID, key); - auto mapKeyIter = watchCounterMap.find(mapKey); - if (mapKeyIter == std::end(watchCounterMap)) { - // Key does not exist. The metadata might be removed by deleteWatchMetadata already. - return 0; - } - - auto& versionSet = mapKeyIter->second; - auto versionIter = versionSet.find(version); - - if (versionIter == std::end(versionSet)) { - // Version not found, the watch might be cleared before. - return versionSet.size(); - } - versionSet.erase(versionIter); - - const auto count = versionSet.size(); - // The metadata might be deleted somewhere else, before calling this decreaseWatchRefCount - if (auto metadata = getWatchMetadata(tenantID, key); metadata.isValid() && versionSet.size() == 0) { - // It is a *must* to cancel the watchFutureSS manually. watchFutureSS waits for watchStorageServerResp, which - // holds a reference to the metadata. If the ACTOR is not cancelled, it indirectly holds a Future waiting for - // itself. - metadata->watchFutureSS.cancel(); - deleteWatchMetadata(tenantID, key); - } - - return count; -} - -void DatabaseContext::deleteWatchMetadata(int64_t tenantId, KeyRef key) { - const WatchMapKey mapKey(tenantId, key); - watchMap.erase(mapKey); - watchCounterMap.erase(mapKey); -} - -void DatabaseContext::clearWatchMetadata() { - watchMap.clear(); - watchCounterMap.clear(); -} diff --git a/fdbclient/NativeAPI.actor.cpp b/fdbclient/NativeAPI.actor.cpp index f550c63afb..fcef80270a 100644 --- a/fdbclient/NativeAPI.actor.cpp +++ b/fdbclient/NativeAPI.actor.cpp @@ -2382,6 +2382,54 @@ Database Database::createSimulatedExtraDatabase(std::string connectionString, Op return db; } +Reference DatabaseContext::getWatchMetadata(int64_t tenantId, KeyRef key) const { + const auto it = watchMap.find(std::make_pair(tenantId, key)); + if (it == watchMap.end()) + return Reference(); + return it->second; +} + +void DatabaseContext::setWatchMetadata(Reference metadata) { + const WatchMapKey key(metadata->parameters->tenant.tenantId, metadata->parameters->key); + watchMap[key] = metadata; + // NOTE Here we do *NOT* update/reset the reference count for the key, see the source code in getWatchFuture +} + +int32_t DatabaseContext::increaseWatchRefCount(const int64_t tenantID, KeyRef key) { + const WatchMapKey mapKey(tenantID, key); + if (watchCounterMap.count(mapKey) == 0) { + watchCounterMap[mapKey] = 0; + } + const auto count = ++watchCounterMap[mapKey]; + return count; +} + +int32_t DatabaseContext::decreaseWatchRefCount(const int64_t tenantID, KeyRef key) { + const WatchMapKey mapKey(tenantID, key); + if (watchCounterMap.count(mapKey) == 0) { + // Key does not exist. The metadata might be removed by deleteWatchMetadata already. + return 0; + } + const auto count = --watchCounterMap[mapKey]; + ASSERT(watchCounterMap[mapKey] >= 0); + if (watchCounterMap[mapKey] == 0) { + getWatchMetadata(tenantID, key)->watchFutureSS.cancel(); + deleteWatchMetadata(tenantID, key); + } + return count; +} + +void DatabaseContext::deleteWatchMetadata(int64_t tenantId, KeyRef key) { + const WatchMapKey mapKey(tenantId, key); + watchMap.erase(mapKey); + watchCounterMap.erase(mapKey); +} + +void DatabaseContext::clearWatchMetadata() { + watchMap.clear(); + watchCounterMap.clear(); +} + const UniqueOrderedOptionList& Database::getTransactionDefaults() const { ASSERT(db); return db->transactionDefaults; @@ -3905,35 +3953,41 @@ class WatchRefCountUpdater { Database cx; int64_t tenantID; KeyRef key; - Version version; + + void tryAddRefCount() { + if (cx.getReference()) { + cx->increaseWatchRefCount(tenantID, key); + } + } + + void tryDelRefCount() { + if (cx.getReference()) { + cx->decreaseWatchRefCount(tenantID, key); + } + } public: WatchRefCountUpdater() {} - WatchRefCountUpdater(const Database& cx_, const int64_t tenantID_, KeyRef key_, const Version& ver) - : cx(cx_), tenantID(tenantID_), key(key_), version(ver) {} + WatchRefCountUpdater(const Database& cx_, const int64_t tenantID_, KeyRef key_) + : cx(cx_), tenantID(tenantID_), key(key_) { + tryAddRefCount(); + } WatchRefCountUpdater& operator=(WatchRefCountUpdater&& other) { - // Since this class is only used by watchValueMap, and it is used *AFTER* a wait statement, this class is first - // initialized by default constructor, then, after the wait, this function is called to assign the actual - // database, key, etc., to re-initialize this object. At this stage, the reference count can be increased. And - // since the database object is moved, the rvalue will have null reference to the DatabaseContext and will not - // reduce the reference count. - cx = std::move(other.cx); - tenantID = std::move(other.tenantID); - key = std::move(other.key); - version = std::move(other.version); + tryDelRefCount(); - cx->increaseWatchRefCount(tenantID, key, version); + // NOTE: Do not use move semantic, this copy allows other delete the reference count properly. + cx = other.cx; + tenantID = other.tenantID; + key = other.key; + + tryAddRefCount(); return *this; } - ~WatchRefCountUpdater() { - if (cx.getReference()) { - cx->decreaseWatchRefCount(tenantID, key, version); - } - } + ~WatchRefCountUpdater() { tryDelRefCount(); } }; } // namespace @@ -3949,7 +4003,7 @@ ACTOR Future watchValueMap(Future version, Optional debugID, UseProvisionalProxies useProvisionalProxies) { state Version ver = wait(version); - state WatchRefCountUpdater watchRefCountUpdater(cx, tenant.tenantId, key, ver); + state WatchRefCountUpdater watchRefCountUpdater(cx, tenant.tenantId, key); wait(getWatchFuture(cx, makeReference( @@ -5426,26 +5480,17 @@ ACTOR Future restartWatch(Database cx, tenantInfo.tenantId = locationInfo.tenantEntry.id; } - if (key == "anotherKey"_sr) - std::cout << cx->dbId.toString() << " restartWatch" << std::endl; + wait(watchValueMap(cx->minAcceptableReadVersion, + tenantInfo, + key, + value, + cx, + tags, + spanContext, + taskID, + debugID, + useProvisionalProxies)); - try { - wait(watchValueMap(cx->minAcceptableReadVersion, - tenantInfo, - key, - value, - cx, - tags, - spanContext, - taskID, - debugID, - useProvisionalProxies)); - } catch (Error& err) { - std::cout << cx->dbId.toString() << " restartWatch fail " << err.code() << std::endl; - return Void(); - } - - std::cout << cx->dbId.toString() << " restartWatch pass" << std::endl; return Void(); } diff --git a/fdbclient/include/fdbclient/DatabaseContext.h b/fdbclient/include/fdbclient/DatabaseContext.h index a0d065bbfd..33c31bf76e 100644 --- a/fdbclient/include/fdbclient/DatabaseContext.h +++ b/fdbclient/include/fdbclient/DatabaseContext.h @@ -157,7 +157,7 @@ public: WatchMetadata(Reference parameters) : watchFuture(watchPromise.getFuture()), parameters(parameters) {} - ~WatchMetadata() { /*watchFutureSS.cancel();*/ } + ~WatchMetadata() { watchFutureSS.cancel(); } }; struct MutationAndVersionStream { @@ -350,11 +350,11 @@ public: void deleteWatchMetadata(int64_t tenant, KeyRef key); // Increases reference count to the given watch. Returns the number of references to the watch. - int32_t increaseWatchRefCount(const int64_t tenant, KeyRef key, const Version& version); + int32_t increaseWatchRefCount(const int64_t tenant, KeyRef key); // Decreases reference count to the given watch. If the reference count is dropped to 0, the watch metadata will be // removed. Returns the number of references to the watch. - int32_t decreaseWatchRefCount(const int64_t tenant, KeyRef key, const Version& version); + int32_t decreaseWatchRefCount(const int64_t tenant, KeyRef key); void clearWatchMetadata(); @@ -730,17 +730,7 @@ private: WatchMap_t watchMap; - // The reason of using a multiset of Versions as counter instead of a simpler integer counter is due to the - // possible race condition: - // - // 1. A watch to key A is set, the watchValueMap ACTOR, noted as X, starts waiting. - // 2. All watches are cleared due to connection string change. - // 3. The watch to key A is restarted with watchValueMap ACTOR Y. - // 4. X receives the cancel exception, and tries to dereference the counter. This causes Y gets cancelled. - // - // By introducing versions, this race condition is solved. - using WatchCounterMapValue = std::multiset; - using WatchCounterMap_t = std::unordered_map; + using WatchCounterMap_t = std::unordered_map; // Maps the number of the WatchMapKey being used. WatchCounterMap_t watchCounterMap; }; diff --git a/flow/include/flow/flow.h b/flow/include/flow/flow.h index bc2fadccea..cd6704f8b8 100644 --- a/flow/include/flow/flow.h +++ b/flow/include/flow/flow.h @@ -788,7 +788,7 @@ public: T const& get() const { return sav->get(); } T getValue() const { return get(); } - bool isValid() const { return sav != nullptr; } + bool isValid() const { return sav != 0; } bool isReady() const { return sav->isSet(); } bool isError() const { return sav->isError(); } // returns true if get can be called on this future (counterpart of canBeSet on Promises) @@ -798,12 +798,12 @@ public: return sav->error_state; } - Future() : sav(nullptr) {} + Future() : sav(0) {} Future(const Future& rhs) : sav(rhs.sav) { if (sav) sav->addFutureRef(); } - Future(Future&& rhs) noexcept : sav(rhs.sav) { rhs.sav = nullptr; } + Future(Future&& rhs) noexcept : sav(rhs.sav) { rhs.sav = 0; } Future(const T& presentValue) : sav(new SAV(1, 0)) { sav->send(presentValue); } Future(T&& presentValue) : sav(new SAV(1, 0)) { sav->send(std::move(presentValue)); } Future(Never) : sav(new SAV(1, 0)) { sav->send(Never()); } @@ -830,7 +830,7 @@ public: if (sav) sav->delFutureRef(); sav = rhs.sav; - rhs.sav = nullptr; + rhs.sav = 0; } } bool operator==(const Future& rhs) { return rhs.sav == sav; } @@ -843,17 +843,17 @@ public: void addCallbackAndClear(Callback* cb) { sav->addCallbackAndDelFutureRef(cb); - sav = nullptr; + sav = 0; } void addYieldedCallbackAndClear(Callback* cb) { sav->addYieldedCallbackAndDelFutureRef(cb); - sav = nullptr; + sav = 0; } void addCallbackChainAndClear(Callback* cb) { sav->addCallbackChainAndDelFutureRef(cb); - sav = nullptr; + sav = 0; } int getFutureReferenceCount() const { return sav->getFutureReferenceCount(); } From fbe9802be5ae804fdbb73388c6aab87cc99771bf Mon Sep 17 00:00:00 2001 From: Jingyu Zhou Date: Thu, 27 Oct 2022 19:46:05 -0700 Subject: [PATCH 87/95] Revert "configurationMonitor does not need to check watch reference count" This reverts commit ab0f827058c21dfab66462c3ce8545c6eec6a6e5. --- fdbserver/ClusterRecovery.actor.cpp | 124 +++++++++++++++++++++------- 1 file changed, 93 insertions(+), 31 deletions(-) diff --git a/fdbserver/ClusterRecovery.actor.cpp b/fdbserver/ClusterRecovery.actor.cpp index 15702ab500..ae6e7a4fa3 100644 --- a/fdbserver/ClusterRecovery.actor.cpp +++ b/fdbserver/ClusterRecovery.actor.cpp @@ -563,44 +563,106 @@ ACTOR Future changeCoordinators(Reference self) { } } -ACTOR Future configurationMonitor(Reference self, Database cx) { +namespace { + +// NOTE: This vector may not be initialized here as the keys might be initialized *AFTER* this vector, causing all +// keys empty. The reason is that the order of the initialization of keys and this vector might not be ordered as wished +// so the vector might be initialized before the keys receives values; thus, all values inside the vector are copied +// from uninitialized KeyRefs.. +// See C++11 standard 3.6.2 for more info. +std::vector configurationMonitorWatchKeys; + +ACTOR Future configurationMonitorImpl(Reference self, + Database cx, + Optional* pTenantId) { + state ReadYourWritesTransaction tr(cx); loop { - state ReadYourWritesTransaction tr(cx); + try { + tr.setOption(FDBTransactionOptions::ACCESS_SYSTEM_KEYS); + RangeResult results = wait(tr.getRange(configKeys, CLIENT_KNOBS->TOO_MANY)); + ASSERT(!results.more && results.size() < CLIENT_KNOBS->TOO_MANY); - loop { - try { - tr.setOption(FDBTransactionOptions::ACCESS_SYSTEM_KEYS); - RangeResult results = wait(tr.getRange(configKeys, CLIENT_KNOBS->TOO_MANY)); - ASSERT(!results.more && results.size() < CLIENT_KNOBS->TOO_MANY); - - DatabaseConfiguration conf; - conf.fromKeyValues((VectorRef)results); - TraceEvent("ConfigurationMonitor", self->dbgid) - .detail(getRecoveryEventName(ClusterRecoveryEventType::CLUSTER_RECOVERY_STATE_EVENT_NAME).c_str(), - self->recoveryState); - if (conf != self->configuration) { - if (self->recoveryState != RecoveryState::ALL_LOGS_RECRUITED && - self->recoveryState != RecoveryState::FULLY_RECOVERED) { - self->controllerData->shouldCommitSuicide = true; - throw restart_cluster_controller(); - } - - self->configuration = conf; - self->registrationTrigger.trigger(); + DatabaseConfiguration conf; + conf.fromKeyValues((VectorRef)results); + TraceEvent("ConfigurationMonitor", self->dbgid) + .detail(getRecoveryEventName(ClusterRecoveryEventType::CLUSTER_RECOVERY_STATE_EVENT_NAME).c_str(), + self->recoveryState); + if (conf != self->configuration) { + if (self->recoveryState != RecoveryState::ALL_LOGS_RECRUITED && + self->recoveryState != RecoveryState::FULLY_RECOVERED) { + self->controllerData->shouldCommitSuicide = true; + throw restart_cluster_controller(); } - state Future watchFuture = - tr.watch(moveKeysLockOwnerKey) || tr.watch(excludedServersVersionKey) || - tr.watch(failedServersVersionKey) || tr.watch(excludedLocalityVersionKey) || - tr.watch(failedLocalityVersionKey); - wait(tr.commit()); - wait(watchFuture); - break; - } catch (Error& e) { - wait(tr.onError(e)); + self->configuration = conf; + self->registrationTrigger.trigger(); } + + std::vector> watchFutures; + std::transform(std::begin(configurationMonitorWatchKeys), + std::end(configurationMonitorWatchKeys), + std::back_inserter(watchFutures), + [this](KeyRef key) { return tr.watch(key); }); + // Only after this stage, where getKeyLocation is called indirectly, the tenant information is updated and + // set to the transaction state. + (*pTenantId) = tr.getTransactionState()->tenantId(); + state Future watchFuture = waitForAny(watchFutures); + + wait(tr.commit()); + + wait(watchFuture); + break; + } catch (Error& e) { + if (e.code() == error_code_actor_cancelled) { + throw e; + } + wait(tr.onError(e)); } } + + return Void(); +} + +} // anonymous namespace + +ACTOR Future configurationMonitor(Reference self, Database cx) { + state Optional tenantId; + + // The keys cannot be initialized where it is defined. see comments at the definition. + if (configurationMonitorWatchKeys.empty()) { + configurationMonitorWatchKeys = std::vector{ moveKeysLockOwnerKey, + excludedServersVersionKey, + failedServersVersionKey, + excludedLocalityVersionKey, + failedLocalityVersionKey }; + } + + try { + while (true) { + wait(configurationMonitorImpl(self, cx, &tenantId)); + } + } catch (Error& e) { + if (e.code() != error_code_actor_cancelled) { + throw e; + } + + // Cancel all watches created by configurationMonitorImpl. Due to a circular reference issue, if the watches are + // not cancelled manually, the DatabaseContext object in cx will not be able to destructed properly, see + // + // https://github.com/apple/foundationdb/issues/8321 + // + // for more detailed discussion. + + if (!tenantId.present()) { + // Tenant ID not set, no watches are created in this case, no cleanup required. + return Void(); + } + std::for_each(std::begin(configurationMonitorWatchKeys), + std::end(configurationMonitorWatchKeys), + [this](KeyRef key) { cx->decreaseWatchRefCount(tenantId.get(), key); }); + } + + return Void(); } ACTOR static Future> getMinBackupVersion(Reference self, Database cx) { From dc60f63f9bfda9c45be30b7042842298591c467e Mon Sep 17 00:00:00 2001 From: Jingyu Zhou Date: Thu, 27 Oct 2022 19:46:05 -0700 Subject: [PATCH 88/95] Revert "Cancel watch when the key is not being waited" This reverts commit 639afbe62cc157a3428261bf8783088becc9ac13. --- fdbclient/NativeAPI.actor.cpp | 94 +------------ fdbclient/include/fdbclient/DatabaseContext.h | 38 +----- fdbserver/ClusterRecovery.actor.cpp | 124 +++++------------- 3 files changed, 43 insertions(+), 213 deletions(-) diff --git a/fdbclient/NativeAPI.actor.cpp b/fdbclient/NativeAPI.actor.cpp index fcef80270a..fa301ef15c 100644 --- a/fdbclient/NativeAPI.actor.cpp +++ b/fdbclient/NativeAPI.actor.cpp @@ -2163,14 +2163,14 @@ void DatabaseContext::setOption(FDBDatabaseOptions::Option option, Optional= maxOutstandingWatches) throw too_many_watches(); ++outstandingWatches; } -void DatabaseContext::removeWatchCounter() { +void DatabaseContext::removeWatch() { --outstandingWatches; ASSERT(outstandingWatches >= 0); } @@ -2390,44 +2390,15 @@ Reference DatabaseContext::getWatchMetadata(int64_t tenantId, Key } void DatabaseContext::setWatchMetadata(Reference metadata) { - const WatchMapKey key(metadata->parameters->tenant.tenantId, metadata->parameters->key); - watchMap[key] = metadata; - // NOTE Here we do *NOT* update/reset the reference count for the key, see the source code in getWatchFuture -} - -int32_t DatabaseContext::increaseWatchRefCount(const int64_t tenantID, KeyRef key) { - const WatchMapKey mapKey(tenantID, key); - if (watchCounterMap.count(mapKey) == 0) { - watchCounterMap[mapKey] = 0; - } - const auto count = ++watchCounterMap[mapKey]; - return count; -} - -int32_t DatabaseContext::decreaseWatchRefCount(const int64_t tenantID, KeyRef key) { - const WatchMapKey mapKey(tenantID, key); - if (watchCounterMap.count(mapKey) == 0) { - // Key does not exist. The metadata might be removed by deleteWatchMetadata already. - return 0; - } - const auto count = --watchCounterMap[mapKey]; - ASSERT(watchCounterMap[mapKey] >= 0); - if (watchCounterMap[mapKey] == 0) { - getWatchMetadata(tenantID, key)->watchFutureSS.cancel(); - deleteWatchMetadata(tenantID, key); - } - return count; + watchMap[std::make_pair(metadata->parameters->tenant.tenantId, metadata->parameters->key)] = metadata; } void DatabaseContext::deleteWatchMetadata(int64_t tenantId, KeyRef key) { - const WatchMapKey mapKey(tenantId, key); - watchMap.erase(mapKey); - watchCounterMap.erase(mapKey); + watchMap.erase(std::make_pair(tenantId, key)); } void DatabaseContext::clearWatchMetadata() { watchMap.clear(); - watchCounterMap.clear(); } const UniqueOrderedOptionList& Database::getTransactionDefaults() const { @@ -3942,56 +3913,6 @@ Future getWatchFuture(Database cx, Reference parameters) return Void(); } -namespace { - -// NOTE: Since an ACTOR could receive multiple exceptions for a single catch clause, e.g. broken promise together with -// operation cancelled, If the decreaseWatchRefCount is placed at the catch clause, it might be triggered for multiple -// times. One could check if the SAV isSet, but seems a more intuitive way is to use RAII-style constructor/destructor -// pair. Yet the object has to be constructed after a wait statement, so it must be trivially-constructible. This -// requires move-assignment operator implemented. -class WatchRefCountUpdater { - Database cx; - int64_t tenantID; - KeyRef key; - - void tryAddRefCount() { - if (cx.getReference()) { - cx->increaseWatchRefCount(tenantID, key); - } - } - - void tryDelRefCount() { - if (cx.getReference()) { - cx->decreaseWatchRefCount(tenantID, key); - } - } - -public: - WatchRefCountUpdater() {} - - WatchRefCountUpdater(const Database& cx_, const int64_t tenantID_, KeyRef key_) - : cx(cx_), tenantID(tenantID_), key(key_) { - tryAddRefCount(); - } - - WatchRefCountUpdater& operator=(WatchRefCountUpdater&& other) { - tryDelRefCount(); - - // NOTE: Do not use move semantic, this copy allows other delete the reference count properly. - cx = other.cx; - tenantID = other.tenantID; - key = other.key; - - tryAddRefCount(); - - return *this; - } - - ~WatchRefCountUpdater() { tryDelRefCount(); } -}; - -} // namespace - ACTOR Future watchValueMap(Future version, TenantInfo tenant, Key key, @@ -4003,7 +3924,6 @@ ACTOR Future watchValueMap(Future version, Optional debugID, UseProvisionalProxies useProvisionalProxies) { state Version ver = wait(version); - state WatchRefCountUpdater watchRefCountUpdater(cx, tenant.tenantId, key); wait(getWatchFuture(cx, makeReference( @@ -5536,11 +5456,11 @@ ACTOR Future watch(Reference watch, } } } catch (Error& e) { - cx->removeWatchCounter(); + cx->removeWatch(); throw; } - cx->removeWatchCounter(); + cx->removeWatch(); return Void(); } @@ -5551,7 +5471,7 @@ Future Transaction::getRawReadVersion() { Future Transaction::watch(Reference watch) { ++trState->cx->transactionWatchRequests; - trState->cx->addWatchCounter(); + trState->cx->addWatch(); watches.push_back(watch); return ::watch( watch, diff --git a/fdbclient/include/fdbclient/DatabaseContext.h b/fdbclient/include/fdbclient/DatabaseContext.h index 33c31bf76e..390873e0ef 100644 --- a/fdbclient/include/fdbclient/DatabaseContext.h +++ b/fdbclient/include/fdbclient/DatabaseContext.h @@ -156,8 +156,6 @@ public: WatchMetadata(Reference parameters) : watchFuture(watchPromise.getFuture()), parameters(parameters) {} - - ~WatchMetadata() { watchFutureSS.cancel(); } }; struct MutationAndVersionStream { @@ -330,32 +328,14 @@ public: // Note: this will never return if the server is running a protocol from FDB 5.0 or older Future getClusterProtocol(Optional expectedVersion = Optional()); - // Increases the counter of the number of watches in this DatabaseContext by 1. If the number of watches is too - // many, throws too_many_watches. - void addWatchCounter(); - - // Decrease the counter of the number of watches in this DatabaseContext by 1 - void removeWatchCounter(); + // Update the watch counter for the database + void addWatch(); + void removeWatch(); // watch map operations - - // Gets the watch metadata per tenant id and key Reference getWatchMetadata(int64_t tenantId, KeyRef key) const; - - // Refreshes the watch metadata. If the same watch is used (this is determined by the tenant id and the key), the - // metadata will be updated. void setWatchMetadata(Reference metadata); - - // Removes the watch metadata void deleteWatchMetadata(int64_t tenant, KeyRef key); - - // Increases reference count to the given watch. Returns the number of references to the watch. - int32_t increaseWatchRefCount(const int64_t tenant, KeyRef key); - - // Decreases reference count to the given watch. If the reference count is dropped to 0, the watch metadata will be - // removed. Returns the number of references to the watch. - int32_t decreaseWatchRefCount(const int64_t tenant, KeyRef key); - void clearWatchMetadata(); void setOption(FDBDatabaseOptions::Option option, Optional value); @@ -723,16 +703,8 @@ public: EventCacheHolder connectToDatabaseEventCacheHolder; private: - using WatchMapKey = std::pair; - using WatchMapKeyHasher = boost::hash; - using WatchMapValue = Reference; - using WatchMap_t = std::unordered_map; - - WatchMap_t watchMap; - - using WatchCounterMap_t = std::unordered_map; - // Maps the number of the WatchMapKey being used. - WatchCounterMap_t watchCounterMap; + std::unordered_map, Reference, boost::hash>> + watchMap; }; // Similar to tr.onError(), but doesn't require a DatabaseContext. diff --git a/fdbserver/ClusterRecovery.actor.cpp b/fdbserver/ClusterRecovery.actor.cpp index ae6e7a4fa3..15702ab500 100644 --- a/fdbserver/ClusterRecovery.actor.cpp +++ b/fdbserver/ClusterRecovery.actor.cpp @@ -563,106 +563,44 @@ ACTOR Future changeCoordinators(Reference self) { } } -namespace { - -// NOTE: This vector may not be initialized here as the keys might be initialized *AFTER* this vector, causing all -// keys empty. The reason is that the order of the initialization of keys and this vector might not be ordered as wished -// so the vector might be initialized before the keys receives values; thus, all values inside the vector are copied -// from uninitialized KeyRefs.. -// See C++11 standard 3.6.2 for more info. -std::vector configurationMonitorWatchKeys; - -ACTOR Future configurationMonitorImpl(Reference self, - Database cx, - Optional* pTenantId) { - state ReadYourWritesTransaction tr(cx); +ACTOR Future configurationMonitor(Reference self, Database cx) { loop { - try { - tr.setOption(FDBTransactionOptions::ACCESS_SYSTEM_KEYS); - RangeResult results = wait(tr.getRange(configKeys, CLIENT_KNOBS->TOO_MANY)); - ASSERT(!results.more && results.size() < CLIENT_KNOBS->TOO_MANY); + state ReadYourWritesTransaction tr(cx); - DatabaseConfiguration conf; - conf.fromKeyValues((VectorRef)results); - TraceEvent("ConfigurationMonitor", self->dbgid) - .detail(getRecoveryEventName(ClusterRecoveryEventType::CLUSTER_RECOVERY_STATE_EVENT_NAME).c_str(), - self->recoveryState); - if (conf != self->configuration) { - if (self->recoveryState != RecoveryState::ALL_LOGS_RECRUITED && - self->recoveryState != RecoveryState::FULLY_RECOVERED) { - self->controllerData->shouldCommitSuicide = true; - throw restart_cluster_controller(); + loop { + try { + tr.setOption(FDBTransactionOptions::ACCESS_SYSTEM_KEYS); + RangeResult results = wait(tr.getRange(configKeys, CLIENT_KNOBS->TOO_MANY)); + ASSERT(!results.more && results.size() < CLIENT_KNOBS->TOO_MANY); + + DatabaseConfiguration conf; + conf.fromKeyValues((VectorRef)results); + TraceEvent("ConfigurationMonitor", self->dbgid) + .detail(getRecoveryEventName(ClusterRecoveryEventType::CLUSTER_RECOVERY_STATE_EVENT_NAME).c_str(), + self->recoveryState); + if (conf != self->configuration) { + if (self->recoveryState != RecoveryState::ALL_LOGS_RECRUITED && + self->recoveryState != RecoveryState::FULLY_RECOVERED) { + self->controllerData->shouldCommitSuicide = true; + throw restart_cluster_controller(); + } + + self->configuration = conf; + self->registrationTrigger.trigger(); } - self->configuration = conf; - self->registrationTrigger.trigger(); + state Future watchFuture = + tr.watch(moveKeysLockOwnerKey) || tr.watch(excludedServersVersionKey) || + tr.watch(failedServersVersionKey) || tr.watch(excludedLocalityVersionKey) || + tr.watch(failedLocalityVersionKey); + wait(tr.commit()); + wait(watchFuture); + break; + } catch (Error& e) { + wait(tr.onError(e)); } - - std::vector> watchFutures; - std::transform(std::begin(configurationMonitorWatchKeys), - std::end(configurationMonitorWatchKeys), - std::back_inserter(watchFutures), - [this](KeyRef key) { return tr.watch(key); }); - // Only after this stage, where getKeyLocation is called indirectly, the tenant information is updated and - // set to the transaction state. - (*pTenantId) = tr.getTransactionState()->tenantId(); - state Future watchFuture = waitForAny(watchFutures); - - wait(tr.commit()); - - wait(watchFuture); - break; - } catch (Error& e) { - if (e.code() == error_code_actor_cancelled) { - throw e; - } - wait(tr.onError(e)); } } - - return Void(); -} - -} // anonymous namespace - -ACTOR Future configurationMonitor(Reference self, Database cx) { - state Optional tenantId; - - // The keys cannot be initialized where it is defined. see comments at the definition. - if (configurationMonitorWatchKeys.empty()) { - configurationMonitorWatchKeys = std::vector{ moveKeysLockOwnerKey, - excludedServersVersionKey, - failedServersVersionKey, - excludedLocalityVersionKey, - failedLocalityVersionKey }; - } - - try { - while (true) { - wait(configurationMonitorImpl(self, cx, &tenantId)); - } - } catch (Error& e) { - if (e.code() != error_code_actor_cancelled) { - throw e; - } - - // Cancel all watches created by configurationMonitorImpl. Due to a circular reference issue, if the watches are - // not cancelled manually, the DatabaseContext object in cx will not be able to destructed properly, see - // - // https://github.com/apple/foundationdb/issues/8321 - // - // for more detailed discussion. - - if (!tenantId.present()) { - // Tenant ID not set, no watches are created in this case, no cleanup required. - return Void(); - } - std::for_each(std::begin(configurationMonitorWatchKeys), - std::end(configurationMonitorWatchKeys), - [this](KeyRef key) { cx->decreaseWatchRefCount(tenantId.get(), key); }); - } - - return Void(); } ACTOR static Future> getMinBackupVersion(Reference self, Database cx) { From 49645a7755fe4d1eb19114d81b61fb2b648b0ebd Mon Sep 17 00:00:00 2001 From: Jingyu Zhou Date: Thu, 27 Oct 2022 19:46:05 -0700 Subject: [PATCH 89/95] Revert "Clean up unused comment in flow.h" This reverts commit 03b102d86aecbe700aa8402ae31d0431bfb0b2b9. --- flow/include/flow/flow.h | 11 +++++++++-- 1 file changed, 9 insertions(+), 2 deletions(-) diff --git a/flow/include/flow/flow.h b/flow/include/flow/flow.h index cd6704f8b8..3cc7470dbf 100644 --- a/flow/include/flow/flow.h +++ b/flow/include/flow/flow.h @@ -802,8 +802,12 @@ public: Future(const Future& rhs) : sav(rhs.sav) { if (sav) sav->addFutureRef(); + // if (sav->endpoint.isValid()) std::cout << "Future copied for " << sav->endpoint.key << std::endl; + } + Future(Future&& rhs) noexcept : sav(rhs.sav) { + rhs.sav = 0; + // if (sav->endpoint.isValid()) std::cout << "Future moved for " << sav->endpoint.key << std::endl; } - Future(Future&& rhs) noexcept : sav(rhs.sav) { rhs.sav = 0; } Future(const T& presentValue) : sav(new SAV(1, 0)) { sav->send(presentValue); } Future(T&& presentValue) : sav(new SAV(1, 0)) { sav->send(std::move(presentValue)); } Future(Never) : sav(new SAV(1, 0)) { sav->send(Never()); } @@ -815,6 +819,7 @@ public: #endif ~Future() { + // if (sav && sav->endpoint.isValid()) std::cout << "Future destroyed for " << sav->endpoint.key << std::endl; if (sav) sav->delFutureRef(); } @@ -859,7 +864,9 @@ public: int getFutureReferenceCount() const { return sav->getFutureReferenceCount(); } int getPromiseReferenceCount() const { return sav->getPromiseReferenceCount(); } - explicit Future(SAV* sav) : sav(sav) {} + explicit Future(SAV* sav) : sav(sav) { + // if (sav->endpoint.isValid()) std::cout << "Future created for " << sav->endpoint.key << std::endl; + } private: SAV* sav; From 0a15f081a185b4750f9b203d8cca69d68b4d4e24 Mon Sep 17 00:00:00 2001 From: Andrew Noyes Date: Fri, 28 Oct 2022 09:07:54 -0700 Subject: [PATCH 90/95] Proactively clean up idempotency ids for successful commits (#8578) * Proactively clean up idempotency ids for successful commits This change also includes some minor changes from my branch working on an idempotency ids cleaner, that I'd like to get merged sooner rather than later. - Adding a timestamp to idempotency values - Making IdempotencyId an actor file - Adding commit_unknown_result_fatal - Checking idempotencyIdsExpiredVersion in determineCommitStatus - Some testing QOL changes * Factor out decodeIdempotencyKey logic * Fix formatting * Update flow/include/flow/error_definitions.h Co-authored-by: A.J. Beamon * Use KeyBackedObjectProperty for idempotencyIdsExpiredVersion * Add IDEMPOTENCY_ID_IN_MEMORY_LIFETIME knob * Rename ExpireIdempotencyKeyValuePairRequest Also add a code probe for the case where an ExpireIdempotencyIdRequest is received before the count is known, and add an assert * Fix formatting and add TODO for nwijetunga Co-authored-by: A.J. Beamon --- design/idempotency_ids.md | 15 ++- ...mpotencyId.cpp => IdempotencyId.actor.cpp} | 56 +++++++-- fdbclient/NativeAPI.actor.cpp | 36 +++++- fdbclient/ServerKnobs.cpp | 4 + .../fdbclient/BuildIdempotencyIdMutations.h | 2 +- .../include/fdbclient/CommitProxyInterface.h | 24 +++- ...{IdempotencyId.h => IdempotencyId.actor.h} | 31 ++++- fdbclient/include/fdbclient/NativeAPI.actor.h | 2 + fdbclient/include/fdbclient/ServerKnobs.h | 3 + fdbclient/vexillographer/fdb.options | 2 +- fdbrpc/include/fdbrpc/LoadBalance.actor.h | 11 +- fdbserver/CommitProxyServer.actor.cpp | 113 +++++++++++++++++- .../include/fdbserver/ProxyCommitData.actor.h | 13 ++ .../AutomaticIdempotencyWorkload.actor.cpp | 94 +++++++++++++-- flow/include/flow/error_definitions.h | 1 + 15 files changed, 367 insertions(+), 40 deletions(-) rename fdbclient/{IdempotencyId.cpp => IdempotencyId.actor.cpp} (78%) rename fdbclient/include/fdbclient/{IdempotencyId.h => IdempotencyId.actor.h} (83%) diff --git a/design/idempotency_ids.md b/design/idempotency_ids.md index 278503713a..cf8106cda2 100644 --- a/design/idempotency_ids.md +++ b/design/idempotency_ids.md @@ -34,20 +34,25 @@ Commit proxies would combine idempotency IDs for transactions within a batch. Th ## Value format ``` -${protocol_version}(${n (1 byte)}${idempotency_id (n bytes)}${low_order_byte_of_batch_index})* +${protocol_version}${timestamp}(${n (1 byte)}${idempotency_id (n bytes)}${low_order_byte_of_batch_index})* ``` The batch index for each idempotency id can be reconstructed from the high order byte and low order bytes stored in the key and value, respectively. This is necessary for an "unknown_committed" transaction to recover their full version stamp. Batch index is a `short int`, i.e. 2 bytes. +The timestamp is the unix epoch stored as a little-endian signed 64-bit integer. + # Cleaning up old idempotency ids After learning the result of an attempt to commit a transaction with an idempotency id, the client may inform the cluster that it's no longer interested in that id and the cluster can reclaim the space used to store the idempotency -id. The happy-path reply to a CommitTransactionRequest will say which proxy this -request should be sent to, and all idempotency ids for a database key will be -sent to the same proxy so that it can clear the key once it receives all of -them. The first proxy will also periodically clean up the oldest idempotency ids, based on a policy determined by two knobs. One knob will control the minimum lifetime of an idempotency id (i.e. don't delete anything younger than 1 day), and the other will control the target byte size of the idempotency keys (e.g. keep 100 MB of idempotency keys around). +id. The commit proxy that committed a batch is responsible for cleaning all +idempotency kv pairs from that batch, so clients must tell that specific proxy +that they're done with the id. The first proxy will also periodically clean up +the oldest idempotency ids, based on a policy determined by two knobs. One knob +will control the minimum lifetime of an idempotency id (i.e. don't delete +anything younger than 1 day), and the other will control the target byte size of +the idempotency keys (e.g. keep 100 MB of idempotency keys around). # Commit protocol diff --git a/fdbclient/IdempotencyId.cpp b/fdbclient/IdempotencyId.actor.cpp similarity index 78% rename from fdbclient/IdempotencyId.cpp rename to fdbclient/IdempotencyId.actor.cpp index eaba38ed34..e0c4839b45 100644 --- a/fdbclient/IdempotencyId.cpp +++ b/fdbclient/IdempotencyId.actor.cpp @@ -1,5 +1,5 @@ /* - * IdempotencyId.cpp + * IdempotencyId.actor.cpp * * This source file is part of the FoundationDB open source project * @@ -18,9 +18,11 @@ * limitations under the License. */ -#include "fdbclient/IdempotencyId.h" +#include "fdbclient/IdempotencyId.actor.h" +#include "fdbclient/ReadYourWrites.h" #include "fdbclient/SystemData.h" #include "flow/UnitTest.h" +#include "flow/actorcompiler.h" // this has to be the last include struct IdempotencyIdKVBuilderImpl { Optional commitVersion; @@ -40,6 +42,7 @@ void IdempotencyIdKVBuilder::add(const IdempotencyIdRef& id, uint16_t batchIndex ASSERT((batchIndex >> 8) == impl->batchIndexHighOrderByte.get()); } else { impl->batchIndexHighOrderByte = batchIndex >> 8; + impl->value << int64_t(now()); } StringRef s = id.asStringRefUnsafe(); impl->value << uint8_t(s.size()); @@ -53,19 +56,17 @@ Optional IdempotencyIdKVBuilder::buildAndClear() { return {}; } - BinaryWriter key{ Unversioned() }; - key.serializeBytes(idempotencyIdKeys.begin); - key << bigEndian64(impl->commitVersion.get()); - key << impl->batchIndexHighOrderByte.get(); - Value v = impl->value.toValue(); + KeyRef key = + makeIdempotencySingleKeyRange(v.arena(), impl->commitVersion.get(), impl->batchIndexHighOrderByte.get()).begin; + impl->value = BinaryWriter(IncludeVersion()); impl->batchIndexHighOrderByte = Optional(); Optional result = KeyValue(); result.get().arena() = v.arena(); - result.get().key = key.toValue(result.get().arena()); + result.get().key = key; result.get().value = v; return result; } @@ -86,6 +87,8 @@ Optional kvContainsIdempotencyId(const KeyValueRef& kv, const Idem // Even if id is a substring of value, it may still not actually contain it. BinaryReader reader(kv.value.begin(), kv.value.size(), IncludeVersion()); + int64_t timestamp; // ignored + reader >> timestamp; while (!reader.empty()) { uint8_t length; reader >> length; @@ -93,13 +96,9 @@ Optional kvContainsIdempotencyId(const KeyValueRef& kv, const Idem uint8_t lowOrderBatchIndex; reader >> lowOrderBatchIndex; if (candidate == needle) { - BinaryReader reader(kv.key.begin(), kv.key.size(), Unversioned()); - reader.readBytes(idempotencyIdKeys.begin.size()); Version commitVersion; - reader >> commitVersion; - commitVersion = bigEndian64(commitVersion); uint8_t highOrderBatchIndex; - reader >> highOrderBatchIndex; + decodeIdempotencyKey(kv.key, commitVersion, highOrderBatchIndex); return CommitResult{ commitVersion, static_cast((uint16_t(highOrderBatchIndex) << 8) | uint16_t(lowOrderBatchIndex)) }; @@ -172,4 +171,35 @@ TEST_CASE("/fdbclient/IdempotencyId/serialization") { ASSERT(t == id); } return Void(); +} + +KeyRangeRef makeIdempotencySingleKeyRange(Arena& arena, Version version, uint8_t highOrderBatchIndex) { + static const auto size = + idempotencyIdKeys.begin.size() + sizeof(version) + sizeof(highOrderBatchIndex) + /*\x00*/ 1; + + StringRef second = makeString(size, arena); + auto* dst = mutateString(second); + + memcpy(dst, idempotencyIdKeys.begin.begin(), idempotencyIdKeys.begin.size()); + dst += idempotencyIdKeys.begin.size(); + + version = bigEndian64(version); + memcpy(dst, &version, sizeof(version)); + dst += sizeof(version); + + *dst++ = highOrderBatchIndex; + + *dst++ = 0; + + ASSERT_EQ(dst - second.begin(), size); + + return KeyRangeRef(second.removeSuffix("\x00"_sr), second); +} + +void decodeIdempotencyKey(KeyRef key, Version& commitVersion, uint8_t& highOrderBatchIndex) { + BinaryReader reader(key, Unversioned()); + reader.readBytes(idempotencyIdKeys.begin.size()); + reader >> commitVersion; + commitVersion = bigEndian64(commitVersion); + reader >> highOrderBatchIndex; } \ No newline at end of file diff --git a/fdbclient/NativeAPI.actor.cpp b/fdbclient/NativeAPI.actor.cpp index fa301ef15c..77083fbdfd 100644 --- a/fdbclient/NativeAPI.actor.cpp +++ b/fdbclient/NativeAPI.actor.cpp @@ -6142,6 +6142,7 @@ ACTOR static Future> determineCommitStatus(Referencecx); state int retries = 0; + state Version expiredVersion; state Span span("NAPI:determineCommitStatus"_loc, trState->spanContext); tr.span.setParent(span.context); loop { @@ -6151,11 +6152,19 @@ ACTOR static Future> determineCommitStatus(ReferenceauthToken = trState->authToken; tr.setOption(FDBTransactionOptions::READ_SYSTEM_KEYS); tr.setOption(FDBTransactionOptions::READ_LOCK_AWARE); + KeyBackedObjectProperty expiredKey(idempotencyIdsExpiredVersion, + Unversioned()); + IdempotencyIdsExpiredVersion expiredVal = wait(expiredKey.getD(&tr)); + expiredVersion = expiredVal.expired; + if (expiredVersion >= minPossibleCommitVersion) { + throw commit_unknown_result_fatal(); + } Version rv = wait(tr.getReadVersion()); TraceEvent("DetermineCommitStatusAttempt") .detail("IdempotencyId", idempotencyId.asStringRefUnsafe()) .detail("Retries", retries) .detail("ReadVersion", rv) + .detail("ExpiredVersion", expiredVersion) .detail("MinPossibleCommitVersion", minPossibleCommitVersion) .detail("MaxPossibleCommitVersion", maxPossibleCommitVersion); KeyRange possibleRange = @@ -6399,6 +6408,12 @@ ACTOR static Future tryCommit(Reference trState, req.debugID = commitID; state Future reply; + // Only gets filled in in the happy path where we don't have to commit on the first proxy or use provisional + // proxies + state int alternativeChosen = -1; + // Only valid if alternativeChosen >= 0 + state Reference proxiesUsed; + if (trState->options.commitOnFirstProxy) { if (trState->cx->clientInfo->get().firstCommitProxy.present()) { reply = throwErrorOr(brokenPromiseToMaybeDelivered( @@ -6409,11 +6424,13 @@ ACTOR static Future tryCommit(Reference trState, : Never(); } } else { - reply = basicLoadBalance(trState->cx->getCommitProxies(trState->useProvisionalProxies), + proxiesUsed = trState->cx->getCommitProxies(trState->useProvisionalProxies); + reply = basicLoadBalance(proxiesUsed, &CommitProxyInterface::commit, req, TaskPriority::DefaultPromiseEndpoint, - AtMostOnce::True); + AtMostOnce::True, + &alternativeChosen); } state double grvTime = now(); choose { @@ -6463,6 +6480,12 @@ ACTOR static Future tryCommit(Reference trState, ci.version, req, trState->tenant())); + if (trState->automaticIdempotency && alternativeChosen >= 0) { + // Automatic idempotency means we're responsible for best effort idempotency id clean up + proxiesUsed->getInterface(alternativeChosen) + .expireIdempotencyId.send(ExpireIdempotencyIdRequest{ + ci.version, uint8_t(ci.txnBatchId >> 8), trState->getTenantInfo() }); + } return Void(); } else { // clear the RYW transaction which contains previous conflicting keys @@ -6948,11 +6971,16 @@ void Transaction::setOption(FDBTransactionOptions::Option option, OptionalautomaticIdempotency = false; break; case FDBTransactionOptions::AUTOMATIC_IDEMPOTENCY: validateOptionValueNotPresent(value); - tr.idempotencyId = IdempotencyIdRef( - tr.arena, IdempotencyIdRef(BinaryWriter::toValue(deterministicRandom()->randomUniqueID(), Unversioned()))); + if (!tr.idempotencyId.valid()) { + tr.idempotencyId = IdempotencyIdRef( + tr.arena, + IdempotencyIdRef(BinaryWriter::toValue(deterministicRandom()->randomUniqueID(), Unversioned()))); + } + trState->automaticIdempotency = true; break; default: diff --git a/fdbclient/ServerKnobs.cpp b/fdbclient/ServerKnobs.cpp index 0275ab7761..4f034714c7 100644 --- a/fdbclient/ServerKnobs.cpp +++ b/fdbclient/ServerKnobs.cpp @@ -1022,6 +1022,10 @@ void ServerKnobs::initialize(Randomize randomize, ClientKnobs* clientKnobs, IsSi // NOTE: 'token-name" can NOT contain '#' character init( REST_KMS_CONNECTOR_VALIDATION_TOKEN_DETAILS, ""); + // Drop in-memory state associated with an idempotency id after this many seconds. Once dropped, this id cannot be + // expired proactively, but will eventually get cleaned up by the idempotency id cleaner. + init( IDEMPOTENCY_ID_IN_MEMORY_LIFETIME, 10); + // clang-format on if (clientKnobs) { diff --git a/fdbclient/include/fdbclient/BuildIdempotencyIdMutations.h b/fdbclient/include/fdbclient/BuildIdempotencyIdMutations.h index 73ee0d57fb..e8617ff657 100644 --- a/fdbclient/include/fdbclient/BuildIdempotencyIdMutations.h +++ b/fdbclient/include/fdbclient/BuildIdempotencyIdMutations.h @@ -22,7 +22,7 @@ #define FDBCLIENT_BUILD_IDEMPOTENCY_ID_MUTATIONS_H #include "fdbclient/CommitProxyInterface.h" -#include "fdbclient/IdempotencyId.h" +#include "fdbclient/IdempotencyId.actor.h" #pragma once diff --git a/fdbclient/include/fdbclient/CommitProxyInterface.h b/fdbclient/include/fdbclient/CommitProxyInterface.h index e1a7ce9000..2791338b02 100644 --- a/fdbclient/include/fdbclient/CommitProxyInterface.h +++ b/fdbclient/include/fdbclient/CommitProxyInterface.h @@ -30,7 +30,7 @@ #include "fdbclient/FDBTypes.h" #include "fdbclient/GlobalConfig.h" #include "fdbclient/GrvProxyInterface.h" -#include "fdbclient/IdempotencyId.h" +#include "fdbclient/IdempotencyId.actor.h" #include "fdbclient/StorageServerInterface.h" #include "fdbclient/TagThrottle.actor.h" #include "fdbclient/VersionVector.h" @@ -61,6 +61,7 @@ struct CommitProxyInterface { RequestStream proxySnapReq; RequestStream exclusionSafetyCheckReq; RequestStream getDDMetrics; + PublicRequestStream expireIdempotencyId; UID id() const { return commit.getEndpoint().token; } std::string toString() const { return id().shortString(); } @@ -87,6 +88,8 @@ struct CommitProxyInterface { exclusionSafetyCheckReq = RequestStream(commit.getEndpoint().getAdjustedEndpoint(8)); getDDMetrics = RequestStream(commit.getEndpoint().getAdjustedEndpoint(9)); + expireIdempotencyId = + PublicRequestStream(commit.getEndpoint().getAdjustedEndpoint(10)); } } @@ -103,6 +106,7 @@ struct CommitProxyInterface { streams.push_back(proxySnapReq.getReceiver()); streams.push_back(exclusionSafetyCheckReq.getReceiver()); streams.push_back(getDDMetrics.getReceiver()); + streams.push_back(expireIdempotencyId.getReceiver()); FlowTransport::transport().addEndpoints(streams); } }; @@ -151,6 +155,24 @@ struct ClientDBInfo { } }; +struct ExpireIdempotencyIdRequest { + constexpr static FileIdentifier file_identifier = 1900933; + Version commitVersion = invalidVersion; + uint8_t batchIndexHighByte = 0; + TenantInfo tenant; + + ExpireIdempotencyIdRequest() {} + ExpireIdempotencyIdRequest(Version commitVersion, uint8_t batchIndexHighByte, TenantInfo tenant) + : commitVersion(commitVersion), batchIndexHighByte(batchIndexHighByte), tenant(tenant) {} + + bool verify() const { return tenant.isAuthorized(); } + + template + void serialize(Ar& ar) { + serializer(ar, commitVersion, batchIndexHighByte, tenant); + } +}; + struct CommitID { constexpr static FileIdentifier file_identifier = 14254927; Version version; // returns invalidVersion if transaction conflicts diff --git a/fdbclient/include/fdbclient/IdempotencyId.h b/fdbclient/include/fdbclient/IdempotencyId.actor.h similarity index 83% rename from fdbclient/include/fdbclient/IdempotencyId.h rename to fdbclient/include/fdbclient/IdempotencyId.actor.h index 19a66a897e..de94b01479 100644 --- a/fdbclient/include/fdbclient/IdempotencyId.h +++ b/fdbclient/include/fdbclient/IdempotencyId.actor.h @@ -1,5 +1,5 @@ /* - * IdempotencyId.h + * IdempotencyId.actor.h * * This source file is part of the FoundationDB open source project * @@ -18,8 +18,13 @@ * limitations under the License. */ -#ifndef FDBCLIENT_IDEMPOTENCYID_H -#define FDBCLIENT_IDEMPOTENCYID_H +// When actually compiled (NO_INTELLISENSE), include the generated version of this file. In intellisense use the source +// version. +#if defined(NO_INTELLISENSE) && !defined(FDBCLIENT_IDEMPOTENCY_ID_ACTOR_G_H) +#define FDBCLIENT_IDEMPOTENCY_ID_ACTOR_G_H +#include "fdbclient/IdempotencyId.actor.g.h" +#elif !defined(FDBCLIENT_IDEMPOTENCY_ID_ACTOR_H) +#define FDBCLIENT_IDEMPOTENCY_ID_ACTOR_H #pragma once @@ -28,12 +33,24 @@ #include "flow/Arena.h" #include "flow/IRandom.h" #include "flow/serialize.h" +#include "flow/actorcompiler.h" // this has to be the last include struct CommitResult { Version commitVersion; uint16_t batchIndex; }; +// The type of the value stored at the key |idempotencyIdsExpiredVersion| +struct IdempotencyIdsExpiredVersion { + static constexpr auto file_identifier = 3746945; + Version expired = 0; + + template + void serialize(Archive& ar) { + serializer(ar, expired); + } +}; + // See design/idempotency_ids.md for more information. Designed so that the common case of a random 16 byte id does not // usually require indirection. Either invalid or an id with length >= 16 and < 256. struct IdempotencyIdRef { @@ -163,4 +180,10 @@ private: // Check if id is present in kv, and if so return the commit version and batchIndex Optional kvContainsIdempotencyId(const KeyValueRef& kv, const IdempotencyIdRef& id); -#endif \ No newline at end of file +// Make a range containing only the idempotency key associated with version and highOrderBatchIndex +KeyRangeRef makeIdempotencySingleKeyRange(Arena& arena, Version version, uint8_t highOrderBatchIndex); + +void decodeIdempotencyKey(KeyRef key, Version& commitVersion, uint8_t& highOrderBatchIndex); + +#include "flow/unactorcompiler.h" +#endif diff --git a/fdbclient/include/fdbclient/NativeAPI.actor.h b/fdbclient/include/fdbclient/NativeAPI.actor.h index 642a4e747a..eb19e0df40 100644 --- a/fdbclient/include/fdbclient/NativeAPI.actor.h +++ b/fdbclient/include/fdbclient/NativeAPI.actor.h @@ -268,6 +268,8 @@ struct TransactionState : ReferenceCounted { // prefix/ : '0' - any keys equal or larger than this key are (definitely) not conflicting keys std::shared_ptr> conflictingKeys; + bool automaticIdempotency = false; + // Only available so that Transaction can have a default constructor, for use in state variables TransactionState(TaskPriority taskID, SpanContext spanContext) : taskID(taskID), spanContext(spanContext), tenantSet(false) {} diff --git a/fdbclient/include/fdbclient/ServerKnobs.h b/fdbclient/include/fdbclient/ServerKnobs.h index 5fdc88347d..8cb24c0e75 100644 --- a/fdbclient/include/fdbclient/ServerKnobs.h +++ b/fdbclient/include/fdbclient/ServerKnobs.h @@ -992,6 +992,9 @@ public: std::string REST_KMS_CONNECTOR_GET_ENCRYPTION_KEYS_ENDPOINT; std::string REST_KMS_CONNECTOR_GET_BLOB_METADATA_ENDPOINT; + // Idempotency ids + double IDEMPOTENCY_ID_IN_MEMORY_LIFETIME; + ServerKnobs(Randomize, ClientKnobs*, IsSimulated); void initialize(Randomize, ClientKnobs*, IsSimulated); }; diff --git a/fdbclient/vexillographer/fdb.options b/fdbclient/vexillographer/fdb.options index 6dbcf6d6c1..499a31e709 100644 --- a/fdbclient/vexillographer/fdb.options +++ b/fdbclient/vexillographer/fdb.options @@ -279,7 +279,7 @@ description is not currently required but encouraged. description="Set the transaction size limit in bytes. The size is calculated by combining the sizes of all keys and values written or mutated, all key ranges cleared, and all read and write conflict ranges. (In other words, it includes the total size of all data included in the request to the cluster to commit the transaction.) Large transactions can cause performance problems on FoundationDB clusters, so setting this limit to a smaller value than the default can help prevent the client from accidentally degrading the cluster's performance. This value must be at least 32 and cannot be set to higher than 10,000,000, the default transaction size limit." />