From 7d25dab96bc200508f273cc54941dc5330cef6a4 Mon Sep 17 00:00:00 2001 From: Josh Slocum Date: Tue, 26 Oct 2021 12:11:37 -0500 Subject: [PATCH 001/413] Re-enabling blob granules for testing --- fdbclient/ClientKnobs.cpp | 2 +- fdbserver/SimulatedCluster.actor.cpp | 32 +++++++++++++++++++--------- tests/CMakeLists.txt | 12 +++++------ 3 files changed, 29 insertions(+), 17 deletions(-) diff --git a/fdbclient/ClientKnobs.cpp b/fdbclient/ClientKnobs.cpp index bb2e000b3f..9306a248f8 100644 --- a/fdbclient/ClientKnobs.cpp +++ b/fdbclient/ClientKnobs.cpp @@ -255,7 +255,7 @@ void ClientKnobs::initialize(Randomize randomize) { init( BUSYNESS_SPIKE_SATURATED_THRESHOLD, 0.500 ); // blob granules - init( ENABLE_BLOB_GRANULES, false ); + init( ENABLE_BLOB_GRANULES, true ); // clang-format on } diff --git a/fdbserver/SimulatedCluster.actor.cpp b/fdbserver/SimulatedCluster.actor.cpp index 9062c20b58..bdef5d8690 100644 --- a/fdbserver/SimulatedCluster.actor.cpp +++ b/fdbserver/SimulatedCluster.actor.cpp @@ -70,6 +70,10 @@ namespace { const int MACHINE_REBOOT_TIME = 10; +// The max number of extra blob worker machines we might (i.e. randomly) add to the simulated cluster. +// Note that this is in addition to the two we always have. +const int NUM_EXTRA_BW_MACHINES = 5; + bool destructed = false; // Configuration details specified in workload test files that change the simulation @@ -2011,16 +2015,16 @@ void setupSimulatedSystem(std::vector>* systemActors, coordinatorCount); ASSERT_LE(dcCoordinators, machines); - // FIXME: temporarily code to test storage cache + // FIXME: we hardcode some machines to specifically test storage cache and blob workers // TODO: caching disabled for this merge - if (dc == 0) { - machines++; - } + int storageCacheMachines = dc == 0 ? 1 : 0; + int blobWorkerMachines = 2 + deterministicRandom()->randomInt(0, NUM_EXTRA_BW_MACHINES + 1); - int useSeedForMachine = deterministicRandom()->randomInt(0, machines); + int totalMachines = machines + storageCacheMachines + blobWorkerMachines; + int useSeedForMachine = deterministicRandom()->randomInt(0, totalMachines); Standalone zoneId; Standalone newZoneId; - for (int machine = 0; machine < machines; machine++) { + for (int machine = 0; machine < totalMachines; machine++) { Standalone machineId(deterministicRandom()->randomUniqueID().toString()); if (machine == 0 || machineCount - dataCenters <= 4 || assignedMachines != 4 || simconfig.db.regions.size() || deterministicRandom()->random01() < 0.5) { @@ -2050,11 +2054,19 @@ void setupSimulatedSystem(std::vector>* systemActors, } } - // FIXME: temporarily code to test storage cache + // FIXME: hack to add machines specifically to test storage cache and blob workers // TODO: caching disabled for this merge - if (machine == machines - 1 && dc == 0) { - processClass = ProcessClass(ProcessClass::StorageCacheClass, ProcessClass::CommandLineSource); - nonVersatileMachines++; + // `machines` here is the normal (non-temporary) machines that totalMachines comprises of + if (machine >= machines) { + if (storageCacheMachines > 0 && dc == 0) { + processClass = ProcessClass(ProcessClass::StorageCacheClass, ProcessClass::CommandLineSource); + nonVersatileMachines++; + storageCacheMachines--; + } else if (blobWorkerMachines > 0) { // add blob workers to every DC + processClass = ProcessClass(ProcessClass::BlobWorkerClass, ProcessClass::CommandLineSource); + nonVersatileMachines++; + blobWorkerMachines--; + } } std::vector ips; diff --git a/tests/CMakeLists.txt b/tests/CMakeLists.txt index 0215a95cd5..7138b8ca3a 100644 --- a/tests/CMakeLists.txt +++ b/tests/CMakeLists.txt @@ -49,8 +49,8 @@ if(WITH_PYTHON) add_fdb_test(TEST_FILES BackupContainers.txt IGNORE) add_fdb_test(TEST_FILES BandwidthThrottle.txt IGNORE) add_fdb_test(TEST_FILES BigInsert.txt IGNORE) - add_fdb_test(TEST_FILES BlobGranuleReaderUnit.txt IGNORE) - add_fdb_test(TEST_FILES BlobManagerUnit.txt IGNORE) + add_fdb_test(TEST_FILES BlobGranuleReaderUnit.txt) + add_fdb_test(TEST_FILES BlobManagerUnit.txt) add_fdb_test(TEST_FILES ConsistencyCheck.txt IGNORE) add_fdb_test(TEST_FILES DDMetricsExclude.txt IGNORE) add_fdb_test(TEST_FILES DataDistributionMetrics.txt IGNORE) @@ -258,10 +258,10 @@ if(WITH_PYTHON) add_fdb_test(TEST_FILES slow/ApiCorrectness.toml) add_fdb_test(TEST_FILES slow/ApiCorrectnessAtomicRestore.toml) add_fdb_test(TEST_FILES slow/ApiCorrectnessSwitchover.toml) - add_fdb_test(TEST_FILES fast/BlobGranuleCorrectness.toml IGNORE) - add_fdb_test(TEST_FILES slow/BlobGranuleCorrectnessLarge.toml IGNORE) - add_fdb_test(TEST_FILES fast/BlobGranuleCorrectnessClean.toml IGNORE) - add_fdb_test(TEST_FILES slow/BlobGranuleCorrectnessLargeClean.toml IGNORE) + add_fdb_test(TEST_FILES fast/BlobGranuleCorrectness.toml) + add_fdb_test(TEST_FILES slow/BlobGranuleCorrectnessLarge.toml) + add_fdb_test(TEST_FILES fast/BlobGranuleCorrectnessClean.toml) + add_fdb_test(TEST_FILES slow/BlobGranuleCorrectnessLargeClean.toml) add_fdb_test(TEST_FILES slow/ClogWithRollbacks.toml) add_fdb_test(TEST_FILES slow/CloggedCycleTest.toml) add_fdb_test(TEST_FILES slow/CloggedStorefront.toml) From c39c631d50d26812541f4f848a2aa32f837d7ecd Mon Sep 17 00:00:00 2001 From: Josh Slocum Date: Tue, 26 Oct 2021 12:16:15 -0500 Subject: [PATCH 002/413] Using change feed stops and known committed version from change feeds --- fdbserver/BlobWorker.actor.cpp | 216 ++++++++++----------------------- 1 file changed, 65 insertions(+), 151 deletions(-) diff --git a/fdbserver/BlobWorker.actor.cpp b/fdbserver/BlobWorker.actor.cpp index d85d113e99..2ec7467238 100644 --- a/fdbserver/BlobWorker.actor.cpp +++ b/fdbserver/BlobWorker.actor.cpp @@ -181,10 +181,6 @@ struct BlobWorkerData : NonCopyable, ReferenceCounted { // FIXME: expire from map after a delay when granule is revoked and the history is no longer needed KeyRangeMap> granuleHistory; - AsyncVar pendingDeltaFileCommitChecks; - AsyncVar knownCommittedVersion; - uint64_t knownCommittedCheckCount = 0; - PromiseStream granuleUpdateErrors; BlobWorkerData(UID id, Database db) : id(id), db(db), stats(id, SERVER_KNOBS->WORKER_LOGGING_INTERVAL) {} @@ -435,18 +431,18 @@ ACTOR Future updateGranuleSplitState(Transaction* tr, tr->clear(singleKeyRange(oldGranuleLockKey)); tr->clear(currentRange); } else { + tr->atomicOp(myStateKey, blobGranuleSplitValueFor(newState), MutationRef::SetVersionstampedValue); if (newState == BlobGranuleSplitState::Assigned && currentState == BlobGranuleSplitState::Started && totalStarted == 1) { + // We are the last one to change from Start -> Assigned, so we can stop the parent change feed. if (BW_DEBUG) { - printf("%s WOULD BE stopping change feed for old granule %s\n", + printf("%s stopping change feed for old granule %s\n", currentGranuleID.toString().c_str(), parentGranuleID.toString().c_str()); } - // FIXME: enable - // wait(updateChangeFeed(tr, KeyRef(parentGranuleID.toString()), - // ChangeFeedStatus::CHANGE_FEED_DESTROY)); + + wait(updateChangeFeed(tr, KeyRef(parentGranuleID.toString()), ChangeFeedStatus::CHANGE_FEED_STOP)); } - tr->atomicOp(myStateKey, blobGranuleSplitValueFor(newState), MutationRef::SetVersionstampedValue); } } else if (BW_DEBUG) { printf("Ignoring granule %s split state from %s %d -> %d\n", @@ -484,14 +480,9 @@ ACTOR Future writeDeltaFile(Reference bwData, GranuleDeltas deltasToWrite, Version currentDeltaVersion, Future previousDeltaFileFuture, + NotifiedVersion* granuleCommittedVersion, Optional> oldGranuleComplete) { wait(delay(0, TaskPriority::BlobWorkerUpdateStorage)); - // potentially kick off delta file commit check, if our version isn't already known to be committed - state uint64_t checkCount = -1; - if (bwData->knownCommittedVersion.get() < currentDeltaVersion) { - bwData->pendingDeltaFileCommitChecks.set(bwData->pendingDeltaFileCommitChecks.get() + 1); - checkCount = bwData->knownCommittedCheckCount; - } // TODO some sort of directory structure would be useful? state std::string fname = deterministicRandom()->randomUniqueID().toString() + "_T" + @@ -514,13 +505,8 @@ ACTOR Future writeDeltaFile(Reference bwData, state int numIterations = 0; try { // before updating FDB, wait for the delta file version to be committed and previous delta files to finish - while (bwData->knownCommittedVersion.get() < currentDeltaVersion) { - if (bwData->knownCommittedCheckCount != checkCount) { - checkCount = bwData->knownCommittedCheckCount; - // a check happened between the start and now, and the version is still lower. Kick off another one. - bwData->pendingDeltaFileCommitChecks.set(bwData->pendingDeltaFileCommitChecks.get() + 1); - } - wait(bwData->knownCommittedVersion.onChange()); + if (currentDeltaVersion > granuleCommittedVersion->get()) { + wait(granuleCommittedVersion->whenAtLeast(currentDeltaVersion)); } BlobFileIndex prev = wait(previousDeltaFileFuture); wait(delay(0, TaskPriority::BlobWorkerUpdateFDB)); @@ -859,46 +845,6 @@ ACTOR Future compactFromBlob(Reference bwData, } } -// When reading from a prior change feed, the prior change feed may contain mutations that don't belong in the new -// granule. And, we only want to read the prior change feed up to the start of the new change feed. -static bool filterOldMutations(const KeyRange& range, - const Standalone>* oldMutations, - Standalone>* mutations, - Version maxVersion) { - Standalone> filteredMutations; - mutations->arena().dependsOn(range.arena()); - mutations->arena().dependsOn(oldMutations->arena()); - for (auto& delta : *oldMutations) { - if (delta.version >= maxVersion) { - return true; - } - MutationsAndVersionRef filteredDelta; - filteredDelta.version = delta.version; - for (auto& m : delta.mutations) { - ASSERT(m.type == MutationRef::SetValue || m.type == MutationRef::ClearRange); - if (m.type == MutationRef::SetValue) { - if (m.param1 >= range.begin && m.param1 < range.end) { - filteredDelta.mutations.push_back(mutations->arena(), m); - } - } else { - if (m.param2 >= range.begin && m.param1 < range.end) { - // clamp clear range down to sub-range - MutationRef m2 = m; - if (range.begin > m.param1) { - m2.param1 = range.begin; - } - if (range.end < m.param2) { - m2.param2 = range.end; - } - filteredDelta.mutations.push_back(mutations->arena(), m2); - } - } - } - mutations->push_back(mutations->arena(), filteredDelta); - } - return false; -} - ACTOR Future handleCompletedDeltaFile(Reference bwData, Reference metadata, BlobFileIndex completedDeltaFile, @@ -1062,10 +1008,10 @@ ACTOR Future blobGranuleUpdateFiles(Reference bwData, state Future changeFeedFuture; state GranuleStartState startState; state bool readOldChangeFeed; - state bool lastFromOldChangeFeed = false; state Optional> oldChangeFeedDataComplete; state Key cfKey; state Optional oldCFKey; + state NotifiedVersion committedVersion; state std::deque> rollbacksInProgress; state std::deque> rollbacksCompleted; @@ -1153,23 +1099,19 @@ ACTOR Future blobGranuleUpdateFiles(Reference bwData, metadata->durableDeltaVersion.set(startVersion); metadata->pendingDeltaVersion = startVersion; metadata->bufferedDeltaVersion.set(startVersion); + committedVersion.set(startVersion); ASSERT(metadata->readable.canBeSet()); metadata->readable.send(Void()); - if (startState.parentGranule.present()) { - // FIXME: once we have empty versions, only include up to startState.changeFeedStartVersion in the read - // stream. Then we can just stop the old stream when we get end_of_stream from this and not handle the - // mutation version truncation stuff - - // FIXME: filtering on key range != change feed range doesn't work + if (startState.parentGranule.present() && startVersion < startState.changeFeedStartVersion) { + // read from parent change feed up until our new change feed is started readOldChangeFeed = true; - oldChangeFeedFuture = - bwData->db->getChangeFeedStream(oldChangeFeedStream, - oldCFKey.get(), - startVersion + 1, - MAX_VERSION, - startState.parentGranule.get().first /*metadata->keyRange*/); + oldChangeFeedFuture = bwData->db->getChangeFeedStream(oldChangeFeedStream, + oldCFKey.get(), + startVersion + 1, + startState.changeFeedStartVersion, + metadata->keyRange); } else { readOldChangeFeed = false; changeFeedFuture = bwData->db->getChangeFeedStream( @@ -1219,27 +1161,37 @@ ACTOR Future blobGranuleUpdateFiles(Reference bwData, } state Standalone> mutations; - if (readOldChangeFeed) { - Standalone> oldMutations = waitNext(oldChangeFeedStream.getFuture()); - // TODO filter old mutations won't be necessary, SS does it already - if (filterOldMutations( - metadata->keyRange, &oldMutations, &mutations, startState.changeFeedStartVersion)) { - // if old change feed has caught up with where new one would start, finish last one and start new - // one - - Key cfKey = StringRef(startState.granuleID.toString()); - changeFeedFuture = bwData->db->getChangeFeedStream( - changeFeedStream, cfKey, startState.changeFeedStartVersion, MAX_VERSION, metadata->keyRange); - oldChangeFeedFuture.cancel(); - lastFromOldChangeFeed = true; - - // now that old change feed is cancelled, clear out any mutations still in buffer by replacing - // promise stream - oldChangeFeedStream = PromiseStream>>(); + try { + if (readOldChangeFeed) { + // TODO efficient way to store next in mutations? + Standalone> oldMutations = + waitNext(oldChangeFeedStream.getFuture()); + mutations = oldMutations; + } else { + Standalone> newMutations = waitNext(changeFeedStream.getFuture()); + mutations = newMutations; } - } else { - Standalone> newMutations = waitNext(changeFeedStream.getFuture()); - mutations = newMutations; + } catch (Error& e) { + // only error we should expect here is when we finish consuming old change feed + if (e.code() != error_code_end_of_stream) { + throw; + } + ASSERT(readOldChangeFeed); + + readOldChangeFeed = false; + // set this so next delta file write updates granule split metadata to done + ASSERT(startState.parentGranule.present()); + oldChangeFeedDataComplete = startState.parentGranule.get(); + if (BW_DEBUG) { + printf("Granule [%s - %s) switching to new change feed %s @ %lld\n", + metadata->keyRange.begin.printable().c_str(), + metadata->keyRange.end.printable().c_str(), + startState.granuleID.toString().c_str(), + metadata->bufferedDeltaVersion.get()); + } + + changeFeedFuture = bwData->db->getChangeFeedStream( + changeFeedStream, cfKey, startState.changeFeedStartVersion, MAX_VERSION, metadata->keyRange); } // process mutations @@ -1253,6 +1205,11 @@ ACTOR Future blobGranuleUpdateFiles(Reference bwData, if (deltas.version > lastVersion) { metadata->bufferedDeltaVersion.set(lastVersion); } + Version knownNoRollbacksPast = std::min(lastVersion, deltas.knownCommittedVersion); + if (knownNoRollbacksPast > committedVersion.get()) { + committedVersion.set(knownNoRollbacksPast); + } + // Write a new delta file IF we have enough bytes, and we have all of the previous version's stuff // there to ensure no versions span multiple delta files. Check this by ensuring the version of this // new delta is larger than the previous largest seen version @@ -1293,6 +1250,7 @@ ACTOR Future blobGranuleUpdateFiles(Reference bwData, metadata->currentDeltas, lastVersion, previousDeltaFileFuture, + &committedVersion, oldChangeFeedDataComplete); inFlightDeltaFiles.push_back( InFlightDeltaFile(dfFuture, lastVersion, metadata->bufferedDeltaBytes)); @@ -1502,17 +1460,22 @@ ACTOR Future blobGranuleUpdateFiles(Reference bwData, rollbacksInProgress, rollbacksCompleted); - // reset change feeds to cfRollbackVersion + // Reset change feeds to cfRollbackVersion if (readOldChangeFeed) { + // It shouldn't be possible to roll back across the parent/child feed boundary, + // because the transaction creating the child change feed had to commit before we + // got here. + ASSERT(cfRollbackVersion < startState.changeFeedStartVersion); oldChangeFeedStream = PromiseStream>>(); - oldChangeFeedFuture = bwData->db->getChangeFeedStream( - oldChangeFeedStream, - oldCFKey.get(), - cfRollbackVersion + 1, - MAX_VERSION, - startState.parentGranule.get().first /*metadata->keyRange*/); + oldChangeFeedFuture = + bwData->db->getChangeFeedStream(oldChangeFeedStream, + oldCFKey.get(), + cfRollbackVersion + 1, + startState.changeFeedStartVersion, + metadata->keyRange); } else { + ASSERT(cfRollbackVersion > startState.changeFeedStartVersion); changeFeedStream = PromiseStream>>(); changeFeedFuture = bwData->db->getChangeFeedStream(changeFeedStream, cfKey, @@ -1548,20 +1511,6 @@ ACTOR Future blobGranuleUpdateFiles(Reference bwData, } lastVersion = deltas.version; } - if (lastFromOldChangeFeed && !justDidRollback) { - readOldChangeFeed = false; - lastFromOldChangeFeed = false; - // set this so next delta file write updates granule split metadata to done - ASSERT(startState.parentGranule.present()); - oldChangeFeedDataComplete = startState.parentGranule.get(); - if (BW_DEBUG) { - printf("Granule [%s - %s) switching to new change feed %s @ %lld\n", - metadata->keyRange.begin.printable().c_str(), - metadata->keyRange.end.printable().c_str(), - startState.granuleID.toString().c_str(), - metadata->bufferedDeltaVersion.get()); - } - } justDidRollback = false; } } catch (Error& e) { @@ -2445,40 +2394,6 @@ ACTOR Future handleRangeRevoke(Reference bwData, RevokeBlo } } -// Because change feeds send uncommitted data and explicit rollback messages, we speculatively buffer/write -// uncommitted data. This means we must ensure the data is actually committed before "committing" those writes in -// the blob granule. The simplest way to do this is to have the blob worker do a periodic GRV, which is guaranteed -// to be an earlier committed version. -ACTOR Future runCommitVersionChecks(Reference bwData) { - state Transaction tr(bwData->db); - loop { - // only do grvs to get committed version if we need it to persist delta files - while (bwData->pendingDeltaFileCommitChecks.get() == 0) { - wait(bwData->pendingDeltaFileCommitChecks.onChange()); - } - - // batch potentially multiple delta files into one GRV, and also rate limit GRVs for this worker - wait(delay(0.1)); // TODO KNOB? - - state int checksToResolve = bwData->pendingDeltaFileCommitChecks.get(); - - tr.reset(); - try { - Version readVersion = wait(tr.getReadVersion()); - - ASSERT(readVersion >= bwData->knownCommittedVersion.get()); - if (readVersion > bwData->knownCommittedVersion.get()) { - ++bwData->knownCommittedCheckCount; - bwData->knownCommittedVersion.set(readVersion); - bwData->pendingDeltaFileCommitChecks.set(bwData->pendingDeltaFileCommitChecks.get() - checksToResolve); - } - ++bwData->stats.commitVersionChecks; - } catch (Error& e) { - wait(tr.onError(e)); - } - } -} - ACTOR Future blobWorker(BlobWorkerInterface bwInterf, ReplyPromise recruitReply, Reference const> dbInfo) { @@ -2531,7 +2446,6 @@ ACTOR Future blobWorker(BlobWorkerInterface bwInterf, recruitReply.send(rep); self->addActor.send(waitFailureServer(bwInterf.waitFailure.getFuture())); - self->addActor.send(runCommitVersionChecks(self)); try { loop choose { From 99606482eaeeedbd1a6cfca6625960ac3091c941 Mon Sep 17 00:00:00 2001 From: Suraj Gupta Date: Wed, 20 Oct 2021 11:54:19 -0400 Subject: [PATCH 003/413] initial thoughts --- fdbserver/BlobManager.actor.cpp | 154 +++++++++++++++++++++++++++++- fdbserver/BlobWorker.actor.cpp | 27 +++++- fdbserver/QuietDatabase.actor.cpp | 2 + 3 files changed, 181 insertions(+), 2 deletions(-) diff --git a/fdbserver/BlobManager.actor.cpp b/fdbserver/BlobManager.actor.cpp index 557352f11c..9e3a6c74a3 100644 --- a/fdbserver/BlobManager.actor.cpp +++ b/fdbserver/BlobManager.actor.cpp @@ -28,6 +28,7 @@ #include "fdbclient/SystemData.h" #include "fdbserver/BlobManagerInterface.h" #include "fdbserver/Knobs.h" +#include "fdbserver/QuietDatabase.h" #include "fdbserver/WaitFailure.h" #include "fdbserver/WorkerInterface.actor.h" #include "flow/IRandom.h" @@ -715,7 +716,32 @@ ACTOR Future maybeSplitRange(BlobManagerData* bmData, return Void(); } -void killBlobWorker(BlobManagerData* bmData, BlobWorkerInterface bwInterf) { +ACTOR Future deregisterBlobWorker(BlobManagerData* bmData, const BlobWorkerInterface& interf) { + state Reference tr = makeReference(bmData->db); + loop { + tr->setOption(FDBTransactionOptions::ACCESS_SYSTEM_KEYS); + tr->setOption(FDBTransactionOptions::PRIORITY_SYSTEM_IMMEDIATE); + try { + Key blobWorkerListKey = blobWorkerListKeyFor(interf.id()); + tr->addReadConflictRange(singleKeyRange(blobWorkerListKey)); + tr->clear(blobWorkerListKey); + + wait(tr->commit()); + + if (BM_DEBUG) { + printf("Deregistered blob worker %s\n", interf.id().toString().c_str()); + } + return Void(); + } catch (Error& e) { + if (BM_DEBUG) { + printf("Deregistering blob worker %s got error %s\n", interf.id().toString().c_str(), e.name()); + } + wait(tr->onError(e)); + } + } +} + +void killBlobWorker(BlobManagerData* bmData, const BlobWorkerInterface& bwInterf) { UID bwId = bwInterf.id(); // Remove blob worker from stats map so that when we try to find a worker to takeover the range, @@ -756,6 +782,8 @@ void killBlobWorker(BlobManagerData* bmData, BlobWorkerInterface bwInterf) { } bmData->addActor.send( brokenPromiseToNever(bwInterf.haltBlobWorker.getReply(HaltBlobWorkerRequest(bmData->epoch, bmData->id)))); + + wait(deregisterBlobWorker(bmData, bwInterf)); } ACTOR Future monitorBlobWorkerStatus(BlobManagerData* bmData, BlobWorkerInterface bwInterf) { @@ -868,6 +896,7 @@ ACTOR Future monitorBlobWorker(BlobManagerData* bmData, BlobWorkerInterfac } } } catch (Error& e) { + // will blob worker get cleaned up in this case? if (e.code() == error_code_operation_cancelled) { throw e; } @@ -895,6 +924,126 @@ ACTOR Future monitorBlobWorker(BlobManagerData* bmData, BlobWorkerInterfac return Void(); } +ACTOR Future ackExistingBlobWorkers(BlobManagerData* bmData) { + // get list of last known blob workers + // note: the list will include every blob worker that the old manager knew about + // but it might also contain blob workers that died while the new manager was being recruited + std::vector blobWorkers = wait(getBlobWorkers(bmData->db)); + + // add all blob workers to this new blob manager's records + for (auto worker : blobWorkers) { + bmData->workersById[worker.id()] = worker; + bmData->workerStats[worker.id()] = BlobWorkerStats(); + + // if the worker died when the new BM was being recruited, the monitor will fail and we + // will clean up the dead blob worker (and recruit new ones in place) + bmData->addActor.send(monitorBlobWorker(bmData, worker)); + } + + // TODO: anyway we can guarantee the monitor ran so that we have a complete set of alive BWs? + // At this point, bmData->workersById is a complete list of blob workers + + state Reference tr = makeReference(bmData->db); + + // TODO: if blob manager dies, and while another is being recruited, a blob worker dies, we need to handle sending + // the ranges for the dead blob worker to new workers. For every range persisted in the DB, send an assign request + // to the worker that it was intended for. If the worker indeed had it, its a noop. If the worker didn't have it, + // it'll persist it. If the worker is dead, i.e. it died while a new BM was being recruited, (easy check: just check + // if that worker is still a worker that we know about via getBlobWorkers), let other workers take over its ranges. + // + // TODO: confirm the below are solved now with the new way of checking BWs liveness. + // Problem: when a blob worker dies while a blob manager is being recruited, we wouldn't recruit new BW to replace + // that one, even though we should have. Easy fix: when going through the list of persisted blob workers + // assignments, if any of them are dead, just recruit a new one. Problem: if the blob worker that died didn't have + // any assignments, it wouldn't be in this list, and so we wouldn't rerecruit for it. + + // 1. Get existing assignments using blobGranuleMappingKeys.begin + // 2. Get split intentions using blobGranuleSplitKeys + // 3. Start sending assign requests to blob workers + // + // + // Cases to consider: + // - Blob Manager revokes and re-assigns a range but only the revoke went through before BM died. Then, + // granuleMappingKeys still has G->oldBW so + // the worst case here is that we end up giving G back to the oldBW + // - Blob Manager revokes and re-assigns a range but neither went through before BM died. Same as above. + // - While the BM is recovering, a BW dies. Then, when the BM comes back, it seems granuleMappings of the form + // G->deadBW. Since + // we have a complete list of alive BWs by now, we can just reassign these G's to the next best worker + // - BM persisted intent to split ranges but died before sending them. We just have to re-assign these. If any of + // the new workers died, then + // it's one of the cases above. + // + // We have to think about how to recreate bmData->granuleAssignments and the order in which to send these assigns. + // If we apply all of granuleMappingKeys in a keyrangemap and then apply the splits over it, I think that works. + // We can't just send assigns for granuleMappingKeys because entries there might be dated. + // + // We need a special type of assign req (something like continue) that if fails, doesn't try to find a different + // worker. The reason is that if we can guarantee the worker was alive by the time it got the req, then if the req + // failed, + // + state RangeResult blobGranuleMappings; + state RangeResult blobGranuleSplits; + // get assignments + loop { + tr->reset(); + try { + wait(checkManagerLock(tr, bmData)); + RangeResult _results = wait(krmGetRanges(tr, + blobGranuleMappingKeys.begin, + KeyRange(normalKeys), + GetRangeLimits::ROW_LIMIT_UNLIMITED, + GetRangeLimits::BYTE_LIMIT_UNLIMITED)); + blobGranuleMappings = _results; + if (blobGranuleMappings.more) { + } + + wait(tr->commit()); + } catch (Error& e) { + wait(tr->onError(e)); + } + } + // get splits + loop { + tr->reset(); + try { + wait(checkManagerLock(tr, bmData)); + RangeResult _results = wait(krmGetRanges(tr, + blobGranuleMappingKeys.begin, + KeyRange(normalKeys), + GetRangeLimits::ROW_LIMIT_UNLIMITED, + GetRangeLimits::BYTE_LIMIT_UNLIMITED)); + results = _results; + if (results.more) { + } + + wait(tr->commit()); + } catch (Error& e) { + wait(tr->onError(e)); + } + } + for (auto range : blobGranuleMappings) { + bmData->workerAssignments.insert(range.key, decodeBlobGranuleMappingValue(range.value)); + } + for (auto range : blobGranuleSplits) { + if bmData + ->workerAssignments.insert(range.key, decodeBlobGranuleMappingValue(range.value)); + } + + /* + for (range : bmData->workerAssignments) { + if (workersbyId.count(range.value)) { + } else { + range.value = UID(); + + } + } + send assign with worker = range.workerId + */ + + return Void(); +} + ACTOR Future chaosRangeMover(BlobManagerData* bmData) { ASSERT(g_network->isSimulated()); loop { @@ -1122,6 +1271,9 @@ ACTOR Future blobManager(BlobManagerInterface bmInterf, auto recruitBlobWorker = IAsyncListener>::create( dbInfo, [](auto const& info) { return info.clusterInterface.recruitBlobWorker; }); + // we need to acknowledge existing blob workers before recruiting any new ones + wait(ackExistingBlobWorkers(&self)); + self.addActor.send(blobWorkerRecruiter(&self, recruitBlobWorker)); self.addActor.send(monitorClientRanges(&self)); self.addActor.send(rangeAssigner(&self)); diff --git a/fdbserver/BlobWorker.actor.cpp b/fdbserver/BlobWorker.actor.cpp index d85d113e99..c818ae4e30 100644 --- a/fdbserver/BlobWorker.actor.cpp +++ b/fdbserver/BlobWorker.actor.cpp @@ -2385,6 +2385,31 @@ ACTOR Future registerBlobWorker(Reference bwData, BlobWork } } +ACTOR Future deregisterBlobWorker(Reference bwData, BlobWorkerInterface interf) { + state Reference tr = makeReference(bwData->db); + loop { + tr->setOption(FDBTransactionOptions::ACCESS_SYSTEM_KEYS); + tr->setOption(FDBTransactionOptions::PRIORITY_SYSTEM_IMMEDIATE); + try { + Key blobWorkerListKey = blobWorkerListKeyFor(interf.id()); + tr->addReadConflictRange(singleKeyRange(blobWorkerListKey)); + tr->clear(blobWorkerListKey); + + wait(tr->commit()); + + if (BW_DEBUG) { + printf("Deregistered blob worker %s\n", interf.id().toString().c_str()); + } + return Void(); + } catch (Error& e) { + if (BW_DEBUG) { + printf("Deregistering blob worker %s got error %s\n", interf.id().toString().c_str(), e.name()); + } + wait(tr->onError(e)); + } + } +} + ACTOR Future handleRangeAssign(Reference bwData, AssignBlobRangeRequest req, bool isSelfReassign) { @@ -2552,7 +2577,7 @@ ACTOR Future blobWorker(BlobWorkerInterface bwInterf, } when(AssignBlobRangeRequest _req = waitNext(bwInterf.assignBlobRangeRequest.getFuture())) { ++self->stats.rangeAssignmentRequests; - --self->stats.numRangesAssigned; + ++self->stats.numRangesAssigned; state AssignBlobRangeRequest assignReq = _req; if (BW_DEBUG) { printf("Worker %s assigned range [%s - %s) @ (%lld, %lld):\n continue=%s\n", diff --git a/fdbserver/QuietDatabase.actor.cpp b/fdbserver/QuietDatabase.actor.cpp index b3f6fc1c7a..d9d18fd553 100644 --- a/fdbserver/QuietDatabase.actor.cpp +++ b/fdbserver/QuietDatabase.actor.cpp @@ -19,6 +19,8 @@ */ #include +#include + #include "fdbclient/SystemData.h" #include "flow/ActorCollection.h" #include "fdbrpc/simulator.h" From 17b30f188a35c5d9a96757f85b0f22dc22159801 Mon Sep 17 00:00:00 2001 From: Suraj Gupta Date: Thu, 21 Oct 2021 17:39:38 -0400 Subject: [PATCH 004/413] Working impl --- fdbclient/BlobWorkerInterface.h | 3 +- fdbclient/NativeAPI.actor.cpp | 7 +- fdbserver/BlobManager.actor.cpp | 211 +++++++++++++++++++++++--------- fdbserver/BlobWorker.actor.cpp | 53 ++++---- 4 files changed, 181 insertions(+), 93 deletions(-) diff --git a/fdbclient/BlobWorkerInterface.h b/fdbclient/BlobWorkerInterface.h index f95bb87b74..15a8fc86e9 100644 --- a/fdbclient/BlobWorkerInterface.h +++ b/fdbclient/BlobWorkerInterface.h @@ -135,6 +135,7 @@ struct AssignBlobRangeRequest { // For an initial assignment, reassignent, split, or merge, continueAssignment==false. bool continueAssignment; + bool specialAssignment; ReplyPromise reply; @@ -142,7 +143,7 @@ struct AssignBlobRangeRequest { template void serialize(Ar& ar) { - serializer(ar, keyRange, managerEpoch, managerSeqno, continueAssignment, reply, arena); + serializer(ar, keyRange, managerEpoch, managerSeqno, continueAssignment, specialAssignment, reply, arena); } }; diff --git a/fdbclient/NativeAPI.actor.cpp b/fdbclient/NativeAPI.actor.cpp index 117524a43a..0115010d6b 100644 --- a/fdbclient/NativeAPI.actor.cpp +++ b/fdbclient/NativeAPI.actor.cpp @@ -7109,7 +7109,7 @@ Future DatabaseContext::popChangeFeedMutations(Key rangeID, Version versio return popChangeFeedMutationsActor(Reference::addRef(this), rangeID, version); } -#define BG_REQUEST_DEBUG false +#define BG_REQUEST_DEBUG true ACTOR Future getBlobGranuleRangesStreamActor(Reference db, PromiseStream results, @@ -7242,7 +7242,10 @@ ACTOR Future readBlobGranulesStreamActor(Reference db, if (!cx->blobWorker_interf.count(workerId)) { Optional workerInterface = wait(tr->get(blobWorkerListKeyFor(workerId))); - ASSERT(workerInterface.present()); + if (!workerInterface.present()) { + throw wrong_shard_server(); + } + // ASSERT(workerInterface.present()); cx->blobWorker_interf[workerId] = decodeBlobWorkerListValue(workerInterface.get()); if (BG_REQUEST_DEBUG) { printf(" decoded worker interface for %s\n", workerId.toString().c_str()); diff --git a/fdbserver/BlobManager.actor.cpp b/fdbserver/BlobManager.actor.cpp index 9e3a6c74a3..a453ae83cc 100644 --- a/fdbserver/BlobManager.actor.cpp +++ b/fdbserver/BlobManager.actor.cpp @@ -35,7 +35,7 @@ #include "flow/UnitTest.h" #include "flow/actorcompiler.h" // has to be last include -#define BM_DEBUG false +#define BM_DEBUG true // FIXME: change all BlobManagerData* to Reference to avoid segfaults if core loop gets error @@ -166,10 +166,13 @@ void getRanges(std::vector>& results, KeyRangeMap workerAssignments; KeyRangeMap knownBlobRanges; + AsyncTrigger startRecruiting; Debouncer restartRecruiting; std::set recruitingLocalities; // the addrs of the workers being recruited on + AsyncVar recruitingStream; int64_t epoch = -1; int64_t seqNo = 1; @@ -220,7 +225,7 @@ struct BlobManagerData { BlobManagerData(UID id, Database db) : id(id), db(db), knownBlobRanges(false, normalKeys.end), - restartRecruiting(SERVER_KNOBS->DEBOUNCE_RECRUITING_DELAY) {} + restartRecruiting(SERVER_KNOBS->DEBOUNCE_RECRUITING_DELAY), recruitingStream(0) {} ~BlobManagerData() { printf("Destroying blob manager data for %s\n", id.toString().c_str()); } }; @@ -269,9 +274,16 @@ ACTOR Future>> splitRange(Reference eligibleWorkers; +ACTOR Future pickWorkerForAssign(BlobManagerData* bmData) { + state int minGranulesAssigned = INT_MAX; + state std::vector eligibleWorkers; + + printf("workerStats.size()=%d, recruitingStream=%d\n", bmData->workerStats.size(), bmData->recruitingStream.get()); + while (bmData->workerStats.size() == 0) { + printf("looping waiting for workerstats\n"); + wait(bmData->recruitingStream.onChange()); + } + printf("workerStats.size()=%d\n", bmData->workerStats.size()); for (auto const& worker : bmData->workerStats) { UID currId = worker.first; @@ -322,6 +334,7 @@ ACTOR Future doRangeAssignment(BlobManagerData* bmData, RangeAssignment as req.managerEpoch = bmData->epoch; req.managerSeqno = seqNo; req.continueAssignment = assignment.assign.get().continueAssignment; + req.specialAssignment = assignment.assign.get().specialAssignment; // if that worker isn't alive anymore, add the range back into the stream if (bmData->workersById.count(workerID) == 0) { @@ -375,7 +388,10 @@ ACTOR Future doRangeAssignment(BlobManagerData* bmData, RangeAssignment as bmData->rangesToAssign.send(revokeOld); // send assignment back to queue as is, clearing designated worker if present - assignment.worker.reset(); + if (assignment.assign.get().specialAssignment && assignment.worker.get() != UID()) { + } else { + assignment.worker.reset(); + } bmData->rangesToAssign.send(assignment); // FIXME: improvement would be to add history of failed workers to assignment so it can try other ones first } else { @@ -409,12 +425,12 @@ ACTOR Future rangeAssigner(BlobManagerData* bmData) { if (BUGGIFY_WITH_PROB(0.05)) { wait(delay(deterministicRandom()->random01())); } - RangeAssignment assignment = waitNext(bmData->rangesToAssign.getFuture()); - int64_t seqNo = bmData->seqNo; + state RangeAssignment assignment = waitNext(bmData->rangesToAssign.getFuture()); + state int64_t seqNo = bmData->seqNo; bmData->seqNo++; // modify the in-memory assignment data structures, and send request off to worker - UID workerId; + state UID workerId; if (assignment.isAssign) { // Ensure range isn't currently assigned anywhere, and there is only 1 intersecting range auto currentAssignments = bmData->workerAssignments.intersectingRanges(assignment.keyRange); @@ -424,19 +440,26 @@ ACTOR Future rangeAssigner(BlobManagerData* bmData) { ASSERT(assignment.worker.present()); ASSERT(it.value() == assignment.worker.get()); } else { - ASSERT(it.value() == UID()); + // ASSERT(it.value() == UID()); } count++; } ASSERT(count == 1); - workerId = assignment.worker.present() ? assignment.worker.get() : pickWorkerForAssign(bmData); + if (assignment.worker.present() && assignment.worker.get().isValid()) { + workerId = assignment.worker.get(); + } else { + UID _workerId = wait(pickWorkerForAssign(bmData)); + workerId = _workerId; + } bmData->workerAssignments.insert(assignment.keyRange, workerId); + /* ASSERT(bmData->workerStats.count(workerId)); if (!assignment.assign.get().continueAssignment) { - bmData->workerStats[workerId].numGranulesAssigned += 1; + bmData->workerStats[workerId].numGranulesAssigned += 1; } + */ // FIXME: if range is assign, have some sort of semaphore for outstanding assignments so we don't assign // a ton ranges at once and blow up FDB with reading initial snapshots. @@ -716,7 +739,7 @@ ACTOR Future maybeSplitRange(BlobManagerData* bmData, return Void(); } -ACTOR Future deregisterBlobWorker(BlobManagerData* bmData, const BlobWorkerInterface& interf) { +ACTOR Future deregisterBlobWorker(BlobManagerData* bmData, BlobWorkerInterface interf) { state Reference tr = makeReference(bmData->db); loop { tr->setOption(FDBTransactionOptions::ACCESS_SYSTEM_KEYS); @@ -741,7 +764,7 @@ ACTOR Future deregisterBlobWorker(BlobManagerData* bmData, const BlobWorke } } -void killBlobWorker(BlobManagerData* bmData, const BlobWorkerInterface& bwInterf) { +ACTOR Future killBlobWorker(BlobManagerData* bmData, BlobWorkerInterface bwInterf) { UID bwId = bwInterf.id(); // Remove blob worker from stats map so that when we try to find a worker to takeover the range, @@ -750,6 +773,8 @@ void killBlobWorker(BlobManagerData* bmData, const BlobWorkerInterface& bwInterf // when we try to recruit new blob workers. bmData->workerStats.erase(bwId); bmData->workersById.erase(bwId); + Future deregister = deregisterBlobWorker(bmData, bwInterf); + bmData->restartRecruiting.trigger(); // for every range owned by this blob worker, we want to // - send a revoke request for that range @@ -783,7 +808,8 @@ void killBlobWorker(BlobManagerData* bmData, const BlobWorkerInterface& bwInterf bmData->addActor.send( brokenPromiseToNever(bwInterf.haltBlobWorker.getReply(HaltBlobWorkerRequest(bmData->epoch, bmData->id)))); - wait(deregisterBlobWorker(bmData, bwInterf)); + wait(deregister); + return Void(); } ACTOR Future monitorBlobWorkerStatus(BlobManagerData* bmData, BlobWorkerInterface bwInterf) { @@ -852,6 +878,10 @@ ACTOR Future monitorBlobWorkerStatus(BlobManagerData* bmData, BlobWorkerIn } } } catch (Error& e) { + printf("BM %s got error %s while monitoring BW %s---------------\n", + bmData->id.toString().c_str(), + e.name(), + bwInterf.id().toString().c_str()); if (e.code() == error_code_operation_cancelled) { throw e; } @@ -880,6 +910,7 @@ ACTOR Future monitorBlobWorkerStatus(BlobManagerData* bmData, BlobWorkerIn ACTOR Future monitorBlobWorker(BlobManagerData* bmData, BlobWorkerInterface bwInterf) { try { + printf("adding waitFailure for BW %s\n", bwInterf.id().toString().c_str()); state Future waitFailure = waitFailureClient(bwInterf.waitFailure, SERVER_KNOBS->BLOB_WORKER_TIMEOUT); state Future monitorStatus = monitorBlobWorkerStatus(bmData, bwInterf); @@ -910,13 +941,13 @@ ACTOR Future monitorBlobWorker(BlobManagerData* bmData, BlobWorkerInterfac } // kill the blob worker - killBlobWorker(bmData, bwInterf); + wait(killBlobWorker(bmData, bwInterf)); // Trigger recruitment for a new blob worker if (BM_DEBUG) { printf("Restarting recruitment to replace dead BW %s\n", bwInterf.id().toString().c_str()); } - bmData->restartRecruiting.trigger(); + // bmData->restartRecruiting.trigger(); if (BM_DEBUG) { printf("No longer monitoring BW %s\n", bwInterf.id().toString().c_str()); @@ -925,6 +956,7 @@ ACTOR Future monitorBlobWorker(BlobManagerData* bmData, BlobWorkerInterfac } ACTOR Future ackExistingBlobWorkers(BlobManagerData* bmData) { + printf("acking\n"); // get list of last known blob workers // note: the list will include every blob worker that the old manager knew about // but it might also contain blob workers that died while the new manager was being recruited @@ -937,15 +969,19 @@ ACTOR Future ackExistingBlobWorkers(BlobManagerData* bmData) { // if the worker died when the new BM was being recruited, the monitor will fail and we // will clean up the dead blob worker (and recruit new ones in place) + printf("adding monitor for BW %s\n", worker.id().toString().c_str()); bmData->addActor.send(monitorBlobWorker(bmData, worker)); } + bmData->startRecruiting.trigger(); - // TODO: anyway we can guarantee the monitor ran so that we have a complete set of alive BWs? + // TODO: is there anyway we can guarantee the monitor ran so that we have a complete set of alive BWs? + // A not so great way of doing this is timeouts // At this point, bmData->workersById is a complete list of blob workers + printf("after adding monitors\n"); state Reference tr = makeReference(bmData->db); - // TODO: if blob manager dies, and while another is being recruited, a blob worker dies, we need to handle sending + // If blob manager dies, and while another is being recruited, a blob worker dies, we need to handle sending // the ranges for the dead blob worker to new workers. For every range persisted in the DB, send an assign request // to the worker that it was intended for. If the worker indeed had it, its a noop. If the worker didn't have it, // it'll persist it. If the worker is dead, i.e. it died while a new BM was being recruited, (easy check: just check @@ -986,60 +1022,102 @@ ACTOR Future ackExistingBlobWorkers(BlobManagerData* bmData) { state RangeResult blobGranuleSplits; // get assignments loop { - tr->reset(); try { + tr->setOption(FDBTransactionOptions::ACCESS_SYSTEM_KEYS); + tr->setOption(FDBTransactionOptions::PRIORITY_SYSTEM_IMMEDIATE); + // tr->reset(); wait(checkManagerLock(tr, bmData)); - RangeResult _results = wait(krmGetRanges(tr, - blobGranuleMappingKeys.begin, - KeyRange(normalKeys), - GetRangeLimits::ROW_LIMIT_UNLIMITED, - GetRangeLimits::BYTE_LIMIT_UNLIMITED)); + RangeResult _results = wait(krmGetRanges( + tr, blobGranuleMappingKeys.begin, KeyRange(normalKeys), 10000, GetRangeLimits::BYTE_LIMIT_UNLIMITED)); blobGranuleMappings = _results; if (blobGranuleMappings.more) { + // TODO: accumulate resutls } - wait(tr->commit()); + wait(tr->commit()); // TODO: don't need commit + break; } catch (Error& e) { wait(tr->onError(e)); } } + + for (int rangeIdx = 0; rangeIdx < blobGranuleMappings.size() - 1; rangeIdx++) { + Key granuleStartKey = blobGranuleMappings[rangeIdx].key; + Key granuleEndKey = blobGranuleMappings[rangeIdx + 1].key; + if (blobGranuleMappings[rangeIdx].value.size()) { + UID existingOwner = decodeBlobGranuleMappingValue(blobGranuleMappings[rangeIdx].value); + UID newOwner = bmData->workersById.count(existingOwner) ? existingOwner : UID(); + printf("about to insert [%s-%s] into workerassignments\n", + granuleStartKey.printable().c_str(), + granuleStartKey.printable().c_str()); + bmData->workerAssignments.insert(KeyRangeRef(granuleStartKey, granuleEndKey), newOwner); + } + } + // get splits + tr->reset(); loop { - tr->reset(); try { + tr->setOption(FDBTransactionOptions::ACCESS_SYSTEM_KEYS); + tr->setOption(FDBTransactionOptions::PRIORITY_SYSTEM_IMMEDIATE); wait(checkManagerLock(tr, bmData)); - RangeResult _results = wait(krmGetRanges(tr, - blobGranuleMappingKeys.begin, - KeyRange(normalKeys), - GetRangeLimits::ROW_LIMIT_UNLIMITED, - GetRangeLimits::BYTE_LIMIT_UNLIMITED)); - results = _results; - if (results.more) { + // tr->reset(); + RangeResult _results = + wait(tr->getRange(KeyRangeRef(blobGranuleSplitKeys.begin, blobGranuleSplitKeys.end), 10000)); + + blobGranuleSplits = _results; + if (blobGranuleSplits.more) { + // TODO: accumulate resutls } - wait(tr->commit()); + wait(tr->commit()); // don't need commit + break; } catch (Error& e) { wait(tr->onError(e)); } } - for (auto range : blobGranuleMappings) { - bmData->workerAssignments.insert(range.key, decodeBlobGranuleMappingValue(range.value)); - } - for (auto range : blobGranuleSplits) { - if bmData - ->workerAssignments.insert(range.key, decodeBlobGranuleMappingValue(range.value)); + + for (auto split : blobGranuleSplits) { + UID parentGranuleID, granuleID; // + BlobGranuleSplitState splitState; // enum + Version version; // version at which split happened + if (split.expectedSize() == 0) { + continue; + } + std::tie(parentGranuleID, granuleID) = decodeBlobGranuleSplitKey(split.key); + std::tie(splitState, version) = decodeBlobGranuleSplitValue(split.value); + const KeyRange range = blobGranuleSplitKeyRangeFor(parentGranuleID); + UID owner = UID(); + // TODO: need actual owner otherwise retry logic will be messed up. we might have sent the + // req to a worker who didn't finish it yet and so the assignment is not in granuleAssignments + // and if we try to assign to someone else, there will be two workers owning the same range which is + // problematic. Is this problematic though? One of the seqno's will becomes stale! + // bmData->workerAssignments.insert(range, UID()); + if (splitState <= BlobGranuleSplitState::Started) { + printf("about to insert [%s-%s] into workerassignments\n", + range.begin.printable().c_str(), + range.end.printable().c_str()); + bmData->workerAssignments.insert(range, UID()); + } } - /* - for (range : bmData->workerAssignments) { - if (workersbyId.count(range.value)) { - } else { - range.value = UID(); + if (bmData->workerAssignments.size() == 1) { + return Void(); + } - } - } - send assign with worker = range.workerId - */ + for (auto& range : bmData->workerAssignments.ranges()) { + printf("assigning range [%s-%s]\n", + range.range().begin.printable().c_str(), + range.range().end.printable().c_str()); + // what should we do if we get here and the worker was alive, but between now and the assignment, the + // worker dies. what's the retry logic going to be? + RangeAssignment raAssign; + raAssign.isAssign = true; + raAssign.worker = range.value(); + raAssign.keyRange = range.range(); + raAssign.assign = RangeAssignmentData(false, true); // special assignment + bmData->rangesToAssign.send(raAssign); + } return Void(); } @@ -1069,8 +1147,8 @@ ACTOR Future chaosRangeMover(BlobManagerData* bmData) { randomRange.value().toString().c_str()); } - // FIXME: with low probability, could immediately revoke it from the new assignment and move it back - // right after to test that race + // FIXME: with low probability, could immediately revoke it from the new assignment and move + // it back right after to test that race RangeAssignment revokeOld; revokeOld.isAssign = false; @@ -1113,6 +1191,8 @@ int numExistingBWOnAddr(BlobManagerData* self, const AddressExclusion& addr) { ACTOR Future initializeBlobWorker(BlobManagerData* self, RecruitBlobWorkerReply candidateWorker) { const NetworkAddress& netAddr = candidateWorker.worker.stableAddress(); AddressExclusion workerAddr(netAddr.ip, netAddr.port); + self->recruitingStream.set(self->recruitingStream.get() + 1); + // Ask the candidateWorker to initialize a BW only if the worker does not have a pending request if (numExistingBWOnAddr(self, workerAddr) == 0 && self->recruitingLocalities.count(candidateWorker.worker.stableAddress()) == 0) { @@ -1157,6 +1237,7 @@ ACTOR Future initializeBlobWorker(BlobManagerData* self, RecruitBlobWorker if (newBlobWorker.present()) { BlobWorkerInterface bwi = newBlobWorker.get().interf; + printf("Adding worker %s to BM Records\n", bwi.id().toString().c_str()); self->workersById[bwi.id()] = bwi; self->workerStats[bwi.id()] = BlobWorkerStats(); self->addActor.send(monitorBlobWorker(self, bwi)); @@ -1176,6 +1257,7 @@ ACTOR Future initializeBlobWorker(BlobManagerData* self, RecruitBlobWorker } // try to recruit more blob workers + self->recruitingStream.set(self->recruitingStream.get() - 1); self->restartRecruiting.trigger(); return Void(); } @@ -1187,6 +1269,9 @@ ACTOR Future blobWorkerRecruiter( state Future fCandidateWorker; state RecruitBlobWorkerRequest lastRequest; + loop choose { + when(wait(self->startRecruiting.onTrigger())) { break; } + } loop { try { state RecruitBlobWorkerRequest recruitReq; @@ -1203,6 +1288,11 @@ ACTOR Future blobWorkerRecruiter( recruitReq.excludeAddresses.emplace_back(AddressExclusion(addr.ip, addr.port)); } + printf("Recruiting now. Excluding: \n"); + for (auto addr : recruitReq.excludeAddresses) { + printf(" - %s\n", addr.toString().c_str()); + } + TraceEvent("BMRecruiting").detail("State", "Sending request to CC"); if (!fCandidateWorker.isValid() || fCandidateWorker.isReady() || @@ -1214,8 +1304,10 @@ ACTOR Future blobWorkerRecruiter( } choose { - // when we get back a worker we can use, we will try to initialize a blob worker onto that process + // when we get back a worker we can use, we will try to initialize a blob worker onto that + // process when(RecruitBlobWorkerReply candidateWorker = wait(fCandidateWorker)) { + printf("About to initialize BW\n"); self->addActor.send(initializeBlobWorker(self, candidateWorker)); } @@ -1267,14 +1359,15 @@ ACTOR Future blobManager(BlobManagerInterface bmInterf, printf("Blob manager acquired lock at epoch %lld\n", epoch); } - // needed to pick up changes to dbinfo in case new CC comes along auto recruitBlobWorker = IAsyncListener>::create( dbInfo, [](auto const& info) { return info.clusterInterface.recruitBlobWorker; }); + self.addActor.send(blobWorkerRecruiter(&self, recruitBlobWorker)); + // we need to acknowledge existing blob workers before recruiting any new ones wait(ackExistingBlobWorkers(&self)); - self.addActor.send(blobWorkerRecruiter(&self, recruitBlobWorker)); + // self.addActor.send(blobWorkerRecruiter(&self, recruitBlobWorker)); self.addActor.send(monitorClientRanges(&self)); self.addActor.send(rangeAssigner(&self)); @@ -1319,8 +1412,8 @@ ACTOR Future blobManager(BlobManagerInterface bmInterf, // DB has [B - D). It should show up coalesced in knownBlobRanges, and [C - D) should be removed. // DB has [A - D). It should show up coalesced in knownBlobRanges, and [A - B) should be removed. // DB has [A - B) and [C - D). They should show up in knownBlobRanges, and [B - C) should be in removed. -// DB has [B - C). It should show up in knownBlobRanges, [B - C) should be in added, and [A - B) and [C - D) should -// be in removed. +// DB has [B - C). It should show up in knownBlobRanges, [B - C) should be in added, and [A - B) and [C - D) +// should be in removed. TEST_CASE("/blobmanager/updateranges") { KeyRangeMap knownBlobRanges(false, normalKeys.end); Arena ar; diff --git a/fdbserver/BlobWorker.actor.cpp b/fdbserver/BlobWorker.actor.cpp index c818ae4e30..03804e8015 100644 --- a/fdbserver/BlobWorker.actor.cpp +++ b/fdbserver/BlobWorker.actor.cpp @@ -42,8 +42,8 @@ #include "flow/IRandom.h" #include "flow/actorcompiler.h" // has to be last include -#define BW_DEBUG false -#define BW_REQUEST_DEBUG false +#define BW_DEBUG true +#define BW_REQUEST_DEBUG true // TODO add comments + documentation struct BlobFileIndex { @@ -2220,7 +2220,8 @@ ACTOR Future changeBlobRange(Reference bwData, int64_t seqno, bool active, bool disposeOnCleanup, - bool selfReassign) { + bool selfReassign, + bool specialAssignment = false) { if (BW_DEBUG) { printf("%s range for [%s - %s): %s @ (%lld, %lld)\n", selfReassign ? "Re-assigning" : "Changing", @@ -2253,6 +2254,16 @@ ACTOR Future changeBlobRange(Reference bwData, } } bool thisAssignmentNewer = newerRangeAssignment(r.value(), epoch, seqno); + + // if this granule already has it, and this was a specialassignment (i.e. a new blob maanger is trying to + // reassign granules), then just continue + if (specialAssignment && r.begin() == keyRange.begin && r.end() == keyRange.end) { + r.value().lastEpoch = epoch; + r.value().lastSeqno = seqno; + alreadyAssigned = true; + break; + } + if (r.value().lastEpoch == epoch && r.value().lastSeqno == seqno) { ASSERT(r.begin() == keyRange.begin); ASSERT(r.end() == keyRange.end); @@ -2318,7 +2329,6 @@ ACTOR Future changeBlobRange(Reference bwData, bwData->granuleMetadata.insert(it.first, it.second); } - printf("returning from changeblobrange"); wait(waitForAll(futures)); return true; } @@ -2385,31 +2395,6 @@ ACTOR Future registerBlobWorker(Reference bwData, BlobWork } } -ACTOR Future deregisterBlobWorker(Reference bwData, BlobWorkerInterface interf) { - state Reference tr = makeReference(bwData->db); - loop { - tr->setOption(FDBTransactionOptions::ACCESS_SYSTEM_KEYS); - tr->setOption(FDBTransactionOptions::PRIORITY_SYSTEM_IMMEDIATE); - try { - Key blobWorkerListKey = blobWorkerListKeyFor(interf.id()); - tr->addReadConflictRange(singleKeyRange(blobWorkerListKey)); - tr->clear(blobWorkerListKey); - - wait(tr->commit()); - - if (BW_DEBUG) { - printf("Deregistered blob worker %s\n", interf.id().toString().c_str()); - } - return Void(); - } catch (Error& e) { - if (BW_DEBUG) { - printf("Deregistering blob worker %s got error %s\n", interf.id().toString().c_str(), e.name()); - } - wait(tr->onError(e)); - } - } -} - ACTOR Future handleRangeAssign(Reference bwData, AssignBlobRangeRequest req, bool isSelfReassign) { @@ -2417,8 +2402,14 @@ ACTOR Future handleRangeAssign(Reference bwData, if (req.continueAssignment) { resumeBlobRange(bwData, req.keyRange, req.managerEpoch, req.managerSeqno); } else { - bool shouldStart = wait( - changeBlobRange(bwData, req.keyRange, req.managerEpoch, req.managerSeqno, true, false, isSelfReassign)); + bool shouldStart = wait(changeBlobRange(bwData, + req.keyRange, + req.managerEpoch, + req.managerSeqno, + true, + false, + isSelfReassign, + req.specialAssignment)); if (shouldStart) { auto m = bwData->granuleMetadata.rangeContaining(req.keyRange.begin); From 5a9d9921d0e53937b508541071c0614802b4e7df Mon Sep 17 00:00:00 2001 From: Suraj Gupta Date: Fri, 22 Oct 2021 22:41:19 -0400 Subject: [PATCH 005/413] Fixes and final cleanup for BM failure handling --- fdbclient/BlobWorkerInterface.h | 13 +- fdbclient/NativeAPI.actor.cpp | 1 - fdbserver/BlobManager.actor.cpp | 279 +++++++++++++++----------------- fdbserver/BlobWorker.actor.cpp | 87 +++++++--- 4 files changed, 201 insertions(+), 179 deletions(-) diff --git a/fdbclient/BlobWorkerInterface.h b/fdbclient/BlobWorkerInterface.h index 15a8fc86e9..c40a64ee28 100644 --- a/fdbclient/BlobWorkerInterface.h +++ b/fdbclient/BlobWorkerInterface.h @@ -124,6 +124,13 @@ struct RevokeBlobRangeRequest { } }; +/* + * Continue: when a worker should continue handling a granule that was evaluated for a split + * Reassign: when a new blob manager takes over, it sends Reassign requests to workers to redistribute granules + * Normal: Neither continue nor reassign + */ +enum AssignRequestType { Normal = 0, Continue = 1, Reassign = 2 }; + struct AssignBlobRangeRequest { constexpr static FileIdentifier file_identifier = 905381; Arena arena; @@ -133,9 +140,7 @@ struct AssignBlobRangeRequest { // If continueAssignment is true, this is just to instruct the worker that it *still* owns the range, so it should // re-snapshot it and continue. - // For an initial assignment, reassignent, split, or merge, continueAssignment==false. - bool continueAssignment; - bool specialAssignment; + AssignRequestType type; ReplyPromise reply; @@ -143,7 +148,7 @@ struct AssignBlobRangeRequest { template void serialize(Ar& ar) { - serializer(ar, keyRange, managerEpoch, managerSeqno, continueAssignment, specialAssignment, reply, arena); + serializer(ar, keyRange, managerEpoch, managerSeqno, type, reply, arena); } }; diff --git a/fdbclient/NativeAPI.actor.cpp b/fdbclient/NativeAPI.actor.cpp index 0115010d6b..e16ef335f2 100644 --- a/fdbclient/NativeAPI.actor.cpp +++ b/fdbclient/NativeAPI.actor.cpp @@ -7245,7 +7245,6 @@ ACTOR Future readBlobGranulesStreamActor(Reference db, if (!workerInterface.present()) { throw wrong_shard_server(); } - // ASSERT(workerInterface.present()); cx->blobWorker_interf[workerId] = decodeBlobWorkerListValue(workerInterface.get()); if (BG_REQUEST_DEBUG) { printf(" decoded worker interface for %s\n", workerId.toString().c_str()); diff --git a/fdbserver/BlobManager.actor.cpp b/fdbserver/BlobManager.actor.cpp index a453ae83cc..b145d770a5 100644 --- a/fdbserver/BlobManager.actor.cpp +++ b/fdbserver/BlobManager.actor.cpp @@ -166,13 +166,10 @@ void getRanges(std::vector>& results, KeyRangeMap pickWorkerForAssign(BlobManagerData* bmData) { state int minGranulesAssigned = INT_MAX; state std::vector eligibleWorkers; - printf("workerStats.size()=%d, recruitingStream=%d\n", bmData->workerStats.size(), bmData->recruitingStream.get()); + // wait until there are BWs to pick from while (bmData->workerStats.size() == 0) { - printf("looping waiting for workerstats\n"); + bmData->restartRecruiting.trigger(); wait(bmData->recruitingStream.onChange()); } - printf("workerStats.size()=%d\n", bmData->workerStats.size()); for (auto const& worker : bmData->workerStats) { UID currId = worker.first; @@ -333,8 +329,7 @@ ACTOR Future doRangeAssignment(BlobManagerData* bmData, RangeAssignment as StringRef(req.arena, assignment.keyRange.end)); req.managerEpoch = bmData->epoch; req.managerSeqno = seqNo; - req.continueAssignment = assignment.assign.get().continueAssignment; - req.specialAssignment = assignment.assign.get().specialAssignment; + req.type = assignment.assign.get().type; // if that worker isn't alive anymore, add the range back into the stream if (bmData->workersById.count(workerID) == 0) { @@ -370,6 +365,9 @@ ACTOR Future doRangeAssignment(BlobManagerData* bmData, RangeAssignment as } } } catch (Error& e) { + if (e.code() == error_code_operation_cancelled) { + throw; + } // TODO confirm: using reliable delivery this should only trigger if the worker is marked as failed, right? // So assignment needs to be retried elsewhere, and a revoke is trivially complete if (assignment.isAssign) { @@ -388,10 +386,7 @@ ACTOR Future doRangeAssignment(BlobManagerData* bmData, RangeAssignment as bmData->rangesToAssign.send(revokeOld); // send assignment back to queue as is, clearing designated worker if present - if (assignment.assign.get().specialAssignment && assignment.worker.get() != UID()) { - } else { - assignment.worker.reset(); - } + assignment.worker.reset(); bmData->rangesToAssign.send(assignment); // FIXME: improvement would be to add history of failed workers to assignment so it can try other ones first } else { @@ -436,12 +431,14 @@ ACTOR Future rangeAssigner(BlobManagerData* bmData) { auto currentAssignments = bmData->workerAssignments.intersectingRanges(assignment.keyRange); int count = 0; for (auto& it : currentAssignments) { - if (assignment.assign.get().continueAssignment) { - ASSERT(assignment.worker.present()); - ASSERT(it.value() == assignment.worker.get()); + /* TODO: rething asserts here + if (assignment.assign.get().type == AssignRequestType::Continue) { + ASSERT(assignment.worker.present()); + ASSERT(it.value() == assignment.worker.get()); } else { - // ASSERT(it.value() == UID()); + ASSERT(it.value() == UID()); } + */ count++; } ASSERT(count == 1); @@ -449,17 +446,21 @@ ACTOR Future rangeAssigner(BlobManagerData* bmData) { if (assignment.worker.present() && assignment.worker.get().isValid()) { workerId = assignment.worker.get(); } else { + if (BM_DEBUG) { + printf("About to pick worker for seqno %d in BM %s\n", seqNo, bmData->id.toString().c_str()); + } UID _workerId = wait(pickWorkerForAssign(bmData)); + if (BM_DEBUG) { + printf("Found worker BW %s for seqno %d\n", _workerId.toString().c_str(), seqNo); + } workerId = _workerId; } bmData->workerAssignments.insert(assignment.keyRange, workerId); - /* - ASSERT(bmData->workerStats.count(workerId)); - if (!assignment.assign.get().continueAssignment) { - bmData->workerStats[workerId].numGranulesAssigned += 1; + // If we know about the worker and this is not a continue, then this is a new range for the worker + if (bmData->workerStats.count(workerId) && assignment.assign.get().type != AssignRequestType::Continue) { + bmData->workerStats[workerId].numGranulesAssigned += 1; } - */ // FIXME: if range is assign, have some sort of semaphore for outstanding assignments so we don't assign // a ton ranges at once and blow up FDB with reading initial snapshots. @@ -554,7 +555,6 @@ ACTOR Future monitorClientRanges(BlobManagerData* bmData) { state std::vector>>> splitFutures; // Divide new ranges up into equal chunks by using SS byte sample for (KeyRangeRef range : rangesToAdd) { - // assert that this range contains no currently assigned ranges in this splitFutures.push_back(splitRange(tr, range)); } @@ -569,15 +569,19 @@ ACTOR Future monitorClientRanges(BlobManagerData* bmData) { for (int i = 0; i < splits.size() - 1; i++) { KeyRange range = KeyRange(KeyRangeRef(splits[i], splits[i + 1])); - if (BM_DEBUG) { - printf(" [%s - %s)\n", range.begin.printable().c_str(), range.end.printable().c_str()); - } + // only add the client range if this is the first BM or it's not already assigned + if (bmData->epoch == 1 || bmData->workerAssignments.intersectingRanges(range).empty()) { + if (BM_DEBUG) { + printf( + " [%s - %s)\n", range.begin.printable().c_str(), range.end.printable().c_str()); + } - RangeAssignment ra; - ra.isAssign = true; - ra.keyRange = range; - ra.assign = RangeAssignmentData(false); // continue=false - bmData->rangesToAssign.send(ra); + RangeAssignment ra; + ra.isAssign = true; + ra.keyRange = range; + ra.assign = RangeAssignmentData(); // type=normal + bmData->rangesToAssign.send(ra); + } } } @@ -626,7 +630,7 @@ ACTOR Future maybeSplitRange(BlobManagerData* bmData, raContinue.isAssign = true; raContinue.worker = currentWorkerId; raContinue.keyRange = granuleRange; - raContinue.assign = RangeAssignmentData(true); // continue assignment and re-snapshot + raContinue.assign = RangeAssignmentData(AssignRequestType::Continue); // continue assignment and re-snapshot bmData->rangesToAssign.send(raContinue); return Void(); } @@ -731,7 +735,7 @@ ACTOR Future maybeSplitRange(BlobManagerData* bmData, RangeAssignment raAssignSplit; raAssignSplit.isAssign = true; raAssignSplit.keyRange = KeyRangeRef(newRanges[i], newRanges[i + 1]); - raAssignSplit.assign = RangeAssignmentData(false); + raAssignSplit.assign = RangeAssignmentData(); // don't care who this range gets assigned to bmData->rangesToAssign.send(raAssignSplit); } @@ -773,7 +777,11 @@ ACTOR Future killBlobWorker(BlobManagerData* bmData, BlobWorkerInterface b // when we try to recruit new blob workers. bmData->workerStats.erase(bwId); bmData->workersById.erase(bwId); + + // Remove blob worker from persisted list of blob workers Future deregister = deregisterBlobWorker(bmData, bwInterf); + + // restart recruiting to replace the dead blob worker bmData->restartRecruiting.trigger(); // for every range owned by this blob worker, we want to @@ -796,7 +804,7 @@ ACTOR Future killBlobWorker(BlobManagerData* bmData, BlobWorkerInterface b raAssign.isAssign = true; raAssign.worker = Optional(); raAssign.keyRange = it.range(); - raAssign.assign = RangeAssignmentData(false); // not a continue + raAssign.assign = RangeAssignmentData(); // not a continue bmData->rangesToAssign.send(raAssign); } } @@ -841,6 +849,12 @@ ACTOR Future monitorBlobWorkerStatus(BlobManagerData* bmData, BlobWorkerIn if (bmData->iAmReplaced.canBeSet()) { bmData->iAmReplaced.send(Void()); } + } else if (rep.epoch < bmData->epoch) { + // TODO: revoke the range from that worker? and send optimistic halt req to other (zombie) BM? it's + // optimistic because such a BM is not necessarily a zombie. it could have gotten killed properly + // but the BW that sent this reply was behind (i.e. it started the req when the old BM was in charge + // and finished by the time the new BM took over) + continue; } // TODO maybe this won't be true eventually, but right now the only time the blob worker reports back is @@ -878,18 +892,18 @@ ACTOR Future monitorBlobWorkerStatus(BlobManagerData* bmData, BlobWorkerIn } } } catch (Error& e) { - printf("BM %s got error %s while monitoring BW %s---------------\n", - bmData->id.toString().c_str(), - e.name(), - bwInterf.id().toString().c_str()); if (e.code() == error_code_operation_cancelled) { throw e; } + + // TODO: figure out why waitFailure in monitorBlobWorker doesn't pick up the connection failure first + if (e.code() == error_code_connection_failed || e.code() == error_code_broken_promise) { + throw e; + } + // if we got an error constructing or reading from stream that is retryable, wait and retry. ASSERT(e.code() != error_code_end_of_stream); - if (e.code() == error_code_connection_failed || e.code() == error_code_request_maybe_delivered) { - // FIXME: this could throw connection_failed and we could handle catch this the same as the failure - // detection triggering + if (e.code() == error_code_request_maybe_delivered) { wait(delay(FLOW_KNOBS->PREVENT_FAST_SPIN_DELAY)); continue; } else { @@ -931,24 +945,22 @@ ACTOR Future monitorBlobWorker(BlobManagerData* bmData, BlobWorkerInterfac if (e.code() == error_code_operation_cancelled) { throw e; } - // FIXME: forward errors somewhere from here - if (BM_DEBUG) { - printf("BM got unexpected error %s monitoring BW %s\n", e.name(), bwInterf.id().toString().c_str()); + + // TODO: re-evaluate the expected errors here once wait failure issue is resolved + // Expected errors here are: [connection_failed, broken_promise] + if (e.code() != error_code_connection_failed && e.code() != error_code_broken_promise) { + if (BM_DEBUG) { + printf("BM got unexpected error %s monitoring BW %s\n", e.name(), bwInterf.id().toString().c_str()); + } + // TODO change back from SevError? + TraceEvent(SevError, "BWMonitoringFailed", bmData->id).detail("BlobWorkerID", bwInterf.id()).error(e); + throw e; } - // TODO change back from SevError? - TraceEvent(SevError, "BWMonitoringFailed", bmData->id).detail("BlobWorkerID", bwInterf.id()).error(e); - throw e; } // kill the blob worker wait(killBlobWorker(bmData, bwInterf)); - // Trigger recruitment for a new blob worker - if (BM_DEBUG) { - printf("Restarting recruitment to replace dead BW %s\n", bwInterf.id().toString().c_str()); - } - // bmData->restartRecruiting.trigger(); - if (BM_DEBUG) { printf("No longer monitoring BW %s\n", bwInterf.id().toString().c_str()); } @@ -956,166 +968,133 @@ ACTOR Future monitorBlobWorker(BlobManagerData* bmData, BlobWorkerInterfac } ACTOR Future ackExistingBlobWorkers(BlobManagerData* bmData) { - printf("acking\n"); - // get list of last known blob workers - // note: the list will include every blob worker that the old manager knew about + // skip this entire algorithm for the first blob manager + if (bmData->epoch == 1) { + bmData->startRecruiting.trigger(); + return Void(); + } + + // Get list of last known blob workers + // note: the list will include every blob worker that the old manager knew about, // but it might also contain blob workers that died while the new manager was being recruited std::vector blobWorkers = wait(getBlobWorkers(bmData->db)); - // add all blob workers to this new blob manager's records + // add all blob workers to this new blob manager's records and start monitoring it for (auto worker : blobWorkers) { bmData->workersById[worker.id()] = worker; bmData->workerStats[worker.id()] = BlobWorkerStats(); - - // if the worker died when the new BM was being recruited, the monitor will fail and we - // will clean up the dead blob worker (and recruit new ones in place) - printf("adding monitor for BW %s\n", worker.id().toString().c_str()); bmData->addActor.send(monitorBlobWorker(bmData, worker)); } + + // Once we acknowledge the existing blob workers, we can go ahead and recruit new ones bmData->startRecruiting.trigger(); - // TODO: is there anyway we can guarantee the monitor ran so that we have a complete set of alive BWs? - // A not so great way of doing this is timeouts - // At this point, bmData->workersById is a complete list of blob workers + // At this point, bmData->workersById is a list of all alive blob workers, but could also include some dead BWs. + // The algorithm below works as follows: + // 1. We get the existing granule mappings that were persisted by blob workers who were assigned ranges and + // add them to bmData->granuleAssignments, which is a key range map. + // Details: re-assignments might have happened between the time the mapping was last updated and now. + // For example, suppose a blob manager sends requests to the range assigner stream to move a granule G. + // However, before sending those requests off to the workers, the BM dies. So the persisting mapping + // still has G->oldWorker. The following algorithm will re-assign G to oldWorker (as long as it is also still + // alive). Note that this is fine because it simply means that the range was not moved optimally, but it is still + // owned. In the above case, even if the revoke goes through, since we don't update the mapping during revokes, + // this is the same as the case above. Another case to consider is when a blob worker dies when the BM is + // recovering. Now the mapping at this time looks like G->deadBW. But the rangeAssigner handles this: we'll try + // to assign a range to a dead worker and fail and reassign it to the next best worker. + // + // 2. We get the existing split intentions that were Started but not acknowledged by any blob workers and + // add them to our key range map, bmData->granuleAssignments. Note that we are adding them on top of + // the granule mappings and since we are using a key range map, we end up with the same set of shard + // boundaries as the old blob manager had. For these splits, we simply assign the range to the next + // best worker. This is not any worst than what the old blob manager would have done. + // Details: Note that this means that if a worker we intended to give a splitted range to dies + // before the new BM recovers, then we'll simply assign the range to the next best worker. + // + // 3. For every range in our granuleAssignments, we send an assign request to the stream of requests, + // ultimately giving every range back to some worker (trying to mimic the state of the old BM). + // If the worker already had the range, this is a no-op. If the worker didn't have it, it will + // begin persisting it. The worker that had the same range before will now be at a lower seqno. - printf("after adding monitors\n"); state Reference tr = makeReference(bmData->db); - - // If blob manager dies, and while another is being recruited, a blob worker dies, we need to handle sending - // the ranges for the dead blob worker to new workers. For every range persisted in the DB, send an assign request - // to the worker that it was intended for. If the worker indeed had it, its a noop. If the worker didn't have it, - // it'll persist it. If the worker is dead, i.e. it died while a new BM was being recruited, (easy check: just check - // if that worker is still a worker that we know about via getBlobWorkers), let other workers take over its ranges. - // - // TODO: confirm the below are solved now with the new way of checking BWs liveness. - // Problem: when a blob worker dies while a blob manager is being recruited, we wouldn't recruit new BW to replace - // that one, even though we should have. Easy fix: when going through the list of persisted blob workers - // assignments, if any of them are dead, just recruit a new one. Problem: if the blob worker that died didn't have - // any assignments, it wouldn't be in this list, and so we wouldn't rerecruit for it. - - // 1. Get existing assignments using blobGranuleMappingKeys.begin - // 2. Get split intentions using blobGranuleSplitKeys - // 3. Start sending assign requests to blob workers - // - // - // Cases to consider: - // - Blob Manager revokes and re-assigns a range but only the revoke went through before BM died. Then, - // granuleMappingKeys still has G->oldBW so - // the worst case here is that we end up giving G back to the oldBW - // - Blob Manager revokes and re-assigns a range but neither went through before BM died. Same as above. - // - While the BM is recovering, a BW dies. Then, when the BM comes back, it seems granuleMappings of the form - // G->deadBW. Since - // we have a complete list of alive BWs by now, we can just reassign these G's to the next best worker - // - BM persisted intent to split ranges but died before sending them. We just have to re-assign these. If any of - // the new workers died, then - // it's one of the cases above. - // - // We have to think about how to recreate bmData->granuleAssignments and the order in which to send these assigns. - // If we apply all of granuleMappingKeys in a keyrangemap and then apply the splits over it, I think that works. - // We can't just send assigns for granuleMappingKeys because entries there might be dated. - // - // We need a special type of assign req (something like continue) that if fails, doesn't try to find a different - // worker. The reason is that if we can guarantee the worker was alive by the time it got the req, then if the req - // failed, - // state RangeResult blobGranuleMappings; state RangeResult blobGranuleSplits; - // get assignments + + // Step 1. Get the latest known mapping of granules to blob workers (i.e. assignments) loop { try { tr->setOption(FDBTransactionOptions::ACCESS_SYSTEM_KEYS); tr->setOption(FDBTransactionOptions::PRIORITY_SYSTEM_IMMEDIATE); - // tr->reset(); wait(checkManagerLock(tr, bmData)); + RangeResult _results = wait(krmGetRanges( tr, blobGranuleMappingKeys.begin, KeyRange(normalKeys), 10000, GetRangeLimits::BYTE_LIMIT_UNLIMITED)); blobGranuleMappings = _results; + if (blobGranuleMappings.more) { // TODO: accumulate resutls } - - wait(tr->commit()); // TODO: don't need commit break; } catch (Error& e) { wait(tr->onError(e)); } } + // Step 1. Add the mappings to our in memory key range map for (int rangeIdx = 0; rangeIdx < blobGranuleMappings.size() - 1; rangeIdx++) { Key granuleStartKey = blobGranuleMappings[rangeIdx].key; Key granuleEndKey = blobGranuleMappings[rangeIdx + 1].key; if (blobGranuleMappings[rangeIdx].value.size()) { + // note: if the old owner is dead, we handle this in rangeAssigner UID existingOwner = decodeBlobGranuleMappingValue(blobGranuleMappings[rangeIdx].value); - UID newOwner = bmData->workersById.count(existingOwner) ? existingOwner : UID(); - printf("about to insert [%s-%s] into workerassignments\n", - granuleStartKey.printable().c_str(), - granuleStartKey.printable().c_str()); - bmData->workerAssignments.insert(KeyRangeRef(granuleStartKey, granuleEndKey), newOwner); + bmData->workerAssignments.insert(KeyRangeRef(granuleStartKey, granuleEndKey), existingOwner); } } - // get splits + // Step 2. Get the latest known split intentions tr->reset(); loop { try { tr->setOption(FDBTransactionOptions::ACCESS_SYSTEM_KEYS); tr->setOption(FDBTransactionOptions::PRIORITY_SYSTEM_IMMEDIATE); wait(checkManagerLock(tr, bmData)); - // tr->reset(); + RangeResult _results = wait(tr->getRange(KeyRangeRef(blobGranuleSplitKeys.begin, blobGranuleSplitKeys.end), 10000)); - blobGranuleSplits = _results; + if (blobGranuleSplits.more) { // TODO: accumulate resutls } - - wait(tr->commit()); // don't need commit break; } catch (Error& e) { wait(tr->onError(e)); } } + // Step 2. Add the granules for the started split intentions to the in-memory key range map for (auto split : blobGranuleSplits) { - UID parentGranuleID, granuleID; // - BlobGranuleSplitState splitState; // enum - Version version; // version at which split happened + UID parentGranuleID, granuleID; + BlobGranuleSplitState splitState; + Version version; if (split.expectedSize() == 0) { continue; } std::tie(parentGranuleID, granuleID) = decodeBlobGranuleSplitKey(split.key); std::tie(splitState, version) = decodeBlobGranuleSplitValue(split.value); const KeyRange range = blobGranuleSplitKeyRangeFor(parentGranuleID); - UID owner = UID(); - // TODO: need actual owner otherwise retry logic will be messed up. we might have sent the - // req to a worker who didn't finish it yet and so the assignment is not in granuleAssignments - // and if we try to assign to someone else, there will be two workers owning the same range which is - // problematic. Is this problematic though? One of the seqno's will becomes stale! - // bmData->workerAssignments.insert(range, UID()); if (splitState <= BlobGranuleSplitState::Started) { - printf("about to insert [%s-%s] into workerassignments\n", - range.begin.printable().c_str(), - range.end.printable().c_str()); bmData->workerAssignments.insert(range, UID()); } } - if (bmData->workerAssignments.size() == 1) { - return Void(); - } - - for (auto& range : bmData->workerAssignments.ranges()) { - printf("assigning range [%s-%s]\n", - range.range().begin.printable().c_str(), - range.range().end.printable().c_str()); - // what should we do if we get here and the worker was alive, but between now and the assignment, the - // worker dies. what's the retry logic going to be? + // Step 3. Send assign requests for all the granules + for (auto& range : bmData->workerAssignments.intersectingRanges(normalKeys)) { RangeAssignment raAssign; raAssign.isAssign = true; raAssign.worker = range.value(); raAssign.keyRange = range.range(); - raAssign.assign = RangeAssignmentData(false, true); // special assignment + raAssign.assign = RangeAssignmentData(AssignRequestType::Reassign); bmData->rangesToAssign.send(raAssign); } @@ -1159,7 +1138,7 @@ ACTOR Future chaosRangeMover(BlobManagerData* bmData) { RangeAssignment assignNew; assignNew.isAssign = true; assignNew.keyRange = randomRange.range(); - assignNew.assign = RangeAssignmentData(false); // not a continue + assignNew.assign = RangeAssignmentData(); // not a continue bmData->rangesToAssign.send(assignNew); break; } @@ -1237,7 +1216,6 @@ ACTOR Future initializeBlobWorker(BlobManagerData* self, RecruitBlobWorker if (newBlobWorker.present()) { BlobWorkerInterface bwi = newBlobWorker.get().interf; - printf("Adding worker %s to BM Records\n", bwi.id().toString().c_str()); self->workersById[bwi.id()] = bwi; self->workerStats[bwi.id()] = BlobWorkerStats(); self->addActor.send(monitorBlobWorker(self, bwi)); @@ -1269,9 +1247,11 @@ ACTOR Future blobWorkerRecruiter( state Future fCandidateWorker; state RecruitBlobWorkerRequest lastRequest; + // wait until existing blob workers have been acknowledged so we don't break recruitment invariants loop choose { when(wait(self->startRecruiting.onTrigger())) { break; } } + loop { try { state RecruitBlobWorkerRequest recruitReq; @@ -1288,11 +1268,6 @@ ACTOR Future blobWorkerRecruiter( recruitReq.excludeAddresses.emplace_back(AddressExclusion(addr.ip, addr.port)); } - printf("Recruiting now. Excluding: \n"); - for (auto addr : recruitReq.excludeAddresses) { - printf(" - %s\n", addr.toString().c_str()); - } - TraceEvent("BMRecruiting").detail("State", "Sending request to CC"); if (!fCandidateWorker.isValid() || fCandidateWorker.isReady() || @@ -1307,7 +1282,6 @@ ACTOR Future blobWorkerRecruiter( // when we get back a worker we can use, we will try to initialize a blob worker onto that // process when(RecruitBlobWorkerReply candidateWorker = wait(fCandidateWorker)) { - printf("About to initialize BW\n"); self->addActor.send(initializeBlobWorker(self, candidateWorker)); } @@ -1359,15 +1333,14 @@ ACTOR Future blobManager(BlobManagerInterface bmInterf, printf("Blob manager acquired lock at epoch %lld\n", epoch); } + // although we start the recruiter, we wait until existing workers are ack'd auto recruitBlobWorker = IAsyncListener>::create( dbInfo, [](auto const& info) { return info.clusterInterface.recruitBlobWorker; }); - self.addActor.send(blobWorkerRecruiter(&self, recruitBlobWorker)); - // we need to acknowledge existing blob workers before recruiting any new ones + // we need to acknowledge existing blob workers and their assignments before recruiting any new ones wait(ackExistingBlobWorkers(&self)); - // self.addActor.send(blobWorkerRecruiter(&self, recruitBlobWorker)); self.addActor.send(monitorClientRanges(&self)); self.addActor.send(rangeAssigner(&self)); diff --git a/fdbserver/BlobWorker.actor.cpp b/fdbserver/BlobWorker.actor.cpp index 03804e8015..be3a82156c 100644 --- a/fdbserver/BlobWorker.actor.cpp +++ b/fdbserver/BlobWorker.actor.cpp @@ -119,8 +119,9 @@ struct GranuleMetadata : NonCopyable, ReferenceCounted { AssignBlobRangeRequest originalReq; void resume() { - ASSERT(resumeSnapshot.canBeSet()); - resumeSnapshot.send(Void()); + if (resumeSnapshot.canBeSet()) { + resumeSnapshot.send(Void()); + } } }; @@ -134,6 +135,13 @@ struct GranuleRangeMetadata { Future fileUpdaterFuture; Future historyLoaderFuture; + void cancel() { + // assignFuture.cancel(); + // fileUpdaterFuture.cancel(); + // historyLoaderFuture.cancel(); + activeMetadata.clear(); + } + GranuleRangeMetadata() : lastEpoch(0), lastSeqno(0) {} GranuleRangeMetadata(int64_t epoch, int64_t seqno, Reference activeMetadata) : lastEpoch(epoch), lastSeqno(seqno), activeMetadata(activeMetadata) {} @@ -1025,6 +1033,8 @@ static Version doGranuleRollback(Reference metadata, // delete all deltas in rollback range, but we can optimize here to just skip the uncommitted mutations // directly and immediately pop the rollback out of inProgress + // TODO: bufferedDeltaVersion is of type Notified so you must set it to something greater than the previous val + // This does not hold true for rollback versions. metadata->bufferedDeltaVersion.set(rollbackVersion); cfRollbackVersion = mutationVersion; } @@ -1385,10 +1395,17 @@ ACTOR Future blobGranuleUpdateFiles(Reference bwData, } } + // TODO: figure out why the status stream on change isn't working + // We could just do something like statusEpoch, save down the original status stream + // and compare it to the current one + if (statusEpoch < bwData->currentManagerEpoch) { + break; + } + choose { + when(wait(bwData->currentManagerStatusStream.onChange())) {} when(wait(metadata->resumeSnapshot.getFuture())) { break; } when(wait(delay(1.0))) {} - when(wait(bwData->currentManagerStatusStream.onChange())) {} } if (BW_DEBUG) { @@ -1575,6 +1592,7 @@ ACTOR Future blobGranuleUpdateFiles(Reference bwData, if (e.code() == error_code_granule_assignment_conflict) { TraceEvent(SevInfo, "GranuleAssignmentConflict", bwData->id).detail("Granule", metadata->keyRange); + return Void(); } else { ++bwData->stats.granuleUpdateErrors; if (BW_DEBUG) { @@ -1687,9 +1705,12 @@ ACTOR Future blobGranuleLoadHistory(Reference bwData, metadata->historyLoaded.send(Void()); return Void(); } catch (Error& e) { - if (e.code() == error_code_operation_cancelled || e.code() == error_code_granule_assignment_conflict) { + if (e.code() == error_code_operation_cancelled) { throw e; } + if (e.code() == error_code_granule_assignment_conflict) { + return Void(); + } if (BW_DEBUG) { printf("Loading blob granule history got unexpected error %s\n", e.name()); } @@ -1978,6 +1999,11 @@ ACTOR Future handleBlobGranuleFileRequest(Reference bwData req.reply.send(rep); --bwData->stats.activeReadRequests; } catch (Error& e) { + if (e.code() == error_code_operation_cancelled) { + req.reply.sendError(wrong_shard_server()); + throw; + } + if (e.code() == error_code_wrong_shard_server) { ++bwData->stats.wrongShardServer; } @@ -2006,7 +2032,7 @@ ACTOR Future> getLatestGranuleHistory(Transaction* tr, } ACTOR Future openGranule(Reference bwData, AssignBlobRangeRequest req) { - ASSERT(!req.continueAssignment); + ASSERT(req.type != AssignRequestType::Continue); state Transaction tr(bwData->db); state Key lockKey = blobGranuleLockKeyFor(req.keyRange); @@ -2221,7 +2247,11 @@ ACTOR Future changeBlobRange(Reference bwData, bool active, bool disposeOnCleanup, bool selfReassign, - bool specialAssignment = false) { + Optional assignType = Optional()) { + // since changeBlobRange is used for assigns and revokes, + // we assert that assign type is specified iff this is an + ASSERT(active == assignType.present()); + if (BW_DEBUG) { printf("%s range for [%s - %s): %s @ (%lld, %lld)\n", selfReassign ? "Re-assigning" : "Changing", @@ -2254,16 +2284,20 @@ ACTOR Future changeBlobRange(Reference bwData, } } bool thisAssignmentNewer = newerRangeAssignment(r.value(), epoch, seqno); + printf("thisAssignmentNewer=%s\n", thisAssignmentNewer ? "true" : "false"); // if this granule already has it, and this was a specialassignment (i.e. a new blob maanger is trying to // reassign granules), then just continue - if (specialAssignment && r.begin() == keyRange.begin && r.end() == keyRange.end) { + if (active && assignType.get() == AssignRequestType::Reassign && r.begin() == keyRange.begin && + r.end() == keyRange.end) { r.value().lastEpoch = epoch; r.value().lastSeqno = seqno; alreadyAssigned = true; break; } + printf("last: (%d, %d). now: (%d, %d)\n", r.value().lastEpoch, r.value().lastSeqno, epoch, seqno); + if (r.value().lastEpoch == epoch && r.value().lastSeqno == seqno) { ASSERT(r.begin() == keyRange.begin); ASSERT(r.end() == keyRange.end); @@ -2290,10 +2324,16 @@ ACTOR Future changeBlobRange(Reference bwData, r.value().lastEpoch, r.value().lastSeqno); } - r.value().activeMetadata.clear(); + r.value().cancel(); } else if (!thisAssignmentNewer) { // this assignment is outdated, re-insert it over the current range + // TODO: do we really want to do this?? if its an outdated assignment, shouldnt + // we not include it anymore newerRanges.push_back(std::pair(r.range(), r.value())); + /* + alreadyAssigned = true; + break; + */ } } @@ -2330,7 +2370,7 @@ ACTOR Future changeBlobRange(Reference bwData, } wait(waitForAll(futures)); - return true; + return newerRanges.size() == 0; } static bool resumeBlobRange(Reference bwData, KeyRange keyRange, int64_t epoch, int64_t seqno) { @@ -2399,22 +2439,19 @@ ACTOR Future handleRangeAssign(Reference bwData, AssignBlobRangeRequest req, bool isSelfReassign) { try { - if (req.continueAssignment) { + if (req.type == AssignRequestType::Continue) { resumeBlobRange(bwData, req.keyRange, req.managerEpoch, req.managerSeqno); } else { - bool shouldStart = wait(changeBlobRange(bwData, - req.keyRange, - req.managerEpoch, - req.managerSeqno, - true, - false, - isSelfReassign, - req.specialAssignment)); + + bool shouldStart = wait(changeBlobRange( + bwData, req.keyRange, req.managerEpoch, req.managerSeqno, true, false, isSelfReassign, req.type)); if (shouldStart) { auto m = bwData->granuleMetadata.rangeContaining(req.keyRange.begin); ASSERT(m.begin() == req.keyRange.begin && m.end() == req.keyRange.end); - wait(start(bwData, &m.value(), req)); + if (m.value().activeMetadata.isValid()) { + wait(start(bwData, &m.value(), req)); + } } } if (!isSelfReassign) { @@ -2424,15 +2461,23 @@ ACTOR Future handleRangeAssign(Reference bwData, return Void(); } catch (Error& e) { if (BW_DEBUG) { - printf("AssignRange [%s - %s) got error %s\n", + printf("AssignRange [%s - %s) in BW %s got error %s\n", req.keyRange.begin.printable().c_str(), req.keyRange.end.printable().c_str(), + bwData->id.toString().c_str(), e.name()); } if (!isSelfReassign) { + if (e.code() == error_code_granule_assignment_conflict) { + req.reply.sendError(e); + bwData->stats.numRangesAssigned--; + return Void(); + } + if (canReplyWith(e)) { req.reply.sendError(e); + // TODO: should we just return here rather than throw and kill BW } } @@ -2577,7 +2622,7 @@ ACTOR Future blobWorker(BlobWorkerInterface bwInterf, assignReq.keyRange.end.printable().c_str(), assignReq.managerEpoch, assignReq.managerSeqno, - assignReq.continueAssignment ? "T" : "F"); + assignReq.type == AssignRequestType::Continue ? "T" : "F"); } if (self->managerEpochOk(assignReq.managerEpoch)) { From 51c0f78b5948b09d3f179725e03aba3f00028e95 Mon Sep 17 00:00:00 2001 From: Josh Slocum Date: Tue, 26 Oct 2021 16:34:23 -0500 Subject: [PATCH 006/413] fixing bug when removing simple config --- .../workloads/ConsistencyCheck.actor.cpp | 23 +++++++++---------- 1 file changed, 11 insertions(+), 12 deletions(-) diff --git a/fdbserver/workloads/ConsistencyCheck.actor.cpp b/fdbserver/workloads/ConsistencyCheck.actor.cpp index 3ec5a0f98a..0184617892 100644 --- a/fdbserver/workloads/ConsistencyCheck.actor.cpp +++ b/fdbserver/workloads/ConsistencyCheck.actor.cpp @@ -1996,25 +1996,24 @@ struct ConsistencyCheckWorkload : TestWorkload { int numBlobWorkerProcesses = 0; for (const auto& worker : workers) { NetworkAddress addr = worker.interf.stableAddress(); + bool inCCDc = worker.interf.locality.dcId() == ccDcId; if (!configuration.isExcludedServer(worker.interf.addresses())) { if (worker.processClass == ProcessClass::BlobWorkerClass) { numBlobWorkerProcesses++; - // this is a worker with processClass == BWClass, so should have exactly one blob worker - if (blobWorkersByAddr[addr] == 0) { - TraceEvent("ConsistencyCheck_NoBWsOnBWClass") + // this is a worker with processClass == BWClass, so should have exactly one blob worker if it's in + // the same DC + int desiredBlobWorkersOnAddr = inCCDc ? 1 : 0; + + if (blobWorkersByAddr[addr] != desiredBlobWorkersOnAddr) { + TraceEvent("ConsistencyCheck_WrongBWCountOnBWClass") .detail("Address", addr) - .detail("NumBlobWorkersOnAddr", blobWorkersByAddr[addr]); + .detail("NumBlobWorkersOnAddr", blobWorkersByAddr[addr]) + .detail("DesiredBlobWorkersOnAddr", desiredBlobWorkersOnAddr) + .detail("BwDcId", worker.interf.locality.dcId()) + .detail("CcDcId", ccDcId); return false; } - /* TODO: replace above code with this once blob manager recovery is handled - if (blobWorkersByAddr[addr] != 1) { - TraceEvent("ConsistencyCheck_NoBWOrManyBWsOnBWClass") - .detail("Address", addr) - .detail("NumBlobWorkersOnAddr", blobWorkersByAddr[addr]); - return false; - } - */ } else { // this is a worker with processClass != BWClass, so there should be no BWs on it if (blobWorkersByAddr[addr] > 0) { From fd50a011b15dd8155e226780d6ae5aa3ace2e0a1 Mon Sep 17 00:00:00 2001 From: Suraj Gupta Date: Tue, 26 Oct 2021 21:39:41 -0400 Subject: [PATCH 007/413] Address PR comments. --- fdbclient/NativeAPI.actor.cpp | 3 + fdbserver/BlobManager.actor.cpp | 100 ++++++++++++++++---------------- fdbserver/BlobWorker.actor.cpp | 13 +---- 3 files changed, 57 insertions(+), 59 deletions(-) diff --git a/fdbclient/NativeAPI.actor.cpp b/fdbclient/NativeAPI.actor.cpp index e16ef335f2..2800c079eb 100644 --- a/fdbclient/NativeAPI.actor.cpp +++ b/fdbclient/NativeAPI.actor.cpp @@ -7242,6 +7242,9 @@ ACTOR Future readBlobGranulesStreamActor(Reference db, if (!cx->blobWorker_interf.count(workerId)) { Optional workerInterface = wait(tr->get(blobWorkerListKeyFor(workerId))); + // from the time the mapping was read from the db, the associated blob worker + // could have died and so its interface wouldn't be present as part of the blobWorkerList + // we persist in the db. So throw wrong_shard_server to get the new mapping if (!workerInterface.present()) { throw wrong_shard_server(); } diff --git a/fdbserver/BlobManager.actor.cpp b/fdbserver/BlobManager.actor.cpp index b145d770a5..f66e97ff92 100644 --- a/fdbserver/BlobManager.actor.cpp +++ b/fdbserver/BlobManager.actor.cpp @@ -272,15 +272,15 @@ ACTOR Future>> splitRange(Reference pickWorkerForAssign(BlobManagerData* bmData) { - state int minGranulesAssigned = INT_MAX; - state std::vector eligibleWorkers; - // wait until there are BWs to pick from while (bmData->workerStats.size() == 0) { bmData->restartRecruiting.trigger(); wait(bmData->recruitingStream.onChange()); } + int minGranulesAssigned = INT_MAX; + std::vector eligibleWorkers; + for (auto const& worker : bmData->workerStats) { UID currId = worker.first; int granulesAssigned = worker.second.numGranulesAssigned; @@ -431,7 +431,7 @@ ACTOR Future rangeAssigner(BlobManagerData* bmData) { auto currentAssignments = bmData->workerAssignments.intersectingRanges(assignment.keyRange); int count = 0; for (auto& it : currentAssignments) { - /* TODO: rething asserts here + /* TODO: rethink asserts here if (assignment.assign.get().type == AssignRequestType::Continue) { ASSERT(assignment.worker.present()); ASSERT(it.value() == assignment.worker.get()); @@ -924,7 +924,6 @@ ACTOR Future monitorBlobWorkerStatus(BlobManagerData* bmData, BlobWorkerIn ACTOR Future monitorBlobWorker(BlobManagerData* bmData, BlobWorkerInterface bwInterf) { try { - printf("adding waitFailure for BW %s\n", bwInterf.id().toString().c_str()); state Future waitFailure = waitFailureClient(bwInterf.waitFailure, SERVER_KNOBS->BLOB_WORKER_TIMEOUT); state Future monitorStatus = monitorBlobWorkerStatus(bmData, bwInterf); @@ -967,7 +966,7 @@ ACTOR Future monitorBlobWorker(BlobManagerData* bmData, BlobWorkerInterfac return Void(); } -ACTOR Future ackExistingBlobWorkers(BlobManagerData* bmData) { +ACTOR Future recoverBlobManager(BlobManagerData* bmData) { // skip this entire algorithm for the first blob manager if (bmData->epoch == 1) { bmData->startRecruiting.trigger(); @@ -1017,77 +1016,79 @@ ACTOR Future ackExistingBlobWorkers(BlobManagerData* bmData) { // begin persisting it. The worker that had the same range before will now be at a lower seqno. state Reference tr = makeReference(bmData->db); - state RangeResult blobGranuleMappings; - state RangeResult blobGranuleSplits; // Step 1. Get the latest known mapping of granules to blob workers (i.e. assignments) + state KeyRef beginKey = normalKeys.begin; loop { try { tr->setOption(FDBTransactionOptions::ACCESS_SYSTEM_KEYS); tr->setOption(FDBTransactionOptions::PRIORITY_SYSTEM_IMMEDIATE); wait(checkManagerLock(tr, bmData)); - RangeResult _results = wait(krmGetRanges( - tr, blobGranuleMappingKeys.begin, KeyRange(normalKeys), 10000, GetRangeLimits::BYTE_LIMIT_UNLIMITED)); - blobGranuleMappings = _results; + // TODO: replace row limit with knob + KeyRange nextRange(KeyRangeRef(beginKey, normalKeys.end)); + RangeResult results = wait( + krmGetRanges(tr, blobGranuleMappingKeys.begin, nextRange, 10000, GetRangeLimits::BYTE_LIMIT_UNLIMITED)); - if (blobGranuleMappings.more) { - // TODO: accumulate resutls + // Add the mappings to our in memory key range map + for (int rangeIdx = 0; rangeIdx < results.size() - 1; rangeIdx++) { + Key granuleStartKey = results[rangeIdx].key; + Key granuleEndKey = results[rangeIdx + 1].key; + if (results[rangeIdx].value.size()) { + // note: if the old owner is dead, we handle this in rangeAssigner + UID existingOwner = decodeBlobGranuleMappingValue(results[rangeIdx].value); + bmData->workerAssignments.insert(KeyRangeRef(granuleStartKey, granuleEndKey), existingOwner); + } } - break; + + if (!results.more) { + break; + } + + beginKey = results.readThrough.get(); } catch (Error& e) { wait(tr->onError(e)); } } - // Step 1. Add the mappings to our in memory key range map - for (int rangeIdx = 0; rangeIdx < blobGranuleMappings.size() - 1; rangeIdx++) { - Key granuleStartKey = blobGranuleMappings[rangeIdx].key; - Key granuleEndKey = blobGranuleMappings[rangeIdx + 1].key; - if (blobGranuleMappings[rangeIdx].value.size()) { - // note: if the old owner is dead, we handle this in rangeAssigner - UID existingOwner = decodeBlobGranuleMappingValue(blobGranuleMappings[rangeIdx].value); - bmData->workerAssignments.insert(KeyRangeRef(granuleStartKey, granuleEndKey), existingOwner); - } - } - // Step 2. Get the latest known split intentions tr->reset(); + beginKey = blobGranuleSplitKeys.begin; loop { try { tr->setOption(FDBTransactionOptions::ACCESS_SYSTEM_KEYS); tr->setOption(FDBTransactionOptions::PRIORITY_SYSTEM_IMMEDIATE); wait(checkManagerLock(tr, bmData)); - RangeResult _results = - wait(tr->getRange(KeyRangeRef(blobGranuleSplitKeys.begin, blobGranuleSplitKeys.end), 10000)); - blobGranuleSplits = _results; + // TODO: replace row limit with knob + RangeResult results = wait(tr->getRange(KeyRangeRef(beginKey, blobGranuleSplitKeys.end), 10000)); - if (blobGranuleSplits.more) { - // TODO: accumulate resutls + // Add the granules for the started split intentions to the in-memory key range map + for (auto split : results) { + UID parentGranuleID, granuleID; + BlobGranuleSplitState splitState; + Version version; + if (split.expectedSize() == 0) { + continue; + } + std::tie(parentGranuleID, granuleID) = decodeBlobGranuleSplitKey(split.key); + std::tie(splitState, version) = decodeBlobGranuleSplitValue(split.value); + const KeyRange range = blobGranuleSplitKeyRangeFor(parentGranuleID); + if (splitState <= BlobGranuleSplitState::Started) { + bmData->workerAssignments.insert(range, UID()); + } } - break; + + if (!results.more) { + break; + } + + beginKey = results.readThrough.get(); } catch (Error& e) { wait(tr->onError(e)); } } - // Step 2. Add the granules for the started split intentions to the in-memory key range map - for (auto split : blobGranuleSplits) { - UID parentGranuleID, granuleID; - BlobGranuleSplitState splitState; - Version version; - if (split.expectedSize() == 0) { - continue; - } - std::tie(parentGranuleID, granuleID) = decodeBlobGranuleSplitKey(split.key); - std::tie(splitState, version) = decodeBlobGranuleSplitValue(split.value); - const KeyRange range = blobGranuleSplitKeyRangeFor(parentGranuleID); - if (splitState <= BlobGranuleSplitState::Started) { - bmData->workerAssignments.insert(range, UID()); - } - } - // Step 3. Send assign requests for all the granules for (auto& range : bmData->workerAssignments.intersectingRanges(normalKeys)) { RangeAssignment raAssign; @@ -1338,8 +1339,9 @@ ACTOR Future blobManager(BlobManagerInterface bmInterf, dbInfo, [](auto const& info) { return info.clusterInterface.recruitBlobWorker; }); self.addActor.send(blobWorkerRecruiter(&self, recruitBlobWorker)); - // we need to acknowledge existing blob workers and their assignments before recruiting any new ones - wait(ackExistingBlobWorkers(&self)); + // we need to recover the old blob manager's state (e.g. granule assignments) before + // before the new blob manager does anything + wait(recoverBlobManager(&self)); self.addActor.send(monitorClientRanges(&self)); self.addActor.send(rangeAssigner(&self)); diff --git a/fdbserver/BlobWorker.actor.cpp b/fdbserver/BlobWorker.actor.cpp index be3a82156c..44d7b7f172 100644 --- a/fdbserver/BlobWorker.actor.cpp +++ b/fdbserver/BlobWorker.actor.cpp @@ -2280,6 +2280,7 @@ ACTOR Future changeBlobRange(Reference bwData, if (BW_DEBUG) { printf("Cancelling activeMetadata\n"); } + bwData->stats.numRangesAssigned--; r.value().activeMetadata->cancelled.send(Void()); } } @@ -2326,14 +2327,8 @@ ACTOR Future changeBlobRange(Reference bwData, } r.value().cancel(); } else if (!thisAssignmentNewer) { - // this assignment is outdated, re-insert it over the current range - // TODO: do we really want to do this?? if its an outdated assignment, shouldnt - // we not include it anymore + // re-insert the known newer range over this existing range newerRanges.push_back(std::pair(r.range(), r.value())); - /* - alreadyAssigned = true; - break; - */ } } @@ -2442,11 +2437,11 @@ ACTOR Future handleRangeAssign(Reference bwData, if (req.type == AssignRequestType::Continue) { resumeBlobRange(bwData, req.keyRange, req.managerEpoch, req.managerSeqno); } else { - bool shouldStart = wait(changeBlobRange( bwData, req.keyRange, req.managerEpoch, req.managerSeqno, true, false, isSelfReassign, req.type)); if (shouldStart) { + bwData->stats.numRangesAssigned++; auto m = bwData->granuleMetadata.rangeContaining(req.keyRange.begin); ASSERT(m.begin() == req.keyRange.begin && m.end() == req.keyRange.end); if (m.value().activeMetadata.isValid()) { @@ -2613,7 +2608,6 @@ ACTOR Future blobWorker(BlobWorkerInterface bwInterf, } when(AssignBlobRangeRequest _req = waitNext(bwInterf.assignBlobRangeRequest.getFuture())) { ++self->stats.rangeAssignmentRequests; - ++self->stats.numRangesAssigned; state AssignBlobRangeRequest assignReq = _req; if (BW_DEBUG) { printf("Worker %s assigned range [%s - %s) @ (%lld, %lld):\n continue=%s\n", @@ -2633,7 +2627,6 @@ ACTOR Future blobWorker(BlobWorkerInterface bwInterf, } when(RevokeBlobRangeRequest _req = waitNext(bwInterf.revokeBlobRangeRequest.getFuture())) { state RevokeBlobRangeRequest revokeReq = _req; - --self->stats.numRangesAssigned; if (BW_DEBUG) { printf("Worker %s revoked range [%s - %s) @ (%lld, %lld):\n dispose=%s\n", self->id.toString().c_str(), From 134aef6011964d64742437cfcb0abbec7a9674d5 Mon Sep 17 00:00:00 2001 From: Suraj Gupta Date: Wed, 27 Oct 2021 10:40:03 -0400 Subject: [PATCH 008/413] Fix compiler warning. --- fdbserver/BlobManager.actor.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/fdbserver/BlobManager.actor.cpp b/fdbserver/BlobManager.actor.cpp index f66e97ff92..e828056a86 100644 --- a/fdbserver/BlobManager.actor.cpp +++ b/fdbserver/BlobManager.actor.cpp @@ -430,7 +430,7 @@ ACTOR Future rangeAssigner(BlobManagerData* bmData) { // Ensure range isn't currently assigned anywhere, and there is only 1 intersecting range auto currentAssignments = bmData->workerAssignments.intersectingRanges(assignment.keyRange); int count = 0; - for (auto& it : currentAssignments) { + for (auto i = currentAssignments.begin(); i != currentAssignments.end(); ++i) { /* TODO: rethink asserts here if (assignment.assign.get().type == AssignRequestType::Continue) { ASSERT(assignment.worker.present()); From 19076ad4d2670cc197ca58fb7be08ea363b79fea Mon Sep 17 00:00:00 2001 From: Josh Slocum Date: Wed, 27 Oct 2021 16:23:51 -0500 Subject: [PATCH 009/413] Changed blob manager checkin to by async and to have blob worker speculatively consume change feed ahead of re-snapshot --- fdbclient/NativeAPI.actor.cpp | 2 +- fdbserver/BlobManager.actor.cpp | 2 +- fdbserver/BlobWorker.actor.cpp | 673 ++++++++++++++------------- fdbserver/SimulatedCluster.actor.cpp | 5 +- 4 files changed, 347 insertions(+), 335 deletions(-) diff --git a/fdbclient/NativeAPI.actor.cpp b/fdbclient/NativeAPI.actor.cpp index 2800c079eb..3e5022bf68 100644 --- a/fdbclient/NativeAPI.actor.cpp +++ b/fdbclient/NativeAPI.actor.cpp @@ -7109,7 +7109,7 @@ Future DatabaseContext::popChangeFeedMutations(Key rangeID, Version versio return popChangeFeedMutationsActor(Reference::addRef(this), rangeID, version); } -#define BG_REQUEST_DEBUG true +#define BG_REQUEST_DEBUG false ACTOR Future getBlobGranuleRangesStreamActor(Reference db, PromiseStream results, diff --git a/fdbserver/BlobManager.actor.cpp b/fdbserver/BlobManager.actor.cpp index e828056a86..c076baeba8 100644 --- a/fdbserver/BlobManager.actor.cpp +++ b/fdbserver/BlobManager.actor.cpp @@ -35,7 +35,7 @@ #include "flow/UnitTest.h" #include "flow/actorcompiler.h" // has to be last include -#define BM_DEBUG true +#define BM_DEBUG false // FIXME: change all BlobManagerData* to Reference to avoid segfaults if core loop gets error diff --git a/fdbserver/BlobWorker.actor.cpp b/fdbserver/BlobWorker.actor.cpp index 34edd0ece7..ed1474aec6 100644 --- a/fdbserver/BlobWorker.actor.cpp +++ b/fdbserver/BlobWorker.actor.cpp @@ -43,7 +43,7 @@ #include "flow/actorcompiler.h" // has to be last include #define BW_DEBUG true -#define BW_REQUEST_DEBUG true +#define BW_REQUEST_DEBUG false // TODO add comments + documentation struct BlobFileIndex { @@ -564,6 +564,9 @@ ACTOR Future writeDeltaFile(Reference bwData, throw e; } + // FIXME: this could also fail due to actor cancelled. Since we are speculatively writing here, we should clean + // stuff up more aggressively + // if commit failed the first time due to granule assignment conflict (which is non-retryable), // then the file key was persisted and we should delete it. Otherwise, the commit failed // for some other reason and the key wasn't persisted, so we should just propogate the error @@ -681,6 +684,9 @@ ACTOR Future writeSnapshot(Reference bwData, throw e; } + // FIXME: this could also fail due to actor cancelled. Since we are speculatively writing here, we should clean + // stuff up more aggressively + // if commit failed the first time due to granule assignment conflict (which is non-retryable), // then the file key was persisted and we should delete it. Otherwise, the commit failed // for some other reason and the key wasn't persisted, so we should just propogate the error @@ -772,7 +778,7 @@ ACTOR Future dumpInitialSnapshotFromFDB(Reference ACTOR Future compactFromBlob(Reference bwData, Reference metadata, UID granuleID, - GranuleFiles files) { + Version version) { wait(delay(0, TaskPriority::BlobWorkerUpdateStorage)); if (BW_DEBUG) { printf("Compacting snapshot from blob for [%s - %s)\n", @@ -780,29 +786,34 @@ ACTOR Future compactFromBlob(Reference bwData, metadata->keyRange.end.printable().c_str()); } - ASSERT(!files.snapshotFiles.empty()); - ASSERT(!files.deltaFiles.empty()); - state Version version = files.deltaFiles.back().version; + ASSERT(!metadata->files.snapshotFiles.empty()); + ASSERT(!metadata->files.deltaFiles.empty()); state Arena filenameArena; state BlobGranuleChunkRef chunk; state int64_t compactBytesRead = 0; - state Version snapshotVersion = files.snapshotFiles.back().version; - BlobFileIndex snapshotF = files.snapshotFiles.back(); + state Version snapshotVersion = metadata->files.snapshotFiles.back().version; + BlobFileIndex snapshotF = metadata->files.snapshotFiles.back(); + + ASSERT(snapshotVersion < version); + chunk.snapshotFile = BlobFilePointerRef(filenameArena, snapshotF.filename, snapshotF.offset, snapshotF.length); compactBytesRead += snapshotF.length; - int deltaIdx = files.deltaFiles.size() - 1; - while (deltaIdx >= 0 && files.deltaFiles[deltaIdx].version > snapshotVersion) { + int deltaIdx = metadata->files.deltaFiles.size() - 1; + while (deltaIdx >= 0 && metadata->files.deltaFiles[deltaIdx].version > snapshotVersion) { deltaIdx--; } deltaIdx++; - while (deltaIdx < files.deltaFiles.size()) { - BlobFileIndex deltaF = files.deltaFiles[deltaIdx]; + Version lastDeltaVersion = invalidVersion; + while (deltaIdx < metadata->files.deltaFiles.size() && metadata->files.deltaFiles[deltaIdx].version <= version) { + BlobFileIndex deltaF = metadata->files.deltaFiles[deltaIdx]; chunk.deltaFiles.emplace_back_deep(filenameArena, deltaF.filename, deltaF.offset, deltaF.length); compactBytesRead += deltaF.length; + lastDeltaVersion = metadata->files.deltaFiles[deltaIdx].version; deltaIdx++; } + ASSERT(lastDeltaVersion == version); chunk.includedVersion = version; if (BW_DEBUG) { @@ -853,6 +864,87 @@ ACTOR Future compactFromBlob(Reference bwData, } } +ACTOR Future checkSplitAndReSnapshot(Reference bwData, + Reference metadata, + UID granuleID, + Version historyVersion, + Future lastDeltaBeforeSnapshot) { + + BlobFileIndex lastDeltaIdx = wait(lastDeltaBeforeSnapshot); + state Version reSnapshotVersion = lastDeltaIdx.version; + wait(delay(0, TaskPriority::BlobWorkerUpdateFDB)); + + if (BW_DEBUG) { + printf("Granule [%s - %s) checking with BM for re-snapshot after %d bytes\n", + metadata->keyRange.begin.printable().c_str(), + metadata->keyRange.end.printable().c_str(), + metadata->bytesInNewDeltaFiles); + } + + TraceEvent("BlobGranuleSnapshotCheck", bwData->id) + .detail("Granule", metadata->keyRange) + .detail("Version", reSnapshotVersion); + + // Save these from the start so repeated requests are idempotent + // Need to retry in case response is dropped or manager changes. Eventually, a manager will + // either reassign the range with continue=true, or will revoke the range. But, we will keep the + // range open at this version for reads until that assignment change happens + metadata->resumeSnapshot.reset(); + state int64_t statusEpoch = metadata->continueEpoch; + state int64_t statusSeqno = metadata->continueSeqno; + loop { + loop { + try { + wait(bwData->currentManagerStatusStream.get().onReady()); + bwData->currentManagerStatusStream.get().send(GranuleStatusReply( + metadata->keyRange, true, statusEpoch, statusSeqno, granuleID, historyVersion, reSnapshotVersion)); + break; + } catch (Error& e) { + wait(bwData->currentManagerStatusStream.onChange()); + } + } + + // TODO: figure out why the status stream on change isn't working + // We could just do something like statusEpoch, save down the original status stream + // and compare it to the current one + if (statusEpoch < bwData->currentManagerEpoch) { + break; + } + + choose { + when(wait(bwData->currentManagerStatusStream.onChange())) {} + when(wait(metadata->resumeSnapshot.getFuture())) { break; } + when(wait(delay(1.0))) {} + } + + if (BW_DEBUG) { + printf("Granule [%s - %s)\n, hasn't heard back from BM in BW %s, re-sending status\n", + metadata->keyRange.begin.printable().c_str(), + metadata->keyRange.end.printable().c_str(), + bwData->id.toString().c_str()); + } + } + + if (BW_DEBUG) { + printf("Granule [%s - %s) re-snapshotting after %d bytes\n", + metadata->keyRange.begin.printable().c_str(), + metadata->keyRange.end.printable().c_str(), + metadata->bytesInNewDeltaFiles); + } + TraceEvent("BlobGranuleSnapshotFile", bwData->id) + .detail("Granule", metadata->keyRange) + .detail("Version", metadata->durableDeltaVersion.get()); + // TODO: this could read from FDB instead if it knew there was a large range clear at the end or + // it knew the granule was small, or something + + // wait for file updater to make sure that last delta file is in the metadata before + while (metadata->files.deltaFiles.back().version < reSnapshotVersion) { + wait(delay(FLOW_KNOBS->PREVENT_FAST_SPIN_DELAY)); + } + BlobFileIndex reSnapshotIdx = wait(compactFromBlob(bwData, metadata, granuleID, reSnapshotVersion)); + return reSnapshotIdx; +} + ACTOR Future handleCompletedDeltaFile(Reference bwData, Reference metadata, BlobFileIndex completedDeltaFile, @@ -894,47 +986,65 @@ static bool granuleCanRetry(const Error& e) { }; } -struct InFlightDeltaFile { +struct InFlightFile { Future future; Version version; uint64_t bytes; + bool snapshot; - InFlightDeltaFile(Future future, Version version, uint64_t bytes) - : future(future), version(version), bytes(bytes) {} + InFlightFile(Future future, Version version, uint64_t bytes, bool snapshot) + : future(future), version(version), bytes(bytes), snapshot(snapshot) {} }; static Version doGranuleRollback(Reference metadata, Version mutationVersion, Version rollbackVersion, - std::deque& inFlightDeltaFiles, + std::deque& inFlightFiles, std::deque>& rollbacksInProgress, std::deque>& rollbacksCompleted) { Version cfRollbackVersion; if (metadata->pendingDeltaVersion > rollbackVersion) { - // if we already started writing mutations to a delta file with version > rollbackVersion, + // if we already started writing mutations to a delta or snapshot file with version > rollbackVersion, // we need to rescind those delta file writes - ASSERT(!inFlightDeltaFiles.empty()); + ASSERT(!inFlightFiles.empty()); cfRollbackVersion = metadata->durableDeltaVersion.get(); + metadata->pendingSnapshotVersion = metadata->durableSnapshotVersion.get(); int toPop = 0; - for (auto& df : inFlightDeltaFiles) { - if (df.version > rollbackVersion) { - df.future.cancel(); - metadata->bytesInNewDeltaFiles -= df.bytes; - toPop++; - if (BW_DEBUG) { - printf("[%s - %s) rollback cancelling delta file @ %lld\n", - metadata->keyRange.begin.printable().c_str(), - metadata->keyRange.end.printable().c_str(), - df.version); + bool pendingSnapshot = false; + for (auto& f : inFlightFiles) { + if (f.snapshot) { + if (f.version > rollbackVersion) { + f.future.cancel(); + } else { + metadata->pendingSnapshotVersion = f.version; + metadata->bytesInNewDeltaFiles = 0; + pendingSnapshot = true; } } else { - ASSERT(df.version > cfRollbackVersion); - cfRollbackVersion = df.version; + if (f.version > rollbackVersion) { + f.future.cancel(); + if (!pendingSnapshot) { + metadata->bytesInNewDeltaFiles -= f.bytes; + } + toPop++; + if (BW_DEBUG) { + printf("[%s - %s) rollback cancelling delta file @ %lld\n", + metadata->keyRange.begin.printable().c_str(), + metadata->keyRange.end.printable().c_str(), + f.version); + } + } else { + ASSERT(f.version > cfRollbackVersion); + cfRollbackVersion = f.version; + if (pendingSnapshot) { + metadata->bytesInNewDeltaFiles += f.bytes; + } + } } } ASSERT(toPop > 0); while (toPop > 0) { - inFlightDeltaFiles.pop_back(); + inFlightFiles.pop_back(); toPop--; } metadata->pendingDeltaVersion = cfRollbackVersion; @@ -949,7 +1059,9 @@ static Version doGranuleRollback(Reference metadata, metadata->deltaArena = Arena(); metadata->currentDeltas = GranuleDeltas(); metadata->bufferedDeltaBytes = 0; - metadata->bufferedDeltaVersion.set(cfRollbackVersion); + // Create new notified version so it doesn't go backwards. Do this before signaling the rollback counter so that + // the callers pick this new one up for the next waitForVersion + metadata->bufferedDeltaVersion = NotifiedVersion(cfRollbackVersion); } else { // No pending delta files to discard, just in-memory mutations @@ -1012,8 +1124,7 @@ ACTOR Future blobGranuleUpdateFiles(Reference bwData, Future assignFuture) { state PromiseStream>> oldChangeFeedStream; state PromiseStream>> changeFeedStream; - state Future inFlightBlobSnapshot; - state std::deque inFlightDeltaFiles; + state std::deque inFlightFiles; state Future oldChangeFeedFuture; state Future changeFeedFuture; state GranuleStartState startState; @@ -1060,8 +1171,6 @@ ACTOR Future blobGranuleUpdateFiles(Reference bwData, state Version startVersion; state BlobFileIndex newSnapshotFile; - inFlightBlobSnapshot = Future(); // not valid! - // if this is a reassign, calculate how close to a snapshot the previous owner was if (startState.existingFiles.present()) { GranuleFiles files = startState.existingFiles.get(); @@ -1084,9 +1193,11 @@ ACTOR Future blobGranuleUpdateFiles(Reference bwData, metadata->durableSnapshotVersion.set(metadata->pendingSnapshotVersion); } else { if (startState.blobFilesToSnapshot.present()) { - inFlightBlobSnapshot = - compactFromBlob(bwData, metadata, startState.granuleID, startState.blobFilesToSnapshot.get()); startVersion = startState.previousDurableVersion; + Future inFlightBlobSnapshot = + compactFromBlob(bwData, metadata, startState.granuleID, startVersion); + inFlightFiles.push_back(InFlightFile(inFlightBlobSnapshot, startVersion, 0, true)); + metadata->durableSnapshotVersion.set(startState.blobFilesToSnapshot.get().snapshotFiles.back().version); } else { ASSERT(startState.previousDurableVersion == invalidVersion); @@ -1128,38 +1239,27 @@ ACTOR Future blobGranuleUpdateFiles(Reference bwData, changeFeedStream, cfKey, startVersion + 1, MAX_VERSION, metadata->keyRange); } - state Version lastVersion = startVersion + 1; loop { // check outstanding snapshot/delta files for completion - if (inFlightBlobSnapshot.isValid() && inFlightBlobSnapshot.isReady()) { - BlobFileIndex completedSnapshot = wait(inFlightBlobSnapshot); - metadata->files.snapshotFiles.push_back(completedSnapshot); - metadata->durableSnapshotVersion.set(completedSnapshot.version); - inFlightBlobSnapshot = Future(); // not valid! - if (BW_DEBUG) { - printf("Async Blob Snapshot completed for [%s - %s)\n", - metadata->keyRange.begin.printable().c_str(), - metadata->keyRange.end.printable().c_str()); - } - - wait(yield(TaskPriority::BlobWorkerUpdateStorage)); - } - if (!inFlightBlobSnapshot.isValid()) { - while (inFlightDeltaFiles.size() > 0) { - if (inFlightDeltaFiles.front().future.isReady()) { - BlobFileIndex completedDeltaFile = wait(inFlightDeltaFiles.front().future); + while (inFlightFiles.size() > 0) { + if (inFlightFiles.front().future.isReady()) { + BlobFileIndex completedFile = wait(inFlightFiles.front().future); + if (inFlightFiles.front().snapshot) { + metadata->files.snapshotFiles.push_back(completedFile); + metadata->durableSnapshotVersion.set(completedFile.version); + } else { wait(handleCompletedDeltaFile(bwData, metadata, - completedDeltaFile, + completedFile, cfKey, startState.changeFeedStartVersion, rollbacksCompleted)); - - inFlightDeltaFiles.pop_front(); - wait(yield(TaskPriority::BlobWorkerUpdateStorage)); - } else { - break; } + + inFlightFiles.pop_front(); + wait(yield(TaskPriority::BlobWorkerUpdateStorage)); + } else { + break; } } @@ -1167,6 +1267,7 @@ ACTOR Future blobGranuleUpdateFiles(Reference bwData, if (BUGGIFY_WITH_PROB(0.001)) { wait(delay(deterministicRandom()->random01(), TaskPriority::BlobWorkerReadChangeFeed)); } else { + // FIXME: if we're already BlobWorkerReadChangeFeed, don't do a delay? wait(delay(0, TaskPriority::BlobWorkerReadChangeFeed)); } @@ -1207,224 +1308,10 @@ ACTOR Future blobGranuleUpdateFiles(Reference bwData, // process mutations for (MutationsAndVersionRef d : mutations) { state MutationsAndVersionRef deltas = d; - ASSERT(deltas.version >= lastVersion); - ASSERT(lastVersion > metadata->bufferedDeltaVersion.get()); - // if lastVersion is complete, update buffered version and potentially write a delta file with - // everything up to lastVersion - if (deltas.version > lastVersion) { - metadata->bufferedDeltaVersion.set(lastVersion); - } - Version knownNoRollbacksPast = std::min(lastVersion, deltas.knownCommittedVersion); - if (knownNoRollbacksPast > committedVersion.get()) { - committedVersion.set(knownNoRollbacksPast); - } - - // Write a new delta file IF we have enough bytes, and we have all of the previous version's stuff - // there to ensure no versions span multiple delta files. Check this by ensuring the version of this - // new delta is larger than the previous largest seen version - if (metadata->bufferedDeltaBytes >= SERVER_KNOBS->BG_DELTA_FILE_TARGET_BYTES && - deltas.version > lastVersion) { - if (BW_DEBUG) { - printf("Granule [%s - %s) flushing delta file after %d bytes @ %lld %lld%s\n", - metadata->keyRange.begin.printable().c_str(), - metadata->keyRange.end.printable().c_str(), - metadata->bufferedDeltaBytes, - lastVersion, - deltas.version, - oldChangeFeedDataComplete.present() ? ". Finalizing " : ""); - } - TraceEvent("BlobGranuleDeltaFile", bwData->id) - .detail("Granule", metadata->keyRange) - .detail("Version", lastVersion); - - // sanity check for version order - ASSERT(lastVersion >= metadata->currentDeltas.back().version); - ASSERT(metadata->pendingDeltaVersion < metadata->currentDeltas.front().version); - - // launch pipelined, but wait for previous operation to complete before persisting to FDB - Future previousDeltaFileFuture; - if (inFlightBlobSnapshot.isValid() && inFlightDeltaFiles.empty()) { - previousDeltaFileFuture = inFlightBlobSnapshot; - } else if (!inFlightDeltaFiles.empty()) { - previousDeltaFileFuture = inFlightDeltaFiles.back().future; - } else { - previousDeltaFileFuture = Future(BlobFileIndex()); - } - Future dfFuture = writeDeltaFile(bwData, - metadata->keyRange, - startState.granuleID, - metadata->originalEpoch, - metadata->originalSeqno, - metadata->deltaArena, - metadata->currentDeltas, - lastVersion, - previousDeltaFileFuture, - &committedVersion, - oldChangeFeedDataComplete); - inFlightDeltaFiles.push_back( - InFlightDeltaFile(dfFuture, lastVersion, metadata->bufferedDeltaBytes)); - - oldChangeFeedDataComplete.reset(); - // add new pending delta file - ASSERT(metadata->pendingDeltaVersion < lastVersion); - metadata->pendingDeltaVersion = lastVersion; - metadata->bytesInNewDeltaFiles += metadata->bufferedDeltaBytes; - - bwData->stats.mutationBytesBuffered -= metadata->bufferedDeltaBytes; - - // reset current deltas - metadata->deltaArena = Arena(); - metadata->currentDeltas = GranuleDeltas(); - metadata->bufferedDeltaBytes = 0; - - // if we just wrote a delta file, check if we need to compact here. - // exhaust old change feed before compacting - otherwise we could end up with an endlessly - // growing list of previous change feeds in the worst case. - snapshotEligible = true; - } - - // FIXME: if we're still reading from old change feed, we should probably compact if we're making a - // bunch of extra delta files at some point, even if we don't consider it for a split yet - if (snapshotEligible && metadata->bytesInNewDeltaFiles >= SERVER_KNOBS->BG_DELTA_BYTES_BEFORE_COMPACT && - !readOldChangeFeed) { - if (BW_DEBUG && (inFlightBlobSnapshot.isValid() || !inFlightDeltaFiles.empty())) { - printf("Granule [%s - %s) ready to re-snapshot, waiting for outstanding %d snapshot and %d " - "deltas to " - "finish\n", - metadata->keyRange.begin.printable().c_str(), - metadata->keyRange.end.printable().c_str(), - inFlightBlobSnapshot.isValid() ? 1 : 0, - inFlightDeltaFiles.size()); - } - // wait for all in flight snapshot/delta files - if (inFlightBlobSnapshot.isValid()) { - BlobFileIndex completedSnapshot = wait(inFlightBlobSnapshot); - metadata->files.snapshotFiles.push_back(completedSnapshot); - metadata->durableSnapshotVersion.set(completedSnapshot.version); - inFlightBlobSnapshot = Future(); // not valid! - wait(yield(TaskPriority::BlobWorkerUpdateStorage)); - } - for (auto& it : inFlightDeltaFiles) { - BlobFileIndex completedDeltaFile = wait(it.future); - wait(handleCompletedDeltaFile(bwData, - metadata, - completedDeltaFile, - cfKey, - startState.changeFeedStartVersion, - rollbacksCompleted)); - wait(yield(TaskPriority::BlobWorkerUpdateStorage)); - } - inFlightDeltaFiles.clear(); - - if (BW_DEBUG) { - printf("Granule [%s - %s) checking with BM for re-snapshot after %d bytes\n", - metadata->keyRange.begin.printable().c_str(), - metadata->keyRange.end.printable().c_str(), - metadata->bytesInNewDeltaFiles); - } - - TraceEvent("BlobGranuleSnapshotCheck", bwData->id) - .detail("Granule", metadata->keyRange) - .detail("Version", metadata->durableDeltaVersion.get()); - - // Save these from the start so repeated requests are idempotent - // Need to retry in case response is dropped or manager changes. Eventually, a manager will - // either reassign the range with continue=true, or will revoke the range. But, we will keep the - // range open at this version for reads until that assignment change happens - metadata->resumeSnapshot.reset(); - state int64_t statusEpoch = metadata->continueEpoch; - state int64_t statusSeqno = metadata->continueSeqno; - loop { - loop { - try { - wait(bwData->currentManagerStatusStream.get().onReady()); - bwData->currentManagerStatusStream.get().send( - GranuleStatusReply(metadata->keyRange, - true, - statusEpoch, - statusSeqno, - startState.granuleID, - startState.history.get().version, - metadata->durableDeltaVersion.get())); - break; - } catch (Error& e) { - wait(bwData->currentManagerStatusStream.onChange()); - } - } - - // TODO: figure out why the status stream on change isn't working - // We could just do something like statusEpoch, save down the original status stream - // and compare it to the current one - if (statusEpoch < bwData->currentManagerEpoch) { - break; - } - - choose { - when(wait(bwData->currentManagerStatusStream.onChange())) {} - when(wait(metadata->resumeSnapshot.getFuture())) { break; } - when(wait(delay(1.0))) {} - } - - if (BW_DEBUG) { - printf("Granule [%s - %s)\n, hasn't heard back from BM in BW %s, re-sending status\n", - metadata->keyRange.begin.printable().c_str(), - metadata->keyRange.end.printable().c_str(), - bwData->id.toString().c_str()); - } - } - - if (BW_DEBUG) { - printf("Granule [%s - %s) re-snapshotting after %d bytes\n", - metadata->keyRange.begin.printable().c_str(), - metadata->keyRange.end.printable().c_str(), - metadata->bytesInNewDeltaFiles); - } - TraceEvent("BlobGranuleSnapshotFile", bwData->id) - .detail("Granule", metadata->keyRange) - .detail("Version", metadata->durableDeltaVersion.get()); - // TODO: this could read from FDB instead if it knew there was a large range clear at the end or - // it knew the granule was small, or something - - // Have to copy files object so that adding to it as we start writing new delta files in - // parallel doesn't conflict. We could also pass the snapshot version and ignore any snapshot - // files >= version and any delta files > version, but that's more complicated - inFlightBlobSnapshot = compactFromBlob(bwData, metadata, startState.granuleID, metadata->files); - metadata->pendingSnapshotVersion = metadata->durableDeltaVersion.get(); - - // reset metadata - metadata->bytesInNewDeltaFiles = 0; - } else if (snapshotEligible && - metadata->bytesInNewDeltaFiles >= SERVER_KNOBS->BG_DELTA_BYTES_BEFORE_COMPACT) { - // if we're in the old change feed case and can't snapshot but we have enough data to, don't - // queue too many delta files in parallel - while (inFlightDeltaFiles.size() > 10) { - if (BW_DEBUG) { - printf("[%s - %s) Waiting on delta file b/c old change feed\n", - metadata->keyRange.begin.printable().c_str(), - metadata->keyRange.end.printable().c_str()); - } - BlobFileIndex completedDeltaFile = wait(inFlightDeltaFiles.front().future); - if (BW_DEBUG) { - printf(" [%s - %s) Got completed delta file\n", - metadata->keyRange.begin.printable().c_str(), - metadata->keyRange.end.printable().c_str()); - } - wait(handleCompletedDeltaFile(bwData, - metadata, - completedDeltaFile, - cfKey, - startState.changeFeedStartVersion, - rollbacksCompleted)); - wait(yield(TaskPriority::BlobWorkerUpdateStorage)); - inFlightDeltaFiles.pop_front(); - } - } - snapshotEligible = false; - - wait(yield(TaskPriority::BlobWorkerReadChangeFeed)); - - // finally, after we optionally write delta and snapshot files, add new mutations to buffer + // buffer mutations at this version. There should not be multiple MutationsAndVersionRef with the same + // version + ASSERT(deltas.version > metadata->bufferedDeltaVersion.get()); if (!deltas.mutations.empty()) { if (deltas.mutations.size() == 1 && deltas.mutations.back().param1 == lastEpochEndPrivateKey) { // Note rollbackVerision is durable, [rollbackVersion+1 - deltas.version] needs to be tossed @@ -1435,8 +1322,6 @@ ACTOR Future blobGranuleUpdateFiles(Reference bwData, BinaryReader br(deltas.mutations[0].param2, Unversioned()); br >> rollbackVersion; - // FIXME: THIS IS FALSE!! delta can commit by getting committed version out of band, without - // seeing rollback mutation. ASSERT(rollbackVersion >= metadata->durableDeltaVersion.get()); if (!rollbacksInProgress.empty()) { @@ -1473,7 +1358,7 @@ ACTOR Future blobGranuleUpdateFiles(Reference bwData, Version cfRollbackVersion = doGranuleRollback(metadata, deltas.version, rollbackVersion, - inFlightDeltaFiles, + inFlightFiles, rollbacksInProgress, rollbacksCompleted); @@ -1526,7 +1411,123 @@ ACTOR Future blobGranuleUpdateFiles(Reference bwData, if (justDidRollback) { break; } - lastVersion = deltas.version; + + // update buffered version and committed version + metadata->bufferedDeltaVersion.set(deltas.version); + Version knownNoRollbacksPast = std::min(deltas.version, deltas.knownCommittedVersion); + if (knownNoRollbacksPast > committedVersion.get()) { + committedVersion.set(knownNoRollbacksPast); + } + + // Write a new delta file IF we have enough bytes + if (metadata->bufferedDeltaBytes >= SERVER_KNOBS->BG_DELTA_FILE_TARGET_BYTES) { + if (BW_DEBUG) { + printf("Granule [%s - %s) flushing delta file after %d bytes @ %lld %lld%s\n", + metadata->keyRange.begin.printable().c_str(), + metadata->keyRange.end.printable().c_str(), + metadata->bufferedDeltaBytes, + deltas.version, + deltas.version, + oldChangeFeedDataComplete.present() ? ". Finalizing " : ""); + } + TraceEvent("BlobGranuleDeltaFile", bwData->id) + .detail("Granule", metadata->keyRange) + .detail("Version", deltas.version); + + // sanity check for version order + ASSERT(deltas.version >= metadata->currentDeltas.back().version); + ASSERT(metadata->pendingDeltaVersion < metadata->currentDeltas.front().version); + + // launch pipelined, but wait for previous operation to complete before persisting to FDB + Future previousFuture; + if (!inFlightFiles.empty()) { + previousFuture = inFlightFiles.back().future; + } else { + previousFuture = Future(BlobFileIndex()); + } + Future dfFuture = writeDeltaFile(bwData, + metadata->keyRange, + startState.granuleID, + metadata->originalEpoch, + metadata->originalSeqno, + metadata->deltaArena, + metadata->currentDeltas, + deltas.version, + previousFuture, + &committedVersion, + oldChangeFeedDataComplete); + inFlightFiles.push_back( + InFlightFile(dfFuture, deltas.version, metadata->bufferedDeltaBytes, false)); + + oldChangeFeedDataComplete.reset(); + // add new pending delta file + ASSERT(metadata->pendingDeltaVersion < deltas.version); + metadata->pendingDeltaVersion = deltas.version; + metadata->bytesInNewDeltaFiles += metadata->bufferedDeltaBytes; + + bwData->stats.mutationBytesBuffered -= metadata->bufferedDeltaBytes; + + // reset current deltas + metadata->deltaArena = Arena(); + metadata->currentDeltas = GranuleDeltas(); + metadata->bufferedDeltaBytes = 0; + + // if we just wrote a delta file, check if we need to compact here. + // exhaust old change feed before compacting - otherwise we could end up with an endlessly + // growing list of previous change feeds in the worst case. + snapshotEligible = true; + } + + // FIXME: if we're still reading from old change feed, we should probably compact if we're making a + // bunch of extra delta files at some point, even if we don't consider it for a split yet + + // If we have enough delta files, try to re-snapshot + if (snapshotEligible && metadata->bytesInNewDeltaFiles >= SERVER_KNOBS->BG_DELTA_BYTES_BEFORE_COMPACT && + !readOldChangeFeed) { + if (BW_DEBUG && !inFlightFiles.empty()) { + printf("Granule [%s - %s) ready to re-snapshot, waiting for %d outstanding files to finish\n", + metadata->keyRange.begin.printable().c_str(), + metadata->keyRange.end.printable().c_str(), + inFlightFiles.size()); + } + + // Speculatively assume we will get the range back. This is both a performance optimization, and + // necessary to keep consuming committed versions from the change feed so that we can realize our + // last delta file is committed and write it + + Future previousFuture; + if (!inFlightFiles.empty()) { + previousFuture = inFlightFiles.back().future; + ASSERT(!inFlightFiles.back().snapshot); + } else { + previousFuture = Future(BlobFileIndex()); + } + Future inFlightBlobSnapshot = checkSplitAndReSnapshot( + bwData, metadata, startState.granuleID, startState.history.get().version, previousFuture); + inFlightFiles.push_back(InFlightFile(inFlightBlobSnapshot, metadata->pendingDeltaVersion, 0, true)); + + metadata->pendingSnapshotVersion = metadata->pendingDeltaVersion; + + // reset metadata + metadata->bytesInNewDeltaFiles = 0; + } else if (snapshotEligible && + metadata->bytesInNewDeltaFiles >= SERVER_KNOBS->BG_DELTA_BYTES_BEFORE_COMPACT) { + // if we're in the old change feed case and can't snapshot but we have enough data to, don't + // queue too many files in parallel, and slow down change feed consuming to let file writing catch + // up + if (inFlightFiles.size() > 10) { + if (BW_DEBUG) { + printf("[%s - %s) Waiting on delta file b/c old change feed\n", + metadata->keyRange.begin.printable().c_str(), + metadata->keyRange.end.printable().c_str()); + } + choose { + when(BlobFileIndex completedDeltaFile = wait(inFlightFiles.front().future)) {} + when(wait(delay(0.1))) {} + } + } + } + snapshotEligible = false; } justDidRollback = false; } @@ -1555,8 +1556,7 @@ ACTOR Future blobGranuleUpdateFiles(Reference bwData, if (granuleCanRetry(e)) { // explicitly cancel all outstanding write futures BEFORE updating promise stream, to ensure they // can't update files after the re-assigned granule acquires the lock - inFlightBlobSnapshot.cancel(); - for (auto& f : inFlightDeltaFiles) { + for (auto& f : inFlightFiles) { f.future.cancel(); } @@ -1724,15 +1724,15 @@ ACTOR Future waitForVersion(Reference metadata, Version v wait(metadata->durableDeltaVersion.whenAtLeast(pendingDeltaV)); } - // This isn't strictly needed, but if we're in the process of re-snapshotting, we'd likely rather return that - // snapshot file than the previous snapshot file and all its delta files. + // This isn't strictly needed, but if we're in the process of re-snapshotting, we'd likely rather + // return that snapshot file than the previous snapshot file and all its delta files. if (pendingSnapshotV > metadata->durableSnapshotVersion.get()) { wait(metadata->durableSnapshotVersion.whenAtLeast(pendingSnapshotV)); } - // There is a race here - we wait for pending delta files before this to finish, but while we do, we kick off - // another delta file and roll the mutations. In that case, we must return the new delta file instead of in - // memory mutations, so we wait for that delta file to complete + // There is a race here - we wait for pending delta files before this to finish, but while we do, we + // kick off another delta file and roll the mutations. In that case, we must return the new delta + // file instead of in memory mutations, so we wait for that delta file to complete if (metadata->pendingDeltaVersion != pendingDeltaV) { wait(metadata->durableDeltaVersion.whenAtLeast(pendingDeltaV + 1)); @@ -1749,7 +1749,8 @@ ACTOR Future handleBlobGranuleFileRequest(Reference bwData state std::vector> granules; auto checkRanges = bwData->granuleMetadata.intersectingRanges(req.keyRange); - // check for gaps as errors and copy references to granule metadata before yielding or doing any work + // check for gaps as errors and copy references to granule metadata before yielding or doing any + // work KeyRef lastRangeEnd = req.keyRange.begin; for (auto& r : checkRanges) { @@ -1786,8 +1787,8 @@ ACTOR Future handleBlobGranuleFileRequest(Reference bwData for (auto m : granules) { if (readThrough >= m->keyRange.end) { // previous read did time travel that already included this granule - // FIXME: this will get more complicated with merges where this could potentially include partial - // boundaries. For now with only splits we can skip the whole range + // FIXME: this will get more complicated with merges where this could potentially + // include partial boundaries. For now with only splits we can skip the whole range continue; } ASSERT(readThrough == m->keyRange.begin); @@ -1811,7 +1812,8 @@ ACTOR Future handleBlobGranuleFileRequest(Reference bwData wait(metadata->historyLoaded.getFuture()); } - // FIXME: doesn't work once we add granule merging, could be multiple ranges and/or multiple parents + // FIXME: doesn't work once we add granule merging, could be multiple ranges and/or + // multiple parents Reference cur = bwData->granuleHistory.rangeContaining(req.keyRange.begin).value(); // FIXME: use skip pointers here while (cur.isValid() && req.readVersion < cur->startVersion) { @@ -1820,7 +1822,8 @@ ACTOR Future handleBlobGranuleFileRequest(Reference bwData if (!cur.isValid()) { // this request predates blob data - // FIXME: probably want a dedicated exception like blob_range_too_old or something instead + // FIXME: probably want a dedicated exception like blob_range_too_old or something + // instead throw transaction_too_old(); } @@ -1853,7 +1856,8 @@ ACTOR Future handleBlobGranuleFileRequest(Reference bwData // didn't yield, so no need to check rollback stuff break; } - // rollback resets all of the version information, so we have to redo wait for version on rollback + // rollback resets all of the version information, so we have to redo wait for + // version on rollback state int rollbackCount = metadata->rollbackCount.get(); choose { when(wait(waitForVersionFuture)) {} @@ -1888,8 +1892,8 @@ ACTOR Future handleBlobGranuleFileRequest(Reference bwData while (i >= 0 && chunkFiles.snapshotFiles[i].version > req.readVersion) { i--; } - // because of granule history, we should always be able to find the desired snapshot version, and have - // thrown transaction_too_old earlier if not possible. + // because of granule history, we should always be able to find the desired snapshot + // version, and have thrown transaction_too_old earlier if not possible. ASSERT(i >= 0); BlobFileIndex snapshotF = chunkFiles.snapshotFiles[i]; @@ -1905,8 +1909,8 @@ ACTOR Future handleBlobGranuleFileRequest(Reference bwData i--; } if (i < lastDeltaFileIdx) { - // we skipped one file at the end with a larger read version, this will actually contain our query - // version, so add it back. + // we skipped one file at the end with a larger read version, this will actually contain + // our query version, so add it back. i++; } // only include delta files after the snapshot file @@ -1925,7 +1929,8 @@ ACTOR Future handleBlobGranuleFileRequest(Reference bwData } // new deltas (if version is larger than version of last delta file) - // FIXME: do trivial key bounds here if key range is not fully contained in request key range + // FIXME: do trivial key bounds here if key range is not fully contained in request key + // range if (req.readVersion > metadata->durableDeltaVersion.get()) { ASSERT(metadata->durableDeltaVersion.get() == metadata->pendingDeltaVersion); @@ -2021,8 +2026,8 @@ ACTOR Future openGranule(Reference bwData, As info.doSnapshot = false; if (!info.history.present()) { - // the only time history can be not present if a lock already exists is if it's a new granule and it - // died before it could persist the initial snapshot from FDB + // the only time history can be not present if a lock already exists is if it's a + // new granule and it died before it could persist the initial snapshot from FDB ASSERT(info.existingFiles.get().snapshotFiles.empty()); } @@ -2036,17 +2041,17 @@ ACTOR Future openGranule(Reference bwData, As info.previousDurableVersion = info.existingFiles.get().deltaFiles.back().version; } - // for the non-splitting cases, this doesn't need to be 100% accurate, it just needs to be - // smaller than the next delta file write. + // for the non-splitting cases, this doesn't need to be 100% accurate, it just needs to + // be smaller than the next delta file write. info.changeFeedStartVersion = info.previousDurableVersion; } else { // else we are first, no need to check for owner conflict - // FIXME: use actual 16 bytes of UID instead of converting it to 32 character string and then that - // to bytes + // FIXME: use actual 16 bytes of UID instead of converting it to 32 character string and + // then that to bytes if (info.history.present()) { - // if this granule is derived from a split or merge, this history entry is already present (written - // by the blob manager) + // if this granule is derived from a split or merge, this history entry is already + // present (written by the blob manager) info.granuleID = info.history.get().value.granuleID; } else { // FIXME: could avoid max uid for granule ids here @@ -2062,8 +2067,9 @@ ACTOR Future openGranule(Reference bwData, As tr.set(lockKey, blobGranuleLockValueFor(req.managerEpoch, req.managerSeqno, info.granuleID)); wait(krmSetRange(&tr, blobGranuleMappingKeys.begin, req.keyRange, blobGranuleMappingValueFor(bwData->id))); - // If anything in previousGranules, need to do the handoff logic and set ret.previousChangeFeedId, and - // the previous durable version will come from the previous granules + // If anything in previousGranules, need to do the handoff logic and set + // ret.previousChangeFeedId, and the previous durable version will come from the previous + // granules if (info.history.present() && info.history.get().value.parentGranules.size() > 0) { // TODO change this for merge ASSERT(info.history.get().value.parentGranules.size() == 1); @@ -2096,8 +2102,8 @@ ACTOR Future openGranule(Reference bwData, As info.parentGranule.get().second, info.granuleID, BlobGranuleSplitState::Assigned)); - // change feed was created as part of this transaction, changeFeedStartVersion will be - // set later + // change feed was created as part of this transaction, changeFeedStartVersion + // will be set later } else { // this sub-granule is done splitting, no need for split logic. ASSERT(granuleSplitState.first == BlobGranuleSplitState::Done); @@ -2175,20 +2181,22 @@ static bool newerRangeAssignment(GranuleRangeMetadata oldMetadata, int64_t epoch // TODO unit test this assignment, particularly out-of-order insertions! // The contract from the blob manager is: -// If a key range [A, B) was assigned to the worker at seqno S1, no part of the keyspace that intersects [A, B] may -// be re-assigned to the worker until the range has been revoked from this worker. This revoking can either happen -// by the blob manager willingly relinquishing the range, or by the blob manager reassigning it somewhere else. This -// means that if the worker gets an assignment for any range that intersects [A, B) at S3, there must have been a -// revoke message for [A, B) with seqno S3 where S1 < S2 < S3, that was delivered out of order. This means that if -// there are any intersecting but not fully overlapping ranges with a new range assignment, they had already been -// revoked. So the worker will mark them as revoked, but leave the sequence number as S1, so that when the actual -// revoke message comes in, it is a no-op, but updates the sequence number. Similarly, if a worker gets an assign -// message for any range that already has a higher sequence number, that range was either revoked, or revoked and -// then re-assigned. Either way, this assignment is no longer valid. +// If a key range [A, B) was assigned to the worker at seqno S1, no part of the keyspace that intersects +// [A, B] may be re-assigned to the worker until the range has been revoked from this worker. This +// revoking can either happen by the blob manager willingly relinquishing the range, or by the blob +// manager reassigning it somewhere else. This means that if the worker gets an assignment for any range +// that intersects [A, B) at S3, there must have been a revoke message for [A, B) with seqno S3 where S1 +// < S2 < S3, that was delivered out of order. This means that if there are any intersecting but not +// fully overlapping ranges with a new range assignment, they had already been revoked. So the worker +// will mark them as revoked, but leave the sequence number as S1, so that when the actual revoke +// message comes in, it is a no-op, but updates the sequence number. Similarly, if a worker gets an +// assign message for any range that already has a higher sequence number, that range was either +// revoked, or revoked and then re-assigned. Either way, this assignment is no longer valid. -// Returns future to wait on to ensure prior work of other granules is done before responding to the manager with a -// successful assignment And if the change produced a new granule that needs to start doing work, returns the new -// granule so that the caller can start() it with the appropriate starting state. +// Returns future to wait on to ensure prior work of other granules is done before responding to the +// manager with a successful assignment And if the change produced a new granule that needs to start +// doing work, returns the new granule so that the caller can start() it with the appropriate starting +// state. ACTOR Future changeBlobRange(Reference bwData, KeyRange keyRange, int64_t epoch, @@ -2212,10 +2220,10 @@ ACTOR Future changeBlobRange(Reference bwData, } // For each range that intersects this update: - // If the identical range already exists at the same assignment sequence number and it is not a self-reassign, - // this is a noop. Otherwise, this will consist of a series of ranges that are either older, or newer. For each - // older range, cancel it if it is active. Insert the current range. Re-insert all newer ranges over the current - // range. + // If the identical range already exists at the same assignment sequence number and it is not a + // self-reassign, this is a noop. Otherwise, this will consist of a series of ranges that are either + // older, or newer. For each older range, cancel it if it is active. Insert the current range. + // Re-insert all newer ranges over the current range. state std::vector> futures; @@ -2236,8 +2244,8 @@ ACTOR Future changeBlobRange(Reference bwData, bool thisAssignmentNewer = newerRangeAssignment(r.value(), epoch, seqno); printf("thisAssignmentNewer=%s\n", thisAssignmentNewer ? "true" : "false"); - // if this granule already has it, and this was a specialassignment (i.e. a new blob maanger is trying to - // reassign granules), then just continue + // if this granule already has it, and this was a specialassignment (i.e. a new blob maanger is + // trying to reassign granules), then just continue if (active && assignType.get() == AssignRequestType::Reassign && r.begin() == keyRange.begin && r.end() == keyRange.end) { r.value().lastEpoch = epoch; @@ -2326,7 +2334,8 @@ static bool resumeBlobRange(Reference bwData, KeyRange keyRange, !existingRange.value().activeMetadata.isValid()) { if (BW_DEBUG) { - printf("BW %s got out of date resume range for [%s - %s) @ (%lld, %lld). Currently [%s - %s) @ (%lld, " + printf("BW %s got out of date resume range for [%s - %s) @ (%lld, %lld). Currently [%s - " + "%s) @ (%lld, " "%lld): %s\n", bwData->id.toString().c_str(), existingRange.begin().printable().c_str(), diff --git a/fdbserver/SimulatedCluster.actor.cpp b/fdbserver/SimulatedCluster.actor.cpp index bdef5d8690..325b15d983 100644 --- a/fdbserver/SimulatedCluster.actor.cpp +++ b/fdbserver/SimulatedCluster.actor.cpp @@ -2018,7 +2018,10 @@ void setupSimulatedSystem(std::vector>* systemActors, // FIXME: we hardcode some machines to specifically test storage cache and blob workers // TODO: caching disabled for this merge int storageCacheMachines = dc == 0 ? 1 : 0; - int blobWorkerMachines = 2 + deterministicRandom()->randomInt(0, NUM_EXTRA_BW_MACHINES + 1); + int blobWorkerMachines = 0; + if (CLIENT_KNOBS->ENABLE_BLOB_GRANULES) { + blobWorkerMachines = 2 + deterministicRandom()->randomInt(0, NUM_EXTRA_BW_MACHINES + 1); + } int totalMachines = machines + storageCacheMachines + blobWorkerMachines; int useSeedForMachine = deterministicRandom()->randomInt(0, totalMachines); From d0c6bbc56a6e7eca225c288c6cd65a8f9b46b9eb Mon Sep 17 00:00:00 2001 From: Josh Slocum Date: Thu, 28 Oct 2021 08:13:52 -0500 Subject: [PATCH 010/413] wait on previous snapshots if we know they can be committed now --- fdbserver/BlobWorker.actor.cpp | 102 +++++++++++++++++++++++++++------ 1 file changed, 84 insertions(+), 18 deletions(-) diff --git a/fdbserver/BlobWorker.actor.cpp b/fdbserver/BlobWorker.actor.cpp index ed1474aec6..6a5ec66fb8 100644 --- a/fdbserver/BlobWorker.actor.cpp +++ b/fdbserver/BlobWorker.actor.cpp @@ -778,6 +778,7 @@ ACTOR Future dumpInitialSnapshotFromFDB(Reference ACTOR Future compactFromBlob(Reference bwData, Reference metadata, UID granuleID, + GranuleFiles files, Version version) { wait(delay(0, TaskPriority::BlobWorkerUpdateStorage)); if (BW_DEBUG) { @@ -786,31 +787,33 @@ ACTOR Future compactFromBlob(Reference bwData, metadata->keyRange.end.printable().c_str()); } - ASSERT(!metadata->files.snapshotFiles.empty()); - ASSERT(!metadata->files.deltaFiles.empty()); + // FIXME: don't use metadata->files + + ASSERT(!files.snapshotFiles.empty()); + ASSERT(!files.deltaFiles.empty()); state Arena filenameArena; state BlobGranuleChunkRef chunk; state int64_t compactBytesRead = 0; - state Version snapshotVersion = metadata->files.snapshotFiles.back().version; - BlobFileIndex snapshotF = metadata->files.snapshotFiles.back(); + state Version snapshotVersion = files.snapshotFiles.back().version; + BlobFileIndex snapshotF = files.snapshotFiles.back(); ASSERT(snapshotVersion < version); chunk.snapshotFile = BlobFilePointerRef(filenameArena, snapshotF.filename, snapshotF.offset, snapshotF.length); compactBytesRead += snapshotF.length; - int deltaIdx = metadata->files.deltaFiles.size() - 1; - while (deltaIdx >= 0 && metadata->files.deltaFiles[deltaIdx].version > snapshotVersion) { + int deltaIdx = files.deltaFiles.size() - 1; + while (deltaIdx >= 0 && files.deltaFiles[deltaIdx].version > snapshotVersion) { deltaIdx--; } deltaIdx++; Version lastDeltaVersion = invalidVersion; - while (deltaIdx < metadata->files.deltaFiles.size() && metadata->files.deltaFiles[deltaIdx].version <= version) { - BlobFileIndex deltaF = metadata->files.deltaFiles[deltaIdx]; + while (deltaIdx < files.deltaFiles.size() && files.deltaFiles[deltaIdx].version <= version) { + BlobFileIndex deltaF = files.deltaFiles[deltaIdx]; chunk.deltaFiles.emplace_back_deep(filenameArena, deltaF.filename, deltaF.offset, deltaF.length); compactBytesRead += deltaF.length; - lastDeltaVersion = metadata->files.deltaFiles[deltaIdx].version; + lastDeltaVersion = files.deltaFiles[deltaIdx].version; deltaIdx++; } ASSERT(lastDeltaVersion == version); @@ -868,6 +871,7 @@ ACTOR Future checkSplitAndReSnapshot(Reference bw Reference metadata, UID granuleID, Version historyVersion, + int64_t bytesInNewDeltaFiles, Future lastDeltaBeforeSnapshot) { BlobFileIndex lastDeltaIdx = wait(lastDeltaBeforeSnapshot); @@ -878,7 +882,7 @@ ACTOR Future checkSplitAndReSnapshot(Reference bw printf("Granule [%s - %s) checking with BM for re-snapshot after %d bytes\n", metadata->keyRange.begin.printable().c_str(), metadata->keyRange.end.printable().c_str(), - metadata->bytesInNewDeltaFiles); + bytesInNewDeltaFiles); } TraceEvent("BlobGranuleSnapshotCheck", bwData->id) @@ -929,7 +933,7 @@ ACTOR Future checkSplitAndReSnapshot(Reference bw printf("Granule [%s - %s) re-snapshotting after %d bytes\n", metadata->keyRange.begin.printable().c_str(), metadata->keyRange.end.printable().c_str(), - metadata->bytesInNewDeltaFiles); + bytesInNewDeltaFiles); } TraceEvent("BlobGranuleSnapshotFile", bwData->id) .detail("Granule", metadata->keyRange) @@ -938,10 +942,11 @@ ACTOR Future checkSplitAndReSnapshot(Reference bw // it knew the granule was small, or something // wait for file updater to make sure that last delta file is in the metadata before - while (metadata->files.deltaFiles.back().version < reSnapshotVersion) { + while (metadata->files.deltaFiles.empty() || metadata->files.deltaFiles.back().version < reSnapshotVersion) { wait(delay(FLOW_KNOBS->PREVENT_FAST_SPIN_DELAY)); } - BlobFileIndex reSnapshotIdx = wait(compactFromBlob(bwData, metadata, granuleID, reSnapshotVersion)); + BlobFileIndex reSnapshotIdx = + wait(compactFromBlob(bwData, metadata, granuleID, metadata->files, reSnapshotVersion)); return reSnapshotIdx; } @@ -1133,6 +1138,7 @@ ACTOR Future blobGranuleUpdateFiles(Reference bwData, state Key cfKey; state Optional oldCFKey; state NotifiedVersion committedVersion; + state int pendingSnapshots = 0; state std::deque> rollbacksInProgress; state std::deque> rollbacksCompleted; @@ -1194,9 +1200,10 @@ ACTOR Future blobGranuleUpdateFiles(Reference bwData, } else { if (startState.blobFilesToSnapshot.present()) { startVersion = startState.previousDurableVersion; - Future inFlightBlobSnapshot = - compactFromBlob(bwData, metadata, startState.granuleID, startVersion); + Future inFlightBlobSnapshot = compactFromBlob( + bwData, metadata, startState.granuleID, startState.blobFilesToSnapshot.get(), startVersion); inFlightFiles.push_back(InFlightFile(inFlightBlobSnapshot, startVersion, 0, true)); + pendingSnapshots++; metadata->durableSnapshotVersion.set(startState.blobFilesToSnapshot.get().snapshotFiles.back().version); } else { @@ -1247,6 +1254,7 @@ ACTOR Future blobGranuleUpdateFiles(Reference bwData, if (inFlightFiles.front().snapshot) { metadata->files.snapshotFiles.push_back(completedFile); metadata->durableSnapshotVersion.set(completedFile.version); + pendingSnapshots--; } else { wait(handleCompletedDeltaFile(bwData, metadata, @@ -1416,6 +1424,10 @@ ACTOR Future blobGranuleUpdateFiles(Reference bwData, metadata->bufferedDeltaVersion.set(deltas.version); Version knownNoRollbacksPast = std::min(deltas.version, deltas.knownCommittedVersion); if (knownNoRollbacksPast > committedVersion.get()) { + // This is the only place it is safe to set committedVersion, as it has to come from the mutation + // stream, or we could have a situation where the blob worker has consumed an uncommitted mutation, + // but not its rollback, from the change feed, and could thus think the uncommitted mutation is + // committed because it saw a higher committed version than the mutation's version. committedVersion.set(knownNoRollbacksPast); } @@ -1485,9 +1497,13 @@ ACTOR Future blobGranuleUpdateFiles(Reference bwData, if (snapshotEligible && metadata->bytesInNewDeltaFiles >= SERVER_KNOBS->BG_DELTA_BYTES_BEFORE_COMPACT && !readOldChangeFeed) { if (BW_DEBUG && !inFlightFiles.empty()) { - printf("Granule [%s - %s) ready to re-snapshot, waiting for %d outstanding files to finish\n", + printf("Granule [%s - %s) ready to re-snapshot after %lld > %lld bytes, waiting for %d " + "outstanding " + "files to finish\n", metadata->keyRange.begin.printable().c_str(), metadata->keyRange.end.printable().c_str(), + metadata->bytesInNewDeltaFiles, + SERVER_KNOBS->BG_DELTA_BYTES_BEFORE_COMPACT, inFlightFiles.size()); } @@ -1502,14 +1518,64 @@ ACTOR Future blobGranuleUpdateFiles(Reference bwData, } else { previousFuture = Future(BlobFileIndex()); } - Future inFlightBlobSnapshot = checkSplitAndReSnapshot( - bwData, metadata, startState.granuleID, startState.history.get().version, previousFuture); + Future inFlightBlobSnapshot = + checkSplitAndReSnapshot(bwData, + metadata, + startState.granuleID, + startState.history.get().version, + metadata->bytesInNewDeltaFiles, + previousFuture); inFlightFiles.push_back(InFlightFile(inFlightBlobSnapshot, metadata->pendingDeltaVersion, 0, true)); + pendingSnapshots++; metadata->pendingSnapshotVersion = metadata->pendingDeltaVersion; // reset metadata metadata->bytesInNewDeltaFiles = 0; + + // If we have more than one snapshot file and that file is unblocked (committedVersion >= + // snapshotVersion), wait for it to finish + + if (pendingSnapshots > 1) { + state int waitIdx = 0; + int idx = 0; + for (auto& f : inFlightFiles) { + if (f.snapshot && f.version < metadata->pendingSnapshotVersion && + f.version <= committedVersion.get()) { + if (BW_DEBUG) { + printf( + "[%s - %s) Waiting on previous snapshot file @ %lld <= known committed %lld\n", + metadata->keyRange.begin.printable().c_str(), + metadata->keyRange.end.printable().c_str(), + f.version, + committedVersion.get()); + } + waitIdx = idx + 1; + } + idx++; + } + while (waitIdx > 0) { + // TODO don't duplicate code + BlobFileIndex completedFile = wait(inFlightFiles.front().future); + if (inFlightFiles.front().snapshot) { + metadata->files.snapshotFiles.push_back(completedFile); + metadata->durableSnapshotVersion.set(completedFile.version); + pendingSnapshots--; + } else { + wait(handleCompletedDeltaFile(bwData, + metadata, + completedFile, + cfKey, + startState.changeFeedStartVersion, + rollbacksCompleted)); + } + + inFlightFiles.pop_front(); + waitIdx--; + wait(yield(TaskPriority::BlobWorkerUpdateStorage)); + } + } + } else if (snapshotEligible && metadata->bytesInNewDeltaFiles >= SERVER_KNOBS->BG_DELTA_BYTES_BEFORE_COMPACT) { // if we're in the old change feed case and can't snapshot but we have enough data to, don't From f1a4363fe667de58193b7c39c6d521bd36b08d61 Mon Sep 17 00:00:00 2001 From: Josh Slocum Date: Thu, 28 Oct 2021 13:18:39 -0500 Subject: [PATCH 011/413] fixing a couple bugs in blob worker --- fdbserver/BlobWorker.actor.cpp | 72 +++++++++++++++++++++++++--------- 1 file changed, 53 insertions(+), 19 deletions(-) diff --git a/fdbserver/BlobWorker.actor.cpp b/fdbserver/BlobWorker.actor.cpp index 6a5ec66fb8..bac1f76473 100644 --- a/fdbserver/BlobWorker.actor.cpp +++ b/fdbserver/BlobWorker.actor.cpp @@ -102,6 +102,7 @@ struct GranuleMetadata : NonCopyable, ReferenceCounted { NotifiedVersion durableDeltaVersion; // largest version persisted in s3/fdb NotifiedVersion durableSnapshotVersion; // same as delta vars, except for snapshots Version pendingSnapshotVersion = 0; + Version initialSnapshotVersion = invalidVersion; AsyncVar rollbackCount; @@ -136,15 +137,18 @@ struct GranuleRangeMetadata { Future historyLoaderFuture; void cancel() { - // assignFuture.cancel(); - // fileUpdaterFuture.cancel(); - // historyLoaderFuture.cancel(); + if (activeMetadata->cancelled.canBeSet()) { + activeMetadata->cancelled.send(Void()); + } activeMetadata.clear(); } GranuleRangeMetadata() : lastEpoch(0), lastSeqno(0) {} GranuleRangeMetadata(int64_t epoch, int64_t seqno, Reference activeMetadata) : lastEpoch(epoch), lastSeqno(seqno), activeMetadata(activeMetadata) {} + + // TODO REMOVE + ~GranuleRangeMetadata() { printf("Destroying granule metadata\n"); } }; // represents a previous version of a granule, and optionally the files that compose it @@ -1197,6 +1201,7 @@ ACTOR Future blobGranuleUpdateFiles(Reference bwData, ASSERT(!metadata->files.snapshotFiles.empty()); metadata->pendingSnapshotVersion = metadata->files.snapshotFiles.back().version; metadata->durableSnapshotVersion.set(metadata->pendingSnapshotVersion); + metadata->initialSnapshotVersion = metadata->files.snapshotFiles.front().version; } else { if (startState.blobFilesToSnapshot.present()) { startVersion = startState.previousDurableVersion; @@ -1221,6 +1226,7 @@ ACTOR Future blobGranuleUpdateFiles(Reference bwData, wait(yield(TaskPriority::BlobWorkerUpdateStorage)); } + metadata->initialSnapshotVersion = startVersion; metadata->pendingSnapshotVersion = startVersion; } @@ -1598,6 +1604,8 @@ ACTOR Future blobGranuleUpdateFiles(Reference bwData, justDidRollback = false; } } catch (Error& e) { + // TODO REMOVE + printf("BGUF got error %s\n", e.name()); if (e.code() == error_code_operation_cancelled) { throw; } @@ -1781,7 +1789,7 @@ ACTOR Future waitForVersion(Reference metadata, Version v wait(metadata->bufferedDeltaVersion.whenAtLeast(v)); } - // wait for any pending delta and snapshot files as of the momemt the change feed version caught up. + // wait for any pending delta and snapshot files as of the moment the change feed version caught up. state Version pendingDeltaV = metadata->pendingDeltaVersion; state Version pendingSnapshotV = metadata->pendingSnapshotVersion; @@ -1870,9 +1878,7 @@ ACTOR Future handleBlobGranuleFileRequest(Reference bwData state KeyRange chunkRange; state GranuleFiles chunkFiles; - if ((!metadata->files.snapshotFiles.empty() && - metadata->files.snapshotFiles.front().version > req.readVersion) || - (metadata->files.snapshotFiles.empty() && metadata->pendingSnapshotVersion > req.readVersion)) { + if (metadata->initialSnapshotVersion > req.readVersion) { // this is a time travel query, find previous granule if (metadata->historyLoaded.canBeSet()) { wait(metadata->historyLoaded.getFuture()); @@ -1911,15 +1917,21 @@ ACTOR Future handleBlobGranuleFileRequest(Reference bwData cur->files = loadHistoryFiles(bwData, cur->granuleID); } - GranuleFiles _f = wait(cur->files); - chunkFiles = _f; + choose { + when(GranuleFiles _f = wait(cur->files)) { chunkFiles = _f; } + when(wait(metadata->cancelled.getFuture())) { throw wrong_shard_server(); } + } + ASSERT(!chunkFiles.snapshotFiles.empty()); + ASSERT(!chunkFiles.deltaFiles.empty()); + ASSERT(chunkFiles.deltaFiles.back().version > req.readVersion); + ASSERT(chunkFiles.snapshotFiles.front().version <= req.readVersion); } else { // this is an active granule query loop { Future waitForVersionFuture = waitForVersion(metadata, req.readVersion); if (waitForVersionFuture.isReady()) { - // didn't yield, so no need to check rollback stuff + // didn't wait, so no need to check rollback stuff break; } // rollback resets all of the version information, so we have to redo wait for @@ -1945,6 +1957,7 @@ ACTOR Future handleBlobGranuleFileRequest(Reference bwData } // granule is up to date, do read + ASSERT(metadata->cancelled.canBeSet()); BlobGranuleChunkRef chunk; // TODO change in V2 @@ -1960,6 +1973,15 @@ ACTOR Future handleBlobGranuleFileRequest(Reference bwData } // because of granule history, we should always be able to find the desired snapshot // version, and have thrown transaction_too_old earlier if not possible. + if (i < 0) { + printf("req @ %lld >= initial snapshot %lld but can't find snapshot in (%d) files:\n", + req.readVersion, + metadata->initialSnapshotVersion, + chunkFiles.snapshotFiles.size()); + for (auto& f : chunkFiles.snapshotFiles) { + printf(" %lld", f.version); + } + } ASSERT(i >= 0); BlobFileIndex snapshotF = chunkFiles.snapshotFiles[i]; @@ -1999,6 +2021,14 @@ ACTOR Future handleBlobGranuleFileRequest(Reference bwData // range if (req.readVersion > metadata->durableDeltaVersion.get()) { + if (metadata->durableDeltaVersion.get() != metadata->pendingDeltaVersion) { + printf("real-time read [%s - %s) @ %lld doesn't have mutations!! durable=%lld, pending=%lld\n", + metadata->keyRange.begin.printable().c_str(), + metadata->keyRange.end.printable().c_str(), + req.readVersion, + metadata->durableDeltaVersion.get(), + metadata->pendingDeltaVersion); + } ASSERT(metadata->durableDeltaVersion.get() == metadata->pendingDeltaVersion); rep.arena.dependsOn(metadata->deltaArena); for (auto& delta : metadata->currentDeltas) { @@ -2298,15 +2328,16 @@ ACTOR Future changeBlobRange(Reference bwData, auto ranges = bwData->granuleMetadata.intersectingRanges(keyRange); bool alreadyAssigned = false; for (auto& r : ranges) { - if (!active) { - if (r.value().activeMetadata.isValid() && r.value().activeMetadata->cancelled.canBeSet()) { - if (BW_DEBUG) { - printf("Cancelling activeMetadata\n"); - } - bwData->stats.numRangesAssigned--; - r.value().activeMetadata->cancelled.send(Void()); - } - } + // I don't think we need this? + /*if (!active) { + if (r.value().activeMetadata.isValid() && r.value().activeMetadata->cancelled.canBeSet()) { + if (BW_DEBUG) { + printf("Cancelling activeMetadata\n"); + } + bwData->stats.numRangesAssigned--; + r.value().activeMetadata->cancelled.send(Void()); + } + }*/ bool thisAssignmentNewer = newerRangeAssignment(r.value(), epoch, seqno); printf("thisAssignmentNewer=%s\n", thisAssignmentNewer ? "true" : "false"); @@ -2348,6 +2379,9 @@ ACTOR Future changeBlobRange(Reference bwData, r.value().lastEpoch, r.value().lastSeqno); } + if (!active) { + bwData->stats.numRangesAssigned--; + } r.value().cancel(); } else if (!thisAssignmentNewer) { // re-insert the known newer range over this existing range From dbf46c200fd1c4c018dbeaf3f77dbd8faa5ab923 Mon Sep 17 00:00:00 2001 From: Josh Slocum Date: Thu, 28 Oct 2021 13:53:54 -0500 Subject: [PATCH 012/413] disabling long keys and adding more debugging stuff --- fdbserver/BlobWorker.actor.cpp | 11 +++++++++++ fdbserver/storageserver.actor.cpp | 13 ++++++++++++- fdbserver/workloads/WriteDuringRead.actor.cpp | 4 +++- 3 files changed, 26 insertions(+), 2 deletions(-) diff --git a/fdbserver/BlobWorker.actor.cpp b/fdbserver/BlobWorker.actor.cpp index bac1f76473..1c90c7fbbc 100644 --- a/fdbserver/BlobWorker.actor.cpp +++ b/fdbserver/BlobWorker.actor.cpp @@ -1258,6 +1258,12 @@ ACTOR Future blobGranuleUpdateFiles(Reference bwData, if (inFlightFiles.front().future.isReady()) { BlobFileIndex completedFile = wait(inFlightFiles.front().future); if (inFlightFiles.front().snapshot) { + if (metadata->files.deltaFiles.empty()) { + ASSERT(completedFile.version == metadata->initialSnapshotVersion); + } else { + ASSERT(completedFile.version == metadata->files.deltaFiles.back().version); + } + metadata->files.snapshotFiles.push_back(completedFile); metadata->durableSnapshotVersion.set(completedFile.version); pendingSnapshots--; @@ -1564,6 +1570,11 @@ ACTOR Future blobGranuleUpdateFiles(Reference bwData, // TODO don't duplicate code BlobFileIndex completedFile = wait(inFlightFiles.front().future); if (inFlightFiles.front().snapshot) { + if (metadata->files.deltaFiles.empty()) { + ASSERT(completedFile.version == metadata->initialSnapshotVersion); + } else { + ASSERT(completedFile.version == metadata->files.deltaFiles.back().version); + } metadata->files.snapshotFiles.push_back(completedFile); metadata->durableSnapshotVersion.set(completedFile.version); pendingSnapshots--; diff --git a/fdbserver/storageserver.actor.cpp b/fdbserver/storageserver.actor.cpp index fcd299abe7..93d84da2fe 100644 --- a/fdbserver/storageserver.actor.cpp +++ b/fdbserver/storageserver.actor.cpp @@ -1840,7 +1840,7 @@ ACTOR Future localChangeFeedStream(StorageServer* data, } } } catch (Error& e) { - TraceEvent(SevError, "LocalChangeFeedError", data->thisServerID).error(e); + TraceEvent(SevError, "LocalChangeFeedError", data->thisServerID).detail("CFID", rangeID.printable()).error(e); throw; } } @@ -4255,6 +4255,10 @@ private: } } else { if (status == ChangeFeedStatus::CHANGE_FEED_DESTROY) { + TraceEvent(SevDebug, "DestroyingChangeFeed", data->thisServerID) + .detail("RangeID", changeFeedId.printable()) + .detail("Range", changeFeedRange.toString()) + .detail("Version", currentVersion); Key beginClearKey = changeFeedId.withPrefix(persistChangeFeedKeys.begin); auto& mLV = data->addVersionToMutationLog(data->data().getLatestVersion()); data->addMutationToMutationLog( @@ -4274,6 +4278,13 @@ private: } data->uidChangeFeed.erase(feed); } else { + // must be pop or stop + if (status == ChangeFeedStatus::CHANGE_FEED_STOP) { + TraceEvent(SevDebug, "StoppingChangeFeed", data->thisServerID) + .detail("RangeID", changeFeedId.printable()) + .detail("Range", changeFeedRange.toString()) + .detail("Version", currentVersion); + } if (popVersion != invalidVersion && popVersion - 1 > feed->second->emptyVersion) { feed->second->emptyVersion = popVersion - 1; while (!feed->second->mutations.empty() && diff --git a/fdbserver/workloads/WriteDuringRead.actor.cpp b/fdbserver/workloads/WriteDuringRead.actor.cpp index e0d47d68e4..c76900b36a 100644 --- a/fdbserver/workloads/WriteDuringRead.actor.cpp +++ b/fdbserver/workloads/WriteDuringRead.actor.cpp @@ -59,7 +59,9 @@ struct WriteDuringReadWorkload : TestWorkload { maximumTotalData = getOption(options, LiteralStringRef("maximumTotalData"), 3e6); minNode = getOption(options, LiteralStringRef("minNode"), 0); useSystemKeys = getOption(options, LiteralStringRef("useSystemKeys"), deterministicRandom()->random01() < 0.5); - adjacentKeys = deterministicRandom()->random01() < 0.5; + // TODO CHANGE BACK!! + // adjacentKeys = deterministicRandom()->random01() < 0.5; + adjacentKeys = false; initialKeyDensity = deterministicRandom()->random01(); // This fraction of keys are present before the first // transaction (and after an unknown result) valueSizeRange = std::make_pair( From 0d1d1d7f9ec8f3a8defeb5512123d7c25b6d82e6 Mon Sep 17 00:00:00 2001 From: Josh Slocum Date: Thu, 28 Oct 2021 18:23:43 -0500 Subject: [PATCH 013/413] fix uninitialized memory and granule bounds on manager recovery --- fdbserver/BlobManager.actor.cpp | 12 +++++- fdbserver/BlobWorker.actor.cpp | 66 ++++++++++++++++++++++----------- 2 files changed, 56 insertions(+), 22 deletions(-) diff --git a/fdbserver/BlobManager.actor.cpp b/fdbserver/BlobManager.actor.cpp index c076baeba8..2c6f753cb7 100644 --- a/fdbserver/BlobManager.actor.cpp +++ b/fdbserver/BlobManager.actor.cpp @@ -35,7 +35,7 @@ #include "flow/UnitTest.h" #include "flow/actorcompiler.h" // has to be last include -#define BM_DEBUG false +#define BM_DEBUG true // FIXME: change all BlobManagerData* to Reference to avoid segfaults if core loop gets error @@ -695,6 +695,11 @@ ACTOR Future maybeSplitRange(BlobManagerData* bmData, historyValue.parentGranules.push_back(historyValue.arena(), std::pair(granuleRange, granuleStartVersion)); + /*printf("Creating history entry [%s - %s) - [%lld - %lld)\n", + newRanges[i].printable().c_str(), + newRanges[i + 1].printable().c_str(), + granuleStartVersion, + latestVersion);*/ tr->set(historyKey, blobGranuleHistoryValueFor(historyValue)); } @@ -1035,6 +1040,11 @@ ACTOR Future recoverBlobManager(BlobManagerData* bmData) { Key granuleStartKey = results[rangeIdx].key; Key granuleEndKey = results[rangeIdx + 1].key; if (results[rangeIdx].value.size()) { + if (granuleStartKey == allKeys.begin && granuleEndKey == allKeys.end) { + // in this case, a second manager started before the first manager assigned any ranges. This + // gets the only range as [ - \xff\xff), so we clamp it to [ - \xff) + granuleEndKey = normalKeys.end; + } // note: if the old owner is dead, we handle this in rangeAssigner UID existingOwner = decodeBlobGranuleMappingValue(results[rangeIdx].value); bmData->workerAssignments.insert(KeyRangeRef(granuleStartKey, granuleEndKey), existingOwner); diff --git a/fdbserver/BlobWorker.actor.cpp b/fdbserver/BlobWorker.actor.cpp index 1c90c7fbbc..e21c11ae5a 100644 --- a/fdbserver/BlobWorker.actor.cpp +++ b/fdbserver/BlobWorker.actor.cpp @@ -148,7 +148,7 @@ struct GranuleRangeMetadata { : lastEpoch(epoch), lastSeqno(seqno), activeMetadata(activeMetadata) {} // TODO REMOVE - ~GranuleRangeMetadata() { printf("Destroying granule metadata\n"); } + // ~GranuleRangeMetadata() { printf("Destroying granule metadata\n"); } }; // represents a previous version of a granule, and optionally the files that compose it @@ -438,7 +438,8 @@ ACTOR Future updateGranuleSplitState(Transaction* tr, parentGranuleID.toString().c_str()); } - wait(updateChangeFeed(tr, KeyRef(parentGranuleID.toString()), ChangeFeedStatus::CHANGE_FEED_DESTROY)); + // FIXME: appears change feed destroy isn't working! ADD BACK + // wait(updateChangeFeed(tr, KeyRef(parentGranuleID.toString()), ChangeFeedStatus::CHANGE_FEED_DESTROY)); Key oldGranuleLockKey = blobGranuleLockKeyFor(parentGranuleRange); tr->clear(singleKeyRange(oldGranuleLockKey)); tr->clear(currentRange); @@ -874,7 +875,6 @@ ACTOR Future compactFromBlob(Reference bwData, ACTOR Future checkSplitAndReSnapshot(Reference bwData, Reference metadata, UID granuleID, - Version historyVersion, int64_t bytesInNewDeltaFiles, Future lastDeltaBeforeSnapshot) { @@ -904,8 +904,13 @@ ACTOR Future checkSplitAndReSnapshot(Reference bw loop { try { wait(bwData->currentManagerStatusStream.get().onReady()); - bwData->currentManagerStatusStream.get().send(GranuleStatusReply( - metadata->keyRange, true, statusEpoch, statusSeqno, granuleID, historyVersion, reSnapshotVersion)); + bwData->currentManagerStatusStream.get().send(GranuleStatusReply(metadata->keyRange, + true, + statusEpoch, + statusSeqno, + granuleID, + metadata->initialSnapshotVersion, + reSnapshotVersion)); break; } catch (Error& e) { wait(bwData->currentManagerStatusStream.onChange()); @@ -1220,10 +1225,6 @@ ACTOR Future blobGranuleUpdateFiles(Reference bwData, metadata->files.snapshotFiles.push_back(newSnapshotFile); metadata->durableSnapshotVersion.set(startVersion); - // construct fake history entry so we can store start version for splitting later - startState.history = - GranuleHistory(metadata->keyRange, startVersion, Standalone()); - wait(yield(TaskPriority::BlobWorkerUpdateStorage)); } metadata->initialSnapshotVersion = startVersion; @@ -1298,9 +1299,11 @@ ACTOR Future blobGranuleUpdateFiles(Reference bwData, Standalone> oldMutations = waitNext(oldChangeFeedStream.getFuture()); mutations = oldMutations; + ASSERT(mutations.back().version < startState.changeFeedStartVersion); } else { Standalone> newMutations = waitNext(changeFeedStream.getFuture()); mutations = newMutations; + ASSERT(mutations.front().version >= startState.changeFeedStartVersion); } } catch (Error& e) { // only error we should expect here is when we finish consuming old change feed @@ -1528,15 +1531,10 @@ ACTOR Future blobGranuleUpdateFiles(Reference bwData, previousFuture = inFlightFiles.back().future; ASSERT(!inFlightFiles.back().snapshot); } else { - previousFuture = Future(BlobFileIndex()); + previousFuture = Future(metadata->files.deltaFiles.back()); } - Future inFlightBlobSnapshot = - checkSplitAndReSnapshot(bwData, - metadata, - startState.granuleID, - startState.history.get().version, - metadata->bytesInNewDeltaFiles, - previousFuture); + Future inFlightBlobSnapshot = checkSplitAndReSnapshot( + bwData, metadata, startState.granuleID, metadata->bytesInNewDeltaFiles, previousFuture); inFlightFiles.push_back(InFlightFile(inFlightBlobSnapshot, metadata->pendingDeltaVersion, 0, true)); pendingSnapshots++; @@ -1892,14 +1890,21 @@ ACTOR Future handleBlobGranuleFileRequest(Reference bwData if (metadata->initialSnapshotVersion > req.readVersion) { // this is a time travel query, find previous granule if (metadata->historyLoaded.canBeSet()) { - wait(metadata->historyLoaded.getFuture()); + choose { + when(wait(metadata->historyLoaded.getFuture())) {} + when(wait(metadata->cancelled.getFuture())) { throw wrong_shard_server(); } + } } // FIXME: doesn't work once we add granule merging, could be multiple ranges and/or // multiple parents Reference cur = bwData->granuleHistory.rangeContaining(req.keyRange.begin).value(); // FIXME: use skip pointers here + Version expectedEndVersion = metadata->initialSnapshotVersion; while (cur.isValid() && req.readVersion < cur->startVersion) { + // assert version of history is contiguous + ASSERT(cur->endVersion == expectedEndVersion); + expectedEndVersion = cur->startVersion; cur = cur->parentGranule; } @@ -1910,6 +1915,9 @@ ACTOR Future handleBlobGranuleFileRequest(Reference bwData throw transaction_too_old(); } + ASSERT(cur->endVersion > req.readVersion); + ASSERT(cur->startVersion <= req.readVersion); + if (BW_REQUEST_DEBUG) { printf("[%s - %s) @ %lld time traveled back to %s [%s - %s) @ [%lld - %lld)\n", req.keyRange.begin.printable().c_str(), @@ -1935,6 +1943,22 @@ ACTOR Future handleBlobGranuleFileRequest(Reference bwData ASSERT(!chunkFiles.snapshotFiles.empty()); ASSERT(!chunkFiles.deltaFiles.empty()); + // TODO remove eventually, for help debugging asserts + if (chunkFiles.deltaFiles.back().version <= req.readVersion || + chunkFiles.snapshotFiles.front().version > req.readVersion) { + printf("Time Travel read version %lld out of bounds!\n current granule initial version: %lld\n " + "snapshot files (%d):\n", + req.readVersion, + metadata->initialSnapshotVersion, + chunkFiles.snapshotFiles.size()); + for (auto& f : chunkFiles.snapshotFiles) { + printf(" %lld\n", f.version); + } + printf(" delta files (%d):\n", chunkFiles.deltaFiles.size()); + for (auto& f : chunkFiles.deltaFiles) { + printf(" %lld\n", f.version); + } + } ASSERT(chunkFiles.deltaFiles.back().version > req.readVersion); ASSERT(chunkFiles.snapshotFiles.front().version <= req.readVersion); } else { @@ -2201,7 +2225,7 @@ ACTOR Future openGranule(Reference bwData, As if (granuleSplitState.first == BlobGranuleSplitState::Assigned) { // was already assigned, use change feed start version - ASSERT(granuleSplitState.second != invalidVersion); + ASSERT(granuleSplitState.second > 0); info.changeFeedStartVersion = granuleSplitState.second; } else if (granuleSplitState.first == BlobGranuleSplitState::Started) { wait(updateGranuleSplitState(&tr, @@ -2644,13 +2668,13 @@ ACTOR Future blobWorker(BlobWorkerInterface bwInterf, ++self->stats.rangeAssignmentRequests; state AssignBlobRangeRequest assignReq = _req; if (BW_DEBUG) { - printf("Worker %s assigned range [%s - %s) @ (%lld, %lld):\n continue=%s\n", + printf("Worker %s assigned range [%s - %s) @ (%lld, %lld):\n type=%d\n", self->id.toString().c_str(), assignReq.keyRange.begin.printable().c_str(), assignReq.keyRange.end.printable().c_str(), assignReq.managerEpoch, assignReq.managerSeqno, - assignReq.type == AssignRequestType::Continue ? "T" : "F"); + assignReq.type); } if (self->managerEpochOk(assignReq.managerEpoch)) { From 3b711af061b779555157b45b634a38526278065e Mon Sep 17 00:00:00 2001 From: Josh Slocum Date: Fri, 29 Oct 2021 12:07:16 -0500 Subject: [PATCH 014/413] Fixed a couple rollback issues and endianness of versionstamp version --- fdbclient/SystemData.cpp | 3 ++- fdbserver/BlobWorker.actor.cpp | 33 +++++++++++++++++++-------------- 2 files changed, 21 insertions(+), 15 deletions(-) diff --git a/fdbclient/SystemData.cpp b/fdbclient/SystemData.cpp index 65ae4519a2..fb87c507b1 100644 --- a/fdbclient/SystemData.cpp +++ b/fdbclient/SystemData.cpp @@ -1249,7 +1249,8 @@ std::pair decodeBlobGranuleSplitValue(const Valu BinaryReader reader(value, IncludeVersion()); reader >> st; reader >> v; - return std::pair(st, v); + + return std::pair(st, bigEndian64(v)); } const Key blobGranuleHistoryKeyFor(KeyRangeRef const& range, Version version) { diff --git a/fdbserver/BlobWorker.actor.cpp b/fdbserver/BlobWorker.actor.cpp index e21c11ae5a..6fd9d8edca 100644 --- a/fdbserver/BlobWorker.actor.cpp +++ b/fdbserver/BlobWorker.actor.cpp @@ -964,7 +964,7 @@ ACTOR Future handleCompletedDeltaFile(Reference bwData, BlobFileIndex completedDeltaFile, Key cfKey, Version cfStartVersion, - std::deque> rollbacksInProgress) { + std::deque> rollbacksCompleted) { metadata->files.deltaFiles.push_back(completedDeltaFile); ASSERT(metadata->durableDeltaVersion.get() < completedDeltaFile.version); metadata->durableDeltaVersion.set(completedDeltaFile.version); @@ -980,8 +980,8 @@ ACTOR Future handleCompletedDeltaFile(Reference bwData, Future popFuture = bwData->db->popChangeFeedMutations(cfKey, completedDeltaFile.version); wait(popFuture); } - while (!rollbacksInProgress.empty() && completedDeltaFile.version >= rollbacksInProgress.front().first) { - rollbacksInProgress.pop_front(); + while (!rollbacksCompleted.empty() && completedDeltaFile.version >= rollbacksCompleted.front().first) { + rollbacksCompleted.pop_front(); } return Void(); } @@ -1077,6 +1077,14 @@ static Version doGranuleRollback(Reference metadata, // the callers pick this new one up for the next waitForVersion metadata->bufferedDeltaVersion = NotifiedVersion(cfRollbackVersion); + // Track that this rollback happened, since we have to re-read mutations up to the rollback + // Add this rollback to in progress, and put all completed ones back in progress + rollbacksInProgress.push_back(std::pair(rollbackVersion, mutationVersion)); + for (int i = rollbacksCompleted.size() - 1; i >= 0; i--) { + rollbacksInProgress.push_front(rollbacksCompleted[i]); + } + rollbacksCompleted.clear(); + } else { // No pending delta files to discard, just in-memory mutations @@ -1105,9 +1113,10 @@ static Version doGranuleRollback(Reference metadata, // delete all deltas in rollback range, but we can optimize here to just skip the uncommitted mutations // directly and immediately pop the rollback out of inProgress - // TODO: bufferedDeltaVersion is of type Notified so you must set it to something greater than the previous val - // This does not hold true for rollback versions. - metadata->bufferedDeltaVersion.set(rollbackVersion); + + // Create new notified version so it doesn't go backwards. Do this before signaling the rollback counter so that + // the callers pick this new one up for the next waitForVersion + metadata->bufferedDeltaVersion = NotifiedVersion(rollbackVersion); cfRollbackVersion = mutationVersion; } @@ -1120,13 +1129,6 @@ static Version doGranuleRollback(Reference metadata, metadata->rollbackCount.set(metadata->rollbackCount.get() + 1); - // add this rollback to in progress, and put all completed ones back in progress - rollbacksInProgress.push_back(std::pair(rollbackVersion, mutationVersion)); - for (int i = rollbacksCompleted.size() - 1; i >= 0; i--) { - rollbacksInProgress.push_front(rollbacksCompleted[i]); - } - rollbacksCompleted.clear(); - return cfRollbackVersion; } @@ -1346,11 +1348,14 @@ ACTOR Future blobGranuleUpdateFiles(Reference bwData, br >> rollbackVersion; ASSERT(rollbackVersion >= metadata->durableDeltaVersion.get()); + ASSERT(rollbackVersion >= committedVersion.get()); if (!rollbacksInProgress.empty()) { ASSERT(rollbacksInProgress.front().first == rollbackVersion); ASSERT(rollbacksInProgress.front().second == deltas.version); - printf("Passed rollback %lld -> %lld\n", deltas.version, rollbackVersion); + if (BW_DEBUG) { + printf("Passed rollback %lld -> %lld\n", deltas.version, rollbackVersion); + } rollbacksCompleted.push_back(rollbacksInProgress.front()); rollbacksInProgress.pop_front(); } else { From ba53045fcdbf5b5a3f022bc27204751bae8374aa Mon Sep 17 00:00:00 2001 From: Josh Slocum Date: Thu, 11 Nov 2021 12:33:08 -0600 Subject: [PATCH 015/413] Fixing BW count for multi-process machines and having BW monitor its own removal since it doesn't get private mutations --- fdbserver/BlobWorker.actor.cpp | 46 ++++++++++++++++++++++++++-- fdbserver/SimulatedCluster.actor.cpp | 3 +- 2 files changed, 45 insertions(+), 4 deletions(-) diff --git a/fdbserver/BlobWorker.actor.cpp b/fdbserver/BlobWorker.actor.cpp index 6fd9d8edca..361a2f7e06 100644 --- a/fdbserver/BlobWorker.actor.cpp +++ b/fdbserver/BlobWorker.actor.cpp @@ -2599,6 +2599,31 @@ ACTOR Future handleRangeRevoke(Reference bwData, RevokeBlo } } +ACTOR Future monitorRemoval(Reference bwData) { + state Key blobWorkerListKey = blobWorkerListKeyFor(bwData->id); + loop { + loop { + state ReadYourWritesTransaction tr(bwData->db); + try { + tr.setOption(FDBTransactionOptions::READ_SYSTEM_KEYS); + tr.setOption(FDBTransactionOptions::PRIORITY_SYSTEM_IMMEDIATE); + + Optional val = wait(tr.get(blobWorkerListKey)); + if (!val.present()) { + return Void(); + } + + state Future watchFuture = tr.watch(blobWorkerListKey); + + wait(tr.commit()); + wait(watchFuture); + } catch (Error& e) { + wait(tr.onError(e)); + } + } + } +} + ACTOR Future blobWorker(BlobWorkerInterface bwInterf, ReplyPromise recruitReply, Reference const> dbInfo) { @@ -2651,6 +2676,9 @@ ACTOR Future blobWorker(BlobWorkerInterface bwInterf, recruitReply.send(rep); self->addActor.send(waitFailureServer(bwInterf.waitFailure.getFuture())); + state Future selfRemoved = monitorRemoval(self); + + TraceEvent("BlobWorkerInit", self->id); try { loop choose { @@ -2712,16 +2740,28 @@ ACTOR Future blobWorker(BlobWorkerInterface bwInterf, when(HaltBlobWorkerRequest req = waitNext(bwInterf.haltBlobWorker.getFuture())) { req.reply.send(Void()); if (self->managerEpochOk(req.managerEpoch)) { - TraceEvent("BlobWorkerHalted", bwInterf.id()).detail("ReqID", req.requesterID); - printf("BW %s was halted\n", bwInterf.id().toString().c_str()); + TraceEvent("BlobWorkerHalted", self->id) + .detail("ReqID", req.requesterID) + .detail("ManagerEpoch", req.managerEpoch); + if (BW_DEBUG) { + printf( + "BW %s was halted by manager %lld\n", bwInterf.id().toString().c_str(), req.managerEpoch); + } break; } } when(wait(collection)) { - TraceEvent("BlobWorkerActorCollectionError"); + TraceEvent("BlobWorkerActorCollectionError", self->id); ASSERT(false); throw internal_error(); } + when(wait(selfRemoved)) { + if (BW_DEBUG) { + printf("Blob worker detected removal. Exiting...\n"); + } + TraceEvent("BlobWorkerRemoved", self->id); + return Void(); + } } } catch (Error& e) { if (e.code() == error_code_operation_cancelled) { diff --git a/fdbserver/SimulatedCluster.actor.cpp b/fdbserver/SimulatedCluster.actor.cpp index 325b15d983..33028f78ec 100644 --- a/fdbserver/SimulatedCluster.actor.cpp +++ b/fdbserver/SimulatedCluster.actor.cpp @@ -2020,7 +2020,8 @@ void setupSimulatedSystem(std::vector>* systemActors, int storageCacheMachines = dc == 0 ? 1 : 0; int blobWorkerMachines = 0; if (CLIENT_KNOBS->ENABLE_BLOB_GRANULES) { - blobWorkerMachines = 2 + deterministicRandom()->randomInt(0, NUM_EXTRA_BW_MACHINES + 1); + int blobWorkerProcesses = 1 + deterministicRandom()->randomInt(0, NUM_EXTRA_BW_MACHINES + 1); + blobWorkerMachines = std::max(1, blobWorkerProcesses / processesPerMachine); } int totalMachines = machines + storageCacheMachines + blobWorkerMachines; From bab8756e17a47047aceffafb509508729999f3b6 Mon Sep 17 00:00:00 2001 From: Josh Slocum Date: Thu, 11 Nov 2021 13:50:19 -0600 Subject: [PATCH 016/413] fixed bugs in small splits --- fdbclient/BlobGranuleCommon.h | 2 +- fdbclient/NativeAPI.actor.cpp | 4 ++-- fdbserver/BlobManager.actor.cpp | 33 ++++++++++++++++----------------- fdbserver/BlobWorker.actor.cpp | 8 ++++---- 4 files changed, 23 insertions(+), 24 deletions(-) diff --git a/fdbclient/BlobGranuleCommon.h b/fdbclient/BlobGranuleCommon.h index c9b4d11d43..7c44838c18 100644 --- a/fdbclient/BlobGranuleCommon.h +++ b/fdbclient/BlobGranuleCommon.h @@ -87,7 +87,7 @@ struct BlobGranuleChunkRef { } }; -enum BlobGranuleSplitState { Unknown = 0, Started = 1, Assigned = 2, Done = 3 }; +enum BlobGranuleSplitState { Unknown = 0, Initialized = 1, Assigned = 2, Done = 3 }; struct BlobGranuleHistoryValue { constexpr static FileIdentifier file_identifier = 991434; diff --git a/fdbclient/NativeAPI.actor.cpp b/fdbclient/NativeAPI.actor.cpp index 3e5022bf68..9c379963e5 100644 --- a/fdbclient/NativeAPI.actor.cpp +++ b/fdbclient/NativeAPI.actor.cpp @@ -256,7 +256,6 @@ std::string printable(const VectorRef& val) { std::string printable(const StringRef& val) { return val.printable(); } - std::string printable(const std::string& str) { return StringRef(str).printable(); } @@ -6432,7 +6431,8 @@ ACTOR Future>> splitStorageMetrics(Database cx, //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)) { + if (used.allLessOrEqual(limit * CLIENT_KNOBS->STORAGE_METRICS_UNFAIR_SPLIT_LIMIT) && + results.size() > 1) { results.resize(results.arena(), results.size() - 1); } diff --git a/fdbserver/BlobManager.actor.cpp b/fdbserver/BlobManager.actor.cpp index 2c6f753cb7..822769d67f 100644 --- a/fdbserver/BlobManager.actor.cpp +++ b/fdbserver/BlobManager.actor.cpp @@ -255,6 +255,7 @@ ACTOR Future>> splitRange(Reference> keys = wait(tr->getTransaction().splitStorageMetrics(range, splitMetrics, estimated)); + ASSERT(keys.size() >= 2); return keys; } else { // printf(" Not splitting range\n"); @@ -613,10 +614,8 @@ ACTOR Future maybeSplitRange(BlobManagerData* bmData, state int64_t newLockSeqno = -1; // first get ranges to split - if (newRanges.empty()) { - Standalone> _newRanges = wait(splitRange(tr, granuleRange)); - newRanges = _newRanges; - } + Standalone> _newRanges = wait(splitRange(tr, granuleRange)); + newRanges = _newRanges; if (newRanges.size() == 2) { // not large enough to split, just reassign back to worker @@ -635,13 +634,23 @@ ACTOR Future maybeSplitRange(BlobManagerData* bmData, return Void(); } + if (BM_DEBUG) { + printf("Splitting range [%s - %s) into (%d):\n", + granuleRange.begin.printable().c_str(), + granuleRange.end.printable().c_str(), + newRanges.size() - 1); + for (int i = 0; i < newRanges.size(); i++) { + printf(" %s\n", newRanges[i].printable().c_str()); + } + } + // Need to split range. Persist intent to split and split metadata to DB BEFORE sending split requests loop { try { tr->reset(); tr->setOption(FDBTransactionOptions::Option::PRIORITY_SYSTEM_IMMEDIATE); tr->setOption(FDBTransactionOptions::Option::ACCESS_SYSTEM_KEYS); - ASSERT(newRanges.size() >= 2); + ASSERT(newRanges.size() > 2); // make sure we're still manager when this transaction gets committed wait(checkManagerLock(tr, bmData)); @@ -685,7 +694,7 @@ ACTOR Future maybeSplitRange(BlobManagerData* bmData, Key splitKey = blobGranuleSplitKeyFor(granuleID, newGranuleID); tr->atomicOp(splitKey, - blobGranuleSplitValueFor(BlobGranuleSplitState::Started), + blobGranuleSplitValueFor(BlobGranuleSplitState::Initialized), MutationRef::SetVersionstampedValue); Key historyKey = blobGranuleHistoryKeyFor(KeyRangeRef(newRanges[i], newRanges[i + 1]), latestVersion); @@ -716,16 +725,6 @@ ACTOR Future maybeSplitRange(BlobManagerData* bmData, } } - if (BM_DEBUG) { - printf("Splitting range [%s - %s) into (%d):\n", - granuleRange.begin.printable().c_str(), - granuleRange.end.printable().c_str(), - newRanges.size() - 1); - for (int i = 0; i < newRanges.size() - 1; i++) { - printf(" [%s - %s)\n", newRanges[i].printable().c_str(), newRanges[i + 1].printable().c_str()); - } - } - // transaction committed, send range assignments // revoke from current worker RangeAssignment raRevoke; @@ -1084,7 +1083,7 @@ ACTOR Future recoverBlobManager(BlobManagerData* bmData) { std::tie(parentGranuleID, granuleID) = decodeBlobGranuleSplitKey(split.key); std::tie(splitState, version) = decodeBlobGranuleSplitValue(split.value); const KeyRange range = blobGranuleSplitKeyRangeFor(parentGranuleID); - if (splitState <= BlobGranuleSplitState::Started) { + if (splitState <= BlobGranuleSplitState::Initialized) { bmData->workerAssignments.insert(range, UID()); } } diff --git a/fdbserver/BlobWorker.actor.cpp b/fdbserver/BlobWorker.actor.cpp index 361a2f7e06..fb10eff2ac 100644 --- a/fdbserver/BlobWorker.actor.cpp +++ b/fdbserver/BlobWorker.actor.cpp @@ -405,7 +405,7 @@ ACTOR Future updateGranuleSplitState(Transaction* tr, BlobGranuleSplitState st = decodeBlobGranuleSplitValue(it.value).first; ASSERT(st != BlobGranuleSplitState::Unknown); - if (st == BlobGranuleSplitState::Started) { + if (st == BlobGranuleSplitState::Initialized) { totalStarted++; } else if (st == BlobGranuleSplitState::Done) { totalDone++; @@ -445,7 +445,7 @@ ACTOR Future updateGranuleSplitState(Transaction* tr, tr->clear(currentRange); } else { tr->atomicOp(myStateKey, blobGranuleSplitValueFor(newState), MutationRef::SetVersionstampedValue); - if (newState == BlobGranuleSplitState::Assigned && currentState == BlobGranuleSplitState::Started && + if (newState == BlobGranuleSplitState::Assigned && currentState == BlobGranuleSplitState::Initialized && totalStarted == 1) { // We are the last one to change from Start -> Assigned, so we can stop the parent change feed. if (BW_DEBUG) { @@ -2221,7 +2221,7 @@ ACTOR Future openGranule(Reference bwData, As info.parentGranule = std::pair(parentGranuleRange, parentGranuleID); state std::pair granuleSplitState = - std::pair(BlobGranuleSplitState::Started, invalidVersion); + std::pair(BlobGranuleSplitState::Initialized, invalidVersion); if (hasPrevOwner) { std::pair _gss = wait(getGranuleSplitState(&tr, parentGranuleID, info.granuleID)); @@ -2232,7 +2232,7 @@ ACTOR Future openGranule(Reference bwData, As // was already assigned, use change feed start version ASSERT(granuleSplitState.second > 0); info.changeFeedStartVersion = granuleSplitState.second; - } else if (granuleSplitState.first == BlobGranuleSplitState::Started) { + } else if (granuleSplitState.first == BlobGranuleSplitState::Initialized) { wait(updateGranuleSplitState(&tr, info.parentGranule.get().first, info.parentGranule.get().second, From fd4f13fba1d755dfced93a8012a64633afdc62d6 Mon Sep 17 00:00:00 2001 From: Josh Slocum Date: Mon, 15 Nov 2021 09:49:51 -0600 Subject: [PATCH 017/413] Added fetch version to change feeds to avoid races and overwrites between updateStorage and fetchChangeFeed --- fdbserver/BlobWorker.actor.cpp | 19 +++++++++++------ fdbserver/storageserver.actor.cpp | 35 ++++++++++++++++++++++--------- 2 files changed, 38 insertions(+), 16 deletions(-) diff --git a/fdbserver/BlobWorker.actor.cpp b/fdbserver/BlobWorker.actor.cpp index fb10eff2ac..2b26e35dc2 100644 --- a/fdbserver/BlobWorker.actor.cpp +++ b/fdbserver/BlobWorker.actor.cpp @@ -196,7 +196,6 @@ struct BlobWorkerData : NonCopyable, ReferenceCounted { PromiseStream granuleUpdateErrors; BlobWorkerData(UID id, Database db) : id(id), db(db), stats(id, SERVER_KNOBS->WORKER_LOGGING_INTERVAL) {} - ~BlobWorkerData() { printf("Destroying blob worker data for %s\n", id.toString().c_str()); } bool managerEpochOk(int64_t epoch) { if (epoch < currentManagerEpoch) { @@ -1619,7 +1618,9 @@ ACTOR Future blobGranuleUpdateFiles(Reference bwData, } } catch (Error& e) { // TODO REMOVE - printf("BGUF got error %s\n", e.name()); + if (BW_DEBUG) { + printf("BGUF got error %s\n", e.name()); + } if (e.code() == error_code_operation_cancelled) { throw; } @@ -2216,7 +2217,9 @@ ACTOR Future openGranule(Reference bwData, As std::tuple parentGranuleLock = decodeBlobGranuleLockValue(parentGranuleLockValue.get()); UID parentGranuleID = std::get<2>(parentGranuleLock); - printf(" parent granule id %s\n", parentGranuleID.toString().c_str()); + if (BW_DEBUG) { + printf(" parent granule id %s\n", parentGranuleID.toString().c_str()); + } info.parentGranule = std::pair(parentGranuleRange, parentGranuleID); @@ -2379,7 +2382,9 @@ ACTOR Future changeBlobRange(Reference bwData, } }*/ bool thisAssignmentNewer = newerRangeAssignment(r.value(), epoch, seqno); - printf("thisAssignmentNewer=%s\n", thisAssignmentNewer ? "true" : "false"); + if (BW_DEBUG) { + printf("thisAssignmentNewer=%s\n", thisAssignmentNewer ? "true" : "false"); + } // if this granule already has it, and this was a specialassignment (i.e. a new blob maanger is // trying to reassign granules), then just continue @@ -2391,7 +2396,9 @@ ACTOR Future changeBlobRange(Reference bwData, break; } - printf("last: (%d, %d). now: (%d, %d)\n", r.value().lastEpoch, r.value().lastSeqno, epoch, seqno); + if (BW_DEBUG) { + printf("last: (%d, %d). now: (%d, %d)\n", r.value().lastEpoch, r.value().lastSeqno, epoch, seqno); + } if (r.value().lastEpoch == epoch && r.value().lastSeqno == seqno) { ASSERT(r.begin() == keyRange.begin); @@ -2760,7 +2767,7 @@ ACTOR Future blobWorker(BlobWorkerInterface bwInterf, printf("Blob worker detected removal. Exiting...\n"); } TraceEvent("BlobWorkerRemoved", self->id); - return Void(); + break; } } } catch (Error& e) { diff --git a/fdbserver/storageserver.actor.cpp b/fdbserver/storageserver.actor.cpp index 93d84da2fe..e5df483c84 100644 --- a/fdbserver/storageserver.actor.cpp +++ b/fdbserver/storageserver.actor.cpp @@ -328,8 +328,13 @@ struct FetchInjectionInfo { struct ChangeFeedInfo : ReferenceCounted { std::deque> mutations; + Version fetchVersion = + invalidVersion; // The version that commits from a fetch have been written to storage, but have not yet been + // committed as part of updateStorage. Because a fetch can merge mutations with incoming data + // before updateStorage updates the storage version, updateStorage must know to skip mutations + // that have already been written to storage by change feed fetch. Version storageVersion = invalidVersion; // The version between the storage version and the durable version are - // currently being written to disk + // being written to disk as part of the current commit in updateStorage Version durableVersion = invalidVersion; // All versions before the durable version are durable on disk Version emptyVersion = 0; // The change feed does not have any mutations before emptyVersion KeyRange range; @@ -1587,6 +1592,7 @@ ACTOR Future changeFeedPopQ(StorageServer* self, ChangeFeedPopRequest req) if (req.version > feed->second->storageVersion) { feed->second->storageVersion = invalidVersion; feed->second->durableVersion = invalidVersion; + feed->second->fetchVersion = invalidVersion; } wait(self->durableVersion.whenAtLeast(self->storageVersion() + 1)); } @@ -3307,8 +3313,8 @@ ACTOR Future fetchChangeFeedApplier(StorageServer* data, data->storage.writeKeyValue( KeyValueRef(changeFeedDurableKey(rangeId, it.version), changeFeedDurableValue(it.mutations, it.knownCommittedVersion))); - changeFeedInfo->storageVersion = std::max(changeFeedInfo->durableVersion, it.version); - changeFeedInfo->durableVersion = changeFeedInfo->storageVersion; + + changeFeedInfo->fetchVersion = std::max(changeFeedInfo->fetchVersion, it.version); } } wait(yield()); @@ -3341,9 +3347,8 @@ ACTOR Future fetchChangeFeedApplier(StorageServer* data, KeyValueRef(changeFeedDurableKey(rangeId, remoteResult[remoteLoc].version), changeFeedDurableValue(remoteResult[remoteLoc].mutations, remoteResult[remoteLoc].knownCommittedVersion))); - changeFeedInfo->storageVersion = - std::max(changeFeedInfo->durableVersion, remoteResult[remoteLoc].version); - changeFeedInfo->durableVersion = changeFeedInfo->storageVersion; + changeFeedInfo->fetchVersion = + std::max(changeFeedInfo->fetchVersion, remoteResult[remoteLoc].version); } remoteLoc++; } else if (remoteResult[remoteLoc].version == localResult.version) { @@ -3355,9 +3360,9 @@ ACTOR Future fetchChangeFeedApplier(StorageServer* data, KeyValueRef(changeFeedDurableKey(rangeId, remoteResult[remoteLoc].version), changeFeedDurableValue(remoteResult[remoteLoc].mutations, remoteResult[remoteLoc].knownCommittedVersion))); - changeFeedInfo->storageVersion = - std::max(changeFeedInfo->durableVersion, remoteResult[remoteLoc].version); - changeFeedInfo->durableVersion = changeFeedInfo->storageVersion; + + changeFeedInfo->fetchVersion = + std::max(changeFeedInfo->fetchVersion, remoteResult[remoteLoc].version); } remoteLoc++; Standalone _localResult = waitNext(localResults.getFuture()); @@ -4297,6 +4302,7 @@ private: if (popVersion > feed->second->storageVersion) { feed->second->storageVersion = invalidVersion; feed->second->durableVersion = invalidVersion; + // don't set fetchVersion to invalidVersion here because there could be an active fetch } } } @@ -4850,15 +4856,23 @@ ACTOR Future updateStorage(StorageServer* data) { while (curFeed < updatedChangeFeeds.size()) { auto info = data->uidChangeFeed.find(updatedChangeFeeds[curFeed]); if (info != data->uidChangeFeed.end()) { + // Cannot yield in mutation updating loop because of race between fetchVersion and storageVersion for (auto& it : info->second->mutations) { - if (it.version > newOldestVersion) { + if (it.version <= info->second->fetchVersion) { + continue; + } else if (it.version > newOldestVersion) { break; } data->storage.writeKeyValue( KeyValueRef(changeFeedDurableKey(info->second->id, it.version), changeFeedDurableValue(it.mutations, it.knownCommittedVersion))); + ASSERT(it.version > info->second->storageVersion); info->second->storageVersion = it.version; } + // handle case where fetch had version ahead of last in-memory mutation + if (info->second->fetchVersion > info->second->storageVersion) { + info->second->storageVersion = std::min(info->second->fetchVersion, newOldestVersion); + } wait(yield(TaskPriority::UpdateStorage)); } curFeed++; @@ -4904,6 +4918,7 @@ ACTOR Future updateStorage(StorageServer* data) { while (!info->second->mutations.empty() && info->second->mutations.front().version < newOldestVersion) { info->second->mutations.pop_front(); } + ASSERT(info->second->storageVersion >= info->second->durableVersion); info->second->durableVersion = info->second->storageVersion; wait(yield(TaskPriority::UpdateStorage)); } From 1d1f3fe74794f30978c01d898902a6bc74ffa446 Mon Sep 17 00:00:00 2001 From: Josh Slocum Date: Mon, 15 Nov 2021 12:24:51 -0600 Subject: [PATCH 018/413] changing assert for now --- fdbserver/storageserver.actor.cpp | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/fdbserver/storageserver.actor.cpp b/fdbserver/storageserver.actor.cpp index e5df483c84..2dc051ffc5 100644 --- a/fdbserver/storageserver.actor.cpp +++ b/fdbserver/storageserver.actor.cpp @@ -4866,7 +4866,9 @@ ACTOR Future updateStorage(StorageServer* data) { data->storage.writeKeyValue( KeyValueRef(changeFeedDurableKey(info->second->id, it.version), changeFeedDurableValue(it.mutations, it.knownCommittedVersion))); - ASSERT(it.version > info->second->storageVersion); + // FIXME: there appears to be a bug somewhere where the exact same mutation appears twice in a row + // in the stream. We should fix this assert to be strictly > and re-enable it + ASSERT(it.version >= info->second->storageVersion); info->second->storageVersion = it.version; } // handle case where fetch had version ahead of last in-memory mutation From ac0a5750e74b9cc8f9d2089ee00c573a378a6e08 Mon Sep 17 00:00:00 2001 From: Suraj Gupta Date: Mon, 15 Nov 2021 16:44:59 -0500 Subject: [PATCH 019/413] Clamp end of system keys down to normal keys. --- fdbserver/BlobManager.actor.cpp | 11 +++++------ 1 file changed, 5 insertions(+), 6 deletions(-) diff --git a/fdbserver/BlobManager.actor.cpp b/fdbserver/BlobManager.actor.cpp index b8b2e7437d..d2e8ba491f 100644 --- a/fdbserver/BlobManager.actor.cpp +++ b/fdbserver/BlobManager.actor.cpp @@ -1039,11 +1039,6 @@ ACTOR Future recoverBlobManager(BlobManagerData* bmData) { Key granuleStartKey = results[rangeIdx].key; Key granuleEndKey = results[rangeIdx + 1].key; if (results[rangeIdx].value.size()) { - if (granuleStartKey == allKeys.begin && granuleEndKey == allKeys.end) { - // in this case, a second manager started before the first manager assigned any ranges. This - // gets the only range as [ - \xff\xff), so we clamp it to [ - \xff) - granuleEndKey = normalKeys.end; - } // note: if the old owner is dead, we handle this in rangeAssigner UID existingOwner = decodeBlobGranuleMappingValue(results[rangeIdx].value); bmData->workerAssignments.insert(KeyRangeRef(granuleStartKey, granuleEndKey), existingOwner); @@ -1100,10 +1095,14 @@ ACTOR Future recoverBlobManager(BlobManagerData* bmData) { // Step 3. Send assign requests for all the granules for (auto& range : bmData->workerAssignments.intersectingRanges(normalKeys)) { + // a second manager started before the first manager assigned any ranges so the + // the only range is [ - \xff\xff), so we clamp it to [ - \xff) + Key end = range.end() == allKeys.end ? normalKeys.end : range.end(); + RangeAssignment raAssign; raAssign.isAssign = true; raAssign.worker = range.value(); - raAssign.keyRange = range.range(); + raAssign.keyRange = KeyRangeRef(range.begin(), end); raAssign.assign = RangeAssignmentData(AssignRequestType::Reassign); bmData->rangesToAssign.send(raAssign); } From 1817b135aca3fecb93b2ec210b96d6b6a7547890 Mon Sep 17 00:00:00 2001 From: Suraj Gupta Date: Mon, 15 Nov 2021 18:19:14 -0500 Subject: [PATCH 020/413] Use a local keyRangeMap to avoid assigning ranges not part of client ranges. --- fdbserver/BlobManager.actor.cpp | 22 +++++++++++++--------- 1 file changed, 13 insertions(+), 9 deletions(-) diff --git a/fdbserver/BlobManager.actor.cpp b/fdbserver/BlobManager.actor.cpp index d2e8ba491f..b077c22da8 100644 --- a/fdbserver/BlobManager.actor.cpp +++ b/fdbserver/BlobManager.actor.cpp @@ -1019,6 +1019,7 @@ ACTOR Future recoverBlobManager(BlobManagerData* bmData) { // If the worker already had the range, this is a no-op. If the worker didn't have it, it will // begin persisting it. The worker that had the same range before will now be at a lower seqno. + state KeyRangeMap> workerAssignments; state Reference tr = makeReference(bmData->db); // Step 1. Get the latest known mapping of granules to blob workers (i.e. assignments) @@ -1041,7 +1042,7 @@ ACTOR Future recoverBlobManager(BlobManagerData* bmData) { if (results[rangeIdx].value.size()) { // note: if the old owner is dead, we handle this in rangeAssigner UID existingOwner = decodeBlobGranuleMappingValue(results[rangeIdx].value); - bmData->workerAssignments.insert(KeyRangeRef(granuleStartKey, granuleEndKey), existingOwner); + workerAssignments.insert(KeyRangeRef(granuleStartKey, granuleEndKey), existingOwner); } } @@ -1079,7 +1080,7 @@ ACTOR Future recoverBlobManager(BlobManagerData* bmData) { std::tie(splitState, version) = decodeBlobGranuleSplitValue(split.value); const KeyRange range = blobGranuleSplitKeyRangeFor(parentGranuleID); if (splitState <= BlobGranuleSplitState::Initialized) { - bmData->workerAssignments.insert(range, UID()); + workerAssignments.insert(range, UID()); } } @@ -1093,16 +1094,19 @@ ACTOR Future recoverBlobManager(BlobManagerData* bmData) { } } - // Step 3. Send assign requests for all the granules - for (auto& range : bmData->workerAssignments.intersectingRanges(normalKeys)) { - // a second manager started before the first manager assigned any ranges so the - // the only range is [ - \xff\xff), so we clamp it to [ - \xff) - Key end = range.end() == allKeys.end ? normalKeys.end : range.end(); + // Step 3. Send assign requests for all the granules and transfer assignments + // from local workerAssignments to bmData + for (auto& range : workerAssignments.intersectingRanges(normalKeys)) { + if (!range.value().present()) { + continue; + } + + bmData->workerAssignments.insert(range.range(), range.value().get()); RangeAssignment raAssign; raAssign.isAssign = true; - raAssign.worker = range.value(); - raAssign.keyRange = KeyRangeRef(range.begin(), end); + raAssign.worker = range.value().get(); + raAssign.keyRange = range.range(); raAssign.assign = RangeAssignmentData(AssignRequestType::Reassign); bmData->rangesToAssign.send(raAssign); } From dba0a2d729fdbc0381bd6fe5c8bd94280f117d5e Mon Sep 17 00:00:00 2001 From: Suraj Gupta Date: Tue, 16 Nov 2021 09:46:51 -0500 Subject: [PATCH 021/413] Add comment to clarify empty UID usage. --- fdbserver/BlobManager.actor.cpp | 1 + 1 file changed, 1 insertion(+) diff --git a/fdbserver/BlobManager.actor.cpp b/fdbserver/BlobManager.actor.cpp index b077c22da8..75114d029a 100644 --- a/fdbserver/BlobManager.actor.cpp +++ b/fdbserver/BlobManager.actor.cpp @@ -1080,6 +1080,7 @@ ACTOR Future recoverBlobManager(BlobManagerData* bmData) { std::tie(splitState, version) = decodeBlobGranuleSplitValue(split.value); const KeyRange range = blobGranuleSplitKeyRangeFor(parentGranuleID); if (splitState <= BlobGranuleSplitState::Initialized) { + // the empty UID signifies that we need to find an owner (worker) for this range workerAssignments.insert(range, UID()); } } From 2c82a27f097a57f244e26e30e1cd870eaba2a42b Mon Sep 17 00:00:00 2001 From: Josh Slocum Date: Wed, 24 Nov 2021 14:35:23 -0600 Subject: [PATCH 022/413] fixed typo in fmt::printf --- fdbserver/BlobWorker.actor.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/fdbserver/BlobWorker.actor.cpp b/fdbserver/BlobWorker.actor.cpp index e0768e3219..6f48522556 100644 --- a/fdbserver/BlobWorker.actor.cpp +++ b/fdbserver/BlobWorker.actor.cpp @@ -1456,7 +1456,7 @@ ACTOR Future blobGranuleUpdateFiles(Reference bwData, // Write a new delta file IF we have enough bytes if (metadata->bufferedDeltaBytes >= SERVER_KNOBS->BG_DELTA_FILE_TARGET_BYTES) { if (BW_DEBUG) { - fmt::print("Granule [{0} - {1}) flushing delta file after {2} bytes @ {3} {5}\n", + fmt::print("Granule [{0} - {1}) flushing delta file after {2} bytes @ {3} {4}\n", metadata->keyRange.begin.printable(), metadata->keyRange.end.printable(), metadata->bufferedDeltaBytes, From f12ac4468eea3e90b3d12df0693b525594a680e2 Mon Sep 17 00:00:00 2001 From: Evan Tschannen Date: Tue, 30 Nov 2021 13:32:12 -0800 Subject: [PATCH 023/413] fix: fetched change feeds did not get their durable version updated --- fdbserver/storageserver.actor.cpp | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/fdbserver/storageserver.actor.cpp b/fdbserver/storageserver.actor.cpp index 954e6936af..529d3041d6 100644 --- a/fdbserver/storageserver.actor.cpp +++ b/fdbserver/storageserver.actor.cpp @@ -622,6 +622,7 @@ public: Deque, Version>> changeFeedVersions; std::map> changeFeedRemovals; std::set currentChangeFeeds; + std::set fetchingChangeFeeds; std::unordered_map> changeFeedClientVersions; // newestAvailableVersion[k] @@ -3981,6 +3982,7 @@ ACTOR Future fetchChangeFeed(StorageServer* data, loop { try { wait(fetchChangeFeedApplier(data, changeFeedInfo, rangeId, range, fetchVersion, existing)); + data->fetchingChangeFeeds.insert(rangeId); return Void(); } catch (Error& e) { if (e.code() != error_code_change_feed_not_registered) { @@ -5457,7 +5459,8 @@ ACTOR Future updateStorage(StorageServer* data) { break; } - std::set modifiedChangeFeeds; + std::set modifiedChangeFeeds = data->fetchingChangeFeeds; + data->fetchingChangeFeeds.clear(); while (!data->changeFeedVersions.empty() && data->changeFeedVersions.front().second <= newOldestVersion) { modifiedChangeFeeds.insert(data->changeFeedVersions.front().first.begin(), data->changeFeedVersions.front().first.end()); From 85c6e30aec59557740daad8f11513c5e58a02656 Mon Sep 17 00:00:00 2001 From: Josh Slocum Date: Thu, 2 Dec 2021 09:51:24 -0600 Subject: [PATCH 024/413] Fix: several empty version issues with large single-version change feed entries --- fdbclient/NativeAPI.actor.cpp | 11 +++++++++++ fdbserver/storageserver.actor.cpp | 21 ++++++++++++++++----- 2 files changed, 27 insertions(+), 5 deletions(-) diff --git a/fdbclient/NativeAPI.actor.cpp b/fdbclient/NativeAPI.actor.cpp index 4c7c7cfa5e..b0bdf119cd 100644 --- a/fdbclient/NativeAPI.actor.cpp +++ b/fdbclient/NativeAPI.actor.cpp @@ -6897,6 +6897,15 @@ ACTOR Future singleChangeFeedStream(StorageServerInterface interf, if (rep.mutations[resultLoc].version >= nextVersion) { results.send(rep.mutations[resultLoc]); } else { + // TODO REMOVE eventually, useful for debugging for now + if (!rep.mutations[resultLoc].mutations.empty()) { + printf( + "non-empty mutations (%d), but versions out of order from %s! mv=%lld, nv=%lld\n", + rep.mutations.size(), + interf.id().toString().substr(0, 4).c_str(), + rep.mutations[resultLoc].version, + nextVersion); + } ASSERT(rep.mutations[resultLoc].mutations.empty()); } resultLoc++; @@ -6930,6 +6939,8 @@ ACTOR Future singleChangeFeedStream(StorageServerInterface interf, } } } catch (Error& e) { + // TODO REMOVE eventually, useful for debugging for now + printf("NAS: CFError %s\n", e.name()); if (e.code() == error_code_actor_cancelled) { throw; } diff --git a/fdbserver/storageserver.actor.cpp b/fdbserver/storageserver.actor.cpp index 529d3041d6..cc48631ead 100644 --- a/fdbserver/storageserver.actor.cpp +++ b/fdbserver/storageserver.actor.cpp @@ -1740,6 +1740,7 @@ ACTOR Future> getChangeFeedMutations(Stor 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(); if (data->version.get() < req.begin) { wait(data->version.whenAtLeast(req.begin)); @@ -1835,7 +1836,10 @@ ACTOR Future> getChangeFeedMutations(Stor } } - return std::make_pair(reply, remainingLimitBytes > 0 && remainingDurableBytes > 0); + // If the SS's version advanced at all during any of the waits, the read from memory may have missed some mutations, + // so gotAll can only be true if data->version didn't change over the course of this actor + return std::make_pair(reply, + remainingLimitBytes > 0 && remainingDurableBytes > 0 && data->version.get() == startVersion); } ACTOR Future localChangeFeedStream(StorageServer* data, @@ -1887,7 +1891,7 @@ ACTOR Future changeFeedStreamQ(StorageServer* data, ChangeFeedStreamReques try { loop { Future onReady = req.reply.onReady(); - if (atLatest && !onReady.isReady()) { + if (atLatest && !onReady.isReady() && !removeUID) { data->changeFeedClientVersions[req.reply.getEndpoint().getPrimaryAddress()][streamUID] = blockedVersion.present() ? blockedVersion.get() : data->prevVersion; removeUID = true; @@ -1908,18 +1912,25 @@ ACTOR Future changeFeedStreamQ(StorageServer* data, ChangeFeedStreamReques if (!atLatest && gotAll) { atLatest = true; } + auto& clientVersions = data->changeFeedClientVersions[req.reply.getEndpoint().getPrimaryAddress()]; Version minVersion = removeUID ? data->version.get() : data->prevVersion; if (removeUID) { - data->changeFeedClientVersions[req.reply.getEndpoint().getPrimaryAddress()].erase(streamUID); - removeUID = false; + if (gotAll) { + data->changeFeedClientVersions[req.reply.getEndpoint().getPrimaryAddress()].erase(streamUID); + removeUID = false; + } else { + data->changeFeedClientVersions[req.reply.getEndpoint().getPrimaryAddress()][streamUID] = + feedReply.mutations.back().version; + } } for (auto& it : clientVersions) { minVersion = std::min(minVersion, it.second); } feedReply.atLatestVersion = atLatest; - feedReply.minStreamVersion = gotAll ? minVersion : feedReply.mutations.back().version; + feedReply.minStreamVersion = minVersion; + req.reply.send(feedReply); if (feedReply.mutations.back().version == req.end - 1) { req.reply.sendError(end_of_stream()); From 632f701c8a9c25527e9b86c5cd4eefd02ff46750 Mon Sep 17 00:00:00 2001 From: Josh Slocum Date: Tue, 16 Nov 2021 12:44:35 -0600 Subject: [PATCH 025/413] First pass at using ChangeFeedData in blob worker --- fdbclient/NativeAPI.actor.cpp | 2 +- fdbserver/BlobWorker.actor.cpp | 109 ++++++++++++++++++--------------- 2 files changed, 60 insertions(+), 51 deletions(-) diff --git a/fdbclient/NativeAPI.actor.cpp b/fdbclient/NativeAPI.actor.cpp index b0bdf119cd..85a515f22e 100644 --- a/fdbclient/NativeAPI.actor.cpp +++ b/fdbclient/NativeAPI.actor.cpp @@ -6808,7 +6808,7 @@ Reference DatabaseContext::getStorageData(StorageServerIn } Version ChangeFeedData::getVersion() { - if (notAtLatest.get() == 0 && mutations.isEmpty()) { + if (notAtLatest.get() == 0 && mutations.isEmpty() && storageData.size() > 0) { Version v = storageData[0]->version.get(); for (int i = 1; i < storageData.size(); i++) { if (storageData[i]->version.get() < v) { diff --git a/fdbserver/BlobWorker.actor.cpp b/fdbserver/BlobWorker.actor.cpp index 6f48522556..7aaafa52cc 100644 --- a/fdbserver/BlobWorker.actor.cpp +++ b/fdbserver/BlobWorker.actor.cpp @@ -98,15 +98,13 @@ struct GranuleMetadata : NonCopyable, ReferenceCounted { uint64_t bufferedDeltaBytes = 0; // for client to know when it is safe to read a certain version and from where (check waitForVersion) - NotifiedVersion bufferedDeltaVersion; // largest delta version in currentDeltas (including empty versions) + Version bufferedDeltaVersion; // largest delta version in currentDeltas (including empty versions) Version pendingDeltaVersion = 0; // largest version in progress writing to s3/fdb NotifiedVersion durableDeltaVersion; // largest version persisted in s3/fdb NotifiedVersion durableSnapshotVersion; // same as delta vars, except for snapshots Version pendingSnapshotVersion = 0; Version initialSnapshotVersion = invalidVersion; - AsyncVar rollbackCount; - int64_t originalEpoch; int64_t originalSeqno; int64_t continueEpoch; @@ -118,6 +116,8 @@ struct GranuleMetadata : NonCopyable, ReferenceCounted { Promise resumeSnapshot; + AsyncVar> activeCFData; + AssignBlobRangeRequest originalReq; void resume() { @@ -1011,6 +1011,13 @@ struct InFlightFile { : future(future), version(version), bytes(bytes), snapshot(snapshot) {} }; +static Reference newChangeFeedData(Version startVersion) { + // FIXME: should changeFeedStream guarantee that this is always set to begin-1 instead? + Reference r = makeReference(); + r->lastReturnedVersion.set(startVersion); + return r; +} + static Version doGranuleRollback(Reference metadata, Version mutationVersion, Version rollbackVersion, @@ -1074,9 +1081,7 @@ static Version doGranuleRollback(Reference metadata, metadata->deltaArena = Arena(); metadata->currentDeltas = GranuleDeltas(); metadata->bufferedDeltaBytes = 0; - // Create new notified version so it doesn't go backwards. Do this before signaling the rollback counter so that - // the callers pick this new one up for the next waitForVersion - metadata->bufferedDeltaVersion = NotifiedVersion(cfRollbackVersion); + metadata->bufferedDeltaVersion = cfRollbackVersion; // Track that this rollback happened, since we have to re-read mutations up to the rollback // Add this rollback to in progress, and put all completed ones back in progress @@ -1115,9 +1120,7 @@ static Version doGranuleRollback(Reference metadata, // delete all deltas in rollback range, but we can optimize here to just skip the uncommitted mutations // directly and immediately pop the rollback out of inProgress - // Create new notified version so it doesn't go backwards. Do this before signaling the rollback counter so that - // the callers pick this new one up for the next waitForVersion - metadata->bufferedDeltaVersion = NotifiedVersion(rollbackVersion); + metadata->bufferedDeltaVersion = rollbackVersion; cfRollbackVersion = mutationVersion; } @@ -1128,8 +1131,6 @@ static Version doGranuleRollback(Reference metadata, cfRollbackVersion); } - metadata->rollbackCount.set(metadata->rollbackCount.get() + 1); - return cfRollbackVersion; } @@ -1139,8 +1140,6 @@ static Version doGranuleRollback(Reference metadata, ACTOR Future blobGranuleUpdateFiles(Reference bwData, Reference metadata, Future assignFuture) { - state Reference oldChangeFeedStream = makeReference(); - state Reference changeFeedStream = makeReference(); state std::deque inFlightFiles; state Future oldChangeFeedFuture; state Future changeFeedFuture; @@ -1237,26 +1236,29 @@ ACTOR Future blobGranuleUpdateFiles(Reference bwData, metadata->durableDeltaVersion.set(startVersion); metadata->pendingDeltaVersion = startVersion; - metadata->bufferedDeltaVersion.set(startVersion); + metadata->bufferedDeltaVersion = startVersion; committedVersion.set(startVersion); - ASSERT(metadata->readable.canBeSet()); - metadata->readable.send(Void()); - + metadata->activeCFData.set(newChangeFeedData(startVersion)); if (startState.parentGranule.present() && startVersion < startState.changeFeedStartVersion) { // read from parent change feed up until our new change feed is started readOldChangeFeed = true; - oldChangeFeedFuture = bwData->db->getChangeFeedStream(oldChangeFeedStream, + + oldChangeFeedFuture = bwData->db->getChangeFeedStream(metadata->activeCFData.get(), oldCFKey.get(), startVersion + 1, startState.changeFeedStartVersion, metadata->keyRange); + } else { readOldChangeFeed = false; changeFeedFuture = bwData->db->getChangeFeedStream( - changeFeedStream, cfKey, startVersion + 1, MAX_VERSION, metadata->keyRange); + metadata->activeCFData.get(), cfKey, startVersion + 1, MAX_VERSION, metadata->keyRange); } + ASSERT(metadata->readable.canBeSet()); + metadata->readable.send(Void()); + loop { // check outstanding snapshot/delta files for completion while (inFlightFiles.size() > 0) { @@ -1298,16 +1300,12 @@ ACTOR Future blobGranuleUpdateFiles(Reference bwData, state Standalone> mutations; try { + state Standalone> _mutations = + waitNext(metadata->activeCFData.get()->mutations.getFuture()); + mutations = _mutations; if (readOldChangeFeed) { - // TODO efficient way to store next in mutations? - Standalone> oldMutations = - waitNext(oldChangeFeedStream->mutations.getFuture()); - mutations = oldMutations; ASSERT(mutations.back().version < startState.changeFeedStartVersion); } else { - Standalone> newMutations = - waitNext(changeFeedStream->mutations.getFuture()); - mutations = newMutations; ASSERT(mutations.front().version >= startState.changeFeedStartVersion); } } catch (Error& e) { @@ -1326,11 +1324,15 @@ ACTOR Future blobGranuleUpdateFiles(Reference bwData, metadata->keyRange.begin.printable(), metadata->keyRange.end.printable(), startState.granuleID.toString(), - metadata->bufferedDeltaVersion.get()); + metadata->bufferedDeltaVersion); } - changeFeedFuture = bwData->db->getChangeFeedStream( - changeFeedStream, cfKey, startState.changeFeedStartVersion, MAX_VERSION, metadata->keyRange); + metadata->activeCFData.set(newChangeFeedData(startState.changeFeedStartVersion - 1)); + changeFeedFuture = bwData->db->getChangeFeedStream(metadata->activeCFData.get(), + cfKey, + startState.changeFeedStartVersion, + MAX_VERSION, + metadata->keyRange); } // process mutations @@ -1339,7 +1341,7 @@ ACTOR Future blobGranuleUpdateFiles(Reference bwData, // buffer mutations at this version. There should not be multiple MutationsAndVersionRef with the same // version - ASSERT(deltas.version > metadata->bufferedDeltaVersion.get()); + ASSERT(deltas.version > metadata->bufferedDeltaVersion); if (!deltas.mutations.empty()) { if (deltas.mutations.size() == 1 && deltas.mutations.back().param1 == lastEpochEndPrivateKey) { // Note rollbackVerision is durable, [rollbackVersion+1 - deltas.version] needs to be tossed @@ -1386,6 +1388,7 @@ ACTOR Future blobGranuleUpdateFiles(Reference bwData, .detail("Version", deltas.version) .detail("RollbackVersion", rollbackVersion); } + Version cfRollbackVersion = doGranuleRollback(metadata, deltas.version, rollbackVersion, @@ -1394,27 +1397,29 @@ ACTOR Future blobGranuleUpdateFiles(Reference bwData, rollbacksCompleted); // Reset change feeds to cfRollbackVersion + metadata->activeCFData.set(newChangeFeedData(cfRollbackVersion)); if (readOldChangeFeed) { // It shouldn't be possible to roll back across the parent/child feed boundary, // because the transaction creating the child change feed had to commit before we // got here. ASSERT(cfRollbackVersion < startState.changeFeedStartVersion); - oldChangeFeedStream = makeReference(); oldChangeFeedFuture = - bwData->db->getChangeFeedStream(oldChangeFeedStream, + bwData->db->getChangeFeedStream(metadata->activeCFData.get(), oldCFKey.get(), cfRollbackVersion + 1, startState.changeFeedStartVersion, metadata->keyRange); + } else { ASSERT(cfRollbackVersion > startState.changeFeedStartVersion); - changeFeedStream = makeReference(); - changeFeedFuture = bwData->db->getChangeFeedStream(changeFeedStream, + + changeFeedFuture = bwData->db->getChangeFeedStream(metadata->activeCFData.get(), cfKey, cfRollbackVersion + 1, MAX_VERSION, metadata->keyRange); } + justDidRollback = true; break; } @@ -1443,7 +1448,7 @@ ACTOR Future blobGranuleUpdateFiles(Reference bwData, } // update buffered version and committed version - metadata->bufferedDeltaVersion.set(deltas.version); + metadata->bufferedDeltaVersion = deltas.version; Version knownNoRollbacksPast = std::min(deltas.version, deltas.knownCommittedVersion); if (knownNoRollbacksPast > committedVersion.get()) { // This is the only place it is safe to set committedVersion, as it has to come from the mutation @@ -1622,6 +1627,9 @@ ACTOR Future blobGranuleUpdateFiles(Reference bwData, if (BW_DEBUG) { printf("BGUF got error %s\n", e.name()); } + // Free last change feed data + metadata->activeCFData.set(Reference()); + if (e.code() == error_code_operation_cancelled) { throw; } @@ -1786,13 +1794,15 @@ ACTOR Future waitForVersion(Reference metadata, Version v metadata->keyRange.end.printable().c_str(), v, metadata->readable.isSet() ? "T" : "F", - metadata->bufferedDeltaVersion.get(), + metadata->activeCFData.get()->getVersion(), metadata->pendingDeltaVersion, metadata->durableDeltaVersion.get(), metadata->pendingSnapshotVersion, metadata->durableSnapshotVersion.get());*/ - if (v <= metadata->bufferedDeltaVersion.get() && + ASSERT(metadata->activeCFData.get().isValid()); + + if (v <= metadata->activeCFData.get()->getVersion() && (v <= metadata->durableDeltaVersion.get() || metadata->durableDeltaVersion.get() == metadata->pendingDeltaVersion) && (v <= metadata->durableSnapshotVersion.get() || @@ -1801,15 +1811,15 @@ ACTOR Future waitForVersion(Reference metadata, Version v } // wait for change feed version to catch up to ensure we have all data - if (metadata->bufferedDeltaVersion.get() < v) { - wait(metadata->bufferedDeltaVersion.whenAtLeast(v)); + if (metadata->activeCFData.get()->getVersion() < v) { + wait(metadata->activeCFData.get()->whenAtLeast(v)); } // wait for any pending delta and snapshot files as of the moment the change feed version caught up. state Version pendingDeltaV = metadata->pendingDeltaVersion; state Version pendingSnapshotV = metadata->pendingSnapshotVersion; - ASSERT(pendingDeltaV <= metadata->bufferedDeltaVersion.get()); + // ASSERT(pendingDeltaV <= metadata->activeCFData.get()->getVersion()); if (pendingDeltaV > metadata->durableDeltaVersion.get()) { wait(metadata->durableDeltaVersion.whenAtLeast(pendingDeltaV)); } @@ -1971,6 +1981,9 @@ ACTOR Future handleBlobGranuleFileRequest(Reference bwData } else { // this is an active granule query loop { + if (!metadata->activeCFData.get().isValid()) { + throw wrong_shard_server(); + } Future waitForVersionFuture = waitForVersion(metadata, req.readVersion); if (waitForVersionFuture.isReady()) { // didn't wait, so no need to check rollback stuff @@ -1978,19 +1991,15 @@ ACTOR Future handleBlobGranuleFileRequest(Reference bwData } // rollback resets all of the version information, so we have to redo wait for // version on rollback - state int rollbackCount = metadata->rollbackCount.get(); choose { - when(wait(waitForVersionFuture)) {} - when(wait(metadata->rollbackCount.onChange())) {} + when(wait(waitForVersionFuture)) { break; } + when(wait(metadata->activeCFData.onChange())) {} when(wait(metadata->cancelled.getFuture())) { throw wrong_shard_server(); } } - - if (rollbackCount == metadata->rollbackCount.get()) { - break; - } else if (BW_REQUEST_DEBUG) { - fmt::print("[{0} - {1}) @ {2} hit rollback, restarting waitForVersion\n", - req.keyRange.begin.printable(), - req.keyRange.end.printable(), + if (BW_REQUEST_DEBUG && metadata->activeCFData.get().isValid()) { + fmt::print("{0} - {1}) @ {2} hit CF change, restarting waitForVersion\n", + req.keyRange.begin.printable().c_str(), + req.keyRange.end.printable().c_str(), req.readVersion); } } From f43169cb7bb9da9c851d85343c1c5fc44b1c7367 Mon Sep 17 00:00:00 2001 From: Josh Slocum Date: Thu, 2 Dec 2021 12:40:37 -0600 Subject: [PATCH 026/413] bug fixes --- fdbclient/NativeAPI.actor.cpp | 3 ++- fdbserver/BlobWorker.actor.cpp | 6 ++++-- 2 files changed, 6 insertions(+), 3 deletions(-) diff --git a/fdbclient/NativeAPI.actor.cpp b/fdbclient/NativeAPI.actor.cpp index 85a515f22e..6b985714da 100644 --- a/fdbclient/NativeAPI.actor.cpp +++ b/fdbclient/NativeAPI.actor.cpp @@ -6808,7 +6808,8 @@ Reference DatabaseContext::getStorageData(StorageServerIn } Version ChangeFeedData::getVersion() { - if (notAtLatest.get() == 0 && mutations.isEmpty() && storageData.size() > 0) { + // TODO uncomment? + if (notAtLatest.get() == 0 && mutations.isEmpty() /*& storageData.size() > 0*/) { Version v = storageData[0]->version.get(); for (int i = 1; i < storageData.size(); i++) { if (storageData[i]->version.get() < v) { diff --git a/fdbserver/BlobWorker.actor.cpp b/fdbserver/BlobWorker.actor.cpp index 7aaafa52cc..e63cc3c2dc 100644 --- a/fdbserver/BlobWorker.actor.cpp +++ b/fdbserver/BlobWorker.actor.cpp @@ -1014,7 +1014,8 @@ struct InFlightFile { static Reference newChangeFeedData(Version startVersion) { // FIXME: should changeFeedStream guarantee that this is always set to begin-1 instead? Reference r = makeReference(); - r->lastReturnedVersion.set(startVersion); + // TODO uncomment? + // r->lastReturnedVersion.set(startVersion); return r; } @@ -1949,7 +1950,7 @@ ACTOR Future handleBlobGranuleFileRequest(Reference bwData // lazily load files for old granule if not present chunkRange = cur->range; - if (cur->files.isError() || !cur->files.isValid()) { + if (!cur->files.isValid() || cur->files.isError()) { cur->files = loadHistoryFiles(bwData, cur->granuleID); } @@ -2100,6 +2101,7 @@ ACTOR Future handleBlobGranuleFileRequest(Reference bwData req.reply.send(rep); --bwData->stats.activeReadRequests; } catch (Error& e) { + printf("Error in BGFRequest %s\n", e.name()); if (e.code() == error_code_operation_cancelled) { req.reply.sendError(wrong_shard_server()); throw; From 9f6c8a123b06b1e6a508ddbad826424df574e27a Mon Sep 17 00:00:00 2001 From: Josh Slocum Date: Thu, 2 Dec 2021 13:29:30 -0600 Subject: [PATCH 027/413] Fix: change feed update ordering --- fdbserver/storageserver.actor.cpp | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/fdbserver/storageserver.actor.cpp b/fdbserver/storageserver.actor.cpp index cc48631ead..1b79cc8265 100644 --- a/fdbserver/storageserver.actor.cpp +++ b/fdbserver/storageserver.actor.cpp @@ -1744,6 +1744,10 @@ ACTOR Future> getChangeFeedMutations(Stor if (data->version.get() < req.begin) { wait(data->version.whenAtLeast(req.begin)); + // we must delay here to ensure that any up-to-date change feeds that are waiting on the + // mutation trigger run BEFORE any blocked change feeds run, in order to preserve the + // correct minStreamVersion ordering + wait(delay(0.0)); } state uint64_t changeCounter = data->shardChangeCounter; if (!inverted && !data->isReadable(req.range)) { From d85eb330e0400a8b5ee5854d1ba89f62ae68422d Mon Sep 17 00:00:00 2001 From: Josh Slocum Date: Thu, 2 Dec 2021 14:52:16 -0600 Subject: [PATCH 028/413] retooling some waitForVersion stuff and adding asserts --- fdbclient/NativeAPI.actor.cpp | 12 +++++++----- fdbserver/BlobWorker.actor.cpp | 17 +++++++++++------ fdbserver/storageserver.actor.cpp | 2 ++ 3 files changed, 20 insertions(+), 11 deletions(-) diff --git a/fdbclient/NativeAPI.actor.cpp b/fdbclient/NativeAPI.actor.cpp index 6b985714da..b27a7650ca 100644 --- a/fdbclient/NativeAPI.actor.cpp +++ b/fdbclient/NativeAPI.actor.cpp @@ -6835,7 +6835,7 @@ ACTOR Future changeFeedWhenAtLatest(ChangeFeedData* self, Version version) } } choose { - when(wait(lastReturned)) { return Void(); } + when(wait(lastReturned)) { break; } when(wait(waitForAll(allAtLeast))) { std::vector> onEmpty; if (!self->mutations.isEmpty()) { @@ -6847,14 +6847,14 @@ ACTOR Future changeFeedWhenAtLatest(ChangeFeedData* self, Version version) } } if (!onEmpty.size()) { - return Void(); + break; } choose { when(wait(waitForAll(onEmpty))) { wait(delay(0)); - return Void(); + break; } - when(wait(lastReturned)) { return Void(); } + when(wait(lastReturned)) { break; } when(wait(self->refresh.getFuture())) {} when(wait(self->notAtLatest.onChange())) {} } @@ -6864,12 +6864,14 @@ ACTOR Future changeFeedWhenAtLatest(ChangeFeedData* self, Version version) } } else { choose { - when(wait(lastReturned)) { return Void(); } + when(wait(lastReturned)) { break; } when(wait(self->notAtLatest.onChange())) {} when(wait(self->refresh.getFuture())) {} } } } + ASSERT(self->getVersion() >= version); + return Void(); } Future ChangeFeedData::whenAtLeast(Version version) { diff --git a/fdbserver/BlobWorker.actor.cpp b/fdbserver/BlobWorker.actor.cpp index e63cc3c2dc..2a450bf5d7 100644 --- a/fdbserver/BlobWorker.actor.cpp +++ b/fdbserver/BlobWorker.actor.cpp @@ -1814,29 +1814,34 @@ ACTOR Future waitForVersion(Reference metadata, Version v // wait for change feed version to catch up to ensure we have all data if (metadata->activeCFData.get()->getVersion() < v) { wait(metadata->activeCFData.get()->whenAtLeast(v)); + ASSERT(metadata->activeCFData.get()->getVersion() >= v); } // wait for any pending delta and snapshot files as of the moment the change feed version caught up. state Version pendingDeltaV = metadata->pendingDeltaVersion; state Version pendingSnapshotV = metadata->pendingSnapshotVersion; - // ASSERT(pendingDeltaV <= metadata->activeCFData.get()->getVersion()); - if (pendingDeltaV > metadata->durableDeltaVersion.get()) { + // If there are mutations that are no longer buffered but have not been + // persisted to a delta file that are necessary for the query, wait for them + if (pendingDeltaV > metadata->durableDeltaVersion.get() && v > metadata->durableDeltaVersion.get()) { wait(metadata->durableDeltaVersion.whenAtLeast(pendingDeltaV)); + ASSERT(metadata->durableDeltaVersion.get() >= pendingDeltaV); } // This isn't strictly needed, but if we're in the process of re-snapshotting, we'd likely rather // return that snapshot file than the previous snapshot file and all its delta files. - if (pendingSnapshotV > metadata->durableSnapshotVersion.get()) { + if (pendingSnapshotV > metadata->durableSnapshotVersion.get() && v > metadata->durableSnapshotVersion.get()) { wait(metadata->durableSnapshotVersion.whenAtLeast(pendingSnapshotV)); + ASSERT(metadata->durableSnapshotVersion.get() >= pendingSnapshotV); } // There is a race here - we wait for pending delta files before this to finish, but while we do, we // kick off another delta file and roll the mutations. In that case, we must return the new delta // file instead of in memory mutations, so we wait for that delta file to complete - if (metadata->pendingDeltaVersion != pendingDeltaV) { - wait(metadata->durableDeltaVersion.whenAtLeast(pendingDeltaV + 1)); + if (metadata->pendingDeltaVersion > v) { + wait(metadata->durableDeltaVersion.whenAtLeast(v)); + ASSERT(metadata->durableDeltaVersion.get() >= v); } return Void(); @@ -2101,7 +2106,7 @@ ACTOR Future handleBlobGranuleFileRequest(Reference bwData req.reply.send(rep); --bwData->stats.activeReadRequests; } catch (Error& e) { - printf("Error in BGFRequest %s\n", e.name()); + // printf("Error in BGFRequest %s\n", e.name()); if (e.code() == error_code_operation_cancelled) { req.reply.sendError(wrong_shard_server()); throw; diff --git a/fdbserver/storageserver.actor.cpp b/fdbserver/storageserver.actor.cpp index cc48631ead..f1bcf67927 100644 --- a/fdbserver/storageserver.actor.cpp +++ b/fdbserver/storageserver.actor.cpp @@ -4650,6 +4650,7 @@ void changeServerKeys(StorageServer* data, } } } + data->keyChangeFeed.coalesce(f.second.contents()); auto feed = data->uidChangeFeed.find(f.first); if (feed != data->uidChangeFeed.end()) { feed->second->removing = true; @@ -4905,6 +4906,7 @@ private: } } } + data->keyChangeFeed.coalesce(feed->second->range.contents()); data->uidChangeFeed.erase(feed); } else { // must be pop or stop From 0c46b8960100b8e6326ccc7b61f651a7ad7f64c1 Mon Sep 17 00:00:00 2001 From: Josh Slocum Date: Thu, 2 Dec 2021 14:58:26 -0600 Subject: [PATCH 029/413] Adding new BG tests --- tests/fast/BlobGranuleCycle.toml | 35 +++++++++++++++++++++++++++ tests/fast/BlobGranuleCycleClean.toml | 12 +++++++++ 2 files changed, 47 insertions(+) create mode 100644 tests/fast/BlobGranuleCycle.toml create mode 100644 tests/fast/BlobGranuleCycleClean.toml diff --git a/tests/fast/BlobGranuleCycle.toml b/tests/fast/BlobGranuleCycle.toml new file mode 100644 index 0000000000..8f8fe9f030 --- /dev/null +++ b/tests/fast/BlobGranuleCycle.toml @@ -0,0 +1,35 @@ +[[test]] +testTitle = 'BlobGranuleCycle' + + [[test.workload]] + testName = 'Cycle' + transactionsPerSecond = 250.0 + testDuration = 60.0 + expectedRate = 0 + + [[test.workload]] + testName = 'BlobGranuleVerifier' + testDuration = 60.0 + + [[test.workload]] + testName = 'RandomClogging' + testDuration = 60.0 + + [[test.workload]] + testName = 'Rollback' + meanDelay = 60.0 + testDuration = 60.0 + + [[test.workload]] + testName = 'Attrition' + machinesToKill = 10 + machinesToLeave = 3 + reboot = true + testDuration = 60.0 + + [[test.workload]] + testName = 'Attrition' + machinesToKill = 10 + machinesToLeave = 3 + reboot = true + testDuration = 60.0 diff --git a/tests/fast/BlobGranuleCycleClean.toml b/tests/fast/BlobGranuleCycleClean.toml new file mode 100644 index 0000000000..57e4676c9d --- /dev/null +++ b/tests/fast/BlobGranuleCycleClean.toml @@ -0,0 +1,12 @@ +[[test]] +testTitle = 'BlobGranuleCycleClean' + + [[test.workload]] + testName = 'Cycle' + transactionsPerSecond = 250.0 + testDuration = 60.0 + expectedRate = 0 + + [[test.workload]] + testName = 'BlobGranuleVerifier' + testDuration = 60.0 From 6db49e87d33015800530c69261622689f850279f Mon Sep 17 00:00:00 2001 From: Josh Slocum Date: Thu, 2 Dec 2021 15:25:48 -0600 Subject: [PATCH 030/413] Enabling new BG tests --- tests/CMakeLists.txt | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/tests/CMakeLists.txt b/tests/CMakeLists.txt index df50bd8219..6a62ff64ed 100644 --- a/tests/CMakeLists.txt +++ b/tests/CMakeLists.txt @@ -127,6 +127,10 @@ if(WITH_PYTHON) add_fdb_test(TEST_FILES fast/BackupCorrectnessClean.toml) add_fdb_test(TEST_FILES fast/BackupToDBCorrectness.toml) add_fdb_test(TEST_FILES fast/BackupToDBCorrectnessClean.toml) + add_fdb_test(TEST_FILES fast/BlobGranuleCorrectness.toml) + add_fdb_test(TEST_FILES fast/BlobGranuleCorrectnessClean.toml) + add_fdb_test(TEST_FILES fast/BlobGranuleCycle.toml) + add_fdb_test(TEST_FILES fast/BlobGranuleCycleClean.toml) add_fdb_test(TEST_FILES fast/CacheTest.toml) add_fdb_test(TEST_FILES fast/CloggedSideband.toml) add_fdb_test(TEST_FILES fast/ConfigureLocked.toml) @@ -259,9 +263,7 @@ if(WITH_PYTHON) add_fdb_test(TEST_FILES slow/ApiCorrectness.toml) add_fdb_test(TEST_FILES slow/ApiCorrectnessAtomicRestore.toml) add_fdb_test(TEST_FILES slow/ApiCorrectnessSwitchover.toml) - add_fdb_test(TEST_FILES fast/BlobGranuleCorrectness.toml) add_fdb_test(TEST_FILES slow/BlobGranuleCorrectnessLarge.toml) - add_fdb_test(TEST_FILES fast/BlobGranuleCorrectnessClean.toml) add_fdb_test(TEST_FILES slow/BlobGranuleCorrectnessLargeClean.toml) add_fdb_test(TEST_FILES slow/ClogWithRollbacks.toml) add_fdb_test(TEST_FILES slow/CloggedCycleTest.toml) From 35781924afa33223e26d8dde9ce644c4a33044d1 Mon Sep 17 00:00:00 2001 From: Josh Slocum Date: Thu, 2 Dec 2021 15:53:31 -0600 Subject: [PATCH 031/413] merge conflicts --- tests/CMakeLists.txt | 5 ----- 1 file changed, 5 deletions(-) diff --git a/tests/CMakeLists.txt b/tests/CMakeLists.txt index 6f0d7da32a..e3768b76b4 100644 --- a/tests/CMakeLists.txt +++ b/tests/CMakeLists.txt @@ -251,13 +251,8 @@ if(WITH_PYTHON) add_fdb_test(TEST_FILES slow/ApiCorrectness.toml) add_fdb_test(TEST_FILES slow/ApiCorrectnessAtomicRestore.toml) add_fdb_test(TEST_FILES slow/ApiCorrectnessSwitchover.toml) -<<<<<<< HEAD add_fdb_test(TEST_FILES slow/BlobGranuleCorrectnessLarge.toml) add_fdb_test(TEST_FILES slow/BlobGranuleCorrectnessLargeClean.toml) -======= - add_fdb_test(TEST_FILES slow/BlobGranuleCorrectnessLarge.toml IGNORE) - add_fdb_test(TEST_FILES slow/BlobGranuleCorrectnessLargeClean.toml IGNORE) ->>>>>>> master add_fdb_test(TEST_FILES slow/ClogWithRollbacks.toml) add_fdb_test(TEST_FILES slow/CloggedCycleTest.toml) add_fdb_test(TEST_FILES slow/CloggedStorefront.toml) From 2d7c44af31685e224a06a5862c5283a4b061577d Mon Sep 17 00:00:00 2001 From: Josh Slocum Date: Thu, 2 Dec 2021 16:42:14 -0600 Subject: [PATCH 032/413] Fixing bug with readVersionOut being null --- fdbclient/NativeAPI.actor.cpp | 30 +++++++++++++++++------------- 1 file changed, 17 insertions(+), 13 deletions(-) diff --git a/fdbclient/NativeAPI.actor.cpp b/fdbclient/NativeAPI.actor.cpp index 1ec61fc51b..ad6a76e852 100644 --- a/fdbclient/NativeAPI.actor.cpp +++ b/fdbclient/NativeAPI.actor.cpp @@ -6562,14 +6562,15 @@ ACTOR Future>> readBlobGranulesActor( state KeyRange keyRange = range; state UID workerId; state int i; + state Version rv; state Standalone> results; if (read.present()) { - *readVersionOut = read.get(); + rv = read.get(); } else { Version _end = wait(self->getReadVersion()); - *readVersionOut = _end; + rv = _end; } self->setOption(FDBTransactionOptions::ACCESS_SYSTEM_KEYS); @@ -6597,7 +6598,7 @@ ACTOR Future>> readBlobGranulesActor( } if (BG_REQUEST_DEBUG) { - fmt::print("Doing blob granule request @ {}\n", *readVersionOut); + fmt::print("Doing blob granule request @ {}\n", rv); fmt::print("blob worker assignments:\n"); } @@ -6658,7 +6659,7 @@ ACTOR Future>> readBlobGranulesActor( state BlobGranuleFileRequest req; req.keyRange = KeyRangeRef(StringRef(req.arena, granuleStartKey), StringRef(req.arena, granuleEndKey)); req.beginVersion = begin; - req.readVersion = *readVersionOut; + req.readVersion = rv; std::vector>> v; v.push_back(makeReference>(cx->blobWorker_interf[workerId])); @@ -6677,7 +6678,7 @@ ACTOR Future>> readBlobGranulesActor( granuleStartKey.printable(), granuleEndKey.printable(), begin, - *readVersionOut, + rv, workerId.toString()); } results.arena().dependsOn(rep.arena); @@ -6703,6 +6704,9 @@ ACTOR Future>> readBlobGranulesActor( keyRange = KeyRangeRef(std::min(chunk.keyRange.end, keyRange.end), keyRange.end); } } + if (readVersionOut != nullptr) { + *readVersionOut = rv; + } return results; } @@ -6769,8 +6773,8 @@ ACTOR Future>> splitStorageMetrics(Database cx, 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 + 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(); @@ -7147,12 +7151,12 @@ ACTOR Future singleChangeFeedStream(StorageServerInterface interf, } else { // TODO REMOVE eventually, useful for debugging for now if (!rep.mutations[resultLoc].mutations.empty()) { - printf( - "non-empty mutations (%d), but versions out of order from %s! mv=%lld, nv=%lld\n", - rep.mutations.size(), - interf.id().toString().substr(0, 4).c_str(), - rep.mutations[resultLoc].version, - nextVersion); + printf("non-empty mutations (%d), but versions out of order from %s! mv=%lld, " + "nv=%lld\n", + rep.mutations.size(), + interf.id().toString().substr(0, 4).c_str(), + rep.mutations[resultLoc].version, + nextVersion); } ASSERT(rep.mutations[resultLoc].mutations.empty()); } From f2838740f1c0a2f935a43b5d65305fa3b575bff0 Mon Sep 17 00:00:00 2001 From: Evan Tschannen Date: Fri, 3 Dec 2021 10:29:22 -0800 Subject: [PATCH 033/413] fix: do not allow more than one blob worker per address --- fdbserver/BlobManager.actor.cpp | 46 +++++++++++++++++++++++++++------ fdbserver/worker.actor.cpp | 17 +++++++----- 2 files changed, 49 insertions(+), 14 deletions(-) diff --git a/fdbserver/BlobManager.actor.cpp b/fdbserver/BlobManager.actor.cpp index 8949b6818c..5bea27cbf1 100644 --- a/fdbserver/BlobManager.actor.cpp +++ b/fdbserver/BlobManager.actor.cpp @@ -204,6 +204,7 @@ struct BlobManagerData { std::unordered_map workersById; std::unordered_map workerStats; // mapping between workerID -> workerStats + std::unordered_set workerAddresses; KeyRangeMap workerAssignments; KeyRangeMap knownBlobRanges; @@ -771,15 +772,19 @@ ACTOR Future deregisterBlobWorker(BlobManagerData* bmData, BlobWorkerInter } } -ACTOR Future killBlobWorker(BlobManagerData* bmData, BlobWorkerInterface bwInterf) { +ACTOR Future killBlobWorker(BlobManagerData* bmData, BlobWorkerInterface bwInterf, bool registered) { UID bwId = bwInterf.id(); // Remove blob worker from stats map so that when we try to find a worker to takeover the range, // the one we just killed isn't considered. // Remove it from workersById also since otherwise that worker addr will remain excluded // when we try to recruit new blob workers. - bmData->workerStats.erase(bwId); - bmData->workersById.erase(bwId); + + if (registered) { + bmData->workerStats.erase(bwId); + bmData->workersById.erase(bwId); + bmData->workerAddresses.erase(bwInterf.stableAddress()); + } // Remove blob worker from persisted list of blob workers Future deregister = deregisterBlobWorker(bmData, bwInterf); @@ -961,7 +966,7 @@ ACTOR Future monitorBlobWorker(BlobManagerData* bmData, BlobWorkerInterfac } // kill the blob worker - wait(killBlobWorker(bmData, bwInterf)); + wait(killBlobWorker(bmData, bwInterf, true)); if (BM_DEBUG) { printf("No longer monitoring BW %s\n", bwInterf.id().toString().c_str()); @@ -969,6 +974,23 @@ ACTOR Future monitorBlobWorker(BlobManagerData* bmData, BlobWorkerInterfac return Void(); } +ACTOR Future checkBlobWorkerList(BlobManagerData* bmData) { + loop { + wait(delay(1.0)); + std::vector blobWorkers = wait(getBlobWorkers(bmData->db)); + for (auto& worker : blobWorkers) { + if (!bmData->workerAddresses.count(worker.stableAddress())) { + bmData->workerAddresses.insert(worker.stableAddress()); + bmData->workersById[worker.id()] = worker; + bmData->workerStats[worker.id()] = BlobWorkerStats(); + bmData->addActor.send(monitorBlobWorker(bmData, worker)); + } else if (!bmData->workersById.count(worker.id())) { + bmData->addActor.send(killBlobWorker(bmData, worker, false)); + } + } + } +} + ACTOR Future recoverBlobManager(BlobManagerData* bmData) { // skip this entire algorithm for the first blob manager if (bmData->epoch == 1) { @@ -982,12 +1004,19 @@ ACTOR Future recoverBlobManager(BlobManagerData* bmData) { std::vector blobWorkers = wait(getBlobWorkers(bmData->db)); // add all blob workers to this new blob manager's records and start monitoring it - for (auto worker : blobWorkers) { - bmData->workersById[worker.id()] = worker; - bmData->workerStats[worker.id()] = BlobWorkerStats(); - bmData->addActor.send(monitorBlobWorker(bmData, worker)); + for (auto& worker : blobWorkers) { + if (!bmData->workerAddresses.count(worker.stableAddress())) { + bmData->workerAddresses.insert(worker.stableAddress()); + bmData->workersById[worker.id()] = worker; + bmData->workerStats[worker.id()] = BlobWorkerStats(); + bmData->addActor.send(monitorBlobWorker(bmData, worker)); + } else if (!bmData->workersById.count(worker.id())) { + bmData->addActor.send(killBlobWorker(bmData, worker, false)); + } } + bmData->addActor.send(checkBlobWorkerList(bmData)); + // Once we acknowledge the existing blob workers, we can go ahead and recruit new ones bmData->startRecruiting.trigger(); @@ -1229,6 +1258,7 @@ ACTOR Future initializeBlobWorker(BlobManagerData* self, RecruitBlobWorker if (newBlobWorker.present()) { BlobWorkerInterface bwi = newBlobWorker.get().interf; + self->workerAddresses.insert(bwi.stableAddress()); self->workersById[bwi.id()] = bwi; self->workerStats[bwi.id()] = BlobWorkerStats(); self->addActor.send(monitorBlobWorker(self, bwi)); diff --git a/fdbserver/worker.actor.cpp b/fdbserver/worker.actor.cpp index 7d2e23c306..0fb24463f6 100644 --- a/fdbserver/worker.actor.cpp +++ b/fdbserver/worker.actor.cpp @@ -1394,6 +1394,7 @@ ACTOR Future workerServer(Reference connRecord, state std::map sharedLogs; state Reference> activeSharedTLog(new AsyncVar()); state WorkerCache backupWorkerCache; + state WorkerCache blobWorkerCache; state std::string coordFolder = abspath(_coordFolder); @@ -2036,13 +2037,17 @@ ACTOR Future workerServer(Reference connRecord, } } when(InitializeBlobWorkerRequest req = waitNext(interf.blobWorker.getFuture())) { - BlobWorkerInterface recruited(locality, req.interfaceId); - recruited.initEndpoints(); - startRole(Role::BLOB_WORKER, recruited.id(), interf.id()); + if (!blobWorkerCache.exists(req.reqId)) { + BlobWorkerInterface recruited(locality, req.interfaceId); + recruited.initEndpoints(); + startRole(Role::BLOB_WORKER, recruited.id(), interf.id()); - ReplyPromise blobWorkerReady = req.reply; - Future bw = blobWorker(recruited, blobWorkerReady, dbInfo); - errorForwarders.add(forwardError(errors, Role::BLOB_WORKER, recruited.id(), bw)); + ReplyPromise blobWorkerReady = req.reply; + Future bw = blobWorker(recruited, blobWorkerReady, dbInfo); + errorForwarders.add(forwardError(errors, Role::BLOB_WORKER, recruited.id(), bw)); + } else { + forwardPromise(req.reply, blobWorkerCache.get(req.reqId)); + } } when(InitializeCommitProxyRequest req = waitNext(interf.commitProxy.getFuture())) { LocalLineage _; From 243927c9648c528f8e0d6cab20cab35e65aa6b60 Mon Sep 17 00:00:00 2001 From: Evan Tschannen Date: Fri, 3 Dec 2021 10:31:51 -0800 Subject: [PATCH 034/413] added a knob --- fdbclient/ServerKnobs.cpp | 1 + fdbclient/ServerKnobs.h | 1 + fdbserver/BlobManager.actor.cpp | 2 +- 3 files changed, 3 insertions(+), 1 deletion(-) diff --git a/fdbclient/ServerKnobs.cpp b/fdbclient/ServerKnobs.cpp index b11bd8e88c..38b1bbb18c 100644 --- a/fdbclient/ServerKnobs.cpp +++ b/fdbclient/ServerKnobs.cpp @@ -787,6 +787,7 @@ void ServerKnobs::initialize(Randomize randomize, ClientKnobs* clientKnobs, IsSi init( BG_DELTA_FILE_TARGET_BYTES, BG_DELTA_BYTES_BEFORE_COMPACT/10 ); init( BLOB_WORKER_TIMEOUT, 10.0 ); if( randomize && BUGGIFY ) BLOB_WORKER_TIMEOUT = 1.0; + init( BLOB_WORKERLIST_FETCH_INTERVAL, 1.0 ); // clang-format on diff --git a/fdbclient/ServerKnobs.h b/fdbclient/ServerKnobs.h index 050804a9a6..68f4456e1f 100644 --- a/fdbclient/ServerKnobs.h +++ b/fdbclient/ServerKnobs.h @@ -742,6 +742,7 @@ public: int BG_DELTA_BYTES_BEFORE_COMPACT; double BLOB_WORKER_TIMEOUT; // Blob Manager's reaction time to a blob worker failure + double BLOB_WORKERLIST_FETCH_INTERVAL; ServerKnobs(Randomize, ClientKnobs*, IsSimulated); void initialize(Randomize, ClientKnobs*, IsSimulated); diff --git a/fdbserver/BlobManager.actor.cpp b/fdbserver/BlobManager.actor.cpp index 5bea27cbf1..c6f210d88e 100644 --- a/fdbserver/BlobManager.actor.cpp +++ b/fdbserver/BlobManager.actor.cpp @@ -976,7 +976,7 @@ ACTOR Future monitorBlobWorker(BlobManagerData* bmData, BlobWorkerInterfac ACTOR Future checkBlobWorkerList(BlobManagerData* bmData) { loop { - wait(delay(1.0)); + wait(delay(SERVER_KNOBS->BLOB_WORKERLIST_FETCH_INTERVAL)); std::vector blobWorkers = wait(getBlobWorkers(bmData->db)); for (auto& worker : blobWorkers) { if (!bmData->workerAddresses.count(worker.stableAddress())) { From 935ec25ae3498c14cd4140051afb7f8f53ba8e09 Mon Sep 17 00:00:00 2001 From: Evan Tschannen Date: Fri, 3 Dec 2021 14:12:08 -0800 Subject: [PATCH 035/413] fix: do not re-add dead blob workers --- fdbserver/BlobManager.actor.cpp | 57 +++++++++++++++++---------------- 1 file changed, 29 insertions(+), 28 deletions(-) diff --git a/fdbserver/BlobManager.actor.cpp b/fdbserver/BlobManager.actor.cpp index c6f210d88e..841d3293c1 100644 --- a/fdbserver/BlobManager.actor.cpp +++ b/fdbserver/BlobManager.actor.cpp @@ -205,6 +205,7 @@ struct BlobManagerData { std::unordered_map workersById; std::unordered_map workerStats; // mapping between workerID -> workerStats std::unordered_set workerAddresses; + std::unordered_set deadWorkers; KeyRangeMap workerAssignments; KeyRangeMap knownBlobRanges; @@ -773,7 +774,7 @@ ACTOR Future deregisterBlobWorker(BlobManagerData* bmData, BlobWorkerInter } ACTOR Future killBlobWorker(BlobManagerData* bmData, BlobWorkerInterface bwInterf, bool registered) { - UID bwId = bwInterf.id(); + state UID bwId = bwInterf.id(); // Remove blob worker from stats map so that when we try to find a worker to takeover the range, // the one we just killed isn't considered. @@ -781,6 +782,7 @@ ACTOR Future killBlobWorker(BlobManagerData* bmData, BlobWorkerInterface b // when we try to recruit new blob workers. if (registered) { + bmData->deadWorkers.insert(bwId); bmData->workerStats.erase(bwId); bmData->workersById.erase(bwId); bmData->workerAddresses.erase(bwInterf.stableAddress()); @@ -825,6 +827,11 @@ ACTOR Future killBlobWorker(BlobManagerData* bmData, BlobWorkerInterface b brokenPromiseToNever(bwInterf.haltBlobWorker.getReply(HaltBlobWorkerRequest(bmData->epoch, bmData->id)))); wait(deregister); + + if (registered) { + bmData->deadWorkers.erase(bwId); + } + return Void(); } @@ -974,20 +981,29 @@ ACTOR Future monitorBlobWorker(BlobManagerData* bmData, BlobWorkerInterfac return Void(); } -ACTOR Future checkBlobWorkerList(BlobManagerData* bmData) { +ACTOR Future checkBlobWorkerList(BlobManagerData* bmData, Promise workerListReady) { loop { - wait(delay(SERVER_KNOBS->BLOB_WORKERLIST_FETCH_INTERVAL)); + // Get list of last known blob workers + // note: the list will include every blob worker that the old manager knew about, + // but it might also contain blob workers that died while the new manager was being recruited std::vector blobWorkers = wait(getBlobWorkers(bmData->db)); + // add all blob workers to this new blob manager's records and start monitoring it for (auto& worker : blobWorkers) { - if (!bmData->workerAddresses.count(worker.stableAddress())) { - bmData->workerAddresses.insert(worker.stableAddress()); - bmData->workersById[worker.id()] = worker; - bmData->workerStats[worker.id()] = BlobWorkerStats(); - bmData->addActor.send(monitorBlobWorker(bmData, worker)); - } else if (!bmData->workersById.count(worker.id())) { - bmData->addActor.send(killBlobWorker(bmData, worker, false)); + if (!bmData->deadWorkers.count(worker.id())) { + if (!bmData->workerAddresses.count(worker.stableAddress())) { + bmData->workerAddresses.insert(worker.stableAddress()); + bmData->workersById[worker.id()] = worker; + bmData->workerStats[worker.id()] = BlobWorkerStats(); + bmData->addActor.send(monitorBlobWorker(bmData, worker)); + } else if (!bmData->workersById.count(worker.id())) { + bmData->addActor.send(killBlobWorker(bmData, worker, false)); + } } } + if (workerListReady.canBeSet()) { + workerListReady.send(Void()); + } + wait(delay(SERVER_KNOBS->BLOB_WORKERLIST_FETCH_INTERVAL)); } } @@ -998,24 +1014,9 @@ ACTOR Future recoverBlobManager(BlobManagerData* bmData) { return Void(); } - // Get list of last known blob workers - // note: the list will include every blob worker that the old manager knew about, - // but it might also contain blob workers that died while the new manager was being recruited - std::vector blobWorkers = wait(getBlobWorkers(bmData->db)); - - // add all blob workers to this new blob manager's records and start monitoring it - for (auto& worker : blobWorkers) { - if (!bmData->workerAddresses.count(worker.stableAddress())) { - bmData->workerAddresses.insert(worker.stableAddress()); - bmData->workersById[worker.id()] = worker; - bmData->workerStats[worker.id()] = BlobWorkerStats(); - bmData->addActor.send(monitorBlobWorker(bmData, worker)); - } else if (!bmData->workersById.count(worker.id())) { - bmData->addActor.send(killBlobWorker(bmData, worker, false)); - } - } - - bmData->addActor.send(checkBlobWorkerList(bmData)); + state Promise workerListReady; + bmData->addActor.send(checkBlobWorkerList(bmData, workerListReady)); + wait(workerListReady.getFuture()); // Once we acknowledge the existing blob workers, we can go ahead and recruit new ones bmData->startRecruiting.trigger(); From f711ef56a331977880d6eafe0fa0ba73215b2355 Mon Sep 17 00:00:00 2001 From: Evan Tschannen Date: Fri, 3 Dec 2021 14:27:11 -0800 Subject: [PATCH 036/413] fix: whenAtLeast updates lastReturnedVersion --- fdbclient/NativeAPI.actor.cpp | 3 +++ 1 file changed, 3 insertions(+) diff --git a/fdbclient/NativeAPI.actor.cpp b/fdbclient/NativeAPI.actor.cpp index ad6a76e852..f00f642d4f 100644 --- a/fdbclient/NativeAPI.actor.cpp +++ b/fdbclient/NativeAPI.actor.cpp @@ -7119,6 +7119,9 @@ ACTOR Future changeFeedWhenAtLatest(ChangeFeedData* self, Version version) } } } + if (self->lastReturnedVersion.get() < version) { + self->lastReturnedVersion.set(version); + } ASSERT(self->getVersion() >= version); return Void(); } From ff4701315898f12352c0d543b02211b2613e41ba Mon Sep 17 00:00:00 2001 From: Evan Tschannen Date: Sat, 4 Dec 2021 17:08:23 -0800 Subject: [PATCH 037/413] fix: check if the master has been killed while waiting for getNextBMEpoch --- fdbserver/ClusterController.actor.cpp | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/fdbserver/ClusterController.actor.cpp b/fdbserver/ClusterController.actor.cpp index f14b987fb2..1ab68c344a 100644 --- a/fdbserver/ClusterController.actor.cpp +++ b/fdbserver/ClusterController.actor.cpp @@ -5284,6 +5284,11 @@ ACTOR Future startBlobManager(ClusterControllerData* self) { id_used); int64_t nextEpoch = wait(getNextBMEpoch(self)); + if (!self->masterProcessId.present() || + self->masterProcessId != self->db.serverInfo->get().master.locality.processId() || + self->db.serverInfo->get().recoveryState < RecoveryState::ACCEPTING_COMMITS) { + continue; + } InitializeBlobManagerRequest req(deterministicRandom()->randomUniqueID(), nextEpoch); state WorkerDetails worker = bmWorker.worker; if (self->onMasterIsBetter(worker, ProcessClass::BlobManager)) { From da9c42fd9e0fea2533e0db89ca35597b02609ab8 Mon Sep 17 00:00:00 2001 From: Evan Tschannen Date: Sun, 5 Dec 2021 14:59:24 -0800 Subject: [PATCH 038/413] do not log transaction metrics in simulation --- fdbclient/NativeAPI.actor.cpp | 123 ++++++++++++++++++---------------- 1 file changed, 64 insertions(+), 59 deletions(-) diff --git a/fdbclient/NativeAPI.actor.cpp b/fdbclient/NativeAPI.actor.cpp index f00f642d4f..e957bbf676 100644 --- a/fdbclient/NativeAPI.actor.cpp +++ b/fdbclient/NativeAPI.actor.cpp @@ -77,6 +77,7 @@ #include "flow/TLSConfig.actor.h" #include "flow/Tracing.h" #include "flow/UnitTest.h" +#include "flow/network.h" #include "flow/serialize.h" #ifdef WIN32 @@ -379,39 +380,41 @@ ACTOR Future databaseLogger(DatabaseContext* cx) { loop { wait(delay(CLIENT_KNOBS->SYSTEM_MONITOR_INTERVAL, TaskPriority::FlushTrace)); - TraceEvent ev("TransactionMetrics", cx->dbId); + if (!g_network->isSimulated()) { + TraceEvent ev("TransactionMetrics", cx->dbId); - ev.detail("Elapsed", (lastLogged == 0) ? 0 : now() - lastLogged) - .detail("Cluster", - cx->getConnectionRecord() - ? cx->getConnectionRecord()->getConnectionString().clusterKeyName().toString() - : "") - .detail("Internal", cx->internal); + ev.detail("Elapsed", (lastLogged == 0) ? 0 : now() - lastLogged) + .detail("Cluster", + cx->getConnectionRecord() + ? cx->getConnectionRecord()->getConnectionString().clusterKeyName().toString() + : "") + .detail("Internal", cx->internal); - cx->cc.logToTraceEvent(ev); + cx->cc.logToTraceEvent(ev); - ev.detail("LocationCacheEntryCount", cx->locationCache.size()); - ev.detail("MeanLatency", cx->latencies.mean()) - .detail("MedianLatency", cx->latencies.median()) - .detail("Latency90", cx->latencies.percentile(0.90)) - .detail("Latency98", cx->latencies.percentile(0.98)) - .detail("MaxLatency", cx->latencies.max()) - .detail("MeanRowReadLatency", cx->readLatencies.mean()) - .detail("MedianRowReadLatency", cx->readLatencies.median()) - .detail("MaxRowReadLatency", cx->readLatencies.max()) - .detail("MeanGRVLatency", cx->GRVLatencies.mean()) - .detail("MedianGRVLatency", cx->GRVLatencies.median()) - .detail("MaxGRVLatency", cx->GRVLatencies.max()) - .detail("MeanCommitLatency", cx->commitLatencies.mean()) - .detail("MedianCommitLatency", cx->commitLatencies.median()) - .detail("MaxCommitLatency", cx->commitLatencies.max()) - .detail("MeanMutationsPerCommit", cx->mutationsPerCommit.mean()) - .detail("MedianMutationsPerCommit", cx->mutationsPerCommit.median()) - .detail("MaxMutationsPerCommit", cx->mutationsPerCommit.max()) - .detail("MeanBytesPerCommit", cx->bytesPerCommit.mean()) - .detail("MedianBytesPerCommit", cx->bytesPerCommit.median()) - .detail("MaxBytesPerCommit", cx->bytesPerCommit.max()) - .detail("NumLocalityCacheEntries", cx->locationCache.size()); + ev.detail("LocationCacheEntryCount", cx->locationCache.size()); + ev.detail("MeanLatency", cx->latencies.mean()) + .detail("MedianLatency", cx->latencies.median()) + .detail("Latency90", cx->latencies.percentile(0.90)) + .detail("Latency98", cx->latencies.percentile(0.98)) + .detail("MaxLatency", cx->latencies.max()) + .detail("MeanRowReadLatency", cx->readLatencies.mean()) + .detail("MedianRowReadLatency", cx->readLatencies.median()) + .detail("MaxRowReadLatency", cx->readLatencies.max()) + .detail("MeanGRVLatency", cx->GRVLatencies.mean()) + .detail("MedianGRVLatency", cx->GRVLatencies.median()) + .detail("MaxGRVLatency", cx->GRVLatencies.max()) + .detail("MeanCommitLatency", cx->commitLatencies.mean()) + .detail("MedianCommitLatency", cx->commitLatencies.median()) + .detail("MaxCommitLatency", cx->commitLatencies.max()) + .detail("MeanMutationsPerCommit", cx->mutationsPerCommit.mean()) + .detail("MedianMutationsPerCommit", cx->mutationsPerCommit.median()) + .detail("MaxMutationsPerCommit", cx->mutationsPerCommit.max()) + .detail("MeanBytesPerCommit", cx->bytesPerCommit.mean()) + .detail("MedianBytesPerCommit", cx->bytesPerCommit.median()) + .detail("MaxBytesPerCommit", cx->bytesPerCommit.max()) + .detail("NumLocalityCacheEntries", cx->locationCache.size()); + } cx->latencies.clear(); cx->readLatencies.clear(); @@ -437,42 +440,44 @@ ACTOR Future databaseLogger(DatabaseContext* cx) { traceTSSErrors("TSS_TSSErrors", it.first, it.second->tssErrorsByCode); } - TraceEvent tssEv("TSSClientMetrics", cx->dbId); - tssEv.detail("TSSID", it.first) - .detail("Elapsed", (lastLogged == 0) ? 0 : now() - lastLogged) - .detail("Internal", cx->internal); + if (!g_network->isSimulated()) { + TraceEvent tssEv("TSSClientMetrics", cx->dbId); + tssEv.detail("TSSID", it.first) + .detail("Elapsed", (lastLogged == 0) ? 0 : now() - lastLogged) + .detail("Internal", cx->internal); - it.second->cc.logToTraceEvent(tssEv); + it.second->cc.logToTraceEvent(tssEv); - tssEv.detail("MeanSSGetValueLatency", it.second->SSgetValueLatency.mean()) - .detail("MedianSSGetValueLatency", it.second->SSgetValueLatency.median()) - .detail("SSGetValueLatency90", it.second->SSgetValueLatency.percentile(0.90)) - .detail("SSGetValueLatency99", it.second->SSgetValueLatency.percentile(0.99)); + tssEv.detail("MeanSSGetValueLatency", it.second->SSgetValueLatency.mean()) + .detail("MedianSSGetValueLatency", it.second->SSgetValueLatency.median()) + .detail("SSGetValueLatency90", it.second->SSgetValueLatency.percentile(0.90)) + .detail("SSGetValueLatency99", it.second->SSgetValueLatency.percentile(0.99)); - tssEv.detail("MeanTSSGetValueLatency", it.second->TSSgetValueLatency.mean()) - .detail("MedianTSSGetValueLatency", it.second->TSSgetValueLatency.median()) - .detail("TSSGetValueLatency90", it.second->TSSgetValueLatency.percentile(0.90)) - .detail("TSSGetValueLatency99", it.second->TSSgetValueLatency.percentile(0.99)); + tssEv.detail("MeanTSSGetValueLatency", it.second->TSSgetValueLatency.mean()) + .detail("MedianTSSGetValueLatency", it.second->TSSgetValueLatency.median()) + .detail("TSSGetValueLatency90", it.second->TSSgetValueLatency.percentile(0.90)) + .detail("TSSGetValueLatency99", it.second->TSSgetValueLatency.percentile(0.99)); - tssEv.detail("MeanSSGetKeyLatency", it.second->SSgetKeyLatency.mean()) - .detail("MedianSSGetKeyLatency", it.second->SSgetKeyLatency.median()) - .detail("SSGetKeyLatency90", it.second->SSgetKeyLatency.percentile(0.90)) - .detail("SSGetKeyLatency99", it.second->SSgetKeyLatency.percentile(0.99)); + tssEv.detail("MeanSSGetKeyLatency", it.second->SSgetKeyLatency.mean()) + .detail("MedianSSGetKeyLatency", it.second->SSgetKeyLatency.median()) + .detail("SSGetKeyLatency90", it.second->SSgetKeyLatency.percentile(0.90)) + .detail("SSGetKeyLatency99", it.second->SSgetKeyLatency.percentile(0.99)); - tssEv.detail("MeanTSSGetKeyLatency", it.second->TSSgetKeyLatency.mean()) - .detail("MedianTSSGetKeyLatency", it.second->TSSgetKeyLatency.median()) - .detail("TSSGetKeyLatency90", it.second->TSSgetKeyLatency.percentile(0.90)) - .detail("TSSGetKeyLatency99", it.second->TSSgetKeyLatency.percentile(0.99)); + tssEv.detail("MeanTSSGetKeyLatency", it.second->TSSgetKeyLatency.mean()) + .detail("MedianTSSGetKeyLatency", it.second->TSSgetKeyLatency.median()) + .detail("TSSGetKeyLatency90", it.second->TSSgetKeyLatency.percentile(0.90)) + .detail("TSSGetKeyLatency99", it.second->TSSgetKeyLatency.percentile(0.99)); - tssEv.detail("MeanSSGetKeyValuesLatency", it.second->SSgetKeyValuesLatency.mean()) - .detail("MedianSSGetKeyValuesLatency", it.second->SSgetKeyValuesLatency.median()) - .detail("SSGetKeyValuesLatency90", it.second->SSgetKeyValuesLatency.percentile(0.90)) - .detail("SSGetKeyValuesLatency99", it.second->SSgetKeyValuesLatency.percentile(0.99)); + tssEv.detail("MeanSSGetKeyValuesLatency", it.second->SSgetKeyValuesLatency.mean()) + .detail("MedianSSGetKeyValuesLatency", it.second->SSgetKeyValuesLatency.median()) + .detail("SSGetKeyValuesLatency90", it.second->SSgetKeyValuesLatency.percentile(0.90)) + .detail("SSGetKeyValuesLatency99", it.second->SSgetKeyValuesLatency.percentile(0.99)); - tssEv.detail("MeanTSSGetKeyValuesLatency", it.second->TSSgetKeyValuesLatency.mean()) - .detail("MedianTSSGetKeyValuesLatency", it.second->TSSgetKeyValuesLatency.median()) - .detail("TSSGetKeyValuesLatency90", it.second->TSSgetKeyValuesLatency.percentile(0.90)) - .detail("TSSGetKeyValuesLatency99", it.second->TSSgetKeyValuesLatency.percentile(0.99)); + tssEv.detail("MeanTSSGetKeyValuesLatency", it.second->TSSgetKeyValuesLatency.mean()) + .detail("MedianTSSGetKeyValuesLatency", it.second->TSSgetKeyValuesLatency.median()) + .detail("TSSGetKeyValuesLatency90", it.second->TSSgetKeyValuesLatency.percentile(0.90)) + .detail("TSSGetKeyValuesLatency99", it.second->TSSgetKeyValuesLatency.percentile(0.99)); + } it.second->clear(); } From 98b4299fb2bbcbb1798a6fe1fbcfd9bf9a753a70 Mon Sep 17 00:00:00 2001 From: Evan Tschannen Date: Sun, 5 Dec 2021 14:59:46 -0800 Subject: [PATCH 039/413] .error() on a trace event must be before the details --- 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 27ea0cd3dc..68ecdf4c49 100644 --- a/fdbserver/storageserver.actor.cpp +++ b/fdbserver/storageserver.actor.cpp @@ -1877,7 +1877,7 @@ ACTOR Future localChangeFeedStream(StorageServer* data, } } } catch (Error& e) { - TraceEvent(SevError, "LocalChangeFeedError", data->thisServerID).detail("CFID", rangeID.printable()).error(e); + TraceEvent(SevError, "LocalChangeFeedError", data->thisServerID).error(e).detail("CFID", rangeID.printable()); throw; } } From 13ef5afb9c850ef9d16677afb52d27de59adafea Mon Sep 17 00:00:00 2001 From: Evan Tschannen Date: Sun, 5 Dec 2021 15:02:25 -0800 Subject: [PATCH 040/413] reject blob workers joining from the wrong data center we must run the checkblobworkers actors even on epoch 1 check for an already killed worker even right after it is recruited --- fdbserver/BlobManager.actor.cpp | 35 ++++++++++++++++++++------------- 1 file changed, 21 insertions(+), 14 deletions(-) diff --git a/fdbserver/BlobManager.actor.cpp b/fdbserver/BlobManager.actor.cpp index 841d3293c1..088c6bdbc3 100644 --- a/fdbserver/BlobManager.actor.cpp +++ b/fdbserver/BlobManager.actor.cpp @@ -200,6 +200,7 @@ struct BlobWorkerStats { struct BlobManagerData { UID id; Database db; + Optional dcId; PromiseStream> addActor; std::unordered_map workersById; @@ -223,8 +224,8 @@ struct BlobManagerData { // assigned sequence numbers PromiseStream rangesToAssign; - BlobManagerData(UID id, Database db) - : id(id), db(db), knownBlobRanges(false, normalKeys.end), + BlobManagerData(UID id, Database db, Optional dcId) + : id(id), db(db), dcId(dcId), knownBlobRanges(false, normalKeys.end), restartRecruiting(SERVER_KNOBS->DEBOUNCE_RECRUITING_DELAY), recruitingStream(0) {} ~BlobManagerData() { printf("Destroying blob manager data for %s\n", id.toString().c_str()); } }; @@ -990,7 +991,7 @@ ACTOR Future checkBlobWorkerList(BlobManagerData* bmData, Promise wo // add all blob workers to this new blob manager's records and start monitoring it for (auto& worker : blobWorkers) { if (!bmData->deadWorkers.count(worker.id())) { - if (!bmData->workerAddresses.count(worker.stableAddress())) { + if (!bmData->workerAddresses.count(worker.stableAddress()) && worker.locality.dcId() == bmData->dcId) { bmData->workerAddresses.insert(worker.stableAddress()); bmData->workersById[worker.id()] = worker; bmData->workerStats[worker.id()] = BlobWorkerStats(); @@ -1008,12 +1009,6 @@ ACTOR Future checkBlobWorkerList(BlobManagerData* bmData, Promise wo } ACTOR Future recoverBlobManager(BlobManagerData* bmData) { - // skip this entire algorithm for the first blob manager - if (bmData->epoch == 1) { - bmData->startRecruiting.trigger(); - return Void(); - } - state Promise workerListReady; bmData->addActor.send(checkBlobWorkerList(bmData, workerListReady)); wait(workerListReady.getFuture()); @@ -1021,6 +1016,11 @@ ACTOR Future recoverBlobManager(BlobManagerData* bmData) { // Once we acknowledge the existing blob workers, we can go ahead and recruit new ones bmData->startRecruiting.trigger(); + // skip them rest of the algorithm for the first blob manager + if (bmData->epoch == 1) { + return Void(); + } + // At this point, bmData->workersById is a list of all alive blob workers, but could also include some dead BWs. // The algorithm below works as follows: // 1. We get the existing granule mappings that were persisted by blob workers who were assigned ranges and @@ -1259,10 +1259,16 @@ ACTOR Future initializeBlobWorker(BlobManagerData* self, RecruitBlobWorker if (newBlobWorker.present()) { BlobWorkerInterface bwi = newBlobWorker.get().interf; - self->workerAddresses.insert(bwi.stableAddress()); - self->workersById[bwi.id()] = bwi; - self->workerStats[bwi.id()] = BlobWorkerStats(); - self->addActor.send(monitorBlobWorker(self, bwi)); + if (!self->deadWorkers.count(bwi.id())) { + if (!self->workerAddresses.count(bwi.stableAddress()) && bwi.locality.dcId() == self->dcId) { + self->workerAddresses.insert(bwi.stableAddress()); + self->workersById[bwi.id()] = bwi; + self->workerStats[bwi.id()] = BlobWorkerStats(); + self->addActor.send(monitorBlobWorker(self, bwi)); + } else if (!self->workersById.count(bwi.id())) { + self->addActor.send(killBlobWorker(self, bwi, false)); + } + } TraceEvent("BMRecruiting") .detail("State", "Finished request") @@ -1349,7 +1355,8 @@ ACTOR Future blobManager(BlobManagerInterface bmInterf, Reference const> dbInfo, int64_t epoch) { state BlobManagerData self(deterministicRandom()->randomUniqueID(), - openDBOnServer(dbInfo, TaskPriority::DefaultEndpoint, LockAware::True)); + openDBOnServer(dbInfo, TaskPriority::DefaultEndpoint, LockAware::True), + bmInterf.locality.dcId()); state Future collection = actorCollection(self.addActor.getFuture()); From 02c650c6e70a51079c0b9d6ab12abe604d00f660 Mon Sep 17 00:00:00 2001 From: Evan Tschannen Date: Sun, 5 Dec 2021 16:16:20 -0800 Subject: [PATCH 041/413] fix: sending to mutations can cause whenAtLeast to update lastReturnedVersion before we have a chance to --- fdbclient/NativeAPI.actor.cpp | 15 ++++++++++++--- 1 file changed, 12 insertions(+), 3 deletions(-) diff --git a/fdbclient/NativeAPI.actor.cpp b/fdbclient/NativeAPI.actor.cpp index e957bbf676..1bd120bc57 100644 --- a/fdbclient/NativeAPI.actor.cpp +++ b/fdbclient/NativeAPI.actor.cpp @@ -7269,8 +7269,11 @@ ACTOR Future mergeChangeFeedStream(Reference db, if (nextStream.next.version != checkVersion) { if (nextOut.size()) { *begin = checkVersion + 1; + ASSERT(nextOut.back().version >= results->lastReturnedVersion.get()); results->mutations.send(nextOut); - results->lastReturnedVersion.set(nextOut.back().version); + if (nextOut.back().version > results->lastReturnedVersion.get()) { + results->lastReturnedVersion.set(nextOut.back().version); + } nextOut = Standalone>(); } checkVersion = nextStream.next.version; @@ -7295,8 +7298,11 @@ ACTOR Future mergeChangeFeedStream(Reference db, } } if (nextOut.size()) { + ASSERT(nextOut.back().version >= results->lastReturnedVersion.get()); results->mutations.send(nextOut); - results->lastReturnedVersion.set(nextOut.back().version); + if (nextOut.back().version > results->lastReturnedVersion.get()) { + results->lastReturnedVersion.set(nextOut.back().version); + } } throw end_of_stream(); } @@ -7437,9 +7443,12 @@ ACTOR Future getChangeFeedStreamActor(Reference db, when(wait(cx->connectionFileChanged())) { break; } when(ChangeFeedStreamReply rep = waitNext(replyStream.getFuture())) { begin = rep.mutations.back().version + 1; + ASSERT(rep.mutations.back().version >= results->lastReturnedVersion.get()); results->mutations.send( Standalone>(rep.mutations, rep.arena)); - results->lastReturnedVersion.set(rep.mutations.back().version); + if (rep.mutations.back().version > results->lastReturnedVersion.get()) { + results->lastReturnedVersion.set(rep.mutations.back().version); + } if (!atLatest && rep.atLatestVersion) { atLatest = true; results->notAtLatest.set(0); From b55b095ed050d494ca729cfbe1e0e92a2d64fa7f Mon Sep 17 00:00:00 2001 From: Evan Tschannen Date: Sun, 5 Dec 2021 20:18:03 -0800 Subject: [PATCH 042/413] fix: long lived stateless processes need to be the last comparison criteria to avoid better master exists from changing behavior from the original recruitment --- fdbserver/ClusterController.actor.cpp | 12 ++++++------ 1 file changed, 6 insertions(+), 6 deletions(-) diff --git a/fdbserver/ClusterController.actor.cpp b/fdbserver/ClusterController.actor.cpp index 1ab68c344a..1a0cf025a7 100644 --- a/fdbserver/ClusterController.actor.cpp +++ b/fdbserver/ClusterController.actor.cpp @@ -872,8 +872,8 @@ public: fitness_workers[std::make_tuple(fitness, id_used[worker_process_id], worker_details.degraded, - isLongLivedStateless(worker_process_id), - inCCDC)] + inCCDC, + isLongLivedStateless(worker_process_id))] .push_back(worker_details); } @@ -1430,8 +1430,8 @@ public: auto sharing = preferredSharing.find(it.first); fitness_workers[std::make_tuple(fitness, id_used[it.first], - isLongLivedStateless(it.first), - sharing != preferredSharing.end() ? sharing->second : 1e6)] + sharing != preferredSharing.end() ? sharing->second : 1e6, + isLongLivedStateless(it.first))] .push_back(it.second.details); } } @@ -1478,8 +1478,8 @@ public: auto sharing = preferredSharing.find(it.first); fitness_workers[std::make_tuple(fitness, id_used[it.first], - isLongLivedStateless(it.first), - sharing != preferredSharing.end() ? sharing->second : 1e6)] + sharing != preferredSharing.end() ? sharing->second : 1e6, + isLongLivedStateless(it.first))] .push_back(it.second.details); } } From a4fff1932087e6d2dd77e8182256630fb1b93e6b Mon Sep 17 00:00:00 2001 From: Evan Tschannen Date: Sun, 5 Dec 2021 20:26:13 -0800 Subject: [PATCH 043/413] Revert "fix: long lived stateless processes need to be the last comparison criteria to avoid better master exists from changing behavior from the original recruitment" This reverts commit b55b095ed050d494ca729cfbe1e0e92a2d64fa7f. --- fdbserver/ClusterController.actor.cpp | 12 ++++++------ 1 file changed, 6 insertions(+), 6 deletions(-) diff --git a/fdbserver/ClusterController.actor.cpp b/fdbserver/ClusterController.actor.cpp index 1a0cf025a7..1ab68c344a 100644 --- a/fdbserver/ClusterController.actor.cpp +++ b/fdbserver/ClusterController.actor.cpp @@ -872,8 +872,8 @@ public: fitness_workers[std::make_tuple(fitness, id_used[worker_process_id], worker_details.degraded, - inCCDC, - isLongLivedStateless(worker_process_id))] + isLongLivedStateless(worker_process_id), + inCCDC)] .push_back(worker_details); } @@ -1430,8 +1430,8 @@ public: auto sharing = preferredSharing.find(it.first); fitness_workers[std::make_tuple(fitness, id_used[it.first], - sharing != preferredSharing.end() ? sharing->second : 1e6, - isLongLivedStateless(it.first))] + isLongLivedStateless(it.first), + sharing != preferredSharing.end() ? sharing->second : 1e6)] .push_back(it.second.details); } } @@ -1478,8 +1478,8 @@ public: auto sharing = preferredSharing.find(it.first); fitness_workers[std::make_tuple(fitness, id_used[it.first], - sharing != preferredSharing.end() ? sharing->second : 1e6, - isLongLivedStateless(it.first))] + isLongLivedStateless(it.first), + sharing != preferredSharing.end() ? sharing->second : 1e6)] .push_back(it.second.details); } } From 528df86a28d7cf2b720067f0f69e94739b8e30af Mon Sep 17 00:00:00 2001 From: Josh Slocum Date: Mon, 6 Dec 2021 14:06:17 -0600 Subject: [PATCH 044/413] fix: refactored and fixed changeFeedWhenAtLatest returning early --- fdbclient/NativeAPI.actor.cpp | 97 ++++++++++++++++++++++------------- 1 file changed, 60 insertions(+), 37 deletions(-) diff --git a/fdbclient/NativeAPI.actor.cpp b/fdbclient/NativeAPI.actor.cpp index 1bd120bc57..7ad6279453 100644 --- a/fdbclient/NativeAPI.actor.cpp +++ b/fdbclient/NativeAPI.actor.cpp @@ -7075,44 +7075,51 @@ Version ChangeFeedData::getVersion() { return lastReturnedVersion.get(); } +ACTOR Future changeFeedWaitLatest(ChangeFeedData* self, Version version) { + // first, wait on SS to have sent up through version + int desired = 0; + int waiting = 0; + std::vector> allAtLeast; + for (auto& it : self->storageData) { + if (it->version.get() < version) { + waiting++; + if (version > it->desired.get()) { + it->desired.set(version); + desired++; + } + allAtLeast.push_back(it->version.whenAtLeast(version)); + } + } + + wait(waitForAll(allAtLeast)); + + // then, wait on ss streams to have processed up through version + std::vector> onEmpty; + for (auto& it : self->streams) { + if (!it.isEmpty()) { + onEmpty.push_back(it.onEmpty()); + } + } + + if (onEmpty.size()) { + wait(waitForAll(onEmpty)); + wait(delay(0)); + } + + // then, wait for client to have consumed up through version + if (!self->mutations.isEmpty()) { + wait(self->mutations.onEmpty()); + wait(delay(0)); + } + return Void(); +} + ACTOR Future changeFeedWhenAtLatest(ChangeFeedData* self, Version version) { state Future lastReturned = self->lastReturnedVersion.whenAtLeast(version); loop { if (self->notAtLatest.get() == 0) { - std::vector> allAtLeast; - for (auto& it : self->storageData) { - if (it->version.get() < version) { - if (version > it->desired.get()) { - it->desired.set(version); - } - allAtLeast.push_back(it->version.whenAtLeast(version)); - } - } choose { - when(wait(lastReturned)) { break; } - when(wait(waitForAll(allAtLeast))) { - std::vector> onEmpty; - if (!self->mutations.isEmpty()) { - onEmpty.push_back(self->mutations.onEmpty()); - } - for (auto& it : self->streams) { - if (!it.isEmpty()) { - onEmpty.push_back(it.onEmpty()); - } - } - if (!onEmpty.size()) { - break; - } - choose { - when(wait(waitForAll(onEmpty))) { - wait(delay(0)); - break; - } - when(wait(lastReturned)) { break; } - when(wait(self->refresh.getFuture())) {} - when(wait(self->notAtLatest.onChange())) {} - } - } + when(wait(changeFeedWaitLatest(self, version))) { return Void(); } when(wait(self->refresh.getFuture())) {} when(wait(self->notAtLatest.onChange())) {} } @@ -7124,6 +7131,7 @@ ACTOR Future changeFeedWhenAtLatest(ChangeFeedData* self, Version version) } } } + if (self->lastReturnedVersion.get() < version) { self->lastReturnedVersion.set(version); } @@ -7421,15 +7429,18 @@ ACTOR Future getChangeFeedStreamActor(Reference db, req.begin = begin; req.end = end; req.range = range; + + results->streams.clear(); + StorageServerInterface interf = locations[0].second->getInterface(chosenLocations[0]); - state ReplyPromiseStream replyStream = - interf.changeFeedStream.getReplyStream(req); + for (auto& it : results->storageData) { if (it->debugGetReferenceCount() == 2) { db->changeFeedUpdaters.erase(it->id); } } - results->streams.clear(); + results->streams.push_back(interf.changeFeedStream.getReplyStream(req)); + results->storageData.clear(); results->storageData.push_back(db->getStorageData(interf)); Promise refresh = results->refresh; @@ -7441,8 +7452,20 @@ ACTOR Future getChangeFeedStreamActor(Reference db, wait(results->mutations.onEmpty()); choose { when(wait(cx->connectionFileChanged())) { break; } - when(ChangeFeedStreamReply rep = waitNext(replyStream.getFuture())) { + when(ChangeFeedStreamReply rep = waitNext(results->streams[0].getFuture())) { begin = rep.mutations.back().version + 1; + + // TODO REMOVE, for debugging + if (rep.mutations.back().version < results->lastReturnedVersion.get()) { + printf("out of order mutation for CF %s from (%d) %s! %lld < %lld\n", + rangeID.toString().substr(0, 6).c_str(), + results->storageData.size(), + results->storageData.empty() + ? "????" + : results->storageData[0]->id.toString().substr(0, 4).c_str(), + rep.mutations.back().version, + results->lastReturnedVersion.get()); + } ASSERT(rep.mutations.back().version >= results->lastReturnedVersion.get()); results->mutations.send( Standalone>(rep.mutations, rep.arena)); From 951bc4acd78cbeaf23c3c8415f6d8ff481087013 Mon Sep 17 00:00:00 2001 From: Evan Tschannen Date: Mon, 6 Dec 2021 13:12:27 -0800 Subject: [PATCH 045/413] fix: do not call better master exists until the long lived stateless processes have settled into their desired locations --- fdbclient/ServerKnobs.cpp | 1 + fdbclient/ServerKnobs.h | 1 + fdbserver/ClusterController.actor.cpp | 19 ++++++++++++++++--- 3 files changed, 18 insertions(+), 3 deletions(-) diff --git a/fdbclient/ServerKnobs.cpp b/fdbclient/ServerKnobs.cpp index 38b1bbb18c..4011c3c80a 100644 --- a/fdbclient/ServerKnobs.cpp +++ b/fdbclient/ServerKnobs.cpp @@ -509,6 +509,7 @@ void ServerKnobs::initialize(Randomize randomize, ClientKnobs* clientKnobs, IsSi init( POLICY_GENERATIONS, 100 ); if( randomize && BUGGIFY ) POLICY_GENERATIONS = 10; init( DBINFO_SEND_AMOUNT, 5 ); init( DBINFO_BATCH_DELAY, 0.1 ); + init( SINGLETON_RECRUIT_BME_DELAY, 10.0 ); //Move Keys init( SHARD_READY_DELAY, 0.25 ); diff --git a/fdbclient/ServerKnobs.h b/fdbclient/ServerKnobs.h index 68f4456e1f..33e3a023cd 100644 --- a/fdbclient/ServerKnobs.h +++ b/fdbclient/ServerKnobs.h @@ -446,6 +446,7 @@ public: double RECRUITMENT_TIMEOUT; int DBINFO_SEND_AMOUNT; double DBINFO_BATCH_DELAY; + double SINGLETON_RECRUIT_BME_DELAY; // Move Keys double SHARD_READY_DELAY; diff --git a/fdbserver/ClusterController.actor.cpp b/fdbserver/ClusterController.actor.cpp index 1ab68c344a..a9412769e2 100644 --- a/fdbserver/ClusterController.actor.cpp +++ b/fdbserver/ClusterController.actor.cpp @@ -3195,6 +3195,7 @@ public: // recruitX is used to signal when role X needs to be (re)recruited. // recruitingXID is used to track the ID of X's interface which is being recruited. // We use AsyncVars to kill (i.e. halt) singletons that have been replaced. + double lastRecruitTime = 0; AsyncVar recruitDistributor; Optional recruitingDistributorID; AsyncVar recruitRatekeeper; @@ -3299,7 +3300,10 @@ struct RatekeeperSingleton : Singleton { brokenPromiseToNever(interface.get().haltRatekeeper.getReply(HaltRatekeeperRequest(cc->id))); } } - void recruit(ClusterControllerData* cc) const { cc->recruitRatekeeper.set(true); } + void recruit(ClusterControllerData* cc) const { + cc->lastRecruitTime = now(); + cc->recruitRatekeeper.set(true); + } }; struct DataDistributorSingleton : Singleton { @@ -3320,7 +3324,10 @@ struct DataDistributorSingleton : Singleton { brokenPromiseToNever(interface.get().haltDataDistributor.getReply(HaltDataDistributorRequest(cc->id))); } } - void recruit(ClusterControllerData* cc) const { cc->recruitDistributor.set(true); } + void recruit(ClusterControllerData* cc) const { + cc->lastRecruitTime = now(); + cc->recruitDistributor.set(true); + } }; struct BlobManagerSingleton : Singleton { @@ -3341,7 +3348,10 @@ struct BlobManagerSingleton : Singleton { brokenPromiseToNever(interface.get().haltBlobManager.getReply(HaltBlobManagerRequest(cc->id))); } } - void recruit(ClusterControllerData* cc) const { cc->recruitBlobManager.set(true); } + void recruit(ClusterControllerData* cc) const { + cc->lastRecruitTime = now(); + cc->recruitBlobManager.set(true); + } }; ACTOR Future clusterWatchDatabase(ClusterControllerData* cluster, ClusterControllerData::DBInfo* db) { @@ -3798,6 +3808,9 @@ ACTOR Future doCheckOutstandingRequests(ClusterControllerData* self) { while (!self->goodRecruitmentTime.isReady()) { wait(self->goodRecruitmentTime); } + while (now() - self->lastRecruitTime < SERVER_KNOBS->SINGLETON_RECRUIT_BME_DELAY) { + wait(delay(SERVER_KNOBS->SINGLETON_RECRUIT_BME_DELAY + 0.001 - (now() - self->lastRecruitTime))); + } checkOutstandingRecruitmentRequests(self); checkOutstandingStorageRequests(self); From 130def7897febd32dd2386f5202cd914205eef0e Mon Sep 17 00:00:00 2001 From: Evan Tschannen Date: Mon, 6 Dec 2021 13:50:31 -0800 Subject: [PATCH 046/413] fix: make sure both conditions are true before better master exists is executed --- fdbserver/ClusterController.actor.cpp | 13 ++++++++----- 1 file changed, 8 insertions(+), 5 deletions(-) diff --git a/fdbserver/ClusterController.actor.cpp b/fdbserver/ClusterController.actor.cpp index a9412769e2..5f24dfcc81 100644 --- a/fdbserver/ClusterController.actor.cpp +++ b/fdbserver/ClusterController.actor.cpp @@ -3805,11 +3805,14 @@ void checkBetterSingletons(ClusterControllerData* self) { ACTOR Future doCheckOutstandingRequests(ClusterControllerData* self) { try { wait(delay(SERVER_KNOBS->CHECK_OUTSTANDING_INTERVAL)); - while (!self->goodRecruitmentTime.isReady()) { - wait(self->goodRecruitmentTime); - } - while (now() - self->lastRecruitTime < SERVER_KNOBS->SINGLETON_RECRUIT_BME_DELAY) { - wait(delay(SERVER_KNOBS->SINGLETON_RECRUIT_BME_DELAY + 0.001 - (now() - self->lastRecruitTime))); + while (now() - self->lastRecruitTime < SERVER_KNOBS->SINGLETON_RECRUIT_BME_DELAY || + !self->goodRecruitmentTime.isReady()) { + if (now() - self->lastRecruitTime < SERVER_KNOBS->SINGLETON_RECRUIT_BME_DELAY) { + wait(delay(SERVER_KNOBS->SINGLETON_RECRUIT_BME_DELAY + 0.001 - (now() - self->lastRecruitTime))); + } + if (!self->goodRecruitmentTime.isReady()) { + wait(self->goodRecruitmentTime); + } } checkOutstandingRecruitmentRequests(self); From fdd35070a1ee36747df564069768aae79aa9e0f5 Mon Sep 17 00:00:00 2001 From: Evan Tschannen Date: Tue, 7 Dec 2021 09:57:39 -0800 Subject: [PATCH 047/413] fix: merge cursors can have more mutations sent to self->mutations after it goes empty --- fdbclient/NativeAPI.actor.cpp | 7 +++++-- 1 file changed, 5 insertions(+), 2 deletions(-) diff --git a/fdbclient/NativeAPI.actor.cpp b/fdbclient/NativeAPI.actor.cpp index 7ad6279453..37e2abb827 100644 --- a/fdbclient/NativeAPI.actor.cpp +++ b/fdbclient/NativeAPI.actor.cpp @@ -7103,11 +7103,14 @@ ACTOR Future changeFeedWaitLatest(ChangeFeedData* self, Version version) { if (onEmpty.size()) { wait(waitForAll(onEmpty)); + } + + if (self->mutations.isEmpty()) { wait(delay(0)); } // then, wait for client to have consumed up through version - if (!self->mutations.isEmpty()) { + while (!self->mutations.isEmpty()) { wait(self->mutations.onEmpty()); wait(delay(0)); } @@ -7119,7 +7122,7 @@ ACTOR Future changeFeedWhenAtLatest(ChangeFeedData* self, Version version) loop { if (self->notAtLatest.get() == 0) { choose { - when(wait(changeFeedWaitLatest(self, version))) { return Void(); } + when(wait(changeFeedWaitLatest(self, version))) { break; } when(wait(self->refresh.getFuture())) {} when(wait(self->notAtLatest.onChange())) {} } From 75267aba80be1298cce8c724b95d7bbac098f85e Mon Sep 17 00:00:00 2001 From: Evan Tschannen Date: Tue, 7 Dec 2021 13:03:50 -0800 Subject: [PATCH 048/413] fix: do not update lastReturnedVersion until the client has consumed the mutations --- fdbclient/NativeAPI.actor.cpp | 25 ++++++++++++++----------- 1 file changed, 14 insertions(+), 11 deletions(-) diff --git a/fdbclient/NativeAPI.actor.cpp b/fdbclient/NativeAPI.actor.cpp index 37e2abb827..3590f2eb81 100644 --- a/fdbclient/NativeAPI.actor.cpp +++ b/fdbclient/NativeAPI.actor.cpp @@ -7282,6 +7282,7 @@ ACTOR Future mergeChangeFeedStream(Reference db, *begin = checkVersion + 1; ASSERT(nextOut.back().version >= results->lastReturnedVersion.get()); results->mutations.send(nextOut); + wait(results->mutations.onEmpty()); if (nextOut.back().version > results->lastReturnedVersion.get()) { results->lastReturnedVersion.set(nextOut.back().version); } @@ -7311,6 +7312,7 @@ ACTOR Future mergeChangeFeedStream(Reference db, if (nextOut.size()) { ASSERT(nextOut.back().version >= results->lastReturnedVersion.get()); results->mutations.send(nextOut); + wait(results->mutations.onEmpty()); if (nextOut.back().version > results->lastReturnedVersion.get()) { results->lastReturnedVersion.set(nextOut.back().version); } @@ -7455,32 +7457,33 @@ ACTOR Future getChangeFeedStreamActor(Reference db, wait(results->mutations.onEmpty()); choose { when(wait(cx->connectionFileChanged())) { break; } - when(ChangeFeedStreamReply rep = waitNext(results->streams[0].getFuture())) { - begin = rep.mutations.back().version + 1; + when(state ChangeFeedStreamReply feedReply = waitNext(results->streams[0].getFuture())) { + begin = feedReply.mutations.back().version + 1; // TODO REMOVE, for debugging - if (rep.mutations.back().version < results->lastReturnedVersion.get()) { + if (feedReply.mutations.back().version < results->lastReturnedVersion.get()) { printf("out of order mutation for CF %s from (%d) %s! %lld < %lld\n", rangeID.toString().substr(0, 6).c_str(), results->storageData.size(), results->storageData.empty() ? "????" : results->storageData[0]->id.toString().substr(0, 4).c_str(), - rep.mutations.back().version, + feedReply.mutations.back().version, results->lastReturnedVersion.get()); } - ASSERT(rep.mutations.back().version >= results->lastReturnedVersion.get()); + ASSERT(feedReply.mutations.back().version >= results->lastReturnedVersion.get()); results->mutations.send( - Standalone>(rep.mutations, rep.arena)); - if (rep.mutations.back().version > results->lastReturnedVersion.get()) { - results->lastReturnedVersion.set(rep.mutations.back().version); + Standalone>(feedReply.mutations, feedReply.arena)); + wait(results->mutations.onEmpty()); + if (feedReply.mutations.back().version > results->lastReturnedVersion.get()) { + results->lastReturnedVersion.set(feedReply.mutations.back().version); } - if (!atLatest && rep.atLatestVersion) { + if (!atLatest && feedReply.atLatestVersion) { atLatest = true; results->notAtLatest.set(0); } - if (rep.minStreamVersion > results->storageData[0]->version.get()) { - results->storageData[0]->version.set(rep.minStreamVersion); + if (feedReply.minStreamVersion > results->storageData[0]->version.get()) { + results->storageData[0]->version.set(feedReply.minStreamVersion); } } } From 9dcdda8192340bc0b6a99d2b8f01d4a075c755d4 Mon Sep 17 00:00:00 2001 From: Evan Tschannen Date: Tue, 7 Dec 2021 20:34:01 -0800 Subject: [PATCH 049/413] fix: do not merge rollbacks when fetching --- fdbserver/storageserver.actor.cpp | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/fdbserver/storageserver.actor.cpp b/fdbserver/storageserver.actor.cpp index 68ecdf4c49..e5e1295154 100644 --- a/fdbserver/storageserver.actor.cpp +++ b/fdbserver/storageserver.actor.cpp @@ -3929,7 +3929,8 @@ ACTOR Future fetchChangeFeedApplier(StorageServer* data, } remoteLoc++; } else if (remoteResult[remoteLoc].version == localResult.version) { - if (remoteResult[remoteLoc].mutations.size()) { + if (remoteResult[remoteLoc].mutations.size() && + remoteResult[remoteLoc].mutations.back().param1 != lastEpochEndPrivateKey) { ASSERT(localResult.mutations.size()); remoteResult[remoteLoc].mutations.append( remoteResult.arena(), localResult.mutations.begin(), localResult.mutations.size()); From 3ee6d69f3d52f5ab6032e28b787221bf177c506b Mon Sep 17 00:00:00 2001 From: Josh Slocum Date: Tue, 7 Dec 2021 14:15:05 -0600 Subject: [PATCH 050/413] Wait for empty even for whenAtLatest>0 --- fdbclient/NativeAPI.actor.cpp | 11 ++++++----- 1 file changed, 6 insertions(+), 5 deletions(-) diff --git a/fdbclient/NativeAPI.actor.cpp b/fdbclient/NativeAPI.actor.cpp index 3590f2eb81..6468b9a4ea 100644 --- a/fdbclient/NativeAPI.actor.cpp +++ b/fdbclient/NativeAPI.actor.cpp @@ -7109,11 +7109,6 @@ ACTOR Future changeFeedWaitLatest(ChangeFeedData* self, Version version) { wait(delay(0)); } - // then, wait for client to have consumed up through version - while (!self->mutations.isEmpty()) { - wait(self->mutations.onEmpty()); - wait(delay(0)); - } return Void(); } @@ -7135,6 +7130,12 @@ ACTOR Future changeFeedWhenAtLatest(ChangeFeedData* self, Version version) } } + // then, wait for client to have consumed up through version + while (!self->mutations.isEmpty()) { + wait(self->mutations.onEmpty()); + wait(delay(0)); + } + if (self->lastReturnedVersion.get() < version) { self->lastReturnedVersion.set(version); } From c5b2b384dabc08b12c8e2f5fefa74d4c66c38a54 Mon Sep 17 00:00:00 2001 From: Josh Slocum Date: Wed, 8 Dec 2021 08:42:27 -0600 Subject: [PATCH 051/413] Fixing ASAN issues --- fdbclient/NativeAPI.actor.cpp | 1 + fdbserver/BlobManager.actor.cpp | 5 +++-- fdbserver/BlobWorker.actor.cpp | 3 +-- 3 files changed, 5 insertions(+), 4 deletions(-) diff --git a/fdbclient/NativeAPI.actor.cpp b/fdbclient/NativeAPI.actor.cpp index 6468b9a4ea..529eac5b45 100644 --- a/fdbclient/NativeAPI.actor.cpp +++ b/fdbclient/NativeAPI.actor.cpp @@ -6532,6 +6532,7 @@ ACTOR Future>> getBlobGranuleRangesActor(Trans KeyRangeRef(blobGranuleMapping[i].key, blobGranuleMapping[i + 1].key)); } } + results.arena().dependsOn(blobGranuleMapping.arena()); if (blobGranuleMapping.more) { currentRange = KeyRangeRef(blobGranuleMapping.back().key, currentRange.end); } else { diff --git a/fdbserver/BlobManager.actor.cpp b/fdbserver/BlobManager.actor.cpp index 088c6bdbc3..ca17045bba 100644 --- a/fdbserver/BlobManager.actor.cpp +++ b/fdbserver/BlobManager.actor.cpp @@ -1172,15 +1172,16 @@ ACTOR Future chaosRangeMover(BlobManagerData* bmData) { // FIXME: with low probability, could immediately revoke it from the new assignment and move // it back right after to test that race + state KeyRange range = randomRange.range(); RangeAssignment revokeOld; revokeOld.isAssign = false; - revokeOld.keyRange = randomRange.range(); + revokeOld.keyRange = range; revokeOld.revoke = RangeRevokeData(false); bmData->rangesToAssign.send(revokeOld); RangeAssignment assignNew; assignNew.isAssign = true; - assignNew.keyRange = randomRange.range(); + assignNew.keyRange = range; assignNew.assign = RangeAssignmentData(); // not a continue bmData->rangesToAssign.send(assignNew); break; diff --git a/fdbserver/BlobWorker.actor.cpp b/fdbserver/BlobWorker.actor.cpp index 19a8349246..b04649fa7f 100644 --- a/fdbserver/BlobWorker.actor.cpp +++ b/fdbserver/BlobWorker.actor.cpp @@ -86,6 +86,7 @@ struct GranuleStartState { Optional history; }; +// TODO add global byte limit for pending and buffered deltas struct GranuleMetadata : NonCopyable, ReferenceCounted { KeyRange keyRange; @@ -2064,12 +2065,10 @@ ACTOR Future handleBlobGranuleFileRequest(Reference bwData j--; } j++; - [[maybe_unused]] Version latestDeltaVersion = invalidVersion; while (j <= i) { BlobFileIndex deltaF = chunkFiles.deltaFiles[j]; chunk.deltaFiles.emplace_back_deep(rep.arena, deltaF.filename, deltaF.offset, deltaF.length); bwData->stats.readReqDeltaBytesReturned += deltaF.length; - latestDeltaVersion = deltaF.version; j++; } From 4a9c2f417231ddf9dc0726033d2be3912fc1b1e0 Mon Sep 17 00:00:00 2001 From: Josh Slocum Date: Wed, 8 Dec 2021 08:47:58 -0600 Subject: [PATCH 052/413] fixing merge conflicts --- fdbclient/NativeAPI.actor.cpp | 12 ++++++------ 1 file changed, 6 insertions(+), 6 deletions(-) diff --git a/fdbclient/NativeAPI.actor.cpp b/fdbclient/NativeAPI.actor.cpp index 529eac5b45..c506894036 100644 --- a/fdbclient/NativeAPI.actor.cpp +++ b/fdbclient/NativeAPI.actor.cpp @@ -7110,6 +7110,12 @@ ACTOR Future changeFeedWaitLatest(ChangeFeedData* self, Version version) { wait(delay(0)); } + // then, wait for client to have consumed up through version + while (!self->mutations.isEmpty()) { + wait(self->mutations.onEmpty()); + wait(delay(0)); + } + return Void(); } @@ -7131,12 +7137,6 @@ ACTOR Future changeFeedWhenAtLatest(ChangeFeedData* self, Version version) } } - // then, wait for client to have consumed up through version - while (!self->mutations.isEmpty()) { - wait(self->mutations.onEmpty()); - wait(delay(0)); - } - if (self->lastReturnedVersion.get() < version) { self->lastReturnedVersion.set(version); } From 6c797504120a895094988b51fb63f6944ada94aa Mon Sep 17 00:00:00 2001 From: Josh Slocum Date: Wed, 8 Dec 2021 12:41:45 -0600 Subject: [PATCH 053/413] Added a bunch of debugging for tracking whenAtLatest problems, and fixed a bug there --- fdbclient/NativeAPI.actor.cpp | 57 +++++++++++++ fdbserver/BlobWorker.actor.cpp | 81 ++++++++++++++++--- .../workloads/BlobGranuleVerifier.actor.cpp | 2 +- 3 files changed, 128 insertions(+), 12 deletions(-) diff --git a/fdbclient/NativeAPI.actor.cpp b/fdbclient/NativeAPI.actor.cpp index c506894036..9f98db20e5 100644 --- a/fdbclient/NativeAPI.actor.cpp +++ b/fdbclient/NativeAPI.actor.cpp @@ -7076,6 +7076,14 @@ Version ChangeFeedData::getVersion() { return lastReturnedVersion.get(); } +// TODO REMOVE when BG is correctness clean +// To debug a waitLatest at wait_version returning early, set wait_version to the version, and start+end to a version +// range that surrounds wait_version enough to figure out what's going on +#define DEBUG_CF_START_VERSION invalidVersion +#define DEBUG_CF_END_VERSION invalidVersion +#define DEBUG_CF_WAIT_VERSION invalidVersion +#define DEBUG_CF_VERSION(v) DEBUG_CF_START_VERSION <= v&& v <= DEBUG_CF_END_VERSION + ACTOR Future changeFeedWaitLatest(ChangeFeedData* self, Version version) { // first, wait on SS to have sent up through version int desired = 0; @@ -7092,6 +7100,14 @@ ACTOR Future changeFeedWaitLatest(ChangeFeedData* self, Version version) { } } + if (DEBUG_CF_WAIT_VERSION == version) { + fmt::print("CFW {0}) WaitLatest: waiting for {1}/{2} ss ({3} < desired)\n", + version, + waiting, + self->storageData.size(), + desired); + } + wait(waitForAll(allAtLeast)); // then, wait on ss streams to have processed up through version @@ -7102,6 +7118,10 @@ ACTOR Future changeFeedWaitLatest(ChangeFeedData* self, Version version) { } } + if (DEBUG_CF_WAIT_VERSION == version) { + fmt::print("CFW {0}) WaitLatest: waiting for {1} ss onEmpty\n", version, onEmpty.size()); + } + if (onEmpty.size()) { wait(waitForAll(onEmpty)); } @@ -7112,16 +7132,29 @@ ACTOR Future changeFeedWaitLatest(ChangeFeedData* self, Version version) { // then, wait for client to have consumed up through version while (!self->mutations.isEmpty()) { + if (DEBUG_CF_WAIT_VERSION == version) { + fmt::print("CFW {0}) WaitLatest: waiting for client onEmpty\n", version); + } wait(self->mutations.onEmpty()); wait(delay(0)); } + if (DEBUG_CF_WAIT_VERSION == version) { + fmt::print("CFW {0}) WaitLatest: done\n", version); + } + return Void(); } ACTOR Future changeFeedWhenAtLatest(ChangeFeedData* self, Version version) { state Future lastReturned = self->lastReturnedVersion.whenAtLeast(version); + if (DEBUG_CF_WAIT_VERSION == version) { + fmt::print("CFW {0}) WhenAtLeast: LR={1}\n", version, self->lastReturnedVersion.get()); + } loop { + if (DEBUG_CF_WAIT_VERSION == version) { + fmt::print("CFW {0}) WhenAtLeast: NotAtLatest={1}\n", version, self->notAtLatest.get()); + } if (self->notAtLatest.get() == 0) { choose { when(wait(changeFeedWaitLatest(self, version))) { break; } @@ -7137,6 +7170,10 @@ ACTOR Future changeFeedWhenAtLatest(ChangeFeedData* self, Version version) } } + if (DEBUG_CF_VERSION(version)) { + fmt::print("CFLR (WAL): {0}\n", version); + } + if (self->lastReturnedVersion.get() < version) { self->lastReturnedVersion.set(version); } @@ -7285,6 +7322,9 @@ ACTOR Future mergeChangeFeedStream(Reference db, ASSERT(nextOut.back().version >= results->lastReturnedVersion.get()); results->mutations.send(nextOut); wait(results->mutations.onEmpty()); + if (DEBUG_CF_VERSION(nextOut.back().version)) { + fmt::print("CFLR (merged): {0} (1)\n", nextOut.back().version); + } if (nextOut.back().version > results->lastReturnedVersion.get()) { results->lastReturnedVersion.set(nextOut.back().version); } @@ -7315,6 +7355,9 @@ ACTOR Future mergeChangeFeedStream(Reference db, ASSERT(nextOut.back().version >= results->lastReturnedVersion.get()); results->mutations.send(nextOut); wait(results->mutations.onEmpty()); + if (DEBUG_CF_VERSION(nextOut.back().version)) { + fmt::print("CFLR (merged): {0} (1)\n", nextOut.back().version); + } if (nextOut.back().version > results->lastReturnedVersion.get()) { results->lastReturnedVersion.set(nextOut.back().version); } @@ -7476,7 +7519,21 @@ ACTOR Future getChangeFeedStreamActor(Reference db, ASSERT(feedReply.mutations.back().version >= results->lastReturnedVersion.get()); results->mutations.send( Standalone>(feedReply.mutations, feedReply.arena)); + + // Because onEmpty returns here before the consuming process, we must do a delay(0) wait(results->mutations.onEmpty()); + wait(delay(0)); + + if (DEBUG_CF_VERSION(feedReply.mutations.back().version)) { + fmt::print("CFLR (single): {0} ({1}), atLatest={2}, rep.atLatest={3}, notAtLatest={4}, " + "minSV={5}\n", + feedReply.mutations.back().version, + feedReply.mutations.size(), + atLatest ? "T" : "F", + feedReply.atLatestVersion ? "T" : "F", + results->notAtLatest.get(), + feedReply.minStreamVersion); + } if (feedReply.mutations.back().version > results->lastReturnedVersion.get()) { results->lastReturnedVersion.set(feedReply.mutations.back().version); } diff --git a/fdbserver/BlobWorker.actor.cpp b/fdbserver/BlobWorker.actor.cpp index b04649fa7f..0dca8481aa 100644 --- a/fdbserver/BlobWorker.actor.cpp +++ b/fdbserver/BlobWorker.actor.cpp @@ -1136,6 +1136,12 @@ static Version doGranuleRollback(Reference metadata, return cfRollbackVersion; } +// TODO REMOVE once correctness clean +#define DEBUG_BW_START_VERSION invalidVersion +#define DEBUG_BW_END_VERSION invalidVersion +#define DEBUG_BW_WAIT_VERSION invalidVersion +#define DEBUG_BW_VERSION(v) DEBUG_BW_START_VERSION <= v&& v <= DEBUG_BW_END_VERSION + // updater for a single granule // TODO: this is getting kind of large. Should try to split out this actor if it continues to grow? // FIXME: handle errors here (forward errors) @@ -1302,8 +1308,15 @@ ACTOR Future blobGranuleUpdateFiles(Reference bwData, state Standalone> mutations; try { + if (DEBUG_BW_VERSION(metadata->bufferedDeltaVersion)) { + fmt::print("BW waiting mutations after ({0})\n", metadata->bufferedDeltaVersion); + } state Standalone> _mutations = waitNext(metadata->activeCFData.get()->mutations.getFuture()); + if (DEBUG_BW_VERSION(metadata->bufferedDeltaVersion)) { + fmt::print( + "BW got mutations after ({0}): ({1})\n", metadata->bufferedDeltaVersion, _mutations.size()); + } mutations = _mutations; if (readOldChangeFeed) { ASSERT(mutations.back().version < startState.changeFeedStartVersion); @@ -1442,6 +1455,9 @@ ACTOR Future blobGranuleUpdateFiles(Reference bwData, .detail("ChangeFeedID", readOldChangeFeed ? oldCFKey.get() : cfKey) .detail("OldChangeFeed", readOldChangeFeed ? "T" : "F"); } + if (DEBUG_BW_VERSION(deltas.version)) { + fmt::print("BW {0}: ({1})\n", deltas.version, deltas.mutations.size()); + } metadata->currentDeltas.push_back_deep(metadata->deltaArena, deltas); } } @@ -1790,17 +1806,20 @@ ACTOR Future waitForVersion(Reference metadata, Version v // if we don't have to wait for change feed version to catch up or wait for any pending file writes to complete, // nothing to do - /*printf(" [%s - %s) waiting for %lld\n readable:%s\n bufferedDelta=%lld\n pendingDelta=%lld\n " - "durableDelta=%lld\n pendingSnapshot=%lld\n durableSnapshot=%lld\n", - metadata->keyRange.begin.printable().c_str(), - metadata->keyRange.end.printable().c_str(), - v, - metadata->readable.isSet() ? "T" : "F", - metadata->activeCFData.get()->getVersion(), - metadata->pendingDeltaVersion, - metadata->durableDeltaVersion.get(), - metadata->pendingSnapshotVersion, - metadata->durableSnapshotVersion.get());*/ + if (v == DEBUG_BW_WAIT_VERSION) { + fmt::print("{0}) [{1} - {2}) waiting for {3}\n readable:{4}\n bufferedDelta={5}\n pendingDelta={6}\n " + "durableDelta={7}\n pendingSnapshot={8}\n durableSnapshot={9}\n", + v, + metadata->keyRange.begin.printable().c_str(), + metadata->keyRange.end.printable().c_str(), + v, + metadata->readable.isSet() ? "T" : "F", + metadata->activeCFData.get()->getVersion(), + metadata->pendingDeltaVersion, + metadata->durableDeltaVersion.get(), + metadata->pendingSnapshotVersion, + metadata->durableSnapshotVersion.get()); + } ASSERT(metadata->activeCFData.get().isValid()); @@ -1809,13 +1828,24 @@ ACTOR Future waitForVersion(Reference metadata, Version v metadata->durableDeltaVersion.get() == metadata->pendingDeltaVersion) && (v <= metadata->durableSnapshotVersion.get() || metadata->durableSnapshotVersion.get() == metadata->pendingSnapshotVersion)) { + if (v == DEBUG_BW_WAIT_VERSION) { + fmt::print("{0}) already done\n", v); + } return Void(); } // wait for change feed version to catch up to ensure we have all data if (metadata->activeCFData.get()->getVersion() < v) { + if (v == DEBUG_BW_WAIT_VERSION) { + fmt::print("{0}) waiting for CF version (currently {1})\n", v, metadata->activeCFData.get()->getVersion()); + } + wait(metadata->activeCFData.get()->whenAtLeast(v)); ASSERT(metadata->activeCFData.get()->getVersion() >= v); + + if (v == DEBUG_BW_WAIT_VERSION) { + fmt::print("{0}) got CF version {1}\n", v, metadata->activeCFData.get()->getVersion()); + } } // wait for any pending delta and snapshot files as of the moment the change feed version caught up. @@ -1825,15 +1855,32 @@ ACTOR Future waitForVersion(Reference metadata, Version v // If there are mutations that are no longer buffered but have not been // persisted to a delta file that are necessary for the query, wait for them if (pendingDeltaV > metadata->durableDeltaVersion.get() && v > metadata->durableDeltaVersion.get()) { + if (v == DEBUG_BW_WAIT_VERSION) { + fmt::print("{0}) waiting for DDV {1} < {2}\n", v, metadata->durableDeltaVersion.get(), pendingDeltaV); + } + wait(metadata->durableDeltaVersion.whenAtLeast(pendingDeltaV)); ASSERT(metadata->durableDeltaVersion.get() >= pendingDeltaV); + + if (v == DEBUG_BW_WAIT_VERSION) { + fmt::print("{0}) waiting for DDV {1} >= {2}\n", v, metadata->durableDeltaVersion.get(), pendingDeltaV); + } } // This isn't strictly needed, but if we're in the process of re-snapshotting, we'd likely rather // return that snapshot file than the previous snapshot file and all its delta files. if (pendingSnapshotV > metadata->durableSnapshotVersion.get() && v > metadata->durableSnapshotVersion.get()) { + if (v == DEBUG_BW_WAIT_VERSION) { + fmt::print("{0}) waiting for DSV {1} < {2}\n", v, metadata->durableSnapshotVersion.get(), pendingSnapshotV); + } + wait(metadata->durableSnapshotVersion.whenAtLeast(pendingSnapshotV)); ASSERT(metadata->durableSnapshotVersion.get() >= pendingSnapshotV); + + if (v == DEBUG_BW_WAIT_VERSION) { + fmt::print( + "{0}) waiting for DSV {1} >= {2}\n", v, metadata->durableSnapshotVersion.get(), pendingSnapshotV); + } } // There is a race here - we wait for pending delta files before this to finish, but while we do, we @@ -1841,8 +1888,20 @@ ACTOR Future waitForVersion(Reference metadata, Version v // file instead of in memory mutations, so we wait for that delta file to complete if (metadata->pendingDeltaVersion > v) { + if (v == DEBUG_BW_WAIT_VERSION) { + fmt::print("{0}) waiting for DDV again {1} < {2}\n", v, metadata->durableDeltaVersion.get(), v); + } + wait(metadata->durableDeltaVersion.whenAtLeast(v)); ASSERT(metadata->durableDeltaVersion.get() >= v); + + if (v == DEBUG_BW_WAIT_VERSION) { + fmt::print("{0}) waiting for DDV again {1} >= {2}\n", v, metadata->durableDeltaVersion.get(), v); + } + } + + if (v == DEBUG_BW_WAIT_VERSION) { + fmt::print("{0}) done\n", v); } return Void(); diff --git a/fdbserver/workloads/BlobGranuleVerifier.actor.cpp b/fdbserver/workloads/BlobGranuleVerifier.actor.cpp index 2bc2709340..66333ddde4 100644 --- a/fdbserver/workloads/BlobGranuleVerifier.actor.cpp +++ b/fdbserver/workloads/BlobGranuleVerifier.actor.cpp @@ -34,7 +34,7 @@ #include "flow/actorcompiler.h" // This must be the last #include. -#define BGV_DEBUG false +#define BGV_DEBUG true /* * This workload is designed to verify the correctness of the blob data produced by the blob workers. From ef74d309d8b11c312bb9cddc22131c7c3a21800a Mon Sep 17 00:00:00 2001 From: Josh Slocum Date: Wed, 8 Dec 2021 12:48:30 -0600 Subject: [PATCH 054/413] Adding delay after onEmpty for merge cursor as well --- fdbclient/NativeAPI.actor.cpp | 2 ++ 1 file changed, 2 insertions(+) diff --git a/fdbclient/NativeAPI.actor.cpp b/fdbclient/NativeAPI.actor.cpp index 9f98db20e5..667f984bc4 100644 --- a/fdbclient/NativeAPI.actor.cpp +++ b/fdbclient/NativeAPI.actor.cpp @@ -7322,6 +7322,7 @@ ACTOR Future mergeChangeFeedStream(Reference db, ASSERT(nextOut.back().version >= results->lastReturnedVersion.get()); results->mutations.send(nextOut); wait(results->mutations.onEmpty()); + wait(delay(0)); if (DEBUG_CF_VERSION(nextOut.back().version)) { fmt::print("CFLR (merged): {0} (1)\n", nextOut.back().version); } @@ -7355,6 +7356,7 @@ ACTOR Future mergeChangeFeedStream(Reference db, ASSERT(nextOut.back().version >= results->lastReturnedVersion.get()); results->mutations.send(nextOut); wait(results->mutations.onEmpty()); + wait(delay(0)); if (DEBUG_CF_VERSION(nextOut.back().version)) { fmt::print("CFLR (merged): {0} (1)\n", nextOut.back().version); } From b438e9822044400960e656457b2ca37c5630cbb2 Mon Sep 17 00:00:00 2001 From: Josh Slocum Date: Wed, 8 Dec 2021 15:47:16 -0600 Subject: [PATCH 055/413] Fixed multiple availability bugs in blob worker --- fdbserver/BlobWorker.actor.cpp | 84 +++++++++++-------- .../workloads/BlobGranuleVerifier.actor.cpp | 2 +- 2 files changed, 51 insertions(+), 35 deletions(-) diff --git a/fdbserver/BlobWorker.actor.cpp b/fdbserver/BlobWorker.actor.cpp index 0dca8481aa..36bd59654e 100644 --- a/fdbserver/BlobWorker.actor.cpp +++ b/fdbserver/BlobWorker.actor.cpp @@ -1311,17 +1311,32 @@ ACTOR Future blobGranuleUpdateFiles(Reference bwData, if (DEBUG_BW_VERSION(metadata->bufferedDeltaVersion)) { fmt::print("BW waiting mutations after ({0})\n", metadata->bufferedDeltaVersion); } - state Standalone> _mutations = - waitNext(metadata->activeCFData.get()->mutations.getFuture()); - if (DEBUG_BW_VERSION(metadata->bufferedDeltaVersion)) { - fmt::print( - "BW got mutations after ({0}): ({1})\n", metadata->bufferedDeltaVersion, _mutations.size()); - } - mutations = _mutations; - if (readOldChangeFeed) { - ASSERT(mutations.back().version < startState.changeFeedStartVersion); - } else { - ASSERT(mutations.front().version >= startState.changeFeedStartVersion); + // Even if there are no new mutations, there still might be readers waiting on durableDeltaVersion + // to advance. We need to check whether any outstanding files have finished so we don't wait on + // mutations forever + choose { + when(Standalone> _mutations = + waitNext(metadata->activeCFData.get()->mutations.getFuture())) { + if (DEBUG_BW_VERSION(metadata->bufferedDeltaVersion)) { + fmt::print("BW got mutations after ({0}): ({1})\n", + metadata->bufferedDeltaVersion, + _mutations.size()); + } + mutations = _mutations; + ASSERT(!mutations.empty()); + if (readOldChangeFeed) { + ASSERT(mutations.back().version < startState.changeFeedStartVersion); + } else { + ASSERT(mutations.front().version >= startState.changeFeedStartVersion); + } + } + when(wait(inFlightFiles.empty() ? Never() : success(inFlightFiles.front().future))) { + // TODO REMOVE + if (DEBUG_BW_VERSION(metadata->bufferedDeltaVersion)) { + fmt::print("BW got file before waiting for mutations after {0}\n", + metadata->bufferedDeltaVersion); + } + } } } catch (Error& e) { // only error we should expect here is when we finish consuming old change feed @@ -1354,8 +1369,8 @@ ACTOR Future blobGranuleUpdateFiles(Reference bwData, for (MutationsAndVersionRef d : mutations) { state MutationsAndVersionRef deltas = d; - // buffer mutations at this version. There should not be multiple MutationsAndVersionRef with the same - // version + // buffer mutations at this version. There should not be multiple MutationsAndVersionRef with the + // same version ASSERT(deltas.version > metadata->bufferedDeltaVersion); if (!deltas.mutations.empty()) { if (deltas.mutations.size() == 1 && deltas.mutations.back().param1 == lastEpochEndPrivateKey) { @@ -1380,8 +1395,8 @@ ACTOR Future blobGranuleUpdateFiles(Reference bwData, rollbacksInProgress.pop_front(); } else { // FIXME: add counter for granule rollbacks and rollbacks skipped? - // explicitly check last delta in currentDeltas because lastVersion and bufferedDeltaVersion - // include empties + // explicitly check last delta in currentDeltas because lastVersion and + // bufferedDeltaVersion include empties if (metadata->pendingDeltaVersion <= rollbackVersion && (metadata->currentDeltas.empty() || metadata->currentDeltas.back().version <= rollbackVersion)) { @@ -1415,8 +1430,8 @@ ACTOR Future blobGranuleUpdateFiles(Reference bwData, metadata->activeCFData.set(newChangeFeedData(cfRollbackVersion)); if (readOldChangeFeed) { // It shouldn't be possible to roll back across the parent/child feed boundary, - // because the transaction creating the child change feed had to commit before we - // got here. + // because the transaction creating the child change feed had to commit before + // we got here. ASSERT(cfRollbackVersion < startState.changeFeedStartVersion); oldChangeFeedFuture = bwData->db->getChangeFeedStream(metadata->activeCFData.get(), @@ -1469,10 +1484,11 @@ ACTOR Future blobGranuleUpdateFiles(Reference bwData, metadata->bufferedDeltaVersion = deltas.version; Version knownNoRollbacksPast = std::min(deltas.version, deltas.knownCommittedVersion); if (knownNoRollbacksPast > committedVersion.get()) { - // This is the only place it is safe to set committedVersion, as it has to come from the mutation - // stream, or we could have a situation where the blob worker has consumed an uncommitted mutation, - // but not its rollback, from the change feed, and could thus think the uncommitted mutation is - // committed because it saw a higher committed version than the mutation's version. + // This is the only place it is safe to set committedVersion, as it has to come from the + // mutation stream, or we could have a situation where the blob worker has consumed an + // uncommitted mutation, but not its rollback, from the change feed, and could thus think the + // uncommitted mutation is committed because it saw a higher committed version than the + // mutation's version. committedVersion.set(knownNoRollbacksPast); } @@ -1551,8 +1567,8 @@ ACTOR Future blobGranuleUpdateFiles(Reference bwData, } // Speculatively assume we will get the range back. This is both a performance optimization, and - // necessary to keep consuming committed versions from the change feed so that we can realize our - // last delta file is committed and write it + // necessary to keep consuming committed versions from the change feed so that we can realize + // our last delta file is committed and write it Future previousFuture; if (!inFlightFiles.empty()) { @@ -1581,12 +1597,12 @@ ACTOR Future blobGranuleUpdateFiles(Reference bwData, if (f.snapshot && f.version < metadata->pendingSnapshotVersion && f.version <= committedVersion.get()) { if (BW_DEBUG) { - fmt::print( - "[{0} - {1}) Waiting on previous snapshot file @ {2} <= known committed {3}\n", - metadata->keyRange.begin.printable(), - metadata->keyRange.end.printable(), - f.version, - committedVersion.get()); + fmt::print("[{0} - {1}) Waiting on previous snapshot file @ {2} <= known " + "committed {3}\n", + metadata->keyRange.begin.printable(), + metadata->keyRange.end.printable(), + f.version, + committedVersion.get()); } waitIdx = idx + 1; } @@ -1622,8 +1638,8 @@ ACTOR Future blobGranuleUpdateFiles(Reference bwData, } else if (snapshotEligible && metadata->bytesInNewDeltaFiles >= SERVER_KNOBS->BG_DELTA_BYTES_BEFORE_COMPACT) { // if we're in the old change feed case and can't snapshot but we have enough data to, don't - // queue too many files in parallel, and slow down change feed consuming to let file writing catch - // up + // queue too many files in parallel, and slow down change feed consuming to let file writing + // catch up if (inFlightFiles.size() > 10) { if (BW_DEBUG) { printf("[%s - %s) Waiting on delta file b/c old change feed\n", @@ -1706,8 +1722,8 @@ ACTOR Future blobGranuleLoadHistory(Reference bwData, state std::vector> historyEntryStack; - // while the start version of the current granule's parent is larger than the last known start version, walk - // backwards + // while the start version of the current granule's parent is larger than the last known start version, + // walk backwards while (curHistory.value.parentGranules.size() > 0 && curHistory.value.parentGranules[0].second > stopVersion) { state GranuleHistory next; @@ -2164,7 +2180,7 @@ ACTOR Future handleBlobGranuleFileRequest(Reference bwData req.reply.send(rep); --bwData->stats.activeReadRequests; } catch (Error& e) { - // printf("Error in BGFRequest %s\n", e.name()); + // fmt::print("Error in BGFRequest {0}\n", e.name()); if (e.code() == error_code_operation_cancelled) { req.reply.sendError(wrong_shard_server()); throw; diff --git a/fdbserver/workloads/BlobGranuleVerifier.actor.cpp b/fdbserver/workloads/BlobGranuleVerifier.actor.cpp index 66333ddde4..bdd424c687 100644 --- a/fdbserver/workloads/BlobGranuleVerifier.actor.cpp +++ b/fdbserver/workloads/BlobGranuleVerifier.actor.cpp @@ -402,7 +402,6 @@ struct BlobGranuleVerifierWorkload : TestWorkload { state Standalone> allRanges = self->granuleRanges.get(); for (auto& range : allRanges) { state KeyRange r = range; - state PromiseStream> chunkStream; if (BGV_DEBUG) { fmt::print("Final availability check [{0} - {1}) @ {2}\n", r.begin.printable(), @@ -419,6 +418,7 @@ struct BlobGranuleVerifierWorkload : TestWorkload { ASSERT(chunks.size() > 0); last = chunks.back().keyRange; checks += chunks.size(); + break; } catch (Error& e) { wait(tr.onError(e)); } From e9e5a80086bf5f8a61b777d407279a87ffb96545 Mon Sep 17 00:00:00 2001 From: Josh Slocum Date: Wed, 8 Dec 2021 16:10:29 -0600 Subject: [PATCH 056/413] Refactored file writing to guarantee consuming all mutations from the change feed promise stream --- fdbserver/BlobWorker.actor.cpp | 243 ++++++++++++++++++--------------- 1 file changed, 130 insertions(+), 113 deletions(-) diff --git a/fdbserver/BlobWorker.actor.cpp b/fdbserver/BlobWorker.actor.cpp index 36bd59654e..6b3ea603f1 100644 --- a/fdbserver/BlobWorker.actor.cpp +++ b/fdbserver/BlobWorker.actor.cpp @@ -1366,131 +1366,149 @@ ACTOR Future blobGranuleUpdateFiles(Reference bwData, } // process mutations - for (MutationsAndVersionRef d : mutations) { - state MutationsAndVersionRef deltas = d; + if (!mutations.empty()) { + bool processedAnyMutations = false; + Version knownNoRollbacksPast = invalidVersion; + Version lastDeltaVersion = invalidVersion; + for (MutationsAndVersionRef deltas : mutations) { - // buffer mutations at this version. There should not be multiple MutationsAndVersionRef with the - // same version - ASSERT(deltas.version > metadata->bufferedDeltaVersion); - if (!deltas.mutations.empty()) { - if (deltas.mutations.size() == 1 && deltas.mutations.back().param1 == lastEpochEndPrivateKey) { - // Note rollbackVerision is durable, [rollbackVersion+1 - deltas.version] needs to be tossed - // For correctness right now, there can be no waits and yields either in rollback handling - // or in handleBlobGranuleFileRequest once waitForVersion has succeeded, otherwise this will - // race and clobber results - Version rollbackVersion; - BinaryReader br(deltas.mutations[0].param2, Unversioned()); - br >> rollbackVersion; + // Buffer mutations at this version. There should not be multiple MutationsAndVersionRef with the + // same version + ASSERT(deltas.version > metadata->bufferedDeltaVersion); + ASSERT(deltas.version > lastDeltaVersion); + if (!deltas.mutations.empty()) { + if (deltas.mutations.size() == 1 && deltas.mutations.back().param1 == lastEpochEndPrivateKey) { + // Note rollbackVerision is durable, [rollbackVersion+1 - deltas.version] needs to be tossed + // For correctness right now, there can be no waits and yields either in rollback handling + // or in handleBlobGranuleFileRequest once waitForVersion has succeeded, otherwise this will + // race and clobber results + Version rollbackVersion; + BinaryReader br(deltas.mutations[0].param2, Unversioned()); + br >> rollbackVersion; - ASSERT(rollbackVersion >= metadata->durableDeltaVersion.get()); - ASSERT(rollbackVersion >= committedVersion.get()); - - if (!rollbacksInProgress.empty()) { - ASSERT(rollbacksInProgress.front().first == rollbackVersion); - ASSERT(rollbacksInProgress.front().second == deltas.version); - if (BW_DEBUG) { - fmt::print("Passed rollback {0} -> {1}\n", deltas.version, rollbackVersion); - } - rollbacksCompleted.push_back(rollbacksInProgress.front()); - rollbacksInProgress.pop_front(); - } else { - // FIXME: add counter for granule rollbacks and rollbacks skipped? - // explicitly check last delta in currentDeltas because lastVersion and - // bufferedDeltaVersion include empties - if (metadata->pendingDeltaVersion <= rollbackVersion && - (metadata->currentDeltas.empty() || - metadata->currentDeltas.back().version <= rollbackVersion)) { + ASSERT(rollbackVersion >= metadata->durableDeltaVersion.get()); + ASSERT(rollbackVersion >= committedVersion.get()); + if (!rollbacksInProgress.empty()) { + ASSERT(rollbacksInProgress.front().first == rollbackVersion); + ASSERT(rollbacksInProgress.front().second == deltas.version); if (BW_DEBUG) { - fmt::print("BW skipping rollback {0} -> {1} completely\n", - deltas.version, - rollbackVersion); + fmt::print("Passed rollback {0} -> {1}\n", deltas.version, rollbackVersion); } + rollbacksCompleted.push_back(rollbacksInProgress.front()); + rollbacksInProgress.pop_front(); } else { - if (BW_DEBUG) { - fmt::print("BW [{0} - {1}) ROLLBACK @ {2} -> {3}\n", - metadata->keyRange.begin.printable(), - metadata->keyRange.end.printable(), - deltas.version, - rollbackVersion); - TraceEvent(SevWarn, "GranuleRollback", bwData->id) - .detail("Granule", metadata->keyRange) - .detail("Version", deltas.version) - .detail("RollbackVersion", rollbackVersion); - } - - Version cfRollbackVersion = doGranuleRollback(metadata, - deltas.version, - rollbackVersion, - inFlightFiles, - rollbacksInProgress, - rollbacksCompleted); - - // Reset change feeds to cfRollbackVersion - metadata->activeCFData.set(newChangeFeedData(cfRollbackVersion)); - if (readOldChangeFeed) { - // It shouldn't be possible to roll back across the parent/child feed boundary, - // because the transaction creating the child change feed had to commit before - // we got here. - ASSERT(cfRollbackVersion < startState.changeFeedStartVersion); - oldChangeFeedFuture = - bwData->db->getChangeFeedStream(metadata->activeCFData.get(), - oldCFKey.get(), - cfRollbackVersion + 1, - startState.changeFeedStartVersion, - metadata->keyRange); + // FIXME: add counter for granule rollbacks and rollbacks skipped? + // explicitly check last delta in currentDeltas because lastVersion and + // bufferedDeltaVersion include empties + if (metadata->pendingDeltaVersion <= rollbackVersion && + (metadata->currentDeltas.empty() || + metadata->currentDeltas.back().version <= rollbackVersion)) { + if (BW_DEBUG) { + fmt::print("BW skipping rollback {0} -> {1} completely\n", + deltas.version, + rollbackVersion); + } } else { - ASSERT(cfRollbackVersion > startState.changeFeedStartVersion); + if (BW_DEBUG) { + fmt::print("BW [{0} - {1}) ROLLBACK @ {2} -> {3}\n", + metadata->keyRange.begin.printable(), + metadata->keyRange.end.printable(), + deltas.version, + rollbackVersion); + TraceEvent(SevWarn, "GranuleRollback", bwData->id) + .detail("Granule", metadata->keyRange) + .detail("Version", deltas.version) + .detail("RollbackVersion", rollbackVersion); + } - changeFeedFuture = bwData->db->getChangeFeedStream(metadata->activeCFData.get(), - cfKey, - cfRollbackVersion + 1, - MAX_VERSION, - metadata->keyRange); + Version cfRollbackVersion = doGranuleRollback(metadata, + deltas.version, + rollbackVersion, + inFlightFiles, + rollbacksInProgress, + rollbacksCompleted); + + // Reset change feeds to cfRollbackVersion + metadata->activeCFData.set(newChangeFeedData(cfRollbackVersion)); + if (readOldChangeFeed) { + // It shouldn't be possible to roll back across the parent/child feed boundary, + // because the transaction creating the child change feed had to commit before + // we got here. + ASSERT(cfRollbackVersion < startState.changeFeedStartVersion); + oldChangeFeedFuture = + bwData->db->getChangeFeedStream(metadata->activeCFData.get(), + oldCFKey.get(), + cfRollbackVersion + 1, + startState.changeFeedStartVersion, + metadata->keyRange); + + } else { + ASSERT(cfRollbackVersion > startState.changeFeedStartVersion); + + changeFeedFuture = bwData->db->getChangeFeedStream(metadata->activeCFData.get(), + cfKey, + cfRollbackVersion + 1, + MAX_VERSION, + metadata->keyRange); + } + + justDidRollback = true; + break; } - - justDidRollback = true; - break; } - } - } else if (!rollbacksInProgress.empty() && rollbacksInProgress.front().first < deltas.version && - rollbacksInProgress.front().second > deltas.version) { - if (BW_DEBUG) { - fmt::print("Skipping mutations @ {} b/c prior rollback\n", deltas.version); - } - } else { - for (auto& delta : deltas.mutations) { - metadata->bufferedDeltaBytes += delta.totalSize(); - bwData->stats.changeFeedInputBytes += delta.totalSize(); - bwData->stats.mutationBytesBuffered += delta.totalSize(); + } else if (!rollbacksInProgress.empty() && rollbacksInProgress.front().first < deltas.version && + rollbacksInProgress.front().second > deltas.version) { + if (BW_DEBUG) { + fmt::print("Skipping mutations @ {} b/c prior rollback\n", deltas.version); + } + } else { + for (auto& delta : deltas.mutations) { + metadata->bufferedDeltaBytes += delta.totalSize(); + bwData->stats.changeFeedInputBytes += delta.totalSize(); + bwData->stats.mutationBytesBuffered += delta.totalSize(); - DEBUG_MUTATION("BlobWorkerBuffer", deltas.version, delta, bwData->id) - .detail("Granule", metadata->keyRange) - .detail("ChangeFeedID", readOldChangeFeed ? oldCFKey.get() : cfKey) - .detail("OldChangeFeed", readOldChangeFeed ? "T" : "F"); + DEBUG_MUTATION("BlobWorkerBuffer", deltas.version, delta, bwData->id) + .detail("Granule", metadata->keyRange) + .detail("ChangeFeedID", readOldChangeFeed ? oldCFKey.get() : cfKey) + .detail("OldChangeFeed", readOldChangeFeed ? "T" : "F"); + } + if (DEBUG_BW_VERSION(deltas.version)) { + fmt::print("BW {0}: ({1})\n", deltas.version, deltas.mutations.size()); + } + metadata->currentDeltas.push_back_deep(metadata->deltaArena, deltas); + + processedAnyMutations = true; + lastDeltaVersion = deltas.version; + + Version nextKnownNoRollbacksPast = std::min(deltas.version, deltas.knownCommittedVersion); + ASSERT(nextKnownNoRollbacksPast >= knownNoRollbacksPast); + knownNoRollbacksPast = nextKnownNoRollbacksPast; } - if (DEBUG_BW_VERSION(deltas.version)) { - fmt::print("BW {0}: ({1})\n", deltas.version, deltas.mutations.size()); - } - metadata->currentDeltas.push_back_deep(metadata->deltaArena, deltas); + } + if (justDidRollback) { + break; } } - if (justDidRollback) { - break; - } + if (!justDidRollback && processedAnyMutations) { + // update buffered version and committed version + ASSERT(lastDeltaVersion != invalidVersion); + ASSERT(knownNoRollbacksPast != invalidVersion); + ASSERT(lastDeltaVersion > metadata->bufferedDeltaVersion); + ASSERT(knownNoRollbacksPast >= committedVersion.get()); - // update buffered version and committed version - metadata->bufferedDeltaVersion = deltas.version; - Version knownNoRollbacksPast = std::min(deltas.version, deltas.knownCommittedVersion); - if (knownNoRollbacksPast > committedVersion.get()) { + metadata->bufferedDeltaVersion = lastDeltaVersion; // This is the only place it is safe to set committedVersion, as it has to come from the // mutation stream, or we could have a situation where the blob worker has consumed an - // uncommitted mutation, but not its rollback, from the change feed, and could thus think the - // uncommitted mutation is committed because it saw a higher committed version than the - // mutation's version. + // uncommitted mutation, but not its rollback, from the change feed, and could thus + // think the uncommitted mutation is committed because it saw a higher committed version + // than the mutation's version. + // We also can only set it after consuming all of the mutations from the vector from the promise + // stream, as yielding when consuming from a change feed can cause bugs committedVersion.set(knownNoRollbacksPast); } + justDidRollback = false; // Write a new delta file IF we have enough bytes if (metadata->bufferedDeltaBytes >= SERVER_KNOBS->BG_DELTA_FILE_TARGET_BYTES) { @@ -1499,15 +1517,15 @@ ACTOR Future blobGranuleUpdateFiles(Reference bwData, metadata->keyRange.begin.printable(), metadata->keyRange.end.printable(), metadata->bufferedDeltaBytes, - deltas.version, + lastDeltaVersion, oldChangeFeedDataComplete.present() ? ". Finalizing " : ""); } TraceEvent("BlobGranuleDeltaFile", bwData->id) .detail("Granule", metadata->keyRange) - .detail("Version", deltas.version); + .detail("Version", lastDeltaVersion); // sanity check for version order - ASSERT(deltas.version >= metadata->currentDeltas.back().version); + ASSERT(lastDeltaVersion >= metadata->currentDeltas.back().version); ASSERT(metadata->pendingDeltaVersion < metadata->currentDeltas.front().version); // launch pipelined, but wait for previous operation to complete before persisting to FDB @@ -1524,17 +1542,17 @@ ACTOR Future blobGranuleUpdateFiles(Reference bwData, metadata->originalSeqno, metadata->deltaArena, metadata->currentDeltas, - deltas.version, + lastDeltaVersion, previousFuture, &committedVersion, oldChangeFeedDataComplete); inFlightFiles.push_back( - InFlightFile(dfFuture, deltas.version, metadata->bufferedDeltaBytes, false)); + InFlightFile(dfFuture, lastDeltaVersion, metadata->bufferedDeltaBytes, false)); oldChangeFeedDataComplete.reset(); // add new pending delta file - ASSERT(metadata->pendingDeltaVersion < deltas.version); - metadata->pendingDeltaVersion = deltas.version; + ASSERT(metadata->pendingDeltaVersion < lastDeltaVersion); + metadata->pendingDeltaVersion = lastDeltaVersion; metadata->bytesInNewDeltaFiles += metadata->bufferedDeltaBytes; bwData->stats.mutationBytesBuffered -= metadata->bufferedDeltaBytes; @@ -1654,7 +1672,6 @@ ACTOR Future blobGranuleUpdateFiles(Reference bwData, } snapshotEligible = false; } - justDidRollback = false; } } catch (Error& e) { // TODO REMOVE From 5f2640f5921b8342ac25b86d5a6ff0c269e3a288 Mon Sep 17 00:00:00 2001 From: Josh Slocum Date: Wed, 8 Dec 2021 16:40:40 -0600 Subject: [PATCH 057/413] Fixed an assert I added during refactor that was incorrect --- fdbserver/BlobWorker.actor.cpp | 19 +++++++++++++------ 1 file changed, 13 insertions(+), 6 deletions(-) diff --git a/fdbserver/BlobWorker.actor.cpp b/fdbserver/BlobWorker.actor.cpp index 6b3ea603f1..cef434931c 100644 --- a/fdbserver/BlobWorker.actor.cpp +++ b/fdbserver/BlobWorker.actor.cpp @@ -1137,8 +1137,8 @@ static Version doGranuleRollback(Reference metadata, } // TODO REMOVE once correctness clean -#define DEBUG_BW_START_VERSION invalidVersion -#define DEBUG_BW_END_VERSION invalidVersion +#define DEBUG_BW_START_VERSION 42629425 +#define DEBUG_BW_END_VERSION 48085149 #define DEBUG_BW_WAIT_VERSION invalidVersion #define DEBUG_BW_VERSION(v) DEBUG_BW_START_VERSION <= v&& v <= DEBUG_BW_END_VERSION @@ -1475,7 +1475,10 @@ ACTOR Future blobGranuleUpdateFiles(Reference bwData, .detail("OldChangeFeed", readOldChangeFeed ? "T" : "F"); } if (DEBUG_BW_VERSION(deltas.version)) { - fmt::print("BW {0}: ({1})\n", deltas.version, deltas.mutations.size()); + fmt::print("BW {0}: ({1}), KCV={2}\n", + deltas.version, + deltas.mutations.size(), + deltas.knownCommittedVersion); } metadata->currentDeltas.push_back_deep(metadata->deltaArena, deltas); @@ -1496,17 +1499,21 @@ ACTOR Future blobGranuleUpdateFiles(Reference bwData, ASSERT(lastDeltaVersion != invalidVersion); ASSERT(knownNoRollbacksPast != invalidVersion); ASSERT(lastDeltaVersion > metadata->bufferedDeltaVersion); - ASSERT(knownNoRollbacksPast >= committedVersion.get()); + // Update buffered delta version so new waitForVersion checks can bypass waiting entirely metadata->bufferedDeltaVersion = lastDeltaVersion; + // This is the only place it is safe to set committedVersion, as it has to come from the // mutation stream, or we could have a situation where the blob worker has consumed an // uncommitted mutation, but not its rollback, from the change feed, and could thus // think the uncommitted mutation is committed because it saw a higher committed version // than the mutation's version. // We also can only set it after consuming all of the mutations from the vector from the promise - // stream, as yielding when consuming from a change feed can cause bugs - committedVersion.set(knownNoRollbacksPast); + // stream, as yielding when consuming from a change feed can cause bugs if this wakes up one of the + // file writers + if (knownNoRollbacksPast > committedVersion.get()) { + committedVersion.set(knownNoRollbacksPast); + } } justDidRollback = false; From 845f1ade428e0933dc8985d8c3fd2d77d18dada3 Mon Sep 17 00:00:00 2001 From: Josh Slocum Date: Wed, 8 Dec 2021 17:19:28 -0600 Subject: [PATCH 058/413] Another incorrect assert --- fdbserver/BlobWorker.actor.cpp | 42 +++++++++++++++++++--------------- 1 file changed, 24 insertions(+), 18 deletions(-) diff --git a/fdbserver/BlobWorker.actor.cpp b/fdbserver/BlobWorker.actor.cpp index cef434931c..28275afb32 100644 --- a/fdbserver/BlobWorker.actor.cpp +++ b/fdbserver/BlobWorker.actor.cpp @@ -1137,8 +1137,8 @@ static Version doGranuleRollback(Reference metadata, } // TODO REMOVE once correctness clean -#define DEBUG_BW_START_VERSION 42629425 -#define DEBUG_BW_END_VERSION 48085149 +#define DEBUG_BW_START_VERSION invalidVersion +#define DEBUG_BW_END_VERSION invalidVersion #define DEBUG_BW_WAIT_VERSION invalidVersion #define DEBUG_BW_VERSION(v) DEBUG_BW_START_VERSION <= v&& v <= DEBUG_BW_END_VERSION @@ -1308,20 +1308,20 @@ ACTOR Future blobGranuleUpdateFiles(Reference bwData, state Standalone> mutations; try { - if (DEBUG_BW_VERSION(metadata->bufferedDeltaVersion)) { - fmt::print("BW waiting mutations after ({0})\n", metadata->bufferedDeltaVersion); - } + /*if (DEBUG_BW_VERSION(metadata->bufferedDeltaVersion)) { + fmt::print("BW waiting mutations after ({0})\n", metadata->bufferedDeltaVersion); + }*/ // Even if there are no new mutations, there still might be readers waiting on durableDeltaVersion // to advance. We need to check whether any outstanding files have finished so we don't wait on // mutations forever choose { when(Standalone> _mutations = waitNext(metadata->activeCFData.get()->mutations.getFuture())) { - if (DEBUG_BW_VERSION(metadata->bufferedDeltaVersion)) { - fmt::print("BW got mutations after ({0}): ({1})\n", - metadata->bufferedDeltaVersion, - _mutations.size()); - } + /*if (DEBUG_BW_VERSION(metadata->bufferedDeltaVersion)) { + fmt::print("BW got mutations after ({0}): ({1})\n", + metadata->bufferedDeltaVersion, + _mutations.size()); + }*/ mutations = _mutations; ASSERT(!mutations.empty()); if (readOldChangeFeed) { @@ -1332,10 +1332,10 @@ ACTOR Future blobGranuleUpdateFiles(Reference bwData, } when(wait(inFlightFiles.empty() ? Never() : success(inFlightFiles.front().future))) { // TODO REMOVE - if (DEBUG_BW_VERSION(metadata->bufferedDeltaVersion)) { - fmt::print("BW got file before waiting for mutations after {0}\n", - metadata->bufferedDeltaVersion); - } + /*if (DEBUG_BW_VERSION(metadata->bufferedDeltaVersion)) { + fmt::print("BW got file before waiting for mutations after {0}\n", + metadata->bufferedDeltaVersion); + }*/ } } } catch (Error& e) { @@ -1483,11 +1483,18 @@ ACTOR Future blobGranuleUpdateFiles(Reference bwData, metadata->currentDeltas.push_back_deep(metadata->deltaArena, deltas); processedAnyMutations = true; + ASSERT(deltas.version != invalidVersion); + ASSERT(deltas.version > lastDeltaVersion); lastDeltaVersion = deltas.version; Version nextKnownNoRollbacksPast = std::min(deltas.version, deltas.knownCommittedVersion); - ASSERT(nextKnownNoRollbacksPast >= knownNoRollbacksPast); - knownNoRollbacksPast = nextKnownNoRollbacksPast; + // FIXME: apparently we can have invalidVersion for knownCommittedVersion + ASSERT(nextKnownNoRollbacksPast >= knownNoRollbacksPast || + nextKnownNoRollbacksPast == invalidVersion); + + if (nextKnownNoRollbacksPast != invalidVersion) { + knownNoRollbacksPast = nextKnownNoRollbacksPast; + } } } if (justDidRollback) { @@ -1497,7 +1504,6 @@ ACTOR Future blobGranuleUpdateFiles(Reference bwData, if (!justDidRollback && processedAnyMutations) { // update buffered version and committed version ASSERT(lastDeltaVersion != invalidVersion); - ASSERT(knownNoRollbacksPast != invalidVersion); ASSERT(lastDeltaVersion > metadata->bufferedDeltaVersion); // Update buffered delta version so new waitForVersion checks can bypass waiting entirely @@ -1505,7 +1511,7 @@ ACTOR Future blobGranuleUpdateFiles(Reference bwData, // This is the only place it is safe to set committedVersion, as it has to come from the // mutation stream, or we could have a situation where the blob worker has consumed an - // uncommitted mutation, but not its rollback, from the change feed, and could thus + // uncommitted mutation, but not its rollback, fro m the change feed, and could thus // think the uncommitted mutation is committed because it saw a higher committed version // than the mutation's version. // We also can only set it after consuming all of the mutations from the vector from the promise From 86f6f73518285327d97b9ee4f55674946175df7d Mon Sep 17 00:00:00 2001 From: Josh Slocum Date: Thu, 9 Dec 2021 10:51:31 -0600 Subject: [PATCH 059/413] Merge cursor debugging and fix in BW::waitForVersion --- fdbclient/NativeAPI.actor.cpp | 47 +++++++++++++++++++++++++++++++--- fdbserver/BlobWorker.actor.cpp | 9 +++---- 2 files changed, 48 insertions(+), 8 deletions(-) diff --git a/fdbclient/NativeAPI.actor.cpp b/fdbclient/NativeAPI.actor.cpp index 667f984bc4..fcbdccec98 100644 --- a/fdbclient/NativeAPI.actor.cpp +++ b/fdbclient/NativeAPI.actor.cpp @@ -7190,7 +7190,8 @@ ACTOR Future singleChangeFeedStream(StorageServerInterface interf, ReplyPromiseStream replyStream, Version end, Reference feedData, - Reference storageData) { + Reference storageData, + int idx /* TODO REMOVE this param after correctness clean */) { state bool atLatestVersion = false; state Version nextVersion = 0; try { @@ -7201,9 +7202,30 @@ ACTOR Future singleChangeFeedStream(StorageServerInterface interf, } choose { when(state ChangeFeedStreamReply rep = waitNext(replyStream.getFuture())) { + if (DEBUG_CF_VERSION(rep.mutations.back().version)) { + fmt::print(" single {0} {1}: response {2} - {3} ({4}), atLatest={5}, rep.atLatest={6}, " + "notAtLatest={7}, " + "minSV={8}\n", + idx, + interf.id().toString().substr(0, 4), + rep.mutations.front().version, + rep.mutations.back().version, + rep.mutations.size(), + atLatestVersion ? "T" : "F", + rep.atLatestVersion ? "T" : "F", + feedData->notAtLatest.get(), + rep.minStreamVersion); + } state int resultLoc = 0; while (resultLoc < rep.mutations.size()) { wait(results.onEmpty()); + if (DEBUG_CF_VERSION(rep.mutations[resultLoc].version)) { + fmt::print(" single {0} {1}: onEmpty, sending {2} ({3})\n", + idx, + interf.id().toString().substr(0, 4), + rep.mutations[resultLoc].version, + rep.mutations[resultLoc].mutations.size()); + } if (rep.mutations[resultLoc].version >= nextVersion) { results.send(rep.mutations[resultLoc]); } else { @@ -7239,13 +7261,24 @@ ACTOR Future singleChangeFeedStream(StorageServerInterface interf, when(wait(atLatestVersion && replyStream.isEmpty() && results.isEmpty() ? storageData->version.whenAtLeast(nextVersion) : Future(Never()))) { + if (DEBUG_CF_VERSION(nextVersion)) { + fmt::print(" single {0} {1}: WAL {2}, sending empty {3})\n", + idx, + interf.id().toString().substr(0, 4), + nextVersion, + storageData->version.get()); + } MutationsAndVersionRef empty; empty.version = storageData->version.get(); results.send(empty); nextVersion = storageData->version.get() + 1; } when(wait(atLatestVersion && replyStream.isEmpty() && !results.isEmpty() ? results.onEmpty() - : Future(Never()))) {} + : Future(Never()))) { + if (DEBUG_CF_VERSION(nextVersion)) { + fmt::print(" single {0} {1}: got onEmpty\n", idx, interf.id().toString().substr(0, 4)); + } + } } } } catch (Error& e) { @@ -7295,7 +7328,7 @@ ACTOR Future mergeChangeFeedStream(Reference db, for (int i = 0; i < interfs.size(); i++) { fetchers[i] = singleChangeFeedStream( - interfs[i].first, streams[i].results, results->streams[i], end, results, results->storageData[i]); + interfs[i].first, streams[i].results, results->streams[i], end, results, results->storageData[i], i); } state int interfNum = 0; while (interfNum < interfs.size()) { @@ -7319,6 +7352,9 @@ ACTOR Future mergeChangeFeedStream(Reference db, if (nextStream.next.version != checkVersion) { if (nextOut.size()) { *begin = checkVersion + 1; + if (DEBUG_CF_VERSION(nextOut.back().version)) { + fmt::print("CFNA (merged): {0} (1)\n", nextOut.back().version); + } ASSERT(nextOut.back().version >= results->lastReturnedVersion.get()); results->mutations.send(nextOut); wait(results->mutations.onEmpty()); @@ -7344,6 +7380,9 @@ ACTOR Future mergeChangeFeedStream(Reference db, } try { Standalone res = waitNext(nextStream.results.getFuture()); + if (DEBUG_CF_VERSION(nextOut.back().version)) { + fmt::print(" CFNA (merge1): {0} (1)\n", res.version, res.mutations.size()); + } nextStream.next = res; mutations.push(nextStream); } catch (Error& e) { @@ -7562,6 +7601,8 @@ ACTOR Future getChangeFeedStreamActor(Reference db, results->refresh.sendError(change_feed_cancelled()); throw; } + // TODO REMOVE + fmt::print("CFNA error {}\n", e.name()); if (results->notAtLatest.get() == 0) { results->notAtLatest.set(1); } diff --git a/fdbserver/BlobWorker.actor.cpp b/fdbserver/BlobWorker.actor.cpp index 28275afb32..c6c918a64a 100644 --- a/fdbserver/BlobWorker.actor.cpp +++ b/fdbserver/BlobWorker.actor.cpp @@ -1909,7 +1909,7 @@ ACTOR Future waitForVersion(Reference metadata, Version v ASSERT(metadata->durableDeltaVersion.get() >= pendingDeltaV); if (v == DEBUG_BW_WAIT_VERSION) { - fmt::print("{0}) waiting for DDV {1} >= {2}\n", v, metadata->durableDeltaVersion.get(), pendingDeltaV); + fmt::print("{0}) got DDV {1} >= {2}\n", v, metadata->durableDeltaVersion.get(), pendingDeltaV); } } @@ -1924,8 +1924,7 @@ ACTOR Future waitForVersion(Reference metadata, Version v ASSERT(metadata->durableSnapshotVersion.get() >= pendingSnapshotV); if (v == DEBUG_BW_WAIT_VERSION) { - fmt::print( - "{0}) waiting for DSV {1} >= {2}\n", v, metadata->durableSnapshotVersion.get(), pendingSnapshotV); + fmt::print("{0}) got DSV {1} >= {2}\n", v, metadata->durableSnapshotVersion.get(), pendingSnapshotV); } } @@ -1933,7 +1932,7 @@ ACTOR Future waitForVersion(Reference metadata, Version v // kick off another delta file and roll the mutations. In that case, we must return the new delta // file instead of in memory mutations, so we wait for that delta file to complete - if (metadata->pendingDeltaVersion > v) { + if (metadata->pendingDeltaVersion >= v) { if (v == DEBUG_BW_WAIT_VERSION) { fmt::print("{0}) waiting for DDV again {1} < {2}\n", v, metadata->durableDeltaVersion.get(), v); } @@ -1942,7 +1941,7 @@ ACTOR Future waitForVersion(Reference metadata, Version v ASSERT(metadata->durableDeltaVersion.get() >= v); if (v == DEBUG_BW_WAIT_VERSION) { - fmt::print("{0}) waiting for DDV again {1} >= {2}\n", v, metadata->durableDeltaVersion.get(), v); + fmt::print("{0}) got DDV again {1} >= {2}\n", v, metadata->durableDeltaVersion.get(), v); } } From 46ac7267001f722d208949310fe8e29bda02206b Mon Sep 17 00:00:00 2001 From: Josh Slocum Date: Thu, 9 Dec 2021 11:01:13 -0600 Subject: [PATCH 060/413] Added assert to more easily detect WFV errors --- fdbserver/BlobWorker.actor.cpp | 20 ++++++++++++++++++++ 1 file changed, 20 insertions(+) diff --git a/fdbserver/BlobWorker.actor.cpp b/fdbserver/BlobWorker.actor.cpp index c6c918a64a..c453d3c301 100644 --- a/fdbserver/BlobWorker.actor.cpp +++ b/fdbserver/BlobWorker.actor.cpp @@ -121,6 +121,9 @@ struct GranuleMetadata : NonCopyable, ReferenceCounted { AssignBlobRangeRequest originalReq; + // TODO FOR DEBUGGING, REMOVE + Version waitForVersionReturned = invalidVersion; + void resume() { if (resumeSnapshot.canBeSet()) { resumeSnapshot.send(Void()); @@ -1329,6 +1332,18 @@ ACTOR Future blobGranuleUpdateFiles(Reference bwData, } else { ASSERT(mutations.front().version >= startState.changeFeedStartVersion); } + + // if we just got mutations, we haven't buffered them yet, so waitForVersion can't have returned + // this version yet + if (mutations.front().version <= metadata->waitForVersionReturned) { + fmt::print("ERROR: WaitForVersion returned early for granule [{0} - {1}). " + "waitForVersionReturned={2}, mutationVersion={3} !!!\n", + metadata->keyRange.begin.printable(), + metadata->keyRange.end.printable(), + metadata->waitForVersionReturned, + mutations.front().version); + } + ASSERT(mutations.front().version > metadata->waitForVersionReturned); } when(wait(inFlightFiles.empty() ? Never() : success(inFlightFiles.front().future))) { // TODO REMOVE @@ -1949,6 +1964,11 @@ ACTOR Future waitForVersion(Reference metadata, Version v fmt::print("{0}) done\n", v); } + // TODO REMOVE debugging + if (v > metadata->waitForVersionReturned) { + metadata->waitForVersionReturned = v; + } + return Void(); } From 1ee0b16bfae9dcd6f2889d96e001230ea7b5784e Mon Sep 17 00:00:00 2001 From: Josh Slocum Date: Thu, 9 Dec 2021 14:19:00 -0600 Subject: [PATCH 061/413] Fixed bug in merge cursor whenAtLeast --- fdbclient/DatabaseContext.h | 1 + fdbclient/NativeAPI.actor.cpp | 57 ++++++++++++++++++++++++---------- fdbserver/BlobWorker.actor.cpp | 4 +++ flow/error_definitions.h | 2 +- 4 files changed, 47 insertions(+), 17 deletions(-) diff --git a/fdbclient/DatabaseContext.h b/fdbclient/DatabaseContext.h index 859e4d9212..2f47524f2f 100644 --- a/fdbclient/DatabaseContext.h +++ b/fdbclient/DatabaseContext.h @@ -174,6 +174,7 @@ struct ChangeFeedData : ReferenceCounted { std::vector> storageData; AsyncVar notAtLatest; Promise refresh; + Version maxSeenVersion; ChangeFeedData() : notAtLatest(1) {} }; diff --git a/fdbclient/NativeAPI.actor.cpp b/fdbclient/NativeAPI.actor.cpp index fcbdccec98..764c87189f 100644 --- a/fdbclient/NativeAPI.actor.cpp +++ b/fdbclient/NativeAPI.actor.cpp @@ -7063,16 +7063,17 @@ Reference DatabaseContext::getStorageData(StorageServerIn } Version ChangeFeedData::getVersion() { - // TODO uncomment? - if (notAtLatest.get() == 0 && mutations.isEmpty() /*& storageData.size() > 0*/) { - Version v = storageData[0]->version.get(); - for (int i = 1; i < storageData.size(); i++) { - if (storageData[i]->version.get() < v) { - v = storageData[i]->version.get(); - } - } - return std::max(v, lastReturnedVersion.get()); + // FIXME: add back in smarter version check later + /*if (notAtLatest.get() == 0 && mutations.isEmpty()) { + Version v = storageData[0]->version.get(); + for (int i = 1; i < storageData.size(); i++) { + if (storageData[i]->version.get() < v) { + v = storageData[i]->version.get(); + } + } + return std::max(v, lastReturnedVersion.get()); } + */ return lastReturnedVersion.get(); } @@ -7131,16 +7132,31 @@ ACTOR Future changeFeedWaitLatest(ChangeFeedData* self, Version version) { } // then, wait for client to have consumed up through version - while (!self->mutations.isEmpty()) { + if (self->maxSeenVersion >= version) { + // merge cursor has something buffered but has not yet sent it to self->mutations, just wait for + // lastReturnedVersion if (DEBUG_CF_WAIT_VERSION == version) { - fmt::print("CFW {0}) WaitLatest: waiting for client onEmpty\n", version); + fmt::print("CFW {0}) WaitLatest: maxSeenVersion -> waiting lastReturned\n", version); } - wait(self->mutations.onEmpty()); - wait(delay(0)); - } - if (DEBUG_CF_WAIT_VERSION == version) { - fmt::print("CFW {0}) WaitLatest: done\n", version); + wait(self->lastReturnedVersion.whenAtLeast(version)); + + if (DEBUG_CF_WAIT_VERSION == version) { + fmt::print("CFW {0}) WaitLatest: maxSeenVersion -> got lastReturned\n", version); + } + } else { + // all mutations <= version are in self->mutations, wait for empty + while (!self->mutations.isEmpty()) { + if (DEBUG_CF_WAIT_VERSION == version) { + fmt::print("CFW {0}) WaitLatest: waiting for client onEmpty\n", version); + } + wait(self->mutations.onEmpty()); + wait(delay(0)); + } + + if (DEBUG_CF_WAIT_VERSION == version) { + fmt::print("CFW {0}) WaitLatest: done\n", version); + } } return Void(); @@ -7242,6 +7258,9 @@ ACTOR Future singleChangeFeedStream(StorageServerInterface interf, } resultLoc++; } + if (rep.mutations.back().version > feedData->maxSeenVersion) { + feedData->maxSeenVersion = rep.mutations.back().version; + } nextVersion = rep.mutations.back().version + 1; if (!atLatestVersion && rep.atLatestVersion) { @@ -7317,6 +7336,7 @@ ACTOR Future mergeChangeFeedStream(Reference db, db->changeFeedUpdaters.erase(it->id); } } + results->maxSeenVersion = invalidVersion; results->storageData.clear(); Promise refresh = results->refresh; results->refresh = Promise(); @@ -7355,6 +7375,10 @@ ACTOR Future mergeChangeFeedStream(Reference db, if (DEBUG_CF_VERSION(nextOut.back().version)) { fmt::print("CFNA (merged): {0} (1)\n", nextOut.back().version); } + + if (nextOut.back().version < results->lastReturnedVersion.get()) { + printf("ERROR: merge cursor pushing next out <= lastReturnedVersion"); + } ASSERT(nextOut.back().version >= results->lastReturnedVersion.get()); results->mutations.send(nextOut); wait(results->mutations.onEmpty()); @@ -7532,6 +7556,7 @@ ACTOR Future getChangeFeedStreamActor(Reference db, } results->streams.push_back(interf.changeFeedStream.getReplyStream(req)); + results->maxSeenVersion = invalidVersion; results->storageData.clear(); results->storageData.push_back(db->getStorageData(interf)); Promise refresh = results->refresh; diff --git a/fdbserver/BlobWorker.actor.cpp b/fdbserver/BlobWorker.actor.cpp index c453d3c301..5e2e191946 100644 --- a/fdbserver/BlobWorker.actor.cpp +++ b/fdbserver/BlobWorker.actor.cpp @@ -1889,6 +1889,10 @@ ACTOR Future waitForVersion(Reference metadata, Version v metadata->durableDeltaVersion.get() == metadata->pendingDeltaVersion) && (v <= metadata->durableSnapshotVersion.get() || metadata->durableSnapshotVersion.get() == metadata->pendingSnapshotVersion)) { + // TODO REMOVE debugging + if (v > metadata->waitForVersionReturned) { + metadata->waitForVersionReturned = v; + } if (v == DEBUG_BW_WAIT_VERSION) { fmt::print("{0}) already done\n", v); } diff --git a/flow/error_definitions.h b/flow/error_definitions.h index 5cd156ccdb..88def0477e 100755 --- a/flow/error_definitions.h +++ b/flow/error_definitions.h @@ -169,7 +169,7 @@ ERROR( quick_get_key_values_has_more, 2033, "One of the mapped range queries is ERROR( quick_get_value_miss, 2034, "Found a mapped key that is not served in the same SS" ) ERROR( quick_get_key_values_miss, 2035, "Found a mapped range that is not served in the same SS" ) ERROR( blob_granule_no_ryw, 2036, "Blob Granule Read Transactions must be specified as ryw-disabled" ) -ERROR( blob_granule_not_materialized, 2037, "Blob Granule Read Transactions must be specified as ryw-disabled" ) +ERROR( blob_granule_not_materialized, 2037, "Blob Granule Read was not materialized" ) ERROR( incompatible_protocol_version, 2100, "Incompatible protocol version" ) ERROR( transaction_too_large, 2101, "Transaction exceeds byte limit" ) From 8dc5f79dc7b83e47b192625dc6c1fdd24c1f5237 Mon Sep 17 00:00:00 2001 From: Josh Slocum Date: Thu, 9 Dec 2021 15:00:46 -0600 Subject: [PATCH 062/413] Made blob worker assume no empty mutations, and made merge cursor not send them --- fdbclient/NativeAPI.actor.cpp | 13 +- fdbserver/BlobWorker.actor.cpp | 216 ++++++++++++++++----------------- 2 files changed, 116 insertions(+), 113 deletions(-) diff --git a/fdbclient/NativeAPI.actor.cpp b/fdbclient/NativeAPI.actor.cpp index 764c87189f..6d2642da62 100644 --- a/fdbclient/NativeAPI.actor.cpp +++ b/fdbclient/NativeAPI.actor.cpp @@ -7379,10 +7379,15 @@ ACTOR Future mergeChangeFeedStream(Reference db, if (nextOut.back().version < results->lastReturnedVersion.get()) { printf("ERROR: merge cursor pushing next out <= lastReturnedVersion"); } - ASSERT(nextOut.back().version >= results->lastReturnedVersion.get()); - results->mutations.send(nextOut); - wait(results->mutations.onEmpty()); - wait(delay(0)); + // We can get an empty version pushed through the stream if whenAtLeast is called. Ignore + // it + ASSERT(nextOut.size() == 1); + if (!nextOut.back().mutations.empty()) { + ASSERT(nextOut.back().version >= results->lastReturnedVersion.get()); + results->mutations.send(nextOut); + wait(results->mutations.onEmpty()); + wait(delay(0)); + } if (DEBUG_CF_VERSION(nextOut.back().version)) { fmt::print("CFLR (merged): {0} (1)\n", nextOut.back().version); } diff --git a/fdbserver/BlobWorker.actor.cpp b/fdbserver/BlobWorker.actor.cpp index 5e2e191946..9ec038bd4e 100644 --- a/fdbserver/BlobWorker.actor.cpp +++ b/fdbserver/BlobWorker.actor.cpp @@ -1391,125 +1391,123 @@ ACTOR Future blobGranuleUpdateFiles(Reference bwData, // same version ASSERT(deltas.version > metadata->bufferedDeltaVersion); ASSERT(deltas.version > lastDeltaVersion); - if (!deltas.mutations.empty()) { - if (deltas.mutations.size() == 1 && deltas.mutations.back().param1 == lastEpochEndPrivateKey) { - // Note rollbackVerision is durable, [rollbackVersion+1 - deltas.version] needs to be tossed - // For correctness right now, there can be no waits and yields either in rollback handling - // or in handleBlobGranuleFileRequest once waitForVersion has succeeded, otherwise this will - // race and clobber results - Version rollbackVersion; - BinaryReader br(deltas.mutations[0].param2, Unversioned()); - br >> rollbackVersion; + ASSERT(!deltas.mutations.empty()); + if (deltas.mutations.size() == 1 && deltas.mutations.back().param1 == lastEpochEndPrivateKey) { + // Note rollbackVerision is durable, [rollbackVersion+1 - deltas.version] needs to be tossed + // For correctness right now, there can be no waits and yields either in rollback handling + // or in handleBlobGranuleFileRequest once waitForVersion has succeeded, otherwise this will + // race and clobber results + Version rollbackVersion; + BinaryReader br(deltas.mutations[0].param2, Unversioned()); + br >> rollbackVersion; - ASSERT(rollbackVersion >= metadata->durableDeltaVersion.get()); - ASSERT(rollbackVersion >= committedVersion.get()); + ASSERT(rollbackVersion >= metadata->durableDeltaVersion.get()); + ASSERT(rollbackVersion >= committedVersion.get()); - if (!rollbacksInProgress.empty()) { - ASSERT(rollbacksInProgress.front().first == rollbackVersion); - ASSERT(rollbacksInProgress.front().second == deltas.version); - if (BW_DEBUG) { - fmt::print("Passed rollback {0} -> {1}\n", deltas.version, rollbackVersion); - } - rollbacksCompleted.push_back(rollbacksInProgress.front()); - rollbacksInProgress.pop_front(); - } else { - // FIXME: add counter for granule rollbacks and rollbacks skipped? - // explicitly check last delta in currentDeltas because lastVersion and - // bufferedDeltaVersion include empties - if (metadata->pendingDeltaVersion <= rollbackVersion && - (metadata->currentDeltas.empty() || - metadata->currentDeltas.back().version <= rollbackVersion)) { - - if (BW_DEBUG) { - fmt::print("BW skipping rollback {0} -> {1} completely\n", - deltas.version, - rollbackVersion); - } - } else { - if (BW_DEBUG) { - fmt::print("BW [{0} - {1}) ROLLBACK @ {2} -> {3}\n", - metadata->keyRange.begin.printable(), - metadata->keyRange.end.printable(), - deltas.version, - rollbackVersion); - TraceEvent(SevWarn, "GranuleRollback", bwData->id) - .detail("Granule", metadata->keyRange) - .detail("Version", deltas.version) - .detail("RollbackVersion", rollbackVersion); - } - - Version cfRollbackVersion = doGranuleRollback(metadata, - deltas.version, - rollbackVersion, - inFlightFiles, - rollbacksInProgress, - rollbacksCompleted); - - // Reset change feeds to cfRollbackVersion - metadata->activeCFData.set(newChangeFeedData(cfRollbackVersion)); - if (readOldChangeFeed) { - // It shouldn't be possible to roll back across the parent/child feed boundary, - // because the transaction creating the child change feed had to commit before - // we got here. - ASSERT(cfRollbackVersion < startState.changeFeedStartVersion); - oldChangeFeedFuture = - bwData->db->getChangeFeedStream(metadata->activeCFData.get(), - oldCFKey.get(), - cfRollbackVersion + 1, - startState.changeFeedStartVersion, - metadata->keyRange); - - } else { - ASSERT(cfRollbackVersion > startState.changeFeedStartVersion); - - changeFeedFuture = bwData->db->getChangeFeedStream(metadata->activeCFData.get(), - cfKey, - cfRollbackVersion + 1, - MAX_VERSION, - metadata->keyRange); - } - - justDidRollback = true; - break; - } - } - } else if (!rollbacksInProgress.empty() && rollbacksInProgress.front().first < deltas.version && - rollbacksInProgress.front().second > deltas.version) { + if (!rollbacksInProgress.empty()) { + ASSERT(rollbacksInProgress.front().first == rollbackVersion); + ASSERT(rollbacksInProgress.front().second == deltas.version); if (BW_DEBUG) { - fmt::print("Skipping mutations @ {} b/c prior rollback\n", deltas.version); + fmt::print("Passed rollback {0} -> {1}\n", deltas.version, rollbackVersion); } + rollbacksCompleted.push_back(rollbacksInProgress.front()); + rollbacksInProgress.pop_front(); } else { - for (auto& delta : deltas.mutations) { - metadata->bufferedDeltaBytes += delta.totalSize(); - bwData->stats.changeFeedInputBytes += delta.totalSize(); - bwData->stats.mutationBytesBuffered += delta.totalSize(); + // FIXME: add counter for granule rollbacks and rollbacks skipped? + // explicitly check last delta in currentDeltas because lastVersion and + // bufferedDeltaVersion include empties + if (metadata->pendingDeltaVersion <= rollbackVersion && + (metadata->currentDeltas.empty() || + metadata->currentDeltas.back().version <= rollbackVersion)) { - DEBUG_MUTATION("BlobWorkerBuffer", deltas.version, delta, bwData->id) - .detail("Granule", metadata->keyRange) - .detail("ChangeFeedID", readOldChangeFeed ? oldCFKey.get() : cfKey) - .detail("OldChangeFeed", readOldChangeFeed ? "T" : "F"); + if (BW_DEBUG) { + fmt::print("BW skipping rollback {0} -> {1} completely\n", + deltas.version, + rollbackVersion); + } + } else { + if (BW_DEBUG) { + fmt::print("BW [{0} - {1}) ROLLBACK @ {2} -> {3}\n", + metadata->keyRange.begin.printable(), + metadata->keyRange.end.printable(), + deltas.version, + rollbackVersion); + TraceEvent(SevWarn, "GranuleRollback", bwData->id) + .detail("Granule", metadata->keyRange) + .detail("Version", deltas.version) + .detail("RollbackVersion", rollbackVersion); + } + + Version cfRollbackVersion = doGranuleRollback(metadata, + deltas.version, + rollbackVersion, + inFlightFiles, + rollbacksInProgress, + rollbacksCompleted); + + // Reset change feeds to cfRollbackVersion + metadata->activeCFData.set(newChangeFeedData(cfRollbackVersion)); + if (readOldChangeFeed) { + // It shouldn't be possible to roll back across the parent/child feed boundary, + // because the transaction creating the child change feed had to commit before + // we got here. + ASSERT(cfRollbackVersion < startState.changeFeedStartVersion); + oldChangeFeedFuture = + bwData->db->getChangeFeedStream(metadata->activeCFData.get(), + oldCFKey.get(), + cfRollbackVersion + 1, + startState.changeFeedStartVersion, + metadata->keyRange); + + } else { + ASSERT(cfRollbackVersion > startState.changeFeedStartVersion); + + changeFeedFuture = bwData->db->getChangeFeedStream(metadata->activeCFData.get(), + cfKey, + cfRollbackVersion + 1, + MAX_VERSION, + metadata->keyRange); + } + + justDidRollback = true; + break; } - if (DEBUG_BW_VERSION(deltas.version)) { - fmt::print("BW {0}: ({1}), KCV={2}\n", - deltas.version, - deltas.mutations.size(), - deltas.knownCommittedVersion); - } - metadata->currentDeltas.push_back_deep(metadata->deltaArena, deltas); + } + } else if (!rollbacksInProgress.empty() && rollbacksInProgress.front().first < deltas.version && + rollbacksInProgress.front().second > deltas.version) { + if (BW_DEBUG) { + fmt::print("Skipping mutations @ {} b/c prior rollback\n", deltas.version); + } + } else { + for (auto& delta : deltas.mutations) { + metadata->bufferedDeltaBytes += delta.totalSize(); + bwData->stats.changeFeedInputBytes += delta.totalSize(); + bwData->stats.mutationBytesBuffered += delta.totalSize(); - processedAnyMutations = true; - ASSERT(deltas.version != invalidVersion); - ASSERT(deltas.version > lastDeltaVersion); - lastDeltaVersion = deltas.version; + DEBUG_MUTATION("BlobWorkerBuffer", deltas.version, delta, bwData->id) + .detail("Granule", metadata->keyRange) + .detail("ChangeFeedID", readOldChangeFeed ? oldCFKey.get() : cfKey) + .detail("OldChangeFeed", readOldChangeFeed ? "T" : "F"); + } + if (DEBUG_BW_VERSION(deltas.version)) { + fmt::print("BW {0}: ({1}), KCV={2}\n", + deltas.version, + deltas.mutations.size(), + deltas.knownCommittedVersion); + } + metadata->currentDeltas.push_back_deep(metadata->deltaArena, deltas); - Version nextKnownNoRollbacksPast = std::min(deltas.version, deltas.knownCommittedVersion); - // FIXME: apparently we can have invalidVersion for knownCommittedVersion - ASSERT(nextKnownNoRollbacksPast >= knownNoRollbacksPast || - nextKnownNoRollbacksPast == invalidVersion); + processedAnyMutations = true; + ASSERT(deltas.version != invalidVersion); + ASSERT(deltas.version > lastDeltaVersion); + lastDeltaVersion = deltas.version; - if (nextKnownNoRollbacksPast != invalidVersion) { - knownNoRollbacksPast = nextKnownNoRollbacksPast; - } + Version nextKnownNoRollbacksPast = std::min(deltas.version, deltas.knownCommittedVersion); + ASSERT(nextKnownNoRollbacksPast >= knownNoRollbacksPast || + nextKnownNoRollbacksPast == invalidVersion); + + if (nextKnownNoRollbacksPast != invalidVersion) { + knownNoRollbacksPast = nextKnownNoRollbacksPast; } } if (justDidRollback) { From c95c93b527b335c30ca4b35c784e9bb0fa310171 Mon Sep 17 00:00:00 2001 From: Josh Slocum Date: Thu, 9 Dec 2021 17:06:20 -0600 Subject: [PATCH 063/413] removed incorrect assert for now --- fdbserver/BlobWorker.actor.cpp | 209 +++++++++++++++++---------------- 1 file changed, 106 insertions(+), 103 deletions(-) diff --git a/fdbserver/BlobWorker.actor.cpp b/fdbserver/BlobWorker.actor.cpp index 9ec038bd4e..d24993db11 100644 --- a/fdbserver/BlobWorker.actor.cpp +++ b/fdbserver/BlobWorker.actor.cpp @@ -1391,123 +1391,126 @@ ACTOR Future blobGranuleUpdateFiles(Reference bwData, // same version ASSERT(deltas.version > metadata->bufferedDeltaVersion); ASSERT(deltas.version > lastDeltaVersion); - ASSERT(!deltas.mutations.empty()); - if (deltas.mutations.size() == 1 && deltas.mutations.back().param1 == lastEpochEndPrivateKey) { - // Note rollbackVerision is durable, [rollbackVersion+1 - deltas.version] needs to be tossed - // For correctness right now, there can be no waits and yields either in rollback handling - // or in handleBlobGranuleFileRequest once waitForVersion has succeeded, otherwise this will - // race and clobber results - Version rollbackVersion; - BinaryReader br(deltas.mutations[0].param2, Unversioned()); - br >> rollbackVersion; + // FIXME: this assert isn't true - why + // ASSERT(!deltas.mutations.empty()); + if (!deltas.mutations.empty()) { + if (deltas.mutations.size() == 1 && deltas.mutations.back().param1 == lastEpochEndPrivateKey) { + // Note rollbackVerision is durable, [rollbackVersion+1 - deltas.version] needs to be tossed + // For correctness right now, there can be no waits and yields either in rollback handling + // or in handleBlobGranuleFileRequest once waitForVersion has succeeded, otherwise this will + // race and clobber results + Version rollbackVersion; + BinaryReader br(deltas.mutations[0].param2, Unversioned()); + br >> rollbackVersion; - ASSERT(rollbackVersion >= metadata->durableDeltaVersion.get()); - ASSERT(rollbackVersion >= committedVersion.get()); - - if (!rollbacksInProgress.empty()) { - ASSERT(rollbacksInProgress.front().first == rollbackVersion); - ASSERT(rollbacksInProgress.front().second == deltas.version); - if (BW_DEBUG) { - fmt::print("Passed rollback {0} -> {1}\n", deltas.version, rollbackVersion); - } - rollbacksCompleted.push_back(rollbacksInProgress.front()); - rollbacksInProgress.pop_front(); - } else { - // FIXME: add counter for granule rollbacks and rollbacks skipped? - // explicitly check last delta in currentDeltas because lastVersion and - // bufferedDeltaVersion include empties - if (metadata->pendingDeltaVersion <= rollbackVersion && - (metadata->currentDeltas.empty() || - metadata->currentDeltas.back().version <= rollbackVersion)) { + ASSERT(rollbackVersion >= metadata->durableDeltaVersion.get()); + ASSERT(rollbackVersion >= committedVersion.get()); + if (!rollbacksInProgress.empty()) { + ASSERT(rollbacksInProgress.front().first == rollbackVersion); + ASSERT(rollbacksInProgress.front().second == deltas.version); if (BW_DEBUG) { - fmt::print("BW skipping rollback {0} -> {1} completely\n", - deltas.version, - rollbackVersion); + fmt::print("Passed rollback {0} -> {1}\n", deltas.version, rollbackVersion); } + rollbacksCompleted.push_back(rollbacksInProgress.front()); + rollbacksInProgress.pop_front(); } else { - if (BW_DEBUG) { - fmt::print("BW [{0} - {1}) ROLLBACK @ {2} -> {3}\n", - metadata->keyRange.begin.printable(), - metadata->keyRange.end.printable(), - deltas.version, - rollbackVersion); - TraceEvent(SevWarn, "GranuleRollback", bwData->id) - .detail("Granule", metadata->keyRange) - .detail("Version", deltas.version) - .detail("RollbackVersion", rollbackVersion); - } - - Version cfRollbackVersion = doGranuleRollback(metadata, - deltas.version, - rollbackVersion, - inFlightFiles, - rollbacksInProgress, - rollbacksCompleted); - - // Reset change feeds to cfRollbackVersion - metadata->activeCFData.set(newChangeFeedData(cfRollbackVersion)); - if (readOldChangeFeed) { - // It shouldn't be possible to roll back across the parent/child feed boundary, - // because the transaction creating the child change feed had to commit before - // we got here. - ASSERT(cfRollbackVersion < startState.changeFeedStartVersion); - oldChangeFeedFuture = - bwData->db->getChangeFeedStream(metadata->activeCFData.get(), - oldCFKey.get(), - cfRollbackVersion + 1, - startState.changeFeedStartVersion, - metadata->keyRange); + // FIXME: add counter for granule rollbacks and rollbacks skipped? + // explicitly check last delta in currentDeltas because lastVersion and + // bufferedDeltaVersion include empties + if (metadata->pendingDeltaVersion <= rollbackVersion && + (metadata->currentDeltas.empty() || + metadata->currentDeltas.back().version <= rollbackVersion)) { + if (BW_DEBUG) { + fmt::print("BW skipping rollback {0} -> {1} completely\n", + deltas.version, + rollbackVersion); + } } else { - ASSERT(cfRollbackVersion > startState.changeFeedStartVersion); + if (BW_DEBUG) { + fmt::print("BW [{0} - {1}) ROLLBACK @ {2} -> {3}\n", + metadata->keyRange.begin.printable(), + metadata->keyRange.end.printable(), + deltas.version, + rollbackVersion); + TraceEvent(SevWarn, "GranuleRollback", bwData->id) + .detail("Granule", metadata->keyRange) + .detail("Version", deltas.version) + .detail("RollbackVersion", rollbackVersion); + } - changeFeedFuture = bwData->db->getChangeFeedStream(metadata->activeCFData.get(), - cfKey, - cfRollbackVersion + 1, - MAX_VERSION, - metadata->keyRange); + Version cfRollbackVersion = doGranuleRollback(metadata, + deltas.version, + rollbackVersion, + inFlightFiles, + rollbacksInProgress, + rollbacksCompleted); + + // Reset change feeds to cfRollbackVersion + metadata->activeCFData.set(newChangeFeedData(cfRollbackVersion)); + if (readOldChangeFeed) { + // It shouldn't be possible to roll back across the parent/child feed boundary, + // because the transaction creating the child change feed had to commit before + // we got here. + ASSERT(cfRollbackVersion < startState.changeFeedStartVersion); + oldChangeFeedFuture = + bwData->db->getChangeFeedStream(metadata->activeCFData.get(), + oldCFKey.get(), + cfRollbackVersion + 1, + startState.changeFeedStartVersion, + metadata->keyRange); + + } else { + ASSERT(cfRollbackVersion > startState.changeFeedStartVersion); + + changeFeedFuture = bwData->db->getChangeFeedStream(metadata->activeCFData.get(), + cfKey, + cfRollbackVersion + 1, + MAX_VERSION, + metadata->keyRange); + } + + justDidRollback = true; + break; } - - justDidRollback = true; - break; } - } - } else if (!rollbacksInProgress.empty() && rollbacksInProgress.front().first < deltas.version && - rollbacksInProgress.front().second > deltas.version) { - if (BW_DEBUG) { - fmt::print("Skipping mutations @ {} b/c prior rollback\n", deltas.version); - } - } else { - for (auto& delta : deltas.mutations) { - metadata->bufferedDeltaBytes += delta.totalSize(); - bwData->stats.changeFeedInputBytes += delta.totalSize(); - bwData->stats.mutationBytesBuffered += delta.totalSize(); + } else if (!rollbacksInProgress.empty() && rollbacksInProgress.front().first < deltas.version && + rollbacksInProgress.front().second > deltas.version) { + if (BW_DEBUG) { + fmt::print("Skipping mutations @ {} b/c prior rollback\n", deltas.version); + } + } else { + for (auto& delta : deltas.mutations) { + metadata->bufferedDeltaBytes += delta.totalSize(); + bwData->stats.changeFeedInputBytes += delta.totalSize(); + bwData->stats.mutationBytesBuffered += delta.totalSize(); - DEBUG_MUTATION("BlobWorkerBuffer", deltas.version, delta, bwData->id) - .detail("Granule", metadata->keyRange) - .detail("ChangeFeedID", readOldChangeFeed ? oldCFKey.get() : cfKey) - .detail("OldChangeFeed", readOldChangeFeed ? "T" : "F"); - } - if (DEBUG_BW_VERSION(deltas.version)) { - fmt::print("BW {0}: ({1}), KCV={2}\n", - deltas.version, - deltas.mutations.size(), - deltas.knownCommittedVersion); - } - metadata->currentDeltas.push_back_deep(metadata->deltaArena, deltas); + DEBUG_MUTATION("BlobWorkerBuffer", deltas.version, delta, bwData->id) + .detail("Granule", metadata->keyRange) + .detail("ChangeFeedID", readOldChangeFeed ? oldCFKey.get() : cfKey) + .detail("OldChangeFeed", readOldChangeFeed ? "T" : "F"); + } + if (DEBUG_BW_VERSION(deltas.version)) { + fmt::print("BW {0}: ({1}), KCV={2}\n", + deltas.version, + deltas.mutations.size(), + deltas.knownCommittedVersion); + } + metadata->currentDeltas.push_back_deep(metadata->deltaArena, deltas); - processedAnyMutations = true; - ASSERT(deltas.version != invalidVersion); - ASSERT(deltas.version > lastDeltaVersion); - lastDeltaVersion = deltas.version; + processedAnyMutations = true; + ASSERT(deltas.version != invalidVersion); + ASSERT(deltas.version > lastDeltaVersion); + lastDeltaVersion = deltas.version; - Version nextKnownNoRollbacksPast = std::min(deltas.version, deltas.knownCommittedVersion); - ASSERT(nextKnownNoRollbacksPast >= knownNoRollbacksPast || - nextKnownNoRollbacksPast == invalidVersion); + Version nextKnownNoRollbacksPast = std::min(deltas.version, deltas.knownCommittedVersion); + ASSERT(nextKnownNoRollbacksPast >= knownNoRollbacksPast || + nextKnownNoRollbacksPast == invalidVersion); - if (nextKnownNoRollbacksPast != invalidVersion) { - knownNoRollbacksPast = nextKnownNoRollbacksPast; + if (nextKnownNoRollbacksPast != invalidVersion) { + knownNoRollbacksPast = nextKnownNoRollbacksPast; + } } } if (justDidRollback) { From d31cb07647cf1702e06f64fd4c52f8afaee49e19 Mon Sep 17 00:00:00 2001 From: Josh Slocum Date: Fri, 10 Dec 2021 12:25:42 -0600 Subject: [PATCH 064/413] ASAN fix --- fdbserver/BlobManager.actor.cpp | 55 ++++++++++++++++++--------------- 1 file changed, 30 insertions(+), 25 deletions(-) diff --git a/fdbserver/BlobManager.actor.cpp b/fdbserver/BlobManager.actor.cpp index ca17045bba..aa27cd2e1a 100644 --- a/fdbserver/BlobManager.actor.cpp +++ b/fdbserver/BlobManager.actor.cpp @@ -801,24 +801,29 @@ ACTOR Future killBlobWorker(BlobManagerData* bmData, BlobWorkerInterface b if (BM_DEBUG) { printf("Taking back ranges from BW %s\n", bwId.toString().c_str()); } + // copy ranges into vector before sending, because send then modifies workerAssignments + state std::vector rangesToMove; for (auto& it : bmData->workerAssignments.ranges()) { if (it.cvalue() == bwId) { - // Send revoke request - RangeAssignment raRevoke; - raRevoke.isAssign = false; - raRevoke.keyRange = it.range(); - raRevoke.revoke = RangeRevokeData(false); - bmData->rangesToAssign.send(raRevoke); - - // Add range back into the stream of ranges to be assigned - RangeAssignment raAssign; - raAssign.isAssign = true; - raAssign.worker = Optional(); - raAssign.keyRange = it.range(); - raAssign.assign = RangeAssignmentData(); // not a continue - bmData->rangesToAssign.send(raAssign); + rangesToMove.push_back(it.range()); } } + for (auto& it : rangesToMove) { + // Send revoke request + RangeAssignment raRevoke; + raRevoke.isAssign = false; + raRevoke.keyRange = it; + raRevoke.revoke = RangeRevokeData(false); + bmData->rangesToAssign.send(raRevoke); + + // Add range back into the stream of ranges to be assigned + RangeAssignment raAssign; + raAssign.isAssign = true; + raAssign.worker = Optional(); + raAssign.keyRange = it; + raAssign.assign = RangeAssignmentData(); // not a continue + bmData->rangesToAssign.send(raAssign); + } // Send halt to blob worker, with no expectation of hearing back if (BM_DEBUG) { @@ -866,15 +871,15 @@ ACTOR Future monitorBlobWorkerStatus(BlobManagerData* bmData, BlobWorkerIn bmData->iAmReplaced.send(Void()); } } else if (rep.epoch < bmData->epoch) { - // TODO: revoke the range from that worker? and send optimistic halt req to other (zombie) BM? it's - // optimistic because such a BM is not necessarily a zombie. it could have gotten killed properly - // but the BW that sent this reply was behind (i.e. it started the req when the old BM was in charge - // and finished by the time the new BM took over) + // TODO: revoke the range from that worker? and send optimistic halt req to other (zombie) BM? + // it's optimistic because such a BM is not necessarily a zombie. it could have gotten killed + // properly but the BW that sent this reply was behind (i.e. it started the req when the old BM + // was in charge and finished by the time the new BM took over) continue; } - // TODO maybe this won't be true eventually, but right now the only time the blob worker reports back is - // to split the range. + // TODO maybe this won't be true eventually, but right now the only time the blob worker reports + // back is to split the range. ASSERT(rep.doSplit); // only evaluate for split if this worker currently owns the granule in this blob manager's mapping @@ -1029,11 +1034,11 @@ ACTOR Future recoverBlobManager(BlobManagerData* bmData) { // For example, suppose a blob manager sends requests to the range assigner stream to move a granule G. // However, before sending those requests off to the workers, the BM dies. So the persisting mapping // still has G->oldWorker. The following algorithm will re-assign G to oldWorker (as long as it is also still - // alive). Note that this is fine because it simply means that the range was not moved optimally, but it is still - // owned. In the above case, even if the revoke goes through, since we don't update the mapping during revokes, - // this is the same as the case above. Another case to consider is when a blob worker dies when the BM is - // recovering. Now the mapping at this time looks like G->deadBW. But the rangeAssigner handles this: we'll try - // to assign a range to a dead worker and fail and reassign it to the next best worker. + // alive). Note that this is fine because it simply means that the range was not moved optimally, but it is + // still owned. In the above case, even if the revoke goes through, since we don't update the mapping during + // revokes, this is the same as the case above. Another case to consider is when a blob worker dies when the + // BM is recovering. Now the mapping at this time looks like G->deadBW. But the rangeAssigner handles this: + // we'll try to assign a range to a dead worker and fail and reassign it to the next best worker. // // 2. We get the existing split intentions that were Started but not acknowledged by any blob workers and // add them to our key range map, bmData->granuleAssignments. Note that we are adding them on top of From fc3376fe8f6fe76bde9b1c01fe5eb18ece64709e Mon Sep 17 00:00:00 2001 From: Suraj Gupta Date: Mon, 6 Dec 2021 15:10:20 -0500 Subject: [PATCH 065/413] Move client knob to database config for blob granules. --- fdbcli/ConfigureCommand.actor.cpp | 3 ++- fdbclient/ClientKnobs.cpp | 3 --- fdbclient/ClientKnobs.h | 3 --- fdbclient/DatabaseConfiguration.cpp | 20 ++++++++++++++++++ fdbclient/DatabaseConfiguration.h | 3 +++ fdbclient/Schemas.cpp | 3 ++- fdbserver/ClusterController.actor.cpp | 21 ++++++++++--------- fdbserver/SimulatedCluster.actor.cpp | 2 +- fdbserver/Status.actor.cpp | 8 +++---- .../workloads/BlobGranuleVerifier.actor.cpp | 10 ++++++--- .../workloads/ConsistencyCheck.actor.cpp | 4 ++-- 11 files changed, 52 insertions(+), 28 deletions(-) diff --git a/fdbcli/ConfigureCommand.actor.cpp b/fdbcli/ConfigureCommand.actor.cpp index ab810454a4..6b8ca1ddbd 100644 --- a/fdbcli/ConfigureCommand.actor.cpp +++ b/fdbcli/ConfigureCommand.actor.cpp @@ -264,7 +264,8 @@ CommandFactory configureFactory( "|" "commit_proxies=|grv_proxies=|logs=|resolvers=>*|" "count=|perpetual_storage_wiggle=|perpetual_storage_wiggle_locality=" - "<:|0>|storage_migration_type={disabled|gradual|aggressive}", + "<:|0>|storage_migration_type={disabled|gradual|aggressive}|" + "blob_granules_enabled={0|1}", "change the database configuration", "The `new' option, if present, initializes a new database with the given configuration rather than changing " "the configuration of an existing one. When used, both a redundancy mode and a storage engine must be " diff --git a/fdbclient/ClientKnobs.cpp b/fdbclient/ClientKnobs.cpp index 456cd863a1..bda87873f6 100644 --- a/fdbclient/ClientKnobs.cpp +++ b/fdbclient/ClientKnobs.cpp @@ -259,9 +259,6 @@ void ClientKnobs::initialize(Randomize randomize) { init( MVC_CLIENTLIB_CHUNK_SIZE, 8*1024 ); init( MVC_CLIENTLIB_CHUNKS_PER_TRANSACTION, 32 ); - // blob granules - init( ENABLE_BLOB_GRANULES, true ); - // clang-format on } diff --git a/fdbclient/ClientKnobs.h b/fdbclient/ClientKnobs.h index fefeceb32f..32c2a93a69 100644 --- a/fdbclient/ClientKnobs.h +++ b/fdbclient/ClientKnobs.h @@ -250,9 +250,6 @@ public: int MVC_CLIENTLIB_CHUNK_SIZE; int MVC_CLIENTLIB_CHUNKS_PER_TRANSACTION; - // blob granules - bool ENABLE_BLOB_GRANULES; - ClientKnobs(Randomize randomize); void initialize(Randomize randomize); }; diff --git a/fdbclient/DatabaseConfiguration.cpp b/fdbclient/DatabaseConfiguration.cpp index 3328443fc7..929d60095a 100644 --- a/fdbclient/DatabaseConfiguration.cpp +++ b/fdbclient/DatabaseConfiguration.cpp @@ -50,6 +50,7 @@ void DatabaseConfiguration::resetInternal() { perpetualStorageWiggleSpeed = 0; perpetualStorageWiggleLocality = "0"; storageMigrationType = StorageMigrationType::DEFAULT; + blobGranulesEnabled = true; } int toInt(ValueRef const& v) { @@ -402,6 +403,7 @@ StatusObject DatabaseConfiguration::toJSON(bool noPolicies) const { result["perpetual_storage_wiggle"] = perpetualStorageWiggleSpeed; result["perpetual_storage_wiggle_locality"] = perpetualStorageWiggleLocality; result["storage_migration_type"] = storageMigrationType.toString(); + result["blob_granules_enabled"] = (int32_t)blobGranulesEnabled; return result; } @@ -627,6 +629,24 @@ bool DatabaseConfiguration::setInternal(KeyRef key, ValueRef value) { storageMigrationType = (StorageMigrationType::MigrationType)type; } else if (ck == LiteralStringRef("proxies")) { overwriteProxiesCount(); + int proxiesCount; + parse(&proxiesCount, value); + if (proxiesCount > 1) { + int derivedGrvProxyCount = + std::max(1, + std::min(CLIENT_KNOBS->DEFAULT_MAX_GRV_PROXIES, + proxiesCount / (CLIENT_KNOBS->DEFAULT_COMMIT_GRV_PROXIES_RATIO + 1))); + int derivedCommitProxyCount = proxiesCount - derivedGrvProxyCount; + if (grvProxyCount == -1) { + grvProxyCount = derivedGrvProxyCount; + } + if (commitProxyCount == -1) { + commitProxyCount = derivedCommitProxyCount; + } + } + } else if (ck == LiteralStringRef("blob_granules_enabled")) { + parse((&type), value); + blobGranulesEnabled = (type != 0); } else { return false; } diff --git a/fdbclient/DatabaseConfiguration.h b/fdbclient/DatabaseConfiguration.h index 81bb6f7084..b29ccbc399 100644 --- a/fdbclient/DatabaseConfiguration.h +++ b/fdbclient/DatabaseConfiguration.h @@ -250,6 +250,9 @@ struct DatabaseConfiguration { // Storage Migration Type StorageMigrationType storageMigrationType; + // Blob Granules + bool blobGranulesEnabled; + // Excluded servers (no state should be here) bool isExcludedServer(NetworkAddressList) const; bool isExcludedLocality(const LocalityData& locality) const; diff --git a/fdbclient/Schemas.cpp b/fdbclient/Schemas.cpp index 10f77d7638..d2725e8f5f 100644 --- a/fdbclient/Schemas.cpp +++ b/fdbclient/Schemas.cpp @@ -770,7 +770,8 @@ const KeyRef JSONSchemas::statusSchema = LiteralStringRef(R"statusSchema( "disabled", "aggressive", "gradual" - ]} + ]}, + "blob_granules_enabled":1, }, "data":{ "least_operating_space_bytes_log_server":0, diff --git a/fdbserver/ClusterController.actor.cpp b/fdbserver/ClusterController.actor.cpp index 5f24dfcc81..d684ba824d 100644 --- a/fdbserver/ClusterController.actor.cpp +++ b/fdbserver/ClusterController.actor.cpp @@ -3720,7 +3720,7 @@ void checkBetterSingletons(ClusterControllerData* self) { WorkerDetails newDDWorker = findNewProcessForSingleton(self, ProcessClass::DataDistributor, id_used); WorkerDetails newBMWorker; - if (CLIENT_KNOBS->ENABLE_BLOB_GRANULES) { + if (self->db.config.blobGranulesEnabled) { newBMWorker = findNewProcessForSingleton(self, ProcessClass::BlobManager, id_used); } @@ -3729,7 +3729,7 @@ void checkBetterSingletons(ClusterControllerData* self) { auto bestFitnessForDD = findBestFitnessForSingleton(self, newDDWorker, ProcessClass::DataDistributor); ProcessClass::Fitness bestFitnessForBM; - if (CLIENT_KNOBS->ENABLE_BLOB_GRANULES) { + if (self->db.config.blobGranulesEnabled) { bestFitnessForBM = findBestFitnessForSingleton(self, newBMWorker, ProcessClass::BlobManager); } @@ -3747,7 +3747,7 @@ void checkBetterSingletons(ClusterControllerData* self) { self, newDDWorker, ddSingleton, bestFitnessForDD, self->recruitingDistributorID); bool bmHealthy = true; - if (CLIENT_KNOBS->ENABLE_BLOB_GRANULES) { + if (self->db.config.blobGranulesEnabled) { bmHealthy = isHealthySingleton( self, newBMWorker, bmSingleton, bestFitnessForBM, self->recruitingBlobManagerID); } @@ -3766,14 +3766,15 @@ void checkBetterSingletons(ClusterControllerData* self) { Optional> newDDProcessId = newDDWorker.interf.locality.processId(); Optional> currBMProcessId, newBMProcessId; - if (CLIENT_KNOBS->ENABLE_BLOB_GRANULES) { + if (self->db.config.blobGranulesEnabled) { + currBMProcessId = bmSingleton.interface.get().locality.processId(); newBMProcessId = newBMWorker.interf.locality.processId(); } std::vector>> currPids = { currRKProcessId, currDDProcessId }; std::vector>> newPids = { newRKProcessId, newDDProcessId }; - if (CLIENT_KNOBS->ENABLE_BLOB_GRANULES) { + if (self->db.config.blobGranulesEnabled) { currPids.emplace_back(currBMProcessId); newPids.emplace_back(newBMProcessId); } @@ -3782,7 +3783,7 @@ void checkBetterSingletons(ClusterControllerData* self) { auto newColocMap = getColocCounts(newPids); // if the knob is disabled, the BM coloc counts should have no affect on the coloc counts check below - if (!CLIENT_KNOBS->ENABLE_BLOB_GRANULES) { + if (!self->db.config.blobGranulesEnabled) { ASSERT(currColocMap[currBMProcessId] == 0); ASSERT(newColocMap[newBMProcessId] == 0); } @@ -3796,7 +3797,7 @@ void checkBetterSingletons(ClusterControllerData* self) { rkSingleton.recruit(self); } else if (newColocMap[newDDProcessId] < currColocMap[currDDProcessId]) { ddSingleton.recruit(self); - } else if (CLIENT_KNOBS->ENABLE_BLOB_GRANULES && newColocMap[newBMProcessId] < currColocMap[currBMProcessId]) { + } else if (self->db.config.blobGranulesEnabled && newColocMap[newBMProcessId] < currColocMap[currBMProcessId]) { bmSingleton.recruit(self); } } @@ -3818,7 +3819,7 @@ ACTOR Future doCheckOutstandingRequests(ClusterControllerData* self) { checkOutstandingRecruitmentRequests(self); checkOutstandingStorageRequests(self); - if (CLIENT_KNOBS->ENABLE_BLOB_GRANULES) { + if (self->db.config.blobGranulesEnabled) { checkOutstandingBlobWorkerRequests(self); } checkBetterSingletons(self); @@ -4368,7 +4369,7 @@ void registerWorker(RegisterWorkerRequest req, ClusterControllerData* self, Conf self, w, currSingleton, registeringSingleton, self->recruitingRatekeeperID); } - if (CLIENT_KNOBS->ENABLE_BLOB_GRANULES && req.blobManagerInterf.present()) { + if (self->db.config.blobGranulesEnabled && req.blobManagerInterf.present()) { auto currSingleton = BlobManagerSingleton(self->db.serverInfo->get().blobManager); auto registeringSingleton = BlobManagerSingleton(req.blobManagerInterf); haltRegisteringOrCurrentSingleton( @@ -5517,7 +5518,7 @@ ACTOR Future clusterControllerCore(ClusterControllerFullInterface interf, self.addActor.send(handleForcedRecoveries(&self, interf)); self.addActor.send(monitorDataDistributor(&self)); self.addActor.send(monitorRatekeeper(&self)); - if (CLIENT_KNOBS->ENABLE_BLOB_GRANULES) { + if (self.db.config.blobGranulesEnabled) { self.addActor.send(monitorBlobManager(&self)); } // self.addActor.send(monitorTSSMapping(&self)); diff --git a/fdbserver/SimulatedCluster.actor.cpp b/fdbserver/SimulatedCluster.actor.cpp index a60ec7e105..fd376c8cab 100644 --- a/fdbserver/SimulatedCluster.actor.cpp +++ b/fdbserver/SimulatedCluster.actor.cpp @@ -2021,7 +2021,7 @@ void setupSimulatedSystem(std::vector>* systemActors, // TODO: caching disabled for this merge int storageCacheMachines = dc == 0 ? 1 : 0; int blobWorkerMachines = 0; - if (CLIENT_KNOBS->ENABLE_BLOB_GRANULES) { + if (simconfig.db.blobGranulesEnabled) { int blobWorkerProcesses = 1 + deterministicRandom()->randomInt(0, NUM_EXTRA_BW_MACHINES + 1); blobWorkerMachines = std::max(1, blobWorkerProcesses / processesPerMachine); } diff --git a/fdbserver/Status.actor.cpp b/fdbserver/Status.actor.cpp index 85cdd71866..b3162c2542 100644 --- a/fdbserver/Status.actor.cpp +++ b/fdbserver/Status.actor.cpp @@ -797,7 +797,7 @@ ACTOR static Future processStatusFetcher( roles.addRole("ratekeeper", db->get().ratekeeper.get()); } - if (CLIENT_KNOBS->ENABLE_BLOB_GRANULES && db->get().blobManager.present()) { + if (configuration.present() && configuration.get().blobGranulesEnabled && db->get().blobManager.present()) { roles.addRole("blob_manager", db->get().blobManager.get()); } @@ -864,7 +864,7 @@ ACTOR static Future processStatusFetcher( wait(yield()); } - if (CLIENT_KNOBS->ENABLE_BLOB_GRANULES) { + if (configuration.present() && configuration.get().blobGranulesEnabled) { for (auto blobWorker : blobWorkers) { roles.addRole("blob_worker", blobWorker); wait(yield()); @@ -2890,7 +2890,7 @@ ACTOR Future clusterGetStatus( errorOr(getGrvProxiesAndMetrics(db, address_workers)); state Future>> blobWorkersFuture; - if (CLIENT_KNOBS->ENABLE_BLOB_GRANULES) { + if (configuration.present() && configuration.get().blobGranulesEnabled) { blobWorkersFuture = errorOr(timeoutError(getBlobWorkers(cx, true), 5.0)); } @@ -3011,7 +3011,7 @@ ACTOR Future clusterGetStatus( } // ...also blob workers - if (CLIENT_KNOBS->ENABLE_BLOB_GRANULES) { + if (configuration.present() && configuration.get().blobGranulesEnabled) { ErrorOr> _blobWorkers = wait(blobWorkersFuture); if (_blobWorkers.present()) { blobWorkers = _blobWorkers.get(); diff --git a/fdbserver/workloads/BlobGranuleVerifier.actor.cpp b/fdbserver/workloads/BlobGranuleVerifier.actor.cpp index bdd424c687..41385f941a 100644 --- a/fdbserver/workloads/BlobGranuleVerifier.actor.cpp +++ b/fdbserver/workloads/BlobGranuleVerifier.actor.cpp @@ -24,6 +24,7 @@ #include "contrib/fmt-8.0.1/include/fmt/format.h" #include "fdbclient/BlobGranuleReader.actor.h" +#include "fdbclient/ManagementAPI.actor.h" #include "fdbclient/NativeAPI.actor.h" #include "fdbclient/ReadYourWrites.h" #include "fdbserver/Knobs.h" @@ -60,6 +61,8 @@ struct BlobGranuleVerifierWorkload : TestWorkload { int64_t bytesRead = 0; std::vector> clients; + DatabaseConfiguration config; + Reference bstore; AsyncVar>> granuleRanges; @@ -127,7 +130,8 @@ struct BlobGranuleVerifierWorkload : TestWorkload { std::string description() const override { return "BlobGranuleVerifier"; } Future setup(Database const& cx) override { - if (!CLIENT_KNOBS->ENABLE_BLOB_GRANULES) { + config = wait(getDatabaseConfiguration(cx)); + if (!config.blobGranulesEnabled) { return Void(); } @@ -377,7 +381,7 @@ struct BlobGranuleVerifierWorkload : TestWorkload { } Future start(Database const& cx) override { - if (!CLIENT_KNOBS->ENABLE_BLOB_GRANULES) { + if (!config.blobGranulesEnabled) { return Void(); } @@ -457,7 +461,7 @@ struct BlobGranuleVerifierWorkload : TestWorkload { } Future check(Database const& cx) override { - if (!CLIENT_KNOBS->ENABLE_BLOB_GRANULES) { + if (!config.blobGranulesEnabled) { return true; } diff --git a/fdbserver/workloads/ConsistencyCheck.actor.cpp b/fdbserver/workloads/ConsistencyCheck.actor.cpp index acee855ac0..49d380bfbf 100644 --- a/fdbserver/workloads/ConsistencyCheck.actor.cpp +++ b/fdbserver/workloads/ConsistencyCheck.actor.cpp @@ -296,7 +296,7 @@ struct ConsistencyCheckWorkload : TestWorkload { wait(::success(self->checkForExtraDataStores(cx, self))); // Check blob workers are operating as expected - if (CLIENT_KNOBS->ENABLE_BLOB_GRANULES) { + if (configuration.blobGranulesEnabled) { bool blobWorkersCorrect = wait(self->checkBlobWorkers(cx, configuration, self)); if (!blobWorkersCorrect) self->testFailure("Blob workers incorrect"); @@ -2352,7 +2352,7 @@ struct ConsistencyCheckWorkload : TestWorkload { } // Check BlobManager - if (CLIENT_KNOBS->ENABLE_BLOB_GRANULES && db.blobManager.present() && + if (configuration.&& db.blobManager.present() && (!nonExcludedWorkerProcessMap.count(db.blobManager.get().address()) || nonExcludedWorkerProcessMap[db.blobManager.get().address()].processClass.machineClassFitness( ProcessClass::BlobManager) > fitnessLowerBound)) { From 0f3318a8badd441c4c0e08cb8ea3a4c7832c1483 Mon Sep 17 00:00:00 2001 From: Suraj Gupta Date: Mon, 6 Dec 2021 15:18:58 -0500 Subject: [PATCH 066/413] add TODO --- fdbclient/NativeAPI.actor.cpp | 2 ++ 1 file changed, 2 insertions(+) diff --git a/fdbclient/NativeAPI.actor.cpp b/fdbclient/NativeAPI.actor.cpp index 6d2642da62..32d7a99eb2 100644 --- a/fdbclient/NativeAPI.actor.cpp +++ b/fdbclient/NativeAPI.actor.cpp @@ -6542,6 +6542,7 @@ ACTOR Future>> getBlobGranuleRangesActor(Trans } Future>> Transaction::getBlobGranuleRanges(const KeyRange& range) { + // TODO: change to use db config if (!CLIENT_KNOBS->ENABLE_BLOB_GRANULES) { throw client_invalid_operation(); } @@ -6720,6 +6721,7 @@ Future>> Transaction::readBlobGranules Version begin, Optional readVersion, Version* readVersionOut) { + // TODO: change to use db config if (!CLIENT_KNOBS->ENABLE_BLOB_GRANULES) { throw client_invalid_operation(); } From fdc94d3356d39a28e31a0132f6802bdf75e80cad Mon Sep 17 00:00:00 2001 From: Suraj Gupta Date: Mon, 6 Dec 2021 16:56:47 -0500 Subject: [PATCH 067/413] Fix blobGranuleVerifier setup. --- fdbclient/DatabaseConfiguration.cpp | 2 +- fdbclient/NativeAPI.actor.cpp | 8 ++++-- .../workloads/BlobGranuleVerifier.actor.cpp | 26 ++++++++++++------- .../workloads/ConsistencyCheck.actor.cpp | 2 +- 4 files changed, 24 insertions(+), 14 deletions(-) diff --git a/fdbclient/DatabaseConfiguration.cpp b/fdbclient/DatabaseConfiguration.cpp index 929d60095a..439c952ea0 100644 --- a/fdbclient/DatabaseConfiguration.cpp +++ b/fdbclient/DatabaseConfiguration.cpp @@ -50,7 +50,7 @@ void DatabaseConfiguration::resetInternal() { perpetualStorageWiggleSpeed = 0; perpetualStorageWiggleLocality = "0"; storageMigrationType = StorageMigrationType::DEFAULT; - blobGranulesEnabled = true; + blobGranulesEnabled = false; } int toInt(ValueRef const& v) { diff --git a/fdbclient/NativeAPI.actor.cpp b/fdbclient/NativeAPI.actor.cpp index 32d7a99eb2..d60699f6ee 100644 --- a/fdbclient/NativeAPI.actor.cpp +++ b/fdbclient/NativeAPI.actor.cpp @@ -6543,9 +6543,11 @@ ACTOR Future>> getBlobGranuleRangesActor(Trans Future>> Transaction::getBlobGranuleRanges(const KeyRange& range) { // TODO: change to use db config + /* if (!CLIENT_KNOBS->ENABLE_BLOB_GRANULES) { - throw client_invalid_operation(); + throw client_invalid_operation(); } + */ return ::getBlobGranuleRangesActor(this, range); } @@ -6722,9 +6724,11 @@ Future>> Transaction::readBlobGranules Optional readVersion, Version* readVersionOut) { // TODO: change to use db config + /* if (!CLIENT_KNOBS->ENABLE_BLOB_GRANULES) { - throw client_invalid_operation(); + throw client_invalid_operation(); } + */ return readBlobGranulesActor(cx, this, range, begin, readVersion, readVersionOut); } diff --git a/fdbserver/workloads/BlobGranuleVerifier.actor.cpp b/fdbserver/workloads/BlobGranuleVerifier.actor.cpp index 41385f941a..38575ecf6d 100644 --- a/fdbserver/workloads/BlobGranuleVerifier.actor.cpp +++ b/fdbserver/workloads/BlobGranuleVerifier.actor.cpp @@ -129,20 +129,26 @@ struct BlobGranuleVerifierWorkload : TestWorkload { } std::string description() const override { return "BlobGranuleVerifier"; } - Future setup(Database const& cx) override { - config = wait(getDatabaseConfiguration(cx)); - if (!config.blobGranulesEnabled) { + Future setup(Database const& cx) override { return _setup(cx, this); } + + ACTOR Future _setup(Database cx, BlobGranuleVerifierWorkload* self) { + if (!self->doSetup) { + wait(delay(0)); return Void(); } - if (doSetup) { - double initialDelay = deterministicRandom()->random01() * (maxDelay - minDelay) + minDelay; - if (BGV_DEBUG) { - printf("BGW setup initial delay of %.3f\n", initialDelay); - } - return setUpBlobRange(cx, delay(initialDelay)); + DatabaseConfiguration _config = wait(getDatabaseConfiguration(cx)); + self->config = _config; + if (!self->config.blobGranulesEnabled) { + return Void(); } - return delay(0); + + double initialDelay = deterministicRandom()->random01() * (self->maxDelay - self->minDelay) + self->minDelay; + if (BGV_DEBUG) { + printf("BGW setup initial delay of %.3f\n", initialDelay); + } + wait(self->setUpBlobRange(cx, delay(initialDelay))); + return Void(); } ACTOR Future findGranules(Database cx, BlobGranuleVerifierWorkload* self) { diff --git a/fdbserver/workloads/ConsistencyCheck.actor.cpp b/fdbserver/workloads/ConsistencyCheck.actor.cpp index 49d380bfbf..23ac04805a 100644 --- a/fdbserver/workloads/ConsistencyCheck.actor.cpp +++ b/fdbserver/workloads/ConsistencyCheck.actor.cpp @@ -2352,7 +2352,7 @@ struct ConsistencyCheckWorkload : TestWorkload { } // Check BlobManager - if (configuration.&& db.blobManager.present() && + if (config.blobGranulesEnabled && db.blobManager.present() && (!nonExcludedWorkerProcessMap.count(db.blobManager.get().address()) || nonExcludedWorkerProcessMap[db.blobManager.get().address()].processClass.machineClassFitness( ProcessClass::BlobManager) > fitnessLowerBound)) { From fdb04129cd6754f6110e2729046d7ed36a9d0074 Mon Sep 17 00:00:00 2001 From: Suraj Gupta Date: Mon, 6 Dec 2021 16:58:22 -0500 Subject: [PATCH 068/413] Fix merge conflict --- fdbclient/DatabaseConfiguration.cpp | 15 --------------- 1 file changed, 15 deletions(-) diff --git a/fdbclient/DatabaseConfiguration.cpp b/fdbclient/DatabaseConfiguration.cpp index 439c952ea0..a6702b0dcc 100644 --- a/fdbclient/DatabaseConfiguration.cpp +++ b/fdbclient/DatabaseConfiguration.cpp @@ -629,21 +629,6 @@ bool DatabaseConfiguration::setInternal(KeyRef key, ValueRef value) { storageMigrationType = (StorageMigrationType::MigrationType)type; } else if (ck == LiteralStringRef("proxies")) { overwriteProxiesCount(); - int proxiesCount; - parse(&proxiesCount, value); - if (proxiesCount > 1) { - int derivedGrvProxyCount = - std::max(1, - std::min(CLIENT_KNOBS->DEFAULT_MAX_GRV_PROXIES, - proxiesCount / (CLIENT_KNOBS->DEFAULT_COMMIT_GRV_PROXIES_RATIO + 1))); - int derivedCommitProxyCount = proxiesCount - derivedGrvProxyCount; - if (grvProxyCount == -1) { - grvProxyCount = derivedGrvProxyCount; - } - if (commitProxyCount == -1) { - commitProxyCount = derivedCommitProxyCount; - } - } } else if (ck == LiteralStringRef("blob_granules_enabled")) { parse((&type), value); blobGranulesEnabled = (type != 0); From 9fbccb21d296fee8aef3d2604e48eef3cf361624 Mon Sep 17 00:00:00 2001 From: Suraj Gupta Date: Tue, 7 Dec 2021 10:05:39 -0500 Subject: [PATCH 069/413] Remove checks from native API. --- fdbclient/NativeAPI.actor.cpp | 12 ------------ 1 file changed, 12 deletions(-) diff --git a/fdbclient/NativeAPI.actor.cpp b/fdbclient/NativeAPI.actor.cpp index d60699f6ee..73b93a39b1 100644 --- a/fdbclient/NativeAPI.actor.cpp +++ b/fdbclient/NativeAPI.actor.cpp @@ -6542,12 +6542,6 @@ ACTOR Future>> getBlobGranuleRangesActor(Trans } Future>> Transaction::getBlobGranuleRanges(const KeyRange& range) { - // TODO: change to use db config - /* - if (!CLIENT_KNOBS->ENABLE_BLOB_GRANULES) { - throw client_invalid_operation(); - } - */ return ::getBlobGranuleRangesActor(this, range); } @@ -6723,12 +6717,6 @@ Future>> Transaction::readBlobGranules Version begin, Optional readVersion, Version* readVersionOut) { - // TODO: change to use db config - /* - if (!CLIENT_KNOBS->ENABLE_BLOB_GRANULES) { - throw client_invalid_operation(); - } - */ return readBlobGranulesActor(cx, this, range, begin, readVersion, readVersionOut); } From cb568bbd55f15ef842eb3c43e7674b4b47bf641f Mon Sep 17 00:00:00 2001 From: Suraj Gupta Date: Wed, 8 Dec 2021 00:43:58 -0500 Subject: [PATCH 070/413] Add watch on config key. --- fdbcli/fdbcli.actor.cpp | 1 + fdbclient/ManagementAPI.actor.cpp | 10 +++++++ fdbserver/BlobManager.actor.cpp | 22 ++++++++++++++ fdbserver/BlobManagerInterface.h | 21 +++++++++++-- fdbserver/BlobWorker.actor.cpp | 2 ++ fdbserver/ClusterController.actor.cpp | 43 ++++++++++++++++++++++++--- 6 files changed, 93 insertions(+), 6 deletions(-) diff --git a/fdbcli/fdbcli.actor.cpp b/fdbcli/fdbcli.actor.cpp index 0bd6a0bdb1..8ce87824fe 100644 --- a/fdbcli/fdbcli.actor.cpp +++ b/fdbcli/fdbcli.actor.cpp @@ -1180,6 +1180,7 @@ void configureGenerator(const char* text, const char* line, std::vector blobWorkerRecruiter( } } +ACTOR Future haltBlobGranules(BlobManagerData* bmData) { + std::vector blobWorkers = wait(getBlobWorkers(bmData->db)); + std::vector> deregisterBlobWorkers; + for (auto& worker : blobWorkers) { + printf("BM: sending halt to BW %s\n", worker.myId.toString().c_str()); + // TODO: send a special req to blob workers so they clean up granules/CFs + bmData->addActor.send( + brokenPromiseToNever(worker.haltBlobWorker.getReply(HaltBlobWorkerRequest(bmData->epoch, bmData->id)))); + deregisterBlobWorkers.emplace_back(deregisterBlobWorker(bmData, worker)); + } + waitForAll(deregisterBlobWorkers); + + return Void(); +} + ACTOR Future blobManager(BlobManagerInterface bmInterf, Reference const> dbInfo, int64_t epoch) { @@ -1420,6 +1435,13 @@ ACTOR Future blobManager(BlobManagerInterface bmInterf, TraceEvent("BlobManagerHalted", bmInterf.id()).detail("ReqID", req.requesterID); break; } + when(state HaltBlobGranulesRequest req = waitNext(bmInterf.haltBlobGranules.getFuture())) { + printf("BM: got haltBlobGranules\n"); + wait(haltBlobGranules(&self)); + req.reply.send(Void()); + TraceEvent("BlobGranulesHalted", bmInterf.id()).detail("ReqID", req.requesterID); + break; + } when(wait(collection)) { TraceEvent("BlobManagerActorCollectionError"); ASSERT(false); diff --git a/fdbserver/BlobManagerInterface.h b/fdbserver/BlobManagerInterface.h index 4f6efcd8c5..1de8174b7b 100644 --- a/fdbserver/BlobManagerInterface.h +++ b/fdbserver/BlobManagerInterface.h @@ -30,6 +30,7 @@ struct BlobManagerInterface { constexpr static FileIdentifier file_identifier = 369169; RequestStream> waitFailure; RequestStream haltBlobManager; + RequestStream haltBlobGranules; struct LocalityData locality; UID myId; @@ -44,7 +45,7 @@ struct BlobManagerInterface { template void serialize(Archive& ar) { - serializer(ar, waitFailure, haltBlobManager, locality, myId); + serializer(ar, waitFailure, haltBlobManager, haltBlobGranules, locality, myId); } }; @@ -52,9 +53,25 @@ struct HaltBlobManagerRequest { constexpr static FileIdentifier file_identifier = 4149140; UID requesterID; ReplyPromise reply; + bool haltBlobGranules; HaltBlobManagerRequest() {} - explicit HaltBlobManagerRequest(UID uid) : requesterID(uid) {} + explicit HaltBlobManagerRequest(UID uid, bool haltBlobGranules = false) + : requesterID(uid), haltBlobGranules(haltBlobGranules) {} + + template + void serialize(Ar& ar) { + serializer(ar, requesterID, reply, haltBlobGranules); + } +}; + +struct HaltBlobGranulesRequest { + constexpr static FileIdentifier file_identifier = 904267; + UID requesterID; + ReplyPromise reply; + + HaltBlobGranulesRequest() {} + explicit HaltBlobGranulesRequest(UID uid) : requesterID(uid) {} template void serialize(Ar& ar) { diff --git a/fdbserver/BlobWorker.actor.cpp b/fdbserver/BlobWorker.actor.cpp index d24993db11..d9519b983d 100644 --- a/fdbserver/BlobWorker.actor.cpp +++ b/fdbserver/BlobWorker.actor.cpp @@ -2890,6 +2890,8 @@ ACTOR Future blobWorker(BlobWorkerInterface bwInterf, self->addActor.send(handleRangeAssign(self, granuleToReassign, true)); } when(HaltBlobWorkerRequest req = waitNext(bwInterf.haltBlobWorker.getFuture())) { + printf("BW %s got halt request\n", self->id.toString().c_str()); + req.reply.send(Void()); if (self->managerEpochOk(req.managerEpoch)) { TraceEvent("BlobWorkerHalted", self->id) diff --git a/fdbserver/ClusterController.actor.cpp b/fdbserver/ClusterController.actor.cpp index d684ba824d..642f08e2f9 100644 --- a/fdbserver/ClusterController.actor.cpp +++ b/fdbserver/ClusterController.actor.cpp @@ -24,6 +24,7 @@ #include #include +#include "fdbclient/SystemData.h" #include "fdbrpc/FailureMonitor.h" #include "flow/ActorCollection.h" #include "flow/SystemMonitor.h" @@ -3352,6 +3353,13 @@ struct BlobManagerSingleton : Singleton { cc->lastRecruitTime = now(); cc->recruitBlobManager.set(true); } + void haltBlobGranules(ClusterControllerData* cc, Optional> pid) const { + printf("CC: about to send haltBlobGranules\n"); + if (interface.present()) { + cc->id_worker[pid].haltBlobManager = + brokenPromiseToNever(interface.get().haltBlobGranules.getReply(HaltBlobGranulesRequest(cc->id))); + } + } }; ACTOR Future clusterWatchDatabase(ClusterControllerData* cluster, ClusterControllerData::DBInfo* db) { @@ -5349,12 +5357,31 @@ ACTOR Future startBlobManager(ClusterControllerData* self) { } } +ACTOR Future watchBlobGranulesConfigKey(ClusterControllerData* self) { + state Reference tr = makeReference(self->cx); + + loop { + try { + tr->setOption(FDBTransactionOptions::ACCESS_SYSTEM_KEYS); + tr->setOption(FDBTransactionOptions::PRIORITY_SYSTEM_IMMEDIATE); + Key blobGranuleConfigKey = configKeysPrefix.withSuffix(StringRef("blob_granules_enabled")); + state Future watch = tr->watch(blobGranuleConfigKey); + wait(tr->commit()); + wait(watch); + return Void(); + } catch (Error& e) { + wait(tr->onError(e)); + } + } +} + ACTOR Future monitorBlobManager(ClusterControllerData* self) { while (self->db.serverInfo->get().recoveryState < RecoveryState::ACCEPTING_COMMITS) { wait(self->db.serverInfo->onChange()); } loop { + state Future watchConfigChange = watchBlobGranulesConfigKey(self); if (self->db.serverInfo->get().blobManager.present() && !self->recruitBlobManager.get()) { choose { when(wait(waitFailureClient(self->db.serverInfo->get().blobManager.get().waitFailure, @@ -5364,9 +5391,19 @@ ACTOR Future monitorBlobManager(ClusterControllerData* self) { self->db.clearInterf(ProcessClass::BlobManagerClass); } when(wait(self->recruitBlobManager.onChange())) {} + when(wait(watchConfigChange)) { + if (!self->db.config.blobGranulesEnabled) { + const auto& blobManager = self->db.serverInfo->get().blobManager; + BlobManagerSingleton(blobManager) + .haltBlobGranules(self, blobManager.get().locality.processId()); + } + } } } else { - wait(startBlobManager(self)); + wait(watchConfigChange); + if (self->db.config.blobGranulesEnabled) { + wait(startBlobManager(self)); + } } } } @@ -5518,9 +5555,7 @@ ACTOR Future clusterControllerCore(ClusterControllerFullInterface interf, self.addActor.send(handleForcedRecoveries(&self, interf)); self.addActor.send(monitorDataDistributor(&self)); self.addActor.send(monitorRatekeeper(&self)); - if (self.db.config.blobGranulesEnabled) { - self.addActor.send(monitorBlobManager(&self)); - } + self.addActor.send(monitorBlobManager(&self)); // self.addActor.send(monitorTSSMapping(&self)); self.addActor.send(dbInfoUpdater(&self)); self.addActor.send(traceCounters("ClusterControllerMetrics", From d3fbad74a24ade0c46f239942e21fc94997a67c7 Mon Sep 17 00:00:00 2001 From: Suraj Gupta Date: Wed, 8 Dec 2021 10:59:36 -0500 Subject: [PATCH 071/413] cleanup debugging --- fdbclient/DatabaseConfiguration.cpp | 2 +- fdbserver/BlobManager.actor.cpp | 2 -- fdbserver/BlobManagerInterface.h | 6 ++---- fdbserver/BlobWorker.actor.cpp | 2 -- fdbserver/ClusterController.actor.cpp | 3 ++- 5 files changed, 5 insertions(+), 10 deletions(-) diff --git a/fdbclient/DatabaseConfiguration.cpp b/fdbclient/DatabaseConfiguration.cpp index a6702b0dcc..5608539b09 100644 --- a/fdbclient/DatabaseConfiguration.cpp +++ b/fdbclient/DatabaseConfiguration.cpp @@ -50,7 +50,7 @@ void DatabaseConfiguration::resetInternal() { perpetualStorageWiggleSpeed = 0; perpetualStorageWiggleLocality = "0"; storageMigrationType = StorageMigrationType::DEFAULT; - blobGranulesEnabled = false; + blobGranulesEnabled = true; // TODO: default to false if merging to master } int toInt(ValueRef const& v) { diff --git a/fdbserver/BlobManager.actor.cpp b/fdbserver/BlobManager.actor.cpp index c81fb4c487..5e906a07c4 100644 --- a/fdbserver/BlobManager.actor.cpp +++ b/fdbserver/BlobManager.actor.cpp @@ -1361,7 +1361,6 @@ ACTOR Future haltBlobGranules(BlobManagerData* bmData) { std::vector blobWorkers = wait(getBlobWorkers(bmData->db)); std::vector> deregisterBlobWorkers; for (auto& worker : blobWorkers) { - printf("BM: sending halt to BW %s\n", worker.myId.toString().c_str()); // TODO: send a special req to blob workers so they clean up granules/CFs bmData->addActor.send( brokenPromiseToNever(worker.haltBlobWorker.getReply(HaltBlobWorkerRequest(bmData->epoch, bmData->id)))); @@ -1436,7 +1435,6 @@ ACTOR Future blobManager(BlobManagerInterface bmInterf, break; } when(state HaltBlobGranulesRequest req = waitNext(bmInterf.haltBlobGranules.getFuture())) { - printf("BM: got haltBlobGranules\n"); wait(haltBlobGranules(&self)); req.reply.send(Void()); TraceEvent("BlobGranulesHalted", bmInterf.id()).detail("ReqID", req.requesterID); diff --git a/fdbserver/BlobManagerInterface.h b/fdbserver/BlobManagerInterface.h index 1de8174b7b..8399e136c2 100644 --- a/fdbserver/BlobManagerInterface.h +++ b/fdbserver/BlobManagerInterface.h @@ -53,15 +53,13 @@ struct HaltBlobManagerRequest { constexpr static FileIdentifier file_identifier = 4149140; UID requesterID; ReplyPromise reply; - bool haltBlobGranules; HaltBlobManagerRequest() {} - explicit HaltBlobManagerRequest(UID uid, bool haltBlobGranules = false) - : requesterID(uid), haltBlobGranules(haltBlobGranules) {} + explicit HaltBlobManagerRequest(UID uid) : requesterID(uid) {} template void serialize(Ar& ar) { - serializer(ar, requesterID, reply, haltBlobGranules); + serializer(ar, requesterID, reply); } }; diff --git a/fdbserver/BlobWorker.actor.cpp b/fdbserver/BlobWorker.actor.cpp index d9519b983d..d24993db11 100644 --- a/fdbserver/BlobWorker.actor.cpp +++ b/fdbserver/BlobWorker.actor.cpp @@ -2890,8 +2890,6 @@ ACTOR Future blobWorker(BlobWorkerInterface bwInterf, self->addActor.send(handleRangeAssign(self, granuleToReassign, true)); } when(HaltBlobWorkerRequest req = waitNext(bwInterf.haltBlobWorker.getFuture())) { - printf("BW %s got halt request\n", self->id.toString().c_str()); - req.reply.send(Void()); if (self->managerEpochOk(req.managerEpoch)) { TraceEvent("BlobWorkerHalted", self->id) diff --git a/fdbserver/ClusterController.actor.cpp b/fdbserver/ClusterController.actor.cpp index 642f08e2f9..226674d39e 100644 --- a/fdbserver/ClusterController.actor.cpp +++ b/fdbserver/ClusterController.actor.cpp @@ -3354,7 +3354,6 @@ struct BlobManagerSingleton : Singleton { cc->recruitBlobManager.set(true); } void haltBlobGranules(ClusterControllerData* cc, Optional> pid) const { - printf("CC: about to send haltBlobGranules\n"); if (interface.present()) { cc->id_worker[pid].haltBlobManager = brokenPromiseToNever(interface.get().haltBlobGranules.getReply(HaltBlobGranulesRequest(cc->id))); @@ -5392,6 +5391,7 @@ ACTOR Future monitorBlobManager(ClusterControllerData* self) { } when(wait(self->recruitBlobManager.onChange())) {} when(wait(watchConfigChange)) { + // if there is a blob manager present but blob granules are now disabled, stop the BM if (!self->db.config.blobGranulesEnabled) { const auto& blobManager = self->db.serverInfo->get().blobManager; BlobManagerSingleton(blobManager) @@ -5401,6 +5401,7 @@ ACTOR Future monitorBlobManager(ClusterControllerData* self) { } } else { wait(watchConfigChange); + // if there is no blob manager present but blob granules are now enabled, recruit a BM if (self->db.config.blobGranulesEnabled) { wait(startBlobManager(self)); } From 640cee20725c11693e0a21bc6fba122e927ecd3f Mon Sep 17 00:00:00 2001 From: Suraj Gupta Date: Wed, 8 Dec 2021 14:12:44 -0500 Subject: [PATCH 072/413] Start BM without config change if config is enabled. --- fdbclient/Schemas.cpp | 2 +- fdbserver/ClusterController.actor.cpp | 10 +++++----- 2 files changed, 6 insertions(+), 6 deletions(-) diff --git a/fdbclient/Schemas.cpp b/fdbclient/Schemas.cpp index d2725e8f5f..b9d6d9a72c 100644 --- a/fdbclient/Schemas.cpp +++ b/fdbclient/Schemas.cpp @@ -771,7 +771,7 @@ const KeyRef JSONSchemas::statusSchema = LiteralStringRef(R"statusSchema( "aggressive", "gradual" ]}, - "blob_granules_enabled":1, + "blob_granules_enabled":0 }, "data":{ "least_operating_space_bytes_log_server":0, diff --git a/fdbserver/ClusterController.actor.cpp b/fdbserver/ClusterController.actor.cpp index 226674d39e..d0949c2e23 100644 --- a/fdbserver/ClusterController.actor.cpp +++ b/fdbserver/ClusterController.actor.cpp @@ -5399,12 +5399,12 @@ ACTOR Future monitorBlobManager(ClusterControllerData* self) { } } } - } else { - wait(watchConfigChange); + } else if (self->db.config.blobGranulesEnabled) { // if there is no blob manager present but blob granules are now enabled, recruit a BM - if (self->db.config.blobGranulesEnabled) { - wait(startBlobManager(self)); - } + wait(startBlobManager(self)); + } else { + // if there is no blob manager present and blob granules are disabled, wait for a config change + wait(watchConfigChange); } } } From 6765d3dff7a8a47ae8acde70139c5fa35ae0ce63 Mon Sep 17 00:00:00 2001 From: Suraj Gupta Date: Wed, 8 Dec 2021 14:15:10 -0500 Subject: [PATCH 073/413] Default to blob granules disabled. --- fdbclient/DatabaseConfiguration.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/fdbclient/DatabaseConfiguration.cpp b/fdbclient/DatabaseConfiguration.cpp index 5608539b09..a6702b0dcc 100644 --- a/fdbclient/DatabaseConfiguration.cpp +++ b/fdbclient/DatabaseConfiguration.cpp @@ -50,7 +50,7 @@ void DatabaseConfiguration::resetInternal() { perpetualStorageWiggleSpeed = 0; perpetualStorageWiggleLocality = "0"; storageMigrationType = StorageMigrationType::DEFAULT; - blobGranulesEnabled = true; // TODO: default to false if merging to master + blobGranulesEnabled = false; } int toInt(ValueRef const& v) { From 968a4f9f50dbb11fd3a62e56e64e285d88c2e463 Mon Sep 17 00:00:00 2001 From: Suraj Gupta Date: Thu, 9 Dec 2021 16:22:11 -0500 Subject: [PATCH 074/413] Don't rely on database config to be updated. --- fdbserver/ClusterController.actor.cpp | 41 ++++++++++++------- fdbserver/SimulatedCluster.actor.cpp | 9 +++- .../workloads/BlobGranuleVerifier.actor.cpp | 18 +------- tests/fast/BlobGranuleCorrectness.toml | 3 ++ tests/fast/BlobGranuleCorrectnessClean.toml | 3 ++ tests/fast/BlobGranuleCycle.toml | 3 ++ tests/fast/BlobGranuleCycleClean.toml | 3 ++ tests/slow/BlobGranuleCorrectnessLarge.toml | 3 ++ .../BlobGranuleCorrectnessLargeClean.toml | 3 ++ 9 files changed, 53 insertions(+), 33 deletions(-) diff --git a/fdbserver/ClusterController.actor.cpp b/fdbserver/ClusterController.actor.cpp index d0949c2e23..3b09bd4f57 100644 --- a/fdbserver/ClusterController.actor.cpp +++ b/fdbserver/ClusterController.actor.cpp @@ -141,6 +141,7 @@ public: std::map> clientStatus; Future clientCounter; int clientCount; + AsyncVar blobGranulesEnabled; DBInfo() : clientInfo(new AsyncVar()), serverInfo(new AsyncVar()), @@ -151,7 +152,8 @@ public: EnableLocalityLoadBalance::True, TaskPriority::DefaultEndpoint, LockAware::True)), // SOMEDAY: Locality! - unfinishedRecoveries(0), logGenerations(0), cachePopulated(false), clientCount(0) { + unfinishedRecoveries(0), logGenerations(0), cachePopulated(false), clientCount(0), + blobGranulesEnabled(config.blobGranulesEnabled) { clientCounter = countClients(this); } @@ -3727,7 +3729,7 @@ void checkBetterSingletons(ClusterControllerData* self) { WorkerDetails newDDWorker = findNewProcessForSingleton(self, ProcessClass::DataDistributor, id_used); WorkerDetails newBMWorker; - if (self->db.config.blobGranulesEnabled) { + if (self->db.blobGranulesEnabled.get()) { newBMWorker = findNewProcessForSingleton(self, ProcessClass::BlobManager, id_used); } @@ -3736,7 +3738,7 @@ void checkBetterSingletons(ClusterControllerData* self) { auto bestFitnessForDD = findBestFitnessForSingleton(self, newDDWorker, ProcessClass::DataDistributor); ProcessClass::Fitness bestFitnessForBM; - if (self->db.config.blobGranulesEnabled) { + if (self->db.blobGranulesEnabled.get()) { bestFitnessForBM = findBestFitnessForSingleton(self, newBMWorker, ProcessClass::BlobManager); } @@ -3754,7 +3756,7 @@ void checkBetterSingletons(ClusterControllerData* self) { self, newDDWorker, ddSingleton, bestFitnessForDD, self->recruitingDistributorID); bool bmHealthy = true; - if (self->db.config.blobGranulesEnabled) { + if (self->db.blobGranulesEnabled.get()) { bmHealthy = isHealthySingleton( self, newBMWorker, bmSingleton, bestFitnessForBM, self->recruitingBlobManagerID); } @@ -3773,7 +3775,7 @@ void checkBetterSingletons(ClusterControllerData* self) { Optional> newDDProcessId = newDDWorker.interf.locality.processId(); Optional> currBMProcessId, newBMProcessId; - if (self->db.config.blobGranulesEnabled) { + if (self->db.blobGranulesEnabled.get()) { currBMProcessId = bmSingleton.interface.get().locality.processId(); newBMProcessId = newBMWorker.interf.locality.processId(); @@ -3781,7 +3783,7 @@ void checkBetterSingletons(ClusterControllerData* self) { std::vector>> currPids = { currRKProcessId, currDDProcessId }; std::vector>> newPids = { newRKProcessId, newDDProcessId }; - if (self->db.config.blobGranulesEnabled) { + if (self->db.blobGranulesEnabled.get()) { currPids.emplace_back(currBMProcessId); newPids.emplace_back(newBMProcessId); } @@ -3790,7 +3792,7 @@ void checkBetterSingletons(ClusterControllerData* self) { auto newColocMap = getColocCounts(newPids); // if the knob is disabled, the BM coloc counts should have no affect on the coloc counts check below - if (!self->db.config.blobGranulesEnabled) { + if (!self->db.blobGranulesEnabled.get()) { ASSERT(currColocMap[currBMProcessId] == 0); ASSERT(newColocMap[newBMProcessId] == 0); } @@ -3804,7 +3806,7 @@ void checkBetterSingletons(ClusterControllerData* self) { rkSingleton.recruit(self); } else if (newColocMap[newDDProcessId] < currColocMap[currDDProcessId]) { ddSingleton.recruit(self); - } else if (self->db.config.blobGranulesEnabled && newColocMap[newBMProcessId] < currColocMap[currBMProcessId]) { + } else if (self->db.blobGranulesEnabled.get() && newColocMap[newBMProcessId] < currColocMap[currBMProcessId]) { bmSingleton.recruit(self); } } @@ -3826,7 +3828,7 @@ ACTOR Future doCheckOutstandingRequests(ClusterControllerData* self) { checkOutstandingRecruitmentRequests(self); checkOutstandingStorageRequests(self); - if (self->db.config.blobGranulesEnabled) { + if (self->db.blobGranulesEnabled.get()) { checkOutstandingBlobWorkerRequests(self); } checkBetterSingletons(self); @@ -4376,7 +4378,7 @@ void registerWorker(RegisterWorkerRequest req, ClusterControllerData* self, Conf self, w, currSingleton, registeringSingleton, self->recruitingRatekeeperID); } - if (self->db.config.blobGranulesEnabled && req.blobManagerInterf.present()) { + if (self->db.blobGranulesEnabled.get() && req.blobManagerInterf.present()) { auto currSingleton = BlobManagerSingleton(self->db.serverInfo->get().blobManager); auto registeringSingleton = BlobManagerSingleton(req.blobManagerInterf); haltRegisteringOrCurrentSingleton( @@ -5358,16 +5360,24 @@ ACTOR Future startBlobManager(ClusterControllerData* self) { ACTOR Future watchBlobGranulesConfigKey(ClusterControllerData* self) { state Reference tr = makeReference(self->cx); + state Key blobGranuleConfigKey = configKeysPrefix.withSuffix(StringRef("blob_granules_enabled")); loop { try { tr->setOption(FDBTransactionOptions::ACCESS_SYSTEM_KEYS); tr->setOption(FDBTransactionOptions::PRIORITY_SYSTEM_IMMEDIATE); - Key blobGranuleConfigKey = configKeysPrefix.withSuffix(StringRef("blob_granules_enabled")); state Future watch = tr->watch(blobGranuleConfigKey); wait(tr->commit()); wait(watch); - return Void(); + + tr->reset(); + + tr->setOption(FDBTransactionOptions::ACCESS_SYSTEM_KEYS); + tr->setOption(FDBTransactionOptions::PRIORITY_SYSTEM_IMMEDIATE); + Optional blobConfig = wait(tr->get(blobGranuleConfigKey)); + if (blobConfig.present()) { + self->db.blobGranulesEnabled.set(blobConfig.get() == LiteralStringRef("1")); + } } catch (Error& e) { wait(tr->onError(e)); } @@ -5392,19 +5402,19 @@ ACTOR Future monitorBlobManager(ClusterControllerData* self) { when(wait(self->recruitBlobManager.onChange())) {} when(wait(watchConfigChange)) { // if there is a blob manager present but blob granules are now disabled, stop the BM - if (!self->db.config.blobGranulesEnabled) { + if (!self->db.blobGranulesEnabled.get()) { const auto& blobManager = self->db.serverInfo->get().blobManager; BlobManagerSingleton(blobManager) .haltBlobGranules(self, blobManager.get().locality.processId()); } } } - } else if (self->db.config.blobGranulesEnabled) { + } else if (self->db.blobGranulesEnabled.get()) { // if there is no blob manager present but blob granules are now enabled, recruit a BM wait(startBlobManager(self)); } else { // if there is no blob manager present and blob granules are disabled, wait for a config change - wait(watchConfigChange); + wait(self->db.blobGranulesEnabled.onChange()); } } } @@ -5557,6 +5567,7 @@ ACTOR Future clusterControllerCore(ClusterControllerFullInterface interf, self.addActor.send(monitorDataDistributor(&self)); self.addActor.send(monitorRatekeeper(&self)); self.addActor.send(monitorBlobManager(&self)); + self.addActor.send(watchBlobGranulesConfigKey(&self)); // self.addActor.send(monitorTSSMapping(&self)); self.addActor.send(dbInfoUpdater(&self)); self.addActor.send(traceCounters("ClusterControllerMetrics", diff --git a/fdbserver/SimulatedCluster.actor.cpp b/fdbserver/SimulatedCluster.actor.cpp index fd376c8cab..653531e106 100644 --- a/fdbserver/SimulatedCluster.actor.cpp +++ b/fdbserver/SimulatedCluster.actor.cpp @@ -263,6 +263,9 @@ class TestConfig { configDBType = configDBTypeFromString(value); } } + if (attrib == "blobGranulesEnabled") { + blobGranulesEnabled = strcmp(value.c_str(), "true") == 0; + } } ifs.close(); @@ -297,6 +300,7 @@ public: Optional generateFearless, buggify; Optional datacenters, desiredTLogCount, commitProxyCount, grvProxyCount, resolverCount, storageEngineType, stderrSeverity, machineCount, processesPerMachine, coordinators; + bool blobGranulesEnabled = false; Optional config; ConfigDBType getConfigDBType() const { return configDBType; } @@ -348,7 +352,8 @@ public: .add("processesPerMachine", &processesPerMachine) .add("coordinators", &coordinators) .add("configDB", &configDBType) - .add("extraMachineCountDC", &extraMachineCountDC); + .add("extraMachineCountDC", &extraMachineCountDC) + .add("blobGranulesEnabled", &blobGranulesEnabled); try { auto file = toml::parse(testFile); if (file.contains("configuration") && toml::find(file, "configuration").is_table()) { @@ -2021,7 +2026,7 @@ void setupSimulatedSystem(std::vector>* systemActors, // TODO: caching disabled for this merge int storageCacheMachines = dc == 0 ? 1 : 0; int blobWorkerMachines = 0; - if (simconfig.db.blobGranulesEnabled) { + if (testConfig.blobGranulesEnabled) { int blobWorkerProcesses = 1 + deterministicRandom()->randomInt(0, NUM_EXTRA_BW_MACHINES + 1); blobWorkerMachines = std::max(1, blobWorkerProcesses / processesPerMachine); } diff --git a/fdbserver/workloads/BlobGranuleVerifier.actor.cpp b/fdbserver/workloads/BlobGranuleVerifier.actor.cpp index 38575ecf6d..f5f4f5cda2 100644 --- a/fdbserver/workloads/BlobGranuleVerifier.actor.cpp +++ b/fdbserver/workloads/BlobGranuleVerifier.actor.cpp @@ -137,11 +137,7 @@ struct BlobGranuleVerifierWorkload : TestWorkload { return Void(); } - DatabaseConfiguration _config = wait(getDatabaseConfiguration(cx)); - self->config = _config; - if (!self->config.blobGranulesEnabled) { - return Void(); - } + wait(success(ManagementAPI::changeConfig(cx.getReference(), "blob_granules_enabled=1", true))); double initialDelay = deterministicRandom()->random01() * (self->maxDelay - self->minDelay) + self->minDelay; if (BGV_DEBUG) { @@ -387,10 +383,6 @@ struct BlobGranuleVerifierWorkload : TestWorkload { } Future start(Database const& cx) override { - if (!config.blobGranulesEnabled) { - return Void(); - } - clients.reserve(threads + 1); clients.push_back(timeout(findGranules(cx, this), testDuration, Void())); for (int i = 0; i < threads; i++) { @@ -466,13 +458,7 @@ struct BlobGranuleVerifierWorkload : TestWorkload { return availabilityPassed && self->mismatches == 0 && checks > 0 && self->timeTravelTooOld == 0; } - Future check(Database const& cx) override { - if (!config.blobGranulesEnabled) { - return true; - } - - return _check(cx, this); - } + Future check(Database const& cx) override { return _check(cx, this); } void getMetrics(std::vector& m) override {} }; diff --git a/tests/fast/BlobGranuleCorrectness.toml b/tests/fast/BlobGranuleCorrectness.toml index 20446ec66c..d6fb875474 100644 --- a/tests/fast/BlobGranuleCorrectness.toml +++ b/tests/fast/BlobGranuleCorrectness.toml @@ -1,3 +1,6 @@ +[configuration] +blobGranulesEnabled = true + [[test]] testTitle = 'BlobGranuleCorrectnessTest' diff --git a/tests/fast/BlobGranuleCorrectnessClean.toml b/tests/fast/BlobGranuleCorrectnessClean.toml index 168790dd9d..b58bba4dbe 100644 --- a/tests/fast/BlobGranuleCorrectnessClean.toml +++ b/tests/fast/BlobGranuleCorrectnessClean.toml @@ -1,3 +1,6 @@ +[configuration] +blobGranulesEnabled = true + [[test]] testTitle = 'BlobGranuleCorrectnessCleanTest' diff --git a/tests/fast/BlobGranuleCycle.toml b/tests/fast/BlobGranuleCycle.toml index 8f8fe9f030..1b112aef41 100644 --- a/tests/fast/BlobGranuleCycle.toml +++ b/tests/fast/BlobGranuleCycle.toml @@ -1,3 +1,6 @@ +[configuration] +blobGranulesEnabled = true + [[test]] testTitle = 'BlobGranuleCycle' diff --git a/tests/fast/BlobGranuleCycleClean.toml b/tests/fast/BlobGranuleCycleClean.toml index 57e4676c9d..dca580d37f 100644 --- a/tests/fast/BlobGranuleCycleClean.toml +++ b/tests/fast/BlobGranuleCycleClean.toml @@ -1,3 +1,6 @@ +[configuration] +blobGranulesEnabled = true + [[test]] testTitle = 'BlobGranuleCycleClean' diff --git a/tests/slow/BlobGranuleCorrectnessLarge.toml b/tests/slow/BlobGranuleCorrectnessLarge.toml index edf400b6d9..b9f13f5d83 100644 --- a/tests/slow/BlobGranuleCorrectnessLarge.toml +++ b/tests/slow/BlobGranuleCorrectnessLarge.toml @@ -1,3 +1,6 @@ +[configuration] +blobGranulesEnabled = true + [[test]] testTitle = 'BlobGranuleCorrectnessLargeTest' diff --git a/tests/slow/BlobGranuleCorrectnessLargeClean.toml b/tests/slow/BlobGranuleCorrectnessLargeClean.toml index 1a12e6f47f..90568e8f39 100644 --- a/tests/slow/BlobGranuleCorrectnessLargeClean.toml +++ b/tests/slow/BlobGranuleCorrectnessLargeClean.toml @@ -1,3 +1,6 @@ +[configuration] +blobGranulesEnabled = true + [[test]] testTitle = 'BlobGranuleCorrectnessLargeCleanTest' From c89fd0c3a80792e9b063fb96696b6bc14d7e1f93 Mon Sep 17 00:00:00 2001 From: Suraj Gupta Date: Thu, 9 Dec 2021 17:06:18 -0500 Subject: [PATCH 075/413] reset txn after getting --- fdbserver/ClusterController.actor.cpp | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/fdbserver/ClusterController.actor.cpp b/fdbserver/ClusterController.actor.cpp index 3b09bd4f57..c56b49eca5 100644 --- a/fdbserver/ClusterController.actor.cpp +++ b/fdbserver/ClusterController.actor.cpp @@ -5364,16 +5364,18 @@ ACTOR Future watchBlobGranulesConfigKey(ClusterControllerData* self) { loop { try { + tr->reset(); tr->setOption(FDBTransactionOptions::ACCESS_SYSTEM_KEYS); tr->setOption(FDBTransactionOptions::PRIORITY_SYSTEM_IMMEDIATE); + state Future watch = tr->watch(blobGranuleConfigKey); wait(tr->commit()); wait(watch); tr->reset(); - tr->setOption(FDBTransactionOptions::ACCESS_SYSTEM_KEYS); tr->setOption(FDBTransactionOptions::PRIORITY_SYSTEM_IMMEDIATE); + Optional blobConfig = wait(tr->get(blobGranuleConfigKey)); if (blobConfig.present()) { self->db.blobGranulesEnabled.set(blobConfig.get() == LiteralStringRef("1")); From f68c1fd3c4d212b24575fb04311ff58e9175538c Mon Sep 17 00:00:00 2001 From: Suraj Gupta Date: Fri, 10 Dec 2021 14:05:59 -0500 Subject: [PATCH 076/413] Use _sr and fix disabling case. --- fdbserver/ClusterController.actor.cpp | 36 +++++++++++++++------------ 1 file changed, 20 insertions(+), 16 deletions(-) diff --git a/fdbserver/ClusterController.actor.cpp b/fdbserver/ClusterController.actor.cpp index c56b49eca5..039df63bbe 100644 --- a/fdbserver/ClusterController.actor.cpp +++ b/fdbserver/ClusterController.actor.cpp @@ -5360,7 +5360,7 @@ ACTOR Future startBlobManager(ClusterControllerData* self) { ACTOR Future watchBlobGranulesConfigKey(ClusterControllerData* self) { state Reference tr = makeReference(self->cx); - state Key blobGranuleConfigKey = configKeysPrefix.withSuffix(StringRef("blob_granules_enabled")); + state Key blobGranuleConfigKey = configKeysPrefix.withSuffix("blob_granules_enabled"_sr); loop { try { @@ -5392,22 +5392,26 @@ ACTOR Future monitorBlobManager(ClusterControllerData* self) { } loop { - state Future watchConfigChange = watchBlobGranulesConfigKey(self); if (self->db.serverInfo->get().blobManager.present() && !self->recruitBlobManager.get()) { - choose { - when(wait(waitFailureClient(self->db.serverInfo->get().blobManager.get().waitFailure, - SERVER_KNOBS->BLOB_MANAGER_FAILURE_TIME))) { - TraceEvent("CCBlobManagerDied", self->id) - .detail("BMID", self->db.serverInfo->get().blobManager.get().id()); - self->db.clearInterf(ProcessClass::BlobManagerClass); - } - when(wait(self->recruitBlobManager.onChange())) {} - when(wait(watchConfigChange)) { - // if there is a blob manager present but blob granules are now disabled, stop the BM - if (!self->db.blobGranulesEnabled.get()) { - const auto& blobManager = self->db.serverInfo->get().blobManager; - BlobManagerSingleton(blobManager) - .haltBlobGranules(self, blobManager.get().locality.processId()); + state Future wfClient = waitFailureClient(self->db.serverInfo->get().blobManager.get().waitFailure, + SERVER_KNOBS->BLOB_MANAGER_FAILURE_TIME); + loop { + choose { + when(wait(wfClient)) { + TraceEvent("CCBlobManagerDied", self->id) + .detail("BMID", self->db.serverInfo->get().blobManager.get().id()); + self->db.clearInterf(ProcessClass::BlobManagerClass); + break; + } + when(wait(self->recruitBlobManager.onChange())) { break; } + when(wait(self->db.blobGranulesEnabled.onChange())) { + // if there is a blob manager present but blob granules are now disabled, stop the BM + if (!self->db.blobGranulesEnabled.get()) { + const auto& blobManager = self->db.serverInfo->get().blobManager; + BlobManagerSingleton(blobManager) + .haltBlobGranules(self, blobManager.get().locality.processId()); + break; + } } } } From ff2cd691cdfacc076f5fc3e967c91a34d2dad46e Mon Sep 17 00:00:00 2001 From: Josh Slocum Date: Fri, 10 Dec 2021 15:27:25 -0600 Subject: [PATCH 077/413] Switching back to GRV for committed version checking, with proper rollback checking --- fdbclient/NativeAPI.actor.cpp | 28 +++--- fdbclient/Notified.h | 2 + fdbserver/BlobWorker.actor.cpp | 161 ++++++++++++++++++++++----------- 3 files changed, 123 insertions(+), 68 deletions(-) diff --git a/fdbclient/NativeAPI.actor.cpp b/fdbclient/NativeAPI.actor.cpp index 6d2642da62..23107e027c 100644 --- a/fdbclient/NativeAPI.actor.cpp +++ b/fdbclient/NativeAPI.actor.cpp @@ -7163,26 +7163,28 @@ ACTOR Future changeFeedWaitLatest(ChangeFeedData* self, Version version) { } ACTOR Future changeFeedWhenAtLatest(ChangeFeedData* self, Version version) { - state Future lastReturned = self->lastReturnedVersion.whenAtLeast(version); if (DEBUG_CF_WAIT_VERSION == version) { fmt::print("CFW {0}) WhenAtLeast: LR={1}\n", version, self->lastReturnedVersion.get()); } + if (version <= self->getVersion()) { + if (DEBUG_CF_WAIT_VERSION == version) { + fmt::print("CFW {0}) WhenAtLeast: Already done\n", version, self->lastReturnedVersion.get()); + } + return Void(); + } + state Future lastReturned = self->lastReturnedVersion.whenAtLeast(version); + loop { if (DEBUG_CF_WAIT_VERSION == version) { fmt::print("CFW {0}) WhenAtLeast: NotAtLatest={1}\n", version, self->notAtLatest.get()); } - if (self->notAtLatest.get() == 0) { - choose { - when(wait(changeFeedWaitLatest(self, version))) { break; } - when(wait(self->refresh.getFuture())) {} - when(wait(self->notAtLatest.onChange())) {} - } - } else { - choose { - when(wait(lastReturned)) { break; } - when(wait(self->notAtLatest.onChange())) {} - when(wait(self->refresh.getFuture())) {} - } + // only allowed to use empty versions if you're caught up + Future waitEmptyVersion = (self->notAtLatest.get() == 0) ? changeFeedWaitLatest(self, version) : Never(); + choose { + when(wait(waitEmptyVersion)) { break; } + when(wait(lastReturned)) { break; } + when(wait(self->refresh.getFuture())) {} + when(wait(self->notAtLatest.onChange())) {} } } diff --git a/fdbclient/Notified.h b/fdbclient/Notified.h index 94a1bb2144..dc18cda123 100644 --- a/fdbclient/Notified.h +++ b/fdbclient/Notified.h @@ -80,6 +80,8 @@ struct Notified { val = std::move(r.val); } + int numWaiting() { return waiting.size(); } + private: using Item = std::pair>; struct ItemCompare { diff --git a/fdbserver/BlobWorker.actor.cpp b/fdbserver/BlobWorker.actor.cpp index d24993db11..aba8332dce 100644 --- a/fdbserver/BlobWorker.actor.cpp +++ b/fdbserver/BlobWorker.actor.cpp @@ -200,6 +200,9 @@ struct BlobWorkerData : NonCopyable, ReferenceCounted { PromiseStream granuleUpdateErrors; + Promise doGRVCheck; + NotifiedVersion grvVersion; + BlobWorkerData(UID id, Database db) : id(id), db(db), stats(id, SERVER_KNOBS->WORKER_LOGGING_INTERVAL) {} bool managerEpochOk(int64_t epoch) { @@ -497,7 +500,7 @@ ACTOR Future writeDeltaFile(Reference bwData, GranuleDeltas deltasToWrite, Version currentDeltaVersion, Future previousDeltaFileFuture, - NotifiedVersion* granuleCommittedVersion, + Future waitCommitted, Optional> oldGranuleComplete) { wait(delay(0, TaskPriority::BlobWorkerUpdateStorage)); @@ -522,9 +525,8 @@ ACTOR Future writeDeltaFile(Reference bwData, state int numIterations = 0; try { // before updating FDB, wait for the delta file version to be committed and previous delta files to finish - if (currentDeltaVersion > granuleCommittedVersion->get()) { - wait(granuleCommittedVersion->whenAtLeast(currentDeltaVersion)); - } + // TODO fix file leak here on error pre-transaction. + wait(waitCommitted); BlobFileIndex prev = wait(previousDeltaFileFuture); wait(delay(0, TaskPriority::BlobWorkerUpdateFDB)); @@ -1139,6 +1141,50 @@ static Version doGranuleRollback(Reference metadata, return cfRollbackVersion; } +ACTOR Future waitVersionCommitted(Reference bwData, + Reference metadata, + Version version) { + // TODO REMOVE debugs + if (version > bwData->grvVersion.get()) { + /*if (BW_DEBUG) { + fmt::print("waitVersionCommitted waiting {0}\n", version); + }*/ + // this order is important, since we need to register a waiter on the notified version before waking the GRV + // actor + Future grvAtLeast = bwData->grvVersion.whenAtLeast(version); + if (bwData->doGRVCheck.canBeSet()) { + bwData->doGRVCheck.send(Void()); + } + wait(grvAtLeast); + } + state Version grvVersion = bwData->grvVersion.get(); + /*if (BW_DEBUG) { + fmt::print("waitVersionCommitted got {0} < {1}, waiting on CF (currently {2})\n", + version, + grvVersion, + metadata->activeCFData.get()->getVersion()); + }*/ + // make sure the change feed has consumed mutations up through grvVersion to ensure none of them are rollbacks + + loop { + state Future atLeast = metadata->activeCFData.get()->whenAtLeast(grvVersion); + choose { + when(wait(atLeast)) { break; } + when(wait(metadata->activeCFData.onChange())) {} + } + } + // sanity check to make sure whenAtLeast didn't return early + if (grvVersion > metadata->waitForVersionReturned) { + metadata->waitForVersionReturned = grvVersion; + } + /*if (BW_DEBUG) { + fmt::print( + "waitVersionCommitted CF whenAtLeast {0}: {1}\n", grvVersion, metadata->activeCFData.get()->getVersion()); + }*/ + + return Void(); +} + // TODO REMOVE once correctness clean #define DEBUG_BW_START_VERSION invalidVersion #define DEBUG_BW_END_VERSION invalidVersion @@ -1159,7 +1205,6 @@ ACTOR Future blobGranuleUpdateFiles(Reference bwData, state Optional> oldChangeFeedDataComplete; state Key cfKey; state Optional oldCFKey; - state NotifiedVersion committedVersion; state int pendingSnapshots = 0; state std::deque> rollbacksInProgress; @@ -1248,7 +1293,6 @@ ACTOR Future blobGranuleUpdateFiles(Reference bwData, metadata->durableDeltaVersion.set(startVersion); metadata->pendingDeltaVersion = startVersion; metadata->bufferedDeltaVersion = startVersion; - committedVersion.set(startVersion); metadata->activeCFData.set(newChangeFeedData(startVersion)); if (startState.parentGranule.present() && startVersion < startState.changeFeedStartVersion) { @@ -1383,7 +1427,6 @@ ACTOR Future blobGranuleUpdateFiles(Reference bwData, // process mutations if (!mutations.empty()) { bool processedAnyMutations = false; - Version knownNoRollbacksPast = invalidVersion; Version lastDeltaVersion = invalidVersion; for (MutationsAndVersionRef deltas : mutations) { @@ -1404,7 +1447,6 @@ ACTOR Future blobGranuleUpdateFiles(Reference bwData, br >> rollbackVersion; ASSERT(rollbackVersion >= metadata->durableDeltaVersion.get()); - ASSERT(rollbackVersion >= committedVersion.get()); if (!rollbacksInProgress.empty()) { ASSERT(rollbacksInProgress.front().first == rollbackVersion); @@ -1492,10 +1534,7 @@ ACTOR Future blobGranuleUpdateFiles(Reference bwData, .detail("OldChangeFeed", readOldChangeFeed ? "T" : "F"); } if (DEBUG_BW_VERSION(deltas.version)) { - fmt::print("BW {0}: ({1}), KCV={2}\n", - deltas.version, - deltas.mutations.size(), - deltas.knownCommittedVersion); + fmt::print("BW {0}: ({1})\n", deltas.version, deltas.mutations.size()); } metadata->currentDeltas.push_back_deep(metadata->deltaArena, deltas); @@ -1503,14 +1542,6 @@ ACTOR Future blobGranuleUpdateFiles(Reference bwData, ASSERT(deltas.version != invalidVersion); ASSERT(deltas.version > lastDeltaVersion); lastDeltaVersion = deltas.version; - - Version nextKnownNoRollbacksPast = std::min(deltas.version, deltas.knownCommittedVersion); - ASSERT(nextKnownNoRollbacksPast >= knownNoRollbacksPast || - nextKnownNoRollbacksPast == invalidVersion); - - if (nextKnownNoRollbacksPast != invalidVersion) { - knownNoRollbacksPast = nextKnownNoRollbacksPast; - } } } if (justDidRollback) { @@ -1518,24 +1549,12 @@ ACTOR Future blobGranuleUpdateFiles(Reference bwData, } } if (!justDidRollback && processedAnyMutations) { - // update buffered version and committed version + // update buffered version ASSERT(lastDeltaVersion != invalidVersion); ASSERT(lastDeltaVersion > metadata->bufferedDeltaVersion); // Update buffered delta version so new waitForVersion checks can bypass waiting entirely metadata->bufferedDeltaVersion = lastDeltaVersion; - - // This is the only place it is safe to set committedVersion, as it has to come from the - // mutation stream, or we could have a situation where the blob worker has consumed an - // uncommitted mutation, but not its rollback, fro m the change feed, and could thus - // think the uncommitted mutation is committed because it saw a higher committed version - // than the mutation's version. - // We also can only set it after consuming all of the mutations from the vector from the promise - // stream, as yielding when consuming from a change feed can cause bugs if this wakes up one of the - // file writers - if (knownNoRollbacksPast > committedVersion.get()) { - committedVersion.set(knownNoRollbacksPast); - } } justDidRollback = false; @@ -1564,17 +1583,18 @@ ACTOR Future blobGranuleUpdateFiles(Reference bwData, } else { previousFuture = Future(BlobFileIndex()); } - Future dfFuture = writeDeltaFile(bwData, - metadata->keyRange, - startState.granuleID, - metadata->originalEpoch, - metadata->originalSeqno, - metadata->deltaArena, - metadata->currentDeltas, - lastDeltaVersion, - previousFuture, - &committedVersion, - oldChangeFeedDataComplete); + Future dfFuture = + writeDeltaFile(bwData, + metadata->keyRange, + startState.granuleID, + metadata->originalEpoch, + metadata->originalSeqno, + metadata->deltaArena, + metadata->currentDeltas, + lastDeltaVersion, + previousFuture, + waitVersionCommitted(bwData, metadata, lastDeltaVersion), + oldChangeFeedDataComplete); inFlightFiles.push_back( InFlightFile(dfFuture, lastDeltaVersion, metadata->bufferedDeltaBytes, false)); @@ -1641,15 +1661,12 @@ ACTOR Future blobGranuleUpdateFiles(Reference bwData, state int waitIdx = 0; int idx = 0; for (auto& f : inFlightFiles) { - if (f.snapshot && f.version < metadata->pendingSnapshotVersion && - f.version <= committedVersion.get()) { + if (f.snapshot && f.version < metadata->pendingSnapshotVersion) { if (BW_DEBUG) { - fmt::print("[{0} - {1}) Waiting on previous snapshot file @ {2} <= known " - "committed {3}\n", + fmt::print("[{0} - {1}) Waiting on previous snapshot file @ {2}\n", metadata->keyRange.begin.printable(), metadata->keyRange.end.printable(), - f.version, - committedVersion.get()); + f.version); } waitIdx = idx + 1; } @@ -1912,6 +1929,10 @@ ACTOR Future waitForVersion(Reference metadata, Version v if (v == DEBUG_BW_WAIT_VERSION) { fmt::print("{0}) got CF version {1}\n", v, metadata->activeCFData.get()->getVersion()); } + // TODO REMOVE debugging + if (v > metadata->waitForVersionReturned) { + metadata->waitForVersionReturned = v; + } } // wait for any pending delta and snapshot files as of the moment the change feed version caught up. @@ -1969,11 +1990,6 @@ ACTOR Future waitForVersion(Reference metadata, Version v fmt::print("{0}) done\n", v); } - // TODO REMOVE debugging - if (v > metadata->waitForVersionReturned) { - metadata->waitForVersionReturned = v; - } - return Void(); } @@ -2776,6 +2792,40 @@ ACTOR Future monitorRemoval(Reference bwData) { } } +// Because change feeds send uncommitted data and explicit rollback messages, we speculatively buffer/write +// uncommitted data. This means we must ensure the data is actually committed before "committing" those writes in +// the blob granule. The simplest way to do this is to have the blob worker do a periodic GRV, which is guaranteed +// to be an earlier committed version. Then, once the change feed has consumed up through the GRV's data, we can +// guarantee nothing will roll back the in-memory mutations +ACTOR Future runGRVChecks(Reference bwData) { + state Transaction tr(bwData->db); + loop { + // only do grvs to get committed version if we need it to persist delta files + while (bwData->grvVersion.numWaiting() == 0) { + // printf("GRV checker sleeping\n"); + wait(bwData->doGRVCheck.getFuture()); + bwData->doGRVCheck.reset(); + // printf("GRV checker waking: %d pending\n", bwData->grvVersion.numWaiting()); + } + + // batch potentially multiple delta files into one GRV, and also rate limit GRVs for this worker + wait(delay(0.1)); // TODO KNOB? + // printf("GRV checker doing grv @ %.2f\n", now()); + + tr.reset(); + try { + Version readVersion = wait(tr.getReadVersion()); + ASSERT(readVersion >= bwData->grvVersion.get()); + // printf("GRV checker got GRV %lld\n", readVersion); + bwData->grvVersion.set(readVersion); + + ++bwData->stats.commitVersionChecks; + } catch (Error& e) { + wait(tr.onError(e)); + } + } +} + ACTOR Future blobWorker(BlobWorkerInterface bwInterf, ReplyPromise recruitReply, Reference const> dbInfo) { @@ -2828,6 +2878,7 @@ ACTOR Future blobWorker(BlobWorkerInterface bwInterf, recruitReply.send(rep); self->addActor.send(waitFailureServer(bwInterf.waitFailure.getFuture())); + self->addActor.send(runGRVChecks(self)); state Future selfRemoved = monitorRemoval(self); TraceEvent("BlobWorkerInit", self->id); From 307d049c9d713aecab5ffd9fc05295562d9a1715 Mon Sep 17 00:00:00 2001 From: Josh Slocum Date: Fri, 10 Dec 2021 16:12:06 -0600 Subject: [PATCH 078/413] Cleaning up some memory lifetime issues --- fdbclient/NativeAPI.actor.cpp | 6 +++--- fdbserver/BlobWorker.actor.cpp | 8 +++++--- 2 files changed, 8 insertions(+), 6 deletions(-) diff --git a/fdbclient/NativeAPI.actor.cpp b/fdbclient/NativeAPI.actor.cpp index 23107e027c..9d6e1136b0 100644 --- a/fdbclient/NativeAPI.actor.cpp +++ b/fdbclient/NativeAPI.actor.cpp @@ -7085,7 +7085,7 @@ Version ChangeFeedData::getVersion() { #define DEBUG_CF_WAIT_VERSION invalidVersion #define DEBUG_CF_VERSION(v) DEBUG_CF_START_VERSION <= v&& v <= DEBUG_CF_END_VERSION -ACTOR Future changeFeedWaitLatest(ChangeFeedData* self, Version version) { +ACTOR Future changeFeedWaitLatest(Reference self, Version version) { // first, wait on SS to have sent up through version int desired = 0; int waiting = 0; @@ -7162,7 +7162,7 @@ ACTOR Future changeFeedWaitLatest(ChangeFeedData* self, Version version) { return Void(); } -ACTOR Future changeFeedWhenAtLatest(ChangeFeedData* self, Version version) { +ACTOR Future changeFeedWhenAtLatest(Reference self, Version version) { if (DEBUG_CF_WAIT_VERSION == version) { fmt::print("CFW {0}) WhenAtLeast: LR={1}\n", version, self->lastReturnedVersion.get()); } @@ -7200,7 +7200,7 @@ ACTOR Future changeFeedWhenAtLatest(ChangeFeedData* self, Version version) } Future ChangeFeedData::whenAtLeast(Version version) { - return changeFeedWhenAtLatest(this, version); + return changeFeedWhenAtLatest(Reference::addRef(this), version); } ACTOR Future singleChangeFeedStream(StorageServerInterface interf, diff --git a/fdbserver/BlobWorker.actor.cpp b/fdbserver/BlobWorker.actor.cpp index aba8332dce..91de0695f6 100644 --- a/fdbserver/BlobWorker.actor.cpp +++ b/fdbserver/BlobWorker.actor.cpp @@ -1167,7 +1167,9 @@ ACTOR Future waitVersionCommitted(Reference bwData, // make sure the change feed has consumed mutations up through grvVersion to ensure none of them are rollbacks loop { - state Future atLeast = metadata->activeCFData.get()->whenAtLeast(grvVersion); + // if not valid, we're about to be cancelled anyway + state Future atLeast = + metadata->activeCFData.get().isValid() ? metadata->activeCFData.get()->whenAtLeast(grvVersion) : Never(); choose { when(wait(atLeast)) { break; } when(wait(metadata->activeCFData.onChange())) {} @@ -1885,6 +1887,8 @@ ACTOR Future waitForVersion(Reference metadata, Version v // if we don't have to wait for change feed version to catch up or wait for any pending file writes to complete, // nothing to do + ASSERT(metadata->activeCFData.get().isValid()); + if (v == DEBUG_BW_WAIT_VERSION) { fmt::print("{0}) [{1} - {2}) waiting for {3}\n readable:{4}\n bufferedDelta={5}\n pendingDelta={6}\n " "durableDelta={7}\n pendingSnapshot={8}\n durableSnapshot={9}\n", @@ -1900,8 +1904,6 @@ ACTOR Future waitForVersion(Reference metadata, Version v metadata->durableSnapshotVersion.get()); } - ASSERT(metadata->activeCFData.get().isValid()); - if (v <= metadata->activeCFData.get()->getVersion() && (v <= metadata->durableDeltaVersion.get() || metadata->durableDeltaVersion.get() == metadata->pendingDeltaVersion) && From 22d72ec9dd8b9e1a9383d3b17aed6cb066a32ed9 Mon Sep 17 00:00:00 2001 From: Suraj Gupta Date: Fri, 19 Nov 2021 20:54:22 -0500 Subject: [PATCH 079/413] init --- fdbclient/BlobGranuleCommon.h | 70 ++++++++++-- fdbclient/SystemData.cpp | 17 +++ fdbclient/SystemData.h | 10 +- fdbserver/BlobManager.actor.cpp | 189 ++++++++++++++++++++++++++++++++ fdbserver/BlobWorker.actor.cpp | 52 +-------- 5 files changed, 275 insertions(+), 63 deletions(-) diff --git a/fdbclient/BlobGranuleCommon.h b/fdbclient/BlobGranuleCommon.h index 7c44838c18..32e7e587ac 100644 --- a/fdbclient/BlobGranuleCommon.h +++ b/fdbclient/BlobGranuleCommon.h @@ -27,6 +27,65 @@ #include "fdbclient/CommitTransaction.h" #include "fdbclient/FDBTypes.h" +struct BlobGranuleHistoryValue { + constexpr static FileIdentifier file_identifier = 991434; + UID granuleID; + VectorRef> parentGranules; + + template + void serialize(Ar& ar) { + serializer(ar, granuleID, parentGranules); + } +}; + +struct GranuleHistory { + KeyRange range; + Version version; + Standalone value; + + GranuleHistory() {} + + GranuleHistory(KeyRange range, Version version, Standalone value) + : range(range), version(version), value(value) {} +}; + +// TODO add comments + documentation +struct BlobFileIndex { + Version version; + std::string filename; + int64_t offset; + int64_t length; + + BlobFileIndex() {} + + BlobFileIndex(Version version, std::string filename, int64_t offset, int64_t length) + : version(version), filename(filename), offset(offset), length(length) {} +}; + +struct GranuleFiles { + std::deque snapshotFiles; + std::deque deltaFiles; +}; + +// represents a previous version of a granule, and optionally the files that compose it +struct GranuleHistoryEntry : NonCopyable, ReferenceCounted { + KeyRange range; + UID granuleID; + Version startVersion; // version of the first snapshot + Version endVersion; // version of the last delta file + + // load files lazily, and allows for clearing old cold-queried files to save memory + Future files; + + // FIXME: do skip pointers with single back-pointer and neighbor pointers + // Just parent reference for now (assumes no merging) + Reference parentGranule; + + GranuleHistoryEntry() : startVersion(invalidVersion), endVersion(invalidVersion) {} + GranuleHistoryEntry(KeyRange range, UID granuleID, Version startVersion, Version endVersion) + : range(range), granuleID(granuleID), startVersion(startVersion), endVersion(endVersion) {} +}; + // file format of actual blob files struct GranuleSnapshot : VectorRef { @@ -89,15 +148,4 @@ struct BlobGranuleChunkRef { enum BlobGranuleSplitState { Unknown = 0, Initialized = 1, Assigned = 2, Done = 3 }; -struct BlobGranuleHistoryValue { - constexpr static FileIdentifier file_identifier = 991434; - UID granuleID; - VectorRef> parentGranules; - - template - void serialize(Ar& ar) { - serializer(ar, granuleID, parentGranules); - } -}; - #endif diff --git a/fdbclient/SystemData.cpp b/fdbclient/SystemData.cpp index 1a434f26de..70041f8e72 100644 --- a/fdbclient/SystemData.cpp +++ b/fdbclient/SystemData.cpp @@ -1130,6 +1130,7 @@ const KeyRangeRef blobGranuleMappingKeys(LiteralStringRef("\xff\x02/bgm/"), Lite const KeyRangeRef blobGranuleLockKeys(LiteralStringRef("\xff\x02/bgl/"), LiteralStringRef("\xff\x02/bgl0")); const KeyRangeRef blobGranuleSplitKeys(LiteralStringRef("\xff\x02/bgs/"), LiteralStringRef("\xff\x02/bgs0")); const KeyRangeRef blobGranuleHistoryKeys(LiteralStringRef("\xff\x02/bgh/"), LiteralStringRef("\xff\x02/bgh0")); +const KeyRangeRef blobGranulePruneKeys(LiteralStringRef("\xff\x02/bgp/"), LiteralStringRef("\xff\x02/bgp0")); const uint8_t BG_FILE_TYPE_DELTA = 'D'; const uint8_t BG_FILE_TYPE_SNAPSHOT = 'S'; @@ -1183,6 +1184,22 @@ std::tuple, int64_t, int64_t> decodeBlobGranuleFileValue(V return std::tuple(filename, offset, length); } +const Value blobGranulePruneValueFor(Version version, bool force) { + BinaryWriter wr(IncludeVersion(ProtocolVersion::withBlobGranule())); + wr << version; + wr << force; + return wr.toValue(); +} + +std::pair decodeBlobGranulePruneValue(ValueRef const& value) { + Version version; + bool force; + BinaryReader reader(value, IncludeVersion()); + reader >> version; + reader >> force; + return std::pair(version, force); +} + const Value blobGranuleMappingValueFor(UID const& workerID) { BinaryWriter wr(IncludeVersion(ProtocolVersion::withBlobGranule())); wr << workerID; diff --git a/fdbclient/SystemData.h b/fdbclient/SystemData.h index c3f3726a0d..cd0b3fba70 100644 --- a/fdbclient/SystemData.h +++ b/fdbclient/SystemData.h @@ -559,13 +559,19 @@ extern const KeyRangeRef blobGranuleSplitKeys; // \xff\x02/bgh/(beginKey,endKey,startVersion) = { granuleUID, [parentGranuleHistoryKeys] } extern const KeyRangeRef blobGranuleHistoryKeys; +// \xff\x02/bgp/(start,end) = (version, force) +extern const KeyRangeRef blobGranulePruneKeys; + const Key blobGranuleFileKeyFor(UID granuleID, uint8_t fileType, Version fileVersion); -std::tuple decodeBlobGranuleFileKey(ValueRef const& value); +std::tuple decodeBlobGranuleFileKey(KeyRef const& key); const KeyRange blobGranuleFileKeyRangeFor(UID granuleID); const Value blobGranuleFileValueFor(StringRef const& filename, int64_t offset, int64_t length); std::tuple, int64_t, int64_t> decodeBlobGranuleFileValue(ValueRef const& value); +const Value blobGranulePruneValueFor(Version version, bool force); +std::pair decodeBlobGranulePruneValue(ValueRef const& value); + const Value blobGranuleMappingValueFor(UID const& workerID); UID decodeBlobGranuleMappingValue(ValueRef const& value); @@ -583,7 +589,7 @@ const Value blobGranuleSplitValueFor(BlobGranuleSplitState st); std::pair decodeBlobGranuleSplitValue(ValueRef const& value); const Key blobGranuleHistoryKeyFor(KeyRangeRef const& range, Version version); -std::pair decodeBlobGranuleHistoryKey(KeyRef const& value); +std::pair decodeBlobGranuleHistoryKey(KeyRef const& key); const KeyRange blobGranuleHistoryKeyRangeFor(KeyRangeRef const& range); const Value blobGranuleHistoryValueFor(Standalone const& historyValue); diff --git a/fdbserver/BlobManager.actor.cpp b/fdbserver/BlobManager.actor.cpp index 5e906a07c4..818c2e77d7 100644 --- a/fdbserver/BlobManager.actor.cpp +++ b/fdbserver/BlobManager.actor.cpp @@ -18,6 +18,7 @@ * limitations under the License. */ +#include #include #include @@ -1371,6 +1372,194 @@ ACTOR Future haltBlobGranules(BlobManagerData* bmData) { return Void(); } +ACTOR Future fullyDeleteGranule(BlobManagerData* self, UID granuleId) { + state Transaction tr(self->db); + tr.setOption(FDBTransactionOptions::ACCESS_SYSTEM_KEYS); + tr.setOption(FDBTransactionOptions::PRIORITY_SYSTEM_IMMEDIATE); +} + +ACTOR Future partiallyDeleteGranule(BlobManagerData* self, UID granuleId, Version pruneVersion) { + state Reference tr = makeReference(self->db); + files = readGranuleFiles(G); +} + +// TODO: refactor this into a common file +ACTOR Future> getLatestGranuleHistory(Transaction* tr, KeyRange range) { + KeyRange historyRange = blobGranuleHistoryKeyRangeFor(range); + RangeResult result = wait(tr->getRange(historyRange, 1, Snapshot::False, Reverse::True)); + ASSERT(result.size() <= 1); + + Optional history; + if (!result.empty()) { + std::pair decodedKey = decodeBlobGranuleHistoryKey(result[0].key); + ASSERT(range == decodedKey.first); + history = GranuleHistory(range, decodedKey.second, decodeBlobGranuleHistoryValue(result[0].value)); + } + return history; +} + +// TODO: tell blob workers to clean up local memory +ACTOR Future pruneGranule(BlobManagerData* self, + KeyRef startKey, + KeyRef endKey, + Version pruneVersion, + bool force) { + // queue of for BFS traversal of history + state std::queue> historyEntryQueue; + + state std::unordered_set visited; // track which granules we have already visited in traversal + state std::vector toFullyDelete, toPartiallyDelete; // stacks of granuleIds to track granules to delete + KeyRange range(KeyRangeRef(startKey, endKey)); // range for [startKey, endKey) + + // find all active granules (that comprise the range) and add to the queue + auto activeRanges = self->workerAssignments.intersectingRanges(range); + state Transaction tr(self->db); + tr.setOption(FDBTransactionOptions::ACCESS_SYSTEM_KEYS); + tr.setOption(FDBTransactionOptions::PRIORITY_SYSTEM_IMMEDIATE); + + for (auto& activeRange : activeRanges) { + if (activeRange.begin() < startKey || activeRange.end() >= endKey) { + continue; + } + Optional history = wait(getLatestGranuleHistory(&tr, activeRange.range())); + ASSERT(history.present()); + historyEntryQueue.push({ activeRange.range(), history.get().version, MAX_VERSION }); + } + + while (!historyEntryQueue.empty()) { + // process the node at the front of the queue + KeyRange currRange; + Version startVersion, endVersion; + std::tie(currRange, startVersion, endVersion) = historyEntryQueue.front(); + historyEntryQueue.pop(); + + // get the persisted history entry for this granule + Optional persistedHistory = wait(tr.get(blobGranuleHistoryKeyFor(currRange, startVersion))); + ASSERT(persistedHistory.present()); + auto currHistoryNode = decodeBlobGranuleHistoryValue(persistedHistory.get()); + + // if we already saw this node, skip it; otherwise, mark it as visited + if (visited.count(currHistoryNode.granuleID)) { + continue; + } + visited.insert(currHistoryNode.granuleID); + + // There are three cases this granule can fall into: + // - if the granule's start version is past the prune version, this is an active granule + // - if the granule's end version is before the prune version, this granule should be completely + // deleted + // - otherwise, G.startVersion <= pruneVersion <= G.endVersion and so this granule should be partially + // deleted + if (endVersion < pruneVersion) { + toFullyDelete.emplace_back(currHistoryNode.granuleID); + } else if (startVersion <= pruneVersion) { + toPartiallyDelete.emplace_back(currHistoryNode.granuleID); + } + + // add all of the node's parents to the queue + for (auto& parent : currHistoryNode.parentGranules) { + // the parent's end version is this node's startVersion + historyEntryQueue.push({ parent.first, parent.second, startVersion }); + } + } + + // The top of the stacks have the oldest ranges. This implies that for a granule located at + // index i, it's parent must be located at some index j, where j > i. For this reason, + // we delete granules in reverse order; this way, we will never end up with unreachable + // nodes in the persisted history. Moreover, for any node that must be fully deleted, + // any node that must be partially deleted must occur later on in the history. Thus, + // we delete the 'toFullyDelete' granules first. + + for (int i = toFullyDelete.size() - 1; i >= 0; --i) { + UID granuleId = toFullyDelete[i]; + wait(fullyDeleteGranule(self, granuleId)); + } + + for (int i = toPartiallyDelete.size() - 1; i >= 0; --i) { + UID granuleId = toPartiallyDelete[i]; + wait(partiallyDeleteGranule(self, granuleId, pruneVersion)); + } + + // There could have been another pruneIntent that got written for this table while we + // were processing this one. If that is the case, we should not clear the key. Otherwise, + // we should clear the key to indicate the work is done. + state Reference rywTr = makeReference(self->db); + rywTr->setOption(FDBTransactionOptions::ACCESS_SYSTEM_KEYS); + rywTr->setOption(FDBTransactionOptions::PRIORITY_SYSTEM_IMMEDIATE); + loop { + try { + state RangeResult pruneIntent = + wait(krmGetRanges(rywTr, blobGranulePruneKeys.begin, range, 1, GetRangeLimits::BYTE_LIMIT_UNLIMITED)); + ASSERT(pruneIntent.size() == 1); + + if (decodeBlobGranulePruneValue(pruneIntent[0].value).first == pruneVersion) { + rywTr->clear(pruneIntent[0].key); + wait(rywTr->commit()); + } + break; + } catch (Error& e) { + wait(rywTr->onError(e)); + } + } +} + +/* + * Watch for changes to a key K that gets updated whenever there is an update to a blobGranulePruneKey + * On this change, scan through all blobGranulePruneKeys (which look like =) and prune + * Once the prune has succeeded, clear the key IF the version is still the same one that was pruned. If not, don't clear + * it. + * - this will add a conflict range on the key so even if there was a concurrent write, + * the txn will fail the first time and then the second time around, the version won't be the same, so it'll abort + * + * We can't watch on a key to do this. We will fall short if there's only one table for example and we fail on pruning + * that table a couple of times in a row. Instead, we'd need to (on some cadence) run the blob pruner to cycle through + * all blobGranulePruneKeys and prune per granule and clear the key once it's done. + * + * could also do a wait on a change in blob keys along with a timeout so that we definitely do the work in an hr + + * + * What happens if a prune comes in for key A-C and then another one for A-D. + * Check key boundaries and version before removing from persisted intents + */ +ACTOR Future monitorPruneKeys(BlobManagerData* self) { + loop { + state Reference tr = makeReference(self->db); + tr->setOption(FDBTransactionOptions::ACCESS_SYSTEM_KEYS); + tr->setOption(FDBTransactionOptions::PRIORITY_SYSTEM_IMMEDIATE); + + state KeyRef beginKey = normalKeys.begin; + loop { + try { + // TODO: refactor into a krmGetAllRanges + // TODO: replace 10000 with a knob + KeyRange nextRange(KeyRangeRef(beginKey, normalKeys.end)); + state RangeResult pruneIntents = wait(krmGetRanges( + tr, blobGranulePruneKeys.begin, nextRange, 10000, GetRangeLimits::BYTE_LIMIT_UNLIMITED)); + + for (int rangeIdx = 0; rangeIdx < pruneIntents.size() - 1; ++rangeIdx) { + KeyRef startKey = pruneIntents[rangeIdx].key; + KeyRef endKey = pruneIntents[rangeIdx + 1].key; + Version version; + bool force; + std::tie(version, force) = decodeBlobGranulePruneValue(pruneIntents[rangeIdx].value); + + pruneGranule(self, startKey, endKey, version, force); + + if (!pruneIntents.more) { + break; + } + + beginKey = pruneIntents.readThrough.get(); + } + } catch (Error& e) { + wait(tr->onError(e)); + } + } + + wait(delay(60 * 60 * 24)); // TODO: knobify + } +} + ACTOR Future blobManager(BlobManagerInterface bmInterf, Reference const> dbInfo, int64_t epoch) { diff --git a/fdbserver/BlobWorker.actor.cpp b/fdbserver/BlobWorker.actor.cpp index d24993db11..8e85b268a7 100644 --- a/fdbserver/BlobWorker.actor.cpp +++ b/fdbserver/BlobWorker.actor.cpp @@ -46,35 +46,6 @@ #define BW_DEBUG true #define BW_REQUEST_DEBUG false -// TODO add comments + documentation -struct BlobFileIndex { - Version version; - std::string filename; - int64_t offset; - int64_t length; - - BlobFileIndex() {} - - BlobFileIndex(Version version, std::string filename, int64_t offset, int64_t length) - : version(version), filename(filename), offset(offset), length(length) {} -}; - -struct GranuleFiles { - std::deque snapshotFiles; - std::deque deltaFiles; -}; - -struct GranuleHistory { - KeyRange range; - Version version; - Standalone value; - - GranuleHistory() {} - - GranuleHistory(KeyRange range, Version version, Standalone value) - : range(range), version(version), value(value) {} -}; - struct GranuleStartState { UID granuleID; Version changeFeedStartVersion; @@ -156,25 +127,6 @@ struct GranuleRangeMetadata { // ~GranuleRangeMetadata() { printf("Destroying granule metadata\n"); } }; -// represents a previous version of a granule, and optionally the files that compose it -struct GranuleHistoryEntry : NonCopyable, ReferenceCounted { - KeyRange range; - UID granuleID; - Version startVersion; // version of the first snapshot - Version endVersion; // version of the last delta file - - // load files lazily, and allows for clearing old cold-queried files to save memory - Future files; - - // FIXME: do skip pointers with single back-pointer and neighbor pointers - // Just parent reference for now (assumes no merging) - Reference parentGranule; - - GranuleHistoryEntry() : startVersion(invalidVersion), endVersion(invalidVersion) {} - GranuleHistoryEntry(KeyRange range, UID granuleID, Version startVersion, Version endVersion) - : range(range), granuleID(granuleID), startVersion(startVersion), endVersion(endVersion) {} -}; - struct BlobWorkerData : NonCopyable, ReferenceCounted { UID id; Database db; @@ -502,7 +454,7 @@ ACTOR Future writeDeltaFile(Reference bwData, wait(delay(0, TaskPriority::BlobWorkerUpdateStorage)); // TODO some sort of directory structure would be useful? - state std::string fname = deterministicRandom()->randomUniqueID().toString() + "_T" + + state std::string fname = granuleID.toString() + "_" deterministicRandom()->randomUniqueID().toString() + "_T" + std::to_string((uint64_t)(1000.0 * now())) + "_V" + std::to_string(currentDeltaVersion) + ".delta"; @@ -603,7 +555,7 @@ ACTOR Future writeSnapshot(Reference bwData, PromiseStream rows, bool createGranuleHistory) { // TODO some sort of directory structure would be useful maybe? - state std::string fname = deterministicRandom()->randomUniqueID().toString() + "_T" + + state std::string fname = granuleID.toString() + "_" deterministicRandom()->randomUniqueID().toString() + "_T" + std::to_string((uint64_t)(1000.0 * now())) + "_V" + std::to_string(version) + ".snapshot"; state Arena arena; state GranuleSnapshot snapshot; From 932f68e1b3279a369326b5a4710cf043a55ae402 Mon Sep 17 00:00:00 2001 From: Suraj Gupta Date: Mon, 22 Nov 2021 15:48:30 -0500 Subject: [PATCH 080/413] Refactor bstore and add watch key. --- fdbclient/ServerKnobs.cpp | 2 +- fdbclient/SystemData.cpp | 1 + fdbclient/SystemData.h | 1 + fdbserver/BlobManager.actor.cpp | 165 +++++++++++++++++++++++++++----- fdbserver/BlobWorker.actor.cpp | 19 ++-- 5 files changed, 149 insertions(+), 39 deletions(-) diff --git a/fdbclient/ServerKnobs.cpp b/fdbclient/ServerKnobs.cpp index 4011c3c80a..05b3aa6ead 100644 --- a/fdbclient/ServerKnobs.cpp +++ b/fdbclient/ServerKnobs.cpp @@ -782,7 +782,7 @@ void ServerKnobs::initialize(Randomize randomize, ClientKnobs* clientKnobs, IsSi init( LATENCY_METRICS_LOGGING_INTERVAL, 60.0 ); // Blob granlues - init( BG_URL, "" ); // TODO: store in system key space, eventually + init( BG_URL, isSimulated ? "file://fdbblob/" : "" ); // TODO: store in system key space, eventually init( BG_SNAPSHOT_FILE_TARGET_BYTES, 10000000 ); if( randomize && BUGGIFY ) { deterministicRandom()->random01() < 0.1 ? BG_SNAPSHOT_FILE_TARGET_BYTES /= 100 : BG_SNAPSHOT_FILE_TARGET_BYTES /= 10; } init( BG_DELTA_BYTES_BEFORE_COMPACT, BG_SNAPSHOT_FILE_TARGET_BYTES/2 ); init( BG_DELTA_FILE_TARGET_BYTES, BG_DELTA_BYTES_BEFORE_COMPACT/10 ); diff --git a/fdbclient/SystemData.cpp b/fdbclient/SystemData.cpp index 70041f8e72..32a0f5a24d 100644 --- a/fdbclient/SystemData.cpp +++ b/fdbclient/SystemData.cpp @@ -1131,6 +1131,7 @@ const KeyRangeRef blobGranuleLockKeys(LiteralStringRef("\xff\x02/bgl/"), Literal const KeyRangeRef blobGranuleSplitKeys(LiteralStringRef("\xff\x02/bgs/"), LiteralStringRef("\xff\x02/bgs0")); const KeyRangeRef blobGranuleHistoryKeys(LiteralStringRef("\xff\x02/bgh/"), LiteralStringRef("\xff\x02/bgh0")); const KeyRangeRef blobGranulePruneKeys(LiteralStringRef("\xff\x02/bgp/"), LiteralStringRef("\xff\x02/bgp0")); +const KeyRef blobGranulePruneChangeKey = LiteralStringRef("\xff\x02/bgpChange"); const uint8_t BG_FILE_TYPE_DELTA = 'D'; const uint8_t BG_FILE_TYPE_SNAPSHOT = 'S'; diff --git a/fdbclient/SystemData.h b/fdbclient/SystemData.h index cd0b3fba70..64cb9ea470 100644 --- a/fdbclient/SystemData.h +++ b/fdbclient/SystemData.h @@ -561,6 +561,7 @@ extern const KeyRangeRef blobGranuleHistoryKeys; // \xff\x02/bgp/(start,end) = (version, force) extern const KeyRangeRef blobGranulePruneKeys; +extern const KeyRef blobGranulePruneChangeKey; const Key blobGranuleFileKeyFor(UID granuleID, uint8_t fileType, Version fileVersion); std::tuple decodeBlobGranuleFileKey(KeyRef const& key); diff --git a/fdbserver/BlobManager.actor.cpp b/fdbserver/BlobManager.actor.cpp index 818c2e77d7..aff94d3514 100644 --- a/fdbserver/BlobManager.actor.cpp +++ b/fdbserver/BlobManager.actor.cpp @@ -23,6 +23,7 @@ #include #include "contrib/fmt-8.0.1/include/fmt/format.h" +#include "fdbclient/BackupContainerFileSystem.h" #include "fdbclient/BlobGranuleCommon.h" #include "fdbclient/BlobWorkerInterface.h" #include "fdbclient/KeyRangeMap.h" @@ -204,6 +205,8 @@ struct BlobManagerData { Optional dcId; PromiseStream> addActor; + Reference bstore; + std::unordered_map workersById; std::unordered_map workerStats; // mapping between workerID -> workerStats std::unordered_set workerAddresses; @@ -1372,15 +1375,90 @@ ACTOR Future haltBlobGranules(BlobManagerData* bmData) { return Void(); } -ACTOR Future fullyDeleteGranule(BlobManagerData* self, UID granuleId) { +// TODO: bring blob store to blob manager +ACTOR Future fullyDeleteGranule(BlobManagerData* self, + UID granuleId, + KeyRange granuleRange, + Version granuleStartVersion) { state Transaction tr(self->db); tr.setOption(FDBTransactionOptions::ACCESS_SYSTEM_KEYS); tr.setOption(FDBTransactionOptions::PRIORITY_SYSTEM_IMMEDIATE); + + // delete files + state GranuleFiles files; + wait(readGranuleFiles(&tr, granuleRange.begin, granuleRange.end, &files, granuleId)); + + std::vector> deletions; + + for (auto snapshotFile : files.snapshotFiles) { + std::string fname = snapshotFile.filename; + deletions.emplace_back(self->bstore->deleteFile(fname)); + } + + for (auto deltaFile : files.deltaFiles) { + std::string fname = deltaFile.filename; + deletions.emplace_back(self->bstore->deleteFile(fname)); + } + + wait(waitForAll(deletions)); + + // delete metadata in FDB (history entry and file keys) + loop { + try { + KeyRef historyKey = blobGranuleHistoryKeyFor(granuleRange, granuleStartVersion); + KeyRange fileRangeKey = blobGranuleFileKeyRangeFor(granuleId); + tr.clear(historyKey); + tr.clear(fileRangeKey); + wait(tr.commit()); + break; + } catch (Error& e) { + wait(tr.onError(e)); + } + } + + return Void(); } -ACTOR Future partiallyDeleteGranule(BlobManagerData* self, UID granuleId, Version pruneVersion) { - state Reference tr = makeReference(self->db); - files = readGranuleFiles(G); +ACTOR Future partiallyDeleteGranule(BlobManagerData* self, + UID granuleId, + KeyRange granuleRange, + Version granuleStartVersion, + Version pruneVersion) { + state Transaction tr(self->db); + tr.setOption(FDBTransactionOptions::ACCESS_SYSTEM_KEYS); + tr.setOption(FDBTransactionOptions::PRIORITY_SYSTEM_IMMEDIATE); + + state GranuleFiles files; + wait(readGranuleFiles(&tr, granuleRange.begin, granuleRange.end, &files, granuleId)); + + Version latestSnaphotVersion = invalidVersion; + + state std::vector> deletions; + + for (int idx = files.snapshotFiles.size() - 1; idx >= 0; --idx) { + // if we already found the latestSnapshotVersion, this snapshot can be deleted + if (latestSnaphotVersion != invalidVersion) { + std::string fname = files.snapshotFiles[idx].filename; + deletions.emplace_back(self->bstore->deleteFile(fname)); + } else if (files.snapshotFiles[idx].version < pruneVersion) { + // otherwise if this is the FIRST snapshot file with version < pruneVersion, + // then we found our latestSnapshotVersion (FIRST since we are traversing in reverse) + latestSnaphotVersion = files.snapshotFiles[idx].version; + } + } + + ASSERT(latestSnaphotVersion != invalidVersion); + + // delete all delta files older than latestSnapshotVersion + for (auto deltaFile : files.deltaFiles) { + if (deltaFile.version < latestSnaphotVersion) { + std::string fname = deltaFile.filename; + deletions.emplace_back(self->bstore->deleteFile(fname)); + } + } + + wait(waitForAll(deletions)); + return Void(); } // TODO: refactor this into a common file @@ -1399,16 +1477,17 @@ ACTOR Future> getLatestGranuleHistory(Transaction* tr, } // TODO: tell blob workers to clean up local memory -ACTOR Future pruneGranule(BlobManagerData* self, - KeyRef startKey, - KeyRef endKey, - Version pruneVersion, - bool force) { +ACTOR Future pruneRange(BlobManagerData* self, KeyRef startKey, KeyRef endKey, Version pruneVersion, bool force) { // queue of for BFS traversal of history + // TODO: consider using GranuleHistoryEntry but it's a little messy state std::queue> historyEntryQueue; + // stacks of to track which granules to delete + state std::vector> toFullyDelete, toPartiallyDelete; + + // set of granuleIds to track which granules we have already visited in traversal state std::unordered_set visited; // track which granules we have already visited in traversal - state std::vector toFullyDelete, toPartiallyDelete; // stacks of granuleIds to track granules to delete + KeyRange range(KeyRangeRef(startKey, endKey)); // range for [startKey, endKey) // find all active granules (that comprise the range) and add to the queue @@ -1445,15 +1524,15 @@ ACTOR Future pruneGranule(BlobManagerData* self, visited.insert(currHistoryNode.granuleID); // There are three cases this granule can fall into: - // - if the granule's start version is past the prune version, this is an active granule - // - if the granule's end version is before the prune version, this granule should be completely - // deleted - // - otherwise, G.startVersion <= pruneVersion <= G.endVersion and so this granule should be partially - // deleted - if (endVersion < pruneVersion) { - toFullyDelete.emplace_back(currHistoryNode.granuleID); + // - if the granule's end version is before the prune version or this is a force delete, + // this granule should be completely deleted + // - else if the startVersion <= pruneVersion, then G.startVersion <= pruneVersion <= G.endVersion + // and so this granule should be partially deleted + // - otherwise, this granule is active, so don't schedule it for deletion + if (force || endVersion < pruneVersion) { + toFullyDelete.push_back({ currHistoryNode.granuleID, currRange, startVersion }); } else if (startVersion <= pruneVersion) { - toPartiallyDelete.emplace_back(currHistoryNode.granuleID); + toPartiallyDelete.push_back({ currHistoryNode.granuleID, currRange, startVersion }); } // add all of the node's parents to the queue @@ -1471,18 +1550,27 @@ ACTOR Future pruneGranule(BlobManagerData* self, // we delete the 'toFullyDelete' granules first. for (int i = toFullyDelete.size() - 1; i >= 0; --i) { - UID granuleId = toFullyDelete[i]; - wait(fullyDeleteGranule(self, granuleId)); + UID granuleId; + KeyRange granuleRange; + Version startVersion; + std::tie(granuleId, granuleRange, startVersion) = toFullyDelete[i]; + wait(fullyDeleteGranule(self, granuleId, granuleRange, startVersion)); } for (int i = toPartiallyDelete.size() - 1; i >= 0; --i) { - UID granuleId = toPartiallyDelete[i]; - wait(partiallyDeleteGranule(self, granuleId, pruneVersion)); + UID granuleId; + KeyRange granuleRange; + Version startVersion; + std::tie(granuleId, granuleRange, startVersion) = toPartiallyDelete[i]; + wait(partiallyDeleteGranule(self, granuleId, granuleRange, startVersion, pruneVersion)); } // There could have been another pruneIntent that got written for this table while we // were processing this one. If that is the case, we should not clear the key. Otherwise, // we should clear the key to indicate the work is done. + // + // TODO: Probably should also check that the force value stayed consistent? + // though I don't think we can have two different pruneIntents at the same version but different force's... state Reference rywTr = makeReference(self->db); rywTr->setOption(FDBTransactionOptions::ACCESS_SYSTEM_KEYS); rywTr->setOption(FDBTransactionOptions::PRIORITY_SYSTEM_IMMEDIATE); @@ -1527,6 +1615,17 @@ ACTOR Future monitorPruneKeys(BlobManagerData* self) { tr->setOption(FDBTransactionOptions::ACCESS_SYSTEM_KEYS); tr->setOption(FDBTransactionOptions::PRIORITY_SYSTEM_IMMEDIATE); + loop { + try { + state Future watchPruneIntentsChange = tr->watch(fdbShouldConsistencyCheckBeSuspended); + wait(tr->commit()); + wait(timeout(watchPruneIntentsChange, 60 * 60 * 24, Void())); // TODO: knobify + break; + } catch (Error& e) { + wait(tr->onError(e)); + } + } + state KeyRef beginKey = normalKeys.begin; loop { try { @@ -1536,6 +1635,8 @@ ACTOR Future monitorPruneKeys(BlobManagerData* self) { state RangeResult pruneIntents = wait(krmGetRanges( tr, blobGranulePruneKeys.begin, nextRange, 10000, GetRangeLimits::BYTE_LIMIT_UNLIMITED)); + // TODO: wouldn't we miss a range [pruneIntents[9999], pruneIntents[10000]) because of the `more`? + // We also do this incorrectly in recoverBlobManager for (int rangeIdx = 0; rangeIdx < pruneIntents.size() - 1; ++rangeIdx) { KeyRef startKey = pruneIntents[rangeIdx].key; KeyRef endKey = pruneIntents[rangeIdx + 1].key; @@ -1543,7 +1644,8 @@ ACTOR Future monitorPruneKeys(BlobManagerData* self) { bool force; std::tie(version, force) = decodeBlobGranulePruneValue(pruneIntents[rangeIdx].value); - pruneGranule(self, startKey, endKey, version, force); + // TODO: should we add this to an actor collection? + pruneRange(self, startKey, endKey, version, force); if (!pruneIntents.more) { break; @@ -1555,8 +1657,6 @@ ACTOR Future monitorPruneKeys(BlobManagerData* self) { wait(tr->onError(e)); } } - - wait(delay(60 * 60 * 24)); // TODO: knobify } } @@ -1593,6 +1693,21 @@ ACTOR Future blobManager(BlobManagerInterface bmInterf, fmt::print("Blob manager acquired lock at epoch {}\n", epoch); } + try { + if (BM_DEBUG) { + printf("BM constructing backup container from %s\n", SERVER_KNOBS->BG_URL.c_str()); + } + self.bstore = BackupContainerFileSystem::openContainerFS(SERVER_KNOBS->BG_URL); + if (BM_DEBUG) { + printf("BM constructed backup container\n"); + } + } catch (Error& e) { + if (BM_DEBUG) { + printf("BM got backup container init error %s\n", e.name()); + } + throw e; + } + // although we start the recruiter, we wait until existing workers are ack'd auto recruitBlobWorker = IAsyncListener>::create( dbInfo, [](auto const& info) { return info.clusterInterface.recruitBlobWorker; }); diff --git a/fdbserver/BlobWorker.actor.cpp b/fdbserver/BlobWorker.actor.cpp index 8e85b268a7..6ca872361e 100644 --- a/fdbserver/BlobWorker.actor.cpp +++ b/fdbserver/BlobWorker.actor.cpp @@ -2743,19 +2743,12 @@ ACTOR Future blobWorker(BlobWorkerInterface bwInterf, } try { - if (g_network->isSimulated()) { - if (BW_DEBUG) { - printf("BW constructing simulated backup container\n"); - } - self->bstore = BackupContainerFileSystem::openContainerFS("file://fdbblob/"); - } else { - if (BW_DEBUG) { - printf("BW constructing backup container from %s\n", SERVER_KNOBS->BG_URL.c_str()); - } - self->bstore = BackupContainerFileSystem::openContainerFS(SERVER_KNOBS->BG_URL); - if (BW_DEBUG) { - printf("BW constructed backup container\n"); - } + if (BW_DEBUG) { + printf("BW constructing backup container from %s\n", SERVER_KNOBS->BG_URL.c_str()); + } + self->bstore = BackupContainerFileSystem::openContainerFS(SERVER_KNOBS->BG_URL); + if (BW_DEBUG) { + printf("BW constructed backup container\n"); } // register the blob worker to the system keyspace From 3fe7a9f553500718d21f4894197c9535ad73c495 Mon Sep 17 00:00:00 2001 From: Suraj Gupta Date: Tue, 23 Nov 2021 09:32:12 -0500 Subject: [PATCH 081/413] More fixes --- fdbclient/ServerKnobs.cpp | 1 + fdbclient/ServerKnobs.h | 2 + fdbserver/BlobManager.actor.cpp | 275 ++++++++++++++++++++------------ 3 files changed, 172 insertions(+), 106 deletions(-) diff --git a/fdbclient/ServerKnobs.cpp b/fdbclient/ServerKnobs.cpp index 05b3aa6ead..e9b14ac2b2 100644 --- a/fdbclient/ServerKnobs.cpp +++ b/fdbclient/ServerKnobs.cpp @@ -786,6 +786,7 @@ void ServerKnobs::initialize(Randomize randomize, ClientKnobs* clientKnobs, IsSi init( BG_SNAPSHOT_FILE_TARGET_BYTES, 10000000 ); if( randomize && BUGGIFY ) { deterministicRandom()->random01() < 0.1 ? BG_SNAPSHOT_FILE_TARGET_BYTES /= 100 : BG_SNAPSHOT_FILE_TARGET_BYTES /= 10; } init( BG_DELTA_BYTES_BEFORE_COMPACT, BG_SNAPSHOT_FILE_TARGET_BYTES/2 ); init( BG_DELTA_FILE_TARGET_BYTES, BG_DELTA_BYTES_BEFORE_COMPACT/10 ); + init( BG_PRUNE_TIMEOUT, 60 * 60); init( BLOB_WORKER_TIMEOUT, 10.0 ); if( randomize && BUGGIFY ) BLOB_WORKER_TIMEOUT = 1.0; init( BLOB_WORKERLIST_FETCH_INTERVAL, 1.0 ); diff --git a/fdbclient/ServerKnobs.h b/fdbclient/ServerKnobs.h index 33e3a023cd..6f1f4a9971 100644 --- a/fdbclient/ServerKnobs.h +++ b/fdbclient/ServerKnobs.h @@ -742,6 +742,8 @@ public: int BG_DELTA_FILE_TARGET_BYTES; int BG_DELTA_BYTES_BEFORE_COMPACT; + double BG_PRUNE_TIMEOUT; + double BLOB_WORKER_TIMEOUT; // Blob Manager's reaction time to a blob worker failure double BLOB_WORKERLIST_FETCH_INTERVAL; diff --git a/fdbserver/BlobManager.actor.cpp b/fdbserver/BlobManager.actor.cpp index aff94d3514..6dbbfd2634 100644 --- a/fdbserver/BlobManager.actor.cpp +++ b/fdbserver/BlobManager.actor.cpp @@ -1375,18 +1375,49 @@ ACTOR Future haltBlobGranules(BlobManagerData* bmData) { return Void(); } -// TODO: bring blob store to blob manager -ACTOR Future fullyDeleteGranule(BlobManagerData* self, - UID granuleId, - KeyRange granuleRange, - Version granuleStartVersion) { +// TODO: refactor this into a common file +ACTOR Future> getLatestGranuleHistory(Transaction* tr, KeyRange range) { + KeyRange historyRange = blobGranuleHistoryKeyRangeFor(range); + RangeResult result = wait(tr->getRange(historyRange, 1, Snapshot::False, Reverse::True)); + ASSERT(result.size() <= 1); + + Optional history; + if (!result.empty()) { + std::pair decodedKey = decodeBlobGranuleHistoryKey(result[0].key); + ASSERT(range == decodedKey.first); + history = GranuleHistory(range, decodedKey.second, decodeBlobGranuleHistoryValue(result[0].value)); + } + return history; +} + +ACTOR Future loadHistoryFiles(BlobManagerData* bmData, UID granuleID) { + state Transaction tr(bmData->db); + state KeyRange range = blobGranuleFileKeyRangeFor(granuleID); + state Key startKey = range.begin; + state GranuleFiles files; + loop { + try { + wait(readGranuleFiles(&tr, &startKey, range.end, &files, granuleID)); + return files; + } catch (Error& e) { + wait(tr.onError(e)); + } + } +} + +/* + * Deletes all files pertaining to the granule with id granuleId and + * also removes the history entry for this granule from the system keyspace + */ +ACTOR Future fullyDeleteGranule(BlobManagerData* self, UID granuleId, KeyRef historyKey) { state Transaction tr(self->db); tr.setOption(FDBTransactionOptions::ACCESS_SYSTEM_KEYS); tr.setOption(FDBTransactionOptions::PRIORITY_SYSTEM_IMMEDIATE); - // delete files - state GranuleFiles files; - wait(readGranuleFiles(&tr, granuleRange.begin, granuleRange.end, &files, granuleId)); + KeyRange filesRange = blobGranuleFileKeyRangeFor(granuleId); + + // get files + GranuleFiles files = wait(loadHistoryFiles(self, granuleId)); std::vector> deletions; @@ -1405,7 +1436,6 @@ ACTOR Future fullyDeleteGranule(BlobManagerData* self, // delete metadata in FDB (history entry and file keys) loop { try { - KeyRef historyKey = blobGranuleHistoryKeyFor(granuleRange, granuleStartVersion); KeyRange fileRangeKey = blobGranuleFileKeyRangeFor(granuleId); tr.clear(historyKey); tr.clear(fileRangeKey); @@ -1419,17 +1449,18 @@ ACTOR Future fullyDeleteGranule(BlobManagerData* self, return Void(); } -ACTOR Future partiallyDeleteGranule(BlobManagerData* self, - UID granuleId, - KeyRange granuleRange, - Version granuleStartVersion, - Version pruneVersion) { +/* + * For the granule with id granuleId, finds the first snapshot file at a + * version <= pruneVersion and deletes all files older than it. + */ +ACTOR Future partiallyDeleteGranule(BlobManagerData* self, UID granuleId, Version pruneVersion) { state Transaction tr(self->db); tr.setOption(FDBTransactionOptions::ACCESS_SYSTEM_KEYS); tr.setOption(FDBTransactionOptions::PRIORITY_SYSTEM_IMMEDIATE); - state GranuleFiles files; - wait(readGranuleFiles(&tr, granuleRange.begin, granuleRange.end, &files, granuleId)); + KeyRange filesRange = blobGranuleFileKeyRangeFor(granuleId); + + GranuleFiles files = wait(loadHistoryFiles(self, granuleId)); Version latestSnaphotVersion = invalidVersion; @@ -1440,7 +1471,7 @@ ACTOR Future partiallyDeleteGranule(BlobManagerData* self, if (latestSnaphotVersion != invalidVersion) { std::string fname = files.snapshotFiles[idx].filename; deletions.emplace_back(self->bstore->deleteFile(fname)); - } else if (files.snapshotFiles[idx].version < pruneVersion) { + } else if (files.snapshotFiles[idx].version <= pruneVersion) { // otherwise if this is the FIRST snapshot file with version < pruneVersion, // then we found our latestSnapshotVersion (FIRST since we are traversing in reverse) latestSnaphotVersion = files.snapshotFiles[idx].version; @@ -1461,29 +1492,24 @@ ACTOR Future partiallyDeleteGranule(BlobManagerData* self, return Void(); } -// TODO: refactor this into a common file -ACTOR Future> getLatestGranuleHistory(Transaction* tr, KeyRange range) { - KeyRange historyRange = blobGranuleHistoryKeyRangeFor(range); - RangeResult result = wait(tr->getRange(historyRange, 1, Snapshot::False, Reverse::True)); - ASSERT(result.size() <= 1); - - Optional history; - if (!result.empty()) { - std::pair decodedKey = decodeBlobGranuleHistoryKey(result[0].key); - ASSERT(range == decodedKey.first); - history = GranuleHistory(range, decodedKey.second, decodeBlobGranuleHistoryValue(result[0].value)); - } - return history; -} - -// TODO: tell blob workers to clean up local memory +/* + * This method is used to prune the range [startKey, endKey) at (and including) pruneVersion. + * To do this, we do a BFS traversal starting at the active granules. Then we classify granules + * in the history as nodes that can be fully deleted (i.e. their files and history can be deleted) + * and nodes that can be partially deleted (i.e. some of their files can be deleted). + * Once all this is done, we finally clear the pruneIntent key, if possible, to indicate we are done + * processing this prune intent. + * + * TODO: communicate the prune to blob workers so they can clean up local memory + */ ACTOR Future pruneRange(BlobManagerData* self, KeyRef startKey, KeyRef endKey, Version pruneVersion, bool force) { // queue of for BFS traversal of history - // TODO: consider using GranuleHistoryEntry but it's a little messy + // TODO: consider using GranuleHistoryEntry, but that also makes it a little messy state std::queue> historyEntryQueue; - // stacks of to track which granules to delete - state std::vector> toFullyDelete, toPartiallyDelete; + // stacks of and to track which granules to delete + state std::vector> toFullyDelete; + state std::vector toPartiallyDelete; // set of granuleIds to track which granules we have already visited in traversal state std::unordered_set visited; // track which granules we have already visited in traversal @@ -1497,25 +1523,42 @@ ACTOR Future pruneRange(BlobManagerData* self, KeyRef startKey, KeyRef end tr.setOption(FDBTransactionOptions::PRIORITY_SYSTEM_IMMEDIATE); for (auto& activeRange : activeRanges) { + // only want to prune exact granules if (activeRange.begin() < startKey || activeRange.end() >= endKey) { continue; } - Optional history = wait(getLatestGranuleHistory(&tr, activeRange.range())); - ASSERT(history.present()); - historyEntryQueue.push({ activeRange.range(), history.get().version, MAX_VERSION }); + loop { + try { + Optional history = wait(getLatestGranuleHistory(&tr, activeRange.range())); + ASSERT(history.present()); + historyEntryQueue.push({ activeRange.range(), history.get().version, MAX_VERSION }); + break; + } catch (Error& e) { + wait(tr.onError(e)); + } + } } while (!historyEntryQueue.empty()) { - // process the node at the front of the queue + // process the node at the front of the queue and remove it KeyRange currRange; Version startVersion, endVersion; std::tie(currRange, startVersion, endVersion) = historyEntryQueue.front(); historyEntryQueue.pop(); // get the persisted history entry for this granule - Optional persistedHistory = wait(tr.get(blobGranuleHistoryKeyFor(currRange, startVersion))); - ASSERT(persistedHistory.present()); - auto currHistoryNode = decodeBlobGranuleHistoryValue(persistedHistory.get()); + state Standalone currHistoryNode; + state KeyRef historyKey = blobGranuleHistoryKeyFor(currRange, startVersion); + loop { + try { + Optional persistedHistory = wait(tr.get(historyKey)); + ASSERT(persistedHistory.present()); + currHistoryNode = decodeBlobGranuleHistoryValue(persistedHistory.get()); + break; + } catch (Error& e) { + wait(tr.onError(e)); + } + } // if we already saw this node, skip it; otherwise, mark it as visited if (visited.count(currHistoryNode.granuleID)) { @@ -1524,15 +1567,15 @@ ACTOR Future pruneRange(BlobManagerData* self, KeyRef startKey, KeyRef end visited.insert(currHistoryNode.granuleID); // There are three cases this granule can fall into: - // - if the granule's end version is before the prune version or this is a force delete, + // - if the granule's end version is at or before the prune version or this is a force delete, // this granule should be completely deleted - // - else if the startVersion <= pruneVersion, then G.startVersion <= pruneVersion <= G.endVersion + // - else if the startVersion <= pruneVersion, then G.startVersion <= pruneVersion < G.endVersion // and so this granule should be partially deleted // - otherwise, this granule is active, so don't schedule it for deletion - if (force || endVersion < pruneVersion) { - toFullyDelete.push_back({ currHistoryNode.granuleID, currRange, startVersion }); + if (force || endVersion <= pruneVersion) { + toFullyDelete.push_back({ currHistoryNode.granuleID, historyKey }); } else if (startVersion <= pruneVersion) { - toPartiallyDelete.push_back({ currHistoryNode.granuleID, currRange, startVersion }); + toPartiallyDelete.push_back({ currHistoryNode.granuleID }); } // add all of the node's parents to the queue @@ -1548,29 +1591,26 @@ ACTOR Future pruneRange(BlobManagerData* self, KeyRef startKey, KeyRef end // nodes in the persisted history. Moreover, for any node that must be fully deleted, // any node that must be partially deleted must occur later on in the history. Thus, // we delete the 'toFullyDelete' granules first. + // + // Unfortunately we can't do multiple deletions in parallel because they might + // race and we'll end up with unreachable nodes in the case of a crash for (int i = toFullyDelete.size() - 1; i >= 0; --i) { UID granuleId; - KeyRange granuleRange; - Version startVersion; - std::tie(granuleId, granuleRange, startVersion) = toFullyDelete[i]; - wait(fullyDeleteGranule(self, granuleId, granuleRange, startVersion)); + KeyRef historyKey; + std::tie(granuleId, historyKey) = toFullyDelete[i]; + wait(fullyDeleteGranule(self, granuleId, historyKey)); } + // TODO: could possibly do the partial deletes in parallel? for (int i = toPartiallyDelete.size() - 1; i >= 0; --i) { - UID granuleId; - KeyRange granuleRange; - Version startVersion; - std::tie(granuleId, granuleRange, startVersion) = toPartiallyDelete[i]; - wait(partiallyDeleteGranule(self, granuleId, granuleRange, startVersion, pruneVersion)); + UID granuleId = toPartiallyDelete[i]; + wait(partiallyDeleteGranule(self, granuleId, pruneVersion)); } // There could have been another pruneIntent that got written for this table while we // were processing this one. If that is the case, we should not clear the key. Otherwise, // we should clear the key to indicate the work is done. - // - // TODO: Probably should also check that the force value stayed consistent? - // though I don't think we can have two different pruneIntents at the same version but different force's... state Reference rywTr = makeReference(self->db); rywTr->setOption(FDBTransactionOptions::ACCESS_SYSTEM_KEYS); rywTr->setOption(FDBTransactionOptions::PRIORITY_SYSTEM_IMMEDIATE); @@ -1592,71 +1632,94 @@ ACTOR Future pruneRange(BlobManagerData* self, KeyRef startKey, KeyRef end } /* - * Watch for changes to a key K that gets updated whenever there is an update to a blobGranulePruneKey - * On this change, scan through all blobGranulePruneKeys (which look like =) and prune - * Once the prune has succeeded, clear the key IF the version is still the same one that was pruned. If not, don't clear - * it. - * - this will add a conflict range on the key so even if there was a concurrent write, - * the txn will fail the first time and then the second time around, the version won't be the same, so it'll abort + * This monitor watches for changes to a key K that gets updated whenever there is a new prune intent. + * On this change, we scan through all blobGranulePruneKeys (which look like =) and prune any intents. * - * We can't watch on a key to do this. We will fall short if there's only one table for example and we fail on pruning - * that table a couple of times in a row. Instead, we'd need to (on some cadence) run the blob pruner to cycle through - * all blobGranulePruneKeys and prune per granule and clear the key once it's done. + * Once the prune has succeeded, we clear the key IF the version is still the same one that was pruned. + * That way, if another prune intent arrived for the same range while we were working on an older one, + * we wouldn't end up clearing the intent. * - * could also do a wait on a change in blob keys along with a timeout so that we definitely do the work in an hr - + * When watching for changes, we might end up in scenarios where we failed to do the work + * for a prune intent even though the watch was triggered (maybe the BM had a blip). This is problematic + * if the intent is a force and there isn't another prune intent for quite some time. To remedy this, + * if we don't see a watch change in X (configurable) seconds, we will just sweep through the prune intents, + * consolidating any work we might have missed before. * - * What happens if a prune comes in for key A-C and then another one for A-D. - * Check key boundaries and version before removing from persisted intents + * Note: we could potentially use a changefeed here to get the exact pruneIntent that was added + * rather than iterating through all of them, but this might have too much overhead for latency + * improvements we don't really need here (also we need to go over all prune intents anyways in the + * case that the timer is up before any new prune intents arrive). */ ACTOR Future monitorPruneKeys(BlobManagerData* self) { - loop { - state Reference tr = makeReference(self->db); - tr->setOption(FDBTransactionOptions::ACCESS_SYSTEM_KEYS); - tr->setOption(FDBTransactionOptions::PRIORITY_SYSTEM_IMMEDIATE); - + try { loop { - try { - state Future watchPruneIntentsChange = tr->watch(fdbShouldConsistencyCheckBeSuspended); - wait(tr->commit()); - wait(timeout(watchPruneIntentsChange, 60 * 60 * 24, Void())); // TODO: knobify - break; - } catch (Error& e) { - wait(tr->onError(e)); + state Reference tr = makeReference(self->db); + tr->setOption(FDBTransactionOptions::ACCESS_SYSTEM_KEYS); + tr->setOption(FDBTransactionOptions::PRIORITY_SYSTEM_IMMEDIATE); + + // Wait for the watch to change, or some time to expire (whichever comes first) + // before checking through the prune intents + loop { + try { + state Future watchPruneIntentsChange = tr->watch(blobGranulePruneChangeKey); + wait(tr->commit()); + wait(timeout(watchPruneIntentsChange, SERVER_KNOBS->BG_PRUNE_TIMEOUT, Void())); + break; + } catch (Error& e) { + wait(tr->onError(e)); + } } - } - state KeyRef beginKey = normalKeys.begin; - loop { - try { - // TODO: refactor into a krmGetAllRanges - // TODO: replace 10000 with a knob - KeyRange nextRange(KeyRangeRef(beginKey, normalKeys.end)); - state RangeResult pruneIntents = wait(krmGetRanges( - tr, blobGranulePruneKeys.begin, nextRange, 10000, GetRangeLimits::BYTE_LIMIT_UNLIMITED)); + // loop through all prune intentions and do prune work accordingly + state KeyRef beginKey = normalKeys.begin; + loop { + try { + // TODO: replace 10000 with a knob + KeyRange nextRange(KeyRangeRef(beginKey, normalKeys.end)); + state RangeResult pruneIntents = wait(krmGetRanges( + tr, blobGranulePruneKeys.begin, nextRange, 10000, GetRangeLimits::BYTE_LIMIT_UNLIMITED)); - // TODO: wouldn't we miss a range [pruneIntents[9999], pruneIntents[10000]) because of the `more`? - // We also do this incorrectly in recoverBlobManager - for (int rangeIdx = 0; rangeIdx < pruneIntents.size() - 1; ++rangeIdx) { - KeyRef startKey = pruneIntents[rangeIdx].key; - KeyRef endKey = pruneIntents[rangeIdx + 1].key; - Version version; - bool force; - std::tie(version, force) = decodeBlobGranulePruneValue(pruneIntents[rangeIdx].value); + // TODO: would we miss a range [pruneIntents[9999], pruneIntents[10000]) because of the `more`? + // Or does `readThrough` take care of this? We also do this in recoverBlobManager + for (int rangeIdx = 0; rangeIdx < pruneIntents.size() - 1; ++rangeIdx) { + if (pruneIntents[rangeIdx].value.size() == 0) { + continue; + } + KeyRef rangeStartKey = pruneIntents[rangeIdx].key; + KeyRef rangeEndKey = pruneIntents[rangeIdx + 1].key; + Version pruneVersion; + bool force; + std::tie(pruneVersion, force) = decodeBlobGranulePruneValue(pruneIntents[rangeIdx].value); - // TODO: should we add this to an actor collection? - pruneRange(self, startKey, endKey, version, force); + // TODO: should we add this to an actor collection or a list of futures? + // Probably because still need to handle the case of one prune at version V and then timer + // expires and we start another prune again at version V. we need to keep track of what's in + // progress. That brings another problem though: what happens if something is in progress and + // fails... One way to prevent this is to not iterate over the prunes until the last iteration + // is done (i.e waitForAll) + // + // ErrorOr would prob be what we need here + + pruneRange(self, rangeStartKey, rangeEndKey, pruneVersion, force); + } if (!pruneIntents.more) { break; } beginKey = pruneIntents.readThrough.get(); + } catch (Error& e) { + // TODO: other errors here from pruneRange? + wait(tr->onError(e)); } - } catch (Error& e) { - wait(tr->onError(e)); } } + } catch (Error& e) { + if (BM_DEBUG) { + printf("monitorPruneKeys got error %s\n", e.name()); + } + throw e; } } From 23de6fa39b0836bf881d5228dd1be1bdaa673ae1 Mon Sep 17 00:00:00 2001 From: Suraj Gupta Date: Wed, 24 Nov 2021 10:12:54 -0500 Subject: [PATCH 082/413] Get it compiled --- fdbclient/BlobGranuleCommon.h | 52 +-------- fdbclient/CMakeLists.txt | 1 + fdbserver/BlobGranuleServerCommon.actor.cpp | 120 ++++++++++++++++++++ fdbserver/BlobGranuleServerCommon.actor.h | 75 ++++++++++++ fdbserver/BlobManager.actor.cpp | 73 ++++++++++-- fdbserver/BlobWorker.actor.cpp | 46 +++++--- fdbserver/CMakeLists.txt | 2 + 7 files changed, 290 insertions(+), 79 deletions(-) create mode 100644 fdbserver/BlobGranuleServerCommon.actor.cpp create mode 100644 fdbserver/BlobGranuleServerCommon.actor.h diff --git a/fdbclient/BlobGranuleCommon.h b/fdbclient/BlobGranuleCommon.h index 32e7e587ac..b0cbd8ef8b 100644 --- a/fdbclient/BlobGranuleCommon.h +++ b/fdbclient/BlobGranuleCommon.h @@ -24,9 +24,12 @@ #include +#include "flow/flow.h" #include "fdbclient/CommitTransaction.h" #include "fdbclient/FDBTypes.h" +#include "flow/actorcompiler.h" // has to be last include +// Stores the value representation of a blob granule history entry in FDB struct BlobGranuleHistoryValue { constexpr static FileIdentifier file_identifier = 991434; UID granuleID; @@ -38,54 +41,6 @@ struct BlobGranuleHistoryValue { } }; -struct GranuleHistory { - KeyRange range; - Version version; - Standalone value; - - GranuleHistory() {} - - GranuleHistory(KeyRange range, Version version, Standalone value) - : range(range), version(version), value(value) {} -}; - -// TODO add comments + documentation -struct BlobFileIndex { - Version version; - std::string filename; - int64_t offset; - int64_t length; - - BlobFileIndex() {} - - BlobFileIndex(Version version, std::string filename, int64_t offset, int64_t length) - : version(version), filename(filename), offset(offset), length(length) {} -}; - -struct GranuleFiles { - std::deque snapshotFiles; - std::deque deltaFiles; -}; - -// represents a previous version of a granule, and optionally the files that compose it -struct GranuleHistoryEntry : NonCopyable, ReferenceCounted { - KeyRange range; - UID granuleID; - Version startVersion; // version of the first snapshot - Version endVersion; // version of the last delta file - - // load files lazily, and allows for clearing old cold-queried files to save memory - Future files; - - // FIXME: do skip pointers with single back-pointer and neighbor pointers - // Just parent reference for now (assumes no merging) - Reference parentGranule; - - GranuleHistoryEntry() : startVersion(invalidVersion), endVersion(invalidVersion) {} - GranuleHistoryEntry(KeyRange range, UID granuleID, Version startVersion, Version endVersion) - : range(range), granuleID(granuleID), startVersion(startVersion), endVersion(endVersion) {} -}; - // file format of actual blob files struct GranuleSnapshot : VectorRef { @@ -147,5 +102,4 @@ struct BlobGranuleChunkRef { }; enum BlobGranuleSplitState { Unknown = 0, Initialized = 1, Assigned = 2, Done = 3 }; - #endif diff --git a/fdbclient/CMakeLists.txt b/fdbclient/CMakeLists.txt index f46d491a28..d105c4500c 100644 --- a/fdbclient/CMakeLists.txt +++ b/fdbclient/CMakeLists.txt @@ -23,6 +23,7 @@ set(FDBCLIENT_SRCS BlobWorkerInterface.h BlobGranuleReader.actor.cpp BlobGranuleReader.actor.h + BlobGranuleCommon.cpp BlobGranuleCommon.h BlobGranuleFiles.cpp BlobGranuleFiles.h diff --git a/fdbserver/BlobGranuleServerCommon.actor.cpp b/fdbserver/BlobGranuleServerCommon.actor.cpp new file mode 100644 index 0000000000..d4ed410c92 --- /dev/null +++ b/fdbserver/BlobGranuleServerCommon.actor.cpp @@ -0,0 +1,120 @@ +/* + * BlobGranuleCommon.actor.cpp + * + * This source file is part of the FoundationDB open source project + * + * Copyright 2013-2021 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/SystemData.h" +#include "fdbclient/BlobGranuleCommon.h" +#include "fdbserver/BlobGranuleServerCommon.actor.h" +#include "fdbclient/CommitTransaction.h" +#include "fdbclient/FDBTypes.h" +#include "fdbclient/ReadYourWrites.h" +#include "flow/Arena.h" +#include "flow/actorcompiler.h" // has to be last include + +// Gets the latest granule history node for range that was persisted +ACTOR Future> getLatestGranuleHistory(Transaction* tr, KeyRange range) { + KeyRange historyRange = blobGranuleHistoryKeyRangeFor(range); + state RangeResult result; + + loop { + try { + RangeResult _result = wait(tr->getRange(historyRange, 1, Snapshot::False, Reverse::True)); + result = _result; + break; + } catch (Error& e) { + wait(tr->onError(e)); + } + } + ASSERT(result.size() <= 1); + + Optional history; + if (!result.empty()) { + std::pair decodedKey = decodeBlobGranuleHistoryKey(result[0].key); + ASSERT(range == decodedKey.first); + history = GranuleHistory(range, decodedKey.second, decodeBlobGranuleHistoryValue(result[0].value)); + } + return history; +} + +// Gets the files based on the file key range [startKey, endKey) +// and populates the files object accordingly +ACTOR Future readGranuleFiles(Transaction* tr, + Key* startKey, + Key endKey, + GranuleFiles* files, + UID granuleID, + bool debug) { + + loop { + int lim = BUGGIFY ? 2 : 1000; + RangeResult res = wait(tr->getRange(KeyRangeRef(*startKey, endKey), lim)); + for (auto& it : res) { + UID gid; + uint8_t fileType; + Version version; + + Standalone filename; + int64_t offset; + int64_t length; + + std::tie(gid, fileType, version) = decodeBlobGranuleFileKey(it.key); + ASSERT(gid == granuleID); + + std::tie(filename, offset, length) = decodeBlobGranuleFileValue(it.value); + + BlobFileIndex idx(version, filename.toString(), offset, length); + if (fileType == 'S') { + ASSERT(files->snapshotFiles.empty() || files->snapshotFiles.back().version < idx.version); + files->snapshotFiles.push_back(idx); + } else { + ASSERT(fileType == 'D'); + ASSERT(files->deltaFiles.empty() || files->deltaFiles.back().version < idx.version); + files->deltaFiles.push_back(idx); + } + } + if (res.more) { + *startKey = keyAfter(res.back().key); + } else { + break; + } + } + if (debug) { + printf("Loaded %d snapshot and %d delta files for %s\n", + files->snapshotFiles.size(), + files->deltaFiles.size(), + granuleID.toString().c_str()); + } + return Void(); +} + +// Wrapper around readGranuleFiles +// Gets all files belonging to the granule with id granule ID +ACTOR Future loadHistoryFiles(Transaction* tr, UID granuleID, bool debug) { + state KeyRange range = blobGranuleFileKeyRangeFor(granuleID); + state Key startKey = range.begin; + state GranuleFiles files; + loop { + try { + wait(readGranuleFiles(tr, &startKey, range.end, &files, granuleID, debug)); + return files; + } catch (Error& e) { + wait(tr->onError(e)); + } + } +} diff --git a/fdbserver/BlobGranuleServerCommon.actor.h b/fdbserver/BlobGranuleServerCommon.actor.h new file mode 100644 index 0000000000..788377b8fc --- /dev/null +++ b/fdbserver/BlobGranuleServerCommon.actor.h @@ -0,0 +1,75 @@ +/* + * BlobGranuleCommon.h + * + * This source file is part of the FoundationDB open source project + * + * Copyright 2013-2018 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. + */ + +#if defined(NO_INTELLISENSE) && !defined(FDBSERVER_BLOBGRANULESERVERCOMMON_ACTOR_G_H) +#define FDBSERVER_BLOBGRANULESERVERCOMMON_ACTOR_G_H +#include "fdbserver/BlobGranuleServerCommon.actor.g.h" +#elif !defined(FDBSERVER_BLOBGRANULESERVERCOMMON_ACTOR_H) +#define FDBSERVER_BLOBGRANULESERVERCOMMON_ACTOR_H + +#pragma once + +#include "flow/flow.h" +#include "fdbclient/CommitTransaction.h" +#include "fdbclient/FDBTypes.h" +#include "fdbclient/BlobGranuleCommon.h" +#include "flow/actorcompiler.h" // has to be last include + +struct GranuleHistory { + KeyRange range; + Version version; + Standalone value; + + GranuleHistory() {} + + GranuleHistory(KeyRange range, Version version, Standalone value) + : range(range), version(version), value(value) {} +}; + +// Stores info about a file in blob storage +struct BlobFileIndex { + Version version; + std::string filename; + int64_t offset; + int64_t length; + + BlobFileIndex() {} + + BlobFileIndex(Version version, std::string filename, int64_t offset, int64_t length) + : version(version), filename(filename), offset(offset), length(length) {} +}; + +// Stores the files that comprise a blob granule +struct GranuleFiles { + std::deque snapshotFiles; + std::deque deltaFiles; +}; + +class Transaction; +ACTOR Future> getLatestGranuleHistory(Transaction* tr, KeyRange range); +ACTOR Future readGranuleFiles(Transaction* tr, + Key* startKey, + Key endKey, + GranuleFiles* files, + UID granuleID, + bool debug); + +ACTOR Future loadHistoryFiles(Transaction* tr, UID granuleID, bool debug); +#endif diff --git a/fdbserver/BlobManager.actor.cpp b/fdbserver/BlobManager.actor.cpp index 6dbbfd2634..d99e637731 100644 --- a/fdbserver/BlobManager.actor.cpp +++ b/fdbserver/BlobManager.actor.cpp @@ -31,6 +31,7 @@ #include "fdbclient/SystemData.h" #include "fdbserver/BlobManagerInterface.h" #include "fdbserver/Knobs.h" +#include "fdbserver/BlobGranuleServerCommon.actor.h" #include "fdbserver/QuietDatabase.h" #include "fdbserver/WaitFailure.h" #include "fdbserver/WorkerInterface.actor.h" @@ -1417,7 +1418,7 @@ ACTOR Future fullyDeleteGranule(BlobManagerData* self, UID granuleId, KeyR KeyRange filesRange = blobGranuleFileKeyRangeFor(granuleId); // get files - GranuleFiles files = wait(loadHistoryFiles(self, granuleId)); + GranuleFiles files = wait(loadHistoryFiles(&tr, granuleId, BM_DEBUG)); std::vector> deletions; @@ -1460,17 +1461,20 @@ ACTOR Future partiallyDeleteGranule(BlobManagerData* self, UID granuleId, KeyRange filesRange = blobGranuleFileKeyRangeFor(granuleId); - GranuleFiles files = wait(loadHistoryFiles(self, granuleId)); + GranuleFiles files = wait(loadHistoryFiles(&tr, granuleId, BM_DEBUG)); Version latestSnaphotVersion = invalidVersion; state std::vector> deletions; + state std::vector deletedFileKeys; + // TODO: binary search these snapshot files for latestSnapshotVersion for (int idx = files.snapshotFiles.size() - 1; idx >= 0; --idx) { // if we already found the latestSnapshotVersion, this snapshot can be deleted if (latestSnaphotVersion != invalidVersion) { std::string fname = files.snapshotFiles[idx].filename; deletions.emplace_back(self->bstore->deleteFile(fname)); + deletedFileKeys.emplace_back(blobGranuleFileKeyFor(granuleId, 'S', files.snapshotFiles[idx].version)); } else if (files.snapshotFiles[idx].version <= pruneVersion) { // otherwise if this is the FIRST snapshot file with version < pruneVersion, // then we found our latestSnapshotVersion (FIRST since we are traversing in reverse) @@ -1485,10 +1489,24 @@ ACTOR Future partiallyDeleteGranule(BlobManagerData* self, UID granuleId, if (deltaFile.version < latestSnaphotVersion) { std::string fname = deltaFile.filename; deletions.emplace_back(self->bstore->deleteFile(fname)); + deletedFileKeys.emplace_back(blobGranuleFileKeyFor(granuleId, 'D', deltaFile.version)); } } wait(waitForAll(deletions)); + + // delete metadata in FDB (deleted file keys) + loop { + try { + for (auto key : deletedFileKeys) { + tr.clear(key); + } + wait(tr.commit()); + break; + } catch (Error& e) { + wait(tr.onError(e)); + } + } return Void(); } @@ -1501,6 +1519,7 @@ ACTOR Future partiallyDeleteGranule(BlobManagerData* self, UID granuleId, * processing this prune intent. * * TODO: communicate the prune to blob workers so they can clean up local memory + * maybe BWs can just watch the prune keys as well! */ ACTOR Future pruneRange(BlobManagerData* self, KeyRef startKey, KeyRef endKey, Version pruneVersion, bool force) { // queue of for BFS traversal of history @@ -1514,19 +1533,20 @@ ACTOR Future pruneRange(BlobManagerData* self, KeyRef startKey, KeyRef end // set of granuleIds to track which granules we have already visited in traversal state std::unordered_set visited; // track which granules we have already visited in traversal - KeyRange range(KeyRangeRef(startKey, endKey)); // range for [startKey, endKey) + state KeyRange range(KeyRangeRef(startKey, endKey)); // range for [startKey, endKey) // find all active granules (that comprise the range) and add to the queue - auto activeRanges = self->workerAssignments.intersectingRanges(range); + state KeyRangeMap::Ranges activeRanges = self->workerAssignments.intersectingRanges(range); + state Transaction tr(self->db); tr.setOption(FDBTransactionOptions::ACCESS_SYSTEM_KEYS); tr.setOption(FDBTransactionOptions::PRIORITY_SYSTEM_IMMEDIATE); - for (auto& activeRange : activeRanges) { + state KeyRangeMap::iterator activeRange; + for (activeRange = activeRanges.begin(); activeRange != activeRanges.end(); ++activeRange) { // only want to prune exact granules - if (activeRange.begin() < startKey || activeRange.end() >= endKey) { - continue; - } + ASSERT(activeRange.begin() >= startKey && activeRange.end() < endKey); + loop { try { Optional history = wait(getLatestGranuleHistory(&tr, activeRange.range())); @@ -1542,7 +1562,8 @@ ACTOR Future pruneRange(BlobManagerData* self, KeyRef startKey, KeyRef end while (!historyEntryQueue.empty()) { // process the node at the front of the queue and remove it KeyRange currRange; - Version startVersion, endVersion; + state Version startVersion; + state Version endVersion; std::tie(currRange, startVersion, endVersion) = historyEntryQueue.front(); historyEntryQueue.pop(); @@ -1595,15 +1616,17 @@ ACTOR Future pruneRange(BlobManagerData* self, KeyRef startKey, KeyRef end // Unfortunately we can't do multiple deletions in parallel because they might // race and we'll end up with unreachable nodes in the case of a crash - for (int i = toFullyDelete.size() - 1; i >= 0; --i) { + state int i; + for (i = toFullyDelete.size() - 1; i >= 0; --i) { UID granuleId; KeyRef historyKey; std::tie(granuleId, historyKey) = toFullyDelete[i]; + // TODO: can possibly batch into a single txn wait(fullyDeleteGranule(self, granuleId, historyKey)); } // TODO: could possibly do the partial deletes in parallel? - for (int i = toPartiallyDelete.size() - 1; i >= 0; --i) { + for (i = toPartiallyDelete.size() - 1; i >= 0; --i) { UID granuleId = toPartiallyDelete[i]; wait(partiallyDeleteGranule(self, granuleId, pruneVersion)); } @@ -1611,6 +1634,8 @@ ACTOR Future pruneRange(BlobManagerData* self, KeyRef startKey, KeyRef end // There could have been another pruneIntent that got written for this table while we // were processing this one. If that is the case, we should not clear the key. Otherwise, // we should clear the key to indicate the work is done. + + // TODO: reuse tr state Reference rywTr = makeReference(self->db); rywTr->setOption(FDBTransactionOptions::ACCESS_SYSTEM_KEYS); rywTr->setOption(FDBTransactionOptions::PRIORITY_SYSTEM_IMMEDIATE); @@ -1629,8 +1654,15 @@ ACTOR Future pruneRange(BlobManagerData* self, KeyRef startKey, KeyRef end wait(rywTr->onError(e)); } } + + return Void(); } +/* +TODO: We need to revoke range from BW so that it doesn't try to add to a granule that we dropped +Will SnowTram reuse table IDs; do we unhybridize a range once it's been revoked/dropped? +*/ + /* * This monitor watches for changes to a key K that gets updated whenever there is a new prune intent. * On this change, we scan through all blobGranulePruneKeys (which look like = monitorPruneKeys(BlobManagerData* self) { // Wait for the watch to change, or some time to expire (whichever comes first) // before checking through the prune intents + + // TODO: pruneIntent written, watch triggered, prune work scheduled, before work is done another prune + // intent is written, then we'd wait a whole timeout to do the next pruneintent + // Use a UID to prevent this loop { try { state Future watchPruneIntentsChange = tr->watch(blobGranulePruneChangeKey); @@ -1674,6 +1710,7 @@ ACTOR Future monitorPruneKeys(BlobManagerData* self) { // loop through all prune intentions and do prune work accordingly state KeyRef beginKey = normalKeys.begin; loop { + state std::vector> prunes; try { // TODO: replace 10000 with a knob KeyRange nextRange(KeyRangeRef(beginKey, normalKeys.end)); @@ -1701,9 +1738,21 @@ ACTOR Future monitorPruneKeys(BlobManagerData* self) { // // ErrorOr would prob be what we need here - pruneRange(self, rangeStartKey, rangeEndKey, pruneVersion, force); + prunes.emplace_back(pruneRange(self, rangeStartKey, rangeEndKey, pruneVersion, force)); + + // TODO: maybe clear the key here if pruneRange succeeded } + // wait for this set of prunes to complete before starting the next ones since if we prune + // a range R at version V and while we are doing that, the time expires, we will end up + // trying to prune the same range again since the work isn't finished + // + // TODO: this isn't that efficient though. Instead we could keep metadata as part of the BM's + // memory that tracks which prunes are active. Once done, we can mark that work as done. If the BM + // fails then all prunes will fail and so the next BM will have a clear set of metadata (i.e. no + // work in progress) so we will end up doing the work in the new BM + wait(waitForAll(prunes)); + if (!pruneIntents.more) { break; } diff --git a/fdbserver/BlobWorker.actor.cpp b/fdbserver/BlobWorker.actor.cpp index 6ca872361e..86ff58ee45 100644 --- a/fdbserver/BlobWorker.actor.cpp +++ b/fdbserver/BlobWorker.actor.cpp @@ -35,6 +35,7 @@ #include "fdbclient/NativeAPI.actor.h" #include "fdbclient/Notified.h" #include "fdbserver/Knobs.h" +#include "fdbserver/BlobGranuleServerCommon.actor.h" #include "fdbserver/MutationTracking.h" #include "fdbserver/WaitFailure.h" #include "fdbserver/ServerDBInfo.h" @@ -46,6 +47,25 @@ #define BW_DEBUG true #define BW_REQUEST_DEBUG false +// represents a previous version of a granule, and optionally the files that compose it +struct GranuleHistoryEntry : NonCopyable, ReferenceCounted { + KeyRange range; + UID granuleID; + Version startVersion; // version of the first snapshot + Version endVersion; // version of the last delta file + + // load files lazily, and allows for clearing old cold-queried files to save memory + Future files; + + // FIXME: do skip pointers with single back-pointer and neighbor pointers + // Just parent reference for now (assumes no merging) + Reference parentGranule; + + GranuleHistoryEntry() : startVersion(invalidVersion), endVersion(invalidVersion) {} + GranuleHistoryEntry(KeyRange range, UID granuleID, Version startVersion, Version endVersion) + : range(range), granuleID(granuleID), startVersion(startVersion), endVersion(endVersion) {} +}; + struct GranuleStartState { UID granuleID; Version changeFeedStartVersion; @@ -230,6 +250,7 @@ ACTOR Future readAndCheckGranuleLock(Reference return Void(); } +<<<<<<< HEAD // used for "business logic" of both versions of loading granule files ACTOR Future readGranuleFiles(Transaction* tr, Key* startKey, Key endKey, GranuleFiles* files, UID granuleID) { @@ -299,7 +320,7 @@ ACTOR Future loadPreviousFiles(Transaction* tr, UID granuleID) { // no need to add conflict range for read b/c of granule lock state Key startKey = range.begin; state GranuleFiles files; - wait(readGranuleFiles(tr, &startKey, range.end, &files, granuleID)); + wait(readGranuleFiles(tr, &startKey, range.end, &files, granuleID, BW_DEBUG)); return files; } @@ -454,7 +475,7 @@ ACTOR Future writeDeltaFile(Reference bwData, wait(delay(0, TaskPriority::BlobWorkerUpdateStorage)); // TODO some sort of directory structure would be useful? - state std::string fname = granuleID.toString() + "_" deterministicRandom()->randomUniqueID().toString() + "_T" + + state std::string fname = deterministicRandom()->randomUniqueID().toString() + "_T" + std::to_string((uint64_t)(1000.0 * now())) + "_V" + std::to_string(currentDeltaVersion) + ".delta"; @@ -555,7 +576,7 @@ ACTOR Future writeSnapshot(Reference bwData, PromiseStream rows, bool createGranuleHistory) { // TODO some sort of directory structure would be useful maybe? - state std::string fname = granuleID.toString() + "_" deterministicRandom()->randomUniqueID().toString() + "_T" + + state std::string fname = deterministicRandom()->randomUniqueID().toString() + "_T" + std::to_string((uint64_t)(1000.0 * now())) + "_V" + std::to_string(version) + ".snapshot"; state Arena arena; state GranuleSnapshot snapshot; @@ -2038,7 +2059,10 @@ ACTOR Future handleBlobGranuleFileRequest(Reference bwData // lazily load files for old granule if not present chunkRange = cur->range; if (!cur->files.isValid() || cur->files.isError()) { - cur->files = loadHistoryFiles(bwData, cur->granuleID); + Transaction tr(bwData->db); + tr.setOption(FDBTransactionOptions::ACCESS_SYSTEM_KEYS); + tr.setOption(FDBTransactionOptions::PRIORITY_SYSTEM_IMMEDIATE); + cur->files = loadHistoryFiles(&tr, cur->granuleID, BW_DEBUG); } choose { @@ -2205,20 +2229,6 @@ ACTOR Future handleBlobGranuleFileRequest(Reference bwData return Void(); } -ACTOR Future> getLatestGranuleHistory(Transaction* tr, KeyRange range) { - KeyRange historyRange = blobGranuleHistoryKeyRangeFor(range); - RangeResult result = wait(tr->getRange(historyRange, 1, Snapshot::False, Reverse::True)); - ASSERT(result.size() <= 1); - - Optional history; - if (!result.empty()) { - std::pair decodedKey = decodeBlobGranuleHistoryKey(result[0].key); - ASSERT(range == decodedKey.first); - history = GranuleHistory(range, decodedKey.second, decodeBlobGranuleHistoryValue(result[0].value)); - } - return history; -} - ACTOR Future openGranule(Reference bwData, AssignBlobRangeRequest req) { ASSERT(req.type != AssignRequestType::Continue); state Transaction tr(bwData->db); diff --git a/fdbserver/CMakeLists.txt b/fdbserver/CMakeLists.txt index 79ed01cda0..fb589fb3cc 100644 --- a/fdbserver/CMakeLists.txt +++ b/fdbserver/CMakeLists.txt @@ -5,6 +5,8 @@ set(FDBSERVER_SRCS BackupProgress.actor.cpp BackupProgress.actor.h BackupWorker.actor.cpp + BlobGranuleServerCommon.actor.cpp + BlobGranuleServerCommon.actor.h BlobManager.actor.cpp BlobManagerInterface.h BlobWorker.actor.cpp From 63b7666f49e5532222229c7f553953104f37ba3c Mon Sep 17 00:00:00 2001 From: Suraj Gupta Date: Wed, 24 Nov 2021 10:17:55 -0500 Subject: [PATCH 083/413] Some more small fixes for compilation --- fdbclient/CMakeLists.txt | 1 - fdbserver/BlobGranuleServerCommon.actor.cpp | 2 +- fdbserver/BlobManager.actor.cpp | 1 + 3 files changed, 2 insertions(+), 2 deletions(-) diff --git a/fdbclient/CMakeLists.txt b/fdbclient/CMakeLists.txt index d105c4500c..f46d491a28 100644 --- a/fdbclient/CMakeLists.txt +++ b/fdbclient/CMakeLists.txt @@ -23,7 +23,6 @@ set(FDBCLIENT_SRCS BlobWorkerInterface.h BlobGranuleReader.actor.cpp BlobGranuleReader.actor.h - BlobGranuleCommon.cpp BlobGranuleCommon.h BlobGranuleFiles.cpp BlobGranuleFiles.h diff --git a/fdbserver/BlobGranuleServerCommon.actor.cpp b/fdbserver/BlobGranuleServerCommon.actor.cpp index d4ed410c92..d621f9dd07 100644 --- a/fdbserver/BlobGranuleServerCommon.actor.cpp +++ b/fdbserver/BlobGranuleServerCommon.actor.cpp @@ -29,7 +29,7 @@ // Gets the latest granule history node for range that was persisted ACTOR Future> getLatestGranuleHistory(Transaction* tr, KeyRange range) { - KeyRange historyRange = blobGranuleHistoryKeyRangeFor(range); + state KeyRange historyRange = blobGranuleHistoryKeyRangeFor(range); state RangeResult result; loop { diff --git a/fdbserver/BlobManager.actor.cpp b/fdbserver/BlobManager.actor.cpp index d99e637731..18d02b99fb 100644 --- a/fdbserver/BlobManager.actor.cpp +++ b/fdbserver/BlobManager.actor.cpp @@ -1831,6 +1831,7 @@ ACTOR Future blobManager(BlobManagerInterface bmInterf, self.addActor.send(monitorClientRanges(&self)); self.addActor.send(rangeAssigner(&self)); + self.addActor.send(monitorPruneKeys(&self)); if (BUGGIFY) { self.addActor.send(chaosRangeMover(&self)); From 2ccb3a4740fd1a4285f95a40a87aa7cdd53493b9 Mon Sep 17 00:00:00 2001 From: Suraj Gupta Date: Mon, 29 Nov 2021 11:53:40 -0500 Subject: [PATCH 084/413] Fix range boundaries and clearing intents. --- fdbserver/BlobManager.actor.cpp | 149 +++++++++++++++++++++++++------- fdbserver/BlobWorker.actor.cpp | 1 + 2 files changed, 120 insertions(+), 30 deletions(-) diff --git a/fdbserver/BlobManager.actor.cpp b/fdbserver/BlobManager.actor.cpp index 18d02b99fb..64e43fb91d 100644 --- a/fdbserver/BlobManager.actor.cpp +++ b/fdbserver/BlobManager.actor.cpp @@ -214,6 +214,7 @@ struct BlobManagerData { std::unordered_set deadWorkers; KeyRangeMap workerAssignments; KeyRangeMap knownBlobRanges; + KeyRangeMap prunesInProgress; AsyncTrigger startRecruiting; Debouncer restartRecruiting; @@ -1493,6 +1494,7 @@ ACTOR Future partiallyDeleteGranule(BlobManagerData* self, UID granuleId, } } + printf("partial deletion: deleting %d files\n", deletions.size()); wait(waitForAll(deletions)); // delete metadata in FDB (deleted file keys) @@ -1544,14 +1546,30 @@ ACTOR Future pruneRange(BlobManagerData* self, KeyRef startKey, KeyRef end state KeyRangeMap::iterator activeRange; for (activeRange = activeRanges.begin(); activeRange != activeRanges.end(); ++activeRange) { - // only want to prune exact granules - ASSERT(activeRange.begin() >= startKey && activeRange.end() < endKey); + // assumption: prune boundaries must respect granule boundaries + printf("looping over active range [%s-%s)=%s\n", + activeRange.begin().printable().c_str(), + activeRange.end().printable().c_str(), + activeRange.value().toString().c_str()); + + // ASSERT(activeRange.begin() >= startKey && activeRange.end() < endKey); + if (activeRange.begin() < startKey || activeRange.end() > endKey) { + continue; + } loop { try { + printf("fetching latest history for worker assignment [%s-%s)=%s\n", + activeRange.begin().printable().c_str(), + activeRange.end().printable().c_str(), + activeRange.value().toString().c_str()); Optional history = wait(getLatestGranuleHistory(&tr, activeRange.range())); - ASSERT(history.present()); - historyEntryQueue.push({ activeRange.range(), history.get().version, MAX_VERSION }); + // ASSERT(history.present()); + // TODO: can we tell from the krm that this range is not valid, so that we don't need to do a get + if (history.present()) { + printf("pushing onto history queue\n"); + historyEntryQueue.push({ activeRange.range(), history.get().version, MAX_VERSION }); + } break; } catch (Error& e) { wait(tr.onError(e)); @@ -1559,6 +1577,7 @@ ACTOR Future pruneRange(BlobManagerData* self, KeyRef startKey, KeyRef end } } + printf("starting to go through history queue\n"); while (!historyEntryQueue.empty()) { // process the node at the front of the queue and remove it KeyRange currRange; @@ -1617,41 +1636,51 @@ ACTOR Future pruneRange(BlobManagerData* self, KeyRef startKey, KeyRef end // race and we'll end up with unreachable nodes in the case of a crash state int i; + printf("%d granules to fully delete\n", toFullyDelete.size()); for (i = toFullyDelete.size() - 1; i >= 0; --i) { UID granuleId; KeyRef historyKey; std::tie(granuleId, historyKey) = toFullyDelete[i]; - // TODO: can possibly batch into a single txn + // FIXME: consider batching into a single txn (need to take care of txn size limit) wait(fullyDeleteGranule(self, granuleId, historyKey)); } - // TODO: could possibly do the partial deletes in parallel? + std::vector> partialDeletions; + printf("%d granules to partially delete\n", toPartiallyDelete.size()); for (i = toPartiallyDelete.size() - 1; i >= 0; --i) { UID granuleId = toPartiallyDelete[i]; - wait(partiallyDeleteGranule(self, granuleId, pruneVersion)); + partialDeletions.emplace_back(partiallyDeleteGranule(self, granuleId, pruneVersion)); } - // There could have been another pruneIntent that got written for this table while we - // were processing this one. If that is the case, we should not clear the key. Otherwise, - // we should clear the key to indicate the work is done. + wait(waitForAll(partialDeletions)); - // TODO: reuse tr - state Reference rywTr = makeReference(self->db); - rywTr->setOption(FDBTransactionOptions::ACCESS_SYSTEM_KEYS); - rywTr->setOption(FDBTransactionOptions::PRIORITY_SYSTEM_IMMEDIATE); + // Now that all the necessary granules and their files have been deleted, we can + // clear the pruneIntent key to signify that the work is done. However, there could have been + // another pruneIntent that got written for this table while we were processing this one. + // If that is the case, we should not clear the key. Otherwise, we can just clear the key. + + tr.reset(); loop { try { - state RangeResult pruneIntent = - wait(krmGetRanges(rywTr, blobGranulePruneKeys.begin, range, 1, GetRangeLimits::BYTE_LIMIT_UNLIMITED)); - ASSERT(pruneIntent.size() == 1); + tr.setOption(FDBTransactionOptions::ACCESS_SYSTEM_KEYS); + tr.setOption(FDBTransactionOptions::PRIORITY_SYSTEM_IMMEDIATE); - if (decodeBlobGranulePruneValue(pruneIntent[0].value).first == pruneVersion) { - rywTr->clear(pruneIntent[0].key); - wait(rywTr->commit()); + state Key pruneIntentKey = blobGranulePruneKeys.begin.withSuffix(startKey); + state Optional pruneIntentValue = wait(tr.get(pruneIntentKey)); + ASSERT(pruneIntentValue.present()); + + Version currPruneVersion; + bool currForce; + std::tie(currPruneVersion, currForce) = decodeBlobGranulePruneValue(pruneIntentValue.get()); + + if (currPruneVersion == pruneVersion && currForce == force) { + tr.clear(pruneIntentKey.withPrefix(blobGranulePruneKeys.begin)); + wait(tr.commit()); } break; } catch (Error& e) { - wait(rywTr->onError(e)); + printf("pruneRange got error %s\n", e.name()); + wait(tr.onError(e)); } } @@ -1661,6 +1690,8 @@ ACTOR Future pruneRange(BlobManagerData* self, KeyRef startKey, KeyRef end /* TODO: We need to revoke range from BW so that it doesn't try to add to a granule that we dropped Will SnowTram reuse table IDs; do we unhybridize a range once it's been revoked/dropped? + +Or can we possibly make the BW give it up upon seeing a change in the watch? */ /* @@ -1685,31 +1716,71 @@ Will SnowTram reuse table IDs; do we unhybridize a range once it's been revoked/ */ ACTOR Future monitorPruneKeys(BlobManagerData* self) { try { + state Value oldPruneWatchVal; loop { state Reference tr = makeReference(self->db); tr->setOption(FDBTransactionOptions::ACCESS_SYSTEM_KEYS); tr->setOption(FDBTransactionOptions::PRIORITY_SYSTEM_IMMEDIATE); // Wait for the watch to change, or some time to expire (whichever comes first) - // before checking through the prune intents - - // TODO: pruneIntent written, watch triggered, prune work scheduled, before work is done another prune - // intent is written, then we'd wait a whole timeout to do the next pruneintent - // Use a UID to prevent this + // before checking through the prune intents. We write a UID into the change key value + // so that we can still recognize when the watch key has been changed while we weren't + // monitoring it loop { try { + tr->setOption(FDBTransactionOptions::ACCESS_SYSTEM_KEYS); + tr->setOption(FDBTransactionOptions::PRIORITY_SYSTEM_IMMEDIATE); + + Optional newPruneWatchVal = wait(tr->get(blobGranulePruneChangeKey)); + + // if the value at the change key has changed, that means there is new work to do + if (newPruneWatchVal.present() && oldPruneWatchVal != newPruneWatchVal.get()) { + oldPruneWatchVal = newPruneWatchVal.get(); + printf("old and new watch don't match\n"); + break; + } + + // otherwise, there are no changes and we should wait until the next change (or timeout) state Future watchPruneIntentsChange = tr->watch(blobGranulePruneChangeKey); wait(tr->commit()); - wait(timeout(watchPruneIntentsChange, SERVER_KNOBS->BG_PRUNE_TIMEOUT, Void())); - break; + choose { + when(wait(watchPruneIntentsChange)) { tr->reset(); } + when(wait(delay(SERVER_KNOBS->BG_PRUNE_TIMEOUT))) { + printf("bg prune timeouts\n"); + break; + } + } + // wait(timeout(watchPruneIntentsChange, SERVER_KNOBS->BG_PRUNE_TIMEOUT, Void())); } catch (Error& e) { wait(tr->onError(e)); } } + tr->reset(); + + loop { + try { + tr->setOption(FDBTransactionOptions::ACCESS_SYSTEM_KEYS); + tr->setOption(FDBTransactionOptions::PRIORITY_SYSTEM_IMMEDIATE); + Version dummyV = wait(tr->getReadVersion()); + Value dummyValue = blobGranulePruneValueFor(dummyV, false); + wait(krmSetRange(tr, blobGranulePruneKeys.begin, normalKeys, dummyValue)); + wait(tr->commit()); + break; + } catch (Error& e) { + printf("dummy txn saw error %s\n", e.name()); + wait(tr->onError(e)); + } + } + + tr->reset(); + // loop through all prune intentions and do prune work accordingly state KeyRef beginKey = normalKeys.begin; loop { + tr->setOption(FDBTransactionOptions::ACCESS_SYSTEM_KEYS); + tr->setOption(FDBTransactionOptions::PRIORITY_SYSTEM_IMMEDIATE); + state std::vector> prunes; try { // TODO: replace 10000 with a knob @@ -1719,12 +1790,14 @@ ACTOR Future monitorPruneKeys(BlobManagerData* self) { // TODO: would we miss a range [pruneIntents[9999], pruneIntents[10000]) because of the `more`? // Or does `readThrough` take care of this? We also do this in recoverBlobManager + printf("pruneIntents.size()==%d\n", pruneIntents.size()); for (int rangeIdx = 0; rangeIdx < pruneIntents.size() - 1; ++rangeIdx) { if (pruneIntents[rangeIdx].value.size() == 0) { continue; } KeyRef rangeStartKey = pruneIntents[rangeIdx].key; KeyRef rangeEndKey = pruneIntents[rangeIdx + 1].key; + KeyRange range(KeyRangeRef(rangeStartKey, rangeEndKey)); Version pruneVersion; bool force; std::tie(pruneVersion, force) = decodeBlobGranulePruneValue(pruneIntents[rangeIdx].value); @@ -1736,11 +1809,26 @@ ACTOR Future monitorPruneKeys(BlobManagerData* self) { // fails... One way to prevent this is to not iterate over the prunes until the last iteration // is done (i.e waitForAll) // - // ErrorOr would prob be what we need here + /* + auto currPrunes = self->prunesInProgress.intersectingRanges(range); + int count = 0; + for (auto currPrune : currPrunes) { + count++; + if (currPrune.value() == pruneVersion) { + } + } + ASSERT(currPrunes.() <= 1); + */ + + printf("about to prune range [%s-%s) @ %d, force=%s\n", + rangeStartKey.printable().c_str(), + rangeEndKey.printable().c_str(), + pruneVersion, + force ? "T" : "F"); prunes.emplace_back(pruneRange(self, rangeStartKey, rangeEndKey, pruneVersion, force)); - // TODO: maybe clear the key here if pruneRange succeeded + // TODO: maybe clear the key here if pruneRange succeeded, but then we'd have to wait here } // wait for this set of prunes to complete before starting the next ones since if we prune @@ -1763,6 +1851,7 @@ ACTOR Future monitorPruneKeys(BlobManagerData* self) { wait(tr->onError(e)); } } + printf("done pruning all ranges. looping back\n"); } } catch (Error& e) { if (BM_DEBUG) { diff --git a/fdbserver/BlobWorker.actor.cpp b/fdbserver/BlobWorker.actor.cpp index 86ff58ee45..e04711996d 100644 --- a/fdbserver/BlobWorker.actor.cpp +++ b/fdbserver/BlobWorker.actor.cpp @@ -1205,6 +1205,7 @@ ACTOR Future blobGranuleUpdateFiles(Reference bwData, metadata->durableSnapshotVersion.set(startState.blobFilesToSnapshot.get().snapshotFiles.back().version); } else { ASSERT(startState.previousDurableVersion == invalidVersion); + printf("About to dump initial snapshot\n"); BlobFileIndex fromFDB = wait(dumpInitialSnapshotFromFDB(bwData, metadata, startState.granuleID)); newSnapshotFile = fromFDB; ASSERT(startState.changeFeedStartVersion <= fromFDB.version); From 310d990b125c688ec2894f7f8c823a9314d6a291 Mon Sep 17 00:00:00 2001 From: Suraj Gupta Date: Fri, 3 Dec 2021 14:46:48 -0500 Subject: [PATCH 085/413] Add debugging. --- fdbclient/NativeAPI.actor.cpp | 270 +++++++++++++++ fdbclient/ServerKnobs.cpp | 2 +- fdbserver/BlobManager.actor.cpp | 310 ++++++++++++------ fdbserver/BlobWorker.actor.cpp | 126 ++++++- fdbserver/MutationTracking.cpp | 10 +- fdbserver/MutationTracking.h | 2 +- .../workloads/BlobGranuleVerifier.actor.cpp | 52 +++ 7 files changed, 668 insertions(+), 104 deletions(-) diff --git a/fdbclient/NativeAPI.actor.cpp b/fdbclient/NativeAPI.actor.cpp index 73b93a39b1..acf2b8af5d 100644 --- a/fdbclient/NativeAPI.actor.cpp +++ b/fdbclient/NativeAPI.actor.cpp @@ -7831,6 +7831,276 @@ Future DatabaseContext::popChangeFeedMutations(Key rangeID, Version versio return popChangeFeedMutationsActor(Reference::addRef(this), rangeID, version); } +#define BG_REQUEST_DEBUG true + +ACTOR Future getBlobGranuleRangesStreamActor(Reference db, + PromiseStream results, + KeyRange keyRange) { + // FIXME: use streaming range read + state Database cx(db); + state Reference tr = makeReference(cx); + state KeyRange currentRange = keyRange; + if (BG_REQUEST_DEBUG) { + printf("Getting Blob Granules for [%s - %s)\n", + keyRange.begin.printable().c_str(), + keyRange.end.printable().c_str()); + } + loop { + try { + tr->setOption(FDBTransactionOptions::ACCESS_SYSTEM_KEYS); + state RangeResult blobGranuleMapping = wait(krmGetRanges( + tr, blobGranuleMappingKeys.begin, currentRange, 1000, GetRangeLimits::BYTE_LIMIT_UNLIMITED)); + + for (int i = 0; i < blobGranuleMapping.size() - 1; i++) { + if (blobGranuleMapping[i].value.size()) { + results.send(KeyRangeRef(blobGranuleMapping[i].key, blobGranuleMapping[i + 1].key)); + } + } + if (blobGranuleMapping.more) { + currentRange = KeyRangeRef(blobGranuleMapping.back().key, currentRange.end); + } else { + results.sendError(end_of_stream()); + return Void(); + } + } catch (Error& e) { + wait(tr->onError(e)); + } + } +} + +Future DatabaseContext::getBlobGranuleRangesStream(const PromiseStream& results, KeyRange range) { + if (!CLIENT_KNOBS->ENABLE_BLOB_GRANULES) { + throw client_invalid_operation(); + } + return getBlobGranuleRangesStreamActor(Reference::addRef(this), results, range); +} + +// hack (for now) to get blob worker interface into load balance +struct BWLocationInfo : MultiInterface> { + using Locations = MultiInterface>; + explicit BWLocationInfo(const std::vector>>& v) : Locations(v) {} +}; + +ACTOR Future readBlobGranulesStreamActor(Reference db, + PromiseStream> results, + KeyRange range, + Version begin, + Optional end) { // end not present is just latest + state Database cx(db); + state Reference tr = makeReference(cx); + state RangeResult blobGranuleMapping; + state Version endVersion; + state Key granuleStartKey; + state Key granuleEndKey; + state KeyRange keyRange = range; + state int i, loopCounter = 0; + state UID workerId; + loop { + try { + // FIXME: Use streaming parallelism? + // Read mapping and worker interfaces from DB + loopCounter++; + loop { + try { + tr->reset(); + tr->setOption(FDBTransactionOptions::ACCESS_SYSTEM_KEYS); + if (loopCounter == 1) { + // if retrying, use new version for mapping but original version for read version + if (end.present()) { + endVersion = end.get(); + } else { + Version _end = wait(tr->getReadVersion()); + endVersion = _end; + } + } + + // Right now just read whole blob range assignments from DB + // FIXME: eventually we probably want to cache this and invalidate similarly to storage servers. + // Cache misses could still read from the DB, or we could add it to the Transaction State Store and + // have proxies serve it from memory. + RangeResult _bgMapping = wait(krmGetRanges( + tr, blobGranuleMappingKeys.begin, keyRange, 1000, GetRangeLimits::BYTE_LIMIT_UNLIMITED)); + blobGranuleMapping = _bgMapping; + if (blobGranuleMapping.more) { + if (BG_REQUEST_DEBUG) { + // printf("BG Mapping for [%s - %s) too large!\n"); + } + throw unsupported_operation(); + } + ASSERT(!blobGranuleMapping.more && blobGranuleMapping.size() < CLIENT_KNOBS->TOO_MANY); + + if (blobGranuleMapping.size() == 0) { + if (BG_REQUEST_DEBUG) { + printf("no blob worker assignments yet \n"); + } + throw transaction_too_old(); + } + + if (BG_REQUEST_DEBUG) { + fmt::print("Doing blob granule request @ {}\n", endVersion); + fmt::print("blob worker assignments:\n"); + } + + for (i = 0; i < blobGranuleMapping.size() - 1; i++) { + granuleStartKey = blobGranuleMapping[i].key; + granuleEndKey = blobGranuleMapping[i + 1].key; + if (!blobGranuleMapping[i].value.size()) { + if (BG_REQUEST_DEBUG) { + printf("Key range [%s - %s) missing worker assignment!\n", + granuleStartKey.printable().c_str(), + granuleEndKey.printable().c_str()); + // TODO probably new exception type instead + } + throw transaction_too_old(); + } + + workerId = decodeBlobGranuleMappingValue(blobGranuleMapping[i].value); + if (BG_REQUEST_DEBUG) { + printf(" [%s - %s): %s\n", + granuleStartKey.printable().c_str(), + granuleEndKey.printable().c_str(), + workerId.toString().c_str()); + } + + if (!cx->blobWorker_interf.count(workerId)) { + Optional workerInterface = wait(tr->get(blobWorkerListKeyFor(workerId))); + // from the time the mapping was read from the db, the associated blob worker + // could have died and so its interface wouldn't be present as part of the blobWorkerList + // we persist in the db. So throw wrong_shard_server to get the new mapping + if (!workerInterface.present()) { + throw wrong_shard_server(); + } + cx->blobWorker_interf[workerId] = decodeBlobWorkerListValue(workerInterface.get()); + if (BG_REQUEST_DEBUG) { + printf(" decoded worker interface for %s\n", workerId.toString().c_str()); + } + } + } + break; + } catch (Error& e) { + wait(tr->onError(e)); + } + } + + // Make request for each granule + for (i = 0; i < blobGranuleMapping.size() - 1; i++) { + granuleStartKey = blobGranuleMapping[i].key; + granuleEndKey = blobGranuleMapping[i + 1].key; + // if this was a time travel and the request returned larger bounds, skip this chunk + if (granuleEndKey <= keyRange.begin) { + continue; + } + workerId = decodeBlobGranuleMappingValue(blobGranuleMapping[i].value); + // prune first/last granules to requested range + if (keyRange.begin > granuleStartKey) { + granuleStartKey = keyRange.begin; + } + if (keyRange.end < granuleEndKey) { + granuleEndKey = keyRange.end; + } + + state BlobGranuleFileRequest req; + req.keyRange = KeyRangeRef(StringRef(req.arena, granuleStartKey), StringRef(req.arena, granuleEndKey)); + req.beginVersion = begin; + req.readVersion = endVersion; + + std::vector>> v; + v.push_back(makeReference>(cx->blobWorker_interf[workerId])); + state Reference>> location = + makeReference(v); + // use load balance with one option for now for retry and error handling + BlobGranuleFileReply rep = wait(loadBalance(location, + &BlobWorkerInterface::blobGranuleFileRequest, + req, + TaskPriority::DefaultPromiseEndpoint, + AtMostOnce::False, + nullptr)); + + if (BG_REQUEST_DEBUG) { + fmt::print("Blob granule request for [{0} - {1}) @ {2} - {3} got reply from {4}:\n", + granuleStartKey.printable(), + granuleEndKey.printable(), + begin, + endVersion, + workerId.toString()); + } + for (auto& chunk : rep.chunks) { + if (BG_REQUEST_DEBUG) { + printf("[%s - %s)\n", + chunk.keyRange.begin.printable().c_str(), + chunk.keyRange.end.printable().c_str()); + + printf(" SnapshotFile:\n %s\n", + chunk.snapshotFile.present() ? chunk.snapshotFile.get().toString().c_str() : ""); + printf(" DeltaFiles:\n"); + for (auto& df : chunk.deltaFiles) { + printf(" %s\n", df.toString().c_str()); + } + printf(" Deltas: (%d)", chunk.newDeltas.size()); + if (chunk.newDeltas.size() > 0) { + fmt::print(" with version [{0} - {1}]", + chunk.newDeltas[0].version, + chunk.newDeltas[chunk.newDeltas.size() - 1].version); + } + fmt::print(" IncludedVersion: {}\n", chunk.includedVersion); + printf("\n\n"); + } + Arena a; + a.dependsOn(rep.arena); + results.send(Standalone(chunk, a)); + keyRange = KeyRangeRef(chunk.keyRange.end, keyRange.end); + } + } + results.sendError(end_of_stream()); + return Void(); + } catch (Error& e) { + if (e.code() == error_code_actor_cancelled) { + throw; + } + if (e.code() == error_code_wrong_shard_server || e.code() == error_code_all_alternatives_failed || + e.code() == error_code_connection_failed) { + // TODO would invalidate mapping cache here if we had it + wait(delay(CLIENT_KNOBS->WRONG_SHARD_SERVER_DELAY)); + } else { + if (BG_REQUEST_DEBUG) { + printf("blob granule file request got unexpected error %s\n", e.name()); + } + results.sendError(e); + return Void(); + } + } + } +} + +Future DatabaseContext::readBlobGranulesStream(const PromiseStream>& results, + KeyRange range, + Version begin, + Optional end) { + if (!CLIENT_KNOBS->ENABLE_BLOB_GRANULES) { + throw client_invalid_operation(); + } + return readBlobGranulesStreamActor(Reference::addRef(this), results, range, begin, end); +} + +ACTOR Future setPerpetualStorageWiggle(Database cx, bool enable, LockAware lockAware) { + state ReadYourWritesTransaction tr(cx); + loop { + try { + tr.setOption(FDBTransactionOptions::ACCESS_SYSTEM_KEYS); + if (lockAware) { + tr.setOption(FDBTransactionOptions::LOCK_AWARE); + } + + tr.set(perpetualStorageWiggleKey, enable ? "1"_sr : "0"_sr); + wait(tr.commit()); + break; + } catch (Error& e) { + wait(tr.onError(e)); + } + } + return Void(); +} + Reference DatabaseContext::createTransaction() { return makeReference(Database(Reference::addRef(this))); } diff --git a/fdbclient/ServerKnobs.cpp b/fdbclient/ServerKnobs.cpp index e9b14ac2b2..fde9d49203 100644 --- a/fdbclient/ServerKnobs.cpp +++ b/fdbclient/ServerKnobs.cpp @@ -786,7 +786,7 @@ void ServerKnobs::initialize(Randomize randomize, ClientKnobs* clientKnobs, IsSi init( BG_SNAPSHOT_FILE_TARGET_BYTES, 10000000 ); if( randomize && BUGGIFY ) { deterministicRandom()->random01() < 0.1 ? BG_SNAPSHOT_FILE_TARGET_BYTES /= 100 : BG_SNAPSHOT_FILE_TARGET_BYTES /= 10; } init( BG_DELTA_BYTES_BEFORE_COMPACT, BG_SNAPSHOT_FILE_TARGET_BYTES/2 ); init( BG_DELTA_FILE_TARGET_BYTES, BG_DELTA_BYTES_BEFORE_COMPACT/10 ); - init( BG_PRUNE_TIMEOUT, 60 * 60); + init( BG_PRUNE_TIMEOUT, 60*60); init( BLOB_WORKER_TIMEOUT, 10.0 ); if( randomize && BUGGIFY ) BLOB_WORKER_TIMEOUT = 1.0; init( BLOB_WORKERLIST_FETCH_INTERVAL, 1.0 ); diff --git a/fdbserver/BlobManager.actor.cpp b/fdbserver/BlobManager.actor.cpp index 64e43fb91d..48302c25c6 100644 --- a/fdbserver/BlobManager.actor.cpp +++ b/fdbserver/BlobManager.actor.cpp @@ -18,6 +18,7 @@ * limitations under the License. */ +#include #include #include #include @@ -1412,30 +1413,49 @@ ACTOR Future loadHistoryFiles(BlobManagerData* bmData, UID granule * also removes the history entry for this granule from the system keyspace */ ACTOR Future fullyDeleteGranule(BlobManagerData* self, UID granuleId, KeyRef historyKey) { + if (BM_DEBUG) { + printf("Fully deleting granule %s: init\n", granuleId.toString().c_str()); + } + state Transaction tr(self->db); tr.setOption(FDBTransactionOptions::ACCESS_SYSTEM_KEYS); tr.setOption(FDBTransactionOptions::PRIORITY_SYSTEM_IMMEDIATE); - KeyRange filesRange = blobGranuleFileKeyRangeFor(granuleId); - // get files GranuleFiles files = wait(loadHistoryFiles(&tr, granuleId, BM_DEBUG)); std::vector> deletions; + std::vector filesToDelete; // TODO: remove, just for debugging for (auto snapshotFile : files.snapshotFiles) { std::string fname = snapshotFile.filename; deletions.emplace_back(self->bstore->deleteFile(fname)); + filesToDelete.emplace_back(fname); } for (auto deltaFile : files.deltaFiles) { std::string fname = deltaFile.filename; deletions.emplace_back(self->bstore->deleteFile(fname)); + filesToDelete.emplace_back(fname); } + if (BM_DEBUG) { + printf("Fully deleting granule %s: deleting %d files\n", granuleId.toString().c_str(), deletions.size()); + for (auto filename : filesToDelete) { + printf(" - %s\n", filename.c_str()); + } + } + + // delete the files before the corresponding metadata. + // this could lead to dangling pointers in fdb, but this granule should + // never be read again anyways, and we can clean up the keys the next time around. + // deleting files before corresponding metadata reduces the # of orphaned files. wait(waitForAll(deletions)); // delete metadata in FDB (history entry and file keys) + if (BM_DEBUG) { + printf("Fully deleting granule %s: deleting history and file keys\n", granuleId.toString().c_str()); + } loop { try { KeyRange fileRangeKey = blobGranuleFileKeyRangeFor(granuleId); @@ -1448,6 +1468,10 @@ ACTOR Future fullyDeleteGranule(BlobManagerData* self, UID granuleId, KeyR } } + if (BM_DEBUG) { + printf("Fully deleting granule %s: success\n", granuleId.toString().c_str()); + } + return Void(); } @@ -1456,51 +1480,88 @@ ACTOR Future fullyDeleteGranule(BlobManagerData* self, UID granuleId, KeyR * version <= pruneVersion and deletes all files older than it. */ ACTOR Future partiallyDeleteGranule(BlobManagerData* self, UID granuleId, Version pruneVersion) { + if (BM_DEBUG) { + printf("Partially deleting granule %s: init\n", granuleId.toString().c_str()); + } + state Transaction tr(self->db); tr.setOption(FDBTransactionOptions::ACCESS_SYSTEM_KEYS); tr.setOption(FDBTransactionOptions::PRIORITY_SYSTEM_IMMEDIATE); - KeyRange filesRange = blobGranuleFileKeyRangeFor(granuleId); - + // get files GranuleFiles files = wait(loadHistoryFiles(&tr, granuleId, BM_DEBUG)); - Version latestSnaphotVersion = invalidVersion; + // represents the version of the latest snapshot file in this granule with G.version < pruneVersion + Version latestSnapshotVersion = invalidVersion; - state std::vector> deletions; - state std::vector deletedFileKeys; + state std::vector> deletions; // deletion work per file + state std::vector deletedFileKeys; // keys for deleted files + state std::vector filesToDelete; // TODO: remove evenutally, just for debugging // TODO: binary search these snapshot files for latestSnapshotVersion for (int idx = files.snapshotFiles.size() - 1; idx >= 0; --idx) { // if we already found the latestSnapshotVersion, this snapshot can be deleted - if (latestSnaphotVersion != invalidVersion) { + if (latestSnapshotVersion != invalidVersion) { std::string fname = files.snapshotFiles[idx].filename; deletions.emplace_back(self->bstore->deleteFile(fname)); deletedFileKeys.emplace_back(blobGranuleFileKeyFor(granuleId, 'S', files.snapshotFiles[idx].version)); + filesToDelete.emplace_back(fname); } else if (files.snapshotFiles[idx].version <= pruneVersion) { // otherwise if this is the FIRST snapshot file with version < pruneVersion, // then we found our latestSnapshotVersion (FIRST since we are traversing in reverse) - latestSnaphotVersion = files.snapshotFiles[idx].version; + latestSnapshotVersion = files.snapshotFiles[idx].version; } } - ASSERT(latestSnaphotVersion != invalidVersion); + // we would have only partially deleted the granule if such a snapshot existed + ASSERT(latestSnapshotVersion != invalidVersion); // delete all delta files older than latestSnapshotVersion for (auto deltaFile : files.deltaFiles) { - if (deltaFile.version < latestSnaphotVersion) { - std::string fname = deltaFile.filename; - deletions.emplace_back(self->bstore->deleteFile(fname)); - deletedFileKeys.emplace_back(blobGranuleFileKeyFor(granuleId, 'D', deltaFile.version)); + // traversing in fwd direction, so stop once we find the first delta file past the latestSnapshotVersion + if (deltaFile.version > latestSnapshotVersion) { + break; + } + + // otherwise deltaFile.version <= latestSnapshotVersion so delete it + // == should also be deleted because the last delta file before a snapshot would have the same version + std::string fname = deltaFile.filename; + deletions.emplace_back(self->bstore->deleteFile(fname)); + deletedFileKeys.emplace_back(blobGranuleFileKeyFor(granuleId, 'D', deltaFile.version)); + filesToDelete.emplace_back(fname); + } + + if (BM_DEBUG) { + printf("Partially deleting granule %s: deleting %d files\n", granuleId.toString().c_str(), deletions.size()); + for (auto filename : filesToDelete) { + printf(" - %s\n", filename.c_str()); } } - printf("partial deletion: deleting %d files\n", deletions.size()); + // TODO: the following comment relies on the assumption that BWs will not get requests to + // read data that was already pruned. confirm assumption is fine. otherwise, we'd need + // to communicate with BWs here and have them ack the pruneVersion + + // delete the files before the corresponding metadata. + // this could lead to dangling pointers in fdb, but we should never read data older than + // pruneVersion anyways, and we can clean up the keys the next time around. + // deleting files before corresponding metadata reduces the # of orphaned files. wait(waitForAll(deletions)); // delete metadata in FDB (deleted file keys) + // TODO: do we need to also update the start version for the history entry? + // it would be a blind write here so might that cause a problem with history traversal in BW? + // do we gain any benefit from updating it? even if we keep the old start version, the worst is + // someone requests a version in [oldStartVersion, newStartVersion) and we fail to return + // any files for that request. + + if (BM_DEBUG) { + printf("Partially deleting granule %s: deleting file keys\n", granuleId.toString().c_str()); + } + loop { try { - for (auto key : deletedFileKeys) { + for (auto& key : deletedFileKeys) { tr.clear(key); } wait(tr.commit()); @@ -1509,6 +1570,10 @@ ACTOR Future partiallyDeleteGranule(BlobManagerData* self, UID granuleId, wait(tr.onError(e)); } } + + if (BM_DEBUG) { + printf("Partially deleting granule %s: success\n", granuleId.toString().c_str()); + } return Void(); } @@ -1519,11 +1584,15 @@ ACTOR Future partiallyDeleteGranule(BlobManagerData* self, UID granuleId, * and nodes that can be partially deleted (i.e. some of their files can be deleted). * Once all this is done, we finally clear the pruneIntent key, if possible, to indicate we are done * processing this prune intent. - * - * TODO: communicate the prune to blob workers so they can clean up local memory - * maybe BWs can just watch the prune keys as well! */ ACTOR Future pruneRange(BlobManagerData* self, KeyRef startKey, KeyRef endKey, Version pruneVersion, bool force) { + if (BM_DEBUG) { + printf("pruneRange starting for range [%s-%s) @ pruneVersion=%lld, force=%s\n", + startKey.printable().c_str(), + endKey.printable().c_str(), + pruneVersion); + } + // queue of for BFS traversal of history // TODO: consider using GranuleHistoryEntry, but that also makes it a little messy state std::queue> historyEntryQueue; @@ -1546,28 +1615,35 @@ ACTOR Future pruneRange(BlobManagerData* self, KeyRef startKey, KeyRef end state KeyRangeMap::iterator activeRange; for (activeRange = activeRanges.begin(); activeRange != activeRanges.end(); ++activeRange) { - // assumption: prune boundaries must respect granule boundaries - printf("looping over active range [%s-%s)=%s\n", - activeRange.begin().printable().c_str(), - activeRange.end().printable().c_str(), - activeRange.value().toString().c_str()); + if (BM_DEBUG) { + printf("Checking if active range [%s-%s), owned by BW %s, should be pruned\n", + activeRange.begin().printable().c_str(), + activeRange.end().printable().c_str(), + activeRange.value().toString().c_str()); + } - // ASSERT(activeRange.begin() >= startKey && activeRange.end() < endKey); + // assumption: prune boundaries must respect granule boundaries if (activeRange.begin() < startKey || activeRange.end() > endKey) { continue; } + // TODO: if this is a force prune, then revoke the assignment from the corresponding BW first + // so that it doesn't try to interact with the granule (i.e. force it to give up gLock). + // we'll need some way to ack that the revoke was successful + loop { try { - printf("fetching latest history for worker assignment [%s-%s)=%s\n", - activeRange.begin().printable().c_str(), - activeRange.end().printable().c_str(), - activeRange.value().toString().c_str()); + if (BM_DEBUG) { + printf("Fetching latest history entry for range [%s-%s)\n", + activeRange.begin().printable().c_str(), + activeRange.end().printable().c_str()) + } Optional history = wait(getLatestGranuleHistory(&tr, activeRange.range())); - // ASSERT(history.present()); // TODO: can we tell from the krm that this range is not valid, so that we don't need to do a get if (history.present()) { - printf("pushing onto history queue\n"); + if (BM_DEBUG) { + printf("Adding range to history queue\n"); + } historyEntryQueue.push({ activeRange.range(), history.get().version, MAX_VERSION }); } break; @@ -1577,7 +1653,9 @@ ACTOR Future pruneRange(BlobManagerData* self, KeyRef startKey, KeyRef end } } - printf("starting to go through history queue\n"); + if (BM_DEBUG) { + printf("Beginning BFS traversal of history\n"); + } while (!historyEntryQueue.empty()) { // process the node at the front of the queue and remove it KeyRange currRange; @@ -1586,6 +1664,14 @@ ACTOR Future pruneRange(BlobManagerData* self, KeyRef startKey, KeyRef end std::tie(currRange, startVersion, endVersion) = historyEntryQueue.front(); historyEntryQueue.pop(); + if (BM_DEBUG) { + printf("Processing history node [%s-%s) with versions [%lld, %lld)\n", + currRange.begin.printable().c_str(), + currRange.end.printable().c_str(), + startVersion, + endVersion); + } + // get the persisted history entry for this granule state Standalone currHistoryNode; state KeyRef historyKey = blobGranuleHistoryKeyFor(currRange, startVersion); @@ -1600,8 +1686,16 @@ ACTOR Future pruneRange(BlobManagerData* self, KeyRef startKey, KeyRef end } } + if (BM_DEBUG) { + printf("Found history entry for this node. It's granuleID is %s\n", + currHistoryNode.granuleID.toString().c_str()); + } + // if we already saw this node, skip it; otherwise, mark it as visited if (visited.count(currHistoryNode.granuleID)) { + if (BM_DEBUG) { + printf("Already processed %s, so skipping it\n", currHistoryNode.granuleID.toString().c_str()); + } continue; } visited.insert(currHistoryNode.granuleID); @@ -1609,18 +1703,32 @@ ACTOR Future pruneRange(BlobManagerData* self, KeyRef startKey, KeyRef end // There are three cases this granule can fall into: // - if the granule's end version is at or before the prune version or this is a force delete, // this granule should be completely deleted - // - else if the startVersion <= pruneVersion, then G.startVersion <= pruneVersion < G.endVersion + // - else if the startVersion <= pruneVersion, then G.startVersion < pruneVersion < G.endVersion // and so this granule should be partially deleted // - otherwise, this granule is active, so don't schedule it for deletion if (force || endVersion <= pruneVersion) { + if (BM_DEBUG) { + printf("Granule %s will be FULLY deleted\n", currHistoryNode.granuleID.toString().c_str()); + } toFullyDelete.push_back({ currHistoryNode.granuleID, historyKey }); - } else if (startVersion <= pruneVersion) { + } else if (startVersion < pruneVersion) { + if (BM_DEBUG) { + printf("Granule %s will be partially deleted\n", currHistoryNode.granuleID.toString().c_str()); + } toPartiallyDelete.push_back({ currHistoryNode.granuleID }); } // add all of the node's parents to the queue for (auto& parent : currHistoryNode.parentGranules) { - // the parent's end version is this node's startVersion + // the parent's end version is this node's startVersion, + // since this node must have started where it's parent finished + if (BM_DEBUG) { + printf("Adding parent [%s-%s) with versions [%lld-%lld) to queue\n", + parent.first.begin.printable().c_str(), + parent.first.end.printable().c_str(), + parent.second, + startVersion); + } historyEntryQueue.push({ parent.first, parent.second, startVersion }); } } @@ -1632,23 +1740,34 @@ ACTOR Future pruneRange(BlobManagerData* self, KeyRef startKey, KeyRef end // any node that must be partially deleted must occur later on in the history. Thus, // we delete the 'toFullyDelete' granules first. // - // Unfortunately we can't do multiple deletions in parallel because they might - // race and we'll end up with unreachable nodes in the case of a crash + // Unfortunately we can't do parallelize _full_ deletions because they might + // race and we'll end up with unreachable nodes in the case of a crash. + // Since partial deletions only occur for "leafs", they can be done in parallel state int i; - printf("%d granules to fully delete\n", toFullyDelete.size()); + if (BM_DEBUG) { + printf("%d granules to fully delete\n", toFullyDelete.size()); + } for (i = toFullyDelete.size() - 1; i >= 0; --i) { UID granuleId; KeyRef historyKey; std::tie(granuleId, historyKey) = toFullyDelete[i]; // FIXME: consider batching into a single txn (need to take care of txn size limit) + if (BM_DEBUG) { + printf("About to fully delete granule %s\n", granuleId.toString().c_str()); + } wait(fullyDeleteGranule(self, granuleId, historyKey)); } + if (BM_DEBUG) { + printf("%d granules to partially delete\n", toPartiallyDelete.size()); + } std::vector> partialDeletions; - printf("%d granules to partially delete\n", toPartiallyDelete.size()); for (i = toPartiallyDelete.size() - 1; i >= 0; --i) { UID granuleId = toPartiallyDelete[i]; + if (BM_DEBUG) { + printf("About to partially delete granule %s\n", granuleId.toString().c_str()); + } partialDeletions.emplace_back(partiallyDeleteGranule(self, granuleId, pruneVersion)); } @@ -1660,6 +1779,9 @@ ACTOR Future pruneRange(BlobManagerData* self, KeyRef startKey, KeyRef end // If that is the case, we should not clear the key. Otherwise, we can just clear the key. tr.reset(); + if (BM_DEBUG) { + printf("About to clear prune intent\n"); + } loop { try { tr.setOption(FDBTransactionOptions::ACCESS_SYSTEM_KEYS); @@ -1679,21 +1801,20 @@ ACTOR Future pruneRange(BlobManagerData* self, KeyRef startKey, KeyRef end } break; } catch (Error& e) { - printf("pruneRange got error %s\n", e.name()); + printf("Attempt to clear prune intent got error %s\n", e.name()); wait(tr.onError(e)); } } + if (BM_DEBUG) { + printf("Successfully pruned range [%s-%s) at pruneVersion=%lld\n", + startKey.printable().c_str(), + endKey.printable().c_str(), + pruneVersion); + } return Void(); } -/* -TODO: We need to revoke range from BW so that it doesn't try to add to a granule that we dropped -Will SnowTram reuse table IDs; do we unhybridize a range once it's been revoked/dropped? - -Or can we possibly make the BW give it up upon seeing a change in the watch? -*/ - /* * This monitor watches for changes to a key K that gets updated whenever there is a new prune intent. * On this change, we scan through all blobGranulePruneKeys (which look like = monitorPruneKeys(BlobManagerData* self) { tr->setOption(FDBTransactionOptions::ACCESS_SYSTEM_KEYS); tr->setOption(FDBTransactionOptions::PRIORITY_SYSTEM_IMMEDIATE); - Optional newPruneWatchVal = wait(tr->get(blobGranulePruneChangeKey)); + state Optional newPruneWatchVal = wait(tr->get(blobGranulePruneChangeKey)); // if the value at the change key has changed, that means there is new work to do if (newPruneWatchVal.present() && oldPruneWatchVal != newPruneWatchVal.get()) { oldPruneWatchVal = newPruneWatchVal.get(); - printf("old and new watch don't match\n"); + if (BM_DEBUG) { + printf("the blobGranulePruneChangeKey changed\n"); + } + + // TODO: debugging code, remove it + if (newPruneWatchVal.get().toString().substr(0, 6) == "random") { + state Reference dummy = + makeReference(self->db); + loop { + try { + dummy->setOption(FDBTransactionOptions::ACCESS_SYSTEM_KEYS); + dummy->setOption(FDBTransactionOptions::PRIORITY_SYSTEM_IMMEDIATE); + std::istringstream iss(newPruneWatchVal.get().toString().substr(6)); + Version version; + iss >> version; + dummy->set(blobGranulePruneKeys.begin.withSuffix(normalKeys.begin), + blobGranulePruneValueFor(version, false)); + wait(dummy->commit()); + break; + + } catch (Error& e) { + wait(dummy->onError(e)); + } + } + } break; } // otherwise, there are no changes and we should wait until the next change (or timeout) state Future watchPruneIntentsChange = tr->watch(blobGranulePruneChangeKey); wait(tr->commit()); + + if (BM_DEBUG) { + printf("monitorPruneKeys waiting for change or timeout\n"); + } + choose { - when(wait(watchPruneIntentsChange)) { tr->reset(); } + when(wait(watchPruneIntentsChange)) { + if (BM_DEBUG) { + printf("monitorPruneKeys saw a change\n"); + } + tr->reset(); + } when(wait(delay(SERVER_KNOBS->BG_PRUNE_TIMEOUT))) { - printf("bg prune timeouts\n"); + if (BM_DEBUG) { + printf("monitorPruneKeys got a timeout\n"); + } break; } } - // wait(timeout(watchPruneIntentsChange, SERVER_KNOBS->BG_PRUNE_TIMEOUT, Void())); } catch (Error& e) { wait(tr->onError(e)); } @@ -1758,23 +1914,10 @@ ACTOR Future monitorPruneKeys(BlobManagerData* self) { tr->reset(); - loop { - try { - tr->setOption(FDBTransactionOptions::ACCESS_SYSTEM_KEYS); - tr->setOption(FDBTransactionOptions::PRIORITY_SYSTEM_IMMEDIATE); - Version dummyV = wait(tr->getReadVersion()); - Value dummyValue = blobGranulePruneValueFor(dummyV, false); - wait(krmSetRange(tr, blobGranulePruneKeys.begin, normalKeys, dummyValue)); - wait(tr->commit()); - break; - } catch (Error& e) { - printf("dummy txn saw error %s\n", e.name()); - wait(tr->onError(e)); - } + if (BM_DEBUG) { + printf("Looping over prune intents\n"); } - tr->reset(); - // loop through all prune intentions and do prune work accordingly state KeyRef beginKey = normalKeys.begin; loop { @@ -1802,43 +1945,22 @@ ACTOR Future monitorPruneKeys(BlobManagerData* self) { bool force; std::tie(pruneVersion, force) = decodeBlobGranulePruneValue(pruneIntents[rangeIdx].value); - // TODO: should we add this to an actor collection or a list of futures? - // Probably because still need to handle the case of one prune at version V and then timer - // expires and we start another prune again at version V. we need to keep track of what's in - // progress. That brings another problem though: what happens if something is in progress and - // fails... One way to prevent this is to not iterate over the prunes until the last iteration - // is done (i.e waitForAll) - // - - /* - auto currPrunes = self->prunesInProgress.intersectingRanges(range); - int count = 0; - for (auto currPrune : currPrunes) { - count++; - if (currPrune.value() == pruneVersion) { - } - } - ASSERT(currPrunes.() <= 1); - */ - printf("about to prune range [%s-%s) @ %d, force=%s\n", rangeStartKey.printable().c_str(), rangeEndKey.printable().c_str(), pruneVersion, force ? "T" : "F"); prunes.emplace_back(pruneRange(self, rangeStartKey, rangeEndKey, pruneVersion, force)); - - // TODO: maybe clear the key here if pruneRange succeeded, but then we'd have to wait here } // wait for this set of prunes to complete before starting the next ones since if we prune // a range R at version V and while we are doing that, the time expires, we will end up - // trying to prune the same range again since the work isn't finished + // trying to prune the same range again since the work isn't finished and the prunes will race // // TODO: this isn't that efficient though. Instead we could keep metadata as part of the BM's - // memory that tracks which prunes are active. Once done, we can mark that work as done. If the BM - // fails then all prunes will fail and so the next BM will have a clear set of metadata (i.e. no - // work in progress) so we will end up doing the work in the new BM + // memory that tracks which prunes are active. Once done, we can mark that work as done. If the + // BM fails then all prunes will fail and so the next BM will have a clear set of metadata (i.e. + // no work in progress) so we will end up doing the work in the new BM wait(waitForAll(prunes)); if (!pruneIntents.more) { @@ -1851,7 +1973,9 @@ ACTOR Future monitorPruneKeys(BlobManagerData* self) { wait(tr->onError(e)); } } - printf("done pruning all ranges. looping back\n"); + if (BM_DEBUG) { + printf("Done pruning current set of prune intents.\n"); + } } } catch (Error& e) { if (BM_DEBUG) { diff --git a/fdbserver/BlobWorker.actor.cpp b/fdbserver/BlobWorker.actor.cpp index e04711996d..a41d1dc688 100644 --- a/fdbserver/BlobWorker.actor.cpp +++ b/fdbserver/BlobWorker.actor.cpp @@ -45,7 +45,7 @@ #include "flow/actorcompiler.h" // has to be last include #define BW_DEBUG true -#define BW_REQUEST_DEBUG false +#define BW_REQUEST_DEBUG true // represents a previous version of a granule, and optionally the files that compose it struct GranuleHistoryEntry : NonCopyable, ReferenceCounted { @@ -250,7 +250,6 @@ ACTOR Future readAndCheckGranuleLock(Reference return Void(); } -<<<<<<< HEAD // used for "business logic" of both versions of loading granule files ACTOR Future readGranuleFiles(Transaction* tr, Key* startKey, Key endKey, GranuleFiles* files, UID granuleID) { @@ -475,9 +474,8 @@ ACTOR Future writeDeltaFile(Reference bwData, wait(delay(0, TaskPriority::BlobWorkerUpdateStorage)); // TODO some sort of directory structure would be useful? - state std::string fname = deterministicRandom()->randomUniqueID().toString() + "_T" + - std::to_string((uint64_t)(1000.0 * now())) + "_V" + std::to_string(currentDeltaVersion) + - ".delta"; + state std::string fname = granuleID.toString() + "_T" + std::to_string((uint64_t)(1000.0 * now())) + "_V" + + std::to_string(currentDeltaVersion) + ".delta"; state Value serialized = ObjectWriter::toValue(deltasToWrite, Unversioned()); @@ -576,8 +574,8 @@ ACTOR Future writeSnapshot(Reference bwData, PromiseStream rows, bool createGranuleHistory) { // TODO some sort of directory structure would be useful maybe? - state std::string fname = deterministicRandom()->randomUniqueID().toString() + "_T" + - std::to_string((uint64_t)(1000.0 * now())) + "_V" + std::to_string(version) + ".snapshot"; + state std::string fname = granuleID.toString() + "_T" + std::to_string((uint64_t)(1000.0 * now())) + "_V" + + std::to_string(version) + ".snapshot"; state Arena arena; state GranuleSnapshot snapshot; @@ -1243,6 +1241,7 @@ ACTOR Future blobGranuleUpdateFiles(Reference bwData, ASSERT(metadata->readable.canBeSet()); metadata->readable.send(Void()); + printf("Got change feed stream\n"); loop { // check outstanding snapshot/delta files for completion @@ -1354,6 +1353,8 @@ ACTOR Future blobGranuleUpdateFiles(Reference bwData, metadata->keyRange); } + printf("About to process mutations\n"); + // process mutations if (!mutations.empty()) { bool processedAnyMutations = false; @@ -1952,6 +1953,11 @@ ACTOR Future waitForVersion(Reference metadata, Version v } ACTOR Future handleBlobGranuleFileRequest(Reference bwData, BlobGranuleFileRequest req) { + printf("BW %s got blobGranuleFileRequest for range [%s-%s) @ %lld\n", + bwData->id.toString().c_str(), + req.keyRange.begin.printable().c_str(), + req.keyRange.end.printable().c_str(), + req.readVersion); try { // TODO REMOVE in api V2 ASSERT(req.beginVersion == 0); @@ -2011,10 +2017,13 @@ ACTOR Future handleBlobGranuleFileRequest(Reference bwData throw wrong_shard_server(); } + printf("is readable\n"); + state KeyRange chunkRange; state GranuleFiles chunkFiles; if (metadata->initialSnapshotVersion > req.readVersion) { + printf("time travel query\n"); // this is a time travel query, find previous granule if (metadata->historyLoaded.canBeSet()) { choose { @@ -2092,6 +2101,7 @@ ACTOR Future handleBlobGranuleFileRequest(Reference bwData ASSERT(chunkFiles.deltaFiles.back().version > req.readVersion); ASSERT(chunkFiles.snapshotFiles.front().version <= req.readVersion); } else { + // this is an active granule query loop { if (!metadata->activeCFData.get().isValid()) { @@ -2739,6 +2749,107 @@ ACTOR Future monitorRemoval(Reference bwData) { } } +ACTOR Future monitorPruneKeys(Reference self) { + try { + state Value oldPruneWatchVal; + loop { + state Reference tr = makeReference(self->db); + tr->setOption(FDBTransactionOptions::ACCESS_SYSTEM_KEYS); + tr->setOption(FDBTransactionOptions::PRIORITY_SYSTEM_IMMEDIATE); + + // Wait for the watch to change, or some time to expire (whichever comes first) + // before checking through the prune intents. We write a UID into the change key value + // so that we can still recognize when the watch key has been changed while we weren't + // monitoring it + loop { + try { + tr->setOption(FDBTransactionOptions::ACCESS_SYSTEM_KEYS); + tr->setOption(FDBTransactionOptions::PRIORITY_SYSTEM_IMMEDIATE); + + Optional newPruneWatchVal = wait(tr->get(blobGranulePruneChangeKey)); + + // if the value at the change key has changed, that means there is new work to do + if (newPruneWatchVal.present() && oldPruneWatchVal != newPruneWatchVal.get()) { + oldPruneWatchVal = newPruneWatchVal.get(); + printf("old and new watch don't match\n"); + break; + } + + // otherwise, there are no changes and we should wait until the next change (or timeout) + state Future watchPruneIntentsChange = tr->watch(blobGranulePruneChangeKey); + wait(tr->commit()); + printf("About to wait for change or timeout\n"); + choose { + when(wait(watchPruneIntentsChange)) { tr->reset(); } + when(wait(delay(SERVER_KNOBS->BG_PRUNE_TIMEOUT))) { + printf("bg prune timeouts\n"); + break; + } + } + // wait(timeout(watchPruneIntentsChange, SERVER_KNOBS->BG_PRUNE_TIMEOUT, Void())); + } catch (Error& e) { + wait(tr->onError(e)); + } + } + + tr->reset(); + + // loop through all prune intentions and do prune work accordingly + state KeyRef beginKey = normalKeys.begin; + loop { + tr->setOption(FDBTransactionOptions::ACCESS_SYSTEM_KEYS); + tr->setOption(FDBTransactionOptions::PRIORITY_SYSTEM_IMMEDIATE); + + state std::vector> prunes; + try { + // TODO: replace 10000 with a knob + KeyRange nextRange(KeyRangeRef(beginKey, normalKeys.end)); + state RangeResult pruneIntents = wait(krmGetRanges( + tr, blobGranulePruneKeys.begin, nextRange, 10000, GetRangeLimits::BYTE_LIMIT_UNLIMITED)); + + // TODO: would we miss a range [pruneIntents[9999], pruneIntents[10000]) because of the `more`? + // Or does `readThrough` take care of this? We also do this in recoverBlobManager + printf("pruneIntents.size()==%d\n", pruneIntents.size()); + for (int rangeIdx = 0; rangeIdx < pruneIntents.size() - 1; ++rangeIdx) { + if (pruneIntents[rangeIdx].value.size() == 0) { + continue; + } + KeyRef rangeStartKey = pruneIntents[rangeIdx].key; + KeyRef rangeEndKey = pruneIntents[rangeIdx + 1].key; + KeyRange range(KeyRangeRef(rangeStartKey, rangeEndKey)); + Version pruneVersion; + bool force; + std::tie(pruneVersion, force) = decodeBlobGranulePruneValue(pruneIntents[rangeIdx].value); + + printf("about to prune range [%s-%s) @ %d, force=%s\n", + rangeStartKey.printable().c_str(), + rangeEndKey.printable().c_str(), + pruneVersion, + force ? "T" : "F"); + + // TODO: clear associated history + } + + if (!pruneIntents.more) { + break; + } + + beginKey = pruneIntents.readThrough.get(); + } catch (Error& e) { + // TODO: other errors here from pruneRange? + wait(tr->onError(e)); + } + } + printf("done pruning all ranges. looping back\n"); + } + } catch (Error& e) { + if (BW_DEBUG) { + printf("monitorPruneKeys got error %s\n", e.name()); + } + throw e; + } +} + ACTOR Future blobWorker(BlobWorkerInterface bwInterf, ReplyPromise recruitReply, Reference const> dbInfo) { @@ -2784,6 +2895,7 @@ ACTOR Future blobWorker(BlobWorkerInterface bwInterf, recruitReply.send(rep); self->addActor.send(waitFailureServer(bwInterf.waitFailure.getFuture())); + self->addActor.send(monitorPruneKeys(self)); state Future selfRemoved = monitorRemoval(self); TraceEvent("BlobWorkerInit", self->id); diff --git a/fdbserver/MutationTracking.cpp b/fdbserver/MutationTracking.cpp index fc698077d7..3393281cd8 100644 --- a/fdbserver/MutationTracking.cpp +++ b/fdbserver/MutationTracking.cpp @@ -33,8 +33,14 @@ // keys in debugKeys and the ranges in debugRanges. // Each entry is a pair of (label, keyOrRange) and the Label will be attached to the // MutationTracking TraceEvent for easier searching/recognition. -std::vector> debugKeys = { { "SomeKey", "foo"_sr } }; -std::vector> debugRanges = { { "Everything", { ""_sr, "\xff\xff\xff\xff"_sr } } }; +std::vector> debugKeys = {}; +std::vector> debugRanges = { { "PruneKeys", + { "\xff" + "\x02" + "/bgp/"_sr, + "\xff" + "\x02" + "/bgp0"_sr } } }; TraceEvent debugMutationEnabled(const char* context, Version version, MutationRef const& mutation, UID id) { const char* label = nullptr; diff --git a/fdbserver/MutationTracking.h b/fdbserver/MutationTracking.h index 44be11145b..76acefd4ff 100644 --- a/fdbserver/MutationTracking.h +++ b/fdbserver/MutationTracking.h @@ -25,7 +25,7 @@ #include "fdbclient/FDBTypes.h" #include "fdbclient/CommitTransaction.h" -#define MUTATION_TRACKING_ENABLED 0 +#define MUTATION_TRACKING_ENABLED 1 // The keys to track are defined in the .cpp file to limit recompilation. #define DEBUG_MUTATION(...) MUTATION_TRACKING_ENABLED&& debugMutation(__VA_ARGS__) diff --git a/fdbserver/workloads/BlobGranuleVerifier.actor.cpp b/fdbserver/workloads/BlobGranuleVerifier.actor.cpp index f5f4f5cda2..9c97edbcce 100644 --- a/fdbserver/workloads/BlobGranuleVerifier.actor.cpp +++ b/fdbserver/workloads/BlobGranuleVerifier.actor.cpp @@ -27,6 +27,7 @@ #include "fdbclient/ManagementAPI.actor.h" #include "fdbclient/NativeAPI.actor.h" #include "fdbclient/ReadYourWrites.h" +#include "fdbclient/SystemData.h" #include "fdbserver/Knobs.h" #include "fdbserver/TesterInterface.actor.h" #include "fdbserver/workloads/workloads.actor.h" @@ -59,6 +60,7 @@ struct BlobGranuleVerifierWorkload : TestWorkload { int64_t timeTravelTooOld = 0; int64_t rowsRead = 0; int64_t bytesRead = 0; + KeyRangeMap latestPruneVersions; std::vector> clients; DatabaseConfiguration config; @@ -305,11 +307,53 @@ struct BlobGranuleVerifierWorkload : TestWorkload { OldRead(KeyRange range, Version v, RangeResult oldResult) : range(range), v(v), oldResult(oldResult) {} }; + ACTOR Future pruneAtVersion(Database cx, KeyRange range, Version version, bool force) { + state Reference tr = makeReference(cx); + loop { + try { + tr->setOption(FDBTransactionOptions::ACCESS_SYSTEM_KEYS); + tr->setOption(FDBTransactionOptions::PRIORITY_SYSTEM_IMMEDIATE); + Optional oldPruneIntent = wait(tr->get(blobGranulePruneKeys.begin.withSuffix(range.begin))); + + if (oldPruneIntent.present()) { + Version oldPruneVersion; + bool oldForce; + std::tie(oldPruneVersion, oldForce) = decodeBlobGranulePruneValue(oldPruneIntent.get()); + if (oldPruneVersion >= version) { + return Void(); + } + } + + Value pruneValue = blobGranulePruneValueFor(version, force); + wait(krmSetRange(tr, blobGranulePruneKeys.begin, range, pruneValue)); + tr->set(blobGranulePruneChangeKey, deterministicRandom()->randomUniqueID().toString()); + wait(tr->commit()); + if (BGV_DEBUG) { + printf("pruneAtVersion for range [%s-%s) at version %lld succeeded\n", + range.begin.printable().c_str(), + range.end.printable().c_str(), + version); + } + return Void(); + } catch (Error& e) { + if (BGV_DEBUG) { + printf("pruneAtVersion for range [%s-%s) at version %lld encountered error %s\n", + range.begin.printable().c_str(), + range.end.printable().c_str(), + version, + e.name()); + } + wait(tr->onError(e)); + } + } + } + ACTOR Future verifyGranules(Database cx, BlobGranuleVerifierWorkload* self) { state double last = now(); state double endTime = last + self->testDuration; state std::map timeTravelChecks; state int64_t timeTravelChecksMemory = 0; + state KeyRangeMap latestPruneVersions; TraceEvent("BlobGranuleVerifierStart"); if (BGV_DEBUG) { @@ -334,10 +378,18 @@ struct BlobGranuleVerifierWorkload : TestWorkload { // advance iterator before doing read, so if it gets error we don't retry it try { + // before reading, prune at some version [0, readVersion) + Version pruneVersion = deterministicRandom()->randomInt(0, oldRead.v); + wait(self->pruneAtVersion(cx, oldRead.range, pruneVersion, false)); + // FIXME: this doesnt actually guarantee that the prune executed. maybe add a delay? + std::pair>> reReadResult = wait(self->readFromBlob(cx, self, oldRead.range, oldRead.v)); self->compareResult(oldRead.oldResult, reReadResult, oldRead.range, oldRead.v, false); self->timeTravelReads++; + + // TODO: read at some version older than pruneVersion and make sure you get txn_too_old + // To achieve this, the BWs are going to have to recognize latest prune versions per granules } catch (Error& e) { if (e.code() == error_code_transaction_too_old) { self->timeTravelTooOld++; From 1d32da5ac6bb09f1f96437ff786130c273fde825 Mon Sep 17 00:00:00 2001 From: Suraj Gupta Date: Fri, 3 Dec 2021 18:08:01 -0500 Subject: [PATCH 086/413] Remove local debugging. --- fdbclient/BlobGranuleCommon.h | 26 ++++---- fdbserver/BlobGranuleServerCommon.actor.cpp | 2 +- fdbserver/BlobGranuleServerCommon.actor.h | 2 +- fdbserver/BlobManager.actor.cpp | 4 +- fdbserver/BlobWorker.actor.cpp | 74 ++++----------------- fdbserver/MutationTracking.cpp | 10 +-- fdbserver/MutationTracking.h | 2 +- 7 files changed, 33 insertions(+), 87 deletions(-) diff --git a/fdbclient/BlobGranuleCommon.h b/fdbclient/BlobGranuleCommon.h index b0cbd8ef8b..7c44838c18 100644 --- a/fdbclient/BlobGranuleCommon.h +++ b/fdbclient/BlobGranuleCommon.h @@ -24,22 +24,8 @@ #include -#include "flow/flow.h" #include "fdbclient/CommitTransaction.h" #include "fdbclient/FDBTypes.h" -#include "flow/actorcompiler.h" // has to be last include - -// Stores the value representation of a blob granule history entry in FDB -struct BlobGranuleHistoryValue { - constexpr static FileIdentifier file_identifier = 991434; - UID granuleID; - VectorRef> parentGranules; - - template - void serialize(Ar& ar) { - serializer(ar, granuleID, parentGranules); - } -}; // file format of actual blob files struct GranuleSnapshot : VectorRef { @@ -102,4 +88,16 @@ struct BlobGranuleChunkRef { }; enum BlobGranuleSplitState { Unknown = 0, Initialized = 1, Assigned = 2, Done = 3 }; + +struct BlobGranuleHistoryValue { + constexpr static FileIdentifier file_identifier = 991434; + UID granuleID; + VectorRef> parentGranules; + + template + void serialize(Ar& ar) { + serializer(ar, granuleID, parentGranules); + } +}; + #endif diff --git a/fdbserver/BlobGranuleServerCommon.actor.cpp b/fdbserver/BlobGranuleServerCommon.actor.cpp index d621f9dd07..a971c8d268 100644 --- a/fdbserver/BlobGranuleServerCommon.actor.cpp +++ b/fdbserver/BlobGranuleServerCommon.actor.cpp @@ -1,5 +1,5 @@ /* - * BlobGranuleCommon.actor.cpp + * BlobGranuleServerCommon.actor.cpp * * This source file is part of the FoundationDB open source project * diff --git a/fdbserver/BlobGranuleServerCommon.actor.h b/fdbserver/BlobGranuleServerCommon.actor.h index 788377b8fc..d9eb372999 100644 --- a/fdbserver/BlobGranuleServerCommon.actor.h +++ b/fdbserver/BlobGranuleServerCommon.actor.h @@ -1,5 +1,5 @@ /* - * BlobGranuleCommon.h + * BlobGranuleServerCommon.h * * This source file is part of the FoundationDB open source project * diff --git a/fdbserver/BlobManager.actor.cpp b/fdbserver/BlobManager.actor.cpp index 48302c25c6..4c1fe94360 100644 --- a/fdbserver/BlobManager.actor.cpp +++ b/fdbserver/BlobManager.actor.cpp @@ -1636,7 +1636,7 @@ ACTOR Future pruneRange(BlobManagerData* self, KeyRef startKey, KeyRef end if (BM_DEBUG) { printf("Fetching latest history entry for range [%s-%s)\n", activeRange.begin().printable().c_str(), - activeRange.end().printable().c_str()) + activeRange.end().printable().c_str()); } Optional history = wait(getLatestGranuleHistory(&tr, activeRange.range())); // TODO: can we tell from the krm that this range is not valid, so that we don't need to do a get @@ -1692,6 +1692,8 @@ ACTOR Future pruneRange(BlobManagerData* self, KeyRef startKey, KeyRef end } // if we already saw this node, skip it; otherwise, mark it as visited + // TODO: doing the visited check here as opposed to when adding this node as a parent + // causes us to do another GET, but the parentGranules field does not contain parent gids if (visited.count(currHistoryNode.granuleID)) { if (BM_DEBUG) { printf("Already processed %s, so skipping it\n", currHistoryNode.granuleID.toString().c_str()); diff --git a/fdbserver/BlobWorker.actor.cpp b/fdbserver/BlobWorker.actor.cpp index a41d1dc688..fbd03bba86 100644 --- a/fdbserver/BlobWorker.actor.cpp +++ b/fdbserver/BlobWorker.actor.cpp @@ -250,51 +250,6 @@ ACTOR Future readAndCheckGranuleLock(Reference return Void(); } -// used for "business logic" of both versions of loading granule files -ACTOR Future readGranuleFiles(Transaction* tr, Key* startKey, Key endKey, GranuleFiles* files, UID granuleID) { - - loop { - int lim = BUGGIFY ? 2 : 1000; - RangeResult res = wait(tr->getRange(KeyRangeRef(*startKey, endKey), lim)); - for (auto& it : res) { - UID gid; - uint8_t fileType; - Version version; - - Standalone filename; - int64_t offset; - int64_t length; - - std::tie(gid, fileType, version) = decodeBlobGranuleFileKey(it.key); - ASSERT(gid == granuleID); - - std::tie(filename, offset, length) = decodeBlobGranuleFileValue(it.value); - - BlobFileIndex idx(version, filename.toString(), offset, length); - if (fileType == 'S') { - ASSERT(files->snapshotFiles.empty() || files->snapshotFiles.back().version < idx.version); - files->snapshotFiles.push_back(idx); - } else { - ASSERT(fileType == 'D'); - ASSERT(files->deltaFiles.empty() || files->deltaFiles.back().version < idx.version); - files->deltaFiles.push_back(idx); - } - } - if (res.more) { - *startKey = keyAfter(res.back().key); - } else { - break; - } - } - if (BW_DEBUG) { - fmt::print("Loaded {0} snapshot and {1} delta files for {2}\n", - files->snapshotFiles.size(), - files->deltaFiles.size(), - granuleID.toString()); - } - return Void(); -} - // Read snapshot and delta files for granule history, for completed granule // Retries on error local to this function ACTOR Future loadHistoryFiles(Reference bwData, UID granuleID) { @@ -304,7 +259,7 @@ ACTOR Future loadHistoryFiles(Reference bwData, UI state GranuleFiles files; loop { try { - wait(readGranuleFiles(&tr, &startKey, range.end, &files, granuleID)); + wait(readGranuleFiles(&tr, &startKey, range.end, &files, granuleID, BW_DEBUG)); return files; } catch (Error& e) { wait(tr.onError(e)); @@ -473,7 +428,8 @@ ACTOR Future writeDeltaFile(Reference bwData, Optional> oldGranuleComplete) { wait(delay(0, TaskPriority::BlobWorkerUpdateStorage)); - // TODO some sort of directory structure would be useful? + // TODO: this will induce S3 hotspotting, so we should rethink if we want to prefix the file name + // with the granuleID or just add it somewhere in the file name state std::string fname = granuleID.toString() + "_T" + std::to_string((uint64_t)(1000.0 * now())) + "_V" + std::to_string(currentDeltaVersion) + ".delta"; @@ -573,7 +529,8 @@ ACTOR Future writeSnapshot(Reference bwData, Version version, PromiseStream rows, bool createGranuleHistory) { - // TODO some sort of directory structure would be useful maybe? + // TODO: this will induce S3 hotspotting, so we should rethink if we want to prefix the file name + // with the granuleID or just add it somewhere in the file name state std::string fname = granuleID.toString() + "_T" + std::to_string((uint64_t)(1000.0 * now())) + "_V" + std::to_string(version) + ".snapshot"; state Arena arena; @@ -1203,7 +1160,6 @@ ACTOR Future blobGranuleUpdateFiles(Reference bwData, metadata->durableSnapshotVersion.set(startState.blobFilesToSnapshot.get().snapshotFiles.back().version); } else { ASSERT(startState.previousDurableVersion == invalidVersion); - printf("About to dump initial snapshot\n"); BlobFileIndex fromFDB = wait(dumpInitialSnapshotFromFDB(bwData, metadata, startState.granuleID)); newSnapshotFile = fromFDB; ASSERT(startState.changeFeedStartVersion <= fromFDB.version); @@ -1241,7 +1197,6 @@ ACTOR Future blobGranuleUpdateFiles(Reference bwData, ASSERT(metadata->readable.canBeSet()); metadata->readable.send(Void()); - printf("Got change feed stream\n"); loop { // check outstanding snapshot/delta files for completion @@ -1353,8 +1308,6 @@ ACTOR Future blobGranuleUpdateFiles(Reference bwData, metadata->keyRange); } - printf("About to process mutations\n"); - // process mutations if (!mutations.empty()) { bool processedAnyMutations = false; @@ -1953,11 +1906,14 @@ ACTOR Future waitForVersion(Reference metadata, Version v } ACTOR Future handleBlobGranuleFileRequest(Reference bwData, BlobGranuleFileRequest req) { - printf("BW %s got blobGranuleFileRequest for range [%s-%s) @ %lld\n", - bwData->id.toString().c_str(), - req.keyRange.begin.printable().c_str(), - req.keyRange.end.printable().c_str(), - req.readVersion); + if (BW_DEBUG) { + printf("BW %s processing blobGranuleFileRequest for range [%s-%s) @ %lld\n", + bwData->id.toString().c_str(), + req.keyRange.begin.printable().c_str(), + req.keyRange.end.printable().c_str(), + req.readVersion); + } + try { // TODO REMOVE in api V2 ASSERT(req.beginVersion == 0); @@ -2017,13 +1973,10 @@ ACTOR Future handleBlobGranuleFileRequest(Reference bwData throw wrong_shard_server(); } - printf("is readable\n"); - state KeyRange chunkRange; state GranuleFiles chunkFiles; if (metadata->initialSnapshotVersion > req.readVersion) { - printf("time travel query\n"); // this is a time travel query, find previous granule if (metadata->historyLoaded.canBeSet()) { choose { @@ -2101,7 +2054,6 @@ ACTOR Future handleBlobGranuleFileRequest(Reference bwData ASSERT(chunkFiles.deltaFiles.back().version > req.readVersion); ASSERT(chunkFiles.snapshotFiles.front().version <= req.readVersion); } else { - // this is an active granule query loop { if (!metadata->activeCFData.get().isValid()) { diff --git a/fdbserver/MutationTracking.cpp b/fdbserver/MutationTracking.cpp index 3393281cd8..fc698077d7 100644 --- a/fdbserver/MutationTracking.cpp +++ b/fdbserver/MutationTracking.cpp @@ -33,14 +33,8 @@ // keys in debugKeys and the ranges in debugRanges. // Each entry is a pair of (label, keyOrRange) and the Label will be attached to the // MutationTracking TraceEvent for easier searching/recognition. -std::vector> debugKeys = {}; -std::vector> debugRanges = { { "PruneKeys", - { "\xff" - "\x02" - "/bgp/"_sr, - "\xff" - "\x02" - "/bgp0"_sr } } }; +std::vector> debugKeys = { { "SomeKey", "foo"_sr } }; +std::vector> debugRanges = { { "Everything", { ""_sr, "\xff\xff\xff\xff"_sr } } }; TraceEvent debugMutationEnabled(const char* context, Version version, MutationRef const& mutation, UID id) { const char* label = nullptr; diff --git a/fdbserver/MutationTracking.h b/fdbserver/MutationTracking.h index 76acefd4ff..44be11145b 100644 --- a/fdbserver/MutationTracking.h +++ b/fdbserver/MutationTracking.h @@ -25,7 +25,7 @@ #include "fdbclient/FDBTypes.h" #include "fdbclient/CommitTransaction.h" -#define MUTATION_TRACKING_ENABLED 1 +#define MUTATION_TRACKING_ENABLED 0 // The keys to track are defined in the .cpp file to limit recompilation. #define DEBUG_MUTATION(...) MUTATION_TRACKING_ENABLED&& debugMutation(__VA_ARGS__) From e9d1f36caef8d60dd00b970f2544961b51a2b592 Mon Sep 17 00:00:00 2001 From: Suraj Gupta Date: Fri, 3 Dec 2021 18:22:10 -0500 Subject: [PATCH 087/413] More cleanup. --- fdbclient/NativeAPI.actor.cpp | 2 +- fdbserver/BlobManager.actor.cpp | 1 - fdbserver/BlobWorker.actor.cpp | 140 +++++--------------------------- 3 files changed, 20 insertions(+), 123 deletions(-) diff --git a/fdbclient/NativeAPI.actor.cpp b/fdbclient/NativeAPI.actor.cpp index acf2b8af5d..8723f8fc29 100644 --- a/fdbclient/NativeAPI.actor.cpp +++ b/fdbclient/NativeAPI.actor.cpp @@ -7831,7 +7831,7 @@ Future DatabaseContext::popChangeFeedMutations(Key rangeID, Version versio return popChangeFeedMutationsActor(Reference::addRef(this), rangeID, version); } -#define BG_REQUEST_DEBUG true +#define BG_REQUEST_DEBUG false ACTOR Future getBlobGranuleRangesStreamActor(Reference db, PromiseStream results, diff --git a/fdbserver/BlobManager.actor.cpp b/fdbserver/BlobManager.actor.cpp index 4c1fe94360..6e0d3135e9 100644 --- a/fdbserver/BlobManager.actor.cpp +++ b/fdbserver/BlobManager.actor.cpp @@ -215,7 +215,6 @@ struct BlobManagerData { std::unordered_set deadWorkers; KeyRangeMap workerAssignments; KeyRangeMap knownBlobRanges; - KeyRangeMap prunesInProgress; AsyncTrigger startRecruiting; Debouncer restartRecruiting; diff --git a/fdbserver/BlobWorker.actor.cpp b/fdbserver/BlobWorker.actor.cpp index fbd03bba86..61a13ebdeb 100644 --- a/fdbserver/BlobWorker.actor.cpp +++ b/fdbserver/BlobWorker.actor.cpp @@ -47,25 +47,6 @@ #define BW_DEBUG true #define BW_REQUEST_DEBUG true -// represents a previous version of a granule, and optionally the files that compose it -struct GranuleHistoryEntry : NonCopyable, ReferenceCounted { - KeyRange range; - UID granuleID; - Version startVersion; // version of the first snapshot - Version endVersion; // version of the last delta file - - // load files lazily, and allows for clearing old cold-queried files to save memory - Future files; - - // FIXME: do skip pointers with single back-pointer and neighbor pointers - // Just parent reference for now (assumes no merging) - Reference parentGranule; - - GranuleHistoryEntry() : startVersion(invalidVersion), endVersion(invalidVersion) {} - GranuleHistoryEntry(KeyRange range, UID granuleID, Version startVersion, Version endVersion) - : range(range), granuleID(granuleID), startVersion(startVersion), endVersion(endVersion) {} -}; - struct GranuleStartState { UID granuleID; Version changeFeedStartVersion; @@ -147,6 +128,25 @@ struct GranuleRangeMetadata { // ~GranuleRangeMetadata() { printf("Destroying granule metadata\n"); } }; +// represents a previous version of a granule, and optionally the files that compose it +struct GranuleHistoryEntry : NonCopyable, ReferenceCounted { + KeyRange range; + UID granuleID; + Version startVersion; // version of the first snapshot + Version endVersion; // version of the last delta file + + // load files lazily, and allows for clearing old cold-queried files to save memory + Future files; + + // FIXME: do skip pointers with single back-pointer and neighbor pointers + // Just parent reference for now (assumes no merging) + Reference parentGranule; + + GranuleHistoryEntry() : startVersion(invalidVersion), endVersion(invalidVersion) {} + GranuleHistoryEntry(KeyRange range, UID granuleID, Version startVersion, Version endVersion) + : range(range), granuleID(granuleID), startVersion(startVersion), endVersion(endVersion) {} +}; + struct BlobWorkerData : NonCopyable, ReferenceCounted { UID id; Database db; @@ -2701,107 +2701,6 @@ ACTOR Future monitorRemoval(Reference bwData) { } } -ACTOR Future monitorPruneKeys(Reference self) { - try { - state Value oldPruneWatchVal; - loop { - state Reference tr = makeReference(self->db); - tr->setOption(FDBTransactionOptions::ACCESS_SYSTEM_KEYS); - tr->setOption(FDBTransactionOptions::PRIORITY_SYSTEM_IMMEDIATE); - - // Wait for the watch to change, or some time to expire (whichever comes first) - // before checking through the prune intents. We write a UID into the change key value - // so that we can still recognize when the watch key has been changed while we weren't - // monitoring it - loop { - try { - tr->setOption(FDBTransactionOptions::ACCESS_SYSTEM_KEYS); - tr->setOption(FDBTransactionOptions::PRIORITY_SYSTEM_IMMEDIATE); - - Optional newPruneWatchVal = wait(tr->get(blobGranulePruneChangeKey)); - - // if the value at the change key has changed, that means there is new work to do - if (newPruneWatchVal.present() && oldPruneWatchVal != newPruneWatchVal.get()) { - oldPruneWatchVal = newPruneWatchVal.get(); - printf("old and new watch don't match\n"); - break; - } - - // otherwise, there are no changes and we should wait until the next change (or timeout) - state Future watchPruneIntentsChange = tr->watch(blobGranulePruneChangeKey); - wait(tr->commit()); - printf("About to wait for change or timeout\n"); - choose { - when(wait(watchPruneIntentsChange)) { tr->reset(); } - when(wait(delay(SERVER_KNOBS->BG_PRUNE_TIMEOUT))) { - printf("bg prune timeouts\n"); - break; - } - } - // wait(timeout(watchPruneIntentsChange, SERVER_KNOBS->BG_PRUNE_TIMEOUT, Void())); - } catch (Error& e) { - wait(tr->onError(e)); - } - } - - tr->reset(); - - // loop through all prune intentions and do prune work accordingly - state KeyRef beginKey = normalKeys.begin; - loop { - tr->setOption(FDBTransactionOptions::ACCESS_SYSTEM_KEYS); - tr->setOption(FDBTransactionOptions::PRIORITY_SYSTEM_IMMEDIATE); - - state std::vector> prunes; - try { - // TODO: replace 10000 with a knob - KeyRange nextRange(KeyRangeRef(beginKey, normalKeys.end)); - state RangeResult pruneIntents = wait(krmGetRanges( - tr, blobGranulePruneKeys.begin, nextRange, 10000, GetRangeLimits::BYTE_LIMIT_UNLIMITED)); - - // TODO: would we miss a range [pruneIntents[9999], pruneIntents[10000]) because of the `more`? - // Or does `readThrough` take care of this? We also do this in recoverBlobManager - printf("pruneIntents.size()==%d\n", pruneIntents.size()); - for (int rangeIdx = 0; rangeIdx < pruneIntents.size() - 1; ++rangeIdx) { - if (pruneIntents[rangeIdx].value.size() == 0) { - continue; - } - KeyRef rangeStartKey = pruneIntents[rangeIdx].key; - KeyRef rangeEndKey = pruneIntents[rangeIdx + 1].key; - KeyRange range(KeyRangeRef(rangeStartKey, rangeEndKey)); - Version pruneVersion; - bool force; - std::tie(pruneVersion, force) = decodeBlobGranulePruneValue(pruneIntents[rangeIdx].value); - - printf("about to prune range [%s-%s) @ %d, force=%s\n", - rangeStartKey.printable().c_str(), - rangeEndKey.printable().c_str(), - pruneVersion, - force ? "T" : "F"); - - // TODO: clear associated history - } - - if (!pruneIntents.more) { - break; - } - - beginKey = pruneIntents.readThrough.get(); - } catch (Error& e) { - // TODO: other errors here from pruneRange? - wait(tr->onError(e)); - } - } - printf("done pruning all ranges. looping back\n"); - } - } catch (Error& e) { - if (BW_DEBUG) { - printf("monitorPruneKeys got error %s\n", e.name()); - } - throw e; - } -} - ACTOR Future blobWorker(BlobWorkerInterface bwInterf, ReplyPromise recruitReply, Reference const> dbInfo) { @@ -2847,7 +2746,6 @@ ACTOR Future blobWorker(BlobWorkerInterface bwInterf, recruitReply.send(rep); self->addActor.send(waitFailureServer(bwInterf.waitFailure.getFuture())); - self->addActor.send(monitorPruneKeys(self)); state Future selfRemoved = monitorRemoval(self); TraceEvent("BlobWorkerInit", self->id); From dab13bd61418049e58c6bb889d29bed46873de2a Mon Sep 17 00:00:00 2001 From: Suraj Gupta Date: Mon, 6 Dec 2021 11:00:15 -0500 Subject: [PATCH 088/413] Fix range iteration for krms in BM. --- fdbserver/BlobManager.actor.cpp | 17 +++++++++-------- 1 file changed, 9 insertions(+), 8 deletions(-) diff --git a/fdbserver/BlobManager.actor.cpp b/fdbserver/BlobManager.actor.cpp index 6e0d3135e9..6cb4648812 100644 --- a/fdbserver/BlobManager.actor.cpp +++ b/fdbserver/BlobManager.actor.cpp @@ -1074,11 +1074,13 @@ ACTOR Future recoverBlobManager(BlobManagerData* bmData) { KeyRange nextRange(KeyRangeRef(beginKey, normalKeys.end)); RangeResult results = wait( krmGetRanges(tr, blobGranuleMappingKeys.begin, nextRange, 10000, GetRangeLimits::BYTE_LIMIT_UNLIMITED)); + Key lastEndKey; // Add the mappings to our in memory key range map for (int rangeIdx = 0; rangeIdx < results.size() - 1; rangeIdx++) { Key granuleStartKey = results[rangeIdx].key; Key granuleEndKey = results[rangeIdx + 1].key; + lastEndKey = granuleEndKey; if (results[rangeIdx].value.size()) { // note: if the old owner is dead, we handle this in rangeAssigner UID existingOwner = decodeBlobGranuleMappingValue(results[rangeIdx].value); @@ -1090,7 +1092,7 @@ ACTOR Future recoverBlobManager(BlobManagerData* bmData) { break; } - beginKey = results.readThrough.get(); + beginKey = lastEndKey; } catch (Error& e) { wait(tr->onError(e)); } @@ -1929,18 +1931,17 @@ ACTOR Future monitorPruneKeys(BlobManagerData* self) { try { // TODO: replace 10000 with a knob KeyRange nextRange(KeyRangeRef(beginKey, normalKeys.end)); - state RangeResult pruneIntents = wait(krmGetRanges( + RangeResult pruneIntents = wait(krmGetRanges( tr, blobGranulePruneKeys.begin, nextRange, 10000, GetRangeLimits::BYTE_LIMIT_UNLIMITED)); + state Key lastEndKey; - // TODO: would we miss a range [pruneIntents[9999], pruneIntents[10000]) because of the `more`? - // Or does `readThrough` take care of this? We also do this in recoverBlobManager - printf("pruneIntents.size()==%d\n", pruneIntents.size()); for (int rangeIdx = 0; rangeIdx < pruneIntents.size() - 1; ++rangeIdx) { + KeyRef rangeStartKey = pruneIntents[rangeIdx].key; + KeyRef rangeEndKey = pruneIntents[rangeIdx + 1].key; + lastEndKey = rangeEndKey; if (pruneIntents[rangeIdx].value.size() == 0) { continue; } - KeyRef rangeStartKey = pruneIntents[rangeIdx].key; - KeyRef rangeEndKey = pruneIntents[rangeIdx + 1].key; KeyRange range(KeyRangeRef(rangeStartKey, rangeEndKey)); Version pruneVersion; bool force; @@ -1968,7 +1969,7 @@ ACTOR Future monitorPruneKeys(BlobManagerData* self) { break; } - beginKey = pruneIntents.readThrough.get(); + beginKey = lastEndKey; } catch (Error& e) { // TODO: other errors here from pruneRange? wait(tr->onError(e)); From 6ff7ad3c6a9dc5c6d502126bce90604a3756eb08 Mon Sep 17 00:00:00 2001 From: Suraj Gupta Date: Mon, 6 Dec 2021 12:09:38 -0500 Subject: [PATCH 089/413] Clarify error handling. --- fdbserver/BlobManager.actor.cpp | 107 ++++++++++++++++++-------------- 1 file changed, 61 insertions(+), 46 deletions(-) diff --git a/fdbserver/BlobManager.actor.cpp b/fdbserver/BlobManager.actor.cpp index 6cb4648812..c152e23990 100644 --- a/fdbserver/BlobManager.actor.cpp +++ b/fdbserver/BlobManager.actor.cpp @@ -36,6 +36,7 @@ #include "fdbserver/QuietDatabase.h" #include "fdbserver/WaitFailure.h" #include "fdbserver/WorkerInterface.actor.h" +#include "flow/Error.h" #include "flow/IRandom.h" #include "flow/UnitTest.h" #include "flow/actorcompiler.h" // has to be last include @@ -1746,6 +1747,10 @@ ACTOR Future pruneRange(BlobManagerData* self, KeyRef startKey, KeyRef end // Unfortunately we can't do parallelize _full_ deletions because they might // race and we'll end up with unreachable nodes in the case of a crash. // Since partial deletions only occur for "leafs", they can be done in parallel + // + // Note about file deletions: although we might be retrying a deletion of a granule, + // we won't run into any issues with trying to "re-delete" a blob file since deleting + // a file that doesn't exist is considered successful state int i; if (BM_DEBUG) { @@ -1922,58 +1927,68 @@ ACTOR Future monitorPruneKeys(BlobManagerData* self) { } // loop through all prune intentions and do prune work accordingly - state KeyRef beginKey = normalKeys.begin; - loop { - tr->setOption(FDBTransactionOptions::ACCESS_SYSTEM_KEYS); - tr->setOption(FDBTransactionOptions::PRIORITY_SYSTEM_IMMEDIATE); + try { + state KeyRef beginKey = normalKeys.begin; + loop { + tr->setOption(FDBTransactionOptions::ACCESS_SYSTEM_KEYS); + tr->setOption(FDBTransactionOptions::PRIORITY_SYSTEM_IMMEDIATE); - state std::vector> prunes; - try { - // TODO: replace 10000 with a knob - KeyRange nextRange(KeyRangeRef(beginKey, normalKeys.end)); - RangeResult pruneIntents = wait(krmGetRanges( - tr, blobGranulePruneKeys.begin, nextRange, 10000, GetRangeLimits::BYTE_LIMIT_UNLIMITED)); - state Key lastEndKey; + state std::vector> prunes; + try { + // TODO: replace 10000 with a knob + KeyRange nextRange(KeyRangeRef(beginKey, normalKeys.end)); + state RangeResult pruneIntents = wait(krmGetRanges( + tr, blobGranulePruneKeys.begin, nextRange, 10000, GetRangeLimits::BYTE_LIMIT_UNLIMITED)); + state Key lastEndKey; - for (int rangeIdx = 0; rangeIdx < pruneIntents.size() - 1; ++rangeIdx) { - KeyRef rangeStartKey = pruneIntents[rangeIdx].key; - KeyRef rangeEndKey = pruneIntents[rangeIdx + 1].key; - lastEndKey = rangeEndKey; - if (pruneIntents[rangeIdx].value.size() == 0) { - continue; + for (int rangeIdx = 0; rangeIdx < pruneIntents.size() - 1; ++rangeIdx) { + KeyRef rangeStartKey = pruneIntents[rangeIdx].key; + KeyRef rangeEndKey = pruneIntents[rangeIdx + 1].key; + lastEndKey = rangeEndKey; + if (pruneIntents[rangeIdx].value.size() == 0) { + continue; + } + KeyRange range(KeyRangeRef(rangeStartKey, rangeEndKey)); + Version pruneVersion; + bool force; + std::tie(pruneVersion, force) = decodeBlobGranulePruneValue(pruneIntents[rangeIdx].value); + + printf("about to prune range [%s-%s) @ %d, force=%s\n", + rangeStartKey.printable().c_str(), + rangeEndKey.printable().c_str(), + pruneVersion, + force ? "T" : "F"); + prunes.emplace_back(pruneRange(self, rangeStartKey, rangeEndKey, pruneVersion, force)); } - KeyRange range(KeyRangeRef(rangeStartKey, rangeEndKey)); - Version pruneVersion; - bool force; - std::tie(pruneVersion, force) = decodeBlobGranulePruneValue(pruneIntents[rangeIdx].value); - printf("about to prune range [%s-%s) @ %d, force=%s\n", - rangeStartKey.printable().c_str(), - rangeEndKey.printable().c_str(), - pruneVersion, - force ? "T" : "F"); - prunes.emplace_back(pruneRange(self, rangeStartKey, rangeEndKey, pruneVersion, force)); + // wait for this set of prunes to complete before starting the next ones since if we prune + // a range R at version V and while we are doing that, the time expires, we will end up + // trying to prune the same range again since the work isn't finished and the prunes will race + // + // TODO: this isn't that efficient though. Instead we could keep metadata as part of the BM's + // memory that tracks which prunes are active. Once done, we can mark that work as done. If the + // BM fails then all prunes will fail and so the next BM will have a clear set of metadata (i.e. + // no work in progress) so we will end up doing the work in the new BM + wait(waitForAll(prunes)); + + if (!pruneIntents.more) { + break; + } + + beginKey = lastEndKey; + } catch (Error& e) { + wait(tr->onError(e)); } - - // wait for this set of prunes to complete before starting the next ones since if we prune - // a range R at version V and while we are doing that, the time expires, we will end up - // trying to prune the same range again since the work isn't finished and the prunes will race - // - // TODO: this isn't that efficient though. Instead we could keep metadata as part of the BM's - // memory that tracks which prunes are active. Once done, we can mark that work as done. If the - // BM fails then all prunes will fail and so the next BM will have a clear set of metadata (i.e. - // no work in progress) so we will end up doing the work in the new BM - wait(waitForAll(prunes)); - - if (!pruneIntents.more) { - break; - } - - beginKey = lastEndKey; - } catch (Error& e) { - // TODO: other errors here from pruneRange? - wait(tr->onError(e)); } + } catch (Error& e) { + if (e.code() == error_code_actor_cancelled) { + throw e; + } + if (BM_DEBUG) { + printf("monitorPruneKeys for BM %s saw error %s\n", self->id.toString().c_str(), e.name()); + } + // don't want to kill the blob manager for errors around pruning + TraceEvent("MonitorPruneKeysError", self->id).detail("Error", e.name()); } if (BM_DEBUG) { printf("Done pruning current set of prune intents.\n"); From 90a652ccfb1868873b0e2514ab950b2c7e293832 Mon Sep 17 00:00:00 2001 From: Suraj Gupta Date: Fri, 10 Dec 2021 15:05:21 -0500 Subject: [PATCH 090/413] fix merge conflict --- fdbclient/NativeAPI.actor.cpp | 270 ---------------------------------- 1 file changed, 270 deletions(-) diff --git a/fdbclient/NativeAPI.actor.cpp b/fdbclient/NativeAPI.actor.cpp index 8723f8fc29..73b93a39b1 100644 --- a/fdbclient/NativeAPI.actor.cpp +++ b/fdbclient/NativeAPI.actor.cpp @@ -7831,276 +7831,6 @@ Future DatabaseContext::popChangeFeedMutations(Key rangeID, Version versio return popChangeFeedMutationsActor(Reference::addRef(this), rangeID, version); } -#define BG_REQUEST_DEBUG false - -ACTOR Future getBlobGranuleRangesStreamActor(Reference db, - PromiseStream results, - KeyRange keyRange) { - // FIXME: use streaming range read - state Database cx(db); - state Reference tr = makeReference(cx); - state KeyRange currentRange = keyRange; - if (BG_REQUEST_DEBUG) { - printf("Getting Blob Granules for [%s - %s)\n", - keyRange.begin.printable().c_str(), - keyRange.end.printable().c_str()); - } - loop { - try { - tr->setOption(FDBTransactionOptions::ACCESS_SYSTEM_KEYS); - state RangeResult blobGranuleMapping = wait(krmGetRanges( - tr, blobGranuleMappingKeys.begin, currentRange, 1000, GetRangeLimits::BYTE_LIMIT_UNLIMITED)); - - for (int i = 0; i < blobGranuleMapping.size() - 1; i++) { - if (blobGranuleMapping[i].value.size()) { - results.send(KeyRangeRef(blobGranuleMapping[i].key, blobGranuleMapping[i + 1].key)); - } - } - if (blobGranuleMapping.more) { - currentRange = KeyRangeRef(blobGranuleMapping.back().key, currentRange.end); - } else { - results.sendError(end_of_stream()); - return Void(); - } - } catch (Error& e) { - wait(tr->onError(e)); - } - } -} - -Future DatabaseContext::getBlobGranuleRangesStream(const PromiseStream& results, KeyRange range) { - if (!CLIENT_KNOBS->ENABLE_BLOB_GRANULES) { - throw client_invalid_operation(); - } - return getBlobGranuleRangesStreamActor(Reference::addRef(this), results, range); -} - -// hack (for now) to get blob worker interface into load balance -struct BWLocationInfo : MultiInterface> { - using Locations = MultiInterface>; - explicit BWLocationInfo(const std::vector>>& v) : Locations(v) {} -}; - -ACTOR Future readBlobGranulesStreamActor(Reference db, - PromiseStream> results, - KeyRange range, - Version begin, - Optional end) { // end not present is just latest - state Database cx(db); - state Reference tr = makeReference(cx); - state RangeResult blobGranuleMapping; - state Version endVersion; - state Key granuleStartKey; - state Key granuleEndKey; - state KeyRange keyRange = range; - state int i, loopCounter = 0; - state UID workerId; - loop { - try { - // FIXME: Use streaming parallelism? - // Read mapping and worker interfaces from DB - loopCounter++; - loop { - try { - tr->reset(); - tr->setOption(FDBTransactionOptions::ACCESS_SYSTEM_KEYS); - if (loopCounter == 1) { - // if retrying, use new version for mapping but original version for read version - if (end.present()) { - endVersion = end.get(); - } else { - Version _end = wait(tr->getReadVersion()); - endVersion = _end; - } - } - - // Right now just read whole blob range assignments from DB - // FIXME: eventually we probably want to cache this and invalidate similarly to storage servers. - // Cache misses could still read from the DB, or we could add it to the Transaction State Store and - // have proxies serve it from memory. - RangeResult _bgMapping = wait(krmGetRanges( - tr, blobGranuleMappingKeys.begin, keyRange, 1000, GetRangeLimits::BYTE_LIMIT_UNLIMITED)); - blobGranuleMapping = _bgMapping; - if (blobGranuleMapping.more) { - if (BG_REQUEST_DEBUG) { - // printf("BG Mapping for [%s - %s) too large!\n"); - } - throw unsupported_operation(); - } - ASSERT(!blobGranuleMapping.more && blobGranuleMapping.size() < CLIENT_KNOBS->TOO_MANY); - - if (blobGranuleMapping.size() == 0) { - if (BG_REQUEST_DEBUG) { - printf("no blob worker assignments yet \n"); - } - throw transaction_too_old(); - } - - if (BG_REQUEST_DEBUG) { - fmt::print("Doing blob granule request @ {}\n", endVersion); - fmt::print("blob worker assignments:\n"); - } - - for (i = 0; i < blobGranuleMapping.size() - 1; i++) { - granuleStartKey = blobGranuleMapping[i].key; - granuleEndKey = blobGranuleMapping[i + 1].key; - if (!blobGranuleMapping[i].value.size()) { - if (BG_REQUEST_DEBUG) { - printf("Key range [%s - %s) missing worker assignment!\n", - granuleStartKey.printable().c_str(), - granuleEndKey.printable().c_str()); - // TODO probably new exception type instead - } - throw transaction_too_old(); - } - - workerId = decodeBlobGranuleMappingValue(blobGranuleMapping[i].value); - if (BG_REQUEST_DEBUG) { - printf(" [%s - %s): %s\n", - granuleStartKey.printable().c_str(), - granuleEndKey.printable().c_str(), - workerId.toString().c_str()); - } - - if (!cx->blobWorker_interf.count(workerId)) { - Optional workerInterface = wait(tr->get(blobWorkerListKeyFor(workerId))); - // from the time the mapping was read from the db, the associated blob worker - // could have died and so its interface wouldn't be present as part of the blobWorkerList - // we persist in the db. So throw wrong_shard_server to get the new mapping - if (!workerInterface.present()) { - throw wrong_shard_server(); - } - cx->blobWorker_interf[workerId] = decodeBlobWorkerListValue(workerInterface.get()); - if (BG_REQUEST_DEBUG) { - printf(" decoded worker interface for %s\n", workerId.toString().c_str()); - } - } - } - break; - } catch (Error& e) { - wait(tr->onError(e)); - } - } - - // Make request for each granule - for (i = 0; i < blobGranuleMapping.size() - 1; i++) { - granuleStartKey = blobGranuleMapping[i].key; - granuleEndKey = blobGranuleMapping[i + 1].key; - // if this was a time travel and the request returned larger bounds, skip this chunk - if (granuleEndKey <= keyRange.begin) { - continue; - } - workerId = decodeBlobGranuleMappingValue(blobGranuleMapping[i].value); - // prune first/last granules to requested range - if (keyRange.begin > granuleStartKey) { - granuleStartKey = keyRange.begin; - } - if (keyRange.end < granuleEndKey) { - granuleEndKey = keyRange.end; - } - - state BlobGranuleFileRequest req; - req.keyRange = KeyRangeRef(StringRef(req.arena, granuleStartKey), StringRef(req.arena, granuleEndKey)); - req.beginVersion = begin; - req.readVersion = endVersion; - - std::vector>> v; - v.push_back(makeReference>(cx->blobWorker_interf[workerId])); - state Reference>> location = - makeReference(v); - // use load balance with one option for now for retry and error handling - BlobGranuleFileReply rep = wait(loadBalance(location, - &BlobWorkerInterface::blobGranuleFileRequest, - req, - TaskPriority::DefaultPromiseEndpoint, - AtMostOnce::False, - nullptr)); - - if (BG_REQUEST_DEBUG) { - fmt::print("Blob granule request for [{0} - {1}) @ {2} - {3} got reply from {4}:\n", - granuleStartKey.printable(), - granuleEndKey.printable(), - begin, - endVersion, - workerId.toString()); - } - for (auto& chunk : rep.chunks) { - if (BG_REQUEST_DEBUG) { - printf("[%s - %s)\n", - chunk.keyRange.begin.printable().c_str(), - chunk.keyRange.end.printable().c_str()); - - printf(" SnapshotFile:\n %s\n", - chunk.snapshotFile.present() ? chunk.snapshotFile.get().toString().c_str() : ""); - printf(" DeltaFiles:\n"); - for (auto& df : chunk.deltaFiles) { - printf(" %s\n", df.toString().c_str()); - } - printf(" Deltas: (%d)", chunk.newDeltas.size()); - if (chunk.newDeltas.size() > 0) { - fmt::print(" with version [{0} - {1}]", - chunk.newDeltas[0].version, - chunk.newDeltas[chunk.newDeltas.size() - 1].version); - } - fmt::print(" IncludedVersion: {}\n", chunk.includedVersion); - printf("\n\n"); - } - Arena a; - a.dependsOn(rep.arena); - results.send(Standalone(chunk, a)); - keyRange = KeyRangeRef(chunk.keyRange.end, keyRange.end); - } - } - results.sendError(end_of_stream()); - return Void(); - } catch (Error& e) { - if (e.code() == error_code_actor_cancelled) { - throw; - } - if (e.code() == error_code_wrong_shard_server || e.code() == error_code_all_alternatives_failed || - e.code() == error_code_connection_failed) { - // TODO would invalidate mapping cache here if we had it - wait(delay(CLIENT_KNOBS->WRONG_SHARD_SERVER_DELAY)); - } else { - if (BG_REQUEST_DEBUG) { - printf("blob granule file request got unexpected error %s\n", e.name()); - } - results.sendError(e); - return Void(); - } - } - } -} - -Future DatabaseContext::readBlobGranulesStream(const PromiseStream>& results, - KeyRange range, - Version begin, - Optional end) { - if (!CLIENT_KNOBS->ENABLE_BLOB_GRANULES) { - throw client_invalid_operation(); - } - return readBlobGranulesStreamActor(Reference::addRef(this), results, range, begin, end); -} - -ACTOR Future setPerpetualStorageWiggle(Database cx, bool enable, LockAware lockAware) { - state ReadYourWritesTransaction tr(cx); - loop { - try { - tr.setOption(FDBTransactionOptions::ACCESS_SYSTEM_KEYS); - if (lockAware) { - tr.setOption(FDBTransactionOptions::LOCK_AWARE); - } - - tr.set(perpetualStorageWiggleKey, enable ? "1"_sr : "0"_sr); - wait(tr.commit()); - break; - } catch (Error& e) { - wait(tr.onError(e)); - } - } - return Void(); -} - Reference DatabaseContext::createTransaction() { return makeReference(Database(Reference::addRef(this))); } From a674edaa6220c1442f6872a00b06cac049d308e7 Mon Sep 17 00:00:00 2001 From: Suraj Gupta Date: Fri, 10 Dec 2021 16:46:22 -0500 Subject: [PATCH 091/413] Address PR comments. --- fdbserver/BlobManager.actor.cpp | 94 ++++++++----------- .../workloads/BlobGranuleVerifier.actor.cpp | 9 +- 2 files changed, 42 insertions(+), 61 deletions(-) diff --git a/fdbserver/BlobManager.actor.cpp b/fdbserver/BlobManager.actor.cpp index c152e23990..cff7cd48d1 100644 --- a/fdbserver/BlobManager.actor.cpp +++ b/fdbserver/BlobManager.actor.cpp @@ -1380,21 +1380,6 @@ ACTOR Future haltBlobGranules(BlobManagerData* bmData) { return Void(); } -// TODO: refactor this into a common file -ACTOR Future> getLatestGranuleHistory(Transaction* tr, KeyRange range) { - KeyRange historyRange = blobGranuleHistoryKeyRangeFor(range); - RangeResult result = wait(tr->getRange(historyRange, 1, Snapshot::False, Reverse::True)); - ASSERT(result.size() <= 1); - - Optional history; - if (!result.empty()) { - std::pair decodedKey = decodeBlobGranuleHistoryKey(result[0].key); - ASSERT(range == decodedKey.first); - history = GranuleHistory(range, decodedKey.second, decodeBlobGranuleHistoryValue(result[0].value)); - } - return history; -} - ACTOR Future loadHistoryFiles(BlobManagerData* bmData, UID granuleID) { state Transaction tr(bmData->db); state KeyRange range = blobGranuleFileKeyRangeFor(granuleID); @@ -1402,7 +1387,7 @@ ACTOR Future loadHistoryFiles(BlobManagerData* bmData, UID granule state GranuleFiles files; loop { try { - wait(readGranuleFiles(&tr, &startKey, range.end, &files, granuleID)); + wait(readGranuleFiles(&tr, &startKey, range.end, &files, granuleID, BM_DEBUG)); return files; } catch (Error& e) { wait(tr.onError(e)); @@ -1480,6 +1465,10 @@ ACTOR Future fullyDeleteGranule(BlobManagerData* self, UID granuleId, KeyR /* * For the granule with id granuleId, finds the first snapshot file at a * version <= pruneVersion and deletes all files older than it. + * + * Assumption: this granule's startVersion might change because the first snapshot + * file might be deleted. We will need to ensure we don't rely on the granule's startVersion + * (that's persisted as part of the key), but rather use the granule's first snapshot's version when needed */ ACTOR Future partiallyDeleteGranule(BlobManagerData* self, UID granuleId, Version pruneVersion) { if (BM_DEBUG) { @@ -1551,12 +1540,6 @@ ACTOR Future partiallyDeleteGranule(BlobManagerData* self, UID granuleId, wait(waitForAll(deletions)); // delete metadata in FDB (deleted file keys) - // TODO: do we need to also update the start version for the history entry? - // it would be a blind write here so might that cause a problem with history traversal in BW? - // do we gain any benefit from updating it? even if we keep the old start version, the worst is - // someone requests a version in [oldStartVersion, newStartVersion) and we fail to return - // any files for that request. - if (BM_DEBUG) { printf("Partially deleting granule %s: deleting file keys\n", granuleId.toString().c_str()); } @@ -1603,8 +1586,10 @@ ACTOR Future pruneRange(BlobManagerData* self, KeyRef startKey, KeyRef end state std::vector> toFullyDelete; state std::vector toPartiallyDelete; - // set of granuleIds to track which granules we have already visited in traversal - state std::unordered_set visited; // track which granules we have already visited in traversal + // track which granules we have already added to traversal + // note: (startKey, startVersion) uniquely identifies a granule + state std::unordered_set, boost::hash>> + visited; state KeyRange range(KeyRangeRef(startKey, endKey)); // range for [startKey, endKey) @@ -1646,6 +1631,7 @@ ACTOR Future pruneRange(BlobManagerData* self, KeyRef startKey, KeyRef end if (BM_DEBUG) { printf("Adding range to history queue\n"); } + visited.insert({ activeRange.range().begin.begin(), history.get().version }); historyEntryQueue.push({ activeRange.range(), history.get().version, MAX_VERSION }); } break; @@ -1693,17 +1679,6 @@ ACTOR Future pruneRange(BlobManagerData* self, KeyRef startKey, KeyRef end currHistoryNode.granuleID.toString().c_str()); } - // if we already saw this node, skip it; otherwise, mark it as visited - // TODO: doing the visited check here as opposed to when adding this node as a parent - // causes us to do another GET, but the parentGranules field does not contain parent gids - if (visited.count(currHistoryNode.granuleID)) { - if (BM_DEBUG) { - printf("Already processed %s, so skipping it\n", currHistoryNode.granuleID.toString().c_str()); - } - continue; - } - visited.insert(currHistoryNode.granuleID); - // There are three cases this granule can fall into: // - if the granule's end version is at or before the prune version or this is a force delete, // this granule should be completely deleted @@ -1719,13 +1694,20 @@ ACTOR Future pruneRange(BlobManagerData* self, KeyRef startKey, KeyRef end if (BM_DEBUG) { printf("Granule %s will be partially deleted\n", currHistoryNode.granuleID.toString().c_str()); } - toPartiallyDelete.push_back({ currHistoryNode.granuleID }); + toPartiallyDelete.push_back({ currHistoryNode.granuleID, currHistoryNode.contents().}); } // add all of the node's parents to the queue for (auto& parent : currHistoryNode.parentGranules) { - // the parent's end version is this node's startVersion, - // since this node must have started where it's parent finished + // if we already added this node to queue, skip it; otherwise, mark it as visited + if (visited.count({ parent.first.begin.begin(), parent.second })) { + if (BM_DEBUG) { + printf("Already added %s to queue, so skipping it\n", currHistoryNode.granuleID.toString().c_str()); + } + continue; + } + visited.insert({ parent.first.begin.begin(), parent.second }); + if (BM_DEBUG) { printf("Adding parent [%s-%s) with versions [%lld-%lld) to queue\n", parent.first.begin.printable().c_str(), @@ -1733,6 +1715,9 @@ ACTOR Future pruneRange(BlobManagerData* self, KeyRef startKey, KeyRef end parent.second, startVersion); } + + // the parent's end version is this node's startVersion, + // since this node must have started where it's parent finished historyEntryQueue.push({ parent.first, parent.second, startVersion }); } } @@ -1844,6 +1829,22 @@ ACTOR Future pruneRange(BlobManagerData* self, KeyRef startKey, KeyRef end * case that the timer is up before any new prune intents arrive). */ ACTOR Future monitorPruneKeys(BlobManagerData* self) { + // setup bstore + try { + if (BM_DEBUG) { + printf("BM constructing backup container from %s\n", SERVER_KNOBS->BG_URL.c_str()); + } + self->bstore = BackupContainerFileSystem::openContainerFS(SERVER_KNOBS->BG_URL); + if (BM_DEBUG) { + printf("BM constructed backup container\n"); + } + } catch (Error& e) { + if (BM_DEBUG) { + printf("BM got backup container init error %s\n", e.name()); + } + throw e; + } + try { state Value oldPruneWatchVal; loop { @@ -1870,7 +1871,7 @@ ACTOR Future monitorPruneKeys(BlobManagerData* self) { } // TODO: debugging code, remove it - if (newPruneWatchVal.get().toString().substr(0, 6) == "random") { + if (newPruneWatchVal.get().toString().substr(0, 6) == "prune=") { state Reference dummy = makeReference(self->db); loop { @@ -2035,21 +2036,6 @@ ACTOR Future blobManager(BlobManagerInterface bmInterf, fmt::print("Blob manager acquired lock at epoch {}\n", epoch); } - try { - if (BM_DEBUG) { - printf("BM constructing backup container from %s\n", SERVER_KNOBS->BG_URL.c_str()); - } - self.bstore = BackupContainerFileSystem::openContainerFS(SERVER_KNOBS->BG_URL); - if (BM_DEBUG) { - printf("BM constructed backup container\n"); - } - } catch (Error& e) { - if (BM_DEBUG) { - printf("BM got backup container init error %s\n", e.name()); - } - throw e; - } - // although we start the recruiter, we wait until existing workers are ack'd auto recruitBlobWorker = IAsyncListener>::create( dbInfo, [](auto const& info) { return info.clusterInterface.recruitBlobWorker; }); diff --git a/fdbserver/workloads/BlobGranuleVerifier.actor.cpp b/fdbserver/workloads/BlobGranuleVerifier.actor.cpp index 9c97edbcce..901a98c581 100644 --- a/fdbserver/workloads/BlobGranuleVerifier.actor.cpp +++ b/fdbserver/workloads/BlobGranuleVerifier.actor.cpp @@ -60,7 +60,6 @@ struct BlobGranuleVerifierWorkload : TestWorkload { int64_t timeTravelTooOld = 0; int64_t rowsRead = 0; int64_t bytesRead = 0; - KeyRangeMap latestPruneVersions; std::vector> clients; DatabaseConfiguration config; @@ -307,6 +306,7 @@ struct BlobGranuleVerifierWorkload : TestWorkload { OldRead(KeyRange range, Version v, RangeResult oldResult) : range(range), v(v), oldResult(oldResult) {} }; + // utility to prune at pruneVersion= with the flag ACTOR Future pruneAtVersion(Database cx, KeyRange range, Version version, bool force) { state Reference tr = makeReference(cx); loop { @@ -353,7 +353,6 @@ struct BlobGranuleVerifierWorkload : TestWorkload { state double endTime = last + self->testDuration; state std::map timeTravelChecks; state int64_t timeTravelChecksMemory = 0; - state KeyRangeMap latestPruneVersions; TraceEvent("BlobGranuleVerifierStart"); if (BGV_DEBUG) { @@ -378,11 +377,7 @@ struct BlobGranuleVerifierWorkload : TestWorkload { // advance iterator before doing read, so if it gets error we don't retry it try { - // before reading, prune at some version [0, readVersion) - Version pruneVersion = deterministicRandom()->randomInt(0, oldRead.v); - wait(self->pruneAtVersion(cx, oldRead.range, pruneVersion, false)); - // FIXME: this doesnt actually guarantee that the prune executed. maybe add a delay? - + // TODO: before reading, prune at some version [0, readVersion) std::pair>> reReadResult = wait(self->readFromBlob(cx, self, oldRead.range, oldRead.v)); self->compareResult(oldRead.oldResult, reReadResult, oldRead.range, oldRead.v, false); From e102738ca985422514aa725f142e3b1042688091 Mon Sep 17 00:00:00 2001 From: Suraj Gupta Date: Fri, 10 Dec 2021 17:01:42 -0500 Subject: [PATCH 092/413] Comment out debugging code. --- fdbserver/BlobManager.actor.cpp | 36 +++++++++++++++++---------------- 1 file changed, 19 insertions(+), 17 deletions(-) diff --git a/fdbserver/BlobManager.actor.cpp b/fdbserver/BlobManager.actor.cpp index cff7cd48d1..ccd2e16e47 100644 --- a/fdbserver/BlobManager.actor.cpp +++ b/fdbserver/BlobManager.actor.cpp @@ -1871,26 +1871,28 @@ ACTOR Future monitorPruneKeys(BlobManagerData* self) { } // TODO: debugging code, remove it + /* if (newPruneWatchVal.get().toString().substr(0, 6) == "prune=") { - state Reference dummy = - makeReference(self->db); - loop { - try { - dummy->setOption(FDBTransactionOptions::ACCESS_SYSTEM_KEYS); - dummy->setOption(FDBTransactionOptions::PRIORITY_SYSTEM_IMMEDIATE); - std::istringstream iss(newPruneWatchVal.get().toString().substr(6)); - Version version; - iss >> version; - dummy->set(blobGranulePruneKeys.begin.withSuffix(normalKeys.begin), - blobGranulePruneValueFor(version, false)); - wait(dummy->commit()); - break; + state Reference dummy = + makeReference(self->db); + loop { + try { + dummy->setOption(FDBTransactionOptions::ACCESS_SYSTEM_KEYS); + dummy->setOption(FDBTransactionOptions::PRIORITY_SYSTEM_IMMEDIATE); + std::istringstream iss(newPruneWatchVal.get().toString().substr(6)); + Version version; + iss >> version; + dummy->set(blobGranulePruneKeys.begin.withSuffix(normalKeys.begin), + blobGranulePruneValueFor(version, false)); + wait(dummy->commit()); + break; - } catch (Error& e) { - wait(dummy->onError(e)); - } - } + } catch (Error& e) { + wait(dummy->onError(e)); + } + } } + */ break; } From 1e36069c8f3ce764830c1fdc7e190730f69bc6cf Mon Sep 17 00:00:00 2001 From: Suraj Gupta Date: Fri, 10 Dec 2021 17:13:33 -0500 Subject: [PATCH 093/413] fix error. --- fdbserver/BlobManager.actor.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/fdbserver/BlobManager.actor.cpp b/fdbserver/BlobManager.actor.cpp index ccd2e16e47..626823613a 100644 --- a/fdbserver/BlobManager.actor.cpp +++ b/fdbserver/BlobManager.actor.cpp @@ -1694,7 +1694,7 @@ ACTOR Future pruneRange(BlobManagerData* self, KeyRef startKey, KeyRef end if (BM_DEBUG) { printf("Granule %s will be partially deleted\n", currHistoryNode.granuleID.toString().c_str()); } - toPartiallyDelete.push_back({ currHistoryNode.granuleID, currHistoryNode.contents().}); + toPartiallyDelete.push_back({ currHistoryNode.granuleID }); } // add all of the node's parents to the queue From 9f69715fec258bc752bbd9401a2c4913dd6eb6bb Mon Sep 17 00:00:00 2001 From: Josh Slocum Date: Mon, 20 Dec 2021 11:33:44 -0600 Subject: [PATCH 094/413] Fixing blob worker committed tracking and ReplyPromiseStream::onEmpty --- fdbserver/BlobWorker.actor.cpp | 34 ++++++++++++++++++++-------------- flow/flow.h | 5 +++++ 2 files changed, 25 insertions(+), 14 deletions(-) diff --git a/fdbserver/BlobWorker.actor.cpp b/fdbserver/BlobWorker.actor.cpp index 3c92bb575d..b24e651458 100644 --- a/fdbserver/BlobWorker.actor.cpp +++ b/fdbserver/BlobWorker.actor.cpp @@ -77,6 +77,7 @@ struct GranuleMetadata : NonCopyable, ReferenceCounted { NotifiedVersion durableSnapshotVersion; // same as delta vars, except for snapshots Version pendingSnapshotVersion = 0; Version initialSnapshotVersion = invalidVersion; + Version knownCommittedVersion; int64_t originalEpoch; int64_t originalSeqno; @@ -953,6 +954,12 @@ static Reference newChangeFeedData(Version startVersion) { return r; } +// TODO REMOVE once correctness clean +#define DEBUG_BW_START_VERSION invalidVersion +#define DEBUG_BW_END_VERSION invalidVersion +#define DEBUG_BW_WAIT_VERSION invalidVersion +#define DEBUG_BW_VERSION(v) DEBUG_BW_START_VERSION <= v&& v <= DEBUG_BW_END_VERSION + static Version doGranuleRollback(Reference metadata, Version mutationVersion, Version rollbackVersion, @@ -1086,12 +1093,12 @@ ACTOR Future waitVersionCommitted(Reference bwData, wait(grvAtLeast); } state Version grvVersion = bwData->grvVersion.get(); - /*if (BW_DEBUG) { - fmt::print("waitVersionCommitted got {0} < {1}, waiting on CF (currently {2})\n", - version, - grvVersion, - metadata->activeCFData.get()->getVersion()); - }*/ + if ((DEBUG_BW_VERSION(version)) || (DEBUG_BW_VERSION(grvVersion))) { + fmt::print("waitVersionCommitted got {0} < {1}, waiting on CF (currently {2})\n", + version, + grvVersion, + metadata->activeCFData.get()->getVersion()); + } // make sure the change feed has consumed mutations up through grvVersion to ensure none of them are rollbacks loop { @@ -1107,6 +1114,9 @@ ACTOR Future waitVersionCommitted(Reference bwData, if (grvVersion > metadata->waitForVersionReturned) { metadata->waitForVersionReturned = grvVersion; } + if (version > metadata->knownCommittedVersion) { + metadata->knownCommittedVersion = version; + } /*if (BW_DEBUG) { fmt::print( "waitVersionCommitted CF whenAtLeast {0}: {1}\n", grvVersion, metadata->activeCFData.get()->getVersion()); @@ -1115,12 +1125,6 @@ ACTOR Future waitVersionCommitted(Reference bwData, return Void(); } -// TODO REMOVE once correctness clean -#define DEBUG_BW_START_VERSION invalidVersion -#define DEBUG_BW_END_VERSION invalidVersion -#define DEBUG_BW_WAIT_VERSION invalidVersion -#define DEBUG_BW_VERSION(v) DEBUG_BW_START_VERSION <= v&& v <= DEBUG_BW_END_VERSION - // updater for a single granule // TODO: this is getting kind of large. Should try to split out this actor if it continues to grow? // FIXME: handle errors here (forward errors) @@ -1591,7 +1595,8 @@ ACTOR Future blobGranuleUpdateFiles(Reference bwData, state int waitIdx = 0; int idx = 0; for (auto& f : inFlightFiles) { - if (f.snapshot && f.version < metadata->pendingSnapshotVersion) { + if (f.snapshot && f.version < metadata->pendingSnapshotVersion && + f.version <= metadata->knownCommittedVersion) { if (BW_DEBUG) { fmt::print("[{0} - {1}) Waiting on previous snapshot file @ {2}\n", metadata->keyRange.begin.printable(), @@ -1634,7 +1639,8 @@ ACTOR Future blobGranuleUpdateFiles(Reference bwData, // if we're in the old change feed case and can't snapshot but we have enough data to, don't // queue too many files in parallel, and slow down change feed consuming to let file writing // catch up - if (inFlightFiles.size() > 10) { + + if (inFlightFiles.size() > 10 && inFlightFiles.front().version <= metadata->knownCommittedVersion) { if (BW_DEBUG) { printf("[%s - %s) Waiting on delta file b/c old change feed\n", metadata->keyRange.begin.printable().c_str(), diff --git a/flow/flow.h b/flow/flow.h index a3d0e5d7a7..27b08db274 100644 --- a/flow/flow.h +++ b/flow/flow.h @@ -1052,6 +1052,11 @@ protected: } auto copy = std::move(queue.front()); queue.pop(); + if (onEmpty.isValid() && queue.empty()) { + Promise hold = onEmpty; + onEmpty = Promise(nullptr); + hold.send(Void()); + } return copy; } From abdaf5c9cdf4391824f2223baf33d3bc84046ca0 Mon Sep 17 00:00:00 2001 From: Josh Slocum Date: Tue, 21 Dec 2021 08:58:20 -0600 Subject: [PATCH 095/413] extra debugging for CF empty versions --- fdbclient/DatabaseContext.h | 2 ++ fdbclient/NativeAPI.actor.cpp | 38 +++++++++++++++++++++++++++++++++++ 2 files changed, 40 insertions(+) diff --git a/fdbclient/DatabaseContext.h b/fdbclient/DatabaseContext.h index 2f47524f2f..b42c502a53 100644 --- a/fdbclient/DatabaseContext.h +++ b/fdbclient/DatabaseContext.h @@ -160,6 +160,8 @@ struct ChangeFeedStorageData : ReferenceCounted { NotifiedVersion desired; Promise destroyed; + bool debug = false; // TODO REMOVE + ~ChangeFeedStorageData() { destroyed.send(Void()); } }; diff --git a/fdbclient/NativeAPI.actor.cpp b/fdbclient/NativeAPI.actor.cpp index 153bbef1f0..c1b5f10ce4 100644 --- a/fdbclient/NativeAPI.actor.cpp +++ b/fdbclient/NativeAPI.actor.cpp @@ -7024,21 +7024,51 @@ ACTOR Future storageFeedVersionUpdater(StorageServerInterface interf, Chan state Promise destroyed = self->destroyed; loop { if (destroyed.isSet()) { + if (self->debug) { + fmt::print("CFSD {0}: destroyed\n", self->id.toString().substr(0, 4)); + } return Void(); } if (self->version.get() < self->desired.get()) { + if (self->debug) { + fmt::print("CFSD {0}: update waiting {1} < {2}\n", + self->id.toString().substr(0, 4), + self->version.get(), + self->desired.get()); + } wait(delay(CLIENT_KNOBS->CHANGE_FEED_EMPTY_BATCH_TIME) || self->version.whenAtLeast(self->desired.get())); if (destroyed.isSet()) { + if (self->debug) { + fmt::print("CFSD {0}: destroyed2\n", self->id.toString().substr(0, 4)); + } return Void(); } + if (self->debug) { + fmt::print("CFSD {0}: updated {1} < {2}\n", + self->id.toString().substr(0, 4), + self->version.get(), + self->desired.get()); + } if (self->version.get() < self->desired.get()) { + if (self->debug) { + fmt::print("CFSD {0}: requesting {1}\n", self->id.toString().substr(0, 4), self->desired.get()); + } ChangeFeedVersionUpdateReply rep = wait(brokenPromiseToNever( interf.changeFeedVersionUpdate.getReply(ChangeFeedVersionUpdateRequest(self->desired.get())))); + if (self->debug) { + fmt::print("CFSD {0}: got {1}\n", self->id.toString().substr(0, 4), rep.version); + } if (rep.version > self->version.get()) { + if (self->debug) { + fmt::print("CFSD {0}: V={1} (req)\n", self->id.toString().substr(0, 4), rep.version); + } self->version.set(rep.version); } } } else { + if (self->debug) { + fmt::print("CFSD {0}: desired.WAL({1})\n", self->id.toString().substr(0, 4), self->version.get() + 1); + } wait(self->desired.whenAtLeast(self->version.get() + 1)); } } @@ -7088,6 +7118,9 @@ ACTOR Future changeFeedWaitLatest(Reference self, Version if (it->version.get() < version) { waiting++; if (version > it->desired.get()) { + if (DEBUG_CF_WAIT_VERSION == version) { + it->debug = true; + } it->desired.set(version); desired++; } @@ -7609,6 +7642,11 @@ ACTOR Future getChangeFeedStreamActor(Reference db, results->notAtLatest.set(0); } if (feedReply.minStreamVersion > results->storageData[0]->version.get()) { + if (results->storageData[0]->debug) { + fmt::print("CFSD {0}: V={1} (CFLR)\n", + results->storageData[0]->id.toString().substr(0, 4), + results->storageData[0]->version.get()); + } results->storageData[0]->version.set(feedReply.minStreamVersion); } } From d337e8fbe8e04753a94e92e637210acafdc235bf Mon Sep 17 00:00:00 2001 From: Josh Slocum Date: Tue, 21 Dec 2021 10:38:04 -0600 Subject: [PATCH 096/413] Handled disconnects explicitly in CF streams --- fdbclient/NativeAPI.actor.cpp | 197 ++++++++++++++++++--------------- fdbserver/BlobWorker.actor.cpp | 14 +-- 2 files changed, 117 insertions(+), 94 deletions(-) diff --git a/fdbclient/NativeAPI.actor.cpp b/fdbclient/NativeAPI.actor.cpp index c1b5f10ce4..53aceeb168 100644 --- a/fdbclient/NativeAPI.actor.cpp +++ b/fdbclient/NativeAPI.actor.cpp @@ -7230,13 +7230,13 @@ Future ChangeFeedData::whenAtLeast(Version version) { return changeFeedWhenAtLatest(Reference::addRef(this), version); } -ACTOR Future singleChangeFeedStream(StorageServerInterface interf, - PromiseStream> results, - ReplyPromiseStream replyStream, - Version end, - Reference feedData, - Reference storageData, - int idx /* TODO REMOVE this param after correctness clean */) { +ACTOR Future partialChangeFeedStream(StorageServerInterface interf, + PromiseStream> results, + ReplyPromiseStream replyStream, + Version end, + Reference feedData, + Reference storageData, + int idx /* TODO REMOVE this param after correctness clean */) { state bool atLatestVersion = false; state Version nextVersion = 0; try { @@ -7376,7 +7376,7 @@ ACTOR Future mergeChangeFeedStream(Reference db, refresh.send(Void()); for (int i = 0; i < interfs.size(); i++) { - fetchers[i] = singleChangeFeedStream( + fetchers[i] = partialChangeFeedStream( interfs[i].first, streams[i].results, results->streams[i], end, results, results->storageData[i], i); } state int interfNum = 0; @@ -7496,6 +7496,97 @@ ACTOR Future getChangeFeedRange(Reference db, Databas } } } +// TODO better name +ACTOR Future singleChangeFeedStream(Reference db, + StorageServerInterface interf, + KeyRange range, + Reference results, + Key rangeID, + Version* begin, + Version end) { + state Database cx(db); + state ChangeFeedStreamRequest req; + req.rangeID = rangeID; + req.begin = *begin; + req.end = end; + req.range = range; + + results->streams.clear(); + + for (auto& it : results->storageData) { + if (it->debugGetReferenceCount() == 2) { + db->changeFeedUpdaters.erase(it->id); + } + } + results->streams.push_back(interf.changeFeedStream.getReplyStream(req)); + + results->maxSeenVersion = invalidVersion; + results->storageData.clear(); + results->storageData.push_back(db->getStorageData(interf)); + Promise refresh = results->refresh; + results->refresh = Promise(); + results->notAtLatest.set(1); + refresh.send(Void()); + state bool atLatest = false; + loop { + wait(results->mutations.onEmpty()); + state ChangeFeedStreamReply feedReply = waitNext(results->streams[0].getFuture()); + *begin = feedReply.mutations.back().version + 1; + + // TODO REMOVE, for debugging + if (feedReply.mutations.back().version < results->lastReturnedVersion.get()) { + printf("out of order mutation for CF %s from (%d) %s! %lld < %lld\n", + rangeID.toString().substr(0, 6).c_str(), + results->storageData.size(), + results->storageData.empty() ? "????" : results->storageData[0]->id.toString().substr(0, 4).c_str(), + feedReply.mutations.back().version, + results->lastReturnedVersion.get()); + } + ASSERT(feedReply.mutations.back().version >= results->lastReturnedVersion.get()); + results->mutations.send(Standalone>(feedReply.mutations, feedReply.arena)); + + // Because onEmpty returns here before the consuming process, we must do a delay(0) + wait(results->mutations.onEmpty()); + wait(delay(0)); + + if (DEBUG_CF_VERSION(feedReply.mutations.back().version)) { + fmt::print("CFLR (single): {0} ({1}), atLatest={2}, rep.atLatest={3}, notAtLatest={4}, " + "minSV={5}\n", + feedReply.mutations.back().version, + feedReply.mutations.size(), + atLatest ? "T" : "F", + feedReply.atLatestVersion ? "T" : "F", + results->notAtLatest.get(), + feedReply.minStreamVersion); + } + if (feedReply.mutations.back().version > results->lastReturnedVersion.get()) { + results->lastReturnedVersion.set(feedReply.mutations.back().version); + } + if (!atLatest && feedReply.atLatestVersion) { + atLatest = true; + results->notAtLatest.set(0); + } + if (feedReply.minStreamVersion > results->storageData[0]->version.get()) { + if (results->storageData[0]->debug) { + fmt::print("CFSD {0}: V={1} (CFLR)\n", + results->storageData[0]->id.toString().substr(0, 4), + results->storageData[0]->version.get()); + } + results->storageData[0]->version.set(feedReply.minStreamVersion); + } + } +} + +ACTOR Future anyCFDisconnect(std::vector> interfs) { + state std::vector> disconnectFutures; + disconnectFutures.reserve(interfs.size()); + for (auto& it : interfs) { + disconnectFutures.push_back( + IFailureMonitor::failureMonitor().onDisconnectOrFailure(it.first.changeFeedStream.getEndpoint())); + } + wait(waitForAny(disconnectFutures)); + return Void(); +} ACTOR Future getChangeFeedStreamActor(Reference db, Reference results, @@ -7571,86 +7662,18 @@ ACTOR Future getChangeFeedStreamActor(Reference db, interfs.push_back(std::make_pair(locations[i].second->getInterface(chosenLocations[i]), locations[i].first & range)); } - wait(mergeChangeFeedStream(db, interfs, results, rangeID, &begin, end) || cx->connectionFileChanged()); - } else { - state ChangeFeedStreamRequest req; - req.rangeID = rangeID; - req.begin = begin; - req.end = end; - req.range = range; - - results->streams.clear(); - - StorageServerInterface interf = locations[0].second->getInterface(chosenLocations[0]); - - for (auto& it : results->storageData) { - if (it->debugGetReferenceCount() == 2) { - db->changeFeedUpdaters.erase(it->id); - } + choose { + when(wait(mergeChangeFeedStream(db, interfs, results, rangeID, &begin, end))) {} + when(wait(cx->connectionFileChanged())) {} + when(wait(anyCFDisconnect(interfs))) {} } - results->streams.push_back(interf.changeFeedStream.getReplyStream(req)); - - results->maxSeenVersion = invalidVersion; - results->storageData.clear(); - results->storageData.push_back(db->getStorageData(interf)); - Promise refresh = results->refresh; - results->refresh = Promise(); - results->notAtLatest.set(1); - refresh.send(Void()); - state bool atLatest = false; - loop { - wait(results->mutations.onEmpty()); - choose { - when(wait(cx->connectionFileChanged())) { break; } - when(state ChangeFeedStreamReply feedReply = waitNext(results->streams[0].getFuture())) { - begin = feedReply.mutations.back().version + 1; - - // TODO REMOVE, for debugging - if (feedReply.mutations.back().version < results->lastReturnedVersion.get()) { - printf("out of order mutation for CF %s from (%d) %s! %lld < %lld\n", - rangeID.toString().substr(0, 6).c_str(), - results->storageData.size(), - results->storageData.empty() - ? "????" - : results->storageData[0]->id.toString().substr(0, 4).c_str(), - feedReply.mutations.back().version, - results->lastReturnedVersion.get()); - } - ASSERT(feedReply.mutations.back().version >= results->lastReturnedVersion.get()); - results->mutations.send( - Standalone>(feedReply.mutations, feedReply.arena)); - - // Because onEmpty returns here before the consuming process, we must do a delay(0) - wait(results->mutations.onEmpty()); - wait(delay(0)); - - if (DEBUG_CF_VERSION(feedReply.mutations.back().version)) { - fmt::print("CFLR (single): {0} ({1}), atLatest={2}, rep.atLatest={3}, notAtLatest={4}, " - "minSV={5}\n", - feedReply.mutations.back().version, - feedReply.mutations.size(), - atLatest ? "T" : "F", - feedReply.atLatestVersion ? "T" : "F", - results->notAtLatest.get(), - feedReply.minStreamVersion); - } - if (feedReply.mutations.back().version > results->lastReturnedVersion.get()) { - results->lastReturnedVersion.set(feedReply.mutations.back().version); - } - if (!atLatest && feedReply.atLatestVersion) { - atLatest = true; - results->notAtLatest.set(0); - } - if (feedReply.minStreamVersion > results->storageData[0]->version.get()) { - if (results->storageData[0]->debug) { - fmt::print("CFSD {0}: V={1} (CFLR)\n", - results->storageData[0]->id.toString().substr(0, 4), - results->storageData[0]->version.get()); - } - results->storageData[0]->version.set(feedReply.minStreamVersion); - } - } - } + } else { + StorageServerInterface interf = locations[0].second->getInterface(chosenLocations[0]); + choose { + when(wait(singleChangeFeedStream(db, interf, range, results, rangeID, &begin, end))) {} + when(wait(cx->connectionFileChanged())) {} + when(wait(IFailureMonitor::failureMonitor().onDisconnectOrFailure( + interf.changeFeedStream.getEndpoint()))) {} } } } catch (Error& e) { diff --git a/fdbserver/BlobWorker.actor.cpp b/fdbserver/BlobWorker.actor.cpp index b24e651458..79146ad3d5 100644 --- a/fdbserver/BlobWorker.actor.cpp +++ b/fdbserver/BlobWorker.actor.cpp @@ -1930,13 +1930,13 @@ ACTOR Future waitForVersion(Reference metadata, Version v } ACTOR Future handleBlobGranuleFileRequest(Reference bwData, BlobGranuleFileRequest req) { - if (BW_DEBUG) { - printf("BW %s processing blobGranuleFileRequest for range [%s-%s) @ %lld\n", - bwData->id.toString().c_str(), - req.keyRange.begin.printable().c_str(), - req.keyRange.end.printable().c_str(), - req.readVersion); - } + /*if (BW_DEBUG) { + printf("BW %s processing blobGranuleFileRequest for range [%s-%s) @ %lld\n", + bwData->id.toString().c_str(), + req.keyRange.begin.printable().c_str(), + req.keyRange.end.printable().c_str(), + req.readVersion); + }*/ try { // TODO REMOVE in api V2 From b0aea918959e86a5035028614f32ae4d7117458f Mon Sep 17 00:00:00 2001 From: Josh Slocum Date: Tue, 21 Dec 2021 14:12:09 -0600 Subject: [PATCH 097/413] Broadening explicit disconnect handling to explicit error handling of all types --- fdbclient/NativeAPI.actor.cpp | 189 ++++++++++++++++-------------- fdbrpc/fdbrpc.h | 13 ++ fdbserver/BlobWorker.actor.cpp | 2 +- fdbserver/storageserver.actor.cpp | 34 +++++- flow/flow.h | 12 +- 5 files changed, 160 insertions(+), 90 deletions(-) diff --git a/fdbclient/NativeAPI.actor.cpp b/fdbclient/NativeAPI.actor.cpp index 53aceeb168..a97841f1b6 100644 --- a/fdbclient/NativeAPI.actor.cpp +++ b/fdbclient/NativeAPI.actor.cpp @@ -7340,45 +7340,13 @@ ACTOR Future partialChangeFeedStream(StorageServerInterface interf, } } -ACTOR Future mergeChangeFeedStream(Reference db, - std::vector> interfs, - Reference results, - Key rangeID, - Version* begin, - Version end) { +// TODO better name +ACTOR Future doCFMerge(Reference results, + std::vector> interfs, + std::vector streams, + Version* begin, + Version end) { state std::priority_queue> mutations; - state std::vector> fetchers(interfs.size()); - state std::vector streams(interfs.size()); - - results->streams.clear(); - for (auto& it : interfs) { - ChangeFeedStreamRequest req; - req.rangeID = rangeID; - req.begin = *begin; - req.end = end; - req.range = it.second; - results->streams.push_back(it.first.changeFeedStream.getReplyStream(req)); - } - - for (auto& it : results->storageData) { - if (it->debugGetReferenceCount() == 2) { - db->changeFeedUpdaters.erase(it->id); - } - } - results->maxSeenVersion = invalidVersion; - results->storageData.clear(); - Promise refresh = results->refresh; - results->refresh = Promise(); - for (int i = 0; i < interfs.size(); i++) { - results->storageData.push_back(db->getStorageData(interfs[i].first)); - } - results->notAtLatest.set(interfs.size()); - refresh.send(Void()); - - for (int i = 0; i < interfs.size(); i++) { - fetchers[i] = partialChangeFeedStream( - interfs[i].first, streams[i].results, results->streams[i], end, results, results->storageData[i], i); - } state int interfNum = 0; while (interfNum < interfs.size()) { try { @@ -7464,6 +7432,64 @@ ACTOR Future mergeChangeFeedStream(Reference db, throw end_of_stream(); } +ACTOR Future onCFErrors(std::vector> onErrors) { + wait(waitForAny(onErrors)); + // propagate error - TODO better way? + for (auto& f : onErrors) { + if (f.isError()) { + throw f.getError(); + } + } + ASSERT(false); + return Void(); +} + +ACTOR Future mergeChangeFeedStream(Reference db, + std::vector> interfs, + Reference results, + Key rangeID, + Version* begin, + Version end) { + state std::vector> fetchers(interfs.size()); + state std::vector> onErrors(interfs.size()); + state std::vector streams(interfs.size()); + + results->streams.clear(); + for (auto& it : interfs) { + ChangeFeedStreamRequest req; + req.rangeID = rangeID; + req.begin = *begin; + req.end = end; + req.range = it.second; + results->streams.push_back(it.first.changeFeedStream.getReplyStream(req)); + } + + for (auto& it : results->storageData) { + if (it->debugGetReferenceCount() == 2) { + db->changeFeedUpdaters.erase(it->id); + } + } + results->maxSeenVersion = invalidVersion; + results->storageData.clear(); + Promise refresh = results->refresh; + results->refresh = Promise(); + for (int i = 0; i < interfs.size(); i++) { + results->storageData.push_back(db->getStorageData(interfs[i].first)); + } + results->notAtLatest.set(interfs.size()); + refresh.send(Void()); + + for (int i = 0; i < interfs.size(); i++) { + onErrors[i] = results->streams[i].onError(); + fetchers[i] = partialChangeFeedStream( + interfs[i].first, streams[i].results, results->streams[i], end, results, results->storageData[i], i); + } + + wait(onCFErrors(onErrors) || doCFMerge(results, interfs, streams, begin, end)); + + return Void(); +} + ACTOR Future getChangeFeedRange(Reference db, Database cx, Key rangeID, Version begin = 0) { state Transaction tr(cx); state Key rangeIDKey = rangeID.withPrefix(changeFeedPrefix); @@ -7496,37 +7522,13 @@ ACTOR Future getChangeFeedRange(Reference db, Databas } } } -// TODO better name -ACTOR Future singleChangeFeedStream(Reference db, - StorageServerInterface interf, - KeyRange range, - Reference results, - Key rangeID, - Version* begin, - Version end) { - state Database cx(db); - state ChangeFeedStreamRequest req; - req.rangeID = rangeID; - req.begin = *begin; - req.end = end; - req.range = range; - results->streams.clear(); +ACTOR Future doSingleCFStream(KeyRange range, + Reference results, + Key rangeID, + Version* begin, + Version end) { - for (auto& it : results->storageData) { - if (it->debugGetReferenceCount() == 2) { - db->changeFeedUpdaters.erase(it->id); - } - } - results->streams.push_back(interf.changeFeedStream.getReplyStream(req)); - - results->maxSeenVersion = invalidVersion; - results->storageData.clear(); - results->storageData.push_back(db->getStorageData(interf)); - Promise refresh = results->refresh; - results->refresh = Promise(); - results->notAtLatest.set(1); - refresh.send(Void()); state bool atLatest = false; loop { wait(results->mutations.onEmpty()); @@ -7577,14 +7579,39 @@ ACTOR Future singleChangeFeedStream(Reference db, } } -ACTOR Future anyCFDisconnect(std::vector> interfs) { - state std::vector> disconnectFutures; - disconnectFutures.reserve(interfs.size()); - for (auto& it : interfs) { - disconnectFutures.push_back( - IFailureMonitor::failureMonitor().onDisconnectOrFailure(it.first.changeFeedStream.getEndpoint())); +ACTOR Future singleChangeFeedStream(Reference db, + StorageServerInterface interf, + KeyRange range, + Reference results, + Key rangeID, + Version* begin, + Version end) { + state Database cx(db); + state ChangeFeedStreamRequest req; + req.rangeID = rangeID; + req.begin = *begin; + req.end = end; + req.range = range; + + results->streams.clear(); + + for (auto& it : results->storageData) { + if (it->debugGetReferenceCount() == 2) { + db->changeFeedUpdaters.erase(it->id); + } } - wait(waitForAny(disconnectFutures)); + results->streams.push_back(interf.changeFeedStream.getReplyStream(req)); + + results->maxSeenVersion = invalidVersion; + results->storageData.clear(); + results->storageData.push_back(db->getStorageData(interf)); + Promise refresh = results->refresh; + results->refresh = Promise(); + results->notAtLatest.set(1); + refresh.send(Void()); + + wait(results->streams[0].onError() || doSingleCFStream(range, results, rangeID, begin, end)); + return Void(); } @@ -7662,19 +7689,11 @@ ACTOR Future getChangeFeedStreamActor(Reference db, interfs.push_back(std::make_pair(locations[i].second->getInterface(chosenLocations[i]), locations[i].first & range)); } - choose { - when(wait(mergeChangeFeedStream(db, interfs, results, rangeID, &begin, end))) {} - when(wait(cx->connectionFileChanged())) {} - when(wait(anyCFDisconnect(interfs))) {} - } + wait(mergeChangeFeedStream(db, interfs, results, rangeID, &begin, end) || cx->connectionFileChanged()); } else { StorageServerInterface interf = locations[0].second->getInterface(chosenLocations[0]); - choose { - when(wait(singleChangeFeedStream(db, interf, range, results, rangeID, &begin, end))) {} - when(wait(cx->connectionFileChanged())) {} - when(wait(IFailureMonitor::failureMonitor().onDisconnectOrFailure( - interf.changeFeedStream.getEndpoint()))) {} - } + wait(singleChangeFeedStream(db, interf, range, results, rangeID, &begin, end) || + cx->connectionFileChanged()); } } catch (Error& e) { if (e.code() == error_code_actor_cancelled) { diff --git a/fdbrpc/fdbrpc.h b/fdbrpc/fdbrpc.h index 6468e19d7d..5c63ec5251 100644 --- a/fdbrpc/fdbrpc.h +++ b/fdbrpc/fdbrpc.h @@ -511,6 +511,19 @@ public: return queue->onEmpty.getFuture(); } + bool isError() const { return !queue->isError(); } + + // throws, used to short circuit waiting on the queue if there has been an unexpected error + Future onError() { + if (queue->hasError() && queue->error.code() != error_code_end_of_stream) { + throw queue->error; + } + if (!queue->onError.isValid()) { + queue->onError = Promise(); + } + return queue->onError.getFuture(); + } + uint32_t size() const { return queue->size(); } // Must be called on the server before sending results on the stream to ratelimit the amount of data outstanding to diff --git a/fdbserver/BlobWorker.actor.cpp b/fdbserver/BlobWorker.actor.cpp index 79146ad3d5..680fd1822e 100644 --- a/fdbserver/BlobWorker.actor.cpp +++ b/fdbserver/BlobWorker.actor.cpp @@ -1930,7 +1930,7 @@ ACTOR Future waitForVersion(Reference metadata, Version v } ACTOR Future handleBlobGranuleFileRequest(Reference bwData, BlobGranuleFileRequest req) { - /*if (BW_DEBUG) { + /*if (BW_REQUEST_DEBUG) { printf("BW %s processing blobGranuleFileRequest for range [%s-%s) @ %lld\n", bwData->id.toString().c_str(), req.keyRange.begin.printable().c_str(), diff --git a/fdbserver/storageserver.actor.cpp b/fdbserver/storageserver.actor.cpp index e5e1295154..05720a1bd2 100644 --- a/fdbserver/storageserver.actor.cpp +++ b/fdbserver/storageserver.actor.cpp @@ -349,8 +349,17 @@ struct ChangeFeedInfo : ReferenceCounted { KeyRange range; Key id; AsyncTrigger newMutations; + Promise onMove; bool stopped = false; // A stopped change feed no longer adds new mutations, but is still queriable bool removing = false; + + // TODO: could change this to a promiseStream, and only cancel stream if the cancelled range overlaps the + // request range + void moved() { + Promise hold = onMove; + onMove = Promise(); + hold.send(Void()); + } }; class ServerWatchMetadata : public ReferenceCounted { @@ -1882,6 +1891,20 @@ ACTOR Future localChangeFeedStream(StorageServer* data, } } +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 Future moved = feed->second->onMove.getFuture(); + wait(moved); + // DO NOT call req.reply.onReady before sending - we want to propagate this error through regardless of how far + // behind client is + req.reply.sendError(wrong_shard_server()); + return Void(); +} + ACTOR Future changeFeedStreamQ(StorageServer* data, ChangeFeedStreamRequest req) { state Span span("SS:getChangeFeedStream"_loc, { req.spanContext }); state bool atLatest = false; @@ -4637,6 +4660,7 @@ void changeServerKeys(StorageServer* data, break; } } + if (!foundAssigned) { Key beginClearKey = f.first.withPrefix(persistChangeFeedKeys.begin); auto& mLV = data->addVersionToMutationLog(data->data().getLatestVersion()); @@ -4658,10 +4682,17 @@ void changeServerKeys(StorageServer* data, data->keyChangeFeed.coalesce(f.second.contents()); auto feed = data->uidChangeFeed.find(f.first); if (feed != data->uidChangeFeed.end()) { + feed->second->moved(); feed->second->removing = true; feed->second->newMutations.trigger(); data->uidChangeFeed.erase(feed); } + } else { + // if just part of feed is moved away, + auto feed = data->uidChangeFeed.find(f.first); + if (feed != data->uidChangeFeed.end()) { + feed->second->moved(); + } } } } @@ -6547,7 +6578,8 @@ ACTOR Future serveChangeFeedStreamRequests(StorageServer* self, FutureStream changeFeedStream) { loop { ChangeFeedStreamRequest req = waitNext(changeFeedStream); - self->actors.add(changeFeedStreamQ(self, req)); + // must notify change feed that its shard is moved away ASAP + self->actors.add(changeFeedStreamQ(self, req) || stopChangeFeedOnMove(self, req)); } } diff --git a/flow/flow.h b/flow/flow.h index 27b08db274..a9377d9d03 100644 --- a/flow/flow.h +++ b/flow/flow.h @@ -962,13 +962,16 @@ struct NotifiedQueue : private SingleCallback, FastAllocated std::queue> queue; Promise onEmpty; Error error; + Promise onError; - NotifiedQueue(int futures, int promises) : promises(promises), futures(futures), onEmpty(nullptr) { + NotifiedQueue(int futures, int promises) + : promises(promises), futures(futures), onEmpty(nullptr), onError(nullptr) { SingleCallback::next = this; } bool isReady() const { return !queue.empty() || error.isValid(); } bool isError() const { return queue.empty() && error.isValid(); } // the *next* thing queued is an error + bool hasError() const { return error.isValid(); } // there is an error queued uint32_t size() const { return queue.size(); } virtual T pop() { @@ -1007,6 +1010,11 @@ struct NotifiedQueue : private SingleCallback, FastAllocated if (shouldFireImmediately()) { SingleCallback::next->error(err); } + + // end_of_stream error is "expected", don't terminate reading stream early for this + if (onError.isValid() && err.code() != error_code_end_of_stream) { + onError.sendError(err); + } } void addPromiseRef() { promises++; } @@ -1060,8 +1068,6 @@ protected: return copy; } - bool hasError() { return error.isValid(); } - bool shouldFireImmediately() { return SingleCallback::next != this; } }; From 738b72918a6542354ac2b99fa39ef8e041ffb105 Mon Sep 17 00:00:00 2001 From: Josh Slocum Date: Tue, 21 Dec 2021 16:32:56 -0600 Subject: [PATCH 098/413] send merge cursor's buffered data if all sub-streams have something at this version --- fdbclient/NativeAPI.actor.cpp | 39 +++++++++++++++++++++++++++++++++- fdbserver/BlobWorker.actor.cpp | 3 +-- 2 files changed, 39 insertions(+), 3 deletions(-) diff --git a/fdbclient/NativeAPI.actor.cpp b/fdbclient/NativeAPI.actor.cpp index a97841f1b6..0b04a0efb6 100644 --- a/fdbclient/NativeAPI.actor.cpp +++ b/fdbclient/NativeAPI.actor.cpp @@ -7360,12 +7360,16 @@ ACTOR Future doCFMerge(Reference results, } interfNum++; } + state int atCheckVersion = 0; state Version checkVersion = invalidVersion; state Standalone> nextOut; while (mutations.size()) { state MutationAndVersionStream nextStream = mutations.top(); mutations.pop(); ASSERT(nextStream.next.version >= checkVersion); + if (nextStream.next.version == checkVersion) { + ASSERT(atCheckVersion > 0); + } if (nextStream.next.version != checkVersion) { if (nextOut.size()) { *begin = checkVersion + 1; @@ -7394,6 +7398,7 @@ ACTOR Future doCFMerge(Reference results, nextOut = Standalone>(); } checkVersion = nextStream.next.version; + atCheckVersion = 0; } if (nextOut.size() && nextStream.next.version == nextOut.back().version) { if (nextStream.next.mutations.size() && @@ -7404,9 +7409,41 @@ ACTOR Future doCFMerge(Reference results, } else { nextOut.push_back_deep(nextOut.arena(), nextStream.next); } + atCheckVersion++; + + // TODO AVOID CODE DUPLICATION + // If all streams have returned something at this version, we know it is complete. + if (atCheckVersion == mutations.size() + 1) { + ASSERT(nextOut.size() == 1); + *begin = checkVersion + 1; + if (DEBUG_CF_VERSION(nextOut.back().version)) { + fmt::print("CFNA (merged@all): {0} (1)\n", nextOut.back().version); + } + + if (nextOut.back().version < results->lastReturnedVersion.get()) { + printf("ERROR: merge cursor@all pushing next out <= lastReturnedVersion"); + } + // We can get an empty version pushed through the stream if whenAtLeast is called. Ignore + // it + + if (!nextOut.back().mutations.empty()) { + ASSERT(nextOut.back().version >= results->lastReturnedVersion.get()); + results->mutations.send(nextOut); + wait(results->mutations.onEmpty()); + wait(delay(0)); + } + if (DEBUG_CF_VERSION(nextOut.back().version)) { + fmt::print("CFLR (merged@all): {0} (1)\n", nextOut.back().version); + } + if (nextOut.back().version > results->lastReturnedVersion.get()) { + results->lastReturnedVersion.set(nextOut.back().version); + } + nextOut = Standalone>(); + atCheckVersion = 0; + } try { Standalone res = waitNext(nextStream.results.getFuture()); - if (DEBUG_CF_VERSION(nextOut.back().version)) { + if (DEBUG_CF_VERSION(res.version)) { fmt::print(" CFNA (merge1): {0} (1)\n", res.version, res.mutations.size()); } nextStream.next = res; diff --git a/fdbserver/BlobWorker.actor.cpp b/fdbserver/BlobWorker.actor.cpp index 680fd1822e..04204d6ef1 100644 --- a/fdbserver/BlobWorker.actor.cpp +++ b/fdbserver/BlobWorker.actor.cpp @@ -716,6 +716,7 @@ ACTOR Future dumpInitialSnapshotFromFDB(Reference // files might not be the current set of files in metadata, in the case of doing the initial snapshot of a granule that // was split. +// FIXME: only pass metadata->keyRange ACTOR Future compactFromBlob(Reference bwData, Reference metadata, UID granuleID, @@ -728,8 +729,6 @@ ACTOR Future compactFromBlob(Reference bwData, metadata->keyRange.end.printable().c_str()); } - // FIXME: don't use metadata->files - ASSERT(!files.snapshotFiles.empty()); ASSERT(!files.deltaFiles.empty()); From 329a4b1c8557333fd5682779ca43ca6eb877bf1a Mon Sep 17 00:00:00 2001 From: Josh Slocum Date: Wed, 22 Dec 2021 12:09:20 -0600 Subject: [PATCH 099/413] Handling end of stream correctly in merge cursor optimization --- fdbclient/NativeAPI.actor.cpp | 13 ++++++++++++- 1 file changed, 12 insertions(+), 1 deletion(-) diff --git a/fdbclient/NativeAPI.actor.cpp b/fdbclient/NativeAPI.actor.cpp index 0b04a0efb6..3cf8d5961c 100644 --- a/fdbclient/NativeAPI.actor.cpp +++ b/fdbclient/NativeAPI.actor.cpp @@ -7361,6 +7361,7 @@ ACTOR Future doCFMerge(Reference results, interfNum++; } state int atCheckVersion = 0; + state int maxAtCheckVersion = -1; state Version checkVersion = invalidVersion; state Standalone> nextOut; while (mutations.size()) { @@ -7368,8 +7369,13 @@ ACTOR Future doCFMerge(Reference results, mutations.pop(); ASSERT(nextStream.next.version >= checkVersion); if (nextStream.next.version == checkVersion) { + // TODO REMOVE + if (atCheckVersion == 0) { + printf("atCheckVersion == 0 at %lld\n", nextStream.next.version); + } ASSERT(atCheckVersion > 0); } + if (nextStream.next.version != checkVersion) { if (nextOut.size()) { *begin = checkVersion + 1; @@ -7399,6 +7405,11 @@ ACTOR Future doCFMerge(Reference results, } checkVersion = nextStream.next.version; atCheckVersion = 0; + // save this at the start of the "round" to check if all streams have a reply at version checkVersion. If + // so, we can send it early. But because mutations.size() can change if one of the streams gets + // end_of_version on its waitNext, we want that to be reflected in the maxAtCheckVersion for the NEXT + // version, not this one. + maxAtCheckVersion = mutations.size() + 1; } if (nextOut.size() && nextStream.next.version == nextOut.back().version) { if (nextStream.next.mutations.size() && @@ -7413,7 +7424,7 @@ ACTOR Future doCFMerge(Reference results, // TODO AVOID CODE DUPLICATION // If all streams have returned something at this version, we know it is complete. - if (atCheckVersion == mutations.size() + 1) { + if (atCheckVersion == maxAtCheckVersion) { ASSERT(nextOut.size() == 1); *begin = checkVersion + 1; if (DEBUG_CF_VERSION(nextOut.back().version)) { From bc69521a911287ecee3a7befb909dbfe96f37746 Mon Sep 17 00:00:00 2001 From: Josh Slocum Date: Wed, 5 Jan 2022 12:48:53 -0600 Subject: [PATCH 100/413] Several fixes with restarting BW/BM --- fdbclient/NativeAPI.actor.cpp | 10 ++- fdbserver/BlobManager.actor.cpp | 63 +++++++++++++++---- fdbserver/BlobWorker.actor.cpp | 28 +++++---- fdbserver/ClusterController.actor.cpp | 14 ++--- .../workloads/BlobGranuleVerifier.actor.cpp | 22 ++++++- 5 files changed, 104 insertions(+), 33 deletions(-) diff --git a/fdbclient/NativeAPI.actor.cpp b/fdbclient/NativeAPI.actor.cpp index 3cf8d5961c..9aba7a8892 100644 --- a/fdbclient/NativeAPI.actor.cpp +++ b/fdbclient/NativeAPI.actor.cpp @@ -6508,7 +6508,7 @@ Future>> Transaction::getRangeSplitPoints(KeyRange return ::getRangeSplitPoints(cx, keys, chunkSize); } -#define BG_REQUEST_DEBUG false +#define BG_REQUEST_DEBUG true // the blob granule requests are a bit funky because they piggyback off the existing transaction to read from the system // keyspace @@ -6619,6 +6619,14 @@ ACTOR Future>> readBlobGranulesActor( } workerId = decodeBlobGranuleMappingValue(blobGranuleMapping[i].value); + if (workerId == UID()) { + if (BG_REQUEST_DEBUG) { + printf("Key range [%s - %s) has no assigned worker yet!\n", + granuleStartKey.printable().c_str(), + granuleEndKey.printable().c_str()); + } + throw transaction_too_old(); + } if (BG_REQUEST_DEBUG) { printf(" [%s - %s): %s\n", granuleStartKey.printable().c_str(), diff --git a/fdbserver/BlobManager.actor.cpp b/fdbserver/BlobManager.actor.cpp index 626823613a..d577206513 100644 --- a/fdbserver/BlobManager.actor.cpp +++ b/fdbserver/BlobManager.actor.cpp @@ -286,6 +286,10 @@ ACTOR Future>> splitRange(Reference pickWorkerForAssign(BlobManagerData* bmData) { // wait until there are BWs to pick from while (bmData->workerStats.size() == 0) { + // TODO REMOVE + if (BM_DEBUG) { + printf("BM waiting for blob workers before assigning granules\n"); + } bmData->restartRecruiting.trigger(); wait(bmData->recruitingStream.onChange()); } @@ -523,6 +527,37 @@ ACTOR Future checkManagerLock(Reference tr, Blo return Void(); } +ACTOR Future writeInitialGranuleMapping(BlobManagerData* bmData, Standalone> boundaries) { + state Reference tr = makeReference(bmData->db); + // don't do too many in one transaction + state int i = 0; + state int transactionChunkSize = BUGGIFY ? deterministicRandom()->randomInt(2, 5) : 1000; + while (i < boundaries.size() - 1) { + TEST(i > 0); // multiple transactions for large granule split + tr->reset(); + tr->setOption(FDBTransactionOptions::Option::PRIORITY_SYSTEM_IMMEDIATE); + tr->setOption(FDBTransactionOptions::Option::ACCESS_SYSTEM_KEYS); + state int j = 0; + while (i + j < boundaries.size() - 1 && j < transactionChunkSize) { + // TODO REMOVE + if (BM_DEBUG) { + printf("Persisting initial mapping for [%s - %s)\n", + boundaries[i + j].printable().c_str(), + boundaries[i + j + 1].printable().c_str()); + } + // set to empty UID - no worker assigned yet + wait(krmSetRange(tr, + blobGranuleMappingKeys.begin, + KeyRangeRef(boundaries[i + j], boundaries[i + j + 1]), + blobGranuleMappingValueFor(UID()))); + j++; + } + wait(tr->commit()); + i += j; + } + return Void(); +} + // FIXME: this does all logic in one transaction. Adding a giant range to an existing database to blobify would // require doing a ton of storage metrics calls, which we should split up across multiple transactions likely. ACTOR Future monitorClientRanges(BlobManagerData* bmData) { @@ -569,7 +604,7 @@ ACTOR Future monitorClientRanges(BlobManagerData* bmData) { } for (auto f : splitFutures) { - Standalone> splits = wait(f); + state Standalone> splits = wait(f); if (BM_DEBUG) { printf("Split client range [%s - %s) into %d ranges:\n", splits[0].printable().c_str(), @@ -577,21 +612,22 @@ ACTOR Future monitorClientRanges(BlobManagerData* bmData) { splits.size() - 1); } + // Write to DB BEFORE sending assign requests, so that if manager dies before/during, new manager + // picks up the same ranges + wait(writeInitialGranuleMapping(bmData, splits)); + for (int i = 0; i < splits.size() - 1; i++) { KeyRange range = KeyRange(KeyRangeRef(splits[i], splits[i + 1])); // only add the client range if this is the first BM or it's not already assigned - if (bmData->epoch == 1 || bmData->workerAssignments.intersectingRanges(range).empty()) { - if (BM_DEBUG) { - printf( - " [%s - %s)\n", range.begin.printable().c_str(), range.end.printable().c_str()); - } - - RangeAssignment ra; - ra.isAssign = true; - ra.keyRange = range; - ra.assign = RangeAssignmentData(); // type=normal - bmData->rangesToAssign.send(ra); + if (BM_DEBUG) { + printf(" [%s - %s)\n", range.begin.printable().c_str(), range.end.printable().c_str()); } + + RangeAssignment ra; + ra.isAssign = true; + ra.keyRange = range; + ra.assign = RangeAssignmentData(); // type=normal + bmData->rangesToAssign.send(ra); } } @@ -1086,6 +1122,7 @@ ACTOR Future recoverBlobManager(BlobManagerData* bmData) { // note: if the old owner is dead, we handle this in rangeAssigner UID existingOwner = decodeBlobGranuleMappingValue(results[rangeIdx].value); workerAssignments.insert(KeyRangeRef(granuleStartKey, granuleEndKey), existingOwner); + bmData->knownBlobRanges.insert(KeyRangeRef(granuleStartKey, granuleEndKey), true); } } @@ -1138,6 +1175,8 @@ ACTOR Future recoverBlobManager(BlobManagerData* bmData) { } } + bmData->knownBlobRanges.coalesce(normalKeys); + // Step 3. Send assign requests for all the granules and transfer assignments // from local workerAssignments to bmData for (auto& range : workerAssignments.intersectingRanges(normalKeys)) { diff --git a/fdbserver/BlobWorker.actor.cpp b/fdbserver/BlobWorker.actor.cpp index 04204d6ef1..a362ca25f8 100644 --- a/fdbserver/BlobWorker.actor.cpp +++ b/fdbserver/BlobWorker.actor.cpp @@ -1162,9 +1162,11 @@ ACTOR Future blobGranuleUpdateFiles(Reference bwData, } if (BW_DEBUG) { - fmt::print("Granule File Updater Starting for [{0} - {1}):\n", + fmt::print("Granule File Updater Starting for [{0} - {1}) @ ({2}, {3}):\n", metadata->keyRange.begin.printable(), - metadata->keyRange.end.printable()); + metadata->keyRange.end.printable(), + metadata->originalEpoch, + metadata->originalSeqno); fmt::print(" CFID: {}\n", startState.granuleID.toString()); fmt::print(" CF Start Version: {}\n", startState.changeFeedStartVersion); fmt::print(" Previous Durable Version: {}\n", startState.previousDurableVersion); @@ -2654,11 +2656,13 @@ ACTOR Future handleRangeAssign(Reference bwData, return Void(); } catch (Error& e) { if (BW_DEBUG) { - printf("AssignRange [%s - %s) in BW %s got error %s\n", - req.keyRange.begin.printable().c_str(), - req.keyRange.end.printable().c_str(), - bwData->id.toString().c_str(), - e.name()); + fmt::print("AssignRange [{0} - {1}) ({2}, {3}) in BW {4} got error {5}\n", + req.keyRange.begin.printable().c_str(), + req.keyRange.end.printable().c_str(), + req.managerEpoch, + req.managerSeqno, + bwData->id.toString().c_str(), + e.name()); } if (!isSelfReassign) { @@ -2687,10 +2691,12 @@ ACTOR Future handleRangeRevoke(Reference bwData, RevokeBlo } catch (Error& e) { // FIXME: retry on error if dispose fails? if (BW_DEBUG) { - printf("RevokeRange [%s - %s) got error %s\n", - req.keyRange.begin.printable().c_str(), - req.keyRange.end.printable().c_str(), - e.name()); + fmt::print("RevokeRange [{0} - {1}) ({2}, {3}) got error {4}\n", + req.keyRange.begin.printable(), + req.keyRange.end.printable(), + req.managerEpoch, + req.managerSeqno, + e.name()); } if (canReplyWith(e)) { req.reply.sendError(e); diff --git a/fdbserver/ClusterController.actor.cpp b/fdbserver/ClusterController.actor.cpp index 039df63bbe..16aaa0fbfc 100644 --- a/fdbserver/ClusterController.actor.cpp +++ b/fdbserver/ClusterController.actor.cpp @@ -5368,18 +5368,18 @@ ACTOR Future watchBlobGranulesConfigKey(ClusterControllerData* self) { tr->setOption(FDBTransactionOptions::ACCESS_SYSTEM_KEYS); tr->setOption(FDBTransactionOptions::PRIORITY_SYSTEM_IMMEDIATE); - state Future watch = tr->watch(blobGranuleConfigKey); - wait(tr->commit()); - wait(watch); + Optional blobConfig = wait(tr->get(blobGranuleConfigKey)); + if (blobConfig.present()) { + self->db.blobGranulesEnabled.set(blobConfig.get() == LiteralStringRef("1")); + } tr->reset(); tr->setOption(FDBTransactionOptions::ACCESS_SYSTEM_KEYS); tr->setOption(FDBTransactionOptions::PRIORITY_SYSTEM_IMMEDIATE); - Optional blobConfig = wait(tr->get(blobGranuleConfigKey)); - if (blobConfig.present()) { - self->db.blobGranulesEnabled.set(blobConfig.get() == LiteralStringRef("1")); - } + state Future watch = tr->watch(blobGranuleConfigKey); + wait(tr->commit()); + wait(watch); } catch (Error& e) { wait(tr->onError(e)); } diff --git a/fdbserver/workloads/BlobGranuleVerifier.actor.cpp b/fdbserver/workloads/BlobGranuleVerifier.actor.cpp index 901a98c581..31459ad929 100644 --- a/fdbserver/workloads/BlobGranuleVerifier.actor.cpp +++ b/fdbserver/workloads/BlobGranuleVerifier.actor.cpp @@ -448,7 +448,25 @@ struct BlobGranuleVerifierWorkload : TestWorkload { state KeyRange last; state bool availabilityPassed = true; - state Standalone> allRanges = self->granuleRanges.get(); + + state Standalone> allRanges; + if (self->granuleRanges.get().empty()) { + if (BGV_DEBUG) { + fmt::print("Waiting to get granule ranges for check\n"); + } + state Future rangeFetcher = self->findGranules(cx, self); + loop { + wait(self->granuleRanges.onChange()); + if (!self->granuleRanges.get().empty()) { + break; + } + } + rangeFetcher.cancel(); + if (BGV_DEBUG) { + fmt::print("Got granule ranges for check\n"); + } + } + allRanges = self->granuleRanges.get(); for (auto& range : allRanges) { state KeyRange r = range; if (BGV_DEBUG) { @@ -502,7 +520,7 @@ struct BlobGranuleVerifierWorkload : TestWorkload { fmt::print(" {} bytes\n", self->bytesRead); // FIXME: add above as details TraceEvent("BlobGranuleVerifierChecked"); - return availabilityPassed && self->mismatches == 0 && checks > 0 && self->timeTravelTooOld == 0; + return availabilityPassed && self->mismatches == 0 && (checks > 0) && (self->timeTravelTooOld == 0); } Future check(Database const& cx) override { return _check(cx, this); } From a96163d9d347ab8755d3187fc10a5036a257f818 Mon Sep 17 00:00:00 2001 From: Josh Slocum Date: Wed, 5 Jan 2022 09:37:55 -0600 Subject: [PATCH 101/413] Fixed ASAN issues --- fdbserver/BlobWorker.actor.cpp | 7 ++++--- flow/flow.h | 6 +++++- 2 files changed, 9 insertions(+), 4 deletions(-) diff --git a/fdbserver/BlobWorker.actor.cpp b/fdbserver/BlobWorker.actor.cpp index a362ca25f8..5b882c5959 100644 --- a/fdbserver/BlobWorker.actor.cpp +++ b/fdbserver/BlobWorker.actor.cpp @@ -1086,8 +1086,9 @@ ACTOR Future waitVersionCommitted(Reference bwData, // this order is important, since we need to register a waiter on the notified version before waking the GRV // actor Future grvAtLeast = bwData->grvVersion.whenAtLeast(version); - if (bwData->doGRVCheck.canBeSet()) { - bwData->doGRVCheck.send(Void()); + Promise doGrvCheck = bwData->doGRVCheck; + if (doGrvCheck.canBeSet()) { + doGrvCheck.send(Void()); } wait(grvAtLeast); } @@ -2742,7 +2743,7 @@ ACTOR Future runGRVChecks(Reference bwData) { while (bwData->grvVersion.numWaiting() == 0) { // printf("GRV checker sleeping\n"); wait(bwData->doGRVCheck.getFuture()); - bwData->doGRVCheck.reset(); + bwData->doGRVCheck = Promise(); // printf("GRV checker waking: %d pending\n", bwData->grvVersion.numWaiting()); } diff --git a/flow/flow.h b/flow/flow.h index a9377d9d03..69eb25d5ec 100644 --- a/flow/flow.h +++ b/flow/flow.h @@ -1006,13 +1006,17 @@ struct NotifiedQueue : private SingleCallback, FastAllocated if (error.isValid()) return; + ASSERT(this->error.code() != error_code_success); this->error = err; if (shouldFireImmediately()) { SingleCallback::next->error(err); } // end_of_stream error is "expected", don't terminate reading stream early for this - if (onError.isValid() && err.code() != error_code_end_of_stream) { + // weird destruction issue with sending broken_promise, so just ignore it. Stream is dead anyway. TODO better + // fix? + if (err.code() != error_code_end_of_stream && err.code() != error_code_broken_promise && onError.isValid()) { + ASSERT(onError.canBeSet()); onError.sendError(err); } } From 0f66cca8e0eb3037f5d52998f2596e4a1c96a884 Mon Sep 17 00:00:00 2001 From: Josh Slocum Date: Wed, 5 Jan 2022 16:40:07 -0600 Subject: [PATCH 102/413] Fixing change feed race with empty mutation and error --- fdbclient/NativeAPI.actor.cpp | 58 ++++++++++++++++++++++++++-------- fdbserver/BlobWorker.actor.cpp | 10 ++++++ 2 files changed, 55 insertions(+), 13 deletions(-) diff --git a/fdbclient/NativeAPI.actor.cpp b/fdbclient/NativeAPI.actor.cpp index 9aba7a8892..d12c865eaf 100644 --- a/fdbclient/NativeAPI.actor.cpp +++ b/fdbclient/NativeAPI.actor.cpp @@ -6508,7 +6508,7 @@ Future>> Transaction::getRangeSplitPoints(KeyRange return ::getRangeSplitPoints(cx, keys, chunkSize); } -#define BG_REQUEST_DEBUG true +#define BG_REQUEST_DEBUG false // the blob granule requests are a bit funky because they piggyback off the existing transaction to read from the system // keyspace @@ -7356,6 +7356,11 @@ ACTOR Future doCFMerge(Reference results, Version end) { state std::priority_queue> mutations; state int interfNum = 0; + + // previous version of change feed may have put a mutation in the promise stream and then immediately died. Wait for + // that mutation first, so the promise stream always starts empty + wait(results->mutations.onEmpty()); + while (interfNum < interfs.size()) { try { Standalone res = waitNext(streams[interfNum].results.getFuture()); @@ -7388,7 +7393,7 @@ ACTOR Future doCFMerge(Reference results, if (nextOut.size()) { *begin = checkVersion + 1; if (DEBUG_CF_VERSION(nextOut.back().version)) { - fmt::print("CFNA (merged): {0} (1)\n", nextOut.back().version); + fmt::print("CFNA (merged): {0} ({1})\n", nextOut.back().version, nextOut.back().mutations.size()); } if (nextOut.back().version < results->lastReturnedVersion.get()) { @@ -7402,9 +7407,11 @@ ACTOR Future doCFMerge(Reference results, results->mutations.send(nextOut); wait(results->mutations.onEmpty()); wait(delay(0)); + } else { + ASSERT(results->mutations.isEmpty()); } if (DEBUG_CF_VERSION(nextOut.back().version)) { - fmt::print("CFLR (merged): {0} (1)\n", nextOut.back().version); + fmt::print("CFLR (merged): {0} (1)\n", nextOut.back().version, nextOut.back().mutations.size()); } if (nextOut.back().version > results->lastReturnedVersion.get()) { results->lastReturnedVersion.set(nextOut.back().version); @@ -7436,7 +7443,7 @@ ACTOR Future doCFMerge(Reference results, ASSERT(nextOut.size() == 1); *begin = checkVersion + 1; if (DEBUG_CF_VERSION(nextOut.back().version)) { - fmt::print("CFNA (merged@all): {0} (1)\n", nextOut.back().version); + fmt::print("CFNA (merged@all): {0} ({1})\n", nextOut.back().version, nextOut.back().mutations.size()); } if (nextOut.back().version < results->lastReturnedVersion.get()) { @@ -7449,10 +7456,13 @@ ACTOR Future doCFMerge(Reference results, ASSERT(nextOut.back().version >= results->lastReturnedVersion.get()); results->mutations.send(nextOut); wait(results->mutations.onEmpty()); + ASSERT(results->mutations.isEmpty()); wait(delay(0)); + } else { + ASSERT(results->mutations.isEmpty()); } if (DEBUG_CF_VERSION(nextOut.back().version)) { - fmt::print("CFLR (merged@all): {0} (1)\n", nextOut.back().version); + fmt::print("CFLR (merged@all): {0} ({1})\n", nextOut.back().version, nextOut.back().mutations.size()); } if (nextOut.back().version > results->lastReturnedVersion.get()) { results->lastReturnedVersion.set(nextOut.back().version); @@ -7463,7 +7473,7 @@ ACTOR Future doCFMerge(Reference results, try { Standalone res = waitNext(nextStream.results.getFuture()); if (DEBUG_CF_VERSION(res.version)) { - fmt::print(" CFNA (merge1): {0} (1)\n", res.version, res.mutations.size()); + fmt::print(" CFNA (merge1): {0} ({1})\n", res.version, res.mutations.size()); } nextStream.next = res; mutations.push(nextStream); @@ -7475,9 +7485,12 @@ ACTOR Future doCFMerge(Reference results, } if (nextOut.size()) { ASSERT(nextOut.back().version >= results->lastReturnedVersion.get()); - results->mutations.send(nextOut); - wait(results->mutations.onEmpty()); - wait(delay(0)); + if (!nextOut.back().mutations.empty()) { + results->mutations.send(nextOut); + wait(results->mutations.onEmpty()); + ASSERT(results->mutations.isEmpty()); + wait(delay(0)); + } if (DEBUG_CF_VERSION(nextOut.back().version)) { fmt::print("CFLR (merged): {0} (1)\n", nextOut.back().version); } @@ -7601,11 +7614,23 @@ ACTOR Future doSingleCFStream(KeyRange range, results->lastReturnedVersion.get()); } ASSERT(feedReply.mutations.back().version >= results->lastReturnedVersion.get()); - results->mutations.send(Standalone>(feedReply.mutations, feedReply.arena)); - // Because onEmpty returns here before the consuming process, we must do a delay(0) - wait(results->mutations.onEmpty()); - wait(delay(0)); + // don't send completely empty set of mutations to promise stream + bool anyMutations = false; + for (auto& it : feedReply.mutations) { + if (!it.mutations.empty()) { + anyMutations = true; + break; + } + } + if (anyMutations) { + results->mutations.send( + Standalone>(feedReply.mutations, feedReply.arena)); + + // Because onEmpty returns here before the consuming process, we must do a delay(0) + wait(results->mutations.onEmpty()); + wait(delay(0)); + } if (DEBUG_CF_VERSION(feedReply.mutations.back().version)) { fmt::print("CFLR (single): {0} ({1}), atLatest={2}, rep.atLatest={3}, notAtLatest={4}, " @@ -7680,6 +7705,13 @@ ACTOR Future getChangeFeedStreamActor(Reference db, state Database cx(db); state Span span("NAPI:GetChangeFeedStream"_loc); + /*printf("CFStream %s [%s - %s): [%lld - %lld]\n", + rangeID.printable().substr(0, 6).c_str(), + range.begin.printable().c_str(), + range.end.printable().c_str(), + begin, + end);*/ + loop { state KeyRange keys; try { diff --git a/fdbserver/BlobWorker.actor.cpp b/fdbserver/BlobWorker.actor.cpp index 5b882c5959..3422aa5e23 100644 --- a/fdbserver/BlobWorker.actor.cpp +++ b/fdbserver/BlobWorker.actor.cpp @@ -1313,6 +1313,16 @@ ACTOR Future blobGranuleUpdateFiles(Reference bwData, ASSERT(mutations.front().version >= startState.changeFeedStartVersion); } + if (mutations.front().version <= metadata->bufferedDeltaVersion) { + fmt::print("ERROR: Mutations went backwards for granule [{0} - {1}). " + "bufferedDeltaVersion={2}, mutationVersion={3} !!!\n", + metadata->keyRange.begin.printable(), + metadata->keyRange.end.printable(), + metadata->bufferedDeltaVersion, + mutations.front().version); + } + ASSERT(mutations.front().version > metadata->bufferedDeltaVersion); + // if we just got mutations, we haven't buffered them yet, so waitForVersion can't have returned // this version yet if (mutations.front().version <= metadata->waitForVersionReturned) { From 2438fee519a6ef38400a1d9a56d2520be417755a Mon Sep 17 00:00:00 2001 From: Josh Slocum Date: Wed, 5 Jan 2022 14:46:01 -0600 Subject: [PATCH 103/413] Made client range watching handle very long/large transactions --- fdbserver/BlobManager.actor.cpp | 64 ++++++++++++++++++++++++--------- 1 file changed, 47 insertions(+), 17 deletions(-) diff --git a/fdbserver/BlobManager.actor.cpp b/fdbserver/BlobManager.actor.cpp index d577206513..fd0e6a6d26 100644 --- a/fdbserver/BlobManager.actor.cpp +++ b/fdbserver/BlobManager.actor.cpp @@ -538,21 +538,30 @@ ACTOR Future writeInitialGranuleMapping(BlobManagerData* bmData, Standalon tr->setOption(FDBTransactionOptions::Option::PRIORITY_SYSTEM_IMMEDIATE); tr->setOption(FDBTransactionOptions::Option::ACCESS_SYSTEM_KEYS); state int j = 0; - while (i + j < boundaries.size() - 1 && j < transactionChunkSize) { - // TODO REMOVE - if (BM_DEBUG) { - printf("Persisting initial mapping for [%s - %s)\n", - boundaries[i + j].printable().c_str(), - boundaries[i + j + 1].printable().c_str()); + loop { + try { + + while (i + j < boundaries.size() - 1 && j < transactionChunkSize) { + // TODO REMOVE + if (BM_DEBUG) { + printf("Persisting initial mapping for [%s - %s)\n", + boundaries[i + j].printable().c_str(), + boundaries[i + j + 1].printable().c_str()); + } + // set to empty UID - no worker assigned yet + wait(krmSetRange(tr, + blobGranuleMappingKeys.begin, + KeyRangeRef(boundaries[i + j], boundaries[i + j + 1]), + blobGranuleMappingValueFor(UID()))); + j++; + } + wait(tr->commit()); + break; + } catch (Error& e) { + wait(tr->onError(e)); + j = 0; } - // set to empty UID - no worker assigned yet - wait(krmSetRange(tr, - blobGranuleMappingKeys.begin, - KeyRangeRef(boundaries[i + j], boundaries[i + j + 1]), - blobGranuleMappingValueFor(UID()))); - j++; } - wait(tr->commit()); i += j; } return Void(); @@ -561,6 +570,7 @@ ACTOR Future writeInitialGranuleMapping(BlobManagerData* bmData, Standalon // FIXME: this does all logic in one transaction. Adding a giant range to an existing database to blobify would // require doing a ton of storage metrics calls, which we should split up across multiple transactions likely. ACTOR Future monitorClientRanges(BlobManagerData* bmData) { + state Optional lastChangeKeyValue; loop { state Reference tr = makeReference(bmData->db); @@ -572,6 +582,9 @@ ACTOR Future monitorClientRanges(BlobManagerData* bmData) { tr->setOption(FDBTransactionOptions::ACCESS_SYSTEM_KEYS); tr->setOption(FDBTransactionOptions::PRIORITY_SYSTEM_IMMEDIATE); + // read change key at this point along with ranges + state Optional ckvBegin = wait(tr->get(blobRangeChangeKey)); + // TODO probably knobs here? This should always be pretty small though RangeResult results = wait(krmGetRanges( tr, blobRangeKeys.begin, KeyRange(normalKeys), 10000, GetRangeLimits::BYTE_LIMIT_UNLIMITED)); @@ -629,13 +642,30 @@ ACTOR Future monitorClientRanges(BlobManagerData* bmData) { ra.assign = RangeAssignmentData(); // type=normal bmData->rangesToAssign.send(ra); } + wait(bmData->rangesToAssign.onEmpty()); } - state Future watchFuture = tr->watch(blobRangeChangeKey); - wait(tr->commit()); - if (BM_DEBUG) { - printf("Blob manager done processing client ranges, awaiting update\n"); + lastChangeKeyValue = + ckvBegin; // the version of the ranges we processed is the one read alongside the ranges + + // do a new transaction, check for change in change key, watch if none + tr->reset(); + tr->setOption(FDBTransactionOptions::ACCESS_SYSTEM_KEYS); + tr->setOption(FDBTransactionOptions::PRIORITY_SYSTEM_IMMEDIATE); + state Future watchFuture; + + Optional ckvEnd = wait(tr->get(blobRangeChangeKey)); + + if (ckvEnd == lastChangeKeyValue) { + watchFuture = tr->watch(blobRangeChangeKey); // watch for change in key + wait(tr->commit()); + if (BM_DEBUG) { + printf("Blob manager done processing client ranges, awaiting update\n"); + } + } else { + watchFuture = Future(Void()); // restart immediately } + wait(watchFuture); break; } catch (Error& e) { From 807ec37e848a23bb824e2ac94f503301cef24689 Mon Sep 17 00:00:00 2001 From: Josh Slocum Date: Wed, 5 Jan 2022 17:38:35 -0600 Subject: [PATCH 104/413] BG request retry on failed blob worker properly --- fdbclient/NativeAPI.actor.cpp | 102 +++++++++++------- .../workloads/BlobGranuleVerifier.actor.cpp | 14 ++- flow/error_definitions.h | 1 + 3 files changed, 74 insertions(+), 43 deletions(-) diff --git a/fdbclient/NativeAPI.actor.cpp b/fdbclient/NativeAPI.actor.cpp index d12c865eaf..4ef181926e 100644 --- a/fdbclient/NativeAPI.actor.cpp +++ b/fdbclient/NativeAPI.actor.cpp @@ -6576,7 +6576,6 @@ ACTOR Future>> readBlobGranulesActor( rv = _end; } self->setOption(FDBTransactionOptions::ACCESS_SYSTEM_KEYS); - // Right now just read whole blob range assignments from DB // FIXME: eventually we probably want to cache this and invalidate similarly to storage servers. // Cache misses could still read from the DB, or we could add it to the Transaction State Store and @@ -6597,7 +6596,7 @@ ACTOR Future>> readBlobGranulesActor( if (BG_REQUEST_DEBUG) { printf("no blob worker assignments yet\n"); } - throw transaction_too_old(); + throw blob_granule_transaction_too_old(); } if (BG_REQUEST_DEBUG) { @@ -6615,7 +6614,7 @@ ACTOR Future>> readBlobGranulesActor( granuleEndKey.printable().c_str()); // TODO probably new exception type instead } - throw transaction_too_old(); + throw blob_granule_transaction_too_old(); } workerId = decodeBlobGranuleMappingValue(blobGranuleMapping[i].value); @@ -6625,7 +6624,7 @@ ACTOR Future>> readBlobGranulesActor( granuleStartKey.printable().c_str(), granuleEndKey.printable().c_str()); } - throw transaction_too_old(); + throw blob_granule_transaction_too_old(); } if (BG_REQUEST_DEBUG) { printf(" [%s - %s): %s\n", @@ -6640,9 +6639,12 @@ ACTOR Future>> readBlobGranulesActor( // could have died and so its interface wouldn't be present as part of the blobWorkerList // we persist in the db. So throw wrong_shard_server to get the new mapping if (!workerInterface.present()) { - throw wrong_shard_server(); + // need to re-read mapping, throw transaction_too_old so client retries. TODO better error? + // throw wrong_shard_server(); + throw transaction_too_old(); } - // FIXME: maybe just want to insert here if there are racing queries for the same worker or something? + // FIXME: maybe just want to insert here if there are racing queries for the same worker or + // something? cx->blobWorker_interf[workerId] = decodeBlobWorkerListValue(workerInterface.get()); if (BG_REQUEST_DEBUG) { printf(" decoded worker interface for %s\n", workerId.toString().c_str()); @@ -6677,42 +6679,64 @@ ACTOR Future>> readBlobGranulesActor( state Reference>> location = makeReference(v); // use load balance with one option for now for retry and error handling - BlobGranuleFileReply rep = wait(loadBalance(location, - &BlobWorkerInterface::blobGranuleFileRequest, - req, - TaskPriority::DefaultPromiseEndpoint, - AtMostOnce::False, - nullptr)); + try { + choose { + when(BlobGranuleFileReply rep = wait(loadBalance(location, + &BlobWorkerInterface::blobGranuleFileRequest, + req, + TaskPriority::DefaultPromiseEndpoint, + AtMostOnce::False, + nullptr))) { + if (BG_REQUEST_DEBUG) { + fmt::print("Blob granule request for [{0} - {1}) @ {2} - {3} got reply from {4}:\n", + granuleStartKey.printable(), + granuleEndKey.printable(), + begin, + rv, + workerId.toString()); + } + results.arena().dependsOn(rep.arena); + for (auto& chunk : rep.chunks) { + if (BG_REQUEST_DEBUG) { + fmt::print( + "[{0} - {1})\n", chunk.keyRange.begin.printable(), chunk.keyRange.end.printable()); - if (BG_REQUEST_DEBUG) { - fmt::print("Blob granule request for [{0} - {1}) @ {2} - {3} got reply from {4}:\n", - granuleStartKey.printable(), - granuleEndKey.printable(), - begin, - rv, - workerId.toString()); - } - results.arena().dependsOn(rep.arena); - for (auto& chunk : rep.chunks) { - if (BG_REQUEST_DEBUG) { - fmt::print("[{0} - {1})\n", chunk.keyRange.begin.printable(), chunk.keyRange.end.printable()); + fmt::print(" SnapshotFile: {0}\n \n DeltaFiles:\n", + chunk.snapshotFile.present() ? chunk.snapshotFile.get().toString().c_str() + : ""); + for (auto& df : chunk.deltaFiles) { + fmt::print(" {0}\n", df.toString()); + } + fmt::print(" Deltas: ({0})", chunk.newDeltas.size()); + if (chunk.newDeltas.size() > 0) { + fmt::print(" with version [{0} - {1}]", + chunk.newDeltas[0].version, + chunk.newDeltas[chunk.newDeltas.size() - 1].version); + } + fmt::print(" IncludedVersion: {0}\n\n\n", chunk.includedVersion); + } - fmt::print(" SnapshotFile: {0}\n \n DeltaFiles:\n", - chunk.snapshotFile.present() ? chunk.snapshotFile.get().toString().c_str() : ""); - for (auto& df : chunk.deltaFiles) { - fmt::print(" {0}\n", df.toString()); + results.push_back(results.arena(), chunk); + keyRange = KeyRangeRef(std::min(chunk.keyRange.end, keyRange.end), keyRange.end); + } } - fmt::print(" Deltas: ({0})", chunk.newDeltas.size()); - if (chunk.newDeltas.size() > 0) { - fmt::print(" with version [{0} - {1}]", - chunk.newDeltas[0].version, - chunk.newDeltas[chunk.newDeltas.size() - 1].version); + // if we detect that this blob worker fails, cancel the request, as otherwise load balance will + // retry indefinitely with one option + when(wait(IFailureMonitor::failureMonitor().onStateEqual( + location->get(i, &BlobWorkerInterface::blobGranuleFileRequest).getEndpoint(), + FailureStatus(true)))) { + printf("readBlobGranules got BW %s failed\n", workerId.toString().c_str()); + + throw connection_failed(); } - fmt::print(" IncludedVersion: {0}\n\n\n", chunk.includedVersion); } - - results.push_back(results.arena(), chunk); - keyRange = KeyRangeRef(std::min(chunk.keyRange.end, keyRange.end), keyRange.end); + } catch (Error& e) { + // worker is up but didn't actually have granule, or connection failed + if (e.code() == error_code_wrong_shard_server || e.code() == error_code_connection_failed) { + // need to re-read mapping, throw transaction_too_old so client retries. TODO better error? + throw transaction_too_old(); + } + throw e; } } if (readVersionOut != nullptr) { @@ -7420,8 +7444,8 @@ ACTOR Future doCFMerge(Reference results, } checkVersion = nextStream.next.version; atCheckVersion = 0; - // save this at the start of the "round" to check if all streams have a reply at version checkVersion. If - // so, we can send it early. But because mutations.size() can change if one of the streams gets + // save this at the start of the "round" to check if all streams have a reply at version checkVersion. + // If so, we can send it early. But because mutations.size() can change if one of the streams gets // end_of_version on its waitNext, we want that to be reflected in the maxAtCheckVersion for the NEXT // version, not this one. maxAtCheckVersion = mutations.size() + 1; diff --git a/fdbserver/workloads/BlobGranuleVerifier.actor.cpp b/fdbserver/workloads/BlobGranuleVerifier.actor.cpp index 31459ad929..b57ca10ac7 100644 --- a/fdbserver/workloads/BlobGranuleVerifier.actor.cpp +++ b/fdbserver/workloads/BlobGranuleVerifier.actor.cpp @@ -508,7 +508,8 @@ struct BlobGranuleVerifierWorkload : TestWorkload { break; } } - fmt::print("Blob Granule Verifier finished with:\n"); + bool result = availabilityPassed && self->mismatches == 0 && (checks > 0) && (self->timeTravelTooOld == 0); + fmt::print("Blob Granule Verifier {0} {1}:\n", self->clientId, result ? "passed" : "failed"); fmt::print(" {} successful final granule checks\n", checks); fmt::print(" {} failed final granule checks\n", availabilityPassed ? 0 : 1); fmt::print(" {} mismatches\n", self->mismatches); @@ -518,9 +519,14 @@ struct BlobGranuleVerifierWorkload : TestWorkload { fmt::print(" {} time travel reads\n", self->timeTravelReads); fmt::print(" {} rows\n", self->rowsRead); fmt::print(" {} bytes\n", self->bytesRead); - // FIXME: add above as details - TraceEvent("BlobGranuleVerifierChecked"); - return availabilityPassed && self->mismatches == 0 && (checks > 0) && (self->timeTravelTooOld == 0); + // FIXME: add above as details to trace event + + TraceEvent("BlobGranuleVerifierChecked").detail("Result", result); + + // For some reason simulation is still passing when this fails?.. so assert for now + ASSERT(result); + + return result; } Future check(Database const& cx) override { return _check(cx, this); } diff --git a/flow/error_definitions.h b/flow/error_definitions.h index 88def0477e..9b5bffeba9 100755 --- a/flow/error_definitions.h +++ b/flow/error_definitions.h @@ -84,6 +84,7 @@ ERROR( change_feed_not_registered, 1060, "Change feed not registered" ) ERROR( granule_assignment_conflict, 1061, "Conflicting attempts to assign blob granules" ) ERROR( change_feed_cancelled, 1062, "Change feed was cancelled" ) ERROR( blob_granule_file_load_error, 1063, "Error loading a blob file during granule materialization" ) +ERROR( blob_granule_transaction_too_old, 1064, "Read version is older than blob granule history supports" ) ERROR( broken_promise, 1100, "Broken promise" ) ERROR( operation_cancelled, 1101, "Asynchronous operation cancelled" ) From 4d2650f4dc02230fa0bd6705142f4dae1a89a8ea Mon Sep 17 00:00:00 2001 From: Josh Slocum Date: Fri, 7 Jan 2022 11:21:05 -0600 Subject: [PATCH 105/413] Fixed a couple issues with failures and the final availability check --- fdbserver/BlobManager.actor.cpp | 18 +++++++++++---- fdbserver/BlobWorker.actor.cpp | 5 ++-- .../workloads/BlobGranuleVerifier.actor.cpp | 23 +++++++++++++++---- 3 files changed, 34 insertions(+), 12 deletions(-) diff --git a/fdbserver/BlobManager.actor.cpp b/fdbserver/BlobManager.actor.cpp index fd0e6a6d26..5cb2727569 100644 --- a/fdbserver/BlobManager.actor.cpp +++ b/fdbserver/BlobManager.actor.cpp @@ -990,14 +990,17 @@ ACTOR Future monitorBlobWorkerStatus(BlobManagerData* bmData, BlobWorkerIn throw e; } - // TODO: figure out why waitFailure in monitorBlobWorker doesn't pick up the connection failure first - if (e.code() == error_code_connection_failed || e.code() == error_code_broken_promise) { + // on known network errors or stream close errors, throw + if (e.code() == error_code_broken_promise) { throw e; } // if we got an error constructing or reading from stream that is retryable, wait and retry. + // Sometimes we get connection_failed without the failure monitor tripping. One example is simulation's + // rollRandomClose. In this case, just reconstruct the stream. If it was a transient failure, it works, and + // if it is permanent, the failure monitor will eventually trip. ASSERT(e.code() != error_code_end_of_stream); - if (e.code() == error_code_request_maybe_delivered) { + if (e.code() == error_code_request_maybe_delivered || e.code() == error_code_connection_failed) { wait(delay(FLOW_KNOBS->PREVENT_FAST_SPIN_DELAY)); continue; } else { @@ -1039,9 +1042,14 @@ ACTOR Future monitorBlobWorker(BlobManagerData* bmData, BlobWorkerInterfac throw e; } + if (BM_DEBUG) { + fmt::print( + "BM {0} got monitoring error {1} from BW {2}\n", bmData->epoch, e.name(), bwInterf.id().toString()); + } + // TODO: re-evaluate the expected errors here once wait failure issue is resolved - // Expected errors here are: [connection_failed, broken_promise] - if (e.code() != error_code_connection_failed && e.code() != error_code_broken_promise) { + // Expected errors here are: [broken_promise] + if (e.code() != error_code_broken_promise) { if (BM_DEBUG) { printf("BM got unexpected error %s monitoring BW %s\n", e.name(), bwInterf.id().toString().c_str()); } diff --git a/fdbserver/BlobWorker.actor.cpp b/fdbserver/BlobWorker.actor.cpp index 3422aa5e23..11c338ec10 100644 --- a/fdbserver/BlobWorker.actor.cpp +++ b/fdbserver/BlobWorker.actor.cpp @@ -2369,7 +2369,6 @@ ACTOR Future openGranule(Reference bwData, As : info.blobFilesToSnapshot.get().deltaFiles.back().version; } } - wait(tr.commit()); if (info.changeFeedStartVersion == invalidVersion) { @@ -2513,7 +2512,9 @@ ACTOR Future changeBlobRange(Reference bwData, if (selfReassign) { thisAssignmentNewer = true; } else { - printf("same assignment\n"); + if (BW_DEBUG) { + printf("same assignment\n"); + } // applied the same assignment twice, make idempotent if (r.value().activeMetadata.isValid()) { futures.push_back(success(r.value().assignFuture)); diff --git a/fdbserver/workloads/BlobGranuleVerifier.actor.cpp b/fdbserver/workloads/BlobGranuleVerifier.actor.cpp index b57ca10ac7..32189a6738 100644 --- a/fdbserver/workloads/BlobGranuleVerifier.actor.cpp +++ b/fdbserver/workloads/BlobGranuleVerifier.actor.cpp @@ -386,7 +386,7 @@ struct BlobGranuleVerifierWorkload : TestWorkload { // TODO: read at some version older than pruneVersion and make sure you get txn_too_old // To achieve this, the BWs are going to have to recognize latest prune versions per granules } catch (Error& e) { - if (e.code() == error_code_transaction_too_old) { + if (e.code() == error_code_blob_granule_transaction_too_old) { self->timeTravelTooOld++; // TODO: add debugging info for when this is a failure } @@ -418,8 +418,7 @@ struct BlobGranuleVerifierWorkload : TestWorkload { if (e.code() == error_code_operation_cancelled) { throw; } - if (e.code() != error_code_transaction_too_old && e.code() != error_code_wrong_shard_server && - BGV_DEBUG) { + if (e.code() != error_code_blob_granule_transaction_too_old && BGV_DEBUG) { printf("BGVerifier got unexpected error %s\n", e.name()); } self->errors++; @@ -444,6 +443,7 @@ struct BlobGranuleVerifierWorkload : TestWorkload { state Transaction tr(cx); state Version readVersion = wait(tr.getReadVersion()); + state Version startReadVersion = readVersion; state int checks = 0; state KeyRange last; @@ -478,16 +478,26 @@ struct BlobGranuleVerifierWorkload : TestWorkload { try { loop { - tr.reset(); try { Standalone> chunks = wait(tr.readBlobGranules(r, 0, readVersion)); ASSERT(chunks.size() > 0); last = chunks.back().keyRange; checks += chunks.size(); + break; } catch (Error& e) { - wait(tr.onError(e)); + // it's possible for blob granules to never get opened for the entire test due to fault + // injection. If we get blob_granule_transaction_too_old, for the latest read version, the + // granule still needs to open. Wait for that to happen at a higher read version. + if (e.code() == error_code_blob_granule_transaction_too_old) { + wait(delay(1.0)); + tr.reset(); + Version rv = wait(tr.getReadVersion()); + readVersion = rv; + } else { + wait(tr.onError(e)); + } } } } catch (Error& e) { @@ -508,6 +518,9 @@ struct BlobGranuleVerifierWorkload : TestWorkload { break; } } + if (BGV_DEBUG && startReadVersion != readVersion) { + fmt::print("Availability check updated read version from {0} to {1}\n", startReadVersion, readVersion); + } bool result = availabilityPassed && self->mismatches == 0 && (checks > 0) && (self->timeTravelTooOld == 0); fmt::print("Blob Granule Verifier {0} {1}:\n", self->clientId, result ? "passed" : "failed"); fmt::print(" {} successful final granule checks\n", checks); From 2c62dee5ba5c9f97a00256f78f9386111bbd0b9e Mon Sep 17 00:00:00 2001 From: Josh Slocum Date: Fri, 7 Jan 2022 13:49:02 -0600 Subject: [PATCH 106/413] Fixed more issues in blob granule requests --- fdbserver/BlobWorker.actor.cpp | 16 ++++++---------- 1 file changed, 6 insertions(+), 10 deletions(-) diff --git a/fdbserver/BlobWorker.actor.cpp b/fdbserver/BlobWorker.actor.cpp index 11c338ec10..de6bde332d 100644 --- a/fdbserver/BlobWorker.actor.cpp +++ b/fdbserver/BlobWorker.actor.cpp @@ -1817,6 +1817,7 @@ ACTOR Future blobGranuleLoadHistory(Reference bwData, namespace { bool canReplyWith(Error e) { switch (e.code()) { + case error_code_blob_granule_transaction_too_old: case error_code_transaction_too_old: case error_code_future_version: // not thrown yet case error_code_wrong_shard_server: @@ -2001,12 +2002,9 @@ ACTOR Future handleBlobGranuleFileRequest(Reference bwData } state Reference metadata = m; - // don't do 'if (canBeSet)' - if (metadata->readable.canBeSet()) { - wait(metadata->readable.getFuture()); - } - if (metadata->cancelled.isSet()) { - throw wrong_shard_server(); + choose { + when(wait(metadata->readable.getFuture())) {} + when(wait(metadata->cancelled.getFuture())) { throw wrong_shard_server(); } } state KeyRange chunkRange; @@ -2035,9 +2033,7 @@ ACTOR Future handleBlobGranuleFileRequest(Reference bwData if (!cur.isValid()) { // this request predates blob data - // FIXME: probably want a dedicated exception like blob_range_too_old or something - // instead - throw transaction_too_old(); + throw blob_granule_transaction_too_old(); } ASSERT(cur->endVersion > req.readVersion); @@ -2134,7 +2130,7 @@ ACTOR Future handleBlobGranuleFileRequest(Reference bwData i--; } // because of granule history, we should always be able to find the desired snapshot - // version, and have thrown transaction_too_old earlier if not possible. + // version, and have thrown blob_granule_transaction_too_old earlier if not possible. if (i < 0) { fmt::print("req @ {0} >= initial snapshot {1} but can't find snapshot in ({2}) files:\n", req.readVersion, From 21309fb55b455dae40200cd53f4c120d8209fe1c Mon Sep 17 00:00:00 2001 From: Josh Slocum Date: Fri, 7 Jan 2022 16:16:29 -0600 Subject: [PATCH 107/413] Change feed merge cursor whenAtLeast fix --- fdbclient/NativeAPI.actor.cpp | 44 +++++++++++++++++++++++++++------- fdbserver/BlobWorker.actor.cpp | 2 +- 2 files changed, 37 insertions(+), 9 deletions(-) diff --git a/fdbclient/NativeAPI.actor.cpp b/fdbclient/NativeAPI.actor.cpp index 4ef181926e..cb1c54a9ee 100644 --- a/fdbclient/NativeAPI.actor.cpp +++ b/fdbclient/NativeAPI.actor.cpp @@ -6723,7 +6723,7 @@ ACTOR Future>> readBlobGranulesActor( // if we detect that this blob worker fails, cancel the request, as otherwise load balance will // retry indefinitely with one option when(wait(IFailureMonitor::failureMonitor().onStateEqual( - location->get(i, &BlobWorkerInterface::blobGranuleFileRequest).getEndpoint(), + location->get(0, &BlobWorkerInterface::blobGranuleFileRequest).getEndpoint(), FailureStatus(true)))) { printf("readBlobGranules got BW %s failed\n", workerId.toString().c_str()); @@ -7141,8 +7141,11 @@ Version ChangeFeedData::getVersion() { #define DEBUG_CF_WAIT_VERSION invalidVersion #define DEBUG_CF_VERSION(v) DEBUG_CF_START_VERSION <= v&& v <= DEBUG_CF_END_VERSION +// This function is essentially bubbling the information about what has been processed from the server through the +// change feed client. First it makes sure the server has returned all mutations up through the target version, the +// native api has consumed and processed, them, and then the fdb client has consumed all of the mutations. ACTOR Future changeFeedWaitLatest(Reference self, Version version) { - // first, wait on SS to have sent up through version + // wait on SS to have sent up through version int desired = 0; int waiting = 0; std::vector> allAtLeast; @@ -7190,9 +7193,23 @@ ACTOR Future changeFeedWaitLatest(Reference self, Version wait(delay(0)); } + // wait for merge cursor to fully process everything it read from its individual promise streams, either until it is + // done processing or we have up through the desired version + while (self->lastReturnedVersion.get() < self->maxSeenVersion && self->lastReturnedVersion.get() < version) { + Version target = std::min(self->maxSeenVersion, version); + if (DEBUG_CF_WAIT_VERSION == version) { + fmt::print("CFW {0}) WaitLatest: waiting merge lastReturned >= {1}\n", version, target); + } + wait(self->lastReturnedVersion.whenAtLeast(target)); + if (DEBUG_CF_WAIT_VERSION == version) { + fmt::print( + "CFW {0}) WaitLatest: got merge lastReturned {1}\n", version, self->lastReturnedVersion.get()); + } + } + // then, wait for client to have consumed up through version if (self->maxSeenVersion >= version) { - // merge cursor has something buffered but has not yet sent it to self->mutations, just wait for + // merge cursor may have something buffered but has not yet sent it to self->mutations, just wait for // lastReturnedVersion if (DEBUG_CF_WAIT_VERSION == version) { fmt::print("CFW {0}) WaitLatest: maxSeenVersion -> waiting lastReturned\n", version); @@ -7293,13 +7310,20 @@ ACTOR Future partialChangeFeedStream(StorageServerInterface interf, feedData->notAtLatest.get(), rep.minStreamVersion); } + + if (rep.mutations.back().version > feedData->maxSeenVersion) { + feedData->maxSeenVersion = rep.mutations.back().version; + } + state int resultLoc = 0; while (resultLoc < rep.mutations.size()) { wait(results.onEmpty()); if (DEBUG_CF_VERSION(rep.mutations[resultLoc].version)) { - fmt::print(" single {0} {1}: onEmpty, sending {2} ({3})\n", + fmt::print(" single {0} {1}: sending {2}/{3} {4} ({5})\n", idx, interf.id().toString().substr(0, 4), + resultLoc, + rep.mutations.size(), rep.mutations[resultLoc].version, rep.mutations[resultLoc].mutations.size()); } @@ -7319,9 +7343,6 @@ ACTOR Future partialChangeFeedStream(StorageServerInterface interf, } resultLoc++; } - if (rep.mutations.back().version > feedData->maxSeenVersion) { - feedData->maxSeenVersion = rep.mutations.back().version; - } nextVersion = rep.mutations.back().version + 1; if (!atLatestVersion && rep.atLatestVersion) { @@ -7421,7 +7442,9 @@ ACTOR Future doCFMerge(Reference results, } if (nextOut.back().version < results->lastReturnedVersion.get()) { - printf("ERROR: merge cursor pushing next out <= lastReturnedVersion"); + fmt::print("ERROR: merge cursor pushing next out {} <= lastReturnedVersion {}\n", + nextOut.back().version, + results->lastReturnedVersion.get()); } // We can get an empty version pushed through the stream if whenAtLeast is called. Ignore // it @@ -7477,6 +7500,11 @@ ACTOR Future doCFMerge(Reference results, // it if (!nextOut.back().mutations.empty()) { + if (nextOut.back().version < results->lastReturnedVersion.get()) { + fmt::print("Merged all version went backwards!! mutation version {} < lastReturnedVersion {}\n", + nextOut.back().version, + results->lastReturnedVersion.get()); + } ASSERT(nextOut.back().version >= results->lastReturnedVersion.get()); results->mutations.send(nextOut); wait(results->mutations.onEmpty()); diff --git a/fdbserver/BlobWorker.actor.cpp b/fdbserver/BlobWorker.actor.cpp index de6bde332d..159c08b42c 100644 --- a/fdbserver/BlobWorker.actor.cpp +++ b/fdbserver/BlobWorker.actor.cpp @@ -45,7 +45,7 @@ #include "flow/actorcompiler.h" // has to be last include #define BW_DEBUG true -#define BW_REQUEST_DEBUG true +#define BW_REQUEST_DEBUG false struct GranuleStartState { UID granuleID; From 91574fa0b9e5752c616980f13366cca7b6d6fb82 Mon Sep 17 00:00:00 2001 From: Evan Tschannen Date: Mon, 10 Jan 2022 10:14:54 -0800 Subject: [PATCH 108/413] fix: poppingChangeFeeds did not properly handle broken promise errors --- 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 cb1c54a9ee..dad8df6e04 100644 --- a/fdbclient/NativeAPI.actor.cpp +++ b/fdbclient/NativeAPI.actor.cpp @@ -8039,7 +8039,7 @@ ACTOR Future popChangeFeedMutationsActor(Reference db, Ke } } catch (Error& e) { if (e.code() != error_code_unknown_change_feed && e.code() != error_code_wrong_shard_server && - e.code() != error_code_all_alternatives_failed) { + e.code() != error_code_all_alternatives_failed && e.code() != error_code_broken_promise) { throw; } db->changeFeedCache.erase(rangeID); From 95e2d0993f3cd5a0deb37fe350f315c6e41f5324 Mon Sep 17 00:00:00 2001 From: Josh Slocum Date: Mon, 10 Jan 2022 10:30:14 -0600 Subject: [PATCH 109/413] Fixed destroy race in NotifiedQueue --- flow/flow.h | 11 ++++++----- 1 file changed, 6 insertions(+), 5 deletions(-) diff --git a/flow/flow.h b/flow/flow.h index 69eb25d5ec..e6d195a527 100644 --- a/flow/flow.h +++ b/flow/flow.h @@ -1008,17 +1008,18 @@ struct NotifiedQueue : private SingleCallback, FastAllocated ASSERT(this->error.code() != error_code_success); this->error = err; - if (shouldFireImmediately()) { - SingleCallback::next->error(err); - } // end_of_stream error is "expected", don't terminate reading stream early for this - // weird destruction issue with sending broken_promise, so just ignore it. Stream is dead anyway. TODO better - // fix? + // onError must be triggered before callback, otherwise callback could cause delPromiseRef/delFutureRef. This + // could destroy *this* in the callback, causing onError to be referenced after this object is destroyed. if (err.code() != error_code_end_of_stream && err.code() != error_code_broken_promise && onError.isValid()) { ASSERT(onError.canBeSet()); onError.sendError(err); } + + if (shouldFireImmediately()) { + SingleCallback::next->error(err); + } } void addPromiseRef() { promises++; } From 17ba3e796d05a93d79f3d14b93f5ecb886059b29 Mon Sep 17 00:00:00 2001 From: Josh Slocum Date: Mon, 10 Jan 2022 12:13:27 -0600 Subject: [PATCH 110/413] Fixed some races in file requests and wait committed --- fdbserver/BlobWorker.actor.cpp | 58 ++++++++++++++++++++++++++-------- 1 file changed, 44 insertions(+), 14 deletions(-) diff --git a/fdbserver/BlobWorker.actor.cpp b/fdbserver/BlobWorker.actor.cpp index 159c08b42c..f3998671ee 100644 --- a/fdbserver/BlobWorker.actor.cpp +++ b/fdbserver/BlobWorker.actor.cpp @@ -1102,12 +1102,27 @@ ACTOR Future waitVersionCommitted(Reference bwData, // make sure the change feed has consumed mutations up through grvVersion to ensure none of them are rollbacks loop { - // if not valid, we're about to be cancelled anyway - state Future atLeast = - metadata->activeCFData.get().isValid() ? metadata->activeCFData.get()->whenAtLeast(grvVersion) : Never(); - choose { - when(wait(atLeast)) { break; } - when(wait(metadata->activeCFData.onChange())) {} + try { + // if not valid, we're about to be cancelled anyway + state Future atLeast = metadata->activeCFData.get().isValid() + ? metadata->activeCFData.get()->whenAtLeast(grvVersion) + : Never(); + choose { + when(wait(atLeast)) { break; } + when(wait(metadata->activeCFData.onChange())) {} + } + } catch (Error& e) { + + // if waiting on a parent granule change feed and we change to the child, the parent will get end_of_stream, + // which could cause this waiting whenAtLeast to get change_feed_cancelled. We should simply retry and wait + // a bit, as blobGranuleUpdateFiles will switch to the new change feed + if (e.code() != error_code_change_feed_cancelled) { + if (BW_DEBUG) { + printf("waitVersionCommitted WAL got unexpected error %s\n", e.name()); + } + throw e; + } + wait(delay(0.05)); } } // sanity check to make sure whenAtLeast didn't return early @@ -1943,13 +1958,13 @@ ACTOR Future waitForVersion(Reference metadata, Version v } ACTOR Future handleBlobGranuleFileRequest(Reference bwData, BlobGranuleFileRequest req) { - /*if (BW_REQUEST_DEBUG) { - printf("BW %s processing blobGranuleFileRequest for range [%s-%s) @ %lld\n", - bwData->id.toString().c_str(), - req.keyRange.begin.printable().c_str(), - req.keyRange.end.printable().c_str(), - req.readVersion); - }*/ + if (BW_REQUEST_DEBUG || DEBUG_BW_WAIT_VERSION == req.readVersion) { + printf("BW %s processing blobGranuleFileRequest for range [%s-%s) @ %lld\n", + bwData->id.toString().c_str(), + req.keyRange.begin.printable().c_str(), + req.keyRange.end.printable().c_str(), + req.readVersion); + } try { // TODO REMOVE in api V2 @@ -2007,6 +2022,11 @@ ACTOR Future handleBlobGranuleFileRequest(Reference bwData when(wait(metadata->cancelled.getFuture())) { throw wrong_shard_server(); } } + // in case both readable and cancelled are ready, check cancelled + if (!metadata->cancelled.canBeSet()) { + throw wrong_shard_server(); + } + state KeyRange chunkRange; state GranuleFiles chunkFiles; @@ -2088,7 +2108,8 @@ ACTOR Future handleBlobGranuleFileRequest(Reference bwData } else { // this is an active granule query loop { - if (!metadata->activeCFData.get().isValid()) { + if (!metadata->activeCFData.get().isValid() || !metadata->cancelled.canBeSet()) { + throw wrong_shard_server(); } Future waitForVersionFuture = waitForVersion(metadata, req.readVersion); @@ -2114,6 +2135,15 @@ ACTOR Future handleBlobGranuleFileRequest(Reference bwData chunkRange = metadata->keyRange; } + if (!metadata->cancelled.canBeSet()) { + printf("ERROR: Request [%s - %s) @ %lld cancelled for granule [%s - %s) after waitForVersion!\n", + req.keyRange.begin.printable().c_str(), + req.keyRange.end.printable().c_str(), + req.readVersion, + metadata->keyRange.begin.printable().c_str(), + metadata->keyRange.end.printable().c_str()); + } + // granule is up to date, do read ASSERT(metadata->cancelled.canBeSet()); From f0b434d9ce87addb533ad9fec26b890c9d911d16 Mon Sep 17 00:00:00 2001 From: Josh Slocum Date: Mon, 10 Jan 2022 12:27:43 -0600 Subject: [PATCH 111/413] Don't log operation_cancelled --- fdbserver/BlobWorker.actor.cpp | 3 +++ 1 file changed, 3 insertions(+) diff --git a/fdbserver/BlobWorker.actor.cpp b/fdbserver/BlobWorker.actor.cpp index f3998671ee..7881ea9790 100644 --- a/fdbserver/BlobWorker.actor.cpp +++ b/fdbserver/BlobWorker.actor.cpp @@ -1112,6 +1112,9 @@ ACTOR Future waitVersionCommitted(Reference bwData, when(wait(metadata->activeCFData.onChange())) {} } } catch (Error& e) { + if (e.code() == error_code_operation_cancelled) { + throw e; + } // if waiting on a parent granule change feed and we change to the child, the parent will get end_of_stream, // which could cause this waiting whenAtLeast to get change_feed_cancelled. We should simply retry and wait From 5bd8350eac30fdc0a501c01329ab4ad33c2a427f Mon Sep 17 00:00:00 2001 From: Josh Slocum Date: Mon, 10 Jan 2022 13:13:00 -0600 Subject: [PATCH 112/413] also handling server overloaded in change feed pop --- fdbclient/NativeAPI.actor.cpp | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/fdbclient/NativeAPI.actor.cpp b/fdbclient/NativeAPI.actor.cpp index dad8df6e04..42437199f1 100644 --- a/fdbclient/NativeAPI.actor.cpp +++ b/fdbclient/NativeAPI.actor.cpp @@ -8039,7 +8039,8 @@ ACTOR Future popChangeFeedMutationsActor(Reference db, Ke } } catch (Error& e) { if (e.code() != error_code_unknown_change_feed && e.code() != error_code_wrong_shard_server && - e.code() != error_code_all_alternatives_failed && e.code() != error_code_broken_promise) { + e.code() != error_code_all_alternatives_failed && e.code() != error_code_broken_promise && + e.code() != error_code_server_overloaded) { throw; } db->changeFeedCache.erase(rangeID); From 01fa42522a268aca5f35e21960aa665b1888254c Mon Sep 17 00:00:00 2001 From: Evan Tschannen Date: Mon, 10 Jan 2022 11:33:18 -0800 Subject: [PATCH 113/413] fix: changed loadHistoryFiles to take a database object --- fdbserver/BlobGranuleServerCommon.actor.cpp | 10 +++++++--- fdbserver/BlobGranuleServerCommon.actor.h | 2 +- fdbserver/BlobManager.actor.cpp | 21 +++++++++++---------- fdbserver/BlobWorker.actor.cpp | 5 +---- 4 files changed, 20 insertions(+), 18 deletions(-) diff --git a/fdbserver/BlobGranuleServerCommon.actor.cpp b/fdbserver/BlobGranuleServerCommon.actor.cpp index a971c8d268..87d6e12064 100644 --- a/fdbserver/BlobGranuleServerCommon.actor.cpp +++ b/fdbserver/BlobGranuleServerCommon.actor.cpp @@ -105,16 +105,20 @@ ACTOR Future readGranuleFiles(Transaction* tr, // Wrapper around readGranuleFiles // Gets all files belonging to the granule with id granule ID -ACTOR Future loadHistoryFiles(Transaction* tr, UID granuleID, bool debug) { +ACTOR Future loadHistoryFiles(Database cx, UID granuleID, bool debug) { state KeyRange range = blobGranuleFileKeyRangeFor(granuleID); state Key startKey = range.begin; state GranuleFiles files; + state Transaction tr(cx); + loop { try { - wait(readGranuleFiles(tr, &startKey, range.end, &files, granuleID, debug)); + tr.setOption(FDBTransactionOptions::ACCESS_SYSTEM_KEYS); + tr.setOption(FDBTransactionOptions::PRIORITY_SYSTEM_IMMEDIATE); + wait(readGranuleFiles(&tr, &startKey, range.end, &files, granuleID, debug)); return files; } catch (Error& e) { - wait(tr->onError(e)); + wait(tr.onError(e)); } } } diff --git a/fdbserver/BlobGranuleServerCommon.actor.h b/fdbserver/BlobGranuleServerCommon.actor.h index d9eb372999..b593cc1361 100644 --- a/fdbserver/BlobGranuleServerCommon.actor.h +++ b/fdbserver/BlobGranuleServerCommon.actor.h @@ -71,5 +71,5 @@ ACTOR Future readGranuleFiles(Transaction* tr, UID granuleID, bool debug); -ACTOR Future loadHistoryFiles(Transaction* tr, UID granuleID, bool debug); +ACTOR Future loadHistoryFiles(Database cx, UID granuleID, bool debug); #endif diff --git a/fdbserver/BlobManager.actor.cpp b/fdbserver/BlobManager.actor.cpp index 5cb2727569..60f8bcca8f 100644 --- a/fdbserver/BlobManager.actor.cpp +++ b/fdbserver/BlobManager.actor.cpp @@ -1481,12 +1481,8 @@ ACTOR Future fullyDeleteGranule(BlobManagerData* self, UID granuleId, KeyR printf("Fully deleting granule %s: init\n", granuleId.toString().c_str()); } - state Transaction tr(self->db); - tr.setOption(FDBTransactionOptions::ACCESS_SYSTEM_KEYS); - tr.setOption(FDBTransactionOptions::PRIORITY_SYSTEM_IMMEDIATE); - // get files - GranuleFiles files = wait(loadHistoryFiles(&tr, granuleId, BM_DEBUG)); + GranuleFiles files = wait(loadHistoryFiles(self->db, granuleId, BM_DEBUG)); std::vector> deletions; std::vector filesToDelete; // TODO: remove, just for debugging @@ -1520,6 +1516,11 @@ ACTOR Future fullyDeleteGranule(BlobManagerData* self, UID granuleId, KeyR if (BM_DEBUG) { printf("Fully deleting granule %s: deleting history and file keys\n", granuleId.toString().c_str()); } + + state Transaction tr(self->db); + tr.setOption(FDBTransactionOptions::ACCESS_SYSTEM_KEYS); + tr.setOption(FDBTransactionOptions::PRIORITY_SYSTEM_IMMEDIATE); + loop { try { KeyRange fileRangeKey = blobGranuleFileKeyRangeFor(granuleId); @@ -1552,12 +1553,8 @@ ACTOR Future partiallyDeleteGranule(BlobManagerData* self, UID granuleId, printf("Partially deleting granule %s: init\n", granuleId.toString().c_str()); } - state Transaction tr(self->db); - tr.setOption(FDBTransactionOptions::ACCESS_SYSTEM_KEYS); - tr.setOption(FDBTransactionOptions::PRIORITY_SYSTEM_IMMEDIATE); - // get files - GranuleFiles files = wait(loadHistoryFiles(&tr, granuleId, BM_DEBUG)); + GranuleFiles files = wait(loadHistoryFiles(self->db, granuleId, BM_DEBUG)); // represents the version of the latest snapshot file in this granule with G.version < pruneVersion Version latestSnapshotVersion = invalidVersion; @@ -1621,6 +1618,10 @@ ACTOR Future partiallyDeleteGranule(BlobManagerData* self, UID granuleId, printf("Partially deleting granule %s: deleting file keys\n", granuleId.toString().c_str()); } + state Transaction tr(self->db); + tr.setOption(FDBTransactionOptions::ACCESS_SYSTEM_KEYS); + tr.setOption(FDBTransactionOptions::PRIORITY_SYSTEM_IMMEDIATE); + loop { try { for (auto& key : deletedFileKeys) { diff --git a/fdbserver/BlobWorker.actor.cpp b/fdbserver/BlobWorker.actor.cpp index 7881ea9790..27a3c8149f 100644 --- a/fdbserver/BlobWorker.actor.cpp +++ b/fdbserver/BlobWorker.actor.cpp @@ -2077,10 +2077,7 @@ ACTOR Future handleBlobGranuleFileRequest(Reference bwData // lazily load files for old granule if not present chunkRange = cur->range; if (!cur->files.isValid() || cur->files.isError()) { - Transaction tr(bwData->db); - tr.setOption(FDBTransactionOptions::ACCESS_SYSTEM_KEYS); - tr.setOption(FDBTransactionOptions::PRIORITY_SYSTEM_IMMEDIATE); - cur->files = loadHistoryFiles(&tr, cur->granuleID, BW_DEBUG); + cur->files = loadHistoryFiles(bwData->db, cur->granuleID, BW_DEBUG); } choose { From 59e6793c6bdc2ff92bd7fd3299c9f0128e1873ae Mon Sep 17 00:00:00 2001 From: Josh Slocum Date: Mon, 10 Jan 2022 13:44:45 -0600 Subject: [PATCH 114/413] Fixed waitForVersion sanity check for rollbacks --- fdbserver/BlobWorker.actor.cpp | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/fdbserver/BlobWorker.actor.cpp b/fdbserver/BlobWorker.actor.cpp index 27a3c8149f..2dcebb3bc7 100644 --- a/fdbserver/BlobWorker.actor.cpp +++ b/fdbserver/BlobWorker.actor.cpp @@ -1454,6 +1454,12 @@ ACTOR Future blobGranuleUpdateFiles(Reference bwData, rollbacksCompleted); // Reset change feeds to cfRollbackVersion + if (cfRollbackVersion < metadata->waitForVersionReturned) { + fmt::print("Rollback resetting waitForVersionReturned {0} -> {1}\n", + metadata->waitForVersionReturned, + cfRollbackVersion); + metadata->waitForVersionReturned = cfRollbackVersion; + } metadata->activeCFData.set(newChangeFeedData(cfRollbackVersion)); if (readOldChangeFeed) { // It shouldn't be possible to roll back across the parent/child feed boundary, From a7421e6ca2f7a07e1868cdbd4653a6430a51a50a Mon Sep 17 00:00:00 2001 From: Evan Tschannen Date: Mon, 10 Jan 2022 12:59:30 -0800 Subject: [PATCH 115/413] fix: localChangeFeeds cannot get wrong_shard_server errors --- 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 05720a1bd2..f0ee40a0d7 100644 --- a/fdbserver/storageserver.actor.cpp +++ b/fdbserver/storageserver.actor.cpp @@ -1795,7 +1795,7 @@ ACTOR Future> getChangeFeedMutations(Stor 1 << 30, remainingDurableBytes)); - if (!req.range.empty()) { + if (!inverted && !req.range.empty()) { data->checkChangeCounter(changeCounter, req.range); } From 6b4b22229b96ae7090f5a5faa40a5388c4720f2a Mon Sep 17 00:00:00 2001 From: Josh Slocum Date: Mon, 10 Jan 2022 17:22:56 -0600 Subject: [PATCH 116/413] Fixed rollback and granule history issue --- fdbserver/BlobManager.actor.cpp | 11 ++--- fdbserver/BlobWorker.actor.cpp | 72 +++++++++++++++++++++++++++------ 2 files changed, 65 insertions(+), 18 deletions(-) diff --git a/fdbserver/BlobManager.actor.cpp b/fdbserver/BlobManager.actor.cpp index 60f8bcca8f..982a2abdf7 100644 --- a/fdbserver/BlobManager.actor.cpp +++ b/fdbserver/BlobManager.actor.cpp @@ -710,12 +710,13 @@ ACTOR Future maybeSplitRange(BlobManagerData* bmData, } if (BM_DEBUG) { - printf("Splitting range [%s - %s) into (%d):\n", - granuleRange.begin.printable().c_str(), - granuleRange.end.printable().c_str(), - newRanges.size() - 1); + fmt::print("Splitting range [{0} - {1}) into {2} granules @ {3}:\n", + granuleRange.begin.printable(), + granuleRange.end.printable(), + newRanges.size() - 1, + latestVersion); for (int i = 0; i < newRanges.size(); i++) { - printf(" %s\n", newRanges[i].printable().c_str()); + fmt::print(" {}\n", newRanges[i].printable()); } } diff --git a/fdbserver/BlobWorker.actor.cpp b/fdbserver/BlobWorker.actor.cpp index 2dcebb3bc7..f6510ab031 100644 --- a/fdbserver/BlobWorker.actor.cpp +++ b/fdbserver/BlobWorker.actor.cpp @@ -1461,10 +1461,20 @@ ACTOR Future blobGranuleUpdateFiles(Reference bwData, metadata->waitForVersionReturned = cfRollbackVersion; } metadata->activeCFData.set(newChangeFeedData(cfRollbackVersion)); + + if (!readOldChangeFeed && cfRollbackVersion < startState.changeFeedStartVersion) { + // It isn't possible to roll back across the parent/child feed boundary, but as + // part of rolling back we may need to cancel in-flight delta files, and those + // delta files may include stuff from before the parent/child boundary. So we + // have to go back to reading the old change feed + ASSERT(cfRollbackVersion >= startState.previousDurableVersion); + ASSERT(cfRollbackVersion >= metadata->durableDeltaVersion.get()); + TEST(true); // rollback crossed change feed boundaries + readOldChangeFeed = true; + oldChangeFeedDataComplete.reset(); + } + if (readOldChangeFeed) { - // It shouldn't be possible to roll back across the parent/child feed boundary, - // because the transaction creating the child change feed had to commit before - // we got here. ASSERT(cfRollbackVersion < startState.changeFeedStartVersion); oldChangeFeedFuture = bwData->db->getChangeFeedStream(metadata->activeCFData.get(), @@ -1474,6 +1484,11 @@ ACTOR Future blobGranuleUpdateFiles(Reference bwData, metadata->keyRange); } else { + if (cfRollbackVersion <= startState.changeFeedStartVersion) { + fmt::print("Rollback past CF start??. rollback={0}, start={1}\n", + cfRollbackVersion, + startState.changeFeedStartVersion); + } ASSERT(cfRollbackVersion > startState.changeFeedStartVersion); changeFeedFuture = bwData->db->getChangeFeedStream(metadata->activeCFData.get(), @@ -1757,10 +1772,10 @@ ACTOR Future blobGranuleLoadHistory(Reference bwData, state std::vector> historyEntryStack; - // while the start version of the current granule's parent is larger than the last known start version, + // while the start version of the current granule's parent not past the last known start version, // walk backwards while (curHistory.value.parentGranules.size() > 0 && - curHistory.value.parentGranules[0].second > stopVersion) { + curHistory.value.parentGranules[0].second >= stopVersion) { state GranuleHistory next; loop { try { @@ -1783,6 +1798,17 @@ ACTOR Future blobGranuleLoadHistory(Reference bwData, curHistory = next; } + if (!historyEntryStack.empty()) { + Version oldestStartVersion = historyEntryStack.back()->startVersion; + // TODO REMOVE eventually, for debugging + if (stopVersion != oldestStartVersion && stopVersion != invalidVersion) { + fmt::print("Finished, stopVersion={0}, curHistory.version={1}\n", stopVersion, oldestStartVersion); + } + ASSERT(stopVersion == oldestStartVersion || stopVersion == invalidVersion); + } else { + ASSERT(stopVersion == invalidVersion); + } + // go back up stack and apply history entries from oldest to newest, skipping ranges that were already // applied by other racing loads. // yielding in this loop would mean we'd need to re-check for load races @@ -1989,7 +2015,7 @@ ACTOR Future handleBlobGranuleFileRequest(Reference bwData for (auto& r : checkRanges) { bool isValid = r.value().activeMetadata.isValid(); if (lastRangeEnd < r.begin() || !isValid) { - if (BW_REQUEST_DEBUG) { + if (BW_REQUEST_DEBUG || DEBUG_BW_WAIT_VERSION == req.readVersion) { printf("No %s blob data for [%s - %s) in request range [%s - %s), skipping request\n", isValid ? "" : "valid", lastRangeEnd.printable().c_str(), @@ -2004,7 +2030,7 @@ ACTOR Future handleBlobGranuleFileRequest(Reference bwData lastRangeEnd = r.end(); } if (lastRangeEnd < req.keyRange.end) { - if (BW_REQUEST_DEBUG) { + if (BW_REQUEST_DEBUG || DEBUG_BW_WAIT_VERSION == req.readVersion) { printf("No blob data for [%s - %s) in request range [%s - %s), skipping request\n", lastRangeEnd.printable().c_str(), req.keyRange.end.printable().c_str(), @@ -2050,9 +2076,28 @@ ACTOR Future handleBlobGranuleFileRequest(Reference bwData // FIXME: doesn't work once we add granule merging, could be multiple ranges and/or // multiple parents - Reference cur = bwData->granuleHistory.rangeContaining(req.keyRange.begin).value(); + Key historySearchKey = std::max(req.keyRange.begin, metadata->keyRange.begin); + Reference cur = bwData->granuleHistory.rangeContaining(historySearchKey).value(); + // FIXME: use skip pointers here Version expectedEndVersion = metadata->initialSnapshotVersion; + if (cur.isValid()) { + // TODO REMOVE, useful for debugging for now + if (cur->endVersion != expectedEndVersion) { + fmt::print("Active granule [{0} - {1}) does not have history ancestor!!. Start is {2}, " + "ancestor is [{3} - {4}) ({5}) V[{6} - {7}). SearchKey={8}\n", + metadata->keyRange.begin.printable(), + metadata->keyRange.end.printable(), + expectedEndVersion, + cur->range.begin.printable(), + cur->range.end.printable(), + cur->granuleID.toString(), + cur->startVersion, + cur->endVersion, + historySearchKey.printable()); + } + ASSERT(cur->endVersion == expectedEndVersion); + } while (cur.isValid() && req.readVersion < cur->startVersion) { // assert version of history is contiguous ASSERT(cur->endVersion == expectedEndVersion); @@ -2065,10 +2110,7 @@ ACTOR Future handleBlobGranuleFileRequest(Reference bwData throw blob_granule_transaction_too_old(); } - ASSERT(cur->endVersion > req.readVersion); - ASSERT(cur->startVersion <= req.readVersion); - - if (BW_REQUEST_DEBUG) { + if (BW_REQUEST_DEBUG || DEBUG_BW_WAIT_VERSION == req.readVersion) { fmt::print("[{0} - {1}) @ {2} time traveled back to {3} [{4} - {5}) @ [{6} - {7})\n", req.keyRange.begin.printable(), req.keyRange.end.printable(), @@ -2080,6 +2122,9 @@ ACTOR Future handleBlobGranuleFileRequest(Reference bwData cur->endVersion); } + ASSERT(cur->endVersion > req.readVersion); + ASSERT(cur->startVersion <= req.readVersion); + // lazily load files for old granule if not present chunkRange = cur->range; if (!cur->files.isValid() || cur->files.isError()) { @@ -2130,7 +2175,8 @@ ACTOR Future handleBlobGranuleFileRequest(Reference bwData when(wait(metadata->activeCFData.onChange())) {} when(wait(metadata->cancelled.getFuture())) { throw wrong_shard_server(); } } - if (BW_REQUEST_DEBUG && metadata->activeCFData.get().isValid()) { + if ((BW_REQUEST_DEBUG || DEBUG_BW_WAIT_VERSION == req.readVersion) && + metadata->activeCFData.get().isValid()) { fmt::print("{0} - {1}) @ {2} hit CF change, restarting waitForVersion\n", req.keyRange.begin.printable().c_str(), req.keyRange.end.printable().c_str(), From 3b2e58ada82bf33d6ce902ac74cd9675947f63e5 Mon Sep 17 00:00:00 2001 From: Josh Slocum Date: Tue, 11 Jan 2022 10:33:39 -0600 Subject: [PATCH 117/413] Fixed rollback of snapshot files bug --- fdbclient/NativeAPI.actor.cpp | 7 ++++++- fdbserver/BlobManager.actor.cpp | 2 +- fdbserver/BlobWorker.actor.cpp | 13 ++++++++++++- 3 files changed, 19 insertions(+), 3 deletions(-) diff --git a/fdbclient/NativeAPI.actor.cpp b/fdbclient/NativeAPI.actor.cpp index 42437199f1..1ca405f803 100644 --- a/fdbclient/NativeAPI.actor.cpp +++ b/fdbclient/NativeAPI.actor.cpp @@ -6725,12 +6725,17 @@ ACTOR Future>> readBlobGranulesActor( when(wait(IFailureMonitor::failureMonitor().onStateEqual( location->get(0, &BlobWorkerInterface::blobGranuleFileRequest).getEndpoint(), FailureStatus(true)))) { - printf("readBlobGranules got BW %s failed\n", workerId.toString().c_str()); + if (BG_REQUEST_DEBUG) { + printf("readBlobGranules got BW %s failed\n", workerId.toString().c_str()); + } throw connection_failed(); } } } catch (Error& e) { + if (BG_REQUEST_DEBUG) { + printf("BGReq got error %s\n", e.name()); + } // worker is up but didn't actually have granule, or connection failed if (e.code() == error_code_wrong_shard_server || e.code() == error_code_connection_failed) { // need to re-read mapping, throw transaction_too_old so client retries. TODO better error? diff --git a/fdbserver/BlobManager.actor.cpp b/fdbserver/BlobManager.actor.cpp index 982a2abdf7..b28443ca06 100644 --- a/fdbserver/BlobManager.actor.cpp +++ b/fdbserver/BlobManager.actor.cpp @@ -969,7 +969,7 @@ ACTOR Future monitorBlobWorkerStatus(BlobManagerData* bmData, BlobWorkerIn rep.granuleRange.end == lastReqForGranule.end() && rep.epoch == lastReqForGranule.value().first && rep.seqno == lastReqForGranule.value().second) { if (BM_DEBUG) { - fmt::print("Manager {0} received repeat status for the same granule [{1} - {2}), ignoring.", + fmt::print("Manager {0} received repeat status for the same granule [{1} - {2}), ignoring.\n", bmData->epoch, rep.granuleRange.begin.printable(), rep.granuleRange.end.printable()); diff --git a/fdbserver/BlobWorker.actor.cpp b/fdbserver/BlobWorker.actor.cpp index f6510ab031..8e070fe3c0 100644 --- a/fdbserver/BlobWorker.actor.cpp +++ b/fdbserver/BlobWorker.actor.cpp @@ -977,7 +977,14 @@ static Version doGranuleRollback(Reference metadata, for (auto& f : inFlightFiles) { if (f.snapshot) { if (f.version > rollbackVersion) { + if (BW_DEBUG) { + fmt::print("[{0} - {1}) rollback cancelling snapshot file @ {2}\n", + metadata->keyRange.begin.printable(), + metadata->keyRange.end.printable(), + f.version); + } f.future.cancel(); + toPop++; } else { metadata->pendingSnapshotVersion = f.version; metadata->bytesInNewDeltaFiles = 0; @@ -1709,7 +1716,11 @@ ACTOR Future blobGranuleUpdateFiles(Reference bwData, } catch (Error& e) { // TODO REMOVE if (BW_DEBUG) { - printf("BGUF got error %s\n", e.name()); + fmt::print("BGUF {0} [{1} - {2}) got error {3}\n", + startState.granuleID.toString(), + metadata->keyRange.begin.printable(), + metadata->keyRange.end.printable(), + e.name()); } // Free last change feed data metadata->activeCFData.set(Reference()); From 6b202fc9a82fb8233dc3b15719053153b243a7dd Mon Sep 17 00:00:00 2001 From: Josh Slocum Date: Tue, 11 Jan 2022 15:35:20 -0600 Subject: [PATCH 118/413] Fixed stuck change feed storage updater and improved debugging --- fdbclient/DatabaseContext.h | 2 + fdbclient/NativeAPI.actor.cpp | 72 ++++++++++++++++++++--------------- fdbserver/worker.actor.cpp | 6 +++ 3 files changed, 49 insertions(+), 31 deletions(-) diff --git a/fdbclient/DatabaseContext.h b/fdbclient/DatabaseContext.h index b42c502a53..ac567fcd76 100644 --- a/fdbclient/DatabaseContext.h +++ b/fdbclient/DatabaseContext.h @@ -159,6 +159,7 @@ struct ChangeFeedStorageData : ReferenceCounted { NotifiedVersion version; NotifiedVersion desired; Promise destroyed; + UID interfToken; bool debug = false; // TODO REMOVE @@ -172,6 +173,7 @@ struct ChangeFeedData : ReferenceCounted { Version getVersion(); Future whenAtLeast(Version version); + Key id; // TODO REMOVE eventually? for debugging NotifiedVersion lastReturnedVersion; std::vector> storageData; AsyncVar notAtLatest; diff --git a/fdbclient/NativeAPI.actor.cpp b/fdbclient/NativeAPI.actor.cpp index 1ca405f803..b7bc59b664 100644 --- a/fdbclient/NativeAPI.actor.cpp +++ b/fdbclient/NativeAPI.actor.cpp @@ -7112,12 +7112,15 @@ ACTOR Future storageFeedVersionUpdater(StorageServerInterface interf, Chan } Reference DatabaseContext::getStorageData(StorageServerInterface interf) { - auto it = changeFeedUpdaters.find(interf.id()); + // use token from interface since that changes on SS restart + UID token = interf.waitFailure.getEndpoint().token; + auto it = changeFeedUpdaters.find(token); if (it == changeFeedUpdaters.end()) { Reference newStorageUpdater = makeReference(); newStorageUpdater->id = interf.id(); + newStorageUpdater->interfToken = token; newStorageUpdater->updater = storageFeedVersionUpdater(interf, newStorageUpdater.getPtr()); - changeFeedUpdaters[interf.id()] = newStorageUpdater; + changeFeedUpdaters[token] = newStorageUpdater; return newStorageUpdater; } return it->second; @@ -7141,10 +7144,15 @@ Version ChangeFeedData::getVersion() { // TODO REMOVE when BG is correctness clean // To debug a waitLatest at wait_version returning early, set wait_version to the version, and start+end to a version // range that surrounds wait_version enough to figure out what's going on +// DEBUG_CF_ID is optional +#define DEBUG_CF_ID ""_sr #define DEBUG_CF_START_VERSION invalidVersion #define DEBUG_CF_END_VERSION invalidVersion #define DEBUG_CF_WAIT_VERSION invalidVersion -#define DEBUG_CF_VERSION(v) DEBUG_CF_START_VERSION <= v&& v <= DEBUG_CF_END_VERSION +#define DEBUG_CF_VERSION(cfId, v) \ + DEBUG_CF_START_VERSION <= v&& v <= DEBUG_CF_END_VERSION && (""_sr == DEBUG_CF_ID || cfId.printable() == DEBUG_CF_ID) + +#define DEBUG_CF_WAIT(cfId, v) DEBUG_CF_WAIT_VERSION == v && (""_sr == DEBUG_CF_ID || cfId.printable() == DEBUG_CF_ID) // This function is essentially bubbling the information about what has been processed from the server through the // change feed client. First it makes sure the server has returned all mutations up through the target version, the @@ -7158,7 +7166,7 @@ ACTOR Future changeFeedWaitLatest(Reference self, Version if (it->version.get() < version) { waiting++; if (version > it->desired.get()) { - if (DEBUG_CF_WAIT_VERSION == version) { + if (DEBUG_CF_WAIT(self->id, version)) { it->debug = true; } it->desired.set(version); @@ -7168,7 +7176,7 @@ ACTOR Future changeFeedWaitLatest(Reference self, Version } } - if (DEBUG_CF_WAIT_VERSION == version) { + if (DEBUG_CF_WAIT(self->id, version)) { fmt::print("CFW {0}) WaitLatest: waiting for {1}/{2} ss ({3} < desired)\n", version, waiting, @@ -7186,7 +7194,7 @@ ACTOR Future changeFeedWaitLatest(Reference self, Version } } - if (DEBUG_CF_WAIT_VERSION == version) { + if (DEBUG_CF_WAIT(self->id, version)) { fmt::print("CFW {0}) WaitLatest: waiting for {1} ss onEmpty\n", version, onEmpty.size()); } @@ -7202,11 +7210,11 @@ ACTOR Future changeFeedWaitLatest(Reference self, Version // done processing or we have up through the desired version while (self->lastReturnedVersion.get() < self->maxSeenVersion && self->lastReturnedVersion.get() < version) { Version target = std::min(self->maxSeenVersion, version); - if (DEBUG_CF_WAIT_VERSION == version) { + if (DEBUG_CF_WAIT(self->id, version)) { fmt::print("CFW {0}) WaitLatest: waiting merge lastReturned >= {1}\n", version, target); } wait(self->lastReturnedVersion.whenAtLeast(target)); - if (DEBUG_CF_WAIT_VERSION == version) { + if (DEBUG_CF_WAIT(self->id, version)) { fmt::print( "CFW {0}) WaitLatest: got merge lastReturned {1}\n", version, self->lastReturnedVersion.get()); } @@ -7216,26 +7224,26 @@ ACTOR Future changeFeedWaitLatest(Reference self, Version if (self->maxSeenVersion >= version) { // merge cursor may have something buffered but has not yet sent it to self->mutations, just wait for // lastReturnedVersion - if (DEBUG_CF_WAIT_VERSION == version) { + if (DEBUG_CF_WAIT(self->id, version)) { fmt::print("CFW {0}) WaitLatest: maxSeenVersion -> waiting lastReturned\n", version); } wait(self->lastReturnedVersion.whenAtLeast(version)); - if (DEBUG_CF_WAIT_VERSION == version) { + if (DEBUG_CF_WAIT(self->id, version)) { fmt::print("CFW {0}) WaitLatest: maxSeenVersion -> got lastReturned\n", version); } } else { // all mutations <= version are in self->mutations, wait for empty while (!self->mutations.isEmpty()) { - if (DEBUG_CF_WAIT_VERSION == version) { + if (DEBUG_CF_WAIT(self->id, version)) { fmt::print("CFW {0}) WaitLatest: waiting for client onEmpty\n", version); } wait(self->mutations.onEmpty()); wait(delay(0)); } - if (DEBUG_CF_WAIT_VERSION == version) { + if (DEBUG_CF_WAIT(self->id, version)) { fmt::print("CFW {0}) WaitLatest: done\n", version); } } @@ -7244,11 +7252,11 @@ ACTOR Future changeFeedWaitLatest(Reference self, Version } ACTOR Future changeFeedWhenAtLatest(Reference self, Version version) { - if (DEBUG_CF_WAIT_VERSION == version) { + if (DEBUG_CF_WAIT(self->id, version)) { fmt::print("CFW {0}) WhenAtLeast: LR={1}\n", version, self->lastReturnedVersion.get()); } if (version <= self->getVersion()) { - if (DEBUG_CF_WAIT_VERSION == version) { + if (DEBUG_CF_WAIT(self->id, version)) { fmt::print("CFW {0}) WhenAtLeast: Already done\n", version, self->lastReturnedVersion.get()); } return Void(); @@ -7256,7 +7264,7 @@ ACTOR Future changeFeedWhenAtLatest(Reference self, Versio state Future lastReturned = self->lastReturnedVersion.whenAtLeast(version); loop { - if (DEBUG_CF_WAIT_VERSION == version) { + if (DEBUG_CF_WAIT(self->id, version)) { fmt::print("CFW {0}) WhenAtLeast: NotAtLatest={1}\n", version, self->notAtLatest.get()); } // only allowed to use empty versions if you're caught up @@ -7269,7 +7277,7 @@ ACTOR Future changeFeedWhenAtLatest(Reference self, Versio } } - if (DEBUG_CF_VERSION(version)) { + if (DEBUG_CF_VERSION(self->id, version)) { fmt::print("CFLR (WAL): {0}\n", version); } @@ -7301,7 +7309,7 @@ ACTOR Future partialChangeFeedStream(StorageServerInterface interf, } choose { when(state ChangeFeedStreamReply rep = waitNext(replyStream.getFuture())) { - if (DEBUG_CF_VERSION(rep.mutations.back().version)) { + if (DEBUG_CF_VERSION(feedData->id, rep.mutations.back().version)) { fmt::print(" single {0} {1}: response {2} - {3} ({4}), atLatest={5}, rep.atLatest={6}, " "notAtLatest={7}, " "minSV={8}\n", @@ -7323,7 +7331,7 @@ ACTOR Future partialChangeFeedStream(StorageServerInterface interf, state int resultLoc = 0; while (resultLoc < rep.mutations.size()) { wait(results.onEmpty()); - if (DEBUG_CF_VERSION(rep.mutations[resultLoc].version)) { + if (DEBUG_CF_VERSION(feedData->id, rep.mutations[resultLoc].version)) { fmt::print(" single {0} {1}: sending {2}/{3} {4} ({5})\n", idx, interf.id().toString().substr(0, 4), @@ -7367,7 +7375,7 @@ ACTOR Future partialChangeFeedStream(StorageServerInterface interf, when(wait(atLatestVersion && replyStream.isEmpty() && results.isEmpty() ? storageData->version.whenAtLeast(nextVersion) : Future(Never()))) { - if (DEBUG_CF_VERSION(nextVersion)) { + if (DEBUG_CF_VERSION(feedData->id, nextVersion)) { fmt::print(" single {0} {1}: WAL {2}, sending empty {3})\n", idx, interf.id().toString().substr(0, 4), @@ -7381,7 +7389,7 @@ ACTOR Future partialChangeFeedStream(StorageServerInterface interf, } when(wait(atLatestVersion && replyStream.isEmpty() && !results.isEmpty() ? results.onEmpty() : Future(Never()))) { - if (DEBUG_CF_VERSION(nextVersion)) { + if (DEBUG_CF_VERSION(feedData->id, nextVersion)) { fmt::print(" single {0} {1}: got onEmpty\n", idx, interf.id().toString().substr(0, 4)); } } @@ -7442,7 +7450,7 @@ ACTOR Future doCFMerge(Reference results, if (nextStream.next.version != checkVersion) { if (nextOut.size()) { *begin = checkVersion + 1; - if (DEBUG_CF_VERSION(nextOut.back().version)) { + if (DEBUG_CF_VERSION(results->id, nextOut.back().version)) { fmt::print("CFNA (merged): {0} ({1})\n", nextOut.back().version, nextOut.back().mutations.size()); } @@ -7462,7 +7470,7 @@ ACTOR Future doCFMerge(Reference results, } else { ASSERT(results->mutations.isEmpty()); } - if (DEBUG_CF_VERSION(nextOut.back().version)) { + if (DEBUG_CF_VERSION(results->id, nextOut.back().version)) { fmt::print("CFLR (merged): {0} (1)\n", nextOut.back().version, nextOut.back().mutations.size()); } if (nextOut.back().version > results->lastReturnedVersion.get()) { @@ -7494,7 +7502,7 @@ ACTOR Future doCFMerge(Reference results, if (atCheckVersion == maxAtCheckVersion) { ASSERT(nextOut.size() == 1); *begin = checkVersion + 1; - if (DEBUG_CF_VERSION(nextOut.back().version)) { + if (DEBUG_CF_VERSION(results->id, nextOut.back().version)) { fmt::print("CFNA (merged@all): {0} ({1})\n", nextOut.back().version, nextOut.back().mutations.size()); } @@ -7518,7 +7526,7 @@ ACTOR Future doCFMerge(Reference results, } else { ASSERT(results->mutations.isEmpty()); } - if (DEBUG_CF_VERSION(nextOut.back().version)) { + if (DEBUG_CF_VERSION(results->id, nextOut.back().version)) { fmt::print("CFLR (merged@all): {0} ({1})\n", nextOut.back().version, nextOut.back().mutations.size()); } if (nextOut.back().version > results->lastReturnedVersion.get()) { @@ -7529,7 +7537,7 @@ ACTOR Future doCFMerge(Reference results, } try { Standalone res = waitNext(nextStream.results.getFuture()); - if (DEBUG_CF_VERSION(res.version)) { + if (DEBUG_CF_VERSION(results->id, res.version)) { fmt::print(" CFNA (merge1): {0} ({1})\n", res.version, res.mutations.size()); } nextStream.next = res; @@ -7548,7 +7556,7 @@ ACTOR Future doCFMerge(Reference results, ASSERT(results->mutations.isEmpty()); wait(delay(0)); } - if (DEBUG_CF_VERSION(nextOut.back().version)) { + if (DEBUG_CF_VERSION(results->id, nextOut.back().version)) { fmt::print("CFLR (merged): {0} (1)\n", nextOut.back().version); } if (nextOut.back().version > results->lastReturnedVersion.get()) { @@ -7592,7 +7600,7 @@ ACTOR Future mergeChangeFeedStream(Reference db, for (auto& it : results->storageData) { if (it->debugGetReferenceCount() == 2) { - db->changeFeedUpdaters.erase(it->id); + db->changeFeedUpdaters.erase(it->interfToken); } } results->maxSeenVersion = invalidVersion; @@ -7689,7 +7697,7 @@ ACTOR Future doSingleCFStream(KeyRange range, wait(delay(0)); } - if (DEBUG_CF_VERSION(feedReply.mutations.back().version)) { + if (DEBUG_CF_VERSION(rangeID, feedReply.mutations.back().version)) { fmt::print("CFLR (single): {0} ({1}), atLatest={2}, rep.atLatest={3}, notAtLatest={4}, " "minSV={5}\n", feedReply.mutations.back().version, @@ -7735,7 +7743,7 @@ ACTOR Future singleChangeFeedStream(Reference db, for (auto& it : results->storageData) { if (it->debugGetReferenceCount() == 2) { - db->changeFeedUpdaters.erase(it->id); + db->changeFeedUpdaters.erase(it->interfToken); } } results->streams.push_back(interf.changeFeedStream.getReplyStream(req)); @@ -7762,6 +7770,8 @@ ACTOR Future getChangeFeedStreamActor(Reference db, state Database cx(db); state Span span("NAPI:GetChangeFeedStream"_loc); + results->id = rangeID; + /*printf("CFStream %s [%s - %s): [%lld - %lld]\n", rangeID.printable().substr(0, 6).c_str(), range.begin.printable().c_str(), @@ -7844,7 +7854,7 @@ ACTOR Future getChangeFeedStreamActor(Reference db, if (e.code() == error_code_actor_cancelled) { for (auto& it : results->storageData) { if (it->debugGetReferenceCount() == 2) { - db->changeFeedUpdaters.erase(it->id); + db->changeFeedUpdaters.erase(it->interfToken); } } results->streams.clear(); @@ -7869,7 +7879,7 @@ ACTOR Future getChangeFeedStreamActor(Reference db, results->refresh.sendError(change_feed_cancelled()); for (auto& it : results->storageData) { if (it->debugGetReferenceCount() == 2) { - db->changeFeedUpdaters.erase(it->id); + db->changeFeedUpdaters.erase(it->interfToken); } } results->streams.clear(); diff --git a/fdbserver/worker.actor.cpp b/fdbserver/worker.actor.cpp index 0fb24463f6..d3d3ec95f3 100644 --- a/fdbserver/worker.actor.cpp +++ b/fdbserver/worker.actor.cpp @@ -1518,6 +1518,9 @@ ACTOR Future workerServer(Reference connRecord, DUMPTOKEN(recruited.watchValue); DUMPTOKEN(recruited.getKeyValuesStream); DUMPTOKEN(recruited.getKeyValuesAndFlatMap); + DUMPTOKEN(recruited.changeFeedStream); + DUMPTOKEN(recruited.changeFeedPop); + DUMPTOKEN(recruited.changeFeedVersionUpdate); Promise recovery; Future f = storageServer(kv, recruited, dbInfo, folder, recovery, connRecord); @@ -1991,6 +1994,9 @@ ACTOR Future workerServer(Reference connRecord, DUMPTOKEN(recruited.watchValue); DUMPTOKEN(recruited.getKeyValuesStream); DUMPTOKEN(recruited.getKeyValuesAndFlatMap); + DUMPTOKEN(recruited.changeFeedStream); + DUMPTOKEN(recruited.changeFeedPop); + DUMPTOKEN(recruited.changeFeedVersionUpdate); // printf("Recruited as storageServer\n"); std::string filename = From 48cad574d433b0236c5b9f82a296a6f229f0ce03 Mon Sep 17 00:00:00 2001 From: Josh Slocum Date: Wed, 12 Jan 2022 09:52:55 -0600 Subject: [PATCH 119/413] Enforce max fanout for blob granule splits --- fdbserver/BlobManager.actor.cpp | 48 +++++++++++++++++++++++++++++++++ 1 file changed, 48 insertions(+) diff --git a/fdbserver/BlobManager.actor.cpp b/fdbserver/BlobManager.actor.cpp index b28443ca06..b7a9b88cab 100644 --- a/fdbserver/BlobManager.actor.cpp +++ b/fdbserver/BlobManager.actor.cpp @@ -678,6 +678,32 @@ ACTOR Future monitorClientRanges(BlobManagerData* bmData) { } } +// split recursively in the middle to guarantee roughly equal splits across different parts of key space +static void downsampleSplit(const Standalone>& splits, + Standalone>& out, + int startIdx, + int endIdx, + int remaining) { + ASSERT(endIdx - startIdx >= remaining); + ASSERT(remaining >= 0); + if (remaining == 0) { + return; + } + if (endIdx - startIdx == remaining) { + out.append(out.arena(), splits.begin() + startIdx, remaining); + } else { + int mid = (startIdx + endIdx) / 2; + int startCount = (remaining - 1) / 2; + int endCount = remaining - startCount - 1; + // ensure no infinite recursion + ASSERT(mid != endIdx); + ASSERT(mid + 1 != startIdx); + downsampleSplit(splits, out, startIdx, mid, startCount); + out.push_back(out.arena(), splits[mid]); + downsampleSplit(splits, out, mid + 1, endIdx, endCount); + } +} + ACTOR Future maybeSplitRange(BlobManagerData* bmData, UID currentWorkerId, KeyRange granuleRange, @@ -709,6 +735,28 @@ ACTOR Future maybeSplitRange(BlobManagerData* bmData, return Void(); } + // TODO KNOB for this. + // Enforce max split fanout of 10 for performance reasons + int maxSplitFanout = 10; + if (newRanges.size() >= maxSplitFanout + 2) { // +2 because this is boundaries, so N keys would have N+1 bounaries. + TEST(true); // downsampling granule split because fanout too high + Standalone> coalescedRanges; + coalescedRanges.arena().dependsOn(newRanges.arena()); + coalescedRanges.push_back(coalescedRanges.arena(), newRanges.front()); + + // since we include start + end boundaries here, only need maxSplitFanout-1 split boundaries to produce + // maxSplitFanout granules + downsampleSplit(newRanges, coalescedRanges, 1, newRanges.size() - 1, maxSplitFanout - 1); + + coalescedRanges.push_back(coalescedRanges.arena(), newRanges.back()); + ASSERT(coalescedRanges.size() == maxSplitFanout + 1); + if (BM_DEBUG) { + fmt::print("Downsampled split from {0} -> {1} granules", newRanges.size() - 1, maxSplitFanout); + } + + newRanges = coalescedRanges; + } + if (BM_DEBUG) { fmt::print("Splitting range [{0} - {1}) into {2} granules @ {3}:\n", granuleRange.begin.printable(), From fe9a3d27d8b2e79af9130e2c3da12fb4da94b90f Mon Sep 17 00:00:00 2001 From: Josh Slocum Date: Wed, 12 Jan 2022 13:16:49 -0600 Subject: [PATCH 120/413] Not allowing whenAtLeast after change feed end version --- fdbclient/DatabaseContext.h | 1 + fdbclient/NativeAPI.actor.cpp | 9 +++++++++ 2 files changed, 10 insertions(+) diff --git a/fdbclient/DatabaseContext.h b/fdbclient/DatabaseContext.h index ac567fcd76..1a89464fe4 100644 --- a/fdbclient/DatabaseContext.h +++ b/fdbclient/DatabaseContext.h @@ -179,6 +179,7 @@ struct ChangeFeedData : ReferenceCounted { AsyncVar notAtLatest; Promise refresh; Version maxSeenVersion; + Version endVersion = invalidVersion; ChangeFeedData() : notAtLatest(1) {} }; diff --git a/fdbclient/NativeAPI.actor.cpp b/fdbclient/NativeAPI.actor.cpp index b7bc59b664..c003319061 100644 --- a/fdbclient/NativeAPI.actor.cpp +++ b/fdbclient/NativeAPI.actor.cpp @@ -7252,6 +7252,14 @@ ACTOR Future changeFeedWaitLatest(Reference self, Version } ACTOR Future changeFeedWhenAtLatest(Reference self, Version version) { + if (version >= self->endVersion) { + if (DEBUG_CF_WAIT(self->id, version)) { + fmt::print( + "CFW {0}) WhenAtLeast: After CF end version {1}, returning Never()\n", version, self->endVersion); + } + return Never(); + } + if (DEBUG_CF_WAIT(self->id, version)) { fmt::print("CFW {0}) WhenAtLeast: LR={1}\n", version, self->lastReturnedVersion.get()); } @@ -7771,6 +7779,7 @@ ACTOR Future getChangeFeedStreamActor(Reference db, state Span span("NAPI:GetChangeFeedStream"_loc); results->id = rangeID; + results->endVersion = end; /*printf("CFStream %s [%s - %s): [%lld - %lld]\n", rangeID.printable().substr(0, 6).c_str(), From cdb433569e2e5083b29c5d7eaf52bc3431628511 Mon Sep 17 00:00:00 2001 From: Josh Slocum Date: Wed, 12 Jan 2022 13:00:06 -0600 Subject: [PATCH 121/413] fixed range bug in filterMutationsInverted --- fdbserver/storageserver.actor.cpp | 32 +++++++++++++++++++++++++++++-- 1 file changed, 30 insertions(+), 2 deletions(-) diff --git a/fdbserver/storageserver.actor.cpp b/fdbserver/storageserver.actor.cpp index f0ee40a0d7..3b2e63dbcd 100644 --- a/fdbserver/storageserver.actor.cpp +++ b/fdbserver/storageserver.actor.cpp @@ -1670,8 +1670,7 @@ MutationsAndVersionRef filterMutationsInverted(Arena& arena, MutationsAndVersion } else { ASSERT(m.mutations[i].type == MutationRef::ClearRange); if (!modifiedMutations.present() && - ((m.mutations[i].param1 < range.begin && m.mutations[i].param2 > range.begin) || - (m.mutations[i].param2 > range.end && m.mutations[i].param1 < range.end))) { + (m.mutations[i].param2 > range.begin && m.mutations[i].param1 < range.end)) { modifiedMutations = m.mutations.slice(0, i); arena.dependsOn(range.arena()); } @@ -3909,6 +3908,14 @@ ACTOR Future fetchChangeFeedApplier(StorageServer* data, Standalone> res = waitNext(feedResults->mutations.getFuture()); for (auto& it : res) { if (it.mutations.size()) { + if (MUTATION_TRACKING_ENABLED) { + for (auto& m : it.mutations) { + DEBUG_MUTATION("ChangeFeedWriteMove", it.version, m, data->thisServerID) + .detail("Range", range) + .detail("Existing", existing) + .detail("ChangeFeedID", rangeId); + } + } data->storage.writeKeyValue( KeyValueRef(changeFeedDurableKey(rangeId, it.version), changeFeedDurableValue(it.mutations, it.knownCommittedVersion))); @@ -3943,6 +3950,15 @@ ACTOR Future fetchChangeFeedApplier(StorageServer* data, while (remoteLoc < remoteResult.size()) { if (remoteResult[remoteLoc].version < localResult.version) { if (remoteResult[remoteLoc].mutations.size()) { + if (MUTATION_TRACKING_ENABLED) { + for (auto& m : remoteResult[remoteLoc].mutations) { + DEBUG_MUTATION( + "ChangeFeedWriteMove", remoteResult[remoteLoc].version, m, data->thisServerID) + .detail("Range", range) + .detail("Existing", existing) + .detail("ChangeFeedID", rangeId); + } + } data->storage.writeKeyValue( KeyValueRef(changeFeedDurableKey(rangeId, remoteResult[remoteLoc].version), changeFeedDurableValue(remoteResult[remoteLoc].mutations, @@ -3954,9 +3970,21 @@ ACTOR Future fetchChangeFeedApplier(StorageServer* data, } else if (remoteResult[remoteLoc].version == localResult.version) { if (remoteResult[remoteLoc].mutations.size() && remoteResult[remoteLoc].mutations.back().param1 != lastEpochEndPrivateKey) { + int remoteSize = remoteResult[remoteLoc].mutations.size(); ASSERT(localResult.mutations.size()); remoteResult[remoteLoc].mutations.append( remoteResult.arena(), localResult.mutations.begin(), localResult.mutations.size()); + if (MUTATION_TRACKING_ENABLED) { + int midx = 0; + for (auto& m : remoteResult[remoteLoc].mutations) { + DEBUG_MUTATION( + "ChangeFeedWriteMoveMerge", remoteResult[remoteLoc].version, m, data->thisServerID) + .detail("Range", range) + .detail("FromLocal", midx >= remoteSize) + .detail("ChangeFeedID", rangeId); + midx++; + } + } data->storage.writeKeyValue( KeyValueRef(changeFeedDurableKey(rangeId, remoteResult[remoteLoc].version), changeFeedDurableValue(remoteResult[remoteLoc].mutations, From 05d2eafd0cf4f11e1edb57f610442dbf7c35b0a5 Mon Sep 17 00:00:00 2001 From: Josh Slocum Date: Wed, 12 Jan 2022 18:28:31 -0600 Subject: [PATCH 122/413] Correctly updating change feed storage and durable versions after a fetch --- fdbserver/storageserver.actor.cpp | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/fdbserver/storageserver.actor.cpp b/fdbserver/storageserver.actor.cpp index 3b2e63dbcd..53c8d2c70d 100644 --- a/fdbserver/storageserver.actor.cpp +++ b/fdbserver/storageserver.actor.cpp @@ -5567,6 +5567,12 @@ ACTOR Future updateStorage(StorageServer* data) { // handle case where fetch had version ahead of last in-memory mutation if (info->second->fetchVersion > info->second->storageVersion) { info->second->storageVersion = std::min(info->second->fetchVersion, newOldestVersion); + if (info->second->fetchVersion > info->second->storageVersion) { + // This change feed still has pending mutations fetched and written to storage that are not yet + // durable. To ensure its storageVersion gets updated once its fetchVersion is durable, we need + // to add it back to fetchingChangeFeeds + data->fetchingChangeFeeds.insert(info->first); + } } wait(yield(TaskPriority::UpdateStorage)); } From 515dc4015354b1fdb301fa11f73260977951a674 Mon Sep 17 00:00:00 2001 From: Josh Slocum Date: Thu, 13 Jan 2022 08:43:02 -0600 Subject: [PATCH 123/413] Fixed issue with out of order empty versions --- fdbclient/NativeAPI.actor.cpp | 8 ++++++-- 1 file changed, 6 insertions(+), 2 deletions(-) diff --git a/fdbclient/NativeAPI.actor.cpp b/fdbclient/NativeAPI.actor.cpp index c003319061..35e6b26142 100644 --- a/fdbclient/NativeAPI.actor.cpp +++ b/fdbclient/NativeAPI.actor.cpp @@ -7364,7 +7364,10 @@ ACTOR Future partialChangeFeedStream(StorageServerInterface interf, } resultLoc++; } - nextVersion = rep.mutations.back().version + 1; + // if we got the empty version that went backwards, don't decrease nextVersion + if (rep.mutations.back().version + 1 > nextVersion) { + nextVersion = rep.mutations.back().version + 1; + } if (!atLatestVersion && rep.atLatestVersion) { atLatestVersion = true; @@ -7450,7 +7453,7 @@ ACTOR Future doCFMerge(Reference results, if (nextStream.next.version == checkVersion) { // TODO REMOVE if (atCheckVersion == 0) { - printf("atCheckVersion == 0 at %lld\n", nextStream.next.version); + printf("atCheckVersion %lld == 0 at %lld\n", checkVersion, nextStream.next.version); } ASSERT(atCheckVersion > 0); } @@ -7548,6 +7551,7 @@ ACTOR Future doCFMerge(Reference results, if (DEBUG_CF_VERSION(results->id, res.version)) { fmt::print(" CFNA (merge1): {0} ({1})\n", res.version, res.mutations.size()); } + ASSERT(res.version > nextStream.next.version); nextStream.next = res; mutations.push(nextStream); } catch (Error& e) { From 6e361a3283d3a3ceb1ba3cd183816c85814a8591 Mon Sep 17 00:00:00 2001 From: Josh Slocum Date: Thu, 13 Jan 2022 09:48:17 -0600 Subject: [PATCH 124/413] Need delay(0) for empty check at start of change feed --- fdbclient/NativeAPI.actor.cpp | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/fdbclient/NativeAPI.actor.cpp b/fdbclient/NativeAPI.actor.cpp index 35e6b26142..0ff08a05c7 100644 --- a/fdbclient/NativeAPI.actor.cpp +++ b/fdbclient/NativeAPI.actor.cpp @@ -7429,6 +7429,8 @@ ACTOR Future doCFMerge(Reference results, // previous version of change feed may have put a mutation in the promise stream and then immediately died. Wait for // that mutation first, so the promise stream always starts empty wait(results->mutations.onEmpty()); + wait(delay(0)); + ASSERT(results->mutations.isEmpty()); while (interfNum < interfs.size()) { try { @@ -7677,7 +7679,10 @@ ACTOR Future doSingleCFStream(KeyRange range, state bool atLatest = false; loop { + // wait for any previous mutations in stream to be consumed wait(results->mutations.onEmpty()); + wait(delay(0)); + ASSERT(results->mutations.isEmpty()); state ChangeFeedStreamReply feedReply = waitNext(results->streams[0].getFuture()); *begin = feedReply.mutations.back().version + 1; From e7d15b29477977725a92288761389cfcf2918a67 Mon Sep 17 00:00:00 2001 From: Josh Slocum Date: Thu, 13 Jan 2022 12:10:10 -0600 Subject: [PATCH 125/413] more explicit empty version problems --- fdbclient/NativeAPI.actor.cpp | 24 +++++++++++++----------- 1 file changed, 13 insertions(+), 11 deletions(-) diff --git a/fdbclient/NativeAPI.actor.cpp b/fdbclient/NativeAPI.actor.cpp index 0ff08a05c7..f223a76d4b 100644 --- a/fdbclient/NativeAPI.actor.cpp +++ b/fdbclient/NativeAPI.actor.cpp @@ -7686,17 +7686,6 @@ ACTOR Future doSingleCFStream(KeyRange range, state ChangeFeedStreamReply feedReply = waitNext(results->streams[0].getFuture()); *begin = feedReply.mutations.back().version + 1; - // TODO REMOVE, for debugging - if (feedReply.mutations.back().version < results->lastReturnedVersion.get()) { - printf("out of order mutation for CF %s from (%d) %s! %lld < %lld\n", - rangeID.toString().substr(0, 6).c_str(), - results->storageData.size(), - results->storageData.empty() ? "????" : results->storageData[0]->id.toString().substr(0, 4).c_str(), - feedReply.mutations.back().version, - results->lastReturnedVersion.get()); - } - ASSERT(feedReply.mutations.back().version >= results->lastReturnedVersion.get()); - // don't send completely empty set of mutations to promise stream bool anyMutations = false; for (auto& it : feedReply.mutations) { @@ -7706,6 +7695,19 @@ ACTOR Future doSingleCFStream(KeyRange range, } } if (anyMutations) { + // empty versions can come out of order, as we sometimes send explicit empty versions when restarting a + // stream. Anything with mutations should be strictly greater than lastReturnedVersion + if (feedReply.mutations.front().version <= results->lastReturnedVersion.get()) { + printf("out of order mutation for CF %s from (%d) %s! %lld < %lld\n", + rangeID.toString().substr(0, 6).c_str(), + results->storageData.size(), + results->storageData.empty() ? "????" + : results->storageData[0]->id.toString().substr(0, 4).c_str(), + feedReply.mutations.front().version, + results->lastReturnedVersion.get()); + } + ASSERT(feedReply.mutations.front().version > results->lastReturnedVersion.get()); + results->mutations.send( Standalone>(feedReply.mutations, feedReply.arena)); From 661c50d29f1b11ecb839c74826c63ee9fec29c9b Mon Sep 17 00:00:00 2001 From: Josh Slocum Date: Thu, 13 Jan 2022 16:29:25 -0600 Subject: [PATCH 126/413] Handled server_overloaded in storageFeedVersionUpdater --- fdbclient/NativeAPI.actor.cpp | 28 ++++++++++++++++++++-------- fdbserver/BlobManager.actor.cpp | 8 +++++++- fdbserver/BlobWorker.actor.cpp | 3 +++ 3 files changed, 30 insertions(+), 9 deletions(-) diff --git a/fdbclient/NativeAPI.actor.cpp b/fdbclient/NativeAPI.actor.cpp index f223a76d4b..f371da41c2 100644 --- a/fdbclient/NativeAPI.actor.cpp +++ b/fdbclient/NativeAPI.actor.cpp @@ -7090,16 +7090,28 @@ ACTOR Future storageFeedVersionUpdater(StorageServerInterface interf, Chan if (self->debug) { fmt::print("CFSD {0}: requesting {1}\n", self->id.toString().substr(0, 4), self->desired.get()); } - ChangeFeedVersionUpdateReply rep = wait(brokenPromiseToNever( - interf.changeFeedVersionUpdate.getReply(ChangeFeedVersionUpdateRequest(self->desired.get())))); - if (self->debug) { - fmt::print("CFSD {0}: got {1}\n", self->id.toString().substr(0, 4), rep.version); - } - if (rep.version > self->version.get()) { + try { + ChangeFeedVersionUpdateReply rep = wait(brokenPromiseToNever( + interf.changeFeedVersionUpdate.getReply(ChangeFeedVersionUpdateRequest(self->desired.get())))); + if (self->debug) { - fmt::print("CFSD {0}: V={1} (req)\n", self->id.toString().substr(0, 4), rep.version); + fmt::print("CFSD {0}: got {1}\n", self->id.toString().substr(0, 4), rep.version); + } + if (rep.version > self->version.get()) { + if (self->debug) { + fmt::print("CFSD {0}: V={1} (req)\n", self->id.toString().substr(0, 4), rep.version); + } + self->version.set(rep.version); + } + } catch (Error& e) { + if (e.code() == error_code_server_overloaded) { + if (FLOW_KNOBS->PREVENT_FAST_SPIN_DELAY > CLIENT_KNOBS->CHANGE_FEED_EMPTY_BATCH_TIME) { + wait(delay(FLOW_KNOBS->PREVENT_FAST_SPIN_DELAY - + CLIENT_KNOBS->CHANGE_FEED_EMPTY_BATCH_TIME)); + } + } else { + throw e; } - self->version.set(rep.version); } } } else { diff --git a/fdbserver/BlobManager.actor.cpp b/fdbserver/BlobManager.actor.cpp index b7a9b88cab..440d03435d 100644 --- a/fdbserver/BlobManager.actor.cpp +++ b/fdbserver/BlobManager.actor.cpp @@ -535,12 +535,14 @@ ACTOR Future writeInitialGranuleMapping(BlobManagerData* bmData, Standalon while (i < boundaries.size() - 1) { TEST(i > 0); // multiple transactions for large granule split tr->reset(); + // TODO COMMENT AND USE BELOW INSTEAD tr->setOption(FDBTransactionOptions::Option::PRIORITY_SYSTEM_IMMEDIATE); tr->setOption(FDBTransactionOptions::Option::ACCESS_SYSTEM_KEYS); state int j = 0; loop { try { - + // tr->setOption(FDBTransactionOptions::Option::PRIORITY_SYSTEM_IMMEDIATE); + // tr->setOption(FDBTransactionOptions::Option::ACCESS_SYSTEM_KEYS); while (i + j < boundaries.size() - 1 && j < transactionChunkSize) { // TODO REMOVE if (BM_DEBUG) { @@ -558,6 +560,9 @@ ACTOR Future writeInitialGranuleMapping(BlobManagerData* bmData, Standalon wait(tr->commit()); break; } catch (Error& e) { + if (BM_DEBUG) { + printf("Persisting initial mapping got error %s\n", e.name()); + } wait(tr->onError(e)); j = 0; } @@ -2152,6 +2157,7 @@ ACTOR Future blobManager(BlobManagerInterface bmInterf, tr->setOption(FDBTransactionOptions::ACCESS_SYSTEM_KEYS); tr->setOption(FDBTransactionOptions::PRIORITY_SYSTEM_IMMEDIATE); + // TODO add loop with retries here!!! try { wait(checkManagerLock(tr, &self)); } catch (Error& e) { diff --git a/fdbserver/BlobWorker.actor.cpp b/fdbserver/BlobWorker.actor.cpp index 8e070fe3c0..b7a1bc57d4 100644 --- a/fdbserver/BlobWorker.actor.cpp +++ b/fdbserver/BlobWorker.actor.cpp @@ -700,6 +700,9 @@ ACTOR Future dumpInitialSnapshotFromFDB(Reference } } BlobFileIndex f = wait(snapshotWriter); + TraceEvent("BlobGranuleSnapshotFile", bwData->id) + .detail("Granule", metadata->keyRange) + .detail("Version", readVersion); DEBUG_KEY_RANGE("BlobWorkerFDBSnapshot", readVersion, metadata->keyRange, bwData->id); return f; } catch (Error& e) { From abfb3a7e82baeaee78166eb5867dd8c2adf65acb Mon Sep 17 00:00:00 2001 From: Josh Slocum Date: Fri, 14 Jan 2022 09:07:42 -0600 Subject: [PATCH 127/413] Fixed bug and removed unecessary check in blob manager --- fdbserver/BlobManager.actor.cpp | 26 ++------------------------ 1 file changed, 2 insertions(+), 24 deletions(-) diff --git a/fdbserver/BlobManager.actor.cpp b/fdbserver/BlobManager.actor.cpp index 440d03435d..f908706e1d 100644 --- a/fdbserver/BlobManager.actor.cpp +++ b/fdbserver/BlobManager.actor.cpp @@ -535,14 +535,11 @@ ACTOR Future writeInitialGranuleMapping(BlobManagerData* bmData, Standalon while (i < boundaries.size() - 1) { TEST(i > 0); // multiple transactions for large granule split tr->reset(); - // TODO COMMENT AND USE BELOW INSTEAD - tr->setOption(FDBTransactionOptions::Option::PRIORITY_SYSTEM_IMMEDIATE); - tr->setOption(FDBTransactionOptions::Option::ACCESS_SYSTEM_KEYS); state int j = 0; loop { try { - // tr->setOption(FDBTransactionOptions::Option::PRIORITY_SYSTEM_IMMEDIATE); - // tr->setOption(FDBTransactionOptions::Option::ACCESS_SYSTEM_KEYS); + tr->setOption(FDBTransactionOptions::Option::PRIORITY_SYSTEM_IMMEDIATE); + tr->setOption(FDBTransactionOptions::Option::ACCESS_SYSTEM_KEYS); while (i + j < boundaries.size() - 1 && j < transactionChunkSize) { // TODO REMOVE if (BM_DEBUG) { @@ -2152,25 +2149,6 @@ ACTOR Future blobManager(BlobManagerInterface bmInterf, self.epoch = epoch; - // make sure the epoch hasn't gotten stale - state Reference tr = makeReference(self.db); - - tr->setOption(FDBTransactionOptions::ACCESS_SYSTEM_KEYS); - tr->setOption(FDBTransactionOptions::PRIORITY_SYSTEM_IMMEDIATE); - // TODO add loop with retries here!!! - try { - wait(checkManagerLock(tr, &self)); - } catch (Error& e) { - if (BM_DEBUG) { - printf("Blob manager lock check got unexpected error %s. Dying...\n", e.name()); - } - return Void(); - } - - if (BM_DEBUG) { - fmt::print("Blob manager acquired lock at epoch {}\n", epoch); - } - // although we start the recruiter, we wait until existing workers are ack'd auto recruitBlobWorker = IAsyncListener>::create( dbInfo, [](auto const& info) { return info.clusterInterface.recruitBlobWorker; }); From 0e58478c4d5c38a34e9c660f0b132ee10854b33a Mon Sep 17 00:00:00 2001 From: Josh Slocum Date: Fri, 14 Jan 2022 10:28:54 -0600 Subject: [PATCH 128/413] Initial rework of merge cursor --- fdbclient/NativeAPI.actor.cpp | 354 +++++++++++++++++++++++----------- 1 file changed, 237 insertions(+), 117 deletions(-) diff --git a/fdbclient/NativeAPI.actor.cpp b/fdbclient/NativeAPI.actor.cpp index f371da41c2..0931fe4db5 100644 --- a/fdbclient/NativeAPI.actor.cpp +++ b/fdbclient/NativeAPI.actor.cpp @@ -7435,8 +7435,9 @@ ACTOR Future doCFMerge(Reference results, std::vector streams, Version* begin, Version end) { + // with empty version handling in the partial cursor, all streams will always have a next element with version >= + // the minimum version of any stream's next element state std::priority_queue> mutations; - state int interfNum = 0; // previous version of change feed may have put a mutation in the promise stream and then immediately died. Wait for // that mutation first, so the promise stream always starts empty @@ -7444,6 +7445,8 @@ ACTOR Future doCFMerge(Reference results, wait(delay(0)); ASSERT(results->mutations.isEmpty()); + // push one initial mutation from each stream + state int interfNum = 0; while (interfNum < interfs.size()) { try { Standalone res = waitNext(streams[interfNum].results.getFuture()); @@ -7456,142 +7459,259 @@ ACTOR Future doCFMerge(Reference results, } interfNum++; } - state int atCheckVersion = 0; - state int maxAtCheckVersion = -1; - state Version checkVersion = invalidVersion; - state Standalone> nextOut; - while (mutations.size()) { - state MutationAndVersionStream nextStream = mutations.top(); + // Without this delay, weird issues with the last stream getting on another stream's callstack can happen + wait(delay(0)); + + // TODO minor optimization - could make this just a vector of indexes if each MutationAndVersionStream remembered + // its version index + state std::vector streamsUsed; + streamsUsed.reserve(interfs.size()); + // TODO REMOVE - i always mess reserve vs resize up + ASSERT(streamsUsed.size() == 0); + ASSERT(streamsUsed.capacity() == interfs.size()); + + loop { + if (streams.empty()) { + throw end_of_stream(); + } + + streamsUsed.clear(); + + // pop first item off queue - this will be mutation with the lowest version + Standalone> nextOut; + state Version nextVersion = nextVersion = mutations.top().next.version; + + streamsUsed.push_back(mutations.top()); + nextOut.push_back_deep(nextOut.arena(), mutations.top().next); mutations.pop(); - ASSERT(nextStream.next.version >= checkVersion); - if (nextStream.next.version == checkVersion) { - // TODO REMOVE - if (atCheckVersion == 0) { - printf("atCheckVersion %lld == 0 at %lld\n", checkVersion, nextStream.next.version); - } - ASSERT(atCheckVersion > 0); - } - if (nextStream.next.version != checkVersion) { - if (nextOut.size()) { - *begin = checkVersion + 1; - if (DEBUG_CF_VERSION(results->id, nextOut.back().version)) { - fmt::print("CFNA (merged): {0} ({1})\n", nextOut.back().version, nextOut.back().mutations.size()); - } - - if (nextOut.back().version < results->lastReturnedVersion.get()) { - fmt::print("ERROR: merge cursor pushing next out {} <= lastReturnedVersion {}\n", - nextOut.back().version, - results->lastReturnedVersion.get()); - } - // We can get an empty version pushed through the stream if whenAtLeast is called. Ignore - // it - ASSERT(nextOut.size() == 1); - if (!nextOut.back().mutations.empty()) { - ASSERT(nextOut.back().version >= results->lastReturnedVersion.get()); - results->mutations.send(nextOut); - wait(results->mutations.onEmpty()); - wait(delay(0)); - } else { - ASSERT(results->mutations.isEmpty()); - } - if (DEBUG_CF_VERSION(results->id, nextOut.back().version)) { - fmt::print("CFLR (merged): {0} (1)\n", nextOut.back().version, nextOut.back().mutations.size()); - } - if (nextOut.back().version > results->lastReturnedVersion.get()) { - results->lastReturnedVersion.set(nextOut.back().version); - } - nextOut = Standalone>(); - } - checkVersion = nextStream.next.version; - atCheckVersion = 0; - // save this at the start of the "round" to check if all streams have a reply at version checkVersion. - // If so, we can send it early. But because mutations.size() can change if one of the streams gets - // end_of_version on its waitNext, we want that to be reflected in the maxAtCheckVersion for the NEXT - // version, not this one. - maxAtCheckVersion = mutations.size() + 1; - } - if (nextOut.size() && nextStream.next.version == nextOut.back().version) { - if (nextStream.next.mutations.size() && - nextStream.next.mutations.front().param1 != lastEpochEndPrivateKey) { + // for each other mutation with the same version, add it to nextOut + while (!mutations.empty() && mutations.top().next.version == nextVersion) { + if (mutations.top().next.mutations.size() && + mutations.top().next.mutations.front().param1 != lastEpochEndPrivateKey) { nextOut.back().mutations.append_deep( - nextOut.arena(), nextStream.next.mutations.begin(), nextStream.next.mutations.size()); + nextOut.arena(), mutations.top().next.mutations.begin(), mutations.top().next.mutations.size()); } - } else { - nextOut.push_back_deep(nextOut.arena(), nextStream.next); + streamsUsed.push_back(mutations.top()); + mutations.pop(); } - atCheckVersion++; - // TODO AVOID CODE DUPLICATION - // If all streams have returned something at this version, we know it is complete. - if (atCheckVersion == maxAtCheckVersion) { - ASSERT(nextOut.size() == 1); - *begin = checkVersion + 1; - if (DEBUG_CF_VERSION(results->id, nextOut.back().version)) { - fmt::print("CFNA (merged@all): {0} ({1})\n", nextOut.back().version, nextOut.back().mutations.size()); - } + ASSERT(nextOut.size() == 1); + ASSERT(nextVersion >= *begin); + *begin = nextVersion + 1; + if (DEBUG_CF_VERSION(results->id, nextOut.back().version)) { + fmt::print("CFNA (merged): {0} ({1})\n", nextOut.back().version, nextOut.back().mutations.size()); + } + + if (nextOut.back().mutations.empty()) { + // TODO REMOVE, for debugging if (nextOut.back().version < results->lastReturnedVersion.get()) { - printf("ERROR: merge cursor@all pushing next out <= lastReturnedVersion"); + printf("ERROR: merge cursor got empty version <= lastReturnedVersion"); } - // We can get an empty version pushed through the stream if whenAtLeast is called. Ignore - // it + ASSERT(nextOut.back().version >= results->lastReturnedVersion.get()); - if (!nextOut.back().mutations.empty()) { - if (nextOut.back().version < results->lastReturnedVersion.get()) { - fmt::print("Merged all version went backwards!! mutation version {} < lastReturnedVersion {}\n", - nextOut.back().version, - results->lastReturnedVersion.get()); - } - ASSERT(nextOut.back().version >= results->lastReturnedVersion.get()); - results->mutations.send(nextOut); - wait(results->mutations.onEmpty()); - ASSERT(results->mutations.isEmpty()); - wait(delay(0)); - } else { - ASSERT(results->mutations.isEmpty()); + ASSERT(results->mutations.isEmpty()); + } else { + // TODO REMOVE, for debugging + if (nextOut.back().version <= results->lastReturnedVersion.get()) { + printf("ERROR: merge cursor got mutations <= lastReturnedVersion"); } - if (DEBUG_CF_VERSION(results->id, nextOut.back().version)) { - fmt::print("CFLR (merged@all): {0} ({1})\n", nextOut.back().version, nextOut.back().mutations.size()); - } - if (nextOut.back().version > results->lastReturnedVersion.get()) { - results->lastReturnedVersion.set(nextOut.back().version); - } - nextOut = Standalone>(); - atCheckVersion = 0; - } - try { - Standalone res = waitNext(nextStream.results.getFuture()); - if (DEBUG_CF_VERSION(results->id, res.version)) { - fmt::print(" CFNA (merge1): {0} ({1})\n", res.version, res.mutations.size()); - } - ASSERT(res.version > nextStream.next.version); - nextStream.next = res; - mutations.push(nextStream); - } catch (Error& e) { - if (e.code() != error_code_end_of_stream) { - throw e; - } - } - } - if (nextOut.size()) { - ASSERT(nextOut.back().version >= results->lastReturnedVersion.get()); - if (!nextOut.back().mutations.empty()) { + ASSERT(nextOut.back().version > results->lastReturnedVersion.get()); + results->mutations.send(nextOut); wait(results->mutations.onEmpty()); - ASSERT(results->mutations.isEmpty()); wait(delay(0)); } - if (DEBUG_CF_VERSION(results->id, nextOut.back().version)) { - fmt::print("CFLR (merged): {0} (1)\n", nextOut.back().version); + + if (DEBUG_CF_VERSION(results->id, nextVersion)) { + fmt::print("CFLR (merged): {0}\n", nextVersion); } - if (nextOut.back().version > results->lastReturnedVersion.get()) { - results->lastReturnedVersion.set(nextOut.back().version); + if (nextVersion > results->lastReturnedVersion.get()) { + results->lastReturnedVersion.set(nextVersion); } + + // after sending the result to the client, advance each of the streams we popped from + interfNum = 0; + while (interfNum < streamsUsed.size()) { + try { + Standalone res = waitNext(streamsUsed[interfNum].results.getFuture()); + streamsUsed[interfNum].next = res; + mutations.push(streamsUsed[interfNum]); + } catch (Error& e) { + if (e.code() != error_code_end_of_stream) { + throw e; + } + } + } + // Without this delay, weird issues with the last stream getting on another stream's callstack can happen + wait(delay(0)); } - throw end_of_stream(); } +// TODO better name +/*ACTOR Future doCFMerge(Reference results, + std::vector> interfs, + std::vector streams, + Version* begin, + Version end) { + state std::priority_queue> mutations; + state int interfNum = 0; + + // previous version of change feed may have put a mutation in the promise stream and then immediately died. Wait for + // that mutation first, so the promise stream always starts empty + wait(results->mutations.onEmpty()); + wait(delay(0)); + ASSERT(results->mutations.isEmpty()); + + while (interfNum < interfs.size()) { + try { + Standalone res = waitNext(streams[interfNum].results.getFuture()); + streams[interfNum].next = res; + mutations.push(streams[interfNum]); + } catch (Error& e) { + if (e.code() != error_code_end_of_stream) { + throw e; + } + } + interfNum++; + } + state int atCheckVersion = 0; + state int maxAtCheckVersion = -1; + state Version checkVersion = invalidVersion; + state Standalone> nextOut; + while (mutations.size()) { + state MutationAndVersionStream nextStream = mutations.top(); + mutations.pop(); + ASSERT(nextStream.next.version >= checkVersion); + if (nextStream.next.version == checkVersion) { + // TODO REMOVE + if (atCheckVersion == 0) { + printf("atCheckVersion %lld == 0 at %lld\n", checkVersion, nextStream.next.version); + } + ASSERT(atCheckVersion > 0); + } + + if (nextStream.next.version != checkVersion) { + if (nextOut.size()) { + *begin = checkVersion + 1; + if (DEBUG_CF_VERSION(results->id, nextOut.back().version)) { + fmt::print("CFNA (merged): {0} ({1})\n", nextOut.back().version, nextOut.back().mutations.size()); + } + + if (nextOut.back().version < results->lastReturnedVersion.get()) { + fmt::print("ERROR: merge cursor pushing next out {} <= lastReturnedVersion {}\n", + nextOut.back().version, + results->lastReturnedVersion.get()); + } + // We can get an empty version pushed through the stream if whenAtLeast is called. Ignore + // it + ASSERT(nextOut.size() == 1); + if (!nextOut.back().mutations.empty()) { + ASSERT(nextOut.back().version >= results->lastReturnedVersion.get()); + results->mutations.send(nextOut); + wait(results->mutations.onEmpty()); + wait(delay(0)); + } else { + ASSERT(results->mutations.isEmpty()); + } + if (DEBUG_CF_VERSION(results->id, nextOut.back().version)) { + fmt::print("CFLR (merged): {0} (1)\n", nextOut.back().version, nextOut.back().mutations.size()); + } + if (nextOut.back().version > results->lastReturnedVersion.get()) { + results->lastReturnedVersion.set(nextOut.back().version); + } + nextOut = Standalone>(); + } + checkVersion = nextStream.next.version; + atCheckVersion = 0; + // save this at the start of the "round" to check if all streams have a reply at version checkVersion. + // If so, we can send it early. But because mutations.size() can change if one of the streams gets + // end_of_version on its waitNext, we want that to be reflected in the maxAtCheckVersion for the NEXT + // version, not this one. + maxAtCheckVersion = mutations.size() + 1; + } + if (nextOut.size() && nextStream.next.version == nextOut.back().version) { + if (nextStream.next.mutations.size() && + nextStream.next.mutations.front().param1 != lastEpochEndPrivateKey) { + nextOut.back().mutations.append_deep( + nextOut.arena(), nextStream.next.mutations.begin(), nextStream.next.mutations.size()); + } + } else { + nextOut.push_back_deep(nextOut.arena(), nextStream.next); + } + atCheckVersion++; + + // TODO AVOID CODE DUPLICATION + // If all streams have returned something at this version, we know it is complete. + if (atCheckVersion == maxAtCheckVersion) { + ASSERT(nextOut.size() == 1); + *begin = checkVersion + 1; + if (DEBUG_CF_VERSION(results->id, nextOut.back().version)) { + fmt::print("CFNA (merged@all): {0} ({1})\n", nextOut.back().version, nextOut.back().mutations.size()); + } + + if (nextOut.back().version < results->lastReturnedVersion.get()) { + printf("ERROR: merge cursor@all pushing next out <= lastReturnedVersion"); + } + // We can get an empty version pushed through the stream if whenAtLeast is called. Ignore + // it + + if (!nextOut.back().mutations.empty()) { + if (nextOut.back().version < results->lastReturnedVersion.get()) { + fmt::print("Merged all version went backwards!! mutation version {} < lastReturnedVersion {}\n", + nextOut.back().version, + results->lastReturnedVersion.get()); + } + ASSERT(nextOut.back().version >= results->lastReturnedVersion.get()); + results->mutations.send(nextOut); + wait(results->mutations.onEmpty()); + ASSERT(results->mutations.isEmpty()); + wait(delay(0)); + } else { + ASSERT(results->mutations.isEmpty()); + } + if (DEBUG_CF_VERSION(results->id, nextOut.back().version)) { + fmt::print("CFLR (merged@all): {0} ({1})\n", nextOut.back().version, nextOut.back().mutations.size()); + } + if (nextOut.back().version > results->lastReturnedVersion.get()) { + results->lastReturnedVersion.set(nextOut.back().version); + } + nextOut = Standalone>(); + atCheckVersion = 0; + } + try { + Standalone res = waitNext(nextStream.results.getFuture()); + if (DEBUG_CF_VERSION(results->id, res.version)) { + fmt::print(" CFNA (merge1): {0} ({1})\n", res.version, res.mutations.size()); + } + ASSERT(res.version > nextStream.next.version); + nextStream.next = res; + mutations.push(nextStream); + } catch (Error& e) { + if (e.code() != error_code_end_of_stream) { + throw e; + } + } + } + if (nextOut.size()) { + ASSERT(nextOut.back().version >= results->lastReturnedVersion.get()); + if (!nextOut.back().mutations.empty()) { + results->mutations.send(nextOut); + wait(results->mutations.onEmpty()); + ASSERT(results->mutations.isEmpty()); + wait(delay(0)); + } + if (DEBUG_CF_VERSION(results->id, nextOut.back().version)) { + fmt::print("CFLR (merged): {0} (1)\n", nextOut.back().version); + } + if (nextOut.back().version > results->lastReturnedVersion.get()) { + results->lastReturnedVersion.set(nextOut.back().version); + } + } + throw end_of_stream(); +}*/ + ACTOR Future onCFErrors(std::vector> onErrors) { wait(waitForAny(onErrors)); // propagate error - TODO better way? From e708072fd8f9ca49119020e86706537940dfe5ec Mon Sep 17 00:00:00 2001 From: Josh Slocum Date: Fri, 14 Jan 2022 14:47:23 -0600 Subject: [PATCH 129/413] Cleaned up new merge cursor and fixed memory lifetime issue with cancelling a change feed --- fdbclient/NativeAPI.actor.cpp | 259 +++++++--------------------------- 1 file changed, 54 insertions(+), 205 deletions(-) diff --git a/fdbclient/NativeAPI.actor.cpp b/fdbclient/NativeAPI.actor.cpp index 0931fe4db5..cdf1a264d8 100644 --- a/fdbclient/NativeAPI.actor.cpp +++ b/fdbclient/NativeAPI.actor.cpp @@ -7158,9 +7158,9 @@ Version ChangeFeedData::getVersion() { // range that surrounds wait_version enough to figure out what's going on // DEBUG_CF_ID is optional #define DEBUG_CF_ID ""_sr -#define DEBUG_CF_START_VERSION invalidVersion -#define DEBUG_CF_END_VERSION invalidVersion -#define DEBUG_CF_WAIT_VERSION invalidVersion +#define DEBUG_CF_START_VERSION 0 +#define DEBUG_CF_END_VERSION 9223372036854775807 +#define DEBUG_CF_WAIT_VERSION 64306986 #define DEBUG_CF_VERSION(cfId, v) \ DEBUG_CF_START_VERSION <= v&& v <= DEBUG_CF_END_VERSION && (""_sr == DEBUG_CF_ID || cfId.printable() == DEBUG_CF_ID) @@ -7319,6 +7319,9 @@ ACTOR Future partialChangeFeedStream(StorageServerInterface interf, Reference feedData, Reference storageData, int idx /* TODO REMOVE this param after correctness clean */) { + + // calling lastReturnedVersion's callbacks could cause us to be cancelled + state Promise refresh = feedData->refresh; state bool atLatestVersion = false; state Version nextVersion = 0; try { @@ -7381,16 +7384,20 @@ ACTOR Future partialChangeFeedStream(StorageServerInterface interf, nextVersion = rep.mutations.back().version + 1; } - if (!atLatestVersion && rep.atLatestVersion) { + // check refresh.canBeSet so that, if we are killed after calling one of these callbacks, we just + // skip to the next wait and get actor_cancelled + + if (refresh.canBeSet() && !atLatestVersion && rep.atLatestVersion) { atLatestVersion = true; feedData->notAtLatest.set(feedData->notAtLatest.get() - 1); } - if (rep.minStreamVersion > storageData->version.get()) { + + if (refresh.canBeSet() && rep.minStreamVersion > storageData->version.get()) { storageData->version.set(rep.minStreamVersion); } for (auto& it : feedData->storageData) { - if (rep.mutations.back().version > it->desired.get()) { + if (refresh.canBeSet() && rep.mutations.back().version > it->desired.get()) { it->desired.set(rep.mutations.back().version); } } @@ -7435,6 +7442,7 @@ ACTOR Future doCFMerge(Reference results, std::vector streams, Version* begin, Version end) { + state Promise refresh = results->refresh; // with empty version handling in the partial cursor, all streams will always have a next element with version >= // the minimum version of any stream's next element state std::priority_queue> mutations; @@ -7446,37 +7454,47 @@ ACTOR Future doCFMerge(Reference results, ASSERT(results->mutations.isEmpty()); // push one initial mutation from each stream - state int interfNum = 0; - while (interfNum < interfs.size()) { - try { - Standalone res = waitNext(streams[interfNum].results.getFuture()); - streams[interfNum].next = res; - mutations.push(streams[interfNum]); - } catch (Error& e) { - if (e.code() != error_code_end_of_stream) { - throw e; - } - } - interfNum++; - } + // Without this delay, weird issues with the last stream getting on another stream's callstack can happen wait(delay(0)); + state int interfNum = 0; // TODO minor optimization - could make this just a vector of indexes if each MutationAndVersionStream remembered // its version index state std::vector streamsUsed; - streamsUsed.reserve(interfs.size()); + // initially, pull from all streams + for (auto& stream : streams) { + streamsUsed.push_back(stream); + } // TODO REMOVE - i always mess reserve vs resize up ASSERT(streamsUsed.size() == 0); ASSERT(streamsUsed.capacity() == interfs.size()); loop { - if (streams.empty()) { + // bring all of the streams up to date to ensure we have the latest element from each stream in mutations + interfNum = 0; + while (interfNum < streamsUsed.size()) { + try { + Standalone res = waitNext(streamsUsed[interfNum].results.getFuture()); + streamsUsed[interfNum].next = res; + mutations.push(streamsUsed[interfNum]); + } catch (Error& e) { + if (e.code() != error_code_end_of_stream) { + throw e; + } + } + interfNum++; + } + + if (mutations.empty()) { throw end_of_stream(); } streamsUsed.clear(); + // Without this delay, weird issues with the last stream getting on another stream's callstack can happen + wait(delay(0)); + // pop first item off queue - this will be mutation with the lowest version Standalone> nextOut; state Version nextVersion = nextVersion = mutations.top().next.version; @@ -7485,7 +7503,7 @@ ACTOR Future doCFMerge(Reference results, nextOut.push_back_deep(nextOut.arena(), mutations.top().next); mutations.pop(); - // for each other mutation with the same version, add it to nextOut + // for each other stream that has mutations with the same version, add it to nextOut while (!mutations.empty() && mutations.top().next.version == nextVersion) { if (mutations.top().next.mutations.size() && mutations.top().next.mutations.front().param1 != lastEpochEndPrivateKey) { @@ -7504,6 +7522,7 @@ ACTOR Future doCFMerge(Reference results, fmt::print("CFNA (merged): {0} ({1})\n", nextOut.back().version, nextOut.back().mutations.size()); } + // send mutations at nextVersion to the client if (nextOut.back().mutations.empty()) { // TODO REMOVE, for debugging if (nextOut.back().version < results->lastReturnedVersion.get()) { @@ -7530,188 +7549,9 @@ ACTOR Future doCFMerge(Reference results, if (nextVersion > results->lastReturnedVersion.get()) { results->lastReturnedVersion.set(nextVersion); } - - // after sending the result to the client, advance each of the streams we popped from - interfNum = 0; - while (interfNum < streamsUsed.size()) { - try { - Standalone res = waitNext(streamsUsed[interfNum].results.getFuture()); - streamsUsed[interfNum].next = res; - mutations.push(streamsUsed[interfNum]); - } catch (Error& e) { - if (e.code() != error_code_end_of_stream) { - throw e; - } - } - } - // Without this delay, weird issues with the last stream getting on another stream's callstack can happen - wait(delay(0)); } } -// TODO better name -/*ACTOR Future doCFMerge(Reference results, - std::vector> interfs, - std::vector streams, - Version* begin, - Version end) { - state std::priority_queue> mutations; - state int interfNum = 0; - - // previous version of change feed may have put a mutation in the promise stream and then immediately died. Wait for - // that mutation first, so the promise stream always starts empty - wait(results->mutations.onEmpty()); - wait(delay(0)); - ASSERT(results->mutations.isEmpty()); - - while (interfNum < interfs.size()) { - try { - Standalone res = waitNext(streams[interfNum].results.getFuture()); - streams[interfNum].next = res; - mutations.push(streams[interfNum]); - } catch (Error& e) { - if (e.code() != error_code_end_of_stream) { - throw e; - } - } - interfNum++; - } - state int atCheckVersion = 0; - state int maxAtCheckVersion = -1; - state Version checkVersion = invalidVersion; - state Standalone> nextOut; - while (mutations.size()) { - state MutationAndVersionStream nextStream = mutations.top(); - mutations.pop(); - ASSERT(nextStream.next.version >= checkVersion); - if (nextStream.next.version == checkVersion) { - // TODO REMOVE - if (atCheckVersion == 0) { - printf("atCheckVersion %lld == 0 at %lld\n", checkVersion, nextStream.next.version); - } - ASSERT(atCheckVersion > 0); - } - - if (nextStream.next.version != checkVersion) { - if (nextOut.size()) { - *begin = checkVersion + 1; - if (DEBUG_CF_VERSION(results->id, nextOut.back().version)) { - fmt::print("CFNA (merged): {0} ({1})\n", nextOut.back().version, nextOut.back().mutations.size()); - } - - if (nextOut.back().version < results->lastReturnedVersion.get()) { - fmt::print("ERROR: merge cursor pushing next out {} <= lastReturnedVersion {}\n", - nextOut.back().version, - results->lastReturnedVersion.get()); - } - // We can get an empty version pushed through the stream if whenAtLeast is called. Ignore - // it - ASSERT(nextOut.size() == 1); - if (!nextOut.back().mutations.empty()) { - ASSERT(nextOut.back().version >= results->lastReturnedVersion.get()); - results->mutations.send(nextOut); - wait(results->mutations.onEmpty()); - wait(delay(0)); - } else { - ASSERT(results->mutations.isEmpty()); - } - if (DEBUG_CF_VERSION(results->id, nextOut.back().version)) { - fmt::print("CFLR (merged): {0} (1)\n", nextOut.back().version, nextOut.back().mutations.size()); - } - if (nextOut.back().version > results->lastReturnedVersion.get()) { - results->lastReturnedVersion.set(nextOut.back().version); - } - nextOut = Standalone>(); - } - checkVersion = nextStream.next.version; - atCheckVersion = 0; - // save this at the start of the "round" to check if all streams have a reply at version checkVersion. - // If so, we can send it early. But because mutations.size() can change if one of the streams gets - // end_of_version on its waitNext, we want that to be reflected in the maxAtCheckVersion for the NEXT - // version, not this one. - maxAtCheckVersion = mutations.size() + 1; - } - if (nextOut.size() && nextStream.next.version == nextOut.back().version) { - if (nextStream.next.mutations.size() && - nextStream.next.mutations.front().param1 != lastEpochEndPrivateKey) { - nextOut.back().mutations.append_deep( - nextOut.arena(), nextStream.next.mutations.begin(), nextStream.next.mutations.size()); - } - } else { - nextOut.push_back_deep(nextOut.arena(), nextStream.next); - } - atCheckVersion++; - - // TODO AVOID CODE DUPLICATION - // If all streams have returned something at this version, we know it is complete. - if (atCheckVersion == maxAtCheckVersion) { - ASSERT(nextOut.size() == 1); - *begin = checkVersion + 1; - if (DEBUG_CF_VERSION(results->id, nextOut.back().version)) { - fmt::print("CFNA (merged@all): {0} ({1})\n", nextOut.back().version, nextOut.back().mutations.size()); - } - - if (nextOut.back().version < results->lastReturnedVersion.get()) { - printf("ERROR: merge cursor@all pushing next out <= lastReturnedVersion"); - } - // We can get an empty version pushed through the stream if whenAtLeast is called. Ignore - // it - - if (!nextOut.back().mutations.empty()) { - if (nextOut.back().version < results->lastReturnedVersion.get()) { - fmt::print("Merged all version went backwards!! mutation version {} < lastReturnedVersion {}\n", - nextOut.back().version, - results->lastReturnedVersion.get()); - } - ASSERT(nextOut.back().version >= results->lastReturnedVersion.get()); - results->mutations.send(nextOut); - wait(results->mutations.onEmpty()); - ASSERT(results->mutations.isEmpty()); - wait(delay(0)); - } else { - ASSERT(results->mutations.isEmpty()); - } - if (DEBUG_CF_VERSION(results->id, nextOut.back().version)) { - fmt::print("CFLR (merged@all): {0} ({1})\n", nextOut.back().version, nextOut.back().mutations.size()); - } - if (nextOut.back().version > results->lastReturnedVersion.get()) { - results->lastReturnedVersion.set(nextOut.back().version); - } - nextOut = Standalone>(); - atCheckVersion = 0; - } - try { - Standalone res = waitNext(nextStream.results.getFuture()); - if (DEBUG_CF_VERSION(results->id, res.version)) { - fmt::print(" CFNA (merge1): {0} ({1})\n", res.version, res.mutations.size()); - } - ASSERT(res.version > nextStream.next.version); - nextStream.next = res; - mutations.push(nextStream); - } catch (Error& e) { - if (e.code() != error_code_end_of_stream) { - throw e; - } - } - } - if (nextOut.size()) { - ASSERT(nextOut.back().version >= results->lastReturnedVersion.get()); - if (!nextOut.back().mutations.empty()) { - results->mutations.send(nextOut); - wait(results->mutations.onEmpty()); - ASSERT(results->mutations.isEmpty()); - wait(delay(0)); - } - if (DEBUG_CF_VERSION(results->id, nextOut.back().version)) { - fmt::print("CFLR (merged): {0} (1)\n", nextOut.back().version); - } - if (nextOut.back().version > results->lastReturnedVersion.get()) { - results->lastReturnedVersion.set(nextOut.back().version); - } - } - throw end_of_stream(); -}*/ - ACTOR Future onCFErrors(std::vector> onErrors) { wait(waitForAny(onErrors)); // propagate error - TODO better way? @@ -7808,7 +7648,10 @@ ACTOR Future doSingleCFStream(KeyRange range, Key rangeID, Version* begin, Version end) { - + state Promise refresh = results->refresh; + printf("Single stream starting %lld - %lld\n", *begin, end); + ASSERT(results->streams.size() == 1); + ASSERT(results->storageData.size() == 1); state bool atLatest = false; loop { // wait for any previous mutations in stream to be consumed @@ -7858,14 +7701,17 @@ ACTOR Future doSingleCFStream(KeyRange range, results->notAtLatest.get(), feedReply.minStreamVersion); } + // check refresh.canBeSet so that, if we are killed after calling one of these callbacks, we just + // skip to the next wait and get actor_cancelled if (feedReply.mutations.back().version > results->lastReturnedVersion.get()) { results->lastReturnedVersion.set(feedReply.mutations.back().version); } - if (!atLatest && feedReply.atLatestVersion) { + + if (refresh.canBeSet() && !atLatest && feedReply.atLatestVersion) { atLatest = true; results->notAtLatest.set(0); } - if (feedReply.minStreamVersion > results->storageData[0]->version.get()) { + if (refresh.canBeSet() && feedReply.minStreamVersion > results->storageData[0]->version.get()) { if (results->storageData[0]->debug) { fmt::print("CFSD {0}: V={1} (CFLR)\n", results->storageData[0]->id.toString().substr(0, 4), @@ -7921,6 +7767,8 @@ ACTOR Future getChangeFeedStreamActor(Reference db, state Database cx(db); state Span span("NAPI:GetChangeFeedStream"_loc); + printf("Get Change Feed Stream %lld - %lld\n", begin, end); + results->id = rangeID; results->endVersion = end; @@ -8003,6 +7851,7 @@ ACTOR Future getChangeFeedStreamActor(Reference db, cx->connectionFileChanged()); } } catch (Error& e) { + fmt::print("CFNA error {}\n", e.name()); if (e.code() == error_code_actor_cancelled) { for (auto& it : results->storageData) { if (it->debugGetReferenceCount() == 2) { @@ -8015,7 +7864,7 @@ ACTOR Future getChangeFeedStreamActor(Reference db, throw; } // TODO REMOVE - fmt::print("CFNA error {}\n", e.name()); + // fmt::print("CFNA error {}\n", e.name()); if (results->notAtLatest.get() == 0) { results->notAtLatest.set(1); } From 680bc53b8fa8ab20222858a1aca01418c9fca9a2 Mon Sep 17 00:00:00 2001 From: Evan Tschannen Date: Fri, 14 Jan 2022 12:55:35 -0800 Subject: [PATCH 130/413] fix: do not submit assignments if the worker has already been removed --- fdbserver/BlobManager.actor.cpp | 3 +++ 1 file changed, 3 insertions(+) diff --git a/fdbserver/BlobManager.actor.cpp b/fdbserver/BlobManager.actor.cpp index b7a9b88cab..04573fcb80 100644 --- a/fdbserver/BlobManager.actor.cpp +++ b/fdbserver/BlobManager.actor.cpp @@ -384,6 +384,9 @@ ACTOR Future doRangeAssignment(BlobManagerData* bmData, RangeAssignment as if (e.code() == error_code_operation_cancelled) { throw; } + if (!bmData->workersById.count(workerID)) { + return Void(); + } // TODO confirm: using reliable delivery this should only trigger if the worker is marked as failed, right? // So assignment needs to be retried elsewhere, and a revoke is trivially complete if (assignment.isAssign) { From 9174bfbb4e5d61934c8e22aa9c29097e6428a2e0 Mon Sep 17 00:00:00 2001 From: Josh Slocum Date: Fri, 14 Jan 2022 15:26:37 -0600 Subject: [PATCH 131/413] bug fix and extra check in merge cursor --- fdbclient/NativeAPI.actor.cpp | 8 +++++--- 1 file changed, 5 insertions(+), 3 deletions(-) diff --git a/fdbclient/NativeAPI.actor.cpp b/fdbclient/NativeAPI.actor.cpp index cdf1a264d8..ea103fff0d 100644 --- a/fdbclient/NativeAPI.actor.cpp +++ b/fdbclient/NativeAPI.actor.cpp @@ -7442,6 +7442,8 @@ ACTOR Future doCFMerge(Reference results, std::vector streams, Version* begin, Version end) { + // TODO REMOVE, a sanity check + state int eosCount = 0; state Promise refresh = results->refresh; // with empty version handling in the partial cursor, all streams will always have a next element with version >= // the minimum version of any stream's next element @@ -7466,9 +7468,6 @@ ACTOR Future doCFMerge(Reference results, for (auto& stream : streams) { streamsUsed.push_back(stream); } - // TODO REMOVE - i always mess reserve vs resize up - ASSERT(streamsUsed.size() == 0); - ASSERT(streamsUsed.capacity() == interfs.size()); loop { // bring all of the streams up to date to ensure we have the latest element from each stream in mutations @@ -7482,10 +7481,13 @@ ACTOR Future doCFMerge(Reference results, if (e.code() != error_code_end_of_stream) { throw e; } + eosCount++; } interfNum++; } + ASSERT(streams.size() - mutations.size() == eosCount); + if (mutations.empty()) { throw end_of_stream(); } From 6b1e387df796d721c8239b2ee65b50c4fa898f78 Mon Sep 17 00:00:00 2001 From: Josh Slocum Date: Fri, 14 Jan 2022 15:44:31 -0600 Subject: [PATCH 132/413] removed debugging --- fdbclient/NativeAPI.actor.cpp | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/fdbclient/NativeAPI.actor.cpp b/fdbclient/NativeAPI.actor.cpp index ea103fff0d..580a2f68b6 100644 --- a/fdbclient/NativeAPI.actor.cpp +++ b/fdbclient/NativeAPI.actor.cpp @@ -7158,9 +7158,9 @@ Version ChangeFeedData::getVersion() { // range that surrounds wait_version enough to figure out what's going on // DEBUG_CF_ID is optional #define DEBUG_CF_ID ""_sr -#define DEBUG_CF_START_VERSION 0 -#define DEBUG_CF_END_VERSION 9223372036854775807 -#define DEBUG_CF_WAIT_VERSION 64306986 +#define DEBUG_CF_START_VERSION invalidVersion +#define DEBUG_CF_END_VERSION invalidVersion +#define DEBUG_CF_WAIT_VERSION invalidVersion #define DEBUG_CF_VERSION(cfId, v) \ DEBUG_CF_START_VERSION <= v&& v <= DEBUG_CF_END_VERSION && (""_sr == DEBUG_CF_ID || cfId.printable() == DEBUG_CF_ID) From e44dae0e5b7ead464d91fbd4d480324ce070b66b Mon Sep 17 00:00:00 2001 From: Josh Slocum Date: Fri, 14 Jan 2022 15:47:21 -0600 Subject: [PATCH 133/413] Removing incorrect assert --- fdbclient/NativeAPI.actor.cpp | 6 ------ 1 file changed, 6 deletions(-) diff --git a/fdbclient/NativeAPI.actor.cpp b/fdbclient/NativeAPI.actor.cpp index 580a2f68b6..4fb295e819 100644 --- a/fdbclient/NativeAPI.actor.cpp +++ b/fdbclient/NativeAPI.actor.cpp @@ -7526,12 +7526,6 @@ ACTOR Future doCFMerge(Reference results, // send mutations at nextVersion to the client if (nextOut.back().mutations.empty()) { - // TODO REMOVE, for debugging - if (nextOut.back().version < results->lastReturnedVersion.get()) { - printf("ERROR: merge cursor got empty version <= lastReturnedVersion"); - } - ASSERT(nextOut.back().version >= results->lastReturnedVersion.get()); - ASSERT(results->mutations.isEmpty()); } else { // TODO REMOVE, for debugging From 58bc3a78ea336abe60373c73b12da9d27ba51dc8 Mon Sep 17 00:00:00 2001 From: Josh Slocum Date: Fri, 14 Jan 2022 17:10:53 -0600 Subject: [PATCH 134/413] Fixed range assignment bug and fixed printf compiler warnings --- fdbclient/NativeAPI.actor.cpp | 62 ++++---- fdbserver/BlobManager.actor.cpp | 269 +++++++++++++++++--------------- fdbserver/BlobWorker.actor.cpp | 182 +++++++++++---------- 3 files changed, 258 insertions(+), 255 deletions(-) diff --git a/fdbclient/NativeAPI.actor.cpp b/fdbclient/NativeAPI.actor.cpp index 4fb295e819..8caad964ca 100644 --- a/fdbclient/NativeAPI.actor.cpp +++ b/fdbclient/NativeAPI.actor.cpp @@ -6517,9 +6517,7 @@ ACTOR Future>> getBlobGranuleRangesActor(Trans state KeyRange currentRange = keyRange; state Standalone> results; if (BG_REQUEST_DEBUG) { - printf("Getting Blob Granules for [%s - %s)\n", - keyRange.begin.printable().c_str(), - keyRange.end.printable().c_str()); + fmt::print("Getting Blob Granules for [{0} - {1})\n", keyRange.begin.printable(), keyRange.end.printable()); } self->setOption(FDBTransactionOptions::ACCESS_SYSTEM_KEYS); loop { @@ -6609,9 +6607,9 @@ ACTOR Future>> readBlobGranulesActor( granuleEndKey = blobGranuleMapping[i + 1].key; if (!blobGranuleMapping[i].value.size()) { if (BG_REQUEST_DEBUG) { - printf("Key range [%s - %s) missing worker assignment!\n", - granuleStartKey.printable().c_str(), - granuleEndKey.printable().c_str()); + fmt::print("Key range [{0} - {1}) missing worker assignment!\n", + granuleStartKey.printable(), + granuleEndKey.printable()); // TODO probably new exception type instead } throw blob_granule_transaction_too_old(); @@ -6620,17 +6618,15 @@ ACTOR Future>> readBlobGranulesActor( workerId = decodeBlobGranuleMappingValue(blobGranuleMapping[i].value); if (workerId == UID()) { if (BG_REQUEST_DEBUG) { - printf("Key range [%s - %s) has no assigned worker yet!\n", - granuleStartKey.printable().c_str(), - granuleEndKey.printable().c_str()); + fmt::print("Key range [{0} - {1}) has no assigned worker yet!\n", + granuleStartKey.printable(), + granuleEndKey.printable()); } throw blob_granule_transaction_too_old(); } if (BG_REQUEST_DEBUG) { - printf(" [%s - %s): %s\n", - granuleStartKey.printable().c_str(), - granuleEndKey.printable().c_str(), - workerId.toString().c_str()); + fmt::print( + " [{0} - {1}): {2}\n", granuleStartKey.printable(), granuleEndKey.printable(), workerId.toString()); } if (!cx->blobWorker_interf.count(workerId)) { @@ -6647,7 +6643,7 @@ ACTOR Future>> readBlobGranulesActor( // something? cx->blobWorker_interf[workerId] = decodeBlobWorkerListValue(workerInterface.get()); if (BG_REQUEST_DEBUG) { - printf(" decoded worker interface for %s\n", workerId.toString().c_str()); + fmt::print(" decoded worker interface for {0}\n", workerId.toString()); } } } @@ -6726,7 +6722,7 @@ ACTOR Future>> readBlobGranulesActor( location->get(0, &BlobWorkerInterface::blobGranuleFileRequest).getEndpoint(), FailureStatus(true)))) { if (BG_REQUEST_DEBUG) { - printf("readBlobGranules got BW %s failed\n", workerId.toString().c_str()); + fmt::print("readBlobGranules got BW {0} failed\n", workerId.toString()); } throw connection_failed(); @@ -6734,7 +6730,7 @@ ACTOR Future>> readBlobGranulesActor( } } catch (Error& e) { if (BG_REQUEST_DEBUG) { - printf("BGReq got error %s\n", e.name()); + fmt::print("BGReq got error {}\n", e.name()); } // worker is up but didn't actually have granule, or connection failed if (e.code() == error_code_wrong_shard_server || e.code() == error_code_connection_failed) { @@ -7368,12 +7364,12 @@ ACTOR Future partialChangeFeedStream(StorageServerInterface interf, } else { // TODO REMOVE eventually, useful for debugging for now if (!rep.mutations[resultLoc].mutations.empty()) { - printf("non-empty mutations (%d), but versions out of order from %s! mv=%lld, " - "nv=%lld\n", - rep.mutations.size(), - interf.id().toString().substr(0, 4).c_str(), - rep.mutations[resultLoc].version, - nextVersion); + fmt::print("non-empty mutations ({0}), but versions out of order from {1}! mv={2}, " + "nv={3}\n", + rep.mutations.size(), + interf.id().toString().substr(0, 4), + rep.mutations[resultLoc].version, + nextVersion); } ASSERT(rep.mutations[resultLoc].mutations.empty()); } @@ -7427,7 +7423,7 @@ ACTOR Future partialChangeFeedStream(StorageServerInterface interf, } } catch (Error& e) { // TODO REMOVE eventually, useful for debugging for now - printf("NAS: CFError %s\n", e.name()); + fmt::print("NAS: CFError {}\n", e.name()); if (e.code() == error_code_actor_cancelled) { throw; } @@ -7530,7 +7526,9 @@ ACTOR Future doCFMerge(Reference results, } else { // TODO REMOVE, for debugging if (nextOut.back().version <= results->lastReturnedVersion.get()) { - printf("ERROR: merge cursor got mutations <= lastReturnedVersion"); + fmt::print("ERROR: merge cursor got mutations {0} <= lastReturnedVersion {1}", + nextOut.back().version, + results->lastReturnedVersion.get()); } ASSERT(nextOut.back().version > results->lastReturnedVersion.get()); @@ -7645,7 +7643,6 @@ ACTOR Future doSingleCFStream(KeyRange range, Version* begin, Version end) { state Promise refresh = results->refresh; - printf("Single stream starting %lld - %lld\n", *begin, end); ASSERT(results->streams.size() == 1); ASSERT(results->storageData.size() == 1); state bool atLatest = false; @@ -7669,13 +7666,12 @@ ACTOR Future doSingleCFStream(KeyRange range, // empty versions can come out of order, as we sometimes send explicit empty versions when restarting a // stream. Anything with mutations should be strictly greater than lastReturnedVersion if (feedReply.mutations.front().version <= results->lastReturnedVersion.get()) { - printf("out of order mutation for CF %s from (%d) %s! %lld < %lld\n", - rangeID.toString().substr(0, 6).c_str(), - results->storageData.size(), - results->storageData.empty() ? "????" - : results->storageData[0]->id.toString().substr(0, 4).c_str(), - feedReply.mutations.front().version, - results->lastReturnedVersion.get()); + fmt::print("out of order mutation for CF {0} from ({1}) {2}! {3} < {4}\n", + rangeID.toString().substr(0, 6), + results->storageData.size(), + results->storageData[0]->id.toString().substr(0, 4).c_str(), + feedReply.mutations.front().version, + results->lastReturnedVersion.get()); } ASSERT(feedReply.mutations.front().version > results->lastReturnedVersion.get()); @@ -7763,8 +7759,6 @@ ACTOR Future getChangeFeedStreamActor(Reference db, state Database cx(db); state Span span("NAPI:GetChangeFeedStream"_loc); - printf("Get Change Feed Stream %lld - %lld\n", begin, end); - results->id = rangeID; results->endVersion = end; diff --git a/fdbserver/BlobManager.actor.cpp b/fdbserver/BlobManager.actor.cpp index 992929282d..848bb47e78 100644 --- a/fdbserver/BlobManager.actor.cpp +++ b/fdbserver/BlobManager.actor.cpp @@ -54,10 +54,8 @@ void handleClientBlobRange(KeyRangeMap* knownBlobRanges, KeyRef rangeEnd, bool rangeActive) { if (BM_DEBUG) { - printf("db range [%s - %s): %s\n", - rangeStart.printable().c_str(), - rangeEnd.printable().c_str(), - rangeActive ? "T" : "F"); + fmt::print( + "db range [{0} - {1}): {2}\n", rangeStart.printable(), rangeEnd.printable(), rangeActive ? "T" : "F"); } KeyRange keyRange(KeyRangeRef(rangeStart, rangeEnd)); auto allRanges = knownBlobRanges->intersectingRanges(keyRange); @@ -68,16 +66,16 @@ void handleClientBlobRange(KeyRangeMap* knownBlobRanges, KeyRangeRef overlap(overlapStart, overlapEnd); if (rangeActive) { if (BM_DEBUG) { - printf("BM Adding client range [%s - %s)\n", - overlapStart.printable().c_str(), - overlapEnd.printable().c_str()); + fmt::print("BM Adding client range [{0} - {1})\n", + overlapStart.printable().c_str(), + overlapEnd.printable().c_str()); } rangesToAdd->push_back_deep(ar, overlap); } else { if (BM_DEBUG) { - printf("BM Removing client range [%s - %s)\n", - overlapStart.printable().c_str(), - overlapEnd.printable().c_str()); + fmt::print("BM Removing client range [{0} - {1})\n", + overlapStart.printable().c_str(), + overlapEnd.printable().c_str()); } rangesToRemove->push_back_deep(ar, overlap); } @@ -92,9 +90,9 @@ void updateClientBlobRanges(KeyRangeMap* knownBlobRanges, VectorRef* rangesToAdd, VectorRef* rangesToRemove) { if (BM_DEBUG) { - printf("Updating %d client blob ranges", dbBlobRanges.size() / 2); + fmt::print("Updating {0} client blob ranges", dbBlobRanges.size() / 2); for (int i = 0; i < dbBlobRanges.size() - 1; i += 2) { - printf(" [%s - %s)", dbBlobRanges[i].key.printable().c_str(), dbBlobRanges[i + 1].key.printable().c_str()); + fmt::print(" [{0} - {1})", dbBlobRanges[i].key.printable(), dbBlobRanges[i + 1].key.printable()); } printf("\n"); } @@ -119,7 +117,7 @@ void updateClientBlobRanges(KeyRangeMap* knownBlobRanges, for (int i = 0; i < dbBlobRanges.size() - 1; i++) { if (dbBlobRanges[i].key >= normalKeys.end) { if (BM_DEBUG) { - printf("Found invalid blob range start %s\n", dbBlobRanges[i].key.printable().c_str()); + fmt::print("Found invalid blob range start {0}\n", dbBlobRanges[i].key.printable()); } break; } @@ -127,17 +125,17 @@ void updateClientBlobRanges(KeyRangeMap* knownBlobRanges, if (active) { ASSERT(dbBlobRanges[i + 1].value == StringRef()); if (BM_DEBUG) { - printf("BM sees client range [%s - %s)\n", - dbBlobRanges[i].key.printable().c_str(), - dbBlobRanges[i + 1].key.printable().c_str()); + fmt::print("BM sees client range [{0} - {1})\n", + dbBlobRanges[i].key.printable(), + dbBlobRanges[i + 1].key.printable()); } } KeyRef endKey = dbBlobRanges[i + 1].key; if (endKey > normalKeys.end) { if (BM_DEBUG) { - printf("Removing system keyspace from blob range [%s - %s)\n", - dbBlobRanges[i].key.printable().c_str(), - endKey.printable().c_str()); + fmt::print("Removing system keyspace from blob range [{0} - {1})\n", + dbBlobRanges[i].key.printable(), + endKey.printable()); } endKey = normalKeys.end; } @@ -165,8 +163,7 @@ void getRanges(std::vector>& results, KeyRangeMap dcId) : id(id), db(db), dcId(dcId), knownBlobRanges(false, normalKeys.end), restartRecruiting(SERVER_KNOBS->DEBOUNCE_RECRUITING_DELAY), recruitingStream(0) {} - ~BlobManagerData() { printf("Destroying blob manager data for %s\n", id.toString().c_str()); } + ~BlobManagerData() { fmt::print("Destroying blob manager data for {}\n", id.toString()); } }; ACTOR Future>> splitRange(Reference tr, KeyRange range) { @@ -243,8 +240,7 @@ ACTOR Future>> splitRange(ReferencegetTransaction().getStorageMetrics(range, CLIENT_KNOBS->TOO_MANY)); @@ -314,9 +310,9 @@ ACTOR Future pickWorkerForAssign(BlobManagerData* bmData) { ASSERT(eligibleWorkers.size() > 0); int idx = deterministicRandom()->randomInt(0, eligibleWorkers.size()); if (BM_DEBUG) { - printf("picked worker %s, which has a minimal number (%d) of granules assigned\n", - eligibleWorkers[idx].toString().c_str(), - minGranulesAssigned); + fmt::print("picked worker {0}, which has a minimal number ({1}) of granules assigned\n", + eligibleWorkers[idx].toString(), + minGranulesAssigned); } return eligibleWorkers[idx]; @@ -325,13 +321,14 @@ ACTOR Future pickWorkerForAssign(BlobManagerData* bmData) { ACTOR Future doRangeAssignment(BlobManagerData* bmData, RangeAssignment assignment, UID workerID, int64_t seqNo) { if (BM_DEBUG) { - fmt::print("BM {0} {1} range [{2} - {3}) @ ({4}, {5})\n", - bmData->id.toString(), + fmt::print("BM {0} {1} range [{2} - {3}) @ ({4}, {5}) to {6}\n", + bmData->epoch, assignment.isAssign ? "assigning" : "revoking", assignment.keyRange.begin.printable(), assignment.keyRange.end.printable(), bmData->epoch, - seqNo); + seqNo, + workerID.toString()); } try { @@ -384,17 +381,28 @@ ACTOR Future doRangeAssignment(BlobManagerData* bmData, RangeAssignment as if (e.code() == error_code_operation_cancelled) { throw; } - if (!bmData->workersById.count(workerID)) { + // If the worker is no longer present, the ranges were already moved off by that function, so don't retry. + // If the request is a reassign though, we need to retry it since the worker it was on is now dead and nobody + // owns it + if (!bmData->workersById.count(workerID) && + (!assignment.assign.present() || assignment.assign.get().type != AssignRequestType::Reassign)) { + if (BM_DEBUG) { + fmt::print("BM {0} got error assigning range [{1} - {2}) to now dead worker {3}, ignoring\n", + bmData->epoch, + assignment.keyRange.begin.printable(), + assignment.keyRange.end.printable(), + workerID.toString()); + } return Void(); } // TODO confirm: using reliable delivery this should only trigger if the worker is marked as failed, right? // So assignment needs to be retried elsewhere, and a revoke is trivially complete if (assignment.isAssign) { if (BM_DEBUG) { - printf("BM got error assigning range [%s - %s) to worker %s, requeueing\n", - assignment.keyRange.begin.printable().c_str(), - assignment.keyRange.end.printable().c_str(), - workerID.toString().c_str()); + fmt::print("BM got error assigning range [%s - %s) to worker %s, requeueing\n", + assignment.keyRange.begin.printable(), + assignment.keyRange.end.printable(), + workerID.toString()); } // re-send revoke to queue to handle range being un-assigned from that worker before the new one RangeAssignment revokeOld; @@ -410,9 +418,9 @@ ACTOR Future doRangeAssignment(BlobManagerData* bmData, RangeAssignment as // FIXME: improvement would be to add history of failed workers to assignment so it can try other ones first } else { if (BM_DEBUG) { - printf("BM got error revoking range [%s - %s) from worker", - assignment.keyRange.begin.printable().c_str(), - assignment.keyRange.end.printable().c_str()); + fmt::print("BM got error revoking range [{0} - {1}) from worker", + assignment.keyRange.begin.printable(), + assignment.keyRange.end.printable()); } if (assignment.revoke.get().dispose) { @@ -463,14 +471,17 @@ ACTOR Future rangeAssigner(BlobManagerData* bmData) { ASSERT(count == 1); if (assignment.worker.present() && assignment.worker.get().isValid()) { + if (BM_DEBUG) { + fmt::print("BW {0} already chosen for seqno {1} in BM {2}\n", + assignment.worker.get().toString(), + seqNo, + bmData->id.toString()); + } workerId = assignment.worker.get(); } else { - if (BM_DEBUG) { - printf("About to pick worker for seqno %d in BM %s\n", seqNo, bmData->id.toString().c_str()); - } UID _workerId = wait(pickWorkerForAssign(bmData)); if (BM_DEBUG) { - printf("Found worker BW %s for seqno %d\n", _workerId.toString().c_str(), seqNo); + fmt::print("Chose BW {0} for seqno {1} in BM {2}\n", _workerId.toString(), seqNo, bmData->epoch); } workerId = _workerId; } @@ -546,9 +557,9 @@ ACTOR Future writeInitialGranuleMapping(BlobManagerData* bmData, Standalon while (i + j < boundaries.size() - 1 && j < transactionChunkSize) { // TODO REMOVE if (BM_DEBUG) { - printf("Persisting initial mapping for [%s - %s)\n", - boundaries[i + j].printable().c_str(), - boundaries[i + j + 1].printable().c_str()); + fmt::print("Persisting initial mapping for [{0} - {1})\n", + boundaries[i + j].printable(), + boundaries[i + j + 1].printable()); } // set to empty UID - no worker assigned yet wait(krmSetRange(tr, @@ -561,7 +572,7 @@ ACTOR Future writeInitialGranuleMapping(BlobManagerData* bmData, Standalon break; } catch (Error& e) { if (BM_DEBUG) { - printf("Persisting initial mapping got error %s\n", e.name()); + fmt::print("Persisting initial mapping got error {}\n", e.name()); } wait(tr->onError(e)); j = 0; @@ -603,9 +614,8 @@ ACTOR Future monitorClientRanges(BlobManagerData* bmData) { for (KeyRangeRef range : rangesToRemove) { if (BM_DEBUG) { - printf("BM Got range to revoke [%s - %s)\n", - range.begin.printable().c_str(), - range.end.printable().c_str()); + fmt::print( + "BM Got range to revoke [{0} - {1})\n", range.begin.printable(), range.end.printable()); } RangeAssignment ra; @@ -624,10 +634,10 @@ ACTOR Future monitorClientRanges(BlobManagerData* bmData) { for (auto f : splitFutures) { state Standalone> splits = wait(f); if (BM_DEBUG) { - printf("Split client range [%s - %s) into %d ranges:\n", - splits[0].printable().c_str(), - splits[splits.size() - 1].printable().c_str(), - splits.size() - 1); + fmt::print("Split client range [{0} - {1}) into {2} ranges:\n", + splits[0].printable(), + splits[splits.size() - 1].printable(), + splits.size() - 1); } // Write to DB BEFORE sending assign requests, so that if manager dies before/during, new manager @@ -638,7 +648,8 @@ ACTOR Future monitorClientRanges(BlobManagerData* bmData) { KeyRange range = KeyRange(KeyRangeRef(splits[i], splits[i + 1])); // only add the client range if this is the first BM or it's not already assigned if (BM_DEBUG) { - printf(" [%s - %s)\n", range.begin.printable().c_str(), range.end.printable().c_str()); + fmt::print( + " [{0} - {1})\n", range.begin.printable().c_str(), range.end.printable().c_str()); } RangeAssignment ra; @@ -675,7 +686,7 @@ ACTOR Future monitorClientRanges(BlobManagerData* bmData) { break; } catch (Error& e) { if (BM_DEBUG) { - printf("Blob manager got error looking for range updates %s\n", e.name()); + fmt::print("Blob manager got error looking for range updates {}\n", e.name()); } wait(tr->onError(e)); } @@ -726,10 +737,10 @@ ACTOR Future maybeSplitRange(BlobManagerData* bmData, if (newRanges.size() == 2) { // not large enough to split, just reassign back to worker if (BM_DEBUG) { - printf("Not splitting existing range [%s - %s). Continuing assignment to %s\n", - granuleRange.begin.printable().c_str(), - granuleRange.end.printable().c_str(), - currentWorkerId.toString().c_str()); + fmt::print("Not splitting existing range [{0} - {1}). Continuing assignment to {2}\n", + granuleRange.begin.printable(), + granuleRange.end.printable(), + currentWorkerId.toString()); } RangeAssignment raContinue; raContinue.isAssign = true; @@ -889,12 +900,12 @@ ACTOR Future deregisterBlobWorker(BlobManagerData* bmData, BlobWorkerInter wait(tr->commit()); if (BM_DEBUG) { - printf("Deregistered blob worker %s\n", interf.id().toString().c_str()); + fmt::print("Deregistered blob worker {0}\n", interf.id().toString()); } return Void(); } catch (Error& e) { if (BM_DEBUG) { - printf("Deregistering blob worker %s got error %s\n", interf.id().toString().c_str(), e.name()); + fmt::print("Deregistering blob worker {0} got error {1}\n", interf.id().toString(), e.name()); } wait(tr->onError(e)); } @@ -926,7 +937,7 @@ ACTOR Future killBlobWorker(BlobManagerData* bmData, BlobWorkerInterface b // - send a revoke request for that range // - add the range back to the stream of ranges to be assigned if (BM_DEBUG) { - printf("Taking back ranges from BW %s\n", bwId.toString().c_str()); + fmt::print("Taking back ranges from BW {0}\n", bwId.toString()); } // copy ranges into vector before sending, because send then modifies workerAssignments state std::vector rangesToMove; @@ -954,7 +965,7 @@ ACTOR Future killBlobWorker(BlobManagerData* bmData, BlobWorkerInterface b // Send halt to blob worker, with no expectation of hearing back if (BM_DEBUG) { - printf("Sending halt to BW %s\n", bwId.toString().c_str()); + fmt::print("Sending halt to BW {}\n", bwId.toString()); } bmData->addActor.send( brokenPromiseToNever(bwInterf.haltBlobWorker.getReply(HaltBlobWorkerRequest(bmData->epoch, bmData->id)))); @@ -991,8 +1002,8 @@ ACTOR Future monitorBlobWorkerStatus(BlobManagerData* bmData, BlobWorkerIn } if (rep.epoch > bmData->epoch) { if (BM_DEBUG) { - printf("BM heard from BW %s that there is a new manager with higher epoch\n", - bwInterf.id().toString().c_str()); + fmt::print("BM heard from BW {0} that there is a new manager with higher epoch\n", + bwInterf.id().toString()); } if (bmData->iAmReplaced.canBeSet()) { bmData->iAmReplaced.send(Void()); @@ -1059,9 +1070,8 @@ ACTOR Future monitorBlobWorkerStatus(BlobManagerData* bmData, BlobWorkerIn continue; } else { if (BM_DEBUG) { - printf("BM got unexpected error %s monitoring BW %s status\n", - e.name(), - bwInterf.id().toString().c_str()); + fmt::print( + "BM got unexpected error {0} monitoring BW {1} status\n", e.name(), bwInterf.id().toString()); } // TODO change back from SevError? TraceEvent(SevError, "BWStatusMonitoringFailed", bmData->id) @@ -1105,7 +1115,7 @@ ACTOR Future monitorBlobWorker(BlobManagerData* bmData, BlobWorkerInterfac // Expected errors here are: [broken_promise] if (e.code() != error_code_broken_promise) { if (BM_DEBUG) { - printf("BM got unexpected error %s monitoring BW %s\n", e.name(), bwInterf.id().toString().c_str()); + fmt::print("BM got unexpected error {0} monitoring BW {1}\n", e.name(), bwInterf.id().toString()); } // TODO change back from SevError? TraceEvent(SevError, "BWMonitoringFailed", bmData->id).detail("BlobWorkerID", bwInterf.id()).error(e); @@ -1117,7 +1127,7 @@ ACTOR Future monitorBlobWorker(BlobManagerData* bmData, BlobWorkerInterfac wait(killBlobWorker(bmData, bwInterf, true)); if (BM_DEBUG) { - printf("No longer monitoring BW %s\n", bwInterf.id().toString().c_str()); + fmt::print("No longer monitoring BW {0}\n", bwInterf.id().toString()); } return Void(); } @@ -1308,10 +1318,10 @@ ACTOR Future chaosRangeMover(BlobManagerData* bmData) { auto randomRange = bmData->workerAssignments.randomRange(); if (randomRange.value() != UID()) { if (BM_DEBUG) { - printf("Range mover moving range [%s - %s): %s\n", - randomRange.begin().printable().c_str(), - randomRange.end().printable().c_str(), - randomRange.value().toString().c_str()); + fmt::print("Range mover moving range [{0} - {1}): {2}\n", + randomRange.begin().printable().c_str(), + randomRange.end().printable().c_str(), + randomRange.value().toString().c_str()); } // FIXME: with low probability, could immediately revoke it from the new assignment and move @@ -1336,7 +1346,7 @@ ACTOR Future chaosRangeMover(BlobManagerData* bmData) { printf("Range mover couldn't find random range to move, skipping\n"); } } else if (BM_DEBUG) { - printf("Range mover found %d workers, skipping\n", bmData->workerAssignments.size()); + fmt::print("Range mover found {0} workers, skipping\n", bmData->workerAssignments.size()); } } } @@ -1532,7 +1542,7 @@ ACTOR Future loadHistoryFiles(BlobManagerData* bmData, UID granule */ ACTOR Future fullyDeleteGranule(BlobManagerData* self, UID granuleId, KeyRef historyKey) { if (BM_DEBUG) { - printf("Fully deleting granule %s: init\n", granuleId.toString().c_str()); + fmt::print("Fully deleting granule {0}: init\n", granuleId.toString()); } // get files @@ -1554,9 +1564,9 @@ ACTOR Future fullyDeleteGranule(BlobManagerData* self, UID granuleId, KeyR } if (BM_DEBUG) { - printf("Fully deleting granule %s: deleting %d files\n", granuleId.toString().c_str(), deletions.size()); + fmt::print("Fully deleting granule {0}: deleting {1} files\n", granuleId.toString(), deletions.size()); for (auto filename : filesToDelete) { - printf(" - %s\n", filename.c_str()); + fmt::print(" - {}\n", filename.c_str()); } } @@ -1568,7 +1578,7 @@ ACTOR Future fullyDeleteGranule(BlobManagerData* self, UID granuleId, KeyR // delete metadata in FDB (history entry and file keys) if (BM_DEBUG) { - printf("Fully deleting granule %s: deleting history and file keys\n", granuleId.toString().c_str()); + fmt::print("Fully deleting granule {0}: deleting history and file keys\n", granuleId.toString()); } state Transaction tr(self->db); @@ -1588,7 +1598,7 @@ ACTOR Future fullyDeleteGranule(BlobManagerData* self, UID granuleId, KeyR } if (BM_DEBUG) { - printf("Fully deleting granule %s: success\n", granuleId.toString().c_str()); + fmt::print("Fully deleting granule {0}: success\n", granuleId.toString()); } return Void(); @@ -1604,7 +1614,7 @@ ACTOR Future fullyDeleteGranule(BlobManagerData* self, UID granuleId, KeyR */ ACTOR Future partiallyDeleteGranule(BlobManagerData* self, UID granuleId, Version pruneVersion) { if (BM_DEBUG) { - printf("Partially deleting granule %s: init\n", granuleId.toString().c_str()); + fmt::print("Partially deleting granule {0}: init\n", granuleId.toString()); } // get files @@ -1651,9 +1661,9 @@ ACTOR Future partiallyDeleteGranule(BlobManagerData* self, UID granuleId, } if (BM_DEBUG) { - printf("Partially deleting granule %s: deleting %d files\n", granuleId.toString().c_str(), deletions.size()); + fmt::print("Partially deleting granule {0}: deleting {1} files\n", granuleId.toString(), deletions.size()); for (auto filename : filesToDelete) { - printf(" - %s\n", filename.c_str()); + fmt::print(" - {0}\n", filename); } } @@ -1669,7 +1679,7 @@ ACTOR Future partiallyDeleteGranule(BlobManagerData* self, UID granuleId, // delete metadata in FDB (deleted file keys) if (BM_DEBUG) { - printf("Partially deleting granule %s: deleting file keys\n", granuleId.toString().c_str()); + fmt::print("Partially deleting granule {0}: deleting file keys\n", granuleId.toString()); } state Transaction tr(self->db); @@ -1689,7 +1699,7 @@ ACTOR Future partiallyDeleteGranule(BlobManagerData* self, UID granuleId, } if (BM_DEBUG) { - printf("Partially deleting granule %s: success\n", granuleId.toString().c_str()); + fmt::print("Partially deleting granule {0}: success\n", granuleId.toString()); } return Void(); } @@ -1704,10 +1714,11 @@ ACTOR Future partiallyDeleteGranule(BlobManagerData* self, UID granuleId, */ ACTOR Future pruneRange(BlobManagerData* self, KeyRef startKey, KeyRef endKey, Version pruneVersion, bool force) { if (BM_DEBUG) { - printf("pruneRange starting for range [%s-%s) @ pruneVersion=%lld, force=%s\n", - startKey.printable().c_str(), - endKey.printable().c_str(), - pruneVersion); + fmt::print("pruneRange starting for range [{0} - {1}) @ pruneVersion={2}, force={3}\n", + startKey.printable(), + endKey.printable(), + pruneVersion, + force); } // queue of for BFS traversal of history @@ -1735,10 +1746,10 @@ ACTOR Future pruneRange(BlobManagerData* self, KeyRef startKey, KeyRef end state KeyRangeMap::iterator activeRange; for (activeRange = activeRanges.begin(); activeRange != activeRanges.end(); ++activeRange) { if (BM_DEBUG) { - printf("Checking if active range [%s-%s), owned by BW %s, should be pruned\n", - activeRange.begin().printable().c_str(), - activeRange.end().printable().c_str(), - activeRange.value().toString().c_str()); + fmt::print("Checking if active range [{0} - {1}), owned by BW {2}, should be pruned\n", + activeRange.begin().printable(), + activeRange.end().printable(), + activeRange.value().toString()); } // assumption: prune boundaries must respect granule boundaries @@ -1753,9 +1764,9 @@ ACTOR Future pruneRange(BlobManagerData* self, KeyRef startKey, KeyRef end loop { try { if (BM_DEBUG) { - printf("Fetching latest history entry for range [%s-%s)\n", - activeRange.begin().printable().c_str(), - activeRange.end().printable().c_str()); + fmt::print("Fetching latest history entry for range [{0} - {1})\n", + activeRange.begin().printable(), + activeRange.end().printable()); } Optional history = wait(getLatestGranuleHistory(&tr, activeRange.range())); // TODO: can we tell from the krm that this range is not valid, so that we don't need to do a get @@ -1785,11 +1796,11 @@ ACTOR Future pruneRange(BlobManagerData* self, KeyRef startKey, KeyRef end historyEntryQueue.pop(); if (BM_DEBUG) { - printf("Processing history node [%s-%s) with versions [%lld, %lld)\n", - currRange.begin.printable().c_str(), - currRange.end.printable().c_str(), - startVersion, - endVersion); + fmt::print("Processing history node [{0} - {1}) with versions [{2}, {3})\n", + currRange.begin.printable(), + currRange.end.printable(), + startVersion, + endVersion); } // get the persisted history entry for this granule @@ -1807,8 +1818,8 @@ ACTOR Future pruneRange(BlobManagerData* self, KeyRef startKey, KeyRef end } if (BM_DEBUG) { - printf("Found history entry for this node. It's granuleID is %s\n", - currHistoryNode.granuleID.toString().c_str()); + fmt::print("Found history entry for this node. It's granuleID is {0}\n", + currHistoryNode.granuleID.toString()); } // There are three cases this granule can fall into: @@ -1819,12 +1830,12 @@ ACTOR Future pruneRange(BlobManagerData* self, KeyRef startKey, KeyRef end // - otherwise, this granule is active, so don't schedule it for deletion if (force || endVersion <= pruneVersion) { if (BM_DEBUG) { - printf("Granule %s will be FULLY deleted\n", currHistoryNode.granuleID.toString().c_str()); + fmt::print("Granule {0} will be FULLY deleted\n", currHistoryNode.granuleID.toString()); } toFullyDelete.push_back({ currHistoryNode.granuleID, historyKey }); } else if (startVersion < pruneVersion) { if (BM_DEBUG) { - printf("Granule %s will be partially deleted\n", currHistoryNode.granuleID.toString().c_str()); + fmt::print("Granule {0} will be partially deleted\n", currHistoryNode.granuleID.toString()); } toPartiallyDelete.push_back({ currHistoryNode.granuleID }); } @@ -1834,18 +1845,18 @@ ACTOR Future pruneRange(BlobManagerData* self, KeyRef startKey, KeyRef end // if we already added this node to queue, skip it; otherwise, mark it as visited if (visited.count({ parent.first.begin.begin(), parent.second })) { if (BM_DEBUG) { - printf("Already added %s to queue, so skipping it\n", currHistoryNode.granuleID.toString().c_str()); + fmt::print("Already added {0} to queue, so skipping it\n", currHistoryNode.granuleID.toString()); } continue; } visited.insert({ parent.first.begin.begin(), parent.second }); if (BM_DEBUG) { - printf("Adding parent [%s-%s) with versions [%lld-%lld) to queue\n", - parent.first.begin.printable().c_str(), - parent.first.end.printable().c_str(), - parent.second, - startVersion); + fmt::print("Adding parent [{0} - {1}) with versions [{2} - {3}) to queue\n", + parent.first.begin.printable(), + parent.first.end.printable(), + parent.second, + startVersion); } // the parent's end version is this node's startVersion, @@ -1871,7 +1882,7 @@ ACTOR Future pruneRange(BlobManagerData* self, KeyRef startKey, KeyRef end state int i; if (BM_DEBUG) { - printf("%d granules to fully delete\n", toFullyDelete.size()); + fmt::print("{0} granules to fully delete\n", toFullyDelete.size()); } for (i = toFullyDelete.size() - 1; i >= 0; --i) { UID granuleId; @@ -1879,19 +1890,19 @@ ACTOR Future pruneRange(BlobManagerData* self, KeyRef startKey, KeyRef end std::tie(granuleId, historyKey) = toFullyDelete[i]; // FIXME: consider batching into a single txn (need to take care of txn size limit) if (BM_DEBUG) { - printf("About to fully delete granule %s\n", granuleId.toString().c_str()); + fmt::print("About to fully delete granule {0}\n", granuleId.toString()); } wait(fullyDeleteGranule(self, granuleId, historyKey)); } if (BM_DEBUG) { - printf("%d granules to partially delete\n", toPartiallyDelete.size()); + fmt::print("{0} granules to partially delete\n", toPartiallyDelete.size()); } std::vector> partialDeletions; for (i = toPartiallyDelete.size() - 1; i >= 0; --i) { UID granuleId = toPartiallyDelete[i]; if (BM_DEBUG) { - printf("About to partially delete granule %s\n", granuleId.toString().c_str()); + fmt::print("About to partially delete granule {0}\n", granuleId.toString()); } partialDeletions.emplace_back(partiallyDeleteGranule(self, granuleId, pruneVersion)); } @@ -1926,16 +1937,16 @@ ACTOR Future pruneRange(BlobManagerData* self, KeyRef startKey, KeyRef end } break; } catch (Error& e) { - printf("Attempt to clear prune intent got error %s\n", e.name()); + fmt::print("Attempt to clear prune intent got error {}\n", e.name()); wait(tr.onError(e)); } } if (BM_DEBUG) { - printf("Successfully pruned range [%s-%s) at pruneVersion=%lld\n", - startKey.printable().c_str(), - endKey.printable().c_str(), - pruneVersion); + fmt::print("Successfully pruned range [{0} - {1}) at pruneVersion={2}\n", + startKey.printable(), + endKey.printable(), + pruneVersion); } return Void(); } @@ -1964,7 +1975,7 @@ ACTOR Future monitorPruneKeys(BlobManagerData* self) { // setup bstore try { if (BM_DEBUG) { - printf("BM constructing backup container from %s\n", SERVER_KNOBS->BG_URL.c_str()); + fmt::print("BM constructing backup container from {}\n", SERVER_KNOBS->BG_URL.c_str()); } self->bstore = BackupContainerFileSystem::openContainerFS(SERVER_KNOBS->BG_URL); if (BM_DEBUG) { @@ -1972,7 +1983,7 @@ ACTOR Future monitorPruneKeys(BlobManagerData* self) { } } catch (Error& e) { if (BM_DEBUG) { - printf("BM got backup container init error %s\n", e.name()); + fmt::print("BM got backup container init error {0}\n", e.name()); } throw e; } @@ -2088,11 +2099,11 @@ ACTOR Future monitorPruneKeys(BlobManagerData* self) { bool force; std::tie(pruneVersion, force) = decodeBlobGranulePruneValue(pruneIntents[rangeIdx].value); - printf("about to prune range [%s-%s) @ %d, force=%s\n", - rangeStartKey.printable().c_str(), - rangeEndKey.printable().c_str(), - pruneVersion, - force ? "T" : "F"); + fmt::print("about to prune range [{0} - {1}) @ {2}, force={3}\n", + rangeStartKey.printable(), + rangeEndKey.printable(), + pruneVersion, + force ? "T" : "F"); prunes.emplace_back(pruneRange(self, rangeStartKey, rangeEndKey, pruneVersion, force)); } @@ -2120,7 +2131,7 @@ ACTOR Future monitorPruneKeys(BlobManagerData* self) { throw e; } if (BM_DEBUG) { - printf("monitorPruneKeys for BM %s saw error %s\n", self->id.toString().c_str(), e.name()); + fmt::print("monitorPruneKeys for BM {0} saw error {1}\n", self->id.toString(), e.name()); } // don't want to kill the blob manager for errors around pruning TraceEvent("MonitorPruneKeysError", self->id).detail("Error", e.name()); @@ -2131,7 +2142,7 @@ ACTOR Future monitorPruneKeys(BlobManagerData* self) { } } catch (Error& e) { if (BM_DEBUG) { - printf("monitorPruneKeys got error %s\n", e.name()); + fmt::print("monitorPruneKeys got error {}\n", e.name()); } throw e; } diff --git a/fdbserver/BlobWorker.actor.cpp b/fdbserver/BlobWorker.actor.cpp index b7a1bc57d4..593b423ec3 100644 --- a/fdbserver/BlobWorker.actor.cpp +++ b/fdbserver/BlobWorker.actor.cpp @@ -319,7 +319,7 @@ ACTOR Future updateGranuleSplitState(Transaction* tr, if (totalState.empty()) { ASSERT(newState == BlobGranuleSplitState::Done); if (BW_DEBUG) { - printf("Found empty split state for parent granule %s\n", parentGranuleID.toString().c_str()); + fmt::print("Found empty split state for parent granule {0}\n", parentGranuleID.toString()); } // must have retried and successfully nuked everything return Void(); @@ -355,11 +355,11 @@ ACTOR Future updateGranuleSplitState(Transaction* tr, if (currentState < newState) { if (BW_DEBUG) { - printf("Updating granule %s split state from %s %d -> %d\n", - currentGranuleID.toString().c_str(), - parentGranuleID.toString().c_str(), - currentState, - newState); + fmt::print("Updating granule {0} split state from {1} {2} -> {3}\n", + currentGranuleID.toString(), + parentGranuleID.toString(), + currentState, + newState); } Key myStateKey = blobGranuleSplitKeyFor(parentGranuleID, currentGranuleID); @@ -368,9 +368,7 @@ ACTOR Future updateGranuleSplitState(Transaction* tr, // we are the last one to change from Assigned -> Done, so everything can be cleaned up for the old // change feed and splitting state if (BW_DEBUG) { - printf("%s destroying old granule %s\n", - currentGranuleID.toString().c_str(), - parentGranuleID.toString().c_str()); + fmt::print("{0} destroying old granule {1}\n", currentGranuleID.toString(), parentGranuleID.toString()); } // FIXME: appears change feed destroy isn't working! ADD BACK @@ -384,20 +382,20 @@ ACTOR Future updateGranuleSplitState(Transaction* tr, totalStarted == 1) { // We are the last one to change from Start -> Assigned, so we can stop the parent change feed. if (BW_DEBUG) { - printf("%s stopping change feed for old granule %s\n", - currentGranuleID.toString().c_str(), - parentGranuleID.toString().c_str()); + fmt::print("{0} stopping change feed for old granule {1}\n", + currentGranuleID.toString().c_str(), + parentGranuleID.toString().c_str()); } wait(updateChangeFeed(tr, KeyRef(parentGranuleID.toString()), ChangeFeedStatus::CHANGE_FEED_STOP)); } } } else if (BW_DEBUG) { - printf("Ignoring granule %s split state from %s %d -> %d\n", - currentGranuleID.toString().c_str(), - parentGranuleID.toString().c_str(), - currentState, - newState); + fmt::print("Ignoring granule {0} split state from {1} {2} -> {3}\n", + currentGranuleID.toString(), + parentGranuleID.toString(), + currentState, + newState); } return Void(); @@ -515,7 +513,7 @@ ACTOR Future writeDeltaFile(Reference bwData, } if (BW_DEBUG) { - printf("deleting s3 delta file %s after error %s\n", fname.c_str(), e.name()); + fmt::print("deleting s3 delta file {0} after error {1}\n", fname, e.name()); } state Error eState = e; ++bwData->stats.s3DeleteReqs; @@ -558,10 +556,10 @@ ACTOR Future writeSnapshot(Reference bwData, wait(delay(0, TaskPriority::BlobWorkerUpdateStorage)); if (BW_DEBUG) { - printf("Granule [%s - %s) read %d snapshot rows\n", - keyRange.begin.printable().c_str(), - keyRange.end.printable().c_str(), - snapshot.size()); + fmt::print("Granule [{0} - {1}) read {2} snapshot rows\n", + keyRange.begin.printable(), + keyRange.end.printable(), + snapshot.size()); } // TODO REMOVE sanity checks! @@ -571,9 +569,9 @@ ACTOR Future writeSnapshot(Reference bwData, } for (int i = 0; i < snapshot.size() - 1; i++) { if (snapshot[i].key >= snapshot[i + 1].key) { - printf("SORT ORDER VIOLATION IN SNAPSHOT FILE: %s, %s\n", - snapshot[i].key.printable().c_str(), - snapshot[i + 1].key.printable().c_str()); + fmt::print("SORT ORDER VIOLATION IN SNAPSHOT FILE: {0}, {1}\n", + snapshot[i].key.printable(), + snapshot[i + 1].key.printable()); } ASSERT(snapshot[i].key < snapshot[i + 1].key); } @@ -636,7 +634,7 @@ ACTOR Future writeSnapshot(Reference bwData, } if (BW_DEBUG) { - printf("deleting s3 snapshot file %s after error %s\n", fname.c_str(), e.name()); + fmt::print("deleting s3 snapshot file {0} after error {1}\n", fname, e.name()); } state Error eState = e; ++bwData->stats.s3DeleteReqs; @@ -645,11 +643,11 @@ ACTOR Future writeSnapshot(Reference bwData, } if (BW_DEBUG) { - printf("Granule [%s - %s) committed new snapshot file %s with %d bytes\n\n", - keyRange.begin.printable().c_str(), - keyRange.end.printable().c_str(), - fname.c_str(), - serialized.size()); + fmt::print("Granule [{0} - {1}) committed new snapshot file {2} with {3} bytes\n\n", + keyRange.begin.printable(), + keyRange.end.printable(), + fname, + serialized.size()); } if (BUGGIFY_WITH_PROB(0.1)) { @@ -663,9 +661,9 @@ ACTOR Future dumpInitialSnapshotFromFDB(Reference Reference metadata, UID granuleID) { if (BW_DEBUG) { - printf("Dumping snapshot from FDB for [%s - %s)\n", - metadata->keyRange.begin.printable().c_str(), - metadata->keyRange.end.printable().c_str()); + fmt::print("Dumping snapshot from FDB for [{0} - {1})\n", + metadata->keyRange.begin.printable(), + metadata->keyRange.end.printable()); } state Reference tr = makeReference(bwData->db); @@ -707,10 +705,10 @@ ACTOR Future dumpInitialSnapshotFromFDB(Reference return f; } catch (Error& e) { if (BW_DEBUG) { - printf("Dumping snapshot from FDB for [%s - %s) got error %s\n", - metadata->keyRange.begin.printable().c_str(), - metadata->keyRange.end.printable().c_str(), - e.name()); + fmt::print("Dumping snapshot from FDB for [{0} - {1}) got error {2}\n", + metadata->keyRange.begin.printable(), + metadata->keyRange.end.printable(), + e.name()); } wait(tr->onError(e)); } @@ -727,9 +725,9 @@ ACTOR Future compactFromBlob(Reference bwData, Version version) { wait(delay(0, TaskPriority::BlobWorkerUpdateStorage)); if (BW_DEBUG) { - printf("Compacting snapshot from blob for [%s - %s)\n", - metadata->keyRange.begin.printable().c_str(), - metadata->keyRange.end.printable().c_str()); + fmt::print("Compacting snapshot from blob for [{0} - {1})\n", + metadata->keyRange.begin.printable().c_str(), + metadata->keyRange.end.printable().c_str()); } ASSERT(!files.snapshotFiles.empty()); @@ -800,10 +798,10 @@ ACTOR Future compactFromBlob(Reference bwData, return f; } catch (Error& e) { if (BW_DEBUG) { - printf("Compacting snapshot from blob for [%s - %s) got error %s\n", - metadata->keyRange.begin.printable().c_str(), - metadata->keyRange.end.printable().c_str(), - e.name()); + fmt::print("Compacting snapshot from blob for [{0} - {1}) got error {2}\n", + metadata->keyRange.begin.printable(), + metadata->keyRange.end.printable(), + e.name()); } throw e; } @@ -1022,10 +1020,10 @@ static Version doGranuleRollback(Reference metadata, } metadata->pendingDeltaVersion = cfRollbackVersion; if (BW_DEBUG) { - printf("[%s - %s) rollback discarding all %d in-memory mutations\n", - metadata->keyRange.begin.printable().c_str(), - metadata->keyRange.end.printable().c_str(), - metadata->currentDeltas.size()); + fmt::print("[{0} - {1}) rollback discarding all {2} in-memory mutations\n", + metadata->keyRange.begin.printable(), + metadata->keyRange.end.printable(), + metadata->currentDeltas.size()); } // discard all in-memory mutations @@ -1131,7 +1129,7 @@ ACTOR Future waitVersionCommitted(Reference bwData, // a bit, as blobGranuleUpdateFiles will switch to the new change feed if (e.code() != error_code_change_feed_cancelled) { if (BW_DEBUG) { - printf("waitVersionCommitted WAL got unexpected error %s\n", e.name()); + fmt::print("waitVersionCommitted WAL got unexpected error {}\n", e.name()); } throw e; } @@ -1703,9 +1701,9 @@ ACTOR Future blobGranuleUpdateFiles(Reference bwData, if (inFlightFiles.size() > 10 && inFlightFiles.front().version <= metadata->knownCommittedVersion) { if (BW_DEBUG) { - printf("[%s - %s) Waiting on delta file b/c old change feed\n", - metadata->keyRange.begin.printable().c_str(), - metadata->keyRange.end.printable().c_str()); + fmt::print("[{0} - {1}) Waiting on delta file b/c old change feed\n", + metadata->keyRange.begin.printable(), + metadata->keyRange.end.printable()); } choose { when(BlobFileIndex completedDeltaFile = wait(inFlightFiles.front().future)) {} @@ -1742,10 +1740,10 @@ ACTOR Future blobGranuleUpdateFiles(Reference bwData, } else { ++bwData->stats.granuleUpdateErrors; if (BW_DEBUG) { - printf("Granule file updater for [%s - %s) got error %s, exiting\n", - metadata->keyRange.begin.printable().c_str(), - metadata->keyRange.end.printable().c_str(), - e.name()); + fmt::print("Granule file updater for [{0} - {1}) got error {2}, exiting\n", + metadata->keyRange.begin.printable(), + metadata->keyRange.end.printable(), + e.name()); } TraceEvent(SevWarn, "GranuleFileUpdaterError", bwData->id).detail("Granule", metadata->keyRange).error(e); @@ -1868,7 +1866,7 @@ ACTOR Future blobGranuleLoadHistory(Reference bwData, return Void(); } if (BW_DEBUG) { - printf("Loading blob granule history got unexpected error %s\n", e.name()); + fmt::print("Loading blob granule history got unexpected error {}\n", e.name()); } // TODO this should never happen? ASSERT(false); @@ -2008,11 +2006,11 @@ ACTOR Future waitForVersion(Reference metadata, Version v ACTOR Future handleBlobGranuleFileRequest(Reference bwData, BlobGranuleFileRequest req) { if (BW_REQUEST_DEBUG || DEBUG_BW_WAIT_VERSION == req.readVersion) { - printf("BW %s processing blobGranuleFileRequest for range [%s-%s) @ %lld\n", - bwData->id.toString().c_str(), - req.keyRange.begin.printable().c_str(), - req.keyRange.end.printable().c_str(), - req.readVersion); + fmt::print("BW {0} processing blobGranuleFileRequest for range [{1} -{2}) @ {3}\n", + bwData->id.toString(), + req.keyRange.begin.printable(), + req.keyRange.end.printable(), + req.readVersion); } try { @@ -2030,12 +2028,12 @@ ACTOR Future handleBlobGranuleFileRequest(Reference bwData bool isValid = r.value().activeMetadata.isValid(); if (lastRangeEnd < r.begin() || !isValid) { if (BW_REQUEST_DEBUG || DEBUG_BW_WAIT_VERSION == req.readVersion) { - printf("No %s blob data for [%s - %s) in request range [%s - %s), skipping request\n", - isValid ? "" : "valid", - lastRangeEnd.printable().c_str(), - r.begin().printable().c_str(), - req.keyRange.begin.printable().c_str(), - req.keyRange.end.printable().c_str()); + fmt::print("No {0} blob data for [{1} - {2}) in request range [{3} - {4}), skipping request\n", + isValid ? "" : "valid", + lastRangeEnd.printable(), + r.begin().printable(), + req.keyRange.begin.printable(), + req.keyRange.end.printable()); } throw wrong_shard_server(); @@ -2045,11 +2043,11 @@ ACTOR Future handleBlobGranuleFileRequest(Reference bwData } if (lastRangeEnd < req.keyRange.end) { if (BW_REQUEST_DEBUG || DEBUG_BW_WAIT_VERSION == req.readVersion) { - printf("No blob data for [%s - %s) in request range [%s - %s), skipping request\n", - lastRangeEnd.printable().c_str(), - req.keyRange.end.printable().c_str(), - req.keyRange.begin.printable().c_str(), - req.keyRange.end.printable().c_str()); + fmt::print("No blob data for [{0} - {1}) in request range [{2} - {3}), skipping request\n", + lastRangeEnd.printable(), + req.keyRange.end.printable(), + req.keyRange.begin.printable(), + req.keyRange.end.printable()); } throw wrong_shard_server(); @@ -2202,12 +2200,12 @@ ACTOR Future handleBlobGranuleFileRequest(Reference bwData } if (!metadata->cancelled.canBeSet()) { - printf("ERROR: Request [%s - %s) @ %lld cancelled for granule [%s - %s) after waitForVersion!\n", - req.keyRange.begin.printable().c_str(), - req.keyRange.end.printable().c_str(), - req.readVersion, - metadata->keyRange.begin.printable().c_str(), - metadata->keyRange.end.printable().c_str()); + fmt::print("ERROR: Request [{0} - {1}) @ {2} cancelled for granule [{3} - {4}) after waitForVersion!\n", + req.keyRange.begin.printable(), + req.keyRange.end.printable(), + req.readVersion, + metadata->keyRange.begin.printable(), + metadata->keyRange.end.printable()); } // granule is up to date, do read @@ -2326,10 +2324,10 @@ ACTOR Future openGranule(Reference bwData, As state Key lockKey = blobGranuleLockKeyFor(req.keyRange); if (BW_DEBUG) { - printf("%s [%s - %s) opening\n", - bwData->id.toString().c_str(), - req.keyRange.begin.printable().c_str(), - req.keyRange.end.printable().c_str()); + fmt::print("{0} [{1} - {2}) opening\n", + bwData->id.toString(), + req.keyRange.begin.printable(), + req.keyRange.end.printable()); } loop { @@ -2416,7 +2414,7 @@ ACTOR Future openGranule(Reference bwData, As decodeBlobGranuleLockValue(parentGranuleLockValue.get()); UID parentGranuleID = std::get<2>(parentGranuleLock); if (BW_DEBUG) { - printf(" parent granule id %s\n", parentGranuleID.toString().c_str()); + fmt::print(" parent granule id {}\n", parentGranuleID.toString()); } info.parentGranule = std::pair(parentGranuleRange, parentGranuleID); @@ -2580,7 +2578,7 @@ ACTOR Future changeBlobRange(Reference bwData, }*/ bool thisAssignmentNewer = newerRangeAssignment(r.value(), epoch, seqno); if (BW_DEBUG) { - printf("thisAssignmentNewer=%s\n", thisAssignmentNewer ? "true" : "false"); + fmt::print("thisAssignmentNewer={}\n", thisAssignmentNewer ? "true" : "false"); } // if this granule already has it, and this was a specialassignment (i.e. a new blob maanger is @@ -2594,7 +2592,7 @@ ACTOR Future changeBlobRange(Reference bwData, } if (BW_DEBUG) { - printf("last: (%d, %d). now: (%d, %d)\n", r.value().lastEpoch, r.value().lastSeqno, epoch, seqno); + fmt::print("last: ({0}, {1}). now: ({2}, {3})\n", r.value().lastEpoch, r.value().lastSeqno, epoch, seqno); } if (r.value().lastEpoch == epoch && r.value().lastSeqno == seqno) { @@ -2722,12 +2720,12 @@ ACTOR Future registerBlobWorker(Reference bwData, BlobWork wait(tr->commit()); if (BW_DEBUG) { - printf("Registered blob worker %s\n", interf.id().toString().c_str()); + fmt::print("Registered blob worker {}\n", interf.id().toString()); } return Void(); } catch (Error& e) { if (BW_DEBUG) { - printf("Registering blob worker %s got error %s\n", interf.id().toString().c_str(), e.name()); + fmt::print("Registering blob worker {0} got error {1}\n", interf.id().toString(), e.name()); } wait(tr->onError(e)); } @@ -2884,7 +2882,7 @@ ACTOR Future blobWorker(BlobWorkerInterface bwInterf, try { if (BW_DEBUG) { - printf("BW constructing backup container from %s\n", SERVER_KNOBS->BG_URL.c_str()); + fmt::print("BW constructing backup container from {0}\n", SERVER_KNOBS->BG_URL); } self->bstore = BackupContainerFileSystem::openContainerFS(SERVER_KNOBS->BG_URL); if (BW_DEBUG) { @@ -2895,7 +2893,7 @@ ACTOR Future blobWorker(BlobWorkerInterface bwInterf, wait(registerBlobWorker(self, bwInterf)); } catch (Error& e) { if (BW_DEBUG) { - printf("BW got backup container init error %s\n", e.name()); + fmt::print("BW got backup container init error {0}\n", e.name()); } // if any errors came up while initializing the blob worker, let the blob manager know // that recruitment failed @@ -2928,7 +2926,7 @@ ACTOR Future blobWorker(BlobWorkerInterface bwInterf, when(state GranuleStatusStreamRequest req = waitNext(bwInterf.granuleStatusStreamRequest.getFuture())) { if (self->managerEpochOk(req.managerEpoch)) { if (BW_DEBUG) { - printf("Worker %s got new granule status endpoint\n", self->id.toString().c_str()); + fmt::print("Worker {0} got new granule status endpoint\n", self->id.toString()); } // TODO: pick a reasonable byte limit instead of just piggy-backing req.reply.setByteLimit(SERVER_KNOBS->RANGESTREAM_LIMIT_BYTES); From 04e88396569bb8199f2055bb7789c710f71a13a9 Mon Sep 17 00:00:00 2001 From: Josh Slocum Date: Fri, 14 Jan 2022 18:43:16 -0600 Subject: [PATCH 135/413] fixed incorrect assert --- fdbserver/BlobWorker.actor.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/fdbserver/BlobWorker.actor.cpp b/fdbserver/BlobWorker.actor.cpp index 593b423ec3..6eff98af51 100644 --- a/fdbserver/BlobWorker.actor.cpp +++ b/fdbserver/BlobWorker.actor.cpp @@ -1492,12 +1492,12 @@ ACTOR Future blobGranuleUpdateFiles(Reference bwData, metadata->keyRange); } else { - if (cfRollbackVersion <= startState.changeFeedStartVersion) { + if (cfRollbackVersion < startState.changeFeedStartVersion) { fmt::print("Rollback past CF start??. rollback={0}, start={1}\n", cfRollbackVersion, startState.changeFeedStartVersion); } - ASSERT(cfRollbackVersion > startState.changeFeedStartVersion); + ASSERT(cfRollbackVersion >= startState.changeFeedStartVersion); changeFeedFuture = bwData->db->getChangeFeedStream(metadata->activeCFData.get(), cfKey, From e4e7b638c8194e953fc09d25b5cbf746be8f8f72 Mon Sep 17 00:00:00 2001 From: Josh Slocum Date: Sat, 15 Jan 2022 09:33:47 -0600 Subject: [PATCH 136/413] reworked blob manager recovery to correctly handle splits --- fdbserver/BlobManager.actor.cpp | 147 +++++++++++++++++++++++++------- 1 file changed, 114 insertions(+), 33 deletions(-) diff --git a/fdbserver/BlobManager.actor.cpp b/fdbserver/BlobManager.actor.cpp index 848bb47e78..b925001e54 100644 --- a/fdbserver/BlobManager.actor.cpp +++ b/fdbserver/BlobManager.actor.cpp @@ -399,7 +399,7 @@ ACTOR Future doRangeAssignment(BlobManagerData* bmData, RangeAssignment as // So assignment needs to be retried elsewhere, and a revoke is trivially complete if (assignment.isAssign) { if (BM_DEBUG) { - fmt::print("BM got error assigning range [%s - %s) to worker %s, requeueing\n", + fmt::print("BM got error assigning range [{0} - {1}) to worker {2}, requeueing\n", assignment.keyRange.begin.printable(), assignment.keyRange.end.printable(), workerID.toString()); @@ -844,11 +844,11 @@ ACTOR Future maybeSplitRange(BlobManagerData* bmData, historyValue.parentGranules.push_back(historyValue.arena(), std::pair(granuleRange, granuleStartVersion)); - /*printf("Creating history entry [%s - %s) - [%lld - %lld)\n", - newRanges[i].printable().c_str(), - newRanges[i + 1].printable().c_str(), - granuleStartVersion, - latestVersion);*/ + /*fmt::print("Creating history entry [{0} - {1}) - [{2} - {3})\n", + newRanges[i].printable(), + newRanges[i + 1].printable(), + granuleStartVersion, + latestVersion);*/ tr->set(historyKey, blobGranuleHistoryValueFor(historyValue)); } @@ -1185,7 +1185,9 @@ ACTOR Future recoverBlobManager(BlobManagerData* bmData) { // BM is recovering. Now the mapping at this time looks like G->deadBW. But the rangeAssigner handles this: // we'll try to assign a range to a dead worker and fail and reassign it to the next best worker. // - // 2. We get the existing split intentions that were Started but not acknowledged by any blob workers and + // 2. We get all granule history entries, to get a mapping from granule id to key range, for step 3. + // + // 3. We get the existing split intentions that were Started but not acknowledged by any blob workers and // add them to our key range map, bmData->granuleAssignments. Note that we are adding them on top of // the granule mappings and since we are using a key range map, we end up with the same set of shard // boundaries as the old blob manager had. For these splits, we simply assign the range to the next @@ -1193,14 +1195,22 @@ ACTOR Future recoverBlobManager(BlobManagerData* bmData) { // Details: Note that this means that if a worker we intended to give a splitted range to dies // before the new BM recovers, then we'll simply assign the range to the next best worker. // - // 3. For every range in our granuleAssignments, we send an assign request to the stream of requests, + // 4. For every range in our granuleAssignments, we send an assign request to the stream of requests, // ultimately giving every range back to some worker (trying to mimic the state of the old BM). // If the worker already had the range, this is a no-op. If the worker didn't have it, it will // begin persisting it. The worker that had the same range before will now be at a lower seqno. state KeyRangeMap> workerAssignments; state Reference tr = makeReference(bmData->db); + state std::unordered_map granuleIdToRange; + // TODO KNOB + state int rowLimit = BUGGIFY ? deterministicRandom()->randomInt(2, 10) : 10000; + + if (BM_DEBUG) { + printf("BM %lld recovering:\n", bmData->epoch); + printf("BM %lld found old assignments:\n", bmData->epoch); + } // Step 1. Get the latest known mapping of granules to blob workers (i.e. assignments) state KeyRef beginKey = normalKeys.begin; loop { @@ -1211,8 +1221,8 @@ ACTOR Future recoverBlobManager(BlobManagerData* bmData) { // TODO: replace row limit with knob KeyRange nextRange(KeyRangeRef(beginKey, normalKeys.end)); - RangeResult results = wait( - krmGetRanges(tr, blobGranuleMappingKeys.begin, nextRange, 10000, GetRangeLimits::BYTE_LIMIT_UNLIMITED)); + RangeResult results = wait(krmGetRanges( + tr, blobGranuleMappingKeys.begin, nextRange, rowLimit, GetRangeLimits::BYTE_LIMIT_UNLIMITED)); Key lastEndKey; // Add the mappings to our in memory key range map @@ -1225,6 +1235,18 @@ ACTOR Future recoverBlobManager(BlobManagerData* bmData) { UID existingOwner = decodeBlobGranuleMappingValue(results[rangeIdx].value); workerAssignments.insert(KeyRangeRef(granuleStartKey, granuleEndKey), existingOwner); bmData->knownBlobRanges.insert(KeyRangeRef(granuleStartKey, granuleEndKey), true); + if (BM_DEBUG) { + fmt::print(" [{0} - {1})={2}\n", + results[rangeIdx].key.printable(), + results[rangeIdx + 1].key.printable(), + results[rangeIdx].value.printable()); + } + } else { + if (BM_DEBUG) { + fmt::print(" [{0} - {1})=\n", + results[rangeIdx].key.printable(), + results[rangeIdx + 1].key.printable()); + } } } @@ -1238,32 +1260,33 @@ ACTOR Future recoverBlobManager(BlobManagerData* bmData) { } } - // Step 2. Get the latest known split intentions + // TODO could avoid if no splits in progress + // Step 2. Read all history entries, so we can know the range of each sub-granule that is splitting tr->reset(); - beginKey = blobGranuleSplitKeys.begin; + beginKey = blobGranuleHistoryKeys.begin; + if (BM_DEBUG) { + printf("BM %lld found history entries:\n", bmData->epoch); + } loop { try { tr->setOption(FDBTransactionOptions::ACCESS_SYSTEM_KEYS); tr->setOption(FDBTransactionOptions::PRIORITY_SYSTEM_IMMEDIATE); wait(checkManagerLock(tr, bmData)); - // TODO: replace row limit with knob - RangeResult results = wait(tr->getRange(KeyRangeRef(beginKey, blobGranuleSplitKeys.end), 10000)); + RangeResult results = wait(tr->getRange(KeyRangeRef(beginKey, blobGranuleHistoryKeys.end), rowLimit)); // Add the granules for the started split intentions to the in-memory key range map - for (auto split : results) { - UID parentGranuleID, granuleID; - BlobGranuleSplitState splitState; + for (auto history : results) { + KeyRange granuleRange; Version version; - if (split.expectedSize() == 0) { - continue; - } - std::tie(parentGranuleID, granuleID) = decodeBlobGranuleSplitKey(split.key); - std::tie(splitState, version) = decodeBlobGranuleSplitValue(split.value); - const KeyRange range = blobGranuleSplitKeyRangeFor(parentGranuleID); - if (splitState <= BlobGranuleSplitState::Initialized) { - // the empty UID signifies that we need to find an owner (worker) for this range - workerAssignments.insert(range, UID()); + std::tie(granuleRange, version) = decodeBlobGranuleHistoryKey(history.key); + Standalone v = decodeBlobGranuleHistoryValue(history.value); + granuleIdToRange[v.granuleID] = granuleRange; + if (BM_DEBUG) { + fmt::print(" {0}=[{1} - {2})\n", + v.granuleID, + granuleRange.begin.printable(), + granuleRange.end.printable()); } } @@ -1277,15 +1300,72 @@ ACTOR Future recoverBlobManager(BlobManagerData* bmData) { } } - bmData->knownBlobRanges.coalesce(normalKeys); + // Step 3. Get the latest known split intentions + tr->reset(); + beginKey = blobGranuleSplitKeys.begin; + if (BM_DEBUG) { + printf("BM %lld found in progress splits:\n", bmData->epoch); + } + loop { + try { + tr->setOption(FDBTransactionOptions::ACCESS_SYSTEM_KEYS); + tr->setOption(FDBTransactionOptions::PRIORITY_SYSTEM_IMMEDIATE); + wait(checkManagerLock(tr, bmData)); - // Step 3. Send assign requests for all the granules and transfer assignments + // TODO: replace row limit with knob + RangeResult results = wait(tr->getRange(KeyRangeRef(beginKey, blobGranuleSplitKeys.end), rowLimit)); + + // Add the granules for the started split intentions to the in-memory key range map + for (auto split : results) { + UID parentGranuleID, granuleID; + BlobGranuleSplitState splitState; + Version version; + + std::tie(parentGranuleID, granuleID) = decodeBlobGranuleSplitKey(split.key); + if (split.value.size() == 0) { + printf("No value for %s/%s split??\n", + parentGranuleID.toString().c_str(), + granuleID.toString().c_str()); + ASSERT(split.value.size() > 0); + } + std::tie(splitState, version) = decodeBlobGranuleSplitValue(split.value); + + // TODO THIS RANGE IS WRONG + ASSERT(granuleIdToRange.count(granuleID) == 1); + const KeyRange range = granuleIdToRange[granuleID]; + if (splitState <= BlobGranuleSplitState::Initialized) { + // the empty UID signifies that we need to find an owner (worker) for this range + workerAssignments.insert(range, UID()); + if (BM_DEBUG) { + fmt::print(" [{0} - {1})\n", range.begin.printable(), range.end.printable()); + } + } + } + + if (!results.more) { + break; + } + + beginKey = results.readThrough.get(); + } catch (Error& e) { + wait(tr->onError(e)); + } + } + + if (BM_DEBUG) { + fmt::print("BM {0} final ranges:\n", bmData->epoch); + } + // Step 4. Send assign requests for all the granules and transfer assignments // from local workerAssignments to bmData for (auto& range : workerAssignments.intersectingRanges(normalKeys)) { if (!range.value().present()) { continue; } + if (BM_DEBUG) { + fmt::print(" [{0} - {1})\n", range.begin().printable(), range.end().printable()); + } + bmData->workerAssignments.insert(range.range(), range.value().get()); RangeAssignment raAssign; @@ -2109,12 +2189,13 @@ ACTOR Future monitorPruneKeys(BlobManagerData* self) { // wait for this set of prunes to complete before starting the next ones since if we prune // a range R at version V and while we are doing that, the time expires, we will end up - // trying to prune the same range again since the work isn't finished and the prunes will race + // trying to prune the same range again since the work isn't finished and the prunes will + // race // - // TODO: this isn't that efficient though. Instead we could keep metadata as part of the BM's - // memory that tracks which prunes are active. Once done, we can mark that work as done. If the - // BM fails then all prunes will fail and so the next BM will have a clear set of metadata (i.e. - // no work in progress) so we will end up doing the work in the new BM + // TODO: this isn't that efficient though. Instead we could keep metadata as part of the + // BM's memory that tracks which prunes are active. Once done, we can mark that work as + // done. If the BM fails then all prunes will fail and so the next BM will have a clear set + // of metadata (i.e. no work in progress) so we will end up doing the work in the new BM wait(waitForAll(prunes)); if (!pruneIntents.more) { From 9d9cb961a1edab9e5d81c7d39f5f0c7f157054d8 Mon Sep 17 00:00:00 2001 From: Josh Slocum Date: Tue, 18 Jan 2022 14:22:34 -0600 Subject: [PATCH 137/413] reworked blob manager recovery to be more efficient --- fdbclient/SystemData.cpp | 29 +++ fdbclient/SystemData.h | 7 + fdbserver/BlobManager.actor.cpp | 302 +++++++++++++++++++++----------- fdbserver/BlobWorker.actor.cpp | 1 + 4 files changed, 239 insertions(+), 100 deletions(-) diff --git a/fdbclient/SystemData.cpp b/fdbclient/SystemData.cpp index 32a0f5a24d..71c51a58b3 100644 --- a/fdbclient/SystemData.cpp +++ b/fdbclient/SystemData.cpp @@ -1129,6 +1129,7 @@ const KeyRangeRef blobGranuleFileKeys(LiteralStringRef("\xff\x02/bgf/"), Literal const KeyRangeRef blobGranuleMappingKeys(LiteralStringRef("\xff\x02/bgm/"), LiteralStringRef("\xff\x02/bgm0")); const KeyRangeRef blobGranuleLockKeys(LiteralStringRef("\xff\x02/bgl/"), LiteralStringRef("\xff\x02/bgl0")); const KeyRangeRef blobGranuleSplitKeys(LiteralStringRef("\xff\x02/bgs/"), LiteralStringRef("\xff\x02/bgs0")); +const KeyRangeRef blobGranuleSplitBoundaryKeys(LiteralStringRef("\xff\x02/bgsb/"), LiteralStringRef("\xff\x02/bgsb0")); const KeyRangeRef blobGranuleHistoryKeys(LiteralStringRef("\xff\x02/bgh/"), LiteralStringRef("\xff\x02/bgh0")); const KeyRangeRef blobGranulePruneKeys(LiteralStringRef("\xff\x02/bgp/"), LiteralStringRef("\xff\x02/bgp0")); const KeyRef blobGranulePruneChangeKey = LiteralStringRef("\xff\x02/bgpChange"); @@ -1283,6 +1284,34 @@ std::pair decodeBlobGranuleSplitValue(const Valu return std::pair(st, bigEndian64(v)); } +const Key blobGranuleSplitBoundaryKeyFor(UID const& parentGranuleID, KeyRef const& granuleStart) { + BinaryWriter wr(AssumeVersion(ProtocolVersion::withBlobGranule())); + wr.serializeBytes(blobGranuleSplitBoundaryKeys.begin); + wr << parentGranuleID; + wr << granuleStart; + return wr.toValue(); +} + +std::pair decodeBlobGranuleSplitBoundaryKey(KeyRef const& key) { + UID parentGranuleID; + Key granuleStart; + BinaryReader reader(key.removePrefix(blobGranuleSplitBoundaryKeys.begin), + AssumeVersion(ProtocolVersion::withBlobGranule())); + + reader >> parentGranuleID; + reader >> granuleStart; + return std::pair(parentGranuleID, granuleStart); +} + +const KeyRange blobGranuleSplitBoundaryKeyRangeFor(UID const& parentGranuleID) { + BinaryWriter wr(AssumeVersion(ProtocolVersion::withBlobGranule())); + wr.serializeBytes(blobGranuleSplitBoundaryKeys.begin); + wr << parentGranuleID; + + Key startKey = wr.toValue(); + return KeyRangeRef(startKey, strinc(startKey)); +} + const Key blobGranuleHistoryKeyFor(KeyRangeRef const& range, Version version) { BinaryWriter wr(AssumeVersion(ProtocolVersion::withBlobGranule())); wr.serializeBytes(blobGranuleHistoryKeys.begin); diff --git a/fdbclient/SystemData.h b/fdbclient/SystemData.h index 64cb9ea470..bd59eb5cae 100644 --- a/fdbclient/SystemData.h +++ b/fdbclient/SystemData.h @@ -556,6 +556,9 @@ extern const KeyRangeRef blobGranuleLockKeys; // \xff\x02/bgs/(parentGranuleUID, granuleUID) = [[BlobGranuleSplitState]] extern const KeyRangeRef blobGranuleSplitKeys; +// \xff\x02/bgsb/(parentGranuleID, granuleStartKey) = [] +extern const KeyRangeRef blobGranuleSplitBoundaryKeys; + // \xff\x02/bgh/(beginKey,endKey,startVersion) = { granuleUID, [parentGranuleHistoryKeys] } extern const KeyRangeRef blobGranuleHistoryKeys; @@ -589,6 +592,10 @@ const KeyRange blobGranuleSplitKeyRangeFor(UID const& parentGranuleID); const Value blobGranuleSplitValueFor(BlobGranuleSplitState st); std::pair decodeBlobGranuleSplitValue(ValueRef const& value); +const Key blobGranuleSplitBoundaryKeyFor(UID const& parentGranuleID, KeyRef const& granuleStart); +std::pair decodeBlobGranuleSplitBoundaryKey(KeyRef const& key); +const KeyRange blobGranuleSplitBoundaryKeyRangeFor(UID const& parentGranuleID); + const Key blobGranuleHistoryKeyFor(KeyRangeRef const& range, Version version); std::pair decodeBlobGranuleHistoryKey(KeyRef const& key); const KeyRange blobGranuleHistoryKeyRangeFor(KeyRangeRef const& range); diff --git a/fdbserver/BlobManager.actor.cpp b/fdbserver/BlobManager.actor.cpp index b925001e54..683a17afc2 100644 --- a/fdbserver/BlobManager.actor.cpp +++ b/fdbserver/BlobManager.actor.cpp @@ -832,6 +832,7 @@ ACTOR Future maybeSplitRange(BlobManagerData* bmData, UID newGranuleID = deterministicRandom()->randomUniqueID(); Key splitKey = blobGranuleSplitKeyFor(granuleID, newGranuleID); + tr->set(blobGranuleSplitBoundaryKeyFor(granuleID, newRanges[i]), Value()); tr->atomicOp(splitKey, blobGranuleSplitValueFor(BlobGranuleSplitState::Initialized), @@ -851,6 +852,7 @@ ACTOR Future maybeSplitRange(BlobManagerData* bmData, latestVersion);*/ tr->set(historyKey, blobGranuleHistoryValueFor(historyValue)); } + tr->set(blobGranuleSplitBoundaryKeyFor(granuleID, newRanges.back()), Value()); wait(tr->commit()); break; @@ -1185,15 +1187,13 @@ ACTOR Future recoverBlobManager(BlobManagerData* bmData) { // BM is recovering. Now the mapping at this time looks like G->deadBW. But the rangeAssigner handles this: // we'll try to assign a range to a dead worker and fail and reassign it to the next best worker. // - // 2. We get all granule history entries, to get a mapping from granule id to key range, for step 3. - // - // 3. We get the existing split intentions that were Started but not acknowledged by any blob workers and - // add them to our key range map, bmData->granuleAssignments. Note that we are adding them on top of - // the granule mappings and since we are using a key range map, we end up with the same set of shard - // boundaries as the old blob manager had. For these splits, we simply assign the range to the next - // best worker. This is not any worst than what the old blob manager would have done. - // Details: Note that this means that if a worker we intended to give a splitted range to dies - // before the new BM recovers, then we'll simply assign the range to the next best worker. + // 3. We get the existing split intentions and boundaries that were Started but not acknowledged by any blob workers + // and add them to our key range map, bmData->granuleAssignments. Note that we are adding them on top of the + // granule mappings and since we are using a key range map, we end up with the same set of shard boundaries as + // the old blob manager had. For these splits, we simply assign the range to the next best worker. This is not + // any worst than what the old blob manager would have done. Details: Note that this means that if a worker we + // intended to give a splitted range to dies before the new BM recovers, then we'll simply assign the range to + // the next best worker. // // 4. For every range in our granuleAssignments, we send an assign request to the stream of requests, // ultimately giving every range back to some worker (trying to mimic the state of the old BM). @@ -1208,16 +1208,17 @@ ACTOR Future recoverBlobManager(BlobManagerData* bmData) { state int rowLimit = BUGGIFY ? deterministicRandom()->randomInt(2, 10) : 10000; if (BM_DEBUG) { - printf("BM %lld recovering:\n", bmData->epoch); - printf("BM %lld found old assignments:\n", bmData->epoch); + fmt::print("BM {0} recovering:\n", bmData->epoch); + fmt::print("BM {0} found old assignments:\n", bmData->epoch); } - // Step 1. Get the latest known mapping of granules to blob workers (i.e. assignments) + + // TODO could populate most/all of this list by just asking existing blob workers for their range sets to reduce DB + // read load on BM restart Step 1. Get the latest known mapping of granules to blob workers (i.e. assignments) state KeyRef beginKey = normalKeys.begin; loop { try { tr->setOption(FDBTransactionOptions::ACCESS_SYSTEM_KEYS); tr->setOption(FDBTransactionOptions::PRIORITY_SYSTEM_IMMEDIATE); - wait(checkManagerLock(tr, bmData)); // TODO: replace row limit with knob KeyRange nextRange(KeyRangeRef(beginKey, normalKeys.end)); @@ -1256,98 +1257,197 @@ ACTOR Future recoverBlobManager(BlobManagerData* bmData) { beginKey = lastEndKey; } catch (Error& e) { + if (BM_DEBUG) { + fmt::print("BM {0} got error reading granule mapping during recovery: {1}\n", bmData->epoch, e.name()); + } wait(tr->onError(e)); } } - // TODO could avoid if no splits in progress - // Step 2. Read all history entries, so we can know the range of each sub-granule that is splitting + // TODO use range stream instead + + state UID currentParentID = UID(); + state Optional nextParentID; + state std::vector splitBoundaries; + state std::vector> splitStates; + + state Key splitBeginKey = blobGranuleSplitKeys.begin; + state RangeResult splitResult; + splitResult.readThrough = splitBeginKey; + splitResult.more = true; + state int splitResultIdx = 0; + + state Key boundaryBeginKey = blobGranuleSplitBoundaryKeys.begin; + state RangeResult boundaryResult; + boundaryResult.readThrough = boundaryBeginKey; + boundaryResult.more = true; + state int boundaryResultIdx = 0; + + // Step 3. Get the latest known split intentions and boundaries tr->reset(); - beginKey = blobGranuleHistoryKeys.begin; + tr->setOption(FDBTransactionOptions::ACCESS_SYSTEM_KEYS); + tr->setOption(FDBTransactionOptions::PRIORITY_SYSTEM_IMMEDIATE); + if (BM_DEBUG) { - printf("BM %lld found history entries:\n", bmData->epoch); + fmt::print("BM {0} found in progress splits:\n", bmData->epoch); } loop { - try { - tr->setOption(FDBTransactionOptions::ACCESS_SYSTEM_KEYS); - tr->setOption(FDBTransactionOptions::PRIORITY_SYSTEM_IMMEDIATE); - wait(checkManagerLock(tr, bmData)); + // Advance both split and boundary readers until we hit another granule or EOS, to get the full state for one + // granule split. Effectively a stream merge. - RangeResult results = wait(tr->getRange(KeyRangeRef(beginKey, blobGranuleHistoryKeys.end), rowLimit)); - - // Add the granules for the started split intentions to the in-memory key range map - for (auto history : results) { - KeyRange granuleRange; - Version version; - std::tie(granuleRange, version) = decodeBlobGranuleHistoryKey(history.key); - Standalone v = decodeBlobGranuleHistoryValue(history.value); - granuleIdToRange[v.granuleID] = granuleRange; - if (BM_DEBUG) { - fmt::print(" {0}=[{1} - {2})\n", - v.granuleID, - granuleRange.begin.printable(), - granuleRange.end.printable()); + // Advance split reader + loop { + if (splitResultIdx >= splitResult.size()) { + if (!splitResult.more) { + break; } - } - - if (!results.more) { - break; - } - - beginKey = results.readThrough.get(); - } catch (Error& e) { - wait(tr->onError(e)); - } - } - - // Step 3. Get the latest known split intentions - tr->reset(); - beginKey = blobGranuleSplitKeys.begin; - if (BM_DEBUG) { - printf("BM %lld found in progress splits:\n", bmData->epoch); - } - loop { - try { - tr->setOption(FDBTransactionOptions::ACCESS_SYSTEM_KEYS); - tr->setOption(FDBTransactionOptions::PRIORITY_SYSTEM_IMMEDIATE); - wait(checkManagerLock(tr, bmData)); - - // TODO: replace row limit with knob - RangeResult results = wait(tr->getRange(KeyRangeRef(beginKey, blobGranuleSplitKeys.end), rowLimit)); - - // Add the granules for the started split intentions to the in-memory key range map - for (auto split : results) { - UID parentGranuleID, granuleID; - BlobGranuleSplitState splitState; - Version version; - - std::tie(parentGranuleID, granuleID) = decodeBlobGranuleSplitKey(split.key); - if (split.value.size() == 0) { - printf("No value for %s/%s split??\n", - parentGranuleID.toString().c_str(), - granuleID.toString().c_str()); - ASSERT(split.value.size() > 0); - } - std::tie(splitState, version) = decodeBlobGranuleSplitValue(split.value); - - // TODO THIS RANGE IS WRONG - ASSERT(granuleIdToRange.count(granuleID) == 1); - const KeyRange range = granuleIdToRange[granuleID]; - if (splitState <= BlobGranuleSplitState::Initialized) { - // the empty UID signifies that we need to find an owner (worker) for this range - workerAssignments.insert(range, UID()); - if (BM_DEBUG) { - fmt::print(" [{0} - {1})\n", range.begin.printable(), range.end.printable()); + ASSERT(splitResult.readThrough.present()); + splitBeginKey = splitResult.readThrough.get(); + loop { + try { + RangeResult r = + wait(tr->getRange(KeyRangeRef(splitBeginKey, blobGranuleSplitKeys.end), rowLimit)); + ASSERT(r.size() > 0 || !r.more); + splitResult = r; + splitResultIdx = 0; + break; + } catch (Error& e) { + if (BM_DEBUG) { + fmt::print("BM {0} got error advancing split cursor: {1}\n", bmData->epoch, e.name()); + } + wait(tr->onError(e)); + tr->setOption(FDBTransactionOptions::ACCESS_SYSTEM_KEYS); + tr->setOption(FDBTransactionOptions::PRIORITY_SYSTEM_IMMEDIATE); } } - } - - if (!results.more) { + // if we got a response and there are zero rows, we are done + if (splitResult.empty()) { + ASSERT(!splitResult.more); + break; + } + } else { break; } + while (splitResultIdx < splitResult.size()) { + UID parentGranuleID, granuleID; - beginKey = results.readThrough.get(); + std::tie(parentGranuleID, granuleID) = decodeBlobGranuleSplitKey(splitResult[splitResultIdx].key); + if (parentGranuleID != currentParentID) { + nextParentID = parentGranuleID; + break; + } + BlobGranuleSplitState splitState; + Version version; + std::tie(splitState, version) = decodeBlobGranuleSplitValue(splitResult[splitResultIdx].value); + splitStates.push_back(std::pair(granuleID, splitState)); + splitResultIdx++; + } + } + + // Advance boundary reader + loop { + if (boundaryResultIdx >= boundaryResult.size()) { + if (!boundaryResult.more) { + break; + } + ASSERT(boundaryResult.readThrough.present()); + boundaryBeginKey = boundaryResult.readThrough.get(); + loop { + try { + RangeResult r = wait( + tr->getRange(KeyRangeRef(boundaryBeginKey, blobGranuleSplitBoundaryKeys.end), rowLimit)); + ASSERT(r.size() > 0 || !r.more); + boundaryResult = r; + boundaryResultIdx = 0; + break; + } catch (Error& e) { + if (BM_DEBUG) { + fmt::print("BM {0} got error advancing boundary cursor: {1}\n", bmData->epoch, e.name()); + } + wait(tr->onError(e)); + tr->setOption(FDBTransactionOptions::ACCESS_SYSTEM_KEYS); + tr->setOption(FDBTransactionOptions::PRIORITY_SYSTEM_IMMEDIATE); + } + } + // if we got a response and there are zero rows, we are done + if (boundaryResult.empty()) { + break; + } + } else { + break; + } + while (boundaryResultIdx < boundaryResult.size()) { + UID parentGranuleID; + Key boundaryKey; + std::tie(parentGranuleID, boundaryKey) = + decodeBlobGranuleSplitBoundaryKey(boundaryResult[boundaryResultIdx].key); + if (parentGranuleID != currentParentID) { + // nextParentID should have already been set by split reader + ASSERT(nextParentID.present()); + ASSERT(nextParentID.get() == parentGranuleID); + break; + } + splitBoundaries.push_back(boundaryKey); + boundaryResultIdx++; + } + } + + // process this split + if (currentParentID != UID()) { + if (BM_DEBUG) { + fmt::print(" [{0} - {1}) {2}:\n", + splitBoundaries.front().printable(), + splitBoundaries.back().printable(), + currentParentID.toString().substr(0, 6)); + } + ASSERT(splitBoundaries.size() - 1 == splitStates.size()); + for (int i = 0; i < splitStates.size(); i++) { + // if this split boundary had not been opened by a blob worker before the last manager crashed, we must + // ensure it gets assigned to one + KeyRange range = KeyRange(KeyRangeRef(splitBoundaries[i], splitBoundaries[i + 1])); + + if (BM_DEBUG) { + printf(" "); + } + if (splitStates[i].second <= BlobGranuleSplitState::Initialized) { + // the empty UID signifies that we need to find an owner (worker) for this range + if (BM_DEBUG) { + printf("*** "); + } + workerAssignments.insert(range, UID()); + } + if (BM_DEBUG) { + fmt::print("[{0} - {1}) {2}\n", + range.begin.printable(), + range.end.printable(), + splitStates[i].first.toString().substr(0, 6)); + } + } + } + splitBoundaries.clear(); + splitStates.clear(); + + if (!nextParentID.present()) { + break; + } + currentParentID = nextParentID.get(); + nextParentID.reset(); + } + + // Step 4. Send assign requests for all the granules and transfer assignments + // from local workerAssignments to bmData + // before we take ownership of all of the ranges, check the manager lock again + tr->reset(); + loop { + try { + tr->setOption(FDBTransactionOptions::ACCESS_SYSTEM_KEYS); + tr->setOption(FDBTransactionOptions::PRIORITY_SYSTEM_IMMEDIATE); + wait(checkManagerLock(tr, bmData)); + break; } catch (Error& e) { + if (BM_DEBUG) { + fmt::print("BM {0} got error checking lock after recovery: {1}\n", bmData->epoch, e.name()); + } wait(tr->onError(e)); } } @@ -1355,8 +1455,7 @@ ACTOR Future recoverBlobManager(BlobManagerData* bmData) { if (BM_DEBUG) { fmt::print("BM {0} final ranges:\n", bmData->epoch); } - // Step 4. Send assign requests for all the granules and transfer assignments - // from local workerAssignments to bmData + for (auto& range : workerAssignments.intersectingRanges(normalKeys)) { if (!range.value().present()) { continue; @@ -1619,6 +1718,7 @@ ACTOR Future loadHistoryFiles(BlobManagerData* bmData, UID granule /* * Deletes all files pertaining to the granule with id granuleId and * also removes the history entry for this granule from the system keyspace + * TODO ensure cannot fully delete granule that is still splitting! */ ACTOR Future fullyDeleteGranule(BlobManagerData* self, UID granuleId, KeyRef historyKey) { if (BM_DEBUG) { @@ -1849,7 +1949,8 @@ ACTOR Future pruneRange(BlobManagerData* self, KeyRef startKey, KeyRef end activeRange.end().printable()); } Optional history = wait(getLatestGranuleHistory(&tr, activeRange.range())); - // TODO: can we tell from the krm that this range is not valid, so that we don't need to do a get + // TODO: can we tell from the krm that this range is not valid, so that we don't need to do a + // get if (history.present()) { if (BM_DEBUG) { printf("Adding range to history queue\n"); @@ -2187,15 +2288,16 @@ ACTOR Future monitorPruneKeys(BlobManagerData* self) { prunes.emplace_back(pruneRange(self, rangeStartKey, rangeEndKey, pruneVersion, force)); } - // wait for this set of prunes to complete before starting the next ones since if we prune - // a range R at version V and while we are doing that, the time expires, we will end up - // trying to prune the same range again since the work isn't finished and the prunes will - // race + // wait for this set of prunes to complete before starting the next ones since if we + // prune a range R at version V and while we are doing that, the time expires, we will + // end up trying to prune the same range again since the work isn't finished and the + // prunes will race // // TODO: this isn't that efficient though. Instead we could keep metadata as part of the // BM's memory that tracks which prunes are active. Once done, we can mark that work as - // done. If the BM fails then all prunes will fail and so the next BM will have a clear set - // of metadata (i.e. no work in progress) so we will end up doing the work in the new BM + // done. If the BM fails then all prunes will fail and so the next BM will have a clear + // set of metadata (i.e. no work in progress) so we will end up doing the work in the + // new BM wait(waitForAll(prunes)); if (!pruneIntents.more) { @@ -2304,8 +2406,8 @@ ACTOR Future blobManager(BlobManagerInterface bmInterf, // DB has [B - D). It should show up coalesced in knownBlobRanges, and [C - D) should be removed. // DB has [A - D). It should show up coalesced in knownBlobRanges, and [A - B) should be removed. // DB has [A - B) and [C - D). They should show up in knownBlobRanges, and [B - C) should be in removed. -// DB has [B - C). It should show up in knownBlobRanges, [B - C) should be in added, and [A - B) and [C - D) should -// be in removed. +// DB has [B - C). It should show up in knownBlobRanges, [B - C) should be in added, and [A - B) and [C - D) +// should be in removed. TEST_CASE(":/blobmanager/updateranges") { KeyRangeMap knownBlobRanges(false, normalKeys.end); Arena ar; diff --git a/fdbserver/BlobWorker.actor.cpp b/fdbserver/BlobWorker.actor.cpp index 6eff98af51..152402a26f 100644 --- a/fdbserver/BlobWorker.actor.cpp +++ b/fdbserver/BlobWorker.actor.cpp @@ -376,6 +376,7 @@ ACTOR Future updateGranuleSplitState(Transaction* tr, Key oldGranuleLockKey = blobGranuleLockKeyFor(parentGranuleRange); tr->clear(singleKeyRange(oldGranuleLockKey)); tr->clear(currentRange); + tr->clear(blobGranuleSplitBoundaryKeyRangeFor(parentGranuleID)); } else { tr->atomicOp(myStateKey, blobGranuleSplitValueFor(newState), MutationRef::SetVersionstampedValue); if (newState == BlobGranuleSplitState::Assigned && currentState == BlobGranuleSplitState::Initialized && From 7ad085ce7360ed368a11f1f26b2d8f6737fd8669 Mon Sep 17 00:00:00 2001 From: Josh Slocum Date: Sat, 15 Jan 2022 10:05:43 -0600 Subject: [PATCH 138/413] Fixed more manager assignment issues --- fdbserver/BlobManager.actor.cpp | 85 +++++++++++++++++++++++++++++---- 1 file changed, 75 insertions(+), 10 deletions(-) diff --git a/fdbserver/BlobManager.actor.cpp b/fdbserver/BlobManager.actor.cpp index 683a17afc2..15c6c0572f 100644 --- a/fdbserver/BlobManager.actor.cpp +++ b/fdbserver/BlobManager.actor.cpp @@ -204,6 +204,7 @@ struct BlobManagerData { Database db; Optional dcId; PromiseStream> addActor; + Promise doLockCheck; Reference bstore; @@ -212,6 +213,7 @@ struct BlobManagerData { std::unordered_set workerAddresses; std::unordered_set deadWorkers; KeyRangeMap workerAssignments; + KeyRangeActorMap assignsInProgress; KeyRangeMap knownBlobRanges; AsyncTrigger startRecruiting; @@ -231,7 +233,13 @@ struct BlobManagerData { BlobManagerData(UID id, Database db, Optional dcId) : id(id), db(db), dcId(dcId), knownBlobRanges(false, normalKeys.end), restartRecruiting(SERVER_KNOBS->DEBOUNCE_RECRUITING_DELAY), recruitingStream(0) {} - ~BlobManagerData() { fmt::print("Destroying blob manager data for {}\n", id.toString()); } + + // TODO REMOVE + ~BlobManagerData() { + if (BM_DEBUG) { + fmt::print("Destroying blob manager data for {0} {1}\n", epoch, id.toString()); + } + } }; ACTOR Future>> splitRange(Reference tr, KeyRange range) { @@ -284,7 +292,7 @@ ACTOR Future pickWorkerForAssign(BlobManagerData* bmData) { while (bmData->workerStats.size() == 0) { // TODO REMOVE if (BM_DEBUG) { - printf("BM waiting for blob workers before assigning granules\n"); + fmt::print("BM {0} waiting for blob workers before assigning granules\n", bmData->epoch); } bmData->restartRecruiting.trigger(); wait(bmData->recruitingStream.onChange()); @@ -381,20 +389,39 @@ ACTOR Future doRangeAssignment(BlobManagerData* bmData, RangeAssignment as if (e.code() == error_code_operation_cancelled) { throw; } - // If the worker is no longer present, the ranges were already moved off by that function, so don't retry. - // If the request is a reassign though, we need to retry it since the worker it was on is now dead and nobody - // owns it - if (!bmData->workersById.count(workerID) && - (!assignment.assign.present() || assignment.assign.get().type != AssignRequestType::Reassign)) { + if (e.code() == error_code_granule_assignment_conflict) { + // Another blob worker already owns the range, don't retry. + // And, if it was us that send the request to another worker for this range, this actor should have been + // cancelled. So if it wasn't, it's likely that the conflict is from a new blob manager. Trigger the lock + // check to make sure, and die if so. if (BM_DEBUG) { - fmt::print("BM {0} got error assigning range [{1} - {2}) to now dead worker {3}, ignoring\n", + fmt::print("BM {0} got conflict assigning [{1} - {2}) to worker {3}, ignoring\n", bmData->epoch, assignment.keyRange.begin.printable(), assignment.keyRange.end.printable(), workerID.toString()); } + if (bmData->doLockCheck.canBeSet()) { + bmData->doLockCheck.send(Void()); + } return Void(); } + + // TODO: i think this is no longer necessary with the cancelling and actor map, but keep it around just in case + // for a bit If the worker is no longer present, the ranges were already moved off by that function, so don't + // retry. If the request is a reassign though, we need to retry it since the worker it was on is now dead and + // nobody owns it + /*if (!bmData->workersById.count(workerID) && + (!assignment.assign.present() || assignment.assign.get().type != AssignRequestType::Reassign)) { + if (BM_DEBUG) { + fmt::print("BM {0} got error assigning range [{1} - {2}) to now dead worker {3}, ignoring\n", + bmData->epoch, + assignment.keyRange.begin.printable(), + assignment.keyRange.end.printable(), + workerID.toString()); + } + return Void(); + }*/ // TODO confirm: using reliable delivery this should only trigger if the worker is marked as failed, right? // So assignment needs to be retried elsewhere, and a revoke is trivially complete if (assignment.isAssign) { @@ -494,7 +521,8 @@ ACTOR Future rangeAssigner(BlobManagerData* bmData) { // FIXME: if range is assign, have some sort of semaphore for outstanding assignments so we don't assign // a ton ranges at once and blow up FDB with reading initial snapshots. - bmData->addActor.send(doRangeAssignment(bmData, assignment, workerId, seqNo)); + bmData->assignsInProgress.insert(assignment.keyRange, + doRangeAssignment(bmData, assignment, workerId, seqNo)); } else { // Revoking a range could be a large range that contains multiple ranges. auto currentAssignments = bmData->workerAssignments.intersectingRanges(assignment.keyRange); @@ -510,6 +538,8 @@ ACTOR Future rangeAssigner(BlobManagerData* bmData) { bmData->workerStats[it.value()].numGranulesAssigned -= 1; } + bmData->assignsInProgress.cancel(assignment.keyRange); + // revoke the range for the worker that owns it, not the worker specified in the revoke bmData->addActor.send(doRangeAssignment(bmData, assignment, it.value(), seqNo)); } @@ -2331,6 +2361,40 @@ ACTOR Future monitorPruneKeys(BlobManagerData* self) { } } +ACTOR Future doLockChecks(BlobManagerData* bmData) { + loop { + Promise check = bmData->doLockCheck; + wait(check.getFuture()); + wait(delay(0.5)); // don't do this too often if a lot of conflict + + state Reference tr = makeReference(bmData->db); + + loop { + try { + tr->setOption(FDBTransactionOptions::ACCESS_SYSTEM_KEYS); + tr->setOption(FDBTransactionOptions::PRIORITY_SYSTEM_IMMEDIATE); + wait(checkManagerLock(tr, bmData)); + break; + } catch (Error& e) { + if (e.code() == error_code_granule_assignment_conflict) { + if (BM_DEBUG) { + fmt::print("BM {0} got lock out of date in lock check on conflict! Dying\n", bmData->epoch); + } + if (bmData->iAmReplaced.canBeSet()) { + bmData->iAmReplaced.send(Void()); + } + return Void(); + } + wait(tr->onError(e)); + if (BM_DEBUG) { + fmt::print("BM {0} still ok after checking lock on conflict\n", bmData->epoch); + } + } + } + bmData->doLockCheck = Promise(); + } +} + ACTOR Future blobManager(BlobManagerInterface bmInterf, Reference const> dbInfo, int64_t epoch) { @@ -2341,7 +2405,7 @@ ACTOR Future blobManager(BlobManagerInterface bmInterf, state Future collection = actorCollection(self.addActor.getFuture()); if (BM_DEBUG) { - printf("Blob manager starting...\n"); + fmt::print("Blob manager {0} starting...\n", epoch); } self.epoch = epoch; @@ -2355,6 +2419,7 @@ ACTOR Future blobManager(BlobManagerInterface bmInterf, // before the new blob manager does anything wait(recoverBlobManager(&self)); + self.addActor.send(doLockChecks(&self)); self.addActor.send(monitorClientRanges(&self)); self.addActor.send(rangeAssigner(&self)); self.addActor.send(monitorPruneKeys(&self)); From 08f8700636a567a2245fc13d9a5f973cfe829502 Mon Sep 17 00:00:00 2001 From: Josh Slocum Date: Tue, 18 Jan 2022 18:47:42 -0600 Subject: [PATCH 139/413] Fixed a couple bugs in blob manager recovery --- fdbclient/SystemData.cpp | 2 +- fdbserver/BlobManager.actor.cpp | 24 +++++++++++++++++++----- 2 files changed, 20 insertions(+), 6 deletions(-) diff --git a/fdbclient/SystemData.cpp b/fdbclient/SystemData.cpp index 71c51a58b3..c965bf9a18 100644 --- a/fdbclient/SystemData.cpp +++ b/fdbclient/SystemData.cpp @@ -1294,7 +1294,7 @@ const Key blobGranuleSplitBoundaryKeyFor(UID const& parentGranuleID, KeyRef cons std::pair decodeBlobGranuleSplitBoundaryKey(KeyRef const& key) { UID parentGranuleID; - Key granuleStart; + KeyRef granuleStart; BinaryReader reader(key.removePrefix(blobGranuleSplitBoundaryKeys.begin), AssumeVersion(ProtocolVersion::withBlobGranule())); diff --git a/fdbserver/BlobManager.actor.cpp b/fdbserver/BlobManager.actor.cpp index 15c6c0572f..5824e35283 100644 --- a/fdbserver/BlobManager.actor.cpp +++ b/fdbserver/BlobManager.actor.cpp @@ -1355,23 +1355,27 @@ ACTOR Future recoverBlobManager(BlobManagerData* bmData) { ASSERT(!splitResult.more); break; } - } else { - break; } + bool foundNext = false; while (splitResultIdx < splitResult.size()) { UID parentGranuleID, granuleID; std::tie(parentGranuleID, granuleID) = decodeBlobGranuleSplitKey(splitResult[splitResultIdx].key); if (parentGranuleID != currentParentID) { nextParentID = parentGranuleID; + foundNext = true; break; } + BlobGranuleSplitState splitState; Version version; std::tie(splitState, version) = decodeBlobGranuleSplitValue(splitResult[splitResultIdx].value); splitStates.push_back(std::pair(granuleID, splitState)); splitResultIdx++; } + if (foundNext) { + break; + } } // Advance boundary reader @@ -1403,34 +1407,44 @@ ACTOR Future recoverBlobManager(BlobManagerData* bmData) { if (boundaryResult.empty()) { break; } - } else { - break; } + bool foundNext = false; while (boundaryResultIdx < boundaryResult.size()) { UID parentGranuleID; Key boundaryKey; + std::tie(parentGranuleID, boundaryKey) = decodeBlobGranuleSplitBoundaryKey(boundaryResult[boundaryResultIdx].key); + if (parentGranuleID != currentParentID) { // nextParentID should have already been set by split reader ASSERT(nextParentID.present()); ASSERT(nextParentID.get() == parentGranuleID); + foundNext = true; break; } + splitBoundaries.push_back(boundaryKey); boundaryResultIdx++; } + if (foundNext) { + break; + } } // process this split if (currentParentID != UID()) { + ASSERT(splitStates.size() > 0); + ASSERT(splitBoundaries.size() - 1 == splitStates.size()); + + std::sort(splitBoundaries.begin(), splitBoundaries.end()); + if (BM_DEBUG) { fmt::print(" [{0} - {1}) {2}:\n", splitBoundaries.front().printable(), splitBoundaries.back().printable(), currentParentID.toString().substr(0, 6)); } - ASSERT(splitBoundaries.size() - 1 == splitStates.size()); for (int i = 0; i < splitStates.size(); i++) { // if this split boundary had not been opened by a blob worker before the last manager crashed, we must // ensure it gets assigned to one From 79704aa05fc400a7d47befe79045a39b729afb86 Mon Sep 17 00:00:00 2001 From: Josh Slocum Date: Tue, 18 Jan 2022 20:17:28 -0600 Subject: [PATCH 140/413] Fixing change feed private mutation handling --- fdbclient/NativeAPI.actor.cpp | 56 +++++++++++++++++++++++++------ fdbserver/storageserver.actor.cpp | 13 ++++++- 2 files changed, 57 insertions(+), 12 deletions(-) diff --git a/fdbclient/NativeAPI.actor.cpp b/fdbclient/NativeAPI.actor.cpp index 8caad964ca..6377a09d90 100644 --- a/fdbclient/NativeAPI.actor.cpp +++ b/fdbclient/NativeAPI.actor.cpp @@ -7160,6 +7160,10 @@ Version ChangeFeedData::getVersion() { #define DEBUG_CF_VERSION(cfId, v) \ DEBUG_CF_START_VERSION <= v&& v <= DEBUG_CF_END_VERSION && (""_sr == DEBUG_CF_ID || cfId.printable() == DEBUG_CF_ID) +#define DEBUG_CF_VERSION_RANGE(cfId, vStart, vEnd) \ + DEBUG_CF_START_VERSION <= vEnd&& vStart <= DEBUG_CF_END_VERSION && \ + (""_sr == DEBUG_CF_ID || cfId.printable() == DEBUG_CF_ID) + #define DEBUG_CF_WAIT(cfId, v) DEBUG_CF_WAIT_VERSION == v && (""_sr == DEBUG_CF_ID || cfId.printable() == DEBUG_CF_ID) // This function is essentially bubbling the information about what has been processed from the server through the @@ -7314,7 +7318,8 @@ ACTOR Future partialChangeFeedStream(StorageServerInterface interf, Version end, Reference feedData, Reference storageData, - int idx /* TODO REMOVE this param after correctness clean */) { + int idx /* TODO REMOVE this param after correctness clean */, + KeyRange range /* TODO REMOVE this param after correctness clean */) { // calling lastReturnedVersion's callbacks could cause us to be cancelled state Promise refresh = feedData->refresh; @@ -7323,12 +7328,20 @@ ACTOR Future partialChangeFeedStream(StorageServerInterface interf, try { loop { if (nextVersion >= end) { + if (DEBUG_CF_VERSION(feedData->id, end)) { + fmt::print(" single {0} {1} [{2} - {3}): sending EOS\n", + idx, + interf.id().toString().substr(0, 4), + range.begin.printable(), + range.end.printable()); + } results.sendError(end_of_stream()); return Void(); } choose { when(state ChangeFeedStreamReply rep = waitNext(replyStream.getFuture())) { - if (DEBUG_CF_VERSION(feedData->id, rep.mutations.back().version)) { + if (DEBUG_CF_VERSION_RANGE( + feedData->id, rep.mutations.front().version, rep.mutations.back().version)) { fmt::print(" single {0} {1}: response {2} - {3} ({4}), atLatest={5}, rep.atLatest={6}, " "notAtLatest={7}, " "minSV={8}\n", @@ -7343,6 +7356,12 @@ ACTOR Future partialChangeFeedStream(StorageServerInterface interf, rep.minStreamVersion); } + // TODO REMOVE, just for debugging + // set next version so debug statements trigger + if (nextVersion == 0) { + nextVersion = rep.mutations.front().version; + } + if (rep.mutations.back().version > feedData->maxSeenVersion) { feedData->maxSeenVersion = rep.mutations.back().version; } @@ -7351,9 +7370,11 @@ ACTOR Future partialChangeFeedStream(StorageServerInterface interf, while (resultLoc < rep.mutations.size()) { wait(results.onEmpty()); if (DEBUG_CF_VERSION(feedData->id, rep.mutations[resultLoc].version)) { - fmt::print(" single {0} {1}: sending {2}/{3} {4} ({5})\n", + fmt::print(" single {0} {1} [{2} - {3}): sending {4}/{5} {6} ({7})\n", idx, interf.id().toString().substr(0, 4), + range.begin.printable(), + range.end.printable(), resultLoc, rep.mutations.size(), rep.mutations[resultLoc].version, @@ -7423,7 +7444,14 @@ ACTOR Future partialChangeFeedStream(StorageServerInterface interf, } } catch (Error& e) { // TODO REMOVE eventually, useful for debugging for now - fmt::print("NAS: CFError {}\n", e.name()); + if (DEBUG_CF_VERSION(feedData->id, nextVersion)) { + fmt::print(" single {0} {1} [{2} - {3}): CFError {4}\n", + idx, + interf.id().toString().substr(0, 4), + range.begin.printable(), + range.end.printable(), + e.name()); + } if (e.code() == error_code_actor_cancelled) { throw; } @@ -7451,10 +7479,6 @@ ACTOR Future doCFMerge(Reference results, wait(delay(0)); ASSERT(results->mutations.isEmpty()); - // push one initial mutation from each stream - - // Without this delay, weird issues with the last stream getting on another stream's callstack can happen - wait(delay(0)); state int interfNum = 0; // TODO minor optimization - could make this just a vector of indexes if each MutationAndVersionStream remembered @@ -7465,6 +7489,7 @@ ACTOR Future doCFMerge(Reference results, streamsUsed.push_back(stream); } + state Version nextVersion; loop { // bring all of the streams up to date to ensure we have the latest element from each stream in mutations interfNum = 0; @@ -7495,7 +7520,7 @@ ACTOR Future doCFMerge(Reference results, // pop first item off queue - this will be mutation with the lowest version Standalone> nextOut; - state Version nextVersion = nextVersion = mutations.top().next.version; + nextVersion = mutations.top().next.version; streamsUsed.push_back(mutations.top()); nextOut.push_back_deep(nextOut.arena(), mutations.top().next); @@ -7593,10 +7618,19 @@ ACTOR Future mergeChangeFeedStream(Reference db, results->notAtLatest.set(interfs.size()); refresh.send(Void()); + if (DEBUG_CF_START_VERSION != invalidVersion) { + fmt::print("Starting merge cursor for {0} @ {1} - {2}\n", interfs.size(), *begin, end); + } for (int i = 0; i < interfs.size(); i++) { onErrors[i] = results->streams[i].onError(); - fetchers[i] = partialChangeFeedStream( - interfs[i].first, streams[i].results, results->streams[i], end, results, results->storageData[i], i); + fetchers[i] = partialChangeFeedStream(interfs[i].first, + streams[i].results, + results->streams[i], + end, + results, + results->storageData[i], + i, + interfs[i].second); } wait(onCFErrors(onErrors) || doCFMerge(results, interfs, streams, begin, end)); diff --git a/fdbserver/storageserver.actor.cpp b/fdbserver/storageserver.actor.cpp index 53c8d2c70d..f4692c89ad 100644 --- a/fdbserver/storageserver.actor.cpp +++ b/fdbserver/storageserver.actor.cpp @@ -4023,6 +4023,7 @@ ACTOR Future fetchChangeFeed(StorageServer* data, TraceEvent(SevDebug, "FetchChangeFeed", data->thisServerID) .detail("RangeID", rangeId.printable()) .detail("Range", range.toString()) + .detail("FetchVersion", fetchVersion) .detail("Existing", existing); if (!existing) { @@ -4068,6 +4069,7 @@ ACTOR Future dispatchChangeFeeds(StorageServer* data, UID fetchKeysID, Key try { state std::vector feeds = wait(data->cx->getOverlappingChangeFeeds(keys, fetchVersion + 1)); + // TODO add trace events for some of these for (auto& feed : feeds) { feedFetches[feed.rangeId] = fetchChangeFeed(data, feed.rangeId, feed.range, feed.stopped, fetchVersion); } @@ -4492,6 +4494,10 @@ void AddingShard::addMutation(Version version, bool fromFetch, MutationRef const } it->mutations.back().mutations.push_back_deep(it->mutations.back().arena(), mutation); server->currentChangeFeeds.insert(it->id); + DEBUG_MUTATION("ChangeFeedWriteSet", version, mutation, server->thisServerID) + .detail("Range", it->range) + .detail("ChangeFeedID", it->id) + .detail("Source", "Adding"); } } } else if (mutation.type == MutationRef::ClearRange) { @@ -4504,6 +4510,10 @@ void AddingShard::addMutation(Version version, bool fromFetch, MutationRef const } it->mutations.back().mutations.push_back_deep(it->mutations.back().arena(), mutation); server->currentChangeFeeds.insert(it->id); + DEBUG_MUTATION("ChangeFeedWriteClear", version, mutation, server->thisServerID) + .detail("Range", it->range) + .detail("ChangeFeedID", it->id) + .detail("Source", "Adding"); } } } @@ -4972,7 +4982,8 @@ private: } data->keyChangeFeed.coalesce(feed->second->range.contents()); data->uidChangeFeed.erase(feed); - } else { + } else if (status != ChangeFeedStatus::CHANGE_FEED_CREATE) { + // Can be a change feed create from move, ignore // must be pop or stop if (status == ChangeFeedStatus::CHANGE_FEED_STOP) { TraceEvent(SevDebug, "StoppingChangeFeed", data->thisServerID) From f914b0860b6b4ff05b76f47091bbb31d593e1c83 Mon Sep 17 00:00:00 2001 From: Josh Slocum Date: Wed, 19 Jan 2022 09:11:55 -0600 Subject: [PATCH 141/413] Fixed race in change feed initialization --- fdbserver/BlobWorker.actor.cpp | 56 ++++++++++++++++------------------ 1 file changed, 27 insertions(+), 29 deletions(-) diff --git a/fdbserver/BlobWorker.actor.cpp b/fdbserver/BlobWorker.actor.cpp index 152402a26f..27045eca61 100644 --- a/fdbserver/BlobWorker.actor.cpp +++ b/fdbserver/BlobWorker.actor.cpp @@ -947,14 +947,6 @@ struct InFlightFile { : future(future), version(version), bytes(bytes), snapshot(snapshot) {} }; -static Reference newChangeFeedData(Version startVersion) { - // FIXME: should changeFeedStream guarantee that this is always set to begin-1 instead? - Reference r = makeReference(); - // TODO uncomment? - // r->lastReturnedVersion.set(startVersion); - return r; -} - // TODO REMOVE once correctness clean #define DEBUG_BW_START_VERSION invalidVersion #define DEBUG_BW_END_VERSION invalidVersion @@ -1257,23 +1249,25 @@ ACTOR Future blobGranuleUpdateFiles(Reference bwData, metadata->pendingDeltaVersion = startVersion; metadata->bufferedDeltaVersion = startVersion; - metadata->activeCFData.set(newChangeFeedData(startVersion)); + Reference newCFData = makeReference(); + if (startState.parentGranule.present() && startVersion < startState.changeFeedStartVersion) { // read from parent change feed up until our new change feed is started readOldChangeFeed = true; - oldChangeFeedFuture = bwData->db->getChangeFeedStream(metadata->activeCFData.get(), - oldCFKey.get(), - startVersion + 1, - startState.changeFeedStartVersion, - metadata->keyRange); + oldChangeFeedFuture = bwData->db->getChangeFeedStream( + newCFData, oldCFKey.get(), startVersion + 1, startState.changeFeedStartVersion, metadata->keyRange); } else { readOldChangeFeed = false; - changeFeedFuture = bwData->db->getChangeFeedStream( - metadata->activeCFData.get(), cfKey, startVersion + 1, MAX_VERSION, metadata->keyRange); + changeFeedFuture = + bwData->db->getChangeFeedStream(newCFData, cfKey, startVersion + 1, MAX_VERSION, metadata->keyRange); } + // Start actors BEFORE setting new change feed data to ensure the change feed data is properly initialized by + // the client + metadata->activeCFData.set(newCFData); + ASSERT(metadata->readable.canBeSet()); metadata->readable.send(Void()); @@ -1389,12 +1383,14 @@ ACTOR Future blobGranuleUpdateFiles(Reference bwData, metadata->bufferedDeltaVersion); } - metadata->activeCFData.set(newChangeFeedData(startState.changeFeedStartVersion - 1)); - changeFeedFuture = bwData->db->getChangeFeedStream(metadata->activeCFData.get(), - cfKey, - startState.changeFeedStartVersion, - MAX_VERSION, - metadata->keyRange); + Reference newCFData = makeReference(); + + changeFeedFuture = bwData->db->getChangeFeedStream( + newCFData, cfKey, startState.changeFeedStartVersion, MAX_VERSION, metadata->keyRange); + + // Start actors BEFORE setting new change feed data to ensure the change feed data is properly + // initialized by the client + metadata->activeCFData.set(newCFData); } // process mutations @@ -1469,7 +1465,8 @@ ACTOR Future blobGranuleUpdateFiles(Reference bwData, cfRollbackVersion); metadata->waitForVersionReturned = cfRollbackVersion; } - metadata->activeCFData.set(newChangeFeedData(cfRollbackVersion)); + + Reference newCFData = makeReference(); if (!readOldChangeFeed && cfRollbackVersion < startState.changeFeedStartVersion) { // It isn't possible to roll back across the parent/child feed boundary, but as @@ -1486,7 +1483,7 @@ ACTOR Future blobGranuleUpdateFiles(Reference bwData, if (readOldChangeFeed) { ASSERT(cfRollbackVersion < startState.changeFeedStartVersion); oldChangeFeedFuture = - bwData->db->getChangeFeedStream(metadata->activeCFData.get(), + bwData->db->getChangeFeedStream(newCFData, oldCFKey.get(), cfRollbackVersion + 1, startState.changeFeedStartVersion, @@ -1500,13 +1497,14 @@ ACTOR Future blobGranuleUpdateFiles(Reference bwData, } ASSERT(cfRollbackVersion >= startState.changeFeedStartVersion); - changeFeedFuture = bwData->db->getChangeFeedStream(metadata->activeCFData.get(), - cfKey, - cfRollbackVersion + 1, - MAX_VERSION, - metadata->keyRange); + changeFeedFuture = bwData->db->getChangeFeedStream( + newCFData, cfKey, cfRollbackVersion + 1, MAX_VERSION, metadata->keyRange); } + // Start actors BEFORE setting new change feed data to ensure the change feed data + // is properly initialized by the client + metadata->activeCFData.set(newCFData); + justDidRollback = true; break; } From 215f5fae9372c95899575286224838af8494d2ff Mon Sep 17 00:00:00 2001 From: Josh Slocum Date: Wed, 19 Jan 2022 15:20:23 -0600 Subject: [PATCH 142/413] Reworked change feed initialization to handle more types of races --- fdbserver/BlobWorker.actor.cpp | 2 +- fdbserver/storageserver.actor.cpp | 185 +++++++++++++++++------------- 2 files changed, 107 insertions(+), 80 deletions(-) diff --git a/fdbserver/BlobWorker.actor.cpp b/fdbserver/BlobWorker.actor.cpp index 27045eca61..3409314b09 100644 --- a/fdbserver/BlobWorker.actor.cpp +++ b/fdbserver/BlobWorker.actor.cpp @@ -1526,7 +1526,7 @@ ACTOR Future blobGranuleUpdateFiles(Reference bwData, .detail("OldChangeFeed", readOldChangeFeed ? "T" : "F"); } if (DEBUG_BW_VERSION(deltas.version)) { - fmt::print("BW {0}: ({1})\n", deltas.version, deltas.mutations.size()); + fmt::print("BWB {0}: ({1})\n", deltas.version, deltas.mutations.size()); } metadata->currentDeltas.push_back_deep(metadata->deltaArena, deltas); diff --git a/fdbserver/storageserver.actor.cpp b/fdbserver/storageserver.actor.cpp index f4692c89ad..ba094001c9 100644 --- a/fdbserver/storageserver.actor.cpp +++ b/fdbserver/storageserver.actor.cpp @@ -4933,88 +4933,115 @@ private: ChangeFeedStatus status; std::tie(changeFeedRange, popVersion, status) = decodeChangeFeedValue(m.param2); auto feed = data->uidChangeFeed.find(changeFeedId); - if (feed == data->uidChangeFeed.end()) { - if (status == ChangeFeedStatus::CHANGE_FEED_CREATE) { - TraceEvent(SevDebug, "AddingChangeFeed", data->thisServerID) - .detail("RangeID", changeFeedId.printable()) - .detail("Range", changeFeedRange.toString()) - .detail("Version", currentVersion); - Reference changeFeedInfo(new ChangeFeedInfo()); - changeFeedInfo->range = changeFeedRange; - changeFeedInfo->id = changeFeedId; - changeFeedInfo->emptyVersion = currentVersion - 1; - data->uidChangeFeed[changeFeedId] = changeFeedInfo; - auto rs = data->keyChangeFeed.modify(changeFeedRange); - for (auto r = rs.begin(); r != rs.end(); ++r) { - r->value().push_back(changeFeedInfo); - } - data->keyChangeFeed.coalesce(changeFeedRange.contents()); - auto& mLV = data->addVersionToMutationLog(data->data().getLatestVersion()); - data->addMutationToMutationLog( - mLV, - MutationRef(MutationRef::SetValue, - persistChangeFeedKeys.begin.toString() + changeFeedId.toString(), - m.param2)); + // TODO REMOVE eventually + TraceEvent(SevDebug, "ChangeFeedPrivateMutation", data->thisServerID) + .detail("RangeID", changeFeedId.printable()) + .detail("Range", changeFeedRange.toString()) + .detail("Version", currentVersion) + .detail("PopVersion", popVersion) + .detail("Status", status); + + // Because of data moves, we can get mutations operating on a change feed we don't yet know about, because + // the fetch hasn't started yet + bool createdFeed = false; + if (feed == data->uidChangeFeed.end() && status != ChangeFeedStatus::CHANGE_FEED_DESTROY) { + createdFeed = true; + + Reference changeFeedInfo(new ChangeFeedInfo()); + changeFeedInfo->range = changeFeedRange; + changeFeedInfo->id = changeFeedId; + if (status == ChangeFeedStatus::CHANGE_FEED_CREATE && popVersion == invalidVersion) { + // for a create, the empty version should be now, otherwise it will be set in a later pop + changeFeedInfo->emptyVersion = currentVersion - 1; + } else { + TEST(true); // SS got non-create change feed private mutation before move created its metadata + changeFeedInfo->emptyVersion = invalidVersion; } - } else { - if (status == ChangeFeedStatus::CHANGE_FEED_DESTROY) { - TraceEvent(SevDebug, "DestroyingChangeFeed", data->thisServerID) - .detail("RangeID", changeFeedId.printable()) - .detail("Range", changeFeedRange.toString()) - .detail("Version", currentVersion); - Key beginClearKey = changeFeedId.withPrefix(persistChangeFeedKeys.begin); - auto& mLV = data->addVersionToMutationLog(data->data().getLatestVersion()); - data->addMutationToMutationLog( - mLV, MutationRef(MutationRef::ClearRange, beginClearKey, keyAfter(beginClearKey))); - data->addMutationToMutationLog(mLV, - MutationRef(MutationRef::ClearRange, - changeFeedDurableKey(feed->second->id, 0), - changeFeedDurableKey(feed->second->id, currentVersion))); - auto rs = data->keyChangeFeed.modify(feed->second->range); - for (auto r = rs.begin(); r != rs.end(); ++r) { - auto& feedList = r->value(); - for (int i = 0; i < feedList.size(); i++) { - if (feedList[i] == feed->second) { - swapAndPop(&feedList, i--); - } - } - } - data->keyChangeFeed.coalesce(feed->second->range.contents()); - data->uidChangeFeed.erase(feed); - } else if (status != ChangeFeedStatus::CHANGE_FEED_CREATE) { - // Can be a change feed create from move, ignore - // must be pop or stop - if (status == ChangeFeedStatus::CHANGE_FEED_STOP) { - TraceEvent(SevDebug, "StoppingChangeFeed", data->thisServerID) - .detail("RangeID", changeFeedId.printable()) - .detail("Range", changeFeedRange.toString()) - .detail("Version", currentVersion); - } - if (popVersion != invalidVersion && popVersion - 1 > feed->second->emptyVersion) { - feed->second->emptyVersion = popVersion - 1; - while (!feed->second->mutations.empty() && - feed->second->mutations.front().version < popVersion) { - feed->second->mutations.pop_front(); - } - if (feed->second->storageVersion != invalidVersion) { - data->storage.clearRange(KeyRangeRef(changeFeedDurableKey(feed->second->id, 0), - changeFeedDurableKey(feed->second->id, popVersion))); - if (popVersion > feed->second->storageVersion) { - feed->second->storageVersion = invalidVersion; - feed->second->durableVersion = invalidVersion; - // don't set fetchVersion to invalidVersion here because there could be an active fetch - } - } - } - feed->second->stopped = (status == ChangeFeedStatus::CHANGE_FEED_STOP); - auto& mLV = data->addVersionToMutationLog(data->data().getLatestVersion()); - data->addMutationToMutationLog( - mLV, - MutationRef(MutationRef::SetValue, - persistChangeFeedKeys.begin.toString() + changeFeedId.toString(), - m.param2)); + data->uidChangeFeed[changeFeedId] = changeFeedInfo; + + feed = data->uidChangeFeed.find(changeFeedId); + ASSERT(feed != data->uidChangeFeed.end()); + + TraceEvent(SevDebug, "AddingChangeFeed", data->thisServerID) + .detail("RangeID", changeFeedId.printable()) + .detail("Range", changeFeedRange.toString()) + .detail("EmptyVersion", feed->second->emptyVersion); + + auto rs = data->keyChangeFeed.modify(changeFeedRange); + for (auto r = rs.begin(); r != rs.end(); ++r) { + r->value().push_back(changeFeedInfo); } + data->keyChangeFeed.coalesce(changeFeedRange.contents()); + } + + bool addMutationToLog = false; + if (popVersion != invalidVersion) { + // pop the change feed at pop version, no matter what state it is in + if (popVersion - 1 > feed->second->emptyVersion) { + feed->second->emptyVersion = popVersion - 1; + while (!feed->second->mutations.empty() && feed->second->mutations.front().version < popVersion) { + feed->second->mutations.pop_front(); + } + if (feed->second->storageVersion != invalidVersion) { + data->storage.clearRange(KeyRangeRef(changeFeedDurableKey(feed->second->id, 0), + changeFeedDurableKey(feed->second->id, popVersion))); + if (popVersion > feed->second->storageVersion) { + feed->second->storageVersion = invalidVersion; + feed->second->durableVersion = invalidVersion; + // don't set fetchVersion to invalidVersion here because there could be an active fetch + } + } + } + feed->second->stopped = (status == ChangeFeedStatus::CHANGE_FEED_STOP); + addMutationToLog = true; + } else if (status == ChangeFeedStatus::CHANGE_FEED_CREATE) { + TraceEvent(SevDebug, "CreatingChangeFeed", data->thisServerID) + .detail("RangeID", changeFeedId.printable()) + .detail("Range", changeFeedRange.toString()) + .detail("Version", currentVersion); + // no-op, already created + addMutationToLog = true; + } else if (status == ChangeFeedStatus::CHANGE_FEED_STOP) { + TraceEvent(SevDebug, "StoppingChangeFeed", data->thisServerID) + .detail("RangeID", changeFeedId.printable()) + .detail("Range", changeFeedRange.toString()) + .detail("Version", currentVersion); + feed->second->stopped = true; + addMutationToLog = true; + } else if (status == ChangeFeedStatus::CHANGE_FEED_DESTROY && !createdFeed) { + TraceEvent(SevDebug, "DestroyingChangeFeed", data->thisServerID) + .detail("RangeID", changeFeedId.printable()) + .detail("Range", changeFeedRange.toString()) + .detail("Version", currentVersion); + Key beginClearKey = changeFeedId.withPrefix(persistChangeFeedKeys.begin); + auto& mLV = data->addVersionToMutationLog(data->data().getLatestVersion()); + data->addMutationToMutationLog( + mLV, MutationRef(MutationRef::ClearRange, beginClearKey, keyAfter(beginClearKey))); + data->addMutationToMutationLog(mLV, + MutationRef(MutationRef::ClearRange, + changeFeedDurableKey(feed->second->id, 0), + changeFeedDurableKey(feed->second->id, currentVersion))); + auto rs = data->keyChangeFeed.modify(feed->second->range); + for (auto r = rs.begin(); r != rs.end(); ++r) { + auto& feedList = r->value(); + for (int i = 0; i < feedList.size(); i++) { + if (feedList[i] == feed->second) { + swapAndPop(&feedList, i--); + } + } + } + data->keyChangeFeed.coalesce(feed->second->range.contents()); + data->uidChangeFeed.erase(feed); + } + + if (addMutationToLog) { + auto& mLV = data->addVersionToMutationLog(data->data().getLatestVersion()); + data->addMutationToMutationLog( + mLV, + MutationRef(MutationRef::SetValue, + persistChangeFeedKeys.begin.toString() + changeFeedId.toString(), + m.param2)); } } else if (m.param1.substr(1).startsWith(tssMappingKeys.begin) && (m.type == MutationRef::SetValue || m.type == MutationRef::ClearRange)) { From 9bce1dcb0f33f4e115eebe300b4f6248d05a9fbf Mon Sep 17 00:00:00 2001 From: Josh Slocum Date: Wed, 19 Jan 2022 17:03:57 -0600 Subject: [PATCH 143/413] Made blob granule split transaction idempotent on retry --- fdbserver/BlobManager.actor.cpp | 37 ++++++++++++++++++++++++++++----- 1 file changed, 32 insertions(+), 5 deletions(-) diff --git a/fdbserver/BlobManager.actor.cpp b/fdbserver/BlobManager.actor.cpp index 5824e35283..c0b8b3f2d0 100644 --- a/fdbserver/BlobManager.actor.cpp +++ b/fdbserver/BlobManager.actor.cpp @@ -764,6 +764,7 @@ ACTOR Future maybeSplitRange(BlobManagerData* bmData, Standalone> _newRanges = wait(splitRange(tr, granuleRange)); newRanges = _newRanges; + ASSERT(newRanges.size() >= 2); if (newRanges.size() == 2) { // not large enough to split, just reassign back to worker if (BM_DEBUG) { @@ -813,8 +814,20 @@ ACTOR Future maybeSplitRange(BlobManagerData* bmData, fmt::print(" {}\n", newRanges[i].printable()); } } + ASSERT(granuleRange.begin == newRanges.front()); + ASSERT(granuleRange.end == newRanges.back()); - // Need to split range. Persist intent to split and split metadata to DB BEFORE sending split requests + // Have to make set of granule ids deterministic across retries to not end up with extra UIDs in the split + // state, which could cause recovery to fail and resources to not be cleaned up. + // This entire transaction must be idempotent across retries for all splitting state + state std::vector newGranuleIDs; + newGranuleIDs.reserve(newRanges.size() - 1); + for (int i = 0; i < newRanges.size() - 1; i++) { + newGranuleIDs.push_back(deterministicRandom()->randomUniqueID()); + } + + // Need to split range. Persist intent to split and split metadata to DB BEFORE sending split assignments to blob + // workers, so that nothing is lost on blob manager recovery loop { try { tr->reset(); @@ -858,10 +871,18 @@ ACTOR Future maybeSplitRange(BlobManagerData* bmData, tr->set(lockKey, blobGranuleLockValueFor(bmData->epoch, newLockSeqno, std::get<2>(prevGranuleLock))); // set up split metadata - for (int i = 0; i < newRanges.size() - 1; i++) { - UID newGranuleID = deterministicRandom()->randomUniqueID(); + /*fmt::print("Persisting granule split {0} [{1} - {2})\n", + granuleID.toString().substr(0, 6), + granuleRange.begin.printable(), + granuleRange.end.printable());*/ - Key splitKey = blobGranuleSplitKeyFor(granuleID, newGranuleID); + for (int i = 0; i < newRanges.size() - 1; i++) { + /*fmt::print(" {0} [{1} - {2})\n", + newGranuleIDs[i].toString().substr(0, 6), + newRanges[i].printable(), + newRanges[i + 1].printable());*/ + + Key splitKey = blobGranuleSplitKeyFor(granuleID, newGranuleIDs[i]); tr->set(blobGranuleSplitBoundaryKeyFor(granuleID, newRanges[i]), Value()); tr->atomicOp(splitKey, @@ -871,7 +892,7 @@ ACTOR Future maybeSplitRange(BlobManagerData* bmData, Key historyKey = blobGranuleHistoryKeyFor(KeyRangeRef(newRanges[i], newRanges[i + 1]), latestVersion); Standalone historyValue; - historyValue.granuleID = newGranuleID; + historyValue.granuleID = newGranuleIDs[i]; historyValue.parentGranules.push_back(historyValue.arena(), std::pair(granuleRange, granuleStartVersion)); @@ -887,6 +908,12 @@ ACTOR Future maybeSplitRange(BlobManagerData* bmData, wait(tr->commit()); break; } catch (Error& e) { + if (e.code() == error_code_operation_cancelled) { + throw; + } + if (BM_DEBUG) { + fmt::print("BM {0} Persisting granule split got error {1}\n", bmData->epoch, e.name()); + } if (e.code() == error_code_granule_assignment_conflict) { if (bmData->iAmReplaced.canBeSet()) { bmData->iAmReplaced.send(Void()); From 6a8e73891fb0726e868f28d087b241075454bd24 Mon Sep 17 00:00:00 2001 From: Josh Slocum Date: Thu, 20 Jan 2022 11:44:53 -0600 Subject: [PATCH 144/413] Bug fixes for blob worker rollbacks --- fdbserver/BlobWorker.actor.cpp | 32 +++++++++++++++++++++++++++----- 1 file changed, 27 insertions(+), 5 deletions(-) diff --git a/fdbserver/BlobWorker.actor.cpp b/fdbserver/BlobWorker.actor.cpp index 3409314b09..a9488a5594 100644 --- a/fdbserver/BlobWorker.actor.cpp +++ b/fdbserver/BlobWorker.actor.cpp @@ -901,7 +901,7 @@ ACTOR Future handleCompletedDeltaFile(Reference bwData, BlobFileIndex completedDeltaFile, Key cfKey, Version cfStartVersion, - std::deque> rollbacksCompleted) { + std::deque>* rollbacksCompleted) { metadata->files.deltaFiles.push_back(completedDeltaFile); ASSERT(metadata->durableDeltaVersion.get() < completedDeltaFile.version); metadata->durableDeltaVersion.set(completedDeltaFile.version); @@ -917,8 +917,12 @@ ACTOR Future handleCompletedDeltaFile(Reference bwData, Future popFuture = bwData->db->popChangeFeedMutations(cfKey, completedDeltaFile.version); wait(popFuture); } - while (!rollbacksCompleted.empty() && completedDeltaFile.version >= rollbacksCompleted.front().first) { - rollbacksCompleted.pop_front(); + while (!rollbacksCompleted->empty() && completedDeltaFile.version >= rollbacksCompleted->front().second) { + fmt::print("Completed rollback {0} -> {1} with delta file {2}\n", + rollbacksCompleted->front().second, + rollbacksCompleted->front().first, + completedDeltaFile.version); + rollbacksCompleted->pop_front(); } return Void(); } @@ -1292,7 +1296,7 @@ ACTOR Future blobGranuleUpdateFiles(Reference bwData, completedFile, cfKey, startState.changeFeedStartVersion, - rollbacksCompleted)); + &rollbacksCompleted)); } inFlightFiles.pop_front(); @@ -1418,7 +1422,20 @@ ACTOR Future blobGranuleUpdateFiles(Reference bwData, ASSERT(rollbackVersion >= metadata->durableDeltaVersion.get()); if (!rollbacksInProgress.empty()) { + // TODO REMOVE, for debugging + if (rollbacksInProgress.front().first != rollbackVersion) { + fmt::print("Found out of order rollbacks! Current in progress: {0}, mutation " + "version: {1}\n", + rollbacksInProgress.front().first, + rollbackVersion); + } ASSERT(rollbacksInProgress.front().first == rollbackVersion); + if (rollbacksInProgress.front().first != rollbackVersion) { + fmt::print("Found out of order rollbacks! Current in progress: {0}, rollback " + "version: {1}\n", + rollbacksInProgress.front().second, + deltas.version); + } ASSERT(rollbacksInProgress.front().second == deltas.version); if (BW_DEBUG) { fmt::print("Passed rollback {0} -> {1}\n", deltas.version, rollbackVersion); @@ -1438,6 +1455,11 @@ ACTOR Future blobGranuleUpdateFiles(Reference bwData, deltas.version, rollbackVersion); } + // Still have to add to rollbacksCompleted. If we later roll the granule back past + // this because of cancelling a delta file, we need to count this as in progress so + // we can match the rollback mutation to a rollbackInProgress when we restart the + // stream. + rollbacksCompleted.push_back(std::pair(rollbackVersion, deltas.version)); } else { if (BW_DEBUG) { fmt::print("BW [{0} - {1}) ROLLBACK @ {2} -> {3}\n", @@ -1683,7 +1705,7 @@ ACTOR Future blobGranuleUpdateFiles(Reference bwData, completedFile, cfKey, startState.changeFeedStartVersion, - rollbacksCompleted)); + &rollbacksCompleted)); } inFlightFiles.pop_front(); From 07f09f1118741ceb22f87120a8f894682a4c176a Mon Sep 17 00:00:00 2001 From: Josh Slocum Date: Thu, 20 Jan 2022 12:33:15 -0600 Subject: [PATCH 145/413] Changed BlobManagerData from pointer to reference to fix ASAN issues --- fdbserver/BlobManager.actor.cpp | 89 ++++++++++++++++++--------------- 1 file changed, 48 insertions(+), 41 deletions(-) diff --git a/fdbserver/BlobManager.actor.cpp b/fdbserver/BlobManager.actor.cpp index c0b8b3f2d0..f8d5579327 100644 --- a/fdbserver/BlobManager.actor.cpp +++ b/fdbserver/BlobManager.actor.cpp @@ -43,8 +43,6 @@ #define BM_DEBUG true -// FIXME: change all BlobManagerData* to Reference to avoid segfaults if core loop gets error - // TODO add comments + documentation void handleClientBlobRange(KeyRangeMap* knownBlobRanges, Arena& ar, @@ -199,7 +197,7 @@ struct BlobWorkerStats { BlobWorkerStats(int numGranulesAssigned = 0) : numGranulesAssigned(numGranulesAssigned) {} }; -struct BlobManagerData { +struct BlobManagerData : NonCopyable, ReferenceCounted { UID id; Database db; Optional dcId; @@ -287,7 +285,7 @@ ACTOR Future>> splitRange(Reference pickWorkerForAssign(BlobManagerData* bmData) { +ACTOR Future pickWorkerForAssign(Reference bmData) { // wait until there are BWs to pick from while (bmData->workerStats.size() == 0) { // TODO REMOVE @@ -326,7 +324,10 @@ ACTOR Future pickWorkerForAssign(BlobManagerData* bmData) { return eligibleWorkers[idx]; } -ACTOR Future doRangeAssignment(BlobManagerData* bmData, RangeAssignment assignment, UID workerID, int64_t seqNo) { +ACTOR Future doRangeAssignment(Reference bmData, + RangeAssignment assignment, + UID workerID, + int64_t seqNo) { if (BM_DEBUG) { fmt::print("BM {0} {1} range [{2} - {3}) @ ({4}, {5}) to {6}\n", @@ -468,7 +469,7 @@ ACTOR Future doRangeAssignment(BlobManagerData* bmData, RangeAssignment as return Void(); } -ACTOR Future rangeAssigner(BlobManagerData* bmData) { +ACTOR Future rangeAssigner(Reference bmData) { loop { // inject delay into range assignments if (BUGGIFY_WITH_PROB(0.05)) { @@ -549,7 +550,7 @@ ACTOR Future rangeAssigner(BlobManagerData* bmData) { } } -ACTOR Future checkManagerLock(Reference tr, BlobManagerData* bmData) { +ACTOR Future checkManagerLock(Reference tr, Reference bmData) { Optional currentLockValue = wait(tr->get(blobManagerEpochKey)); ASSERT(currentLockValue.present()); int64_t currentEpoch = decodeBlobManagerEpochValue(currentLockValue.get()); @@ -571,7 +572,8 @@ ACTOR Future checkManagerLock(Reference tr, Blo return Void(); } -ACTOR Future writeInitialGranuleMapping(BlobManagerData* bmData, Standalone> boundaries) { +ACTOR Future writeInitialGranuleMapping(Reference bmData, + Standalone> boundaries) { state Reference tr = makeReference(bmData->db); // don't do too many in one transaction state int i = 0; @@ -615,7 +617,7 @@ ACTOR Future writeInitialGranuleMapping(BlobManagerData* bmData, Standalon // FIXME: this does all logic in one transaction. Adding a giant range to an existing database to blobify would // require doing a ton of storage metrics calls, which we should split up across multiple transactions likely. -ACTOR Future monitorClientRanges(BlobManagerData* bmData) { +ACTOR Future monitorClientRanges(Reference bmData) { state Optional lastChangeKeyValue; loop { state Reference tr = makeReference(bmData->db); @@ -750,7 +752,7 @@ static void downsampleSplit(const Standalone>& splits, } } -ACTOR Future maybeSplitRange(BlobManagerData* bmData, +ACTOR Future maybeSplitRange(Reference bmData, UID currentWorkerId, KeyRange granuleRange, UID granuleID, @@ -946,7 +948,7 @@ ACTOR Future maybeSplitRange(BlobManagerData* bmData, return Void(); } -ACTOR Future deregisterBlobWorker(BlobManagerData* bmData, BlobWorkerInterface interf) { +ACTOR Future deregisterBlobWorker(Reference bmData, BlobWorkerInterface interf) { state Reference tr = makeReference(bmData->db); loop { tr->setOption(FDBTransactionOptions::ACCESS_SYSTEM_KEYS); @@ -971,7 +973,7 @@ ACTOR Future deregisterBlobWorker(BlobManagerData* bmData, BlobWorkerInter } } -ACTOR Future killBlobWorker(BlobManagerData* bmData, BlobWorkerInterface bwInterf, bool registered) { +ACTOR Future killBlobWorker(Reference bmData, BlobWorkerInterface bwInterf, bool registered) { state UID bwId = bwInterf.id(); // Remove blob worker from stats map so that when we try to find a worker to takeover the range, @@ -1038,7 +1040,7 @@ ACTOR Future killBlobWorker(BlobManagerData* bmData, BlobWorkerInterface b return Void(); } -ACTOR Future monitorBlobWorkerStatus(BlobManagerData* bmData, BlobWorkerInterface bwInterf) { +ACTOR Future monitorBlobWorkerStatus(Reference bmData, BlobWorkerInterface bwInterf) { state KeyRangeMap> lastSeenSeqno; // outer loop handles reconstructing stream if it got a retryable error loop { @@ -1142,7 +1144,7 @@ ACTOR Future monitorBlobWorkerStatus(BlobManagerData* bmData, BlobWorkerIn } } -ACTOR Future monitorBlobWorker(BlobManagerData* bmData, BlobWorkerInterface bwInterf) { +ACTOR Future monitorBlobWorker(Reference bmData, BlobWorkerInterface bwInterf) { try { state Future waitFailure = waitFailureClient(bwInterf.waitFailure, SERVER_KNOBS->BLOB_WORKER_TIMEOUT); state Future monitorStatus = monitorBlobWorkerStatus(bmData, bwInterf); @@ -1191,7 +1193,7 @@ ACTOR Future monitorBlobWorker(BlobManagerData* bmData, BlobWorkerInterfac return Void(); } -ACTOR Future checkBlobWorkerList(BlobManagerData* bmData, Promise workerListReady) { +ACTOR Future checkBlobWorkerList(Reference bmData, Promise workerListReady) { loop { // Get list of last known blob workers // note: the list will include every blob worker that the old manager knew about, @@ -1217,7 +1219,7 @@ ACTOR Future checkBlobWorkerList(BlobManagerData* bmData, Promise wo } } -ACTOR Future recoverBlobManager(BlobManagerData* bmData) { +ACTOR Future recoverBlobManager(Reference bmData) { state Promise workerListReady; bmData->addActor.send(checkBlobWorkerList(bmData, workerListReady)); wait(workerListReady.getFuture()); @@ -1549,7 +1551,7 @@ ACTOR Future recoverBlobManager(BlobManagerData* bmData) { return Void(); } -ACTOR Future chaosRangeMover(BlobManagerData* bmData) { +ACTOR Future chaosRangeMover(Reference bmData) { ASSERT(g_network->isSimulated()); loop { wait(delay(30.0)); @@ -1602,7 +1604,7 @@ ACTOR Future chaosRangeMover(BlobManagerData* bmData) { } // Returns the number of blob workers on addr -int numExistingBWOnAddr(BlobManagerData* self, const AddressExclusion& addr) { +int numExistingBWOnAddr(Reference self, const AddressExclusion& addr) { int numExistingBW = 0; for (auto& server : self->workersById) { const NetworkAddress& netAddr = server.second.stableAddress(); @@ -1616,7 +1618,7 @@ int numExistingBWOnAddr(BlobManagerData* self, const AddressExclusion& addr) { } // Tries to recruit a blob worker on the candidateWorker process -ACTOR Future initializeBlobWorker(BlobManagerData* self, RecruitBlobWorkerReply candidateWorker) { +ACTOR Future initializeBlobWorker(Reference self, RecruitBlobWorkerReply candidateWorker) { const NetworkAddress& netAddr = candidateWorker.worker.stableAddress(); AddressExclusion workerAddr(netAddr.ip, netAddr.port); self->recruitingStream.set(self->recruitingStream.get() + 1); @@ -1698,7 +1700,7 @@ ACTOR Future initializeBlobWorker(BlobManagerData* self, RecruitBlobWorker // Recruits blob workers in a loop ACTOR Future blobWorkerRecruiter( - BlobManagerData* self, + Reference self, Reference>> recruitBlobWorker) { state Future fCandidateWorker; state RecruitBlobWorkerRequest lastRequest; @@ -1757,7 +1759,7 @@ ACTOR Future blobWorkerRecruiter( } } -ACTOR Future haltBlobGranules(BlobManagerData* bmData) { +ACTOR Future haltBlobGranules(Reference bmData) { std::vector blobWorkers = wait(getBlobWorkers(bmData->db)); std::vector> deregisterBlobWorkers; for (auto& worker : blobWorkers) { @@ -1771,7 +1773,7 @@ ACTOR Future haltBlobGranules(BlobManagerData* bmData) { return Void(); } -ACTOR Future loadHistoryFiles(BlobManagerData* bmData, UID granuleID) { +ACTOR Future loadHistoryFiles(Reference bmData, UID granuleID) { state Transaction tr(bmData->db); state KeyRange range = blobGranuleFileKeyRangeFor(granuleID); state Key startKey = range.begin; @@ -1791,7 +1793,7 @@ ACTOR Future loadHistoryFiles(BlobManagerData* bmData, UID granule * also removes the history entry for this granule from the system keyspace * TODO ensure cannot fully delete granule that is still splitting! */ -ACTOR Future fullyDeleteGranule(BlobManagerData* self, UID granuleId, KeyRef historyKey) { +ACTOR Future fullyDeleteGranule(Reference self, UID granuleId, KeyRef historyKey) { if (BM_DEBUG) { fmt::print("Fully deleting granule {0}: init\n", granuleId.toString()); } @@ -1863,7 +1865,7 @@ ACTOR Future fullyDeleteGranule(BlobManagerData* self, UID granuleId, KeyR * file might be deleted. We will need to ensure we don't rely on the granule's startVersion * (that's persisted as part of the key), but rather use the granule's first snapshot's version when needed */ -ACTOR Future partiallyDeleteGranule(BlobManagerData* self, UID granuleId, Version pruneVersion) { +ACTOR Future partiallyDeleteGranule(Reference self, UID granuleId, Version pruneVersion) { if (BM_DEBUG) { fmt::print("Partially deleting granule {0}: init\n", granuleId.toString()); } @@ -1963,7 +1965,11 @@ ACTOR Future partiallyDeleteGranule(BlobManagerData* self, UID granuleId, * Once all this is done, we finally clear the pruneIntent key, if possible, to indicate we are done * processing this prune intent. */ -ACTOR Future pruneRange(BlobManagerData* self, KeyRef startKey, KeyRef endKey, Version pruneVersion, bool force) { +ACTOR Future pruneRange(Reference self, + KeyRef startKey, + KeyRef endKey, + Version pruneVersion, + bool force) { if (BM_DEBUG) { fmt::print("pruneRange starting for range [{0} - {1}) @ pruneVersion={2}, force={3}\n", startKey.printable(), @@ -2223,7 +2229,7 @@ ACTOR Future pruneRange(BlobManagerData* self, KeyRef startKey, KeyRef end * improvements we don't really need here (also we need to go over all prune intents anyways in the * case that the timer is up before any new prune intents arrive). */ -ACTOR Future monitorPruneKeys(BlobManagerData* self) { +ACTOR Future monitorPruneKeys(Reference self) { // setup bstore try { if (BM_DEBUG) { @@ -2402,7 +2408,7 @@ ACTOR Future monitorPruneKeys(BlobManagerData* self) { } } -ACTOR Future doLockChecks(BlobManagerData* bmData) { +ACTOR Future doLockChecks(Reference bmData) { loop { Promise check = bmData->doLockCheck; wait(check.getFuture()); @@ -2439,40 +2445,41 @@ ACTOR Future doLockChecks(BlobManagerData* bmData) { ACTOR Future blobManager(BlobManagerInterface bmInterf, Reference const> dbInfo, int64_t epoch) { - state BlobManagerData self(deterministicRandom()->randomUniqueID(), - openDBOnServer(dbInfo, TaskPriority::DefaultEndpoint, LockAware::True), - bmInterf.locality.dcId()); + state Reference self = + makeReference(deterministicRandom()->randomUniqueID(), + openDBOnServer(dbInfo, TaskPriority::DefaultEndpoint, LockAware::True), + bmInterf.locality.dcId()); - state Future collection = actorCollection(self.addActor.getFuture()); + state Future collection = actorCollection(self->addActor.getFuture()); if (BM_DEBUG) { fmt::print("Blob manager {0} starting...\n", epoch); } - self.epoch = epoch; + self->epoch = epoch; // although we start the recruiter, we wait until existing workers are ack'd auto recruitBlobWorker = IAsyncListener>::create( dbInfo, [](auto const& info) { return info.clusterInterface.recruitBlobWorker; }); - self.addActor.send(blobWorkerRecruiter(&self, recruitBlobWorker)); + self->addActor.send(blobWorkerRecruiter(self, recruitBlobWorker)); // we need to recover the old blob manager's state (e.g. granule assignments) before // before the new blob manager does anything - wait(recoverBlobManager(&self)); + wait(recoverBlobManager(self)); - self.addActor.send(doLockChecks(&self)); - self.addActor.send(monitorClientRanges(&self)); - self.addActor.send(rangeAssigner(&self)); - self.addActor.send(monitorPruneKeys(&self)); + self->addActor.send(doLockChecks(self)); + self->addActor.send(monitorClientRanges(self)); + self->addActor.send(rangeAssigner(self)); + self->addActor.send(monitorPruneKeys(self)); if (BUGGIFY) { - self.addActor.send(chaosRangeMover(&self)); + self->addActor.send(chaosRangeMover(self)); } // TODO probably other things here eventually try { loop choose { - when(wait(self.iAmReplaced.getFuture())) { + when(wait(self->iAmReplaced.getFuture())) { if (BM_DEBUG) { printf("Blob Manager exiting because it is replaced\n"); } @@ -2484,7 +2491,7 @@ ACTOR Future blobManager(BlobManagerInterface bmInterf, break; } when(state HaltBlobGranulesRequest req = waitNext(bmInterf.haltBlobGranules.getFuture())) { - wait(haltBlobGranules(&self)); + wait(haltBlobGranules(self)); req.reply.send(Void()); TraceEvent("BlobGranulesHalted", bmInterf.id()).detail("ReqID", req.requesterID); break; From 05c3aeb93f1141150288c5017e5b342a7705c0d9 Mon Sep 17 00:00:00 2001 From: Josh Slocum Date: Wed, 19 Jan 2022 17:49:19 -0600 Subject: [PATCH 146/413] Fixed range read bug in blob manager recovery --- fdbserver/BlobManager.actor.cpp | 14 ++++++++++---- 1 file changed, 10 insertions(+), 4 deletions(-) diff --git a/fdbserver/BlobManager.actor.cpp b/fdbserver/BlobManager.actor.cpp index f8d5579327..cd4b27c4e6 100644 --- a/fdbserver/BlobManager.actor.cpp +++ b/fdbserver/BlobManager.actor.cpp @@ -1360,13 +1360,18 @@ ACTOR Future recoverBlobManager(Reference bmData) { if (!splitResult.more) { break; } - ASSERT(splitResult.readThrough.present()); - splitBeginKey = splitResult.readThrough.get(); + ASSERT(splitResult.readThrough.present() || splitResult.size() > 0); + splitBeginKey = splitResult.readThrough.present() ? splitResult.readThrough.get() + : keyAfter(splitResult.back().key); loop { try { RangeResult r = wait(tr->getRange(KeyRangeRef(splitBeginKey, blobGranuleSplitKeys.end), rowLimit)); ASSERT(r.size() > 0 || !r.more); + fmt::print("Split cursor got {0} rows, readThrough={1}, more={2}\n", + r.size(), + r.readThrough.present() ? r.readThrough.get().printable().c_str() : "", + r.more ? "T" : "F"); splitResult = r; splitResultIdx = 0; break; @@ -1413,8 +1418,9 @@ ACTOR Future recoverBlobManager(Reference bmData) { if (!boundaryResult.more) { break; } - ASSERT(boundaryResult.readThrough.present()); - boundaryBeginKey = boundaryResult.readThrough.get(); + ASSERT(boundaryResult.readThrough.present() || boundaryResult.size() > 0); + boundaryBeginKey = boundaryResult.readThrough.present() ? boundaryResult.readThrough.get() + : keyAfter(boundaryResult.back().key); loop { try { RangeResult r = wait( From 3be1bcd588ea9cf9203c7143199e9679b17fce47 Mon Sep 17 00:00:00 2001 From: Josh Slocum Date: Thu, 20 Jan 2022 14:42:13 -0600 Subject: [PATCH 147/413] Fix move race in change feeds and added extra debugging to track down similar problems --- fdbserver/BlobManager.actor.cpp | 4 - fdbserver/storageserver.actor.cpp | 155 ++++++++++++++++++++++++++++-- 2 files changed, 145 insertions(+), 14 deletions(-) diff --git a/fdbserver/BlobManager.actor.cpp b/fdbserver/BlobManager.actor.cpp index cd4b27c4e6..c4a7b66262 100644 --- a/fdbserver/BlobManager.actor.cpp +++ b/fdbserver/BlobManager.actor.cpp @@ -1368,10 +1368,6 @@ ACTOR Future recoverBlobManager(Reference bmData) { RangeResult r = wait(tr->getRange(KeyRangeRef(splitBeginKey, blobGranuleSplitKeys.end), rowLimit)); ASSERT(r.size() > 0 || !r.more); - fmt::print("Split cursor got {0} rows, readThrough={1}, more={2}\n", - r.size(), - r.readThrough.present() ? r.readThrough.get().printable().c_str() : "", - r.more ? "T" : "F"); splitResult = r; splitResultIdx = 0; break; diff --git a/fdbserver/storageserver.actor.cpp b/fdbserver/storageserver.actor.cpp index ba094001c9..baa72ff25e 100644 --- a/fdbserver/storageserver.actor.cpp +++ b/fdbserver/storageserver.actor.cpp @@ -1741,6 +1741,14 @@ MutationsAndVersionRef filterMutations(Arena& arena, return m; } +// TODO REMOVE!!! when BG is correctness clean +#define DEBUG_SS_ID ""_sr +#define DEBUG_SS_CF_ID ""_sr +#define DEBUG_SS_CF_BEGIN_VERSION invalidVersion +#define DEBUG_SS_CFM(ssId, cfId, v) \ + ssId.toString().substr(0, 4) == DEBUG_SS_ID&& cfId.printable().substr(0, 6) == DEBUG_SS_CF_ID&& v == \ + DEBUG_SS_CF_BEGIN_VERSION + ACTOR Future> getChangeFeedMutations(StorageServer* data, ChangeFeedStreamRequest req, bool inverted) { @@ -1750,6 +1758,22 @@ ACTOR Future> getChangeFeedMutations(Stor state int remainingDurableBytes = CLIENT_KNOBS->REPLY_BYTE_LIMIT; state Version startVersion = data->version.get(); + if (DEBUG_SS_CFM(data->thisServerID, req.rangeID, req.begin)) { + printf("CFM: SS %s CF %s: GCFM [%s - %s) %lld - %lld\n", + data->thisServerID.toString().substr(0, 4).c_str(), + req.rangeID.printable().substr(0, 6).c_str(), + req.range.begin.printable().c_str(), + req.range.end.printable().c_str(), + req.begin, + req.end); + // TODO REMOVE + TraceEvent(SevDebug, "ChangeFeedMutations", data->thisServerID) + .detail("FeedID", req.rangeID) + .detail("Range", req.range) + .detail("Begin", req.begin) + .detail("End", req.end); + } + if (data->version.get() < req.begin) { wait(data->version.whenAtLeast(req.begin)); // we must delay here to ensure that any up-to-date change feeds that are waiting on the @@ -1767,11 +1791,31 @@ ACTOR Future> getChangeFeedMutations(Stor throw unknown_change_feed(); } + if (DEBUG_SS_CFM(data->thisServerID, req.rangeID, req.begin)) { + printf("CFM: SS %s CF %s: got version %lld >= %lld\n", + data->thisServerID.toString().substr(0, 4).c_str(), + req.rangeID.printable().substr(0, 6).c_str(), + data->version.get(), + req.begin); + } + // We must copy the mutationDeque when fetching the durable bytes in case mutations are popped from memory while // waiting for the results state Version dequeVersion = data->version.get(); state Version dequeKnownCommit = data->knownCommittedVersion; + if (DEBUG_SS_CFM(data->thisServerID, req.rangeID, req.begin)) { + printf("CFM: SS %s CF %s: dequeVersion=%lld, emptyVersion=%lld, storageVersion=%lld, durableVersion=%lld, " + "fetchVersion=%lld\n", + data->thisServerID.toString().substr(0, 4).c_str(), + req.rangeID.printable().substr(0, 6).c_str(), + dequeVersion, + feed->second->emptyVersion, + feed->second->storageVersion, + feed->second->durableVersion, + feed->second->fetchVersion); + } + if (req.end > feed->second->emptyVersion + 1) { for (auto& it : feed->second->mutations) { if (it.version >= req.end || it.version > dequeVersion || remainingLimitBytes <= 0) { @@ -1784,6 +1828,14 @@ ACTOR Future> getChangeFeedMutations(Stor remainingLimitBytes -= sizeof(MutationsAndVersionRef) + m.expectedSize(); } } + if (DEBUG_SS_CFM(data->thisServerID, req.rangeID, req.begin)) { + printf("CFM: SS %s CF %s: got %lld - %lld (%d) from memory\n", + data->thisServerID.toString().substr(0, 4).c_str(), + req.rangeID.printable().substr(0, 6).c_str(), + memoryReply.mutations.empty() ? invalidVersion : memoryReply.mutations.front().version, + memoryReply.mutations.empty() ? invalidVersion : memoryReply.mutations.back().version, + memoryReply.mutations.size()); + } } if (req.end > feed->second->emptyVersion + 1 && feed->second->durableVersion != invalidVersion && @@ -1815,6 +1867,14 @@ ACTOR Future> getChangeFeedMutations(Stor // because we cannot add mutations from memory if there are potentially more on disk lastVersion = version; } + if (DEBUG_SS_CFM(data->thisServerID, req.rangeID, req.begin)) { + printf("CFM: SS %s CF %s: got %lld - %lld (%d) from disk\n", + data->thisServerID.toString().substr(0, 4).c_str(), + req.rangeID.printable().substr(0, 6).c_str(), + reply.mutations.empty() ? invalidVersion : reply.mutations.front().version, + reply.mutations.empty() ? invalidVersion : reply.mutations.back().version, + reply.mutations.size()); + } if (remainingDurableBytes > 0) { reply.arena.dependsOn(memoryReply.arena); auto it = memoryReply.mutations.begin(); @@ -1832,6 +1892,12 @@ ACTOR Future> getChangeFeedMutations(Stor Version finalVersion = std::min(req.end - 1, dequeVersion); if ((reply.mutations.empty() || reply.mutations.back().version < finalVersion) && remainingLimitBytes > 0 && remainingDurableBytes > 0) { + if (DEBUG_SS_CFM(data->thisServerID, req.rangeID, req.begin)) { + printf("CFM: SS %s CF %s: adding empty %lld\n", + data->thisServerID.toString().substr(0, 4).c_str(), + req.rangeID.printable().substr(0, 6).c_str(), + finalVersion); + } reply.mutations.push_back( reply.arena, MutationsAndVersionRef(finalVersion, finalVersion == dequeVersion ? dequeKnownCommit : 0)); } @@ -1848,6 +1914,24 @@ ACTOR Future> getChangeFeedMutations(Stor } } + if (DEBUG_SS_CFM(data->thisServerID, req.rangeID, req.begin)) { + printf("CFM: SS %s CF %s: result %lld - %lld (%d)\n", + data->thisServerID.toString().substr(0, 4).c_str(), + req.rangeID.printable().substr(0, 6).c_str(), + reply.mutations.empty() ? invalidVersion : reply.mutations.front().version, + reply.mutations.empty() ? invalidVersion : reply.mutations.back().version, + reply.mutations.size()); + // TODO REMOVE + TraceEvent(SevDebug, "ChangeFeedMutationsDone", data->thisServerID) + .detail("FeedID", req.rangeID) + .detail("Range", req.range) + .detail("Begin", req.begin) + .detail("End", req.end) + .detail("FirstVersion", reply.mutations.empty() ? invalidVersion : reply.mutations.front().version) + .detail("LastVersion", reply.mutations.empty() ? invalidVersion : reply.mutations.back().version) + .detail("Count", reply.mutations.size()); + } + // If the SS's version advanced at all during any of the waits, the read from memory may have missed some mutations, // so gotAll can only be true if data->version didn't change over the course of this actor return std::make_pair(reply, @@ -1914,6 +1998,16 @@ ACTOR Future changeFeedStreamQ(StorageServer* data, ChangeFeedStreamReques wait(delay(0, TaskPriority::DefaultEndpoint)); + if (DEBUG_SS_CFM(data->thisServerID, req.rangeID, req.begin)) { + printf("CFM: SS %s CF %s: got CFSQ [%s - %s) %lld - %lld\n", + data->thisServerID.toString().substr(0, 4).c_str(), + req.rangeID.printable().substr(0, 6).c_str(), + req.range.begin.printable().c_str(), + req.range.end.printable().c_str(), + req.begin, + req.end); + } + try { loop { Future onReady = req.reply.onReady(); @@ -3902,6 +3996,11 @@ ACTOR Future fetchChangeFeedApplier(StorageServer* data, state Future feed = data->cx->getChangeFeedStream( feedResults, rangeId, 0, existing ? fetchVersion + 1 : data->version.get() + 1, range); + // TODO remove debugging eventually? + state Version firstVersion = invalidVersion; + state Version lastVersion = invalidVersion; + state int64_t versionsFetched = 0; + if (!existing) { try { loop { @@ -3921,6 +4020,11 @@ ACTOR Future fetchChangeFeedApplier(StorageServer* data, changeFeedDurableValue(it.mutations, it.knownCommittedVersion))); changeFeedInfo->fetchVersion = std::max(changeFeedInfo->fetchVersion, it.version); + if (firstVersion == invalidVersion) { + firstVersion = it.version; + } + lastVersion = it.version; + versionsFetched++; } } wait(yield()); @@ -3929,6 +4033,15 @@ ACTOR Future fetchChangeFeedApplier(StorageServer* data, if (e.code() != error_code_end_of_stream) { throw; } + // TODO REMOVE? + TraceEvent(SevDebug, "FetchChangeFeedDone", data->thisServerID) + .detail("RangeID", rangeId.printable()) + .detail("Range", range.toString()) + .detail("FetchVersion", fetchVersion) + .detail("FirstFetchedVersion", firstVersion) + .detail("LastFetchedVersion", lastVersion) + .detail("VersionsFetched", versionsFetched) + .detail("Existing", existing); return Void(); } } @@ -3948,6 +4061,12 @@ ACTOR Future fetchChangeFeedApplier(StorageServer* data, waitNext(feedResults->mutations.getFuture()); state int remoteLoc = 0; while (remoteLoc < remoteResult.size()) { + if (firstVersion == invalidVersion) { + firstVersion = remoteResult[remoteLoc].version; + } + lastVersion = remoteResult[remoteLoc].version; + versionsFetched++; + if (remoteResult[remoteLoc].version < localResult.version) { if (remoteResult[remoteLoc].mutations.size()) { if (MUTATION_TRACKING_ENABLED) { @@ -4008,6 +4127,16 @@ ACTOR Future fetchChangeFeedApplier(StorageServer* data, throw; } } + + // TODO REMOVE? + TraceEvent(SevDebug, "FetchChangeFeedDone", data->thisServerID) + .detail("RangeID", rangeId.printable()) + .detail("Range", range.toString()) + .detail("FetchVersion", fetchVersion) + .detail("FirstFetchedVersion", firstVersion) + .detail("LastFetchedVersion", lastVersion) + .detail("VersionsFetched", versionsFetched) + .detail("Existing", existing); return Void(); } @@ -4681,6 +4810,7 @@ void changeServerKeys(StorageServer* data, } validate(data); + // find any change feeds that no longer have shards on this server, and clean them up if (!nowAssigned) { std::map candidateFeeds; auto ranges = data->keyChangeFeed.intersectingRanges(keys); @@ -4700,14 +4830,11 @@ void changeServerKeys(StorageServer* data, } if (!foundAssigned) { - Key beginClearKey = f.first.withPrefix(persistChangeFeedKeys.begin); - auto& mLV = data->addVersionToMutationLog(data->data().getLatestVersion()); - data->addMutationToMutationLog( - mLV, MutationRef(MutationRef::ClearRange, beginClearKey, keyAfter(beginClearKey))); - data->addMutationToMutationLog(mLV, - MutationRef(MutationRef::ClearRange, - changeFeedDurableKey(f.first, 0), - changeFeedDurableKey(f.first, version))); + // TODO REMOVE + TraceEvent(SevDebug, "ChangeFeedCleanup", data->thisServerID) + .detail("FeedID", f.first) + .detail("Version", version); + auto rs = data->keyChangeFeed.modify(f.second); for (auto r = rs.begin(); r != rs.end(); ++r) { auto& feedList = r->value(); @@ -4720,13 +4847,21 @@ void changeServerKeys(StorageServer* data, data->keyChangeFeed.coalesce(f.second.contents()); auto feed = data->uidChangeFeed.find(f.first); if (feed != data->uidChangeFeed.end()) { - feed->second->moved(); feed->second->removing = true; + feed->second->moved(); feed->second->newMutations.trigger(); data->uidChangeFeed.erase(feed); } + + Key beginClearKey = f.first.withPrefix(persistChangeFeedKeys.begin); + + // all fetching actors should be cancelled by now because removing=true and moved(), so it's safe to + // clear storage directly + data->storage.clearRange(KeyRangeRef(beginClearKey, keyAfter(beginClearKey))); + data->storage.clearRange( + KeyRangeRef(changeFeedDurableKey(f.first, 0), changeFeedDurableKey(f.first, version))); } else { - // if just part of feed is moved away, + // if just part of feed's range is moved away auto feed = data->uidChangeFeed.find(f.first); if (feed != data->uidChangeFeed.end()) { feed->second->moved(); From 62acbcfe190aee61c3486510c8d0a6118385cec2 Mon Sep 17 00:00:00 2001 From: Josh Slocum Date: Thu, 20 Jan 2022 11:43:34 -0600 Subject: [PATCH 148/413] Added explicit error for old blob manager instead of attaching it to response objects --- fdbclient/BlobWorkerInterface.h | 17 ++------- fdbserver/BlobManager.actor.cpp | 62 +++++++++++++++++++++++---------- fdbserver/BlobWorker.actor.cpp | 18 ++++++---- flow/error_definitions.h | 1 + 4 files changed, 58 insertions(+), 40 deletions(-) diff --git a/fdbclient/BlobWorkerInterface.h b/fdbclient/BlobWorkerInterface.h index c40a64ee28..480883b263 100644 --- a/fdbclient/BlobWorkerInterface.h +++ b/fdbclient/BlobWorkerInterface.h @@ -94,19 +94,6 @@ struct BlobGranuleFileRequest { } }; -struct AssignBlobRangeReply { - constexpr static FileIdentifier file_identifier = 6431923; - bool epochOk; // false if the worker has seen a new manager - - AssignBlobRangeReply() {} - explicit AssignBlobRangeReply(bool epochOk) : epochOk(epochOk) {} - - template - void serialize(Ar& ar) { - serializer(ar, epochOk); - } -}; - struct RevokeBlobRangeRequest { constexpr static FileIdentifier file_identifier = 4844288; Arena arena; @@ -114,7 +101,7 @@ struct RevokeBlobRangeRequest { int64_t managerEpoch; int64_t managerSeqno; bool dispose; - ReplyPromise reply; + ReplyPromise reply; RevokeBlobRangeRequest() {} @@ -142,7 +129,7 @@ struct AssignBlobRangeRequest { AssignRequestType type; - ReplyPromise reply; + ReplyPromise reply; AssignBlobRangeRequest() {} diff --git a/fdbserver/BlobManager.actor.cpp b/fdbserver/BlobManager.actor.cpp index c4a7b66262..5750965068 100644 --- a/fdbserver/BlobManager.actor.cpp +++ b/fdbserver/BlobManager.actor.cpp @@ -341,7 +341,6 @@ ACTOR Future doRangeAssignment(Reference bmData, } try { - state AssignBlobRangeReply rep; if (assignment.isAssign) { ASSERT(assignment.assign.present()); ASSERT(!assignment.revoke.present()); @@ -357,8 +356,7 @@ ACTOR Future doRangeAssignment(Reference bmData, if (bmData->workersById.count(workerID) == 0) { throw no_more_servers(); } - AssignBlobRangeReply _rep = wait(bmData->workersById[workerID].assignBlobRangeRequest.getReply(req)); - rep = _rep; + wait(bmData->workersById[workerID].assignBlobRangeRequest.getReply(req)); } else { ASSERT(!assignment.assign.present()); ASSERT(assignment.revoke.present()); @@ -372,24 +370,21 @@ ACTOR Future doRangeAssignment(Reference bmData, // if that worker isn't alive anymore, this is a noop if (bmData->workersById.count(workerID)) { - AssignBlobRangeReply _rep = wait(bmData->workersById[workerID].revokeBlobRangeRequest.getReply(req)); - rep = _rep; + wait(bmData->workersById[workerID].revokeBlobRangeRequest.getReply(req)); } else { return Void(); } } - if (!rep.epochOk) { - if (BM_DEBUG) { - printf("BM heard from BW that there is a new manager with higher epoch\n"); - } - if (bmData->iAmReplaced.canBeSet()) { - bmData->iAmReplaced.send(Void()); - } - } } catch (Error& e) { if (e.code() == error_code_operation_cancelled) { throw; } + if (e.code() == error_code_blob_manager_replaced) { + if (bmData->iAmReplaced.canBeSet()) { + bmData->iAmReplaced.send(Void()); + } + return Void(); + } if (e.code() == error_code_granule_assignment_conflict) { // Another blob worker already owns the range, don't retry. // And, if it was us that send the request to another worker for this range, this actor should have been @@ -565,7 +560,7 @@ ACTOR Future checkManagerLock(Reference tr, Ref bmData->iAmReplaced.send(Void()); } - throw granule_assignment_conflict(); + throw blob_manager_replaced(); } tr->addReadConflictRange(singleKeyRange(blobManagerEpochKey)); @@ -973,6 +968,31 @@ ACTOR Future deregisterBlobWorker(Reference bmData, BlobW } } +ACTOR Future haltBlobWorker(Reference bmData, BlobWorkerInterface bwInterf) { + loop { + try { + wait(bwInterf.haltBlobWorker.getReply(HaltBlobWorkerRequest(bmData->epoch, bmData->id))); + return Void(); + } catch (Error& e) { + // throw other errors instead of returning? + if (e.code() == error_code_operation_cancelled) { + throw; + } + // TODO REMOVE + fmt::print("BM {0} got error {1} trying to halt blob worker {2}\n", + bmData->epoch, + e.name(), + bwInterf.id().toString()); + if (e.code() != error_code_blob_manager_replaced) { + return Void(); + } + if (bmData->iAmReplaced.canBeSet()) { + bmData->iAmReplaced.send(Void()); + } + } + } +} + ACTOR Future killBlobWorker(Reference bmData, BlobWorkerInterface bwInterf, bool registered) { state UID bwId = bwInterf.id(); @@ -1028,8 +1048,7 @@ ACTOR Future killBlobWorker(Reference bmData, BlobWorkerI if (BM_DEBUG) { fmt::print("Sending halt to BW {}\n", bwId.toString()); } - bmData->addActor.send( - brokenPromiseToNever(bwInterf.haltBlobWorker.getReply(HaltBlobWorkerRequest(bmData->epoch, bmData->id)))); + bmData->addActor.send(haltBlobWorker(bmData, bwInterf)); wait(deregister); @@ -1121,6 +1140,14 @@ ACTOR Future monitorBlobWorkerStatus(Reference bmData, Bl throw e; } + // if manager is replaced, die + if (e.code() == error_code_blob_manager_replaced) { + if (bmData->iAmReplaced.canBeSet()) { + bmData->iAmReplaced.send(Void()); + } + return Void(); + } + // if we got an error constructing or reading from stream that is retryable, wait and retry. // Sometimes we get connection_failed without the failure monitor tripping. One example is simulation's // rollRandomClose. In this case, just reconstruct the stream. If it was a transient failure, it works, and @@ -1766,8 +1793,7 @@ ACTOR Future haltBlobGranules(Reference bmData) { std::vector> deregisterBlobWorkers; for (auto& worker : blobWorkers) { // TODO: send a special req to blob workers so they clean up granules/CFs - bmData->addActor.send( - brokenPromiseToNever(worker.haltBlobWorker.getReply(HaltBlobWorkerRequest(bmData->epoch, bmData->id)))); + bmData->addActor.send(haltBlobWorker(bmData, worker)); deregisterBlobWorkers.emplace_back(deregisterBlobWorker(bmData, worker)); } waitForAll(deregisterBlobWorkers); diff --git a/fdbserver/BlobWorker.actor.cpp b/fdbserver/BlobWorker.actor.cpp index a9488a5594..5e049265f0 100644 --- a/fdbserver/BlobWorker.actor.cpp +++ b/fdbserver/BlobWorker.actor.cpp @@ -2774,7 +2774,7 @@ ACTOR Future handleRangeAssign(Reference bwData, } if (!isSelfReassign) { ASSERT(!req.reply.isSet()); - req.reply.send(AssignBlobRangeReply(true)); + req.reply.send(Void()); } return Void(); } catch (Error& e) { @@ -2807,9 +2807,9 @@ ACTOR Future handleRangeAssign(Reference bwData, ACTOR Future handleRangeRevoke(Reference bwData, RevokeBlobRangeRequest req) { try { - bool _shouldStart = - wait(changeBlobRange(bwData, req.keyRange, req.managerEpoch, req.managerSeqno, false, req.dispose, false)); - req.reply.send(AssignBlobRangeReply(true)); + wait(success( + changeBlobRange(bwData, req.keyRange, req.managerEpoch, req.managerSeqno, false, req.dispose, false))); + req.reply.send(Void()); return Void(); } catch (Error& e) { // FIXME: retry on error if dispose fails? @@ -2952,6 +2952,8 @@ ACTOR Future blobWorker(BlobWorkerInterface bwInterf, // TODO: pick a reasonable byte limit instead of just piggy-backing req.reply.setByteLimit(SERVER_KNOBS->RANGESTREAM_LIMIT_BYTES); self->currentManagerStatusStream.set(req.reply); + } else { + req.reply.sendError(blob_manager_replaced()); } } when(AssignBlobRangeRequest _req = waitNext(bwInterf.assignBlobRangeRequest.getFuture())) { @@ -2970,7 +2972,7 @@ ACTOR Future blobWorker(BlobWorkerInterface bwInterf, if (self->managerEpochOk(assignReq.managerEpoch)) { self->addActor.send(handleRangeAssign(self, assignReq, false)); } else { - assignReq.reply.send(AssignBlobRangeReply(false)); + assignReq.reply.sendError(blob_manager_replaced()); } } when(RevokeBlobRangeRequest _req = waitNext(bwInterf.revokeBlobRangeRequest.getFuture())) { @@ -2988,14 +2990,13 @@ ACTOR Future blobWorker(BlobWorkerInterface bwInterf, if (self->managerEpochOk(revokeReq.managerEpoch)) { self->addActor.send(handleRangeRevoke(self, revokeReq)); } else { - revokeReq.reply.send(AssignBlobRangeReply(false)); + revokeReq.reply.sendError(blob_manager_replaced()); } } when(AssignBlobRangeRequest granuleToReassign = waitNext(self->granuleUpdateErrors.getFuture())) { self->addActor.send(handleRangeAssign(self, granuleToReassign, true)); } when(HaltBlobWorkerRequest req = waitNext(bwInterf.haltBlobWorker.getFuture())) { - req.reply.send(Void()); if (self->managerEpochOk(req.managerEpoch)) { TraceEvent("BlobWorkerHalted", self->id) .detail("ReqID", req.requesterID) @@ -3003,7 +3004,10 @@ ACTOR Future blobWorker(BlobWorkerInterface bwInterf, if (BW_DEBUG) { fmt::print("BW {0} was halted by manager {1}\n", bwInterf.id().toString(), req.managerEpoch); } + req.reply.send(Void()); break; + } else { + req.reply.sendError(blob_manager_replaced()); } } when(wait(collection)) { diff --git a/flow/error_definitions.h b/flow/error_definitions.h index 9b5bffeba9..41d69a0573 100755 --- a/flow/error_definitions.h +++ b/flow/error_definitions.h @@ -85,6 +85,7 @@ ERROR( granule_assignment_conflict, 1061, "Conflicting attempts to assign blob g ERROR( change_feed_cancelled, 1062, "Change feed was cancelled" ) ERROR( blob_granule_file_load_error, 1063, "Error loading a blob file during granule materialization" ) ERROR( blob_granule_transaction_too_old, 1064, "Read version is older than blob granule history supports" ) +ERROR( blob_manager_replaced, 1065, "This blob manager has been replaced." ) ERROR( broken_promise, 1100, "Broken promise" ) ERROR( operation_cancelled, 1101, "Asynchronous operation cancelled" ) From 0f6bced510afcb2826e5d26268f3067eb1a6809d Mon Sep 17 00:00:00 2001 From: Josh Slocum Date: Thu, 20 Jan 2022 16:28:08 -0600 Subject: [PATCH 149/413] Fixing assert in blob manager failure --- fdbserver/BlobManager.actor.cpp | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/fdbserver/BlobManager.actor.cpp b/fdbserver/BlobManager.actor.cpp index 5750965068..27373c43a7 100644 --- a/fdbserver/BlobManager.actor.cpp +++ b/fdbserver/BlobManager.actor.cpp @@ -418,6 +418,7 @@ ACTOR Future doRangeAssignment(Reference bmData, } return Void(); }*/ + // TODO confirm: using reliable delivery this should only trigger if the worker is marked as failed, right? // So assignment needs to be retried elsewhere, and a revoke is trivially complete if (assignment.isAssign) { @@ -427,6 +428,7 @@ ACTOR Future doRangeAssignment(Reference bmData, assignment.keyRange.end.printable(), workerID.toString()); } + // re-send revoke to queue to handle range being un-assigned from that worker before the new one RangeAssignment revokeOld; revokeOld.isAssign = false; @@ -1184,8 +1186,8 @@ ACTOR Future monitorBlobWorker(Reference bmData, BlobWork TraceEvent("BlobWorkerFailed", bmData->id).detail("BlobWorkerID", bwInterf.id()); } when(wait(monitorStatus)) { - ASSERT(false); - throw internal_error(); + // should only return when manager got replaced + ASSERT(!bmData->iAmReplaced.canBeSet()); } } } catch (Error& e) { From 951b28498b6642220328973cdf0cfb04b3c5f813 Mon Sep 17 00:00:00 2001 From: Josh Slocum Date: Thu, 20 Jan 2022 16:45:13 -0600 Subject: [PATCH 150/413] Fixed a couple issues with manager recovery and granule locks --- fdbserver/BlobManager.actor.cpp | 59 ++++++++++++++------------------- fdbserver/BlobWorker.actor.cpp | 16 +++++++-- 2 files changed, 38 insertions(+), 37 deletions(-) diff --git a/fdbserver/BlobManager.actor.cpp b/fdbserver/BlobManager.actor.cpp index 27373c43a7..e33f672938 100644 --- a/fdbserver/BlobManager.actor.cpp +++ b/fdbserver/BlobManager.actor.cpp @@ -403,27 +403,12 @@ ACTOR Future doRangeAssignment(Reference bmData, return Void(); } - // TODO: i think this is no longer necessary with the cancelling and actor map, but keep it around just in case - // for a bit If the worker is no longer present, the ranges were already moved off by that function, so don't - // retry. If the request is a reassign though, we need to retry it since the worker it was on is now dead and - // nobody owns it - /*if (!bmData->workersById.count(workerID) && - (!assignment.assign.present() || assignment.assign.get().type != AssignRequestType::Reassign)) { - if (BM_DEBUG) { - fmt::print("BM {0} got error assigning range [{1} - {2}) to now dead worker {3}, ignoring\n", - bmData->epoch, - assignment.keyRange.begin.printable(), - assignment.keyRange.end.printable(), - workerID.toString()); - } - return Void(); - }*/ - // TODO confirm: using reliable delivery this should only trigger if the worker is marked as failed, right? // So assignment needs to be retried elsewhere, and a revoke is trivially complete if (assignment.isAssign) { if (BM_DEBUG) { - fmt::print("BM got error assigning range [{0} - {1}) to worker {2}, requeueing\n", + fmt::print("BM got error {0} assigning range [{1} - {2}) to worker {3}, requeueing\n", + e.name(), assignment.keyRange.begin.printable(), assignment.keyRange.end.printable(), workerID.toString()); @@ -438,6 +423,10 @@ ACTOR Future doRangeAssignment(Reference bmData, bmData->rangesToAssign.send(revokeOld); // send assignment back to queue as is, clearing designated worker if present + // if we failed to send continue or reassign to the worker we thought owned the shard, it should be retried + // as a normal assign + ASSERT(assignment.assign.present()); + assignment.assign.get().type = AssignRequestType::Normal; assignment.worker.reset(); bmData->rangesToAssign.send(assignment); // FIXME: improvement would be to add history of failed workers to assignment so it can try other ones first @@ -970,11 +959,13 @@ ACTOR Future deregisterBlobWorker(Reference bmData, BlobW } } -ACTOR Future haltBlobWorker(Reference bmData, BlobWorkerInterface bwInterf) { +ACTOR Future haltBlobWorker(Reference bmData, + BlobWorkerInterface bwInterf, + bool removeFromDead) { loop { try { wait(bwInterf.haltBlobWorker.getReply(HaltBlobWorkerRequest(bmData->epoch, bmData->id))); - return Void(); + break; } catch (Error& e) { // throw other errors instead of returning? if (e.code() == error_code_operation_cancelled) { @@ -986,13 +977,25 @@ ACTOR Future haltBlobWorker(Reference bmData, BlobWorkerI e.name(), bwInterf.id().toString()); if (e.code() != error_code_blob_manager_replaced) { - return Void(); + break; } if (bmData->iAmReplaced.canBeSet()) { bmData->iAmReplaced.send(Void()); } } } + + // Remove blob worker from persisted list of blob workers + Future deregister = deregisterBlobWorker(bmData, bwInterf); + + // restart recruiting to replace the dead blob worker + bmData->restartRecruiting.trigger(); + + if (removeFromDead) { + bmData->deadWorkers.erase(bwInterf.id()); + } + + return Void(); } ACTOR Future killBlobWorker(Reference bmData, BlobWorkerInterface bwInterf, bool registered) { @@ -1010,12 +1013,6 @@ ACTOR Future killBlobWorker(Reference bmData, BlobWorkerI bmData->workerAddresses.erase(bwInterf.stableAddress()); } - // Remove blob worker from persisted list of blob workers - Future deregister = deregisterBlobWorker(bmData, bwInterf); - - // restart recruiting to replace the dead blob worker - bmData->restartRecruiting.trigger(); - // for every range owned by this blob worker, we want to // - send a revoke request for that range // - add the range back to the stream of ranges to be assigned @@ -1050,13 +1047,7 @@ ACTOR Future killBlobWorker(Reference bmData, BlobWorkerI if (BM_DEBUG) { fmt::print("Sending halt to BW {}\n", bwId.toString()); } - bmData->addActor.send(haltBlobWorker(bmData, bwInterf)); - - wait(deregister); - - if (registered) { - bmData->deadWorkers.erase(bwId); - } + bmData->addActor.send(haltBlobWorker(bmData, bwInterf, registered)); return Void(); } @@ -1795,7 +1786,7 @@ ACTOR Future haltBlobGranules(Reference bmData) { std::vector> deregisterBlobWorkers; for (auto& worker : blobWorkers) { // TODO: send a special req to blob workers so they clean up granules/CFs - bmData->addActor.send(haltBlobWorker(bmData, worker)); + bmData->addActor.send(haltBlobWorker(bmData, worker, false)); deregisterBlobWorkers.emplace_back(deregisterBlobWorker(bmData, worker)); } waitForAll(deregisterBlobWorkers); diff --git a/fdbserver/BlobWorker.actor.cpp b/fdbserver/BlobWorker.actor.cpp index 5e049265f0..a5657d8765 100644 --- a/fdbserver/BlobWorker.actor.cpp +++ b/fdbserver/BlobWorker.actor.cpp @@ -1775,6 +1775,12 @@ ACTOR Future blobGranuleUpdateFiles(Reference bwData, f.future.cancel(); } + // if we retry and re-open, we need to use a normal request (no continue or reassign) and update the + // seqno + metadata->originalReq.managerEpoch = metadata->continueEpoch; + metadata->originalReq.managerSeqno = metadata->continueSeqno; + metadata->originalReq.type = AssignRequestType::Normal; + bwData->granuleUpdateErrors.send(metadata->originalReq); } } @@ -2602,12 +2608,16 @@ ACTOR Future changeBlobRange(Reference bwData, fmt::print("thisAssignmentNewer={}\n", thisAssignmentNewer ? "true" : "false"); } - // if this granule already has it, and this was a specialassignment (i.e. a new blob maanger is + // if this granule already has it, and this was a special assignment (i.e. a new blob manager is // trying to reassign granules), then just continue - if (active && assignType.get() == AssignRequestType::Reassign && r.begin() == keyRange.begin && - r.end() == keyRange.end) { + + // TODO this needs to also have the condition + if (active && assignType.get() == AssignRequestType::Reassign && r.value().activeMetadata.isValid() && + r.begin() == keyRange.begin && r.end() == keyRange.end) { r.value().lastEpoch = epoch; r.value().lastSeqno = seqno; + r.value().activeMetadata->continueEpoch = epoch; + r.value().activeMetadata->continueSeqno = seqno; alreadyAssigned = true; break; } From a7a6e0ee9cc17bb5ca62fae4592e2cc7a49b1a69 Mon Sep 17 00:00:00 2001 From: Josh Slocum Date: Thu, 20 Jan 2022 18:37:25 -0600 Subject: [PATCH 151/413] More blob worker failure fixes --- fdbserver/BlobManager.actor.cpp | 27 ++++++++++++++------------- 1 file changed, 14 insertions(+), 13 deletions(-) diff --git a/fdbserver/BlobManager.actor.cpp b/fdbserver/BlobManager.actor.cpp index e33f672938..61de2a23f2 100644 --- a/fdbserver/BlobManager.actor.cpp +++ b/fdbserver/BlobManager.actor.cpp @@ -940,6 +940,7 @@ ACTOR Future deregisterBlobWorker(Reference bmData, BlobW tr->setOption(FDBTransactionOptions::ACCESS_SYSTEM_KEYS); tr->setOption(FDBTransactionOptions::PRIORITY_SYSTEM_IMMEDIATE); try { + checkManagerLock(tr, bmData); Key blobWorkerListKey = blobWorkerListKeyFor(interf.id()); tr->addReadConflictRange(singleKeyRange(blobWorkerListKey)); tr->clear(blobWorkerListKey); @@ -959,9 +960,7 @@ ACTOR Future deregisterBlobWorker(Reference bmData, BlobW } } -ACTOR Future haltBlobWorker(Reference bmData, - BlobWorkerInterface bwInterf, - bool removeFromDead) { +ACTOR Future haltBlobWorker(Reference bmData, BlobWorkerInterface bwInterf) { loop { try { wait(bwInterf.haltBlobWorker.getReply(HaltBlobWorkerRequest(bmData->epoch, bmData->id))); @@ -985,16 +984,6 @@ ACTOR Future haltBlobWorker(Reference bmData, } } - // Remove blob worker from persisted list of blob workers - Future deregister = deregisterBlobWorker(bmData, bwInterf); - - // restart recruiting to replace the dead blob worker - bmData->restartRecruiting.trigger(); - - if (removeFromDead) { - bmData->deadWorkers.erase(bwInterf.id()); - } - return Void(); } @@ -1013,6 +1002,9 @@ ACTOR Future killBlobWorker(Reference bmData, BlobWorkerI bmData->workerAddresses.erase(bwInterf.stableAddress()); } + // Remove blob worker from persisted list of blob workers + Future deregister = deregisterBlobWorker(bmData, bwInterf); + // for every range owned by this blob worker, we want to // - send a revoke request for that range // - add the range back to the stream of ranges to be assigned @@ -1049,6 +1041,15 @@ ACTOR Future killBlobWorker(Reference bmData, BlobWorkerI } bmData->addActor.send(haltBlobWorker(bmData, bwInterf, registered)); + wait(deregister); + + // restart recruiting to replace the dead blob worker + bmData->restartRecruiting.trigger(); + + if (registered) { + bmData->deadWorkers.erase(bwInterf.id()); + } + return Void(); } From a8f1d6fa0062d50536252073228dd6723cdb045c Mon Sep 17 00:00:00 2001 From: Josh Slocum Date: Thu, 20 Jan 2022 19:20:11 -0600 Subject: [PATCH 152/413] StorageMetrics and and halting fixes --- fdbclient/NativeAPI.actor.cpp | 14 +++++++++----- fdbclient/NativeAPI.actor.h | 3 ++- fdbserver/BlobManager.actor.cpp | 29 ++++++++++++++++++++++------- 3 files changed, 33 insertions(+), 13 deletions(-) diff --git a/fdbclient/NativeAPI.actor.cpp b/fdbclient/NativeAPI.actor.cpp index 6377a09d90..9d5fb8ab62 100644 --- a/fdbclient/NativeAPI.actor.cpp +++ b/fdbclient/NativeAPI.actor.cpp @@ -6775,7 +6775,8 @@ ACTOR Future setPerpetualStorageWiggle(Database cx, bool enable, LockAware ACTOR Future>> splitStorageMetrics(Database cx, KeyRange keys, StorageMetrics limit, - StorageMetrics estimated) { + StorageMetrics estimated, + bool allowPartial) { state Span span("NAPI:SplitStorageMetrics"_loc); loop { state std::vector>> locations = @@ -6790,7 +6791,7 @@ ACTOR Future>> splitStorageMetrics(Database cx, // 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) { + if (locations.size() == CLIENT_KNOBS->STORAGE_METRICS_SHARD_LIMIT && !allowPartial) { wait(delay(CLIENT_KNOBS->STORAGE_METRICS_TOO_MANY_SHARDS_DELAY, TaskPriority::DataDistribution)); cx->invalidateCache(keys); } else { @@ -6826,7 +6827,9 @@ ACTOR Future>> splitStorageMetrics(Database cx, results.resize(results.arena(), results.size() - 1); } - results.push_back_deep(results.arena(), keys.end); + if (!allowPartial || keys.end <= locations.back().first.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) { @@ -6842,8 +6845,9 @@ ACTOR Future>> splitStorageMetrics(Database cx, Future>> Transaction::splitStorageMetrics(KeyRange const& keys, StorageMetrics const& limit, - StorageMetrics const& estimated) { - return ::splitStorageMetrics(cx, keys, limit, estimated); + StorageMetrics const& estimated, + bool allowPartial) { + return ::splitStorageMetrics(cx, keys, limit, estimated, allowPartial); } void Transaction::checkDeferredError() const { diff --git a/fdbclient/NativeAPI.actor.h b/fdbclient/NativeAPI.actor.h index c713d5744b..e38fbc5421 100644 --- a/fdbclient/NativeAPI.actor.h +++ b/fdbclient/NativeAPI.actor.h @@ -366,7 +366,8 @@ public: Future getStorageMetrics(KeyRange const& keys, int shardLimit); Future>> splitStorageMetrics(KeyRange const& keys, StorageMetrics const& limit, - StorageMetrics const& estimated); + StorageMetrics const& estimated, + bool allowPartial = false); Future>> getReadHotRanges(KeyRange const& keys); // Try to split the given range into equally sized chunks based on estimated size. diff --git a/fdbserver/BlobManager.actor.cpp b/fdbserver/BlobManager.actor.cpp index 61de2a23f2..59ca8776ae 100644 --- a/fdbserver/BlobManager.actor.cpp +++ b/fdbserver/BlobManager.actor.cpp @@ -248,7 +248,8 @@ ACTOR Future>> splitRange(ReferencegetTransaction().getStorageMetrics(range, CLIENT_KNOBS->TOO_MANY)); + state StorageMetrics estimated = + wait(tr->getTransaction().getStorageMetrics(range, CLIENT_KNOBS->TOO_MANY)); if (BM_DEBUG) { fmt::print("Estimated bytes for [{0} - {1}): {2}\n", @@ -260,14 +261,28 @@ ACTOR Future>> splitRange(Reference SERVER_KNOBS->BG_SNAPSHOT_FILE_TARGET_BYTES) { // printf(" Splitting range\n"); // only split on bytes - StorageMetrics splitMetrics; + state Standalone> keys; + state StorageMetrics splitMetrics; splitMetrics.bytes = SERVER_KNOBS->BG_SNAPSHOT_FILE_TARGET_BYTES; splitMetrics.bytesPerKSecond = splitMetrics.infinity; splitMetrics.iosPerKSecond = splitMetrics.infinity; splitMetrics.bytesReadPerKSecond = splitMetrics.infinity; // Don't split by readBandwidth - Standalone> keys = - wait(tr->getTransaction().splitStorageMetrics(range, splitMetrics, estimated)); + while (keys.empty() || keys.back() < range.end) { + // allow partial in case we have a large split + Standalone> newKeys = + wait(tr->getTransaction().splitStorageMetrics(range, splitMetrics, estimated, true)); + ASSERT(!newKeys.empty()); + if (keys.empty()) { + keys = newKeys; + } else { + TEST(true); // large split that requires multiple rounds + // start key was repeated with last request, so don't include it + ASSERT(newKeys[0] == keys.back()); + keys.append_deep(keys.arena(), newKeys.begin() + 1, newKeys.size() - 1); + } + range = KeyRangeRef(keys.back(), range.end); + } ASSERT(keys.size() >= 2); return keys; } else { @@ -940,7 +955,7 @@ ACTOR Future deregisterBlobWorker(Reference bmData, BlobW tr->setOption(FDBTransactionOptions::ACCESS_SYSTEM_KEYS); tr->setOption(FDBTransactionOptions::PRIORITY_SYSTEM_IMMEDIATE); try { - checkManagerLock(tr, bmData); + wait(checkManagerLock(tr, bmData)); Key blobWorkerListKey = blobWorkerListKeyFor(interf.id()); tr->addReadConflictRange(singleKeyRange(blobWorkerListKey)); tr->clear(blobWorkerListKey); @@ -1039,7 +1054,7 @@ ACTOR Future killBlobWorker(Reference bmData, BlobWorkerI if (BM_DEBUG) { fmt::print("Sending halt to BW {}\n", bwId.toString()); } - bmData->addActor.send(haltBlobWorker(bmData, bwInterf, registered)); + bmData->addActor.send(haltBlobWorker(bmData, bwInterf)); wait(deregister); @@ -1787,7 +1802,7 @@ ACTOR Future haltBlobGranules(Reference bmData) { std::vector> deregisterBlobWorkers; for (auto& worker : blobWorkers) { // TODO: send a special req to blob workers so they clean up granules/CFs - bmData->addActor.send(haltBlobWorker(bmData, worker, false)); + bmData->addActor.send(haltBlobWorker(bmData, worker)); deregisterBlobWorkers.emplace_back(deregisterBlobWorker(bmData, worker)); } waitForAll(deregisterBlobWorkers); From be98fc9a58c6f4c03f9c0463004a52d164bb7a32 Mon Sep 17 00:00:00 2001 From: Josh Slocum Date: Fri, 21 Jan 2022 08:28:36 -0600 Subject: [PATCH 153/413] Fixed incorrect assert in blob manager --- fdbserver/BlobManager.actor.cpp | 30 +++++++++++++++++++++++++++--- 1 file changed, 27 insertions(+), 3 deletions(-) diff --git a/fdbserver/BlobManager.actor.cpp b/fdbserver/BlobManager.actor.cpp index 59ca8776ae..601da224ab 100644 --- a/fdbserver/BlobManager.actor.cpp +++ b/fdbserver/BlobManager.actor.cpp @@ -861,13 +861,37 @@ ACTOR Future maybeSplitRange(Reference bmData, } return Void(); } + int64_t ownerEpoch = std::get<0>(prevGranuleLock); + int64_t ownerSeqno = std::get<1>(prevGranuleLock); if (newLockSeqno == -1) { newLockSeqno = bmData->seqNo; bmData->seqNo++; - ASSERT(newLockSeqno > std::get<1>(prevGranuleLock)); + if (!(bmData->epoch > ownerEpoch || (bmData->epoch == ownerEpoch && newLockSeqno > ownerSeqno))) { + printf("BM seqno for granule [%s - %s) out of order for lock! manager: (%lld, %lld), owner: %lld, " + "%lld)\n", + granuleRange.begin.printable().c_str(), + granuleRange.end.printable().c_str(), + bmData->epoch, + newLockSeqno, + ownerEpoch, + ownerSeqno); + } + ASSERT(bmData->epoch > ownerEpoch || (bmData->epoch == ownerEpoch && newLockSeqno > ownerSeqno)); } else { - // previous transaction could have succeeded but got commit_unknown_result - ASSERT(newLockSeqno >= std::get<1>(prevGranuleLock)); + if (!(bmData->epoch > ownerEpoch || (bmData->epoch == ownerEpoch && newLockSeqno >= ownerSeqno))) { + printf("BM seqno for granule [%s - %s) out of order for lock on retry! manager: (%lld, %lld), " + "owner: %lld, " + "%lld)\n", + granuleRange.begin.printable().c_str(), + granuleRange.end.printable().c_str(), + bmData->epoch, + newLockSeqno, + ownerEpoch, + ownerSeqno); + } + // previous transaction could have succeeded but got commit_unknown_result, so use >= instead of > for + // seqno if epochs are equal + ASSERT(bmData->epoch > ownerEpoch || (bmData->epoch == ownerEpoch && newLockSeqno >= ownerSeqno)); } // acquire granule lock so nobody else can make changes to this granule. From 269dfb79cf84e3f61aa521fbd152ca26dc239a20 Mon Sep 17 00:00:00 2001 From: Josh Slocum Date: Fri, 21 Jan 2022 10:29:54 -0600 Subject: [PATCH 154/413] Fixes to checkBlobWorkerList --- fdbserver/BlobManager.actor.cpp | 63 ++++++++++++++++++++++----------- 1 file changed, 43 insertions(+), 20 deletions(-) diff --git a/fdbserver/BlobManager.actor.cpp b/fdbserver/BlobManager.actor.cpp index 601da224ab..d16dc0bd12 100644 --- a/fdbserver/BlobManager.actor.cpp +++ b/fdbserver/BlobManager.actor.cpp @@ -218,6 +218,7 @@ struct BlobManagerData : NonCopyable, ReferenceCounted { Debouncer restartRecruiting; std::set recruitingLocalities; // the addrs of the workers being recruited on AsyncVar recruitingStream; + Promise foundBlobWorkers; int64_t epoch = -1; int64_t seqNo = 1; @@ -308,7 +309,7 @@ ACTOR Future pickWorkerForAssign(Reference bmData) { fmt::print("BM {0} waiting for blob workers before assigning granules\n", bmData->epoch); } bmData->restartRecruiting.trigger(); - wait(bmData->recruitingStream.onChange()); + wait(bmData->recruitingStream.onChange() || bmData->foundBlobWorkers.getFuture()); } int minGranulesAssigned = INT_MAX; @@ -1080,7 +1081,12 @@ ACTOR Future killBlobWorker(Reference bmData, BlobWorkerI } bmData->addActor.send(haltBlobWorker(bmData, bwInterf)); - wait(deregister); + // wait for blob worker to be removed from DB and in-memory mapping to have reassigned all shards from this worker + // before removing it from deadWorkers, to avoid a race with checkBlobWorkerList + wait(deregister && bmData->rangesToAssign.onEmpty()); + // delay(0) after onEmpty to yield back to the range assigner on the final pop to ensure it gets processed before + // deadWorkers.erase + wait(delay(0)); // restart recruiting to replace the dead blob worker bmData->restartRecruiting.trigger(); @@ -1254,28 +1260,45 @@ ACTOR Future monitorBlobWorker(Reference bmData, BlobWork } ACTOR Future checkBlobWorkerList(Reference bmData, Promise workerListReady) { - loop { - // Get list of last known blob workers - // note: the list will include every blob worker that the old manager knew about, - // but it might also contain blob workers that died while the new manager was being recruited - std::vector blobWorkers = wait(getBlobWorkers(bmData->db)); - // add all blob workers to this new blob manager's records and start monitoring it - for (auto& worker : blobWorkers) { - if (!bmData->deadWorkers.count(worker.id())) { - if (!bmData->workerAddresses.count(worker.stableAddress()) && worker.locality.dcId() == bmData->dcId) { - bmData->workerAddresses.insert(worker.stableAddress()); - bmData->workersById[worker.id()] = worker; - bmData->workerStats[worker.id()] = BlobWorkerStats(); - bmData->addActor.send(monitorBlobWorker(bmData, worker)); - } else if (!bmData->workersById.count(worker.id())) { - bmData->addActor.send(killBlobWorker(bmData, worker, false)); + + try { + loop { + // Get list of last known blob workers + // note: the list will include every blob worker that the old manager knew about, + // but it might also contain blob workers that died while the new manager was being recruited + std::vector blobWorkers = wait(getBlobWorkers(bmData->db)); + // add all blob workers to this new blob manager's records and start monitoring it + bool foundAnyNew = false; + for (auto& worker : blobWorkers) { + if (!bmData->deadWorkers.count(worker.id())) { + if (!bmData->workerAddresses.count(worker.stableAddress()) && + worker.locality.dcId() == bmData->dcId) { + bmData->workerAddresses.insert(worker.stableAddress()); + bmData->workersById[worker.id()] = worker; + bmData->workerStats[worker.id()] = BlobWorkerStats(); + bmData->addActor.send(monitorBlobWorker(bmData, worker)); + foundAnyNew = true; + } else if (!bmData->workersById.count(worker.id())) { + bmData->addActor.send(killBlobWorker(bmData, worker, false)); + } } } + if (workerListReady.canBeSet()) { + workerListReady.send(Void()); + } + // if any assigns are stuck on workers, and we have workers, wake them + if (foundAnyNew || !bmData->workersById.empty()) { + Promise hold = bmData->foundBlobWorkers; + bmData->foundBlobWorkers = Promise(); + hold.send(Void()); + } + wait(delay(SERVER_KNOBS->BLOB_WORKERLIST_FETCH_INTERVAL)); } - if (workerListReady.canBeSet()) { - workerListReady.send(Void()); + } catch (Error& e) { + if (BM_DEBUG) { + fmt::print("BM {0} got error {1} reading blob worker list!!\n", bmData->epoch, e.name()); } - wait(delay(SERVER_KNOBS->BLOB_WORKERLIST_FETCH_INTERVAL)); + throw e; } } From 558779d782a37c49d8e999d561f9daa9e66f1fb2 Mon Sep 17 00:00:00 2001 From: Josh Slocum Date: Fri, 21 Jan 2022 13:12:16 -0600 Subject: [PATCH 155/413] Fix open granule races --- fdbserver/BlobGranuleServerCommon.actor.cpp | 11 +---------- fdbserver/BlobWorker.actor.cpp | 2 ++ 2 files changed, 3 insertions(+), 10 deletions(-) diff --git a/fdbserver/BlobGranuleServerCommon.actor.cpp b/fdbserver/BlobGranuleServerCommon.actor.cpp index 87d6e12064..7700d6152b 100644 --- a/fdbserver/BlobGranuleServerCommon.actor.cpp +++ b/fdbserver/BlobGranuleServerCommon.actor.cpp @@ -30,17 +30,8 @@ // Gets the latest granule history node for range that was persisted ACTOR Future> getLatestGranuleHistory(Transaction* tr, KeyRange range) { state KeyRange historyRange = blobGranuleHistoryKeyRangeFor(range); - state RangeResult result; + state RangeResult result = wait(tr->getRange(historyRange, 1, Snapshot::False, Reverse::True)); - loop { - try { - RangeResult _result = wait(tr->getRange(historyRange, 1, Snapshot::False, Reverse::True)); - result = _result; - break; - } catch (Error& e) { - wait(tr->onError(e)); - } - } ASSERT(result.size() <= 1); Optional history; diff --git a/fdbserver/BlobWorker.actor.cpp b/fdbserver/BlobWorker.actor.cpp index a5657d8765..ec7190db8d 100644 --- a/fdbserver/BlobWorker.actor.cpp +++ b/fdbserver/BlobWorker.actor.cpp @@ -119,6 +119,8 @@ struct GranuleRangeMetadata { activeMetadata->cancelled.send(Void()); } activeMetadata.clear(); + assignFuture.cancel(); + historyLoaderFuture.cancel(); } GranuleRangeMetadata() : lastEpoch(0), lastSeqno(0) {} From 7f9bce40a75f816619307f55d5075f61a4604f3b Mon Sep 17 00:00:00 2001 From: Josh Slocum Date: Fri, 21 Jan 2022 13:22:19 -0600 Subject: [PATCH 156/413] retrying GRVs in BlobGranuleVerifier --- .../workloads/BlobGranuleVerifier.actor.cpp | 21 +++++++++++++++++-- 1 file changed, 19 insertions(+), 2 deletions(-) diff --git a/fdbserver/workloads/BlobGranuleVerifier.actor.cpp b/fdbserver/workloads/BlobGranuleVerifier.actor.cpp index 32189a6738..ccc5ef34d9 100644 --- a/fdbserver/workloads/BlobGranuleVerifier.actor.cpp +++ b/fdbserver/workloads/BlobGranuleVerifier.actor.cpp @@ -438,11 +438,28 @@ struct BlobGranuleVerifierWorkload : TestWorkload { return delay(testDuration); } + // handle retries + errors + // It's ok to reset the transaction here because its read version is only used for reading the granule mapping from + // the system keyspace + ACTOR Future doGrv(Transaction* tr) { + loop { + try { + Version readVersion = wait(tr->getReadVersion()); + return readVersion; + } catch (Error& e) { + // TODO REMOVE print + printf("BGV GRV got error %s\n", e.name()); + wait(tr->onError(e)); + } + } + } + ACTOR Future _check(Database cx, BlobGranuleVerifierWorkload* self) { // check error counts, and do an availability check at the end + // TODO need to have retry loop for getReadVersion, it's throwing tag throttled for some reason? state Transaction tr(cx); - state Version readVersion = wait(tr.getReadVersion()); + state Version readVersion = wait(self->doGrv(&tr)); state Version startReadVersion = readVersion; state int checks = 0; @@ -493,7 +510,7 @@ struct BlobGranuleVerifierWorkload : TestWorkload { if (e.code() == error_code_blob_granule_transaction_too_old) { wait(delay(1.0)); tr.reset(); - Version rv = wait(tr.getReadVersion()); + Version rv = wait(self->doGrv(&tr)); readVersion = rv; } else { wait(tr.onError(e)); From fee9e932e700a1bc86daf7d8b22bb1b1c34c8123 Mon Sep 17 00:00:00 2001 From: Evan Tschannen Date: Fri, 21 Jan 2022 12:06:00 -0800 Subject: [PATCH 157/413] fix: do not report errors when the blob verifier is actor_cancelled --- fdbserver/workloads/BlobGranuleVerifier.actor.cpp | 3 +++ 1 file changed, 3 insertions(+) diff --git a/fdbserver/workloads/BlobGranuleVerifier.actor.cpp b/fdbserver/workloads/BlobGranuleVerifier.actor.cpp index 32189a6738..b103c9b1fa 100644 --- a/fdbserver/workloads/BlobGranuleVerifier.actor.cpp +++ b/fdbserver/workloads/BlobGranuleVerifier.actor.cpp @@ -501,6 +501,9 @@ struct BlobGranuleVerifierWorkload : TestWorkload { } } } catch (Error& e) { + if (e.code() == error_code_actor_cancelled) { + throw; + } if (e.code() == error_code_end_of_stream) { break; } From 9ccffe911b140521abf9da0afbb4cdf043aae93c Mon Sep 17 00:00:00 2001 From: Josh Slocum Date: Fri, 21 Jan 2022 14:20:52 -0600 Subject: [PATCH 158/413] Added exponential backoff for re-establishing blob worker monitoring --- fdbserver/BlobManager.actor.cpp | 9 ++++++++- 1 file changed, 8 insertions(+), 1 deletion(-) diff --git a/fdbserver/BlobManager.actor.cpp b/fdbserver/BlobManager.actor.cpp index d16dc0bd12..d2ec79e9af 100644 --- a/fdbserver/BlobManager.actor.cpp +++ b/fdbserver/BlobManager.actor.cpp @@ -1101,6 +1101,10 @@ ACTOR Future killBlobWorker(Reference bmData, BlobWorkerI ACTOR Future monitorBlobWorkerStatus(Reference bmData, BlobWorkerInterface bwInterf) { state KeyRangeMap> lastSeenSeqno; // outer loop handles reconstructing stream if it got a retryable error + // do backoff, we can get a lot of retries in a row + + // TODO knob? + state double backoff = 0.1; loop { try { state ReplyPromiseStream statusStream = @@ -1119,6 +1123,8 @@ ACTOR Future monitorBlobWorkerStatus(Reference bmData, Bl bwInterf.id().toString(), rep.doSplit ? "split" : ""); } + // if we get a reply from the stream, reset backoff + backoff = 0.1; if (rep.epoch > bmData->epoch) { if (BM_DEBUG) { fmt::print("BM heard from BW {0} that there is a new manager with higher epoch\n", @@ -1193,7 +1199,8 @@ ACTOR Future monitorBlobWorkerStatus(Reference bmData, Bl // if it is permanent, the failure monitor will eventually trip. ASSERT(e.code() != error_code_end_of_stream); if (e.code() == error_code_request_maybe_delivered || e.code() == error_code_connection_failed) { - wait(delay(FLOW_KNOBS->PREVENT_FAST_SPIN_DELAY)); + wait(delay(backoff)); + backoff = std::min(backoff * 1.5, 5.0); continue; } else { if (BM_DEBUG) { From 1180eb6e44088cde429047ffcc32ed262b3dddf5 Mon Sep 17 00:00:00 2001 From: Josh Slocum Date: Fri, 21 Jan 2022 18:02:30 -0600 Subject: [PATCH 159/413] Fixed uncaught error in blob worker requests --- fdbserver/BlobWorker.actor.cpp | 23 ++++++++++++++++++----- fdbserver/storageserver.actor.cpp | 1 + 2 files changed, 19 insertions(+), 5 deletions(-) diff --git a/fdbserver/BlobWorker.actor.cpp b/fdbserver/BlobWorker.actor.cpp index ec7190db8d..3039e9b229 100644 --- a/fdbserver/BlobWorker.actor.cpp +++ b/fdbserver/BlobWorker.actor.cpp @@ -179,6 +179,11 @@ struct BlobWorkerData : NonCopyable, ReferenceCounted { NotifiedVersion grvVersion; BlobWorkerData(UID id, Database db) : id(id), db(db), stats(id, SERVER_KNOBS->WORKER_LOGGING_INTERVAL) {} + ~BlobWorkerData() { + if (BW_DEBUG) { + printf("Destroying BW %s data\n", id.toString().c_str()); + } + } bool managerEpochOk(int64_t epoch) { if (epoch < currentManagerEpoch) { @@ -2201,7 +2206,6 @@ ACTOR Future handleBlobGranuleFileRequest(Reference bwData // this is an active granule query loop { if (!metadata->activeCFData.get().isValid() || !metadata->cancelled.canBeSet()) { - throw wrong_shard_server(); } Future waitForVersionFuture = waitForVersion(metadata, req.readVersion); @@ -2211,10 +2215,19 @@ ACTOR Future handleBlobGranuleFileRequest(Reference bwData } // rollback resets all of the version information, so we have to redo wait for // version on rollback - choose { - when(wait(waitForVersionFuture)) { break; } - when(wait(metadata->activeCFData.onChange())) {} - when(wait(metadata->cancelled.getFuture())) { throw wrong_shard_server(); } + try { + choose { + when(wait(waitForVersionFuture)) { break; } + when(wait(metadata->activeCFData.onChange())) {} + when(wait(metadata->cancelled.getFuture())) { throw wrong_shard_server(); } + } + } catch (Error& e) { + // we can get change feed cancelled from whenAtLeast. This is effectively + if (e.code() != error_code_change_feed_cancelled) { + throw e; + } + // wait 1ms and try again + wait(delay(0.001)); } if ((BW_REQUEST_DEBUG || DEBUG_BW_WAIT_VERSION == req.readVersion) && metadata->activeCFData.get().isValid()) { diff --git a/fdbserver/storageserver.actor.cpp b/fdbserver/storageserver.actor.cpp index baa72ff25e..16211f0c74 100644 --- a/fdbserver/storageserver.actor.cpp +++ b/fdbserver/storageserver.actor.cpp @@ -1982,6 +1982,7 @@ ACTOR Future stopChangeFeedOnMove(StorageServer* data, ChangeFeedStreamReq } state Future moved = feed->second->onMove.getFuture(); wait(moved); + printf("CF Moved! %lld - %lld. sending WSS\n", req.begin, req.end); // DO NOT call req.reply.onReady before sending - we want to propagate this error through regardless of how far // behind client is req.reply.sendError(wrong_shard_server()); From 42a36dc7564f3a11799ed11216070692f89b6dcc Mon Sep 17 00:00:00 2001 From: Josh Slocum Date: Mon, 24 Jan 2022 09:46:14 -0600 Subject: [PATCH 160/413] Fixed Blob Manager recruitment error and Blob Worker monitoring error --- fdbserver/BlobManager.actor.cpp | 1 + fdbserver/BlobWorker.actor.cpp | 14 +++++++++++--- fdbserver/worker.actor.cpp | 10 +++++++++- 3 files changed, 21 insertions(+), 4 deletions(-) diff --git a/fdbserver/BlobManager.actor.cpp b/fdbserver/BlobManager.actor.cpp index d2ec79e9af..dcaef06398 100644 --- a/fdbserver/BlobManager.actor.cpp +++ b/fdbserver/BlobManager.actor.cpp @@ -2546,6 +2546,7 @@ ACTOR Future blobManager(BlobManagerInterface bmInterf, if (BM_DEBUG) { fmt::print("Blob manager {0} starting...\n", epoch); } + TraceEvent("BlobManagerInit", bmInterf.id()).detail("Epoch", epoch).log(); self->epoch = epoch; diff --git a/fdbserver/BlobWorker.actor.cpp b/fdbserver/BlobWorker.actor.cpp index 3039e9b229..d1a1d0df80 100644 --- a/fdbserver/BlobWorker.actor.cpp +++ b/fdbserver/BlobWorker.actor.cpp @@ -188,7 +188,7 @@ struct BlobWorkerData : NonCopyable, ReferenceCounted { bool managerEpochOk(int64_t epoch) { if (epoch < currentManagerEpoch) { if (BW_DEBUG) { - fmt::print("BW {0} got request from old epoch {1}, notifying manager it is out of date\n", + fmt::print("BW {0} got request from old epoch {1}, notifying them they are out of date\n", id.toString(), epoch); } @@ -2960,7 +2960,7 @@ ACTOR Future blobWorker(BlobWorkerInterface bwInterf, self->addActor.send(runGRVChecks(self)); state Future selfRemoved = monitorRemoval(self); - TraceEvent("BlobWorkerInit", self->id); + TraceEvent("BlobWorkerInit", self->id).log(); try { loop choose { @@ -2972,8 +2972,16 @@ ACTOR Future blobWorker(BlobWorkerInterface bwInterf, when(state GranuleStatusStreamRequest req = waitNext(bwInterf.granuleStatusStreamRequest.getFuture())) { if (self->managerEpochOk(req.managerEpoch)) { if (BW_DEBUG) { - fmt::print("Worker {0} got new granule status endpoint\n", self->id.toString()); + fmt::print("Worker {0} got new granule status endpoint {1} from BM {2}\n", + self->id.toString(), + req.reply.getEndpoint().token.toString().c_str(), + req.managerEpoch); } + + // send an error to the old stream before closing it, so it doesn't get broken_promise and mark this + // endpoint as failed + self->currentManagerStatusStream.get().sendError(connection_failed()); + // TODO: pick a reasonable byte limit instead of just piggy-backing req.reply.setByteLimit(SERVER_KNOBS->RANGESTREAM_LIMIT_BYTES); self->currentManagerStatusStream.set(req.reply); diff --git a/fdbserver/worker.actor.cpp b/fdbserver/worker.actor.cpp index d3d3ec95f3..5ce2f3fbef 100644 --- a/fdbserver/worker.actor.cpp +++ b/fdbserver/worker.actor.cpp @@ -1376,6 +1376,7 @@ ACTOR Future workerServer(Reference connRecord, new AsyncVar>()); state Reference>> rkInterf(new AsyncVar>()); state Reference>> bmInterf(new AsyncVar>()); + state int64_t myBMEpoch = -1; state Future handleErrors = workerHandleErrors(errors.getFuture()); // Needs to be stopped last state ActorCollection errorForwarders(false); state Future loggingTrigger = Void(); @@ -1847,10 +1848,17 @@ ACTOR Future workerServer(Reference connRecord, BlobManagerInterface recruited(locality, req.reqId); recruited.initEndpoints(); - if (bmInterf->get().present()) { + if (bmInterf->get().present() && myBMEpoch == req.epoch) { recruited = bmInterf->get().get(); + TEST(true); // Recruited while already a blob manager. } else { + // TODO: it'd be more optimal to halt the last manager here, but it will figure it out via the epoch + // check + // Also, not halting lets us handle the case here where the last BM had a higher epoch and somehow + // the epochs got out of order by a delayed initialize request. The one we start here will just halt + // on the lock check. + myBMEpoch = req.epoch; startRole(Role::BLOB_MANAGER, recruited.id(), interf.id()); DUMPTOKEN(recruited.waitFailure); DUMPTOKEN(recruited.haltBlobManager); From 4d7d1f0e8ea7968431b6cd2c48663a492849b1e5 Mon Sep 17 00:00:00 2001 From: Josh Slocum Date: Mon, 24 Jan 2022 12:50:41 -0600 Subject: [PATCH 161/413] Rollback tracking fix for blob worker --- fdbserver/BlobWorker.actor.cpp | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/fdbserver/BlobWorker.actor.cpp b/fdbserver/BlobWorker.actor.cpp index d1a1d0df80..62c0a8fef5 100644 --- a/fdbserver/BlobWorker.actor.cpp +++ b/fdbserver/BlobWorker.actor.cpp @@ -1071,10 +1071,11 @@ static Version doGranuleRollback(Reference metadata, metadata->currentDeltas.resize(metadata->deltaArena, mIdx); // delete all deltas in rollback range, but we can optimize here to just skip the uncommitted mutations - // directly and immediately pop the rollback out of inProgress + // directly and immediately pop the rollback out of inProgress to completed metadata->bufferedDeltaVersion = rollbackVersion; cfRollbackVersion = mutationVersion; + rollbacksCompleted.push_back(std::pair(rollbackVersion, mutationVersion)); } if (BW_DEBUG) { From 4262241c92bc8d3dc7af7becf39bb805a3056844 Mon Sep 17 00:00:00 2001 From: Josh Slocum Date: Mon, 24 Jan 2022 10:38:33 -0600 Subject: [PATCH 162/413] Removed incorrect assert --- fdbserver/BlobWorker.actor.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/fdbserver/BlobWorker.actor.cpp b/fdbserver/BlobWorker.actor.cpp index 62c0a8fef5..84b3c9c9c5 100644 --- a/fdbserver/BlobWorker.actor.cpp +++ b/fdbserver/BlobWorker.actor.cpp @@ -2643,8 +2643,8 @@ ACTOR Future changeBlobRange(Reference bwData, } if (r.value().lastEpoch == epoch && r.value().lastSeqno == seqno) { - ASSERT(r.begin() == keyRange.begin); - ASSERT(r.end() == keyRange.end); + // the range in our map can be different if later the range was split, but then an old request gets retried. + // Assume that it's the same as initially if (selfReassign) { thisAssignmentNewer = true; From 1494f8216b4ce8e556bb173f4d068e043f44125c Mon Sep 17 00:00:00 2001 From: Josh Slocum Date: Mon, 24 Jan 2022 14:12:36 -0600 Subject: [PATCH 163/413] Reworked blob manager recovery to be more efficient and handle overlapping ongoing splits --- fdbclient/SystemData.cpp | 15 ++ fdbclient/SystemData.h | 3 + fdbserver/BlobManager.actor.cpp | 370 +++++++++++++++----------------- 3 files changed, 196 insertions(+), 192 deletions(-) diff --git a/fdbclient/SystemData.cpp b/fdbclient/SystemData.cpp index c965bf9a18..abeeac97db 100644 --- a/fdbclient/SystemData.cpp +++ b/fdbclient/SystemData.cpp @@ -1312,6 +1312,21 @@ const KeyRange blobGranuleSplitBoundaryKeyRangeFor(UID const& parentGranuleID) { return KeyRangeRef(startKey, strinc(startKey)); } +const Key blobGranuleSplitBoundaryValueFor(int64_t epoch, int64_t seqno) { + BinaryWriter wr(IncludeVersion(ProtocolVersion::withBlobGranule())); + wr << epoch; + wr << seqno; + return wr.toValue(); +} + +std::pair decodeBlobGranuleSplitBoundaryValue(ValueRef const& value) { + int64_t epoch, seqno; + BinaryReader reader(value, IncludeVersion()); + reader >> epoch; + reader >> seqno; + return std::pair(epoch, seqno); +} + const Key blobGranuleHistoryKeyFor(KeyRangeRef const& range, Version version) { BinaryWriter wr(AssumeVersion(ProtocolVersion::withBlobGranule())); wr.serializeBytes(blobGranuleHistoryKeys.begin); diff --git a/fdbclient/SystemData.h b/fdbclient/SystemData.h index bd59eb5cae..27749ffee2 100644 --- a/fdbclient/SystemData.h +++ b/fdbclient/SystemData.h @@ -596,6 +596,9 @@ const Key blobGranuleSplitBoundaryKeyFor(UID const& parentGranuleID, KeyRef cons std::pair decodeBlobGranuleSplitBoundaryKey(KeyRef const& key); const KeyRange blobGranuleSplitBoundaryKeyRangeFor(UID const& parentGranuleID); +const Key blobGranuleSplitBoundaryValueFor(int64_t epoch, int64_t seqno); +std::pair decodeBlobGranuleSplitBoundaryValue(ValueRef const& value); + const Key blobGranuleHistoryKeyFor(KeyRangeRef const& range, Version version); std::pair decodeBlobGranuleHistoryKey(KeyRef const& key); const KeyRange blobGranuleHistoryKeyRangeFor(KeyRangeRef const& range); diff --git a/fdbserver/BlobManager.actor.cpp b/fdbserver/BlobManager.actor.cpp index dcaef06398..5d6ca4366c 100644 --- a/fdbserver/BlobManager.actor.cpp +++ b/fdbserver/BlobManager.actor.cpp @@ -43,6 +43,11 @@ #define BM_DEBUG true +// DO NOT CHANGE THIS +// Special key where the value means the epoch + sequence number of the split, instead of the actual boundary +// Chosen because this should not be a start or end key in any split +static Key splitBoundarySpecialKey = "\xff\xff\xff"_sr; + // TODO add comments + documentation void handleClientBlobRange(KeyRangeMap* knownBlobRanges, Arena& ar, @@ -830,6 +835,9 @@ ACTOR Future maybeSplitRange(Reference bmData, newGranuleIDs.push_back(deterministicRandom()->randomUniqueID()); } + state int64_t splitSeqno = bmData->seqNo; + bmData->seqNo++; + // Need to split range. Persist intent to split and split metadata to DB BEFORE sending split assignments to blob // workers, so that nothing is lost on blob manager recovery loop { @@ -904,6 +912,9 @@ ACTOR Future maybeSplitRange(Reference bmData, granuleRange.begin.printable(), granuleRange.end.printable());*/ + // first key in split boundaries is special: key that doesn't occur normally to the (epoch, seqno) of split + tr->set(blobGranuleSplitBoundaryKeyFor(granuleID, splitBoundarySpecialKey), + blobGranuleSplitBoundaryValueFor(bmData->epoch, splitSeqno)); for (int i = 0; i < newRanges.size() - 1; i++) { /*fmt::print(" {0} [{1} - {2})\n", newGranuleIDs[i].toString().substr(0, 6), @@ -1324,7 +1335,13 @@ ACTOR Future recoverBlobManager(Reference bmData) { // At this point, bmData->workersById is a list of all alive blob workers, but could also include some dead BWs. // The algorithm below works as follows: - // 1. We get the existing granule mappings that were persisted by blob workers who were assigned ranges and + // 1. We get the ongoing split boundaries to construct the set of granules we should have. For these splits, we + // simply assign the range to the next best worker if it is not present in the assignment mapping. This is not + // any worse than what the old blob manager would have done. Details: Note that this means that if a worker we + // intended to give a splitted range to dies before the new BM recovers, then we'll simply assign the range to + // the next best worker. + // + // 2. We get the existing granule mappings that were persisted by blob workers who were assigned ranges and // add them to bmData->granuleAssignments, which is a key range map. // Details: re-assignments might have happened between the time the mapping was last updated and now. // For example, suppose a blob manager sends requests to the range assigner stream to move a granule G. @@ -1334,35 +1351,169 @@ ACTOR Future recoverBlobManager(Reference bmData) { // still owned. In the above case, even if the revoke goes through, since we don't update the mapping during // revokes, this is the same as the case above. Another case to consider is when a blob worker dies when the // BM is recovering. Now the mapping at this time looks like G->deadBW. But the rangeAssigner handles this: - // we'll try to assign a range to a dead worker and fail and reassign it to the next best worker. + // we'll try to assign a range to a dead worker and fail and reassign it to the next best worker. It will also + // handle the case where the mapping does not reflect the desired set of granules based on the ongoing spits, and + // correct it. // - // 3. We get the existing split intentions and boundaries that were Started but not acknowledged by any blob workers - // and add them to our key range map, bmData->granuleAssignments. Note that we are adding them on top of the - // granule mappings and since we are using a key range map, we end up with the same set of shard boundaries as - // the old blob manager had. For these splits, we simply assign the range to the next best worker. This is not - // any worst than what the old blob manager would have done. Details: Note that this means that if a worker we - // intended to give a splitted range to dies before the new BM recovers, then we'll simply assign the range to - // the next best worker. - // - // 4. For every range in our granuleAssignments, we send an assign request to the stream of requests, + // 3. For every range in our granuleAssignments, we send an assign request to the stream of requests, // ultimately giving every range back to some worker (trying to mimic the state of the old BM). // If the worker already had the range, this is a no-op. If the worker didn't have it, it will // begin persisting it. The worker that had the same range before will now be at a lower seqno. state KeyRangeMap> workerAssignments; state Reference tr = makeReference(bmData->db); - state std::unordered_map granuleIdToRange; // TODO KNOB state int rowLimit = BUGGIFY ? deterministicRandom()->randomInt(2, 10) : 10000; if (BM_DEBUG) { fmt::print("BM {0} recovering:\n", bmData->epoch); - fmt::print("BM {0} found old assignments:\n", bmData->epoch); + fmt::print("BM {0} found in progress splits:\n", bmData->epoch); } + // TODO use range stream instead + + state UID currentParentID = UID(); + state Optional nextParentID; + state std::vector splitBoundaries; + state std::pair + splitEpochSeqno; // used to order splits since we can have multiple splits of the same range in progress at once + + state Key boundaryBeginKey = blobGranuleSplitBoundaryKeys.begin; + state RangeResult boundaryResult; + boundaryResult.readThrough = boundaryBeginKey; + boundaryResult.more = true; + state int boundaryResultIdx = 0; + + // Step 3. Get the latest known split and merge state. Because we can have multiple splits in progress at the same + // time, and we don't know which parts of those are reflected in the current set of worker assignments we read, we + // have to construct the current desired set of granules from the set of ongoing splits and merges. Then, if any of + // those are not represented in the worker mapping, we must add them. + state KeyRangeMap> inProgressSplits; + + tr->reset(); + tr->setOption(FDBTransactionOptions::ACCESS_SYSTEM_KEYS); + tr->setOption(FDBTransactionOptions::PRIORITY_SYSTEM_IMMEDIATE); + + loop { + // Advance boundary reader + loop { + if (boundaryResultIdx >= boundaryResult.size()) { + if (!boundaryResult.more) { + break; + } + ASSERT(boundaryResult.readThrough.present() || boundaryResult.size() > 0); + boundaryBeginKey = boundaryResult.readThrough.present() ? boundaryResult.readThrough.get() + : keyAfter(boundaryResult.back().key); + loop { + try { + RangeResult r = wait( + tr->getRange(KeyRangeRef(boundaryBeginKey, blobGranuleSplitBoundaryKeys.end), rowLimit)); + ASSERT(r.size() > 0 || !r.more); + boundaryResult = r; + boundaryResultIdx = 0; + break; + } catch (Error& e) { + if (BM_DEBUG) { + fmt::print("BM {0} got error advancing boundary cursor: {1}\n", bmData->epoch, e.name()); + } + wait(tr->onError(e)); + tr->setOption(FDBTransactionOptions::ACCESS_SYSTEM_KEYS); + tr->setOption(FDBTransactionOptions::PRIORITY_SYSTEM_IMMEDIATE); + } + } + // if we got a response and there are zero rows, we are done + if (boundaryResult.empty()) { + break; + } + } + bool foundNext = false; + while (boundaryResultIdx < boundaryResult.size()) { + UID parentGranuleID; + Key boundaryKey; + + std::tie(parentGranuleID, boundaryKey) = + decodeBlobGranuleSplitBoundaryKey(boundaryResult[boundaryResultIdx].key); + if (parentGranuleID != currentParentID) { + // nextParentID should have already been set by split reader + nextParentID = parentGranuleID; + foundNext = true; + break; + } + + if (splitBoundarySpecialKey == boundaryKey) { + ASSERT(splitEpochSeqno.first == 0 && splitEpochSeqno.second == 0); + ASSERT(boundaryResult[boundaryResultIdx].value.size() > 0); + splitEpochSeqno = decodeBlobGranuleSplitBoundaryValue(boundaryResult[boundaryResultIdx].value); + ASSERT(splitEpochSeqno.first != 0 && splitEpochSeqno.second != 0); + } else { + ASSERT(boundaryResult[boundaryResultIdx].value.size() == 0); + splitBoundaries.push_back(boundaryKey); + } + + boundaryResultIdx++; + } + if (foundNext) { + break; + } + } + + // process this split + if (currentParentID != UID()) { + std::sort(splitBoundaries.begin(), splitBoundaries.end()); + + if (BM_DEBUG) { + fmt::print(" [{0} - {1}) {2} @ ({3}, {4}):\n", + splitBoundaries.front().printable(), + splitBoundaries.back().printable(), + currentParentID.toString().substr(0, 6), + splitEpochSeqno.first, + splitEpochSeqno.second); + } + for (int i = 0; i < splitBoundaries.size() - 1; i++) { + // if this split boundary has not been opened by a blob worker yet, or was not in the assignment list + // when we previously read it, we must ensure it gets assigned to one + KeyRange range = KeyRange(KeyRangeRef(splitBoundaries[i], splitBoundaries[i + 1])); + + // same algorithm as worker map. If we read boundary changes from the log out of order, save the newer + // ones, apply this one, and re-apply the other ones over this one don't concurrently modify with + // iterator + std::vector>> newer; + newer.reserve(splitBoundaries.size() - 1); + auto intersecting = inProgressSplits.intersectingRanges(range); + for (auto& it : intersecting) { + if (splitEpochSeqno.first < it.value().first || + (splitEpochSeqno.first == it.value().first && splitEpochSeqno.second > it.value().second)) { + newer.push_back(std::pair(it.range(), it.value())); + } + } + inProgressSplits.insert(range, splitEpochSeqno); + + for (auto& it : newer) { + inProgressSplits.insert(it.first, it.second); + } + if (BM_DEBUG) { + fmt::print(" [{0} - {1})\n", range.begin.printable(), range.end.printable()); + } + } + } + splitBoundaries.clear(); + splitEpochSeqno = std::pair(0, 0); + + if (!nextParentID.present()) { + break; + } + currentParentID = nextParentID.get(); + nextParentID.reset(); + } + + if (BM_DEBUG) { + fmt::print("BM {0} found old assignments:\n", bmData->epoch); + } // TODO could populate most/all of this list by just asking existing blob workers for their range sets to reduce DB // read load on BM restart Step 1. Get the latest known mapping of granules to blob workers (i.e. assignments) + // This must happen causally AFTER reading the split boundaries, since the blob workers can clear the split + // boundaries for a granule as part of persisting their assignment. state KeyRef beginKey = normalKeys.begin; loop { try { @@ -1413,190 +1564,25 @@ ACTOR Future recoverBlobManager(Reference bmData) { } } - // TODO use range stream instead - - state UID currentParentID = UID(); - state Optional nextParentID; - state std::vector splitBoundaries; - state std::vector> splitStates; - - state Key splitBeginKey = blobGranuleSplitKeys.begin; - state RangeResult splitResult; - splitResult.readThrough = splitBeginKey; - splitResult.more = true; - state int splitResultIdx = 0; - - state Key boundaryBeginKey = blobGranuleSplitBoundaryKeys.begin; - state RangeResult boundaryResult; - boundaryResult.readThrough = boundaryBeginKey; - boundaryResult.more = true; - state int boundaryResultIdx = 0; - - // Step 3. Get the latest known split intentions and boundaries - tr->reset(); - tr->setOption(FDBTransactionOptions::ACCESS_SYSTEM_KEYS); - tr->setOption(FDBTransactionOptions::PRIORITY_SYSTEM_IMMEDIATE); - if (BM_DEBUG) { - fmt::print("BM {0} found in progress splits:\n", bmData->epoch); + printf("Splits overriding the following ranges:\n"); } - loop { - // Advance both split and boundary readers until we hit another granule or EOS, to get the full state for one - // granule split. Effectively a stream merge. - - // Advance split reader - loop { - if (splitResultIdx >= splitResult.size()) { - if (!splitResult.more) { - break; - } - ASSERT(splitResult.readThrough.present() || splitResult.size() > 0); - splitBeginKey = splitResult.readThrough.present() ? splitResult.readThrough.get() - : keyAfter(splitResult.back().key); - loop { - try { - RangeResult r = - wait(tr->getRange(KeyRangeRef(splitBeginKey, blobGranuleSplitKeys.end), rowLimit)); - ASSERT(r.size() > 0 || !r.more); - splitResult = r; - splitResultIdx = 0; - break; - } catch (Error& e) { - if (BM_DEBUG) { - fmt::print("BM {0} got error advancing split cursor: {1}\n", bmData->epoch, e.name()); - } - wait(tr->onError(e)); - tr->setOption(FDBTransactionOptions::ACCESS_SYSTEM_KEYS); - tr->setOption(FDBTransactionOptions::PRIORITY_SYSTEM_IMMEDIATE); - } - } - // if we got a response and there are zero rows, we are done - if (splitResult.empty()) { - ASSERT(!splitResult.more); - break; - } - } - bool foundNext = false; - while (splitResultIdx < splitResult.size()) { - UID parentGranuleID, granuleID; - - std::tie(parentGranuleID, granuleID) = decodeBlobGranuleSplitKey(splitResult[splitResultIdx].key); - if (parentGranuleID != currentParentID) { - nextParentID = parentGranuleID; - foundNext = true; - break; - } - - BlobGranuleSplitState splitState; - Version version; - std::tie(splitState, version) = decodeBlobGranuleSplitValue(splitResult[splitResultIdx].value); - splitStates.push_back(std::pair(granuleID, splitState)); - splitResultIdx++; - } - if (foundNext) { - break; - } + // Apply current granule boundaries to the assignment map. If they don't exactly match what is currently in the map, + // override and assign it to a new worker + auto splits = inProgressSplits.intersectingRanges(normalKeys); + for (auto& it : splits) { + if (it.value().first == 0 || it.value().second == 0) { + // no in-progress splits for this range + continue; } + auto r = workerAssignments.rangeContaining(it.begin()); - // Advance boundary reader - loop { - if (boundaryResultIdx >= boundaryResult.size()) { - if (!boundaryResult.more) { - break; - } - ASSERT(boundaryResult.readThrough.present() || boundaryResult.size() > 0); - boundaryBeginKey = boundaryResult.readThrough.present() ? boundaryResult.readThrough.get() - : keyAfter(boundaryResult.back().key); - loop { - try { - RangeResult r = wait( - tr->getRange(KeyRangeRef(boundaryBeginKey, blobGranuleSplitBoundaryKeys.end), rowLimit)); - ASSERT(r.size() > 0 || !r.more); - boundaryResult = r; - boundaryResultIdx = 0; - break; - } catch (Error& e) { - if (BM_DEBUG) { - fmt::print("BM {0} got error advancing boundary cursor: {1}\n", bmData->epoch, e.name()); - } - wait(tr->onError(e)); - tr->setOption(FDBTransactionOptions::ACCESS_SYSTEM_KEYS); - tr->setOption(FDBTransactionOptions::PRIORITY_SYSTEM_IMMEDIATE); - } - } - // if we got a response and there are zero rows, we are done - if (boundaryResult.empty()) { - break; - } - } - bool foundNext = false; - while (boundaryResultIdx < boundaryResult.size()) { - UID parentGranuleID; - Key boundaryKey; - - std::tie(parentGranuleID, boundaryKey) = - decodeBlobGranuleSplitBoundaryKey(boundaryResult[boundaryResultIdx].key); - - if (parentGranuleID != currentParentID) { - // nextParentID should have already been set by split reader - ASSERT(nextParentID.present()); - ASSERT(nextParentID.get() == parentGranuleID); - foundNext = true; - break; - } - - splitBoundaries.push_back(boundaryKey); - boundaryResultIdx++; - } - if (foundNext) { - break; - } + // if this range is at all different from the worker mapping, the mapping is out of date + if (r.begin() != it.begin() || r.end() != it.end()) { + // the empty UID signifies that we need to find an owner (worker) for this range + workerAssignments.insert(it.range(), UID()); + fmt::print(" [{0} - {1})\n", it.begin().printable().c_str(), it.end().printable().c_str()); } - - // process this split - if (currentParentID != UID()) { - ASSERT(splitStates.size() > 0); - ASSERT(splitBoundaries.size() - 1 == splitStates.size()); - - std::sort(splitBoundaries.begin(), splitBoundaries.end()); - - if (BM_DEBUG) { - fmt::print(" [{0} - {1}) {2}:\n", - splitBoundaries.front().printable(), - splitBoundaries.back().printable(), - currentParentID.toString().substr(0, 6)); - } - for (int i = 0; i < splitStates.size(); i++) { - // if this split boundary had not been opened by a blob worker before the last manager crashed, we must - // ensure it gets assigned to one - KeyRange range = KeyRange(KeyRangeRef(splitBoundaries[i], splitBoundaries[i + 1])); - - if (BM_DEBUG) { - printf(" "); - } - if (splitStates[i].second <= BlobGranuleSplitState::Initialized) { - // the empty UID signifies that we need to find an owner (worker) for this range - if (BM_DEBUG) { - printf("*** "); - } - workerAssignments.insert(range, UID()); - } - if (BM_DEBUG) { - fmt::print("[{0} - {1}) {2}\n", - range.begin.printable(), - range.end.printable(), - splitStates[i].first.toString().substr(0, 6)); - } - } - } - splitBoundaries.clear(); - splitStates.clear(); - - if (!nextParentID.present()) { - break; - } - currentParentID = nextParentID.get(); - nextParentID.reset(); } // Step 4. Send assign requests for all the granules and transfer assignments From 672b7ab89df615b1cd011e43686ae227992855d1 Mon Sep 17 00:00:00 2001 From: Josh Slocum Date: Mon, 24 Jan 2022 15:15:27 -0600 Subject: [PATCH 164/413] Added new test for blob granules and had more consistent naming --- tests/CMakeLists.txt | 14 +++-- ...Cycle.toml => BlobGranuleVerifyCycle.toml} | 2 +- ....toml => BlobGranuleVerifyCycleClean.toml} | 2 +- ...tness.toml => BlobGranuleVerifySmall.toml} | 2 +- ....toml => BlobGranuleVerifySmallClean.toml} | 2 +- tests/slow/BlobGranuleVerifyBalance.toml | 55 +++++++++++++++++++ tests/slow/BlobGranuleVerifyBalanceClean.toml | 20 +++++++ ...Large.toml => BlobGranuleVerifyLarge.toml} | 2 +- ....toml => BlobGranuleVerifyLargeClean.toml} | 2 +- 9 files changed, 89 insertions(+), 12 deletions(-) rename tests/fast/{BlobGranuleCycle.toml => BlobGranuleVerifyCycle.toml} (95%) rename tests/fast/{BlobGranuleCycleClean.toml => BlobGranuleVerifyCycleClean.toml} (86%) rename tests/fast/{BlobGranuleCorrectness.toml => BlobGranuleVerifySmall.toml} (94%) rename tests/fast/{BlobGranuleCorrectnessClean.toml => BlobGranuleVerifySmallClean.toml} (82%) create mode 100644 tests/slow/BlobGranuleVerifyBalance.toml create mode 100644 tests/slow/BlobGranuleVerifyBalanceClean.toml rename tests/slow/{BlobGranuleCorrectnessLarge.toml => BlobGranuleVerifyLarge.toml} (95%) rename tests/slow/{BlobGranuleCorrectnessLargeClean.toml => BlobGranuleVerifyLargeClean.toml} (90%) diff --git a/tests/CMakeLists.txt b/tests/CMakeLists.txt index e3768b76b4..c63ea04aeb 100644 --- a/tests/CMakeLists.txt +++ b/tests/CMakeLists.txt @@ -127,10 +127,10 @@ if(WITH_PYTHON) add_fdb_test(TEST_FILES fast/BackupCorrectnessClean.toml) add_fdb_test(TEST_FILES fast/BackupToDBCorrectness.toml) add_fdb_test(TEST_FILES fast/BackupToDBCorrectnessClean.toml) - add_fdb_test(TEST_FILES fast/BlobGranuleCorrectness.toml) - add_fdb_test(TEST_FILES fast/BlobGranuleCorrectnessClean.toml) - add_fdb_test(TEST_FILES fast/BlobGranuleCycle.toml) - add_fdb_test(TEST_FILES fast/BlobGranuleCycleClean.toml) + add_fdb_test(TEST_FILES fast/BlobGranuleVerifySmall.toml) + add_fdb_test(TEST_FILES fast/BlobGranuleVerifySmallClean.toml) + add_fdb_test(TEST_FILES fast/BlobGranuleVerifyCycle.toml) + add_fdb_test(TEST_FILES fast/BlobGranuleVerifyCycleClean.toml) add_fdb_test(TEST_FILES fast/CacheTest.toml) add_fdb_test(TEST_FILES fast/CloggedSideband.toml) add_fdb_test(TEST_FILES fast/ConfigureLocked.toml) @@ -251,8 +251,10 @@ if(WITH_PYTHON) add_fdb_test(TEST_FILES slow/ApiCorrectness.toml) add_fdb_test(TEST_FILES slow/ApiCorrectnessAtomicRestore.toml) add_fdb_test(TEST_FILES slow/ApiCorrectnessSwitchover.toml) - add_fdb_test(TEST_FILES slow/BlobGranuleCorrectnessLarge.toml) - add_fdb_test(TEST_FILES slow/BlobGranuleCorrectnessLargeClean.toml) + add_fdb_test(TEST_FILES slow/BlobGranuleVerifyLarge.toml) + add_fdb_test(TEST_FILES slow/BlobGranuleVerifyLargeClean.toml) + add_fdb_test(TEST_FILES slow/BlobGranuleVerifyBalance.toml) + add_fdb_test(TEST_FILES slow/BlobGranuleVerifyBalanceClean.toml) add_fdb_test(TEST_FILES slow/ClogWithRollbacks.toml) add_fdb_test(TEST_FILES slow/CloggedCycleTest.toml) add_fdb_test(TEST_FILES slow/CloggedStorefront.toml) diff --git a/tests/fast/BlobGranuleCycle.toml b/tests/fast/BlobGranuleVerifyCycle.toml similarity index 95% rename from tests/fast/BlobGranuleCycle.toml rename to tests/fast/BlobGranuleVerifyCycle.toml index 1b112aef41..b15bc34a85 100644 --- a/tests/fast/BlobGranuleCycle.toml +++ b/tests/fast/BlobGranuleVerifyCycle.toml @@ -2,7 +2,7 @@ blobGranulesEnabled = true [[test]] -testTitle = 'BlobGranuleCycle' +testTitle = 'BlobGranuleVerifyCycle' [[test.workload]] testName = 'Cycle' diff --git a/tests/fast/BlobGranuleCycleClean.toml b/tests/fast/BlobGranuleVerifyCycleClean.toml similarity index 86% rename from tests/fast/BlobGranuleCycleClean.toml rename to tests/fast/BlobGranuleVerifyCycleClean.toml index dca580d37f..abe1d498aa 100644 --- a/tests/fast/BlobGranuleCycleClean.toml +++ b/tests/fast/BlobGranuleVerifyCycleClean.toml @@ -2,7 +2,7 @@ blobGranulesEnabled = true [[test]] -testTitle = 'BlobGranuleCycleClean' +testTitle = 'BlobGranuleVerifyCycleClean' [[test.workload]] testName = 'Cycle' diff --git a/tests/fast/BlobGranuleCorrectness.toml b/tests/fast/BlobGranuleVerifySmall.toml similarity index 94% rename from tests/fast/BlobGranuleCorrectness.toml rename to tests/fast/BlobGranuleVerifySmall.toml index d6fb875474..03022154a2 100644 --- a/tests/fast/BlobGranuleCorrectness.toml +++ b/tests/fast/BlobGranuleVerifySmall.toml @@ -2,7 +2,7 @@ blobGranulesEnabled = true [[test]] -testTitle = 'BlobGranuleCorrectnessTest' +testTitle = 'BlobGranuleVerifySmall' [[test.workload]] testName = 'WriteDuringRead' diff --git a/tests/fast/BlobGranuleCorrectnessClean.toml b/tests/fast/BlobGranuleVerifySmallClean.toml similarity index 82% rename from tests/fast/BlobGranuleCorrectnessClean.toml rename to tests/fast/BlobGranuleVerifySmallClean.toml index b58bba4dbe..a758c46d88 100644 --- a/tests/fast/BlobGranuleCorrectnessClean.toml +++ b/tests/fast/BlobGranuleVerifySmallClean.toml @@ -2,7 +2,7 @@ blobGranulesEnabled = true [[test]] -testTitle = 'BlobGranuleCorrectnessCleanTest' +testTitle = 'BlobGranuleVerifySmallClean' [[test.workload]] testName = 'WriteDuringRead' diff --git a/tests/slow/BlobGranuleVerifyBalance.toml b/tests/slow/BlobGranuleVerifyBalance.toml new file mode 100644 index 0000000000..ee93253c3b --- /dev/null +++ b/tests/slow/BlobGranuleVerifyBalance.toml @@ -0,0 +1,55 @@ +[configuration] +blobGranulesEnabled = true + +[[test]] +testTitle = 'BlobGranuleVerifyBalance' + + [[test.workload]] + testName = 'DDBalance' + testDuration = 120.0 + transactionsPerSecond = 250.0 + binCount = 1000 + writesPerTransaction = 5 + keySpaceDriftFactor = 10 + moversPerClient = 10 + actorsPerClient = 100 + nodes = 100000 + + [[test.workload]] + testName = 'BlobGranuleVerifier' + testDuration = 120.0 + + [[test.workload]] + testName = 'RandomClogging' + testDuration = 120.0 + swizzle = 1 + + [[test.workload]] + testName = 'Rollback' + testDuration = 120.0 + meanDelay = 10.0 + + [[test.workload]] + testName = 'Attrition' + testDuration = 120.0 + + [[test.workload]] + testName = 'Attrition' + machinesToKill = 10 + machinesToLeave = 3 + reboot = true + testDuration = 120.0 + + [[test.workload]] + testName = 'Attrition' + machinesToKill = 10 + machinesToLeave = 3 + reboot = true + testDuration = 120.0 + + [[test.workload]] + testName = 'RemoveServersSafely' + minDelay = 0 + maxDelay = 100 + kill1Timeout = 30 + kill2Timeout = 6000 diff --git a/tests/slow/BlobGranuleVerifyBalanceClean.toml b/tests/slow/BlobGranuleVerifyBalanceClean.toml new file mode 100644 index 0000000000..65bb8ad15c --- /dev/null +++ b/tests/slow/BlobGranuleVerifyBalanceClean.toml @@ -0,0 +1,20 @@ +[configuration] +blobGranulesEnabled = true + +[[test]] +testTitle = 'BlobGranuleVerifyBalanceClean' + + [[test.workload]] + testName = 'DDBalance' + testDuration = 120.0 + transactionsPerSecond = 250.0 + binCount = 1000 + writesPerTransaction = 5 + keySpaceDriftFactor = 10 + moversPerClient = 10 + actorsPerClient = 100 + nodes = 100000 + + [[test.workload]] + testName = 'BlobGranuleVerifier' + testDuration = 120.0 diff --git a/tests/slow/BlobGranuleCorrectnessLarge.toml b/tests/slow/BlobGranuleVerifyLarge.toml similarity index 95% rename from tests/slow/BlobGranuleCorrectnessLarge.toml rename to tests/slow/BlobGranuleVerifyLarge.toml index b9f13f5d83..ad180c55b3 100644 --- a/tests/slow/BlobGranuleCorrectnessLarge.toml +++ b/tests/slow/BlobGranuleVerifyLarge.toml @@ -2,7 +2,7 @@ blobGranulesEnabled = true [[test]] -testTitle = 'BlobGranuleCorrectnessLargeTest' +testTitle = 'BlobGranuleVerifyLarge' [[test.workload]] testName = 'ReadWrite' diff --git a/tests/slow/BlobGranuleCorrectnessLargeClean.toml b/tests/slow/BlobGranuleVerifyLargeClean.toml similarity index 90% rename from tests/slow/BlobGranuleCorrectnessLargeClean.toml rename to tests/slow/BlobGranuleVerifyLargeClean.toml index 90568e8f39..353c73bb23 100644 --- a/tests/slow/BlobGranuleCorrectnessLargeClean.toml +++ b/tests/slow/BlobGranuleVerifyLargeClean.toml @@ -2,7 +2,7 @@ blobGranulesEnabled = true [[test]] -testTitle = 'BlobGranuleCorrectnessLargeCleanTest' +testTitle = 'BlobGranuleVerifyLargeClean' [[test.workload]] testName = 'ReadWrite' From 14cc0a8b02cc0aedb0370d734546862c76e0e117 Mon Sep 17 00:00:00 2001 From: Josh Slocum Date: Tue, 25 Jan 2022 15:41:38 -0600 Subject: [PATCH 165/413] Got BlobGranuleCorrectnessWorkload passing a single test --- fdbserver/BlobManager.actor.cpp | 1 - fdbserver/CMakeLists.txt | 1 + .../BlobGranuleCorrectnessWorkload.actor.cpp | 844 ++++++++++++++++++ tests/CMakeLists.txt | 2 + tests/slow/BlobGranuleCorrectness.toml | 32 + tests/slow/BlobGranuleCorrectnessClean.toml | 9 + 6 files changed, 888 insertions(+), 1 deletion(-) create mode 100644 fdbserver/workloads/BlobGranuleCorrectnessWorkload.actor.cpp create mode 100644 tests/slow/BlobGranuleCorrectness.toml create mode 100644 tests/slow/BlobGranuleCorrectnessClean.toml diff --git a/fdbserver/BlobManager.actor.cpp b/fdbserver/BlobManager.actor.cpp index 5d6ca4366c..72d163c38a 100644 --- a/fdbserver/BlobManager.actor.cpp +++ b/fdbserver/BlobManager.actor.cpp @@ -126,7 +126,6 @@ void updateClientBlobRanges(KeyRangeMap* knownBlobRanges, } bool active = dbBlobRanges[i].value == LiteralStringRef("1"); if (active) { - ASSERT(dbBlobRanges[i + 1].value == StringRef()); if (BM_DEBUG) { fmt::print("BM sees client range [{0} - {1})\n", dbBlobRanges[i].key.printable(), diff --git a/fdbserver/CMakeLists.txt b/fdbserver/CMakeLists.txt index fb589fb3cc..1870ae8f67 100644 --- a/fdbserver/CMakeLists.txt +++ b/fdbserver/CMakeLists.txt @@ -161,6 +161,7 @@ set(FDBSERVER_SRCS workloads/BackupToDBCorrectness.actor.cpp workloads/BackupToDBUpgrade.actor.cpp workloads/BlobStoreWorkload.h + workloads/BlobGranuleCorrectnessWorkload.actor.cpp workloads/BlobGranuleVerifier.actor.cpp workloads/BulkLoad.actor.cpp workloads/BulkSetup.actor.h diff --git a/fdbserver/workloads/BlobGranuleCorrectnessWorkload.actor.cpp b/fdbserver/workloads/BlobGranuleCorrectnessWorkload.actor.cpp new file mode 100644 index 0000000000..2e7d7bd21b --- /dev/null +++ b/fdbserver/workloads/BlobGranuleCorrectnessWorkload.actor.cpp @@ -0,0 +1,844 @@ +/* + * BlobGranuleCorrectnessWorkload.actor.cpp + * + * This source file is part of the FoundationDB open source project + * + * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +#include +#include +#include +#include + +#include "contrib/fmt-8.0.1/include/fmt/format.h" +#include "fdbclient/BlobGranuleReader.actor.h" +#include "fdbclient/ManagementAPI.actor.h" +#include "fdbclient/NativeAPI.actor.h" +#include "fdbclient/ReadYourWrites.h" +#include "fdbclient/SystemData.h" +#include "fdbserver/Knobs.h" +#include "fdbserver/TesterInterface.actor.h" +#include "fdbserver/workloads/workloads.actor.h" +#include "flow/Arena.h" +#include "flow/IRandom.h" +#include "flow/genericactors.actor.h" + +#include "flow/actorcompiler.h" // This must be the last #include. + +#define BGW_DEBUG true + +struct WriteData { + Version writeVersion; + Version clearVersion; + int32_t val; + int16_t valLength; + + // start as MAX_VERSION while uncommitted/uncleared so that they're ignored by concurrent readers + explicit WriteData(int32_t val, int16_t valLength) + : writeVersion(MAX_VERSION), clearVersion(MAX_VERSION), val(val), valLength(valLength) {} +}; + +struct KeyData { + int nextClearIdx; + std::vector writes; +}; + +static std::vector targetValSizes = { 20, 100, 500 }; + +struct ThreadData : ReferenceCounted, NonCopyable { + // directory info + int32_t directoryID; + KeyRange directoryRange; + + // key + value gen data + // in vector for efficient random selection + std::vector usedKeys; + // by key for tracking data + std::map keyData; + + std::deque writeVersions; + + // randomized parameters that can be different per directory + int targetByteRate; + bool nextKeySequential; + int16_t targetValLength; + double reuseKeyProb; + int targetIDsPerKey; + + // communication between workers + Promise firstWriteSuccessful; + Version minSuccessfulReadVersion = MAX_VERSION; + + // stats + int64_t errors = 0; + int64_t mismatches = 0; + int64_t reads = 0; + int64_t timeTravelReads = 0; + int64_t timeTravelTooOld = 0; + int64_t rowsRead = 0; + int64_t rowsWritten = 0; + int64_t bytesRead = 0; + + ThreadData(uint32_t directoryID, int64_t targetByteRate) + : directoryID(directoryID), targetByteRate(targetByteRate) { + directoryRange = + KeyRangeRef(StringRef(format("%08x", directoryID)), StringRef(format("%08x", directoryID + 1))); + + targetByteRate *= (0.5 + deterministicRandom()->random01()); + + targetValLength = deterministicRandom()->randomChoice(targetValSizes); + targetValLength *= (0.5 + deterministicRandom()->random01()); + + nextKeySequential = deterministicRandom()->random01() < 0.5; + reuseKeyProb = 0.1 + (deterministicRandom()->random01() * 0.8); + targetIDsPerKey = 1 + deterministicRandom()->randomInt(1, 10); + + if (BGW_DEBUG) { + fmt::print("Directory {0} initialized with the following parameters:\n", directoryID); + fmt::print(" targetByteRate={0}\n", targetByteRate); + fmt::print(" targetValLength={0}\n", targetValLength); + fmt::print(" nextKeySequential={0}\n", nextKeySequential); + fmt::print(" reuseKeyProb={0}\n", reuseKeyProb); + fmt::print(" targetIDsPerKey={0}\n", targetIDsPerKey); + } + } + + // TODO could make keys variable length? + Key getKey(uint32_t key, uint32_t id) { return StringRef(format("%08x/%08x/%08x", directoryID, key, id)); } +}; + +// TODO REMOVE eventually? +// For debugging mismatches on what data should be and why +// set mismatch to true, dir id and key id to the directory and key id that are wrong, and rv to read version that read +// the wrong value +#define DEBUG_MISMATCH false +#define DEBUG_DIR_ID 0 +#define DEBUG_KEY_ID 0 +#define DEBUG_RV invalidVersion + +#define DEBUG_KEY_OP(dirId, keyId) BGW_DEBUG&& DEBUG_MISMATCH&& dirId == DEBUG_DIR_ID&& DEBUG_KEY_ID == keyId +#define DEBUG_READ_OP(dirId, rv) BGW_DEBUG&& DEBUG_MISMATCH&& dirId == DEBUG_DIR_ID&& rv == DEBUG_RV + +/* + * This is a stand-alone workload designed to validate blob granule correctness. + * By enabling distinct ranges and writing to those parts of the key space, we can control what parts of the key space + * are written to blob, and can validate that the granule data is correct at any desired version. + */ +struct BlobGranuleCorrectnessWorkload : TestWorkload { + bool doSetup; + double testDuration; + + // parameters global across all clients + int64_t targetByteRate; + + std::vector> directories; + std::vector> clients; + DatabaseConfiguration config; + Reference bstore; + + BlobGranuleCorrectnessWorkload(WorkloadContext const& wcx) : TestWorkload(wcx) { + doSetup = !clientId; // only do this on the "first" client + testDuration = getOption(options, LiteralStringRef("testDuration"), 120.0); + + // randomize global test settings based on shared parameter to get similar workload across tests, but then vary + // different parameters within those constraints + int64_t randomness = sharedRandomNumber; + + // randomize between low and high directory count + int64_t targetDirectories = 1 + (randomness % 8); + randomness /= 8; + + int64_t targetMyDirectories = + (targetDirectories / clientCount) + ((targetDirectories % clientCount > clientId) ? 1 : 0); + + if (targetMyDirectories > 0) { + int myDirectories = deterministicRandom()->randomInt(1, 2 * targetMyDirectories + 1); + + // anywhere from 2 delta files per second to 1 delta file every 4 seconds, spread across all directories + targetByteRate = 2 * SERVER_KNOBS->BG_DELTA_FILE_TARGET_BYTES / (1 + (randomness % 8)) / myDirectories; + randomness /= 8; + for (int i = 0; i < myDirectories; i++) { + // set up directory with its own randomness + uint32_t dirId = i * clientCount + clientId; + if (BGW_DEBUG) { + printf("Client %d/%d creating directory %d\n", clientId, clientCount, dirId); + } + directories.push_back(makeReference(dirId, targetByteRate)); + } + } + } + + ACTOR Future setUpBlobRange(Database cx, KeyRange range) { + state Reference tr = makeReference(cx); + loop { + try { + tr->setOption(FDBTransactionOptions::ACCESS_SYSTEM_KEYS); + tr->setOption(FDBTransactionOptions::PRIORITY_SYSTEM_IMMEDIATE); + tr->set(blobRangeChangeKey, deterministicRandom()->randomUniqueID().toString()); + wait(krmSetRange(tr, blobRangeKeys.begin, range, LiteralStringRef("1"))); + wait(tr->commit()); + if (BGW_DEBUG) { + fmt::print("Successfully set up blob granule range for [{0} - {1})\n", + range.begin.printable(), + range.end.printable()); + } + return Void(); + } catch (Error& e) { + wait(tr->onError(e)); + } + } + } + + std::string description() const override { return "BlobGranuleCorrectnessWorkload"; } + Future setup(Database const& cx) override { return _setup(cx, this); } + + ACTOR Future _setup(Database cx, BlobGranuleCorrectnessWorkload* self) { + if (self->doSetup) { + // FIXME: run the actual FDBCLI command instead of copy/pasting its implementation + wait(success(ManagementAPI::changeConfig(cx.getReference(), "blob_granules_enabled=1", true))); + } + + if (self->directories.empty()) { + return Void(); + } + + if (BGW_DEBUG) { + printf("Initializing Blob Granule Correctness s3 stuff\n"); + } + try { + if (g_network->isSimulated()) { + if (BGW_DEBUG) { + printf("Blob Granule Correctness constructing simulated backup container\n"); + } + self->bstore = BackupContainerFileSystem::openContainerFS("file://fdbblob/"); + } else { + if (BGW_DEBUG) { + printf("Blob Granule Correctness constructing backup container from %s\n", + SERVER_KNOBS->BG_URL.c_str()); + } + self->bstore = BackupContainerFileSystem::openContainerFS(SERVER_KNOBS->BG_URL); + if (BGW_DEBUG) { + printf("Blob Granule Correctness constructed backup container\n"); + } + } + } catch (Error& e) { + if (BGW_DEBUG) { + printf("Blob Granule Correctness got backup container init error %s\n", e.name()); + } + throw e; + } + + return Void(); + } + + // FIXME: typedef this pair type and/or chunk list + ACTOR Future>>> + readFromBlob(Database cx, BlobGranuleCorrectnessWorkload* self, KeyRange range, Version version) { + state RangeResult out; + state Standalone> chunks; + state Transaction tr(cx); + + loop { + try { + Standalone> chunks_ = wait(tr.readBlobGranules(range, 0, version)); + chunks = chunks_; + break; + } catch (Error& e) { + wait(tr.onError(e)); + } + } + + for (const BlobGranuleChunkRef& chunk : chunks) { + RangeResult chunkRows = wait(readBlobGranule(chunk, range, version, self->bstore)); + out.arena().dependsOn(chunkRows.arena()); + out.append(out.arena(), chunkRows.begin(), chunkRows.size()); + } + return std::pair(out, chunks); + } + + // handle retries + errors + // It's ok to reset the transaction here because its read version is only used for reading the granule mapping from + // the system keyspace + ACTOR Future doGrv(Transaction* tr) { + loop { + try { + Version readVersion = wait(tr->getReadVersion()); + return readVersion; + } catch (Error& e) { + // TODO REMOVE print + printf("BGV GRV got error %s\n", e.name()); + wait(tr->onError(e)); + } + } + } + + ACTOR Future waitFirstSnapshot(BlobGranuleCorrectnessWorkload* self, + Database cx, + Reference threadData, + bool doSetup) { + if (doSetup) { + // Set up the blob range first + wait(self->setUpBlobRange(cx, threadData->directoryRange)); + } + + // read entire keyspace at the start until granules for the entire thing are available + loop { + state Transaction tr(cx); + try { + Version rv = wait(self->doGrv(&tr)); + state Version readVersion = rv; + std::pair>> blob = + wait(self->readFromBlob(cx, self, threadData->directoryRange, readVersion)); + printf("Directory %d got %s RV %lld\n", + threadData->directoryID, + doSetup ? "initial" : "final", + readVersion); + threadData->minSuccessfulReadVersion = readVersion; + return Void(); + } catch (Error& e) { + if (e.code() == error_code_operation_cancelled) { + throw e; + } + if (e.code() != error_code_blob_granule_transaction_too_old) { + wait(tr.onError(e)); + } else { + wait(delay(1.0)); + } + } + } + } + + void logMismatch(Reference threadData, + const Optional& lastMatching, + const Optional& expectedKey, + const Optional& blobKey, + const Optional& expectedValue, + const Optional& blobValue, + uint32_t startKey, + uint32_t endKey, + Version readVersion, + const std::pair>>& blob) { + threadData->mismatches++; + if (!BGW_DEBUG) { + return; + } + + fmt::print("Found mismatch! Request for dir {0} [{1} - {2}) @ {3}\n", + format("%08x", threadData->directoryID), + format("%08x", startKey), + format("%08x", endKey), + readVersion); + if (lastMatching.present()) { + printf(" last correct: %s\n", lastMatching.get().printable().c_str()); + } + if (expectedValue.present() || blobValue.present()) { + // value mismatch + ASSERT(blobKey.present()); + ASSERT(blobKey == expectedKey); + fmt::print(" Value mismatch for {0}.\n Expected={1}\n Actual={2}\n", + blobKey.get().printable(), + expectedValue.get().printable(), + blobValue.get().printable()); + } else { + // key mismatch + fmt::print(" Expected Key: {0}\n", expectedKey.present() ? expectedKey.get().printable() : ""); + fmt::print(" Actual Key: {0}\n", blobKey.present() ? blobKey.get().printable() : ""); + } + + printf("Chunks:\n"); + for (auto& chunk : blob.second) { + printf("[%s - %s)\n", chunk.keyRange.begin.printable().c_str(), chunk.keyRange.end.printable().c_str()); + + printf(" SnapshotFile:\n %s\n", + chunk.snapshotFile.present() ? chunk.snapshotFile.get().toString().c_str() : ""); + printf(" DeltaFiles:\n"); + for (auto& df : chunk.deltaFiles) { + printf(" %s\n", df.toString().c_str()); + } + printf(" Deltas: (%d)", chunk.newDeltas.size()); + if (chunk.newDeltas.size() > 0) { + fmt::print(" with version [{0} - {1}]", + chunk.newDeltas[0].version, + chunk.newDeltas[chunk.newDeltas.size() - 1].version); + } + fmt::print(" IncludedVersion: {}\n", chunk.includedVersion); + } + printf("\n"); + } + + Value genVal(uint32_t val, uint16_t valLen) { + std::string v(valLen, 'x'); + auto valFormatted = format("%08x", val); + if (valFormatted.size() > v.size()) { + printf("valFormatted=%d, v.size=%d\n", valFormatted.size(), v.size()); + } + ASSERT(valFormatted.size() <= v.size()); + + for (int i = 0; i < valFormatted.size(); i++) { + v[i] = valFormatted[i]; + } + // copy into an arena + // TODO do this in original arena? a bit more efficient that way + Arena a; + return Standalone(StringRef(a, v), a); + } + + bool validateValue(const Value& v, uint32_t val, uint16_t valLen) { + if (v.size() != valLen) { + return false; + } + // check for correct value portion + auto valFormatted = format("%08x", val); + ASSERT(valFormatted.size() <= v.size()); + if (v.substr(0, valFormatted.size()) != valFormatted) { + return false; + } + // check for corruption + for (int i = valFormatted.size(); i < v.size(); i++) { + if (v[i] != 'x') { + return false; + } + } + return true; + } + + bool validateResult(Reference threadData, + std::pair>> blob, + int startKeyInclusive, + int endKeyExclusive, + Version beginVersion, + Version readVersion) { + auto checkIt = threadData->keyData.lower_bound(startKeyInclusive); + if (checkIt != threadData->keyData.end() && checkIt->first < startKeyInclusive) { + checkIt++; + } + int resultIdx = 0; + Optional lastMatching; + if (DEBUG_READ_OP(threadData->directoryID, readVersion)) { + fmt::print("DBG READ: [{0} - {1}) @ {2}\n", + format("%08x", startKeyInclusive), + format("%08x", endKeyExclusive), + readVersion); + } + + while (checkIt != threadData->keyData.end() && checkIt->first < endKeyExclusive) { + uint32_t key = checkIt->first; + if (DEBUG_READ_OP(threadData->directoryID, readVersion)) { + fmt::print("DBG READ: Key {0}\n", format("%08x", key)); + } + + // TODO could binary search this to find clearVersion if it gets long + int idIdx = 0; + for (; idIdx < checkIt->second.writes.size() && checkIt->second.writes[idIdx].clearVersion <= readVersion; + idIdx++) { + // iterate until we find the oldest tag that should have not been cleared + /*if (DEBUG_READ_OP(threadData->directoryID, readVersion)) { + fmt::print( + "DBG READ: Skip ID {0} cleared @ {1}\n", idIdx, checkIt->second.writes[idIdx].clearVersion); + }*/ + } + if (DEBUG_READ_OP(threadData->directoryID, readVersion)) { + printf("Finished skipping\n"); + } + for (; idIdx < checkIt->second.writes.size() && checkIt->second.writes[idIdx].writeVersion <= readVersion; + idIdx++) { + Key nextKeyShouldBe = threadData->getKey(key, idIdx); + if (DEBUG_READ_OP(threadData->directoryID, readVersion)) { + fmt::print("DBG READ: Checking ID {0} ({1}) written @ {2}\n", + format("%08x", idIdx), + idIdx, + checkIt->second.writes[idIdx].writeVersion); + } + if (resultIdx >= blob.first.size()) { + // missing at end!! + logMismatch(threadData, + lastMatching, + nextKeyShouldBe, + Optional(), + Optional(), + Optional(), + startKeyInclusive, + endKeyExclusive, + readVersion, + blob); + return false; + } + + if (nextKeyShouldBe != blob.first[resultIdx].key) { + // key mismatch! + if (DEBUG_READ_OP(threadData->directoryID, readVersion)) { + printf("key mismatch!\n"); + } + logMismatch(threadData, + lastMatching, + nextKeyShouldBe, + blob.first[resultIdx].key, + Optional(), + Optional(), + startKeyInclusive, + endKeyExclusive, + readVersion, + blob); + return false; + } else if (!validateValue(blob.first[resultIdx].value, + checkIt->second.writes[idIdx].val, + checkIt->second.writes[idIdx].valLength)) { + logMismatch(threadData, + lastMatching, + nextKeyShouldBe, + blob.first[resultIdx].key, + genVal(checkIt->second.writes[idIdx].val, checkIt->second.writes[idIdx].valLength), + blob.first[resultIdx].value, + startKeyInclusive, + endKeyExclusive, + readVersion, + blob); + return false; + // value mismatch for same key + } else { + lastMatching = nextKeyShouldBe; + } + resultIdx++; + } + checkIt++; + } + + if (resultIdx < blob.first.size()) { + // blob has extra stuff!! + logMismatch(threadData, + lastMatching, + Optional(), + blob.first[resultIdx].key, + Optional(), + Optional(), + startKeyInclusive, + endKeyExclusive, + readVersion, + blob); + return false; + } + + return true; + } + + ACTOR Future readWorker(BlobGranuleCorrectnessWorkload* self, + Future firstSnapshot, + Database cx, + Reference threadData) { + state double last = now(); + state double targetBytesReadPerQuery = + SERVER_KNOBS->BG_SNAPSHOT_FILE_TARGET_BYTES * 2.0 / deterministicRandom()->randomInt(1, 11); + state int targetKeysReadPerQuery = + (int)(targetBytesReadPerQuery / (threadData->targetValLength * threadData->targetIDsPerKey)); + // read at higher read rate than write rate + state double targetTps = threadData->targetByteRate * 2 / targetBytesReadPerQuery; + + state Version readVersion; + + TraceEvent("BlobGranuleCorrectnessReaderStart").log(); + if (BGW_DEBUG) { + printf("BGW read thread starting\n"); + } + + // wait for data to read + wait(firstSnapshot); + wait(threadData->firstWriteSuccessful.getFuture()); + + TraceEvent("BlobGranuleCorrectnessReaderReady").log(); + if (BGW_DEBUG) { + printf("BGW read thread ready\n"); + } + + loop { + try { + // Do 1 read + + // pick key range by doing random start key, and then picking the end key based on that + int startKeyIdx = deterministicRandom()->randomInt(0, threadData->usedKeys.size()); + state uint32_t startKey = threadData->usedKeys[startKeyIdx]; + auto endKeyIt = threadData->keyData.find(startKey); + ASSERT(endKeyIt != threadData->keyData.end()); + + for (int i = 0; i < targetKeysReadPerQuery && endKeyIt != threadData->keyData.end(); i++, endKeyIt++) { + // iterate forward until end or target keys have passed + } + + state uint32_t endKey; + if (endKeyIt == threadData->keyData.end()) { + endKey = std::numeric_limits::max(); + } else { + endKey = endKeyIt->first; + } + + state KeyRange range = KeyRangeRef(threadData->getKey(startKey, 0), threadData->getKey(endKey, 0)); + + // pick read version + // TODO could also pick begin version here + ASSERT(threadData->writeVersions.back() >= threadData->minSuccessfulReadVersion); + // randomly choose up to date vs time travel read + if (deterministicRandom()->random01() < 0.5) { + threadData->reads++; + readVersion = threadData->writeVersions.back(); + } else { + threadData->timeTravelReads++; + loop { + int readVersionIdx = deterministicRandom()->randomInt(0, threadData->writeVersions.size()); + readVersion = threadData->writeVersions[readVersionIdx]; + if (readVersion >= threadData->minSuccessfulReadVersion) { + break; + } + } + } + + std::pair>> blob = + wait(self->readFromBlob(cx, self, range, readVersion)); + self->validateResult(threadData, blob, startKey, endKey, 0, readVersion); + + threadData->rowsRead += blob.first.size(); + threadData->bytesRead += blob.first.expectedSize(); + } catch (Error& e) { + if (e.code() == error_code_operation_cancelled) { + throw; + } + if (e.code() == error_code_blob_granule_transaction_too_old) { + threadData->timeTravelTooOld++; + } else { + threadData->errors++; + if (BGW_DEBUG) { + printf("BGWorkload got unexpected error %s\n", e.name()); + } + } + } + wait(poisson(&last, 1.0 / targetTps)); + } + } + + ACTOR Future writeWorker(BlobGranuleCorrectnessWorkload* self, + Future firstSnapshot, + Database cx, + Reference threadData) { + + state double last = now(); + state int keysPerQuery = 100; + state int targetBytesPerQuery = threadData->targetValLength * keysPerQuery; + state double targetTps = (1.0 * threadData->targetByteRate) / targetBytesPerQuery; + state uint32_t nextVal = 0; + + TraceEvent("BlobGranuleCorrectnessWriterStart").log(); + + wait(firstSnapshot); + + TraceEvent("BlobGranuleCorrectnessWriterReady").log(); + + loop { + state Transaction tr(cx); + + // pick rows to write and clear, generate values for writes + state std::vector> keyAndIdToWrite; + state std::vector> keyAndIdToClear; + + for (int i = 0; i < keysPerQuery; i++) { + uint32_t key; + if (threadData->keyData.empty() || deterministicRandom()->random01() > threadData->reuseKeyProb) { + // new key + if (threadData->nextKeySequential) { + key = threadData->usedKeys.size(); + } else { + key = std::numeric_limits::max(); + while (key == std::numeric_limits::max() || + threadData->keyData.find(key) != threadData->keyData.end()) { + key = deterministicRandom()->randomUInt32(); + } + } + + // add new key to data structures + threadData->usedKeys.push_back(key); + threadData->keyData.insert({ key, KeyData() }); + } else { + int keyIdx = deterministicRandom()->randomInt(0, threadData->usedKeys.size()); + key = threadData->usedKeys[keyIdx]; + } + + auto keyIt = threadData->keyData.find(key); + ASSERT(keyIt != threadData->keyData.end()); + + int unclearedIds = keyIt->second.writes.size() - keyIt->second.nextClearIdx; + // if we are at targetIDs, 50% chance of adding one or clearing. If we are closer to 0, higher chance of + // adding one, if we are closer to 2x target IDs, higher chance of clearing one + double probAddId = (threadData->targetIDsPerKey * 2.0 - unclearedIds) / threadData->targetIDsPerKey; + if (deterministicRandom()->random01() < probAddId || + keyIt->second.nextClearIdx == keyIt->second.writes.size()) { + int32_t val = nextVal++; + int16_t valLen = (0.5 + deterministicRandom()->random01()) * threadData->targetValLength; + if (valLen < 10) { + valLen = 10; + } + + uint32_t nextId = keyIt->second.writes.size(); + keyIt->second.writes.push_back(WriteData(val, valLen)); + + keyAndIdToWrite.push_back(std::tuple(key, nextId, val, valLen)); + } else { + uint32_t idToClear = keyIt->second.nextClearIdx++; + keyAndIdToClear.push_back(std::pair(key, idToClear)); + } + } + + loop { + try { + // write rows in txn + for (auto& it : keyAndIdToWrite) { + Value v = self->genVal(std::get<2>(it), std::get<3>(it)); + tr.set(threadData->getKey(std::get<0>(it), std::get<1>(it)), v); + } + for (auto& it : keyAndIdToClear) { + tr.clear(singleKeyRange(threadData->getKey(it.first, it.second))); + } + + wait(tr.commit()); + break; + } catch (Error& e) { + wait(tr.onError(e)); + } + } + + Version commitVersion = tr.getCommittedVersion(); + + // once txn is committed, update write map + + for (auto& it : keyAndIdToWrite) { + uint32_t key = std::get<0>(it); + uint32_t id = std::get<1>(it); + auto keyIt = threadData->keyData.find(key); + ASSERT(keyIt != threadData->keyData.end()); + keyIt->second.writes[id].writeVersion = commitVersion; + if (DEBUG_KEY_OP(threadData->directoryID, key)) { + fmt::print("DBG: {0} WRITE {1} = {2}:{3}\n", + commitVersion, + format("%08x/%08x/%08x", threadData->directoryID, key, id), + std::get<2>(it), + std::get<3>(it)); + } + } + + for (auto& it : keyAndIdToClear) { + auto keyIt = threadData->keyData.find(it.first); + ASSERT(keyIt != threadData->keyData.end()); + keyIt->second.writes[it.second].clearVersion = commitVersion; + if (DEBUG_KEY_OP(threadData->directoryID, it.first)) { + fmt::print("DBG: {0} CLEAR {1}\n", + commitVersion, + format("%08x/%08x/%08x", threadData->directoryID, it.first, it.second)); + } + } + + threadData->writeVersions.push_back(commitVersion); + + if (threadData->firstWriteSuccessful.canBeSet()) { + threadData->firstWriteSuccessful.send(Void()); + } + + // wait + wait(poisson(&last, 1.0 / targetTps)); + } + } + + Future start(Database const& cx) override { + // TODO need to make thing that waits for granules to exist before ANY of the actors start! + // Then can reuse that for final checks or something? + clients.reserve(3 * directories.size()); + for (auto& it : directories) { + Future start = waitFirstSnapshot(this, cx, it, true); + clients.push_back(timeout(writeWorker(this, start, cx, it), testDuration, Void())); + clients.push_back(timeout(readWorker(this, start, cx, it), testDuration, Void())); + } + return delay(testDuration); + } + + ACTOR Future checkDirectory(Database cx, + BlobGranuleCorrectnessWorkload* self, + Reference threadData) { + + state bool result = true; + state int finalRowsValidated; + if (threadData->minSuccessfulReadVersion == MAX_VERSION) { + // never had a successful read during the test, likely due to many chaos events. Just wait for granules to + // become available and call that a pass, since writer is stopped + wait(self->waitFirstSnapshot(self, cx, threadData, false)); + } else { + // otherwise, read at last write version and ensure everything becomes available and matches + // it's possible that waitFirstSnapshot finished but then writer never wrote anything before test timed out + state Version readVersion; + if (threadData->writeVersions.size()) { + readVersion = threadData->writeVersions.back(); + } else { + state Transaction tr(cx); + Version rv = wait(self->doGrv(&tr)); + readVersion = rv; + } + if (BGW_DEBUG) { + fmt::print("Directory {0} doing final data check @ {1}\n", threadData->directoryID, readVersion); + } + std::pair>> blob = + wait(self->readFromBlob(cx, self, threadData->directoryRange, readVersion)); + result = self->validateResult(threadData, blob, 0, std::numeric_limits::max(), 0, readVersion); + finalRowsValidated = blob.first.size(); + } + + bool initialCheck = result; + result &= threadData->mismatches == 0 && (threadData->timeTravelTooOld == 0); + + fmt::print("Blob Granule Workload Directory {0} {1}:\n", threadData->directoryID, result ? "passed" : "failed"); + fmt::print(" Final granule check {0}successful\n", initialCheck ? "" : "un"); + fmt::print(" {} Rows read in final check\n", finalRowsValidated); + fmt::print(" {} mismatches\n", threadData->mismatches); + fmt::print(" {} time travel too old\n", threadData->timeTravelTooOld); + fmt::print(" {} errors\n", threadData->errors); + fmt::print(" {} rows written\n", threadData->rowsWritten); + fmt::print(" {} unique keys\n", threadData->usedKeys.size()); + fmt::print(" {} real-time reads\n", threadData->reads); + fmt::print(" {} time travel reads\n", threadData->timeTravelReads); + fmt::print(" {} rows read\n", threadData->rowsRead); + fmt::print(" {} bytes read\n", threadData->bytesRead); + // FIXME: add above as details to trace event + + TraceEvent("BlobGranuleWorkloadChecked").detail("Directory", threadData->directoryID).detail("Result", result); + + // For some reason simulation is still passing when this fails?.. so assert for now + ASSERT(result); + + return result; + } + + ACTOR Future _check(Database cx, BlobGranuleCorrectnessWorkload* self) { + // check error counts, and do an availability check at the end + state std::vector> results; + for (auto& it : self->directories) { + results.push_back(self->checkDirectory(cx, self, it)); + } + state bool allSuccessful = true; + for (auto& f : results) { + bool dirSuccess = wait(f); + allSuccessful &= dirSuccess; + } + return allSuccessful; + } + + Future check(Database const& cx) override { return _check(cx, this); } + void getMetrics(std::vector& m) override {} +}; + +WorkloadFactory BlobGranuleCorrectnessWorkloadFactory("BlobGranuleCorrectnessWorkload"); diff --git a/tests/CMakeLists.txt b/tests/CMakeLists.txt index c63ea04aeb..3f7909635e 100644 --- a/tests/CMakeLists.txt +++ b/tests/CMakeLists.txt @@ -255,6 +255,8 @@ if(WITH_PYTHON) add_fdb_test(TEST_FILES slow/BlobGranuleVerifyLargeClean.toml) add_fdb_test(TEST_FILES slow/BlobGranuleVerifyBalance.toml) add_fdb_test(TEST_FILES slow/BlobGranuleVerifyBalanceClean.toml) + add_fdb_test(TEST_FILES slow/BlobGranuleCorrectnessClean.toml) + add_fdb_test(TEST_FILES slow/BlobGranuleCorrectness.toml) add_fdb_test(TEST_FILES slow/ClogWithRollbacks.toml) add_fdb_test(TEST_FILES slow/CloggedCycleTest.toml) add_fdb_test(TEST_FILES slow/CloggedStorefront.toml) diff --git a/tests/slow/BlobGranuleCorrectness.toml b/tests/slow/BlobGranuleCorrectness.toml new file mode 100644 index 0000000000..10c03ab63b --- /dev/null +++ b/tests/slow/BlobGranuleCorrectness.toml @@ -0,0 +1,32 @@ +[configuration] +blobGranulesEnabled = true + +[[test]] +testTitle = 'BlobGranuleCorrectness' + + [[test.workload]] + testName = 'BlobGranuleCorrectnessWorkload' + testDuration = 120.0 + + [[test.workload]] + testName = 'RandomClogging' + testDuration = 120.0 + + [[test.workload]] + testName = 'Rollback' + meanDelay = 30.0 + testDuration = 120.0 + + [[test.workload]] + testName = 'Attrition' + machinesToKill = 10 + machinesToLeave = 3 + reboot = true + testDuration = 120.0 + + [[test.workload]] + testName = 'Attrition' + machinesToKill = 10 + machinesToLeave = 3 + reboot = true + testDuration = 120.0 \ No newline at end of file diff --git a/tests/slow/BlobGranuleCorrectnessClean.toml b/tests/slow/BlobGranuleCorrectnessClean.toml new file mode 100644 index 0000000000..a538e7203b --- /dev/null +++ b/tests/slow/BlobGranuleCorrectnessClean.toml @@ -0,0 +1,9 @@ +[configuration] +blobGranulesEnabled = true + +[[test]] +testTitle = 'BlobGranuleCorrectness' + + [[test.workload]] + testName = 'BlobGranuleCorrectnessWorkload' + testDuration = 120.0 \ No newline at end of file From 005421dfc87222064b163ec7ed27a1b53ac52134 Mon Sep 17 00:00:00 2001 From: Josh Slocum Date: Tue, 25 Jan 2022 16:53:44 -0600 Subject: [PATCH 166/413] Fixing final check bug in new workload --- .../BlobGranuleCorrectnessWorkload.actor.cpp | 23 ++++++++----------- 1 file changed, 9 insertions(+), 14 deletions(-) diff --git a/fdbserver/workloads/BlobGranuleCorrectnessWorkload.actor.cpp b/fdbserver/workloads/BlobGranuleCorrectnessWorkload.actor.cpp index 2e7d7bd21b..c111d339c0 100644 --- a/fdbserver/workloads/BlobGranuleCorrectnessWorkload.actor.cpp +++ b/fdbserver/workloads/BlobGranuleCorrectnessWorkload.actor.cpp @@ -450,9 +450,6 @@ struct BlobGranuleCorrectnessWorkload : TestWorkload { "DBG READ: Skip ID {0} cleared @ {1}\n", idIdx, checkIt->second.writes[idIdx].clearVersion); }*/ } - if (DEBUG_READ_OP(threadData->directoryID, readVersion)) { - printf("Finished skipping\n"); - } for (; idIdx < checkIt->second.writes.size() && checkIt->second.writes[idIdx].writeVersion <= readVersion; idIdx++) { Key nextKeyShouldBe = threadData->getKey(key, idIdx); @@ -724,6 +721,7 @@ struct BlobGranuleCorrectnessWorkload : TestWorkload { uint32_t id = std::get<1>(it); auto keyIt = threadData->keyData.find(key); ASSERT(keyIt != threadData->keyData.end()); + keyIt->second.writes[id].writeVersion = commitVersion; if (DEBUG_KEY_OP(threadData->directoryID, key)) { fmt::print("DBG: {0} WRITE {1} = {2}:{3}\n", @@ -774,21 +772,18 @@ struct BlobGranuleCorrectnessWorkload : TestWorkload { state bool result = true; state int finalRowsValidated; - if (threadData->minSuccessfulReadVersion == MAX_VERSION) { - // never had a successful read during the test, likely due to many chaos events. Just wait for granules to - // become available and call that a pass, since writer is stopped + if (threadData->writeVersions.size()) { + // never had a successful write during the test, likely due to many chaos events. Just wait for granules to + // become available and call that a pass, since writer is stopped and will never guarantee anything is + // written + if (BGW_DEBUG) { + fmt::print("Directory {0} doing final availability check\n", threadData->directoryID); + } wait(self->waitFirstSnapshot(self, cx, threadData, false)); } else { // otherwise, read at last write version and ensure everything becomes available and matches // it's possible that waitFirstSnapshot finished but then writer never wrote anything before test timed out - state Version readVersion; - if (threadData->writeVersions.size()) { - readVersion = threadData->writeVersions.back(); - } else { - state Transaction tr(cx); - Version rv = wait(self->doGrv(&tr)); - readVersion = rv; - } + state Version readVersion = threadData->writeVersions.back(); if (BGW_DEBUG) { fmt::print("Directory {0} doing final data check @ {1}\n", threadData->directoryID, readVersion); } From c2d334d8431cf056cc24d530999fa3bc3bc161af Mon Sep 17 00:00:00 2001 From: Josh Slocum Date: Tue, 25 Jan 2022 19:36:51 -0600 Subject: [PATCH 167/413] Fixing more bugs in new workload --- .../BlobGranuleCorrectnessWorkload.actor.cpp | 17 +++++++++++------ 1 file changed, 11 insertions(+), 6 deletions(-) diff --git a/fdbserver/workloads/BlobGranuleCorrectnessWorkload.actor.cpp b/fdbserver/workloads/BlobGranuleCorrectnessWorkload.actor.cpp index c111d339c0..fe33ae9f47 100644 --- a/fdbserver/workloads/BlobGranuleCorrectnessWorkload.actor.cpp +++ b/fdbserver/workloads/BlobGranuleCorrectnessWorkload.actor.cpp @@ -182,6 +182,10 @@ struct BlobGranuleCorrectnessWorkload : TestWorkload { } ACTOR Future setUpBlobRange(Database cx, KeyRange range) { + if (BGW_DEBUG) { + fmt::print( + "Setting up blob granule range for [{0} - {1})\n", range.begin.printable(), range.end.printable()); + } state Reference tr = makeReference(cx); loop { try { @@ -215,6 +219,12 @@ struct BlobGranuleCorrectnessWorkload : TestWorkload { return Void(); } + state int directoryIdx = 0; + for (; directoryIdx < self->directories.size(); directoryIdx++) { + // Set up the blob range first + wait(self->setUpBlobRange(cx, self->directories[directoryIdx]->directoryRange)); + } + if (BGW_DEBUG) { printf("Initializing Blob Granule Correctness s3 stuff\n"); } @@ -289,11 +299,6 @@ struct BlobGranuleCorrectnessWorkload : TestWorkload { Database cx, Reference threadData, bool doSetup) { - if (doSetup) { - // Set up the blob range first - wait(self->setUpBlobRange(cx, threadData->directoryRange)); - } - // read entire keyspace at the start until granules for the entire thing are available loop { state Transaction tr(cx); @@ -772,7 +777,7 @@ struct BlobGranuleCorrectnessWorkload : TestWorkload { state bool result = true; state int finalRowsValidated; - if (threadData->writeVersions.size()) { + if (threadData->writeVersions.empty()) { // never had a successful write during the test, likely due to many chaos events. Just wait for granules to // become available and call that a pass, since writer is stopped and will never guarantee anything is // written From 826f9569903698e1ce02dc035b3aa31252fc86e3 Mon Sep 17 00:00:00 2001 From: Josh Slocum Date: Wed, 26 Jan 2022 08:38:24 -0600 Subject: [PATCH 168/413] Fixed manager recovery order of in progress splits --- fdbserver/BlobManager.actor.cpp | 15 +++++++++------ 1 file changed, 9 insertions(+), 6 deletions(-) diff --git a/fdbserver/BlobManager.actor.cpp b/fdbserver/BlobManager.actor.cpp index 72d163c38a..cb8fe38e2e 100644 --- a/fdbserver/BlobManager.actor.cpp +++ b/fdbserver/BlobManager.actor.cpp @@ -1384,7 +1384,7 @@ ACTOR Future recoverBlobManager(Reference bmData) { boundaryResult.more = true; state int boundaryResultIdx = 0; - // Step 3. Get the latest known split and merge state. Because we can have multiple splits in progress at the same + // Step 2. Get the latest known split and merge state. Because we can have multiple splits in progress at the same // time, and we don't know which parts of those are reflected in the current set of worker assignments we read, we // have to construct the current desired set of granules from the set of ongoing splits and merges. Then, if any of // those are not represented in the worker mapping, we must add them. @@ -1473,6 +1473,9 @@ ACTOR Future recoverBlobManager(Reference bmData) { // if this split boundary has not been opened by a blob worker yet, or was not in the assignment list // when we previously read it, we must ensure it gets assigned to one KeyRange range = KeyRange(KeyRangeRef(splitBoundaries[i], splitBoundaries[i + 1])); + if (BM_DEBUG) { + fmt::print(" [{0} - {1})\n", range.begin.printable(), range.end.printable()); + } // same algorithm as worker map. If we read boundary changes from the log out of order, save the newer // ones, apply this one, and re-apply the other ones over this one don't concurrently modify with @@ -1482,7 +1485,8 @@ ACTOR Future recoverBlobManager(Reference bmData) { auto intersecting = inProgressSplits.intersectingRanges(range); for (auto& it : intersecting) { if (splitEpochSeqno.first < it.value().first || - (splitEpochSeqno.first == it.value().first && splitEpochSeqno.second > it.value().second)) { + (splitEpochSeqno.first == it.value().first && splitEpochSeqno.second < it.value().second)) { + // range currently there is newer than this range. newer.push_back(std::pair(it.range(), it.value())); } } @@ -1491,9 +1495,6 @@ ACTOR Future recoverBlobManager(Reference bmData) { for (auto& it : newer) { inProgressSplits.insert(it.first, it.second); } - if (BM_DEBUG) { - fmt::print(" [{0} - {1})\n", range.begin.printable(), range.end.printable()); - } } } splitBoundaries.clear(); @@ -1510,7 +1511,9 @@ ACTOR Future recoverBlobManager(Reference bmData) { fmt::print("BM {0} found old assignments:\n", bmData->epoch); } // TODO could populate most/all of this list by just asking existing blob workers for their range sets to reduce DB - // read load on BM restart Step 1. Get the latest known mapping of granules to blob workers (i.e. assignments) + // read load on BM restart + + // Step 3. Get the latest known mapping of granules to blob workers (i.e. assignments) // This must happen causally AFTER reading the split boundaries, since the blob workers can clear the split // boundaries for a granule as part of persisting their assignment. state KeyRef beginKey = normalKeys.begin; From 311a7656c0bcdb06c37c9c99666c451e70025259 Mon Sep 17 00:00:00 2001 From: Josh Slocum Date: Wed, 26 Jan 2022 12:22:20 -0600 Subject: [PATCH 169/413] Fixing change feed getting stuck not quite caught up, which causes empty version problems --- fdbserver/storageserver.actor.cpp | 10 ++++++---- 1 file changed, 6 insertions(+), 4 deletions(-) diff --git a/fdbserver/storageserver.actor.cpp b/fdbserver/storageserver.actor.cpp index 16211f0c74..0303849392 100644 --- a/fdbserver/storageserver.actor.cpp +++ b/fdbserver/storageserver.actor.cpp @@ -1746,8 +1746,8 @@ MutationsAndVersionRef filterMutations(Arena& arena, #define DEBUG_SS_CF_ID ""_sr #define DEBUG_SS_CF_BEGIN_VERSION invalidVersion #define DEBUG_SS_CFM(ssId, cfId, v) \ - ssId.toString().substr(0, 4) == DEBUG_SS_ID&& cfId.printable().substr(0, 6) == DEBUG_SS_CF_ID&& v == \ - DEBUG_SS_CF_BEGIN_VERSION + ssId.toString().substr(0, 4) == DEBUG_SS_ID&& cfId.printable().substr(0, 6) == DEBUG_SS_CF_ID && \ + (v == DEBUG_SS_CF_BEGIN_VERSION || latestVersion == DEBUG_SS_CF_BEGIN_VERSION) ACTOR Future> getChangeFeedMutations(StorageServer* data, ChangeFeedStreamRequest req, @@ -1889,6 +1889,7 @@ ACTOR Future> getChangeFeedMutations(Stor reply = memoryReply; } + bool gotAll = remainingLimitBytes > 0 && remainingDurableBytes > 0 && data->version.get() == startVersion; Version finalVersion = std::min(req.end - 1, dequeVersion); if ((reply.mutations.empty() || reply.mutations.back().version < finalVersion) && remainingLimitBytes > 0 && remainingDurableBytes > 0) { @@ -1900,6 +1901,8 @@ ACTOR Future> getChangeFeedMutations(Stor } reply.mutations.push_back( reply.arena, MutationsAndVersionRef(finalVersion, finalVersion == dequeVersion ? dequeKnownCommit : 0)); + // if we add empty mutation, we gotAll + gotAll = true; } if (MUTATION_TRACKING_ENABLED) { @@ -1934,8 +1937,7 @@ ACTOR Future> getChangeFeedMutations(Stor // If the SS's version advanced at all during any of the waits, the read from memory may have missed some mutations, // so gotAll can only be true if data->version didn't change over the course of this actor - return std::make_pair(reply, - remainingLimitBytes > 0 && remainingDurableBytes > 0 && data->version.get() == startVersion); + return std::make_pair(reply, gotAll); } ACTOR Future localChangeFeedStream(StorageServer* data, From 992fc8ed7cb5bd4f8c54b38d121a1767e9264513 Mon Sep 17 00:00:00 2001 From: Josh Slocum Date: Wed, 26 Jan 2022 15:28:58 -0600 Subject: [PATCH 170/413] Using non-expanded clears in change feeds --- fdbserver/storageserver.actor.cpp | 151 ++++++++++++++++-------------- 1 file changed, 81 insertions(+), 70 deletions(-) diff --git a/fdbserver/storageserver.actor.cpp b/fdbserver/storageserver.actor.cpp index 16211f0c74..a4772486d9 100644 --- a/fdbserver/storageserver.actor.cpp +++ b/fdbserver/storageserver.actor.cpp @@ -3574,42 +3574,9 @@ Optional clipMutation(MutationRef const& m, KeyRangeRef range) { return Optional(); } -// Return true if the mutation need to be applied, otherwise (it's a CompareAndClear mutation and failed the comparison) -// false. -bool expandMutation(MutationRef& m, - StorageServer::VersionedData const& data, - UpdateEagerReadInfo* eager, - KeyRef eagerTrustedEnd, - Arena& ar) { +bool convertAtomicOp(MutationRef& m, StorageServer::VersionedData const& data, UpdateEagerReadInfo* eager, Arena& ar) { // After this function call, m should be copied into an arena immediately (before modifying data, shards, or eager) - if (m.type == MutationRef::ClearRange) { - // Expand the clear - const auto& d = data.atLatest(); - - // If another clear overlaps the beginning of this one, engulf it - auto i = d.lastLess(m.param1); - if (i && i->isClearTo() && i->getEndKey() >= m.param1) - m.param1 = i.key(); - - // If another clear overlaps the end of this one, engulf it; otherwise expand - i = d.lastLessOrEqual(m.param2); - if (i && i->isClearTo() && i->getEndKey() >= m.param2) { - m.param2 = i->getEndKey(); - } else if (SERVER_KNOBS->ENABLE_CLEAR_RANGE_EAGER_READS) { - // Expand to the next set or clear (from storage or latestVersion), and if it - // is a clear, engulf it as well - i = d.lower_bound(m.param2); - KeyRef endKeyAtStorageVersion = - m.param2 == eagerTrustedEnd ? eagerTrustedEnd : std::min(eager->getKeyEnd(m.param2), eagerTrustedEnd); - if (!i || endKeyAtStorageVersion < i.key()) - m.param2 = endKeyAtStorageVersion; - else if (i->isClearTo()) - m.param2 = i->getEndKey(); - else - m.param2 = i.key(); - } - } else if (m.type != MutationRef::SetValue && (m.type)) { - + if (m.type != MutationRef::ClearRange && m.type != MutationRef::SetValue) { Optional oldVal; auto it = data.atLatest().lastLessOrEqual(m.param1); if (it != data.atLatest().end() && it->isValue() && it.key() == m.param1) @@ -3660,22 +3627,53 @@ bool expandMutation(MutationRef& m, if (oldVal.present() && m.param2 == oldVal.get()) { m.type = MutationRef::ClearRange; m.param2 = keyAfter(m.param1, ar); - return expandMutation(m, data, eager, eagerTrustedEnd, ar); + return true; } return false; } m.type = MutationRef::SetValue; } - return true; } +void expandClear(MutationRef& m, + StorageServer::VersionedData const& data, + UpdateEagerReadInfo* eager, + KeyRef eagerTrustedEnd) { + // After this function call, m should be copied into an arena immediately (before modifying data, shards, or eager) + ASSERT(m.type == MutationRef::ClearRange); + // Expand the clear + const auto& d = data.atLatest(); + + // If another clear overlaps the beginning of this one, engulf it + auto i = d.lastLess(m.param1); + if (i && i->isClearTo() && i->getEndKey() >= m.param1) + m.param1 = i.key(); + + // If another clear overlaps the end of this one, engulf it; otherwise expand + i = d.lastLessOrEqual(m.param2); + if (i && i->isClearTo() && i->getEndKey() >= m.param2) { + m.param2 = i->getEndKey(); + } else if (SERVER_KNOBS->ENABLE_CLEAR_RANGE_EAGER_READS) { + // Expand to the next set or clear (from storage or latestVersion), and if it + // is a clear, engulf it as well + i = d.lower_bound(m.param2); + KeyRef endKeyAtStorageVersion = + m.param2 == eagerTrustedEnd ? eagerTrustedEnd : std::min(eager->getKeyEnd(m.param2), eagerTrustedEnd); + if (!i || endKeyAtStorageVersion < i.key()) + m.param2 = endKeyAtStorageVersion; + else if (i->isClearTo()) + m.param2 = i->getEndKey(); + else + m.param2 = i.key(); + } +} + void applyMutation(StorageServer* self, MutationRef const& m, Arena& arena, StorageServer::VersionedData& data, - Version version, - bool fromFetch) { + Version version) { // m is expected to be in arena already // Clear split keys are added to arena StorageMetrics metrics; @@ -3708,43 +3706,43 @@ void applyMutation(StorageServer* self, } data.insert(m.param1, ValueOrClearToRef::value(m.param2)); self->watches.trigger(m.param1); - - if (!fromFetch) { - for (auto& it : self->keyChangeFeed[m.param1]) { - if (!it->stopped) { - if (it->mutations.empty() || it->mutations.back().version != version) { - it->mutations.push_back(MutationsAndVersionRef(version, self->knownCommittedVersion)); - } - it->mutations.back().mutations.push_back_deep(it->mutations.back().arena(), m); - self->currentChangeFeeds.insert(it->id); - - DEBUG_MUTATION("ChangeFeedWriteSet", version, m, self->thisServerID) - .detail("Range", it->range) - .detail("ChangeFeedID", it->id); - } - } - } } else if (m.type == MutationRef::ClearRange) { data.erase(m.param1, m.param2); ASSERT(m.param2 > m.param1); ASSERT(!data.isClearContaining(data.atLatest(), m.param1)); data.insert(m.param1, ValueOrClearToRef::clearTo(m.param2)); self->watches.triggerRange(m.param1, m.param2); + } +} - if (!fromFetch) { - auto ranges = self->keyChangeFeed.intersectingRanges(KeyRangeRef(m.param1, m.param2)); - for (auto& r : ranges) { - for (auto& it : r.value()) { - if (!it->stopped) { - if (it->mutations.empty() || it->mutations.back().version != version) { - it->mutations.push_back(MutationsAndVersionRef(version, self->knownCommittedVersion)); - } - it->mutations.back().mutations.push_back_deep(it->mutations.back().arena(), m); - self->currentChangeFeeds.insert(it->id); - DEBUG_MUTATION("ChangeFeedWriteClear", version, m, self->thisServerID) - .detail("Range", it->range) - .detail("ChangeFeedID", it->id); +void applyChangeFeedMutation(StorageServer* self, MutationRef const& m, Version version) { + if (m.type == MutationRef::SetValue) { + for (auto& it : self->keyChangeFeed[m.param1]) { + if (!it->stopped) { + if (it->mutations.empty() || it->mutations.back().version != version) { + it->mutations.push_back(MutationsAndVersionRef(version, self->knownCommittedVersion)); + } + it->mutations.back().mutations.push_back_deep(it->mutations.back().arena(), m); + self->currentChangeFeeds.insert(it->id); + + DEBUG_MUTATION("ChangeFeedWriteSet", version, m, self->thisServerID) + .detail("Range", it->range) + .detail("ChangeFeedID", it->id); + } + } + } else if (m.type == MutationRef::ClearRange) { + auto ranges = self->keyChangeFeed.intersectingRanges(KeyRangeRef(m.param1, m.param2)); + for (auto& r : ranges) { + for (auto& it : r.value()) { + if (!it->stopped) { + if (it->mutations.empty() || it->mutations.back().version != version) { + it->mutations.push_back(MutationsAndVersionRef(version, self->knownCommittedVersion)); } + it->mutations.back().mutations.push_back_deep(it->mutations.back().arena(), m); + self->currentChangeFeeds.insert(it->id); + DEBUG_MUTATION("ChangeFeedWriteClear", version, m, self->thisServerID) + .detail("Range", it->range) + .detail("ChangeFeedID", it->id); } } } @@ -4894,16 +4892,29 @@ void StorageServer::addMutation(Version version, KeyRangeRef const& shard, UpdateEagerReadInfo* eagerReads) { MutationRef expanded = mutation; + MutationRef + nonExpanded; // need to keep non-expanded but atomic converted version of clear mutations for change feeds auto& mLog = addVersionToMutationLog(version); - if (!expandMutation(expanded, data(), eagerReads, shard.end, mLog.arena())) { + if (!convertAtomicOp(expanded, data(), eagerReads, mLog.arena())) { return; } + if (expanded.type == MutationRef::ClearRange) { + nonExpanded = expanded; + expandClear(expanded, data(), eagerReads, shard.end); + } expanded = addMutationToMutationLog(mLog, expanded); DEBUG_MUTATION("applyMutation", version, expanded, thisServerID) .detail("ShardBegin", shard.begin) .detail("ShardEnd", shard.end); - applyMutation(this, expanded, mLog.arena(), mutableData(), version, fromFetch); + + if (!fromFetch) { + // have to do change feed before applyMutation because nonExpanded wasn't copied into the mutation log arena, + // and thus would go out of scope + applyChangeFeedMutation(this, expanded.type == MutationRef::ClearRange ? nonExpanded : expanded, version); + } + applyMutation(this, expanded, mLog.arena(), mutableData(), version); + // printf("\nSSUpdate: Printing versioned tree after applying mutation\n"); // mutableData().printTree(version); } From 74c8167745aaf02c03e8bd2b5a62ebd91425fb50 Mon Sep 17 00:00:00 2001 From: Josh Slocum Date: Thu, 27 Jan 2022 10:04:06 -0600 Subject: [PATCH 171/413] Not using krm for blob manager worker mapping recovery --- fdbserver/BlobManager.actor.cpp | 33 ++++++++++++++++++++++----------- 1 file changed, 22 insertions(+), 11 deletions(-) diff --git a/fdbserver/BlobManager.actor.cpp b/fdbserver/BlobManager.actor.cpp index cb8fe38e2e..3a42011216 100644 --- a/fdbserver/BlobManager.actor.cpp +++ b/fdbserver/BlobManager.actor.cpp @@ -607,6 +607,13 @@ ACTOR Future writeInitialGranuleMapping(Reference bmData, j++; } wait(tr->commit()); + if (BM_DEBUG) { + for (int k = 0; k < j; k++) { + fmt::print("Persisted initial mapping for [{0} - {1})\n", + boundaries[i + k].printable(), + boundaries[i + k + 1].printable()); + } + } break; } catch (Error& e) { if (BM_DEBUG) { @@ -1516,23 +1523,23 @@ ACTOR Future recoverBlobManager(Reference bmData) { // Step 3. Get the latest known mapping of granules to blob workers (i.e. assignments) // This must happen causally AFTER reading the split boundaries, since the blob workers can clear the split // boundaries for a granule as part of persisting their assignment. - state KeyRef beginKey = normalKeys.begin; + state KeyRef beginKey = blobGranuleMappingKeys.begin; loop { try { tr->setOption(FDBTransactionOptions::ACCESS_SYSTEM_KEYS); tr->setOption(FDBTransactionOptions::PRIORITY_SYSTEM_IMMEDIATE); // TODO: replace row limit with knob - KeyRange nextRange(KeyRangeRef(beginKey, normalKeys.end)); - RangeResult results = wait(krmGetRanges( - tr, blobGranuleMappingKeys.begin, nextRange, rowLimit, GetRangeLimits::BYTE_LIMIT_UNLIMITED)); - Key lastEndKey; + KeyRange nextRange(KeyRangeRef(beginKey, blobGranuleMappingKeys.end)); + // using the krm functions can produce incorrect behavior here as it does weird stuff with beginKey + state GetRangeLimits limits(rowLimit, GetRangeLimits::BYTE_LIMIT_UNLIMITED); + limits.minRows = 2; + RangeResult results = wait(tr->getRange(nextRange, limits)); // Add the mappings to our in memory key range map for (int rangeIdx = 0; rangeIdx < results.size() - 1; rangeIdx++) { - Key granuleStartKey = results[rangeIdx].key; - Key granuleEndKey = results[rangeIdx + 1].key; - lastEndKey = granuleEndKey; + Key granuleStartKey = results[rangeIdx].key.removePrefix(blobGranuleMappingKeys.begin); + Key granuleEndKey = results[rangeIdx + 1].key.removePrefix(blobGranuleMappingKeys.begin); if (results[rangeIdx].value.size()) { // note: if the old owner is dead, we handle this in rangeAssigner UID existingOwner = decodeBlobGranuleMappingValue(results[rangeIdx].value); @@ -1546,18 +1553,19 @@ ACTOR Future recoverBlobManager(Reference bmData) { } } else { if (BM_DEBUG) { - fmt::print(" [{0} - {1})=\n", + fmt::print(" [{0} - {1})\n", results[rangeIdx].key.printable(), results[rangeIdx + 1].key.printable()); } } } - if (!results.more) { + if (!results.more || results.size() <= 1) { break; } - beginKey = lastEndKey; + // re-read last key to get range that starts there + beginKey = results.back().key; } catch (Error& e) { if (BM_DEBUG) { fmt::print("BM {0} got error reading granule mapping during recovery: {1}\n", bmData->epoch, e.name()); @@ -1611,6 +1619,9 @@ ACTOR Future recoverBlobManager(Reference bmData) { for (auto& range : workerAssignments.intersectingRanges(normalKeys)) { if (!range.value().present()) { + /*if (BM_DEBUG) { + fmt::print(" [{0} - {1}) invalid\n", range.begin().printable(), range.end().printable()); + }*/ continue; } From bf6ed8348f0a13e4dc991f4cbccb43fed916af7b Mon Sep 17 00:00:00 2001 From: Josh Slocum Date: Thu, 27 Jan 2022 10:26:21 -0600 Subject: [PATCH 172/413] A couple workload improvements --- fdbserver/BlobManager.actor.cpp | 8 +++---- .../BlobGranuleCorrectnessWorkload.actor.cpp | 21 ++++++++++++------- tests/fast/BlobGranuleVerifySmall.toml | 1 + tests/fast/BlobGranuleVerifySmallClean.toml | 1 + 4 files changed, 19 insertions(+), 12 deletions(-) diff --git a/fdbserver/BlobManager.actor.cpp b/fdbserver/BlobManager.actor.cpp index 3a42011216..1117a25a96 100644 --- a/fdbserver/BlobManager.actor.cpp +++ b/fdbserver/BlobManager.actor.cpp @@ -1547,15 +1547,13 @@ ACTOR Future recoverBlobManager(Reference bmData) { bmData->knownBlobRanges.insert(KeyRangeRef(granuleStartKey, granuleEndKey), true); if (BM_DEBUG) { fmt::print(" [{0} - {1})={2}\n", - results[rangeIdx].key.printable(), - results[rangeIdx + 1].key.printable(), + granuleStartKey.printable(), + granuleEndKey.printable(), results[rangeIdx].value.printable()); } } else { if (BM_DEBUG) { - fmt::print(" [{0} - {1})\n", - results[rangeIdx].key.printable(), - results[rangeIdx + 1].key.printable()); + fmt::print(" [{0} - {1})\n", granuleStartKey.printable(), granuleEndKey.printable()); } } } diff --git a/fdbserver/workloads/BlobGranuleCorrectnessWorkload.actor.cpp b/fdbserver/workloads/BlobGranuleCorrectnessWorkload.actor.cpp index fe33ae9f47..175c27b71a 100644 --- a/fdbserver/workloads/BlobGranuleCorrectnessWorkload.actor.cpp +++ b/fdbserver/workloads/BlobGranuleCorrectnessWorkload.actor.cpp @@ -543,10 +543,10 @@ struct BlobGranuleCorrectnessWorkload : TestWorkload { state double last = now(); state double targetBytesReadPerQuery = SERVER_KNOBS->BG_SNAPSHOT_FILE_TARGET_BYTES * 2.0 / deterministicRandom()->randomInt(1, 11); - state int targetKeysReadPerQuery = - (int)(targetBytesReadPerQuery / (threadData->targetValLength * threadData->targetIDsPerKey)); - // read at higher read rate than write rate - state double targetTps = threadData->targetByteRate * 2 / targetBytesReadPerQuery; + + // read at higher read rate than write rate to validate data + state double targetReadBytesPerSec = threadData->targetByteRate * 4; + ASSERT(targetReadBytesPerSec > 0); state Version readVersion; @@ -574,8 +574,13 @@ struct BlobGranuleCorrectnessWorkload : TestWorkload { auto endKeyIt = threadData->keyData.find(startKey); ASSERT(endKeyIt != threadData->keyData.end()); - for (int i = 0; i < targetKeysReadPerQuery && endKeyIt != threadData->keyData.end(); i++, endKeyIt++) { + int targetQueryBytes = (deterministicRandom()->randomInt(1, 20) * targetBytesReadPerQuery) / 10; + int estimatedQueryBytes = 0; + for (int i = 0; estimatedQueryBytes < targetQueryBytes && endKeyIt != threadData->keyData.end(); + i++, endKeyIt++) { // iterate forward until end or target keys have passed + estimatedQueryBytes += (1 + endKeyIt->second.writes.size() - endKeyIt->second.nextClearIdx) * + threadData->targetValLength; } state uint32_t endKey; @@ -609,8 +614,11 @@ struct BlobGranuleCorrectnessWorkload : TestWorkload { wait(self->readFromBlob(cx, self, range, readVersion)); self->validateResult(threadData, blob, startKey, endKey, 0, readVersion); + int resultBytes = blob.first.expectedSize(); threadData->rowsRead += blob.first.size(); - threadData->bytesRead += blob.first.expectedSize(); + threadData->bytesRead += resultBytes; + + wait(poisson(&last, (resultBytes + 1) / targetReadBytesPerSec)); } catch (Error& e) { if (e.code() == error_code_operation_cancelled) { throw; @@ -624,7 +632,6 @@ struct BlobGranuleCorrectnessWorkload : TestWorkload { } } } - wait(poisson(&last, 1.0 / targetTps)); } } diff --git a/tests/fast/BlobGranuleVerifySmall.toml b/tests/fast/BlobGranuleVerifySmall.toml index 03022154a2..60296446eb 100644 --- a/tests/fast/BlobGranuleVerifySmall.toml +++ b/tests/fast/BlobGranuleVerifySmall.toml @@ -7,6 +7,7 @@ testTitle = 'BlobGranuleVerifySmall' [[test.workload]] testName = 'WriteDuringRead' testDuration = 120.0 + useSystemKeys = false [[test.workload]] testName = 'BlobGranuleVerifier' diff --git a/tests/fast/BlobGranuleVerifySmallClean.toml b/tests/fast/BlobGranuleVerifySmallClean.toml index a758c46d88..0a72412d22 100644 --- a/tests/fast/BlobGranuleVerifySmallClean.toml +++ b/tests/fast/BlobGranuleVerifySmallClean.toml @@ -7,6 +7,7 @@ testTitle = 'BlobGranuleVerifySmallClean' [[test.workload]] testName = 'WriteDuringRead' testDuration = 120.0 + useSystemKeys = false [[test.workload]] testName = 'BlobGranuleVerifier' From bc7b76b40757474f1c6d70a9b80a823944ff0a0c Mon Sep 17 00:00:00 2001 From: Josh Slocum Date: Thu, 27 Jan 2022 14:25:08 -0600 Subject: [PATCH 173/413] Fix for gotAll in getChangeFeedMutations --- fdbclient/NativeAPI.actor.cpp | 14 ++++++++++++-- fdbserver/storageserver.actor.cpp | 6 ++++-- 2 files changed, 16 insertions(+), 4 deletions(-) diff --git a/fdbclient/NativeAPI.actor.cpp b/fdbclient/NativeAPI.actor.cpp index 9d5fb8ab62..71ceae180a 100644 --- a/fdbclient/NativeAPI.actor.cpp +++ b/fdbclient/NativeAPI.actor.cpp @@ -7389,12 +7389,22 @@ ACTOR Future partialChangeFeedStream(StorageServerInterface interf, } else { // TODO REMOVE eventually, useful for debugging for now if (!rep.mutations[resultLoc].mutations.empty()) { - fmt::print("non-empty mutations ({0}), but versions out of order from {1}! mv={2}, " - "nv={3}\n", + fmt::print("non-empty mutations ({0}), but versions out of order from {1} for {2} cf " + "{3}! mv={4}, nv={5}\n", rep.mutations.size(), interf.id().toString().substr(0, 4), + idx, + feedData->id.toString().substr(0, 6), rep.mutations[resultLoc].version, nextVersion); + for (auto& it : rep.mutations[resultLoc].mutations) { + if (it.type == MutationRef::SetValue) { + printf(" %s=", it.param1.printable().c_str()); + } else { + printf( + " %s - %s", it.param1.printable().c_str(), it.param2.printable().c_str()); + } + } } ASSERT(rep.mutations[resultLoc].mutations.empty()); } diff --git a/fdbserver/storageserver.actor.cpp b/fdbserver/storageserver.actor.cpp index 8be69f878d..1691eb363d 100644 --- a/fdbserver/storageserver.actor.cpp +++ b/fdbserver/storageserver.actor.cpp @@ -1901,8 +1901,10 @@ ACTOR Future> getChangeFeedMutations(Stor } reply.mutations.push_back( reply.arena, MutationsAndVersionRef(finalVersion, finalVersion == dequeVersion ? dequeKnownCommit : 0)); - // if we add empty mutation, we gotAll - gotAll = true; + // if we add empty mutation after the last thing in memory, and didn't read from disk, gotAll is true + if (data->version.get() == startVersion) { + gotAll = true; + } } if (MUTATION_TRACKING_ENABLED) { From c26e11c2c310eb47d8433be0bdc6e4ec3f49409d Mon Sep 17 00:00:00 2001 From: Josh Slocum Date: Thu, 27 Jan 2022 18:53:14 -0600 Subject: [PATCH 174/413] Adding extra debugid to change feed stream requests to track things better --- fdbclient/NativeAPI.actor.cpp | 30 ++++++++++++++++++++---------- fdbclient/StorageServerInterface.h | 4 +++- fdbserver/storageserver.actor.cpp | 4 +++- 3 files changed, 26 insertions(+), 12 deletions(-) diff --git a/fdbclient/NativeAPI.actor.cpp b/fdbclient/NativeAPI.actor.cpp index 71ceae180a..89bc339b6e 100644 --- a/fdbclient/NativeAPI.actor.cpp +++ b/fdbclient/NativeAPI.actor.cpp @@ -7323,7 +7323,8 @@ ACTOR Future partialChangeFeedStream(StorageServerInterface interf, Reference feedData, Reference storageData, int idx /* TODO REMOVE this param after correctness clean */, - KeyRange range /* TODO REMOVE this param after correctness clean */) { + KeyRange range /* TODO REMOVE this param after correctness clean */, + UID debugID /*TODO REMOVE this param after correctness clean*/) { // calling lastReturnedVersion's callbacks could cause us to be cancelled state Promise refresh = feedData->refresh; @@ -7346,11 +7347,11 @@ ACTOR Future partialChangeFeedStream(StorageServerInterface interf, when(state ChangeFeedStreamReply rep = waitNext(replyStream.getFuture())) { if (DEBUG_CF_VERSION_RANGE( feedData->id, rep.mutations.front().version, rep.mutations.back().version)) { - fmt::print(" single {0} {1}: response {2} - {3} ({4}), atLatest={5}, rep.atLatest={6}, " - "notAtLatest={7}, " - "minSV={8}\n", + fmt::print(" single {0} {1} {2}: response {3} - {4} ({5}), atLatest={6}, rep.atLatest={7}, " + "notAtLatest={8}, minSV={9}\n", idx, interf.id().toString().substr(0, 4), + debugID.toString().substr(0, 8).c_str(), rep.mutations.front().version, rep.mutations.back().version, rep.mutations.size(), @@ -7607,6 +7608,7 @@ ACTOR Future mergeChangeFeedStream(Reference db, state std::vector> onErrors(interfs.size()); state std::vector streams(interfs.size()); + std::vector debugIDs; results->streams.clear(); for (auto& it : interfs) { ChangeFeedStreamRequest req; @@ -7614,6 +7616,9 @@ ACTOR Future mergeChangeFeedStream(Reference db, req.begin = *begin; req.end = end; req.range = it.second; + UID debugID = deterministicRandom()->randomUniqueID(); + debugIDs.push_back(debugID); + req.debugID = debugID; results->streams.push_back(it.first.changeFeedStream.getReplyStream(req)); } @@ -7644,7 +7649,8 @@ ACTOR Future mergeChangeFeedStream(Reference db, results, results->storageData[i], i, - interfs[i].second); + interfs[i].second, + debugIDs[i]); } wait(onCFErrors(onErrors) || doCFMerge(results, interfs, streams, begin, end)); @@ -7689,7 +7695,8 @@ ACTOR Future doSingleCFStream(KeyRange range, Reference results, Key rangeID, Version* begin, - Version end) { + Version end, + UID debugID /*TODO REMOVE this parameter once BG is correctness clean*/) { state Promise refresh = results->refresh; ASSERT(results->streams.size() == 1); ASSERT(results->storageData.size() == 1); @@ -7714,8 +7721,9 @@ ACTOR Future doSingleCFStream(KeyRange range, // empty versions can come out of order, as we sometimes send explicit empty versions when restarting a // stream. Anything with mutations should be strictly greater than lastReturnedVersion if (feedReply.mutations.front().version <= results->lastReturnedVersion.get()) { - fmt::print("out of order mutation for CF {0} from ({1}) {2}! {3} < {4}\n", + fmt::print("out of order mutation for CF {0} Req {1} from ({2}) {3}! {4} < {5}\n", rangeID.toString().substr(0, 6), + debugID.toString().substr(0, 8), results->storageData.size(), results->storageData[0]->id.toString().substr(0, 4).c_str(), feedReply.mutations.front().version, @@ -7732,8 +7740,8 @@ ACTOR Future doSingleCFStream(KeyRange range, } if (DEBUG_CF_VERSION(rangeID, feedReply.mutations.back().version)) { - fmt::print("CFLR (single): {0} ({1}), atLatest={2}, rep.atLatest={3}, notAtLatest={4}, " - "minSV={5}\n", + fmt::print("CFLR (single) {0}: {1} ({2}), atLatest={3}, rep.atLatest={4}, notAtLatest={5}, minSV={6}\n", + debugID.toString().substr(0, 8), feedReply.mutations.back().version, feedReply.mutations.size(), atLatest ? "T" : "F", @@ -7771,10 +7779,12 @@ ACTOR Future singleChangeFeedStream(Reference db, Version end) { state Database cx(db); state ChangeFeedStreamRequest req; + state UID debugID = deterministicRandom()->randomUniqueID(); req.rangeID = rangeID; req.begin = *begin; req.end = end; req.range = range; + req.debugID = debugID; results->streams.clear(); @@ -7793,7 +7803,7 @@ ACTOR Future singleChangeFeedStream(Reference db, results->notAtLatest.set(1); refresh.send(Void()); - wait(results->streams[0].onError() || doSingleCFStream(range, results, rangeID, begin, end)); + wait(results->streams[0].onError() || doSingleCFStream(range, results, rangeID, begin, end, debugID)); return Void(); } diff --git a/fdbclient/StorageServerInterface.h b/fdbclient/StorageServerInterface.h index 9e40f95e1f..ea58e3b3ed 100644 --- a/fdbclient/StorageServerInterface.h +++ b/fdbclient/StorageServerInterface.h @@ -715,12 +715,14 @@ struct ChangeFeedStreamRequest { Version begin = 0; Version end = 0; KeyRange range; + // TODO REMOVE once BG is correctness clean!! Useful for debugging + UID debugID; ReplyPromiseStream reply; ChangeFeedStreamRequest() {} template void serialize(Ar& ar) { - serializer(ar, rangeID, begin, end, range, reply, spanContext, arena); + serializer(ar, rangeID, begin, end, range, reply, spanContext, debugID, arena); } }; diff --git a/fdbserver/storageserver.actor.cpp b/fdbserver/storageserver.actor.cpp index 1691eb363d..b3c099f187 100644 --- a/fdbserver/storageserver.actor.cpp +++ b/fdbserver/storageserver.actor.cpp @@ -1996,7 +1996,9 @@ ACTOR Future stopChangeFeedOnMove(StorageServer* data, ChangeFeedStreamReq ACTOR Future changeFeedStreamQ(StorageServer* data, ChangeFeedStreamRequest req) { state Span span("SS:getChangeFeedStream"_loc, { req.spanContext }); state bool atLatest = false; - state UID streamUID = deterministicRandom()->randomUniqueID(); + // TODO CHANGE BACK after BG is correctness clean + // state UID streamUID = deterministicRandom()->randomUniqueID(); + state UID streamUID = req.debugID; state bool removeUID = false; state Optional blockedVersion; req.reply.setByteLimit(SERVER_KNOBS->RANGESTREAM_LIMIT_BYTES); From 91df569fdd7f37653b7678d808dcf439bc3335f8 Mon Sep 17 00:00:00 2001 From: Josh Slocum Date: Thu, 27 Jan 2022 18:46:22 -0600 Subject: [PATCH 175/413] Fetch Change Feed metadata at the start of fetchKeys to fix mutation race --- fdbserver/storageserver.actor.cpp | 116 +++++++++++------- .../BlobGranuleCorrectnessWorkload.actor.cpp | 5 + 2 files changed, 77 insertions(+), 44 deletions(-) diff --git a/fdbserver/storageserver.actor.cpp b/fdbserver/storageserver.actor.cpp index b3c099f187..ea14244b74 100644 --- a/fdbserver/storageserver.actor.cpp +++ b/fdbserver/storageserver.actor.cpp @@ -4146,45 +4146,20 @@ ACTOR Future fetchChangeFeedApplier(StorageServer* data, } ACTOR Future fetchChangeFeed(StorageServer* data, - Key rangeId, - KeyRange range, - bool stopped, + Reference changeFeedInfo, Version fetchVersion) { - state Reference changeFeedInfo; wait(delay(0)); // allow this actor to be cancelled by removals - state bool existing = data->uidChangeFeed.count(rangeId); TraceEvent(SevDebug, "FetchChangeFeed", data->thisServerID) - .detail("RangeID", rangeId.printable()) - .detail("Range", range.toString()) - .detail("FetchVersion", fetchVersion) - .detail("Existing", existing); - - if (!existing) { - changeFeedInfo = Reference(new ChangeFeedInfo()); - changeFeedInfo->range = range; - changeFeedInfo->id = rangeId; - changeFeedInfo->stopped = stopped; - data->uidChangeFeed[rangeId] = changeFeedInfo; - auto rs = data->keyChangeFeed.modify(range); - for (auto r = rs.begin(); r != rs.end(); ++r) { - r->value().push_back(changeFeedInfo); - } - data->keyChangeFeed.coalesce(range.contents()); - auto& mLV = data->addVersionToMutationLog(data->data().getLatestVersion()); - data->addMutationToMutationLog( - mLV, - MutationRef(MutationRef::SetValue, - persistChangeFeedKeys.begin.toString() + rangeId.toString(), - changeFeedValue(range, invalidVersion, ChangeFeedStatus::CHANGE_FEED_CREATE))); - } else { - changeFeedInfo = data->uidChangeFeed[rangeId]; - } - + .detail("RangeID", changeFeedInfo->id.printable()) + .detail("Range", changeFeedInfo->range.toString()) + .detail("FetchVersion", fetchVersion); loop { try { - wait(fetchChangeFeedApplier(data, changeFeedInfo, rangeId, range, fetchVersion, existing)); - data->fetchingChangeFeeds.insert(rangeId); + // TODO clean up existing param for !existing + wait(fetchChangeFeedApplier( + data, changeFeedInfo, changeFeedInfo->id, changeFeedInfo->range, fetchVersion, false)); + data->fetchingChangeFeeds.insert(changeFeedInfo->id); return Void(); } catch (Error& e) { if (e.code() != error_code_change_feed_not_registered) { @@ -4195,17 +4170,62 @@ ACTOR Future fetchChangeFeed(StorageServer* data, } } -ACTOR Future dispatchChangeFeeds(StorageServer* data, UID fetchKeysID, KeyRange keys, Version fetchVersion) { +ACTOR Future> fetchChangeFeedMetadata(StorageServer* data, KeyRange keys, Version fetchVersion) { + std::vector feeds = wait(data->cx->getOverlappingChangeFeeds(keys, fetchVersion + 1)); + std::vector feedIds; + feedIds.reserve(feeds.size()); + // create change feed metadata if it does not exist + for (auto& cfEntry : feeds) { + feedIds.push_back(cfEntry.rangeId); + bool existing = data->uidChangeFeed.count(cfEntry.rangeId); + if (!existing) { + TraceEvent(SevDebug, "FetchChangeFeedMetadata", data->thisServerID) + .detail("RangeID", cfEntry.rangeId.printable()) + .detail("Range", cfEntry.range.toString()) + .detail("FetchVersion", fetchVersion) + .detail("Stopped", cfEntry.stopped); + + Reference changeFeedInfo = Reference(new ChangeFeedInfo()); + changeFeedInfo->range = cfEntry.range; + changeFeedInfo->id = cfEntry.rangeId; + + changeFeedInfo->stopped = cfEntry.stopped; + data->uidChangeFeed[cfEntry.rangeId] = changeFeedInfo; + auto rs = data->keyChangeFeed.modify(cfEntry.range); + for (auto r = rs.begin(); r != rs.end(); ++r) { + r->value().push_back(changeFeedInfo); + } + data->keyChangeFeed.coalesce(cfEntry.range.contents()); + auto& mLV = data->addVersionToMutationLog(data->data().getLatestVersion()); + data->addMutationToMutationLog( + mLV, + MutationRef(MutationRef::SetValue, + persistChangeFeedKeys.begin.toString() + cfEntry.rangeId.toString(), + changeFeedValue(cfEntry.range, invalidVersion, ChangeFeedStatus::CHANGE_FEED_CREATE))); + } + } + return feedIds; +} + +ACTOR Future dispatchChangeFeeds(StorageServer* data, + UID fetchKeysID, + KeyRange keys, + Version fetchVersion, + std::vector feedIds) { + // find overlapping range feeds state std::map> feedFetches; state PromiseStream removals; data->changeFeedRemovals[fetchKeysID] = removals; try { - state std::vector feeds = - wait(data->cx->getOverlappingChangeFeeds(keys, fetchVersion + 1)); + // TODO add trace events for some of these - for (auto& feed : feeds) { - feedFetches[feed.rangeId] = fetchChangeFeed(data, feed.rangeId, feed.range, feed.stopped, fetchVersion); + for (auto& feedId : feedIds) { + auto feedIt = data->uidChangeFeed.find(feedId); + // TODO REMOVE this assert once we enable change feed deletion + ASSERT(feedIt != data->uidChangeFeed.end()); + Reference feed = feedIt->second; + feedFetches[feed->id] = fetchChangeFeed(data, feed, fetchVersion); } loop { @@ -4264,7 +4284,10 @@ ACTOR Future fetchKeys(StorageServer* data, AddingShard* shard) { TraceEvent(SevDebug, interval.begin(), data->thisServerID) .detail("KeyBegin", shard->keys.begin) - .detail("KeyEnd", shard->keys.end); + .detail("KeyEnd", shard->keys.end) + .detail("Version", data->version.get()); + + state Future> fetchCFMetadata = fetchChangeFeedMetadata(data, keys, data->version.get()); validate(data); @@ -4297,6 +4320,12 @@ ACTOR Future fetchKeys(StorageServer* data, AddingShard* shard) { // Fetch keys gets called while the update actor is processing mutations. data->version will not be updated // until all mutations for a version have been processed. We need to take the durableVersionLock to ensure // data->version is greater than the version of the mutation which caused the fetch to be initiated. + + // We must also ensure we have fetched all change feed metadata BEFORE changing the phase to fetching to ensure + // change feed mutations get applied correctly + state std::vector changeFeedsToFetch; + std::vector _cfToFetch = wait(fetchCFMetadata); + changeFeedsToFetch = _cfToFetch; wait(data->durableVersionLock.take()); shard->phase = AddingShard::Fetching; @@ -4465,7 +4494,7 @@ ACTOR Future fetchKeys(StorageServer* data, AddingShard* shard) { // we have written) state Future fetchDurable = data->durableVersion.whenAtLeast(data->storageVersion() + 1); - wait(dispatchChangeFeeds(data, fetchKeysID, keys, fetchVersion)); + wait(dispatchChangeFeeds(data, fetchKeysID, keys, fetchVersion, changeFeedsToFetch)); holdingFKPL.release(); wait(fetchDurable); @@ -4960,10 +4989,9 @@ public: applyPrivateData(data, m); } } else { - // FIXME: enable when DEBUG_MUTATION is active - // for(auto m = changes[c].mutations.begin(); m; ++m) { - // DEBUG_MUTATION("SSUpdateMutation", changes[c].version, *m, data->thisServerID); - //} + if (MUTATION_TRACKING_ENABLED) { + DEBUG_MUTATION("SSUpdateMutation", ver, m, data->thisServerID).detail("FromFetch", fromFetch); + } splitMutation(data, data->shards, m, ver, fromFetch); } diff --git a/fdbserver/workloads/BlobGranuleCorrectnessWorkload.actor.cpp b/fdbserver/workloads/BlobGranuleCorrectnessWorkload.actor.cpp index 175c27b71a..d8f0495f40 100644 --- a/fdbserver/workloads/BlobGranuleCorrectnessWorkload.actor.cpp +++ b/fdbserver/workloads/BlobGranuleCorrectnessWorkload.actor.cpp @@ -341,6 +341,11 @@ struct BlobGranuleCorrectnessWorkload : TestWorkload { return; } + TraceEvent ev(SevError, "BGMismatch"); + ev.detail("DirectoryID", format("%08x", threadData->directoryID)) + .detail("RangeStart", format("%08x", startKey)) + .detail("RangeEnd", format("%08x", endKey)) + .detail("Version", readVersion); fmt::print("Found mismatch! Request for dir {0} [{1} - {2}) @ {3}\n", format("%08x", threadData->directoryID), format("%08x", startKey), From df1a21564b6ca7fa1ee2390ba50ba5cf5d0532ad Mon Sep 17 00:00:00 2001 From: Josh Slocum Date: Fri, 28 Jan 2022 06:21:07 -0600 Subject: [PATCH 176/413] Fixed Blob Worker Rollback issue --- fdbserver/BlobWorker.actor.cpp | 12 +++++++++--- 1 file changed, 9 insertions(+), 3 deletions(-) diff --git a/fdbserver/BlobWorker.actor.cpp b/fdbserver/BlobWorker.actor.cpp index 84b3c9c9c5..0dfabcb3e4 100644 --- a/fdbserver/BlobWorker.actor.cpp +++ b/fdbserver/BlobWorker.actor.cpp @@ -1039,10 +1039,16 @@ static Version doGranuleRollback(Reference metadata, // Track that this rollback happened, since we have to re-read mutations up to the rollback // Add this rollback to in progress, and put all completed ones back in progress rollbacksInProgress.push_back(std::pair(rollbackVersion, mutationVersion)); - for (int i = rollbacksCompleted.size() - 1; i >= 0; i--) { - rollbacksInProgress.push_front(rollbacksCompleted[i]); + while (!rollbacksCompleted.empty()) { + if (rollbacksCompleted.back().first >= cfRollbackVersion) { + rollbacksInProgress.push_front(rollbacksCompleted.back()); + rollbacksCompleted.pop_back(); + } else { + // some rollbacks in completed could still have a delta file in flight after this rollback, they should + // remain in completed + break; + } } - rollbacksCompleted.clear(); } else { // No pending delta files to discard, just in-memory mutations From cfbb3f5b2bd5b304a60f82f0c48ad237f9e7d5c7 Mon Sep 17 00:00:00 2001 From: Josh Slocum Date: Fri, 28 Jan 2022 07:43:34 -0600 Subject: [PATCH 177/413] Adding random prefix to blob worker generated files --- fdbserver/BlobWorker.actor.cpp | 16 +++++++++------- 1 file changed, 9 insertions(+), 7 deletions(-) diff --git a/fdbserver/BlobWorker.actor.cpp b/fdbserver/BlobWorker.actor.cpp index 0dfabcb3e4..afc4b209b8 100644 --- a/fdbserver/BlobWorker.actor.cpp +++ b/fdbserver/BlobWorker.actor.cpp @@ -438,9 +438,10 @@ ACTOR Future writeDeltaFile(Reference bwData, Optional> oldGranuleComplete) { wait(delay(0, TaskPriority::BlobWorkerUpdateStorage)); - // TODO: this will induce S3 hotspotting, so we should rethink if we want to prefix the file name - // with the granuleID or just add it somewhere in the file name - state std::string fname = granuleID.toString() + "_T" + std::to_string((uint64_t)(1000.0 * now())) + "_V" + + // Prefix filename with random chars both to avoid hotspotting on granuleID, and to have unique file names if + // multiple blob workers try to create the exact same file at the same millisecond (which observably happens) + state std::string fname = deterministicRandom()->randomUniqueID().shortString() + "_" + granuleID.toString() + + "_T" + std::to_string((uint64_t)(1000.0 * now())) + "_V" + std::to_string(currentDeltaVersion) + ".delta"; state Value serialized = ObjectWriter::toValue(deltasToWrite, Unversioned()); @@ -538,10 +539,11 @@ ACTOR Future writeSnapshot(Reference bwData, Version version, PromiseStream rows, bool createGranuleHistory) { - // TODO: this will induce S3 hotspotting, so we should rethink if we want to prefix the file name - // with the granuleID or just add it somewhere in the file name - state std::string fname = granuleID.toString() + "_T" + std::to_string((uint64_t)(1000.0 * now())) + "_V" + - std::to_string(version) + ".snapshot"; + // Prefix filename with random chars both to avoid hotspotting on granuleID, and to have unique file names if + // multiple blob workers try to create the exact same file at the same millisecond (which observably happens) + state std::string fname = deterministicRandom()->randomUniqueID().shortString() + "_" + granuleID.toString() + + "_T" + std::to_string((uint64_t)(1000.0 * now())) + "_V" + std::to_string(version) + + ".snapshot"; state Arena arena; state GranuleSnapshot snapshot; From 10c3cc870fbfd0d1a066a8861fe6d5386f0551b4 Mon Sep 17 00:00:00 2001 From: Josh Slocum Date: Fri, 28 Jan 2022 09:25:58 -0600 Subject: [PATCH 178/413] Fixing new race in change feed cleanup while still working for fetch+cleanup race --- fdbserver/storageserver.actor.cpp | 59 +++++++++++++++++++++++++------ 1 file changed, 48 insertions(+), 11 deletions(-) diff --git a/fdbserver/storageserver.actor.cpp b/fdbserver/storageserver.actor.cpp index ea14244b74..fc41c113e6 100644 --- a/fdbserver/storageserver.actor.cpp +++ b/fdbserver/storageserver.actor.cpp @@ -633,6 +633,7 @@ public: std::set currentChangeFeeds; std::set fetchingChangeFeeds; std::unordered_map> changeFeedClientVersions; + std::unordered_map changeFeedCleanupDurable; // newestAvailableVersion[k] // == invalidVersion -> k is unavailable at all versions @@ -4154,6 +4155,18 @@ ACTOR Future fetchChangeFeed(StorageServer* data, .detail("RangeID", changeFeedInfo->id.printable()) .detail("Range", changeFeedInfo->range.toString()) .detail("FetchVersion", fetchVersion); + + auto cleanupPending = data->changeFeedCleanupDurable.find(changeFeedInfo->id); + if (cleanupPending != data->changeFeedCleanupDurable.end()) { + TraceEvent(SevDebug, "FetchChangeFeedWaitCleanup", data->thisServerID) + .detail("RangeID", changeFeedInfo->id.printable()) + .detail("Range", changeFeedInfo->range.toString()) + .detail("CleanupVersion", cleanupPending->second) + .detail("FetchVersion", fetchVersion); + wait(data->durableVersion.whenAtLeast(cleanupPending->second + 1)); + ASSERT(!data->changeFeedCleanupDurable.count(changeFeedInfo->id)); + } + loop { try { // TODO clean up existing param for !existing @@ -4176,14 +4189,18 @@ ACTOR Future> fetchChangeFeedMetadata(StorageServer* data, KeyR feedIds.reserve(feeds.size()); // create change feed metadata if it does not exist for (auto& cfEntry : feeds) { + bool cleanupPending = data->changeFeedCleanupDurable.count(cfEntry.rangeId); feedIds.push_back(cfEntry.rangeId); bool existing = data->uidChangeFeed.count(cfEntry.rangeId); if (!existing) { + TEST(cleanupPending); // Fetch change feed which is cleanup pending. This means there was a move away and a + // move back, this will remake the metadata TraceEvent(SevDebug, "FetchChangeFeedMetadata", data->thisServerID) .detail("RangeID", cfEntry.rangeId.printable()) .detail("Range", cfEntry.range.toString()) .detail("FetchVersion", fetchVersion) - .detail("Stopped", cfEntry.stopped); + .detail("Stopped", cfEntry.stopped) + .detail("CleanupPending", cleanupPending); Reference changeFeedInfo = Reference(new ChangeFeedInfo()); changeFeedInfo->range = cfEntry.range; @@ -4202,6 +4219,8 @@ ACTOR Future> fetchChangeFeedMetadata(StorageServer* data, KeyR MutationRef(MutationRef::SetValue, persistChangeFeedKeys.begin.toString() + cfEntry.rangeId.toString(), changeFeedValue(cfEntry.range, invalidVersion, ChangeFeedStatus::CHANGE_FEED_CREATE))); + } else { + ASSERT(!cleanupPending); } } return feedIds; @@ -4865,9 +4884,23 @@ void changeServerKeys(StorageServer* data, if (!foundAssigned) { // TODO REMOVE + + Version durableVersion = data->data().getLatestVersion(); TraceEvent(SevDebug, "ChangeFeedCleanup", data->thisServerID) .detail("FeedID", f.first) - .detail("Version", version); + .detail("Version", version) + .detail("DurableVersion", durableVersion); + + data->changeFeedCleanupDurable[f.first] = durableVersion; + + Key beginClearKey = f.first.withPrefix(persistChangeFeedKeys.begin); + auto& mLV = data->addVersionToMutationLog(durableVersion); + data->addMutationToMutationLog( + mLV, MutationRef(MutationRef::ClearRange, beginClearKey, keyAfter(beginClearKey))); + data->addMutationToMutationLog(mLV, + MutationRef(MutationRef::ClearRange, + changeFeedDurableKey(f.first, 0), + changeFeedDurableKey(f.first, version))); auto rs = data->keyChangeFeed.modify(f.second); for (auto r = rs.begin(); r != rs.end(); ++r) { @@ -4886,14 +4919,6 @@ void changeServerKeys(StorageServer* data, feed->second->newMutations.trigger(); data->uidChangeFeed.erase(feed); } - - Key beginClearKey = f.first.withPrefix(persistChangeFeedKeys.begin); - - // all fetching actors should be cancelled by now because removing=true and moved(), so it's safe to - // clear storage directly - data->storage.clearRange(KeyRangeRef(beginClearKey, keyAfter(beginClearKey))); - data->storage.clearRange( - KeyRangeRef(changeFeedDurableKey(f.first, 0), changeFeedDurableKey(f.first, version))); } else { // if just part of feed's range is moved away auto feed = data->uidChangeFeed.find(f.first); @@ -5196,7 +5221,8 @@ private: .detail("Range", changeFeedRange.toString()) .detail("Version", currentVersion); Key beginClearKey = changeFeedId.withPrefix(persistChangeFeedKeys.begin); - auto& mLV = data->addVersionToMutationLog(data->data().getLatestVersion()); + Version cleanupVersion = data->data().getLatestVersion(); + auto& mLV = data->addVersionToMutationLog(cleanupVersion); data->addMutationToMutationLog( mLV, MutationRef(MutationRef::ClearRange, beginClearKey, keyAfter(beginClearKey))); data->addMutationToMutationLog(mLV, @@ -5214,6 +5240,7 @@ private: } data->keyChangeFeed.coalesce(feed->second->range.contents()); data->uidChangeFeed.erase(feed); + data->changeFeedCleanupDurable[feed->first] = cleanupVersion; } if (addMutationToLog) { @@ -5845,6 +5872,16 @@ ACTOR Future updateStorage(StorageServer* data) { curFeed++; } + // remove any entries from changeFeedCleanupPending that were persisted + auto cfCleanup = data->changeFeedCleanupDurable.begin(); + while (cfCleanup != data->changeFeedCleanupDurable.end()) { + if (cfCleanup->second <= newOldestVersion) { + cfCleanup = data->changeFeedCleanupDurable.erase(cfCleanup); + } else { + cfCleanup++; + } + } + durableInProgress.send(Void()); wait(delay(0, TaskPriority::UpdateStorage)); // Setting durableInProgess could cause the storage server to shut // down, so delay to check for cancellation From ac1fd056dd9fd7d96f1ea0c40e3b35010fe1f12d Mon Sep 17 00:00:00 2001 From: Josh Slocum Date: Fri, 28 Jan 2022 10:44:58 -0600 Subject: [PATCH 179/413] Added change feed popped read guard for blob workers --- fdbclient/DatabaseContext.h | 3 ++- fdbclient/NativeAPI.actor.cpp | 40 ++++++++++++++++++++++++------ fdbclient/StorageServerInterface.h | 3 ++- fdbserver/BlobWorker.actor.cpp | 32 +++++++++++++++++------- fdbserver/storageserver.actor.cpp | 35 ++++++++++++++++++-------- flow/error_definitions.h | 1 + 6 files changed, 85 insertions(+), 29 deletions(-) diff --git a/fdbclient/DatabaseContext.h b/fdbclient/DatabaseContext.h index 1a89464fe4..f9ad1a63a8 100644 --- a/fdbclient/DatabaseContext.h +++ b/fdbclient/DatabaseContext.h @@ -295,7 +295,8 @@ public: Key rangeID, Version begin = 0, Version end = std::numeric_limits::max(), - KeyRange range = allKeys); + KeyRange range = allKeys, + bool canReadPopped = true); Future> getOverlappingChangeFeeds(KeyRangeRef ranges, Version minVersion); Future popChangeFeedMutations(Key rangeID, Version version); diff --git a/fdbclient/NativeAPI.actor.cpp b/fdbclient/NativeAPI.actor.cpp index 89bc339b6e..d7858d5ee6 100644 --- a/fdbclient/NativeAPI.actor.cpp +++ b/fdbclient/NativeAPI.actor.cpp @@ -7603,7 +7603,8 @@ ACTOR Future mergeChangeFeedStream(Reference db, Reference results, Key rangeID, Version* begin, - Version end) { + Version end, + bool canReadPopped) { state std::vector> fetchers(interfs.size()); state std::vector> onErrors(interfs.size()); state std::vector streams(interfs.size()); @@ -7616,6 +7617,7 @@ ACTOR Future mergeChangeFeedStream(Reference db, req.begin = *begin; req.end = end; req.range = it.second; + req.canReadPopped = canReadPopped; UID debugID = deterministicRandom()->randomUniqueID(); debugIDs.push_back(debugID); req.debugID = debugID; @@ -7651,6 +7653,13 @@ ACTOR Future mergeChangeFeedStream(Reference db, i, interfs[i].second, debugIDs[i]); + if (DEBUG_CF_START_VERSION != invalidVersion) { + fmt::print(" [{0} - {1}): {2} {3}\n", + interfs[i].second.begin.printable(), + interfs[i].second.end.printable(), + i, + debugIDs[i].toString().substr(0, 8)); + } } wait(onCFErrors(onErrors) || doCFMerge(results, interfs, streams, begin, end)); @@ -7697,6 +7706,7 @@ ACTOR Future doSingleCFStream(KeyRange range, Version* begin, Version end, UID debugID /*TODO REMOVE this parameter once BG is correctness clean*/) { + state Promise refresh = results->refresh; ASSERT(results->streams.size() == 1); ASSERT(results->storageData.size() == 1); @@ -7776,7 +7786,8 @@ ACTOR Future singleChangeFeedStream(Reference db, Reference results, Key rangeID, Version* begin, - Version end) { + Version end, + bool canReadPopped) { state Database cx(db); state ChangeFeedStreamRequest req; state UID debugID = deterministicRandom()->randomUniqueID(); @@ -7784,6 +7795,7 @@ ACTOR Future singleChangeFeedStream(Reference db, req.begin = *begin; req.end = end; req.range = range; + req.canReadPopped = canReadPopped; req.debugID = debugID; results->streams.clear(); @@ -7803,6 +7815,16 @@ ACTOR Future singleChangeFeedStream(Reference db, results->notAtLatest.set(1); refresh.send(Void()); + if (DEBUG_CF_START_VERSION != invalidVersion) { + fmt::print("Starting single cursor {0} for [{1} - {2}) @ {3} - {4} from {5}\n", + debugID.toString().substr(0, 8), + range.begin.printable(), + range.end.printable(), + *begin, + end, + interf.id().toString().c_str()); + } + wait(results->streams[0].onError() || doSingleCFStream(range, results, rangeID, begin, end, debugID)); return Void(); @@ -7813,7 +7835,8 @@ ACTOR Future getChangeFeedStreamActor(Reference db, Key rangeID, Version begin, Version end, - KeyRange range) { + KeyRange range, + bool canReadPopped) { state Database cx(db); state Span span("NAPI:GetChangeFeedStream"_loc); @@ -7892,10 +7915,11 @@ ACTOR Future getChangeFeedStreamActor(Reference db, interfs.push_back(std::make_pair(locations[i].second->getInterface(chosenLocations[i]), locations[i].first & range)); } - wait(mergeChangeFeedStream(db, interfs, results, rangeID, &begin, end) || cx->connectionFileChanged()); + wait(mergeChangeFeedStream(db, interfs, results, rangeID, &begin, end, canReadPopped) || + cx->connectionFileChanged()); } else { StorageServerInterface interf = locations[0].second->getInterface(chosenLocations[0]); - wait(singleChangeFeedStream(db, interf, range, results, rangeID, &begin, end) || + wait(singleChangeFeedStream(db, interf, range, results, rangeID, &begin, end, canReadPopped) || cx->connectionFileChanged()); } } catch (Error& e) { @@ -7943,8 +7967,10 @@ Future DatabaseContext::getChangeFeedStream(Reference resu Key rangeID, Version begin, Version end, - KeyRange range) { - return getChangeFeedStreamActor(Reference::addRef(this), results, rangeID, begin, end, range); + KeyRange range, + bool canReadPopped) { + return getChangeFeedStreamActor( + Reference::addRef(this), results, rangeID, begin, end, range, canReadPopped); } ACTOR Future> singleLocationOverlappingChangeFeeds( diff --git a/fdbclient/StorageServerInterface.h b/fdbclient/StorageServerInterface.h index ea58e3b3ed..65b622170b 100644 --- a/fdbclient/StorageServerInterface.h +++ b/fdbclient/StorageServerInterface.h @@ -715,6 +715,7 @@ struct ChangeFeedStreamRequest { Version begin = 0; Version end = 0; KeyRange range; + bool canReadPopped = true; // TODO REMOVE once BG is correctness clean!! Useful for debugging UID debugID; ReplyPromiseStream reply; @@ -722,7 +723,7 @@ struct ChangeFeedStreamRequest { ChangeFeedStreamRequest() {} template void serialize(Ar& ar) { - serializer(ar, rangeID, begin, end, range, reply, spanContext, debugID, arena); + serializer(ar, rangeID, begin, end, range, reply, spanContext, canReadPopped, debugID, arena); } }; diff --git a/fdbserver/BlobWorker.actor.cpp b/fdbserver/BlobWorker.actor.cpp index afc4b209b8..cce388428d 100644 --- a/fdbserver/BlobWorker.actor.cpp +++ b/fdbserver/BlobWorker.actor.cpp @@ -1273,15 +1273,24 @@ ACTOR Future blobGranuleUpdateFiles(Reference bwData, if (startState.parentGranule.present() && startVersion < startState.changeFeedStartVersion) { // read from parent change feed up until our new change feed is started + // Required to have canReadPopped = false, otherwise another granule can take over the change feed, and pop + // it. That could cause this worker to think it has the full correct set of data if it then reads the data, + // until it checks the granule lock again. + // passing false for canReadPopped means we will get an exception if we try to read any popped data, killing + // this actor readOldChangeFeed = true; - oldChangeFeedFuture = bwData->db->getChangeFeedStream( - newCFData, oldCFKey.get(), startVersion + 1, startState.changeFeedStartVersion, metadata->keyRange); + oldChangeFeedFuture = bwData->db->getChangeFeedStream(newCFData, + oldCFKey.get(), + startVersion + 1, + startState.changeFeedStartVersion, + metadata->keyRange, + false); } else { readOldChangeFeed = false; - changeFeedFuture = - bwData->db->getChangeFeedStream(newCFData, cfKey, startVersion + 1, MAX_VERSION, metadata->keyRange); + changeFeedFuture = bwData->db->getChangeFeedStream( + newCFData, cfKey, startVersion + 1, MAX_VERSION, metadata->keyRange, false); } // Start actors BEFORE setting new change feed data to ensure the change feed data is properly initialized by @@ -1406,7 +1415,7 @@ ACTOR Future blobGranuleUpdateFiles(Reference bwData, Reference newCFData = makeReference(); changeFeedFuture = bwData->db->getChangeFeedStream( - newCFData, cfKey, startState.changeFeedStartVersion, MAX_VERSION, metadata->keyRange); + newCFData, cfKey, startState.changeFeedStartVersion, MAX_VERSION, metadata->keyRange, false); // Start actors BEFORE setting new change feed data to ensure the change feed data is properly // initialized by the client @@ -1525,7 +1534,8 @@ ACTOR Future blobGranuleUpdateFiles(Reference bwData, oldCFKey.get(), cfRollbackVersion + 1, startState.changeFeedStartVersion, - metadata->keyRange); + metadata->keyRange, + false); } else { if (cfRollbackVersion < startState.changeFeedStartVersion) { @@ -1535,8 +1545,12 @@ ACTOR Future blobGranuleUpdateFiles(Reference bwData, } ASSERT(cfRollbackVersion >= startState.changeFeedStartVersion); - changeFeedFuture = bwData->db->getChangeFeedStream( - newCFData, cfKey, cfRollbackVersion + 1, MAX_VERSION, metadata->keyRange); + changeFeedFuture = bwData->db->getChangeFeedStream(newCFData, + cfKey, + cfRollbackVersion + 1, + MAX_VERSION, + metadata->keyRange, + false); } // Start actors BEFORE setting new change feed data to ensure the change feed data @@ -2049,7 +2063,7 @@ ACTOR Future waitForVersion(Reference metadata, Version v ACTOR Future handleBlobGranuleFileRequest(Reference bwData, BlobGranuleFileRequest req) { if (BW_REQUEST_DEBUG || DEBUG_BW_WAIT_VERSION == req.readVersion) { - fmt::print("BW {0} processing blobGranuleFileRequest for range [{1} -{2}) @ {3}\n", + fmt::print("BW {0} processing blobGranuleFileRequest for range [{1} - {2}) @ {3}\n", bwData->id.toString(), req.keyRange.begin.printable(), req.keyRange.end.printable(), diff --git a/fdbserver/storageserver.actor.cpp b/fdbserver/storageserver.actor.cpp index fc41c113e6..433bdc0ece 100644 --- a/fdbserver/storageserver.actor.cpp +++ b/fdbserver/storageserver.actor.cpp @@ -85,6 +85,7 @@ bool canReplyWith(Error e) { case error_code_watch_cancelled: case error_code_unknown_change_feed: case error_code_server_overloaded: + case error_code_change_feed_popped: // getRangeAndMap related exceptions that are not retriable: case error_code_mapper_bad_index: case error_code_mapper_no_such_key: @@ -1792,6 +1793,8 @@ ACTOR Future> getChangeFeedMutations(Stor throw unknown_change_feed(); } + state Reference feedInfo = feed->second; + if (DEBUG_SS_CFM(data->thisServerID, req.rangeID, req.begin)) { printf("CFM: SS %s CF %s: got version %lld >= %lld\n", data->thisServerID.toString().substr(0, 4).c_str(), @@ -1800,6 +1803,10 @@ ACTOR Future> getChangeFeedMutations(Stor req.begin); } + if (!req.canReadPopped && req.begin <= feedInfo->emptyVersion) { + throw change_feed_popped(); + } + // We must copy the mutationDeque when fetching the durable bytes in case mutations are popped from memory while // waiting for the results state Version dequeVersion = data->version.get(); @@ -1811,14 +1818,14 @@ ACTOR Future> getChangeFeedMutations(Stor data->thisServerID.toString().substr(0, 4).c_str(), req.rangeID.printable().substr(0, 6).c_str(), dequeVersion, - feed->second->emptyVersion, - feed->second->storageVersion, - feed->second->durableVersion, - feed->second->fetchVersion); + feedInfo->emptyVersion, + feedInfo->storageVersion, + feedInfo->durableVersion, + feedInfo->fetchVersion); } - if (req.end > feed->second->emptyVersion + 1) { - for (auto& it : feed->second->mutations) { + if (req.end > feedInfo->emptyVersion + 1) { + for (auto& it : feedInfo->mutations) { if (it.version >= req.end || it.version > dequeVersion || remainingLimitBytes <= 0) { break; } @@ -1839,8 +1846,8 @@ ACTOR Future> getChangeFeedMutations(Stor } } - if (req.end > feed->second->emptyVersion + 1 && feed->second->durableVersion != invalidVersion && - req.begin <= feed->second->durableVersion) { + if (req.end > feedInfo->emptyVersion + 1 && feedInfo->durableVersion != invalidVersion && + req.begin <= feedInfo->durableVersion) { RangeResult res = wait(data->storage.readRange( KeyRangeRef(changeFeedDurableKey(req.rangeID, std::max(req.begin, feed->second->emptyVersion)), changeFeedDurableKey(req.rangeID, req.end)), @@ -1890,6 +1897,11 @@ ACTOR Future> getChangeFeedMutations(Stor reply = memoryReply; } + // check if pop happened concurrently with read + if (!req.canReadPopped && req.begin <= feedInfo->emptyVersion) { + throw change_feed_popped(); + } + bool gotAll = remainingLimitBytes > 0 && remainingDurableBytes > 0 && data->version.get() == startVersion; Version finalVersion = std::min(req.end - 1, dequeVersion); if ((reply.mutations.empty() || reply.mutations.back().version < finalVersion) && remainingLimitBytes > 0 && @@ -2007,13 +2019,14 @@ ACTOR Future changeFeedStreamQ(StorageServer* data, ChangeFeedStreamReques wait(delay(0, TaskPriority::DefaultEndpoint)); if (DEBUG_SS_CFM(data->thisServerID, req.rangeID, req.begin)) { - printf("CFM: SS %s CF %s: got CFSQ [%s - %s) %lld - %lld\n", + printf("CFM: SS %s CF %s: got CFSQ [%s - %s) %lld - %lld, crp=%s\n", data->thisServerID.toString().substr(0, 4).c_str(), req.rangeID.printable().substr(0, 6).c_str(), req.range.begin.printable().c_str(), req.range.end.printable().c_str(), req.begin, - req.end); + req.end, + req.canReadPopped ? "T" : "F"); } try { @@ -4000,7 +4013,7 @@ ACTOR Future fetchChangeFeedApplier(StorageServer* data, bool existing) { state Reference feedResults = makeReference(); state Future feed = data->cx->getChangeFeedStream( - feedResults, rangeId, 0, existing ? fetchVersion + 1 : data->version.get() + 1, range); + feedResults, rangeId, 0, existing ? fetchVersion + 1 : data->version.get() + 1, range, true); // TODO remove debugging eventually? state Version firstVersion = invalidVersion; diff --git a/flow/error_definitions.h b/flow/error_definitions.h index 41d69a0573..b0d1b6ec37 100755 --- a/flow/error_definitions.h +++ b/flow/error_definitions.h @@ -86,6 +86,7 @@ ERROR( change_feed_cancelled, 1062, "Change feed was cancelled" ) ERROR( blob_granule_file_load_error, 1063, "Error loading a blob file during granule materialization" ) ERROR( blob_granule_transaction_too_old, 1064, "Read version is older than blob granule history supports" ) ERROR( blob_manager_replaced, 1065, "This blob manager has been replaced." ) +ERROR( change_feed_popped, 1066, "Tried to read a version older than what has been popped from the change feed" ) ERROR( broken_promise, 1100, "Broken promise" ) ERROR( operation_cancelled, 1101, "Asynchronous operation cancelled" ) From 2d3b216f60e0a3143ce4b2ce50ede397edb0411d Mon Sep 17 00:00:00 2001 From: Josh Slocum Date: Fri, 28 Jan 2022 12:27:04 -0600 Subject: [PATCH 180/413] Persisting change feed popped version in all cases to enable popped checking --- fdbclient/StorageServerInterface.h | 9 +- fdbserver/storageserver.actor.cpp | 156 +++++++++++++++++++---------- 2 files changed, 106 insertions(+), 59 deletions(-) diff --git a/fdbclient/StorageServerInterface.h b/fdbclient/StorageServerInterface.h index 65b622170b..556c8548e0 100644 --- a/fdbclient/StorageServerInterface.h +++ b/fdbclient/StorageServerInterface.h @@ -747,19 +747,20 @@ struct ChangeFeedPopRequest { struct OverlappingChangeFeedEntry { Key rangeId; KeyRange range; + Version emptyVersion; bool stopped = false; bool operator==(const OverlappingChangeFeedEntry& r) const { - return rangeId == r.rangeId && range == r.range && stopped == r.stopped; + return rangeId == r.rangeId && range == r.range && stopped == r.stopped && emptyVersion == r.emptyVersion; } OverlappingChangeFeedEntry() {} - OverlappingChangeFeedEntry(Key const& rangeId, KeyRange const& range, bool stopped) - : rangeId(rangeId), range(range), stopped(stopped) {} + OverlappingChangeFeedEntry(Key const& rangeId, KeyRange const& range, Version emptyVersion, bool stopped) + : rangeId(rangeId), range(range), emptyVersion(emptyVersion), stopped(stopped) {} template void serialize(Ar& ar) { - serializer(ar, rangeId, range, stopped); + serializer(ar, rangeId, range, emptyVersion, stopped); } }; diff --git a/fdbserver/storageserver.actor.cpp b/fdbserver/storageserver.actor.cpp index 433bdc0ece..a2b47e0913 100644 --- a/fdbserver/storageserver.actor.cpp +++ b/fdbserver/storageserver.actor.cpp @@ -1597,43 +1597,6 @@ ACTOR Future watchValueSendReply(StorageServer* data, } } -ACTOR Future changeFeedPopQ(StorageServer* self, ChangeFeedPopRequest req) { - wait(delay(0)); - - TraceEvent(SevDebug, "ChangeFeedPopQuery", self->thisServerID) - .detail("RangeID", req.rangeID.printable()) - .detail("Version", req.version) - .detail("Range", req.range.toString()); - - if (!self->isReadable(req.range)) { - req.reply.sendError(wrong_shard_server()); - return Void(); - } - auto feed = self->uidChangeFeed.find(req.rangeID); - if (feed == self->uidChangeFeed.end()) { - req.reply.sendError(unknown_change_feed()); - return Void(); - } - if (req.version - 1 > feed->second->emptyVersion) { - feed->second->emptyVersion = req.version - 1; - while (!feed->second->mutations.empty() && feed->second->mutations.front().version < req.version) { - feed->second->mutations.pop_front(); - } - if (feed->second->storageVersion != invalidVersion) { - self->storage.clearRange(KeyRangeRef(changeFeedDurableKey(feed->second->id, 0), - changeFeedDurableKey(feed->second->id, req.version))); - if (req.version > feed->second->storageVersion) { - feed->second->storageVersion = invalidVersion; - feed->second->durableVersion = invalidVersion; - feed->second->fetchVersion = invalidVersion; - } - wait(self->durableVersion.whenAtLeast(self->storageVersion() + 1)); - } - } - req.reply.send(Void()); - return Void(); -} - ACTOR Future overlappingChangeFeedsQ(StorageServer* data, OverlappingChangeFeedsRequest req) { wait(delay(0)); wait(data->version.whenAtLeast(req.minVersion)); @@ -1644,15 +1607,16 @@ ACTOR Future overlappingChangeFeedsQ(StorageServer* data, OverlappingChang } auto ranges = data->keyChangeFeed.intersectingRanges(req.range); - std::map> rangeIds; + std::map> rangeIds; for (auto r : ranges) { for (auto& it : r.value()) { - rangeIds[it->id] = std::make_pair(it->range, it->stopped); + rangeIds[it->id] = std::tuple(it->range, it->emptyVersion, it->stopped); } } OverlappingChangeFeedsReply reply; for (auto& it : rangeIds) { - reply.rangeIds.push_back(OverlappingChangeFeedEntry(it.first, it.second.first, it.second.second)); + reply.rangeIds.push_back(OverlappingChangeFeedEntry( + it.first, std::get<0>(it.second), std::get<1>(it.second), std::get<2>(it.second))); } req.reply.send(reply); return Void(); @@ -4005,15 +3969,64 @@ static const KeyRangeRef persistChangeFeedKeys = KeyRangeRef(LiteralStringRef(PERSIST_PREFIX "RF/"), LiteralStringRef(PERSIST_PREFIX "RF0")); // data keys are unmangled (but never start with PERSIST_PREFIX because they are always in allKeys) +ACTOR Future changeFeedPopQ(StorageServer* self, ChangeFeedPopRequest req) { + wait(delay(0)); + + TraceEvent(SevDebug, "ChangeFeedPopQuery", self->thisServerID) + .detail("RangeID", req.rangeID.printable()) + .detail("Version", req.version) + .detail("Range", req.range.toString()); + + if (!self->isReadable(req.range)) { + req.reply.sendError(wrong_shard_server()); + return Void(); + } + auto feed = self->uidChangeFeed.find(req.rangeID); + if (feed == self->uidChangeFeed.end()) { + req.reply.sendError(unknown_change_feed()); + return Void(); + } + if (req.version - 1 > feed->second->emptyVersion) { + feed->second->emptyVersion = req.version - 1; + while (!feed->second->mutations.empty() && feed->second->mutations.front().version < req.version) { + feed->second->mutations.pop_front(); + } + Version durableVersion = self->data().getLatestVersion(); + auto& mLV = self->addVersionToMutationLog(durableVersion); + self->addMutationToMutationLog( + mLV, + MutationRef(MutationRef::SetValue, + persistChangeFeedKeys.begin.toString() + feed->second->id.toString(), + changeFeedValue(feed->second->range, + feed->second->emptyVersion + 1, + feed->second->stopped ? ChangeFeedStatus::CHANGE_FEED_STOP + : ChangeFeedStatus::CHANGE_FEED_CREATE))); + if (feed->second->storageVersion != invalidVersion) { + self->addMutationToMutationLog(mLV, + MutationRef(MutationRef::ClearRange, + changeFeedDurableKey(feed->second->id, 0), + changeFeedDurableKey(feed->second->id, req.version))); + if (req.version > feed->second->storageVersion) { + feed->second->storageVersion = invalidVersion; + feed->second->durableVersion = invalidVersion; + } + } + wait(self->durableVersion.whenAtLeast(durableVersion)); + } + req.reply.send(Void()); + return Void(); +} + ACTOR Future fetchChangeFeedApplier(StorageServer* data, Reference changeFeedInfo, Key rangeId, KeyRange range, + Version emptyVersion, Version fetchVersion, bool existing) { state Reference feedResults = makeReference(); state Future feed = data->cx->getChangeFeedStream( - feedResults, rangeId, 0, existing ? fetchVersion + 1 : data->version.get() + 1, range, true); + feedResults, rangeId, emptyVersion + 1, existing ? fetchVersion + 1 : data->version.get() + 1, range, true); // TODO remove debugging eventually? state Version firstVersion = invalidVersion; @@ -4069,7 +4082,8 @@ ACTOR Future fetchChangeFeedApplier(StorageServer* data, // Add 2 to fetch version to make sure the local stream will have more versions in the stream than the remote stream // to avoid edge cases in the merge logic - state Future localStream = localChangeFeedStream(data, localResults, rangeId, 0, fetchVersion + 2, range); + state Future localStream = + localChangeFeedStream(data, localResults, rangeId, emptyVersion + 1, fetchVersion + 2, range); state Standalone localResult; Standalone _localResult = waitNext(localResults.getFuture()); @@ -4183,8 +4197,13 @@ ACTOR Future fetchChangeFeed(StorageServer* data, loop { try { // TODO clean up existing param for !existing - wait(fetchChangeFeedApplier( - data, changeFeedInfo, changeFeedInfo->id, changeFeedInfo->range, fetchVersion, false)); + wait(fetchChangeFeedApplier(data, + changeFeedInfo, + changeFeedInfo->id, + changeFeedInfo->range, + changeFeedInfo->emptyVersion, + fetchVersion, + false)); data->fetchingChangeFeeds.insert(changeFeedInfo->id); return Void(); } catch (Error& e) { @@ -4197,43 +4216,70 @@ ACTOR Future fetchChangeFeed(StorageServer* data, } ACTOR Future> fetchChangeFeedMetadata(StorageServer* data, KeyRange keys, Version fetchVersion) { + TraceEvent(SevDebug, "FetchChangeFeedMetadata", data->thisServerID) + .detail("Range", keys.toString()) + .detail("FetchVersion", fetchVersion); std::vector feeds = wait(data->cx->getOverlappingChangeFeeds(keys, fetchVersion + 1)); std::vector feedIds; feedIds.reserve(feeds.size()); // create change feed metadata if it does not exist for (auto& cfEntry : feeds) { - bool cleanupPending = data->changeFeedCleanupDurable.count(cfEntry.rangeId); + auto cleanupEntry = data->changeFeedCleanupDurable.find(cfEntry.rangeId); + bool cleanupPending = cleanupEntry != data->changeFeedCleanupDurable.end(); feedIds.push_back(cfEntry.rangeId); - bool existing = data->uidChangeFeed.count(cfEntry.rangeId); + auto existingEntry = data->uidChangeFeed.find(cfEntry.rangeId); + bool existing = existingEntry != data->uidChangeFeed.end(); + + TraceEvent(SevDebug, "FetchedChangeFeedInfo", data->thisServerID) + .detail("RangeID", cfEntry.rangeId.printable()) + .detail("Range", cfEntry.range.toString()) + .detail("FetchVersion", fetchVersion) + .detail("EmptyVersion", cfEntry.emptyVersion) + .detail("Stopped", cfEntry.stopped) + .detail("Existing", existing) + .detail("CleanupPendingVersion", cleanupPending ? cleanupEntry->second : invalidVersion); + if (!existing) { TEST(cleanupPending); // Fetch change feed which is cleanup pending. This means there was a move away and a // move back, this will remake the metadata - TraceEvent(SevDebug, "FetchChangeFeedMetadata", data->thisServerID) - .detail("RangeID", cfEntry.rangeId.printable()) - .detail("Range", cfEntry.range.toString()) - .detail("FetchVersion", fetchVersion) - .detail("Stopped", cfEntry.stopped) - .detail("CleanupPending", cleanupPending); Reference changeFeedInfo = Reference(new ChangeFeedInfo()); changeFeedInfo->range = cfEntry.range; changeFeedInfo->id = cfEntry.rangeId; changeFeedInfo->stopped = cfEntry.stopped; + changeFeedInfo->emptyVersion = cfEntry.emptyVersion; data->uidChangeFeed[cfEntry.rangeId] = changeFeedInfo; auto rs = data->keyChangeFeed.modify(cfEntry.range); for (auto r = rs.begin(); r != rs.end(); ++r) { r->value().push_back(changeFeedInfo); } data->keyChangeFeed.coalesce(cfEntry.range.contents()); + auto& mLV = data->addVersionToMutationLog(data->data().getLatestVersion()); data->addMutationToMutationLog( mLV, MutationRef(MutationRef::SetValue, persistChangeFeedKeys.begin.toString() + cfEntry.rangeId.toString(), - changeFeedValue(cfEntry.range, invalidVersion, ChangeFeedStatus::CHANGE_FEED_CREATE))); + changeFeedValue(cfEntry.range, + changeFeedInfo->emptyVersion + 1, + cfEntry.stopped ? ChangeFeedStatus::CHANGE_FEED_STOP + : ChangeFeedStatus::CHANGE_FEED_CREATE))); } else { - ASSERT(!cleanupPending); + auto changeFeedInfo = existingEntry->second; + if (changeFeedInfo->emptyVersion < cfEntry.emptyVersion) { + TEST(true); // Got updated CF emptyVersion from a parallel fetchChangeFeedMetadata + changeFeedInfo->emptyVersion = cfEntry.emptyVersion; + auto& mLV = data->addVersionToMutationLog(data->data().getLatestVersion()); + data->addMutationToMutationLog( + mLV, + MutationRef(MutationRef::SetValue, + persistChangeFeedKeys.begin.toString() + cfEntry.rangeId.toString(), + changeFeedValue(cfEntry.range, + changeFeedInfo->emptyVersion + 1, + cfEntry.stopped ? ChangeFeedStatus::CHANGE_FEED_STOP + : ChangeFeedStatus::CHANGE_FEED_CREATE))); + } } } return feedIds; @@ -5211,9 +5257,9 @@ private: // don't set fetchVersion to invalidVersion here because there could be an active fetch } } + addMutationToLog = true; } feed->second->stopped = (status == ChangeFeedStatus::CHANGE_FEED_STOP); - addMutationToLog = true; } else if (status == ChangeFeedStatus::CHANGE_FEED_CREATE) { TraceEvent(SevDebug, "CreatingChangeFeed", data->thisServerID) .detail("RangeID", changeFeedId.printable()) From cf7ed4ee40c4b188a8debca48484af912406a364 Mon Sep 17 00:00:00 2001 From: Josh Slocum Date: Fri, 28 Jan 2022 14:21:26 -0600 Subject: [PATCH 181/413] Simplify change feed fetching to not inject anything at all during fetch and rely on reading from the remote feeds --- fdbserver/storageserver.actor.cpp | 41 +++++-------------------------- 1 file changed, 6 insertions(+), 35 deletions(-) diff --git a/fdbserver/storageserver.actor.cpp b/fdbserver/storageserver.actor.cpp index a2b47e0913..b14c1c667d 100644 --- a/fdbserver/storageserver.actor.cpp +++ b/fdbserver/storageserver.actor.cpp @@ -4572,7 +4572,6 @@ ACTOR Future fetchKeys(StorageServer* data, AddingShard* shard) { // we have written) state Future fetchDurable = data->durableVersion.whenAtLeast(data->storageVersion() + 1); - wait(dispatchChangeFeeds(data, fetchKeysID, keys, fetchVersion, changeFeedsToFetch)); holdingFKPL.release(); wait(fetchDurable); @@ -4607,6 +4606,9 @@ ACTOR Future fetchKeys(StorageServer* data, AddingShard* shard) { ASSERT(shard->transferredVersion > data->storageVersion()); ASSERT(shard->transferredVersion == data->data().getLatestVersion()); + state Future cfFetch = + dispatchChangeFeeds(data, fetchKeysID, keys, shard->transferredVersion, changeFeedsToFetch); + TraceEvent(SevDebug, "FetchKeysHaveData", data->thisServerID) .detail("FKID", interval.pairID) .detail("Version", shard->transferredVersion) @@ -4643,6 +4645,9 @@ ACTOR Future fetchKeys(StorageServer* data, AddingShard* shard) { shard->updates.clear(); + // wait on change feed fetch to complete before marking data as available + wait(cfFetch); + setAvailableStatus(data, keys, true); // keys will be available when getLatestVersion()==transferredVersion is durable @@ -4726,40 +4731,6 @@ void AddingShard::addMutation(Version version, bool fromFetch, MutationRef const } // Add the mutation to the version. updates.back().mutations.push_back_deep(updates.back().arena(), mutation); - if (!fromFetch) { - if (mutation.type == MutationRef::SetValue) { - for (auto& it : server->keyChangeFeed[mutation.param1]) { - if (!it->stopped) { - if (it->mutations.empty() || it->mutations.back().version != version) { - it->mutations.push_back(MutationsAndVersionRef(version, server->knownCommittedVersion)); - } - it->mutations.back().mutations.push_back_deep(it->mutations.back().arena(), mutation); - server->currentChangeFeeds.insert(it->id); - DEBUG_MUTATION("ChangeFeedWriteSet", version, mutation, server->thisServerID) - .detail("Range", it->range) - .detail("ChangeFeedID", it->id) - .detail("Source", "Adding"); - } - } - } else if (mutation.type == MutationRef::ClearRange) { - auto ranges = server->keyChangeFeed.intersectingRanges(KeyRangeRef(mutation.param1, mutation.param2)); - for (auto& r : ranges) { - for (auto& it : r.value()) { - if (!it->stopped) { - if (it->mutations.empty() || it->mutations.back().version != version) { - it->mutations.push_back(MutationsAndVersionRef(version, server->knownCommittedVersion)); - } - it->mutations.back().mutations.push_back_deep(it->mutations.back().arena(), mutation); - server->currentChangeFeeds.insert(it->id); - DEBUG_MUTATION("ChangeFeedWriteClear", version, mutation, server->thisServerID) - .detail("Range", it->range) - .detail("ChangeFeedID", it->id) - .detail("Source", "Adding"); - } - } - } - } - } } else if (phase == Waiting) { server->addMutation(version, fromFetch, mutation, keys, server->updateEagerReads); } else From bc9572028fc4bfbc00304fe1ed100f04602806ec Mon Sep 17 00:00:00 2001 From: Josh Slocum Date: Fri, 28 Jan 2022 14:40:43 -0600 Subject: [PATCH 182/413] Added AtomicOps tests for blob granules --- tests/CMakeLists.txt | 2 + tests/fast/BlobGranuleVerifyAtomicOps.toml | 37 +++++++++++++++++++ .../fast/BlobGranuleVerifyAtomicOpsClean.toml | 14 +++++++ 3 files changed, 53 insertions(+) create mode 100644 tests/fast/BlobGranuleVerifyAtomicOps.toml create mode 100644 tests/fast/BlobGranuleVerifyAtomicOpsClean.toml diff --git a/tests/CMakeLists.txt b/tests/CMakeLists.txt index 3f7909635e..f41414b7f4 100644 --- a/tests/CMakeLists.txt +++ b/tests/CMakeLists.txt @@ -129,6 +129,8 @@ if(WITH_PYTHON) add_fdb_test(TEST_FILES fast/BackupToDBCorrectnessClean.toml) add_fdb_test(TEST_FILES fast/BlobGranuleVerifySmall.toml) add_fdb_test(TEST_FILES fast/BlobGranuleVerifySmallClean.toml) + add_fdb_test(TEST_FILES fast/BlobGranuleVerifyAtomicOps.toml) + add_fdb_test(TEST_FILES fast/BlobGranuleVerifyAtomicOpsClean.toml) add_fdb_test(TEST_FILES fast/BlobGranuleVerifyCycle.toml) add_fdb_test(TEST_FILES fast/BlobGranuleVerifyCycleClean.toml) add_fdb_test(TEST_FILES fast/CacheTest.toml) diff --git a/tests/fast/BlobGranuleVerifyAtomicOps.toml b/tests/fast/BlobGranuleVerifyAtomicOps.toml new file mode 100644 index 0000000000..4831d8b985 --- /dev/null +++ b/tests/fast/BlobGranuleVerifyAtomicOps.toml @@ -0,0 +1,37 @@ +[configuration] +blobGranulesEnabled = true + +[[test]] +testTitle = 'BlobGranuleVerifyAtomicOps' + + [[test.workload]] + testName = 'AtomicOps' + transactionsPerSecond = 2500.0 + testDuration = 30.0 + + [[test.workload]] + testName = 'BlobGranuleVerifier' + testDuration = 30.0 + + [[test.workload]] + testName = 'RandomClogging' + testDuration = 30.0 + + [[test.workload]] + testName = 'Rollback' + meanDelay = 30.0 + testDuration = 30.0 + + [[test.workload]] + testName = 'Attrition' + machinesToKill = 10 + machinesToLeave = 3 + reboot = true + testDuration = 30.0 + + [[test.workload]] + testName = 'Attrition' + machinesToKill = 10 + machinesToLeave = 3 + reboot = true + testDuration = 30.0 diff --git a/tests/fast/BlobGranuleVerifyAtomicOpsClean.toml b/tests/fast/BlobGranuleVerifyAtomicOpsClean.toml new file mode 100644 index 0000000000..89ecf6d0f8 --- /dev/null +++ b/tests/fast/BlobGranuleVerifyAtomicOpsClean.toml @@ -0,0 +1,14 @@ +[configuration] +blobGranulesEnabled = true + +[[test]] +testTitle = 'BlobGranuleVerifyAtomicOpsClean' + + [[test.workload]] + testName = 'AtomicOps' + transactionsPerSecond = 2500.0 + testDuration = 30.0 + + [[test.workload]] + testName = 'BlobGranuleVerifier' + testDuration = 30.0 \ No newline at end of file From c3cece552c1da9d7825b7d54cee4c4dc8f857ee4 Mon Sep 17 00:00:00 2001 From: Josh Slocum Date: Fri, 28 Jan 2022 14:45:56 -0600 Subject: [PATCH 183/413] Fixes for previous --- fdbserver/storageserver.actor.cpp | 23 +++++++++++++++-------- 1 file changed, 15 insertions(+), 8 deletions(-) diff --git a/fdbserver/storageserver.actor.cpp b/fdbserver/storageserver.actor.cpp index b14c1c667d..d1668217e3 100644 --- a/fdbserver/storageserver.actor.cpp +++ b/fdbserver/storageserver.actor.cpp @@ -4025,8 +4025,14 @@ ACTOR Future fetchChangeFeedApplier(StorageServer* data, Version fetchVersion, bool existing) { state Reference feedResults = makeReference(); - state Future feed = data->cx->getChangeFeedStream( - feedResults, rangeId, emptyVersion + 1, existing ? fetchVersion + 1 : data->version.get() + 1, range, true); + printf("DBG: fetching %lld %lld %lld\n", emptyVersion, emptyVersion + 1, fetchVersion); + Version startVersion = emptyVersion + 1; + if (startVersion < 0) { + startVersion = 0; + } + // TODO somwhere this is initialized to -2 instead of -1 but it's fine + state Future feed = + data->cx->getChangeFeedStream(feedResults, rangeId, startVersion, fetchVersion, range, true); // TODO remove debugging eventually? state Version firstVersion = invalidVersion; @@ -4175,13 +4181,13 @@ ACTOR Future fetchChangeFeedApplier(StorageServer* data, ACTOR Future fetchChangeFeed(StorageServer* data, Reference changeFeedInfo, - Version fetchVersion) { + Version transferredVersion) { wait(delay(0)); // allow this actor to be cancelled by removals TraceEvent(SevDebug, "FetchChangeFeed", data->thisServerID) .detail("RangeID", changeFeedInfo->id.printable()) .detail("Range", changeFeedInfo->range.toString()) - .detail("FetchVersion", fetchVersion); + .detail("FetchVersion", transferredVersion); auto cleanupPending = data->changeFeedCleanupDurable.find(changeFeedInfo->id); if (cleanupPending != data->changeFeedCleanupDurable.end()) { @@ -4189,7 +4195,8 @@ ACTOR Future fetchChangeFeed(StorageServer* data, .detail("RangeID", changeFeedInfo->id.printable()) .detail("Range", changeFeedInfo->range.toString()) .detail("CleanupVersion", cleanupPending->second) - .detail("FetchVersion", fetchVersion); + .detail("EmptyVersion", changeFeedInfo->emptyVersion) + .detail("FetchVersion", transferredVersion); wait(data->durableVersion.whenAtLeast(cleanupPending->second + 1)); ASSERT(!data->changeFeedCleanupDurable.count(changeFeedInfo->id)); } @@ -4202,7 +4209,7 @@ ACTOR Future fetchChangeFeed(StorageServer* data, changeFeedInfo->id, changeFeedInfo->range, changeFeedInfo->emptyVersion, - fetchVersion, + transferredVersion, false)); data->fetchingChangeFeeds.insert(changeFeedInfo->id); return Void(); @@ -4288,7 +4295,7 @@ ACTOR Future> fetchChangeFeedMetadata(StorageServer* data, KeyR ACTOR Future dispatchChangeFeeds(StorageServer* data, UID fetchKeysID, KeyRange keys, - Version fetchVersion, + Version transferredVersion, std::vector feedIds) { // find overlapping range feeds @@ -4303,7 +4310,7 @@ ACTOR Future dispatchChangeFeeds(StorageServer* data, // TODO REMOVE this assert once we enable change feed deletion ASSERT(feedIt != data->uidChangeFeed.end()); Reference feed = feedIt->second; - feedFetches[feed->id] = fetchChangeFeed(data, feed, fetchVersion); + feedFetches[feed->id] = fetchChangeFeed(data, feed, transferredVersion); } loop { From 05f27eac50d45e5783ace55fd8eac711a75f4123 Mon Sep 17 00:00:00 2001 From: Josh Slocum Date: Mon, 31 Jan 2022 08:27:37 -0600 Subject: [PATCH 184/413] range mover fix --- fdbserver/BlobManager.actor.cpp | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/fdbserver/BlobManager.actor.cpp b/fdbserver/BlobManager.actor.cpp index 1117a25a96..65b87fe294 100644 --- a/fdbserver/BlobManager.actor.cpp +++ b/fdbserver/BlobManager.actor.cpp @@ -1641,6 +1641,9 @@ ACTOR Future recoverBlobManager(Reference bmData) { } ACTOR Future chaosRangeMover(Reference bmData) { + // Only move each granule once during the test, otherwise it can cause availability issues + // KeyRange isn't hashable and this is only for simulation, so just use toString of range + state std::unordered_set alreadyMoved; ASSERT(g_network->isSimulated()); loop { wait(delay(30.0)); @@ -1657,13 +1660,14 @@ ACTOR Future chaosRangeMover(Reference bmData) { while (tries > 0) { tries--; auto randomRange = bmData->workerAssignments.randomRange(); - if (randomRange.value() != UID()) { + if (randomRange.value() != UID() && !alreadyMoved.count(randomRange.range().toString())) { if (BM_DEBUG) { fmt::print("Range mover moving range [{0} - {1}): {2}\n", randomRange.begin().printable().c_str(), randomRange.end().printable().c_str(), randomRange.value().toString().c_str()); } + alreadyMoved.insert(randomRange.range().toString()); // FIXME: with low probability, could immediately revoke it from the new assignment and move // it back right after to test that race From d6dc46879353c0e8bf4c6d68232bcdee39675faf Mon Sep 17 00:00:00 2001 From: Josh Slocum Date: Mon, 31 Jan 2022 09:21:20 -0600 Subject: [PATCH 185/413] Fixing potential bug of mixed-version FDB result in granule verifier --- fdbserver/BlobWorker.actor.cpp | 4 ++++ .../workloads/BlobGranuleVerifier.actor.cpp | 21 ++++++++++++++++--- 2 files changed, 22 insertions(+), 3 deletions(-) diff --git a/fdbserver/BlobWorker.actor.cpp b/fdbserver/BlobWorker.actor.cpp index cce388428d..fcb73307b9 100644 --- a/fdbserver/BlobWorker.actor.cpp +++ b/fdbserver/BlobWorker.actor.cpp @@ -42,6 +42,7 @@ #include "flow/Arena.h" #include "flow/Error.h" #include "flow/IRandom.h" +#include "flow/Trace.h" #include "flow/actorcompiler.h" // has to be last include #define BW_DEBUG true @@ -720,6 +721,9 @@ ACTOR Future dumpInitialSnapshotFromFDB(Reference metadata->keyRange.end.printable(), e.name()); } + TraceEvent(SevWarn, "BlobGranuleInitialSnapshotError", bwData->id) + .detail("Granule", metadata->keyRange) + .error(e); wait(tr->onError(e)); } } diff --git a/fdbserver/workloads/BlobGranuleVerifier.actor.cpp b/fdbserver/workloads/BlobGranuleVerifier.actor.cpp index 5c33d0fd87..6767da00ef 100644 --- a/fdbserver/workloads/BlobGranuleVerifier.actor.cpp +++ b/fdbserver/workloads/BlobGranuleVerifier.actor.cpp @@ -166,20 +166,35 @@ struct BlobGranuleVerifierWorkload : TestWorkload { // assumes we can read the whole range in one transaction at a single version ACTOR Future> readFromFDB(Database cx, KeyRange range) { + state bool first = true; state Version v; state RangeResult out; state Transaction tr(cx); state KeyRange currentRange = range; loop { try { - RangeResult r = wait(tr.getRange(currentRange, CLIENT_KNOBS->TOO_MANY)); + state RangeResult r = wait(tr.getRange(currentRange, CLIENT_KNOBS->TOO_MANY)); + Version grv = wait(tr.getReadVersion()); + // need consistent version snapshot of range + if (first) { + v = grv; + first = false; + } else if (v != grv) { + // reset the range and restart the read at a higher version + TraceEvent(SevDebug, "BGVFDBReadReset").detail("ReadVersion", v); + TEST(true); // BGV transaction reset + printf("Resetting BGV GRV {0} -> {1}\n", v, grv); + first = true; + out = RangeResult(); + currentRange = range; + tr.reset(); + continue; + } out.arena().dependsOn(r.arena()); out.append(out.arena(), r.begin(), r.size()); if (r.more) { currentRange = KeyRangeRef(keyAfter(r.back().key), currentRange.end); } else { - Version _v = wait(tr.getReadVersion()); - v = _v; break; } } catch (Error& e) { From f2b9eb1d4bfd69757cee33b701433f3f0ffb1097 Mon Sep 17 00:00:00 2001 From: Josh Slocum Date: Mon, 31 Jan 2022 09:21:20 -0600 Subject: [PATCH 186/413] Fixing potential bug of mixed-version FDB result in granule verifier --- fdbserver/BlobWorker.actor.cpp | 8 +++++++ .../workloads/BlobGranuleVerifier.actor.cpp | 21 ++++++++++++++++--- 2 files changed, 26 insertions(+), 3 deletions(-) diff --git a/fdbserver/BlobWorker.actor.cpp b/fdbserver/BlobWorker.actor.cpp index cce388428d..4cdd94258b 100644 --- a/fdbserver/BlobWorker.actor.cpp +++ b/fdbserver/BlobWorker.actor.cpp @@ -42,6 +42,7 @@ #include "flow/Arena.h" #include "flow/Error.h" #include "flow/IRandom.h" +#include "flow/Trace.h" #include "flow/actorcompiler.h" // has to be last include #define BW_DEBUG true @@ -714,13 +715,20 @@ ACTOR Future dumpInitialSnapshotFromFDB(Reference DEBUG_KEY_RANGE("BlobWorkerFDBSnapshot", readVersion, metadata->keyRange, bwData->id); return f; } catch (Error& e) { + if (e.code() == error_code_operation_cancelled) { + throw e; + } if (BW_DEBUG) { fmt::print("Dumping snapshot from FDB for [{0} - {1}) got error {2}\n", metadata->keyRange.begin.printable(), metadata->keyRange.end.printable(), e.name()); } + state Error err = e; wait(tr->onError(e)); + TraceEvent(SevWarn, "BlobGranuleInitialSnapshotRetry", bwData->id) + .detail("Granule", metadata->keyRange) + .error(err); } } } diff --git a/fdbserver/workloads/BlobGranuleVerifier.actor.cpp b/fdbserver/workloads/BlobGranuleVerifier.actor.cpp index 5c33d0fd87..6767da00ef 100644 --- a/fdbserver/workloads/BlobGranuleVerifier.actor.cpp +++ b/fdbserver/workloads/BlobGranuleVerifier.actor.cpp @@ -166,20 +166,35 @@ struct BlobGranuleVerifierWorkload : TestWorkload { // assumes we can read the whole range in one transaction at a single version ACTOR Future> readFromFDB(Database cx, KeyRange range) { + state bool first = true; state Version v; state RangeResult out; state Transaction tr(cx); state KeyRange currentRange = range; loop { try { - RangeResult r = wait(tr.getRange(currentRange, CLIENT_KNOBS->TOO_MANY)); + state RangeResult r = wait(tr.getRange(currentRange, CLIENT_KNOBS->TOO_MANY)); + Version grv = wait(tr.getReadVersion()); + // need consistent version snapshot of range + if (first) { + v = grv; + first = false; + } else if (v != grv) { + // reset the range and restart the read at a higher version + TraceEvent(SevDebug, "BGVFDBReadReset").detail("ReadVersion", v); + TEST(true); // BGV transaction reset + printf("Resetting BGV GRV {0} -> {1}\n", v, grv); + first = true; + out = RangeResult(); + currentRange = range; + tr.reset(); + continue; + } out.arena().dependsOn(r.arena()); out.append(out.arena(), r.begin(), r.size()); if (r.more) { currentRange = KeyRangeRef(keyAfter(r.back().key), currentRange.end); } else { - Version _v = wait(tr.getReadVersion()); - v = _v; break; } } catch (Error& e) { From d4931348c1c3dbd43e80021d200c39d0878b1640 Mon Sep 17 00:00:00 2001 From: Josh Slocum Date: Mon, 31 Jan 2022 11:18:36 -0600 Subject: [PATCH 187/413] Fix for new change feed movement --- fdbserver/storageserver.actor.cpp | 28 ++++++++++++++++++++-------- 1 file changed, 20 insertions(+), 8 deletions(-) diff --git a/fdbserver/storageserver.actor.cpp b/fdbserver/storageserver.actor.cpp index d1668217e3..b9eb05c709 100644 --- a/fdbserver/storageserver.actor.cpp +++ b/fdbserver/storageserver.actor.cpp @@ -1775,6 +1775,7 @@ ACTOR Future> getChangeFeedMutations(Stor // waiting for the results state Version dequeVersion = data->version.get(); state Version dequeKnownCommit = data->knownCommittedVersion; + state Version emptyVersion = feedInfo->emptyVersion; if (DEBUG_SS_CFM(data->thisServerID, req.rangeID, req.begin)) { printf("CFM: SS %s CF %s: dequeVersion=%lld, emptyVersion=%lld, storageVersion=%lld, durableVersion=%lld, " @@ -1788,7 +1789,7 @@ ACTOR Future> getChangeFeedMutations(Stor feedInfo->fetchVersion); } - if (req.end > feedInfo->emptyVersion + 1) { + if (req.end > emptyVersion + 1) { for (auto& it : feedInfo->mutations) { if (it.version >= req.end || it.version > dequeVersion || remainingLimitBytes <= 0) { break; @@ -1810,13 +1811,24 @@ ACTOR Future> getChangeFeedMutations(Stor } } - if (req.end > feedInfo->emptyVersion + 1 && feedInfo->durableVersion != invalidVersion && - req.begin <= feedInfo->durableVersion) { - RangeResult res = wait(data->storage.readRange( - KeyRangeRef(changeFeedDurableKey(req.rangeID, std::max(req.begin, feed->second->emptyVersion)), - changeFeedDurableKey(req.rangeID, req.end)), - 1 << 30, - remainingDurableBytes)); + bool readDurable = feedInfo->durableVersion != invalidVersion && req.begin <= feedInfo->durableVersion; + bool readFetched = feedInfo->durableVersion < feedInfo->fetchVersion && req.begin <= feedInfo->fetchVersion; + if (req.end > emptyVersion + 1 && (readDurable || readFetched)) { + if (readFetched && feedInfo->durableVersion == invalidVersion) { + // To not block fetchKeys on making change feed data written to storage, we wait in here instead for all + // fetched data to become readable from the storage engine. + while (feedInfo->durableVersion == invalidVersion) { + TEST(true); // getChangeFeedMutations before any fetched data durable + // wait for next commit + wait(data->durableVersion.whenAtLeast(data->durableVersion.get() + 1)); + // TODO it may be safer to always just wait for durableVersion whenAtLeast feedVersion? + } + } + RangeResult res = wait( + data->storage.readRange(KeyRangeRef(changeFeedDurableKey(req.rangeID, std::max(req.begin, emptyVersion)), + changeFeedDurableKey(req.rangeID, req.end)), + 1 << 30, + remainingDurableBytes)); if (!inverted && !req.range.empty()) { data->checkChangeCounter(changeCounter, req.range); From 25a0d857fa48be31813405e9a95b668ebce82761 Mon Sep 17 00:00:00 2001 From: Josh Slocum Date: Mon, 31 Jan 2022 17:19:52 -0600 Subject: [PATCH 188/413] Fixing issue with change_feed_popped --- fdbserver/storageserver.actor.cpp | 58 +++++++++++++++++++++++-------- 1 file changed, 43 insertions(+), 15 deletions(-) diff --git a/fdbserver/storageserver.actor.cpp b/fdbserver/storageserver.actor.cpp index b9eb05c709..63095412e3 100644 --- a/fdbserver/storageserver.actor.cpp +++ b/fdbserver/storageserver.actor.cpp @@ -1717,7 +1717,9 @@ MutationsAndVersionRef filterMutations(Arena& arena, ACTOR Future> getChangeFeedMutations(StorageServer* data, ChangeFeedStreamRequest req, - bool inverted) { + bool inverted, + UID streamUID + /*TODO REMOVE*/) { state ChangeFeedStreamReply reply; state ChangeFeedStreamReply memoryReply; state int remainingLimitBytes = CLIENT_KNOBS->REPLY_BYTE_LIMIT; @@ -1725,9 +1727,10 @@ ACTOR Future> getChangeFeedMutations(Stor state Version startVersion = data->version.get(); if (DEBUG_SS_CFM(data->thisServerID, req.rangeID, req.begin)) { - printf("CFM: SS %s CF %s: GCFM [%s - %s) %lld - %lld\n", + printf("CFM: SS %s CF %s: GCFM %s [%s - %s) %lld - %lld\n", data->thisServerID.toString().substr(0, 4).c_str(), req.rangeID.printable().substr(0, 6).c_str(), + streamUID.toString().substr(0, 8).c_str(), req.range.begin.printable().c_str(), req.range.end.printable().c_str(), req.begin, @@ -1760,7 +1763,7 @@ ACTOR Future> getChangeFeedMutations(Stor state Reference feedInfo = feed->second; if (DEBUG_SS_CFM(data->thisServerID, req.rangeID, req.begin)) { - printf("CFM: SS %s CF %s: got version %lld >= %lld\n", + printf("CFM: SS %s CF %s: got version %lld >= %lld\n", data->thisServerID.toString().substr(0, 4).c_str(), req.rangeID.printable().substr(0, 6).c_str(), data->version.get(), @@ -1768,6 +1771,12 @@ ACTOR Future> getChangeFeedMutations(Stor } if (!req.canReadPopped && req.begin <= feedInfo->emptyVersion) { + printf("CFM: SS %s CF %s: %s popped! req.begin=%lld, emptyVersion=%lld\n", + data->thisServerID.toString().substr(0, 4).c_str(), + req.rangeID.printable().substr(0, 6).c_str(), + streamUID.toString().substr(0, 8).c_str(), + req.begin, + feedInfo->emptyVersion); throw change_feed_popped(); } @@ -1778,7 +1787,7 @@ ACTOR Future> getChangeFeedMutations(Stor state Version emptyVersion = feedInfo->emptyVersion; if (DEBUG_SS_CFM(data->thisServerID, req.rangeID, req.begin)) { - printf("CFM: SS %s CF %s: dequeVersion=%lld, emptyVersion=%lld, storageVersion=%lld, durableVersion=%lld, " + printf("CFM: SS %s CF %s: dequeVersion=%lld, emptyVersion=%lld, storageVersion=%lld, durableVersion=%lld, " "fetchVersion=%lld\n", data->thisServerID.toString().substr(0, 4).c_str(), req.rangeID.printable().substr(0, 6).c_str(), @@ -1802,7 +1811,7 @@ ACTOR Future> getChangeFeedMutations(Stor } } if (DEBUG_SS_CFM(data->thisServerID, req.rangeID, req.begin)) { - printf("CFM: SS %s CF %s: got %lld - %lld (%d) from memory\n", + printf("CFM: SS %s CF %s: got %lld - %lld (%d) from memory\n", data->thisServerID.toString().substr(0, 4).c_str(), req.rangeID.printable().substr(0, 6).c_str(), memoryReply.mutations.empty() ? invalidVersion : memoryReply.mutations.front().version, @@ -1852,7 +1861,7 @@ ACTOR Future> getChangeFeedMutations(Stor lastVersion = version; } if (DEBUG_SS_CFM(data->thisServerID, req.rangeID, req.begin)) { - printf("CFM: SS %s CF %s: got %lld - %lld (%d) from disk\n", + printf("CFM: SS %s CF %s: got %lld - %lld (%d) from disk\n", data->thisServerID.toString().substr(0, 4).c_str(), req.rangeID.printable().substr(0, 6).c_str(), reply.mutations.empty() ? invalidVersion : reply.mutations.front().version, @@ -1875,6 +1884,12 @@ ACTOR Future> getChangeFeedMutations(Stor // check if pop happened concurrently with read if (!req.canReadPopped && req.begin <= feedInfo->emptyVersion) { + printf("SS %s: CF %s popped after read! req.begin=%lld, emptyVersion=%lld, emptyBeforeRead=%lld\n", + data->thisServerID.toString().substr(0, 4).c_str(), + req.rangeID.printable().substr(0, 6).c_str(), + req.begin, + feedInfo->emptyVersion, + emptyVersion); throw change_feed_popped(); } @@ -1883,7 +1898,7 @@ ACTOR Future> getChangeFeedMutations(Stor if ((reply.mutations.empty() || reply.mutations.back().version < finalVersion) && remainingLimitBytes > 0 && remainingDurableBytes > 0) { if (DEBUG_SS_CFM(data->thisServerID, req.rangeID, req.begin)) { - printf("CFM: SS %s CF %s: adding empty %lld\n", + printf("CFM: SS %s CF %s: adding empty %lld\n", data->thisServerID.toString().substr(0, 4).c_str(), req.rangeID.printable().substr(0, 6).c_str(), finalVersion); @@ -1909,7 +1924,7 @@ ACTOR Future> getChangeFeedMutations(Stor } if (DEBUG_SS_CFM(data->thisServerID, req.rangeID, req.begin)) { - printf("CFM: SS %s CF %s: result %lld - %lld (%d)\n", + printf("CFM: SS %s CF %s: result %lld - %lld (%d)\n", data->thisServerID.toString().substr(0, 4).c_str(), req.rangeID.printable().substr(0, 6).c_str(), reply.mutations.empty() ? invalidVersion : reply.mutations.front().version, @@ -1945,7 +1960,7 @@ ACTOR Future localChangeFeedStream(StorageServer* data, feedRequest.end = end; feedRequest.range = range; state std::pair feedReply = - wait(getChangeFeedMutations(data, feedRequest, true)); + wait(getChangeFeedMutations(data, feedRequest, true, UID())); begin = feedReply.first.mutations.back().version + 1; state int resultLoc = 0; while (resultLoc < feedReply.first.mutations.size()) { @@ -1995,9 +2010,10 @@ ACTOR Future changeFeedStreamQ(StorageServer* data, ChangeFeedStreamReques wait(delay(0, TaskPriority::DefaultEndpoint)); if (DEBUG_SS_CFM(data->thisServerID, req.rangeID, req.begin)) { - printf("CFM: SS %s CF %s: got CFSQ [%s - %s) %lld - %lld, crp=%s\n", + printf("CFM: SS %s CF %s: got CFSQ %s [%s - %s) %lld - %lld, crp=%s\n", data->thisServerID.toString().substr(0, 4).c_str(), req.rangeID.printable().substr(0, 6).c_str(), + streamUID.toString().substr(0, 8).c_str(), req.range.begin.printable().c_str(), req.range.end.printable().c_str(), req.begin, @@ -2015,7 +2031,7 @@ ACTOR Future changeFeedStreamQ(StorageServer* data, ChangeFeedStreamReques } wait(onReady); state Future> feedReplyFuture = - getChangeFeedMutations(data, req, false); + getChangeFeedMutations(data, req, false, streamUID); if (atLatest && !removeUID && !feedReplyFuture.isReady()) { data->changeFeedClientVersions[req.reply.getEndpoint().getPrimaryAddress()][streamUID] = blockedVersion.present() ? blockedVersion.get() : data->prevVersion; @@ -2060,10 +2076,9 @@ ACTOR Future changeFeedStreamQ(StorageServer* data, ChangeFeedStreamReques req.reply.sendError(unknown_change_feed()); return Void(); } + state Version emptyBefore = feed->second->emptyVersion; choose { - when(wait(feed->second->newMutations.onTrigger())) { - } // FIXME: check that this is triggered when the range is moved to a different - // server, also check that the stream is closed + when(wait(feed->second->newMutations.onTrigger())) {} when(wait(req.end == std::numeric_limits::max() ? Future(Never()) : data->version.whenAtLeast(req.end))) {} } @@ -2072,11 +2087,21 @@ ACTOR Future changeFeedStreamQ(StorageServer* data, ChangeFeedStreamReques req.reply.sendError(unknown_change_feed()); return Void(); } + if (emptyBefore != feed->second->emptyVersion && !req.canReadPopped && + req.begin <= feed->second->emptyVersion) { + // Change feed was popped with no new mutations, update its begin version to skip those versions so + // it doesn't get change_feed_popped. This is safe because change_feed_popped is to ensure an old + // read can't miss mutations from a change feed stream, and this read is guaranteed not to (it was + // caught up before the pop, and trigger wasn't called on any new mutations before the pop) + req.begin = feed->second->emptyVersion + 1; + } } else { blockedVersion = feedReply.mutations.back().version; } } } catch (Error& e) { + // TODO REMOVE + printf("CFSQ %s got error %s\n", streamUID.toString().substr(0, 8).c_str(), e.name()); auto it = data->changeFeedClientVersions.find(req.reply.getEndpoint().getPrimaryAddress()); if (it != data->changeFeedClientVersions.end()) { if (removeUID) { @@ -4023,6 +4048,8 @@ ACTOR Future changeFeedPopQ(StorageServer* self, ChangeFeedPopRequest req) feed->second->durableVersion = invalidVersion; } } + // wake up requests that didn't get any mutations since the last pop to update their beginVersion + feed->second->newMutations.trigger(); wait(self->durableVersion.whenAtLeast(durableVersion)); } req.reply.send(Void()); @@ -4037,7 +4064,6 @@ ACTOR Future fetchChangeFeedApplier(StorageServer* data, Version fetchVersion, bool existing) { state Reference feedResults = makeReference(); - printf("DBG: fetching %lld %lld %lld\n", emptyVersion, emptyVersion + 1, fetchVersion); Version startVersion = emptyVersion + 1; if (startVersion < 0) { startVersion = 0; @@ -5250,6 +5276,8 @@ private: addMutationToLog = true; } feed->second->stopped = (status == ChangeFeedStatus::CHANGE_FEED_STOP); + // wake up requests that didn't get any mutations since the last pop to update their beginVersion + feed->second->newMutations.trigger(); } else if (status == ChangeFeedStatus::CHANGE_FEED_CREATE) { TraceEvent(SevDebug, "CreatingChangeFeed", data->thisServerID) .detail("RangeID", changeFeedId.printable()) From d0113a67760e7fc8439a150b3c3b79d06f206cef Mon Sep 17 00:00:00 2001 From: Josh Slocum Date: Mon, 31 Jan 2022 16:09:11 -0600 Subject: [PATCH 189/413] Added mechanism for blob manager to poll blob workers for their granule assignments, and used that to improve manager recovery --- fdbclient/BlobWorkerInterface.h | 41 ++++++ fdbserver/BlobManager.actor.cpp | 237 ++++++++++++++++++++++++-------- fdbserver/BlobWorker.actor.cpp | 92 +++++++++---- 3 files changed, 283 insertions(+), 87 deletions(-) diff --git a/fdbclient/BlobWorkerInterface.h b/fdbclient/BlobWorkerInterface.h index 480883b263..3381b54da4 100644 --- a/fdbclient/BlobWorkerInterface.h +++ b/fdbclient/BlobWorkerInterface.h @@ -34,6 +34,7 @@ struct BlobWorkerInterface { RequestStream blobGranuleFileRequest; RequestStream assignBlobRangeRequest; RequestStream revokeBlobRangeRequest; + RequestStream granuleAssignmentsRequest; RequestStream granuleStatusStreamRequest; RequestStream haltBlobWorker; @@ -58,6 +59,7 @@ struct BlobWorkerInterface { blobGranuleFileRequest, assignBlobRangeRequest, revokeBlobRangeRequest, + granuleAssignmentsRequest, granuleStatusStreamRequest, haltBlobWorker, locality, @@ -116,6 +118,7 @@ struct RevokeBlobRangeRequest { * Reassign: when a new blob manager takes over, it sends Reassign requests to workers to redistribute granules * Normal: Neither continue nor reassign */ +// TODO REMOVE reassign now! enum AssignRequestType { Normal = 0, Continue = 1, Reassign = 2 }; struct AssignBlobRangeRequest { @@ -213,4 +216,42 @@ struct HaltBlobWorkerRequest { } }; +struct GranuleAssignmentRef { + KeyRangeRef range; + int64_t epochAssigned; + int64_t seqnoAssigned; + + GranuleAssignmentRef() {} + + explicit GranuleAssignmentRef(KeyRangeRef range, int64_t epochAssigned, int64_t seqnoAssigned) + : range(range), epochAssigned(epochAssigned), seqnoAssigned(seqnoAssigned) {} + + template + void serialize(Ar& ar) { + serializer(ar, range, epochAssigned, seqnoAssigned); + } +}; + +struct GetGranuleAssignmentsReply { + constexpr static FileIdentifier file_identifier = 9191718; + Arena arena; + VectorRef assignments; + + template + void serialize(Ar& ar) { + serializer(ar, assignments, arena); + } +}; + +struct GetGranuleAssignmentsRequest { + constexpr static FileIdentifier file_identifier = 4121494; + int64_t managerEpoch; + ReplyPromise reply; + + template + void serialize(Ar& ar) { + serializer(ar, managerEpoch, reply); + } +}; + #endif diff --git a/fdbserver/BlobManager.actor.cpp b/fdbserver/BlobManager.actor.cpp index 65b87fe294..567843cd72 100644 --- a/fdbserver/BlobManager.actor.cpp +++ b/fdbserver/BlobManager.actor.cpp @@ -1326,19 +1326,79 @@ ACTOR Future checkBlobWorkerList(Reference bmData, Promis } } +// Shared code for handling KeyRangeMap that is used several places in blob manager recovery +// when there can be conflicting sources of what assignments exist or which workers owns a granule. +// Resolves these conflicts by comparing the epoch + seqno for the range +// Special epoch/seqnos: +// (0,0): range is not mapped +// (0,1): range is mapped, but worker is unknown +static void addAssignment(KeyRangeMap>& map, + const KeyRangeRef& newRange, + UID newId, + int64_t newEpoch, + int64_t newSeqno, + std::vector>* outOfDate = nullptr) { + std::vector>> newer; + auto intersecting = map.intersectingRanges(newRange); + bool allNewer = true; + for (auto& old : intersecting) { + UID oldWorker = std::get<0>(old.value()); + int64_t oldEpoch = std::get<1>(old.value()); + int64_t oldSeqno = std::get<2>(old.value()); + if (oldEpoch > newEpoch || (oldEpoch == newEpoch && oldSeqno > newSeqno)) { + newer.push_back(std::pair(old.range(), std::tuple(oldWorker, oldEpoch, oldSeqno))); + } else { + allNewer = false; + if (newId != UID()) { + // different workers can't have same epoch and seqno for granule assignment + ASSERT(oldEpoch != newEpoch || oldSeqno != newSeqno); + } + if (outOfDate != nullptr && oldEpoch > 0) { + outOfDate->push_back(std::pair(oldWorker, old.range())); + } + } + } + + if (!allNewer) { + // if this range supercedes an old range insert it over that + map.insert(newRange, std::tuple(newId, newEpoch, newSeqno)); + + // then, if there were any ranges superceded by this one, insert them over this one + if (newer.size()) { + if (outOfDate != nullptr) { + outOfDate->push_back(std::pair(newId, newRange)); + } + for (auto& it : newer) { + map.insert(it.first, it.second); + } + } + } else { + if (outOfDate != nullptr) { + outOfDate->push_back(std::pair(newId, newRange)); + } + } +} + ACTOR Future recoverBlobManager(Reference bmData) { state Promise workerListReady; bmData->addActor.send(checkBlobWorkerList(bmData, workerListReady)); wait(workerListReady.getFuture()); + state std::vector startingWorkers; + for (auto& it : bmData->workersById) { + startingWorkers.push_back(it.second); + } + // Once we acknowledge the existing blob workers, we can go ahead and recruit new ones bmData->startRecruiting.trigger(); - // skip them rest of the algorithm for the first blob manager + // skip the rest of the algorithm for the first blob manager if (bmData->epoch == 1) { return Void(); } + wait(delay(0)); + // At this point, bmData->workersById is a list of all alive blob workers, but could also include some dead BWs. // The algorithm below works as follows: // 1. We get the ongoing split boundaries to construct the set of granules we should have. For these splits, we @@ -1347,26 +1407,21 @@ ACTOR Future recoverBlobManager(Reference bmData) { // intended to give a splitted range to dies before the new BM recovers, then we'll simply assign the range to // the next best worker. // - // 2. We get the existing granule mappings that were persisted by blob workers who were assigned ranges and - // add them to bmData->granuleAssignments, which is a key range map. - // Details: re-assignments might have happened between the time the mapping was last updated and now. - // For example, suppose a blob manager sends requests to the range assigner stream to move a granule G. - // However, before sending those requests off to the workers, the BM dies. So the persisting mapping - // still has G->oldWorker. The following algorithm will re-assign G to oldWorker (as long as it is also still - // alive). Note that this is fine because it simply means that the range was not moved optimally, but it is - // still owned. In the above case, even if the revoke goes through, since we don't update the mapping during - // revokes, this is the same as the case above. Another case to consider is when a blob worker dies when the - // BM is recovering. Now the mapping at this time looks like G->deadBW. But the rangeAssigner handles this: - // we'll try to assign a range to a dead worker and fail and reassign it to the next best worker. It will also - // handle the case where the mapping does not reflect the desired set of granules based on the ongoing spits, and - // correct it. + // 2. We get the existing granule mappings. We do this by asking all active blob workers for their current granule + // assignments. This guarantees a consistent snapshot of the state of that worker's assignments: Any request it + // recieved and processed from the old manager before the granule assignment request will be included in the + // assignments, and any request it recieves from the old manager afterwards will be rejected with + // blob_manager_replaced. We will then read any gaps in the mapping from the database. We will reconcile the set + // of ongoing splits to this mapping, and any ranges that are not already assigned to existing blob workers will + // be reassigned. // // 3. For every range in our granuleAssignments, we send an assign request to the stream of requests, // ultimately giving every range back to some worker (trying to mimic the state of the old BM). // If the worker already had the range, this is a no-op. If the worker didn't have it, it will // begin persisting it. The worker that had the same range before will now be at a lower seqno. - state KeyRangeMap> workerAssignments; + state KeyRangeMap> workerAssignments; + workerAssignments.insert(normalKeys, std::tuple(UID(), 0, 0)); state Reference tr = makeReference(bmData->db); // TODO KNOB @@ -1395,7 +1450,8 @@ ACTOR Future recoverBlobManager(Reference bmData) { // time, and we don't know which parts of those are reflected in the current set of worker assignments we read, we // have to construct the current desired set of granules from the set of ongoing splits and merges. Then, if any of // those are not represented in the worker mapping, we must add them. - state KeyRangeMap> inProgressSplits; + state KeyRangeMap> inProgressSplits; + inProgressSplits.insert(normalKeys, std::tuple(UID(), 0, 0)); tr->reset(); tr->setOption(FDBTransactionOptions::ACCESS_SYSTEM_KEYS); @@ -1483,25 +1539,7 @@ ACTOR Future recoverBlobManager(Reference bmData) { if (BM_DEBUG) { fmt::print(" [{0} - {1})\n", range.begin.printable(), range.end.printable()); } - - // same algorithm as worker map. If we read boundary changes from the log out of order, save the newer - // ones, apply this one, and re-apply the other ones over this one don't concurrently modify with - // iterator - std::vector>> newer; - newer.reserve(splitBoundaries.size() - 1); - auto intersecting = inProgressSplits.intersectingRanges(range); - for (auto& it : intersecting) { - if (splitEpochSeqno.first < it.value().first || - (splitEpochSeqno.first == it.value().first && splitEpochSeqno.second < it.value().second)) { - // range currently there is newer than this range. - newer.push_back(std::pair(it.range(), it.value())); - } - } - inProgressSplits.insert(range, splitEpochSeqno); - - for (auto& it : newer) { - inProgressSplits.insert(it.first, it.second); - } + addAssignment(inProgressSplits, range, UID(), splitEpochSeqno.first, splitEpochSeqno.second); } } splitBoundaries.clear(); @@ -1514,15 +1552,62 @@ ACTOR Future recoverBlobManager(Reference bmData) { nextParentID.reset(); } - if (BM_DEBUG) { - fmt::print("BM {0} found old assignments:\n", bmData->epoch); - } - // TODO could populate most/all of this list by just asking existing blob workers for their range sets to reduce DB - // read load on BM restart - // Step 3. Get the latest known mapping of granules to blob workers (i.e. assignments) // This must happen causally AFTER reading the split boundaries, since the blob workers can clear the split // boundaries for a granule as part of persisting their assignment. + + // First, ask existing workers for their mapping + if (BM_DEBUG) { + fmt::print("BM {0} requesting assignments from {1} workers:\n", bmData->epoch, startingWorkers.size()); + } + state std::vector>> aliveAssignments; + aliveAssignments.reserve(startingWorkers.size()); + for (auto& it : startingWorkers) { + GetGranuleAssignmentsRequest req; + req.managerEpoch = bmData->epoch; + aliveAssignments.push_back(timeout(brokenPromiseToNever(it.granuleAssignmentsRequest.getReply(req)), + SERVER_KNOBS->BLOB_WORKER_TIMEOUT)); + } + waitForAll(aliveAssignments); + + state std::vector> outOfDateAssignments; + state int successful = 0; + state int assignIdx = 0; + // FIXME: more CPU efficient to do sorted merge of assignments? + for (; assignIdx < aliveAssignments.size(); assignIdx++) { + Optional reply = wait(aliveAssignments[assignIdx]); + UID workerId = startingWorkers[assignIdx].id(); + if (reply.present()) { + successful++; + for (auto& assignment : reply.get().assignments) { + bmData->knownBlobRanges.insert(assignment.range, true); + addAssignment(workerAssignments, + assignment.range, + workerId, + assignment.epochAssigned, + assignment.seqnoAssigned, + &outOfDateAssignments); + } + wait(yield()); + } else { + // TODO mark as failed and kill it + } + } + + if (BM_DEBUG) { + fmt::print("BM {0} got assignments from {1}/{2} workers:\n", bmData->epoch, successful, startingWorkers.size()); + } + + if (BM_DEBUG) { + fmt::print("BM {0} found old assignments:\n", bmData->epoch); + } + + // then, read any gaps in worker assignment from FDB + // With a small number of blob workers, if even one is missing, doing numGranules/numWorkers small range reads from + // FDB is probably less efficient than just reading the whole mapping anyway + // Plus, we don't have a consistent snapshot of the mapping ACROSS blob workers, so we need the DB to reconcile any + // differences (eg blob manager revoked from worker A, assigned to B, the revoke from A was processed but the assign + // to B wasn't, meaning in the snapshot nobody owns the granule) state KeyRef beginKey = blobGranuleMappingKeys.begin; loop { try { @@ -1543,7 +1628,8 @@ ACTOR Future recoverBlobManager(Reference bmData) { if (results[rangeIdx].value.size()) { // note: if the old owner is dead, we handle this in rangeAssigner UID existingOwner = decodeBlobGranuleMappingValue(results[rangeIdx].value); - workerAssignments.insert(KeyRangeRef(granuleStartKey, granuleEndKey), existingOwner); + addAssignment(workerAssignments, KeyRangeRef(granuleStartKey, granuleEndKey), existingOwner, 0, 1); + bmData->knownBlobRanges.insert(KeyRangeRef(granuleStartKey, granuleEndKey), true); if (BM_DEBUG) { fmt::print(" [{0} - {1})={2}\n", @@ -1579,18 +1665,14 @@ ACTOR Future recoverBlobManager(Reference bmData) { // override and assign it to a new worker auto splits = inProgressSplits.intersectingRanges(normalKeys); for (auto& it : splits) { - if (it.value().first == 0 || it.value().second == 0) { + int64_t epoch = std::get<1>(it.value()); + int64_t seqno = std::get<2>(it.value()); + if (epoch == 0 || seqno == 0) { // no in-progress splits for this range continue; } - auto r = workerAssignments.rangeContaining(it.begin()); - // if this range is at all different from the worker mapping, the mapping is out of date - if (r.begin() != it.begin() || r.end() != it.end()) { - // the empty UID signifies that we need to find an owner (worker) for this range - workerAssignments.insert(it.range(), UID()); - fmt::print(" [{0} - {1})\n", it.begin().printable().c_str(), it.end().printable().c_str()); - } + addAssignment(workerAssignments, it.range(), UID(), epoch, seqno, &outOfDateAssignments); } // Step 4. Send assign requests for all the granules and transfer assignments @@ -1611,30 +1693,67 @@ ACTOR Future recoverBlobManager(Reference bmData) { } } + // revoke assignments that are old and incorrect + TEST(!outOfDateAssignments.empty()); // BM resolved conflicting assignments on recovery + for (auto& it : outOfDateAssignments) { + if (BM_DEBUG) { + fmt::print("BM {0} revoking out of date assignment [%s - %s): %s:\n", + bmData->epoch, + it.second.begin.printable().c_str(), + it.second.end.printable().c_str(), + it.first.toString().c_str()); + } + RangeAssignment raRevoke; + raRevoke.isAssign = false; + raRevoke.worker = it.first; + raRevoke.keyRange = it.second; + bmData->rangesToAssign.send(raRevoke); + } + if (BM_DEBUG) { fmt::print("BM {0} final ranges:\n", bmData->epoch); } for (auto& range : workerAssignments.intersectingRanges(normalKeys)) { - if (!range.value().present()) { + int64_t epoch = std::get<1>(range.value()); + int64_t seqno = std::get<2>(range.value()); + if (epoch == 0 && seqno == 0) { /*if (BM_DEBUG) { fmt::print(" [{0} - {1}) invalid\n", range.begin().printable(), range.end().printable()); }*/ continue; } + UID workerId = std::get<0>(range.value()); + bmData->workerAssignments.insert(range.range(), workerId); + if (BM_DEBUG) { - fmt::print(" [{0} - {1})\n", range.begin().printable(), range.end().printable()); + fmt::print(" [{0} - {1}){2}\n", + range.begin().printable(), + range.end().printable(), + workerId == UID() ? " (*)" : ""); } - bmData->workerAssignments.insert(range.range(), range.value().get()); + // if worker id is already set to a known worker, range is already assigned there. If not, need to explicitly + // assign it + if (workerId == UID()) { + RangeAssignment raAssign; + raAssign.isAssign = true; + raAssign.worker = workerId; + raAssign.keyRange = range.range(); + raAssign.assign = RangeAssignmentData(AssignRequestType::Normal); + bmData->rangesToAssign.send(raAssign); + } + } - RangeAssignment raAssign; - raAssign.isAssign = true; - raAssign.worker = range.value().get(); - raAssign.keyRange = range.range(); - raAssign.assign = RangeAssignmentData(AssignRequestType::Reassign); - bmData->rangesToAssign.send(raAssign); + // coalesce known blob ranges within boundaries at the very end + RangeResult results = + wait(krmGetRanges(tr, blobRangeKeys.begin, KeyRange(normalKeys), 10000, GetRangeLimits::BYTE_LIMIT_UNLIMITED)); + ASSERT(!results.more && results.size() < CLIENT_KNOBS->TOO_MANY); + for (int i = 0; i < results.size() - 1; i++) { + if (results[i].value.size()) { + bmData->knownBlobRanges.coalesce(KeyRangeRef(results[i].key, results[i + 1].key)); + } } return Void(); diff --git a/fdbserver/BlobWorker.actor.cpp b/fdbserver/BlobWorker.actor.cpp index 4cdd94258b..1076d9f22f 100644 --- a/fdbserver/BlobWorker.actor.cpp +++ b/fdbserver/BlobWorker.actor.cpp @@ -2604,14 +2604,17 @@ static bool newerRangeAssignment(GranuleRangeMetadata oldMetadata, int64_t epoch // manager with a successful assignment And if the change produced a new granule that needs to start // doing work, returns the new granule so that the caller can start() it with the appropriate starting // state. -ACTOR Future changeBlobRange(Reference bwData, - KeyRange keyRange, - int64_t epoch, - int64_t seqno, - bool active, - bool disposeOnCleanup, - bool selfReassign, - Optional assignType = Optional()) { + +// Not an actor because we need to guarantee it changes the synchronously as part of the request +static bool changeBlobRange(Reference bwData, + KeyRange keyRange, + int64_t epoch, + int64_t seqno, + bool active, + bool disposeOnCleanup, + bool selfReassign, + std::vector>& toWaitOut, + Optional assignType = Optional()) { // since changeBlobRange is used for assigns and revokes, // we assert that assign type is specified iff this is an ASSERT(active == assignType.present()); @@ -2632,23 +2635,11 @@ ACTOR Future changeBlobRange(Reference bwData, // older, or newer. For each older range, cancel it if it is active. Insert the current range. // Re-insert all newer ranges over the current range. - state std::vector> futures; - - state std::vector> newerRanges; + std::vector> newerRanges; auto ranges = bwData->granuleMetadata.intersectingRanges(keyRange); bool alreadyAssigned = false; for (auto& r : ranges) { - // I don't think we need this? - /*if (!active) { - if (r.value().activeMetadata.isValid() && r.value().activeMetadata->cancelled.canBeSet()) { - if (BW_DEBUG) { - printf("Cancelling activeMetadata\n"); - } - bwData->stats.numRangesAssigned--; - r.value().activeMetadata->cancelled.send(Void()); - } - }*/ bool thisAssignmentNewer = newerRangeAssignment(r.value(), epoch, seqno); if (BW_DEBUG) { fmt::print("thisAssignmentNewer={}\n", thisAssignmentNewer ? "true" : "false"); @@ -2684,7 +2675,7 @@ ACTOR Future changeBlobRange(Reference bwData, } // applied the same assignment twice, make idempotent if (r.value().activeMetadata.isValid()) { - futures.push_back(success(r.value().assignFuture)); + toWaitOut.push_back(success(r.value().assignFuture)); } alreadyAssigned = true; break; @@ -2711,7 +2702,6 @@ ACTOR Future changeBlobRange(Reference bwData, } if (alreadyAssigned) { - wait(waitForAll(futures)); // already applied, nothing to do return false; } @@ -2742,7 +2732,6 @@ ACTOR Future changeBlobRange(Reference bwData, bwData->granuleMetadata.insert(it.first, it.second); } - wait(waitForAll(futures)); return newerRanges.size() == 0; } @@ -2809,6 +2798,8 @@ ACTOR Future registerBlobWorker(Reference bwData, BlobWork } } +// the contract of handleRangeAssign and handleRangeRevoke is that they change the mapping before doing any waiting. +// This ensures GetGranuleAssignment returns an up-to-date set of ranges ACTOR Future handleRangeAssign(Reference bwData, AssignBlobRangeRequest req, bool isSelfReassign) { @@ -2816,8 +2807,17 @@ ACTOR Future handleRangeAssign(Reference bwData, if (req.type == AssignRequestType::Continue) { resumeBlobRange(bwData, req.keyRange, req.managerEpoch, req.managerSeqno); } else { - bool shouldStart = wait(changeBlobRange( - bwData, req.keyRange, req.managerEpoch, req.managerSeqno, true, false, isSelfReassign, req.type)); + std::vector> toWait; + state bool shouldStart = changeBlobRange(bwData, + req.keyRange, + req.managerEpoch, + req.managerSeqno, + true, + false, + isSelfReassign, + toWait, + req.type); + wait(waitForAll(toWait)); if (shouldStart) { bwData->stats.numRangesAssigned++; @@ -2863,8 +2863,9 @@ ACTOR Future handleRangeAssign(Reference bwData, ACTOR Future handleRangeRevoke(Reference bwData, RevokeBlobRangeRequest req) { try { - wait(success( - changeBlobRange(bwData, req.keyRange, req.managerEpoch, req.managerSeqno, false, req.dispose, false))); + std::vector> toWait; + changeBlobRange(bwData, req.keyRange, req.managerEpoch, req.managerSeqno, false, req.dispose, false, toWait); + wait(waitForAll(toWait)); req.reply.send(Void()); return Void(); } catch (Error& e) { @@ -2943,6 +2944,29 @@ ACTOR Future runGRVChecks(Reference bwData) { } } +static void handleGetGranuleAssignmentsRequest(Reference self, + const GetGranuleAssignmentsRequest& req) { + GetGranuleAssignmentsReply reply; + auto allRanges = self->granuleMetadata.intersectingRanges(normalKeys); + for (auto& it : allRanges) { + if (it.value().activeMetadata.isValid()) { + // range is active, copy into reply's arena + StringRef start = StringRef(reply.arena, it.begin()); + StringRef end = StringRef(reply.arena, it.end()); + + reply.assignments.push_back( + reply.arena, GranuleAssignmentRef(KeyRangeRef(start, end), it.value().lastEpoch, it.value().lastSeqno)); + } + } + if (BW_DEBUG) { + fmt::print("Worker {0} sending {1} granule assignments back to BM {2}\n", + self->id.toString(), + reply.assignments.size(), + req.managerEpoch); + } + req.reply.send(reply); +} + ACTOR Future blobWorker(BlobWorkerInterface bwInterf, ReplyPromise recruitReply, Reference const> dbInfo) { @@ -3060,6 +3084,18 @@ ACTOR Future blobWorker(BlobWorkerInterface bwInterf, when(AssignBlobRangeRequest granuleToReassign = waitNext(self->granuleUpdateErrors.getFuture())) { self->addActor.send(handleRangeAssign(self, granuleToReassign, true)); } + when(GetGranuleAssignmentsRequest req = waitNext(bwInterf.granuleAssignmentsRequest.getFuture())) { + if (self->managerEpochOk(req.managerEpoch)) { + if (BW_DEBUG) { + fmt::print("Worker {0} got granule assignments request from BM {1}\n", + self->id.toString(), + req.managerEpoch); + } + handleGetGranuleAssignmentsRequest(self, req); + } else { + req.reply.sendError(blob_manager_replaced()); + } + } when(HaltBlobWorkerRequest req = waitNext(bwInterf.haltBlobWorker.getFuture())) { if (self->managerEpochOk(req.managerEpoch)) { TraceEvent("BlobWorkerHalted", self->id) From 30d8c593c8e5903c16a3135d834d97abca41339e Mon Sep 17 00:00:00 2001 From: Josh Slocum Date: Mon, 31 Jan 2022 17:10:08 -0600 Subject: [PATCH 190/413] Fix for new manager recovery --- fdbserver/BlobManager.actor.cpp | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/fdbserver/BlobManager.actor.cpp b/fdbserver/BlobManager.actor.cpp index 567843cd72..3aeac4892c 100644 --- a/fdbserver/BlobManager.actor.cpp +++ b/fdbserver/BlobManager.actor.cpp @@ -1731,12 +1731,12 @@ ACTOR Future recoverBlobManager(Reference bmData) { fmt::print(" [{0} - {1}){2}\n", range.begin().printable(), range.end().printable(), - workerId == UID() ? " (*)" : ""); + workerId == UID() || epoch == 0 ? " (*)" : ""); } - // if worker id is already set to a known worker, range is already assigned there. If not, need to explicitly - // assign it - if (workerId == UID()) { + // if worker id is already set to a known worker that replied with it in the mapping, range is already assigned + // there. If not, need to explicitly assign it to someone + if (workerId == UID() || epoch == 0) { RangeAssignment raAssign; raAssign.isAssign = true; raAssign.worker = workerId; From d46ef148bb9aba673e2418c8dd8aea4ebd79b03d Mon Sep 17 00:00:00 2001 From: Josh Slocum Date: Mon, 31 Jan 2022 19:46:20 -0600 Subject: [PATCH 191/413] Fixing granule revoke on recover --- fdbserver/BlobManager.actor.cpp | 66 +++++++++++++++++++++++---------- 1 file changed, 46 insertions(+), 20 deletions(-) diff --git a/fdbserver/BlobManager.actor.cpp b/fdbserver/BlobManager.actor.cpp index 3aeac4892c..9988435744 100644 --- a/fdbserver/BlobManager.actor.cpp +++ b/fdbserver/BlobManager.actor.cpp @@ -531,26 +531,32 @@ ACTOR Future rangeAssigner(Reference bmData) { bmData->assignsInProgress.insert(assignment.keyRange, doRangeAssignment(bmData, assignment, workerId, seqNo)); } else { - // Revoking a range could be a large range that contains multiple ranges. - auto currentAssignments = bmData->workerAssignments.intersectingRanges(assignment.keyRange); - for (auto& it : currentAssignments) { - // ensure range doesn't truncate existing ranges - ASSERT(it.begin() >= assignment.keyRange.begin); - ASSERT(it.end() <= assignment.keyRange.end); - - // It is fine for multiple disjoint sub-ranges to have the same sequence number since they were part of - // the same logical change - - if (bmData->workerStats.count(it.value())) { - bmData->workerStats[it.value()].numGranulesAssigned -= 1; + if (assignment.worker.present()) { + // revoke this specific range from this specific worker. Either part of recovery or failing a worker + if (bmData->workerStats.count(assignment.worker.get())) { + bmData->workerStats[assignment.worker.get()].numGranulesAssigned -= 1; } + bmData->addActor.send(doRangeAssignment(bmData, assignment, assignment.worker.get(), seqNo)); + } else { + auto currentAssignments = bmData->workerAssignments.intersectingRanges(assignment.keyRange); + for (auto& it : currentAssignments) { + // ensure range doesn't truncate existing ranges + ASSERT(it.begin() >= assignment.keyRange.begin); + ASSERT(it.end() <= assignment.keyRange.end); - bmData->assignsInProgress.cancel(assignment.keyRange); + // It is fine for multiple disjoint sub-ranges to have the same sequence number since they were part + // of the same logical change - // revoke the range for the worker that owns it, not the worker specified in the revoke - bmData->addActor.send(doRangeAssignment(bmData, assignment, it.value(), seqNo)); + if (bmData->workerStats.count(it.value())) { + bmData->workerStats[it.value()].numGranulesAssigned -= 1; + } + + // revoke the range for the worker that owns it, not the worker specified in the revoke + bmData->addActor.send(doRangeAssignment(bmData, assignment, it.value(), seqNo)); + } } + bmData->assignsInProgress.cancel(assignment.keyRange); bmData->workerAssignments.insert(assignment.keyRange, UID()); } } @@ -1573,13 +1579,25 @@ ACTOR Future recoverBlobManager(Reference bmData) { state std::vector> outOfDateAssignments; state int successful = 0; state int assignIdx = 0; + // FIXME: more CPU efficient to do sorted merge of assignments? for (; assignIdx < aliveAssignments.size(); assignIdx++) { Optional reply = wait(aliveAssignments[assignIdx]); UID workerId = startingWorkers[assignIdx].id(); + if (reply.present()) { + if (BM_DEBUG) { + fmt::print(" Worker {}: ({})\n", workerId.toString().substr(0, 5), reply.get().assignments.size()); + } successful++; for (auto& assignment : reply.get().assignments) { + if (BM_DEBUG) { + fmt::print(" [{0} - {1}): ({2}, {3})\n", + assignment.range.begin.printable(), + assignment.range.end.printable(), + assignment.epochAssigned, + assignment.seqnoAssigned); + } bmData->knownBlobRanges.insert(assignment.range, true); addAssignment(workerAssignments, assignment.range, @@ -1588,9 +1606,14 @@ ACTOR Future recoverBlobManager(Reference bmData) { assignment.seqnoAssigned, &outOfDateAssignments); } - wait(yield()); + if (bmData->workerStats.count(workerId)) { + bmData->workerStats[workerId].numGranulesAssigned = reply.get().assignments.size(); + } } else { // TODO mark as failed and kill it + if (BM_DEBUG) { + fmt::print(" Worker {}: failed\n", workerId.toString().substr(0, 5)); + } } } @@ -1635,7 +1658,7 @@ ACTOR Future recoverBlobManager(Reference bmData) { fmt::print(" [{0} - {1})={2}\n", granuleStartKey.printable(), granuleEndKey.printable(), - results[rangeIdx].value.printable()); + existingOwner.toString().substr(0, 5)); } } else { if (BM_DEBUG) { @@ -1697,7 +1720,7 @@ ACTOR Future recoverBlobManager(Reference bmData) { TEST(!outOfDateAssignments.empty()); // BM resolved conflicting assignments on recovery for (auto& it : outOfDateAssignments) { if (BM_DEBUG) { - fmt::print("BM {0} revoking out of date assignment [%s - %s): %s:\n", + fmt::print("BM {0} revoking out of date assignment [{1} - {2}): {3}:\n", bmData->epoch, it.second.begin.printable().c_str(), it.second.end.printable().c_str(), @@ -1707,6 +1730,7 @@ ACTOR Future recoverBlobManager(Reference bmData) { raRevoke.isAssign = false; raRevoke.worker = it.first; raRevoke.keyRange = it.second; + raRevoke.revoke = RangeRevokeData(false); bmData->rangesToAssign.send(raRevoke); } @@ -1728,10 +1752,12 @@ ACTOR Future recoverBlobManager(Reference bmData) { bmData->workerAssignments.insert(range.range(), workerId); if (BM_DEBUG) { - fmt::print(" [{0} - {1}){2}\n", + fmt::print(" [{0} - {1}) @ ({2}, {3}): {4}\n", range.begin().printable(), range.end().printable(), - workerId == UID() || epoch == 0 ? " (*)" : ""); + epoch, + seqno, + workerId == UID() || epoch == 0 ? " (?)" : workerId.toString().substr(0, 5).c_str()); } // if worker id is already set to a known worker that replied with it in the mapping, range is already assigned From 45f4235f2bf8a317bb4f3d76dbb00d66e2f3f6e8 Mon Sep 17 00:00:00 2001 From: Josh Slocum Date: Mon, 31 Jan 2022 20:59:58 -0600 Subject: [PATCH 192/413] Fixing worker dying in BM recovery race --- fdbserver/BlobManager.actor.cpp | 8 +++++++- 1 file changed, 7 insertions(+), 1 deletion(-) diff --git a/fdbserver/BlobManager.actor.cpp b/fdbserver/BlobManager.actor.cpp index 9988435744..645ff246a1 100644 --- a/fdbserver/BlobManager.actor.cpp +++ b/fdbserver/BlobManager.actor.cpp @@ -1716,6 +1716,12 @@ ACTOR Future recoverBlobManager(Reference bmData) { } } + // Get set of workers again. Some could have died after reporting assignments + std::unordered_set endingWorkers; + for (auto& it : bmData->workersById) { + endingWorkers.insert(it.first); + } + // revoke assignments that are old and incorrect TEST(!outOfDateAssignments.empty()); // BM resolved conflicting assignments on recovery for (auto& it : outOfDateAssignments) { @@ -1762,7 +1768,7 @@ ACTOR Future recoverBlobManager(Reference bmData) { // if worker id is already set to a known worker that replied with it in the mapping, range is already assigned // there. If not, need to explicitly assign it to someone - if (workerId == UID() || epoch == 0) { + if (workerId == UID() || epoch == 0 || !endingWorkers.count(workerId)) { RangeAssignment raAssign; raAssign.isAssign = true; raAssign.worker = workerId; From 2c22d07b39a5eb2890a076a38cebb4b793bea87a Mon Sep 17 00:00:00 2001 From: Josh Slocum Date: Tue, 1 Feb 2022 10:27:41 -0600 Subject: [PATCH 193/413] Several change feed popping fixes for behind SS --- fdbserver/storageserver.actor.cpp | 37 ++++++++++++++++++++++++++++--- 1 file changed, 34 insertions(+), 3 deletions(-) diff --git a/fdbserver/storageserver.actor.cpp b/fdbserver/storageserver.actor.cpp index 63095412e3..3857021904 100644 --- a/fdbserver/storageserver.actor.cpp +++ b/fdbserver/storageserver.actor.cpp @@ -1831,6 +1831,8 @@ ACTOR Future> getChangeFeedMutations(Stor // wait for next commit wait(data->durableVersion.whenAtLeast(data->durableVersion.get() + 1)); // TODO it may be safer to always just wait for durableVersion whenAtLeast feedVersion? + // To return control back to updateStorage + wait(delay(0)); } } RangeResult res = wait( @@ -3739,7 +3741,7 @@ void applyMutation(StorageServer* self, void applyChangeFeedMutation(StorageServer* self, MutationRef const& m, Version version) { if (m.type == MutationRef::SetValue) { for (auto& it : self->keyChangeFeed[m.param1]) { - if (!it->stopped) { + if (!it->stopped && version > it->emptyVersion) { if (it->mutations.empty() || it->mutations.back().version != version) { it->mutations.push_back(MutationsAndVersionRef(version, self->knownCommittedVersion)); } @@ -3749,13 +3751,21 @@ void applyChangeFeedMutation(StorageServer* self, MutationRef const& m, Version DEBUG_MUTATION("ChangeFeedWriteSet", version, m, self->thisServerID) .detail("Range", it->range) .detail("ChangeFeedID", it->id); + } else { + TEST(it->stopped); // Skip CF write because stopped + TEST(version <= it->emptyVersion); // Skip CF write because popped and SS behind + DEBUG_MUTATION("ChangeFeedIgnoreSet", version, m, self->thisServerID) + .detail("Range", it->range) + .detail("ChangeFeedID", it->id) + .detail("Stopped", it->stopped) + .detail("EmptyVersion", it->emptyVersion); } } } else if (m.type == MutationRef::ClearRange) { auto ranges = self->keyChangeFeed.intersectingRanges(KeyRangeRef(m.param1, m.param2)); for (auto& r : ranges) { for (auto& it : r.value()) { - if (!it->stopped) { + if (!it->stopped && version > it->emptyVersion) { if (it->mutations.empty() || it->mutations.back().version != version) { it->mutations.push_back(MutationsAndVersionRef(version, self->knownCommittedVersion)); } @@ -3764,6 +3774,14 @@ void applyChangeFeedMutation(StorageServer* self, MutationRef const& m, Version DEBUG_MUTATION("ChangeFeedWriteClear", version, m, self->thisServerID) .detail("Range", it->range) .detail("ChangeFeedID", it->id); + } else { + TEST(it->stopped); // Skip CF clear because stopped + TEST(version <= it->emptyVersion); // Skip CF clear because popped and SS behind + DEBUG_MUTATION("ChangeFeedIgnoreClear", version, m, self->thisServerID) + .detail("Range", it->range) + .detail("ChangeFeedID", it->id) + .detail("Stopped", it->stopped) + .detail("EmptyVersion", it->emptyVersion); } } } @@ -4056,6 +4074,7 @@ ACTOR Future changeFeedPopQ(StorageServer* self, ChangeFeedPopRequest req) return Void(); } +// TODO handle being popped in the middle by removing after fetch ACTOR Future fetchChangeFeedApplier(StorageServer* data, Reference changeFeedInfo, Key rangeId, @@ -4063,11 +4082,23 @@ ACTOR Future fetchChangeFeedApplier(StorageServer* data, Version emptyVersion, Version fetchVersion, bool existing) { - state Reference feedResults = makeReference(); + Version startVersion = emptyVersion + 1; if (startVersion < 0) { startVersion = 0; } + + if (startVersion >= fetchVersion) { + TEST(true); // Change Feed popped before fetch + TraceEvent(SevDebug, "FetchChangeFeedNoOp", data->thisServerID) + .detail("RangeID", rangeId.printable()) + .detail("Range", range.toString()) + .detail("StartVersion", startVersion) + .detail("FetchVersion", fetchVersion); + return Void(); + } + + state Reference feedResults = makeReference(); // TODO somwhere this is initialized to -2 instead of -1 but it's fine state Future feed = data->cx->getChangeFeedStream(feedResults, rangeId, startVersion, fetchVersion, range, true); From 25f291cf6d1839649c095fd2413ed81c544ab1ff Mon Sep 17 00:00:00 2001 From: Josh Slocum Date: Tue, 1 Feb 2022 14:38:54 -0600 Subject: [PATCH 194/413] Improving blob manager recovery of client blob ranges --- fdbserver/BlobManager.actor.cpp | 24 ++++++++++++++---------- 1 file changed, 14 insertions(+), 10 deletions(-) diff --git a/fdbserver/BlobManager.actor.cpp b/fdbserver/BlobManager.actor.cpp index 645ff246a1..08305d906e 100644 --- a/fdbserver/BlobManager.actor.cpp +++ b/fdbserver/BlobManager.actor.cpp @@ -638,6 +638,7 @@ ACTOR Future writeInitialGranuleMapping(Reference bmData, // require doing a ton of storage metrics calls, which we should split up across multiple transactions likely. ACTOR Future monitorClientRanges(Reference bmData) { state Optional lastChangeKeyValue; + state bool needToCoalesce = bmData->epoch > 1; loop { state Reference tr = makeReference(bmData->db); @@ -663,6 +664,19 @@ ACTOR Future monitorClientRanges(Reference bmData) { VectorRef rangesToRemove; updateClientBlobRanges(&bmData->knownBlobRanges, results, ar, &rangesToAdd, &rangesToRemove); + if (needToCoalesce) { + // recovery has granules instead of known ranges in here. We need to do so to identify any parts of + // known client ranges the last manager didn't finish blob-ifying. + // To coalesce the map, we simply override known ranges with the current DB ranges after computing + // rangesToAdd + rangesToRemove + needToCoalesce = false; + + for (int i = 0; i < results.size() - 1; i++) { + bool active = results[i].value == LiteralStringRef("1"); + bmData->knownBlobRanges.insert(KeyRangeRef(results[i].key, results[i + 1].key), active); + } + } + for (KeyRangeRef range : rangesToRemove) { if (BM_DEBUG) { fmt::print( @@ -1778,16 +1792,6 @@ ACTOR Future recoverBlobManager(Reference bmData) { } } - // coalesce known blob ranges within boundaries at the very end - RangeResult results = - wait(krmGetRanges(tr, blobRangeKeys.begin, KeyRange(normalKeys), 10000, GetRangeLimits::BYTE_LIMIT_UNLIMITED)); - ASSERT(!results.more && results.size() < CLIENT_KNOBS->TOO_MANY); - for (int i = 0; i < results.size() - 1; i++) { - if (results[i].value.size()) { - bmData->knownBlobRanges.coalesce(KeyRangeRef(results[i].key, results[i + 1].key)); - } - } - return Void(); } From d0e89ecdd5ac09578604adfadb7f2a397fc88772 Mon Sep 17 00:00:00 2001 From: Josh Slocum Date: Tue, 1 Feb 2022 14:27:30 -0600 Subject: [PATCH 195/413] Including Change Feeds in fetch keys budget --- fdbserver/storageserver.actor.cpp | 15 +++++++++++++++ 1 file changed, 15 insertions(+) diff --git a/fdbserver/storageserver.actor.cpp b/fdbserver/storageserver.actor.cpp index 3857021904..f6954b16ba 100644 --- a/fdbserver/storageserver.actor.cpp +++ b/fdbserver/storageserver.actor.cpp @@ -4111,6 +4111,9 @@ ACTOR Future fetchChangeFeedApplier(StorageServer* data, if (!existing) { try { loop { + while (data->fetchKeysBudgetUsed.get()) { + wait(data->fetchKeysBudgetUsed.onChange()); + } Standalone> res = waitNext(feedResults->mutations.getFuture()); for (auto& it : res) { if (it.mutations.size()) { @@ -4134,6 +4137,10 @@ ACTOR Future fetchChangeFeedApplier(StorageServer* data, versionsFetched++; } } + data->fetchKeysBytesBudget -= res.expectedSize(); + if (data->fetchKeysBytesBudget <= 0) { + data->fetchKeysBudgetUsed.set(true); + } wait(yield()); } } catch (Error& e) { @@ -4165,6 +4172,10 @@ ACTOR Future fetchChangeFeedApplier(StorageServer* data, localResult = _localResult; try { loop { + while (data->fetchKeysBudgetUsed.get()) { + wait(data->fetchKeysBudgetUsed.onChange()); + } + state Standalone> remoteResult = waitNext(feedResults->mutations.getFuture()); state int remoteLoc = 0; @@ -4228,6 +4239,10 @@ ACTOR Future fetchChangeFeedApplier(StorageServer* data, localResult = _localResult; } } + data->fetchKeysBytesBudget -= remoteResult.expectedSize(); + if (data->fetchKeysBytesBudget <= 0) { + data->fetchKeysBudgetUsed.set(true); + } wait(yield()); } } catch (Error& e) { From a42c80faa96786cd474e26d89499bcba0952e088 Mon Sep 17 00:00:00 2001 From: Josh Slocum Date: Tue, 1 Feb 2022 14:52:28 -0600 Subject: [PATCH 196/413] Tightening up memory management in the blob worker --- fdbserver/BlobWorker.actor.cpp | 64 ++++++++++++++++++---------------- 1 file changed, 34 insertions(+), 30 deletions(-) diff --git a/fdbserver/BlobWorker.actor.cpp b/fdbserver/BlobWorker.actor.cpp index 1076d9f22f..3e4768568d 100644 --- a/fdbserver/BlobWorker.actor.cpp +++ b/fdbserver/BlobWorker.actor.cpp @@ -64,9 +64,7 @@ struct GranuleMetadata : NonCopyable, ReferenceCounted { KeyRange keyRange; GranuleFiles files; - GranuleDeltas currentDeltas; // only contain deltas in pendingDeltaVersion + 1, bufferedDeltaVersion - // TODO get rid of this and do Reference>? - Arena deltaArena; + Standalone currentDeltas; // only contain deltas in pendingDeltaVersion + 1, bufferedDeltaVersion uint64_t bytesInNewDeltaFiles = 0; uint64_t bufferedDeltaBytes = 0; @@ -431,8 +429,7 @@ ACTOR Future writeDeltaFile(Reference bwData, UID granuleID, int64_t epoch, int64_t seqno, - Arena deltaArena, - GranuleDeltas deltasToWrite, + Standalone deltasToWrite, Version currentDeltaVersion, Future previousDeltaFileFuture, Future waitCommitted, @@ -446,18 +443,23 @@ ACTOR Future writeDeltaFile(Reference bwData, std::to_string(currentDeltaVersion) + ".delta"; state Value serialized = ObjectWriter::toValue(deltasToWrite, Unversioned()); + state size_t serializedSize = serialized.size(); - // FIXME: technically we can free up deltaArena here to reduce memory + // Free up deltasToWrite here to reduce memory + deltasToWrite = Standalone(); state Reference objectFile = wait(bwData->bstore->writeFile(fname)); ++bwData->stats.s3PutReqs; ++bwData->stats.deltaFilesWritten; - bwData->stats.deltaBytesWritten += serialized.size(); + bwData->stats.deltaBytesWritten += serializedSize; - wait(objectFile->append(serialized.begin(), serialized.size())); + wait(objectFile->append(serialized.begin(), serializedSize)); wait(objectFile->finish()); + // free serialized since it is persisted in blob + serialized = Value(); + state int numIterations = 0; try { // before updating FDB, wait for the delta file version to be committed and previous delta files to finish @@ -474,7 +476,7 @@ ACTOR Future writeDeltaFile(Reference bwData, wait(readAndCheckGranuleLock(tr, keyRange, epoch, seqno)); Key dfKey = blobGranuleFileKeyFor(granuleID, 'D', currentDeltaVersion); - Value dfValue = blobGranuleFileValueFor(fname, 0, serialized.size()); + Value dfValue = blobGranuleFileValueFor(fname, 0, serializedSize); tr->set(dfKey, dfValue); if (oldGranuleComplete.present()) { @@ -493,7 +495,7 @@ ACTOR Future writeDeltaFile(Reference bwData, keyRange.begin.printable(), keyRange.end.printable(), fname, - serialized.size(), + serializedSize, currentDeltaVersion, tr->getCommittedVersion()); } @@ -501,7 +503,7 @@ ACTOR Future writeDeltaFile(Reference bwData, if (BUGGIFY_WITH_PROB(0.01)) { wait(delay(deterministicRandom()->random01())); } - return BlobFileIndex(currentDeltaVersion, fname, 0, serialized.size()); + return BlobFileIndex(currentDeltaVersion, fname, 0, serializedSize); } catch (Error& e) { numIterations++; wait(tr->onError(e)); @@ -545,16 +547,15 @@ ACTOR Future writeSnapshot(Reference bwData, state std::string fname = deterministicRandom()->randomUniqueID().shortString() + "_" + granuleID.toString() + "_T" + std::to_string((uint64_t)(1000.0 * now())) + "_V" + std::to_string(version) + ".snapshot"; - state Arena arena; - state GranuleSnapshot snapshot; + state Standalone snapshot; wait(delay(0, TaskPriority::BlobWorkerUpdateStorage)); loop { try { RangeResult res = waitNext(rows.getFuture()); - arena.dependsOn(res.arena()); - snapshot.append(arena, res.begin(), res.size()); + snapshot.arena().dependsOn(res.arena()); + snapshot.append(snapshot.arena(), res.begin(), res.size()); wait(yield(TaskPriority::BlobWorkerUpdateStorage)); } catch (Error& e) { if (e.code() == error_code_end_of_stream) { @@ -587,20 +588,26 @@ ACTOR Future writeSnapshot(Reference bwData, ASSERT(snapshot[i].key < snapshot[i + 1].key); } - // TODO is this easy to read as a flatbuffer from reader? Need to be sure about this data format state Value serialized = ObjectWriter::toValue(snapshot, Unversioned()); + state size_t serializedSize = serialized.size(); - // write to s3 using multi part upload + // free snapshot to reduce memory + snapshot = Standalone(); + + // write to blob using multi part upload state Reference objectFile = wait(bwData->bstore->writeFile(fname)); ++bwData->stats.s3PutReqs; ++bwData->stats.snapshotFilesWritten; - bwData->stats.snapshotBytesWritten += serialized.size(); + bwData->stats.snapshotBytesWritten += serializedSize; // TODO: inject write error - wait(objectFile->append(serialized.begin(), serialized.size())); + wait(objectFile->append(serialized.begin(), serializedSize)); wait(objectFile->finish()); + // free serialized since it is persisted in blob + serialized = Value(); + wait(delay(0, TaskPriority::BlobWorkerUpdateFDB)); // object uploaded successfully, save it to system key space @@ -613,7 +620,7 @@ ACTOR Future writeSnapshot(Reference bwData, try { wait(readAndCheckGranuleLock(tr, keyRange, epoch, seqno)); Key snapshotFileKey = blobGranuleFileKeyFor(granuleID, 'S', version); - Key snapshotFileValue = blobGranuleFileValueFor(fname, 0, serialized.size()); + Key snapshotFileValue = blobGranuleFileValueFor(fname, 0, serializedSize); tr->set(snapshotFileKey, snapshotFileValue); // create granule history at version if this is a new granule with the initial dump from FDB if (createGranuleHistory) { @@ -658,14 +665,14 @@ ACTOR Future writeSnapshot(Reference bwData, keyRange.begin.printable(), keyRange.end.printable(), fname, - serialized.size()); + serializedSize); } if (BUGGIFY_WITH_PROB(0.1)) { wait(delay(deterministicRandom()->random01())); } - return BlobFileIndex(version, fname, 0, serialized.size()); + return BlobFileIndex(version, fname, 0, serializedSize); } ACTOR Future dumpInitialSnapshotFromFDB(Reference bwData, @@ -1041,8 +1048,7 @@ static Version doGranuleRollback(Reference metadata, } // discard all in-memory mutations - metadata->deltaArena = Arena(); - metadata->currentDeltas = GranuleDeltas(); + metadata->currentDeltas = Standalone(); metadata->bufferedDeltaBytes = 0; metadata->bufferedDeltaVersion = cfRollbackVersion; @@ -1084,7 +1090,7 @@ static Version doGranuleRollback(Reference metadata, metadata->bufferedDeltaBytes); } - metadata->currentDeltas.resize(metadata->deltaArena, mIdx); + metadata->currentDeltas.resize(metadata->currentDeltas.arena(), mIdx); // delete all deltas in rollback range, but we can optimize here to just skip the uncommitted mutations // directly and immediately pop the rollback out of inProgress to completed @@ -1588,7 +1594,7 @@ ACTOR Future blobGranuleUpdateFiles(Reference bwData, if (DEBUG_BW_VERSION(deltas.version)) { fmt::print("BWB {0}: ({1})\n", deltas.version, deltas.mutations.size()); } - metadata->currentDeltas.push_back_deep(metadata->deltaArena, deltas); + metadata->currentDeltas.push_back_deep(metadata->currentDeltas.arena(), deltas); processedAnyMutations = true; ASSERT(deltas.version != invalidVersion); @@ -1641,7 +1647,6 @@ ACTOR Future blobGranuleUpdateFiles(Reference bwData, startState.granuleID, metadata->originalEpoch, metadata->originalSeqno, - metadata->deltaArena, metadata->currentDeltas, lastDeltaVersion, previousFuture, @@ -1659,8 +1664,7 @@ ACTOR Future blobGranuleUpdateFiles(Reference bwData, bwData->stats.mutationBytesBuffered -= metadata->bufferedDeltaBytes; // reset current deltas - metadata->deltaArena = Arena(); - metadata->currentDeltas = GranuleDeltas(); + metadata->currentDeltas = Standalone(); metadata->bufferedDeltaBytes = 0; // if we just wrote a delta file, check if we need to compact here. @@ -2353,7 +2357,7 @@ ACTOR Future handleBlobGranuleFileRequest(Reference bwData metadata->pendingDeltaVersion); } ASSERT(metadata->durableDeltaVersion.get() == metadata->pendingDeltaVersion); - rep.arena.dependsOn(metadata->deltaArena); + rep.arena.dependsOn(metadata->currentDeltas.arena()); for (auto& delta : metadata->currentDeltas) { if (delta.version > req.readVersion) { break; From 3baad66b4c8b8147c043d70bc56caa0e9f0530c4 Mon Sep 17 00:00:00 2001 From: Josh Slocum Date: Tue, 1 Feb 2022 16:16:46 -0600 Subject: [PATCH 197/413] Better bound on knowing if a delta file can be committed --- fdbserver/BlobWorker.actor.cpp | 22 +++++++++++++++------- 1 file changed, 15 insertions(+), 7 deletions(-) diff --git a/fdbserver/BlobWorker.actor.cpp b/fdbserver/BlobWorker.actor.cpp index 3e4768568d..fd182e53b5 100644 --- a/fdbserver/BlobWorker.actor.cpp +++ b/fdbserver/BlobWorker.actor.cpp @@ -1127,11 +1127,18 @@ ACTOR Future waitVersionCommitted(Reference bwData, } wait(grvAtLeast); } - state Version grvVersion = bwData->grvVersion.get(); - if ((DEBUG_BW_VERSION(version)) || (DEBUG_BW_VERSION(grvVersion))) { - fmt::print("waitVersionCommitted got {0} < {1}, waiting on CF (currently {2})\n", + Version grvVersion = bwData->grvVersion.get(); + // If GRV is way in the future, we know we can't roll back more than 5 seconds (or whatever this knob is set to) + // worth of versions + Version cantRollbackVersion = version + SERVER_KNOBS->MAX_READ_TRANSACTION_LIFE_VERSIONS; + state Version committedWaitVersion = std::min(grvVersion, cantRollbackVersion); + + if ((DEBUG_BW_VERSION(version)) || (DEBUG_BW_VERSION(committedWaitVersion))) { + fmt::print("waitVersionCommitted got {0} < {1} - min of ({2}, {3}) waiting on CF (currently {4})\n", version, + committedWaitVersion, grvVersion, + cantRollbackVersion, metadata->activeCFData.get()->getVersion()); } // make sure the change feed has consumed mutations up through grvVersion to ensure none of them are rollbacks @@ -1140,7 +1147,7 @@ ACTOR Future waitVersionCommitted(Reference bwData, try { // if not valid, we're about to be cancelled anyway state Future atLeast = metadata->activeCFData.get().isValid() - ? metadata->activeCFData.get()->whenAtLeast(grvVersion) + ? metadata->activeCFData.get()->whenAtLeast(committedWaitVersion) : Never(); choose { when(wait(atLeast)) { break; } @@ -1164,15 +1171,16 @@ ACTOR Future waitVersionCommitted(Reference bwData, } } // sanity check to make sure whenAtLeast didn't return early - if (grvVersion > metadata->waitForVersionReturned) { - metadata->waitForVersionReturned = grvVersion; + if (committedWaitVersion > metadata->waitForVersionReturned) { + metadata->waitForVersionReturned = committedWaitVersion; } if (version > metadata->knownCommittedVersion) { metadata->knownCommittedVersion = version; } /*if (BW_DEBUG) { fmt::print( - "waitVersionCommitted CF whenAtLeast {0}: {1}\n", grvVersion, metadata->activeCFData.get()->getVersion()); + "waitVersionCommitted CF whenAtLeast {0}: {1}\n", committedWaitVersion, + metadata->activeCFData.get()->getVersion()); }*/ return Void(); From 3a7fb164c69ce1827200438a5b1006765114cae0 Mon Sep 17 00:00:00 2001 From: Josh Slocum Date: Wed, 2 Feb 2022 08:49:56 -0600 Subject: [PATCH 198/413] Small pipelining improvements for behind blob worker --- fdbserver/BlobWorker.actor.cpp | 99 +++++++++++++++++----------------- 1 file changed, 51 insertions(+), 48 deletions(-) diff --git a/fdbserver/BlobWorker.actor.cpp b/fdbserver/BlobWorker.actor.cpp index fd182e53b5..bfed522808 100644 --- a/fdbserver/BlobWorker.actor.cpp +++ b/fdbserver/BlobWorker.actor.cpp @@ -1110,45 +1110,12 @@ static Version doGranuleRollback(Reference metadata, return cfRollbackVersion; } -ACTOR Future waitVersionCommitted(Reference bwData, - Reference metadata, - Version version) { - // TODO REMOVE debugs - if (version > bwData->grvVersion.get()) { - /*if (BW_DEBUG) { - fmt::print("waitVersionCommitted waiting {0}\n", version); - }*/ - // this order is important, since we need to register a waiter on the notified version before waking the GRV - // actor - Future grvAtLeast = bwData->grvVersion.whenAtLeast(version); - Promise doGrvCheck = bwData->doGRVCheck; - if (doGrvCheck.canBeSet()) { - doGrvCheck.send(Void()); - } - wait(grvAtLeast); - } - Version grvVersion = bwData->grvVersion.get(); - // If GRV is way in the future, we know we can't roll back more than 5 seconds (or whatever this knob is set to) - // worth of versions - Version cantRollbackVersion = version + SERVER_KNOBS->MAX_READ_TRANSACTION_LIFE_VERSIONS; - state Version committedWaitVersion = std::min(grvVersion, cantRollbackVersion); - - if ((DEBUG_BW_VERSION(version)) || (DEBUG_BW_VERSION(committedWaitVersion))) { - fmt::print("waitVersionCommitted got {0} < {1} - min of ({2}, {3}) waiting on CF (currently {4})\n", - version, - committedWaitVersion, - grvVersion, - cantRollbackVersion, - metadata->activeCFData.get()->getVersion()); - } - // make sure the change feed has consumed mutations up through grvVersion to ensure none of them are rollbacks - +ACTOR Future waitOnCFVersion(Reference metadata, Version cfVersion) { loop { try { // if not valid, we're about to be cancelled anyway - state Future atLeast = metadata->activeCFData.get().isValid() - ? metadata->activeCFData.get()->whenAtLeast(committedWaitVersion) - : Never(); + state Future atLeast = + metadata->activeCFData.get().isValid() ? metadata->activeCFData.get()->whenAtLeast(cfVersion) : Never(); choose { when(wait(atLeast)) { break; } when(wait(metadata->activeCFData.onChange())) {} @@ -1170,19 +1137,51 @@ ACTOR Future waitVersionCommitted(Reference bwData, wait(delay(0.05)); } } + // sanity check to make sure whenAtLeast didn't return early - if (committedWaitVersion > metadata->waitForVersionReturned) { - metadata->waitForVersionReturned = committedWaitVersion; + if (cfVersion > metadata->waitForVersionReturned) { + metadata->waitForVersionReturned = cfVersion; } + + // stop after change feed callback + wait(delay(0, TaskPriority::BlobWorkerReadChangeFeed)); + + return Void(); +} + +ACTOR Future waitCommittedGrv(Reference bwData, + Reference metadata, + Version version) { + // TODO REMOVE debugs + if (version > bwData->grvVersion.get()) { + /*if (BW_DEBUG) { + fmt::print("waitVersionCommitted waiting {0}\n", version); + }*/ + // this order is important, since we need to register a waiter on the notified version before waking the GRV + // actor + Future grvAtLeast = bwData->grvVersion.whenAtLeast(version); + Promise doGrvCheck = bwData->doGRVCheck; + if (doGrvCheck.canBeSet()) { + doGrvCheck.send(Void()); + } + wait(grvAtLeast); + } + + Version grvVersion = bwData->grvVersion.get(); + wait(waitOnCFVersion(metadata, grvVersion)); + return Void(); +} + +ACTOR Future waitVersionCommitted(Reference bwData, + Reference metadata, + Version version) { + // If GRV is way in the future, we know we can't roll back more than 5 seconds (or whatever this knob is set to) + // worth of versions + wait(waitCommittedGrv(bwData, metadata, version) || + waitOnCFVersion(metadata, version + SERVER_KNOBS->MAX_READ_TRANSACTION_LIFE_VERSIONS)); if (version > metadata->knownCommittedVersion) { metadata->knownCommittedVersion = version; } - /*if (BW_DEBUG) { - fmt::print( - "waitVersionCommitted CF whenAtLeast {0}: {1}\n", committedWaitVersion, - metadata->activeCFData.get()->getVersion()); - }*/ - return Void(); } @@ -1698,7 +1697,7 @@ ACTOR Future blobGranuleUpdateFiles(Reference bwData, } // Speculatively assume we will get the range back. This is both a performance optimization, and - // necessary to keep consuming committed versions from the change feed so that we can realize + // necessary to keep consuming versions from the change feed so that we can realize // our last delta file is committed and write it Future previousFuture; @@ -1724,14 +1723,18 @@ ACTOR Future blobGranuleUpdateFiles(Reference bwData, if (pendingSnapshots > 1) { state int waitIdx = 0; int idx = 0; + Version safeVersion = + std::max(metadata->knownCommittedVersion, + metadata->bufferedDeltaVersion - SERVER_KNOBS->MAX_READ_TRANSACTION_LIFE_VERSIONS); for (auto& f : inFlightFiles) { if (f.snapshot && f.version < metadata->pendingSnapshotVersion && - f.version <= metadata->knownCommittedVersion) { + f.version <= safeVersion) { if (BW_DEBUG) { - fmt::print("[{0} - {1}) Waiting on previous snapshot file @ {2}\n", + fmt::print("[{0} - {1}) Waiting on previous snapshot file @ {2} <= {3}\n", metadata->keyRange.begin.printable(), metadata->keyRange.end.printable(), - f.version); + f.version, + safeVersion); } waitIdx = idx + 1; } From d58b0325b3de679ce4f5cf52ac6be900c8a4fe27 Mon Sep 17 00:00:00 2001 From: Josh Slocum Date: Wed, 2 Feb 2022 09:45:18 -0600 Subject: [PATCH 199/413] Added write-hot granule splitting and splitting based on write rate --- fdbclient/BlobWorkerInterface.h | 7 ++++-- fdbserver/BlobManager.actor.cpp | 44 +++++++++++++++++++++++---------- fdbserver/BlobWorker.actor.cpp | 16 +++++++++--- 3 files changed, 49 insertions(+), 18 deletions(-) diff --git a/fdbclient/BlobWorkerInterface.h b/fdbclient/BlobWorkerInterface.h index 3381b54da4..8be48404de 100644 --- a/fdbclient/BlobWorkerInterface.h +++ b/fdbclient/BlobWorkerInterface.h @@ -149,6 +149,7 @@ struct GranuleStatusReply : public ReplyPromiseStreamReply { KeyRange granuleRange; bool doSplit; + bool writeHotSplit; int64_t epoch; int64_t seqno; UID granuleID; @@ -158,13 +159,14 @@ struct GranuleStatusReply : public ReplyPromiseStreamReply { GranuleStatusReply() {} explicit GranuleStatusReply(KeyRange range, bool doSplit, + bool writeHotSplit, int64_t epoch, int64_t seqno, UID granuleID, Version startVersion, Version latestVersion) - : granuleRange(range), doSplit(doSplit), epoch(epoch), seqno(seqno), granuleID(granuleID), - startVersion(startVersion), latestVersion(latestVersion) {} + : granuleRange(range), doSplit(doSplit), writeHotSplit(writeHotSplit), epoch(epoch), seqno(seqno), + granuleID(granuleID), startVersion(startVersion), latestVersion(latestVersion) {} int expectedSize() const { return sizeof(GranuleStatusReply) + granuleRange.expectedSize(); } @@ -175,6 +177,7 @@ struct GranuleStatusReply : public ReplyPromiseStreamReply { ReplyPromiseStreamReply::sequence, granuleRange, doSplit, + writeHotSplit, epoch, seqno, granuleID, diff --git a/fdbserver/BlobManager.actor.cpp b/fdbserver/BlobManager.actor.cpp index 08305d906e..baaea2dfba 100644 --- a/fdbserver/BlobManager.actor.cpp +++ b/fdbserver/BlobManager.actor.cpp @@ -245,13 +245,18 @@ struct BlobManagerData : NonCopyable, ReferenceCounted { } }; -ACTOR Future>> splitRange(Reference tr, KeyRange range) { +ACTOR Future>> splitRange(Reference tr, + KeyRange range, + bool writeHot) { // TODO is it better to just pass empty metrics to estimated? // redo split if previous txn failed to calculate it loop { try { if (BM_DEBUG) { - fmt::print("Splitting new range [{0} - {1})\n", range.begin.printable(), range.end.printable()); + fmt::print("Splitting new range [{0} - {1}): {2}\n", + range.begin.printable(), + range.end.printable(), + writeHot ? "hot" : "normal"); } state StorageMetrics estimated = wait(tr->getTransaction().getStorageMetrics(range, CLIENT_KNOBS->TOO_MANY)); @@ -263,15 +268,21 @@ ACTOR Future>> splitRange(Reference SERVER_KNOBS->BG_SNAPSHOT_FILE_TARGET_BYTES) { + if (estimated.bytes > SERVER_KNOBS->BG_SNAPSHOT_FILE_TARGET_BYTES || writeHot) { // printf(" Splitting range\n"); - // only split on bytes + // only split on bytes and write rate state Standalone> keys; state StorageMetrics splitMetrics; splitMetrics.bytes = SERVER_KNOBS->BG_SNAPSHOT_FILE_TARGET_BYTES; - splitMetrics.bytesPerKSecond = splitMetrics.infinity; + splitMetrics.bytesPerKSecond = SERVER_KNOBS->SHARD_SPLIT_BYTES_PER_KSEC; + if (writeHot) { + splitMetrics.bytesPerKSecond = + std::min(splitMetrics.bytesPerKSecond, estimated.bytesPerKSecond / 2); + splitMetrics.bytesPerKSecond = + std::max(splitMetrics.bytesPerKSecond, SERVER_KNOBS->SHARD_MIN_BYTES_PER_KSEC); + } splitMetrics.iosPerKSecond = splitMetrics.infinity; - splitMetrics.bytesReadPerKSecond = splitMetrics.infinity; // Don't split by readBandwidth + splitMetrics.bytesReadPerKSecond = splitMetrics.infinity; while (keys.empty() || keys.back() < range.end) { // allow partial in case we have a large split @@ -693,7 +704,7 @@ ACTOR Future monitorClientRanges(Reference bmData) { state std::vector>>> splitFutures; // Divide new ranges up into equal chunks by using SS byte sample for (KeyRangeRef range : rangesToAdd) { - splitFutures.push_back(splitRange(tr, range)); + splitFutures.push_back(splitRange(tr, range, false)); } for (auto f : splitFutures) { @@ -790,13 +801,14 @@ ACTOR Future maybeSplitRange(Reference bmData, KeyRange granuleRange, UID granuleID, Version granuleStartVersion, - Version latestVersion) { + Version latestVersion, + bool writeHot) { state Reference tr = makeReference(bmData->db); state Standalone> newRanges; state int64_t newLockSeqno = -1; // first get ranges to split - Standalone> _newRanges = wait(splitRange(tr, granuleRange)); + Standalone> _newRanges = wait(splitRange(tr, granuleRange, writeHot)); newRanges = _newRanges; ASSERT(newRanges.size() >= 2); @@ -1151,14 +1163,15 @@ ACTOR Future monitorBlobWorkerStatus(Reference bmData, Bl GranuleStatusReply rep = waitNext(statusStream.getFuture()); if (BM_DEBUG) { - fmt::print("BM {0} got status of [{1} - {2}) @ ({3}, {4}) from BW {5}: {6}\n", + fmt::print("BM {0} got status of [{1} - {2}) @ ({3}, {4}) from BW {5}: {6} {7}\n", bmData->epoch, rep.granuleRange.begin.printable(), rep.granuleRange.end.printable(), rep.epoch, rep.seqno, bwInterf.id().toString(), - rep.doSplit ? "split" : ""); + rep.doSplit ? "split" : "", + rep.writeHotSplit ? "hot" : "normal"); } // if we get a reply from the stream, reset backoff backoff = 0.1; @@ -1208,8 +1221,13 @@ ACTOR Future monitorBlobWorkerStatus(Reference bmData, Bl rep.granuleRange.end.printable().c_str()); } lastSeenSeqno.insert(rep.granuleRange, std::pair(rep.epoch, rep.seqno)); - bmData->addActor.send(maybeSplitRange( - bmData, bwInterf.id(), rep.granuleRange, rep.granuleID, rep.startVersion, rep.latestVersion)); + bmData->addActor.send(maybeSplitRange(bmData, + bwInterf.id(), + rep.granuleRange, + rep.granuleID, + rep.startVersion, + rep.latestVersion, + rep.writeHotSplit)); } } } catch (Error& e) { diff --git a/fdbserver/BlobWorker.actor.cpp b/fdbserver/BlobWorker.actor.cpp index bfed522808..658ddbaa19 100644 --- a/fdbserver/BlobWorker.actor.cpp +++ b/fdbserver/BlobWorker.actor.cpp @@ -837,7 +837,8 @@ ACTOR Future checkSplitAndReSnapshot(Reference bw Reference metadata, UID granuleID, int64_t bytesInNewDeltaFiles, - Future lastDeltaBeforeSnapshot) { + Future lastDeltaBeforeSnapshot, + int64_t versionsSinceLastSnapshot) { BlobFileIndex lastDeltaIdx = wait(lastDeltaBeforeSnapshot); state Version reSnapshotVersion = lastDeltaIdx.version; @@ -861,12 +862,15 @@ ACTOR Future checkSplitAndReSnapshot(Reference bw metadata->resumeSnapshot.reset(); state int64_t statusEpoch = metadata->continueEpoch; state int64_t statusSeqno = metadata->continueSeqno; + // TODO its own knob or something better? This is wrong in case of rollbacks + state bool writeHot = versionsSinceLastSnapshot <= SERVER_KNOBS->MAX_READ_TRANSACTION_LIFE_VERSIONS; loop { loop { try { wait(bwData->currentManagerStatusStream.get().onReady()); bwData->currentManagerStatusStream.get().send(GranuleStatusReply(metadata->keyRange, true, + writeHot, statusEpoch, statusSeqno, granuleID, @@ -1707,8 +1711,14 @@ ACTOR Future blobGranuleUpdateFiles(Reference bwData, } else { previousFuture = Future(metadata->files.deltaFiles.back()); } - Future inFlightBlobSnapshot = checkSplitAndReSnapshot( - bwData, metadata, startState.granuleID, metadata->bytesInNewDeltaFiles, previousFuture); + int64_t versionsSinceLastSnapshot = + metadata->pendingDeltaVersion - metadata->pendingSnapshotVersion; + Future inFlightBlobSnapshot = checkSplitAndReSnapshot(bwData, + metadata, + startState.granuleID, + metadata->bytesInNewDeltaFiles, + previousFuture, + versionsSinceLastSnapshot); inFlightFiles.push_back(InFlightFile(inFlightBlobSnapshot, metadata->pendingDeltaVersion, 0, true)); pendingSnapshots++; From c90c356c2cc39b2184ef4b6752df8e41792e5365 Mon Sep 17 00:00:00 2001 From: Josh Slocum Date: Wed, 2 Feb 2022 11:28:13 -0600 Subject: [PATCH 200/413] Improving orphaned file cleanup in Blob Worker --- fdbserver/BlobWorker.actor.cpp | 55 ++++++++++++---------------------- 1 file changed, 19 insertions(+), 36 deletions(-) diff --git a/fdbserver/BlobWorker.actor.cpp b/fdbserver/BlobWorker.actor.cpp index 658ddbaa19..29a2a10666 100644 --- a/fdbserver/BlobWorker.actor.cpp +++ b/fdbserver/BlobWorker.actor.cpp @@ -474,6 +474,7 @@ ACTOR Future writeDeltaFile(Reference bwData, tr->setOption(FDBTransactionOptions::ACCESS_SYSTEM_KEYS); try { wait(readAndCheckGranuleLock(tr, keyRange, epoch, seqno)); + numIterations++; Key dfKey = blobGranuleFileKeyFor(granuleID, 'D', currentDeltaVersion); Value dfValue = blobGranuleFileValueFor(fname, 0, serializedSize); @@ -505,32 +506,22 @@ ACTOR Future writeDeltaFile(Reference bwData, } return BlobFileIndex(currentDeltaVersion, fname, 0, serializedSize); } catch (Error& e) { - numIterations++; wait(tr->onError(e)); } } } catch (Error& e) { - if (e.code() == error_code_operation_cancelled) { + // If this actor was cancelled, doesn't own the granule anymore, or got some other error before trying to + // commit a transaction, we can and want to safely delete the file we wrote. Otherwise, we may have updated FDB + // with file and cannot safely delete it. + if (numIterations > 0) { throw e; } - - // FIXME: this could also fail due to actor cancelled. Since we are speculatively writing here, we should clean - // stuff up more aggressively - - // if commit failed the first time due to granule assignment conflict (which is non-retryable), - // then the file key was persisted and we should delete it. Otherwise, the commit failed - // for some other reason and the key wasn't persisted, so we should just propogate the error - if (numIterations != 1 || e.code() != error_code_granule_assignment_conflict) { - throw e; - } - if (BW_DEBUG) { - fmt::print("deleting s3 delta file {0} after error {1}\n", fname, e.name()); + fmt::print("deleting delta file {0} after error {1}\n", fname, e.name()); } - state Error eState = e; ++bwData->stats.s3DeleteReqs; - wait(bwData->bstore->deleteFile(fname)); - throw eState; + bwData->addActor.send(bwData->bstore->deleteFile(fname)); + throw e; } } @@ -619,6 +610,7 @@ ACTOR Future writeSnapshot(Reference bwData, tr->setOption(FDBTransactionOptions::ACCESS_SYSTEM_KEYS); try { wait(readAndCheckGranuleLock(tr, keyRange, epoch, seqno)); + numIterations++; Key snapshotFileKey = blobGranuleFileKeyFor(granuleID, 'S', version); Key snapshotFileValue = blobGranuleFileValueFor(fname, 0, serializedSize); tr->set(snapshotFileKey, snapshotFileValue); @@ -632,32 +624,22 @@ ACTOR Future writeSnapshot(Reference bwData, wait(tr->commit()); break; } catch (Error& e) { - numIterations++; wait(tr->onError(e)); } } } catch (Error& e) { - if (e.code() == error_code_operation_cancelled) { + // If this actor was cancelled, doesn't own the granule anymore, or got some other error before trying to + // commit a transaction, we can and want to safely delete the file we wrote. Otherwise, we may have updated FDB + // with file and cannot safely delete it. + if (numIterations > 0) { throw e; } - - // FIXME: this could also fail due to actor cancelled. Since we are speculatively writing here, we should clean - // stuff up more aggressively - - // if commit failed the first time due to granule assignment conflict (which is non-retryable), - // then the file key was persisted and we should delete it. Otherwise, the commit failed - // for some other reason and the key wasn't persisted, so we should just propogate the error - if (numIterations != 1 || e.code() != error_code_granule_assignment_conflict) { - throw e; - } - if (BW_DEBUG) { - fmt::print("deleting s3 snapshot file {0} after error {1}\n", fname, e.name()); + fmt::print("deleting snapshot file {0} after error {1}\n", fname, e.name()); } - state Error eState = e; ++bwData->stats.s3DeleteReqs; - wait(bwData->bstore->deleteFile(fname)); - throw eState; + bwData->addActor.send(bwData->bstore->deleteFile(fname)); + throw e; } if (BW_DEBUG) { @@ -1834,8 +1816,9 @@ ACTOR Future blobGranuleUpdateFiles(Reference bwData, if (granuleCanRetry(e)) { // explicitly cancel all outstanding write futures BEFORE updating promise stream, to ensure they // can't update files after the re-assigned granule acquires the lock - for (auto& f : inFlightFiles) { - f.future.cancel(); + // do it backwards though because future depends on previous one, so it could cause a cascade + for (int i = inFlightFiles.size() - 1; i >= 0; i--) { + inFlightFiles[i].future.cancel(); } // if we retry and re-open, we need to use a normal request (no continue or reassign) and update the From f187d01b71fe6be9b5a01fb2cafd2d8e47cc2095 Mon Sep 17 00:00:00 2001 From: Josh Slocum Date: Wed, 2 Feb 2022 11:10:32 -0600 Subject: [PATCH 201/413] Improving change feed performance and fixing durability issue --- fdbserver/storageserver.actor.cpp | 203 +++++++++++++++++++----------- 1 file changed, 131 insertions(+), 72 deletions(-) diff --git a/fdbserver/storageserver.actor.cpp b/fdbserver/storageserver.actor.cpp index f6954b16ba..385dfde657 100644 --- a/fdbserver/storageserver.actor.cpp +++ b/fdbserver/storageserver.actor.cpp @@ -338,18 +338,16 @@ struct FetchInjectionInfo { struct ChangeFeedInfo : ReferenceCounted { std::deque> mutations; - Version fetchVersion = - invalidVersion; // The version that commits from a fetch have been written to storage, but have not yet been - // committed as part of updateStorage. Because a fetch can merge mutations with incoming data - // before updateStorage updates the storage version, updateStorage must know to skip mutations - // that have already been written to storage by change feed fetch. + Version fetchVersion = invalidVersion; // The version that commits from a fetch have been written to storage, but + // have not yet been committed as part of updateStorage. Version storageVersion = invalidVersion; // The version between the storage version and the durable version are - // being written to disk as part of the current commit in updateStorage + // being written to disk as part of the current commit in updateStorage. Version durableVersion = invalidVersion; // All versions before the durable version are durable on disk Version emptyVersion = 0; // The change feed does not have any mutations before emptyVersion KeyRange range; Key id; AsyncTrigger newMutations; + NotifiedVersion durableFetchVersion; Promise onMove; bool stopped = false; // A stopped change feed no longer adds new mutations, but is still queriable bool removing = false; @@ -1713,7 +1711,7 @@ MutationsAndVersionRef filterMutations(Arena& arena, #define DEBUG_SS_CF_BEGIN_VERSION invalidVersion #define DEBUG_SS_CFM(ssId, cfId, v) \ ssId.toString().substr(0, 4) == DEBUG_SS_ID&& cfId.printable().substr(0, 6) == DEBUG_SS_CF_ID && \ - (v == DEBUG_SS_CF_BEGIN_VERSION || latestVersion == DEBUG_SS_CF_BEGIN_VERSION) + (v >= DEBUG_SS_CF_BEGIN_VERSION || latestVersion == DEBUG_SS_CF_BEGIN_VERSION) ACTOR Future> getChangeFeedMutations(StorageServer* data, ChangeFeedStreamRequest req, @@ -1823,17 +1821,15 @@ ACTOR Future> getChangeFeedMutations(Stor bool readDurable = feedInfo->durableVersion != invalidVersion && req.begin <= feedInfo->durableVersion; bool readFetched = feedInfo->durableVersion < feedInfo->fetchVersion && req.begin <= feedInfo->fetchVersion; if (req.end > emptyVersion + 1 && (readDurable || readFetched)) { - if (readFetched && feedInfo->durableVersion == invalidVersion) { - // To not block fetchKeys on making change feed data written to storage, we wait in here instead for all - // fetched data to become readable from the storage engine. - while (feedInfo->durableVersion == invalidVersion) { - TEST(true); // getChangeFeedMutations before any fetched data durable - // wait for next commit - wait(data->durableVersion.whenAtLeast(data->durableVersion.get() + 1)); - // TODO it may be safer to always just wait for durableVersion whenAtLeast feedVersion? - // To return control back to updateStorage - wait(delay(0)); - } + if (readFetched) { + // To not block fetchKeys making normal SS data readable on making change feed data written to storage, we + // wait in here instead for all fetched data to become readable from the storage engine. + TEST(true); // getChangeFeedMutations before fetched data durable + // wait for next commit to write feed data to storage + wait(feedInfo->durableFetchVersion.whenAtLeast( + std::min(feedInfo->fetchVersion, feedInfo->durableFetchVersion.get() + 1))); + // To return control back to updateStorage + wait(delay(0)); } RangeResult res = wait( data->storage.readRange(KeyRangeRef(changeFeedDurableKey(req.rangeID, std::max(req.begin, emptyVersion)), @@ -4075,33 +4071,32 @@ ACTOR Future changeFeedPopQ(StorageServer* self, ChangeFeedPopRequest req) } // TODO handle being popped in the middle by removing after fetch -ACTOR Future fetchChangeFeedApplier(StorageServer* data, - Reference changeFeedInfo, - Key rangeId, - KeyRange range, - Version emptyVersion, - Version fetchVersion, - bool existing) { +// Returns max version fetched +ACTOR Future fetchChangeFeedApplier(StorageServer* data, + Reference changeFeedInfo, + Key rangeId, + KeyRange range, + Version emptyVersion, + Version beginVersion, + Version endVersion, + bool existing) { - Version startVersion = emptyVersion + 1; - if (startVersion < 0) { - startVersion = 0; - } + state Version startVersion = std::max(beginVersion, emptyVersion + 1); + ASSERT(startVersion >= 0); - if (startVersion >= fetchVersion) { + if (startVersion >= endVersion) { TEST(true); // Change Feed popped before fetch TraceEvent(SevDebug, "FetchChangeFeedNoOp", data->thisServerID) .detail("RangeID", rangeId.printable()) .detail("Range", range.toString()) .detail("StartVersion", startVersion) - .detail("FetchVersion", fetchVersion); - return Void(); + .detail("EndVersion", endVersion); + return invalidVersion; } state Reference feedResults = makeReference(); - // TODO somwhere this is initialized to -2 instead of -1 but it's fine state Future feed = - data->cx->getChangeFeedStream(feedResults, rangeId, startVersion, fetchVersion, range, true); + data->cx->getChangeFeedStream(feedResults, rangeId, startVersion, endVersion, range, true); // TODO remove debugging eventually? state Version firstVersion = invalidVersion; @@ -4151,21 +4146,22 @@ ACTOR Future fetchChangeFeedApplier(StorageServer* data, TraceEvent(SevDebug, "FetchChangeFeedDone", data->thisServerID) .detail("RangeID", rangeId.printable()) .detail("Range", range.toString()) - .detail("FetchVersion", fetchVersion) + .detail("StartVersion", startVersion) + .detail("EndVersion", endVersion) .detail("FirstFetchedVersion", firstVersion) .detail("LastFetchedVersion", lastVersion) .detail("VersionsFetched", versionsFetched) .detail("Existing", existing); - return Void(); + return lastVersion; } } state PromiseStream> localResults; - // Add 2 to fetch version to make sure the local stream will have more versions in the stream than the remote stream + // Add 1 to fetch version to make sure the local stream will have more versions in the stream than the remote stream // to avoid edge cases in the merge logic state Future localStream = - localChangeFeedStream(data, localResults, rangeId, emptyVersion + 1, fetchVersion + 2, range); + localChangeFeedStream(data, localResults, rangeId, startVersion, endVersion + 1, range); state Standalone localResult; Standalone _localResult = waitNext(localResults.getFuture()); @@ -4255,23 +4251,27 @@ ACTOR Future fetchChangeFeedApplier(StorageServer* data, TraceEvent(SevDebug, "FetchChangeFeedDone", data->thisServerID) .detail("RangeID", rangeId.printable()) .detail("Range", range.toString()) - .detail("FetchVersion", fetchVersion) + .detail("StartVersion", startVersion) + .detail("EndVersion", endVersion) .detail("FirstFetchedVersion", firstVersion) .detail("LastFetchedVersion", lastVersion) .detail("VersionsFetched", versionsFetched) .detail("Existing", existing); - return Void(); + return lastVersion; } -ACTOR Future fetchChangeFeed(StorageServer* data, - Reference changeFeedInfo, - Version transferredVersion) { +// returns largest version fetched +ACTOR Future fetchChangeFeed(StorageServer* data, + Reference changeFeedInfo, + Version beginVersion, + Version endVersion) { wait(delay(0)); // allow this actor to be cancelled by removals TraceEvent(SevDebug, "FetchChangeFeed", data->thisServerID) .detail("RangeID", changeFeedInfo->id.printable()) .detail("Range", changeFeedInfo->range.toString()) - .detail("FetchVersion", transferredVersion); + .detail("BeginVersion", beginVersion) + .detail("EndVersion", endVersion); auto cleanupPending = data->changeFeedCleanupDurable.find(changeFeedInfo->id); if (cleanupPending != data->changeFeedCleanupDurable.end()) { @@ -4280,7 +4280,8 @@ ACTOR Future fetchChangeFeed(StorageServer* data, .detail("Range", changeFeedInfo->range.toString()) .detail("CleanupVersion", cleanupPending->second) .detail("EmptyVersion", changeFeedInfo->emptyVersion) - .detail("FetchVersion", transferredVersion); + .detail("BeginVersion", beginVersion) + .detail("EndVersion", endVersion); wait(data->durableVersion.whenAtLeast(cleanupPending->second + 1)); ASSERT(!data->changeFeedCleanupDurable.count(changeFeedInfo->id)); } @@ -4288,15 +4289,16 @@ ACTOR Future fetchChangeFeed(StorageServer* data, loop { try { // TODO clean up existing param for !existing - wait(fetchChangeFeedApplier(data, - changeFeedInfo, - changeFeedInfo->id, - changeFeedInfo->range, - changeFeedInfo->emptyVersion, - transferredVersion, - false)); + Version maxFetched = wait(fetchChangeFeedApplier(data, + changeFeedInfo, + changeFeedInfo->id, + changeFeedInfo->range, + changeFeedInfo->emptyVersion, + beginVersion, + endVersion, + false)); data->fetchingChangeFeeds.insert(changeFeedInfo->id); - return Void(); + return maxFetched; } catch (Error& e) { if (e.code() != error_code_change_feed_not_registered) { throw; @@ -4376,33 +4378,38 @@ ACTOR Future> fetchChangeFeedMetadata(StorageServer* data, KeyR return feedIds; } -ACTOR Future dispatchChangeFeeds(StorageServer* data, - UID fetchKeysID, - KeyRange keys, - Version transferredVersion, - std::vector feedIds) { +// returns max version fetched for each feed +ACTOR Future> dispatchChangeFeeds(StorageServer* data, + UID fetchKeysID, + KeyRange keys, + Version beginVersion, + Version endVersion, + std::vector feedIds) { // find overlapping range feeds - state std::map> feedFetches; + state std::unordered_map feedMaxFetched; + state std::map> feedFetches; state PromiseStream removals; data->changeFeedRemovals[fetchKeysID] = removals; try { - - // TODO add trace events for some of these for (auto& feedId : feedIds) { auto feedIt = data->uidChangeFeed.find(feedId); // TODO REMOVE this assert once we enable change feed deletion ASSERT(feedIt != data->uidChangeFeed.end()); Reference feed = feedIt->second; - feedFetches[feed->id] = fetchChangeFeed(data, feed, transferredVersion); + feedFetches[feed->id] = fetchChangeFeed(data, feed, beginVersion, endVersion); } loop { - Future nextFeed = Never(); + Future nextFeed = Never(); if (!removals.getFuture().isReady()) { bool done = true; while (!feedFetches.empty()) { if (feedFetches.begin()->second.isReady()) { + Version maxFetched = feedFetches.begin()->second.get(); + if (maxFetched != invalidVersion) { + feedFetches[feedFetches.begin()->first] = maxFetched; + } feedFetches.erase(feedFetches.begin()); } else { nextFeed = feedFetches.begin()->second; @@ -4412,12 +4419,12 @@ ACTOR Future dispatchChangeFeeds(StorageServer* data, } if (done) { data->changeFeedRemovals.erase(fetchKeysID); - return Void(); + return feedMaxFetched; } } choose { when(Key remove = waitNext(removals.getFuture())) { feedFetches.erase(remove); } - when(wait(nextFeed)) {} + when(wait(success(nextFeed))) {} } } @@ -4662,11 +4669,16 @@ ACTOR Future fetchKeys(StorageServer* data, AddingShard* shard) { // being recovered. Instead we wait for the updateStorage loop to commit something (and consequently also what // we have written) + state Future> feedFetchMain = + dispatchChangeFeeds(data, fetchKeysID, keys, 0, fetchVersion + 1, changeFeedsToFetch); + state Future fetchDurable = data->durableVersion.whenAtLeast(data->storageVersion() + 1); holdingFKPL.release(); wait(fetchDurable); + state std::unordered_map feedFetchedVersions = wait(feedFetchMain); + TraceEvent(SevDebug, "FKAfterFinalCommit", data->thisServerID) .detail("FKID", interval.pairID) .detail("SV", data->storageVersion()) @@ -4697,8 +4709,11 @@ ACTOR Future fetchKeys(StorageServer* data, AddingShard* shard) { ASSERT(shard->transferredVersion > data->storageVersion()); ASSERT(shard->transferredVersion == data->data().getLatestVersion()); - state Future cfFetch = - dispatchChangeFeeds(data, fetchKeysID, keys, shard->transferredVersion, changeFeedsToFetch); + // This is split into two fetches to reduce tail. Fetch [0 - fetchVersion+1) + // once fetchVersion is finalized, and [fetchVersion+1, transferredVersion) here once transferredVersion is + // finalized + Future> feedFetchTransferred = dispatchChangeFeeds( + data, fetchKeysID, keys, fetchVersion + 1, shard->transferredVersion, changeFeedsToFetch); TraceEvent(SevDebug, "FetchKeysHaveData", data->thisServerID) .detail("FKID", interval.pairID) @@ -4736,8 +4751,16 @@ ACTOR Future fetchKeys(StorageServer* data, AddingShard* shard) { shard->updates.clear(); - // wait on change feed fetch to complete before marking data as available - wait(cfFetch); + // wait on change feed fetch to complete writing to storage before marking data as available + std::unordered_map feedFetchedVersions2 = wait(feedFetchTransferred); + for (auto& newFetch : feedFetchedVersions2) { + auto prevFetch = feedFetchedVersions.find(newFetch.first); + if (prevFetch != feedFetchedVersions.end()) { + prevFetch->second = std::max(prevFetch->second, newFetch.second); + } else { + feedFetchedVersions[newFetch.first] = newFetch.second; + } + } setAvailableStatus(data, keys, @@ -4749,6 +4772,21 @@ ACTOR Future fetchKeys(StorageServer* data, AddingShard* shard) { // Wait for the transferredVersion (and therefore the shard data) to be committed and durable. wait(data->durableVersion.whenAtLeast(shard->transferredVersion)); + // Also wait on all fetched change feed data to become committed and durable + while (!feedFetchedVersions.empty()) { + auto feed = feedFetchedVersions.begin(); + state Key feedId = feed->first; + Version maxFetched = feed->second; + feedFetchedVersions.erase(feed); + + auto feedIt = data->uidChangeFeed.find(feedId); + if (feedIt != data->uidChangeFeed.end() && feedIt->second->durableFetchVersion.get() < maxFetched) { + wait(feedIt->second->durableFetchVersion.whenAtLeast(maxFetched)); + // return to updateStorage + wait(delay(0)); + } + } + ASSERT(data->shards[shard->keys.begin]->assigned() && data->shards[shard->keys.begin]->keys == shard->keys); // We aren't changing whether the shard is assigned @@ -5091,7 +5129,7 @@ void StorageServer::addMutation(Version version, if (!fromFetch) { // have to do change feed before applyMutation because nonExpanded wasn't copied into the mutation log arena, - // and thus would go out of scope + // and thus would go out of scope if it wasn't copied into the change feed arena applyChangeFeedMutation(this, expanded.type == MutationRef::ClearRange ? nonExpanded : expanded, version); } applyMutation(this, expanded, mLog.arena(), mutableData(), version); @@ -5913,14 +5951,17 @@ ACTOR Future updateStorage(StorageServer* data) { data->changeFeedVersions.pop_front(); } + state std::vector> feedFetchVersions; + state std::vector updatedChangeFeeds(modifiedChangeFeeds.begin(), modifiedChangeFeeds.end()); state int curFeed = 0; while (curFeed < updatedChangeFeeds.size()) { auto info = data->uidChangeFeed.find(updatedChangeFeeds[curFeed]); if (info != data->uidChangeFeed.end()) { - // Cannot yield in mutation updating loop because of race between fetchVersion and storageVersion + // Cannot yield in mutation updating loop because of race with fetchVersion + Version alreadyFetched = std::max(info->second->fetchVersion, info->second->durableFetchVersion.get()); for (auto& it : info->second->mutations) { - if (it.version <= info->second->fetchVersion) { + if (it.version <= alreadyFetched) { continue; } else if (it.version > newOldestVersion) { break; @@ -5933,6 +5974,10 @@ ACTOR Future updateStorage(StorageServer* data) { ASSERT(it.version >= info->second->storageVersion); info->second->storageVersion = it.version; } + + if (info->second->fetchVersion != invalidVersion) { + feedFetchVersions.push_back(std::pair(info->second->id, info->second->fetchVersion)); + } // handle case where fetch had version ahead of last in-memory mutation if (info->second->fetchVersion > info->second->storageVersion) { info->second->storageVersion = std::min(info->second->fetchVersion, newOldestVersion); @@ -5942,6 +5987,8 @@ ACTOR Future updateStorage(StorageServer* data) { // to add it back to fetchingChangeFeeds data->fetchingChangeFeeds.insert(info->first); } + } else { + info->second->fetchVersion = invalidVersion; } wait(yield(TaskPriority::UpdateStorage)); } @@ -5995,6 +6042,18 @@ ACTOR Future updateStorage(StorageServer* data) { curFeed++; } + // if commit included fetched data from this change feed, update the fetched durable version + curFeed = 0; + while (curFeed < feedFetchVersions.size()) { + auto info = data->uidChangeFeed.find(feedFetchVersions[curFeed].first); + if (info != data->uidChangeFeed.end()) { + if (feedFetchVersions[curFeed].second > info->second->durableFetchVersion.get()) { + info->second->durableFetchVersion.set(feedFetchVersions[curFeed].second); + } + } + curFeed++; + } + // remove any entries from changeFeedCleanupPending that were persisted auto cfCleanup = data->changeFeedCleanupDurable.begin(); while (cfCleanup != data->changeFeedCleanupDurable.end()) { From 9c8afdf35c2775fcb0a897f488d8cdc78918462d Mon Sep 17 00:00:00 2001 From: Josh Slocum Date: Wed, 2 Feb 2022 15:15:54 -0600 Subject: [PATCH 202/413] Debugged and fixed stuck change feed issue --- fdbclient/NativeAPI.actor.cpp | 20 ++++++++-- fdbserver/BlobWorker.actor.cpp | 72 ++++++++++++++++++++++++++++------ 2 files changed, 76 insertions(+), 16 deletions(-) diff --git a/fdbclient/NativeAPI.actor.cpp b/fdbclient/NativeAPI.actor.cpp index d7858d5ee6..3506137759 100644 --- a/fdbclient/NativeAPI.actor.cpp +++ b/fdbclient/NativeAPI.actor.cpp @@ -7494,6 +7494,11 @@ ACTOR Future doCFMerge(Reference results, wait(delay(0)); ASSERT(results->mutations.isEmpty()); + // update lastReturned once the previous mutation has been consumed + if (*begin - 1 > results->lastReturnedVersion.get()) { + results->lastReturnedVersion.set(*begin - 1); + } + state int interfNum = 0; // TODO minor optimization - could make this just a vector of indexes if each MutationAndVersionStream remembered @@ -7711,11 +7716,18 @@ ACTOR Future doSingleCFStream(KeyRange range, ASSERT(results->streams.size() == 1); ASSERT(results->storageData.size() == 1); state bool atLatest = false; + + // wait for any previous mutations in stream to be consumed + wait(results->mutations.onEmpty()); + wait(delay(0)); + ASSERT(results->mutations.isEmpty()); + // update lastReturned once the previous mutation has been consumed + if (*begin - 1 > results->lastReturnedVersion.get()) { + results->lastReturnedVersion.set(*begin - 1); + } + loop { - // wait for any previous mutations in stream to be consumed - wait(results->mutations.onEmpty()); - wait(delay(0)); - ASSERT(results->mutations.isEmpty()); + state ChangeFeedStreamReply feedReply = waitNext(results->streams[0].getFuture()); *begin = feedReply.mutations.back().version + 1; diff --git a/fdbserver/BlobWorker.actor.cpp b/fdbserver/BlobWorker.actor.cpp index 29a2a10666..6806a7cac7 100644 --- a/fdbserver/BlobWorker.actor.cpp +++ b/fdbserver/BlobWorker.actor.cpp @@ -1096,17 +1096,44 @@ static Version doGranuleRollback(Reference metadata, return cfRollbackVersion; } -ACTOR Future waitOnCFVersion(Reference metadata, Version cfVersion) { +#define DEBUG_WAIT_VERSION_COMMITTED false +ACTOR Future waitOnCFVersion(Reference metadata, + Version original /*TODO REMOVE, just for debugging*/, + Version waitVersion) { + if (DEBUG_BW_VERSION(original) && DEBUG_WAIT_VERSION_COMMITTED) { + fmt::print("WVC {0}: waiting for {1} \n", original, waitVersion); + } loop { try { + if (DEBUG_BW_VERSION(original) && DEBUG_WAIT_VERSION_COMMITTED) { + if (metadata->activeCFData.get().isValid()) { + fmt::print( + "WVC {0}: WAL (currently {1})\n", original, metadata->activeCFData.get()->getVersion()); + } else { + fmt::print("WVC {0}: invalid\n", original, metadata->activeCFData.get()->getVersion()); + } + } // if not valid, we're about to be cancelled anyway - state Future atLeast = - metadata->activeCFData.get().isValid() ? metadata->activeCFData.get()->whenAtLeast(cfVersion) : Never(); + state Future atLeast = metadata->activeCFData.get().isValid() + ? metadata->activeCFData.get()->whenAtLeast(waitVersion) + : Never(); choose { - when(wait(atLeast)) { break; } - when(wait(metadata->activeCFData.onChange())) {} + when(wait(atLeast)) { + if (DEBUG_BW_VERSION(original) && DEBUG_WAIT_VERSION_COMMITTED) { + fmt::print("WVC {0}: got at least {1} \n", original, waitVersion); + } + break; + } + when(wait(metadata->activeCFData.onChange())) { + if (DEBUG_BW_VERSION(original) && DEBUG_WAIT_VERSION_COMMITTED) { + fmt::print("WVC {0}: cfOnChange \n", original); + } + } } } catch (Error& e) { + if (DEBUG_BW_VERSION(original) && DEBUG_WAIT_VERSION_COMMITTED) { + fmt::print("WVC {0}: got error {1} \n", original, e.name()); + } if (e.code() == error_code_operation_cancelled) { throw e; } @@ -1114,9 +1141,9 @@ ACTOR Future waitOnCFVersion(Reference metadata, Version // if waiting on a parent granule change feed and we change to the child, the parent will get end_of_stream, // which could cause this waiting whenAtLeast to get change_feed_cancelled. We should simply retry and wait // a bit, as blobGranuleUpdateFiles will switch to the new change feed - if (e.code() != error_code_change_feed_cancelled) { + if (DEBUG_BW_VERSION(original) && DEBUG_WAIT_VERSION_COMMITTED) { if (BW_DEBUG) { - fmt::print("waitVersionCommitted WAL got unexpected error {}\n", e.name()); + fmt::print("WVC {0}: unexpected error {1}\n", original, e.name()); } throw e; } @@ -1124,9 +1151,13 @@ ACTOR Future waitOnCFVersion(Reference metadata, Version } } + if (DEBUG_BW_VERSION(original) && DEBUG_WAIT_VERSION_COMMITTED) { + fmt::print("WVC {0}: got \n", original); + } + // sanity check to make sure whenAtLeast didn't return early - if (cfVersion > metadata->waitForVersionReturned) { - metadata->waitForVersionReturned = cfVersion; + if (waitVersion > metadata->waitForVersionReturned) { + metadata->waitForVersionReturned = waitVersion; } // stop after change feed callback @@ -1138,8 +1169,14 @@ ACTOR Future waitOnCFVersion(Reference metadata, Version ACTOR Future waitCommittedGrv(Reference bwData, Reference metadata, Version version) { + if (DEBUG_BW_VERSION(version) && DEBUG_WAIT_VERSION_COMMITTED) { + fmt::print("WVC {0}: grv start\n", version); + } // TODO REMOVE debugs if (version > bwData->grvVersion.get()) { + if (DEBUG_BW_VERSION(version) && DEBUG_WAIT_VERSION_COMMITTED) { + fmt::print("WVC {0}: getting grv\n", version); + } /*if (BW_DEBUG) { fmt::print("waitVersionCommitted waiting {0}\n", version); }*/ @@ -1154,7 +1191,10 @@ ACTOR Future waitCommittedGrv(Reference bwData, } Version grvVersion = bwData->grvVersion.get(); - wait(waitOnCFVersion(metadata, grvVersion)); + if (DEBUG_BW_VERSION(version) && DEBUG_WAIT_VERSION_COMMITTED) { + fmt::print("WVC {0}: got grv\n", version); + } + wait(waitOnCFVersion(metadata, version, grvVersion)); return Void(); } @@ -1163,8 +1203,14 @@ ACTOR Future waitVersionCommitted(Reference bwData, Version version) { // If GRV is way in the future, we know we can't roll back more than 5 seconds (or whatever this knob is set to) // worth of versions + if (DEBUG_BW_VERSION(version) && DEBUG_WAIT_VERSION_COMMITTED) { + fmt::print("WVC {0}: starting\n", version); + } wait(waitCommittedGrv(bwData, metadata, version) || - waitOnCFVersion(metadata, version + SERVER_KNOBS->MAX_READ_TRANSACTION_LIFE_VERSIONS)); + waitOnCFVersion(metadata, version, version + SERVER_KNOBS->MAX_READ_TRANSACTION_LIFE_VERSIONS)); + if (DEBUG_BW_VERSION(version) && DEBUG_WAIT_VERSION_COMMITTED) { + fmt::print("WVC {0}: done\n", version); + } if (version > metadata->knownCommittedVersion) { metadata->knownCommittedVersion = version; } @@ -1358,8 +1404,10 @@ ACTOR Future blobGranuleUpdateFiles(Reference bwData, when(Standalone> _mutations = waitNext(metadata->activeCFData.get()->mutations.getFuture())) { /*if (DEBUG_BW_VERSION(metadata->bufferedDeltaVersion)) { - fmt::print("BW got mutations after ({0}): ({1})\n", + fmt::print("BW got mutations after ({0}): {1} - {2} ({3})\n", metadata->bufferedDeltaVersion, + _mutations.front().version, + _mutations.back().version, _mutations.size()); }*/ mutations = _mutations; From d3f9c0112015cac49660c46174dc277ce4313d24 Mon Sep 17 00:00:00 2001 From: Josh Slocum Date: Wed, 2 Feb 2022 15:20:57 -0600 Subject: [PATCH 203/413] Handling change feed fetch for feeds added during fetching a range --- fdbserver/storageserver.actor.cpp | 32 +++++++++++++++++++++++++------ 1 file changed, 26 insertions(+), 6 deletions(-) diff --git a/fdbserver/storageserver.actor.cpp b/fdbserver/storageserver.actor.cpp index 385dfde657..f19044c0f2 100644 --- a/fdbserver/storageserver.actor.cpp +++ b/fdbserver/storageserver.actor.cpp @@ -4379,12 +4379,14 @@ ACTOR Future> fetchChangeFeedMetadata(StorageServer* data, KeyR } // returns max version fetched for each feed +// newFeedIds is used for the second fetch to get data for new feeds that weren't there for the first fetch ACTOR Future> dispatchChangeFeeds(StorageServer* data, UID fetchKeysID, KeyRange keys, Version beginVersion, Version endVersion, - std::vector feedIds) { + std::vector feedIds, + std::unordered_set newFeedIds) { // find overlapping range feeds state std::unordered_map feedMaxFetched; @@ -4399,6 +4401,13 @@ ACTOR Future> dispatchChangeFeeds(StorageServer Reference feed = feedIt->second; feedFetches[feed->id] = fetchChangeFeed(data, feed, beginVersion, endVersion); } + for (auto& feedId : newFeedIds) { + auto feedIt = data->uidChangeFeed.find(feedId); + // TODO REMOVE this assert once we enable change feed deletion + ASSERT(feedIt != data->uidChangeFeed.end()); + Reference feed = feedIt->second; + feedFetches[feed->id] = fetchChangeFeed(data, feed, 0, endVersion); + } loop { Future nextFeed = Never(); @@ -4669,8 +4678,8 @@ ACTOR Future fetchKeys(StorageServer* data, AddingShard* shard) { // being recovered. Instead we wait for the updateStorage loop to commit something (and consequently also what // we have written) - state Future> feedFetchMain = - dispatchChangeFeeds(data, fetchKeysID, keys, 0, fetchVersion + 1, changeFeedsToFetch); + state Future> feedFetchMain = dispatchChangeFeeds( + data, fetchKeysID, keys, 0, fetchVersion + 1, changeFeedsToFetch, std::unordered_set()); state Future fetchDurable = data->durableVersion.whenAtLeast(data->storageVersion() + 1); @@ -4709,11 +4718,22 @@ ACTOR Future fetchKeys(StorageServer* data, AddingShard* shard) { ASSERT(shard->transferredVersion > data->storageVersion()); ASSERT(shard->transferredVersion == data->data().getLatestVersion()); + // find new change feeds for this range that didn't exist when we started the fetch + auto ranges = data->keyChangeFeed.intersectingRanges(keys); + std::unordered_set newChangeFeeds; + for (auto& r : ranges) { + for (auto& cfInfo : r.value()) { + newChangeFeeds.insert(cfInfo->id); + } + } + for (auto& cfId : changeFeedsToFetch) { + newChangeFeeds.erase(cfId); + } // This is split into two fetches to reduce tail. Fetch [0 - fetchVersion+1) // once fetchVersion is finalized, and [fetchVersion+1, transferredVersion) here once transferredVersion is - // finalized - Future> feedFetchTransferred = dispatchChangeFeeds( - data, fetchKeysID, keys, fetchVersion + 1, shard->transferredVersion, changeFeedsToFetch); + // finalized. Also fetch new change feeds alongside it + state Future> feedFetchTransferred = dispatchChangeFeeds( + data, fetchKeysID, keys, fetchVersion + 1, shard->transferredVersion, changeFeedsToFetch, newChangeFeeds); TraceEvent(SevDebug, "FetchKeysHaveData", data->thisServerID) .detail("FKID", interval.pairID) From f16142e639bf71e1c7d865d88897d5f0527631cc Mon Sep 17 00:00:00 2001 From: Josh Slocum Date: Wed, 2 Feb 2022 15:57:32 -0600 Subject: [PATCH 204/413] Fixed memory lifetime issue in blob manager recovery --- fdbserver/BlobManager.actor.cpp | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/fdbserver/BlobManager.actor.cpp b/fdbserver/BlobManager.actor.cpp index baaea2dfba..5160805cc7 100644 --- a/fdbserver/BlobManager.actor.cpp +++ b/fdbserver/BlobManager.actor.cpp @@ -1663,7 +1663,7 @@ ACTOR Future recoverBlobManager(Reference bmData) { // Plus, we don't have a consistent snapshot of the mapping ACROSS blob workers, so we need the DB to reconcile any // differences (eg blob manager revoked from worker A, assigned to B, the revoke from A was processed but the assign // to B wasn't, meaning in the snapshot nobody owns the granule) - state KeyRef beginKey = blobGranuleMappingKeys.begin; + state Key beginKey = blobGranuleMappingKeys.begin; loop { try { tr->setOption(FDBTransactionOptions::ACCESS_SYSTEM_KEYS); @@ -1678,6 +1678,9 @@ ACTOR Future recoverBlobManager(Reference bmData) { // Add the mappings to our in memory key range map for (int rangeIdx = 0; rangeIdx < results.size() - 1; rangeIdx++) { + // TODO REMOVE asserts eventually + ASSERT(results[rangeIdx].key.startsWith(blobGranuleMappingKeys.begin)); + ASSERT(results[rangeIdx + 1].key.startsWith(blobGranuleMappingKeys.begin)); Key granuleStartKey = results[rangeIdx].key.removePrefix(blobGranuleMappingKeys.begin); Key granuleEndKey = results[rangeIdx + 1].key.removePrefix(blobGranuleMappingKeys.begin); if (results[rangeIdx].value.size()) { From feccea05e5f02a15f8e3b999b894cf41ceb6a6e0 Mon Sep 17 00:00:00 2001 From: Josh Slocum Date: Thu, 3 Feb 2022 06:15:22 -0600 Subject: [PATCH 205/413] Fixing incorrect revokes on blob manager recovery --- fdbserver/BlobManager.actor.cpp | 7 ++++--- 1 file changed, 4 insertions(+), 3 deletions(-) diff --git a/fdbserver/BlobManager.actor.cpp b/fdbserver/BlobManager.actor.cpp index 5160805cc7..d0548ca55c 100644 --- a/fdbserver/BlobManager.actor.cpp +++ b/fdbserver/BlobManager.actor.cpp @@ -1391,7 +1391,8 @@ static void addAssignment(KeyRangeMap>& map, // different workers can't have same epoch and seqno for granule assignment ASSERT(oldEpoch != newEpoch || oldSeqno != newSeqno); } - if (outOfDate != nullptr && oldEpoch > 0) { + if (outOfDate != nullptr && oldWorker != UID() && + (oldEpoch < newEpoch || (oldEpoch == newEpoch && oldSeqno < newSeqno))) { outOfDate->push_back(std::pair(oldWorker, old.range())); } } @@ -1403,7 +1404,7 @@ static void addAssignment(KeyRangeMap>& map, // then, if there were any ranges superceded by this one, insert them over this one if (newer.size()) { - if (outOfDate != nullptr) { + if (outOfDate != nullptr && newId != UID()) { outOfDate->push_back(std::pair(newId, newRange)); } for (auto& it : newer) { @@ -1411,7 +1412,7 @@ static void addAssignment(KeyRangeMap>& map, } } } else { - if (outOfDate != nullptr) { + if (outOfDate != nullptr && newId != UID()) { outOfDate->push_back(std::pair(newId, newRange)); } } From 0f9b2f7c48957867fa71d9eece6575ed844cd6c7 Mon Sep 17 00:00:00 2001 From: Josh Slocum Date: Thu, 3 Feb 2022 10:49:30 -0600 Subject: [PATCH 206/413] Change feed fetch refactor and pop during fetch improvements --- fdbserver/storageserver.actor.cpp | 272 ++++++++++++++++-------------- 1 file changed, 141 insertions(+), 131 deletions(-) diff --git a/fdbserver/storageserver.actor.cpp b/fdbserver/storageserver.actor.cpp index f19044c0f2..19d1ffb4b9 100644 --- a/fdbserver/storageserver.actor.cpp +++ b/fdbserver/storageserver.actor.cpp @@ -1706,9 +1706,9 @@ MutationsAndVersionRef filterMutations(Arena& arena, } // TODO REMOVE!!! when BG is correctness clean -#define DEBUG_SS_ID ""_sr -#define DEBUG_SS_CF_ID ""_sr -#define DEBUG_SS_CF_BEGIN_VERSION invalidVersion +#define DEBUG_SS_ID "bdd3"_sr +#define DEBUG_SS_CF_ID "b65dda"_sr +#define DEBUG_SS_CF_BEGIN_VERSION 244526755 #define DEBUG_SS_CFM(ssId, cfId, v) \ ssId.toString().substr(0, 4) == DEBUG_SS_ID&& cfId.printable().substr(0, 6) == DEBUG_SS_CF_ID && \ (v >= DEBUG_SS_CF_BEGIN_VERSION || latestVersion == DEBUG_SS_CF_BEGIN_VERSION) @@ -1716,6 +1716,7 @@ MutationsAndVersionRef filterMutations(Arena& arena, ACTOR Future> getChangeFeedMutations(StorageServer* data, ChangeFeedStreamRequest req, bool inverted, + bool atLatest, UID streamUID /*TODO REMOVE*/) { state ChangeFeedStreamReply reply; @@ -1725,7 +1726,7 @@ ACTOR Future> getChangeFeedMutations(Stor state Version startVersion = data->version.get(); if (DEBUG_SS_CFM(data->thisServerID, req.rangeID, req.begin)) { - printf("CFM: SS %s CF %s: GCFM %s [%s - %s) %lld - %lld\n", + printf("CFM: SS %s CF %s: SQ %s [%s - %s) %lld - %lld\n", data->thisServerID.toString().substr(0, 4).c_str(), req.rangeID.printable().substr(0, 6).c_str(), streamUID.toString().substr(0, 8).c_str(), @@ -1746,7 +1747,7 @@ ACTOR Future> getChangeFeedMutations(Stor // we must delay here to ensure that any up-to-date change feeds that are waiting on the // mutation trigger run BEFORE any blocked change feeds run, in order to preserve the // correct minStreamVersion ordering - wait(delay(0.0)); + wait(delay(0)); } state uint64_t changeCounter = data->shardChangeCounter; if (!inverted && !data->isReadable(req.range)) { @@ -1785,10 +1786,13 @@ ACTOR Future> getChangeFeedMutations(Stor state Version emptyVersion = feedInfo->emptyVersion; if (DEBUG_SS_CFM(data->thisServerID, req.rangeID, req.begin)) { - printf("CFM: SS %s CF %s: dequeVersion=%lld, emptyVersion=%lld, storageVersion=%lld, durableVersion=%lld, " + printf("CFM: SS %s CF %s: SQ %s atLatest=%s, dequeVersion=%lld, emptyVersion=%lld, storageVersion=%lld, " + "durableVersion=%lld, " "fetchVersion=%lld\n", data->thisServerID.toString().substr(0, 4).c_str(), req.rangeID.printable().substr(0, 6).c_str(), + streamUID.toString().substr(0, 8).c_str(), + atLatest ? "T" : "F", dequeVersion, feedInfo->emptyVersion, feedInfo->storageVersion, @@ -1797,15 +1801,18 @@ ACTOR Future> getChangeFeedMutations(Stor } if (req.end > emptyVersion + 1) { + // FIXME: do exponential backwards search from end to find beginVersion if atLatest to reduce cpu for (auto& it : feedInfo->mutations) { if (it.version >= req.end || it.version > dequeVersion || remainingLimitBytes <= 0) { break; } if (it.version >= req.begin) { - memoryReply.arena.dependsOn(it.arena()); auto m = filterMutations(memoryReply.arena, it, req.range, inverted); - memoryReply.mutations.push_back(memoryReply.arena, m); - remainingLimitBytes -= sizeof(MutationsAndVersionRef) + m.expectedSize(); + if (m.mutations.size()) { + memoryReply.arena.dependsOn(it.arena()); + memoryReply.mutations.push_back(memoryReply.arena, m); + remainingLimitBytes -= sizeof(MutationsAndVersionRef) + m.expectedSize(); + } } } if (DEBUG_SS_CFM(data->thisServerID, req.rangeID, req.begin)) { @@ -1818,8 +1825,9 @@ ACTOR Future> getChangeFeedMutations(Stor } } - bool readDurable = feedInfo->durableVersion != invalidVersion && req.begin <= feedInfo->durableVersion; - bool readFetched = feedInfo->durableVersion < feedInfo->fetchVersion && req.begin <= feedInfo->fetchVersion; + state bool readDurable = feedInfo->durableVersion != invalidVersion && req.begin <= feedInfo->durableVersion; + state bool readFetched = feedInfo->durableVersion < feedInfo->fetchVersion && req.begin <= feedInfo->fetchVersion; + state bool readAnyFromDisk = false; if (req.end > emptyVersion + 1 && (readDurable || readFetched)) { if (readFetched) { // To not block fetchKeys making normal SS data readable on making change feed data written to storage, we @@ -1842,6 +1850,7 @@ ACTOR Future> getChangeFeedMutations(Stor } Version lastVersion = req.begin - 1; + Version lastKnownCommitted = invalidVersion; for (auto& kv : res) { Key id; Version version, knownCommittedVersion; @@ -1851,12 +1860,16 @@ ACTOR Future> getChangeFeedMutations(Stor reply.arena.dependsOn(mutations.arena()); auto m = filterMutations( reply.arena, MutationsAndVersionRef(mutations, version, knownCommittedVersion), req.range, inverted); - reply.mutations.push_back(reply.arena, m); + if (m.mutations.size()) { + readAnyFromDisk = true; + reply.mutations.push_back(reply.arena, m); + } remainingDurableBytes -= sizeof(KeyValueRef) + kv.expectedSize(); // This is tracking the size on disk rather than the reply size // because we cannot add mutations from memory if there are potentially more on disk lastVersion = version; + lastKnownCommitted = knownCommittedVersion; } if (DEBUG_SS_CFM(data->thisServerID, req.rangeID, req.begin)) { printf("CFM: SS %s CF %s: got %lld - %lld (%d) from disk\n", @@ -1865,6 +1878,15 @@ ACTOR Future> getChangeFeedMutations(Stor reply.mutations.empty() ? invalidVersion : reply.mutations.front().version, reply.mutations.empty() ? invalidVersion : reply.mutations.back().version, reply.mutations.size()); + if (!reply.mutations.empty() && reply.mutations.front().version == 252133030) { + for (auto& it : reply.mutations) { + printf("CFM: SS %s CF %s: %lld (%d)\n", + data->thisServerID.toString().substr(0, 4).c_str(), + req.rangeID.printable().substr(0, 6).c_str(), + it.version, + it.mutations.size()); + } + } } if (remainingDurableBytes > 0) { reply.arena.dependsOn(memoryReply.arena); @@ -1875,20 +1897,41 @@ ACTOR Future> getChangeFeedMutations(Stor --totalCount; } reply.mutations.append(reply.arena, it, totalCount); + } else if (!readAnyFromDisk) { + if (DEBUG_SS_CFM(data->thisServerID, req.rangeID, req.begin)) { + printf("CFM: SS %s CF %s: adding empty from disk %lld\n", + data->thisServerID.toString().substr(0, 4).c_str(), + req.rangeID.printable().substr(0, 6).c_str(), + lastVersion); + } + reply.mutations.push_back(reply.arena, MutationsAndVersionRef(lastVersion, lastKnownCommitted)); } } else { reply = memoryReply; } - // check if pop happened concurrently with read + // check if pop happened concurrently with disk read if (!req.canReadPopped && req.begin <= feedInfo->emptyVersion) { - printf("SS %s: CF %s popped after read! req.begin=%lld, emptyVersion=%lld, emptyBeforeRead=%lld\n", - data->thisServerID.toString().substr(0, 4).c_str(), - req.rangeID.printable().substr(0, 6).c_str(), - req.begin, - feedInfo->emptyVersion, - emptyVersion); - throw change_feed_popped(); + // this can happen under normal circumstances on one condition: a feed atLatest has to wait for a fetch to read + // from disk, and the feed is popped during that. Since pop can't guarantee cleaning up the entire fetch, if the + // read only got empty mutations after filtering, this is fine + bool ok = atLatest && readFetched && !readDurable && !readAnyFromDisk; + TEST(ok); // feed popped while read waiting for fetch + if (!ok) { + printf("SS %s: CF %s SQ %s popped after read! req.begin=%lld, emptyVersion=%lld, emptyBeforeRead=%lld, " + "atLatest=%s, readFetched=%s, readDurable=%s, readAnyFromDisk=%s\n", + data->thisServerID.toString().substr(0, 4).c_str(), + req.rangeID.printable().substr(0, 6).c_str(), + streamUID.toString().substr(0, 8).c_str(), + req.begin, + feedInfo->emptyVersion, + emptyVersion, + atLatest ? "T" : "F", + readFetched ? "T" : "F", + readDurable ? "T" : "F", + readAnyFromDisk ? "T" : "F"); + throw change_feed_popped(); + } } bool gotAll = remainingLimitBytes > 0 && remainingDurableBytes > 0 && data->version.get() == startVersion; @@ -1958,7 +2001,7 @@ ACTOR Future localChangeFeedStream(StorageServer* data, feedRequest.end = end; feedRequest.range = range; state std::pair feedReply = - wait(getChangeFeedMutations(data, feedRequest, true, UID())); + wait(getChangeFeedMutations(data, feedRequest, true, false, UID())); begin = feedReply.first.mutations.back().version + 1; state int resultLoc = 0; while (resultLoc < feedReply.first.mutations.size()) { @@ -2029,7 +2072,7 @@ ACTOR Future changeFeedStreamQ(StorageServer* data, ChangeFeedStreamReques } wait(onReady); state Future> feedReplyFuture = - getChangeFeedMutations(data, req, false, streamUID); + getChangeFeedMutations(data, req, false, atLatest, streamUID); if (atLatest && !removeUID && !feedReplyFuture.isReady()) { data->changeFeedClientVersions[req.reply.getEndpoint().getPrimaryAddress()][streamUID] = blockedVersion.present() ? blockedVersion.get() : data->prevVersion; @@ -2039,6 +2082,7 @@ ACTOR Future changeFeedStreamQ(StorageServer* data, ChangeFeedStreamReques ChangeFeedStreamReply feedReply = _feedReply.first; bool gotAll = _feedReply.second; + ASSERT(feedReply.mutations.size() > 0); req.begin = feedReply.mutations.back().version + 1; if (!atLatest && gotAll) { atLatest = true; @@ -4061,6 +4105,9 @@ ACTOR Future changeFeedPopQ(StorageServer* self, ChangeFeedPopRequest req) feed->second->storageVersion = invalidVersion; feed->second->durableVersion = invalidVersion; } + if (req.version > feed->second->fetchVersion) { + feed->second->fetchVersion = invalidVersion; + } } // wake up requests that didn't get any mutations since the last pop to update their beginVersion feed->second->newMutations.trigger(); @@ -4070,7 +4117,6 @@ ACTOR Future changeFeedPopQ(StorageServer* self, ChangeFeedPopRequest req) return Void(); } -// TODO handle being popped in the middle by removing after fetch // Returns max version fetched ACTOR Future fetchChangeFeedApplier(StorageServer* data, Reference changeFeedInfo, @@ -4103,63 +4149,11 @@ ACTOR Future fetchChangeFeedApplier(StorageServer* data, state Version lastVersion = invalidVersion; state int64_t versionsFetched = 0; - if (!existing) { - try { - loop { - while (data->fetchKeysBudgetUsed.get()) { - wait(data->fetchKeysBudgetUsed.onChange()); - } - Standalone> res = waitNext(feedResults->mutations.getFuture()); - for (auto& it : res) { - if (it.mutations.size()) { - if (MUTATION_TRACKING_ENABLED) { - for (auto& m : it.mutations) { - DEBUG_MUTATION("ChangeFeedWriteMove", it.version, m, data->thisServerID) - .detail("Range", range) - .detail("Existing", existing) - .detail("ChangeFeedID", rangeId); - } - } - data->storage.writeKeyValue( - KeyValueRef(changeFeedDurableKey(rangeId, it.version), - changeFeedDurableValue(it.mutations, it.knownCommittedVersion))); - - changeFeedInfo->fetchVersion = std::max(changeFeedInfo->fetchVersion, it.version); - if (firstVersion == invalidVersion) { - firstVersion = it.version; - } - lastVersion = it.version; - versionsFetched++; - } - } - data->fetchKeysBytesBudget -= res.expectedSize(); - if (data->fetchKeysBytesBudget <= 0) { - data->fetchKeysBudgetUsed.set(true); - } - wait(yield()); - } - } catch (Error& e) { - if (e.code() != error_code_end_of_stream) { - throw; - } - // TODO REMOVE? - TraceEvent(SevDebug, "FetchChangeFeedDone", data->thisServerID) - .detail("RangeID", rangeId.printable()) - .detail("Range", range.toString()) - .detail("StartVersion", startVersion) - .detail("EndVersion", endVersion) - .detail("FirstFetchedVersion", firstVersion) - .detail("LastFetchedVersion", lastVersion) - .detail("VersionsFetched", versionsFetched) - .detail("Existing", existing); - return lastVersion; - } - } - state PromiseStream> localResults; // Add 1 to fetch version to make sure the local stream will have more versions in the stream than the remote stream // to avoid edge cases in the merge logic + state Future localStream = localChangeFeedStream(data, localResults, rangeId, startVersion, endVersion + 1, range); state Standalone localResult; @@ -4175,62 +4169,64 @@ ACTOR Future fetchChangeFeedApplier(StorageServer* data, state Standalone> remoteResult = waitNext(feedResults->mutations.getFuture()); state int remoteLoc = 0; + while (remoteLoc < remoteResult.size()) { - if (firstVersion == invalidVersion) { - firstVersion = remoteResult[remoteLoc].version; + Version localVersion = localResult.version; + Version remoteVersion = remoteResult[remoteLoc].version; + if (remoteVersion <= localVersion) { + if (remoteVersion > changeFeedInfo->emptyVersion) { + // merge if same version + if (remoteVersion == localVersion && remoteResult[remoteLoc].mutations.size() && + remoteResult[remoteLoc].mutations.back().param1 != lastEpochEndPrivateKey) { + int remoteSize = remoteResult[remoteLoc].mutations.size(); + ASSERT(localResult.mutations.size()); + remoteResult[remoteLoc].mutations.append( + remoteResult.arena(), localResult.mutations.begin(), localResult.mutations.size()); + if (MUTATION_TRACKING_ENABLED) { + int midx = 0; + for (auto& m : remoteResult[remoteLoc].mutations) { + DEBUG_MUTATION("ChangeFeedWriteMoveMerge", remoteVersion, m, data->thisServerID) + .detail("Range", range) + .detail("FromLocal", midx >= remoteSize) + .detail("ChangeFeedID", rangeId); + midx++; + } + } + } else { + if (MUTATION_TRACKING_ENABLED) { + for (auto& m : remoteResult[remoteLoc].mutations) { + DEBUG_MUTATION("ChangeFeedWriteMove", remoteVersion, m, data->thisServerID) + .detail("Range", range) + .detail("Existing", existing) + .detail("ChangeFeedID", rangeId); + } + } + } + + data->storage.writeKeyValue( + KeyValueRef(changeFeedDurableKey(rangeId, remoteVersion), + changeFeedDurableValue(remoteResult[remoteLoc].mutations, + remoteResult[remoteLoc].knownCommittedVersion))); + changeFeedInfo->fetchVersion = std::max(changeFeedInfo->fetchVersion, remoteVersion); + + if (firstVersion == invalidVersion) { + firstVersion = remoteVersion; + } + lastVersion = remoteVersion; + versionsFetched++; + } else if (versionsFetched > 0) { + ASSERT(firstVersion != invalidVersion); + ASSERT(lastVersion != invalidVersion); + data->storage.clearRange( + KeyRangeRef(changeFeedDurableKey(changeFeedInfo->id, firstVersion), + changeFeedDurableKey(changeFeedInfo->id, lastVersion + 1))); + firstVersion = invalidVersion; + lastVersion = invalidVersion; + versionsFetched = 0; + } + remoteLoc++; } - lastVersion = remoteResult[remoteLoc].version; - versionsFetched++; - - if (remoteResult[remoteLoc].version < localResult.version) { - if (remoteResult[remoteLoc].mutations.size()) { - if (MUTATION_TRACKING_ENABLED) { - for (auto& m : remoteResult[remoteLoc].mutations) { - DEBUG_MUTATION( - "ChangeFeedWriteMove", remoteResult[remoteLoc].version, m, data->thisServerID) - .detail("Range", range) - .detail("Existing", existing) - .detail("ChangeFeedID", rangeId); - } - } - data->storage.writeKeyValue( - KeyValueRef(changeFeedDurableKey(rangeId, remoteResult[remoteLoc].version), - changeFeedDurableValue(remoteResult[remoteLoc].mutations, - remoteResult[remoteLoc].knownCommittedVersion))); - changeFeedInfo->fetchVersion = - std::max(changeFeedInfo->fetchVersion, remoteResult[remoteLoc].version); - } - remoteLoc++; - } else if (remoteResult[remoteLoc].version == localResult.version) { - if (remoteResult[remoteLoc].mutations.size() && - remoteResult[remoteLoc].mutations.back().param1 != lastEpochEndPrivateKey) { - int remoteSize = remoteResult[remoteLoc].mutations.size(); - ASSERT(localResult.mutations.size()); - remoteResult[remoteLoc].mutations.append( - remoteResult.arena(), localResult.mutations.begin(), localResult.mutations.size()); - if (MUTATION_TRACKING_ENABLED) { - int midx = 0; - for (auto& m : remoteResult[remoteLoc].mutations) { - DEBUG_MUTATION( - "ChangeFeedWriteMoveMerge", remoteResult[remoteLoc].version, m, data->thisServerID) - .detail("Range", range) - .detail("FromLocal", midx >= remoteSize) - .detail("ChangeFeedID", rangeId); - midx++; - } - } - data->storage.writeKeyValue( - KeyValueRef(changeFeedDurableKey(rangeId, remoteResult[remoteLoc].version), - changeFeedDurableValue(remoteResult[remoteLoc].mutations, - remoteResult[remoteLoc].knownCommittedVersion))); - - changeFeedInfo->fetchVersion = - std::max(changeFeedInfo->fetchVersion, remoteResult[remoteLoc].version); - } - remoteLoc++; - Standalone _localResult = waitNext(localResults.getFuture()); - localResult = _localResult; - } else { + if (localVersion <= remoteVersion) { Standalone _localResult = waitNext(localResults.getFuture()); localResult = _localResult; } @@ -4247,6 +4243,18 @@ ACTOR Future fetchChangeFeedApplier(StorageServer* data, } } + // if we were popped while fetching but it didn't pass the fetch version while writing, clean up here + if (versionsFetched > 0 && startVersion < changeFeedInfo->emptyVersion) { + ASSERT(firstVersion != invalidVersion); + ASSERT(lastVersion != invalidVersion); + data->storage.clearRange(KeyRangeRef( + changeFeedDurableKey(changeFeedInfo->id, firstVersion), + changeFeedDurableKey(changeFeedInfo->id, std::min(lastVersion + 1, changeFeedInfo->emptyVersion)))); + firstVersion = invalidVersion; + lastVersion = invalidVersion; + versionsFetched = 0; + } + // TODO REMOVE? TraceEvent(SevDebug, "FetchChangeFeedDone", data->thisServerID) .detail("RangeID", rangeId.printable()) @@ -5374,7 +5382,9 @@ private: if (popVersion > feed->second->storageVersion) { feed->second->storageVersion = invalidVersion; feed->second->durableVersion = invalidVersion; - // don't set fetchVersion to invalidVersion here because there could be an active fetch + } + if (popVersion > feed->second->fetchVersion) { + feed->second->fetchVersion = invalidVersion; } } addMutationToLog = true; From d0c4c6856fc4443a2993a290588060930423d97f Mon Sep 17 00:00:00 2001 From: Josh Slocum Date: Thu, 3 Feb 2022 12:38:37 -0600 Subject: [PATCH 207/413] Fix pop at end of CF fetch --- fdbserver/storageserver.actor.cpp | 11 +++++------ 1 file changed, 5 insertions(+), 6 deletions(-) diff --git a/fdbserver/storageserver.actor.cpp b/fdbserver/storageserver.actor.cpp index 19d1ffb4b9..a4e7cf379f 100644 --- a/fdbserver/storageserver.actor.cpp +++ b/fdbserver/storageserver.actor.cpp @@ -4247,12 +4247,11 @@ ACTOR Future fetchChangeFeedApplier(StorageServer* data, if (versionsFetched > 0 && startVersion < changeFeedInfo->emptyVersion) { ASSERT(firstVersion != invalidVersion); ASSERT(lastVersion != invalidVersion); - data->storage.clearRange(KeyRangeRef( - changeFeedDurableKey(changeFeedInfo->id, firstVersion), - changeFeedDurableKey(changeFeedInfo->id, std::min(lastVersion + 1, changeFeedInfo->emptyVersion)))); - firstVersion = invalidVersion; - lastVersion = invalidVersion; - versionsFetched = 0; + Version endClear = std::min(lastVersion + 1, changeFeedInfo->emptyVersion); + if (endClear > firstVersion) { + data->storage.clearRange(KeyRangeRef(changeFeedDurableKey(changeFeedInfo->id, firstVersion), + changeFeedDurableKey(changeFeedInfo->id, endClear))); + } } // TODO REMOVE? From c1559e6392b977a3f5abcd70d234f374afce5146 Mon Sep 17 00:00:00 2001 From: Josh Slocum Date: Thu, 3 Feb 2022 18:14:54 -0600 Subject: [PATCH 208/413] more blob manager recovery issues --- fdbserver/BlobManager.actor.cpp | 18 +++++++++++++++++- 1 file changed, 17 insertions(+), 1 deletion(-) diff --git a/fdbserver/BlobManager.actor.cpp b/fdbserver/BlobManager.actor.cpp index d0548ca55c..8132d750d4 100644 --- a/fdbserver/BlobManager.actor.cpp +++ b/fdbserver/BlobManager.actor.cpp @@ -1384,7 +1384,15 @@ static void addAssignment(KeyRangeMap>& map, int64_t oldEpoch = std::get<1>(old.value()); int64_t oldSeqno = std::get<2>(old.value()); if (oldEpoch > newEpoch || (oldEpoch == newEpoch && oldSeqno > newSeqno)) { - newer.push_back(std::pair(old.range(), std::tuple(oldWorker, oldEpoch, oldSeqno))); + if (newId != oldWorker && newId != UID() && newEpoch == 0 && newSeqno == 1 && + old.begin() == newRange.begin && old.end() == newRange.end) { + // granule mapping disagrees with worker with highest value. Just do an explicit reassign to a random + // worker for now to ensure the conflict is resolved. + newer.push_back(std::pair(old.range(), std::tuple(UID(), oldEpoch, oldSeqno))); + allNewer = false; + } else { + newer.push_back(std::pair(old.range(), std::tuple(oldWorker, oldEpoch, oldSeqno))); + } } else { allNewer = false; if (newId != UID()) { @@ -1780,6 +1788,7 @@ ACTOR Future recoverBlobManager(Reference bmData) { fmt::print("BM {0} final ranges:\n", bmData->epoch); } + int explicitAssignments = 0; for (auto& range : workerAssignments.intersectingRanges(normalKeys)) { int64_t epoch = std::get<1>(range.value()); int64_t seqno = std::get<2>(range.value()); @@ -1811,9 +1820,16 @@ ACTOR Future recoverBlobManager(Reference bmData) { raAssign.keyRange = range.range(); raAssign.assign = RangeAssignmentData(AssignRequestType::Normal); bmData->rangesToAssign.send(raAssign); + explicitAssignments++; } } + TraceEvent("BlobManagerRecovered", bmData->id) + .detail("Epoch", bmData->epoch) + .detail("Granules", bmData->workerAssignments.size()) + .detail("Assigned", explicitAssignments) + .detail("Revoked", outOfDateAssignments.size()); + return Void(); } From 2592c3f0ae5fad744e46bb7b02f9da83339dfa83 Mon Sep 17 00:00:00 2001 From: Josh Slocum Date: Thu, 3 Feb 2022 18:56:01 -0600 Subject: [PATCH 209/413] Implemented less efficient version of reply stream closing --- fdbclient/NativeAPI.actor.cpp | 33 ++++++++++++++++++++++----------- fdbrpc/fdbrpc.h | 28 ++++++++++++++++++++++++++-- fdbrpc/genericactors.actor.h | 12 +++++++++++- 3 files changed, 59 insertions(+), 14 deletions(-) diff --git a/fdbclient/NativeAPI.actor.cpp b/fdbclient/NativeAPI.actor.cpp index 3506137759..ae13509d9c 100644 --- a/fdbclient/NativeAPI.actor.cpp +++ b/fdbclient/NativeAPI.actor.cpp @@ -7158,9 +7158,9 @@ Version ChangeFeedData::getVersion() { // range that surrounds wait_version enough to figure out what's going on // DEBUG_CF_ID is optional #define DEBUG_CF_ID ""_sr -#define DEBUG_CF_START_VERSION invalidVersion -#define DEBUG_CF_END_VERSION invalidVersion -#define DEBUG_CF_WAIT_VERSION invalidVersion +#define DEBUG_CF_START_VERSION 322240646 +#define DEBUG_CF_END_VERSION 393714633 +#define DEBUG_CF_WAIT_VERSION 383714633 #define DEBUG_CF_VERSION(cfId, v) \ DEBUG_CF_START_VERSION <= v&& v <= DEBUG_CF_END_VERSION && (""_sr == DEBUG_CF_ID || cfId.printable() == DEBUG_CF_ID) @@ -7459,14 +7459,15 @@ ACTOR Future partialChangeFeedStream(StorageServerInterface interf, } } catch (Error& e) { // TODO REMOVE eventually, useful for debugging for now - if (DEBUG_CF_VERSION(feedData->id, nextVersion)) { - fmt::print(" single {0} {1} [{2} - {3}): CFError {4}\n", - idx, - interf.id().toString().substr(0, 4), - range.begin.printable(), - range.end.printable(), - e.name()); - } + // if (DEBUG_CF_VERSION(feedData->id, nextVersion)) { + fmt::print(" single {0} {1} {2} [{3} - {4}): CFError {5}\n", + idx, + interf.id().toString().substr(0, 4), + debugID.toString().substr(0, 8).c_str(), + range.begin.printable(), + range.end.printable(), + e.name()); + // } if (e.code() == error_code_actor_cancelled) { throw; } @@ -7627,6 +7628,16 @@ ACTOR Future mergeChangeFeedStream(Reference db, debugIDs.push_back(debugID); req.debugID = debugID; results->streams.push_back(it.first.changeFeedStream.getReplyStream(req)); + if (debugID.toString().substr(0, 8) == "637bfa4e") { + printf( + "Good example: %s: %p\n", debugID.toString().substr(0, 8).c_str(), results->streams.back().debugAddr()); + printf("\n"); + } + if (debugID.toString().substr(0, 8) == "1ad27675") { + printf( + "Bad example: %s: %p\n", debugID.toString().substr(0, 8).c_str(), results->streams.back().debugAddr()); + printf("\n"); + } } for (auto& it : results->storageData) { diff --git a/fdbrpc/fdbrpc.h b/fdbrpc/fdbrpc.h index 5c63ec5251..666454ed25 100644 --- a/fdbrpc/fdbrpc.h +++ b/fdbrpc/fdbrpc.h @@ -324,10 +324,12 @@ struct NetNotifiedQueueWithAcknowledgements final : NotifiedQueue, AcknowledgementReceiver acknowledgements; Endpoint requestStreamEndpoint; bool sentError = false; + Promise onConnect; - NetNotifiedQueueWithAcknowledgements(int futures, int promises) : NotifiedQueue(futures, promises) {} + NetNotifiedQueueWithAcknowledgements(int futures, int promises) + : NotifiedQueue(futures, promises), onConnect(nullptr) {} NetNotifiedQueueWithAcknowledgements(int futures, int promises, const Endpoint& remoteEndpoint) - : NotifiedQueue(futures, promises), FlowReceiver(remoteEndpoint, true) { + : NotifiedQueue(futures, promises), FlowReceiver(remoteEndpoint, true), onConnect(nullptr) { // A ReplyPromiseStream will be terminated on the server side if the network connection with the client breaks acknowledgements.failures = tagError( makeDependent(IFailureMonitor::failureMonitor()).onDisconnect(remoteEndpoint.getPrimaryAddress()), @@ -348,11 +350,17 @@ struct NetNotifiedQueueWithAcknowledgements final : NotifiedQueue, // GetKeyValuesStream requests on the same endpoint will fail IFailureMonitor::failureMonitor().endpointNotFound(requestStreamEndpoint); } + if (onConnect.isValid() && onConnect.canBeSet()) { + onConnect.send(Void()); + } this->sendError(message.getError()); } else { if (message.get().asUnderlyingType().acknowledgeToken.present()) { acknowledgements = AcknowledgementReceiver( FlowTransport::transport().loadedEndpoint(message.get().asUnderlyingType().acknowledgeToken.get())); + if (onConnect.isValid() && onConnect.canBeSet()) { + onConnect.send(Void()); + } } if (acknowledgements.sequence != message.get().asUnderlyingType().sequence) { TraceEvent(SevError, "StreamSequenceMismatch") @@ -439,6 +447,9 @@ public: } } + // TODO REMOVE + const void* debugAddr() const { return queue; } + template void sendError(const E& exc) const { if (queue->isRemoteEndpoint()) { @@ -485,6 +496,18 @@ public: void setRequestStreamEndpoint(const Endpoint& endpoint) { queue->requestStreamEndpoint = endpoint; } + bool connected() { return queue->acknowledgements.getRawEndpoint().isValid() || queue->error.isValid(); } + + Future onConnected() { + if (connected()) { + return Future(Void()); + } + if (!queue->onConnect.isValid()) { + queue->onConnect = Promise(); + } + return queue->onConnect.getFuture(); + } + ~ReplyPromiseStream() { if (queue) queue->delPromiseRef(); @@ -744,6 +767,7 @@ public: FlowTransport::transport().sendUnreliable(SerializeSource(value), getEndpoint(), true); // FIXME: defer sending the message until we know the connection is established endStreamOnDisconnect(disc, p, getEndpoint(), peer); + holdUntilConnected(disc, p); return p; } else { send(value); diff --git a/fdbrpc/genericactors.actor.h b/fdbrpc/genericactors.actor.h index 46a79d29cf..11c9c38395 100644 --- a/fdbrpc/genericactors.actor.h +++ b/fdbrpc/genericactors.actor.h @@ -197,6 +197,11 @@ struct PeerHolder { } }; +ACTOR template +void holdUntilConnected(Future signal, ReplyPromiseStream stream) { + wait(stream.onConnected() || signal); +} + // Implements getReplyStream, this a void actor with the same lifetime as the input ReplyPromiseStream. // Because this actor holds a reference to the stream, normally it would be impossible to know when there are no other // references. To get around this, there is a SAV inside the stream that has one less promise reference than it should @@ -212,7 +217,12 @@ void endStreamOnDisconnect(Future signal, stream.setRequestStreamEndpoint(endpoint); choose { when(wait(signal)) { stream.sendError(connection_failed()); } - when(wait(stream.getErrorFutureAndDelPromiseRef())) {} + when(wait(stream.getErrorFutureAndDelPromiseRef())) { + // Wait for a response from the server + /*if (!stream.connected()) { + // TODO WANT TO DO holdAndConnected ACTOR HERE INSTEAD! + }*/ + } } } From a9747736766712c8b23cae30367dc6da3435b2c0 Mon Sep 17 00:00:00 2001 From: Josh Slocum Date: Fri, 4 Feb 2022 09:33:04 -0600 Subject: [PATCH 210/413] Handled change feed getting moved away or cancelled during move --- fdbclient/NativeAPI.actor.cpp | 6 +++--- fdbserver/storageserver.actor.cpp | 19 ++++++++++--------- 2 files changed, 13 insertions(+), 12 deletions(-) diff --git a/fdbclient/NativeAPI.actor.cpp b/fdbclient/NativeAPI.actor.cpp index ae13509d9c..2ddf1cd674 100644 --- a/fdbclient/NativeAPI.actor.cpp +++ b/fdbclient/NativeAPI.actor.cpp @@ -7158,9 +7158,9 @@ Version ChangeFeedData::getVersion() { // range that surrounds wait_version enough to figure out what's going on // DEBUG_CF_ID is optional #define DEBUG_CF_ID ""_sr -#define DEBUG_CF_START_VERSION 322240646 -#define DEBUG_CF_END_VERSION 393714633 -#define DEBUG_CF_WAIT_VERSION 383714633 +#define DEBUG_CF_START_VERSION invalidVersion +#define DEBUG_CF_END_VERSION invalidVersion +#define DEBUG_CF_WAIT_VERSION invalidVersion #define DEBUG_CF_VERSION(cfId, v) \ DEBUG_CF_START_VERSION <= v&& v <= DEBUG_CF_END_VERSION && (""_sr == DEBUG_CF_ID || cfId.printable() == DEBUG_CF_ID) diff --git a/fdbserver/storageserver.actor.cpp b/fdbserver/storageserver.actor.cpp index a4e7cf379f..fbdbdbc318 100644 --- a/fdbserver/storageserver.actor.cpp +++ b/fdbserver/storageserver.actor.cpp @@ -4243,7 +4243,7 @@ ACTOR Future fetchChangeFeedApplier(StorageServer* data, } } - // if we were popped while fetching but it didn't pass the fetch version while writing, clean up here + // if we were popped or removed while fetching but it didn't pass the fetch version while writing, clean up here if (versionsFetched > 0 && startVersion < changeFeedInfo->emptyVersion) { ASSERT(firstVersion != invalidVersion); ASSERT(lastVersion != invalidVersion); @@ -4263,7 +4263,8 @@ ACTOR Future fetchChangeFeedApplier(StorageServer* data, .detail("FirstFetchedVersion", firstVersion) .detail("LastFetchedVersion", lastVersion) .detail("VersionsFetched", versionsFetched) - .detail("Existing", existing); + .detail("Existing", existing) + .detail("Removed", changeFeedInfo->removing); return lastVersion; } @@ -4403,17 +4404,16 @@ ACTOR Future> dispatchChangeFeeds(StorageServer try { for (auto& feedId : feedIds) { auto feedIt = data->uidChangeFeed.find(feedId); - // TODO REMOVE this assert once we enable change feed deletion - ASSERT(feedIt != data->uidChangeFeed.end()); - Reference feed = feedIt->second; - feedFetches[feed->id] = fetchChangeFeed(data, feed, beginVersion, endVersion); + // 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); + } } for (auto& feedId : newFeedIds) { auto feedIt = data->uidChangeFeed.find(feedId); - // TODO REMOVE this assert once we enable change feed deletion + // we just read the change feed data map earlier in fetchKeys without yielding, so these feeds must exist ASSERT(feedIt != data->uidChangeFeed.end()); - Reference feed = feedIt->second; - feedFetches[feed->id] = fetchChangeFeed(data, feed, 0, endVersion); + feedFetches[feedIt->second->id] = fetchChangeFeed(data, feedIt->second, 0, endVersion); } loop { @@ -5100,6 +5100,7 @@ void changeServerKeys(StorageServer* data, data->keyChangeFeed.coalesce(f.second.contents()); auto feed = data->uidChangeFeed.find(f.first); if (feed != data->uidChangeFeed.end()) { + feed->second->emptyVersion = version - 1; feed->second->removing = true; feed->second->moved(); feed->second->newMutations.trigger(); From 03d91fd42d4f6fd5888804776b7e08571a52fd8d Mon Sep 17 00:00:00 2001 From: Josh Slocum Date: Fri, 4 Feb 2022 09:36:38 -0600 Subject: [PATCH 211/413] Fixing change feed fetch durability semantics --- fdbserver/storageserver.actor.cpp | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/fdbserver/storageserver.actor.cpp b/fdbserver/storageserver.actor.cpp index fbdbdbc318..f4f2ca5284 100644 --- a/fdbserver/storageserver.actor.cpp +++ b/fdbserver/storageserver.actor.cpp @@ -6017,8 +6017,6 @@ ACTOR Future updateStorage(StorageServer* data) { // to add it back to fetchingChangeFeeds data->fetchingChangeFeeds.insert(info->first); } - } else { - info->second->fetchVersion = invalidVersion; } wait(yield(TaskPriority::UpdateStorage)); } @@ -6080,6 +6078,10 @@ ACTOR Future updateStorage(StorageServer* data) { if (feedFetchVersions[curFeed].second > info->second->durableFetchVersion.get()) { info->second->durableFetchVersion.set(feedFetchVersions[curFeed].second); } + if (feedFetchVersions[curFeed].second == info->second->fetchVersion) { + // haven't fetched anything else since commit started, reset fetch version + info->second->fetchVersion = invalidVersion; + } } curFeed++; } From 0a3608a33b805929c4acac2529997588f96fb676 Mon Sep 17 00:00:00 2001 From: Josh Slocum Date: Fri, 4 Feb 2022 11:03:29 -0600 Subject: [PATCH 212/413] Fixed issue with change feed fetching durability --- fdbserver/storageserver.actor.cpp | 16 ++++++++++------ 1 file changed, 10 insertions(+), 6 deletions(-) diff --git a/fdbserver/storageserver.actor.cpp b/fdbserver/storageserver.actor.cpp index f4f2ca5284..7f4d352b50 100644 --- a/fdbserver/storageserver.actor.cpp +++ b/fdbserver/storageserver.actor.cpp @@ -1784,11 +1784,12 @@ ACTOR Future> getChangeFeedMutations(Stor state Version dequeVersion = data->version.get(); state Version dequeKnownCommit = data->knownCommittedVersion; state Version emptyVersion = feedInfo->emptyVersion; + Version fetchStorageVersion = std::max(feedInfo->fetchVersion, feedInfo->durableFetchVersion.get()); if (DEBUG_SS_CFM(data->thisServerID, req.rangeID, req.begin)) { printf("CFM: SS %s CF %s: SQ %s atLatest=%s, dequeVersion=%lld, emptyVersion=%lld, storageVersion=%lld, " "durableVersion=%lld, " - "fetchVersion=%lld\n", + "fetchStorageVersion=%lld\n", data->thisServerID.toString().substr(0, 4).c_str(), req.rangeID.printable().substr(0, 6).c_str(), streamUID.toString().substr(0, 8).c_str(), @@ -1797,7 +1798,7 @@ ACTOR Future> getChangeFeedMutations(Stor feedInfo->emptyVersion, feedInfo->storageVersion, feedInfo->durableVersion, - feedInfo->fetchVersion); + fetchStorageVersion); } if (req.end > emptyVersion + 1) { @@ -1826,17 +1827,20 @@ ACTOR Future> getChangeFeedMutations(Stor } state bool readDurable = feedInfo->durableVersion != invalidVersion && req.begin <= feedInfo->durableVersion; - state bool readFetched = feedInfo->durableVersion < feedInfo->fetchVersion && req.begin <= feedInfo->fetchVersion; + state bool readFetched = feedInfo->durableVersion < fetchStorageVersion && req.begin <= fetchStorageVersion; state bool readAnyFromDisk = false; if (req.end > emptyVersion + 1 && (readDurable || readFetched)) { - if (readFetched) { + if (readFetched && req.begin > feedInfo->durableFetchVersion.get()) { + // Request needs data that has been written to storage by a change feed fetch, but not committed yet // To not block fetchKeys making normal SS data readable on making change feed data written to storage, we // wait in here instead for all fetched data to become readable from the storage engine. + ASSERT(req.begin <= feedInfo->fetchVersion); TEST(true); // getChangeFeedMutations before fetched data durable - // wait for next commit to write feed data to storage + + // Wait for next commit to write pending feed data to storage wait(feedInfo->durableFetchVersion.whenAtLeast( std::min(feedInfo->fetchVersion, feedInfo->durableFetchVersion.get() + 1))); - // To return control back to updateStorage + // To let update storage finish wait(delay(0)); } RangeResult res = wait( From ddfc301d745c31a15e4074e8cd2492179fc49690 Mon Sep 17 00:00:00 2001 From: Josh Slocum Date: Fri, 4 Feb 2022 16:41:25 -0600 Subject: [PATCH 213/413] Improving memory footprint of change feeds and making it configurable --- fdbclient/DatabaseContext.h | 1 + fdbclient/NativeAPI.actor.cpp | 16 +++++++++---- fdbclient/ServerKnobs.cpp | 2 ++ fdbclient/ServerKnobs.h | 2 ++ fdbclient/StorageServerInterface.h | 4 +++- fdbrpc/fdbrpc.h | 2 +- fdbrpc/genericactors.actor.h | 23 +++++++++--------- fdbserver/BlobWorker.actor.cpp | 38 +++++++++++++++++++++--------- fdbserver/storageserver.actor.cpp | 10 +++++--- 9 files changed, 66 insertions(+), 32 deletions(-) diff --git a/fdbclient/DatabaseContext.h b/fdbclient/DatabaseContext.h index f9ad1a63a8..92a9004c3a 100644 --- a/fdbclient/DatabaseContext.h +++ b/fdbclient/DatabaseContext.h @@ -296,6 +296,7 @@ public: Version begin = 0, Version end = std::numeric_limits::max(), KeyRange range = allKeys, + int replyBufferSize = -1, bool canReadPopped = true); Future> getOverlappingChangeFeeds(KeyRangeRef ranges, Version minVersion); diff --git a/fdbclient/NativeAPI.actor.cpp b/fdbclient/NativeAPI.actor.cpp index 2ddf1cd674..6d7e84489c 100644 --- a/fdbclient/NativeAPI.actor.cpp +++ b/fdbclient/NativeAPI.actor.cpp @@ -7610,6 +7610,7 @@ ACTOR Future mergeChangeFeedStream(Reference db, Key rangeID, Version* begin, Version end, + int replyBufferSize, bool canReadPopped) { state std::vector> fetchers(interfs.size()); state std::vector> onErrors(interfs.size()); @@ -7624,6 +7625,7 @@ ACTOR Future mergeChangeFeedStream(Reference db, req.end = end; req.range = it.second; req.canReadPopped = canReadPopped; + req.replyBufferSize = replyBufferSize / interfs.size(); UID debugID = deterministicRandom()->randomUniqueID(); debugIDs.push_back(debugID); req.debugID = debugID; @@ -7810,6 +7812,7 @@ ACTOR Future singleChangeFeedStream(Reference db, Key rangeID, Version* begin, Version end, + int replyBufferSize, bool canReadPopped) { state Database cx(db); state ChangeFeedStreamRequest req; @@ -7819,6 +7822,7 @@ ACTOR Future singleChangeFeedStream(Reference db, req.end = end; req.range = range; req.canReadPopped = canReadPopped; + req.replyBufferSize = replyBufferSize; req.debugID = debugID; results->streams.clear(); @@ -7859,6 +7863,7 @@ ACTOR Future getChangeFeedStreamActor(Reference db, Version begin, Version end, KeyRange range, + int replyBufferSize, bool canReadPopped) { state Database cx(db); state Span span("NAPI:GetChangeFeedStream"_loc); @@ -7938,11 +7943,13 @@ ACTOR Future getChangeFeedStreamActor(Reference db, interfs.push_back(std::make_pair(locations[i].second->getInterface(chosenLocations[i]), locations[i].first & range)); } - wait(mergeChangeFeedStream(db, interfs, results, rangeID, &begin, end, canReadPopped) || - cx->connectionFileChanged()); + wait( + mergeChangeFeedStream(db, interfs, results, rangeID, &begin, end, replyBufferSize, canReadPopped) || + cx->connectionFileChanged()); } else { StorageServerInterface interf = locations[0].second->getInterface(chosenLocations[0]); - wait(singleChangeFeedStream(db, interf, range, results, rangeID, &begin, end, canReadPopped) || + wait(singleChangeFeedStream( + db, interf, range, results, rangeID, &begin, end, replyBufferSize, canReadPopped) || cx->connectionFileChanged()); } } catch (Error& e) { @@ -7991,9 +7998,10 @@ Future DatabaseContext::getChangeFeedStream(Reference resu Version begin, Version end, KeyRange range, + int replyBufferSize, bool canReadPopped) { return getChangeFeedStreamActor( - Reference::addRef(this), results, rangeID, begin, end, range, canReadPopped); + Reference::addRef(this), results, rangeID, begin, end, range, replyBufferSize, canReadPopped); } ACTOR Future> singleLocationOverlappingChangeFeeds( diff --git a/fdbclient/ServerKnobs.cpp b/fdbclient/ServerKnobs.cpp index fde9d49203..2e0d7658cf 100644 --- a/fdbclient/ServerKnobs.cpp +++ b/fdbclient/ServerKnobs.cpp @@ -649,6 +649,8 @@ void ServerKnobs::initialize(Randomize randomize, ClientKnobs* clientKnobs, IsSi init( FETCH_KEYS_TOO_LONG_TIME_CRITERIA, 300.0 ); init( MAX_STORAGE_COMMIT_TIME, 120.0 ); //The max fsync stall time on the storage server and tlog before marking a disk as failed init( RANGESTREAM_LIMIT_BYTES, 2e6 ); if( randomize && BUGGIFY ) RANGESTREAM_LIMIT_BYTES = 1; + init( CHANGEFEEDSTREAM_LIMIT_BYTES, 1e6 ); if( randomize && BUGGIFY ) CHANGEFEEDSTREAM_LIMIT_BYTES = 1; + init( BLOBWORKERSTATUSSTREAM_LIMIT_BYTES, 1e4 ); if( randomize && BUGGIFY ) BLOBWORKERSTATUSSTREAM_LIMIT_BYTES = 1; init( ENABLE_CLEAR_RANGE_EAGER_READS, true ); init( QUICK_GET_VALUE_FALLBACK, true ); init( QUICK_GET_KEY_VALUES_FALLBACK, true ); diff --git a/fdbclient/ServerKnobs.h b/fdbclient/ServerKnobs.h index 6f1f4a9971..9e02ca63d5 100644 --- a/fdbclient/ServerKnobs.h +++ b/fdbclient/ServerKnobs.h @@ -590,6 +590,8 @@ public: double FETCH_KEYS_TOO_LONG_TIME_CRITERIA; double MAX_STORAGE_COMMIT_TIME; int64_t RANGESTREAM_LIMIT_BYTES; + int64_t CHANGEFEEDSTREAM_LIMIT_BYTES; + int64_t BLOBWORKERSTATUSSTREAM_LIMIT_BYTES; bool ENABLE_CLEAR_RANGE_EAGER_READS; bool QUICK_GET_VALUE_FALLBACK; bool QUICK_GET_KEY_VALUES_FALLBACK; diff --git a/fdbclient/StorageServerInterface.h b/fdbclient/StorageServerInterface.h index 556c8548e0..db6fb9f9e4 100644 --- a/fdbclient/StorageServerInterface.h +++ b/fdbclient/StorageServerInterface.h @@ -715,15 +715,17 @@ struct ChangeFeedStreamRequest { Version begin = 0; Version end = 0; KeyRange range; + int replyBufferSize = -1; bool canReadPopped = true; // TODO REMOVE once BG is correctness clean!! Useful for debugging UID debugID; + ReplyPromiseStream reply; ChangeFeedStreamRequest() {} template void serialize(Ar& ar) { - serializer(ar, rangeID, begin, end, range, reply, spanContext, canReadPopped, debugID, arena); + serializer(ar, rangeID, begin, end, range, reply, spanContext, replyBufferSize, canReadPopped, debugID, arena); } }; diff --git a/fdbrpc/fdbrpc.h b/fdbrpc/fdbrpc.h index 666454ed25..fb726309cb 100644 --- a/fdbrpc/fdbrpc.h +++ b/fdbrpc/fdbrpc.h @@ -500,7 +500,7 @@ public: Future onConnected() { if (connected()) { - return Future(Void()); + return Void(); } if (!queue->onConnect.isValid()) { queue->onConnect = Promise(); diff --git a/fdbrpc/genericactors.actor.h b/fdbrpc/genericactors.actor.h index 11c9c38395..e93825b43b 100644 --- a/fdbrpc/genericactors.actor.h +++ b/fdbrpc/genericactors.actor.h @@ -197,11 +197,6 @@ struct PeerHolder { } }; -ACTOR template -void holdUntilConnected(Future signal, ReplyPromiseStream stream) { - wait(stream.onConnected() || signal); -} - // Implements getReplyStream, this a void actor with the same lifetime as the input ReplyPromiseStream. // Because this actor holds a reference to the stream, normally it would be impossible to know when there are no other // references. To get around this, there is a SAV inside the stream that has one less promise reference than it should @@ -215,13 +210,17 @@ void endStreamOnDisconnect(Future signal, Reference peer = Reference()) { state PeerHolder holder = PeerHolder(peer); stream.setRequestStreamEndpoint(endpoint); - choose { - when(wait(signal)) { stream.sendError(connection_failed()); } - when(wait(stream.getErrorFutureAndDelPromiseRef())) { - // Wait for a response from the server - /*if (!stream.connected()) { - // TODO WANT TO DO holdAndConnected ACTOR HERE INSTEAD! - }*/ + try { + choose { + when(wait(signal)) { stream.sendError(connection_failed()); } + when(wait(stream.getErrorFutureAndDelPromiseRef())) {} + } + } catch (Error& e) { + if (e.code() == error_code_broken_promise) { + // getErrorFutureAndDelPromiseRef returned, wait on stream connect or error + if (!stream.connected()) { + wait(signal || stream.onConnected()); + } } } } diff --git a/fdbserver/BlobWorker.actor.cpp b/fdbserver/BlobWorker.actor.cpp index 6806a7cac7..e64e293f98 100644 --- a/fdbserver/BlobWorker.actor.cpp +++ b/fdbserver/BlobWorker.actor.cpp @@ -177,6 +177,8 @@ struct BlobWorkerData : NonCopyable, ReferenceCounted { Promise doGRVCheck; NotifiedVersion grvVersion; + int changeFeedStreamReplyBufferSize = SERVER_KNOBS->BG_DELTA_FILE_TARGET_BYTES / 2; + BlobWorkerData(UID id, Database db) : id(id), db(db), stats(id, SERVER_KNOBS->WORKER_LOGGING_INTERVAL) {} ~BlobWorkerData() { if (BW_DEBUG) { @@ -1338,12 +1340,18 @@ ACTOR Future blobGranuleUpdateFiles(Reference bwData, startVersion + 1, startState.changeFeedStartVersion, metadata->keyRange, + bwData->changeFeedStreamReplyBufferSize, false); } else { readOldChangeFeed = false; - changeFeedFuture = bwData->db->getChangeFeedStream( - newCFData, cfKey, startVersion + 1, MAX_VERSION, metadata->keyRange, false); + changeFeedFuture = bwData->db->getChangeFeedStream(newCFData, + cfKey, + startVersion + 1, + MAX_VERSION, + metadata->keyRange, + bwData->changeFeedStreamReplyBufferSize, + false); } // Start actors BEFORE setting new change feed data to ensure the change feed data is properly initialized by @@ -1469,8 +1477,13 @@ ACTOR Future blobGranuleUpdateFiles(Reference bwData, Reference newCFData = makeReference(); - changeFeedFuture = bwData->db->getChangeFeedStream( - newCFData, cfKey, startState.changeFeedStartVersion, MAX_VERSION, metadata->keyRange, false); + changeFeedFuture = bwData->db->getChangeFeedStream(newCFData, + cfKey, + startState.changeFeedStartVersion, + MAX_VERSION, + metadata->keyRange, + bwData->changeFeedStreamReplyBufferSize, + false); // Start actors BEFORE setting new change feed data to ensure the change feed data is properly // initialized by the client @@ -1590,6 +1603,7 @@ ACTOR Future blobGranuleUpdateFiles(Reference bwData, cfRollbackVersion + 1, startState.changeFeedStartVersion, metadata->keyRange, + bwData->changeFeedStreamReplyBufferSize, false); } else { @@ -1600,12 +1614,14 @@ ACTOR Future blobGranuleUpdateFiles(Reference bwData, } ASSERT(cfRollbackVersion >= startState.changeFeedStartVersion); - changeFeedFuture = bwData->db->getChangeFeedStream(newCFData, - cfKey, - cfRollbackVersion + 1, - MAX_VERSION, - metadata->keyRange, - false); + changeFeedFuture = + bwData->db->getChangeFeedStream(newCFData, + cfKey, + cfRollbackVersion + 1, + MAX_VERSION, + metadata->keyRange, + bwData->changeFeedStreamReplyBufferSize, + false); } // Start actors BEFORE setting new change feed data to ensure the change feed data @@ -3094,7 +3110,7 @@ ACTOR Future blobWorker(BlobWorkerInterface bwInterf, self->currentManagerStatusStream.get().sendError(connection_failed()); // TODO: pick a reasonable byte limit instead of just piggy-backing - req.reply.setByteLimit(SERVER_KNOBS->RANGESTREAM_LIMIT_BYTES); + req.reply.setByteLimit(SERVER_KNOBS->BLOBWORKERSTATUSSTREAM_LIMIT_BYTES); self->currentManagerStatusStream.set(req.reply); } else { req.reply.sendError(blob_manager_replaced()); diff --git a/fdbserver/storageserver.actor.cpp b/fdbserver/storageserver.actor.cpp index 7f4d352b50..2334ca6ed5 100644 --- a/fdbserver/storageserver.actor.cpp +++ b/fdbserver/storageserver.actor.cpp @@ -2050,7 +2050,11 @@ ACTOR Future changeFeedStreamQ(StorageServer* data, ChangeFeedStreamReques state UID streamUID = req.debugID; state bool removeUID = false; state Optional blockedVersion; - req.reply.setByteLimit(SERVER_KNOBS->RANGESTREAM_LIMIT_BYTES); + if (req.replyBufferSize <= 0) { + req.reply.setByteLimit(SERVER_KNOBS->CHANGEFEEDSTREAM_LIMIT_BYTES); + } else { + req.reply.setByteLimit(req.replyBufferSize); + } wait(delay(0, TaskPriority::DefaultEndpoint)); @@ -4145,8 +4149,8 @@ ACTOR Future fetchChangeFeedApplier(StorageServer* data, } state Reference feedResults = makeReference(); - state Future feed = - data->cx->getChangeFeedStream(feedResults, rangeId, startVersion, endVersion, range, true); + state Future feed = data->cx->getChangeFeedStream( + feedResults, rangeId, startVersion, endVersion, range, SERVER_KNOBS->CHANGEFEEDSTREAM_LIMIT_BYTES, true); // TODO remove debugging eventually? state Version firstVersion = invalidVersion; From 41a274ed9c0aac115f7fd798ba132fa7146f1ff9 Mon Sep 17 00:00:00 2001 From: Josh Slocum Date: Fri, 4 Feb 2022 17:10:14 -0600 Subject: [PATCH 214/413] Fixing bug and refactoring iterative range splitting --- fdbclient/NativeAPI.actor.cpp | 85 +++++++++++++++++++++++++++++++-- fdbclient/NativeAPI.actor.h | 7 ++- fdbserver/BlobManager.actor.cpp | 38 ++++++++------- 3 files changed, 108 insertions(+), 22 deletions(-) diff --git a/fdbclient/NativeAPI.actor.cpp b/fdbclient/NativeAPI.actor.cpp index 6d7e84489c..d3ab67b0f6 100644 --- a/fdbclient/NativeAPI.actor.cpp +++ b/fdbclient/NativeAPI.actor.cpp @@ -6772,11 +6772,88 @@ ACTOR Future setPerpetualStorageWiggle(Database cx, bool enable, LockAware return Void(); } +ACTOR Future splitStorageMetricsStream(Database cx, + KeyRange keys, + StorageMetrics limit, + StorageMetrics estimated, + PromiseStream resultStream) { + state Span span("NAPI:SplitStorageMetricsStream"_loc); + Key beginKey = keys.begin; + resultStream.send(beginKey); + // track used across loops in case one loop does not find any split points + state StorageMetrics used; + loop { + state std::vector>> locations = + wait(getKeyRangeLocations(cx, + KeyRangeRef(beginKey, keys.end), + CLIENT_KNOBS->STORAGE_METRICS_SHARD_LIMIT, + Reverse::False, + &StorageServerInterface::splitMetrics, + TransactionInfo(TaskPriority::DataDistribution, span.context))); + + try { + //TraceEvent("SplitStorageMetrics").detail("Locations", locations.size()); + + state Standalone> results; + state int i = 0; + for (; i < locations.size(); i++) { + SplitMetricsRequest req(locations[i].first, limit, used, estimated, i == locations.size() - 1); + SplitMetricsReply res = wait(loadBalance(locations[i].second->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()); + } + + // only truncate split at end + if (keys.end <= locations.back().first.end && + used.allLessOrEqual(limit * CLIENT_KNOBS->STORAGE_METRICS_UNFAIR_SPLIT_LIMIT) && results.size() > 1) { + results.resize(results.arena(), results.size() - 1); + } + + for (auto& splitKey : results) { + resultStream.send(splitKey); + } + + if (keys.end <= locations.back().first.end) { + resultStream.send(keys.end); + resultStream.sendError(end_of_stream()); + break; + } else { + ASSERT(results.size() > 0); + beginKey = locations.back().first.end; + } + } catch (Error& e) { + if (e.code() == error_code_operation_cancelled) { + throw e; + } + if (e.code() != error_code_wrong_shard_server && e.code() != error_code_all_alternatives_failed) { + TraceEvent(SevError, "SplitStorageMetricsStreamError").error(e); + resultStream.sendError(e); + throw; + } + cx->invalidateCache(keys); + wait(delay(CLIENT_KNOBS->WRONG_SHARD_SERVER_DELAY, TaskPriority::DataDistribution)); + } + } +} + ACTOR Future>> splitStorageMetrics(Database cx, KeyRange keys, StorageMetrics limit, - StorageMetrics estimated, - bool allowPartial) { + StorageMetrics estimated) { state Span span("NAPI:SplitStorageMetrics"_loc); loop { state std::vector>> locations = @@ -6791,7 +6868,7 @@ ACTOR Future>> splitStorageMetrics(Database cx, // 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 && !allowPartial) { + if (locations.size() == CLIENT_KNOBS->STORAGE_METRICS_SHARD_LIMIT) { wait(delay(CLIENT_KNOBS->STORAGE_METRICS_TOO_MANY_SHARDS_DELAY, TaskPriority::DataDistribution)); cx->invalidateCache(keys); } else { @@ -6827,7 +6904,7 @@ ACTOR Future>> splitStorageMetrics(Database cx, results.resize(results.arena(), results.size() - 1); } - if (!allowPartial || keys.end <= locations.back().first.end) { + if (keys.end <= locations.back().first.end) { results.push_back_deep(results.arena(), keys.end); } return results; diff --git a/fdbclient/NativeAPI.actor.h b/fdbclient/NativeAPI.actor.h index e38fbc5421..95cf2b2db4 100644 --- a/fdbclient/NativeAPI.actor.h +++ b/fdbclient/NativeAPI.actor.h @@ -364,10 +364,13 @@ public: int expectedShardCount); // Pass a negative value for `shardLimit` to indicate no limit on the shard number. Future getStorageMetrics(KeyRange const& keys, int shardLimit); + Future splitStorageMetricsStream(PromiseStream resultsStream, + KeyRange const& keys, + StorageMetrics const& limit, + StorageMetrics const& estimated); Future>> splitStorageMetrics(KeyRange const& keys, StorageMetrics const& limit, - StorageMetrics const& estimated, - bool allowPartial = false); + StorageMetrics const& estimated); Future>> getReadHotRanges(KeyRange const& keys); // Try to split the given range into equally sized chunks based on estimated size. diff --git a/fdbserver/BlobManager.actor.cpp b/fdbserver/BlobManager.actor.cpp index 8132d750d4..b4495983c6 100644 --- a/fdbserver/BlobManager.actor.cpp +++ b/fdbserver/BlobManager.actor.cpp @@ -269,9 +269,7 @@ ACTOR Future>> splitRange(Reference SERVER_KNOBS->BG_SNAPSHOT_FILE_TARGET_BYTES || writeHot) { - // printf(" Splitting range\n"); // only split on bytes and write rate - state Standalone> keys; state StorageMetrics splitMetrics; splitMetrics.bytes = SERVER_KNOBS->BG_SNAPSHOT_FILE_TARGET_BYTES; splitMetrics.bytesPerKSecond = SERVER_KNOBS->SHARD_SPLIT_BYTES_PER_KSEC; @@ -284,31 +282,39 @@ ACTOR Future>> splitRange(Reference> newKeys = - wait(tr->getTransaction().splitStorageMetrics(range, splitMetrics, estimated, true)); - ASSERT(!newKeys.empty()); - if (keys.empty()) { - keys = newKeys; - } else { - TEST(true); // large split that requires multiple rounds - // start key was repeated with last request, so don't include it - ASSERT(newKeys[0] == keys.back()); - keys.append_deep(keys.arena(), newKeys.begin() + 1, newKeys.size() - 1); + state PromiseStream resultStream; + state Standalone> keys; + state Future streamFuture = + tr->getTransaction().splitStorageMetricsStream(resultStream, range, splitMetrics, estimated); + loop { + try { + Key k = waitNext(resultStream.getFuture()); + keys.push_back_deep(keys.arena(), k); + } catch (Error& e) { + if (e.code() != error_code_end_of_stream) { + throw; + } + break; } - range = KeyRangeRef(keys.back(), range.end); } + ASSERT(keys.size() >= 2); + ASSERT(keys.front() == range.begin); + ASSERT(keys.back() == range.end); return keys; } else { - // printf(" Not splitting range\n"); + if (BM_DEBUG) { + printf("Not splitting range\n"); + } Standalone> keys; keys.push_back_deep(keys.arena(), range.begin); keys.push_back_deep(keys.arena(), range.end); return keys; } } catch (Error& e) { + if (BM_DEBUG) { + printf("Splitting range got error %s\n", e.name()); + } wait(tr->onError(e)); } } From c695958823d36126d1c719dd46f5333eb3bc5647 Mon Sep 17 00:00:00 2001 From: Josh Slocum Date: Fri, 4 Feb 2022 17:24:21 -0600 Subject: [PATCH 215/413] remove old actor --- fdbrpc/fdbrpc.h | 1 - 1 file changed, 1 deletion(-) diff --git a/fdbrpc/fdbrpc.h b/fdbrpc/fdbrpc.h index fb726309cb..686cc26ea5 100644 --- a/fdbrpc/fdbrpc.h +++ b/fdbrpc/fdbrpc.h @@ -767,7 +767,6 @@ public: FlowTransport::transport().sendUnreliable(SerializeSource(value), getEndpoint(), true); // FIXME: defer sending the message until we know the connection is established endStreamOnDisconnect(disc, p, getEndpoint(), peer); - holdUntilConnected(disc, p); return p; } else { send(value); From 1736eabd33d18625507a9c995e4dfc30cc9736e6 Mon Sep 17 00:00:00 2001 From: Josh Slocum Date: Fri, 4 Feb 2022 18:24:47 -0600 Subject: [PATCH 216/413] fixes for split metrics stream --- fdbclient/NativeAPI.actor.cpp | 28 ++++++++++++++++++---------- fdbclient/NativeAPI.actor.h | 2 +- 2 files changed, 19 insertions(+), 11 deletions(-) diff --git a/fdbclient/NativeAPI.actor.cpp b/fdbclient/NativeAPI.actor.cpp index d3ab67b0f6..fee33a43f5 100644 --- a/fdbclient/NativeAPI.actor.cpp +++ b/fdbclient/NativeAPI.actor.cpp @@ -6772,13 +6772,14 @@ ACTOR Future setPerpetualStorageWiggle(Database cx, bool enable, LockAware return Void(); } -ACTOR Future splitStorageMetricsStream(Database cx, +ACTOR Future splitStorageMetricsStream(PromiseStream resultStream, + Database cx, KeyRange keys, StorageMetrics limit, - StorageMetrics estimated, - PromiseStream resultStream) { + StorageMetrics estimated) { state Span span("NAPI:SplitStorageMetricsStream"_loc); - Key beginKey = keys.begin; + state Key beginKey = keys.begin; + state Key lastKey = beginKey; resultStream.send(beginKey); // track used across loops in case one loop does not find any split points state StorageMetrics used; @@ -6802,9 +6803,8 @@ ACTOR Future splitStorageMetricsStream(Database cx, &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 + if (res.splits.size() && res.splits[0] <= lastKey) { // 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(); } @@ -6825,6 +6825,7 @@ ACTOR Future splitStorageMetricsStream(Database cx, for (auto& splitKey : results) { resultStream.send(splitKey); + lastKey = splitKey; } if (keys.end <= locations.back().first.end) { @@ -6848,6 +6849,14 @@ ACTOR Future splitStorageMetricsStream(Database cx, wait(delay(CLIENT_KNOBS->WRONG_SHARD_SERVER_DELAY, TaskPriority::DataDistribution)); } } + return Void(); +} + +Future Transaction::splitStorageMetricsStream(const PromiseStream& resultStream, + KeyRange const& keys, + StorageMetrics const& limit, + StorageMetrics const& estimated) { + return ::splitStorageMetricsStream(resultStream, cx, keys, limit, estimated); } ACTOR Future>> splitStorageMetrics(Database cx, @@ -6922,9 +6931,8 @@ ACTOR Future>> splitStorageMetrics(Database cx, Future>> Transaction::splitStorageMetrics(KeyRange const& keys, StorageMetrics const& limit, - StorageMetrics const& estimated, - bool allowPartial) { - return ::splitStorageMetrics(cx, keys, limit, estimated, allowPartial); + StorageMetrics const& estimated) { + return ::splitStorageMetrics(cx, keys, limit, estimated); } void Transaction::checkDeferredError() const { diff --git a/fdbclient/NativeAPI.actor.h b/fdbclient/NativeAPI.actor.h index 95cf2b2db4..ef6d738cc2 100644 --- a/fdbclient/NativeAPI.actor.h +++ b/fdbclient/NativeAPI.actor.h @@ -364,7 +364,7 @@ public: int expectedShardCount); // Pass a negative value for `shardLimit` to indicate no limit on the shard number. Future getStorageMetrics(KeyRange const& keys, int shardLimit); - Future splitStorageMetricsStream(PromiseStream resultsStream, + Future splitStorageMetricsStream(PromiseStream const& resultsStream, KeyRange const& keys, StorageMetrics const& limit, StorageMetrics const& estimated); From 4d34ac95459536c7194498c08140c195eb1d83fb Mon Sep 17 00:00:00 2001 From: Josh Slocum Date: Mon, 7 Feb 2022 08:15:45 -0600 Subject: [PATCH 217/413] disabling redwood and memory storage engines for tests for now --- fdbserver/SimulatedCluster.actor.cpp | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/fdbserver/SimulatedCluster.actor.cpp b/fdbserver/SimulatedCluster.actor.cpp index 653531e106..0b1c929df5 100644 --- a/fdbserver/SimulatedCluster.actor.cpp +++ b/fdbserver/SimulatedCluster.actor.cpp @@ -1332,6 +1332,11 @@ void SimulationConfig::setStorageEngine(const TestConfig& testConfig) { } } + // TODO CHANGE BACK BEFORE MERGING!! + // Avoid memory storage engines for now to avoid extra memory pressure of change feeds, and redwood is ocasionally + // hitting a root too large issue with change feeds + storage_engine_type = 0; + switch (storage_engine_type) { case 0: { TEST(true); // Simulated cluster using ssd storage engine From 809ec24184f17cd102b24f80300e273f33ca2021 Mon Sep 17 00:00:00 2001 From: Josh Slocum Date: Mon, 7 Feb 2022 08:19:19 -0600 Subject: [PATCH 218/413] More change feed buffer tuning --- fdbclient/NativeAPI.actor.cpp | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/fdbclient/NativeAPI.actor.cpp b/fdbclient/NativeAPI.actor.cpp index fee33a43f5..4c93b3ab7f 100644 --- a/fdbclient/NativeAPI.actor.cpp +++ b/fdbclient/NativeAPI.actor.cpp @@ -7710,7 +7710,11 @@ ACTOR Future mergeChangeFeedStream(Reference db, req.end = end; req.range = it.second; req.canReadPopped = canReadPopped; + // divide total buffer size among sub-streams, but keep individual streams large enough to be efficient req.replyBufferSize = replyBufferSize / interfs.size(); + if (replyBufferSize != -1 && req.replyBufferSize < CLIENT_KNOBS->REPLY_BYTE_LIMIT) { + req.replyBufferSize = CLIENT_KNOBS->REPLY_BYTE_LIMIT; + } UID debugID = deterministicRandom()->randomUniqueID(); debugIDs.push_back(debugID); req.debugID = debugID; From c48ca9430d73dc4b900bbc85cdaa91550d87ab72 Mon Sep 17 00:00:00 2001 From: Josh Slocum Date: Mon, 7 Feb 2022 10:36:22 -0600 Subject: [PATCH 219/413] Fixes for streaming split keys --- fdbclient/NativeAPI.actor.cpp | 25 +++++++++++++++++-------- 1 file changed, 17 insertions(+), 8 deletions(-) diff --git a/fdbclient/NativeAPI.actor.cpp b/fdbclient/NativeAPI.actor.cpp index 4c93b3ab7f..22f12a0f72 100644 --- a/fdbclient/NativeAPI.actor.cpp +++ b/fdbclient/NativeAPI.actor.cpp @@ -6781,8 +6781,8 @@ ACTOR Future splitStorageMetricsStream(PromiseStream resultStream, state Key beginKey = keys.begin; state Key lastKey = beginKey; resultStream.send(beginKey); - // track used across loops in case one loop does not find any split points - state StorageMetrics used; + // track used across loops + state StorageMetrics globalUsed; loop { state std::vector>> locations = wait(getKeyRangeLocations(cx, @@ -6791,41 +6791,50 @@ ACTOR Future splitStorageMetricsStream(PromiseStream resultStream, Reverse::False, &StorageServerInterface::splitMetrics, TransactionInfo(TaskPriority::DataDistribution, span.context))); - try { //TraceEvent("SplitStorageMetrics").detail("Locations", locations.size()); + state StorageMetrics localUsed = globalUsed; state Standalone> results; state int i = 0; for (; i < locations.size(); i++) { - SplitMetricsRequest req(locations[i].first, limit, used, estimated, i == locations.size() - 1); + SplitMetricsRequest req(locations[i].first, + limit, + localUsed, + estimated, + i == locations.size() - 1 && keys.end <= locations.back().first.end); SplitMetricsReply res = wait(loadBalance(locations[i].second->locations(), &StorageServerInterface::splitMetrics, req, TaskPriority::DataDistribution)); if (res.splits.size() && res.splits[0] <= lastKey) { // split points are out of order, possibly because - // of moving data, throw error to retry + // 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()); + lastKey = res.splits.back(); } - used = res.used; + localUsed = res.used; //TraceEvent("SplitStorageMetricsResult").detail("Used", used.bytes).detail("Location", i).detail("Size", res.splits.size()); } + globalUsed = localUsed; + // only truncate split at end if (keys.end <= locations.back().first.end && - used.allLessOrEqual(limit * CLIENT_KNOBS->STORAGE_METRICS_UNFAIR_SPLIT_LIMIT) && results.size() > 1) { + globalUsed.allLessOrEqual(limit * CLIENT_KNOBS->STORAGE_METRICS_UNFAIR_SPLIT_LIMIT) && + results.size() > 1) { results.resize(results.arena(), results.size() - 1); + lastKey = results.back(); } for (auto& splitKey : results) { resultStream.send(splitKey); - lastKey = splitKey; } if (keys.end <= locations.back().first.end) { From 88cab7fb67635a7498fa52222f0167603bfa79eb Mon Sep 17 00:00:00 2001 From: Josh Slocum Date: Mon, 7 Feb 2022 11:06:23 -0600 Subject: [PATCH 220/413] More change feed fetching improvements and optimizations --- fdbclient/ClientKnobs.cpp | 1 + fdbclient/ClientKnobs.h | 1 + fdbclient/NativeAPI.actor.cpp | 14 ++------- fdbclient/ServerKnobs.cpp | 1 + fdbclient/ServerKnobs.h | 1 + fdbserver/storageserver.actor.cpp | 48 +++++++++++++++++++++++-------- 6 files changed, 42 insertions(+), 24 deletions(-) diff --git a/fdbclient/ClientKnobs.cpp b/fdbclient/ClientKnobs.cpp index bda87873f6..0549de8379 100644 --- a/fdbclient/ClientKnobs.cpp +++ b/fdbclient/ClientKnobs.cpp @@ -78,6 +78,7 @@ void ClientKnobs::initialize(Randomize randomize) { init( CHANGE_FEED_LOCATION_LIMIT, 10000 ); init( CHANGE_FEED_CACHE_SIZE, 100000 ); if( randomize && BUGGIFY ) CHANGE_FEED_CACHE_SIZE = 1; init( CHANGE_FEED_POP_TIMEOUT, 5.0 ); + init( CHANGE_FEED_STREAM_MIN_BYTES, 1e4 ); if( randomize && BUGGIFY ) CHANGE_FEED_STREAM_MIN_BYTES = 1; init( MAX_BATCH_SIZE, 1000 ); if( randomize && BUGGIFY ) MAX_BATCH_SIZE = 1; init( GRV_BATCH_TIMEOUT, 0.005 ); if( randomize && BUGGIFY ) GRV_BATCH_TIMEOUT = 0.1; diff --git a/fdbclient/ClientKnobs.h b/fdbclient/ClientKnobs.h index 32c2a93a69..92fdd33e8c 100644 --- a/fdbclient/ClientKnobs.h +++ b/fdbclient/ClientKnobs.h @@ -77,6 +77,7 @@ public: int64_t CHANGE_FEED_LOCATION_LIMIT; int64_t CHANGE_FEED_CACHE_SIZE; double CHANGE_FEED_POP_TIMEOUT; + int64_t CHANGE_FEED_STREAM_MIN_BYTES; int MAX_BATCH_SIZE; double GRV_BATCH_TIMEOUT; diff --git a/fdbclient/NativeAPI.actor.cpp b/fdbclient/NativeAPI.actor.cpp index 22f12a0f72..c3d7d23835 100644 --- a/fdbclient/NativeAPI.actor.cpp +++ b/fdbclient/NativeAPI.actor.cpp @@ -7721,23 +7721,13 @@ ACTOR Future mergeChangeFeedStream(Reference db, req.canReadPopped = canReadPopped; // divide total buffer size among sub-streams, but keep individual streams large enough to be efficient req.replyBufferSize = replyBufferSize / interfs.size(); - if (replyBufferSize != -1 && req.replyBufferSize < CLIENT_KNOBS->REPLY_BYTE_LIMIT) { - req.replyBufferSize = CLIENT_KNOBS->REPLY_BYTE_LIMIT; + if (replyBufferSize != -1 && req.replyBufferSize < CLIENT_KNOBS->CHANGE_FEED_STREAM_MIN_BYTES) { + req.replyBufferSize = CLIENT_KNOBS->CHANGE_FEED_STREAM_MIN_BYTES; } UID debugID = deterministicRandom()->randomUniqueID(); debugIDs.push_back(debugID); req.debugID = debugID; results->streams.push_back(it.first.changeFeedStream.getReplyStream(req)); - if (debugID.toString().substr(0, 8) == "637bfa4e") { - printf( - "Good example: %s: %p\n", debugID.toString().substr(0, 8).c_str(), results->streams.back().debugAddr()); - printf("\n"); - } - if (debugID.toString().substr(0, 8) == "1ad27675") { - printf( - "Bad example: %s: %p\n", debugID.toString().substr(0, 8).c_str(), results->streams.back().debugAddr()); - printf("\n"); - } } for (auto& it : results->storageData) { diff --git a/fdbclient/ServerKnobs.cpp b/fdbclient/ServerKnobs.cpp index 2e0d7658cf..865d082d21 100644 --- a/fdbclient/ServerKnobs.cpp +++ b/fdbclient/ServerKnobs.cpp @@ -619,6 +619,7 @@ void ServerKnobs::initialize(Randomize randomize, ClientKnobs* clientKnobs, IsSi init( FETCH_KEYS_PARALLELISM_BYTES, 4e6 ); if( randomize && BUGGIFY ) FETCH_KEYS_PARALLELISM_BYTES = 3e6; init( FETCH_KEYS_PARALLELISM, 2 ); init( FETCH_KEYS_LOWER_PRIORITY, 0 ); + init( FETCH_CHANGEFEED_PARALLELISM, 2 ); init( BUGGIFY_BLOCK_BYTES, 10000 ); init( STORAGE_COMMIT_BYTES, 10000000 ); if( randomize && BUGGIFY ) STORAGE_COMMIT_BYTES = 2000000; init( STORAGE_FETCH_BYTES, 2500000 ); if( randomize && BUGGIFY ) STORAGE_FETCH_BYTES = 500000; diff --git a/fdbclient/ServerKnobs.h b/fdbclient/ServerKnobs.h index 9e02ca63d5..965bf53ad7 100644 --- a/fdbclient/ServerKnobs.h +++ b/fdbclient/ServerKnobs.h @@ -560,6 +560,7 @@ public: int FETCH_KEYS_PARALLELISM_BYTES; int FETCH_KEYS_PARALLELISM; int FETCH_KEYS_LOWER_PRIORITY; + int FETCH_CHANGEFEED_PARALLELISM; int BUGGIFY_BLOCK_BYTES; double STORAGE_DURABILITY_LAG_REJECT_THRESHOLD; double STORAGE_DURABILITY_LAG_MIN_RATE; diff --git a/fdbserver/storageserver.actor.cpp b/fdbserver/storageserver.actor.cpp index 2334ca6ed5..de16662757 100644 --- a/fdbserver/storageserver.actor.cpp +++ b/fdbserver/storageserver.actor.cpp @@ -702,6 +702,7 @@ public: FlowLock durableVersionLock; FlowLock fetchKeysParallelismLock; + FlowLock fetchChangeFeedParallelismLock; int64_t fetchKeysBytesBudget; AsyncVar fetchKeysBudgetUsed; std::vector> readyFetchKeys; @@ -808,7 +809,7 @@ public: CounterCollection cc; Counter allQueries, getKeyQueries, getValueQueries, getRangeQueries, getRangeAndFlatMapQueries, getRangeStreamQueries, finishedQueries, lowPriorityQueries, rowsQueried, bytesQueried, watchQueries, - emptyQueries; + emptyQueries, feedRowsQueried, feedBytesQueried; // Bytes of the mutations that have been added to the memory of the storage server. When the data is durable // and cleared from the memory, we do not subtract it but add it to bytesDurable. @@ -824,6 +825,9 @@ public: // and the lengths of both parameters. Counter mutationBytes; + // Bytes fetched by fetchChangeFeed for data movements. + Counter feedBytesFetched; + Counter sampledBytesCleared; // The number of key-value pairs fetched by fetchKeys() Counter kvFetched; @@ -850,17 +854,19 @@ public: getRangeStreamQueries("GetRangeStreamQueries", cc), finishedQueries("FinishedQueries", cc), lowPriorityQueries("LowPriorityQueries", cc), rowsQueried("RowsQueried", cc), bytesQueried("BytesQueried", cc), watchQueries("WatchQueries", cc), emptyQueries("EmptyQueries", cc), + feedRowsQueried("FeedRowsQueried", cc), feedBytesQueried("FeedBytesQueried", cc), bytesInput("BytesInput", cc), bytesDurable("BytesDurable", cc), bytesFetched("BytesFetched", cc), - mutationBytes("MutationBytes", cc), sampledBytesCleared("SampledBytesCleared", cc), - kvFetched("KVFetched", cc), mutations("Mutations", cc), setMutations("SetMutations", cc), - clearRangeMutations("ClearRangeMutations", cc), atomicMutations("AtomicMutations", cc), - updateBatches("UpdateBatches", cc), updateVersions("UpdateVersions", cc), loops("Loops", cc), - fetchWaitingMS("FetchWaitingMS", cc), fetchWaitingCount("FetchWaitingCount", cc), - fetchExecutingMS("FetchExecutingMS", cc), fetchExecutingCount("FetchExecutingCount", cc), - readsRejected("ReadsRejected", cc), wrongShardServer("WrongShardServer", cc), - fetchedVersions("FetchedVersions", cc), fetchesFromLogs("FetchesFromLogs", cc), - quickGetValueHit("QuickGetValueHit", cc), quickGetValueMiss("QuickGetValueMiss", cc), - quickGetKeyValuesHit("QuickGetKeyValuesHit", cc), quickGetKeyValuesMiss("QuickGetKeyValuesMiss", cc), + mutationBytes("MutationBytes", cc), feedBytesFetched("FeedBytesFetched", cc), + sampledBytesCleared("SampledBytesCleared", cc), kvFetched("KVFetched", cc), mutations("Mutations", cc), + setMutations("SetMutations", cc), clearRangeMutations("ClearRangeMutations", cc), + atomicMutations("AtomicMutations", cc), updateBatches("UpdateBatches", cc), + updateVersions("UpdateVersions", cc), loops("Loops", cc), fetchWaitingMS("FetchWaitingMS", cc), + fetchWaitingCount("FetchWaitingCount", cc), fetchExecutingMS("FetchExecutingMS", cc), + fetchExecutingCount("FetchExecutingCount", cc), readsRejected("ReadsRejected", cc), + wrongShardServer("WrongShardServer", cc), fetchedVersions("FetchedVersions", cc), + fetchesFromLogs("FetchesFromLogs", cc), quickGetValueHit("QuickGetValueHit", cc), + quickGetValueMiss("QuickGetValueMiss", cc), quickGetKeyValuesHit("QuickGetKeyValuesHit", cc), + quickGetKeyValuesMiss("QuickGetKeyValuesMiss", cc), readLatencySample("ReadLatencyMetrics", self->thisServerID, SERVER_KNOBS->LATENCY_METRICS_LOGGING_INTERVAL, @@ -878,6 +884,11 @@ public: specialCounter( cc, "FetchKeysFetchActive", [self]() { return self->fetchKeysParallelismLock.activePermits(); }); specialCounter(cc, "FetchKeysWaiting", [self]() { return self->fetchKeysParallelismLock.waiters(); }); + specialCounter(cc, "FetchChangeFeedFetchActive", [self]() { + return self->fetchChangeFeedParallelismLock.activePermits(); + }); + specialCounter( + cc, "FetchChangeFeedWaiting", [self]() { return self->fetchChangeFeedParallelismLock.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; }); @@ -926,6 +937,7 @@ public: numWatches(0), noRecentUpdates(false), lastUpdate(now()), readQueueSizeMetric(LiteralStringRef("StorageServer.ReadQueueSize")), updateEagerReads(nullptr), fetchKeysParallelismLock(SERVER_KNOBS->FETCH_KEYS_PARALLELISM), + fetchChangeFeedParallelismLock(SERVER_KNOBS->FETCH_KEYS_PARALLELISM), fetchKeysBytesBudget(SERVER_KNOBS->STORAGE_FETCH_BYTES), fetchKeysBudgetUsed(false), instanceID(deterministicRandom()->randomUniqueID().first()), shuttingDown(false), behind(false), versionBehind(false), debug_inApplyUpdate(false), debug_lastValidateTime(0), lastBytesInputEBrake(0), @@ -2114,6 +2126,9 @@ ACTOR Future changeFeedStreamQ(StorageServer* data, ChangeFeedStreamReques feedReply.atLatestVersion = atLatest; feedReply.minStreamVersion = minVersion; + data->counters.feedRowsQueried += feedReply.mutations.size(); + data->counters.feedBytesQueried += feedReply.mutations.expectedSize(); + req.reply.send(feedReply); if (feedReply.mutations.back().version == req.end - 1) { req.reply.sendError(end_of_stream()); @@ -4135,7 +4150,11 @@ ACTOR Future fetchChangeFeedApplier(StorageServer* data, Version endVersion, bool existing) { - state Version startVersion = std::max(beginVersion, emptyVersion + 1); + state Version startVersion = beginVersion; + startVersion = std::max(startVersion, emptyVersion + 1); + startVersion = std::max(startVersion, changeFeedInfo->fetchVersion + 1); + startVersion = std::max(startVersion, changeFeedInfo->durableFetchVersion.get() + 1); + ASSERT(startVersion >= 0); if (startVersion >= endVersion) { @@ -4239,6 +4258,7 @@ ACTOR Future fetchChangeFeedApplier(StorageServer* data, localResult = _localResult; } } + data->counters.feedBytesFetched += remoteResult.expectedSize(); data->fetchKeysBytesBudget -= remoteResult.expectedSize(); if (data->fetchKeysBytesBudget <= 0) { data->fetchKeysBudgetUsed.set(true); @@ -4283,6 +4303,10 @@ ACTOR Future fetchChangeFeed(StorageServer* data, Version endVersion) { wait(delay(0)); // allow this actor to be cancelled by removals + // bound active change feed fetches + wait(data->fetchChangeFeedParallelismLock.take(TaskPriority::DefaultYield)); + state FlowLock::Releaser holdingFCFPL(data->fetchChangeFeedParallelismLock); + TraceEvent(SevDebug, "FetchChangeFeed", data->thisServerID) .detail("RangeID", changeFeedInfo->id.printable()) .detail("Range", changeFeedInfo->range.toString()) From 88b1593681af5330edf32ccbfd44250237d57043 Mon Sep 17 00:00:00 2001 From: Josh Slocum Date: Mon, 7 Feb 2022 11:56:14 -0600 Subject: [PATCH 221/413] Removing incorrect assert --- fdbclient/NativeAPI.actor.cpp | 1 - 1 file changed, 1 deletion(-) diff --git a/fdbclient/NativeAPI.actor.cpp b/fdbclient/NativeAPI.actor.cpp index c3d7d23835..1d428f4272 100644 --- a/fdbclient/NativeAPI.actor.cpp +++ b/fdbclient/NativeAPI.actor.cpp @@ -6842,7 +6842,6 @@ ACTOR Future splitStorageMetricsStream(PromiseStream resultStream, resultStream.sendError(end_of_stream()); break; } else { - ASSERT(results.size() > 0); beginKey = locations.back().first.end; } } catch (Error& e) { From 9b3f6fe45257a69467bbf234e1df74d306d2cd0f Mon Sep 17 00:00:00 2001 From: Josh Slocum Date: Mon, 7 Feb 2022 13:25:14 -0600 Subject: [PATCH 222/413] More bugs in streaming split --- fdbclient/NativeAPI.actor.cpp | 11 +++++++---- fdbserver/storageserver.actor.cpp | 6 +++--- 2 files changed, 10 insertions(+), 7 deletions(-) diff --git a/fdbclient/NativeAPI.actor.cpp b/fdbclient/NativeAPI.actor.cpp index 1d428f4272..26759ecde5 100644 --- a/fdbclient/NativeAPI.actor.cpp +++ b/fdbclient/NativeAPI.actor.cpp @@ -6779,7 +6779,7 @@ ACTOR Future splitStorageMetricsStream(PromiseStream resultStream, StorageMetrics estimated) { state Span span("NAPI:SplitStorageMetricsStream"_loc); state Key beginKey = keys.begin; - state Key lastKey = beginKey; + state Key globalLastKey = beginKey; resultStream.send(beginKey); // track used across loops state StorageMetrics globalUsed; @@ -6795,6 +6795,7 @@ ACTOR Future splitStorageMetricsStream(PromiseStream resultStream, //TraceEvent("SplitStorageMetrics").detail("Locations", locations.size()); state StorageMetrics localUsed = globalUsed; + state Key localLastKey = globalLastKey; state Standalone> results; state int i = 0; for (; i < locations.size(); i++) { @@ -6807,7 +6808,8 @@ ACTOR Future splitStorageMetricsStream(PromiseStream resultStream, &StorageServerInterface::splitMetrics, req, TaskPriority::DataDistribution)); - if (res.splits.size() && res.splits[0] <= lastKey) { // split points are out of order, possibly because + if (res.splits.size() && + res.splits[0] <= localLastKey) { // 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(); @@ -6816,7 +6818,7 @@ ACTOR Future splitStorageMetricsStream(PromiseStream resultStream, if (res.splits.size()) { results.append(results.arena(), res.splits.begin(), res.splits.size()); results.arena().dependsOn(res.splits.arena()); - lastKey = res.splits.back(); + localLastKey = res.splits.back(); } localUsed = res.used; @@ -6830,8 +6832,9 @@ ACTOR Future splitStorageMetricsStream(PromiseStream resultStream, globalUsed.allLessOrEqual(limit * CLIENT_KNOBS->STORAGE_METRICS_UNFAIR_SPLIT_LIMIT) && results.size() > 1) { results.resize(results.arena(), results.size() - 1); - lastKey = results.back(); + localLastKey = results.back(); } + globalLastKey = localLastKey; for (auto& splitKey : results) { resultStream.send(splitKey); diff --git a/fdbserver/storageserver.actor.cpp b/fdbserver/storageserver.actor.cpp index de16662757..57ceaff9b8 100644 --- a/fdbserver/storageserver.actor.cpp +++ b/fdbserver/storageserver.actor.cpp @@ -1718,9 +1718,9 @@ MutationsAndVersionRef filterMutations(Arena& arena, } // TODO REMOVE!!! when BG is correctness clean -#define DEBUG_SS_ID "bdd3"_sr -#define DEBUG_SS_CF_ID "b65dda"_sr -#define DEBUG_SS_CF_BEGIN_VERSION 244526755 +#define DEBUG_SS_ID ""_sr +#define DEBUG_SS_CF_ID ""_sr +#define DEBUG_SS_CF_BEGIN_VERSION invalidVersion #define DEBUG_SS_CFM(ssId, cfId, v) \ ssId.toString().substr(0, 4) == DEBUG_SS_ID&& cfId.printable().substr(0, 6) == DEBUG_SS_CF_ID && \ (v >= DEBUG_SS_CF_BEGIN_VERSION || latestVersion == DEBUG_SS_CF_BEGIN_VERSION) From 9885d2cc9060975b4cc93334759f0fac705e4b39 Mon Sep 17 00:00:00 2001 From: Josh Slocum Date: Mon, 7 Feb 2022 16:11:26 -0600 Subject: [PATCH 223/413] More change feed fetch and read race fixes --- fdbserver/storageserver.actor.cpp | 26 ++++++++++++++++++++------ 1 file changed, 20 insertions(+), 6 deletions(-) diff --git a/fdbserver/storageserver.actor.cpp b/fdbserver/storageserver.actor.cpp index 57ceaff9b8..00f0a73f90 100644 --- a/fdbserver/storageserver.actor.cpp +++ b/fdbserver/storageserver.actor.cpp @@ -1801,7 +1801,7 @@ ACTOR Future> getChangeFeedMutations(Stor if (DEBUG_SS_CFM(data->thisServerID, req.rangeID, req.begin)) { printf("CFM: SS %s CF %s: SQ %s atLatest=%s, dequeVersion=%lld, emptyVersion=%lld, storageVersion=%lld, " "durableVersion=%lld, " - "fetchStorageVersion=%lld\n", + "fetchStorageVersion=%lld (%lld, %lld)\n", data->thisServerID.toString().substr(0, 4).c_str(), req.rangeID.printable().substr(0, 6).c_str(), streamUID.toString().substr(0, 8).c_str(), @@ -1810,7 +1810,9 @@ ACTOR Future> getChangeFeedMutations(Stor feedInfo->emptyVersion, feedInfo->storageVersion, feedInfo->durableVersion, - fetchStorageVersion); + fetchStorageVersion, + feedInfo->fetchVersion, + feedInfo->durableFetchVersion.get()); } if (req.end > emptyVersion + 1) { @@ -1839,21 +1841,33 @@ ACTOR Future> getChangeFeedMutations(Stor } state bool readDurable = feedInfo->durableVersion != invalidVersion && req.begin <= feedInfo->durableVersion; - state bool readFetched = feedInfo->durableVersion < fetchStorageVersion && req.begin <= fetchStorageVersion; + state bool readFetched = req.begin <= fetchStorageVersion; state bool readAnyFromDisk = false; if (req.end > emptyVersion + 1 && (readDurable || readFetched)) { - if (readFetched && req.begin > feedInfo->durableFetchVersion.get()) { + if (readFetched && req.begin <= feedInfo->fetchVersion) { // Request needs data that has been written to storage by a change feed fetch, but not committed yet // To not block fetchKeys making normal SS data readable on making change feed data written to storage, we // wait in here instead for all fetched data to become readable from the storage engine. ASSERT(req.begin <= feedInfo->fetchVersion); TEST(true); // getChangeFeedMutations before fetched data durable + if (DEBUG_SS_CFM(data->thisServerID, req.rangeID, req.begin)) { + printf("CFM: SS %s CF %s: waiting on fetch durable up to %lld\n", + data->thisServerID.toString().substr(0, 4).c_str(), + req.rangeID.printable().substr(0, 6).c_str(), + feedInfo->fetchVersion); + } + // Wait for next commit to write pending feed data to storage - wait(feedInfo->durableFetchVersion.whenAtLeast( - std::min(feedInfo->fetchVersion, feedInfo->durableFetchVersion.get() + 1))); + wait(feedInfo->durableFetchVersion.whenAtLeast(feedInfo->fetchVersion)); // To let update storage finish wait(delay(0)); + if (DEBUG_SS_CFM(data->thisServerID, req.rangeID, req.begin)) { + printf("CFM: SS %s CF %s: got fetch durable up to %lld\n", + data->thisServerID.toString().substr(0, 4).c_str(), + req.rangeID.printable().substr(0, 6).c_str(), + feedInfo->durableFetchVersion.get()); + } } RangeResult res = wait( data->storage.readRange(KeyRangeRef(changeFeedDurableKey(req.rangeID, std::max(req.begin, emptyVersion)), From 9aaee145b2bc4533ebdc11089f8207f2ca4ac040 Mon Sep 17 00:00:00 2001 From: Josh Slocum Date: Mon, 7 Feb 2022 17:48:56 -0600 Subject: [PATCH 224/413] Fixing range assignment bug --- fdbserver/BlobManager.actor.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/fdbserver/BlobManager.actor.cpp b/fdbserver/BlobManager.actor.cpp index b4495983c6..3cf1859ef9 100644 --- a/fdbserver/BlobManager.actor.cpp +++ b/fdbserver/BlobManager.actor.cpp @@ -571,10 +571,10 @@ ACTOR Future rangeAssigner(Reference bmData) { // revoke the range for the worker that owns it, not the worker specified in the revoke bmData->addActor.send(doRangeAssignment(bmData, assignment, it.value(), seqNo)); } + bmData->workerAssignments.insert(assignment.keyRange, UID()); } bmData->assignsInProgress.cancel(assignment.keyRange); - bmData->workerAssignments.insert(assignment.keyRange, UID()); } } } From 90a870962d0476281e700ba3255103d73c3ee81f Mon Sep 17 00:00:00 2001 From: Josh Slocum Date: Tue, 8 Feb 2022 13:22:17 -0600 Subject: [PATCH 225/413] Unblocking bounded change feed request in changeFeedClientVersions once it hit end --- fdbserver/storageserver.actor.cpp | 14 ++++++++++---- 1 file changed, 10 insertions(+), 4 deletions(-) diff --git a/fdbserver/storageserver.actor.cpp b/fdbserver/storageserver.actor.cpp index 00f0a73f90..eb4038ffea 100644 --- a/fdbserver/storageserver.actor.cpp +++ b/fdbserver/storageserver.actor.cpp @@ -1749,6 +1749,7 @@ ACTOR Future> getChangeFeedMutations(Stor // TODO REMOVE TraceEvent(SevDebug, "ChangeFeedMutations", data->thisServerID) .detail("FeedID", req.rangeID) + .detail("StreamUID", streamUID) .detail("Range", req.range) .detail("Begin", req.begin) .detail("End", req.end); @@ -1987,6 +1988,7 @@ ACTOR Future> getChangeFeedMutations(Stor for (auto& m : mutations.mutations) { DEBUG_MUTATION("ChangeFeedRead", mutations.version, m, data->thisServerID) .detail("ChangeFeedID", req.rangeID) + .detail("StreamUID", streamUID) .detail("ReqBegin", req.begin) .detail("ReqEnd", req.end) .detail("ReqRange", req.range); @@ -2004,6 +2006,7 @@ ACTOR Future> getChangeFeedMutations(Stor // TODO REMOVE TraceEvent(SevDebug, "ChangeFeedMutationsDone", data->thisServerID) .detail("FeedID", req.rangeID) + .detail("StreamUID", streamUID) .detail("Range", req.range) .detail("Begin", req.begin) .detail("End", req.end) @@ -2125,7 +2128,7 @@ ACTOR Future changeFeedStreamQ(StorageServer* data, ChangeFeedStreamReques auto& clientVersions = data->changeFeedClientVersions[req.reply.getEndpoint().getPrimaryAddress()]; Version minVersion = removeUID ? data->version.get() : data->prevVersion; if (removeUID) { - if (gotAll) { + if (gotAll || req.begin == req.end) { data->changeFeedClientVersions[req.reply.getEndpoint().getPrimaryAddress()].erase(streamUID); removeUID = false; } else { @@ -2144,7 +2147,7 @@ ACTOR Future changeFeedStreamQ(StorageServer* data, ChangeFeedStreamReques data->counters.feedBytesQueried += feedReply.mutations.expectedSize(); req.reply.send(feedReply); - if (feedReply.mutations.back().version == req.end - 1) { + if (req.begin == req.end) { req.reply.sendError(end_of_stream()); return Void(); } @@ -2153,7 +2156,8 @@ ACTOR Future changeFeedStreamQ(StorageServer* data, ChangeFeedStreamReques auto feed = data->uidChangeFeed.find(req.rangeID); if (feed == data->uidChangeFeed.end() || feed->second->removing) { req.reply.sendError(unknown_change_feed()); - return Void(); + // throw to delete from changeFeedClientVersions if present + throw unknown_change_feed(); } state Version emptyBefore = feed->second->emptyVersion; choose { @@ -2164,7 +2168,8 @@ ACTOR Future changeFeedStreamQ(StorageServer* data, ChangeFeedStreamReques auto feed = data->uidChangeFeed.find(req.rangeID); if (feed == data->uidChangeFeed.end() || feed->second->removing) { req.reply.sendError(unknown_change_feed()); - return Void(); + // throw to delete from changeFeedClientVersions if present + throw unknown_change_feed(); } if (emptyBefore != feed->second->emptyVersion && !req.canReadPopped && req.begin <= feed->second->emptyVersion) { @@ -2205,6 +2210,7 @@ ACTOR Future changeFeedVersionUpdateQ(StorageServer* data, ChangeFeedVersi auto& clientVersions = data->changeFeedClientVersions[req.reply.getEndpoint().getPrimaryAddress()]; Version minVersion = data->version.get(); for (auto& it : clientVersions) { + // printf("Blocked client %s @ %lld\n", it.first.toString().substr(0, 8).c_str(), it.second); minVersion = std::min(minVersion, it.second); } req.reply.send(ChangeFeedVersionUpdateReply(minVersion)); From 6adf119e3d7f8ee102856b801a9e68b125962d41 Mon Sep 17 00:00:00 2001 From: Josh Slocum Date: Tue, 8 Feb 2022 13:58:56 -0600 Subject: [PATCH 226/413] Fixing change feed durableFetchVersion not getting updated --- fdbserver/storageserver.actor.cpp | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/fdbserver/storageserver.actor.cpp b/fdbserver/storageserver.actor.cpp index eb4038ffea..63e8b638ff 100644 --- a/fdbserver/storageserver.actor.cpp +++ b/fdbserver/storageserver.actor.cpp @@ -4274,10 +4274,15 @@ ACTOR Future fetchChangeFeedApplier(StorageServer* data, remoteLoc++; } if (localVersion <= remoteVersion) { + // Do this once per wait instead of once per version for efficiency + data->fetchingChangeFeeds.insert(changeFeedInfo->id); Standalone _localResult = waitNext(localResults.getFuture()); localResult = _localResult; } } + // Do this once per wait instead of once per version for efficiency + data->fetchingChangeFeeds.insert(changeFeedInfo->id); + data->counters.feedBytesFetched += remoteResult.expectedSize(); data->fetchKeysBytesBudget -= remoteResult.expectedSize(); if (data->fetchKeysBytesBudget <= 0) { From 713c05bdcd509282a4262fe9a315513ba01c4346 Mon Sep 17 00:00:00 2001 From: Josh Slocum Date: Tue, 8 Feb 2022 14:21:11 -0600 Subject: [PATCH 227/413] Improved availability checking at end of BG workload --- .../workloads/BlobGranuleCorrectnessWorkload.actor.cpp | 10 ++++++++++ 1 file changed, 10 insertions(+) diff --git a/fdbserver/workloads/BlobGranuleCorrectnessWorkload.actor.cpp b/fdbserver/workloads/BlobGranuleCorrectnessWorkload.actor.cpp index d8f0495f40..efca15df5c 100644 --- a/fdbserver/workloads/BlobGranuleCorrectnessWorkload.actor.cpp +++ b/fdbserver/workloads/BlobGranuleCorrectnessWorkload.actor.cpp @@ -808,6 +808,16 @@ struct BlobGranuleCorrectnessWorkload : TestWorkload { wait(self->readFromBlob(cx, self, threadData->directoryRange, readVersion)); result = self->validateResult(threadData, blob, 0, std::numeric_limits::max(), 0, readVersion); finalRowsValidated = blob.first.size(); + + // then if we are still good, do another check at a higher version (not checking data) to ensure availabiity + // of empty versions + if (result) { + if (BGW_DEBUG) { + fmt::print("Directory {0} doing final availability check after data check\n", + threadData->directoryID); + } + wait(self->waitFirstSnapshot(self, cx, threadData, false)); + } } bool initialCheck = result; From d7955a90177ee5710f1e3c8e3f58dec86da474cd Mon Sep 17 00:00:00 2001 From: Josh Slocum Date: Tue, 8 Feb 2022 14:21:47 -0600 Subject: [PATCH 228/413] Much better mechanism for change feed popped checking --- fdbserver/storageserver.actor.cpp | 89 +++++++++++++------------------ 1 file changed, 37 insertions(+), 52 deletions(-) diff --git a/fdbserver/storageserver.actor.cpp b/fdbserver/storageserver.actor.cpp index 63e8b638ff..4f34a13620 100644 --- a/fdbserver/storageserver.actor.cpp +++ b/fdbserver/storageserver.actor.cpp @@ -1093,6 +1093,16 @@ public: } return fun(this, request); } + + Version minFeedVersionForAddress(const NetworkAddress& addr) { + auto& clientVersions = changeFeedClientVersions[addr]; + Version minVersion = version.get(); + for (auto& it : clientVersions) { + // printf("Blocked client %s @ %lld\n", it.first.toString().substr(0, 8).c_str(), it.second); + minVersion = std::min(minVersion, it.second); + } + return minVersion; + } }; const StringRef StorageServer::CurrentRunningFetchKeys::emptyString = LiteralStringRef(""); @@ -1782,16 +1792,6 @@ ACTOR Future> getChangeFeedMutations(Stor req.begin); } - if (!req.canReadPopped && req.begin <= feedInfo->emptyVersion) { - printf("CFM: SS %s CF %s: %s popped! req.begin=%lld, emptyVersion=%lld\n", - data->thisServerID.toString().substr(0, 4).c_str(), - req.rangeID.printable().substr(0, 6).c_str(), - streamUID.toString().substr(0, 8).c_str(), - req.begin, - feedInfo->emptyVersion); - throw change_feed_popped(); - } - // We must copy the mutationDeque when fetching the durable bytes in case mutations are popped from memory while // waiting for the results state Version dequeVersion = data->version.get(); @@ -1941,30 +1941,6 @@ ACTOR Future> getChangeFeedMutations(Stor reply = memoryReply; } - // check if pop happened concurrently with disk read - if (!req.canReadPopped && req.begin <= feedInfo->emptyVersion) { - // this can happen under normal circumstances on one condition: a feed atLatest has to wait for a fetch to read - // from disk, and the feed is popped during that. Since pop can't guarantee cleaning up the entire fetch, if the - // read only got empty mutations after filtering, this is fine - bool ok = atLatest && readFetched && !readDurable && !readAnyFromDisk; - TEST(ok); // feed popped while read waiting for fetch - if (!ok) { - printf("SS %s: CF %s SQ %s popped after read! req.begin=%lld, emptyVersion=%lld, emptyBeforeRead=%lld, " - "atLatest=%s, readFetched=%s, readDurable=%s, readAnyFromDisk=%s\n", - data->thisServerID.toString().substr(0, 4).c_str(), - req.rangeID.printable().substr(0, 6).c_str(), - streamUID.toString().substr(0, 8).c_str(), - req.begin, - feedInfo->emptyVersion, - emptyVersion, - atLatest ? "T" : "F", - readFetched ? "T" : "F", - readDurable ? "T" : "F", - readAnyFromDisk ? "T" : "F"); - throw change_feed_popped(); - } - } - bool gotAll = remainingLimitBytes > 0 && remainingDurableBytes > 0 && data->version.get() == startVersion; Version finalVersion = std::min(req.end - 1, dequeVersion); if ((reply.mutations.empty() || reply.mutations.back().version < finalVersion) && remainingLimitBytes > 0 && @@ -1983,6 +1959,32 @@ ACTOR Future> getChangeFeedMutations(Stor } } + // This check is done just before returning, after all waits in this function + // Check if pop happened concurently + if (!req.canReadPopped && req.begin <= feedInfo->emptyVersion) { + // This can happen under normal circumstances if this part of a change feed got no updates, but then the feed + // was popped. We can check by confirming that the client was sent empty versions as part of another feed's + // response's minStorageVersion, or a ChangeFeedUpdateRequest. If this was the case, we know no updates could + // have happened between req.begin and minVersion. + Version minVersion = data->minFeedVersionForAddress(req.reply.getEndpoint().getPrimaryAddress()); + bool ok = atLatest && minVersion > feedInfo->emptyVersion; + TEST(ok); // feed popped while valid read waiting + TEST(!ok); // feed popped while invalid read waiting + if (!ok) { + printf("SS %s: CF %s SQ %s popped after read! req.begin=%lld, emptyVersion=%lld, emptyBeforeRead=%lld, " + "atLatest=%s, minVersionSent=%lld\n", + data->thisServerID.toString().substr(0, 4).c_str(), + req.rangeID.printable().substr(0, 6).c_str(), + streamUID.toString().substr(0, 8).c_str(), + req.begin, + feedInfo->emptyVersion, + emptyVersion, + atLatest ? "T" : "F", + minVersion); + throw change_feed_popped(); + } + } + if (MUTATION_TRACKING_ENABLED) { for (auto& mutations : reply.mutations) { for (auto& m : mutations.mutations) { @@ -2171,14 +2173,6 @@ ACTOR Future changeFeedStreamQ(StorageServer* data, ChangeFeedStreamReques // throw to delete from changeFeedClientVersions if present throw unknown_change_feed(); } - if (emptyBefore != feed->second->emptyVersion && !req.canReadPopped && - req.begin <= feed->second->emptyVersion) { - // Change feed was popped with no new mutations, update its begin version to skip those versions so - // it doesn't get change_feed_popped. This is safe because change_feed_popped is to ensure an old - // read can't miss mutations from a change feed stream, and this read is guaranteed not to (it was - // caught up before the pop, and trigger wasn't called on any new mutations before the pop) - req.begin = feed->second->emptyVersion + 1; - } } else { blockedVersion = feedReply.mutations.back().version; } @@ -2207,12 +2201,7 @@ ACTOR Future changeFeedStreamQ(StorageServer* data, ChangeFeedStreamReques ACTOR Future changeFeedVersionUpdateQ(StorageServer* data, ChangeFeedVersionUpdateRequest req) { wait(data->version.whenAtLeast(req.minVersion)); wait(delay(0)); - auto& clientVersions = data->changeFeedClientVersions[req.reply.getEndpoint().getPrimaryAddress()]; - Version minVersion = data->version.get(); - for (auto& it : clientVersions) { - // printf("Blocked client %s @ %lld\n", it.first.toString().substr(0, 8).c_str(), it.second); - minVersion = std::min(minVersion, it.second); - } + Version minVersion = data->minFeedVersionForAddress(req.reply.getEndpoint().getPrimaryAddress()); req.reply.send(ChangeFeedVersionUpdateReply(minVersion)); return Void(); } @@ -4152,8 +4141,6 @@ ACTOR Future changeFeedPopQ(StorageServer* self, ChangeFeedPopRequest req) feed->second->fetchVersion = invalidVersion; } } - // wake up requests that didn't get any mutations since the last pop to update their beginVersion - feed->second->newMutations.trigger(); wait(self->durableVersion.whenAtLeast(durableVersion)); } req.reply.send(Void()); @@ -5447,8 +5434,6 @@ private: addMutationToLog = true; } feed->second->stopped = (status == ChangeFeedStatus::CHANGE_FEED_STOP); - // wake up requests that didn't get any mutations since the last pop to update their beginVersion - feed->second->newMutations.trigger(); } else if (status == ChangeFeedStatus::CHANGE_FEED_CREATE) { TraceEvent(SevDebug, "CreatingChangeFeed", data->thisServerID) .detail("RangeID", changeFeedId.printable()) From b385db68a4a558dae60da59d1e720c85bb7e6ed6 Mon Sep 17 00:00:00 2001 From: Josh Slocum Date: Tue, 8 Feb 2022 16:47:44 -0600 Subject: [PATCH 229/413] capped time and total data for BGVerifySmall tests to avoid OOM --- fdbserver/workloads/WriteDuringRead.actor.cpp | 17 +++++++++++++++-- tests/fast/BlobGranuleVerifySmall.toml | 13 +++++++------ tests/fast/BlobGranuleVerifySmallClean.toml | 5 +++-- 3 files changed, 25 insertions(+), 10 deletions(-) diff --git a/fdbserver/workloads/WriteDuringRead.actor.cpp b/fdbserver/workloads/WriteDuringRead.actor.cpp index c76900b36a..3ddc775617 100644 --- a/fdbserver/workloads/WriteDuringRead.actor.cpp +++ b/fdbserver/workloads/WriteDuringRead.actor.cpp @@ -45,6 +45,9 @@ struct WriteDuringReadWorkload : TestWorkload { bool useSystemKeys; std::string keyPrefix; int64_t maximumTotalData; + int64_t maximumDataWritten; + + int64_t dataWritten = 0; bool success; Database extraDB; @@ -57,6 +60,8 @@ struct WriteDuringReadWorkload : TestWorkload { numOps = getOption(options, LiteralStringRef("numOps"), 21); rarelyCommit = getOption(options, LiteralStringRef("rarelyCommit"), false); maximumTotalData = getOption(options, LiteralStringRef("maximumTotalData"), 3e6); + maximumDataWritten = + getOption(options, LiteralStringRef("maximumDataWritten"), std::numeric_limits::max()); minNode = getOption(options, LiteralStringRef("minNode"), 0); useSystemKeys = getOption(options, LiteralStringRef("useSystemKeys"), deterministicRandom()->random01() < 0.5); // TODO CHANGE BACK!! @@ -75,6 +80,7 @@ struct WriteDuringReadWorkload : TestWorkload { nodes = deterministicRandom()->randomInt(1, 4 << deterministicRandom()->randomInt(0, 20)); } + dataWritten = 0; int newNodes = std::min(nodes, maximumTotalData / (getKeyForIndex(nodes).size() + valueSizeRange.second)); minNode = std::max(minNode, nodes - newNodes); nodes = newNodes; @@ -536,11 +542,13 @@ ACTOR Future commitAndUpdateMemory(ReadYourWritesTransaction* tr, } } + state int64_t txnSize = tr->getApproximateSize(); state std::map committedDB = self->memoryDatabase; *doingCommit = true; wait(tr->commit()); *doingCommit = false; self->finished.trigger(); + self->dataWritten += txnSize; if (readYourWritesDisabled) tr->setOption(FDBTransactionOptions::READ_YOUR_WRITES_DISABLE); @@ -591,9 +599,10 @@ ACTOR Future loadAndRun(Database cx, WriteDuringReadWorkload* self) { for (; i < self->nodes; i += keysPerBatch) { state Transaction tr(cx); loop { - if (now() - startTime > self->testDuration) + if (now() - startTime > self->testDuration || self->dataWritten >= self->maximumDataWritten) return Void(); try { + state int64_t txnSize = 0; if (i == 0) { tr.setOption(FDBTransactionOptions::ACCESS_SYSTEM_KEYS); tr.addWriteConflictRange( @@ -618,10 +627,14 @@ ACTOR Future loadAndRun(Database cx, WriteDuringReadWorkload* self) { value = value.substr(0, std::min(value.size(), CLIENT_KNOBS->VALUE_SIZE_LIMIT)); self->memoryDatabase[key] = value; tr.set(key, value); + int64_t rowSize = key.expectedSize() + value.expectedSize(); + txnSize += rowSize; } } } wait(tr.commit()); + self->dataWritten += txnSize; + //TraceEvent("WDRInitBatch").detail("I", i).detail("CommittedVersion", tr.getCommittedVersion()); break; } catch (Error& e) { @@ -648,7 +661,7 @@ ACTOR Future loadAndRun(Database cx, WriteDuringReadWorkload* self) { throw; break; } - if (now() - startTime > self->testDuration) + if (now() - startTime > self->testDuration || self->dataWritten >= self->maximumDataWritten) return Void(); } } diff --git a/tests/fast/BlobGranuleVerifySmall.toml b/tests/fast/BlobGranuleVerifySmall.toml index 60296446eb..b3852c428f 100644 --- a/tests/fast/BlobGranuleVerifySmall.toml +++ b/tests/fast/BlobGranuleVerifySmall.toml @@ -6,33 +6,34 @@ testTitle = 'BlobGranuleVerifySmall' [[test.workload]] testName = 'WriteDuringRead' - testDuration = 120.0 + testDuration = 60.0 useSystemKeys = false + maximumDataWritten=50000000 [[test.workload]] testName = 'BlobGranuleVerifier' - testDuration = 120.0 + testDuration = 60.0 [[test.workload]] testName = 'RandomClogging' - testDuration = 120.0 + testDuration = 60.0 [[test.workload]] testName = 'Rollback' meanDelay = 30.0 - testDuration = 120.0 + testDuration = 60.0 [[test.workload]] testName = 'Attrition' machinesToKill = 10 machinesToLeave = 3 reboot = true - testDuration = 120.0 + testDuration = 60.0 [[test.workload]] testName = 'Attrition' machinesToKill = 10 machinesToLeave = 3 reboot = true - testDuration = 120.0 + testDuration = 60.0 diff --git a/tests/fast/BlobGranuleVerifySmallClean.toml b/tests/fast/BlobGranuleVerifySmallClean.toml index 0a72412d22..9378ead90c 100644 --- a/tests/fast/BlobGranuleVerifySmallClean.toml +++ b/tests/fast/BlobGranuleVerifySmallClean.toml @@ -6,9 +6,10 @@ testTitle = 'BlobGranuleVerifySmallClean' [[test.workload]] testName = 'WriteDuringRead' - testDuration = 120.0 + testDuration = 60.0 useSystemKeys = false + maximumDataWritten=50000000 [[test.workload]] testName = 'BlobGranuleVerifier' - testDuration = 120.0 + testDuration = 60.0 From f05a589669712d4ba65de77a4bc1e71d9c9f6a30 Mon Sep 17 00:00:00 2001 From: Josh Slocum Date: Tue, 8 Feb 2022 17:17:20 -0600 Subject: [PATCH 230/413] Fixing blob manager recovery/blob worker status stream races --- fdbserver/BlobManager.actor.cpp | 29 +++++++++++++++++++++-------- fdbserver/BlobWorker.actor.cpp | 6 ++++++ 2 files changed, 27 insertions(+), 8 deletions(-) diff --git a/fdbserver/BlobManager.actor.cpp b/fdbserver/BlobManager.actor.cpp index 3cf1859ef9..0a34004c4f 100644 --- a/fdbserver/BlobManager.actor.cpp +++ b/fdbserver/BlobManager.actor.cpp @@ -223,6 +223,7 @@ struct BlobManagerData : NonCopyable, ReferenceCounted { std::set recruitingLocalities; // the addrs of the workers being recruited on AsyncVar recruitingStream; Promise foundBlobWorkers; + Promise doneRecovering; int64_t epoch = -1; int64_t seqNo = 1; @@ -1158,6 +1159,9 @@ ACTOR Future monitorBlobWorkerStatus(Reference bmData, Bl // outer loop handles reconstructing stream if it got a retryable error // do backoff, we can get a lot of retries in a row + // wait for blob manager to be done recovering, so it has initial granule mapping and worker data + wait(bmData->doneRecovering.getFuture()); + // TODO knob? state double backoff = 0.1; loop { @@ -1189,12 +1193,6 @@ ACTOR Future monitorBlobWorkerStatus(Reference bmData, Bl if (bmData->iAmReplaced.canBeSet()) { bmData->iAmReplaced.send(Void()); } - } else if (rep.epoch < bmData->epoch) { - // TODO: revoke the range from that worker? and send optimistic halt req to other (zombie) BM? - // it's optimistic because such a BM is not necessarily a zombie. it could have gotten killed - // properly but the BW that sent this reply was behind (i.e. it started the req when the old BM - // was in charge and finished by the time the new BM took over) - continue; } // TODO maybe this won't be true eventually, but right now the only time the blob worker reports @@ -1206,6 +1204,16 @@ ACTOR Future monitorBlobWorkerStatus(Reference bmData, Bl if (!(currGranuleAssignment.begin() == rep.granuleRange.begin && currGranuleAssignment.end() == rep.granuleRange.end && currGranuleAssignment.cvalue() == bwInterf.id())) { + if (BM_DEBUG) { + fmt::print( + "Manager {0} ignoring status from BW {1} for granule [{2} - {3}) since BW {4} owns it.\n", + bmData->epoch, + bwInterf.id().toString().substr(0, 5), + rep.granuleRange.begin.printable(), + rep.granuleRange.end.printable(), + currGranuleAssignment.cvalue().toString().substr(0, 5)); + } + // FIXME: could send revoke request continue; } @@ -1221,10 +1229,12 @@ ACTOR Future monitorBlobWorkerStatus(Reference bmData, Bl } } else { if (BM_DEBUG) { - fmt::print("Manager {0} evaluating [{1} - {2}) for split\n", + fmt::print("Manager {0} evaluating [{1} - {2}) @ ({3}, {4}) for split\n", bmData->epoch, rep.granuleRange.begin.printable().c_str(), - rep.granuleRange.end.printable().c_str()); + rep.granuleRange.end.printable().c_str(), + rep.epoch, + rep.seqno); } lastSeenSeqno.insert(rep.granuleRange, std::pair(rep.epoch, rep.seqno)); bmData->addActor.send(maybeSplitRange(bmData, @@ -1836,6 +1846,9 @@ ACTOR Future recoverBlobManager(Reference bmData) { .detail("Assigned", explicitAssignments) .detail("Revoked", outOfDateAssignments.size()); + ASSERT(bmData->doneRecovering.canBeSet()); + bmData->doneRecovering.send(Void()); + return Void(); } diff --git a/fdbserver/BlobWorker.actor.cpp b/fdbserver/BlobWorker.actor.cpp index e64e293f98..8d4a6feca4 100644 --- a/fdbserver/BlobWorker.actor.cpp +++ b/fdbserver/BlobWorker.actor.cpp @@ -161,6 +161,7 @@ struct BlobWorkerData : NonCopyable, ReferenceCounted { int64_t currentManagerEpoch = -1; AsyncVar> currentManagerStatusStream; + bool statusStreamInitialized = false; // FIXME: refactor out the parts of this that are just for interacting with blob stores from the backup business // logic @@ -848,6 +849,10 @@ ACTOR Future checkSplitAndReSnapshot(Reference bw state int64_t statusSeqno = metadata->continueSeqno; // TODO its own knob or something better? This is wrong in case of rollbacks state bool writeHot = versionsSinceLastSnapshot <= SERVER_KNOBS->MAX_READ_TRANSACTION_LIFE_VERSIONS; + + while (!bwData->statusStreamInitialized) { + wait(bwData->currentManagerStatusStream.onChange()); + } loop { loop { try { @@ -3111,6 +3116,7 @@ ACTOR Future blobWorker(BlobWorkerInterface bwInterf, // TODO: pick a reasonable byte limit instead of just piggy-backing req.reply.setByteLimit(SERVER_KNOBS->BLOBWORKERSTATUSSTREAM_LIMIT_BYTES); + self->statusStreamInitialized = true; self->currentManagerStatusStream.set(req.reply); } else { req.reply.sendError(blob_manager_replaced()); From 927729610274c2f9701372d3aca2ab30e03acf99 Mon Sep 17 00:00:00 2001 From: Josh Slocum Date: Tue, 8 Feb 2022 17:47:08 -0600 Subject: [PATCH 231/413] Set doneRecovering for first blob manager --- fdbserver/BlobManager.actor.cpp | 1 + fdbserver/BlobWorker.actor.cpp | 7 +++---- 2 files changed, 4 insertions(+), 4 deletions(-) diff --git a/fdbserver/BlobManager.actor.cpp b/fdbserver/BlobManager.actor.cpp index 0a34004c4f..51fdcbaf1a 100644 --- a/fdbserver/BlobManager.actor.cpp +++ b/fdbserver/BlobManager.actor.cpp @@ -1457,6 +1457,7 @@ ACTOR Future recoverBlobManager(Reference bmData) { // skip the rest of the algorithm for the first blob manager if (bmData->epoch == 1) { + bmData->doneRecovering.send(Void()); return Void(); } diff --git a/fdbserver/BlobWorker.actor.cpp b/fdbserver/BlobWorker.actor.cpp index 8d4a6feca4..5d1b17cc20 100644 --- a/fdbserver/BlobWorker.actor.cpp +++ b/fdbserver/BlobWorker.actor.cpp @@ -826,6 +826,9 @@ ACTOR Future checkSplitAndReSnapshot(Reference bw int64_t versionsSinceLastSnapshot) { BlobFileIndex lastDeltaIdx = wait(lastDeltaBeforeSnapshot); + while (!bwData->statusStreamInitialized) { + wait(bwData->currentManagerStatusStream.onChange()); + } state Version reSnapshotVersion = lastDeltaIdx.version; wait(delay(0, TaskPriority::BlobWorkerUpdateFDB)); @@ -849,10 +852,6 @@ ACTOR Future checkSplitAndReSnapshot(Reference bw state int64_t statusSeqno = metadata->continueSeqno; // TODO its own knob or something better? This is wrong in case of rollbacks state bool writeHot = versionsSinceLastSnapshot <= SERVER_KNOBS->MAX_READ_TRANSACTION_LIFE_VERSIONS; - - while (!bwData->statusStreamInitialized) { - wait(bwData->currentManagerStatusStream.onChange()); - } loop { loop { try { From 4d77932deff99825c39405c0c916904be0483b50 Mon Sep 17 00:00:00 2001 From: Josh Slocum Date: Wed, 9 Feb 2022 08:05:20 -0600 Subject: [PATCH 232/413] fixed reordering --- fdbserver/BlobWorker.actor.cpp | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/fdbserver/BlobWorker.actor.cpp b/fdbserver/BlobWorker.actor.cpp index 5d1b17cc20..b80fe174fd 100644 --- a/fdbserver/BlobWorker.actor.cpp +++ b/fdbserver/BlobWorker.actor.cpp @@ -826,10 +826,11 @@ ACTOR Future checkSplitAndReSnapshot(Reference bw int64_t versionsSinceLastSnapshot) { BlobFileIndex lastDeltaIdx = wait(lastDeltaBeforeSnapshot); + state Version reSnapshotVersion = lastDeltaIdx.version; while (!bwData->statusStreamInitialized) { wait(bwData->currentManagerStatusStream.onChange()); } - state Version reSnapshotVersion = lastDeltaIdx.version; + wait(delay(0, TaskPriority::BlobWorkerUpdateFDB)); if (BW_DEBUG) { From 0fb512489e71106e7a4981d487014159aaf1880d Mon Sep 17 00:00:00 2001 From: Josh Slocum Date: Wed, 9 Feb 2022 09:33:57 -0600 Subject: [PATCH 233/413] Removing incorrect epoch check --- fdbserver/BlobWorker.actor.cpp | 7 ------- 1 file changed, 7 deletions(-) diff --git a/fdbserver/BlobWorker.actor.cpp b/fdbserver/BlobWorker.actor.cpp index b80fe174fd..8741a4921d 100644 --- a/fdbserver/BlobWorker.actor.cpp +++ b/fdbserver/BlobWorker.actor.cpp @@ -871,13 +871,6 @@ ACTOR Future checkSplitAndReSnapshot(Reference bw } } - // TODO: figure out why the status stream on change isn't working - // We could just do something like statusEpoch, save down the original status stream - // and compare it to the current one - if (statusEpoch < bwData->currentManagerEpoch) { - break; - } - choose { when(wait(bwData->currentManagerStatusStream.onChange())) {} when(wait(metadata->resumeSnapshot.getFuture())) { break; } From f4c235f1340375d307202dbc1de2ccc738c2535c Mon Sep 17 00:00:00 2001 From: Josh Slocum Date: Wed, 9 Feb 2022 09:33:13 -0600 Subject: [PATCH 234/413] propagating change_feed_popped correctly --- fdbclient/NativeAPI.actor.cpp | 8 ++++++-- fdbserver/BlobWorker.actor.cpp | 5 ++++- 2 files changed, 10 insertions(+), 3 deletions(-) diff --git a/fdbclient/NativeAPI.actor.cpp b/fdbclient/NativeAPI.actor.cpp index 26759ecde5..8440759bd8 100644 --- a/fdbclient/NativeAPI.actor.cpp +++ b/fdbclient/NativeAPI.actor.cpp @@ -8044,7 +8044,7 @@ ACTOR Future getChangeFeedStreamActor(Reference db, } } catch (Error& e) { fmt::print("CFNA error {}\n", e.name()); - if (e.code() == error_code_actor_cancelled) { + if (e.code() == error_code_actor_cancelled || e.code() == error_code_change_feed_popped) { for (auto& it : results->storageData) { if (it->debugGetReferenceCount() == 2) { db->changeFeedUpdaters.erase(it->interfToken); @@ -8052,7 +8052,11 @@ ACTOR Future getChangeFeedStreamActor(Reference db, } results->streams.clear(); results->storageData.clear(); - results->refresh.sendError(change_feed_cancelled()); + if (e.code() == error_code_change_feed_popped) { + results->refresh.sendError(change_feed_popped()); + } else { + results->refresh.sendError(change_feed_cancelled()); + } throw; } // TODO REMOVE diff --git a/fdbserver/BlobWorker.actor.cpp b/fdbserver/BlobWorker.actor.cpp index 8741a4921d..3790b1c0fc 100644 --- a/fdbserver/BlobWorker.actor.cpp +++ b/fdbserver/BlobWorker.actor.cpp @@ -1134,7 +1134,7 @@ ACTOR Future waitOnCFVersion(Reference metadata, if (DEBUG_BW_VERSION(original) && DEBUG_WAIT_VERSION_COMMITTED) { fmt::print("WVC {0}: got error {1} \n", original, e.name()); } - if (e.code() == error_code_operation_cancelled) { + if (e.code() == error_code_operation_cancelled || e.code() == error_code_change_feed_popped) { throw e; } @@ -2324,6 +2324,9 @@ ACTOR Future handleBlobGranuleFileRequest(Reference bwData } } catch (Error& e) { // we can get change feed cancelled from whenAtLeast. This is effectively + if (e.code() == error_code_change_feed_popped) { + throw wrong_shard_server(); + } if (e.code() != error_code_change_feed_cancelled) { throw e; } From 9189ea0448394cdcda3840480513d2767ff2c13a Mon Sep 17 00:00:00 2001 From: Josh Slocum Date: Wed, 9 Feb 2022 15:37:00 -0600 Subject: [PATCH 235/413] Fixed SS restart and mutation log race --- fdbserver/storageserver.actor.cpp | 9 +++++++++ 1 file changed, 9 insertions(+) diff --git a/fdbserver/storageserver.actor.cpp b/fdbserver/storageserver.actor.cpp index 4f34a13620..9c9c245f72 100644 --- a/fdbserver/storageserver.actor.cpp +++ b/fdbserver/storageserver.actor.cpp @@ -4515,6 +4515,15 @@ ACTOR Future fetchKeys(StorageServer* data, AddingShard* shard) { // This allows adding->start() to be called inline with CSK. wait(data->coreStarted.getFuture() && delay(0)); + // On SS Reboot, durableVersion == latestVersion, so any mutations we add to the mutation log would be skipped if + // added before latest version advances. + // To ensure this doesn't happen, we wait for version to increase by one if this fetchKeys was initiated by a + // changeServerKeys from restoreDurableState + if (data->version.get() == data->durableVersion.get()) { + wait(data->version.whenAtLeast(data->version.get() + 1)); + wait(delay(0)); + } + try { DEBUG_KEY_RANGE("fetchKeysBegin", data->version.get(), shard->keys, data->thisServerID); From c8cd8c06229e836d15bf7e550a4102e9509d7d87 Mon Sep 17 00:00:00 2001 From: Josh Slocum Date: Wed, 9 Feb 2022 15:35:49 -0600 Subject: [PATCH 236/413] Adding request timeout for blob worker --- fdbclient/BlobWorkerCommon.h | 4 +++- fdbclient/ServerKnobs.cpp | 3 ++- fdbclient/ServerKnobs.h | 1 + fdbserver/BlobWorker.actor.cpp | 23 ++++++++++++++++++++++- 4 files changed, 28 insertions(+), 3 deletions(-) diff --git a/fdbclient/BlobWorkerCommon.h b/fdbclient/BlobWorkerCommon.h index bc4c2a687c..5910dfd9f3 100644 --- a/fdbclient/BlobWorkerCommon.h +++ b/fdbclient/BlobWorkerCommon.h @@ -37,6 +37,7 @@ struct BlobWorkerStats { Counter readReqDeltaBytesReturned; Counter commitVersionChecks; Counter granuleUpdateErrors; + Counter granuleRequestTimeouts; int numRangesAssigned; int mutationBytesBuffered; @@ -57,7 +58,8 @@ struct BlobWorkerStats { wrongShardServer("WrongShardServer", cc), changeFeedInputBytes("RangeFeedInputBytes", cc), readReqTotalFilesReturned("ReadReqTotalFilesReturned", cc), readReqDeltaBytesReturned("ReadReqDeltaBytesReturned", cc), commitVersionChecks("CommitVersionChecks", cc), - granuleUpdateErrors("GranuleUpdateErrors", cc), numRangesAssigned(0), mutationBytesBuffered(0) { + granuleUpdateErrors("GranuleUpdateErrors", cc), granuleRequestTimeouts("GranuleRequestTimeouts", cc), + numRangesAssigned(0), mutationBytesBuffered(0) { 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/ServerKnobs.cpp b/fdbclient/ServerKnobs.cpp index 865d082d21..28c677f564 100644 --- a/fdbclient/ServerKnobs.cpp +++ b/fdbclient/ServerKnobs.cpp @@ -784,7 +784,7 @@ void ServerKnobs::initialize(Randomize randomize, ClientKnobs* clientKnobs, IsSi init( LATENCY_SAMPLE_SIZE, 100000 ); init( LATENCY_METRICS_LOGGING_INTERVAL, 60.0 ); - // Blob granlues + // Blob granules init( BG_URL, isSimulated ? "file://fdbblob/" : "" ); // TODO: store in system key space, eventually init( BG_SNAPSHOT_FILE_TARGET_BYTES, 10000000 ); if( randomize && BUGGIFY ) { deterministicRandom()->random01() < 0.1 ? BG_SNAPSHOT_FILE_TARGET_BYTES /= 100 : BG_SNAPSHOT_FILE_TARGET_BYTES /= 10; } init( BG_DELTA_BYTES_BEFORE_COMPACT, BG_SNAPSHOT_FILE_TARGET_BYTES/2 ); @@ -792,6 +792,7 @@ void ServerKnobs::initialize(Randomize randomize, ClientKnobs* clientKnobs, IsSi init( BG_PRUNE_TIMEOUT, 60*60); 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 ); // clang-format on diff --git a/fdbclient/ServerKnobs.h b/fdbclient/ServerKnobs.h index 965bf53ad7..1d2c326cd9 100644 --- a/fdbclient/ServerKnobs.h +++ b/fdbclient/ServerKnobs.h @@ -748,6 +748,7 @@ public: double BG_PRUNE_TIMEOUT; 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 double BLOB_WORKERLIST_FETCH_INTERVAL; ServerKnobs(Randomize, ClientKnobs*, IsSimulated); diff --git a/fdbserver/BlobWorker.actor.cpp b/fdbserver/BlobWorker.actor.cpp index 3790b1c0fc..d5c6ccd7a4 100644 --- a/fdbserver/BlobWorker.actor.cpp +++ b/fdbserver/BlobWorker.actor.cpp @@ -2139,7 +2139,7 @@ ACTOR Future waitForVersion(Reference metadata, Version v return Void(); } -ACTOR Future handleBlobGranuleFileRequest(Reference bwData, BlobGranuleFileRequest req) { +ACTOR Future doBlobGranuleFileRequest(Reference bwData, BlobGranuleFileRequest req) { if (BW_REQUEST_DEBUG || DEBUG_BW_WAIT_VERSION == req.readVersion) { fmt::print("BW {0} processing blobGranuleFileRequest for range [{1} - {2}) @ {3}\n", bwData->id.toString(), @@ -2464,6 +2464,27 @@ ACTOR Future handleBlobGranuleFileRequest(Reference bwData return Void(); } +ACTOR Future handleBlobGranuleFileRequest(Reference bwData, BlobGranuleFileRequest req) { + choose { + when(wait(doBlobGranuleFileRequest(bwData, req))) {} + when(wait(delay(SERVER_KNOBS->BLOB_WORKER_REQUEST_TIMEOUT))) { + TEST(true); // Blob Worker request timeout hit + if (BW_DEBUG) { + fmt::print("BW {0} request [{1} - {2}) @ {3} timed out, sending WSS\n", + bwData->id.toString().substr(0, 5), + req.keyRange.begin.printable(), + req.keyRange.end.printable(), + req.readVersion); + } + --bwData->stats.activeReadRequests; + ++bwData->stats.granuleRequestTimeouts; + // return wrong_shard_server because it's possible that someone else actually owns the granule now + req.reply.sendError(wrong_shard_server()); + } + } + return Void(); +} + ACTOR Future openGranule(Reference bwData, AssignBlobRangeRequest req) { ASSERT(req.type != AssignRequestType::Continue); state Transaction tr(bwData->db); From 7a76b86b53fa2d13842ada3699817ad8830ff47a Mon Sep 17 00:00:00 2001 From: Josh Slocum Date: Wed, 9 Feb 2022 16:22:56 -0600 Subject: [PATCH 237/413] Blob worker request timeout race --- fdbserver/BlobWorker.actor.cpp | 26 +++++++++++++++----------- 1 file changed, 15 insertions(+), 11 deletions(-) diff --git a/fdbserver/BlobWorker.actor.cpp b/fdbserver/BlobWorker.actor.cpp index d5c6ccd7a4..53e3b74c95 100644 --- a/fdbserver/BlobWorker.actor.cpp +++ b/fdbserver/BlobWorker.actor.cpp @@ -2442,6 +2442,7 @@ ACTOR Future doBlobGranuleFileRequest(Reference bwData, Bl wait(yield(TaskPriority::DefaultEndpoint)); } + ASSERT(!req.reply.isSet()); req.reply.send(rep); --bwData->stats.activeReadRequests; } catch (Error& e) { @@ -2468,18 +2469,21 @@ ACTOR Future handleBlobGranuleFileRequest(Reference bwData choose { when(wait(doBlobGranuleFileRequest(bwData, req))) {} when(wait(delay(SERVER_KNOBS->BLOB_WORKER_REQUEST_TIMEOUT))) { - TEST(true); // Blob Worker request timeout hit - if (BW_DEBUG) { - fmt::print("BW {0} request [{1} - {2}) @ {3} timed out, sending WSS\n", - bwData->id.toString().substr(0, 5), - req.keyRange.begin.printable(), - req.keyRange.end.printable(), - req.readVersion); + if (!req.reply.isSet()) { + TEST(true); // Blob Worker request timeout hit + if (BW_DEBUG) { + fmt::print("BW {0} request [{1} - {2}) @ {3} timed out, sending WSS\n", + bwData->id.toString().substr(0, 5), + req.keyRange.begin.printable(), + req.keyRange.end.printable(), + req.readVersion); + } + --bwData->stats.activeReadRequests; + ++bwData->stats.granuleRequestTimeouts; + + // return wrong_shard_server because it's possible that someone else actually owns the granule now + req.reply.sendError(wrong_shard_server()); } - --bwData->stats.activeReadRequests; - ++bwData->stats.granuleRequestTimeouts; - // return wrong_shard_server because it's possible that someone else actually owns the granule now - req.reply.sendError(wrong_shard_server()); } } return Void(); From 4af1b24c7412f694a2718f26e3ed4361174491ca Mon Sep 17 00:00:00 2001 From: Josh Slocum Date: Thu, 10 Feb 2022 07:59:23 -0600 Subject: [PATCH 238/413] Using higher transferred version when fetching change feeds --- fdbserver/storageserver.actor.cpp | 28 +++++++++++++++++++++------- 1 file changed, 21 insertions(+), 7 deletions(-) diff --git a/fdbserver/storageserver.actor.cpp b/fdbserver/storageserver.actor.cpp index 9c9c245f72..fbea6cd23a 100644 --- a/fdbserver/storageserver.actor.cpp +++ b/fdbserver/storageserver.actor.cpp @@ -4439,9 +4439,12 @@ ACTOR Future> dispatchChangeFeeds(StorageServer Version endVersion, std::vector feedIds, std::unordered_set newFeedIds) { + state std::unordered_map feedMaxFetched; + if (feedIds.empty() && newFeedIds.empty()) { + return feedMaxFetched; + } // find overlapping range feeds - state std::unordered_map feedMaxFetched; state std::map> feedFetches; state PromiseStream removals; data->changeFeedRemovals[fetchKeysID] = removals; @@ -4846,12 +4849,20 @@ ACTOR Future fetchKeys(StorageServer* data, AddingShard* shard) { keys, true); // keys will be available when getLatestVersion()==transferredVersion is durable + // Similar to transferred version, but wait for all feed data and + Version feedTransferredVersion = data->version.get() + 1; + TraceEvent(SevDebug, "FetchKeysHaveFeedData", data->thisServerID) + .detail("FKID", interval.pairID) + .detail("Version", feedTransferredVersion) + .detail("StorageVersion", data->storageVersion()); + // Note that since it receives a pointer to FetchInjectionInfo, the thread does not leave this actor until this // point. - // Wait for the transferredVersion (and therefore the shard data) to be committed and durable. - wait(data->durableVersion.whenAtLeast(shard->transferredVersion)); + // Wait for the transferred version (and therefore the shard data) to be committed and durable. + wait(data->durableVersion.whenAtLeast(feedTransferredVersion)); + // TODO if this works, remove all of the fetch version stuff // Also wait on all fetched change feed data to become committed and durable while (!feedFetchedVersions.empty()) { auto feed = feedFetchedVersions.begin(); @@ -4860,10 +4871,13 @@ ACTOR Future fetchKeys(StorageServer* data, AddingShard* shard) { feedFetchedVersions.erase(feed); auto feedIt = data->uidChangeFeed.find(feedId); - if (feedIt != data->uidChangeFeed.end() && feedIt->second->durableFetchVersion.get() < maxFetched) { - wait(feedIt->second->durableFetchVersion.whenAtLeast(maxFetched)); - // return to updateStorage - wait(delay(0)); + /*if (feedIt != data->uidChangeFeed.end() && feedIt->second->durableFetchVersion.get() < maxFetched) { + wait(feedIt->second->durableFetchVersion.whenAtLeast(maxFetched)); + // return to updateStorage + wait(delay(0)); + }*/ + if (feedIt != data->uidChangeFeed.end()) { + ASSERT(feedIt->second->durableFetchVersion.get() >= maxFetched); } } From 031a11116bbec5d0932d724822c64e8b548b42c2 Mon Sep 17 00:00:00 2001 From: Josh Slocum Date: Thu, 10 Feb 2022 11:50:51 -0600 Subject: [PATCH 239/413] Fixing race between range mover and range split reassign --- fdbserver/BlobManager.actor.cpp | 26 ++++++++++++++++++++------ 1 file changed, 20 insertions(+), 6 deletions(-) diff --git a/fdbserver/BlobManager.actor.cpp b/fdbserver/BlobManager.actor.cpp index 51fdcbaf1a..afe3d3c6b0 100644 --- a/fdbserver/BlobManager.actor.cpp +++ b/fdbserver/BlobManager.actor.cpp @@ -506,21 +506,35 @@ ACTOR Future rangeAssigner(Reference bmData) { // modify the in-memory assignment data structures, and send request off to worker state UID workerId; if (assignment.isAssign) { + bool skip = false; // Ensure range isn't currently assigned anywhere, and there is only 1 intersecting range auto currentAssignments = bmData->workerAssignments.intersectingRanges(assignment.keyRange); int count = 0; for (auto i = currentAssignments.begin(); i != currentAssignments.end(); ++i) { - /* TODO: rethink asserts here if (assignment.assign.get().type == AssignRequestType::Continue) { - ASSERT(assignment.worker.present()); - ASSERT(it.value() == assignment.worker.get()); - } else { - ASSERT(it.value() == UID()); + ASSERT(assignment.worker.present()); + if (i.range() != assignment.keyRange || i.cvalue() != assignment.worker.get()) { + if (BM_DEBUG) { + fmt::print("Out of date re-assign for ({0}, {1}). Assignment must have changed while " + "checking split.\n Reassign: [{2} - {3}): {4}\n Existing: [{5} - {6}): {7}\n", + bmData->epoch, + seqNo, + assignment.keyRange.begin.printable(), + assignment.keyRange.end.printable(), + assignment.worker.get().toString().substr(0, 5), + i.begin().printable(), + i.end().printable(), + i.cvalue().toString().substr(0, 5)); + } + skip = true; + } } - */ count++; } ASSERT(count == 1); + if (skip) { + continue; + } if (assignment.worker.present() && assignment.worker.get().isValid()) { if (BM_DEBUG) { From f997b2dab4bd27599ffd77028029bbe6bfb2458f Mon Sep 17 00:00:00 2001 From: Josh Slocum Date: Thu, 10 Feb 2022 12:19:37 -0600 Subject: [PATCH 240/413] Smarter change feed cancelling when data moved away --- fdbserver/storageserver.actor.cpp | 44 +++++++++++++++++++++++-------- 1 file changed, 33 insertions(+), 11 deletions(-) diff --git a/fdbserver/storageserver.actor.cpp b/fdbserver/storageserver.actor.cpp index fbea6cd23a..287531b026 100644 --- a/fdbserver/storageserver.actor.cpp +++ b/fdbserver/storageserver.actor.cpp @@ -352,12 +352,30 @@ struct ChangeFeedInfo : ReferenceCounted { bool stopped = false; // A stopped change feed no longer adds new mutations, but is still queriable bool removing = false; - // TODO: could change this to a promiseStream, and only cancel stream if the cancelled range overlaps the - // request range - void moved() { - Promise hold = onMove; - onMove = Promise(); - hold.send(Void()); + KeyRangeMap>> moveTriggers; + + void triggerOnMove(KeyRange range, Promise p) { + printf("DBG: Trigger onMove registering [%s - %s)\n", + range.begin.printable().c_str(), + range.end.printable().c_str()); + auto toInsert = moveTriggers.modify(range); + for (auto triggerRange = toInsert.begin(); triggerRange != toInsert.end(); ++triggerRange) { + triggerRange->value().push_back(p); + } + } + + void moved(KeyRange range) { + printf("DBG: onMove triggered [%s - %s)\n", range.begin.printable().c_str(), range.end.printable().c_str()); + auto toTrigger = moveTriggers.intersectingRanges(range); + for (auto triggerRange : toTrigger) { + for (int i = 0; i < triggerRange.cvalue().size(); i++) { + if (triggerRange.cvalue()[i].canBeSet()) { + triggerRange.cvalue()[i].send(Void()); + } + } + } + // coalesce doesn't work with promises + moveTriggers.insert(range, std::vector>()); } }; @@ -2058,16 +2076,20 @@ ACTOR Future localChangeFeedStream(StorageServer* data, } } +// 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) { + wait(delay(0, TaskPriority::DefaultEndpoint)); + auto feed = data->uidChangeFeed.find(req.rangeID); if (feed == data->uidChangeFeed.end() || feed->second->removing) { req.reply.sendError(unknown_change_feed()); return Void(); } - state Future moved = feed->second->onMove.getFuture(); - wait(moved); + state Promise moved; + feed->second->triggerOnMove(req.range, moved); + wait(moved.getFuture()); printf("CF Moved! %lld - %lld. sending WSS\n", req.begin, req.end); - // DO NOT call req.reply.onReady before sending - we want to propagate this error through regardless of how far + // DO NOT call req.reply.onReady before sending - we need to propagate this error through regardless of how far // behind client is req.reply.sendError(wrong_shard_server()); return Void(); @@ -5169,7 +5191,7 @@ void changeServerKeys(StorageServer* data, if (feed != data->uidChangeFeed.end()) { feed->second->emptyVersion = version - 1; feed->second->removing = true; - feed->second->moved(); + feed->second->moved(feed->second->range); feed->second->newMutations.trigger(); data->uidChangeFeed.erase(feed); } @@ -5177,7 +5199,7 @@ void changeServerKeys(StorageServer* data, // if just part of feed's range is moved away auto feed = data->uidChangeFeed.find(f.first); if (feed != data->uidChangeFeed.end()) { - feed->second->moved(); + feed->second->moved(keys); } } } From 631cf0b085ae88b34bda67dcf1e1b071eae89a27 Mon Sep 17 00:00:00 2001 From: Josh Slocum Date: Thu, 10 Feb 2022 10:46:16 -0600 Subject: [PATCH 241/413] minor memory optimizations for SS change feeds --- fdbserver/storageserver.actor.cpp | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/fdbserver/storageserver.actor.cpp b/fdbserver/storageserver.actor.cpp index 287531b026..91f02afdda 100644 --- a/fdbserver/storageserver.actor.cpp +++ b/fdbserver/storageserver.actor.cpp @@ -1906,10 +1906,10 @@ ACTOR Future> getChangeFeedMutations(Stor Standalone> mutations; std::tie(id, version) = decodeChangeFeedDurableKey(kv.key); std::tie(mutations, knownCommittedVersion) = decodeChangeFeedDurableValue(kv.value); - reply.arena.dependsOn(mutations.arena()); auto m = filterMutations( reply.arena, MutationsAndVersionRef(mutations, version, knownCommittedVersion), req.range, inverted); if (m.mutations.size()) { + reply.arena.dependsOn(mutations.arena()); readAnyFromDisk = true; reply.mutations.push_back(reply.arena, m); } @@ -2132,7 +2132,8 @@ ACTOR Future changeFeedStreamQ(StorageServer* data, ChangeFeedStreamReques removeUID = true; } wait(onReady); - state Future> feedReplyFuture = + // keep this as not state variable so it is freed after sending to reduce memory + Future> feedReplyFuture = getChangeFeedMutations(data, req, false, atLatest, streamUID); if (atLatest && !removeUID && !feedReplyFuture.isReady()) { data->changeFeedClientVersions[req.reply.getEndpoint().getPrimaryAddress()][streamUID] = From 36742bcd7e6ecee0862d271fc0b41509df8c0cac Mon Sep 17 00:00:00 2001 From: Josh Slocum Date: Thu, 10 Feb 2022 18:52:17 -0600 Subject: [PATCH 242/413] Fixing durability model of change feed cleanup --- fdbserver/storageserver.actor.cpp | 99 +++++++++++++++++++++++-------- 1 file changed, 74 insertions(+), 25 deletions(-) diff --git a/fdbserver/storageserver.actor.cpp b/fdbserver/storageserver.actor.cpp index 91f02afdda..ee97b610b8 100644 --- a/fdbserver/storageserver.actor.cpp +++ b/fdbserver/storageserver.actor.cpp @@ -355,9 +355,6 @@ struct ChangeFeedInfo : ReferenceCounted { KeyRangeMap>> moveTriggers; void triggerOnMove(KeyRange range, Promise p) { - printf("DBG: Trigger onMove registering [%s - %s)\n", - range.begin.printable().c_str(), - range.end.printable().c_str()); auto toInsert = moveTriggers.modify(range); for (auto triggerRange = toInsert.begin(); triggerRange != toInsert.end(); ++triggerRange) { triggerRange->value().push_back(p); @@ -365,7 +362,6 @@ struct ChangeFeedInfo : ReferenceCounted { } void moved(KeyRange range) { - printf("DBG: onMove triggered [%s - %s)\n", range.begin.printable().c_str(), range.end.printable().c_str()); auto toTrigger = moveTriggers.intersectingRanges(range); for (auto triggerRange : toTrigger) { for (int i = 0; i < triggerRange.cvalue().size(); i++) { @@ -4435,7 +4431,16 @@ ACTOR Future> fetchChangeFeedMetadata(StorageServer* data, KeyR : ChangeFeedStatus::CHANGE_FEED_CREATE))); } else { auto changeFeedInfo = existingEntry->second; - if (changeFeedInfo->emptyVersion < cfEntry.emptyVersion) { + auto feedCleanup = data->changeFeedCleanupDurable.find(cfEntry.rangeId); + + if (feedCleanup != data->changeFeedCleanupDurable.end()) { + TEST(true); // re-fetching feed scheduled for deletion! + ASSERT(fetchVersion > feedCleanup->second); + changeFeedInfo->emptyVersion = cfEntry.emptyVersion; + changeFeedInfo->stopped = cfEntry.stopped; + changeFeedInfo->removing = false; + data->changeFeedCleanupDurable.erase(feedCleanup); + } else if (changeFeedInfo->emptyVersion < cfEntry.emptyVersion) { TEST(true); // Got updated CF emptyVersion from a parallel fetchChangeFeedMetadata changeFeedInfo->emptyVersion = cfEntry.emptyVersion; auto& mLV = data->addVersionToMutationLog(data->data().getLatestVersion()); @@ -4483,6 +4488,7 @@ ACTOR Future> dispatchChangeFeeds(StorageServer auto feedIt = data->uidChangeFeed.find(feedId); // 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); } @@ -5178,23 +5184,29 @@ void changeServerKeys(StorageServer* data, changeFeedDurableKey(f.first, 0), changeFeedDurableKey(f.first, version))); - auto rs = data->keyChangeFeed.modify(f.second); + // do this cleanup later!! + /*auto rs = data->keyChangeFeed.modify(f.second); for (auto r = rs.begin(); r != rs.end(); ++r) { - auto& feedList = r->value(); - for (int i = 0; i < feedList.size(); i++) { - if (feedList[i]->id == f.first) { - swapAndPop(&feedList, i--); - } - } - } + auto& feedList = r->value(); + for (int i = 0; i < feedList.size(); i++) { + if (feedList[i]->id == f.first) { + swapAndPop(&feedList, i--); + } + } + }*/ + // We can't actually remove this change feed fully until the mutations clearing its data become durable. + // If the SS restarted at version R before the clearing mutations became durable at version D (R < D), + // then the restarted SS would restore the change feed clients would be able to read data and would miss + // mutations from versions [R, D), up until we got the private mutation triggering the cleanup again. data->keyChangeFeed.coalesce(f.second.contents()); auto feed = data->uidChangeFeed.find(f.first); if (feed != data->uidChangeFeed.end()) { feed->second->emptyVersion = version - 1; + feed->second->stopped = true; feed->second->removing = true; feed->second->moved(feed->second->range); feed->second->newMutations.trigger(); - data->uidChangeFeed.erase(feed); + // data->uidChangeFeed.erase(feed); } } else { // if just part of feed's range is moved away @@ -5368,9 +5380,11 @@ private: rollback(data, rollbackVersion, currentVersion); } for (auto& it : data->uidChangeFeed) { - it.second->mutations.push_back(MutationsAndVersionRef(currentVersion, rollbackVersion)); - it.second->mutations.back().mutations.push_back_deep(it.second->mutations.back().arena(), m); - data->currentChangeFeeds.insert(it.first); + if (!it.second->stopped && !it.second->removing) { + it.second->mutations.push_back(MutationsAndVersionRef(currentVersion, rollbackVersion)); + it.second->mutations.back().mutations.push_back_deep(it.second->mutations.back().arena(), m); + data->currentChangeFeeds.insert(it.first); + } } data->recoveryVersionSkips.emplace_back(rollbackVersion, currentVersion - rollbackVersion); @@ -5508,17 +5522,24 @@ private: MutationRef(MutationRef::ClearRange, changeFeedDurableKey(feed->second->id, 0), changeFeedDurableKey(feed->second->id, currentVersion))); - auto rs = data->keyChangeFeed.modify(feed->second->range); + /*auto rs = data->keyChangeFeed.modify(feed->second->range); for (auto r = rs.begin(); r != rs.end(); ++r) { - auto& feedList = r->value(); - for (int i = 0; i < feedList.size(); i++) { - if (feedList[i] == feed->second) { - swapAndPop(&feedList, i--); - } - } + auto& feedList = r->value(); + for (int i = 0; i < feedList.size(); i++) { + if (feedList[i] == feed->second) { + swapAndPop(&feedList, i--); + } + } } data->keyChangeFeed.coalesce(feed->second->range.contents()); - data->uidChangeFeed.erase(feed); + data->uidChangeFeed.erase(feed);*/ + + feed->second->emptyVersion = currentVersion - 1; + feed->second->stopped = true; + feed->second->removing = true; + feed->second->moved(feed->second->range); + feed->second->newMutations.trigger(); + data->changeFeedCleanupDurable[feed->first] = cleanupVersion; } @@ -6178,6 +6199,34 @@ ACTOR Future updateStorage(StorageServer* data) { auto cfCleanup = data->changeFeedCleanupDurable.begin(); while (cfCleanup != data->changeFeedCleanupDurable.end()) { if (cfCleanup->second <= newOldestVersion) { + // remove from the data structure here, if it wasn't added back by another fetch or something + auto feed = data->uidChangeFeed.find(cfCleanup->first); + ASSERT(feed != data->uidChangeFeed.end()); + if (feed->second->removing) { + // TODO REMOVE + printf("DBG: SS %s Feed %s removing @ %lld!\n", + data->thisServerID.toString().substr(0, 4).c_str(), + feed->first.printable().substr(0, 6).c_str(), + cfCleanup->second); + auto rs = data->keyChangeFeed.modify(feed->second->range); + for (auto r = rs.begin(); r != rs.end(); ++r) { + auto& feedList = r->value(); + for (int i = 0; i < feedList.size(); i++) { + if (feedList[i]->id == cfCleanup->first) { + swapAndPop(&feedList, i--); + } + } + } + data->uidChangeFeed.erase(feed); + } else { + TEST(true); // Feed re-fetched after remove + // TODO REMOVE + printf("DBG: SS %s Feed %s not removing @ %lld, must have been re-fetched after moved away!\n", + data->thisServerID.toString().substr(0, 4).c_str(), + feed->first.printable().substr(0, 6).c_str(), + cfCleanup->second); + } + cfCleanup = data->changeFeedCleanupDurable.erase(cfCleanup); } else { cfCleanup++; From 99628bd3b542dd911193b543259f91ca450b1d8f Mon Sep 17 00:00:00 2001 From: Josh Slocum Date: Thu, 10 Feb 2022 19:02:03 -0600 Subject: [PATCH 243/413] Handling replyPromiseStream already disconnected --- fdbrpc/fdbrpc.h | 12 ++++++++---- 1 file changed, 8 insertions(+), 4 deletions(-) diff --git a/fdbrpc/fdbrpc.h b/fdbrpc/fdbrpc.h index 686cc26ea5..b3e4e8be24 100644 --- a/fdbrpc/fdbrpc.h +++ b/fdbrpc/fdbrpc.h @@ -763,10 +763,14 @@ public: Future disc = makeDependent(IFailureMonitor::failureMonitor()).onDisconnectOrFailure(getEndpoint()); auto& p = getReplyPromiseStream(value); - Reference peer = - FlowTransport::transport().sendUnreliable(SerializeSource(value), getEndpoint(), true); - // FIXME: defer sending the message until we know the connection is established - endStreamOnDisconnect(disc, p, getEndpoint(), peer); + if (disc.isReady()) { + p.sendError(request_maybe_delivered()); + } else { + Reference peer = + FlowTransport::transport().sendUnreliable(SerializeSource(value), getEndpoint(), true); + // FIXME: defer sending the message until we know the connection is established + endStreamOnDisconnect(disc, p, getEndpoint(), peer); + } return p; } else { send(value); From f9c16ed9371d003676b1dc83bd0b58ae559d7c90 Mon Sep 17 00:00:00 2001 From: Josh Slocum Date: Fri, 11 Feb 2022 08:14:00 -0600 Subject: [PATCH 244/413] misc change feed improvements --- fdbserver/storageserver.actor.cpp | 23 ++++++++--------------- 1 file changed, 8 insertions(+), 15 deletions(-) diff --git a/fdbserver/storageserver.actor.cpp b/fdbserver/storageserver.actor.cpp index ee97b610b8..8c4202dc62 100644 --- a/fdbserver/storageserver.actor.cpp +++ b/fdbserver/storageserver.actor.cpp @@ -647,6 +647,7 @@ public: std::set fetchingChangeFeeds; std::unordered_map> changeFeedClientVersions; std::unordered_map changeFeedCleanupDurable; + int64_t activeFeedQueries = 0; // newestAvailableVersion[k] // == invalidVersion -> k is unavailable at all versions @@ -910,6 +911,8 @@ public: specialCounter(cc, "KvstoreSizeTotal", [self]() { return std::get<0>(self->storage.getSize()); }); specialCounter(cc, "KvstoreNodeTotal", [self]() { return std::get<1>(self->storage.getSize()); }); specialCounter(cc, "KvstoreInlineKey", [self]() { return std::get<2>(self->storage.getSize()); }); + specialCounter(cc, "ActiveChangeFeeds", [self]() { return self->uidChangeFeed.size(); }); + specialCounter(cc, "ActiveChangeFeedQueries", [self]() { return self->activeFeedQueries; }); } } counters; @@ -1857,7 +1860,6 @@ ACTOR Future> getChangeFeedMutations(Stor state bool readDurable = feedInfo->durableVersion != invalidVersion && req.begin <= feedInfo->durableVersion; state bool readFetched = req.begin <= fetchStorageVersion; - state bool readAnyFromDisk = false; if (req.end > emptyVersion + 1 && (readDurable || readFetched)) { if (readFetched && req.begin <= feedInfo->fetchVersion) { // Request needs data that has been written to storage by a change feed fetch, but not committed yet @@ -1906,7 +1908,6 @@ ACTOR Future> getChangeFeedMutations(Stor reply.arena, MutationsAndVersionRef(mutations, version, knownCommittedVersion), req.range, inverted); if (m.mutations.size()) { reply.arena.dependsOn(mutations.arena()); - readAnyFromDisk = true; reply.mutations.push_back(reply.arena, m); } remainingDurableBytes -= @@ -1923,15 +1924,6 @@ ACTOR Future> getChangeFeedMutations(Stor reply.mutations.empty() ? invalidVersion : reply.mutations.front().version, reply.mutations.empty() ? invalidVersion : reply.mutations.back().version, reply.mutations.size()); - if (!reply.mutations.empty() && reply.mutations.front().version == 252133030) { - for (auto& it : reply.mutations) { - printf("CFM: SS %s CF %s: %lld (%d)\n", - data->thisServerID.toString().substr(0, 4).c_str(), - req.rangeID.printable().substr(0, 6).c_str(), - it.version, - it.mutations.size()); - } - } } if (remainingDurableBytes > 0) { reply.arena.dependsOn(memoryReply.arena); @@ -1942,7 +1934,7 @@ ACTOR Future> getChangeFeedMutations(Stor --totalCount; } reply.mutations.append(reply.arena, it, totalCount); - } else if (!readAnyFromDisk) { + } else if (reply.mutations.empty() || reply.mutations.back().version < lastVersion) { if (DEBUG_SS_CFM(data->thisServerID, req.rangeID, req.begin)) { printf("CFM: SS %s CF %s: adding empty from disk %lld\n", data->thisServerID.toString().substr(0, 4).c_str(), @@ -2120,6 +2112,7 @@ ACTOR Future changeFeedStreamQ(StorageServer* data, ChangeFeedStreamReques } try { + data->activeFeedQueries++; loop { Future onReady = req.reply.onReady(); if (atLatest && !onReady.isReady() && !removeUID) { @@ -2197,6 +2190,7 @@ ACTOR Future changeFeedStreamQ(StorageServer* data, ChangeFeedStreamReques } } } catch (Error& e) { + data->activeFeedQueries--; // TODO REMOVE printf("CFSQ %s got error %s\n", streamUID.toString().substr(0, 8).c_str(), e.name()); auto it = data->changeFeedClientVersions.find(req.reply.getEndpoint().getPrimaryAddress()); @@ -4433,13 +4427,12 @@ ACTOR Future> fetchChangeFeedMetadata(StorageServer* data, KeyR auto changeFeedInfo = existingEntry->second; auto feedCleanup = data->changeFeedCleanupDurable.find(cfEntry.rangeId); - if (feedCleanup != data->changeFeedCleanupDurable.end()) { - TEST(true); // re-fetching feed scheduled for deletion! + if (feedCleanup != data->changeFeedCleanupDurable.end() && changeFeedInfo->removing) { + TEST(true); // re-fetching feed scheduled for deletion! Un-mark it as removing ASSERT(fetchVersion > feedCleanup->second); changeFeedInfo->emptyVersion = cfEntry.emptyVersion; changeFeedInfo->stopped = cfEntry.stopped; changeFeedInfo->removing = false; - data->changeFeedCleanupDurable.erase(feedCleanup); } else if (changeFeedInfo->emptyVersion < cfEntry.emptyVersion) { TEST(true); // Got updated CF emptyVersion from a parallel fetchChangeFeedMetadata changeFeedInfo->emptyVersion = cfEntry.emptyVersion; From 08505325d3b015afba9f68ece6663f84fbf37a5d Mon Sep 17 00:00:00 2001 From: Josh Slocum Date: Fri, 11 Feb 2022 08:41:01 -0600 Subject: [PATCH 245/413] Fixing change feed mapping in cleanup refactor --- fdbserver/storageserver.actor.cpp | 8 ++++++-- 1 file changed, 6 insertions(+), 2 deletions(-) diff --git a/fdbserver/storageserver.actor.cpp b/fdbserver/storageserver.actor.cpp index 8c4202dc62..a990643e99 100644 --- a/fdbserver/storageserver.actor.cpp +++ b/fdbserver/storageserver.actor.cpp @@ -5186,12 +5186,15 @@ void changeServerKeys(StorageServer* data, swapAndPop(&feedList, i--); } } - }*/ + } + data->keyChangeFeed.coalesce(f.second.contents()); + */ + // We can't actually remove this change feed fully until the mutations clearing its data become durable. // If the SS restarted at version R before the clearing mutations became durable at version D (R < D), // then the restarted SS would restore the change feed clients would be able to read data and would miss // mutations from versions [R, D), up until we got the private mutation triggering the cleanup again. - data->keyChangeFeed.coalesce(f.second.contents()); + auto feed = data->uidChangeFeed.find(f.first); if (feed != data->uidChangeFeed.end()) { feed->second->emptyVersion = version - 1; @@ -6210,6 +6213,7 @@ ACTOR Future updateStorage(StorageServer* data) { } } } + data->keyChangeFeed.coalesce(feed->second->range.contents()); data->uidChangeFeed.erase(feed); } else { TEST(true); // Feed re-fetched after remove From 8a56349ea3a091da1f31e6f8319d715efb9a2287 Mon Sep 17 00:00:00 2001 From: Josh Slocum Date: Fri, 11 Feb 2022 10:12:49 -0600 Subject: [PATCH 246/413] Fixing shard move away then back then away race --- fdbserver/storageserver.actor.cpp | 31 ++++++++++++++++++++++++++----- 1 file changed, 26 insertions(+), 5 deletions(-) diff --git a/fdbserver/storageserver.actor.cpp b/fdbserver/storageserver.actor.cpp index a990643e99..2208d22d5b 100644 --- a/fdbserver/storageserver.actor.cpp +++ b/fdbserver/storageserver.actor.cpp @@ -4340,6 +4340,10 @@ ACTOR Future fetchChangeFeed(StorageServer* data, auto cleanupPending = data->changeFeedCleanupDurable.find(changeFeedInfo->id); if (cleanupPending != data->changeFeedCleanupDurable.end()) { + /*printf("SS %s waiting for CF %s cleanup @ %lld\n", + data->thisServerID.toString().substr(0, 4).c_str(), + changeFeedInfo->id.toString().substr(0, 6).c_str(), + cleanupPending->second);*/ TraceEvent(SevDebug, "FetchChangeFeedWaitCleanup", data->thisServerID) .detail("RangeID", changeFeedInfo->id.printable()) .detail("Range", changeFeedInfo->range.toString()) @@ -4348,7 +4352,18 @@ ACTOR Future fetchChangeFeed(StorageServer* data, .detail("BeginVersion", beginVersion) .detail("EndVersion", endVersion); wait(data->durableVersion.whenAtLeast(cleanupPending->second + 1)); - ASSERT(!data->changeFeedCleanupDurable.count(changeFeedInfo->id)); + wait(delay(0)); + // shard might have gotten moved away (again) while we were waiting + auto cleanupPendingAfter = data->changeFeedCleanupDurable.find(changeFeedInfo->id); + if (cleanupPendingAfter != data->changeFeedCleanupDurable.end()) { + ASSERT(cleanupPendingAfter->second >= endVersion); + TraceEvent(SevDebug, "FetchChangeFeedCancelledByCleanup", data->thisServerID) + .detail("RangeID", changeFeedInfo->id.printable()) + .detail("Range", changeFeedInfo->range.toString()) + .detail("BeginVersion", beginVersion) + .detail("EndVersion", endVersion); + return invalidVersion; + } } loop { @@ -6200,10 +6215,10 @@ ACTOR Future updateStorage(StorageServer* data) { ASSERT(feed != data->uidChangeFeed.end()); if (feed->second->removing) { // TODO REMOVE - printf("DBG: SS %s Feed %s removing @ %lld!\n", + /*printf("DBG: SS %s Feed %s removing metadata @ %lld!\n", data->thisServerID.toString().substr(0, 4).c_str(), feed->first.printable().substr(0, 6).c_str(), - cfCleanup->second); + cfCleanup->second);*/ auto rs = data->keyChangeFeed.modify(feed->second->range); for (auto r = rs.begin(); r != rs.end(); ++r) { auto& feedList = r->value(); @@ -6218,12 +6233,18 @@ ACTOR Future updateStorage(StorageServer* data) { } else { TEST(true); // Feed re-fetched after remove // TODO REMOVE - printf("DBG: SS %s Feed %s not removing @ %lld, must have been re-fetched after moved away!\n", + /*printf("DBG: SS %s Feed %s not removing metadata @ %lld, must have been re-fetched after moved " + "away!\n", data->thisServerID.toString().substr(0, 4).c_str(), feed->first.printable().substr(0, 6).c_str(), - cfCleanup->second); + cfCleanup->second);*/ } + // TODO REMOVE + /*printf("DBG: SS %s Feed %s removing cleanup entry @ %lld!\n", + data->thisServerID.toString().substr(0, 4).c_str(), + feed->first.printable().substr(0, 6).c_str(), + cfCleanup->second);*/ cfCleanup = data->changeFeedCleanupDurable.erase(cfCleanup); } else { cfCleanup++; From 3bbddce668ba0790de82d910b81ced42a61ffb65 Mon Sep 17 00:00:00 2001 From: Josh Slocum Date: Fri, 11 Feb 2022 10:14:15 -0600 Subject: [PATCH 247/413] Fixing shard move away and back race --- fdbserver/storageserver.actor.cpp | 14 ++++++++++++++ 1 file changed, 14 insertions(+) diff --git a/fdbserver/storageserver.actor.cpp b/fdbserver/storageserver.actor.cpp index 2208d22d5b..2572954c3b 100644 --- a/fdbserver/storageserver.actor.cpp +++ b/fdbserver/storageserver.actor.cpp @@ -4177,6 +4177,17 @@ ACTOR Future fetchChangeFeedApplier(StorageServer* data, ASSERT(startVersion >= 0); + // TODO REMOVE + TraceEvent(SevDebug, "FetchChangeFeedStarting", data->thisServerID) + .detail("RangeID", rangeId.printable()) + .detail("Range", range.toString()) + .detail("StartVersion", startVersion) + .detail("EndVersion", endVersion) + .detail("BeginVersion", beginVersion) + .detail("EmptyVersion", emptyVersion) + .detail("FetchVersion", changeFeedInfo->fetchVersion) + .detail("DurableFetchVersion", changeFeedInfo->durableFetchVersion.get()); + if (startVersion >= endVersion) { TEST(true); // Change Feed popped before fetch TraceEvent(SevDebug, "FetchChangeFeedNoOp", data->thisServerID) @@ -4448,6 +4459,9 @@ ACTOR Future> fetchChangeFeedMetadata(StorageServer* data, KeyR changeFeedInfo->emptyVersion = cfEntry.emptyVersion; changeFeedInfo->stopped = cfEntry.stopped; changeFeedInfo->removing = false; + // reset fetch versions because everything previously fetched was cleaned up + changeFeedInfo->fetchVersion = invalidVersion; + changeFeedInfo->durableFetchVersion = NotifiedVersion(); } else if (changeFeedInfo->emptyVersion < cfEntry.emptyVersion) { TEST(true); // Got updated CF emptyVersion from a parallel fetchChangeFeedMetadata changeFeedInfo->emptyVersion = cfEntry.emptyVersion; From e6d68a0fdb7c6c95bee8d64dc7ad354ee3056e4d Mon Sep 17 00:00:00 2001 From: Josh Slocum Date: Fri, 11 Feb 2022 13:35:39 -0600 Subject: [PATCH 248/413] more variability in write workload for BGCorrectness and making tiny values drive slightly less hard --- .../BlobGranuleCorrectnessWorkload.actor.cpp | 37 +++++++++++++++---- 1 file changed, 29 insertions(+), 8 deletions(-) diff --git a/fdbserver/workloads/BlobGranuleCorrectnessWorkload.actor.cpp b/fdbserver/workloads/BlobGranuleCorrectnessWorkload.actor.cpp index efca15df5c..d0dd9a061d 100644 --- a/fdbserver/workloads/BlobGranuleCorrectnessWorkload.actor.cpp +++ b/fdbserver/workloads/BlobGranuleCorrectnessWorkload.actor.cpp @@ -56,7 +56,7 @@ struct KeyData { std::vector writes; }; -static std::vector targetValSizes = { 20, 100, 500 }; +static std::vector targetValSizes = { 40, 100, 500 }; struct ThreadData : ReferenceCounted, NonCopyable { // directory info @@ -89,8 +89,9 @@ struct ThreadData : ReferenceCounted, NonCopyable { int64_t timeTravelReads = 0; int64_t timeTravelTooOld = 0; int64_t rowsRead = 0; - int64_t rowsWritten = 0; int64_t bytesRead = 0; + int64_t rowsWritten = 0; + int64_t bytesWritten = 0; ThreadData(uint32_t directoryID, int64_t targetByteRate) : directoryID(directoryID), targetByteRate(targetByteRate) { @@ -168,8 +169,20 @@ struct BlobGranuleCorrectnessWorkload : TestWorkload { int myDirectories = deterministicRandom()->randomInt(1, 2 * targetMyDirectories + 1); // anywhere from 2 delta files per second to 1 delta file every 4 seconds, spread across all directories - targetByteRate = 2 * SERVER_KNOBS->BG_DELTA_FILE_TARGET_BYTES / (1 + (randomness % 8)) / myDirectories; + targetByteRate = 2 * SERVER_KNOBS->BG_DELTA_FILE_TARGET_BYTES / (1 + (randomness % 8)); randomness /= 8; + + // either do equal across all of my directories, or skewed + bool skewed = myDirectories > 1 && deterministicRandom()->random01() < 0.4; + int skewMultiplier; + if (skewed) { + // first directory has 1/2, second has 1/4, third has 1/8, etc... + skewMultiplier = 2; + targetByteRate /= 2; + } else { + skewMultiplier = 1; + targetByteRate /= myDirectories; + } for (int i = 0; i < myDirectories; i++) { // set up directory with its own randomness uint32_t dirId = i * clientCount + clientId; @@ -177,6 +190,7 @@ struct BlobGranuleCorrectnessWorkload : TestWorkload { printf("Client %d/%d creating directory %d\n", clientId, clientCount, dirId); } directories.push_back(makeReference(dirId, targetByteRate)); + targetByteRate /= skewMultiplier; } } } @@ -647,8 +661,8 @@ struct BlobGranuleCorrectnessWorkload : TestWorkload { state double last = now(); state int keysPerQuery = 100; - state int targetBytesPerQuery = threadData->targetValLength * keysPerQuery; - state double targetTps = (1.0 * threadData->targetByteRate) / targetBytesPerQuery; + // state int targetBytesPerQuery = threadData->targetValLength * keysPerQuery; + // state double targetTps = (1.0 * threadData->targetByteRate) / targetBytesPerQuery; state uint32_t nextVal = 0; TraceEvent("BlobGranuleCorrectnessWriterStart").log(); @@ -664,7 +678,9 @@ struct BlobGranuleCorrectnessWorkload : TestWorkload { state std::vector> keyAndIdToWrite; state std::vector> keyAndIdToClear; - for (int i = 0; i < keysPerQuery; i++) { + state int queryKeys = + keysPerQuery * (0.1 + deterministicRandom()->random01() * 1.8); // 10% to 190% of target keys per query + for (int i = 0; i < queryKeys; i++) { uint32_t key; if (threadData->keyData.empty() || deterministicRandom()->random01() > threadData->reuseKeyProb) { // new key @@ -711,6 +727,7 @@ struct BlobGranuleCorrectnessWorkload : TestWorkload { } } + state int64_t txnBytes; loop { try { // write rows in txn @@ -721,7 +738,7 @@ struct BlobGranuleCorrectnessWorkload : TestWorkload { for (auto& it : keyAndIdToClear) { tr.clear(singleKeyRange(threadData->getKey(it.first, it.second))); } - + txnBytes = tr.getSize(); wait(tr.commit()); break; } catch (Error& e) { @@ -766,8 +783,11 @@ struct BlobGranuleCorrectnessWorkload : TestWorkload { threadData->firstWriteSuccessful.send(Void()); } + threadData->rowsWritten += queryKeys; + threadData->bytesWritten += txnBytes; + // wait - wait(poisson(&last, 1.0 / targetTps)); + wait(poisson(&last, (txnBytes + 1.0) / threadData->targetByteRate)); } } @@ -830,6 +850,7 @@ struct BlobGranuleCorrectnessWorkload : TestWorkload { fmt::print(" {} time travel too old\n", threadData->timeTravelTooOld); fmt::print(" {} errors\n", threadData->errors); fmt::print(" {} rows written\n", threadData->rowsWritten); + fmt::print(" {} bytes written\n", threadData->bytesWritten); fmt::print(" {} unique keys\n", threadData->usedKeys.size()); fmt::print(" {} real-time reads\n", threadData->reads); fmt::print(" {} time travel reads\n", threadData->timeTravelReads); From 0db843bc564c78f8dcddc3b0f6edcbbd9587708b Mon Sep 17 00:00:00 2001 From: Josh Slocum Date: Fri, 11 Feb 2022 16:04:05 -0600 Subject: [PATCH 249/413] more CF movement and cleanup races --- fdbserver/BlobWorker.actor.cpp | 18 +++++++++++----- fdbserver/storageserver.actor.cpp | 35 +++++++++++++++++++++++++++++-- 2 files changed, 46 insertions(+), 7 deletions(-) diff --git a/fdbserver/BlobWorker.actor.cpp b/fdbserver/BlobWorker.actor.cpp index 53e3b74c95..31840cf9b9 100644 --- a/fdbserver/BlobWorker.actor.cpp +++ b/fdbserver/BlobWorker.actor.cpp @@ -928,7 +928,10 @@ ACTOR Future handleCompletedDeltaFile(Reference bwData, wait(popFuture); } while (!rollbacksCompleted->empty() && completedDeltaFile.version >= rollbacksCompleted->front().second) { - fmt::print("Completed rollback {0} -> {1} with delta file {2}\n", + fmt::print("Granule [{0} - {1}) on BW {2} completed rollback {3} -> {4} with delta file {5}\n", + metadata->keyRange.begin.printable().c_str(), + metadata->keyRange.end.printable().c_str(), + bwData->id.toString().substr(0, 5).c_str(), rollbacksCompleted->front().second, rollbacksCompleted->front().first, completedDeltaFile.version); @@ -1542,9 +1545,13 @@ ACTOR Future blobGranuleUpdateFiles(Reference bwData, metadata->currentDeltas.back().version <= rollbackVersion)) { if (BW_DEBUG) { - fmt::print("BW skipping rollback {0} -> {1} completely\n", - deltas.version, - rollbackVersion); + fmt::print( + "Granule [{0} - {1}) on BW {2} skipping rollback {0} -> {1} completely\n", + metadata->keyRange.begin.printable().c_str(), + metadata->keyRange.end.printable().c_str(), + bwData->id.toString().substr(0, 5).c_str(), + deltas.version, + rollbackVersion); } // Still have to add to rollbacksCompleted. If we later roll the granule back past // this because of cancelling a delta file, we need to count this as in progress so @@ -1553,9 +1560,10 @@ ACTOR Future blobGranuleUpdateFiles(Reference bwData, rollbacksCompleted.push_back(std::pair(rollbackVersion, deltas.version)); } else { if (BW_DEBUG) { - fmt::print("BW [{0} - {1}) ROLLBACK @ {2} -> {3}\n", + fmt::print("[{0} - {1}) on BW {2} ROLLBACK @ {2} -> {3}\n", metadata->keyRange.begin.printable(), metadata->keyRange.end.printable(), + bwData->id.toString().substr(0, 5).c_str(), deltas.version, rollbackVersion); TraceEvent(SevWarn, "GranuleRollback", bwData->id) diff --git a/fdbserver/storageserver.actor.cpp b/fdbserver/storageserver.actor.cpp index 2572954c3b..13dddf83e4 100644 --- a/fdbserver/storageserver.actor.cpp +++ b/fdbserver/storageserver.actor.cpp @@ -4462,6 +4462,13 @@ ACTOR Future> fetchChangeFeedMetadata(StorageServer* data, KeyR // reset fetch versions because everything previously fetched was cleaned up changeFeedInfo->fetchVersion = invalidVersion; changeFeedInfo->durableFetchVersion = NotifiedVersion(); + // TODO REMOVE + TraceEvent(SevDebug, "FetchedChangeFeedInfoReset", data->thisServerID) + .detail("RangeID", cfEntry.rangeId.printable()) + .detail("Range", cfEntry.range.toString()) + .detail("FetchVersion", fetchVersion) + .detail("EmptyVersion", cfEntry.emptyVersion) + .detail("Stopped", cfEntry.stopped); } else if (changeFeedInfo->emptyVersion < cfEntry.emptyVersion) { TEST(true); // Got updated CF emptyVersion from a parallel fetchChangeFeedMetadata changeFeedInfo->emptyVersion = cfEntry.emptyVersion; @@ -6139,7 +6146,14 @@ ACTOR Future updateStorage(StorageServer* data) { info->second->storageVersion = it.version; } - if (info->second->fetchVersion != invalidVersion) { + if (info->second->fetchVersion != invalidVersion && !info->second->removing) { + // TODO REMOVE trace + TraceEvent(SevDebug, "UpdateStorageChangeFeedStart", data->thisServerID) + .detail("RangeID", info->second->id.printable()) + .detail("Range", info->second->range.toString()) + .detail("FetchVersion", info->second->fetchVersion) + .detail("Stopped", info->second->stopped) + .detail("Removing", info->second->removing); feedFetchVersions.push_back(std::pair(info->second->id, info->second->fetchVersion)); } // handle case where fetch had version ahead of last in-memory mutation @@ -6208,7 +6222,18 @@ ACTOR Future updateStorage(StorageServer* data) { curFeed = 0; while (curFeed < feedFetchVersions.size()) { auto info = data->uidChangeFeed.find(feedFetchVersions[curFeed].first); - if (info != data->uidChangeFeed.end()) { + // Don't update if the feed is pending cleanup. Either it will get cleaned up and destroyed, or it will get + // fetched again, where the fetch version will get reset. + if (info != data->uidChangeFeed.end() && !data->changeFeedCleanupDurable.count(info->second->id)) { + // TODO REMOVE trace + TraceEvent(SevDebug, "UpdateStorageChangeFeedDurable", data->thisServerID) + .detail("RangeID", info->second->id.printable()) + .detail("Range", info->second->range.toString()) + .detail("FetchVersion", info->second->fetchVersion) + .detail("OldDurableVersion", info->second->durableFetchVersion.get()) + .detail("NewDurableVersion", feedFetchVersions[curFeed].second) + .detail("Stopped", info->second->stopped) + .detail("Removing", info->second->removing); if (feedFetchVersions[curFeed].second > info->second->durableFetchVersion.get()) { info->second->durableFetchVersion.set(feedFetchVersions[curFeed].second); } @@ -6243,6 +6268,12 @@ ACTOR Future updateStorage(StorageServer* data) { } } data->keyChangeFeed.coalesce(feed->second->range.contents()); + + // TODO REMOVE + TraceEvent(SevDebug, "UpdateStorageChangeFeedCleanup", data->thisServerID) + .detail("RangeID", feed->second->id.printable()) + .detail("Range", feed->second->range.toString()); + data->uidChangeFeed.erase(feed); } else { TEST(true); // Feed re-fetched after remove From 9175ba354a51be5dae69bf81f00fa74b57965f67 Mon Sep 17 00:00:00 2001 From: Josh Slocum Date: Fri, 11 Feb 2022 17:25:46 -0600 Subject: [PATCH 250/413] Adding a separate phase for the end of change feed fetching to handle changes in fetchKeys available durability --- fdbserver/storageserver.actor.cpp | 32 +++++++++++++++++++++---------- 1 file changed, 22 insertions(+), 10 deletions(-) diff --git a/fdbserver/storageserver.actor.cpp b/fdbserver/storageserver.actor.cpp index 13dddf83e4..b1dbace948 100644 --- a/fdbserver/storageserver.actor.cpp +++ b/fdbserver/storageserver.actor.cpp @@ -123,6 +123,9 @@ struct AddingShard : NonCopyable { // During Fetching phase, it fetches data before fetchVersion and write it to storage, then let updater know it // is ready to update the deferred updates` (see the comment of member variable `updates` above). Fetching, + // During the FetchingCF phase, the shard data is transferred but the remaining change feed data is still being + // transferred. This is equivalent to the waiting phase for non-changefeed data. + FetchingCF, // During Waiting phase, it sends updater the deferred updates, and wait until they are durable. Waiting // The shard's state is changed from adding to readWrite then. @@ -145,7 +148,8 @@ struct AddingShard : NonCopyable { void addMutation(Version version, bool fromFetch, MutationRef const& mutation); - bool isTransferred() const { return phase == Waiting; } + bool isDataTransferred() const { return phase >= FetchingCF; } + bool isDataAndCFTransferred() const { return phase >= Waiting; } }; class ShardInfo : public ReferenceCounted, NonCopyable { @@ -171,14 +175,17 @@ public: bool isReadable() const { return readWrite != nullptr; } bool notAssigned() const { return !readWrite && !adding; } bool assigned() const { return readWrite || adding; } - bool isInVersionedData() const { return readWrite || (adding && adding->isTransferred()); } + bool isInVersionedData() const { return readWrite || (adding && adding->isDataTransferred()); } + bool isCFInVersionedData() const { return readWrite || (adding && adding->isDataAndCFTransferred()); } void addMutation(Version version, bool fromFetch, MutationRef const& mutation); bool isFetched() const { return readWrite || (adding && adding->fetchComplete.isSet()); } const char* debugDescribeState() const { if (notAssigned()) return "NotAssigned"; - else if (adding && !adding->isTransferred()) + else if (adding && !adding->isDataAndCFTransferred()) + return "AddingFetchingCF"; + else if (adding && !adding->isDataTransferred()) return "AddingFetching"; else if (adding) return "AddingTransferred"; @@ -1251,6 +1258,8 @@ void validate(StorageServer* data, bool force = false) { } } + // FIXME: do some change feed validation? + latest.validate(); validateRange(latest, allKeys, data->version.get(), data->thisServerID, data->durableVersion.get()); @@ -4824,7 +4833,7 @@ ACTOR Future fetchKeys(StorageServer* data, AddingShard* shard) { FetchInjectionInfo* batch = wait(p.getFuture()); TraceEvent(SevDebug, "FKUpdateBatch", data->thisServerID).detail("FKID", interval.pairID); - shard->phase = AddingShard::Waiting; + shard->phase = AddingShard::FetchingCF; // Choose a transferredVersion. This choice and timing ensure that // * The transferredVersion can be mutated in versionedData @@ -4903,17 +4912,20 @@ ACTOR Future fetchKeys(StorageServer* data, AddingShard* shard) { } } - setAvailableStatus(data, - keys, - true); // keys will be available when getLatestVersion()==transferredVersion is durable + shard->phase = AddingShard::Waiting; // Similar to transferred version, but wait for all feed data and Version feedTransferredVersion = data->version.get() + 1; + TraceEvent(SevDebug, "FetchKeysHaveFeedData", data->thisServerID) .detail("FKID", interval.pairID) .detail("Version", feedTransferredVersion) .detail("StorageVersion", data->storageVersion()); + setAvailableStatus(data, + keys, + true); // keys will be available when getLatestVersion()==transferredVersion is durable + // Note that since it receives a pointer to FetchInjectionInfo, the thread does not leave this actor until this // point. @@ -4957,7 +4969,7 @@ ACTOR Future fetchKeys(StorageServer* data, AddingShard* shard) { TraceEvent(SevDebug, interval.end(), data->thisServerID).error(e, true).detail("Version", data->version.get()); if (e.code() == error_code_actor_cancelled && !data->shuttingDown && shard->phase >= AddingShard::Fetching) { - if (shard->phase < AddingShard::Waiting) { + if (shard->phase < AddingShard::FetchingCF) { data->storage.clearRange(keys); data->byteSampleApplyClear(keys, invalidVersion); } else { @@ -5012,7 +5024,7 @@ void AddingShard::addMutation(Version version, bool fromFetch, MutationRef const } // Add the mutation to the version. updates.back().mutations.push_back_deep(updates.back().arena(), mutation); - } else if (phase == Waiting) { + } else if (phase == FetchingCF || phase == Waiting) { server->addMutation(version, fromFetch, mutation, keys, server->updateEagerReads); } else ASSERT(false); @@ -6366,7 +6378,7 @@ void setAvailableStatus(StorageServer* self, KeyRangeRef keys, bool available) { availableKeys.begin, available ? LiteralStringRef("1") : LiteralStringRef("0"))); if (keys.end != allKeys.end) { - bool endAvailable = self->shards.rangeContaining(keys.end)->value()->isInVersionedData(); + bool endAvailable = self->shards.rangeContaining(keys.end)->value()->isCFInVersionedData(); self->addMutationToMutationLog(mLV, MutationRef(MutationRef::SetValue, availableKeys.end, From 7f3f77fd77f6b441713a920632b68490f6b3e07f Mon Sep 17 00:00:00 2001 From: Josh Slocum Date: Mon, 14 Feb 2022 11:29:21 -0600 Subject: [PATCH 251/413] Not cleaning up granule locks immediately after split --- fdbserver/BlobWorker.actor.cpp | 8 +++++++- 1 file changed, 7 insertions(+), 1 deletion(-) diff --git a/fdbserver/BlobWorker.actor.cpp b/fdbserver/BlobWorker.actor.cpp index 31840cf9b9..dbe0e4f7cf 100644 --- a/fdbserver/BlobWorker.actor.cpp +++ b/fdbserver/BlobWorker.actor.cpp @@ -383,7 +383,13 @@ ACTOR Future updateGranuleSplitState(Transaction* tr, // FIXME: appears change feed destroy isn't working! ADD BACK // wait(updateChangeFeed(tr, KeyRef(parentGranuleID.toString()), ChangeFeedStatus::CHANGE_FEED_DESTROY)); Key oldGranuleLockKey = blobGranuleLockKeyFor(parentGranuleRange); - tr->clear(singleKeyRange(oldGranuleLockKey)); + // FIXME: deleting granule lock can cause races where another granule with the same range starts way later + // and thinks it can own the granule! Need to change file cleanup to destroy these, if there is no more + // granule in the history with that exact key range! + // Alternative fix could be to, on granule open, query for all overlapping granule locks and ensure none of + // them have higher (epoch, seqno), but that is much more expensive + + // tr->clear(singleKeyRange(oldGranuleLockKey)); tr->clear(currentRange); tr->clear(blobGranuleSplitBoundaryKeyRangeFor(parentGranuleID)); } else { From b5ff2006b27162c0b15f6b46793556c682ab47e2 Mon Sep 17 00:00:00 2001 From: Josh Slocum Date: Mon, 14 Feb 2022 11:36:41 -0600 Subject: [PATCH 252/413] Removing incorrect assert --- 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 b1dbace948..487e928fe4 100644 --- a/fdbserver/storageserver.actor.cpp +++ b/fdbserver/storageserver.actor.cpp @@ -4464,7 +4464,6 @@ ACTOR Future> fetchChangeFeedMetadata(StorageServer* data, KeyR if (feedCleanup != data->changeFeedCleanupDurable.end() && changeFeedInfo->removing) { TEST(true); // re-fetching feed scheduled for deletion! Un-mark it as removing - ASSERT(fetchVersion > feedCleanup->second); changeFeedInfo->emptyVersion = cfEntry.emptyVersion; changeFeedInfo->stopped = cfEntry.stopped; changeFeedInfo->removing = false; @@ -4477,6 +4476,7 @@ ACTOR Future> fetchChangeFeedMetadata(StorageServer* data, KeyR .detail("Range", cfEntry.range.toString()) .detail("FetchVersion", fetchVersion) .detail("EmptyVersion", cfEntry.emptyVersion) + .detail("CleanupVersion", feedCleanup->second) .detail("Stopped", cfEntry.stopped); } else if (changeFeedInfo->emptyVersion < cfEntry.emptyVersion) { TEST(true); // Got updated CF emptyVersion from a parallel fetchChangeFeedMetadata From abdbc5aafb797a64a38ab87d8bc8ee84bde38165 Mon Sep 17 00:00:00 2001 From: Josh Slocum Date: Mon, 14 Feb 2022 13:20:54 -0600 Subject: [PATCH 253/413] Fixing granule opening after not deleting granule locks --- fdbserver/BlobWorker.actor.cpp | 13 ++++++++----- 1 file changed, 8 insertions(+), 5 deletions(-) diff --git a/fdbserver/BlobWorker.actor.cpp b/fdbserver/BlobWorker.actor.cpp index dbe0e4f7cf..597a950c5c 100644 --- a/fdbserver/BlobWorker.actor.cpp +++ b/fdbserver/BlobWorker.actor.cpp @@ -382,6 +382,7 @@ ACTOR Future updateGranuleSplitState(Transaction* tr, // FIXME: appears change feed destroy isn't working! ADD BACK // wait(updateChangeFeed(tr, KeyRef(parentGranuleID.toString()), ChangeFeedStatus::CHANGE_FEED_DESTROY)); + Key oldGranuleLockKey = blobGranuleLockKeyFor(parentGranuleRange); // FIXME: deleting granule lock can cause races where another granule with the same range starts way later // and thinks it can own the granule! Need to change file cleanup to destroy these, if there is no more @@ -417,16 +418,19 @@ ACTOR Future updateGranuleSplitState(Transaction* tr, return Void(); } -// returns the split state for a given granule on granule reassignment. Assumes granule is in fact splitting, by the -// presence of the previous granule's lock key +// Returns the split state for a given granule on granule reassignment, or unknown if it doesn't exist (meaning the +// granule splitting finished) ACTOR Future> getGranuleSplitState(Transaction* tr, UID parentGranuleID, UID currentGranuleID) { Key myStateKey = blobGranuleSplitKeyFor(parentGranuleID, currentGranuleID); Optional st = wait(tr->get(myStateKey)); - ASSERT(st.present()); - return decodeBlobGranuleSplitValue(st.get()); + if (st.present()) { + return decodeBlobGranuleSplitValue(st.get()); + } else { + return std::pair(BlobGranuleSplitState::Unknown, invalidVersion); + } } // writeDelta file writes speculatively in the common case to optimize throughput. It creates the s3 object even though @@ -2626,7 +2630,6 @@ ACTOR Future openGranule(Reference bwData, As // will be set later } else { // this sub-granule is done splitting, no need for split logic. - ASSERT(granuleSplitState.first == BlobGranuleSplitState::Done); info.parentGranule.reset(); } } From c5810b2127d30c86f2e856ae61e33ee30119af12 Mon Sep 17 00:00:00 2001 From: Josh Slocum Date: Mon, 14 Feb 2022 15:45:46 -0600 Subject: [PATCH 254/413] Reducing possibility for large write rate in BGCorrectness --- .../BlobGranuleCorrectnessWorkload.actor.cpp | 12 ++++++++---- 1 file changed, 8 insertions(+), 4 deletions(-) diff --git a/fdbserver/workloads/BlobGranuleCorrectnessWorkload.actor.cpp b/fdbserver/workloads/BlobGranuleCorrectnessWorkload.actor.cpp index d0dd9a061d..8ecaeec694 100644 --- a/fdbserver/workloads/BlobGranuleCorrectnessWorkload.actor.cpp +++ b/fdbserver/workloads/BlobGranuleCorrectnessWorkload.actor.cpp @@ -166,11 +166,15 @@ struct BlobGranuleCorrectnessWorkload : TestWorkload { (targetDirectories / clientCount) + ((targetDirectories % clientCount > clientId) ? 1 : 0); if (targetMyDirectories > 0) { - int myDirectories = deterministicRandom()->randomInt(1, 2 * targetMyDirectories + 1); + int myDirectories = 1; + if (targetMyDirectories > 1) { + myDirectories = deterministicRandom()->randomInt(1, 2 * targetMyDirectories + 1); + } - // anywhere from 2 delta files per second to 1 delta file every 4 seconds, spread across all directories - targetByteRate = 2 * SERVER_KNOBS->BG_DELTA_FILE_TARGET_BYTES / (1 + (randomness % 8)); - randomness /= 8; + // anywhere from 2 delta files per second to 1 delta file every 2 seconds, spread across all directories + int denom = std::min(clientCount, (int)targetDirectories); + targetByteRate = 2 * SERVER_KNOBS->BG_DELTA_FILE_TARGET_BYTES / (1 + (randomness % 4)) / denom; + randomness /= 4; // either do equal across all of my directories, or skewed bool skewed = myDirectories > 1 && deterministicRandom()->random01() < 0.4; From 53573e764822528d93c94b6f70ae01ee1831329c Mon Sep 17 00:00:00 2001 From: Josh Slocum Date: Tue, 15 Feb 2022 14:11:46 -0600 Subject: [PATCH 255/413] Don't reset CF fetch version on pop --- fdbserver/storageserver.actor.cpp | 6 ------ 1 file changed, 6 deletions(-) diff --git a/fdbserver/storageserver.actor.cpp b/fdbserver/storageserver.actor.cpp index 487e928fe4..143f71863b 100644 --- a/fdbserver/storageserver.actor.cpp +++ b/fdbserver/storageserver.actor.cpp @@ -4159,9 +4159,6 @@ ACTOR Future changeFeedPopQ(StorageServer* self, ChangeFeedPopRequest req) feed->second->storageVersion = invalidVersion; feed->second->durableVersion = invalidVersion; } - if (req.version > feed->second->fetchVersion) { - feed->second->fetchVersion = invalidVersion; - } } wait(self->durableVersion.whenAtLeast(durableVersion)); } @@ -5531,9 +5528,6 @@ private: feed->second->storageVersion = invalidVersion; feed->second->durableVersion = invalidVersion; } - if (popVersion > feed->second->fetchVersion) { - feed->second->fetchVersion = invalidVersion; - } } addMutationToLog = true; } From 26480c4705f0f2370e53342da03f0a8551422766 Mon Sep 17 00:00:00 2001 From: Josh Slocum Date: Tue, 15 Feb 2022 15:11:32 -0600 Subject: [PATCH 256/413] change feed popping fixes --- fdbclient/NativeAPI.actor.cpp | 3 +- fdbserver/storageserver.actor.cpp | 62 +++++++++++++++++++++++++++++-- 2 files changed, 61 insertions(+), 4 deletions(-) diff --git a/fdbclient/NativeAPI.actor.cpp b/fdbclient/NativeAPI.actor.cpp index 8440759bd8..cbd3f36e3e 100644 --- a/fdbclient/NativeAPI.actor.cpp +++ b/fdbclient/NativeAPI.actor.cpp @@ -8053,7 +8053,8 @@ ACTOR Future getChangeFeedStreamActor(Reference db, results->streams.clear(); results->storageData.clear(); if (e.code() == error_code_change_feed_popped) { - results->refresh.sendError(change_feed_popped()); + results->mutations.sendError(e); + results->refresh.sendError(e); } else { results->refresh.sendError(change_feed_cancelled()); } diff --git a/fdbserver/storageserver.actor.cpp b/fdbserver/storageserver.actor.cpp index 143f71863b..6fe4aa18fc 100644 --- a/fdbserver/storageserver.actor.cpp +++ b/fdbserver/storageserver.actor.cpp @@ -1761,6 +1761,15 @@ MutationsAndVersionRef filterMutations(Arena& arena, ssId.toString().substr(0, 4) == DEBUG_SS_ID&& cfId.printable().substr(0, 6) == DEBUG_SS_CF_ID && \ (v >= DEBUG_SS_CF_BEGIN_VERSION || latestVersion == DEBUG_SS_CF_BEGIN_VERSION) +#define DO_DEBUG_CF_MISSING false +#define DEBUG_CF_MISSING_CF ""_sr +#define DEBUG_CF_MISSING_KEY ""_sr +#define DEBUG_CF_MISSING_VERSION invalidVersion +#define DEBUG_CF_MISSING(cfId, keyRange, beginVersion, lastVersion) \ + DO_DEBUG_CF_MISSING&& cfId.printable().substr(0, 6) == \ + DEBUG_CF_MISSING_CF&& keyRange.contains(DEBUG_CF_MISSING_KEY) && \ + beginVersion <= DEBUG_CF_MISSING_VERSION&& lastVersion >= DEBUG_CF_MISSING_VERSION + ACTOR Future> getChangeFeedMutations(StorageServer* data, ChangeFeedStreamRequest req, bool inverted, @@ -1868,7 +1877,7 @@ ACTOR Future> getChangeFeedMutations(Stor } state bool readDurable = feedInfo->durableVersion != invalidVersion && req.begin <= feedInfo->durableVersion; - state bool readFetched = req.begin <= fetchStorageVersion; + state bool readFetched = req.begin <= fetchStorageVersion && !atLatest; if (req.end > emptyVersion + 1 && (readDurable || readFetched)) { if (readFetched && req.begin <= feedInfo->fetchVersion) { // Request needs data that has been written to storage by a change feed fetch, but not committed yet @@ -2032,8 +2041,55 @@ ACTOR Future> getChangeFeedMutations(Stor .detail("Count", reply.mutations.size()); } - // If the SS's version advanced at all during any of the waits, the read from memory may have missed some mutations, - // so gotAll can only be true if data->version didn't change over the course of this actor + if (DEBUG_CF_MISSING(req.rangeID, req.range, req.begin, reply.mutations.back().version) && !req.canReadPopped) { + bool foundVersion = false; + bool foundKey = false; + for (auto& it : reply.mutations) { + if (it.version == DEBUG_CF_MISSING_VERSION) { + foundVersion = true; + for (auto& m : it.mutations) { + if (m.type == MutationRef::SetValue && m.param1 == DEBUG_CF_MISSING_KEY) { + foundKey = true; + break; + } + } + break; + } + } + if (!foundVersion || !foundKey) { + printf("ERROR: SS %s CF %s missing %s @ %lld from request for [%s - %s) %lld - %lld\n", + data->thisServerID.toString().substr(0, 4).c_str(), + req.rangeID.printable().substr(0, 6).c_str(), + foundVersion ? "key" : "version", + DEBUG_CF_MISSING_VERSION, + req.range.begin.printable().c_str(), + req.range.end.printable().c_str(), + req.begin, + req.end); + printf("ERROR: %d versions in response %lld - %lld:\n", + reply.mutations.size(), + reply.mutations.front().version, + reply.mutations.back().version); + for (auto& it : reply.mutations) { + printf("ERROR: %lld (%d)%s\n", + it.version, + it.mutations.size(), + it.version == DEBUG_CF_MISSING_VERSION ? "<-------" : ""); + } + } else { + printf("DBG: SS %s CF %s correct @ %lld from request for [%s - %s) %lld - %lld\n", + data->thisServerID.toString().substr(0, 4).c_str(), + req.rangeID.printable().substr(0, 6).c_str(), + DEBUG_CF_MISSING_VERSION, + req.range.begin.printable().c_str(), + req.range.end.printable().c_str(), + req.begin, + req.end); + } + } + + // If the SS's version advanced at all during any of the waits, the read from memory may have missed some + // mutations, so gotAll can only be true if data->version didn't change over the course of this actor return std::make_pair(reply, gotAll); } From bfb1902a7ab7ba70e6b9b968ac1fe0d6e9578573 Mon Sep 17 00:00:00 2001 From: Josh Slocum Date: Wed, 16 Feb 2022 11:34:22 -0600 Subject: [PATCH 257/413] Fixing logic for when granule split is valid --- fdbserver/BlobWorker.actor.cpp | 7 ++++--- 1 file changed, 4 insertions(+), 3 deletions(-) diff --git a/fdbserver/BlobWorker.actor.cpp b/fdbserver/BlobWorker.actor.cpp index 597a950c5c..9f81cff53b 100644 --- a/fdbserver/BlobWorker.actor.cpp +++ b/fdbserver/BlobWorker.actor.cpp @@ -1334,6 +1334,7 @@ ACTOR Future blobGranuleUpdateFiles(Reference bwData, metadata->durableDeltaVersion.set(startVersion); metadata->pendingDeltaVersion = startVersion; metadata->bufferedDeltaVersion = startVersion; + metadata->knownCommittedVersion = startVersion; Reference newCFData = makeReference(); @@ -1556,7 +1557,7 @@ ACTOR Future blobGranuleUpdateFiles(Reference bwData, if (BW_DEBUG) { fmt::print( - "Granule [{0} - {1}) on BW {2} skipping rollback {0} -> {1} completely\n", + "Granule [{0} - {1}) on BW {2} skipping rollback {3} -> {4} completely\n", metadata->keyRange.begin.printable().c_str(), metadata->keyRange.end.printable().c_str(), bwData->id.toString().substr(0, 5).c_str(), @@ -1570,7 +1571,7 @@ ACTOR Future blobGranuleUpdateFiles(Reference bwData, rollbacksCompleted.push_back(std::pair(rollbackVersion, deltas.version)); } else { if (BW_DEBUG) { - fmt::print("[{0} - {1}) on BW {2} ROLLBACK @ {2} -> {3}\n", + fmt::print("[{0} - {1}) on BW {2} ROLLBACK @ {3} -> {4}\n", metadata->keyRange.begin.printable(), metadata->keyRange.end.printable(), bwData->id.toString().substr(0, 5).c_str(), @@ -1751,7 +1752,7 @@ ACTOR Future blobGranuleUpdateFiles(Reference bwData, // If we have enough delta files, try to re-snapshot if (snapshotEligible && metadata->bytesInNewDeltaFiles >= SERVER_KNOBS->BG_DELTA_BYTES_BEFORE_COMPACT && - !readOldChangeFeed) { + metadata->pendingDeltaVersion >= startState.changeFeedStartVersion) { if (BW_DEBUG && !inFlightFiles.empty()) { fmt::print("Granule [{0} - {1}) ready to re-snapshot after {2} > {3} bytes, waiting for " "outstanding {4} files to finish\n", From 662618a5bceb3419feb73b049bb084891aaecf5c Mon Sep 17 00:00:00 2001 From: Josh Slocum Date: Thu, 17 Feb 2022 10:01:10 -0600 Subject: [PATCH 258/413] Blob Manager lock fixes --- fdbserver/BlobManager.actor.cpp | 14 ++++++++++---- 1 file changed, 10 insertions(+), 4 deletions(-) diff --git a/fdbserver/BlobManager.actor.cpp b/fdbserver/BlobManager.actor.cpp index afe3d3c6b0..c2648b810d 100644 --- a/fdbserver/BlobManager.actor.cpp +++ b/fdbserver/BlobManager.actor.cpp @@ -612,6 +612,7 @@ ACTOR Future checkManagerLock(Reference tr, Ref throw blob_manager_replaced(); } tr->addReadConflictRange(singleKeyRange(blobManagerEpochKey)); + tr->addWriteConflictRange(singleKeyRange(blobManagerEpochKey)); return Void(); } @@ -630,10 +631,12 @@ ACTOR Future writeInitialGranuleMapping(Reference bmData, try { tr->setOption(FDBTransactionOptions::Option::PRIORITY_SYSTEM_IMMEDIATE); tr->setOption(FDBTransactionOptions::Option::ACCESS_SYSTEM_KEYS); + wait(checkManagerLock(tr, bmData)); while (i + j < boundaries.size() - 1 && j < transactionChunkSize) { // TODO REMOVE if (BM_DEBUG) { - fmt::print("Persisting initial mapping for [{0} - {1})\n", + fmt::print("BM {0} Persisting initial mapping for [{1} - {2})\n", + bmData->epoch, boundaries[i + j].printable(), boundaries[i + j + 1].printable()); } @@ -647,7 +650,8 @@ ACTOR Future writeInitialGranuleMapping(Reference bmData, wait(tr->commit()); if (BM_DEBUG) { for (int k = 0; k < j; k++) { - fmt::print("Persisted initial mapping for [{0} - {1})\n", + fmt::print("BM {0} Persisted initial mapping for [{1} - {2})\n", + bmData->epoch, boundaries[i + k].printable(), boundaries[i + k + 1].printable()); } @@ -655,7 +659,7 @@ ACTOR Future writeInitialGranuleMapping(Reference bmData, break; } catch (Error& e) { if (BM_DEBUG) { - fmt::print("Persisting initial mapping got error {}\n", e.name()); + fmt::print("BM {} Persisting initial mapping got error {}\n", bmData->epoch, e.name()); } wait(tr->onError(e)); j = 0; @@ -1782,6 +1786,7 @@ ACTOR Future recoverBlobManager(Reference bmData) { tr->setOption(FDBTransactionOptions::ACCESS_SYSTEM_KEYS); tr->setOption(FDBTransactionOptions::PRIORITY_SYSTEM_IMMEDIATE); wait(checkManagerLock(tr, bmData)); + wait(tr->commit()); break; } catch (Error& e) { if (BM_DEBUG) { @@ -2740,6 +2745,7 @@ ACTOR Future doLockChecks(Reference bmData) { tr->setOption(FDBTransactionOptions::ACCESS_SYSTEM_KEYS); tr->setOption(FDBTransactionOptions::PRIORITY_SYSTEM_IMMEDIATE); wait(checkManagerLock(tr, bmData)); + wait(tr->commit()); break; } catch (Error& e) { if (e.code() == error_code_granule_assignment_conflict) { @@ -2801,7 +2807,7 @@ ACTOR Future blobManager(BlobManagerInterface bmInterf, loop choose { when(wait(self->iAmReplaced.getFuture())) { if (BM_DEBUG) { - printf("Blob Manager exiting because it is replaced\n"); + fmt::print("BM {} exiting because it is replaced\n", self->epoch); } break; } From f60420a8e69c426370f1a6b15ec689eda5582d85 Mon Sep 17 00:00:00 2001 From: Josh Slocum Date: Thu, 17 Feb 2022 10:36:22 -0600 Subject: [PATCH 259/413] Correctly updating change feed storage/durable versions after fetch --- fdbserver/storageserver.actor.cpp | 27 +++++++++++++++++++++------ 1 file changed, 21 insertions(+), 6 deletions(-) diff --git a/fdbserver/storageserver.actor.cpp b/fdbserver/storageserver.actor.cpp index 6fe4aa18fc..075deee550 100644 --- a/fdbserver/storageserver.actor.cpp +++ b/fdbserver/storageserver.actor.cpp @@ -1849,6 +1849,20 @@ ACTOR Future> getChangeFeedMutations(Stor fetchStorageVersion, feedInfo->fetchVersion, feedInfo->durableFetchVersion.get()); + TraceEvent(SevDebug, "ChangeFeedMutationsDetails", data->thisServerID) + .detail("FeedID", req.rangeID) + .detail("StreamUID", streamUID) + .detail("Range", req.range) + .detail("Begin", req.begin) + .detail("End", req.end) + .detail("AtLatest", atLatest) + .detail("DequeVersion", dequeVersion) + .detail("EmptyVersion", feedInfo->emptyVersion) + .detail("StorageVersion", feedInfo->storageVersion) + .detail("DurableVersion", feedInfo->durableVersion) + .detail("FetchStorageVersion", fetchStorageVersion) + .detail("FetchVersion", feedInfo->fetchVersion) + .detail("DurableFetchVersion", feedInfo->durableFetchVersion.get()); } if (req.end > emptyVersion + 1) { @@ -2038,7 +2052,8 @@ ACTOR Future> getChangeFeedMutations(Stor .detail("End", req.end) .detail("FirstVersion", reply.mutations.empty() ? invalidVersion : reply.mutations.front().version) .detail("LastVersion", reply.mutations.empty() ? invalidVersion : reply.mutations.back().version) - .detail("Count", reply.mutations.size()); + .detail("Count", reply.mutations.size()) + .detail("GotAll", gotAll); } if (DEBUG_CF_MISSING(req.rangeID, req.range, req.begin, reply.mutations.back().version) && !req.canReadPopped) { @@ -6219,11 +6234,11 @@ ACTOR Future updateStorage(StorageServer* data) { feedFetchVersions.push_back(std::pair(info->second->id, info->second->fetchVersion)); } // handle case where fetch had version ahead of last in-memory mutation - if (info->second->fetchVersion > info->second->storageVersion) { - info->second->storageVersion = std::min(info->second->fetchVersion, newOldestVersion); - if (info->second->fetchVersion > info->second->storageVersion) { - // This change feed still has pending mutations fetched and written to storage that are not yet - // durable. To ensure its storageVersion gets updated once its fetchVersion is durable, we need + if (alreadyFetched > info->second->storageVersion) { + info->second->storageVersion = std::min(alreadyFetched, newOldestVersion); + if (alreadyFetched > info->second->storageVersion) { + // This change feed still has pending mutations fetched and written to storage that are higher + // than the new durableVersion. To ensure its storage and durable version get updated, we need // to add it back to fetchingChangeFeeds data->fetchingChangeFeeds.insert(info->first); } From c87d868dd24752fedcdb951758dfe71038093060 Mon Sep 17 00:00:00 2001 From: Josh Slocum Date: Thu, 17 Feb 2022 15:55:24 -0600 Subject: [PATCH 260/413] Fix SS restart and change feed pop races --- fdbserver/storageserver.actor.cpp | 29 ++++++++++++++++++----------- 1 file changed, 18 insertions(+), 11 deletions(-) diff --git a/fdbserver/storageserver.actor.cpp b/fdbserver/storageserver.actor.cpp index 075deee550..6f41126e41 100644 --- a/fdbserver/storageserver.actor.cpp +++ b/fdbserver/storageserver.actor.cpp @@ -2072,9 +2072,10 @@ ACTOR Future> getChangeFeedMutations(Stor } } if (!foundVersion || !foundKey) { - printf("ERROR: SS %s CF %s missing %s @ %lld from request for [%s - %s) %lld - %lld\n", + printf("ERROR: SS %s CF %s SQ %s missing %s @ %lld from request for [%s - %s) %lld - %lld\n", data->thisServerID.toString().substr(0, 4).c_str(), req.rangeID.printable().substr(0, 6).c_str(), + streamUID.toString().substr(0, 8).c_str(), foundVersion ? "key" : "version", DEBUG_CF_MISSING_VERSION, req.range.begin.printable().c_str(), @@ -2092,9 +2093,10 @@ ACTOR Future> getChangeFeedMutations(Stor it.version == DEBUG_CF_MISSING_VERSION ? "<-------" : ""); } } else { - printf("DBG: SS %s CF %s correct @ %lld from request for [%s - %s) %lld - %lld\n", + printf("DBG: SS %s CF %s SQ %s correct @ %lld from request for [%s - %s) %lld - %lld\n", data->thisServerID.toString().substr(0, 4).c_str(), req.rangeID.printable().substr(0, 6).c_str(), + streamUID.toString().substr(0, 8).c_str(), DEBUG_CF_MISSING_VERSION, req.range.begin.printable().c_str(), req.range.end.printable().c_str(), @@ -3906,7 +3908,8 @@ void applyMutation(StorageServer* self, void applyChangeFeedMutation(StorageServer* self, MutationRef const& m, Version version) { if (m.type == MutationRef::SetValue) { for (auto& it : self->keyChangeFeed[m.param1]) { - if (!it->stopped && version > it->emptyVersion) { + if (!it->stopped) { + ASSERT(version > it->emptyVersion); if (it->mutations.empty() || it->mutations.back().version != version) { it->mutations.push_back(MutationsAndVersionRef(version, self->knownCommittedVersion)); } @@ -3919,7 +3922,7 @@ void applyChangeFeedMutation(StorageServer* self, MutationRef const& m, Version } else { TEST(it->stopped); // Skip CF write because stopped TEST(version <= it->emptyVersion); // Skip CF write because popped and SS behind - DEBUG_MUTATION("ChangeFeedIgnoreSet", version, m, self->thisServerID) + DEBUG_MUTATION("ChangeFeedWriteSetIgnore", version, m, self->thisServerID) .detail("Range", it->range) .detail("ChangeFeedID", it->id) .detail("Stopped", it->stopped) @@ -3930,7 +3933,8 @@ void applyChangeFeedMutation(StorageServer* self, MutationRef const& m, Version auto ranges = self->keyChangeFeed.intersectingRanges(KeyRangeRef(m.param1, m.param2)); for (auto& r : ranges) { for (auto& it : r.value()) { - if (!it->stopped && version > it->emptyVersion) { + if (!it->stopped) { + ASSERT(version > it->emptyVersion); if (it->mutations.empty() || it->mutations.back().version != version) { it->mutations.push_back(MutationsAndVersionRef(version, self->knownCommittedVersion)); } @@ -3942,7 +3946,7 @@ void applyChangeFeedMutation(StorageServer* self, MutationRef const& m, Version } else { TEST(it->stopped); // Skip CF clear because stopped TEST(version <= it->emptyVersion); // Skip CF clear because popped and SS behind - DEBUG_MUTATION("ChangeFeedIgnoreClear", version, m, self->thisServerID) + DEBUG_MUTATION("ChangeFeedWriteClearIgnore", version, m, self->thisServerID) .detail("Range", it->range) .detail("ChangeFeedID", it->id) .detail("Stopped", it->stopped) @@ -4190,13 +4194,10 @@ static const KeyRangeRef persistChangeFeedKeys = // data keys are unmangled (but never start with PERSIST_PREFIX because they are always in allKeys) ACTOR Future changeFeedPopQ(StorageServer* self, ChangeFeedPopRequest req) { + // if a SS restarted and is way behind, wait for it to at least have caught up through the pop version + wait(self->version.whenAtLeast(req.version)); wait(delay(0)); - TraceEvent(SevDebug, "ChangeFeedPopQuery", self->thisServerID) - .detail("RangeID", req.rangeID.printable()) - .detail("Version", req.version) - .detail("Range", req.range.toString()); - if (!self->isReadable(req.range)) { req.reply.sendError(wrong_shard_server()); return Void(); @@ -4206,6 +4207,12 @@ ACTOR Future changeFeedPopQ(StorageServer* self, ChangeFeedPopRequest req) req.reply.sendError(unknown_change_feed()); return Void(); } + + TraceEvent(SevDebug, "ChangeFeedPopQuery", self->thisServerID) + .detail("RangeID", req.rangeID.printable()) + .detail("Version", req.version) + .detail("Range", req.range.toString()); + if (req.version - 1 > feed->second->emptyVersion) { feed->second->emptyVersion = req.version - 1; while (!feed->second->mutations.empty() && feed->second->mutations.front().version < req.version) { From fc0c0ac21ed35460d2217702909cc9735b12126a Mon Sep 17 00:00:00 2001 From: Josh Slocum Date: Thu, 17 Feb 2022 16:09:05 -0600 Subject: [PATCH 261/413] script for debugging MemSample traces --- contrib/alloc_instrumentation_traces.py | 100 ++++++++++++++++++++++++ 1 file changed, 100 insertions(+) create mode 100755 contrib/alloc_instrumentation_traces.py diff --git a/contrib/alloc_instrumentation_traces.py b/contrib/alloc_instrumentation_traces.py new file mode 100755 index 0000000000..42268dc700 --- /dev/null +++ b/contrib/alloc_instrumentation_traces.py @@ -0,0 +1,100 @@ +#!/usr/bin/env python3 +# +# alloc_instrumentation_traces.py +# +# 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. +# + + +""" +Example trace: +{ "Severity": "10", "Time": "194.878474", "DateTime": "2022-02-01T16:28:27Z", "Type": "MemSample", "Machine": "2.1.1.0:2", "ID": "0000000000000000", "Count": "943", "TotalSize": "540000000", "SampleCount": "54", "Hash": "980074757", "Bt": "addr2line -e fdbserver.debug -p -C -f -i 0x1919b72 0x3751d43 0x37518cc 0x19930f8 0x199dac3 0x1999e7c 0x21a1061 0x31e8fc5 0x31e784a 0x10ab3a8 0x36bf4c6 0x36bf304 0x36beea4 0x36bf352 0x36bfa1c 0x10ab3a8 0x37b22fe 0x37a16ee 0x368c754 0x19202d5 0x7fb3fe2d6555 0x1077029", "ThreadID": "10074331651862410074", "LogGroup": "default" } +""" + + +# This program analyzes MemSample trace events produced by setting ALLOC_INSTRUMENTATION in FastAlloc.h +# It outputs the top memory users by total size as well as number of allocations. + +# Example usage: cat trace.* | ./alloc_instrumentation_traces.py + +import sys +import json + +byCnt = [] +bySize = [] +totalSize = 0 + +lastTimestamp = "" + +for line in sys.stdin: + ev = json.loads(line.rstrip()) + type = ev["Type"] + + if (type != 'MemSample'): + continue + bt = ev["Bt"] + + if (bt == "na"): + continue + + timestamp = ev["Time"] + cnt = int(ev["Count"]) + scnt = int(ev["SampleCount"]) + size = int(ev["TotalSize"]) + h = ev["Hash"] + + if (timestamp != lastTimestamp): + byCnt = [] + bySize = [] + totalSize = 0 + lastTimestamp = timestamp + + + # print(str(cnt) + " " + str(scnt) + " " + str(size) + " " + h) + + byCnt.append( (cnt, scnt, size, h, bt) ) + bySize.append( (size, cnt, size, h, bt) ) + totalSize += size + +byCnt.sort(reverse=True) +bySize.sort(reverse=True) + +btByHash = {} + +byte_suffix = ["Bytes", "KB", "MB", "GB", "TB"] +def byte_str(bytes): + suffix_idx = 0 + while (bytes >= 1024 * 10): + suffix_idx += 1 + bytes //= 1024 + return str(bytes) + ' ' + byte_suffix[suffix_idx] + +print("By Size") +print("-------\r\n") +for x in bySize[:10]: + # print(str(x[0]) + ": " + x[3]) + print(str(x[1]) + " / " + byte_str(x[0]) + " (" + byte_str(x[0] // x[1]) + " per alloc):\r\n" + x[4] + "\r\n") + btByHash[x[3]] = x[4] + +print() +print("By Count") +print("--------\r\n") +for x in byCnt[:5]: + # print(str(x[0]) + ": " + x[3]) + print(str(x[0]) + " / " + byte_str(x[2]) + " (" + byte_str(x[2] // x[0]) + " per alloc):\r\n" + x[4] + "\r\n") + btByHash[x[3]] = x[4] + From ef25e4fdfbe5e206bb157793ce7c250573935315 Mon Sep 17 00:00:00 2001 From: Josh Slocum Date: Thu, 17 Feb 2022 18:10:02 -0600 Subject: [PATCH 262/413] Sending empty version back immediately to establish stream connection and acknowledgement queue for change feed stream requests --- fdbclient/NativeAPI.actor.cpp | 12 +++++++----- fdbserver/storageserver.actor.cpp | 17 +++++++++++++++++ 2 files changed, 24 insertions(+), 5 deletions(-) diff --git a/fdbclient/NativeAPI.actor.cpp b/fdbclient/NativeAPI.actor.cpp index 535eccf701..68ced7ed6e 100644 --- a/fdbclient/NativeAPI.actor.cpp +++ b/fdbclient/NativeAPI.actor.cpp @@ -7412,6 +7412,7 @@ Future ChangeFeedData::whenAtLeast(Version version) { ACTOR Future partialChangeFeedStream(StorageServerInterface interf, PromiseStream> results, ReplyPromiseStream replyStream, + Version begin, Version end, Reference feedData, Reference storageData, @@ -7422,7 +7423,7 @@ ACTOR Future partialChangeFeedStream(StorageServerInterface interf, // calling lastReturnedVersion's callbacks could cause us to be cancelled state Promise refresh = feedData->refresh; state bool atLatestVersion = false; - state Version nextVersion = 0; + state Version nextVersion = begin; try { loop { if (nextVersion >= end) { @@ -7454,10 +7455,10 @@ ACTOR Future partialChangeFeedStream(StorageServerInterface interf, rep.minStreamVersion); } - // TODO REMOVE, just for debugging - // set next version so debug statements trigger - if (nextVersion == 0) { - nextVersion = rep.mutations.front().version; + // handle first empty mutation on stream establishment explicitly + if (nextVersion == begin && rep.mutations.size() == 1 && rep.mutations[0].mutations.size() == 0 && + rep.mutations[0].version == begin - 1) { + continue; } if (rep.mutations.back().version > feedData->maxSeenVersion) { @@ -7752,6 +7753,7 @@ ACTOR Future mergeChangeFeedStream(Reference db, fetchers[i] = partialChangeFeedStream(interfs[i].first, streams[i].results, results->streams[i], + *begin, end, results, results->storageData[i], diff --git a/fdbserver/storageserver.actor.cpp b/fdbserver/storageserver.actor.cpp index 30c301514e..cbe1011770 100644 --- a/fdbserver/storageserver.actor.cpp +++ b/fdbserver/storageserver.actor.cpp @@ -2252,6 +2252,23 @@ ACTOR Future changeFeedStreamQ(StorageServer* data, ChangeFeedStreamReques req.canReadPopped ? "T" : "F"); } + // send an empty version at begin - 1 to establish the stream quickly + ChangeFeedStreamReply emptyInitialReply; + MutationsAndVersionRef emptyInitialVersion; + emptyInitialVersion.version = req.begin - 1; + emptyInitialReply.mutations.push_back_deep(emptyInitialReply.arena, emptyInitialVersion); + ASSERT(emptyInitialReply.atLatestVersion == false); + ASSERT(emptyInitialReply.minStreamVersion == invalidVersion); + req.reply.send(emptyInitialReply); + + if (DEBUG_SS_CFM(data->thisServerID, req.rangeID, req.begin)) { + printf("CFM: SS %s CF %s: CFSQ %s send empty initial version %lld\n", + data->thisServerID.toString().substr(0, 4).c_str(), + req.rangeID.printable().substr(0, 6).c_str(), + streamUID.toString().substr(0, 8).c_str(), + req.begin - 1); + } + try { data->activeFeedQueries++; loop { From a7a3dccba52f189f2d92b4e342341d951ba57c76 Mon Sep 17 00:00:00 2001 From: Josh Slocum Date: Thu, 17 Feb 2022 14:30:09 -0600 Subject: [PATCH 263/413] TSS Mismatch Quarantine fix --- fdbclient/NativeAPI.actor.cpp | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/fdbclient/NativeAPI.actor.cpp b/fdbclient/NativeAPI.actor.cpp index 68ced7ed6e..af29c4f450 100644 --- a/fdbclient/NativeAPI.actor.cpp +++ b/fdbclient/NativeAPI.actor.cpp @@ -433,7 +433,7 @@ ACTOR Future databaseLogger(DatabaseContext* cx) { for (const auto& it : cx->tssMetrics) { // TODO could skip this whole thing if tss if request counter is zero? // That would potentially complicate elapsed calculation though - if (it.second->mismatches.getIntervalDelta()) { + if (it.second->detailedMismatches.size()) { cx->tssMismatchStream.send( std::pair>(it.first, it.second->detailedMismatches)); } @@ -943,6 +943,8 @@ ACTOR static Future handleTssMismatches(DatabaseContext* cx) { loop { // state std::pair> data = waitNext(cx->tssMismatchStream.getFuture()); + // return to calling actor, don't do this as part of metrics loop + wait(delay(0)); // find ss pair id so we can remove it from the mapping state UID tssPairID; bool found = false; From f9f676abf9dc0433db85da2e62993d9dcfb4955e Mon Sep 17 00:00:00 2001 From: Josh Slocum Date: Thu, 17 Feb 2022 19:15:51 -0600 Subject: [PATCH 264/413] Also re-persisting change feed metadata on fetch after cleanup --- fdbserver/storageserver.actor.cpp | 12 ++++++++++++ 1 file changed, 12 insertions(+) diff --git a/fdbserver/storageserver.actor.cpp b/fdbserver/storageserver.actor.cpp index cbe1011770..67d7f3d4e2 100644 --- a/fdbserver/storageserver.actor.cpp +++ b/fdbserver/storageserver.actor.cpp @@ -4331,6 +4331,7 @@ ACTOR Future changeFeedPopQ(StorageServer* self, ChangeFeedPopRequest req) TraceEvent(SevDebug, "ChangeFeedPopQuery", self->thisServerID) .detail("RangeID", req.rangeID.printable()) .detail("Version", req.version) + .detail("SSVersion", self->version.get()) .detail("Range", req.range.toString()); if (req.version - 1 > feed->second->emptyVersion) { @@ -4662,6 +4663,7 @@ ACTOR Future> fetchChangeFeedMetadata(StorageServer* data, KeyR auto changeFeedInfo = existingEntry->second; auto feedCleanup = data->changeFeedCleanupDurable.find(cfEntry.rangeId); + bool writeToMutationLog = false; if (feedCleanup != data->changeFeedCleanupDurable.end() && changeFeedInfo->removing) { TEST(true); // re-fetching feed scheduled for deletion! Un-mark it as removing changeFeedInfo->emptyVersion = cfEntry.emptyVersion; @@ -4678,9 +4680,19 @@ ACTOR Future> fetchChangeFeedMetadata(StorageServer* data, KeyR .detail("EmptyVersion", cfEntry.emptyVersion) .detail("CleanupVersion", feedCleanup->second) .detail("Stopped", cfEntry.stopped); + + // Since cleanup put a mutation in the log to delete the change feed data, put one in the log to restore + // it + // We may just want to refactor this so updateStorage does explicit deletes based on + // changeFeedCleanupDurable and not use the mutation log at all for the change feed metadata cleanup. + // Then we wouldn't have to reset anything here + writeToMutationLog = true; } else if (changeFeedInfo->emptyVersion < cfEntry.emptyVersion) { TEST(true); // Got updated CF emptyVersion from a parallel fetchChangeFeedMetadata changeFeedInfo->emptyVersion = cfEntry.emptyVersion; + writeToMutationLog = true; + } + if (writeToMutationLog) { auto& mLV = data->addVersionToMutationLog(data->data().getLatestVersion()); data->addMutationToMutationLog( mLV, From e75ab5f3a009aab86aeef355516e54289fae58d2 Mon Sep 17 00:00:00 2001 From: Josh Slocum Date: Tue, 22 Feb 2022 11:22:35 -0600 Subject: [PATCH 265/413] not advancing popped version of change feed sideband from another server --- fdbserver/storageserver.actor.cpp | 7 ------- 1 file changed, 7 deletions(-) diff --git a/fdbserver/storageserver.actor.cpp b/fdbserver/storageserver.actor.cpp index 67d7f3d4e2..d123ae4166 100644 --- a/fdbserver/storageserver.actor.cpp +++ b/fdbserver/storageserver.actor.cpp @@ -4686,13 +4686,6 @@ ACTOR Future> fetchChangeFeedMetadata(StorageServer* data, KeyR // We may just want to refactor this so updateStorage does explicit deletes based on // changeFeedCleanupDurable and not use the mutation log at all for the change feed metadata cleanup. // Then we wouldn't have to reset anything here - writeToMutationLog = true; - } else if (changeFeedInfo->emptyVersion < cfEntry.emptyVersion) { - TEST(true); // Got updated CF emptyVersion from a parallel fetchChangeFeedMetadata - changeFeedInfo->emptyVersion = cfEntry.emptyVersion; - writeToMutationLog = true; - } - if (writeToMutationLog) { auto& mLV = data->addVersionToMutationLog(data->data().getLatestVersion()); data->addMutationToMutationLog( mLV, From 5dca00a21d1baaa499993fd7df07f3f0728523cf Mon Sep 17 00:00:00 2001 From: Josh Slocum Date: Tue, 22 Feb 2022 13:48:57 -0600 Subject: [PATCH 266/413] Cleaning up change feed on move triggers when stream is cancelled --- fdbserver/storageserver.actor.cpp | 55 +++++++++++++++++++++---------- 1 file changed, 38 insertions(+), 17 deletions(-) diff --git a/fdbserver/storageserver.actor.cpp b/fdbserver/storageserver.actor.cpp index d123ae4166..25bb838e61 100644 --- a/fdbserver/storageserver.actor.cpp +++ b/fdbserver/storageserver.actor.cpp @@ -371,30 +371,39 @@ struct ChangeFeedInfo : ReferenceCounted { Key id; AsyncTrigger newMutations; NotifiedVersion durableFetchVersion; - Promise onMove; bool stopped = false; // A stopped change feed no longer adds new mutations, but is still queriable bool removing = false; - KeyRangeMap>> moveTriggers; + KeyRangeMap>> moveTriggers; - void triggerOnMove(KeyRange range, Promise p) { + void triggerOnMove(KeyRange range, UID streamUID, Promise p) { auto toInsert = moveTriggers.modify(range); for (auto triggerRange = toInsert.begin(); triggerRange != toInsert.end(); ++triggerRange) { - triggerRange->value().push_back(p); + triggerRange->value().insert({ streamUID, p }); } } void moved(KeyRange range) { auto toTrigger = moveTriggers.intersectingRanges(range); for (auto triggerRange : toTrigger) { - for (int i = 0; i < triggerRange.cvalue().size(); i++) { - if (triggerRange.cvalue()[i].canBeSet()) { - triggerRange.cvalue()[i].send(Void()); + for (auto triggerStream : triggerRange.cvalue()) { + if (triggerStream.second.canBeSet()) { + triggerStream.second.send(Void()); } } } // coalesce doesn't work with promises - moveTriggers.insert(range, std::vector>()); + moveTriggers.insert(range, std::unordered_map>()); + } + + void removeOnMoveTrigger(KeyRange range, UID streamUID) { + auto toRemove = moveTriggers.modify(range); + for (auto triggerRange = toRemove.begin(); triggerRange != toRemove.end(); ++triggerRange) { + auto streamToRemove = triggerRange->value().find(streamUID); + ASSERT(streamToRemove != triggerRange->cvalue().end()); + triggerRange->value().erase(streamToRemove); + } + // TODO: could clean up on } }; @@ -2206,7 +2215,7 @@ ACTOR Future localChangeFeedStream(StorageServer* data, } // 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) { +ACTOR Future stopChangeFeedOnMove(StorageServer* data, ChangeFeedStreamRequest req, UID streamUID) { wait(delay(0, TaskPriority::DefaultEndpoint)); auto feed = data->uidChangeFeed.find(req.rangeID); @@ -2215,21 +2224,29 @@ ACTOR Future stopChangeFeedOnMove(StorageServer* data, ChangeFeedStreamReq return Void(); } state Promise moved; - feed->second->triggerOnMove(req.range, moved); - wait(moved.getFuture()); - printf("CF Moved! %lld - %lld. sending WSS\n", req.begin, req.end); + feed->second->triggerOnMove(req.range, streamUID, moved); + try { + wait(moved.getFuture()); + } catch (Error& e) { + ASSERT(e.code() == error_code_operation_cancelled); + // remove from tracking + + auto feed = data->uidChangeFeed.find(req.rangeID); + if (feed != data->uidChangeFeed.end()) { + feed->second->removeOnMoveTrigger(req.range, streamUID); + } + return Void(); + } + printf("CFSQ %s Moved! %lld - %lld. sending WSS\n", streamUID.toString().substr(0, 8).c_str(), req.begin, req.end); // DO NOT call req.reply.onReady before sending - we need to propagate this error through regardless of how far // behind client is req.reply.sendError(wrong_shard_server()); return Void(); } -ACTOR Future changeFeedStreamQ(StorageServer* data, ChangeFeedStreamRequest req) { +ACTOR Future changeFeedStreamQ(StorageServer* data, ChangeFeedStreamRequest req, UID streamUID) { state Span span("SS:getChangeFeedStream"_loc, { req.spanContext }); state bool atLatest = false; - // TODO CHANGE BACK after BG is correctness clean - // state UID streamUID = deterministicRandom()->randomUniqueID(); - state UID streamUID = req.debugID; state bool removeUID = false; state Optional blockedVersion; if (req.replyBufferSize <= 0) { @@ -7525,7 +7542,11 @@ 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) || stopChangeFeedOnMove(self, req)); + + // TODO CHANGE BACK after BG is correctness clean + // state UID streamUID = deterministicRandom()->randomUniqueID(); + UID streamUID = req.debugID; + self->actors.add(changeFeedStreamQ(self, req, streamUID) || stopChangeFeedOnMove(self, req, streamUID)); } } From d89c6bb3dee4b4a08dd07a33f14c4582197ecb88 Mon Sep 17 00:00:00 2001 From: Josh Slocum Date: Tue, 22 Feb 2022 13:55:30 -0600 Subject: [PATCH 267/413] Fixing initial change feed stream reply to be in try/catch --- fdbserver/storageserver.actor.cpp | 35 ++++++++++++++++--------------- 1 file changed, 18 insertions(+), 17 deletions(-) diff --git a/fdbserver/storageserver.actor.cpp b/fdbserver/storageserver.actor.cpp index 25bb838e61..b250b0439c 100644 --- a/fdbserver/storageserver.actor.cpp +++ b/fdbserver/storageserver.actor.cpp @@ -2269,25 +2269,26 @@ ACTOR Future changeFeedStreamQ(StorageServer* data, ChangeFeedStreamReques req.canReadPopped ? "T" : "F"); } - // send an empty version at begin - 1 to establish the stream quickly - ChangeFeedStreamReply emptyInitialReply; - MutationsAndVersionRef emptyInitialVersion; - emptyInitialVersion.version = req.begin - 1; - emptyInitialReply.mutations.push_back_deep(emptyInitialReply.arena, emptyInitialVersion); - ASSERT(emptyInitialReply.atLatestVersion == false); - ASSERT(emptyInitialReply.minStreamVersion == invalidVersion); - req.reply.send(emptyInitialReply); - - if (DEBUG_SS_CFM(data->thisServerID, req.rangeID, req.begin)) { - printf("CFM: SS %s CF %s: CFSQ %s send empty initial version %lld\n", - data->thisServerID.toString().substr(0, 4).c_str(), - req.rangeID.printable().substr(0, 6).c_str(), - streamUID.toString().substr(0, 8).c_str(), - req.begin - 1); - } - try { data->activeFeedQueries++; + + // send an empty version at begin - 1 to establish the stream quickly + ChangeFeedStreamReply emptyInitialReply; + MutationsAndVersionRef emptyInitialVersion; + emptyInitialVersion.version = req.begin - 1; + emptyInitialReply.mutations.push_back_deep(emptyInitialReply.arena, emptyInitialVersion); + ASSERT(emptyInitialReply.atLatestVersion == false); + ASSERT(emptyInitialReply.minStreamVersion == invalidVersion); + req.reply.send(emptyInitialReply); + + if (DEBUG_SS_CFM(data->thisServerID, req.rangeID, req.begin)) { + printf("CFM: SS %s CF %s: CFSQ %s send empty initial version %lld\n", + data->thisServerID.toString().substr(0, 4).c_str(), + req.rangeID.printable().substr(0, 6).c_str(), + streamUID.toString().substr(0, 8).c_str(), + req.begin - 1); + } + loop { Future onReady = req.reply.onReady(); if (atLatest && !onReady.isReady() && !removeUID) { From 330b2b48ec06664bd9a70e2157f0a12db4ff04b7 Mon Sep 17 00:00:00 2001 From: Evan Tschannen Date: Tue, 22 Feb 2022 12:00:09 -0800 Subject: [PATCH 268/413] improved file cleanup execution and testing --- fdbclient/BlobGranuleCommon.h | 3 +- fdbclient/ServerKnobs.cpp | 1 - fdbclient/ServerKnobs.h | 2 - fdbclient/SystemData.cpp | 10 +- fdbclient/SystemData.h | 5 +- fdbserver/BlobManager.actor.cpp | 292 ++++++------------ fdbserver/BlobWorker.actor.cpp | 4 + .../workloads/BlobGranuleVerifier.actor.cpp | 83 ++++- 8 files changed, 182 insertions(+), 218 deletions(-) diff --git a/fdbclient/BlobGranuleCommon.h b/fdbclient/BlobGranuleCommon.h index 7c44838c18..b33cbabb7b 100644 --- a/fdbclient/BlobGranuleCommon.h +++ b/fdbclient/BlobGranuleCommon.h @@ -77,13 +77,14 @@ struct BlobGranuleChunkRef { constexpr static FileIdentifier file_identifier = 865198; KeyRangeRef keyRange; Version includedVersion; + Version startVersion; Optional snapshotFile; // not set if it's an incremental read VectorRef deltaFiles; GranuleDeltas newDeltas; template void serialize(Ar& ar) { - serializer(ar, keyRange, includedVersion, snapshotFile, deltaFiles, newDeltas); + serializer(ar, keyRange, includedVersion, startVersion, snapshotFile, deltaFiles, newDeltas); } }; diff --git a/fdbclient/ServerKnobs.cpp b/fdbclient/ServerKnobs.cpp index 3ed62aa08c..2c1e93f3c0 100644 --- a/fdbclient/ServerKnobs.cpp +++ b/fdbclient/ServerKnobs.cpp @@ -813,7 +813,6 @@ void ServerKnobs::initialize(Randomize randomize, ClientKnobs* clientKnobs, IsSi init( BG_SNAPSHOT_FILE_TARGET_BYTES, 10000000 ); if( buggifySmallShards || (randomize && BUGGIFY) ) { deterministicRandom()->random01() < 0.1 ? BG_SNAPSHOT_FILE_TARGET_BYTES /= 100 : BG_SNAPSHOT_FILE_TARGET_BYTES /= 10; } init( BG_DELTA_BYTES_BEFORE_COMPACT, BG_SNAPSHOT_FILE_TARGET_BYTES/2 ); init( BG_DELTA_FILE_TARGET_BYTES, BG_DELTA_BYTES_BEFORE_COMPACT/10 ); - init( BG_PRUNE_TIMEOUT, 60*60); 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; diff --git a/fdbclient/ServerKnobs.h b/fdbclient/ServerKnobs.h index cfd7fe4c59..7e41df3756 100644 --- a/fdbclient/ServerKnobs.h +++ b/fdbclient/ServerKnobs.h @@ -767,8 +767,6 @@ public: int BG_DELTA_FILE_TARGET_BYTES; int BG_DELTA_BYTES_BEFORE_COMPACT; - double BG_PRUNE_TIMEOUT; - 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 double BLOB_WORKERLIST_FETCH_INTERVAL; diff --git a/fdbclient/SystemData.cpp b/fdbclient/SystemData.cpp index eb892dff03..3ac30b8dc0 100644 --- a/fdbclient/SystemData.cpp +++ b/fdbclient/SystemData.cpp @@ -1128,6 +1128,7 @@ const KeyRangeRef blobGranuleSplitKeys(LiteralStringRef("\xff\x02/bgs/"), Litera const KeyRangeRef blobGranuleSplitBoundaryKeys(LiteralStringRef("\xff\x02/bgsb/"), LiteralStringRef("\xff\x02/bgsb0")); const KeyRangeRef blobGranuleHistoryKeys(LiteralStringRef("\xff\x02/bgh/"), LiteralStringRef("\xff\x02/bgh0")); const KeyRangeRef blobGranulePruneKeys(LiteralStringRef("\xff\x02/bgp/"), LiteralStringRef("\xff\x02/bgp0")); +const KeyRangeRef blobGranuleVersionKeys(LiteralStringRef("\xff\x02/bgv/"), LiteralStringRef("\xff\x02/bgv0")); const KeyRef blobGranulePruneChangeKey = LiteralStringRef("\xff\x02/bgpChange"); const uint8_t BG_FILE_TYPE_DELTA = 'D'; @@ -1182,20 +1183,23 @@ std::tuple, int64_t, int64_t> decodeBlobGranuleFileValue(V return std::tuple(filename, offset, length); } -const Value blobGranulePruneValueFor(Version version, bool force) { +const Value blobGranulePruneValueFor(Version version, KeyRange range, bool force) { BinaryWriter wr(IncludeVersion(ProtocolVersion::withBlobGranule())); wr << version; + wr << range; wr << force; return wr.toValue(); } -std::pair decodeBlobGranulePruneValue(ValueRef const& value) { +std::tuple decodeBlobGranulePruneValue(ValueRef const& value) { Version version; + KeyRange range; bool force; BinaryReader reader(value, IncludeVersion()); reader >> version; + reader >> range; reader >> force; - return std::pair(version, force); + return std::tuple(version, range, force); } const Value blobGranuleMappingValueFor(UID const& workerID) { diff --git a/fdbclient/SystemData.h b/fdbclient/SystemData.h index c76c2225a6..1a1b024064 100644 --- a/fdbclient/SystemData.h +++ b/fdbclient/SystemData.h @@ -560,6 +560,7 @@ extern const KeyRangeRef blobGranuleHistoryKeys; // \xff\x02/bgp/(start,end) = (version, force) extern const KeyRangeRef blobGranulePruneKeys; +extern const KeyRangeRef blobGranuleVersionKeys; extern const KeyRef blobGranulePruneChangeKey; const Key blobGranuleFileKeyFor(UID granuleID, uint8_t fileType, Version fileVersion); @@ -569,8 +570,8 @@ const KeyRange blobGranuleFileKeyRangeFor(UID granuleID); const Value blobGranuleFileValueFor(StringRef const& filename, int64_t offset, int64_t length); std::tuple, int64_t, int64_t> decodeBlobGranuleFileValue(ValueRef const& value); -const Value blobGranulePruneValueFor(Version version, bool force); -std::pair decodeBlobGranulePruneValue(ValueRef const& value); +const Value blobGranulePruneValueFor(Version version, KeyRange range, bool force); +std::tuple decodeBlobGranulePruneValue(ValueRef const& value); const Value blobGranuleMappingValueFor(UID const& workerID); UID decodeBlobGranuleMappingValue(ValueRef const& value); diff --git a/fdbserver/BlobManager.actor.cpp b/fdbserver/BlobManager.actor.cpp index 687299d207..c0016b1f72 100644 --- a/fdbserver/BlobManager.actor.cpp +++ b/fdbserver/BlobManager.actor.cpp @@ -2290,15 +2290,11 @@ ACTOR Future partiallyDeleteGranule(Reference self, UID g * Once all this is done, we finally clear the pruneIntent key, if possible, to indicate we are done * processing this prune intent. */ -ACTOR Future pruneRange(Reference self, - KeyRef startKey, - KeyRef endKey, - Version pruneVersion, - bool force) { +ACTOR Future pruneRange(Reference self, KeyRangeRef range, Version pruneVersion, bool force) { if (BM_DEBUG) { fmt::print("pruneRange starting for range [{0} - {1}) @ pruneVersion={2}, force={3}\n", - startKey.printable(), - endKey.printable(), + range.begin.printable(), + range.end.printable(), pruneVersion, force); } @@ -2316,8 +2312,6 @@ ACTOR Future pruneRange(Reference self, state std::unordered_set, boost::hash>> visited; - state KeyRange range(KeyRangeRef(startKey, endKey)); // range for [startKey, endKey) - // find all active granules (that comprise the range) and add to the queue state KeyRangeMap::Ranges activeRanges = self->workerAssignments.intersectingRanges(range); @@ -2335,7 +2329,7 @@ ACTOR Future pruneRange(Reference self, } // assumption: prune boundaries must respect granule boundaries - if (activeRange.begin() < startKey || activeRange.end() > endKey) { + if (activeRange.begin() < range.begin || activeRange.end() > range.end) { continue; } @@ -2388,7 +2382,7 @@ ACTOR Future pruneRange(Reference self, // get the persisted history entry for this granule state Standalone currHistoryNode; - state KeyRef historyKey = blobGranuleHistoryKeyFor(currRange, startVersion); + state Key historyKey = blobGranuleHistoryKeyFor(currRange, startVersion); loop { try { Optional persistedHistory = wait(tr.get(historyKey)); @@ -2497,38 +2491,10 @@ ACTOR Future pruneRange(Reference self, // another pruneIntent that got written for this table while we were processing this one. // If that is the case, we should not clear the key. Otherwise, we can just clear the key. - tr.reset(); - if (BM_DEBUG) { - printf("About to clear prune intent\n"); - } - loop { - try { - tr.setOption(FDBTransactionOptions::ACCESS_SYSTEM_KEYS); - tr.setOption(FDBTransactionOptions::PRIORITY_SYSTEM_IMMEDIATE); - - state Key pruneIntentKey = blobGranulePruneKeys.begin.withSuffix(startKey); - state Optional pruneIntentValue = wait(tr.get(pruneIntentKey)); - ASSERT(pruneIntentValue.present()); - - Version currPruneVersion; - bool currForce; - std::tie(currPruneVersion, currForce) = decodeBlobGranulePruneValue(pruneIntentValue.get()); - - if (currPruneVersion == pruneVersion && currForce == force) { - tr.clear(pruneIntentKey.withPrefix(blobGranulePruneKeys.begin)); - wait(tr.commit()); - } - break; - } catch (Error& e) { - fmt::print("Attempt to clear prune intent got error {}\n", e.name()); - wait(tr.onError(e)); - } - } - if (BM_DEBUG) { fmt::print("Successfully pruned range [{0} - {1}) at pruneVersion={2}\n", - startKey.printable(), - endKey.printable(), + range.begin.printable(), + range.end.printable(), pruneVersion); } return Void(); @@ -2556,180 +2522,112 @@ ACTOR Future pruneRange(Reference self, */ ACTOR Future monitorPruneKeys(Reference self) { // setup bstore - try { - if (BM_DEBUG) { - fmt::print("BM constructing backup container from {}\n", SERVER_KNOBS->BG_URL.c_str()); - } - self->bstore = BackupContainerFileSystem::openContainerFS(SERVER_KNOBS->BG_URL); - if (BM_DEBUG) { - printf("BM constructed backup container\n"); - } - } catch (Error& e) { - if (BM_DEBUG) { - fmt::print("BM got backup container init error {0}\n", e.name()); - } - throw e; + if (BM_DEBUG) { + fmt::print("BM constructing backup container from {}\n", SERVER_KNOBS->BG_URL.c_str()); + } + self->bstore = BackupContainerFileSystem::openContainerFS(SERVER_KNOBS->BG_URL); + if (BM_DEBUG) { + printf("BM constructed backup container\n"); } - try { - state Value oldPruneWatchVal; + loop { + state Reference tr = makeReference(self->db); + tr->setOption(FDBTransactionOptions::ACCESS_SYSTEM_KEYS); + tr->setOption(FDBTransactionOptions::PRIORITY_SYSTEM_IMMEDIATE); + + // Wait for the watch to change, or some time to expire (whichever comes first) + // before checking through the prune intents. We write a UID into the change key value + // so that we can still recognize when the watch key has been changed while we weren't + // monitoring it + + state Key lastPruneKey = blobGranulePruneKeys.begin; + loop { - state Reference tr = makeReference(self->db); tr->setOption(FDBTransactionOptions::ACCESS_SYSTEM_KEYS); tr->setOption(FDBTransactionOptions::PRIORITY_SYSTEM_IMMEDIATE); - // Wait for the watch to change, or some time to expire (whichever comes first) - // before checking through the prune intents. We write a UID into the change key value - // so that we can still recognize when the watch key has been changed while we weren't - // monitoring it - loop { - try { - tr->setOption(FDBTransactionOptions::ACCESS_SYSTEM_KEYS); - tr->setOption(FDBTransactionOptions::PRIORITY_SYSTEM_IMMEDIATE); - - state Optional newPruneWatchVal = wait(tr->get(blobGranulePruneChangeKey)); - - // if the value at the change key has changed, that means there is new work to do - if (newPruneWatchVal.present() && oldPruneWatchVal != newPruneWatchVal.get()) { - oldPruneWatchVal = newPruneWatchVal.get(); - if (BM_DEBUG) { - printf("the blobGranulePruneChangeKey changed\n"); + state std::vector> prunes; + state CoalescedKeyRangeMap> pruneMap; + pruneMap.insert(allKeys, std::make_pair(0, false)); + try { + // TODO: replace 10000 with a knob + state RangeResult pruneIntents = wait(tr->getRange(blobGranulePruneKeys, 10000)); + if (pruneIntents.size()) { + int rangeIdx = 0; + for (; rangeIdx < pruneIntents.size(); ++rangeIdx) { + Version pruneVersion; + KeyRange range; + bool force; + std::tie(pruneVersion, range, force) = + decodeBlobGranulePruneValue(pruneIntents[rangeIdx].value); + auto ranges = pruneMap.intersectingRanges(range); + bool foundConflict = false; + for (auto it : ranges) { + if ((it.value().second && !force && it.value().first < pruneVersion) || + (!it.value().second && force && pruneVersion < it.value().first)) { + foundConflict = true; + break; + } } - - // TODO: debugging code, remove it - /* - if (newPruneWatchVal.get().toString().substr(0, 6) == "prune=") { - state Reference dummy = - makeReference(self->db); - loop { - try { - dummy->setOption(FDBTransactionOptions::ACCESS_SYSTEM_KEYS); - dummy->setOption(FDBTransactionOptions::PRIORITY_SYSTEM_IMMEDIATE); - std::istringstream iss(newPruneWatchVal.get().toString().substr(6)); - Version version; - iss >> version; - dummy->set(blobGranulePruneKeys.begin.withSuffix(normalKeys.begin), - blobGranulePruneValueFor(version, false)); - wait(dummy->commit()); - break; - - } catch (Error& e) { - wait(dummy->onError(e)); - } - } + if (foundConflict) { + break; + } + pruneMap.insert(range, std::make_pair(pruneVersion, force)); + + fmt::print("about to prune range [{0} - {1}) @ {2}, force={3}\n", + range.begin.printable(), + range.end.printable(), + pruneVersion, + force ? "T" : "F"); + } + lastPruneKey = pruneIntents[rangeIdx - 1].key; + + for (auto it : pruneMap.ranges()) { + if (it.value().first > 0) { + prunes.emplace_back(pruneRange(self, it.range(), it.value().first, it.value().second)); } - */ - break; } - // otherwise, there are no changes and we should wait until the next change (or timeout) + // wait for this set of prunes to complete before starting the next ones since if we + // prune a range R at version V and while we are doing that, the time expires, we will + // end up trying to prune the same range again since the work isn't finished and the + // prunes will race + // + // TODO: this isn't that efficient though. Instead we could keep metadata as part of the + // BM's memory that tracks which prunes are active. Once done, we can mark that work as + // done. If the BM fails then all prunes will fail and so the next BM will have a clear + // set of metadata (i.e. no work in progress) so we will end up doing the work in the + // new BM + + wait(waitForAll(prunes)); + break; + } else { state Future watchPruneIntentsChange = tr->watch(blobGranulePruneChangeKey); wait(tr->commit()); - - if (BM_DEBUG) { - printf("monitorPruneKeys waiting for change or timeout\n"); - } - - choose { - when(wait(watchPruneIntentsChange)) { - if (BM_DEBUG) { - printf("monitorPruneKeys saw a change\n"); - } - tr->reset(); - } - when(wait(delay(SERVER_KNOBS->BG_PRUNE_TIMEOUT))) { - if (BM_DEBUG) { - printf("monitorPruneKeys got a timeout\n"); - } - break; - } - } - } catch (Error& e) { - wait(tr->onError(e)); - } - } - - tr->reset(); - - if (BM_DEBUG) { - printf("Looping over prune intents\n"); - } - - // loop through all prune intentions and do prune work accordingly - try { - state KeyRef beginKey = normalKeys.begin; - loop { - tr->setOption(FDBTransactionOptions::ACCESS_SYSTEM_KEYS); - tr->setOption(FDBTransactionOptions::PRIORITY_SYSTEM_IMMEDIATE); - - state std::vector> prunes; - try { - // TODO: replace 10000 with a knob - KeyRange nextRange(KeyRangeRef(beginKey, normalKeys.end)); - state RangeResult pruneIntents = wait(krmGetRanges( - tr, blobGranulePruneKeys.begin, nextRange, 10000, GetRangeLimits::BYTE_LIMIT_UNLIMITED)); - state Key lastEndKey; - - for (int rangeIdx = 0; rangeIdx < pruneIntents.size() - 1; ++rangeIdx) { - KeyRef rangeStartKey = pruneIntents[rangeIdx].key; - KeyRef rangeEndKey = pruneIntents[rangeIdx + 1].key; - lastEndKey = rangeEndKey; - if (pruneIntents[rangeIdx].value.size() == 0) { - continue; - } - KeyRange range(KeyRangeRef(rangeStartKey, rangeEndKey)); - Version pruneVersion; - bool force; - std::tie(pruneVersion, force) = decodeBlobGranulePruneValue(pruneIntents[rangeIdx].value); - - fmt::print("about to prune range [{0} - {1}) @ {2}, force={3}\n", - rangeStartKey.printable(), - rangeEndKey.printable(), - pruneVersion, - force ? "T" : "F"); - prunes.emplace_back(pruneRange(self, rangeStartKey, rangeEndKey, pruneVersion, force)); - } - - // wait for this set of prunes to complete before starting the next ones since if we - // prune a range R at version V and while we are doing that, the time expires, we will - // end up trying to prune the same range again since the work isn't finished and the - // prunes will race - // - // TODO: this isn't that efficient though. Instead we could keep metadata as part of the - // BM's memory that tracks which prunes are active. Once done, we can mark that work as - // done. If the BM fails then all prunes will fail and so the next BM will have a clear - // set of metadata (i.e. no work in progress) so we will end up doing the work in the - // new BM - wait(waitForAll(prunes)); - - if (!pruneIntents.more) { - break; - } - - beginKey = lastEndKey; - } catch (Error& e) { - wait(tr->onError(e)); - } + wait(watchPruneIntentsChange); + tr->reset(); } } catch (Error& e) { - if (e.code() == error_code_actor_cancelled) { - throw e; - } - if (BM_DEBUG) { - fmt::print("monitorPruneKeys for BM {0} saw error {1}\n", self->id.toString(), e.name()); - } - // don't want to kill the blob manager for errors around pruning - TraceEvent("MonitorPruneKeysError", self->id).detail("Error", e.name()); - } - if (BM_DEBUG) { - printf("Done pruning current set of prune intents.\n"); + wait(tr->onError(e)); } } - } catch (Error& e) { + + tr->reset(); + loop { + try { + tr->setOption(FDBTransactionOptions::ACCESS_SYSTEM_KEYS); + tr->setOption(FDBTransactionOptions::PRIORITY_SYSTEM_IMMEDIATE); + tr->clear(KeyRangeRef(blobGranulePruneKeys.begin, keyAfter(lastPruneKey))); + wait(tr->commit()); + break; + } catch (Error& e) { + wait(tr->onError(e)); + } + } + if (BM_DEBUG) { - fmt::print("monitorPruneKeys got error {}\n", e.name()); + printf("Done pruning current set of prune intents.\n"); } - throw e; } } diff --git a/fdbserver/BlobWorker.actor.cpp b/fdbserver/BlobWorker.actor.cpp index 9f81cff53b..d5f788724f 100644 --- a/fdbserver/BlobWorker.actor.cpp +++ b/fdbserver/BlobWorker.actor.cpp @@ -2230,6 +2230,7 @@ ACTOR Future doBlobGranuleFileRequest(Reference bwData, Bl state KeyRange chunkRange; state GranuleFiles chunkFiles; + state Version startVer; if (metadata->initialSnapshotVersion > req.readVersion) { // this is a time travel query, find previous granule @@ -2290,6 +2291,7 @@ ACTOR Future doBlobGranuleFileRequest(Reference bwData, Bl ASSERT(cur->endVersion > req.readVersion); ASSERT(cur->startVersion <= req.readVersion); + startVer = cur->startVersion; // lazily load files for old granule if not present chunkRange = cur->range; @@ -2324,6 +2326,7 @@ ACTOR Future doBlobGranuleFileRequest(Reference bwData, Bl ASSERT(chunkFiles.snapshotFiles.front().version <= req.readVersion); } else { // this is an active granule query + startVer = metadata->initialSnapshotVersion; loop { if (!metadata->activeCFData.get().isValid() || !metadata->cancelled.canBeSet()) { throw wrong_shard_server(); @@ -2379,6 +2382,7 @@ ACTOR Future doBlobGranuleFileRequest(Reference bwData, Bl BlobGranuleChunkRef chunk; // TODO change in V2 chunk.includedVersion = req.readVersion; + chunk.startVersion = startVer; chunk.keyRange = KeyRangeRef(StringRef(rep.arena, chunkRange.begin), StringRef(rep.arena, chunkRange.end)); // handle snapshot files diff --git a/fdbserver/workloads/BlobGranuleVerifier.actor.cpp b/fdbserver/workloads/BlobGranuleVerifier.actor.cpp index 6767da00ef..7e7042c7cc 100644 --- a/fdbserver/workloads/BlobGranuleVerifier.actor.cpp +++ b/fdbserver/workloads/BlobGranuleVerifier.actor.cpp @@ -31,6 +31,7 @@ #include "fdbserver/Knobs.h" #include "fdbserver/TesterInterface.actor.h" #include "fdbserver/workloads/workloads.actor.h" +#include "flow/Error.h" #include "flow/IRandom.h" #include "flow/genericactors.actor.h" @@ -328,19 +329,15 @@ struct BlobGranuleVerifierWorkload : TestWorkload { try { tr->setOption(FDBTransactionOptions::ACCESS_SYSTEM_KEYS); tr->setOption(FDBTransactionOptions::PRIORITY_SYSTEM_IMMEDIATE); - Optional oldPruneIntent = wait(tr->get(blobGranulePruneKeys.begin.withSuffix(range.begin))); - if (oldPruneIntent.present()) { - Version oldPruneVersion; - bool oldForce; - std::tie(oldPruneVersion, oldForce) = decodeBlobGranulePruneValue(oldPruneIntent.get()); - if (oldPruneVersion >= version) { - return Void(); - } - } + Value pruneValue = blobGranulePruneValueFor(version, range, force); - Value pruneValue = blobGranulePruneValueFor(version, force); - wait(krmSetRange(tr, blobGranulePruneKeys.begin, range, pruneValue)); + Key pruneKey = KeyRef(blobGranulePruneKeys.begin.withSuffix(std::string(14, '\x00'))); + int32_t pos = pruneKey.size() - 14; + pos = littleEndian32(pos); + uint8_t* data = mutateString(pruneKey); + memcpy(data + pruneKey.size() - sizeof(int32_t), &pos, sizeof(int32_t)); + tr->atomicOp(pruneKey, pruneValue, MutationRef::SetVersionstampedKey); tr->set(blobGranulePruneChangeKey, deterministicRandom()->randomUniqueID().toString()); wait(tr->commit()); if (BGV_DEBUG) { @@ -363,11 +360,53 @@ struct BlobGranuleVerifierWorkload : TestWorkload { } } + ACTOR Future killBlobWorkers(Database cx, BlobGranuleVerifierWorkload* self) { + state Transaction tr(cx); + state std::set knownWorkers; + state bool first = true; + loop { + try { + RangeResult r = wait(tr.getRange(blobWorkerListKeys, CLIENT_KNOBS->TOO_MANY)); + + state std::vector haltIds; + state std::vector>> haltRequests; + for (auto& it : r) { + BlobWorkerInterface interf = decodeBlobWorkerListValue(it.value); + if (first) { + knownWorkers.insert(interf.id()); + } + if (knownWorkers.count(interf.id())) { + haltIds.push_back(interf.id()); + haltRequests.push_back(interf.haltBlobWorker.tryGetReply(HaltBlobWorkerRequest(1e6, UID()))); + } + } + first = false; + wait(waitForAll(haltRequests)); + bool allPresent = true; + for (int i = 0; i < haltRequests.size(); i++) { + if (haltRequests[i].get().present()) { + knownWorkers.erase(haltIds[i]); + } else { + allPresent = false; + } + } + if (allPresent) { + return Void(); + } else { + wait(delay(1.0)); + } + } catch (Error& e) { + wait(tr.onError(e)); + } + } + } + ACTOR Future verifyGranules(Database cx, BlobGranuleVerifierWorkload* self) { state double last = now(); state double endTime = last + self->testDuration; state std::map timeTravelChecks; state int64_t timeTravelChecksMemory = 0; + state Version pruneVersion = 1; TraceEvent("BlobGranuleVerifierStart"); if (BGV_DEBUG) { @@ -392,12 +431,32 @@ struct BlobGranuleVerifierWorkload : TestWorkload { // advance iterator before doing read, so if it gets error we don't retry it try { - // TODO: before reading, prune at some version [0, readVersion) + state Version newPruneVersion = deterministicRandom()->randomInt64(1, oldRead.v); + pruneVersion = std::max(pruneVersion, newPruneVersion); + wait(self->pruneAtVersion(cx, oldRead.range, newPruneVersion, false)); std::pair>> reReadResult = wait(self->readFromBlob(cx, self, oldRead.range, oldRead.v)); self->compareResult(oldRead.oldResult, reReadResult, oldRead.range, oldRead.v, false); self->timeTravelReads++; + wait(self->killBlobWorkers(cx, self)); + try { + std::pair>> versionRead = + wait(self->readFromBlob(cx, self, oldRead.range, pruneVersion)); + Version minStartVer = newPruneVersion; + for (auto& it : versionRead.second) { + minStartVer = std::min(minStartVer, it.startVersion); + } + std::pair>> versionRead = + wait(self->readFromBlob(cx, self, oldRead.range, minStartVer - 1)); + ASSERT(false); + } catch (Error& e) { + if (e.code() == error_code_actor_cancelled) { + throw; + } + ASSERT(e.code() == error_code_blob_granule_transaction_too_old); + } + // TODO: read at some version older than pruneVersion and make sure you get txn_too_old // To achieve this, the BWs are going to have to recognize latest prune versions per granules } catch (Error& e) { From 1990216e896f59f965eaa8b4c11192aafd917bb8 Mon Sep 17 00:00:00 2001 From: Evan Tschannen Date: Tue, 22 Feb 2022 13:03:57 -0800 Subject: [PATCH 269/413] use global knowledge inside simulation to determine when it should be possible to get a too_old error --- fdbserver/workloads/BlobGranuleVerifier.actor.cpp | 9 +++++---- 1 file changed, 5 insertions(+), 4 deletions(-) diff --git a/fdbserver/workloads/BlobGranuleVerifier.actor.cpp b/fdbserver/workloads/BlobGranuleVerifier.actor.cpp index 7e7042c7cc..aaef5a468f 100644 --- a/fdbserver/workloads/BlobGranuleVerifier.actor.cpp +++ b/fdbserver/workloads/BlobGranuleVerifier.actor.cpp @@ -39,6 +39,8 @@ #define BGV_DEBUG true +Version dbgPruneVersion = 0; + /* * This workload is designed to verify the correctness of the blob data produced by the blob workers. * As a read-only validation workload, it can piggyback off of other write or read/write workloads. @@ -406,7 +408,6 @@ struct BlobGranuleVerifierWorkload : TestWorkload { state double endTime = last + self->testDuration; state std::map timeTravelChecks; state int64_t timeTravelChecksMemory = 0; - state Version pruneVersion = 1; TraceEvent("BlobGranuleVerifierStart"); if (BGV_DEBUG) { @@ -432,7 +433,7 @@ struct BlobGranuleVerifierWorkload : TestWorkload { try { state Version newPruneVersion = deterministicRandom()->randomInt64(1, oldRead.v); - pruneVersion = std::max(pruneVersion, newPruneVersion); + dbgPruneVersion = std::max(dbgPruneVersion, newPruneVersion); wait(self->pruneAtVersion(cx, oldRead.range, newPruneVersion, false)); std::pair>> reReadResult = wait(self->readFromBlob(cx, self, oldRead.range, oldRead.v)); @@ -442,7 +443,7 @@ struct BlobGranuleVerifierWorkload : TestWorkload { wait(self->killBlobWorkers(cx, self)); try { std::pair>> versionRead = - wait(self->readFromBlob(cx, self, oldRead.range, pruneVersion)); + wait(self->readFromBlob(cx, self, oldRead.range, dbgPruneVersion)); Version minStartVer = newPruneVersion; for (auto& it : versionRead.second) { minStartVer = std::min(minStartVer, it.startVersion); @@ -460,7 +461,7 @@ struct BlobGranuleVerifierWorkload : TestWorkload { // TODO: read at some version older than pruneVersion and make sure you get txn_too_old // To achieve this, the BWs are going to have to recognize latest prune versions per granules } catch (Error& e) { - if (e.code() == error_code_blob_granule_transaction_too_old) { + if (e.code() == error_code_blob_granule_transaction_too_old && oldRead.v >= dbgPruneVersion) { self->timeTravelTooOld++; // TODO: add debugging info for when this is a failure } From 6739d506c44e0d53283519952a0e2c79817b8477 Mon Sep 17 00:00:00 2001 From: Josh Slocum Date: Tue, 22 Feb 2022 16:23:32 -0600 Subject: [PATCH 270/413] More fixes for handling change feed empty version >= ss version --- fdbserver/storageserver.actor.cpp | 61 +++++++++++++++++++------------ 1 file changed, 37 insertions(+), 24 deletions(-) diff --git a/fdbserver/storageserver.actor.cpp b/fdbserver/storageserver.actor.cpp index b250b0439c..7de9218f18 100644 --- a/fdbserver/storageserver.actor.cpp +++ b/fdbserver/storageserver.actor.cpp @@ -4653,39 +4653,52 @@ ACTOR Future> fetchChangeFeedMetadata(StorageServer* data, KeyR if (!existing) { TEST(cleanupPending); // Fetch change feed which is cleanup pending. This means there was a move away and a - // move back, this will remake the metadata + // move back, this will remake the metadata - Reference changeFeedInfo = Reference(new ChangeFeedInfo()); - changeFeedInfo->range = cfEntry.range; - changeFeedInfo->id = cfEntry.rangeId; + if (cfEntry.emptyVersion < data->version.get()) { + Reference changeFeedInfo = Reference(new ChangeFeedInfo()); + changeFeedInfo->range = cfEntry.range; + changeFeedInfo->id = cfEntry.rangeId; - changeFeedInfo->stopped = cfEntry.stopped; - changeFeedInfo->emptyVersion = cfEntry.emptyVersion; - data->uidChangeFeed[cfEntry.rangeId] = changeFeedInfo; - auto rs = data->keyChangeFeed.modify(cfEntry.range); - for (auto r = rs.begin(); r != rs.end(); ++r) { - r->value().push_back(changeFeedInfo); + changeFeedInfo->stopped = cfEntry.stopped; + changeFeedInfo->emptyVersion = cfEntry.emptyVersion; + data->uidChangeFeed[cfEntry.rangeId] = changeFeedInfo; + auto rs = data->keyChangeFeed.modify(cfEntry.range); + for (auto r = rs.begin(); r != rs.end(); ++r) { + r->value().push_back(changeFeedInfo); + } + data->keyChangeFeed.coalesce(cfEntry.range.contents()); + + auto& mLV = data->addVersionToMutationLog(data->data().getLatestVersion()); + data->addMutationToMutationLog( + mLV, + MutationRef(MutationRef::SetValue, + persistChangeFeedKeys.begin.toString() + cfEntry.rangeId.toString(), + changeFeedValue(cfEntry.range, + changeFeedInfo->emptyVersion + 1, + cfEntry.stopped ? ChangeFeedStatus::CHANGE_FEED_STOP + : ChangeFeedStatus::CHANGE_FEED_CREATE))); + } else { + // don't include in list of change feeds to fetch + feedIds.pop_back(); + TraceEvent(SevDebug, "FetchedChangeFeedInfoIgnored", data->thisServerID) + .detail("RangeID", cfEntry.rangeId.printable()) + .detail("Range", cfEntry.range.toString()) + .detail("FetchVersion", fetchVersion) + .detail("SSVersion", data->version.get()) + .detail("EmptyVersion", cfEntry.emptyVersion); } - data->keyChangeFeed.coalesce(cfEntry.range.contents()); - - auto& mLV = data->addVersionToMutationLog(data->data().getLatestVersion()); - data->addMutationToMutationLog( - mLV, - MutationRef(MutationRef::SetValue, - persistChangeFeedKeys.begin.toString() + cfEntry.rangeId.toString(), - changeFeedValue(cfEntry.range, - changeFeedInfo->emptyVersion + 1, - cfEntry.stopped ? ChangeFeedStatus::CHANGE_FEED_STOP - : ChangeFeedStatus::CHANGE_FEED_CREATE))); } else { auto changeFeedInfo = existingEntry->second; auto feedCleanup = data->changeFeedCleanupDurable.find(cfEntry.rangeId); - bool writeToMutationLog = false; if (feedCleanup != data->changeFeedCleanupDurable.end() && changeFeedInfo->removing) { TEST(true); // re-fetching feed scheduled for deletion! Un-mark it as removing - changeFeedInfo->emptyVersion = cfEntry.emptyVersion; - changeFeedInfo->stopped = cfEntry.stopped; + if (cfEntry.emptyVersion < data->version.get()) { + changeFeedInfo->emptyVersion = cfEntry.emptyVersion; + changeFeedInfo->stopped = cfEntry.stopped; + } + changeFeedInfo->removing = false; // reset fetch versions because everything previously fetched was cleaned up changeFeedInfo->fetchVersion = invalidVersion; From 0714b7a250b10444dd27f0c996b7df883bfc4ea6 Mon Sep 17 00:00:00 2001 From: Josh Slocum Date: Wed, 23 Feb 2022 08:48:45 -0600 Subject: [PATCH 271/413] Fixed blob manager split and reassign race --- fdbserver/BlobManager.actor.cpp | 29 ++++++++++------------------- 1 file changed, 10 insertions(+), 19 deletions(-) diff --git a/fdbserver/BlobManager.actor.cpp b/fdbserver/BlobManager.actor.cpp index 687299d207..143705f288 100644 --- a/fdbserver/BlobManager.actor.cpp +++ b/fdbserver/BlobManager.actor.cpp @@ -919,11 +919,14 @@ ACTOR Future maybeSplitRange(Reference bmData, Optional lockValue = wait(tr->get(lockKey)); ASSERT(lockValue.present()); std::tuple prevGranuleLock = decodeBlobGranuleLockValue(lockValue.get()); - if (std::get<0>(prevGranuleLock) > bmData->epoch) { + int64_t ownerEpoch = std::get<0>(prevGranuleLock); + int64_t ownerSeqno = std::get<1>(prevGranuleLock); + + if (ownerEpoch > bmData->epoch) { if (BM_DEBUG) { fmt::print("BM {0} found a higher epoch {1} than {2} for granule lock of [{3} - {4})\n", bmData->id.toString(), - std::get<0>(prevGranuleLock), + ownerEpoch, bmData->epoch, granuleRange.begin.printable(), granuleRange.end.printable()); @@ -934,8 +937,7 @@ ACTOR Future maybeSplitRange(Reference bmData, } return Void(); } - int64_t ownerEpoch = std::get<0>(prevGranuleLock); - int64_t ownerSeqno = std::get<1>(prevGranuleLock); + if (newLockSeqno == -1) { newLockSeqno = bmData->seqNo; bmData->seqNo++; @@ -950,21 +952,10 @@ ACTOR Future maybeSplitRange(Reference bmData, ownerSeqno); } ASSERT(bmData->epoch > ownerEpoch || (bmData->epoch == ownerEpoch && newLockSeqno > ownerSeqno)); - } else { - if (!(bmData->epoch > ownerEpoch || (bmData->epoch == ownerEpoch && newLockSeqno >= ownerSeqno))) { - printf("BM seqno for granule [%s - %s) out of order for lock on retry! manager: (%lld, %lld), " - "owner: %lld, " - "%lld)\n", - granuleRange.begin.printable().c_str(), - granuleRange.end.printable().c_str(), - bmData->epoch, - newLockSeqno, - ownerEpoch, - ownerSeqno); - } - // previous transaction could have succeeded but got commit_unknown_result, so use >= instead of > for - // seqno if epochs are equal - ASSERT(bmData->epoch > ownerEpoch || (bmData->epoch == ownerEpoch && newLockSeqno >= ownerSeqno)); + } else if (bmData->epoch == ownerEpoch && newLockSeqno < ownerSeqno) { + // we retried, and between retries we reassigned this range elsewhere. Cancel this split + TEST(true); // BM maybe split cancelled by subsequent move + return Void(); } // acquire granule lock so nobody else can make changes to this granule. From 2701e44564dcfe6a4f9cb13850541ba14696aa31 Mon Sep 17 00:00:00 2001 From: Josh Slocum Date: Wed, 23 Feb 2022 15:55:16 -0600 Subject: [PATCH 272/413] Fixing error handling and race for blob worker status stream to manager --- fdbserver/BlobWorker.actor.cpp | 18 ++++++++++++++++-- 1 file changed, 16 insertions(+), 2 deletions(-) diff --git a/fdbserver/BlobWorker.actor.cpp b/fdbserver/BlobWorker.actor.cpp index 9f81cff53b..9521b968b0 100644 --- a/fdbserver/BlobWorker.actor.cpp +++ b/fdbserver/BlobWorker.actor.cpp @@ -866,7 +866,12 @@ ACTOR Future checkSplitAndReSnapshot(Reference bw loop { loop { try { - wait(bwData->currentManagerStatusStream.get().onReady()); + loop { + choose { + when(wait(bwData->currentManagerStatusStream.get().onReady())) { break; } + when(wait(bwData->currentManagerStatusStream.onChange())) {} + } + } bwData->currentManagerStatusStream.get().send(GranuleStatusReply(metadata->keyRange, true, writeHot, @@ -877,7 +882,16 @@ ACTOR Future checkSplitAndReSnapshot(Reference bw reSnapshotVersion)); break; } catch (Error& e) { - wait(bwData->currentManagerStatusStream.onChange()); + if (e.code() == error_code_operation_cancelled) { + throw e; + } + // if we got broken promise while waiting, the old stream was killed, so we don't need to wait on + // change, just retry + if (e.code() == error_code_broken_promise) { + wait(delay(FLOW_KNOBS->PREVENT_FAST_SPIN_DELAY)); + } else { + wait(bwData->currentManagerStatusStream.onChange()); + } } } From 1de4fb1b9e7fb0143040aa690489067b0a51c7b7 Mon Sep 17 00:00:00 2001 From: Josh Slocum Date: Thu, 24 Feb 2022 10:49:10 -0600 Subject: [PATCH 273/413] Adding explicit stopVersion instead of stopped status to change feed durable data, several other minor fixes --- fdbclient/StorageServerInterface.h | 11 +- fdbserver/storageserver.actor.cpp | 256 +++++++++++++++++------------ 2 files changed, 161 insertions(+), 106 deletions(-) diff --git a/fdbclient/StorageServerInterface.h b/fdbclient/StorageServerInterface.h index db6fb9f9e4..e314fbc05d 100644 --- a/fdbclient/StorageServerInterface.h +++ b/fdbclient/StorageServerInterface.h @@ -750,19 +750,20 @@ struct OverlappingChangeFeedEntry { Key rangeId; KeyRange range; Version emptyVersion; - bool stopped = false; + Version stopVersion; bool operator==(const OverlappingChangeFeedEntry& r) const { - return rangeId == r.rangeId && range == r.range && stopped == r.stopped && emptyVersion == r.emptyVersion; + return rangeId == r.rangeId && range == r.range && emptyVersion == r.emptyVersion && + stopVersion == r.stopVersion; } OverlappingChangeFeedEntry() {} - OverlappingChangeFeedEntry(Key const& rangeId, KeyRange const& range, Version emptyVersion, bool stopped) - : rangeId(rangeId), range(range), emptyVersion(emptyVersion), stopped(stopped) {} + OverlappingChangeFeedEntry(Key const& rangeId, KeyRange const& range, Version emptyVersion, Version stopVersion) + : rangeId(rangeId), range(range), emptyVersion(emptyVersion), stopVersion(stopVersion) {} template void serialize(Ar& ar) { - serializer(ar, rangeId, range, emptyVersion, stopped); + serializer(ar, rangeId, range, emptyVersion, stopVersion); } }; diff --git a/fdbserver/storageserver.actor.cpp b/fdbserver/storageserver.actor.cpp index 7de9218f18..6b3d3f1f45 100644 --- a/fdbserver/storageserver.actor.cpp +++ b/fdbserver/storageserver.actor.cpp @@ -371,7 +371,10 @@ struct ChangeFeedInfo : ReferenceCounted { Key id; AsyncTrigger newMutations; NotifiedVersion durableFetchVersion; - bool stopped = false; // A stopped change feed no longer adds new mutations, but is still queriable + // A stopped change feed no longer adds new mutations, but is still queriable. + // stopVersion = MAX_VERSION means the feed has not been stopped + Version stopVersion = MAX_VERSION; + bool removing = false; KeyRangeMap>> moveTriggers; @@ -703,7 +706,7 @@ public: NotifiedVersion durableVersion; // At least this version will be readable from storage after a power failure Version rebootAfterDurableVersion; int8_t primaryLocality; - Version knownCommittedVersion; + NotifiedVersion knownCommittedVersion; Deque> recoveryVersionSkips; int64_t versionLag; // An estimate for how many versions it takes for the data to move from the logs to this storage @@ -1718,18 +1721,35 @@ ACTOR Future overlappingChangeFeedsQ(StorageServer* data, OverlappingChang return Void(); } + Version knownCommittedRequired = invalidVersion; + auto ranges = data->keyChangeFeed.intersectingRanges(req.range); - std::map> rangeIds; + std::map> rangeIds; for (auto r : ranges) { for (auto& it : r.value()) { - rangeIds[it->id] = std::tuple(it->range, it->emptyVersion, it->stopped); + // Can't tell other SS about a stopVersion that may get rolled back, and we only need to tell it about the + // stopVersion if req.minVersion > stopVersion, since it will get the information from its own private + // mutations if it hasn't processed up to stopVersion yet + Version stopVersion; + if (it->stopVersion != MAX_VERSION && req.minVersion > it->stopVersion) { + stopVersion = it->stopVersion; + knownCommittedRequired = std::max(knownCommittedRequired, stopVersion); + } else { + stopVersion = MAX_VERSION; + } + rangeIds[it->id] = std::tuple(it->range, it->emptyVersion, stopVersion); } } - OverlappingChangeFeedsReply reply; + state OverlappingChangeFeedsReply reply; for (auto& it : rangeIds) { reply.rangeIds.push_back(OverlappingChangeFeedEntry( it.first, std::get<0>(it.second), std::get<1>(it.second), std::get<2>(it.second))); } + + // Make sure all of the stop versions we are sending aren't going to get rolled back + if (knownCommittedRequired != invalidVersion && knownCommittedRequired > data->knownCommittedVersion.get()) { + wait(data->knownCommittedVersion.whenAtLeast(knownCommittedRequired)); + } req.reply.send(reply); return Void(); } @@ -1896,7 +1916,7 @@ ACTOR Future> getChangeFeedMutations(Stor // We must copy the mutationDeque when fetching the durable bytes in case mutations are popped from memory while // waiting for the results state Version dequeVersion = data->version.get(); - state Version dequeKnownCommit = data->knownCommittedVersion; + state Version dequeKnownCommit = data->knownCommittedVersion.get(); state Version emptyVersion = feedInfo->emptyVersion; Version fetchStorageVersion = std::max(feedInfo->fetchVersion, feedInfo->durableFetchVersion.get()); @@ -1996,6 +2016,9 @@ ACTOR Future> getChangeFeedMutations(Stor data->checkChangeCounter(changeCounter, req.range); } + // TODO eventually: only do verify in simulation? + int memoryVerifyIdx = 0; + Version lastVersion = req.begin - 1; Version lastKnownCommitted = invalidVersion; for (auto& kv : res) { @@ -2009,6 +2032,25 @@ ACTOR Future> getChangeFeedMutations(Stor if (m.mutations.size()) { reply.arena.dependsOn(mutations.arena()); reply.mutations.push_back(reply.arena, m); + + // if there is overlap between the memory and disk mutations, we can do relatively cheap validation that + // they are the same. In particular this validates the consistency of change feed data recieved from the + // tlog mutations vs change feed data fetched from another storage server + if (memoryVerifyIdx < memoryReply.mutations.size()) { + ASSERT(version <= memoryReply.mutations[memoryVerifyIdx].version); + if (version == memoryReply.mutations[memoryVerifyIdx].version) { + ASSERT(m.mutations.size() == memoryReply.mutations[memoryVerifyIdx].mutations.size()); + for (int i = 0; i < m.mutations.size(); i++) { + ASSERT(m.mutations[i].type == memoryReply.mutations[memoryVerifyIdx].mutations[i].type); + ASSERT(m.mutations[i].param1 == memoryReply.mutations[memoryVerifyIdx].mutations[i].param1); + ASSERT(m.mutations[i].param2 == memoryReply.mutations[memoryVerifyIdx].mutations[i].param2); + } + memoryVerifyIdx++; + } + } + } else if (memoryVerifyIdx < memoryReply.mutations.size() && + version == memoryReply.mutations[memoryVerifyIdx].version) { + ASSERT(false); } remainingDurableBytes -= sizeof(KeyValueRef) + @@ -2257,19 +2299,18 @@ ACTOR Future changeFeedStreamQ(StorageServer* data, ChangeFeedStreamReques wait(delay(0, TaskPriority::DefaultEndpoint)); - if (DEBUG_SS_CFM(data->thisServerID, req.rangeID, req.begin)) { - printf("CFM: SS %s CF %s: got CFSQ %s [%s - %s) %lld - %lld, crp=%s\n", - data->thisServerID.toString().substr(0, 4).c_str(), - req.rangeID.printable().substr(0, 6).c_str(), - streamUID.toString().substr(0, 8).c_str(), - req.range.begin.printable().c_str(), - req.range.end.printable().c_str(), - req.begin, - req.end, - req.canReadPopped ? "T" : "F"); - } - try { + if (DEBUG_SS_CFM(data->thisServerID, req.rangeID, req.begin)) { + printf("CFM: SS %s CF %s: got CFSQ %s [%s - %s) %lld - %lld, crp=%s\n", + data->thisServerID.toString().substr(0, 4).c_str(), + req.rangeID.printable().substr(0, 6).c_str(), + streamUID.toString().substr(0, 8).c_str(), + req.range.begin.printable().c_str(), + req.range.end.printable().c_str(), + req.begin, + req.end, + req.canReadPopped ? "T" : "F"); + } data->activeFeedQueries++; // send an empty version at begin - 1 to establish the stream quickly @@ -4045,10 +4086,9 @@ void applyMutation(StorageServer* self, void applyChangeFeedMutation(StorageServer* self, MutationRef const& m, Version version) { if (m.type == MutationRef::SetValue) { for (auto& it : self->keyChangeFeed[m.param1]) { - if (!it->stopped) { - ASSERT(version > it->emptyVersion); + if (version < it->stopVersion && !it->removing && version > it->emptyVersion) { if (it->mutations.empty() || it->mutations.back().version != version) { - it->mutations.push_back(MutationsAndVersionRef(version, self->knownCommittedVersion)); + it->mutations.push_back(MutationsAndVersionRef(version, self->knownCommittedVersion.get())); } it->mutations.back().mutations.push_back_deep(it->mutations.back().arena(), m); self->currentChangeFeeds.insert(it->id); @@ -4057,23 +4097,24 @@ void applyChangeFeedMutation(StorageServer* self, MutationRef const& m, Version .detail("Range", it->range) .detail("ChangeFeedID", it->id); } else { - TEST(it->stopped); // Skip CF write because stopped - TEST(version <= it->emptyVersion); // Skip CF write because popped and SS behind + TEST(version <= it->emptyVersion); // Skip CF write because version <= emptyVersion + TEST(it->removing); // Skip CF write because removing + TEST(version >= it->stopVersion); // Skip CF write because stopped DEBUG_MUTATION("ChangeFeedWriteSetIgnore", version, m, self->thisServerID) .detail("Range", it->range) .detail("ChangeFeedID", it->id) - .detail("Stopped", it->stopped) - .detail("EmptyVersion", it->emptyVersion); + .detail("StopVersion", it->stopVersion) + .detail("EmptyVersion", it->emptyVersion) + .detail("Removing", it->removing); } } } else if (m.type == MutationRef::ClearRange) { auto ranges = self->keyChangeFeed.intersectingRanges(KeyRangeRef(m.param1, m.param2)); for (auto& r : ranges) { for (auto& it : r.value()) { - if (!it->stopped) { - ASSERT(version > it->emptyVersion); + if (version < it->stopVersion && !it->removing && version > it->emptyVersion) { if (it->mutations.empty() || it->mutations.back().version != version) { - it->mutations.push_back(MutationsAndVersionRef(version, self->knownCommittedVersion)); + it->mutations.push_back(MutationsAndVersionRef(version, self->knownCommittedVersion.get())); } it->mutations.back().mutations.push_back_deep(it->mutations.back().arena(), m); self->currentChangeFeeds.insert(it->id); @@ -4081,13 +4122,15 @@ void applyChangeFeedMutation(StorageServer* self, MutationRef const& m, Version .detail("Range", it->range) .detail("ChangeFeedID", it->id); } else { - TEST(it->stopped); // Skip CF clear because stopped - TEST(version <= it->emptyVersion); // Skip CF clear because popped and SS behind + TEST(version <= it->emptyVersion); // Skip CF clear because version <= emptyVersion + TEST(it->removing); // Skip CF clear because removing + TEST(version >= it->stopVersion); // Skip CF clear because stopped DEBUG_MUTATION("ChangeFeedWriteClearIgnore", version, m, self->thisServerID) .detail("Range", it->range) .detail("ChangeFeedID", it->id) - .detail("Stopped", it->stopped) - .detail("EmptyVersion", it->emptyVersion); + .detail("StopVersion", it->stopVersion) + .detail("EmptyVersion", it->emptyVersion) + .detail("Removing", it->removing); } } } @@ -4328,9 +4371,30 @@ static const KeyRangeRef persistByteSampleSampleKeys = static const KeyRef persistLogProtocol = LiteralStringRef(PERSIST_PREFIX "LogProtocol"); static const KeyRef persistPrimaryLocality = LiteralStringRef(PERSIST_PREFIX "PrimaryLocality"); static const KeyRangeRef persistChangeFeedKeys = - KeyRangeRef(LiteralStringRef(PERSIST_PREFIX "RF/"), LiteralStringRef(PERSIST_PREFIX "RF0")); + KeyRangeRef(LiteralStringRef(PERSIST_PREFIX "CF/"), LiteralStringRef(PERSIST_PREFIX "CF0")); // data keys are unmangled (but never start with PERSIST_PREFIX because they are always in allKeys) +// We have to store the version the change feed was stopped at in the SS instead of just the stopped status +// In addition to simplifying stopping logic, it enables communicating stopped status when fetching change feeds +// from other SS correctly +const Value changeFeedSSValue(KeyRangeRef const& range, Version popVersion, Version stopVersion) { + BinaryWriter wr(IncludeVersion(ProtocolVersion::withChangeFeed())); + wr << range; + wr << popVersion; + wr << stopVersion; + return wr.toValue(); +} + +std::tuple decodeChangeFeedSSValue(ValueRef const& value) { + KeyRange range; + Version popVersion, stopVersion; + BinaryReader reader(value, IncludeVersion()); + reader >> range; + reader >> popVersion; + reader >> stopVersion; + return std::make_tuple(range, popVersion, stopVersion); +} + ACTOR Future changeFeedPopQ(StorageServer* self, ChangeFeedPopRequest req) { // if a SS restarted and is way behind, wait for it to at least have caught up through the pop version wait(self->version.whenAtLeast(req.version)); @@ -4361,12 +4425,10 @@ ACTOR Future changeFeedPopQ(StorageServer* self, ChangeFeedPopRequest req) auto& mLV = self->addVersionToMutationLog(durableVersion); self->addMutationToMutationLog( mLV, - MutationRef(MutationRef::SetValue, - persistChangeFeedKeys.begin.toString() + feed->second->id.toString(), - changeFeedValue(feed->second->range, - feed->second->emptyVersion + 1, - feed->second->stopped ? ChangeFeedStatus::CHANGE_FEED_STOP - : ChangeFeedStatus::CHANGE_FEED_CREATE))); + MutationRef( + MutationRef::SetValue, + persistChangeFeedKeys.begin.toString() + feed->second->id.toString(), + changeFeedSSValue(feed->second->range, feed->second->emptyVersion + 1, feed->second->stopVersion))); if (feed->second->storageVersion != invalidVersion) { ++self->counters.kvSystemClearRanges; self->addMutationToMutationLog(mLV, @@ -4647,56 +4709,44 @@ ACTOR Future> fetchChangeFeedMetadata(StorageServer* data, KeyR .detail("Range", cfEntry.range.toString()) .detail("FetchVersion", fetchVersion) .detail("EmptyVersion", cfEntry.emptyVersion) - .detail("Stopped", cfEntry.stopped) + .detail("StopVersion", cfEntry.stopVersion) .detail("Existing", existing) .detail("CleanupPendingVersion", cleanupPending ? cleanupEntry->second : invalidVersion); + bool addMutationToLog = false; + Reference changeFeedInfo; + if (!existing) { TEST(cleanupPending); // Fetch change feed which is cleanup pending. This means there was a move away and a // move back, this will remake the metadata - if (cfEntry.emptyVersion < data->version.get()) { - Reference changeFeedInfo = Reference(new ChangeFeedInfo()); - changeFeedInfo->range = cfEntry.range; - changeFeedInfo->id = cfEntry.rangeId; + changeFeedInfo = Reference(new ChangeFeedInfo()); + changeFeedInfo->range = cfEntry.range; + changeFeedInfo->id = cfEntry.rangeId; - changeFeedInfo->stopped = cfEntry.stopped; - changeFeedInfo->emptyVersion = cfEntry.emptyVersion; - data->uidChangeFeed[cfEntry.rangeId] = changeFeedInfo; - auto rs = data->keyChangeFeed.modify(cfEntry.range); - for (auto r = rs.begin(); r != rs.end(); ++r) { - r->value().push_back(changeFeedInfo); - } - data->keyChangeFeed.coalesce(cfEntry.range.contents()); - - auto& mLV = data->addVersionToMutationLog(data->data().getLatestVersion()); - data->addMutationToMutationLog( - mLV, - MutationRef(MutationRef::SetValue, - persistChangeFeedKeys.begin.toString() + cfEntry.rangeId.toString(), - changeFeedValue(cfEntry.range, - changeFeedInfo->emptyVersion + 1, - cfEntry.stopped ? ChangeFeedStatus::CHANGE_FEED_STOP - : ChangeFeedStatus::CHANGE_FEED_CREATE))); - } else { - // don't include in list of change feeds to fetch - feedIds.pop_back(); - TraceEvent(SevDebug, "FetchedChangeFeedInfoIgnored", data->thisServerID) - .detail("RangeID", cfEntry.rangeId.printable()) - .detail("Range", cfEntry.range.toString()) - .detail("FetchVersion", fetchVersion) - .detail("SSVersion", data->version.get()) - .detail("EmptyVersion", cfEntry.emptyVersion); + changeFeedInfo->emptyVersion = cfEntry.emptyVersion; + changeFeedInfo->stopVersion = cfEntry.stopVersion; + data->uidChangeFeed[cfEntry.rangeId] = changeFeedInfo; + auto rs = data->keyChangeFeed.modify(cfEntry.range); + for (auto r = rs.begin(); r != rs.end(); ++r) { + r->value().push_back(changeFeedInfo); } + data->keyChangeFeed.coalesce(cfEntry.range.contents()); + + addMutationToLog = true; } else { auto changeFeedInfo = existingEntry->second; auto feedCleanup = data->changeFeedCleanupDurable.find(cfEntry.rangeId); + if (cfEntry.stopVersion < changeFeedInfo->stopVersion) { + changeFeedInfo->stopVersion = cfEntry.stopVersion; + addMutationToLog = true; + } + if (feedCleanup != data->changeFeedCleanupDurable.end() && changeFeedInfo->removing) { TEST(true); // re-fetching feed scheduled for deletion! Un-mark it as removing if (cfEntry.emptyVersion < data->version.get()) { changeFeedInfo->emptyVersion = cfEntry.emptyVersion; - changeFeedInfo->stopped = cfEntry.stopped; } changeFeedInfo->removing = false; @@ -4710,24 +4760,25 @@ ACTOR Future> fetchChangeFeedMetadata(StorageServer* data, KeyR .detail("FetchVersion", fetchVersion) .detail("EmptyVersion", cfEntry.emptyVersion) .detail("CleanupVersion", feedCleanup->second) - .detail("Stopped", cfEntry.stopped); + .detail("StopVersion", cfEntry.stopVersion); // Since cleanup put a mutation in the log to delete the change feed data, put one in the log to restore // it // We may just want to refactor this so updateStorage does explicit deletes based on // changeFeedCleanupDurable and not use the mutation log at all for the change feed metadata cleanup. // Then we wouldn't have to reset anything here - auto& mLV = data->addVersionToMutationLog(data->data().getLatestVersion()); - data->addMutationToMutationLog( - mLV, - MutationRef(MutationRef::SetValue, - persistChangeFeedKeys.begin.toString() + cfEntry.rangeId.toString(), - changeFeedValue(cfEntry.range, - changeFeedInfo->emptyVersion + 1, - cfEntry.stopped ? ChangeFeedStatus::CHANGE_FEED_STOP - : ChangeFeedStatus::CHANGE_FEED_CREATE))); + addMutationToLog = true; } } + if (addMutationToLog) { + auto& mLV = data->addVersionToMutationLog(data->data().getLatestVersion()); + data->addMutationToMutationLog( + mLV, + MutationRef( + MutationRef::SetValue, + persistChangeFeedKeys.begin.toString() + cfEntry.rangeId.toString(), + changeFeedSSValue(cfEntry.range, changeFeedInfo->emptyVersion + 1, changeFeedInfo->stopVersion))); + } } return feedIds; } @@ -5486,7 +5537,6 @@ void changeServerKeys(StorageServer* data, auto feed = data->uidChangeFeed.find(f.first); if (feed != data->uidChangeFeed.end()) { feed->second->emptyVersion = version - 1; - feed->second->stopped = true; feed->second->removing = true; feed->second->moved(feed->second->range); feed->second->newMutations.trigger(); @@ -5664,7 +5714,7 @@ private: rollback(data, rollbackVersion, currentVersion); } for (auto& it : data->uidChangeFeed) { - if (!it.second->stopped && !it.second->removing) { + if (!it.second->removing && currentVersion < it.second->stopVersion) { it.second->mutations.push_back(MutationsAndVersionRef(currentVersion, rollbackVersion)); it.second->mutations.back().mutations.push_back_deep(it.second->mutations.back().arena(), m); data->currentChangeFeeds.insert(it.first); @@ -5757,7 +5807,7 @@ private: } bool addMutationToLog = false; - if (popVersion != invalidVersion) { + if (popVersion != invalidVersion && status != ChangeFeedStatus::CHANGE_FEED_DESTROY) { // pop the change feed at pop version, no matter what state it is in if (popVersion - 1 > feed->second->emptyVersion) { feed->second->emptyVersion = popVersion - 1; @@ -5775,22 +5825,24 @@ private: } addMutationToLog = true; } - feed->second->stopped = (status == ChangeFeedStatus::CHANGE_FEED_STOP); - } else if (status == ChangeFeedStatus::CHANGE_FEED_CREATE) { + + } else if (status == ChangeFeedStatus::CHANGE_FEED_CREATE && createdFeed) { TraceEvent(SevDebug, "CreatingChangeFeed", data->thisServerID) .detail("RangeID", changeFeedId.printable()) .detail("Range", changeFeedRange.toString()) .detail("Version", currentVersion); - // no-op, already created + // no-op, already created metadata addMutationToLog = true; - } else if (status == ChangeFeedStatus::CHANGE_FEED_STOP) { + } + if (status == ChangeFeedStatus::CHANGE_FEED_STOP && currentVersion < feed->second->stopVersion) { TraceEvent(SevDebug, "StoppingChangeFeed", data->thisServerID) .detail("RangeID", changeFeedId.printable()) .detail("Range", changeFeedRange.toString()) .detail("Version", currentVersion); - feed->second->stopped = true; + feed->second->stopVersion = currentVersion; addMutationToLog = true; - } else if (status == ChangeFeedStatus::CHANGE_FEED_DESTROY && !createdFeed) { + } + if (status == ChangeFeedStatus::CHANGE_FEED_DESTROY && !createdFeed) { TraceEvent(SevDebug, "DestroyingChangeFeed", data->thisServerID) .detail("RangeID", changeFeedId.printable()) .detail("Range", changeFeedRange.toString()) @@ -5808,7 +5860,7 @@ private: ++data->counters.kvSystemClearRanges; feed->second->emptyVersion = currentVersion - 1; - feed->second->stopped = true; + feed->second->stopVersion = currentVersion; feed->second->removing = true; feed->second->moved(feed->second->range); feed->second->newMutations.trigger(); @@ -5822,7 +5874,8 @@ private: mLV, MutationRef(MutationRef::SetValue, persistChangeFeedKeys.begin.toString() + changeFeedId.toString(), - m.param2)); + changeFeedSSValue( + feed->second->range, feed->second->emptyVersion + 1, feed->second->stopVersion))); } } else if (m.param1.substr(1).startsWith(tssMappingKeys.begin) && (m.type == MutationRef::SetValue || m.type == MutationRef::ClearRange)) { @@ -5973,7 +6026,9 @@ ACTOR Future update(StorageServer* data, bool* pReceivedUpdate) { ++data->counters.updateBatches; data->lastTLogVersion = cursor->getMaxKnownVersion(); - data->knownCommittedVersion = cursor->getMinKnownCommittedVersion(); + if (cursor->getMinKnownCommittedVersion() > data->knownCommittedVersion.get()) { + data->knownCommittedVersion.set(cursor->getMinKnownCommittedVersion()); + } data->versionLag = std::max(0, data->lastTLogVersion - data->version.get()); ASSERT(*pReceivedUpdate == false); @@ -6394,7 +6449,7 @@ ACTOR Future updateStorage(StorageServer* data) { .detail("RangeID", info->second->id.printable()) .detail("Range", info->second->range.toString()) .detail("FetchVersion", info->second->fetchVersion) - .detail("Stopped", info->second->stopped) + .detail("StopVersion", info->second->stopVersion) .detail("Removing", info->second->removing); feedFetchVersions.push_back(std::pair(info->second->id, info->second->fetchVersion)); } @@ -6475,7 +6530,7 @@ ACTOR Future updateStorage(StorageServer* data) { .detail("FetchVersion", info->second->fetchVersion) .detail("OldDurableVersion", info->second->durableFetchVersion.get()) .detail("NewDurableVersion", feedFetchVersions[curFeed].second) - .detail("Stopped", info->second->stopped) + .detail("StopVersion", info->second->stopVersion) .detail("Removing", info->second->removing); if (feedFetchVersions[curFeed].second > info->second->durableFetchVersion.get()) { info->second->durableFetchVersion.set(feedFetchVersions[curFeed].second); @@ -6995,13 +7050,12 @@ ACTOR Future restoreDurableState(StorageServer* data, IKeyValueStore* stor for (feedLoc = 0; feedLoc < changeFeeds.size(); feedLoc++) { Key changeFeedId = changeFeeds[feedLoc].key.removePrefix(persistChangeFeedKeys.begin); KeyRange changeFeedRange; - Version popVersion; - ChangeFeedStatus status; - std::tie(changeFeedRange, popVersion, status) = decodeChangeFeedValue(changeFeeds[feedLoc].value); + Version popVersion, stopVersion; + std::tie(changeFeedRange, popVersion, stopVersion) = decodeChangeFeedSSValue(changeFeeds[feedLoc].value); TraceEvent(SevDebug, "RestoringChangeFeed", data->thisServerID) .detail("RangeID", changeFeedId.printable()) .detail("Range", changeFeedRange.toString()) - .detail("Status", status) + .detail("StopVersion", stopVersion) .detail("PopVer", popVersion); Reference changeFeedInfo(new ChangeFeedInfo()); changeFeedInfo->range = changeFeedRange; @@ -7009,7 +7063,7 @@ ACTOR Future restoreDurableState(StorageServer* data, IKeyValueStore* stor changeFeedInfo->durableVersion = version; changeFeedInfo->storageVersion = version; changeFeedInfo->emptyVersion = popVersion - 1; - changeFeedInfo->stopped = status == ChangeFeedStatus::CHANGE_FEED_STOP; + changeFeedInfo->stopVersion = stopVersion; data->uidChangeFeed[changeFeedId] = changeFeedInfo; auto rs = data->keyChangeFeed.modify(changeFeedRange); for (auto r = rs.begin(); r != rs.end(); ++r) { From b67930cb9355a86dd8c16ad71977060715dda970 Mon Sep 17 00:00:00 2001 From: Josh Slocum Date: Thu, 24 Feb 2022 11:31:36 -0600 Subject: [PATCH 274/413] Fixing bug in new change feed fetch logic --- fdbserver/storageserver.actor.cpp | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/fdbserver/storageserver.actor.cpp b/fdbserver/storageserver.actor.cpp index 6b3d3f1f45..8c53aaec38 100644 --- a/fdbserver/storageserver.actor.cpp +++ b/fdbserver/storageserver.actor.cpp @@ -1748,6 +1748,7 @@ ACTOR Future overlappingChangeFeedsQ(StorageServer* data, OverlappingChang // Make sure all of the stop versions we are sending aren't going to get rolled back if (knownCommittedRequired != invalidVersion && knownCommittedRequired > data->knownCommittedVersion.get()) { + TEST(true); // overlapping change feeds waiting for stop version to be committed wait(data->knownCommittedVersion.whenAtLeast(knownCommittedRequired)); } req.reply.send(reply); @@ -4735,7 +4736,7 @@ ACTOR Future> fetchChangeFeedMetadata(StorageServer* data, KeyR addMutationToLog = true; } else { - auto changeFeedInfo = existingEntry->second; + changeFeedInfo = existingEntry->second; auto feedCleanup = data->changeFeedCleanupDurable.find(cfEntry.rangeId); if (cfEntry.stopVersion < changeFeedInfo->stopVersion) { @@ -4771,6 +4772,7 @@ ACTOR Future> fetchChangeFeedMetadata(StorageServer* data, KeyR } } if (addMutationToLog) { + ASSERT(changeFeedInfo.isValid()); auto& mLV = data->addVersionToMutationLog(data->data().getLatestVersion()); data->addMutationToMutationLog( mLV, From 86236b766dabbdef41aea9d197fb8f993c0221ec Mon Sep 17 00:00:00 2001 From: Josh Slocum Date: Thu, 24 Feb 2022 12:11:06 -0600 Subject: [PATCH 275/413] Added print debugging for CF memory vs disk validation --- fdbserver/storageserver.actor.cpp | 46 +++++++++++++++++++++++++++++++ 1 file changed, 46 insertions(+) diff --git a/fdbserver/storageserver.actor.cpp b/fdbserver/storageserver.actor.cpp index 8c53aaec38..ca021534fe 100644 --- a/fdbserver/storageserver.actor.cpp +++ b/fdbserver/storageserver.actor.cpp @@ -2040,6 +2040,37 @@ ACTOR Future> getChangeFeedMutations(Stor if (memoryVerifyIdx < memoryReply.mutations.size()) { ASSERT(version <= memoryReply.mutations[memoryVerifyIdx].version); if (version == memoryReply.mutations[memoryVerifyIdx].version) { + // TODO REMOVE debugging eventually + if (m.mutations.size() != memoryReply.mutations[memoryVerifyIdx].mutations.size()) { + printf("ERROR: SS %s CF %s SQ %s has different sizes for mutations at %lld in memory %d vs " + "on disk %d!!!\n", + data->thisServerID.toString().substr(0, 4).c_str(), + req.rangeID.printable().substr(0, 6).c_str(), + streamUID.toString().substr(0, 8).c_str(), + version, + m.mutations.size(), + memoryReply.mutations[memoryVerifyIdx].mutations.size()); + + printf(" Memory: (%d)\n", memoryReply.mutations[memoryVerifyIdx].mutations.size()); + for (auto& it : memoryReply.mutations[memoryVerifyIdx].mutations) { + if (it.type == MutationRef::SetValue) { + printf(" %s=\n", it.param1.printable().c_str()); + } else { + printf( + " %s - %s\n", it.param1.printable().c_str(), it.param2.printable().c_str()); + } + } + + printf(" Disk: (%d)\n", m.mutations.size()); + for (auto& it : m.mutations) { + if (it.type == MutationRef::SetValue) { + printf(" %s=\n", it.param1.printable().c_str()); + } else { + printf( + " %s - %s\n", it.param1.printable().c_str(), it.param2.printable().c_str()); + } + } + } ASSERT(m.mutations.size() == memoryReply.mutations[memoryVerifyIdx].mutations.size()); for (int i = 0; i < m.mutations.size(); i++) { ASSERT(m.mutations[i].type == memoryReply.mutations[memoryVerifyIdx].mutations[i].type); @@ -2051,6 +2082,21 @@ ACTOR Future> getChangeFeedMutations(Stor } } else if (memoryVerifyIdx < memoryReply.mutations.size() && version == memoryReply.mutations[memoryVerifyIdx].version) { + // TODO REMOVE debugging eventually + printf("ERROR: SS %s CF %s SQ %s has mutation at %lld in memory but all filtered out on disk!\n", + data->thisServerID.toString().substr(0, 4).c_str(), + req.rangeID.printable().substr(0, 6).c_str(), + streamUID.toString().substr(0, 8).c_str(), + version); + + printf(" Memory: (%d)\n", memoryReply.mutations[memoryVerifyIdx].mutations.size()); + for (auto& it : memoryReply.mutations[memoryVerifyIdx].mutations) { + if (it.type == MutationRef::SetValue) { + printf(" %s=\n", it.param1.printable().c_str()); + } else { + printf(" %s - %s\n", it.param1.printable().c_str(), it.param2.printable().c_str()); + } + } ASSERT(false); } remainingDurableBytes -= From b6d02478cd28ec34587b23e0398b4671e27d4865 Mon Sep 17 00:00:00 2001 From: Josh Slocum Date: Thu, 24 Feb 2022 12:19:25 -0600 Subject: [PATCH 276/413] Adding stream uid tracking for ss feed debugging --- fdbserver/storageserver.actor.cpp | 32 ++++++++++++++++--------------- 1 file changed, 17 insertions(+), 15 deletions(-) diff --git a/fdbserver/storageserver.actor.cpp b/fdbserver/storageserver.actor.cpp index ca021534fe..ed9adff414 100644 --- a/fdbserver/storageserver.actor.cpp +++ b/fdbserver/storageserver.actor.cpp @@ -1843,10 +1843,12 @@ MutationsAndVersionRef filterMutations(Arena& arena, // TODO REMOVE!!! when BG is correctness clean #define DEBUG_SS_ID ""_sr #define DEBUG_SS_CF_ID ""_sr +#define DEBUG_SS_STREAM_ID ""_sr #define DEBUG_SS_CF_BEGIN_VERSION invalidVersion -#define DEBUG_SS_CFM(ssId, cfId, v) \ - ssId.toString().substr(0, 4) == DEBUG_SS_ID&& cfId.printable().substr(0, 6) == DEBUG_SS_CF_ID && \ - (v >= DEBUG_SS_CF_BEGIN_VERSION || latestVersion == DEBUG_SS_CF_BEGIN_VERSION) +#define DEBUG_SS_CFM(ssId, cfId, streamId, v) \ + ((ssId.toString().substr(0, 4) == DEBUG_SS_ID && cfId.printable().substr(0, 6) == DEBUG_SS_CF_ID && \ + (v >= DEBUG_SS_CF_BEGIN_VERSION || latestVersion == DEBUG_SS_CF_BEGIN_VERSION)) || \ + (streamId.toString().substr(0, 8) == DEBUG_SS_STREAM_ID)) #define DO_DEBUG_CF_MISSING false #define DEBUG_CF_MISSING_CF ""_sr @@ -1869,7 +1871,7 @@ ACTOR Future> getChangeFeedMutations(Stor state int remainingDurableBytes = CLIENT_KNOBS->REPLY_BYTE_LIMIT; state Version startVersion = data->version.get(); - if (DEBUG_SS_CFM(data->thisServerID, req.rangeID, req.begin)) { + if (DEBUG_SS_CFM(data->thisServerID, req.rangeID, streamUID, req.begin)) { printf("CFM: SS %s CF %s: SQ %s [%s - %s) %lld - %lld\n", data->thisServerID.toString().substr(0, 4).c_str(), req.rangeID.printable().substr(0, 6).c_str(), @@ -1906,7 +1908,7 @@ ACTOR Future> getChangeFeedMutations(Stor state Reference feedInfo = feed->second; - if (DEBUG_SS_CFM(data->thisServerID, req.rangeID, req.begin)) { + if (DEBUG_SS_CFM(data->thisServerID, req.rangeID, streamUID, req.begin)) { printf("CFM: SS %s CF %s: got version %lld >= %lld\n", data->thisServerID.toString().substr(0, 4).c_str(), req.rangeID.printable().substr(0, 6).c_str(), @@ -1921,7 +1923,7 @@ ACTOR Future> getChangeFeedMutations(Stor state Version emptyVersion = feedInfo->emptyVersion; Version fetchStorageVersion = std::max(feedInfo->fetchVersion, feedInfo->durableFetchVersion.get()); - if (DEBUG_SS_CFM(data->thisServerID, req.rangeID, req.begin)) { + if (DEBUG_SS_CFM(data->thisServerID, req.rangeID, streamUID, req.begin)) { printf("CFM: SS %s CF %s: SQ %s atLatest=%s, dequeVersion=%lld, emptyVersion=%lld, storageVersion=%lld, " "durableVersion=%lld, " "fetchStorageVersion=%lld (%lld, %lld)\n", @@ -1967,7 +1969,7 @@ ACTOR Future> getChangeFeedMutations(Stor } } } - if (DEBUG_SS_CFM(data->thisServerID, req.rangeID, req.begin)) { + if (DEBUG_SS_CFM(data->thisServerID, req.rangeID, streamUID, req.begin)) { printf("CFM: SS %s CF %s: got %lld - %lld (%d) from memory\n", data->thisServerID.toString().substr(0, 4).c_str(), req.rangeID.printable().substr(0, 6).c_str(), @@ -1987,7 +1989,7 @@ ACTOR Future> getChangeFeedMutations(Stor ASSERT(req.begin <= feedInfo->fetchVersion); TEST(true); // getChangeFeedMutations before fetched data durable - if (DEBUG_SS_CFM(data->thisServerID, req.rangeID, req.begin)) { + if (DEBUG_SS_CFM(data->thisServerID, req.rangeID, streamUID, req.begin)) { printf("CFM: SS %s CF %s: waiting on fetch durable up to %lld\n", data->thisServerID.toString().substr(0, 4).c_str(), req.rangeID.printable().substr(0, 6).c_str(), @@ -1998,7 +2000,7 @@ ACTOR Future> getChangeFeedMutations(Stor wait(feedInfo->durableFetchVersion.whenAtLeast(feedInfo->fetchVersion)); // To let update storage finish wait(delay(0)); - if (DEBUG_SS_CFM(data->thisServerID, req.rangeID, req.begin)) { + if (DEBUG_SS_CFM(data->thisServerID, req.rangeID, streamUID, req.begin)) { printf("CFM: SS %s CF %s: got fetch durable up to %lld\n", data->thisServerID.toString().substr(0, 4).c_str(), req.rangeID.printable().substr(0, 6).c_str(), @@ -2106,7 +2108,7 @@ ACTOR Future> getChangeFeedMutations(Stor lastVersion = version; lastKnownCommitted = knownCommittedVersion; } - if (DEBUG_SS_CFM(data->thisServerID, req.rangeID, req.begin)) { + if (DEBUG_SS_CFM(data->thisServerID, req.rangeID, streamUID, req.begin)) { printf("CFM: SS %s CF %s: got %lld - %lld (%d) from disk\n", data->thisServerID.toString().substr(0, 4).c_str(), req.rangeID.printable().substr(0, 6).c_str(), @@ -2124,7 +2126,7 @@ ACTOR Future> getChangeFeedMutations(Stor } reply.mutations.append(reply.arena, it, totalCount); } else if (reply.mutations.empty() || reply.mutations.back().version < lastVersion) { - if (DEBUG_SS_CFM(data->thisServerID, req.rangeID, req.begin)) { + if (DEBUG_SS_CFM(data->thisServerID, req.rangeID, streamUID, req.begin)) { printf("CFM: SS %s CF %s: adding empty from disk %lld\n", data->thisServerID.toString().substr(0, 4).c_str(), req.rangeID.printable().substr(0, 6).c_str(), @@ -2140,7 +2142,7 @@ ACTOR Future> getChangeFeedMutations(Stor Version finalVersion = std::min(req.end - 1, dequeVersion); if ((reply.mutations.empty() || reply.mutations.back().version < finalVersion) && remainingLimitBytes > 0 && remainingDurableBytes > 0) { - if (DEBUG_SS_CFM(data->thisServerID, req.rangeID, req.begin)) { + if (DEBUG_SS_CFM(data->thisServerID, req.rangeID, streamUID, req.begin)) { printf("CFM: SS %s CF %s: adding empty %lld\n", data->thisServerID.toString().substr(0, 4).c_str(), req.rangeID.printable().substr(0, 6).c_str(), @@ -2193,7 +2195,7 @@ ACTOR Future> getChangeFeedMutations(Stor } } - if (DEBUG_SS_CFM(data->thisServerID, req.rangeID, req.begin)) { + if (DEBUG_SS_CFM(data->thisServerID, req.rangeID, streamUID, req.begin)) { printf("CFM: SS %s CF %s: result %lld - %lld (%d)\n", data->thisServerID.toString().substr(0, 4).c_str(), req.rangeID.printable().substr(0, 6).c_str(), @@ -2347,7 +2349,7 @@ ACTOR Future changeFeedStreamQ(StorageServer* data, ChangeFeedStreamReques wait(delay(0, TaskPriority::DefaultEndpoint)); try { - if (DEBUG_SS_CFM(data->thisServerID, req.rangeID, req.begin)) { + if (DEBUG_SS_CFM(data->thisServerID, req.rangeID, streamUID, req.begin)) { printf("CFM: SS %s CF %s: got CFSQ %s [%s - %s) %lld - %lld, crp=%s\n", data->thisServerID.toString().substr(0, 4).c_str(), req.rangeID.printable().substr(0, 6).c_str(), @@ -2369,7 +2371,7 @@ ACTOR Future changeFeedStreamQ(StorageServer* data, ChangeFeedStreamReques ASSERT(emptyInitialReply.minStreamVersion == invalidVersion); req.reply.send(emptyInitialReply); - if (DEBUG_SS_CFM(data->thisServerID, req.rangeID, req.begin)) { + if (DEBUG_SS_CFM(data->thisServerID, req.rangeID, streamUID, req.begin)) { printf("CFM: SS %s CF %s: CFSQ %s send empty initial version %lld\n", data->thisServerID.toString().substr(0, 4).c_str(), req.rangeID.printable().substr(0, 6).c_str(), From 43543b0d0e6987d757ff0f05d722100c036cc214 Mon Sep 17 00:00:00 2001 From: Josh Slocum Date: Thu, 24 Feb 2022 12:39:15 -0600 Subject: [PATCH 277/413] Removing incorrect CF validation asserts --- fdbserver/storageserver.actor.cpp | 39 ++----------------------------- 1 file changed, 2 insertions(+), 37 deletions(-) diff --git a/fdbserver/storageserver.actor.cpp b/fdbserver/storageserver.actor.cpp index ed9adff414..65c7abaa04 100644 --- a/fdbserver/storageserver.actor.cpp +++ b/fdbserver/storageserver.actor.cpp @@ -2042,43 +2042,8 @@ ACTOR Future> getChangeFeedMutations(Stor if (memoryVerifyIdx < memoryReply.mutations.size()) { ASSERT(version <= memoryReply.mutations[memoryVerifyIdx].version); if (version == memoryReply.mutations[memoryVerifyIdx].version) { - // TODO REMOVE debugging eventually - if (m.mutations.size() != memoryReply.mutations[memoryVerifyIdx].mutations.size()) { - printf("ERROR: SS %s CF %s SQ %s has different sizes for mutations at %lld in memory %d vs " - "on disk %d!!!\n", - data->thisServerID.toString().substr(0, 4).c_str(), - req.rangeID.printable().substr(0, 6).c_str(), - streamUID.toString().substr(0, 8).c_str(), - version, - m.mutations.size(), - memoryReply.mutations[memoryVerifyIdx].mutations.size()); - - printf(" Memory: (%d)\n", memoryReply.mutations[memoryVerifyIdx].mutations.size()); - for (auto& it : memoryReply.mutations[memoryVerifyIdx].mutations) { - if (it.type == MutationRef::SetValue) { - printf(" %s=\n", it.param1.printable().c_str()); - } else { - printf( - " %s - %s\n", it.param1.printable().c_str(), it.param2.printable().c_str()); - } - } - - printf(" Disk: (%d)\n", m.mutations.size()); - for (auto& it : m.mutations) { - if (it.type == MutationRef::SetValue) { - printf(" %s=\n", it.param1.printable().c_str()); - } else { - printf( - " %s - %s\n", it.param1.printable().c_str(), it.param2.printable().c_str()); - } - } - } - ASSERT(m.mutations.size() == memoryReply.mutations[memoryVerifyIdx].mutations.size()); - for (int i = 0; i < m.mutations.size(); i++) { - ASSERT(m.mutations[i].type == memoryReply.mutations[memoryVerifyIdx].mutations[i].type); - ASSERT(m.mutations[i].param1 == memoryReply.mutations[memoryVerifyIdx].mutations[i].param1); - ASSERT(m.mutations[i].param2 == memoryReply.mutations[memoryVerifyIdx].mutations[i].param2); - } + // TODO: we could do some validation here too, but it's complicated because clears can get split + // and stuff memoryVerifyIdx++; } } From e5b4fb3d8065c2743ab3df11c4851a93ade38162 Mon Sep 17 00:00:00 2001 From: Josh Slocum Date: Thu, 24 Feb 2022 15:09:38 -0600 Subject: [PATCH 278/413] Adding explicit popVersion to change feed reply, like tlogs have --- fdbclient/DatabaseContext.h | 2 ++ fdbclient/NativeAPI.actor.cpp | 7 +++++++ fdbclient/StorageServerInterface.h | 2 ++ fdbserver/BlobWorker.actor.cpp | 3 +++ fdbserver/storageserver.actor.cpp | 27 +++++++++++++++++++++++++++ 5 files changed, 41 insertions(+) diff --git a/fdbclient/DatabaseContext.h b/fdbclient/DatabaseContext.h index 2af9530ce8..7b97f5db90 100644 --- a/fdbclient/DatabaseContext.h +++ b/fdbclient/DatabaseContext.h @@ -200,6 +200,8 @@ struct ChangeFeedData : ReferenceCounted { Promise refresh; Version maxSeenVersion; Version endVersion = invalidVersion; + Version popVersion = + invalidVersion; // like TLog pop version, set by SS and client can check it to see if they missed data ChangeFeedData() : notAtLatest(1) {} }; diff --git a/fdbclient/NativeAPI.actor.cpp b/fdbclient/NativeAPI.actor.cpp index af29c4f450..7033ebc792 100644 --- a/fdbclient/NativeAPI.actor.cpp +++ b/fdbclient/NativeAPI.actor.cpp @@ -7466,6 +7466,9 @@ ACTOR Future partialChangeFeedStream(StorageServerInterface interf, if (rep.mutations.back().version > feedData->maxSeenVersion) { feedData->maxSeenVersion = rep.mutations.back().version; } + if (rep.popVersion > feedData->popVersion) { + feedData->popVersion = rep.popVersion; + } state int resultLoc = 0; while (resultLoc < rep.mutations.size()) { @@ -7836,6 +7839,10 @@ ACTOR Future doSingleCFStream(KeyRange range, state ChangeFeedStreamReply feedReply = waitNext(results->streams[0].getFuture()); *begin = feedReply.mutations.back().version + 1; + if (feedReply.popVersion > results->popVersion) { + results->popVersion = feedReply.popVersion; + } + // don't send completely empty set of mutations to promise stream bool anyMutations = false; for (auto& it : feedReply.mutations) { diff --git a/fdbclient/StorageServerInterface.h b/fdbclient/StorageServerInterface.h index e314fbc05d..db8fd1e841 100644 --- a/fdbclient/StorageServerInterface.h +++ b/fdbclient/StorageServerInterface.h @@ -690,6 +690,7 @@ struct ChangeFeedStreamReply : public ReplyPromiseStreamReply { VectorRef mutations; bool atLatestVersion = false; Version minStreamVersion = invalidVersion; + Version popVersion = invalidVersion; ChangeFeedStreamReply() {} @@ -703,6 +704,7 @@ struct ChangeFeedStreamReply : public ReplyPromiseStreamReply { mutations, atLatestVersion, minStreamVersion, + popVersion, arena); } }; diff --git a/fdbserver/BlobWorker.actor.cpp b/fdbserver/BlobWorker.actor.cpp index 9521b968b0..71c6592e56 100644 --- a/fdbserver/BlobWorker.actor.cpp +++ b/fdbserver/BlobWorker.actor.cpp @@ -1473,6 +1473,9 @@ ACTOR Future blobGranuleUpdateFiles(Reference bwData, mutations.front().version); } ASSERT(mutations.front().version > metadata->waitForVersionReturned); + + // If this assert trips we should have gotten change_feed_popped from SS and didn't + ASSERT(mutations.front().version >= metadata->activeCFData.get()->popVersion); } when(wait(inFlightFiles.empty() ? Never() : success(inFlightFiles.front().future))) { // TODO REMOVE diff --git a/fdbserver/storageserver.actor.cpp b/fdbserver/storageserver.actor.cpp index 65c7abaa04..1494d7acc1 100644 --- a/fdbserver/storageserver.actor.cpp +++ b/fdbserver/storageserver.actor.cpp @@ -2229,6 +2229,8 @@ ACTOR Future> getChangeFeedMutations(Stor } } + reply.popVersion = feedInfo->emptyVersion + 1; + // If the SS's version advanced at all during any of the waits, the read from memory may have missed some // mutations, so gotAll can only be true if data->version didn't change over the course of this actor return std::make_pair(reply, gotAll); @@ -4529,6 +4531,19 @@ ACTOR Future fetchChangeFeedApplier(StorageServer* data, state int remoteLoc = 0; while (remoteLoc < remoteResult.size()) { + if (feedResults->popVersion - 1 > changeFeedInfo->emptyVersion) { + TEST(true); // CF fetched updated popped version from src SS + changeFeedInfo->emptyVersion = feedResults->popVersion - 1; + auto& mLV = data->addVersionToMutationLog(data->data().getLatestVersion()); + data->addMutationToMutationLog( + mLV, + MutationRef(MutationRef::SetValue, + persistChangeFeedKeys.begin.toString() + changeFeedInfo->id.toString(), + changeFeedSSValue(changeFeedInfo->range, + changeFeedInfo->emptyVersion + 1, + changeFeedInfo->stopVersion))); + } + Version localVersion = localResult.version; Version remoteVersion = remoteResult[remoteLoc].version; if (remoteVersion <= localVersion) { @@ -4609,6 +4624,18 @@ ACTOR Future fetchChangeFeedApplier(StorageServer* data, } } + if (feedResults->popVersion - 1 > changeFeedInfo->emptyVersion) { + TEST(true); // CF fetched updated popped version from src SS at end + changeFeedInfo->emptyVersion = feedResults->popVersion - 1; + auto& mLV = data->addVersionToMutationLog(data->data().getLatestVersion()); + data->addMutationToMutationLog( + mLV, + MutationRef(MutationRef::SetValue, + persistChangeFeedKeys.begin.toString() + changeFeedInfo->id.toString(), + changeFeedSSValue( + changeFeedInfo->range, changeFeedInfo->emptyVersion + 1, changeFeedInfo->stopVersion))); + } + // if we were popped or removed while fetching but it didn't pass the fetch version while writing, clean up here if (versionsFetched > 0 && startVersion < changeFeedInfo->emptyVersion) { ASSERT(firstVersion != invalidVersion); From cee9f420990693ceb72ba4722bccb59227ee04c7 Mon Sep 17 00:00:00 2001 From: Josh Slocum Date: Thu, 24 Feb 2022 16:39:24 -0600 Subject: [PATCH 279/413] More fixes and debugging improvements found by validation --- fdbserver/storageserver.actor.cpp | 55 +++++++++++++++++++++++++------ 1 file changed, 45 insertions(+), 10 deletions(-) diff --git a/fdbserver/storageserver.actor.cpp b/fdbserver/storageserver.actor.cpp index 1494d7acc1..a0ee41fcbe 100644 --- a/fdbserver/storageserver.actor.cpp +++ b/fdbserver/storageserver.actor.cpp @@ -2040,6 +2040,25 @@ ACTOR Future> getChangeFeedMutations(Stor // they are the same. In particular this validates the consistency of change feed data recieved from the // tlog mutations vs change feed data fetched from another storage server if (memoryVerifyIdx < memoryReply.mutations.size()) { + if (version > memoryReply.mutations[memoryVerifyIdx].version) { + printf("ERROR: SS %s CF %s SQ %s has mutation at %lld in memory but not on disk " + "(emptyVersion=%lld, emptyBefore=%lld)!\n", + data->thisServerID.toString().substr(0, 4).c_str(), + req.rangeID.printable().substr(0, 6).c_str(), + streamUID.toString().substr(0, 8).c_str(), + memoryReply.mutations[memoryVerifyIdx].version, + feedInfo->emptyVersion, + emptyVersion); + + printf(" Memory: (%d)\n", memoryReply.mutations[memoryVerifyIdx].mutations.size()); + for (auto& it : memoryReply.mutations[memoryVerifyIdx].mutations) { + if (it.type == MutationRef::SetValue) { + printf(" %s=\n", it.param1.printable().c_str()); + } else { + printf(" %s - %s\n", it.param1.printable().c_str(), it.param2.printable().c_str()); + } + } + } ASSERT(version <= memoryReply.mutations[memoryVerifyIdx].version); if (version == memoryReply.mutations[memoryVerifyIdx].version) { // TODO: we could do some validation here too, but it's complicated because clears can get split @@ -4588,16 +4607,27 @@ ACTOR Future fetchChangeFeedApplier(StorageServer* data, } lastVersion = remoteVersion; versionsFetched++; - } else if (versionsFetched > 0) { - ASSERT(firstVersion != invalidVersion); - ASSERT(lastVersion != invalidVersion); - data->storage.clearRange( - KeyRangeRef(changeFeedDurableKey(changeFeedInfo->id, firstVersion), - changeFeedDurableKey(changeFeedInfo->id, lastVersion + 1))); - ++data->counters.kvSystemClearRanges; - firstVersion = invalidVersion; - lastVersion = invalidVersion; - versionsFetched = 0; + } else { + if (MUTATION_TRACKING_ENABLED) { + for (auto& m : remoteResult[remoteLoc].mutations) { + DEBUG_MUTATION("ChangeFeedWriteMoveIgnore", remoteVersion, m, data->thisServerID) + .detail("Range", range) + .detail("Existing", existing) + .detail("ChangeFeedID", rangeId) + .detail("EmptyVersion", changeFeedInfo->emptyVersion); + } + } + if (versionsFetched > 0) { + ASSERT(firstVersion != invalidVersion); + ASSERT(lastVersion != invalidVersion); + data->storage.clearRange( + KeyRangeRef(changeFeedDurableKey(changeFeedInfo->id, firstVersion), + changeFeedDurableKey(changeFeedInfo->id, lastVersion + 1))); + ++data->counters.kvSystemClearRanges; + firstVersion = invalidVersion; + lastVersion = invalidVersion; + versionsFetched = 0; + } } remoteLoc++; } @@ -4647,6 +4677,10 @@ ACTOR Future fetchChangeFeedApplier(StorageServer* data, changeFeedDurableKey(changeFeedInfo->id, endClear))); ++data->counters.kvSystemClearRanges; } + while (!changeFeedInfo->mutations.empty() && + changeFeedInfo->mutations.front().version <= changeFeedInfo->emptyVersion) { + changeFeedInfo->mutations.pop_front(); + } } // TODO REMOVE? @@ -4655,6 +4689,7 @@ ACTOR Future fetchChangeFeedApplier(StorageServer* data, .detail("Range", range.toString()) .detail("StartVersion", startVersion) .detail("EndVersion", endVersion) + .detail("EmptyVersion", changeFeedInfo->emptyVersion) .detail("FirstFetchedVersion", firstVersion) .detail("LastFetchedVersion", lastVersion) .detail("VersionsFetched", versionsFetched) From f3f4764916494b49f3eaf106b81a76d982ba3082 Mon Sep 17 00:00:00 2001 From: Josh Slocum Date: Thu, 24 Feb 2022 17:12:05 -0600 Subject: [PATCH 280/413] Fixing change feed validation to handle special case --- fdbserver/storageserver.actor.cpp | 49 +++++++++++++++++++++---------- 1 file changed, 33 insertions(+), 16 deletions(-) diff --git a/fdbserver/storageserver.actor.cpp b/fdbserver/storageserver.actor.cpp index a0ee41fcbe..d5f2b5bcc5 100644 --- a/fdbserver/storageserver.actor.cpp +++ b/fdbserver/storageserver.actor.cpp @@ -1981,8 +1981,10 @@ ACTOR Future> getChangeFeedMutations(Stor state bool readDurable = feedInfo->durableVersion != invalidVersion && req.begin <= feedInfo->durableVersion; state bool readFetched = req.begin <= fetchStorageVersion && !atLatest; + state bool waitFetched = false; if (req.end > emptyVersion + 1 && (readDurable || readFetched)) { if (readFetched && req.begin <= feedInfo->fetchVersion) { + waitFetched = true; // Request needs data that has been written to storage by a change feed fetch, but not committed yet // To not block fetchKeys making normal SS data readable on making change feed data written to storage, we // wait in here instead for all fetched data to become readable from the storage engine. @@ -2040,26 +2042,41 @@ ACTOR Future> getChangeFeedMutations(Stor // they are the same. In particular this validates the consistency of change feed data recieved from the // tlog mutations vs change feed data fetched from another storage server if (memoryVerifyIdx < memoryReply.mutations.size()) { - if (version > memoryReply.mutations[memoryVerifyIdx].version) { - printf("ERROR: SS %s CF %s SQ %s has mutation at %lld in memory but not on disk " - "(emptyVersion=%lld, emptyBefore=%lld)!\n", - data->thisServerID.toString().substr(0, 4).c_str(), - req.rangeID.printable().substr(0, 6).c_str(), - streamUID.toString().substr(0, 8).c_str(), - memoryReply.mutations[memoryVerifyIdx].version, - feedInfo->emptyVersion, - emptyVersion); + while (version > memoryReply.mutations[memoryVerifyIdx].version) { + // there is a case where this can happen - if we wait on a fetching change feed, and the feed is + // popped while we wait, we could have copied the memory mutations into memoryReply before the + // pop, but they could have been skipped writing to disk + if (waitFetched && feedInfo->emptyVersion > emptyVersion && version > feedInfo->emptyVersion && + memoryReply.mutations[memoryVerifyIdx].version <= feedInfo->emptyVersion && + memoryReply.mutations[memoryVerifyIdx].version > emptyVersion) { + // ok + memoryVerifyIdx++; + continue; + } else { + printf( + "ERROR: SS %s CF %s SQ %s has mutation at %lld in memory but not on disk (next disk is " + "%lld) " + "(emptyVersion=%lld, emptyBefore=%lld)!\n", + data->thisServerID.toString().substr(0, 4).c_str(), + req.rangeID.printable().substr(0, 6).c_str(), + streamUID.toString().substr(0, 8).c_str(), + memoryReply.mutations[memoryVerifyIdx].version, + version, + feedInfo->emptyVersion, + emptyVersion); - printf(" Memory: (%d)\n", memoryReply.mutations[memoryVerifyIdx].mutations.size()); - for (auto& it : memoryReply.mutations[memoryVerifyIdx].mutations) { - if (it.type == MutationRef::SetValue) { - printf(" %s=\n", it.param1.printable().c_str()); - } else { - printf(" %s - %s\n", it.param1.printable().c_str(), it.param2.printable().c_str()); + printf(" Memory: (%d)\n", memoryReply.mutations[memoryVerifyIdx].mutations.size()); + for (auto& it : memoryReply.mutations[memoryVerifyIdx].mutations) { + if (it.type == MutationRef::SetValue) { + printf(" %s=\n", it.param1.printable().c_str()); + } else { + printf( + " %s - %s\n", it.param1.printable().c_str(), it.param2.printable().c_str()); + } } + ASSERT(false); } } - ASSERT(version <= memoryReply.mutations[memoryVerifyIdx].version); if (version == memoryReply.mutations[memoryVerifyIdx].version) { // TODO: we could do some validation here too, but it's complicated because clears can get split // and stuff From eac1cd6366208d3b06a2a0011d0d13c3e4b2fd09 Mon Sep 17 00:00:00 2001 From: Josh Slocum Date: Fri, 25 Feb 2022 08:16:03 -0600 Subject: [PATCH 281/413] Fixing another case in change feed validation --- fdbserver/storageserver.actor.cpp | 10 ++++++---- 1 file changed, 6 insertions(+), 4 deletions(-) diff --git a/fdbserver/storageserver.actor.cpp b/fdbserver/storageserver.actor.cpp index d5f2b5bcc5..db0386f3ba 100644 --- a/fdbserver/storageserver.actor.cpp +++ b/fdbserver/storageserver.actor.cpp @@ -2042,11 +2042,12 @@ ACTOR Future> getChangeFeedMutations(Stor // they are the same. In particular this validates the consistency of change feed data recieved from the // tlog mutations vs change feed data fetched from another storage server if (memoryVerifyIdx < memoryReply.mutations.size()) { - while (version > memoryReply.mutations[memoryVerifyIdx].version) { + while (memoryVerifyIdx < memoryReply.mutations.size() && + version > memoryReply.mutations[memoryVerifyIdx].version) { // there is a case where this can happen - if we wait on a fetching change feed, and the feed is // popped while we wait, we could have copied the memory mutations into memoryReply before the - // pop, but they could have been skipped writing to disk - if (waitFetched && feedInfo->emptyVersion > emptyVersion && version > feedInfo->emptyVersion && + // pop, but they may or may not have been skipped writing to disk + if (waitFetched && feedInfo->emptyVersion > emptyVersion && memoryReply.mutations[memoryVerifyIdx].version <= feedInfo->emptyVersion && memoryReply.mutations[memoryVerifyIdx].version > emptyVersion) { // ok @@ -2077,7 +2078,8 @@ ACTOR Future> getChangeFeedMutations(Stor ASSERT(false); } } - if (version == memoryReply.mutations[memoryVerifyIdx].version) { + if (memoryVerifyIdx < memoryReply.mutations.size() && + version == memoryReply.mutations[memoryVerifyIdx].version) { // TODO: we could do some validation here too, but it's complicated because clears can get split // and stuff memoryVerifyIdx++; From 623db663dcef22ae4195f15cc816c2e5e2296fb0 Mon Sep 17 00:00:00 2001 From: Josh Slocum Date: Fri, 25 Feb 2022 08:47:24 -0600 Subject: [PATCH 282/413] don't reset watch config transaction --- fdbserver/ClusterController.actor.cpp | 4 ---- 1 file changed, 4 deletions(-) diff --git a/fdbserver/ClusterController.actor.cpp b/fdbserver/ClusterController.actor.cpp index f27e2f520f..42628eaf4e 100644 --- a/fdbserver/ClusterController.actor.cpp +++ b/fdbserver/ClusterController.actor.cpp @@ -2304,10 +2304,6 @@ ACTOR Future watchBlobGranulesConfigKey(ClusterControllerData* self) { self->db.blobGranulesEnabled.set(blobConfig.get() == LiteralStringRef("1")); } - tr->reset(); - tr->setOption(FDBTransactionOptions::ACCESS_SYSTEM_KEYS); - tr->setOption(FDBTransactionOptions::PRIORITY_SYSTEM_IMMEDIATE); - state Future watch = tr->watch(blobGranuleConfigKey); wait(tr->commit()); wait(watch); From d41b18d132065c7eb5324989ddedb323861766e2 Mon Sep 17 00:00:00 2001 From: Josh Slocum Date: Fri, 25 Feb 2022 09:08:09 -0600 Subject: [PATCH 283/413] more change feed validation fixes --- fdbserver/storageserver.actor.cpp | 22 ++++++++++++++++------ 1 file changed, 16 insertions(+), 6 deletions(-) diff --git a/fdbserver/storageserver.actor.cpp b/fdbserver/storageserver.actor.cpp index db0386f3ba..d2c49e9883 100644 --- a/fdbserver/storageserver.actor.cpp +++ b/fdbserver/storageserver.actor.cpp @@ -2048,8 +2048,7 @@ ACTOR Future> getChangeFeedMutations(Stor // popped while we wait, we could have copied the memory mutations into memoryReply before the // pop, but they may or may not have been skipped writing to disk if (waitFetched && feedInfo->emptyVersion > emptyVersion && - memoryReply.mutations[memoryVerifyIdx].version <= feedInfo->emptyVersion && - memoryReply.mutations[memoryVerifyIdx].version > emptyVersion) { + memoryReply.mutations[memoryVerifyIdx].version <= feedInfo->emptyVersion) { // ok memoryVerifyIdx++; continue; @@ -4500,6 +4499,7 @@ ACTOR Future changeFeedPopQ(StorageServer* self, ChangeFeedPopRequest req) return Void(); } +// FIXME: there's a decent amount of duplicated code around fetching and popping change feeds // Returns max version fetched ACTOR Future fetchChangeFeedApplier(StorageServer* data, Reference changeFeedInfo, @@ -4572,6 +4572,11 @@ ACTOR Future fetchChangeFeedApplier(StorageServer* data, if (feedResults->popVersion - 1 > changeFeedInfo->emptyVersion) { TEST(true); // CF fetched updated popped version from src SS changeFeedInfo->emptyVersion = feedResults->popVersion - 1; + // pop mutations + while (!changeFeedInfo->mutations.empty() && + changeFeedInfo->mutations.front().version <= changeFeedInfo->emptyVersion) { + changeFeedInfo->mutations.pop_front(); + } auto& mLV = data->addVersionToMutationLog(data->data().getLatestVersion()); data->addMutationToMutationLog( mLV, @@ -4676,6 +4681,10 @@ ACTOR Future fetchChangeFeedApplier(StorageServer* data, if (feedResults->popVersion - 1 > changeFeedInfo->emptyVersion) { TEST(true); // CF fetched updated popped version from src SS at end changeFeedInfo->emptyVersion = feedResults->popVersion - 1; + while (!changeFeedInfo->mutations.empty() && + changeFeedInfo->mutations.front().version <= changeFeedInfo->emptyVersion) { + changeFeedInfo->mutations.pop_front(); + } auto& mLV = data->addVersionToMutationLog(data->data().getLatestVersion()); data->addMutationToMutationLog( mLV, @@ -4696,10 +4705,6 @@ ACTOR Future fetchChangeFeedApplier(StorageServer* data, changeFeedDurableKey(changeFeedInfo->id, endClear))); ++data->counters.kvSystemClearRanges; } - while (!changeFeedInfo->mutations.empty() && - changeFeedInfo->mutations.front().version <= changeFeedInfo->emptyVersion) { - changeFeedInfo->mutations.pop_front(); - } } // TODO REMOVE? @@ -4874,6 +4879,11 @@ ACTOR Future> fetchChangeFeedMetadata(StorageServer* data, KeyR MutationRef::SetValue, persistChangeFeedKeys.begin.toString() + cfEntry.rangeId.toString(), changeFeedSSValue(cfEntry.range, changeFeedInfo->emptyVersion + 1, changeFeedInfo->stopVersion))); + // if we updated pop version, remove mutations + while (!changeFeedInfo->mutations.empty() && + changeFeedInfo->mutations.front().version <= changeFeedInfo->emptyVersion) { + changeFeedInfo->mutations.pop_front(); + } } } return feedIds; From bc7cc984b016f76541d26a883ec5b0111fa5b14e Mon Sep 17 00:00:00 2001 From: Josh Slocum Date: Fri, 25 Feb 2022 11:30:21 -0600 Subject: [PATCH 284/413] Fixing BGVerifyBalance test killing issues --- fdbserver/BlobManager.actor.cpp | 36 ++++++++++++++++++++++++ fdbserver/BlobManagerInterface.h | 31 +++++++++++++++++++- fdbserver/CommitProxyServer.actor.cpp | 14 +++++++-- tests/slow/BlobGranuleVerifyBalance.toml | 4 --- 4 files changed, 78 insertions(+), 7 deletions(-) diff --git a/fdbserver/BlobManager.actor.cpp b/fdbserver/BlobManager.actor.cpp index 143705f288..b71947d721 100644 --- a/fdbserver/BlobManager.actor.cpp +++ b/fdbserver/BlobManager.actor.cpp @@ -2759,6 +2759,38 @@ ACTOR Future doLockChecks(Reference bmData) { } } +ACTOR Future blobManagerExclusionSafetyCheck(Reference self, + BlobManagerExclusionSafetyCheckRequest req) { + TraceEvent("BMExclusionSafetyCheckBegin", self->id).log(); + BlobManagerExclusionSafetyCheckReply reply(true); + // make sure at least one blob worker remains after exclusions + if (self->workersById.empty()) { + TraceEvent("BMExclusionSafetyCheckNoWorkers", self->id).log(); + reply.safe = false; + } else { + // TODO REMOVE prints + std::set remainingWorkers; + for (auto& worker : self->workersById) { + remainingWorkers.insert(worker.first); + } + for (const AddressExclusion& excl : req.exclusions) { + for (auto& worker : self->workersById) { + if (excl.excludes(worker.second.address())) { + remainingWorkers.erase(worker.first); + } + } + } + + TraceEvent("BMExclusionSafetyChecked", self->id).detail("RemainingWorkers", remainingWorkers.size()).log(); + reply.safe = !remainingWorkers.empty(); + } + + TraceEvent("BMExclusionSafetyCheckEnd", self->id).log(); + req.reply.send(reply); + + return Void(); +} + ACTOR Future blobManager(BlobManagerInterface bmInterf, Reference const> dbInfo, int64_t epoch) { @@ -2814,6 +2846,10 @@ ACTOR Future blobManager(BlobManagerInterface bmInterf, TraceEvent("BlobGranulesHalted", bmInterf.id()).detail("ReqID", req.requesterID); break; } + when(BlobManagerExclusionSafetyCheckRequest exclCheckReq = + waitNext(bmInterf.blobManagerExclCheckReq.getFuture())) { + self->addActor.send(blobManagerExclusionSafetyCheck(self, exclCheckReq)); + } when(wait(collection)) { TraceEvent("BlobManagerActorCollectionError"); ASSERT(false); diff --git a/fdbserver/BlobManagerInterface.h b/fdbserver/BlobManagerInterface.h index 8399e136c2..64adbc4443 100644 --- a/fdbserver/BlobManagerInterface.h +++ b/fdbserver/BlobManagerInterface.h @@ -31,6 +31,7 @@ struct BlobManagerInterface { RequestStream> waitFailure; RequestStream haltBlobManager; RequestStream haltBlobGranules; + RequestStream blobManagerExclCheckReq; struct LocalityData locality; UID myId; @@ -45,7 +46,7 @@ struct BlobManagerInterface { template void serialize(Archive& ar) { - serializer(ar, waitFailure, haltBlobManager, haltBlobGranules, locality, myId); + serializer(ar, waitFailure, haltBlobManager, haltBlobGranules, blobManagerExclCheckReq, locality, myId); } }; @@ -77,4 +78,32 @@ struct HaltBlobGranulesRequest { } }; +struct BlobManagerExclusionSafetyCheckReply { + constexpr static FileIdentifier file_identifier = 8068627; + bool safe; + + BlobManagerExclusionSafetyCheckReply() : safe(false) {} + explicit BlobManagerExclusionSafetyCheckReply(bool safe) : safe(safe) {} + + template + void serialize(Ar& ar) { + serializer(ar, safe); + } +}; + +struct BlobManagerExclusionSafetyCheckRequest { + constexpr static FileIdentifier file_identifier = 1996387; + std::vector exclusions; + ReplyPromise reply; + + BlobManagerExclusionSafetyCheckRequest() {} + explicit BlobManagerExclusionSafetyCheckRequest(std::vector exclusions) + : exclusions(exclusions) {} + + template + void serialize(Ar& ar) { + serializer(ar, exclusions, reply); + } +}; + #endif diff --git a/fdbserver/CommitProxyServer.actor.cpp b/fdbserver/CommitProxyServer.actor.cpp index eb826a0aa6..eab677e307 100644 --- a/fdbserver/CommitProxyServer.actor.cpp +++ b/fdbserver/CommitProxyServer.actor.cpp @@ -1802,11 +1802,21 @@ ACTOR Future proxyCheckSafeExclusion(Reference cons return Void(); } try { - state Future> safeFuture = + state Future> ddSafeFuture = db->get().distributor.get().distributorExclCheckReq.tryGetReply( DistributorExclusionSafetyCheckRequest(req.exclusions)); - DistributorExclusionSafetyCheckReply _reply = wait(throwErrorOr(safeFuture)); + DistributorExclusionSafetyCheckReply _reply = wait(throwErrorOr(ddSafeFuture)); reply.safe = _reply.safe; + if (db->get().blobManager.present()) { + TraceEvent("SafetyCheckCommitProxyBM").detail("BMID", db->get().blobManager.get().id()); + state Future> bmSafeFuture = + db->get().blobManager.get().blobManagerExclCheckReq.tryGetReply( + BlobManagerExclusionSafetyCheckRequest(req.exclusions)); + BlobManagerExclusionSafetyCheckReply _reply = wait(throwErrorOr(bmSafeFuture)); + reply.safe &= _reply.safe; + } else { + TraceEvent("SafetyCheckCommitProxyNoBM"); + } } catch (Error& e) { TraceEvent("SafetyCheckCommitProxyResponseError").error(e); if (e.code() != error_code_operation_cancelled) { diff --git a/tests/slow/BlobGranuleVerifyBalance.toml b/tests/slow/BlobGranuleVerifyBalance.toml index ee93253c3b..385b88ff69 100644 --- a/tests/slow/BlobGranuleVerifyBalance.toml +++ b/tests/slow/BlobGranuleVerifyBalance.toml @@ -29,10 +29,6 @@ testTitle = 'BlobGranuleVerifyBalance' testDuration = 120.0 meanDelay = 10.0 - [[test.workload]] - testName = 'Attrition' - testDuration = 120.0 - [[test.workload]] testName = 'Attrition' machinesToKill = 10 From ed9f95b2996677df5b30937d6694c23816d888f1 Mon Sep 17 00:00:00 2001 From: Josh Slocum Date: Fri, 25 Feb 2022 13:34:45 -0600 Subject: [PATCH 285/413] simplifying blob manager recovery by always treating the DB granule mapping as the source of truth, and persisting all boundary changes there BEFORE updating workers --- fdbclient/SystemData.cpp | 44 ------ fdbclient/SystemData.h | 10 -- fdbserver/BlobManager.actor.cpp | 237 +++++++------------------------- fdbserver/BlobWorker.actor.cpp | 1 - 4 files changed, 52 insertions(+), 240 deletions(-) diff --git a/fdbclient/SystemData.cpp b/fdbclient/SystemData.cpp index eb892dff03..fa5db048e1 100644 --- a/fdbclient/SystemData.cpp +++ b/fdbclient/SystemData.cpp @@ -1125,7 +1125,6 @@ const KeyRangeRef blobGranuleFileKeys(LiteralStringRef("\xff\x02/bgf/"), Literal const KeyRangeRef blobGranuleMappingKeys(LiteralStringRef("\xff\x02/bgm/"), LiteralStringRef("\xff\x02/bgm0")); const KeyRangeRef blobGranuleLockKeys(LiteralStringRef("\xff\x02/bgl/"), LiteralStringRef("\xff\x02/bgl0")); const KeyRangeRef blobGranuleSplitKeys(LiteralStringRef("\xff\x02/bgs/"), LiteralStringRef("\xff\x02/bgs0")); -const KeyRangeRef blobGranuleSplitBoundaryKeys(LiteralStringRef("\xff\x02/bgsb/"), LiteralStringRef("\xff\x02/bgsb0")); const KeyRangeRef blobGranuleHistoryKeys(LiteralStringRef("\xff\x02/bgh/"), LiteralStringRef("\xff\x02/bgh0")); const KeyRangeRef blobGranulePruneKeys(LiteralStringRef("\xff\x02/bgp/"), LiteralStringRef("\xff\x02/bgp0")); const KeyRef blobGranulePruneChangeKey = LiteralStringRef("\xff\x02/bgpChange"); @@ -1280,49 +1279,6 @@ std::pair decodeBlobGranuleSplitValue(const Valu return std::pair(st, bigEndian64(v)); } -const Key blobGranuleSplitBoundaryKeyFor(UID const& parentGranuleID, KeyRef const& granuleStart) { - BinaryWriter wr(AssumeVersion(ProtocolVersion::withBlobGranule())); - wr.serializeBytes(blobGranuleSplitBoundaryKeys.begin); - wr << parentGranuleID; - wr << granuleStart; - return wr.toValue(); -} - -std::pair decodeBlobGranuleSplitBoundaryKey(KeyRef const& key) { - UID parentGranuleID; - KeyRef granuleStart; - BinaryReader reader(key.removePrefix(blobGranuleSplitBoundaryKeys.begin), - AssumeVersion(ProtocolVersion::withBlobGranule())); - - reader >> parentGranuleID; - reader >> granuleStart; - return std::pair(parentGranuleID, granuleStart); -} - -const KeyRange blobGranuleSplitBoundaryKeyRangeFor(UID const& parentGranuleID) { - BinaryWriter wr(AssumeVersion(ProtocolVersion::withBlobGranule())); - wr.serializeBytes(blobGranuleSplitBoundaryKeys.begin); - wr << parentGranuleID; - - Key startKey = wr.toValue(); - return KeyRangeRef(startKey, strinc(startKey)); -} - -const Key blobGranuleSplitBoundaryValueFor(int64_t epoch, int64_t seqno) { - BinaryWriter wr(IncludeVersion(ProtocolVersion::withBlobGranule())); - wr << epoch; - wr << seqno; - return wr.toValue(); -} - -std::pair decodeBlobGranuleSplitBoundaryValue(ValueRef const& value) { - int64_t epoch, seqno; - BinaryReader reader(value, IncludeVersion()); - reader >> epoch; - reader >> seqno; - return std::pair(epoch, seqno); -} - const Key blobGranuleHistoryKeyFor(KeyRangeRef const& range, Version version) { BinaryWriter wr(AssumeVersion(ProtocolVersion::withBlobGranule())); wr.serializeBytes(blobGranuleHistoryKeys.begin); diff --git a/fdbclient/SystemData.h b/fdbclient/SystemData.h index c76c2225a6..121bd5c95d 100644 --- a/fdbclient/SystemData.h +++ b/fdbclient/SystemData.h @@ -552,9 +552,6 @@ extern const KeyRangeRef blobGranuleLockKeys; // \xff\x02/bgs/(parentGranuleUID, granuleUID) = [[BlobGranuleSplitState]] extern const KeyRangeRef blobGranuleSplitKeys; -// \xff\x02/bgsb/(parentGranuleID, granuleStartKey) = [] -extern const KeyRangeRef blobGranuleSplitBoundaryKeys; - // \xff\x02/bgh/(beginKey,endKey,startVersion) = { granuleUID, [parentGranuleHistoryKeys] } extern const KeyRangeRef blobGranuleHistoryKeys; @@ -588,13 +585,6 @@ const KeyRange blobGranuleSplitKeyRangeFor(UID const& parentGranuleID); const Value blobGranuleSplitValueFor(BlobGranuleSplitState st); std::pair decodeBlobGranuleSplitValue(ValueRef const& value); -const Key blobGranuleSplitBoundaryKeyFor(UID const& parentGranuleID, KeyRef const& granuleStart); -std::pair decodeBlobGranuleSplitBoundaryKey(KeyRef const& key); -const KeyRange blobGranuleSplitBoundaryKeyRangeFor(UID const& parentGranuleID); - -const Key blobGranuleSplitBoundaryValueFor(int64_t epoch, int64_t seqno); -std::pair decodeBlobGranuleSplitBoundaryValue(ValueRef const& value); - const Key blobGranuleHistoryKeyFor(KeyRangeRef const& range, Version version); std::pair decodeBlobGranuleHistoryKey(KeyRef const& key); const KeyRange blobGranuleHistoryKeyRangeFor(KeyRangeRef const& range); diff --git a/fdbserver/BlobManager.actor.cpp b/fdbserver/BlobManager.actor.cpp index b71947d721..34dbb3427d 100644 --- a/fdbserver/BlobManager.actor.cpp +++ b/fdbserver/BlobManager.actor.cpp @@ -18,6 +18,7 @@ * limitations under the License. */ +#include #include #include #include @@ -899,9 +900,6 @@ ACTOR Future maybeSplitRange(Reference bmData, newGranuleIDs.push_back(deterministicRandom()->randomUniqueID()); } - state int64_t splitSeqno = bmData->seqNo; - bmData->seqNo++; - // Need to split range. Persist intent to split and split metadata to DB BEFORE sending split assignments to blob // workers, so that nothing is lost on blob manager recovery loop { @@ -967,17 +965,15 @@ ACTOR Future maybeSplitRange(Reference bmData, granuleRange.begin.printable(), granuleRange.end.printable());*/ - // first key in split boundaries is special: key that doesn't occur normally to the (epoch, seqno) of split - tr->set(blobGranuleSplitBoundaryKeyFor(granuleID, splitBoundarySpecialKey), - blobGranuleSplitBoundaryValueFor(bmData->epoch, splitSeqno)); - for (int i = 0; i < newRanges.size() - 1; i++) { + // set up splits in granule mapping, but point each part to the old owner (until they get reassigned) + state int i; + for (i = 0; i < newRanges.size() - 1; i++) { /*fmt::print(" {0} [{1} - {2})\n", newGranuleIDs[i].toString().substr(0, 6), newRanges[i].printable(), newRanges[i + 1].printable());*/ Key splitKey = blobGranuleSplitKeyFor(granuleID, newGranuleIDs[i]); - tr->set(blobGranuleSplitBoundaryKeyFor(granuleID, newRanges[i]), Value()); tr->atomicOp(splitKey, blobGranuleSplitValueFor(BlobGranuleSplitState::Initialized), @@ -996,8 +992,15 @@ ACTOR Future maybeSplitRange(Reference bmData, granuleStartVersion, latestVersion);*/ tr->set(historyKey, blobGranuleHistoryValueFor(historyValue)); + + // split the assignment but still pointing to the same worker + // FIXME: could pick new random workers here, they'll get overridden shortly unless the BM immediately + // restarts + wait(krmSetRange(tr, + blobGranuleMappingKeys.begin, + KeyRangeRef(newRanges[i], newRanges[i + 1]), + blobGranuleMappingValueFor(currentWorkerId))); } - tr->set(blobGranuleSplitBoundaryKeyFor(granuleID, newRanges.back()), Value()); wait(tr->commit()); break; @@ -1389,7 +1392,6 @@ ACTOR Future checkBlobWorkerList(Reference bmData, Promis throw e; } } - // Shared code for handling KeyRangeMap that is used several places in blob manager recovery // when there can be conflicting sources of what assignments exist or which workers owns a granule. // Resolves these conflicts by comparing the epoch + seqno for the range @@ -1401,53 +1403,56 @@ static void addAssignment(KeyRangeMap>& map, UID newId, int64_t newEpoch, int64_t newSeqno, - std::vector>* outOfDate = nullptr) { + std::vector>& outOfDate) { std::vector>> newer; auto intersecting = map.intersectingRanges(newRange); - bool allNewer = true; + bool allExistingNewer = true; + bool anyConflicts = false; for (auto& old : intersecting) { UID oldWorker = std::get<0>(old.value()); int64_t oldEpoch = std::get<1>(old.value()); int64_t oldSeqno = std::get<2>(old.value()); if (oldEpoch > newEpoch || (oldEpoch == newEpoch && oldSeqno > newSeqno)) { - if (newId != oldWorker && newId != UID() && newEpoch == 0 && newSeqno == 1 && - old.begin() == newRange.begin && old.end() == newRange.end) { - // granule mapping disagrees with worker with highest value. Just do an explicit reassign to a random - // worker for now to ensure the conflict is resolved. - newer.push_back(std::pair(old.range(), std::tuple(UID(), oldEpoch, oldSeqno))); - allNewer = false; - } else { - newer.push_back(std::pair(old.range(), std::tuple(oldWorker, oldEpoch, oldSeqno))); - } + newer.push_back(std::pair(old.range(), std::tuple(oldWorker, oldEpoch, oldSeqno))); } else { - allNewer = false; + allExistingNewer = false; if (newId != UID()) { // different workers can't have same epoch and seqno for granule assignment ASSERT(oldEpoch != newEpoch || oldSeqno != newSeqno); } - if (outOfDate != nullptr && oldWorker != UID() && - (oldEpoch < newEpoch || (oldEpoch == newEpoch && oldSeqno < newSeqno))) { - outOfDate->push_back(std::pair(oldWorker, old.range())); + if (newEpoch == std::numeric_limits::max() && (oldWorker != newId || old.range() != newRange)) { + // new one is from DB (source of truth on boundaries) and existing mapping disagrees on boundary or + // assignment, do explicit revoke and re-assign to converge + anyConflicts = true; + if (outOfDate.empty() || outOfDate.back() != std::pair(oldWorker, KeyRange(old.range()))) { + outOfDate.push_back(std::pair(oldWorker, old.range())); + } + } else if (oldWorker != UID() && (oldEpoch < newEpoch || (oldEpoch == newEpoch && oldSeqno < newSeqno))) { + // 2 blob workers reported conflicting mappings, add old one to out of date (if not already added by a + // previous intersecting range in the split case) + if (outOfDate.empty() || outOfDate.back() != std::pair(oldWorker, KeyRange(old.range()))) { + outOfDate.push_back(std::pair(oldWorker, old.range())); + } } } } - if (!allNewer) { + if (!allExistingNewer) { // if this range supercedes an old range insert it over that - map.insert(newRange, std::tuple(newId, newEpoch, newSeqno)); + map.insert(newRange, std::tuple(anyConflicts ? UID() : newId, newEpoch, newSeqno)); // then, if there were any ranges superceded by this one, insert them over this one if (newer.size()) { - if (outOfDate != nullptr && newId != UID()) { - outOfDate->push_back(std::pair(newId, newRange)); + if (newId != UID()) { + outOfDate.push_back(std::pair(newId, newRange)); } for (auto& it : newer) { map.insert(it.first, it.second); } } } else { - if (outOfDate != nullptr && newId != UID()) { - outOfDate->push_back(std::pair(newId, newRange)); + if (newId != UID()) { + outOfDate.push_back(std::pair(newId, newRange)); } } } @@ -1475,13 +1480,8 @@ ACTOR Future recoverBlobManager(Reference bmData) { // At this point, bmData->workersById is a list of all alive blob workers, but could also include some dead BWs. // The algorithm below works as follows: - // 1. We get the ongoing split boundaries to construct the set of granules we should have. For these splits, we - // simply assign the range to the next best worker if it is not present in the assignment mapping. This is not - // any worse than what the old blob manager would have done. Details: Note that this means that if a worker we - // intended to give a splitted range to dies before the new BM recovers, then we'll simply assign the range to - // the next best worker. // - // 2. We get the existing granule mappings. We do this by asking all active blob workers for their current granule + // 1. We get the existing granule mappings. We do this by asking all active blob workers for their current granule // assignments. This guarantees a consistent snapshot of the state of that worker's assignments: Any request it // recieved and processed from the old manager before the granule assignment request will be included in the // assignments, and any request it recieves from the old manager afterwards will be rejected with @@ -1489,7 +1489,7 @@ ACTOR Future recoverBlobManager(Reference bmData) { // of ongoing splits to this mapping, and any ranges that are not already assigned to existing blob workers will // be reassigned. // - // 3. For every range in our granuleAssignments, we send an assign request to the stream of requests, + // 2. For every range in our granuleAssignments, we send an assign request to the stream of requests, // ultimately giving every range back to some worker (trying to mimic the state of the old BM). // If the worker already had the range, this is a no-op. If the worker didn't have it, it will // begin persisting it. The worker that had the same range before will now be at a lower seqno. @@ -1503,130 +1503,9 @@ ACTOR Future recoverBlobManager(Reference bmData) { if (BM_DEBUG) { fmt::print("BM {0} recovering:\n", bmData->epoch); - fmt::print("BM {0} found in progress splits:\n", bmData->epoch); } - // TODO use range stream instead - - state UID currentParentID = UID(); - state Optional nextParentID; - state std::vector splitBoundaries; - state std::pair - splitEpochSeqno; // used to order splits since we can have multiple splits of the same range in progress at once - - state Key boundaryBeginKey = blobGranuleSplitBoundaryKeys.begin; - state RangeResult boundaryResult; - boundaryResult.readThrough = boundaryBeginKey; - boundaryResult.more = true; - state int boundaryResultIdx = 0; - - // Step 2. Get the latest known split and merge state. Because we can have multiple splits in progress at the same - // time, and we don't know which parts of those are reflected in the current set of worker assignments we read, we - // have to construct the current desired set of granules from the set of ongoing splits and merges. Then, if any of - // those are not represented in the worker mapping, we must add them. - state KeyRangeMap> inProgressSplits; - inProgressSplits.insert(normalKeys, std::tuple(UID(), 0, 0)); - - tr->reset(); - tr->setOption(FDBTransactionOptions::ACCESS_SYSTEM_KEYS); - tr->setOption(FDBTransactionOptions::PRIORITY_SYSTEM_IMMEDIATE); - - loop { - // Advance boundary reader - loop { - if (boundaryResultIdx >= boundaryResult.size()) { - if (!boundaryResult.more) { - break; - } - ASSERT(boundaryResult.readThrough.present() || boundaryResult.size() > 0); - boundaryBeginKey = boundaryResult.readThrough.present() ? boundaryResult.readThrough.get() - : keyAfter(boundaryResult.back().key); - loop { - try { - RangeResult r = wait( - tr->getRange(KeyRangeRef(boundaryBeginKey, blobGranuleSplitBoundaryKeys.end), rowLimit)); - ASSERT(r.size() > 0 || !r.more); - boundaryResult = r; - boundaryResultIdx = 0; - break; - } catch (Error& e) { - if (BM_DEBUG) { - fmt::print("BM {0} got error advancing boundary cursor: {1}\n", bmData->epoch, e.name()); - } - wait(tr->onError(e)); - tr->setOption(FDBTransactionOptions::ACCESS_SYSTEM_KEYS); - tr->setOption(FDBTransactionOptions::PRIORITY_SYSTEM_IMMEDIATE); - } - } - // if we got a response and there are zero rows, we are done - if (boundaryResult.empty()) { - break; - } - } - bool foundNext = false; - while (boundaryResultIdx < boundaryResult.size()) { - UID parentGranuleID; - Key boundaryKey; - - std::tie(parentGranuleID, boundaryKey) = - decodeBlobGranuleSplitBoundaryKey(boundaryResult[boundaryResultIdx].key); - if (parentGranuleID != currentParentID) { - // nextParentID should have already been set by split reader - nextParentID = parentGranuleID; - foundNext = true; - break; - } - - if (splitBoundarySpecialKey == boundaryKey) { - ASSERT(splitEpochSeqno.first == 0 && splitEpochSeqno.second == 0); - ASSERT(boundaryResult[boundaryResultIdx].value.size() > 0); - splitEpochSeqno = decodeBlobGranuleSplitBoundaryValue(boundaryResult[boundaryResultIdx].value); - ASSERT(splitEpochSeqno.first != 0 && splitEpochSeqno.second != 0); - } else { - ASSERT(boundaryResult[boundaryResultIdx].value.size() == 0); - splitBoundaries.push_back(boundaryKey); - } - - boundaryResultIdx++; - } - if (foundNext) { - break; - } - } - - // process this split - if (currentParentID != UID()) { - std::sort(splitBoundaries.begin(), splitBoundaries.end()); - - if (BM_DEBUG) { - fmt::print(" [{0} - {1}) {2} @ ({3}, {4}):\n", - splitBoundaries.front().printable(), - splitBoundaries.back().printable(), - currentParentID.toString().substr(0, 6), - splitEpochSeqno.first, - splitEpochSeqno.second); - } - for (int i = 0; i < splitBoundaries.size() - 1; i++) { - // if this split boundary has not been opened by a blob worker yet, or was not in the assignment list - // when we previously read it, we must ensure it gets assigned to one - KeyRange range = KeyRange(KeyRangeRef(splitBoundaries[i], splitBoundaries[i + 1])); - if (BM_DEBUG) { - fmt::print(" [{0} - {1})\n", range.begin.printable(), range.end.printable()); - } - addAssignment(inProgressSplits, range, UID(), splitEpochSeqno.first, splitEpochSeqno.second); - } - } - splitBoundaries.clear(); - splitEpochSeqno = std::pair(0, 0); - - if (!nextParentID.present()) { - break; - } - currentParentID = nextParentID.get(); - nextParentID.reset(); - } - - // Step 3. Get the latest known mapping of granules to blob workers (i.e. assignments) + // Step 1. Get the latest known mapping of granules to blob workers (i.e. assignments) // This must happen causally AFTER reading the split boundaries, since the blob workers can clear the split // boundaries for a granule as part of persisting their assignment. @@ -1672,7 +1551,7 @@ ACTOR Future recoverBlobManager(Reference bmData) { workerId, assignment.epochAssigned, assignment.seqnoAssigned, - &outOfDateAssignments); + outOfDateAssignments); } if (bmData->workerStats.count(workerId)) { bmData->workerStats[workerId].numGranulesAssigned = reply.get().assignments.size(); @@ -1693,12 +1572,11 @@ ACTOR Future recoverBlobManager(Reference bmData) { fmt::print("BM {0} found old assignments:\n", bmData->epoch); } - // then, read any gaps in worker assignment from FDB - // With a small number of blob workers, if even one is missing, doing numGranules/numWorkers small range reads from - // FDB is probably less efficient than just reading the whole mapping anyway - // Plus, we don't have a consistent snapshot of the mapping ACROSS blob workers, so we need the DB to reconcile any + // DB is the source of truth, so read from here, and resolve any conflicts with current worker mapping + // We don't have a consistent snapshot of the mapping ACROSS blob workers, so we need the DB to reconcile any // differences (eg blob manager revoked from worker A, assigned to B, the revoke from A was processed but the assign - // to B wasn't, meaning in the snapshot nobody owns the granule) + // to B wasn't, meaning in the snapshot nobody owns the granule). This also handles races with a BM persisting a + // boundary change, then dying before notifying the workers state Key beginKey = blobGranuleMappingKeys.begin; loop { try { @@ -1722,7 +1600,13 @@ ACTOR Future recoverBlobManager(Reference bmData) { if (results[rangeIdx].value.size()) { // note: if the old owner is dead, we handle this in rangeAssigner UID existingOwner = decodeBlobGranuleMappingValue(results[rangeIdx].value); - addAssignment(workerAssignments, KeyRangeRef(granuleStartKey, granuleEndKey), existingOwner, 0, 1); + // use (max int64_t, 0) to be higher than anything that existing workers have + addAssignment(workerAssignments, + KeyRangeRef(granuleStartKey, granuleEndKey), + existingOwner, + std::numeric_limits::max(), + 0, + outOfDateAssignments); bmData->knownBlobRanges.insert(KeyRangeRef(granuleStartKey, granuleEndKey), true); if (BM_DEBUG) { @@ -1752,24 +1636,7 @@ ACTOR Future recoverBlobManager(Reference bmData) { } } - if (BM_DEBUG) { - printf("Splits overriding the following ranges:\n"); - } - // Apply current granule boundaries to the assignment map. If they don't exactly match what is currently in the map, - // override and assign it to a new worker - auto splits = inProgressSplits.intersectingRanges(normalKeys); - for (auto& it : splits) { - int64_t epoch = std::get<1>(it.value()); - int64_t seqno = std::get<2>(it.value()); - if (epoch == 0 || seqno == 0) { - // no in-progress splits for this range - continue; - } - - addAssignment(workerAssignments, it.range(), UID(), epoch, seqno, &outOfDateAssignments); - } - - // Step 4. Send assign requests for all the granules and transfer assignments + // Step 2. Send assign requests for all the granules and transfer assignments // from local workerAssignments to bmData // before we take ownership of all of the ranges, check the manager lock again tr->reset(); diff --git a/fdbserver/BlobWorker.actor.cpp b/fdbserver/BlobWorker.actor.cpp index 71c6592e56..73460df49c 100644 --- a/fdbserver/BlobWorker.actor.cpp +++ b/fdbserver/BlobWorker.actor.cpp @@ -392,7 +392,6 @@ ACTOR Future updateGranuleSplitState(Transaction* tr, // tr->clear(singleKeyRange(oldGranuleLockKey)); tr->clear(currentRange); - tr->clear(blobGranuleSplitBoundaryKeyRangeFor(parentGranuleID)); } else { tr->atomicOp(myStateKey, blobGranuleSplitValueFor(newState), MutationRef::SetVersionstampedValue); if (newState == BlobGranuleSplitState::Assigned && currentState == BlobGranuleSplitState::Initialized && From 921be1dedf349e592557a3fe6d053db52a561941 Mon Sep 17 00:00:00 2001 From: Josh Slocum Date: Mon, 28 Feb 2022 08:44:12 -0600 Subject: [PATCH 286/413] fixed revoking when same worker has granule --- fdbserver/BlobManager.actor.cpp | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/fdbserver/BlobManager.actor.cpp b/fdbserver/BlobManager.actor.cpp index 34dbb3427d..fdeb5cde41 100644 --- a/fdbserver/BlobManager.actor.cpp +++ b/fdbserver/BlobManager.actor.cpp @@ -1427,7 +1427,8 @@ static void addAssignment(KeyRangeMap>& map, if (outOfDate.empty() || outOfDate.back() != std::pair(oldWorker, KeyRange(old.range()))) { outOfDate.push_back(std::pair(oldWorker, old.range())); } - } else if (oldWorker != UID() && (oldEpoch < newEpoch || (oldEpoch == newEpoch && oldSeqno < newSeqno))) { + } else if (oldWorker != UID() && oldWorker != newId && + (oldEpoch < newEpoch || (oldEpoch == newEpoch && oldSeqno < newSeqno))) { // 2 blob workers reported conflicting mappings, add old one to out of date (if not already added by a // previous intersecting range in the split case) if (outOfDate.empty() || outOfDate.back() != std::pair(oldWorker, KeyRange(old.range()))) { From a55658204146f5bcd84f1bc0930b70ed34a53e08 Mon Sep 17 00:00:00 2001 From: Josh Slocum Date: Mon, 28 Feb 2022 10:36:28 -0600 Subject: [PATCH 287/413] Fixes for new BM reecovery --- fdbserver/BlobManager.actor.cpp | 11 +++++++---- 1 file changed, 7 insertions(+), 4 deletions(-) diff --git a/fdbserver/BlobManager.actor.cpp b/fdbserver/BlobManager.actor.cpp index fdeb5cde41..b51556f9d6 100644 --- a/fdbserver/BlobManager.actor.cpp +++ b/fdbserver/BlobManager.actor.cpp @@ -1416,7 +1416,7 @@ static void addAssignment(KeyRangeMap>& map, newer.push_back(std::pair(old.range(), std::tuple(oldWorker, oldEpoch, oldSeqno))); } else { allExistingNewer = false; - if (newId != UID()) { + if (newId != UID() && newEpoch != std::numeric_limits::max()) { // different workers can't have same epoch and seqno for granule assignment ASSERT(oldEpoch != newEpoch || oldSeqno != newSeqno); } @@ -1424,6 +1424,11 @@ static void addAssignment(KeyRangeMap>& map, // new one is from DB (source of truth on boundaries) and existing mapping disagrees on boundary or // assignment, do explicit revoke and re-assign to converge anyConflicts = true; + // if ranges don't match, need to explicitly reaassign all parts of old range, as it could be from a + // yet-unassigned split + if (old.range() != newRange) { + std::get<0>(old.value()) = UID(); + } if (outOfDate.empty() || outOfDate.back() != std::pair(oldWorker, KeyRange(old.range()))) { outOfDate.push_back(std::pair(oldWorker, old.range())); } @@ -1699,11 +1704,9 @@ ACTOR Future recoverBlobManager(Reference bmData) { bmData->workerAssignments.insert(range.range(), workerId); if (BM_DEBUG) { - fmt::print(" [{0} - {1}) @ ({2}, {3}): {4}\n", + fmt::print(" [{0} - {1}): {2}\n", range.begin().printable(), range.end().printable(), - epoch, - seqno, workerId == UID() || epoch == 0 ? " (?)" : workerId.toString().substr(0, 5).c_str()); } From f277662419ea918d1bb0379859b30389f7cb723a Mon Sep 17 00:00:00 2001 From: Josh Slocum Date: Mon, 28 Feb 2022 10:50:28 -0600 Subject: [PATCH 288/413] Fixing some change feed pop durability inconsistencies --- fdbserver/storageserver.actor.cpp | 32 ++++++++++++++++++++++++++----- 1 file changed, 27 insertions(+), 5 deletions(-) diff --git a/fdbserver/storageserver.actor.cpp b/fdbserver/storageserver.actor.cpp index d2c49e9883..726f923efe 100644 --- a/fdbserver/storageserver.actor.cpp +++ b/fdbserver/storageserver.actor.cpp @@ -4585,6 +4585,12 @@ ACTOR Future fetchChangeFeedApplier(StorageServer* data, changeFeedSSValue(changeFeedInfo->range, changeFeedInfo->emptyVersion + 1, changeFeedInfo->stopVersion))); + data->addMutationToMutationLog( + mLV, + MutationRef(MutationRef::ClearRange, + changeFeedDurableKey(changeFeedInfo->id, 0), + changeFeedDurableKey(changeFeedInfo->id, feedResults->popVersion))); + ++data->counters.kvSystemClearRanges; } Version localVersion = localResult.version; @@ -4692,6 +4698,11 @@ ACTOR Future fetchChangeFeedApplier(StorageServer* data, persistChangeFeedKeys.begin.toString() + changeFeedInfo->id.toString(), changeFeedSSValue( changeFeedInfo->range, changeFeedInfo->emptyVersion + 1, changeFeedInfo->stopVersion))); + data->addMutationToMutationLog(mLV, + MutationRef(MutationRef::ClearRange, + changeFeedDurableKey(changeFeedInfo->id, 0), + changeFeedDurableKey(changeFeedInfo->id, feedResults->popVersion))); + ++data->counters.kvSystemClearRanges; } // if we were popped or removed while fetching but it didn't pass the fetch version while writing, clean up here @@ -4700,9 +4711,11 @@ ACTOR Future fetchChangeFeedApplier(StorageServer* data, ASSERT(lastVersion != invalidVersion); Version endClear = std::min(lastVersion + 1, changeFeedInfo->emptyVersion); if (endClear > firstVersion) { - - data->storage.clearRange(KeyRangeRef(changeFeedDurableKey(changeFeedInfo->id, firstVersion), - changeFeedDurableKey(changeFeedInfo->id, endClear))); + auto& mLV2 = data->addVersionToMutationLog(data->data().getLatestVersion()); + data->addMutationToMutationLog(mLV2, + MutationRef(MutationRef::ClearRange, + changeFeedDurableKey(changeFeedInfo->id, firstVersion), + changeFeedDurableKey(changeFeedInfo->id, endClear))); ++data->counters.kvSystemClearRanges; } } @@ -5912,6 +5925,7 @@ private: data->keyChangeFeed.coalesce(changeFeedRange.contents()); } + bool popMutationLog = false; bool addMutationToLog = false; if (popVersion != invalidVersion && status != ChangeFeedStatus::CHANGE_FEED_DESTROY) { // pop the change feed at pop version, no matter what state it is in @@ -5922,8 +5936,9 @@ private: } if (feed->second->storageVersion != invalidVersion) { ++data->counters.kvSystemClearRanges; - data->storage.clearRange(KeyRangeRef(changeFeedDurableKey(feed->second->id, 0), - changeFeedDurableKey(feed->second->id, popVersion))); + // do this clear in the mutation log, as we want it to be committed consistently with the + // popVersion update + popMutationLog = true; if (popVersion > feed->second->storageVersion) { feed->second->storageVersion = invalidVersion; feed->second->durableVersion = invalidVersion; @@ -5982,6 +5997,13 @@ private: persistChangeFeedKeys.begin.toString() + changeFeedId.toString(), changeFeedSSValue( feed->second->range, feed->second->emptyVersion + 1, feed->second->stopVersion))); + if (popMutationLog) { + ++data->counters.kvSystemClearRanges; + data->addMutationToMutationLog(mLV, + MutationRef(MutationRef::ClearRange, + changeFeedDurableKey(feed->second->id, 0), + changeFeedDurableKey(feed->second->id, popVersion))); + } } } else if (m.param1.substr(1).startsWith(tssMappingKeys.begin) && (m.type == MutationRef::SetValue || m.type == MutationRef::ClearRange)) { From 642aab2b8be85938b07eaff09a335c93828286c5 Mon Sep 17 00:00:00 2001 From: Josh Slocum Date: Mon, 28 Feb 2022 16:11:59 -0600 Subject: [PATCH 289/413] blob manager range assignment race fixes --- fdbserver/BlobManager.actor.cpp | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/fdbserver/BlobManager.actor.cpp b/fdbserver/BlobManager.actor.cpp index b51556f9d6..8d53d5051d 100644 --- a/fdbserver/BlobManager.actor.cpp +++ b/fdbserver/BlobManager.actor.cpp @@ -570,6 +570,11 @@ ACTOR Future rangeAssigner(Reference bmData) { if (bmData->workerStats.count(assignment.worker.get())) { bmData->workerStats[assignment.worker.get()].numGranulesAssigned -= 1; } + // if this revoke matches the worker assignment state, mark the range as unassigned + auto existingRange = bmData->workerAssignments.rangeContaining(assignment.keyRange.begin); + if (existingRange.range() == assignment.keyRange && existingRange.cvalue() == assignment.worker.get()) { + bmData->workerAssignments.insert(assignment.keyRange, UID()); + } bmData->addActor.send(doRangeAssignment(bmData, assignment, assignment.worker.get(), seqNo)); } else { auto currentAssignments = bmData->workerAssignments.intersectingRanges(assignment.keyRange); From 839edbf5ea5eb5cc72a3efc01d5d6982b0071902 Mon Sep 17 00:00:00 2001 From: Josh Slocum Date: Mon, 28 Feb 2022 16:49:50 -0600 Subject: [PATCH 290/413] moved granule split evaluation to not be specific to any individual blob worker, to simplify it and handle additional cases --- fdbserver/BlobManager.actor.cpp | 84 +++++++++++++++++++++++++-------- 1 file changed, 64 insertions(+), 20 deletions(-) diff --git a/fdbserver/BlobManager.actor.cpp b/fdbserver/BlobManager.actor.cpp index 8d53d5051d..d27f94c595 100644 --- a/fdbserver/BlobManager.actor.cpp +++ b/fdbserver/BlobManager.actor.cpp @@ -203,6 +203,17 @@ struct BlobWorkerStats { BlobWorkerStats(int numGranulesAssigned = 0) : numGranulesAssigned(numGranulesAssigned) {} }; +struct SplitEvaluation { + int64_t epoch; + int64_t seqno; + Version version; + Future inProgress; + + SplitEvaluation() : epoch(0), seqno(0), version(invalidVersion) {} + SplitEvaluation(int64_t epoch, int64_t seqno, int64_t version, Future inProgress) + : epoch(epoch), seqno(seqno), version(version), inProgress(inProgress) {} +}; + struct BlobManagerData : NonCopyable, ReferenceCounted { UID id; Database db; @@ -218,6 +229,7 @@ struct BlobManagerData : NonCopyable, ReferenceCounted { std::unordered_set deadWorkers; KeyRangeMap workerAssignments; KeyRangeActorMap assignsInProgress; + KeyRangeMap splitEvaluations; KeyRangeMap knownBlobRanges; AsyncTrigger startRecruiting; @@ -505,6 +517,16 @@ ACTOR Future rangeAssigner(Reference bmData) { state int64_t seqNo = bmData->seqNo; bmData->seqNo++; + if (BM_DEBUG) { + fmt::print("DBGRA: BM {0} {1} range [{2} - {3}) @ ({4}, {5})\n", + bmData->epoch, + assignment.isAssign ? "assign" : "revoke", + assignment.keyRange.begin.printable(), + assignment.keyRange.end.printable(), + bmData->epoch, + seqNo); + } + // modify the in-memory assignment data structures, and send request off to worker state UID workerId; if (assignment.isAssign) { @@ -1173,7 +1195,6 @@ ACTOR Future killBlobWorker(Reference bmData, BlobWorkerI } ACTOR Future monitorBlobWorkerStatus(Reference bmData, BlobWorkerInterface bwInterf) { - state KeyRangeMap> lastSeenSeqno; // outer loop handles reconstructing stream if it got a retryable error // do backoff, we can get a lot of retries in a row @@ -1235,10 +1256,10 @@ ACTOR Future monitorBlobWorkerStatus(Reference bmData, Bl continue; } - auto lastReqForGranule = lastSeenSeqno.rangeContaining(rep.granuleRange.begin); - if (rep.granuleRange.begin == lastReqForGranule.begin() && - rep.granuleRange.end == lastReqForGranule.end() && rep.epoch == lastReqForGranule.value().first && - rep.seqno == lastReqForGranule.value().second) { + auto lastSplitEval = bmData->splitEvaluations.rangeContaining(rep.granuleRange.begin); + if (rep.granuleRange.begin == lastSplitEval.begin() && rep.granuleRange.end == lastSplitEval.end() && + rep.epoch == lastSplitEval.cvalue().epoch && rep.seqno == lastSplitEval.cvalue().seqno) { + ASSERT(lastSplitEval.cvalue().version == rep.latestVersion); if (BM_DEBUG) { fmt::print("Manager {0} received repeat status for the same granule [{1} - {2}), ignoring.\n", bmData->epoch, @@ -1246,22 +1267,45 @@ ACTOR Future monitorBlobWorkerStatus(Reference bmData, Bl rep.granuleRange.end.printable()); } } else { - if (BM_DEBUG) { - fmt::print("Manager {0} evaluating [{1} - {2}) @ ({3}, {4}) for split\n", - bmData->epoch, - rep.granuleRange.begin.printable().c_str(), - rep.granuleRange.end.printable().c_str(), - rep.epoch, - rep.seqno); + ASSERT(lastSplitEval.cvalue().epoch < rep.epoch || + (lastSplitEval.cvalue().epoch == rep.epoch && lastSplitEval.cvalue().seqno < rep.seqno)); + if (lastSplitEval.cvalue().inProgress.isValid() && !lastSplitEval.cvalue().inProgress.isReady()) { + TEST(true); // racing BM splits + // For example, one worker asked BM to split, then died, granule was moved, new worker asks to + // split on recovery. We need to ensure that they are semantically the same split (same range + + // version). We will just rely on the in-progress split to finish + ASSERT(lastSplitEval.cvalue().version == rep.latestVersion); + ASSERT(lastSplitEval.range() == rep.granuleRange); + if (BM_DEBUG) { + fmt::print("Manager {0} got split request for [{1} - {2}) @ ({3}, {4}), but already in " + "progress from ({5}, {6})\n", + bmData->epoch, + rep.granuleRange.begin.printable().c_str(), + rep.granuleRange.end.printable().c_str(), + rep.epoch, + rep.seqno, + lastSplitEval.cvalue().epoch, + lastSplitEval.cvalue().seqno); + } + } else { + if (BM_DEBUG) { + fmt::print("Manager {0} evaluating [{1} - {2}) @ ({3}, {4}) for split\n", + bmData->epoch, + rep.granuleRange.begin.printable().c_str(), + rep.granuleRange.end.printable().c_str(), + rep.epoch, + rep.seqno); + } + Future doSplitEval = maybeSplitRange(bmData, + bwInterf.id(), + rep.granuleRange, + rep.granuleID, + rep.startVersion, + rep.latestVersion, + rep.writeHotSplit); + bmData->splitEvaluations.insert( + rep.granuleRange, SplitEvaluation(rep.epoch, rep.seqno, rep.latestVersion, doSplitEval)); } - lastSeenSeqno.insert(rep.granuleRange, std::pair(rep.epoch, rep.seqno)); - bmData->addActor.send(maybeSplitRange(bmData, - bwInterf.id(), - rep.granuleRange, - rep.granuleID, - rep.startVersion, - rep.latestVersion, - rep.writeHotSplit)); } } } catch (Error& e) { From c27204d401474bd4ecf7b02a5134509286e8a7e8 Mon Sep 17 00:00:00 2001 From: Josh Slocum Date: Tue, 1 Mar 2022 07:18:43 -0600 Subject: [PATCH 291/413] more BM split race fixes --- fdbserver/BlobManager.actor.cpp | 8 +++----- fdbserver/BlobWorker.actor.cpp | 6 ++++-- 2 files changed, 7 insertions(+), 7 deletions(-) diff --git a/fdbserver/BlobManager.actor.cpp b/fdbserver/BlobManager.actor.cpp index d27f94c595..6382e2d8ea 100644 --- a/fdbserver/BlobManager.actor.cpp +++ b/fdbserver/BlobManager.actor.cpp @@ -1049,10 +1049,9 @@ ACTOR Future maybeSplitRange(Reference bmData, } // transaction committed, send range assignments - // revoke from current worker + // range could have been moved since split eval started, so just revoke from whoever has it RangeAssignment raRevoke; raRevoke.isAssign = false; - raRevoke.worker = currentWorkerId; raRevoke.keyRange = granuleRange; raRevoke.revoke = RangeRevokeData(false); // not a dispose bmData->rangesToAssign.send(raRevoke); @@ -1272,9 +1271,8 @@ ACTOR Future monitorBlobWorkerStatus(Reference bmData, Bl if (lastSplitEval.cvalue().inProgress.isValid() && !lastSplitEval.cvalue().inProgress.isReady()) { TEST(true); // racing BM splits // For example, one worker asked BM to split, then died, granule was moved, new worker asks to - // split on recovery. We need to ensure that they are semantically the same split (same range + - // version). We will just rely on the in-progress split to finish - ASSERT(lastSplitEval.cvalue().version == rep.latestVersion); + // split on recovery. We need to ensure that they are semantically the same split. + // We will just rely on the in-progress split to finish ASSERT(lastSplitEval.range() == rep.granuleRange); if (BM_DEBUG) { fmt::print("Manager {0} got split request for [{1} - {2}) @ ({3}, {4}), but already in " diff --git a/fdbserver/BlobWorker.actor.cpp b/fdbserver/BlobWorker.actor.cpp index 73460df49c..2e092fd33b 100644 --- a/fdbserver/BlobWorker.actor.cpp +++ b/fdbserver/BlobWorker.actor.cpp @@ -1310,6 +1310,7 @@ ACTOR Future blobGranuleUpdateFiles(Reference bwData, } } } + metadata->files = startState.existingFiles.get(); snapshotEligible = true; } @@ -1770,10 +1771,11 @@ ACTOR Future blobGranuleUpdateFiles(Reference bwData, if (snapshotEligible && metadata->bytesInNewDeltaFiles >= SERVER_KNOBS->BG_DELTA_BYTES_BEFORE_COMPACT && metadata->pendingDeltaVersion >= startState.changeFeedStartVersion) { if (BW_DEBUG && !inFlightFiles.empty()) { - fmt::print("Granule [{0} - {1}) ready to re-snapshot after {2} > {3} bytes, waiting for " - "outstanding {4} files to finish\n", + fmt::print("Granule [{0} - {1}) ready to re-snapshot at {2} after {3} > {4} bytes, waiting for " + "outstanding {5} files to finish\n", metadata->keyRange.begin.printable(), metadata->keyRange.end.printable(), + metadata->pendingDeltaVersion, metadata->bytesInNewDeltaFiles, SERVER_KNOBS->BG_DELTA_BYTES_BEFORE_COMPACT, inFlightFiles.size()); From 3090fb174cefd7ee385a9777321fb8c716c5bbf1 Mon Sep 17 00:00:00 2001 From: Josh Slocum Date: Tue, 1 Mar 2022 08:52:29 -0600 Subject: [PATCH 292/413] Tweaks to granule initial snapshot --- fdbserver/BlobWorker.actor.cpp | 37 +++++++++++++++++++++++++++------- 1 file changed, 30 insertions(+), 7 deletions(-) diff --git a/fdbserver/BlobWorker.actor.cpp b/fdbserver/BlobWorker.actor.cpp index 2e092fd33b..63096b3d4d 100644 --- a/fdbserver/BlobWorker.actor.cpp +++ b/fdbserver/BlobWorker.actor.cpp @@ -671,19 +671,26 @@ ACTOR Future writeSnapshot(Reference bwData, ACTOR Future dumpInitialSnapshotFromFDB(Reference bwData, Reference metadata, - UID granuleID) { + UID granuleID, + Key cfKey) { if (BW_DEBUG) { fmt::print("Dumping snapshot from FDB for [{0} - {1})\n", metadata->keyRange.begin.printable(), metadata->keyRange.end.printable()); } state Reference tr = makeReference(bwData->db); + state int64_t bytesRead = 0; + state int retries = 0; + state Version lastReadVersion = invalidVersion; + state Version readVersion = invalidVersion; loop { state Key beginKey = metadata->keyRange.begin; tr->setOption(FDBTransactionOptions::ACCESS_SYSTEM_KEYS); try { - state Version readVersion = wait(tr->getReadVersion()); + Version rv = wait(tr->getReadVersion()); + readVersion = rv; + ASSERT(lastReadVersion <= readVersion); state PromiseStream rowsStream; state Future snapshotWriter = writeSnapshot(bwData, metadata->keyRange, @@ -698,9 +705,10 @@ ACTOR Future dumpInitialSnapshotFromFDB(Reference // TODO: use streaming range read // TODO: inject read error // TODO knob for limit? - int lim = BUGGIFY ? 2 : 1000; + int lim = BUGGIFY && retries < 5 ? 2 : 10000; RangeResult res = wait(tr->getRange(KeyRangeRef(beginKey, metadata->keyRange.end), lim)); bwData->stats.bytesReadFromFDBForInitialSnapshot += res.size(); + bytesRead += res.expectedSize(); rowsStream.send(res); if (res.more) { beginKey = keyAfter(res.back().key); @@ -709,27 +717,42 @@ ACTOR Future dumpInitialSnapshotFromFDB(Reference break; } } - BlobFileIndex f = wait(snapshotWriter); + state BlobFileIndex f = wait(snapshotWriter); TraceEvent("BlobGranuleSnapshotFile", bwData->id) .detail("Granule", metadata->keyRange) .detail("Version", readVersion); DEBUG_KEY_RANGE("BlobWorkerFDBSnapshot", readVersion, metadata->keyRange, bwData->id); + + // initial snapshot is committed in fdb, we can pop the change feed up to this version + bwData->addActor.send(bwData->db->popChangeFeedMutations(cfKey, readVersion)); return f; } catch (Error& e) { if (e.code() == error_code_operation_cancelled) { throw e; } if (BW_DEBUG) { - fmt::print("Dumping snapshot from FDB for [{0} - {1}) got error {2}\n", + fmt::print("Dumping snapshot {0} from FDB for [{1} - {2}) got error {3} after {4} bytes\n", + retries + 1, metadata->keyRange.begin.printable(), metadata->keyRange.end.printable(), - e.name()); + e.name(), + bytesRead); } state Error err = e; wait(tr->onError(e)); + retries++; TraceEvent(SevWarn, "BlobGranuleInitialSnapshotRetry", bwData->id) .detail("Granule", metadata->keyRange) + .detail("Count", retries) .error(err); + bytesRead = 0; + lastReadVersion = readVersion; + // Pop change feed up to readVersion, because that data will be before the next snapshot + // Do this to prevent a large amount of CF data from accumulating if we have consecutive failures to + // snapshot + // Also somewhat servers as a rate limiting function and checking that the database is available for this + // key range + wait(bwData->db->popChangeFeedMutations(cfKey, readVersion)); } } } @@ -1332,7 +1355,7 @@ ACTOR Future blobGranuleUpdateFiles(Reference bwData, metadata->durableSnapshotVersion.set(startState.blobFilesToSnapshot.get().snapshotFiles.back().version); } else { ASSERT(startState.previousDurableVersion == invalidVersion); - BlobFileIndex fromFDB = wait(dumpInitialSnapshotFromFDB(bwData, metadata, startState.granuleID)); + BlobFileIndex fromFDB = wait(dumpInitialSnapshotFromFDB(bwData, metadata, startState.granuleID, cfKey)); newSnapshotFile = fromFDB; ASSERT(startState.changeFeedStartVersion <= fromFDB.version); startVersion = newSnapshotFile.version; From cebe367037d5f9fd553929423cad6db87c63ff5f Mon Sep 17 00:00:00 2001 From: Josh Slocum Date: Tue, 1 Mar 2022 08:59:58 -0600 Subject: [PATCH 293/413] Shortening BGVerifyLarge to match duration of other slow/BG* tests --- tests/slow/BlobGranuleVerifyLarge.toml | 16 ++++++++-------- tests/slow/BlobGranuleVerifyLargeClean.toml | 8 ++++---- 2 files changed, 12 insertions(+), 12 deletions(-) diff --git a/tests/slow/BlobGranuleVerifyLarge.toml b/tests/slow/BlobGranuleVerifyLarge.toml index ad180c55b3..de55422d89 100644 --- a/tests/slow/BlobGranuleVerifyLarge.toml +++ b/tests/slow/BlobGranuleVerifyLarge.toml @@ -6,10 +6,10 @@ testTitle = 'BlobGranuleVerifyLarge' [[test.workload]] testName = 'ReadWrite' - testDuration = 200.0 + testDuration = 120.0 transactionsPerSecond = 200 - writesPerTransactionA = 0 - readsPerTransactionA = 10 + writesPerTransactionA = 5 + readsPerTransactionA = 1 writesPerTransactionB = 10 readsPerTransactionB = 1 alpha = 0.5 @@ -21,28 +21,28 @@ testTitle = 'BlobGranuleVerifyLarge' [[test.workload]] testName = 'BlobGranuleVerifier' - testDuration = 200.0 + testDuration = 120.0 [[test.workload]] testName = 'RandomClogging' - testDuration = 200.0 + testDuration = 120.0 [[test.workload]] testName = 'Rollback' meanDelay = 30.0 - testDuration = 200.0 + testDuration = 120.0 [[test.workload]] testName = 'Attrition' machinesToKill = 10 machinesToLeave = 3 reboot = true - testDuration = 200.0 + testDuration = 120.0 [[test.workload]] testName = 'Attrition' machinesToKill = 10 machinesToLeave = 3 reboot = true - testDuration = 200.0 + testDuration = 120.0 diff --git a/tests/slow/BlobGranuleVerifyLargeClean.toml b/tests/slow/BlobGranuleVerifyLargeClean.toml index 353c73bb23..782935a68b 100644 --- a/tests/slow/BlobGranuleVerifyLargeClean.toml +++ b/tests/slow/BlobGranuleVerifyLargeClean.toml @@ -6,10 +6,10 @@ testTitle = 'BlobGranuleVerifyLargeClean' [[test.workload]] testName = 'ReadWrite' - testDuration = 200.0 + testDuration = 120.0 transactionsPerSecond = 200 - writesPerTransactionA = 0 - readsPerTransactionA = 10 + writesPerTransactionA = 5 + readsPerTransactionA = 1 writesPerTransactionB = 10 readsPerTransactionB = 1 alpha = 0.5 @@ -21,4 +21,4 @@ testTitle = 'BlobGranuleVerifyLargeClean' [[test.workload]] testName = 'BlobGranuleVerifier' - testDuration = 200.0 + testDuration = 120.0 From ce176e846edb4dfec9dc67d44fe5e263bd334a96 Mon Sep 17 00:00:00 2001 From: Josh Slocum Date: Tue, 1 Mar 2022 11:20:44 -0600 Subject: [PATCH 294/413] Switching initial granule snapshot to use range stream --- fdbserver/BlobWorker.actor.cpp | 21 +++------------------ 1 file changed, 3 insertions(+), 18 deletions(-) diff --git a/fdbserver/BlobWorker.actor.cpp b/fdbserver/BlobWorker.actor.cpp index 63096b3d4d..a76d471e5f 100644 --- a/fdbserver/BlobWorker.actor.cpp +++ b/fdbserver/BlobWorker.actor.cpp @@ -685,7 +685,6 @@ ACTOR Future dumpInitialSnapshotFromFDB(Reference state Version readVersion = invalidVersion; loop { - state Key beginKey = metadata->keyRange.begin; tr->setOption(FDBTransactionOptions::ACCESS_SYSTEM_KEYS); try { Version rv = wait(tr->getReadVersion()); @@ -700,23 +699,9 @@ ACTOR Future dumpInitialSnapshotFromFDB(Reference readVersion, rowsStream, true); - - loop { - // TODO: use streaming range read - // TODO: inject read error - // TODO knob for limit? - int lim = BUGGIFY && retries < 5 ? 2 : 10000; - RangeResult res = wait(tr->getRange(KeyRangeRef(beginKey, metadata->keyRange.end), lim)); - bwData->stats.bytesReadFromFDBForInitialSnapshot += res.size(); - bytesRead += res.expectedSize(); - rowsStream.send(res); - if (res.more) { - beginKey = keyAfter(res.back().key); - } else { - rowsStream.sendError(end_of_stream()); - break; - } - } + Future streamFuture = + tr->getTransaction().getRangeStream(rowsStream, metadata->keyRange, GetRangeLimits(), Snapshot::True); + wait(streamFuture); state BlobFileIndex f = wait(snapshotWriter); TraceEvent("BlobGranuleSnapshotFile", bwData->id) .detail("Granule", metadata->keyRange) From 06f58f848d68e2851184dca15e12a8ccdf4e1688 Mon Sep 17 00:00:00 2001 From: Josh Slocum Date: Tue, 1 Mar 2022 12:05:48 -0600 Subject: [PATCH 295/413] Improving change feed validation, and another popped while reading race fixed --- fdbserver/BlobManager.actor.cpp | 10 --- fdbserver/storageserver.actor.cpp | 104 +++++++++++++++++------------- 2 files changed, 59 insertions(+), 55 deletions(-) diff --git a/fdbserver/BlobManager.actor.cpp b/fdbserver/BlobManager.actor.cpp index 6382e2d8ea..763337f884 100644 --- a/fdbserver/BlobManager.actor.cpp +++ b/fdbserver/BlobManager.actor.cpp @@ -517,16 +517,6 @@ ACTOR Future rangeAssigner(Reference bmData) { state int64_t seqNo = bmData->seqNo; bmData->seqNo++; - if (BM_DEBUG) { - fmt::print("DBGRA: BM {0} {1} range [{2} - {3}) @ ({4}, {5})\n", - bmData->epoch, - assignment.isAssign ? "assign" : "revoke", - assignment.keyRange.begin.printable(), - assignment.keyRange.end.printable(), - bmData->epoch, - seqNo); - } - // modify the in-memory assignment data structures, and send request off to worker state UID workerId; if (assignment.isAssign) { diff --git a/fdbserver/storageserver.actor.cpp b/fdbserver/storageserver.actor.cpp index 726f923efe..e52d9a0e74 100644 --- a/fdbserver/storageserver.actor.cpp +++ b/fdbserver/storageserver.actor.cpp @@ -2032,57 +2032,52 @@ ACTOR Future> getChangeFeedMutations(Stor Standalone> mutations; std::tie(id, version) = decodeChangeFeedDurableKey(kv.key); std::tie(mutations, knownCommittedVersion) = decodeChangeFeedDurableValue(kv.value); + + // gap validation + while (memoryVerifyIdx < memoryReply.mutations.size() && + version > memoryReply.mutations[memoryVerifyIdx].version) { + // There is a case where this can happen - if we wait on a fetching change feed, and the feed is + // popped while we wait, we could have copied the memory mutations into memoryReply before the + // pop, but they may or may not have been skipped writing to disk + if (waitFetched && feedInfo->emptyVersion > emptyVersion && + memoryReply.mutations[memoryVerifyIdx].version <= feedInfo->emptyVersion) { + memoryVerifyIdx++; + continue; + } else { + printf("ERROR: SS %s CF %s SQ %s has mutation at %lld in memory but not on disk (next disk is " + "%lld) " + "(emptyVersion=%lld, emptyBefore=%lld)!\n", + data->thisServerID.toString().substr(0, 4).c_str(), + req.rangeID.printable().substr(0, 6).c_str(), + streamUID.toString().substr(0, 8).c_str(), + memoryReply.mutations[memoryVerifyIdx].version, + version, + feedInfo->emptyVersion, + emptyVersion); + + printf(" Memory: (%d)\n", memoryReply.mutations[memoryVerifyIdx].mutations.size()); + for (auto& it : memoryReply.mutations[memoryVerifyIdx].mutations) { + if (it.type == MutationRef::SetValue) { + printf(" %s=\n", it.param1.printable().c_str()); + } else { + printf(" %s - %s\n", it.param1.printable().c_str(), it.param2.printable().c_str()); + } + } + ASSERT(false); + } + } + auto m = filterMutations( reply.arena, MutationsAndVersionRef(mutations, version, knownCommittedVersion), req.range, inverted); if (m.mutations.size()) { reply.arena.dependsOn(mutations.arena()); reply.mutations.push_back(reply.arena, m); - // if there is overlap between the memory and disk mutations, we can do relatively cheap validation that - // they are the same. In particular this validates the consistency of change feed data recieved from the - // tlog mutations vs change feed data fetched from another storage server - if (memoryVerifyIdx < memoryReply.mutations.size()) { - while (memoryVerifyIdx < memoryReply.mutations.size() && - version > memoryReply.mutations[memoryVerifyIdx].version) { - // there is a case where this can happen - if we wait on a fetching change feed, and the feed is - // popped while we wait, we could have copied the memory mutations into memoryReply before the - // pop, but they may or may not have been skipped writing to disk - if (waitFetched && feedInfo->emptyVersion > emptyVersion && - memoryReply.mutations[memoryVerifyIdx].version <= feedInfo->emptyVersion) { - // ok - memoryVerifyIdx++; - continue; - } else { - printf( - "ERROR: SS %s CF %s SQ %s has mutation at %lld in memory but not on disk (next disk is " - "%lld) " - "(emptyVersion=%lld, emptyBefore=%lld)!\n", - data->thisServerID.toString().substr(0, 4).c_str(), - req.rangeID.printable().substr(0, 6).c_str(), - streamUID.toString().substr(0, 8).c_str(), - memoryReply.mutations[memoryVerifyIdx].version, - version, - feedInfo->emptyVersion, - emptyVersion); - - printf(" Memory: (%d)\n", memoryReply.mutations[memoryVerifyIdx].mutations.size()); - for (auto& it : memoryReply.mutations[memoryVerifyIdx].mutations) { - if (it.type == MutationRef::SetValue) { - printf(" %s=\n", it.param1.printable().c_str()); - } else { - printf( - " %s - %s\n", it.param1.printable().c_str(), it.param2.printable().c_str()); - } - } - ASSERT(false); - } - } - if (memoryVerifyIdx < memoryReply.mutations.size() && - version == memoryReply.mutations[memoryVerifyIdx].version) { - // TODO: we could do some validation here too, but it's complicated because clears can get split - // and stuff - memoryVerifyIdx++; - } + if (memoryVerifyIdx < memoryReply.mutations.size() && + version == memoryReply.mutations[memoryVerifyIdx].version) { + // TODO: we could do some validation here too, but it's complicated because clears can get split + // and stuff + memoryVerifyIdx++; } } else if (memoryVerifyIdx < memoryReply.mutations.size() && version == memoryReply.mutations[memoryVerifyIdx].version) { @@ -2127,6 +2122,17 @@ ACTOR Future> getChangeFeedMutations(Stor --totalCount; } reply.mutations.append(reply.arena, it, totalCount); + // If still empty, that means disk results were filtered out, but skipped all memory results. Add an empty, + // either the last version from disk + if (reply.mutations.empty() && res.size()) { + if (DEBUG_SS_CFM(data->thisServerID, req.rangeID, streamUID, req.begin)) { + printf("CFM: SS %s CF %s: adding empty from disk and memory %lld\n", + data->thisServerID.toString().substr(0, 4).c_str(), + req.rangeID.printable().substr(0, 6).c_str(), + lastVersion); + } + reply.mutations.push_back(reply.arena, MutationsAndVersionRef(lastVersion, lastKnownCommitted)); + } } else if (reply.mutations.empty() || reply.mutations.back().version < lastVersion) { if (DEBUG_SS_CFM(data->thisServerID, req.rangeID, streamUID, req.begin)) { printf("CFM: SS %s CF %s: adding empty from disk %lld\n", @@ -2403,6 +2409,14 @@ ACTOR Future changeFeedStreamQ(StorageServer* data, ChangeFeedStreamReques ChangeFeedStreamReply feedReply = _feedReply.first; bool gotAll = _feedReply.second; + // TODO REMOVE debugging + if (feedReply.mutations.size() == 0) { + printf("CFM: SS %s CF %s: CFSQ %s empty results for begin=%lld\n", + data->thisServerID.toString().substr(0, 4).c_str(), + req.rangeID.printable().substr(0, 6).c_str(), + streamUID.toString().substr(0, 8).c_str(), + req.begin); + } ASSERT(feedReply.mutations.size() > 0); req.begin = feedReply.mutations.back().version + 1; if (!atLatest && gotAll) { From 1407a23618decc4b8c2ac1e1c2b09e39977758a1 Mon Sep 17 00:00:00 2001 From: Josh Slocum Date: Tue, 1 Mar 2022 12:07:30 -0600 Subject: [PATCH 296/413] Removing clean versions of basic BG tests --- tests/CMakeLists.txt | 2 -- tests/fast/BlobGranuleVerifyAtomicOpsClean.toml | 14 -------------- tests/fast/BlobGranuleVerifyCycleClean.toml | 15 --------------- 3 files changed, 31 deletions(-) delete mode 100644 tests/fast/BlobGranuleVerifyAtomicOpsClean.toml delete mode 100644 tests/fast/BlobGranuleVerifyCycleClean.toml diff --git a/tests/CMakeLists.txt b/tests/CMakeLists.txt index 1d1a8de3db..db728fd1ff 100644 --- a/tests/CMakeLists.txt +++ b/tests/CMakeLists.txt @@ -132,9 +132,7 @@ if(WITH_PYTHON) add_fdb_test(TEST_FILES fast/BlobGranuleVerifySmall.toml) add_fdb_test(TEST_FILES fast/BlobGranuleVerifySmallClean.toml) add_fdb_test(TEST_FILES fast/BlobGranuleVerifyAtomicOps.toml) - add_fdb_test(TEST_FILES fast/BlobGranuleVerifyAtomicOpsClean.toml) add_fdb_test(TEST_FILES fast/BlobGranuleVerifyCycle.toml) - add_fdb_test(TEST_FILES fast/BlobGranuleVerifyCycleClean.toml) add_fdb_test(TEST_FILES fast/CacheTest.toml) add_fdb_test(TEST_FILES fast/CloggedSideband.toml) add_fdb_test(TEST_FILES fast/ConfigureLocked.toml) diff --git a/tests/fast/BlobGranuleVerifyAtomicOpsClean.toml b/tests/fast/BlobGranuleVerifyAtomicOpsClean.toml deleted file mode 100644 index 89ecf6d0f8..0000000000 --- a/tests/fast/BlobGranuleVerifyAtomicOpsClean.toml +++ /dev/null @@ -1,14 +0,0 @@ -[configuration] -blobGranulesEnabled = true - -[[test]] -testTitle = 'BlobGranuleVerifyAtomicOpsClean' - - [[test.workload]] - testName = 'AtomicOps' - transactionsPerSecond = 2500.0 - testDuration = 30.0 - - [[test.workload]] - testName = 'BlobGranuleVerifier' - testDuration = 30.0 \ No newline at end of file diff --git a/tests/fast/BlobGranuleVerifyCycleClean.toml b/tests/fast/BlobGranuleVerifyCycleClean.toml deleted file mode 100644 index abe1d498aa..0000000000 --- a/tests/fast/BlobGranuleVerifyCycleClean.toml +++ /dev/null @@ -1,15 +0,0 @@ -[configuration] -blobGranulesEnabled = true - -[[test]] -testTitle = 'BlobGranuleVerifyCycleClean' - - [[test.workload]] - testName = 'Cycle' - transactionsPerSecond = 250.0 - testDuration = 60.0 - expectedRate = 0 - - [[test.workload]] - testName = 'BlobGranuleVerifier' - testDuration = 60.0 From 6f1fb719494f17b353e9a12d9651c9fca0bce5db Mon Sep 17 00:00:00 2001 From: Evan Tschannen Date: Tue, 1 Mar 2022 10:13:20 -0800 Subject: [PATCH 297/413] Fixed a number of different pruning problems --- fdbclient/BlobGranuleCommon.h | 4 +- fdbserver/BlobManager.actor.cpp | 2 +- fdbserver/BlobWorker.actor.cpp | 5 +- .../workloads/BlobGranuleVerifier.actor.cpp | 86 +++++++++++++------ 4 files changed, 63 insertions(+), 34 deletions(-) diff --git a/fdbclient/BlobGranuleCommon.h b/fdbclient/BlobGranuleCommon.h index b33cbabb7b..4559f75819 100644 --- a/fdbclient/BlobGranuleCommon.h +++ b/fdbclient/BlobGranuleCommon.h @@ -77,14 +77,14 @@ struct BlobGranuleChunkRef { constexpr static FileIdentifier file_identifier = 865198; KeyRangeRef keyRange; Version includedVersion; - Version startVersion; + Version snapshotVersion; Optional snapshotFile; // not set if it's an incremental read VectorRef deltaFiles; GranuleDeltas newDeltas; template void serialize(Ar& ar) { - serializer(ar, keyRange, includedVersion, startVersion, snapshotFile, deltaFiles, newDeltas); + serializer(ar, keyRange, includedVersion, snapshotVersion, snapshotFile, deltaFiles, newDeltas); } }; diff --git a/fdbserver/BlobManager.actor.cpp b/fdbserver/BlobManager.actor.cpp index a55118ec73..0ad0aaf53e 100644 --- a/fdbserver/BlobManager.actor.cpp +++ b/fdbserver/BlobManager.actor.cpp @@ -2450,7 +2450,7 @@ ACTOR Future monitorPruneKeys(Reference self) { pruneMap.insert(allKeys, std::make_pair(0, false)); try { // TODO: replace 10000 with a knob - state RangeResult pruneIntents = wait(tr->getRange(blobGranulePruneKeys, 10000)); + state RangeResult pruneIntents = wait(tr->getRange(blobGranulePruneKeys, BUGGIFY ? 1 : 10000)); if (pruneIntents.size()) { int rangeIdx = 0; for (; rangeIdx < pruneIntents.size(); ++rangeIdx) { diff --git a/fdbserver/BlobWorker.actor.cpp b/fdbserver/BlobWorker.actor.cpp index c51e1c5bbd..a005bca190 100644 --- a/fdbserver/BlobWorker.actor.cpp +++ b/fdbserver/BlobWorker.actor.cpp @@ -2256,7 +2256,6 @@ ACTOR Future doBlobGranuleFileRequest(Reference bwData, Bl state KeyRange chunkRange; state GranuleFiles chunkFiles; - state Version startVer; if (metadata->initialSnapshotVersion > req.readVersion) { // this is a time travel query, find previous granule @@ -2317,7 +2316,6 @@ ACTOR Future doBlobGranuleFileRequest(Reference bwData, Bl ASSERT(cur->endVersion > req.readVersion); ASSERT(cur->startVersion <= req.readVersion); - startVer = cur->startVersion; // lazily load files for old granule if not present chunkRange = cur->range; @@ -2352,7 +2350,6 @@ ACTOR Future doBlobGranuleFileRequest(Reference bwData, Bl ASSERT(chunkFiles.snapshotFiles.front().version <= req.readVersion); } else { // this is an active granule query - startVer = metadata->initialSnapshotVersion; loop { if (!metadata->activeCFData.get().isValid() || !metadata->cancelled.canBeSet()) { throw wrong_shard_server(); @@ -2408,7 +2405,6 @@ ACTOR Future doBlobGranuleFileRequest(Reference bwData, Bl BlobGranuleChunkRef chunk; // TODO change in V2 chunk.includedVersion = req.readVersion; - chunk.startVersion = startVer; chunk.keyRange = KeyRangeRef(StringRef(rep.arena, chunkRange.begin), StringRef(rep.arena, chunkRange.end)); // handle snapshot files @@ -2434,6 +2430,7 @@ ACTOR Future doBlobGranuleFileRequest(Reference bwData, Bl BlobFileIndex snapshotF = chunkFiles.snapshotFiles[i]; chunk.snapshotFile = BlobFilePointerRef(rep.arena, snapshotF.filename, snapshotF.offset, snapshotF.length); Version snapshotVersion = chunkFiles.snapshotFiles[i].version; + chunk.snapshotVersion = snapshotVersion; // handle delta files // cast this to an int so i going to -1 still compares properly diff --git a/fdbserver/workloads/BlobGranuleVerifier.actor.cpp b/fdbserver/workloads/BlobGranuleVerifier.actor.cpp index aaef5a468f..bbd58cd8cc 100644 --- a/fdbserver/workloads/BlobGranuleVerifier.actor.cpp +++ b/fdbserver/workloads/BlobGranuleVerifier.actor.cpp @@ -327,28 +327,28 @@ struct BlobGranuleVerifierWorkload : TestWorkload { // utility to prune at pruneVersion= with the flag ACTOR Future pruneAtVersion(Database cx, KeyRange range, Version version, bool force) { state Reference tr = makeReference(cx); + state Version commitVersion = 0; + state Key pruneKey; loop { try { tr->setOption(FDBTransactionOptions::ACCESS_SYSTEM_KEYS); tr->setOption(FDBTransactionOptions::PRIORITY_SYSTEM_IMMEDIATE); Value pruneValue = blobGranulePruneValueFor(version, range, force); - - Key pruneKey = KeyRef(blobGranulePruneKeys.begin.withSuffix(std::string(14, '\x00'))); - int32_t pos = pruneKey.size() - 14; - pos = littleEndian32(pos); - uint8_t* data = mutateString(pruneKey); - memcpy(data + pruneKey.size() - sizeof(int32_t), &pos, sizeof(int32_t)); - tr->atomicOp(pruneKey, pruneValue, MutationRef::SetVersionstampedKey); + tr->atomicOp( + addVersionStampAtEnd(blobGranulePruneKeys.begin), pruneValue, MutationRef::SetVersionstampedKey); tr->set(blobGranulePruneChangeKey, deterministicRandom()->randomUniqueID().toString()); + state Future> fTrVs = tr->getVersionstamp(); wait(tr->commit()); + Standalone vs = wait(fTrVs); + pruneKey = blobGranulePruneKeys.begin.withSuffix(vs); if (BGV_DEBUG) { printf("pruneAtVersion for range [%s-%s) at version %lld succeeded\n", range.begin.printable().c_str(), range.end.printable().c_str(), version); } - return Void(); + break; } catch (Error& e) { if (BGV_DEBUG) { printf("pruneAtVersion for range [%s-%s) at version %lld encountered error %s\n", @@ -360,6 +360,23 @@ struct BlobGranuleVerifierWorkload : TestWorkload { wait(tr->onError(e)); } } + tr->reset(); + loop { + try { + tr->setOption(FDBTransactionOptions::ACCESS_SYSTEM_KEYS); + tr->setOption(FDBTransactionOptions::PRIORITY_SYSTEM_IMMEDIATE); + + Optional pruneVal = wait(tr->get(pruneKey)); + if (!pruneVal.present()) { + return Void(); + } + state Future watchFuture = tr->watch(pruneKey); + wait(tr->commit()); + wait(watchFuture); + } catch (Error& e) { + wait(tr->onError(e)); + } + } } ACTOR Future killBlobWorkers(Database cx, BlobGranuleVerifierWorkload* self) { @@ -403,11 +420,13 @@ struct BlobGranuleVerifierWorkload : TestWorkload { } } - ACTOR Future verifyGranules(Database cx, BlobGranuleVerifierWorkload* self) { + ACTOR Future verifyGranules(Database cx, BlobGranuleVerifierWorkload* self, bool allowPruning) { state double last = now(); state double endTime = last + self->testDuration; state std::map timeTravelChecks; state int64_t timeTravelChecksMemory = 0; + state Version prevPruneVersion = -1; + state UID dbgId = debugRandom()->randomUniqueID(); TraceEvent("BlobGranuleVerifierStart"); if (BGV_DEBUG) { @@ -429,33 +448,44 @@ struct BlobGranuleVerifierWorkload : TestWorkload { state OldRead oldRead = timeTravelIt->second; timeTravelChecksMemory -= oldRead.oldResult.expectedSize(); timeTravelIt = timeTravelChecks.erase(timeTravelIt); + if (prevPruneVersion == -1) { + prevPruneVersion = oldRead.v; + } // advance iterator before doing read, so if it gets error we don't retry it try { - state Version newPruneVersion = deterministicRandom()->randomInt64(1, oldRead.v); - dbgPruneVersion = std::max(dbgPruneVersion, newPruneVersion); - wait(self->pruneAtVersion(cx, oldRead.range, newPruneVersion, false)); + state Version newPruneVersion = 0; + state bool doPruning = + allowPruning && prevPruneVersion < oldRead.v && deterministicRandom()->random01() < 0.5; + if (doPruning) { + newPruneVersion = deterministicRandom()->randomInt64(prevPruneVersion, oldRead.v); + prevPruneVersion = std::max(prevPruneVersion, newPruneVersion); + dbgPruneVersion = prevPruneVersion; + wait(self->pruneAtVersion(cx, oldRead.range, newPruneVersion, false)); + } std::pair>> reReadResult = wait(self->readFromBlob(cx, self, oldRead.range, oldRead.v)); self->compareResult(oldRead.oldResult, reReadResult, oldRead.range, oldRead.v, false); self->timeTravelReads++; - wait(self->killBlobWorkers(cx, self)); - try { + if (doPruning) { + wait(self->killBlobWorkers(cx, self)); std::pair>> versionRead = - wait(self->readFromBlob(cx, self, oldRead.range, dbgPruneVersion)); - Version minStartVer = newPruneVersion; - for (auto& it : versionRead.second) { - minStartVer = std::min(minStartVer, it.startVersion); + wait(self->readFromBlob(cx, self, oldRead.range, prevPruneVersion)); + try { + Version minSnapshotVersion = newPruneVersion; + for (auto& it : versionRead.second) { + minSnapshotVersion = std::min(minSnapshotVersion, it.snapshotVersion); + } + std::pair>> versionRead = + wait(self->readFromBlob(cx, self, oldRead.range, minSnapshotVersion - 1)); + ASSERT(false); + } catch (Error& e) { + if (e.code() == error_code_actor_cancelled) { + throw; + } + ASSERT(e.code() == error_code_blob_granule_transaction_too_old); } - std::pair>> versionRead = - wait(self->readFromBlob(cx, self, oldRead.range, minStartVer - 1)); - ASSERT(false); - } catch (Error& e) { - if (e.code() == error_code_actor_cancelled) { - throw; - } - ASSERT(e.code() == error_code_blob_granule_transaction_too_old); } // TODO: read at some version older than pruneVersion and make sure you get txn_too_old @@ -508,7 +538,9 @@ struct BlobGranuleVerifierWorkload : TestWorkload { clients.push_back(timeout(findGranules(cx, this), testDuration, Void())); for (int i = 0; i < threads; i++) { clients.push_back( - timeout(reportErrors(verifyGranules(cx, this), "BlobGranuleVerifier"), testDuration, Void())); + timeout(reportErrors(verifyGranules(cx, this, clientId == 0 && i == 0), "BlobGranuleVerifier"), + testDuration, + Void())); } return delay(testDuration); } From e0578f24d24361cc67a4fae01f23ce4fbffbf0db Mon Sep 17 00:00:00 2001 From: Josh Slocum Date: Tue, 1 Mar 2022 12:45:49 -0600 Subject: [PATCH 298/413] Fixes for range streaming BG snapshot --- fdbserver/BlobManager.actor.cpp | 5 ----- fdbserver/BlobWorker.actor.cpp | 5 ++--- 2 files changed, 2 insertions(+), 8 deletions(-) diff --git a/fdbserver/BlobManager.actor.cpp b/fdbserver/BlobManager.actor.cpp index 0ad0aaf53e..21d2143d06 100644 --- a/fdbserver/BlobManager.actor.cpp +++ b/fdbserver/BlobManager.actor.cpp @@ -45,11 +45,6 @@ #define BM_DEBUG true -// DO NOT CHANGE THIS -// Special key where the value means the epoch + sequence number of the split, instead of the actual boundary -// Chosen because this should not be a start or end key in any split -static Key splitBoundarySpecialKey = "\xff\xff\xff"_sr; - // TODO add comments + documentation void handleClientBlobRange(KeyRangeMap* knownBlobRanges, Arena& ar, diff --git a/fdbserver/BlobWorker.actor.cpp b/fdbserver/BlobWorker.actor.cpp index a005bca190..c6370cecdc 100644 --- a/fdbserver/BlobWorker.actor.cpp +++ b/fdbserver/BlobWorker.actor.cpp @@ -701,8 +701,7 @@ ACTOR Future dumpInitialSnapshotFromFDB(Reference true); Future streamFuture = tr->getTransaction().getRangeStream(rowsStream, metadata->keyRange, GetRangeLimits(), Snapshot::True); - wait(streamFuture); - state BlobFileIndex f = wait(snapshotWriter); + wait(streamFuture && success(snapshotWriter)); TraceEvent("BlobGranuleSnapshotFile", bwData->id) .detail("Granule", metadata->keyRange) .detail("Version", readVersion); @@ -710,7 +709,7 @@ ACTOR Future dumpInitialSnapshotFromFDB(Reference // initial snapshot is committed in fdb, we can pop the change feed up to this version bwData->addActor.send(bwData->db->popChangeFeedMutations(cfKey, readVersion)); - return f; + return snapshotWriter.get(); } catch (Error& e) { if (e.code() == error_code_operation_cancelled) { throw e; From a14f3ff067a42379711dd2174c5dd95bc9a976f9 Mon Sep 17 00:00:00 2001 From: Josh Slocum Date: Tue, 1 Mar 2022 12:52:26 -0600 Subject: [PATCH 299/413] Do some verify tests with no pruning --- fdbserver/workloads/BlobGranuleVerifier.actor.cpp | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/fdbserver/workloads/BlobGranuleVerifier.actor.cpp b/fdbserver/workloads/BlobGranuleVerifier.actor.cpp index bbd58cd8cc..f9b9621ce8 100644 --- a/fdbserver/workloads/BlobGranuleVerifier.actor.cpp +++ b/fdbserver/workloads/BlobGranuleVerifier.actor.cpp @@ -538,7 +538,9 @@ struct BlobGranuleVerifierWorkload : TestWorkload { clients.push_back(timeout(findGranules(cx, this), testDuration, Void())); for (int i = 0; i < threads; i++) { clients.push_back( - timeout(reportErrors(verifyGranules(cx, this, clientId == 0 && i == 0), "BlobGranuleVerifier"), + timeout(reportErrors(verifyGranules( + cx, this, clientId == 0 && i == 0 && deterministicRandom()->random01() < 0.5), + "BlobGranuleVerifier"), testDuration, Void())); } From 2b8882ca0a5451671cd2b6e71847aafceabcdaf9 Mon Sep 17 00:00:00 2001 From: Josh Slocum Date: Wed, 2 Mar 2022 18:46:40 -0600 Subject: [PATCH 300/413] Adding exponential backoff for consecutive change feed stream failures --- fdbclient/NativeAPI.actor.cpp | 13 ++++++++++++- 1 file changed, 12 insertions(+), 1 deletion(-) diff --git a/fdbclient/NativeAPI.actor.cpp b/fdbclient/NativeAPI.actor.cpp index 7033ebc792..3c24304c81 100644 --- a/fdbclient/NativeAPI.actor.cpp +++ b/fdbclient/NativeAPI.actor.cpp @@ -7970,6 +7970,9 @@ ACTOR Future getChangeFeedStreamActor(Reference db, results->id = rangeID; results->endVersion = end; + state double sleepWithBackoff = CLIENT_KNOBS->WRONG_SHARD_SERVER_DELAY; + state Version lastBeginVersion = invalidVersion; + /*printf("CFStream %s [%s - %s): [%lld - %lld]\n", rangeID.printable().substr(0, 6).c_str(), range.begin.printable().c_str(), @@ -7980,6 +7983,7 @@ ACTOR Future getChangeFeedStreamActor(Reference db, loop { state KeyRange keys; try { + lastBeginVersion = begin; KeyRange fullRange = wait(getChangeFeedRange(db, cx, rangeID, begin)); keys = fullRange & range; state std::vector>> locations = @@ -8084,7 +8088,14 @@ ACTOR Future getChangeFeedStreamActor(Reference db, // beginVersion at all before getting an error db->changeFeedCache.erase(rangeID); cx->invalidateCache(keys); - wait(delay(CLIENT_KNOBS->WRONG_SHARD_SERVER_DELAY)); + if (begin == lastBeginVersion) { + // We didn't read anything since the last failure before failing again. + // Do exponential backoff, up to 1 second + sleepWithBackoff = std::min(1.0, sleepWithBackoff * 1.5); + } else { + sleepWithBackoff = CLIENT_KNOBS->WRONG_SHARD_SERVER_DELAY; + } + wait(delay(sleepWithBackoff)); } else { results->mutations.sendError(e); results->refresh.sendError(change_feed_cancelled()); From ea9f935d29f17dfcd5775ed50f87670e3e29c5fd Mon Sep 17 00:00:00 2001 From: Josh Slocum Date: Wed, 2 Mar 2022 18:10:37 -0600 Subject: [PATCH 301/413] Initializing field for valgrind error --- fdbclient/BlobWorkerCommon.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/fdbclient/BlobWorkerCommon.h b/fdbclient/BlobWorkerCommon.h index 5910dfd9f3..b890b57f62 100644 --- a/fdbclient/BlobWorkerCommon.h +++ b/fdbclient/BlobWorkerCommon.h @@ -59,7 +59,7 @@ struct BlobWorkerStats { readReqTotalFilesReturned("ReadReqTotalFilesReturned", cc), readReqDeltaBytesReturned("ReadReqDeltaBytesReturned", cc), commitVersionChecks("CommitVersionChecks", cc), granuleUpdateErrors("GranuleUpdateErrors", cc), granuleRequestTimeouts("GranuleRequestTimeouts", cc), - numRangesAssigned(0), mutationBytesBuffered(0) { + numRangesAssigned(0), mutationBytesBuffered(0), activeReadRequests(0) { specialCounter(cc, "NumRangesAssigned", [this]() { return this->numRangesAssigned; }); specialCounter(cc, "MutationBytesBuffered", [this]() { return this->mutationBytesBuffered; }); specialCounter(cc, "ActiveReadRequests", [this]() { return this->activeReadRequests; }); From 976375466f8a297bad581e9aa6d266c35a97c74b Mon Sep 17 00:00:00 2001 From: Evan Tschannen Date: Wed, 2 Mar 2022 17:03:37 -0800 Subject: [PATCH 302/413] fixed more pruning bugs --- fdbserver/BlobManager.actor.cpp | 11 +++++++++-- fdbserver/BlobWorker.actor.cpp | 7 +++++-- fdbserver/workloads/BlobGranuleVerifier.actor.cpp | 2 +- 3 files changed, 15 insertions(+), 5 deletions(-) diff --git a/fdbserver/BlobManager.actor.cpp b/fdbserver/BlobManager.actor.cpp index 21d2143d06..ecd725d12f 100644 --- a/fdbserver/BlobManager.actor.cpp +++ b/fdbserver/BlobManager.actor.cpp @@ -2277,17 +2277,24 @@ ACTOR Future pruneRange(Reference self, KeyRangeRef range // get the persisted history entry for this granule state Standalone currHistoryNode; state Key historyKey = blobGranuleHistoryKeyFor(currRange, startVersion); + state bool foundHistory = false; loop { try { Optional persistedHistory = wait(tr.get(historyKey)); - ASSERT(persistedHistory.present()); - currHistoryNode = decodeBlobGranuleHistoryValue(persistedHistory.get()); + if (persistedHistory.present()) { + currHistoryNode = decodeBlobGranuleHistoryValue(persistedHistory.get()); + foundHistory = true; + } break; } catch (Error& e) { wait(tr.onError(e)); } } + if (!foundHistory) { + continue; + } + if (BM_DEBUG) { fmt::print("Found history entry for this node. It's granuleID is {0}\n", currHistoryNode.granuleID.toString()); diff --git a/fdbserver/BlobWorker.actor.cpp b/fdbserver/BlobWorker.actor.cpp index c6370cecdc..a84b63c9c6 100644 --- a/fdbserver/BlobWorker.actor.cpp +++ b/fdbserver/BlobWorker.actor.cpp @@ -2338,7 +2338,7 @@ ACTOR Future doBlobGranuleFileRequest(Reference bwData, Bl metadata->initialSnapshotVersion, chunkFiles.snapshotFiles.size()); for (auto& f : chunkFiles.snapshotFiles) { - fmt::print(" {0}}\n", f.version); + fmt::print(" {0}\n", f.version); } fmt::print(" delta files {0}:\n", chunkFiles.deltaFiles.size()); for (auto& f : chunkFiles.deltaFiles) { @@ -2346,7 +2346,10 @@ ACTOR Future doBlobGranuleFileRequest(Reference bwData, Bl } } ASSERT(chunkFiles.deltaFiles.back().version > req.readVersion); - ASSERT(chunkFiles.snapshotFiles.front().version <= req.readVersion); + if (chunkFiles.snapshotFiles.front().version > req.readVersion) { + // a snapshot file must have been pruned + throw blob_granule_transaction_too_old(); + } } else { // this is an active granule query loop { diff --git a/fdbserver/workloads/BlobGranuleVerifier.actor.cpp b/fdbserver/workloads/BlobGranuleVerifier.actor.cpp index f9b9621ce8..81750c2bcd 100644 --- a/fdbserver/workloads/BlobGranuleVerifier.actor.cpp +++ b/fdbserver/workloads/BlobGranuleVerifier.actor.cpp @@ -461,7 +461,7 @@ struct BlobGranuleVerifierWorkload : TestWorkload { newPruneVersion = deterministicRandom()->randomInt64(prevPruneVersion, oldRead.v); prevPruneVersion = std::max(prevPruneVersion, newPruneVersion); dbgPruneVersion = prevPruneVersion; - wait(self->pruneAtVersion(cx, oldRead.range, newPruneVersion, false)); + wait(self->pruneAtVersion(cx, normalKeys, newPruneVersion, false)); } std::pair>> reReadResult = wait(self->readFromBlob(cx, self, oldRead.range, oldRead.v)); From f230ac0e4164a4b40aa84eb4c962c7f8ba1c0806 Mon Sep 17 00:00:00 2001 From: Josh Slocum Date: Wed, 9 Mar 2022 09:13:03 -0600 Subject: [PATCH 303/413] Fixing more merge issues --- fdbserver/BlobManager.actor.cpp | 2 +- fdbserver/BlobWorker.actor.cpp | 4 ++-- fdbserver/workloads/BlobGranuleCorrectnessWorkload.actor.cpp | 2 +- fdbserver/workloads/BlobGranuleVerifier.actor.cpp | 2 +- 4 files changed, 5 insertions(+), 5 deletions(-) diff --git a/fdbserver/BlobManager.actor.cpp b/fdbserver/BlobManager.actor.cpp index fae4c86d89..2fe9f034ca 100644 --- a/fdbserver/BlobManager.actor.cpp +++ b/fdbserver/BlobManager.actor.cpp @@ -1368,7 +1368,7 @@ ACTOR Future monitorBlobWorker(Reference bmData, BlobWork fmt::print("BM got unexpected error {0} monitoring BW {1}\n", e.name(), bwInterf.id().toString()); } // TODO change back from SevError? - TraceEvent(SevError, "BWMonitoringFailed", bmData->id).detail("BlobWorkerID", bwInterf.id()).error(e); + TraceEvent(SevError, "BWMonitoringFailed", bmData->id).error(e).detail("BlobWorkerID", bwInterf.id()); throw e; } } diff --git a/fdbserver/BlobWorker.actor.cpp b/fdbserver/BlobWorker.actor.cpp index e578063174..15d83b6acd 100644 --- a/fdbserver/BlobWorker.actor.cpp +++ b/fdbserver/BlobWorker.actor.cpp @@ -726,9 +726,9 @@ ACTOR Future dumpInitialSnapshotFromFDB(Reference wait(tr->onError(e)); retries++; TraceEvent(SevWarn, "BlobGranuleInitialSnapshotRetry", bwData->id) + .error(err) .detail("Granule", metadata->keyRange) - .detail("Count", retries) - .error(err); + .detail("Count", retries); bytesRead = 0; lastReadVersion = readVersion; // Pop change feed up to readVersion, because that data will be before the next snapshot diff --git a/fdbserver/workloads/BlobGranuleCorrectnessWorkload.actor.cpp b/fdbserver/workloads/BlobGranuleCorrectnessWorkload.actor.cpp index 8ecaeec694..c23b78758e 100644 --- a/fdbserver/workloads/BlobGranuleCorrectnessWorkload.actor.cpp +++ b/fdbserver/workloads/BlobGranuleCorrectnessWorkload.actor.cpp @@ -23,7 +23,7 @@ #include #include -#include "contrib/fmt-8.0.1/include/fmt/format.h" +#include "contrib/fmt-8.1.1/include/fmt/format.h" #include "fdbclient/BlobGranuleReader.actor.h" #include "fdbclient/ManagementAPI.actor.h" #include "fdbclient/NativeAPI.actor.h" diff --git a/fdbserver/workloads/BlobGranuleVerifier.actor.cpp b/fdbserver/workloads/BlobGranuleVerifier.actor.cpp index de77ecb34c..187cb90c9f 100644 --- a/fdbserver/workloads/BlobGranuleVerifier.actor.cpp +++ b/fdbserver/workloads/BlobGranuleVerifier.actor.cpp @@ -186,7 +186,7 @@ struct BlobGranuleVerifierWorkload : TestWorkload { // reset the range and restart the read at a higher version TraceEvent(SevDebug, "BGVFDBReadReset").detail("ReadVersion", v); TEST(true); // BGV transaction reset - printf("Resetting BGV GRV {0} -> {1}\n", v, grv); + fmt::print("Resetting BGV GRV {0} -> {1}\n", v, grv); first = true; out = RangeResult(); currentRange = range; From 4d909abbd1ef6c02bd482ea000fe4c75e96e5309 Mon Sep 17 00:00:00 2001 From: Josh Slocum Date: Wed, 9 Mar 2022 09:21:45 -0600 Subject: [PATCH 304/413] re-enabling all storage engine types --- fdbserver/SimulatedCluster.actor.cpp | 5 ----- 1 file changed, 5 deletions(-) diff --git a/fdbserver/SimulatedCluster.actor.cpp b/fdbserver/SimulatedCluster.actor.cpp index d5fe34cf34..b4b78a8f0d 100644 --- a/fdbserver/SimulatedCluster.actor.cpp +++ b/fdbserver/SimulatedCluster.actor.cpp @@ -1344,11 +1344,6 @@ void SimulationConfig::setStorageEngine(const TestConfig& testConfig) { } } - // TODO CHANGE BACK BEFORE MERGING!! - // Avoid memory storage engines for now to avoid extra memory pressure of change feeds, and redwood is ocasionally - // hitting a root too large issue with change feeds - storage_engine_type = 0; - switch (storage_engine_type) { case 0: { TEST(true); // Simulated cluster using ssd storage engine From b21d0943b9c5199672acc5fbe43618d326fa1c33 Mon Sep 17 00:00:00 2001 From: Josh Slocum Date: Wed, 9 Mar 2022 09:49:49 -0600 Subject: [PATCH 305/413] client-focused cleanup --- fdbclient/BlobWorkerInterface.h | 8 +- fdbclient/DatabaseContext.h | 3 - fdbclient/NativeAPI.actor.cpp | 328 ++---------------- fdbclient/StorageServerInterface.h | 4 +- fdbrpc/fdbrpc.h | 4 - fdbserver/BlobManager.actor.cpp | 18 +- fdbserver/BlobWorker.actor.cpp | 16 +- fdbserver/storageserver.actor.cpp | 4 +- fdbserver/worker.actor.cpp | 22 +- .../BlobGranuleCorrectnessWorkload.actor.cpp | 3 +- .../workloads/BlobGranuleVerifier.actor.cpp | 25 +- 11 files changed, 65 insertions(+), 370 deletions(-) diff --git a/fdbclient/BlobWorkerInterface.h b/fdbclient/BlobWorkerInterface.h index 8be48404de..29318d6a1a 100644 --- a/fdbclient/BlobWorkerInterface.h +++ b/fdbclient/BlobWorkerInterface.h @@ -114,12 +114,10 @@ struct RevokeBlobRangeRequest { }; /* - * Continue: when a worker should continue handling a granule that was evaluated for a split - * Reassign: when a new blob manager takes over, it sends Reassign requests to workers to redistribute granules - * Normal: Neither continue nor reassign + * Continue: Blob worker should continue handling a granule that was evaluated for a split + * Normal: Blob worker should open the granule and start processing it */ -// TODO REMOVE reassign now! -enum AssignRequestType { Normal = 0, Continue = 1, Reassign = 2 }; +enum AssignRequestType { Normal = 0, Continue = 1 }; struct AssignBlobRangeRequest { constexpr static FileIdentifier file_identifier = 905381; diff --git a/fdbclient/DatabaseContext.h b/fdbclient/DatabaseContext.h index 890c1c8214..485617c102 100644 --- a/fdbclient/DatabaseContext.h +++ b/fdbclient/DatabaseContext.h @@ -181,8 +181,6 @@ struct ChangeFeedStorageData : ReferenceCounted { Promise destroyed; UID interfToken; - bool debug = false; // TODO REMOVE - ~ChangeFeedStorageData() { destroyed.send(Void()); } }; @@ -193,7 +191,6 @@ struct ChangeFeedData : ReferenceCounted { Version getVersion(); Future whenAtLeast(Version version); - Key id; // TODO REMOVE eventually? for debugging NotifiedVersion lastReturnedVersion; std::vector> storageData; AsyncVar notAtLatest; diff --git a/fdbclient/NativeAPI.actor.cpp b/fdbclient/NativeAPI.actor.cpp index 482cc651fd..1963fb908c 100644 --- a/fdbclient/NativeAPI.actor.cpp +++ b/fdbclient/NativeAPI.actor.cpp @@ -7427,46 +7427,19 @@ ACTOR Future storageFeedVersionUpdater(StorageServerInterface interf, Chan state Promise destroyed = self->destroyed; loop { if (destroyed.isSet()) { - if (self->debug) { - fmt::print("CFSD {0}: destroyed\n", self->id.toString().substr(0, 4)); - } return Void(); } if (self->version.get() < self->desired.get()) { - if (self->debug) { - fmt::print("CFSD {0}: update waiting {1} < {2}\n", - self->id.toString().substr(0, 4), - self->version.get(), - self->desired.get()); - } wait(delay(CLIENT_KNOBS->CHANGE_FEED_EMPTY_BATCH_TIME) || self->version.whenAtLeast(self->desired.get())); if (destroyed.isSet()) { - if (self->debug) { - fmt::print("CFSD {0}: destroyed2\n", self->id.toString().substr(0, 4)); - } return Void(); } - if (self->debug) { - fmt::print("CFSD {0}: updated {1} < {2}\n", - self->id.toString().substr(0, 4), - self->version.get(), - self->desired.get()); - } if (self->version.get() < self->desired.get()) { - if (self->debug) { - fmt::print("CFSD {0}: requesting {1}\n", self->id.toString().substr(0, 4), self->desired.get()); - } try { ChangeFeedVersionUpdateReply rep = wait(brokenPromiseToNever( interf.changeFeedVersionUpdate.getReply(ChangeFeedVersionUpdateRequest(self->desired.get())))); - if (self->debug) { - fmt::print("CFSD {0}: got {1}\n", self->id.toString().substr(0, 4), rep.version); - } if (rep.version > self->version.get()) { - if (self->debug) { - fmt::print("CFSD {0}: V={1} (req)\n", self->id.toString().substr(0, 4), rep.version); - } self->version.set(rep.version); } } catch (Error& e) { @@ -7481,9 +7454,6 @@ ACTOR Future storageFeedVersionUpdater(StorageServerInterface interf, Chan } } } else { - if (self->debug) { - fmt::print("CFSD {0}: desired.WAL({1})\n", self->id.toString().substr(0, 4), self->version.get() + 1); - } wait(self->desired.whenAtLeast(self->version.get() + 1)); } } @@ -7505,37 +7475,9 @@ Reference DatabaseContext::getStorageData(StorageServerIn } Version ChangeFeedData::getVersion() { - // FIXME: add back in smarter version check later - /*if (notAtLatest.get() == 0 && mutations.isEmpty()) { - Version v = storageData[0]->version.get(); - for (int i = 1; i < storageData.size(); i++) { - if (storageData[i]->version.get() < v) { - v = storageData[i]->version.get(); - } - } - return std::max(v, lastReturnedVersion.get()); - } - */ return lastReturnedVersion.get(); } -// TODO REMOVE when BG is correctness clean -// To debug a waitLatest at wait_version returning early, set wait_version to the version, and start+end to a version -// range that surrounds wait_version enough to figure out what's going on -// DEBUG_CF_ID is optional -#define DEBUG_CF_ID ""_sr -#define DEBUG_CF_START_VERSION invalidVersion -#define DEBUG_CF_END_VERSION invalidVersion -#define DEBUG_CF_WAIT_VERSION invalidVersion -#define DEBUG_CF_VERSION(cfId, v) \ - DEBUG_CF_START_VERSION <= v&& v <= DEBUG_CF_END_VERSION && (""_sr == DEBUG_CF_ID || cfId.printable() == DEBUG_CF_ID) - -#define DEBUG_CF_VERSION_RANGE(cfId, vStart, vEnd) \ - DEBUG_CF_START_VERSION <= vEnd&& vStart <= DEBUG_CF_END_VERSION && \ - (""_sr == DEBUG_CF_ID || cfId.printable() == DEBUG_CF_ID) - -#define DEBUG_CF_WAIT(cfId, v) DEBUG_CF_WAIT_VERSION == v && (""_sr == DEBUG_CF_ID || cfId.printable() == DEBUG_CF_ID) - // This function is essentially bubbling the information about what has been processed from the server through the // change feed client. First it makes sure the server has returned all mutations up through the target version, the // native api has consumed and processed, them, and then the fdb client has consumed all of the mutations. @@ -7548,9 +7490,6 @@ ACTOR Future changeFeedWaitLatest(Reference self, Version if (it->version.get() < version) { waiting++; if (version > it->desired.get()) { - if (DEBUG_CF_WAIT(self->id, version)) { - it->debug = true; - } it->desired.set(version); desired++; } @@ -7558,14 +7497,6 @@ ACTOR Future changeFeedWaitLatest(Reference self, Version } } - if (DEBUG_CF_WAIT(self->id, version)) { - fmt::print("CFW {0}) WaitLatest: waiting for {1}/{2} ss ({3} < desired)\n", - version, - waiting, - self->storageData.size(), - desired); - } - wait(waitForAll(allAtLeast)); // then, wait on ss streams to have processed up through version @@ -7576,10 +7507,6 @@ ACTOR Future changeFeedWaitLatest(Reference self, Version } } - if (DEBUG_CF_WAIT(self->id, version)) { - fmt::print("CFW {0}) WaitLatest: waiting for {1} ss onEmpty\n", version, onEmpty.size()); - } - if (onEmpty.size()) { wait(waitForAll(onEmpty)); } @@ -7592,42 +7519,20 @@ ACTOR Future changeFeedWaitLatest(Reference self, Version // done processing or we have up through the desired version while (self->lastReturnedVersion.get() < self->maxSeenVersion && self->lastReturnedVersion.get() < version) { Version target = std::min(self->maxSeenVersion, version); - if (DEBUG_CF_WAIT(self->id, version)) { - fmt::print("CFW {0}) WaitLatest: waiting merge lastReturned >= {1}\n", version, target); - } wait(self->lastReturnedVersion.whenAtLeast(target)); - if (DEBUG_CF_WAIT(self->id, version)) { - fmt::print( - "CFW {0}) WaitLatest: got merge lastReturned {1}\n", version, self->lastReturnedVersion.get()); - } } // then, wait for client to have consumed up through version if (self->maxSeenVersion >= version) { // merge cursor may have something buffered but has not yet sent it to self->mutations, just wait for // lastReturnedVersion - if (DEBUG_CF_WAIT(self->id, version)) { - fmt::print("CFW {0}) WaitLatest: maxSeenVersion -> waiting lastReturned\n", version); - } - wait(self->lastReturnedVersion.whenAtLeast(version)); - - if (DEBUG_CF_WAIT(self->id, version)) { - fmt::print("CFW {0}) WaitLatest: maxSeenVersion -> got lastReturned\n", version); - } } else { // all mutations <= version are in self->mutations, wait for empty while (!self->mutations.isEmpty()) { - if (DEBUG_CF_WAIT(self->id, version)) { - fmt::print("CFW {0}) WaitLatest: waiting for client onEmpty\n", version); - } wait(self->mutations.onEmpty()); wait(delay(0)); } - - if (DEBUG_CF_WAIT(self->id, version)) { - fmt::print("CFW {0}) WaitLatest: done\n", version); - } } return Void(); @@ -7635,28 +7540,14 @@ ACTOR Future changeFeedWaitLatest(Reference self, Version ACTOR Future changeFeedWhenAtLatest(Reference self, Version version) { if (version >= self->endVersion) { - if (DEBUG_CF_WAIT(self->id, version)) { - fmt::print( - "CFW {0}) WhenAtLeast: After CF end version {1}, returning Never()\n", version, self->endVersion); - } return Never(); } - - if (DEBUG_CF_WAIT(self->id, version)) { - fmt::print("CFW {0}) WhenAtLeast: LR={1}\n", version, self->lastReturnedVersion.get()); - } if (version <= self->getVersion()) { - if (DEBUG_CF_WAIT(self->id, version)) { - fmt::print("CFW {0}) WhenAtLeast: Already done\n", version, self->lastReturnedVersion.get()); - } return Void(); } state Future lastReturned = self->lastReturnedVersion.whenAtLeast(version); loop { - if (DEBUG_CF_WAIT(self->id, version)) { - fmt::print("CFW {0}) WhenAtLeast: NotAtLatest={1}\n", version, self->notAtLatest.get()); - } // only allowed to use empty versions if you're caught up Future waitEmptyVersion = (self->notAtLatest.get() == 0) ? changeFeedWaitLatest(self, version) : Never(); choose { @@ -7667,10 +7558,6 @@ ACTOR Future changeFeedWhenAtLatest(Reference self, Versio } } - if (DEBUG_CF_VERSION(self->id, version)) { - fmt::print("CFLR (WAL): {0}\n", version); - } - if (self->lastReturnedVersion.get() < version) { self->lastReturnedVersion.set(version); } @@ -7688,10 +7575,7 @@ ACTOR Future partialChangeFeedStream(StorageServerInterface interf, Version begin, Version end, Reference feedData, - Reference storageData, - int idx /* TODO REMOVE this param after correctness clean */, - KeyRange range /* TODO REMOVE this param after correctness clean */, - UID debugID /*TODO REMOVE this param after correctness clean*/) { + Reference storageData) { // calling lastReturnedVersion's callbacks could cause us to be cancelled state Promise refresh = feedData->refresh; @@ -7700,34 +7584,11 @@ ACTOR Future partialChangeFeedStream(StorageServerInterface interf, try { loop { if (nextVersion >= end) { - if (DEBUG_CF_VERSION(feedData->id, end)) { - fmt::print(" single {0} {1} [{2} - {3}): sending EOS\n", - idx, - interf.id().toString().substr(0, 4), - range.begin.printable(), - range.end.printable()); - } results.sendError(end_of_stream()); return Void(); } choose { when(state ChangeFeedStreamReply rep = waitNext(replyStream.getFuture())) { - if (DEBUG_CF_VERSION_RANGE( - feedData->id, rep.mutations.front().version, rep.mutations.back().version)) { - fmt::print(" single {0} {1} {2}: response {3} - {4} ({5}), atLatest={6}, rep.atLatest={7}, " - "notAtLatest={8}, minSV={9}\n", - idx, - interf.id().toString().substr(0, 4), - debugID.toString().substr(0, 8).c_str(), - rep.mutations.front().version, - rep.mutations.back().version, - rep.mutations.size(), - atLatestVersion ? "T" : "F", - rep.atLatestVersion ? "T" : "F", - feedData->notAtLatest.get(), - rep.minStreamVersion); - } - // handle first empty mutation on stream establishment explicitly if (nextVersion == begin && rep.mutations.size() == 1 && rep.mutations[0].mutations.size() == 0 && rep.mutations[0].version == begin - 1) { @@ -7744,39 +7605,9 @@ ACTOR Future partialChangeFeedStream(StorageServerInterface interf, state int resultLoc = 0; while (resultLoc < rep.mutations.size()) { wait(results.onEmpty()); - if (DEBUG_CF_VERSION(feedData->id, rep.mutations[resultLoc].version)) { - fmt::print(" single {0} {1} [{2} - {3}): sending {4}/{5} {6} ({7})\n", - idx, - interf.id().toString().substr(0, 4), - range.begin.printable(), - range.end.printable(), - resultLoc, - rep.mutations.size(), - rep.mutations[resultLoc].version, - rep.mutations[resultLoc].mutations.size()); - } if (rep.mutations[resultLoc].version >= nextVersion) { results.send(rep.mutations[resultLoc]); } else { - // TODO REMOVE eventually, useful for debugging for now - if (!rep.mutations[resultLoc].mutations.empty()) { - fmt::print("non-empty mutations ({0}), but versions out of order from {1} for {2} cf " - "{3}! mv={4}, nv={5}\n", - rep.mutations.size(), - interf.id().toString().substr(0, 4), - idx, - feedData->id.toString().substr(0, 6), - rep.mutations[resultLoc].version, - nextVersion); - for (auto& it : rep.mutations[resultLoc].mutations) { - if (it.type == MutationRef::SetValue) { - printf(" %s=", it.param1.printable().c_str()); - } else { - printf( - " %s - %s", it.param1.printable().c_str(), it.param2.printable().c_str()); - } - } - } ASSERT(rep.mutations[resultLoc].mutations.empty()); } resultLoc++; @@ -7807,37 +7638,16 @@ ACTOR Future partialChangeFeedStream(StorageServerInterface interf, when(wait(atLatestVersion && replyStream.isEmpty() && results.isEmpty() ? storageData->version.whenAtLeast(nextVersion) : Future(Never()))) { - if (DEBUG_CF_VERSION(feedData->id, nextVersion)) { - fmt::print(" single {0} {1}: WAL {2}, sending empty {3})\n", - idx, - interf.id().toString().substr(0, 4), - nextVersion, - storageData->version.get()); - } MutationsAndVersionRef empty; empty.version = storageData->version.get(); results.send(empty); nextVersion = storageData->version.get() + 1; } when(wait(atLatestVersion && replyStream.isEmpty() && !results.isEmpty() ? results.onEmpty() - : Future(Never()))) { - if (DEBUG_CF_VERSION(feedData->id, nextVersion)) { - fmt::print(" single {0} {1}: got onEmpty\n", idx, interf.id().toString().substr(0, 4)); - } - } + : Future(Never()))) {} } } } catch (Error& e) { - // TODO REMOVE eventually, useful for debugging for now - // if (DEBUG_CF_VERSION(feedData->id, nextVersion)) { - fmt::print(" single {0} {1} {2} [{3} - {4}): CFError {5}\n", - idx, - interf.id().toString().substr(0, 4), - debugID.toString().substr(0, 8).c_str(), - range.begin.printable(), - range.end.printable(), - e.name()); - // } if (e.code() == error_code_actor_cancelled) { throw; } @@ -7846,14 +7656,11 @@ ACTOR Future partialChangeFeedStream(StorageServerInterface interf, } } -// TODO better name -ACTOR Future doCFMerge(Reference results, - std::vector> interfs, - std::vector streams, - Version* begin, - Version end) { - // TODO REMOVE, a sanity check - state int eosCount = 0; +ACTOR Future mergeChangeFeedStreamInternal(Reference results, + std::vector> interfs, + std::vector streams, + Version* begin, + Version end) { state Promise refresh = results->refresh; // with empty version handling in the partial cursor, all streams will always have a next element with version >= // the minimum version of any stream's next element @@ -7872,8 +7679,6 @@ ACTOR Future doCFMerge(Reference results, state int interfNum = 0; - // TODO minor optimization - could make this just a vector of indexes if each MutationAndVersionStream remembered - // its version index state std::vector streamsUsed; // initially, pull from all streams for (auto& stream : streams) { @@ -7893,13 +7698,10 @@ ACTOR Future doCFMerge(Reference results, if (e.code() != error_code_end_of_stream) { throw e; } - eosCount++; } interfNum++; } - ASSERT(streams.size() - mutations.size() == eosCount); - if (mutations.empty()) { throw end_of_stream(); } @@ -7932,20 +7734,11 @@ ACTOR Future doCFMerge(Reference results, ASSERT(nextVersion >= *begin); *begin = nextVersion + 1; - if (DEBUG_CF_VERSION(results->id, nextOut.back().version)) { - fmt::print("CFNA (merged): {0} ({1})\n", nextOut.back().version, nextOut.back().mutations.size()); - } // send mutations at nextVersion to the client if (nextOut.back().mutations.empty()) { ASSERT(results->mutations.isEmpty()); } else { - // TODO REMOVE, for debugging - if (nextOut.back().version <= results->lastReturnedVersion.get()) { - fmt::print("ERROR: merge cursor got mutations {0} <= lastReturnedVersion {1}", - nextOut.back().version, - results->lastReturnedVersion.get()); - } ASSERT(nextOut.back().version > results->lastReturnedVersion.get()); results->mutations.send(nextOut); @@ -7953,9 +7746,6 @@ ACTOR Future doCFMerge(Reference results, wait(delay(0)); } - if (DEBUG_CF_VERSION(results->id, nextVersion)) { - fmt::print("CFLR (merged): {0}\n", nextVersion); - } if (nextVersion > results->lastReturnedVersion.get()) { results->lastReturnedVersion.set(nextVersion); } @@ -7964,12 +7754,7 @@ ACTOR Future doCFMerge(Reference results, ACTOR Future onCFErrors(std::vector> onErrors) { wait(waitForAny(onErrors)); - // propagate error - TODO better way? - for (auto& f : onErrors) { - if (f.isError()) { - throw f.getError(); - } - } + // TODO: just do waitForAny and remove onCFErrors if this doesn't trip ASSERT(false); return Void(); } @@ -7986,7 +7771,9 @@ ACTOR Future mergeChangeFeedStream(Reference db, state std::vector> onErrors(interfs.size()); state std::vector streams(interfs.size()); - std::vector debugIDs; + TEST(interfs.size() > 10); // Large change feed merge cursor + TEST(interfs.size() > 100); // Very large change feed merge cursor + results->streams.clear(); for (auto& it : interfs) { ChangeFeedStreamRequest req; @@ -8000,9 +7787,6 @@ 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 debugID = deterministicRandom()->randomUniqueID(); - debugIDs.push_back(debugID); - req.debugID = debugID; results->streams.push_back(it.first.changeFeedStream.getReplyStream(req)); } @@ -8021,31 +7805,13 @@ ACTOR Future mergeChangeFeedStream(Reference db, results->notAtLatest.set(interfs.size()); refresh.send(Void()); - if (DEBUG_CF_START_VERSION != invalidVersion) { - fmt::print("Starting merge cursor for {0} @ {1} - {2}\n", interfs.size(), *begin, end); - } for (int i = 0; i < interfs.size(); i++) { onErrors[i] = results->streams[i].onError(); - fetchers[i] = partialChangeFeedStream(interfs[i].first, - streams[i].results, - results->streams[i], - *begin, - end, - results, - results->storageData[i], - i, - interfs[i].second, - debugIDs[i]); - if (DEBUG_CF_START_VERSION != invalidVersion) { - fmt::print(" [{0} - {1}): {2} {3}\n", - interfs[i].second.begin.printable(), - interfs[i].second.end.printable(), - i, - debugIDs[i].toString().substr(0, 8)); - } + fetchers[i] = partialChangeFeedStream( + interfs[i].first, streams[i].results, results->streams[i], *begin, end, results, results->storageData[i]); } - wait(onCFErrors(onErrors) || doCFMerge(results, interfs, streams, begin, end)); + wait(onCFErrors(onErrors) || mergeChangeFeedStreamInternal(results, interfs, streams, begin, end)); return Void(); } @@ -8084,12 +7850,11 @@ ACTOR Future getChangeFeedRange(Reference db, Databas } } -ACTOR Future doSingleCFStream(KeyRange range, - Reference results, - Key rangeID, - Version* begin, - Version end, - UID debugID /*TODO REMOVE this parameter once BG is correctness clean*/) { +ACTOR Future singleChangeFeedStreamInternal(KeyRange range, + Reference results, + Key rangeID, + Version* begin, + Version end) { state Promise refresh = results->refresh; ASSERT(results->streams.size() == 1); @@ -8125,15 +7890,6 @@ ACTOR Future doSingleCFStream(KeyRange range, if (anyMutations) { // empty versions can come out of order, as we sometimes send explicit empty versions when restarting a // stream. Anything with mutations should be strictly greater than lastReturnedVersion - if (feedReply.mutations.front().version <= results->lastReturnedVersion.get()) { - fmt::print("out of order mutation for CF {0} Req {1} from ({2}) {3}! {4} < {5}\n", - rangeID.toString().substr(0, 6), - debugID.toString().substr(0, 8), - results->storageData.size(), - results->storageData[0]->id.toString().substr(0, 4).c_str(), - feedReply.mutations.front().version, - results->lastReturnedVersion.get()); - } ASSERT(feedReply.mutations.front().version > results->lastReturnedVersion.get()); results->mutations.send( @@ -8144,16 +7900,6 @@ ACTOR Future doSingleCFStream(KeyRange range, wait(delay(0)); } - if (DEBUG_CF_VERSION(rangeID, feedReply.mutations.back().version)) { - fmt::print("CFLR (single) {0}: {1} ({2}), atLatest={3}, rep.atLatest={4}, notAtLatest={5}, minSV={6}\n", - debugID.toString().substr(0, 8), - feedReply.mutations.back().version, - feedReply.mutations.size(), - atLatest ? "T" : "F", - feedReply.atLatestVersion ? "T" : "F", - results->notAtLatest.get(), - feedReply.minStreamVersion); - } // check refresh.canBeSet so that, if we are killed after calling one of these callbacks, we just // skip to the next wait and get actor_cancelled if (feedReply.mutations.back().version > results->lastReturnedVersion.get()) { @@ -8165,11 +7911,6 @@ ACTOR Future doSingleCFStream(KeyRange range, results->notAtLatest.set(0); } if (refresh.canBeSet() && feedReply.minStreamVersion > results->storageData[0]->version.get()) { - if (results->storageData[0]->debug) { - fmt::print("CFSD {0}: V={1} (CFLR)\n", - results->storageData[0]->id.toString().substr(0, 4), - results->storageData[0]->version.get()); - } results->storageData[0]->version.set(feedReply.minStreamVersion); } } @@ -8186,14 +7927,12 @@ ACTOR Future singleChangeFeedStream(Reference db, bool canReadPopped) { state Database cx(db); state ChangeFeedStreamRequest req; - state UID debugID = deterministicRandom()->randomUniqueID(); req.rangeID = rangeID; req.begin = *begin; req.end = end; req.range = range; req.canReadPopped = canReadPopped; req.replyBufferSize = replyBufferSize; - req.debugID = debugID; results->streams.clear(); @@ -8212,17 +7951,7 @@ ACTOR Future singleChangeFeedStream(Reference db, results->notAtLatest.set(1); refresh.send(Void()); - if (DEBUG_CF_START_VERSION != invalidVersion) { - fmt::print("Starting single cursor {0} for [{1} - {2}) @ {3} - {4} from {5}\n", - debugID.toString().substr(0, 8), - range.begin.printable(), - range.end.printable(), - *begin, - end, - interf.id().toString().c_str()); - } - - wait(results->streams[0].onError() || doSingleCFStream(range, results, rangeID, begin, end, debugID)); + wait(results->streams[0].onError() || singleChangeFeedStreamInternal(range, results, rangeID, begin, end)); return Void(); } @@ -8238,19 +7967,11 @@ ACTOR Future getChangeFeedStreamActor(Reference db, state Database cx(db); state Span span("NAPI:GetChangeFeedStream"_loc); - results->id = rangeID; results->endVersion = end; state double sleepWithBackoff = CLIENT_KNOBS->WRONG_SHARD_SERVER_DELAY; state Version lastBeginVersion = invalidVersion; - /*printf("CFStream %s [%s - %s): [%lld - %lld]\n", - rangeID.printable().substr(0, 6).c_str(), - range.begin.printable().c_str(), - range.end.printable().c_str(), - begin, - end);*/ - loop { state KeyRange keys; try { @@ -8319,17 +8040,19 @@ ACTOR Future getChangeFeedStreamActor(Reference db, interfs.emplace_back(locations[i].second->getInterface(chosenLocations[i]), locations[i].first & range); } + TEST(true); // Change feed merge cursor + // TODO (jslocum): validate connectionFileChanged behavior wait( mergeChangeFeedStream(db, interfs, results, rangeID, &begin, end, replyBufferSize, canReadPopped) || cx->connectionFileChanged()); } else { + TEST(true); // Change feed single cursor StorageServerInterface interf = locations[0].second->getInterface(chosenLocations[0]); wait(singleChangeFeedStream( db, interf, range, results, rangeID, &begin, end, replyBufferSize, canReadPopped) || cx->connectionFileChanged()); } } catch (Error& e) { - fmt::print("CFNA error {}\n", e.name()); if (e.code() == error_code_actor_cancelled || e.code() == error_code_change_feed_popped) { for (auto& it : results->storageData) { if (it->debugGetReferenceCount() == 2) { @@ -8339,6 +8062,7 @@ ACTOR Future getChangeFeedStreamActor(Reference db, results->streams.clear(); results->storageData.clear(); if (e.code() == error_code_change_feed_popped) { + TEST(true); // getChangeFeedStreamActor got popped results->mutations.sendError(e); results->refresh.sendError(e); } else { @@ -8346,8 +8070,6 @@ ACTOR Future getChangeFeedStreamActor(Reference db, } throw; } - // TODO REMOVE - // fmt::print("CFNA error {}\n", e.name()); if (results->notAtLatest.get() == 0) { results->notAtLatest.set(1); } @@ -8355,8 +8077,6 @@ ACTOR Future getChangeFeedStreamActor(Reference db, if (e.code() == error_code_wrong_shard_server || e.code() == error_code_all_alternatives_failed || e.code() == error_code_connection_failed || e.code() == error_code_unknown_change_feed || e.code() == error_code_broken_promise) { - // TODO: add some exponential backoff (with a reasonable cap) when retrying if we didn't advance - // beginVersion at all before getting an error db->changeFeedCache.erase(rangeID); cx->invalidateCache(keys); if (begin == lastBeginVersion) { diff --git a/fdbclient/StorageServerInterface.h b/fdbclient/StorageServerInterface.h index 84bdfd95fe..cbe6937f2f 100644 --- a/fdbclient/StorageServerInterface.h +++ b/fdbclient/StorageServerInterface.h @@ -793,15 +793,13 @@ struct ChangeFeedStreamRequest { KeyRange range; int replyBufferSize = -1; bool canReadPopped = true; - // TODO REMOVE once BG is correctness clean!! Useful for debugging - UID debugID; ReplyPromiseStream reply; ChangeFeedStreamRequest() {} template void serialize(Ar& ar) { - serializer(ar, rangeID, begin, end, range, reply, spanContext, replyBufferSize, canReadPopped, debugID, arena); + serializer(ar, rangeID, begin, end, range, reply, spanContext, replyBufferSize, canReadPopped, arena); } }; diff --git a/fdbrpc/fdbrpc.h b/fdbrpc/fdbrpc.h index c575788b1d..7490542138 100644 --- a/fdbrpc/fdbrpc.h +++ b/fdbrpc/fdbrpc.h @@ -449,9 +449,6 @@ public: } } - // TODO REMOVE - const void* debugAddr() const { return queue; } - template void sendError(const E& exc) const { if (queue->isRemoteEndpoint()) { @@ -770,7 +767,6 @@ public: } else { Reference peer = FlowTransport::transport().sendUnreliable(SerializeSource(value), getEndpoint(), true); - // FIXME: defer sending the message until we know the connection is established endStreamOnDisconnect(disc, p, getEndpoint(), peer); } return p; diff --git a/fdbserver/BlobManager.actor.cpp b/fdbserver/BlobManager.actor.cpp index 2fe9f034ca..d21a1a9a79 100644 --- a/fdbserver/BlobManager.actor.cpp +++ b/fdbserver/BlobManager.actor.cpp @@ -470,7 +470,7 @@ ACTOR Future doRangeAssignment(Reference bmData, bmData->rangesToAssign.send(revokeOld); // send assignment back to queue as is, clearing designated worker if present - // if we failed to send continue or reassign to the worker we thought owned the shard, it should be retried + // if we failed to send continue to the worker we thought owned the shard, it should be retried // as a normal assign ASSERT(assignment.assign.present()); assignment.assign.get().type = AssignRequestType::Normal; @@ -952,14 +952,14 @@ ACTOR Future maybeSplitRange(Reference bmData, newLockSeqno = bmData->seqNo; bmData->seqNo++; if (!(bmData->epoch > ownerEpoch || (bmData->epoch == ownerEpoch && newLockSeqno > ownerSeqno))) { - printf("BM seqno for granule [%s - %s) out of order for lock! manager: (%lld, %lld), owner: %lld, " - "%lld)\n", - granuleRange.begin.printable().c_str(), - granuleRange.end.printable().c_str(), - bmData->epoch, - newLockSeqno, - ownerEpoch, - ownerSeqno); + fmt::print("BM seqno for granule [{0} - {1}) out of order for lock! manager: ({2}, {3}), owner: " + "({4}, {5}})\n", + granuleRange.begin.printable().c_str(), + granuleRange.end.printable().c_str(), + bmData->epoch, + newLockSeqno, + ownerEpoch, + ownerSeqno); } ASSERT(bmData->epoch > ownerEpoch || (bmData->epoch == ownerEpoch && newLockSeqno > ownerSeqno)); } else if (bmData->epoch == ownerEpoch && newLockSeqno < ownerSeqno) { diff --git a/fdbserver/BlobWorker.actor.cpp b/fdbserver/BlobWorker.actor.cpp index 15d83b6acd..48b73727d8 100644 --- a/fdbserver/BlobWorker.actor.cpp +++ b/fdbserver/BlobWorker.actor.cpp @@ -1927,7 +1927,7 @@ ACTOR Future blobGranuleUpdateFiles(Reference bwData, inFlightFiles[i].future.cancel(); } - // if we retry and re-open, we need to use a normal request (no continue or reassign) and update the + // if we retry and re-open, we need to use a normal request (no continue) and update the // seqno metadata->originalReq.managerEpoch = metadata->continueEpoch; metadata->originalReq.managerSeqno = metadata->continueSeqno; @@ -2787,20 +2787,6 @@ static bool changeBlobRange(Reference bwData, fmt::print("thisAssignmentNewer={}\n", thisAssignmentNewer ? "true" : "false"); } - // if this granule already has it, and this was a special assignment (i.e. a new blob manager is - // trying to reassign granules), then just continue - - // TODO this needs to also have the condition - if (active && assignType.get() == AssignRequestType::Reassign && r.value().activeMetadata.isValid() && - r.begin() == keyRange.begin && r.end() == keyRange.end) { - r.value().lastEpoch = epoch; - r.value().lastSeqno = seqno; - r.value().activeMetadata->continueEpoch = epoch; - r.value().activeMetadata->continueSeqno = seqno; - alreadyAssigned = true; - break; - } - if (BW_DEBUG) { fmt::print("last: ({0}, {1}). now: ({2}, {3})\n", r.value().lastEpoch, r.value().lastSeqno, epoch, seqno); } diff --git a/fdbserver/storageserver.actor.cpp b/fdbserver/storageserver.actor.cpp index 5d62d9ff91..1b6fdaf4b3 100644 --- a/fdbserver/storageserver.actor.cpp +++ b/fdbserver/storageserver.actor.cpp @@ -8085,9 +8085,7 @@ ACTOR Future serveChangeFeedStreamRequests(StorageServer* self, ChangeFeedStreamRequest req = waitNext(changeFeedStream); // must notify change feed that its shard is moved away ASAP - // TODO CHANGE BACK after BG is correctness clean - // state UID streamUID = deterministicRandom()->randomUniqueID(); - UID streamUID = req.debugID; + UID streamUID = deterministicRandom()->randomUniqueID(); self->actors.add(changeFeedStreamQ(self, req, streamUID) || stopChangeFeedOnMove(self, req, streamUID)); } } diff --git a/fdbserver/worker.actor.cpp b/fdbserver/worker.actor.cpp index 0aedfb0397..ff6adde754 100644 --- a/fdbserver/worker.actor.cpp +++ b/fdbserver/worker.actor.cpp @@ -1138,7 +1138,6 @@ ACTOR Future storageServerRollbackRebooter(std::set(), Reference(nullptr)); @@ -1880,15 +1879,17 @@ ACTOR Future workerServer(Reference connRecord, TEST(true); // Recruited while already a blob manager. } else { - // TODO: it'd be more optimal to halt the last manager here, but it will figure it out via the epoch - // check - // Also, not halting lets us handle the case here where the last BM had a higher epoch and somehow - // the epochs got out of order by a delayed initialize request. The one we start here will just halt - // on the lock check. + // TODO: it'd be more optimal to halt the last manager if present here, but it will figure it out + // via the epoch check + // Also, not halting lets us handle the case here where the last BM had a higher + // epoch and somehow the epochs got out of order by a delayed initialize request. The one we start + // here will just halt on the lock check. myBMEpoch = req.epoch; startRole(Role::BLOB_MANAGER, recruited.id(), interf.id()); DUMPTOKEN(recruited.waitFailure); DUMPTOKEN(recruited.haltBlobManager); + DUMPTOKEN(recruited.haltBlobGranules); + DUMPTOKEN(recruited.blobManagerExclCheckReq); Future blobManagerProcess = blobManager(recruited, dbInfo, req.epoch); errorForwarders.add(forwardError( @@ -2106,9 +2107,18 @@ ACTOR Future workerServer(Reference connRecord, recruited.initEndpoints(); startRole(Role::BLOB_WORKER, recruited.id(), interf.id()); + DUMPTOKEN(recruited.waitFailure); + DUMPTOKEN(recruited.blobGranuleFileRequest); + DUMPTOKEN(recruited.assignBlobRangeRequest); + DUMPTOKEN(recruited.revokeBlobRangeRequest); + DUMPTOKEN(recruited.granuleAssignmentsRequest); + DUMPTOKEN(recruited.granuleStatusStreamRequest); + DUMPTOKEN(recruited.haltBlobWorker); + ReplyPromise blobWorkerReady = req.reply; Future bw = blobWorker(recruited, blobWorkerReady, dbInfo); errorForwarders.add(forwardError(errors, Role::BLOB_WORKER, recruited.id(), bw)); + } else { forwardPromise(req.reply, blobWorkerCache.get(req.reqId)); } diff --git a/fdbserver/workloads/BlobGranuleCorrectnessWorkload.actor.cpp b/fdbserver/workloads/BlobGranuleCorrectnessWorkload.actor.cpp index c23b78758e..352511fbb8 100644 --- a/fdbserver/workloads/BlobGranuleCorrectnessWorkload.actor.cpp +++ b/fdbserver/workloads/BlobGranuleCorrectnessWorkload.actor.cpp @@ -796,10 +796,9 @@ struct BlobGranuleCorrectnessWorkload : TestWorkload { } Future start(Database const& cx) override { - // TODO need to make thing that waits for granules to exist before ANY of the actors start! - // Then can reuse that for final checks or something? clients.reserve(3 * directories.size()); for (auto& it : directories) { + // Wait for blob worker to initialize snapshot before starting test for that range Future start = waitFirstSnapshot(this, cx, it, true); clients.push_back(timeout(writeWorker(this, start, cx, it), testDuration, Void())); clients.push_back(timeout(readWorker(this, start, cx, it), testDuration, Void())); diff --git a/fdbserver/workloads/BlobGranuleVerifier.actor.cpp b/fdbserver/workloads/BlobGranuleVerifier.actor.cpp index 187cb90c9f..04aa5e936d 100644 --- a/fdbserver/workloads/BlobGranuleVerifier.actor.cpp +++ b/fdbserver/workloads/BlobGranuleVerifier.actor.cpp @@ -327,7 +327,6 @@ struct BlobGranuleVerifierWorkload : TestWorkload { // utility to prune at pruneVersion= with the flag ACTOR Future pruneAtVersion(Database cx, KeyRange range, Version version, bool force) { state Reference tr = makeReference(cx); - state Version commitVersion = 0; state Key pruneKey; loop { try { @@ -343,19 +342,19 @@ struct BlobGranuleVerifierWorkload : TestWorkload { Standalone vs = wait(fTrVs); pruneKey = blobGranulePruneKeys.begin.withSuffix(vs); if (BGV_DEBUG) { - printf("pruneAtVersion for range [%s-%s) at version %lld succeeded\n", - range.begin.printable().c_str(), - range.end.printable().c_str(), - version); + fmt::print("pruneAtVersion for range [{0} - {1}) at version {2} succeeded\n", + range.begin.printable(), + range.end.printable(), + version); } break; } catch (Error& e) { if (BGV_DEBUG) { - printf("pruneAtVersion for range [%s-%s) at version %lld encountered error %s\n", - range.begin.printable().c_str(), - range.end.printable().c_str(), - version, - e.name()); + fmt::print("pruneAtVersion for range [{0} - {1}) at version {2} encountered error {3}\n", + range.begin.printable(), + range.end.printable(), + version, + e.name()); } wait(tr->onError(e)); } @@ -487,9 +486,6 @@ struct BlobGranuleVerifierWorkload : TestWorkload { ASSERT(e.code() == error_code_blob_granule_transaction_too_old); } } - - // TODO: read at some version older than pruneVersion and make sure you get txn_too_old - // To achieve this, the BWs are going to have to recognize latest prune versions per granules } catch (Error& e) { if (e.code() == error_code_blob_granule_transaction_too_old && oldRead.v >= dbgPruneVersion) { self->timeTravelTooOld++; @@ -556,8 +552,6 @@ struct BlobGranuleVerifierWorkload : TestWorkload { Version readVersion = wait(tr->getReadVersion()); return readVersion; } catch (Error& e) { - // TODO REMOVE print - printf("BGV GRV got error %s\n", e.name()); wait(tr->onError(e)); } } @@ -566,7 +560,6 @@ struct BlobGranuleVerifierWorkload : TestWorkload { ACTOR Future _check(Database cx, BlobGranuleVerifierWorkload* self) { // check error counts, and do an availability check at the end - // TODO need to have retry loop for getReadVersion, it's throwing tag throttled for some reason? state Transaction tr(cx); state Version readVersion = wait(self->doGrv(&tr)); state Version startReadVersion = readVersion; From 9dbb6d216e75a96752f29f13345b97a96a743b8f Mon Sep 17 00:00:00 2001 From: Josh Slocum Date: Wed, 9 Mar 2022 10:39:04 -0600 Subject: [PATCH 306/413] SS-focused cleanup --- fdbserver/BlobManager.actor.cpp | 2 +- fdbserver/BlobWorker.actor.cpp | 2 +- fdbserver/storageserver.actor.cpp | 388 +++++++----------------------- 3 files changed, 86 insertions(+), 306 deletions(-) diff --git a/fdbserver/BlobManager.actor.cpp b/fdbserver/BlobManager.actor.cpp index d21a1a9a79..7c16df3c11 100644 --- a/fdbserver/BlobManager.actor.cpp +++ b/fdbserver/BlobManager.actor.cpp @@ -43,7 +43,7 @@ #include "flow/UnitTest.h" #include "flow/actorcompiler.h" // has to be last include -#define BM_DEBUG true +#define BM_DEBUG false // TODO add comments + documentation void handleClientBlobRange(KeyRangeMap* knownBlobRanges, diff --git a/fdbserver/BlobWorker.actor.cpp b/fdbserver/BlobWorker.actor.cpp index 48b73727d8..f8afcf0e73 100644 --- a/fdbserver/BlobWorker.actor.cpp +++ b/fdbserver/BlobWorker.actor.cpp @@ -45,7 +45,7 @@ #include "flow/Trace.h" #include "flow/actorcompiler.h" // has to be last include -#define BW_DEBUG true +#define BW_DEBUG false #define BW_REQUEST_DEBUG false struct GranuleStartState { diff --git a/fdbserver/storageserver.actor.cpp b/fdbserver/storageserver.actor.cpp index 1b6fdaf4b3..b35fe3f611 100644 --- a/fdbserver/storageserver.actor.cpp +++ b/fdbserver/storageserver.actor.cpp @@ -23,6 +23,7 @@ #include #include +#include "contrib/fmt-8.1.1/include/fmt/format.h" #include "fdbrpc/fdbrpc.h" #include "fdbrpc/LoadBalance.h" #include "flow/ActorCollection.h" @@ -392,8 +393,8 @@ struct ChangeFeedInfo : ReferenceCounted { void moved(KeyRange range) { auto toTrigger = moveTriggers.intersectingRanges(range); - for (auto triggerRange : toTrigger) { - for (auto triggerStream : triggerRange.cvalue()) { + for (auto& triggerRange : toTrigger) { + for (auto& triggerStream : triggerRange.cvalue()) { if (triggerStream.second.canBeSet()) { triggerStream.second.send(Void()); } @@ -410,7 +411,7 @@ struct ChangeFeedInfo : ReferenceCounted { ASSERT(streamToRemove != triggerRange->cvalue().end()); triggerRange->value().erase(streamToRemove); } - // TODO: could clean up on + // TODO: may be more cleanup possible here } }; @@ -1203,7 +1204,7 @@ public: auto& clientVersions = changeFeedClientVersions[addr]; Version minVersion = version.get(); for (auto& it : clientVersions) { - // printf("Blocked client %s @ %lld\n", it.first.toString().substr(0, 8).c_str(), it.second); + // fmt::print("Blocked client {0} @ {1}\n", it.first.toString().substr(0, 8), it.second); minVersion = std::min(minVersion, it.second); } return minVersion; @@ -1892,16 +1893,11 @@ MutationsAndVersionRef filterMutations(Arena& arena, return m; } -// TODO REMOVE!!! when BG is correctness clean -#define DEBUG_SS_ID ""_sr -#define DEBUG_SS_CF_ID ""_sr -#define DEBUG_SS_STREAM_ID ""_sr -#define DEBUG_SS_CF_BEGIN_VERSION invalidVersion -#define DEBUG_SS_CFM(ssId, cfId, streamId, v) \ - ((ssId.toString().substr(0, 4) == DEBUG_SS_ID && cfId.printable().substr(0, 6) == DEBUG_SS_CF_ID && \ - (v >= DEBUG_SS_CF_BEGIN_VERSION || latestVersion == DEBUG_SS_CF_BEGIN_VERSION)) || \ - (streamId.toString().substr(0, 8) == DEBUG_SS_STREAM_ID)) +// set this for VERY verbose logs on change feed SS reads +#define DEBUG_CF_TRACE false +// To easily find if a change feed read missed data. Set the CF to the feedId, the key to the missing key, and the +// version to the version the mutation is missing at. #define DO_DEBUG_CF_MISSING false #define DEBUG_CF_MISSING_CF ""_sr #define DEBUG_CF_MISSING_KEY ""_sr @@ -1915,25 +1911,15 @@ ACTOR Future> getChangeFeedMutations(Stor ChangeFeedStreamRequest req, bool inverted, bool atLatest, - UID streamUID - /*TODO REMOVE*/) { + UID streamUID /* for debugging */) { 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(); - if (DEBUG_SS_CFM(data->thisServerID, req.rangeID, streamUID, req.begin)) { - printf("CFM: SS %s CF %s: SQ %s [%s - %s) %lld - %lld\n", - data->thisServerID.toString().substr(0, 4).c_str(), - req.rangeID.printable().substr(0, 6).c_str(), - streamUID.toString().substr(0, 8).c_str(), - req.range.begin.printable().c_str(), - req.range.end.printable().c_str(), - req.begin, - req.end); - // TODO REMOVE - TraceEvent(SevDebug, "ChangeFeedMutations", data->thisServerID) + if (DEBUG_CF_TRACE) { + TraceEvent(SevDebug, "TraceChangeFeedMutationsBegin", data->thisServerID) .detail("FeedID", req.rangeID) .detail("StreamUID", streamUID) .detail("Range", req.range) @@ -1961,14 +1947,6 @@ ACTOR Future> getChangeFeedMutations(Stor state Reference feedInfo = feed->second; - if (DEBUG_SS_CFM(data->thisServerID, req.rangeID, streamUID, req.begin)) { - printf("CFM: SS %s CF %s: got version %lld >= %lld\n", - data->thisServerID.toString().substr(0, 4).c_str(), - req.rangeID.printable().substr(0, 6).c_str(), - data->version.get(), - req.begin); - } - // We must copy the mutationDeque when fetching the durable bytes in case mutations are popped from memory while // waiting for the results state Version dequeVersion = data->version.get(); @@ -1976,22 +1954,8 @@ ACTOR Future> getChangeFeedMutations(Stor state Version emptyVersion = feedInfo->emptyVersion; Version fetchStorageVersion = std::max(feedInfo->fetchVersion, feedInfo->durableFetchVersion.get()); - if (DEBUG_SS_CFM(data->thisServerID, req.rangeID, streamUID, req.begin)) { - printf("CFM: SS %s CF %s: SQ %s atLatest=%s, dequeVersion=%lld, emptyVersion=%lld, storageVersion=%lld, " - "durableVersion=%lld, " - "fetchStorageVersion=%lld (%lld, %lld)\n", - data->thisServerID.toString().substr(0, 4).c_str(), - req.rangeID.printable().substr(0, 6).c_str(), - streamUID.toString().substr(0, 8).c_str(), - atLatest ? "T" : "F", - dequeVersion, - feedInfo->emptyVersion, - feedInfo->storageVersion, - feedInfo->durableVersion, - fetchStorageVersion, - feedInfo->fetchVersion, - feedInfo->durableFetchVersion.get()); - TraceEvent(SevDebug, "ChangeFeedMutationsDetails", data->thisServerID) + if (DEBUG_CF_TRACE) { + TraceEvent(SevDebug, "TraceChangeFeedMutationsDetails", data->thisServerID) .detail("FeedID", req.rangeID) .detail("StreamUID", streamUID) .detail("Range", req.range) @@ -2022,14 +1986,6 @@ ACTOR Future> getChangeFeedMutations(Stor } } } - if (DEBUG_SS_CFM(data->thisServerID, req.rangeID, streamUID, req.begin)) { - printf("CFM: SS %s CF %s: got %lld - %lld (%d) from memory\n", - data->thisServerID.toString().substr(0, 4).c_str(), - req.rangeID.printable().substr(0, 6).c_str(), - memoryReply.mutations.empty() ? invalidVersion : memoryReply.mutations.front().version, - memoryReply.mutations.empty() ? invalidVersion : memoryReply.mutations.back().version, - memoryReply.mutations.size()); - } } state bool readDurable = feedInfo->durableVersion != invalidVersion && req.begin <= feedInfo->durableVersion; @@ -2044,23 +2000,10 @@ ACTOR Future> getChangeFeedMutations(Stor ASSERT(req.begin <= feedInfo->fetchVersion); TEST(true); // getChangeFeedMutations before fetched data durable - if (DEBUG_SS_CFM(data->thisServerID, req.rangeID, streamUID, req.begin)) { - printf("CFM: SS %s CF %s: waiting on fetch durable up to %lld\n", - data->thisServerID.toString().substr(0, 4).c_str(), - req.rangeID.printable().substr(0, 6).c_str(), - feedInfo->fetchVersion); - } - // Wait for next commit to write pending feed data to storage wait(feedInfo->durableFetchVersion.whenAtLeast(feedInfo->fetchVersion)); // To let update storage finish wait(delay(0)); - if (DEBUG_SS_CFM(data->thisServerID, req.rangeID, streamUID, req.begin)) { - printf("CFM: SS %s CF %s: got fetch durable up to %lld\n", - data->thisServerID.toString().substr(0, 4).c_str(), - req.rangeID.printable().substr(0, 6).c_str(), - feedInfo->durableFetchVersion.get()); - } } RangeResult res = wait( data->storage.readRange(KeyRangeRef(changeFeedDurableKey(req.rangeID, std::max(req.begin, emptyVersion)), @@ -2097,23 +2040,22 @@ ACTOR Future> getChangeFeedMutations(Stor memoryVerifyIdx++; continue; } else { - printf("ERROR: SS %s CF %s SQ %s has mutation at %lld in memory but not on disk (next disk is " - "%lld) " - "(emptyVersion=%lld, emptyBefore=%lld)!\n", - data->thisServerID.toString().substr(0, 4).c_str(), - req.rangeID.printable().substr(0, 6).c_str(), - streamUID.toString().substr(0, 8).c_str(), - memoryReply.mutations[memoryVerifyIdx].version, - version, - feedInfo->emptyVersion, - emptyVersion); + fmt::print("ERROR: SS {0} CF {1} SQ {2} has mutation at {3} in memory but not on disk (next disk " + "is {4}) (emptyVersion={5}, emptyBefore={6})!\n", + data->thisServerID.toString().substr(0, 4), + req.rangeID.printable().substr(0, 6), + streamUID.toString().substr(0, 8), + memoryReply.mutations[memoryVerifyIdx].version, + version, + feedInfo->emptyVersion, + emptyVersion); - printf(" Memory: (%d)\n", memoryReply.mutations[memoryVerifyIdx].mutations.size()); + fmt::print(" Memory: ({})\n", memoryReply.mutations[memoryVerifyIdx].mutations.size()); for (auto& it : memoryReply.mutations[memoryVerifyIdx].mutations) { if (it.type == MutationRef::SetValue) { - printf(" %s=\n", it.param1.printable().c_str()); + fmt::print(" {}=\n", it.param1.printable()); } else { - printf(" %s - %s\n", it.param1.printable().c_str(), it.param2.printable().c_str()); + fmt::print(" {} - {}\n", it.param1.printable(), it.param2.printable()); } } ASSERT(false); @@ -2128,25 +2070,24 @@ ACTOR Future> getChangeFeedMutations(Stor if (memoryVerifyIdx < memoryReply.mutations.size() && version == memoryReply.mutations[memoryVerifyIdx].version) { - // TODO: we could do some validation here too, but it's complicated because clears can get split + // We could do validation of mutations here too, but it's complicated because clears can get split // and stuff memoryVerifyIdx++; } } else if (memoryVerifyIdx < memoryReply.mutations.size() && version == memoryReply.mutations[memoryVerifyIdx].version) { - // TODO REMOVE debugging eventually - printf("ERROR: SS %s CF %s SQ %s has mutation at %lld in memory but all filtered out on disk!\n", - data->thisServerID.toString().substr(0, 4).c_str(), - req.rangeID.printable().substr(0, 6).c_str(), - streamUID.toString().substr(0, 8).c_str(), - version); + fmt::print("ERROR: SS {0} CF {1} SQ {2} has mutation at {3} in memory but all filtered out on disk!\n", + data->thisServerID.toString().substr(0, 4), + req.rangeID.printable().substr(0, 6), + streamUID.toString().substr(0, 8), + version); - printf(" Memory: (%d)\n", memoryReply.mutations[memoryVerifyIdx].mutations.size()); + fmt::print(" Memory: ({})\n", memoryReply.mutations[memoryVerifyIdx].mutations.size()); for (auto& it : memoryReply.mutations[memoryVerifyIdx].mutations) { if (it.type == MutationRef::SetValue) { - printf(" %s=\n", it.param1.printable().c_str()); + fmt::print(" {}=\n", it.param1.printable().c_str()); } else { - printf(" %s - %s\n", it.param1.printable().c_str(), it.param2.printable().c_str()); + fmt::print(" {} - {}\n", it.param1.printable().c_str(), it.param2.printable().c_str()); } } ASSERT(false); @@ -2158,14 +2099,6 @@ ACTOR Future> getChangeFeedMutations(Stor lastVersion = version; lastKnownCommitted = knownCommittedVersion; } - if (DEBUG_SS_CFM(data->thisServerID, req.rangeID, streamUID, req.begin)) { - printf("CFM: SS %s CF %s: got %lld - %lld (%d) from disk\n", - data->thisServerID.toString().substr(0, 4).c_str(), - req.rangeID.printable().substr(0, 6).c_str(), - reply.mutations.empty() ? invalidVersion : reply.mutations.front().version, - reply.mutations.empty() ? invalidVersion : reply.mutations.back().version, - reply.mutations.size()); - } if (remainingDurableBytes > 0) { reply.arena.dependsOn(memoryReply.arena); auto it = memoryReply.mutations.begin(); @@ -2178,21 +2111,9 @@ ACTOR Future> getChangeFeedMutations(Stor // If still empty, that means disk results were filtered out, but skipped all memory results. Add an empty, // either the last version from disk if (reply.mutations.empty() && res.size()) { - if (DEBUG_SS_CFM(data->thisServerID, req.rangeID, streamUID, req.begin)) { - printf("CFM: SS %s CF %s: adding empty from disk and memory %lld\n", - data->thisServerID.toString().substr(0, 4).c_str(), - req.rangeID.printable().substr(0, 6).c_str(), - lastVersion); - } reply.mutations.push_back(reply.arena, MutationsAndVersionRef(lastVersion, lastKnownCommitted)); } } else if (reply.mutations.empty() || reply.mutations.back().version < lastVersion) { - if (DEBUG_SS_CFM(data->thisServerID, req.rangeID, streamUID, req.begin)) { - printf("CFM: SS %s CF %s: adding empty from disk %lld\n", - data->thisServerID.toString().substr(0, 4).c_str(), - req.rangeID.printable().substr(0, 6).c_str(), - lastVersion); - } reply.mutations.push_back(reply.arena, MutationsAndVersionRef(lastVersion, lastKnownCommitted)); } } else { @@ -2203,12 +2124,6 @@ ACTOR Future> getChangeFeedMutations(Stor Version finalVersion = std::min(req.end - 1, dequeVersion); if ((reply.mutations.empty() || reply.mutations.back().version < finalVersion) && remainingLimitBytes > 0 && remainingDurableBytes > 0) { - if (DEBUG_SS_CFM(data->thisServerID, req.rangeID, streamUID, req.begin)) { - printf("CFM: SS %s CF %s: adding empty %lld\n", - data->thisServerID.toString().substr(0, 4).c_str(), - req.rangeID.printable().substr(0, 6).c_str(), - finalVersion); - } reply.mutations.push_back( reply.arena, MutationsAndVersionRef(finalVersion, finalVersion == dequeVersion ? dequeKnownCommit : 0)); // if we add empty mutation after the last thing in memory, and didn't read from disk, gotAll is true @@ -2229,16 +2144,15 @@ ACTOR Future> getChangeFeedMutations(Stor TEST(ok); // feed popped while valid read waiting TEST(!ok); // feed popped while invalid read waiting if (!ok) { - printf("SS %s: CF %s SQ %s popped after read! req.begin=%lld, emptyVersion=%lld, emptyBeforeRead=%lld, " - "atLatest=%s, minVersionSent=%lld\n", - data->thisServerID.toString().substr(0, 4).c_str(), - req.rangeID.printable().substr(0, 6).c_str(), - streamUID.toString().substr(0, 8).c_str(), - req.begin, - feedInfo->emptyVersion, - emptyVersion, - atLatest ? "T" : "F", - minVersion); + TraceEvent("ChangeFeedMutationsPopped", data->thisServerID) + .detail("FeedID", req.rangeID) + .detail("StreamUID", streamUID) + .detail("Range", req.range) + .detail("Begin", req.begin) + .detail("End", req.end) + .detail("EmptyVersion", feedInfo->emptyVersion) + .detail("AtLatest", atLatest) + .detail("MinVersionSent", minVersion); throw change_feed_popped(); } } @@ -2246,7 +2160,7 @@ ACTOR Future> getChangeFeedMutations(Stor if (MUTATION_TRACKING_ENABLED) { for (auto& mutations : reply.mutations) { for (auto& m : mutations.mutations) { - DEBUG_MUTATION("ChangeFeedRead", mutations.version, m, data->thisServerID) + DEBUG_MUTATION("ChangeFeedSSRead", mutations.version, m, data->thisServerID) .detail("ChangeFeedID", req.rangeID) .detail("StreamUID", streamUID) .detail("ReqBegin", req.begin) @@ -2256,14 +2170,7 @@ ACTOR Future> getChangeFeedMutations(Stor } } - if (DEBUG_SS_CFM(data->thisServerID, req.rangeID, streamUID, req.begin)) { - printf("CFM: SS %s CF %s: result %lld - %lld (%d)\n", - data->thisServerID.toString().substr(0, 4).c_str(), - req.rangeID.printable().substr(0, 6).c_str(), - reply.mutations.empty() ? invalidVersion : reply.mutations.front().version, - reply.mutations.empty() ? invalidVersion : reply.mutations.back().version, - reply.mutations.size()); - // TODO REMOVE + if (DEBUG_CF_TRACE) { TraceEvent(SevDebug, "ChangeFeedMutationsDone", data->thisServerID) .detail("FeedID", req.rangeID) .detail("StreamUID", streamUID) @@ -2292,36 +2199,36 @@ ACTOR Future> getChangeFeedMutations(Stor } } if (!foundVersion || !foundKey) { - printf("ERROR: SS %s CF %s SQ %s missing %s @ %lld from request for [%s - %s) %lld - %lld\n", - data->thisServerID.toString().substr(0, 4).c_str(), - req.rangeID.printable().substr(0, 6).c_str(), - streamUID.toString().substr(0, 8).c_str(), - foundVersion ? "key" : "version", - DEBUG_CF_MISSING_VERSION, - req.range.begin.printable().c_str(), - req.range.end.printable().c_str(), - req.begin, - req.end); - printf("ERROR: %d versions in response %lld - %lld:\n", - reply.mutations.size(), - reply.mutations.front().version, - reply.mutations.back().version); + 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), + foundVersion ? "key" : "version", + DEBUG_CF_MISSING_VERSION, + req.range.begin.printable(), + req.range.end.printable(), + req.begin, + req.end); + fmt::print("ERROR: {0} versions in response {1} - {2}:\n", + reply.mutations.size(), + reply.mutations.front().version, + reply.mutations.back().version); for (auto& it : reply.mutations) { - printf("ERROR: %lld (%d)%s\n", - it.version, - it.mutations.size(), - it.version == DEBUG_CF_MISSING_VERSION ? "<-------" : ""); + fmt::print("ERROR: {0} ({1}){2}\n", + it.version, + it.mutations.size(), + it.version == DEBUG_CF_MISSING_VERSION ? "<-------" : ""); } } else { - printf("DBG: SS %s CF %s SQ %s correct @ %lld from request for [%s - %s) %lld - %lld\n", - data->thisServerID.toString().substr(0, 4).c_str(), - req.rangeID.printable().substr(0, 6).c_str(), - streamUID.toString().substr(0, 8).c_str(), - DEBUG_CF_MISSING_VERSION, - req.range.begin.printable().c_str(), - req.range.end.printable().c_str(), - req.begin, - req.end); + 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), + DEBUG_CF_MISSING_VERSION, + req.range.begin.printable(), + req.range.end.printable(), + req.begin, + req.end); } } @@ -2391,7 +2298,6 @@ ACTOR Future stopChangeFeedOnMove(StorageServer* data, ChangeFeedStreamReq } return Void(); } - printf("CFSQ %s Moved! %lld - %lld. sending WSS\n", streamUID.toString().substr(0, 8).c_str(), req.begin, req.end); // DO NOT call req.reply.onReady before sending - we need to propagate this error through regardless of how far // behind client is req.reply.sendError(wrong_shard_server()); @@ -2406,22 +2312,20 @@ ACTOR Future changeFeedStreamQ(StorageServer* data, ChangeFeedStreamReques if (req.replyBufferSize <= 0) { req.reply.setByteLimit(SERVER_KNOBS->CHANGEFEEDSTREAM_LIMIT_BYTES); } else { - req.reply.setByteLimit(req.replyBufferSize); + req.reply.setByteLimit(std::min((int64_t)req.replyBufferSize, SERVER_KNOBS->CHANGEFEEDSTREAM_LIMIT_BYTES)); } wait(delay(0, TaskPriority::DefaultEndpoint)); try { - if (DEBUG_SS_CFM(data->thisServerID, req.rangeID, streamUID, req.begin)) { - printf("CFM: SS %s CF %s: got CFSQ %s [%s - %s) %lld - %lld, crp=%s\n", - data->thisServerID.toString().substr(0, 4).c_str(), - req.rangeID.printable().substr(0, 6).c_str(), - streamUID.toString().substr(0, 8).c_str(), - req.range.begin.printable().c_str(), - req.range.end.printable().c_str(), - req.begin, - req.end, - req.canReadPopped ? "T" : "F"); + if (DEBUG_CF_TRACE) { + TraceEvent(SevDebug, "TraceChangeFeedMutationsBegin", data->thisServerID) + .detail("FeedID", req.rangeID) + .detail("StreamUID", streamUID) + .detail("Range", req.range) + .detail("Begin", req.begin) + .detail("End", req.end) + .detail("CanReadPopped", req.canReadPopped); } data->activeFeedQueries++; @@ -2434,14 +2338,6 @@ ACTOR Future changeFeedStreamQ(StorageServer* data, ChangeFeedStreamReques ASSERT(emptyInitialReply.minStreamVersion == invalidVersion); req.reply.send(emptyInitialReply); - if (DEBUG_SS_CFM(data->thisServerID, req.rangeID, streamUID, req.begin)) { - printf("CFM: SS %s CF %s: CFSQ %s send empty initial version %lld\n", - data->thisServerID.toString().substr(0, 4).c_str(), - req.rangeID.printable().substr(0, 6).c_str(), - streamUID.toString().substr(0, 8).c_str(), - req.begin - 1); - } - loop { Future onReady = req.reply.onReady(); if (atLatest && !onReady.isReady() && !removeUID) { @@ -2462,14 +2358,6 @@ ACTOR Future changeFeedStreamQ(StorageServer* data, ChangeFeedStreamReques ChangeFeedStreamReply feedReply = _feedReply.first; bool gotAll = _feedReply.second; - // TODO REMOVE debugging - if (feedReply.mutations.size() == 0) { - printf("CFM: SS %s CF %s: CFSQ %s empty results for begin=%lld\n", - data->thisServerID.toString().substr(0, 4).c_str(), - req.rangeID.printable().substr(0, 6).c_str(), - streamUID.toString().substr(0, 8).c_str(), - req.begin); - } ASSERT(feedReply.mutations.size() > 0); req.begin = feedReply.mutations.back().version + 1; if (!atLatest && gotAll) { @@ -2515,7 +2403,6 @@ ACTOR Future changeFeedStreamQ(StorageServer* data, ChangeFeedStreamReques when(wait(feed->second->newMutations.onTrigger())) {} when(wait(req.end == std::numeric_limits::max() ? Future(Never()) : data->version.whenAtLeast(req.end))) {} - when(wait(delay(5.0))) {} // TODO REMOVE this once empty version logic is fully implemented } auto feed = data->uidChangeFeed.find(req.rangeID); if (feed == data->uidChangeFeed.end() || feed->second->removing) { @@ -2529,8 +2416,6 @@ ACTOR Future changeFeedStreamQ(StorageServer* data, ChangeFeedStreamReques } } catch (Error& e) { data->activeFeedQueries--; - // TODO REMOVE - printf("CFSQ %s got error %s\n", streamUID.toString().substr(0, 8).c_str(), e.name()); auto it = data->changeFeedClientVersions.find(req.reply.getEndpoint().getPrimaryAddress()); if (it != data->changeFeedClientVersions.end()) { if (removeUID) { @@ -4750,8 +4635,7 @@ ACTOR Future fetchChangeFeedApplier(StorageServer* data, KeyRange range, Version emptyVersion, Version beginVersion, - Version endVersion, - bool existing) { + Version endVersion) { state Version startVersion = beginVersion; startVersion = std::max(startVersion, emptyVersion + 1); @@ -4760,17 +4644,6 @@ ACTOR Future fetchChangeFeedApplier(StorageServer* data, ASSERT(startVersion >= 0); - // TODO REMOVE - TraceEvent(SevDebug, "FetchChangeFeedStarting", data->thisServerID) - .detail("RangeID", rangeId.printable()) - .detail("Range", range.toString()) - .detail("StartVersion", startVersion) - .detail("EndVersion", endVersion) - .detail("BeginVersion", beginVersion) - .detail("EmptyVersion", emptyVersion) - .detail("FetchVersion", changeFeedInfo->fetchVersion) - .detail("DurableFetchVersion", changeFeedInfo->durableFetchVersion.get()); - if (startVersion >= endVersion) { TEST(true); // Change Feed popped before fetch TraceEvent(SevDebug, "FetchChangeFeedNoOp", data->thisServerID) @@ -4785,7 +4658,6 @@ ACTOR Future fetchChangeFeedApplier(StorageServer* data, state Future feed = data->cx->getChangeFeedStream( feedResults, rangeId, startVersion, endVersion, range, SERVER_KNOBS->CHANGEFEEDSTREAM_LIMIT_BYTES, true); - // TODO remove debugging eventually? state Version firstVersion = invalidVersion; state Version lastVersion = invalidVersion; state int64_t versionsFetched = 0; @@ -4862,7 +4734,6 @@ ACTOR Future fetchChangeFeedApplier(StorageServer* data, for (auto& m : remoteResult[remoteLoc].mutations) { DEBUG_MUTATION("ChangeFeedWriteMove", remoteVersion, m, data->thisServerID) .detail("Range", range) - .detail("Existing", existing) .detail("ChangeFeedID", rangeId); } } @@ -4885,7 +4756,6 @@ ACTOR Future fetchChangeFeedApplier(StorageServer* data, for (auto& m : remoteResult[remoteLoc].mutations) { DEBUG_MUTATION("ChangeFeedWriteMoveIgnore", remoteVersion, m, data->thisServerID) .detail("Range", range) - .detail("Existing", existing) .detail("ChangeFeedID", rangeId) .detail("EmptyVersion", changeFeedInfo->emptyVersion); } @@ -4963,7 +4833,6 @@ ACTOR Future fetchChangeFeedApplier(StorageServer* data, } } - // TODO REMOVE? TraceEvent(SevDebug, "FetchChangeFeedDone", data->thisServerID) .detail("RangeID", rangeId.printable()) .detail("Range", range.toString()) @@ -4973,7 +4842,6 @@ ACTOR Future fetchChangeFeedApplier(StorageServer* data, .detail("FirstFetchedVersion", firstVersion) .detail("LastFetchedVersion", lastVersion) .detail("VersionsFetched", versionsFetched) - .detail("Existing", existing) .detail("Removed", changeFeedInfo->removing); return lastVersion; } @@ -4997,10 +4865,6 @@ ACTOR Future fetchChangeFeed(StorageServer* data, auto cleanupPending = data->changeFeedCleanupDurable.find(changeFeedInfo->id); if (cleanupPending != data->changeFeedCleanupDurable.end()) { - /*printf("SS %s waiting for CF %s cleanup @ %lld\n", - data->thisServerID.toString().substr(0, 4).c_str(), - changeFeedInfo->id.toString().substr(0, 6).c_str(), - cleanupPending->second);*/ TraceEvent(SevDebug, "FetchChangeFeedWaitCleanup", data->thisServerID) .detail("RangeID", changeFeedInfo->id.printable()) .detail("Range", changeFeedInfo->range.toString()) @@ -5025,15 +4889,13 @@ ACTOR Future fetchChangeFeed(StorageServer* data, loop { try { - // TODO clean up existing param for !existing Version maxFetched = wait(fetchChangeFeedApplier(data, changeFeedInfo, changeFeedInfo->id, changeFeedInfo->range, changeFeedInfo->emptyVersion, beginVersion, - endVersion, - false)); + endVersion)); data->fetchingChangeFeeds.insert(changeFeedInfo->id); return maxFetched; } catch (Error& e) { @@ -5109,14 +4971,6 @@ ACTOR Future> fetchChangeFeedMetadata(StorageServer* data, KeyR // reset fetch versions because everything previously fetched was cleaned up changeFeedInfo->fetchVersion = invalidVersion; changeFeedInfo->durableFetchVersion = NotifiedVersion(); - // TODO REMOVE - TraceEvent(SevDebug, "FetchedChangeFeedInfoReset", data->thisServerID) - .detail("RangeID", cfEntry.rangeId.printable()) - .detail("Range", cfEntry.range.toString()) - .detail("FetchVersion", fetchVersion) - .detail("EmptyVersion", cfEntry.emptyVersion) - .detail("CleanupVersion", feedCleanup->second) - .detail("StopVersion", cfEntry.stopVersion); // Since cleanup put a mutation in the log to delete the change feed data, put one in the log to restore // it @@ -5581,25 +5435,6 @@ ACTOR Future fetchKeys(StorageServer* data, AddingShard* shard) { // Wait for the transferred version (and therefore the shard data) to be committed and durable. wait(data->durableVersion.whenAtLeast(feedTransferredVersion)); - // TODO if this works, remove all of the fetch version stuff - // Also wait on all fetched change feed data to become committed and durable - while (!feedFetchedVersions.empty()) { - auto feed = feedFetchedVersions.begin(); - state Key feedId = feed->first; - Version maxFetched = feed->second; - feedFetchedVersions.erase(feed); - - auto feedIt = data->uidChangeFeed.find(feedId); - /*if (feedIt != data->uidChangeFeed.end() && feedIt->second->durableFetchVersion.get() < maxFetched) { - wait(feedIt->second->durableFetchVersion.whenAtLeast(maxFetched)); - // return to updateStorage - wait(delay(0)); - }*/ - if (feedIt != data->uidChangeFeed.end()) { - ASSERT(feedIt->second->durableFetchVersion.get() >= maxFetched); - } - } - ASSERT(data->shards[shard->keys.begin]->assigned() && data->shards[shard->keys.begin]->keys == shard->keys); // We aren't changing whether the shard is assigned @@ -5860,8 +5695,6 @@ void changeServerKeys(StorageServer* data, } if (!foundAssigned) { - // TODO REMOVE - Version durableVersion = data->data().getLatestVersion(); TraceEvent(SevDebug, "ChangeFeedCleanup", data->thisServerID) .detail("FeedID", f.first) @@ -5880,19 +5713,6 @@ void changeServerKeys(StorageServer* data, changeFeedDurableKey(f.first, 0), changeFeedDurableKey(f.first, version))); - // do this cleanup later!! - /*auto rs = data->keyChangeFeed.modify(f.second); - for (auto r = rs.begin(); r != rs.end(); ++r) { - auto& feedList = r->value(); - for (int i = 0; i < feedList.size(); i++) { - if (feedList[i]->id == f.first) { - swapAndPop(&feedList, i--); - } - } - } - data->keyChangeFeed.coalesce(f.second.contents()); - */ - // We can't actually remove this change feed fully until the mutations clearing its data become durable. // If the SS restarted at version R before the clearing mutations became durable at version D (R < D), // then the restarted SS would restore the change feed clients would be able to read data and would miss @@ -5904,7 +5724,6 @@ void changeServerKeys(StorageServer* data, feed->second->removing = true; feed->second->moved(feed->second->range); feed->second->newMutations.trigger(); - // data->uidChangeFeed.erase(feed); } } else { // if just part of feed's range is moved away @@ -6129,7 +5948,6 @@ private: std::tie(changeFeedRange, popVersion, status) = decodeChangeFeedValue(m.param2); auto feed = data->uidChangeFeed.find(changeFeedId); - // TODO REMOVE eventually TraceEvent(SevDebug, "ChangeFeedPrivateMutation", data->thisServerID) .detail("RangeID", changeFeedId.printable()) .detail("Range", changeFeedRange.toString()) @@ -6878,13 +6696,6 @@ ACTOR Future updateStorage(StorageServer* data) { } if (info->second->fetchVersion != invalidVersion && !info->second->removing) { - // TODO REMOVE trace - TraceEvent(SevDebug, "UpdateStorageChangeFeedStart", data->thisServerID) - .detail("RangeID", info->second->id.printable()) - .detail("Range", info->second->range.toString()) - .detail("FetchVersion", info->second->fetchVersion) - .detail("StopVersion", info->second->stopVersion) - .detail("Removing", info->second->removing); feedFetchVersions.push_back(std::pair(info->second->id, info->second->fetchVersion)); } // handle case where fetch had version ahead of last in-memory mutation @@ -6958,15 +6769,6 @@ ACTOR Future updateStorage(StorageServer* data) { // Don't update if the feed is pending cleanup. Either it will get cleaned up and destroyed, or it will get // fetched again, where the fetch version will get reset. if (info != data->uidChangeFeed.end() && !data->changeFeedCleanupDurable.count(info->second->id)) { - // TODO REMOVE trace - TraceEvent(SevDebug, "UpdateStorageChangeFeedDurable", data->thisServerID) - .detail("RangeID", info->second->id.printable()) - .detail("Range", info->second->range.toString()) - .detail("FetchVersion", info->second->fetchVersion) - .detail("OldDurableVersion", info->second->durableFetchVersion.get()) - .detail("NewDurableVersion", feedFetchVersions[curFeed].second) - .detail("StopVersion", info->second->stopVersion) - .detail("Removing", info->second->removing); if (feedFetchVersions[curFeed].second > info->second->durableFetchVersion.get()) { info->second->durableFetchVersion.set(feedFetchVersions[curFeed].second); } @@ -6986,11 +6788,6 @@ ACTOR Future updateStorage(StorageServer* data) { auto feed = data->uidChangeFeed.find(cfCleanup->first); ASSERT(feed != data->uidChangeFeed.end()); if (feed->second->removing) { - // TODO REMOVE - /*printf("DBG: SS %s Feed %s removing metadata @ %lld!\n", - data->thisServerID.toString().substr(0, 4).c_str(), - feed->first.printable().substr(0, 6).c_str(), - cfCleanup->second);*/ auto rs = data->keyChangeFeed.modify(feed->second->range); for (auto r = rs.begin(); r != rs.end(); ++r) { auto& feedList = r->value(); @@ -7002,27 +6799,10 @@ ACTOR Future updateStorage(StorageServer* data) { } data->keyChangeFeed.coalesce(feed->second->range.contents()); - // TODO REMOVE - TraceEvent(SevDebug, "UpdateStorageChangeFeedCleanup", data->thisServerID) - .detail("RangeID", feed->second->id.printable()) - .detail("Range", feed->second->range.toString()); - data->uidChangeFeed.erase(feed); } else { TEST(true); // Feed re-fetched after remove - // TODO REMOVE - /*printf("DBG: SS %s Feed %s not removing metadata @ %lld, must have been re-fetched after moved " - "away!\n", - data->thisServerID.toString().substr(0, 4).c_str(), - feed->first.printable().substr(0, 6).c_str(), - cfCleanup->second);*/ } - - // TODO REMOVE - /*printf("DBG: SS %s Feed %s removing cleanup entry @ %lld!\n", - data->thisServerID.toString().substr(0, 4).c_str(), - feed->first.printable().substr(0, 6).c_str(), - cfCleanup->second);*/ cfCleanup = data->changeFeedCleanupDurable.erase(cfCleanup); } else { cfCleanup++; From 1f964ac0854e37b9f757030a14e7410f7ec2629a Mon Sep 17 00:00:00 2001 From: Josh Slocum Date: Wed, 9 Mar 2022 11:44:45 -0600 Subject: [PATCH 307/413] BM focused cleanup --- fdbclient/NativeAPI.actor.cpp | 52 +-- fdbclient/ServerKnobs.cpp | 7 +- fdbclient/ServerKnobs.h | 5 + fdbserver/BlobManager.actor.cpp | 302 ++++++++---------- fdbserver/BlobWorker.actor.cpp | 16 +- .../BlobGranuleCorrectnessWorkload.actor.cpp | 31 +- 6 files changed, 193 insertions(+), 220 deletions(-) diff --git a/fdbclient/NativeAPI.actor.cpp b/fdbclient/NativeAPI.actor.cpp index 1963fb908c..bb0d0143fc 100644 --- a/fdbclient/NativeAPI.actor.cpp +++ b/fdbclient/NativeAPI.actor.cpp @@ -7010,6 +7010,32 @@ ACTOR Future setPerpetualStorageWiggle(Database cx, bool enable, LockAware return Void(); } +ACTOR Future>> readStorageWiggleValues(Database cx, + bool primary, + bool use_system_priority) { + state const Key readKey = perpetualStorageWiggleIDPrefix.withSuffix(primary ? "primary/"_sr : "remote/"_sr); + state KeyBackedObjectMap metadataMap(readKey, + IncludeVersion()); + state Reference tr(new ReadYourWritesTransaction(cx)); + state std::vector> res; + // read the wiggling pairs + loop { + try { + tr->setOption(FDBTransactionOptions::READ_SYSTEM_KEYS); + tr->setOption(FDBTransactionOptions::READ_LOCK_AWARE); + if (use_system_priority) { + tr->setOption(FDBTransactionOptions::PRIORITY_SYSTEM_IMMEDIATE); + } + wait(store(res, metadataMap.getRange(tr, UID(0, 0), Optional(), CLIENT_KNOBS->TOO_MANY))); + wait(tr->commit()); + break; + } catch (Error& e) { + wait(tr->onError(e)); + } + } + return res; +} + ACTOR Future splitStorageMetricsStream(PromiseStream resultStream, Database cx, KeyRange keys, @@ -7111,32 +7137,6 @@ Future DatabaseContext::splitStorageMetricsStream(const PromiseStream resultStream, Database(Reference::addRef(this)), keys, limit, estimated); } -ACTOR Future>> readStorageWiggleValues(Database cx, - bool primary, - bool use_system_priority) { - state const Key readKey = perpetualStorageWiggleIDPrefix.withSuffix(primary ? "primary/"_sr : "remote/"_sr); - state KeyBackedObjectMap metadataMap(readKey, - IncludeVersion()); - state Reference tr(new ReadYourWritesTransaction(cx)); - state std::vector> res; - // read the wiggling pairs - loop { - try { - tr->setOption(FDBTransactionOptions::READ_SYSTEM_KEYS); - tr->setOption(FDBTransactionOptions::READ_LOCK_AWARE); - if (use_system_priority) { - tr->setOption(FDBTransactionOptions::PRIORITY_SYSTEM_IMMEDIATE); - } - wait(store(res, metadataMap.getRange(tr, UID(0, 0), Optional(), CLIENT_KNOBS->TOO_MANY))); - wait(tr->commit()); - break; - } catch (Error& e) { - wait(tr->onError(e)); - } - } - return res; -} - ACTOR Future>> splitStorageMetrics(Database cx, KeyRange keys, StorageMetrics limit, diff --git a/fdbclient/ServerKnobs.cpp b/fdbclient/ServerKnobs.cpp index 58d678a310..e8ddd6b7d9 100644 --- a/fdbclient/ServerKnobs.cpp +++ b/fdbclient/ServerKnobs.cpp @@ -821,7 +821,7 @@ void ServerKnobs::initialize(Randomize randomize, ClientKnobs* clientKnobs, IsSi init( ENABLE_ENCRYPT_KEY_PROXY, false ); // Blob granlues - init( BG_URL, isSimulated ? "file://fdbblob/" : "" ); // TODO: store in system key space, eventually + init( BG_URL, isSimulated ? "file://fdbblob/" : "" ); // TODO: store in system key space or something, eventually init( BG_SNAPSHOT_FILE_TARGET_BYTES, 10000000 ); if( buggifySmallShards || (randomize && BUGGIFY) ) { deterministicRandom()->random01() < 0.1 ? BG_SNAPSHOT_FILE_TARGET_BYTES /= 100 : BG_SNAPSHOT_FILE_TARGET_BYTES /= 10; } init( BG_DELTA_BYTES_BEFORE_COMPACT, BG_SNAPSHOT_FILE_TARGET_BYTES/2 ); init( BG_DELTA_FILE_TARGET_BYTES, BG_DELTA_BYTES_BEFORE_COMPACT/10 ); @@ -829,6 +829,11 @@ void ServerKnobs::initialize(Randomize randomize, ClientKnobs* clientKnobs, IsSi 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 ); + init( BG_MAX_SPLIT_FANOUT, 10 ); if( randomize && BUGGIFY ) BLOB_WORKER_REQUEST_TIMEOUT = deterministicRandom()->randomInt(5, 15); + + init( BLOB_MANAGER_STATUS_EXP_BACKOFF_MIN, 0.1 ); + init( BLOB_MANAGER_STATUS_EXP_BACKOFF_MAX, 5.0 ); + init( BLOB_MANAGER_STATUS_EXP_BACKOFF_EXPONENT, 1.5 ); // clang-format on diff --git a/fdbclient/ServerKnobs.h b/fdbclient/ServerKnobs.h index 4a783da91d..0a0023719c 100644 --- a/fdbclient/ServerKnobs.h +++ b/fdbclient/ServerKnobs.h @@ -780,6 +780,11 @@ public: 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 double BLOB_WORKERLIST_FETCH_INTERVAL; + int BG_MAX_SPLIT_FANOUT; // Decreasing this knob can be unsafe + + double BLOB_MANAGER_STATUS_EXP_BACKOFF_MIN; + double BLOB_MANAGER_STATUS_EXP_BACKOFF_MAX; + double BLOB_MANAGER_STATUS_EXP_BACKOFF_EXPONENT; ServerKnobs(Randomize, ClientKnobs*, IsSimulated); void initialize(Randomize, ClientKnobs*, IsSimulated); diff --git a/fdbserver/BlobManager.actor.cpp b/fdbserver/BlobManager.actor.cpp index 7c16df3c11..4f94391dd2 100644 --- a/fdbserver/BlobManager.actor.cpp +++ b/fdbserver/BlobManager.actor.cpp @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. @@ -43,9 +43,12 @@ #include "flow/UnitTest.h" #include "flow/actorcompiler.h" // has to be last include +/* + * The Blob Manager is responsible for managing range granules, and recruiting and monitoring Blob Workers. + */ + #define BM_DEBUG false -// TODO add comments + documentation void handleClientBlobRange(KeyRangeMap* knownBlobRanges, Arena& ar, VectorRef* rangesToAdd, @@ -103,7 +106,7 @@ void updateClientBlobRanges(KeyRangeMap* knownBlobRanges, // worker. for any range that isn't set in results that is set in ranges, revoke the range from the // worker. and, update ranges to match results as you go - // FIXME: could change this to O(N) instead of O(NLogN) by doing a sorted merge instead of requesting the + // SOMEDAY: could change this to O(N) instead of O(NLogN) by doing a sorted merge instead of requesting the // intersection for each insert, but this operation is pretty infrequent so it's probably not necessary if (dbBlobRanges.size() == 0) { // special case. Nothing in the DB, reset knownBlobRanges and revoke all existing ranges from workers @@ -191,7 +194,7 @@ struct RangeAssignment { Optional revoke; }; -// TODO: track worker's reads/writes eventually +// SOMEDAY: track worker's reads/writes eventually struct BlobWorkerStats { int numGranulesAssigned; @@ -246,87 +249,82 @@ struct BlobManagerData : NonCopyable, ReferenceCounted { BlobManagerData(UID id, Database db, Optional dcId) : id(id), db(db), dcId(dcId), knownBlobRanges(false, normalKeys.end), restartRecruiting(SERVER_KNOBS->DEBOUNCE_RECRUITING_DELAY), recruitingStream(0) {} - - // TODO REMOVE - ~BlobManagerData() { - if (BM_DEBUG) { - fmt::print("Destroying blob manager data for {0} {1}\n", epoch, id.toString()); - } - } }; -ACTOR Future>> splitRange(Reference tr, +ACTOR Future>> splitRange(Reference bmData, KeyRange range, bool writeHot) { - // TODO is it better to just pass empty metrics to estimated? - // redo split if previous txn failed to calculate it - loop { - try { - if (BM_DEBUG) { - fmt::print("Splitting new range [{0} - {1}): {2}\n", - range.begin.printable(), - range.end.printable(), - writeHot ? "hot" : "normal"); - } - state StorageMetrics estimated = - wait(tr->getTransaction().getDatabase()->getStorageMetrics(range, CLIENT_KNOBS->TOO_MANY)); - - if (BM_DEBUG) { - fmt::print("Estimated bytes for [{0} - {1}): {2}\n", - range.begin.printable(), - range.end.printable(), - estimated.bytes); - } - - if (estimated.bytes > SERVER_KNOBS->BG_SNAPSHOT_FILE_TARGET_BYTES || writeHot) { - // only split on bytes and write rate - state StorageMetrics splitMetrics; - splitMetrics.bytes = SERVER_KNOBS->BG_SNAPSHOT_FILE_TARGET_BYTES; - splitMetrics.bytesPerKSecond = SERVER_KNOBS->SHARD_SPLIT_BYTES_PER_KSEC; - if (writeHot) { - splitMetrics.bytesPerKSecond = - std::min(splitMetrics.bytesPerKSecond, estimated.bytesPerKSecond / 2); - splitMetrics.bytesPerKSecond = - std::max(splitMetrics.bytesPerKSecond, SERVER_KNOBS->SHARD_MIN_BYTES_PER_KSEC); - } - splitMetrics.iosPerKSecond = splitMetrics.infinity; - splitMetrics.bytesReadPerKSecond = splitMetrics.infinity; - - state PromiseStream resultStream; - state Standalone> keys; - state Future streamFuture = tr->getTransaction().getDatabase()->splitStorageMetricsStream( - resultStream, range, splitMetrics, estimated); - loop { - try { - Key k = waitNext(resultStream.getFuture()); - keys.push_back_deep(keys.arena(), k); - } catch (Error& e) { - if (e.code() != error_code_end_of_stream) { - throw; - } - break; - } - } - - ASSERT(keys.size() >= 2); - ASSERT(keys.front() == range.begin); - ASSERT(keys.back() == range.end); - return keys; - } else { - if (BM_DEBUG) { - printf("Not splitting range\n"); - } - Standalone> keys; - keys.push_back_deep(keys.arena(), range.begin); - keys.push_back_deep(keys.arena(), range.end); - return keys; - } - } catch (Error& e) { - if (BM_DEBUG) { - printf("Splitting range got error %s\n", e.name()); - } - wait(tr->onError(e)); + try { + if (BM_DEBUG) { + fmt::print("Splitting new range [{0} - {1}): {2}\n", + range.begin.printable(), + range.end.printable(), + writeHot ? "hot" : "normal"); } + state StorageMetrics estimated = wait(bmData->db->getStorageMetrics(range, CLIENT_KNOBS->TOO_MANY)); + + if (BM_DEBUG) { + fmt::print("Estimated bytes for [{0} - {1}): {2}\n", + range.begin.printable(), + range.end.printable(), + estimated.bytes); + } + + if (estimated.bytes > SERVER_KNOBS->BG_SNAPSHOT_FILE_TARGET_BYTES || writeHot) { + // only split on bytes and write rate + state StorageMetrics splitMetrics; + splitMetrics.bytes = SERVER_KNOBS->BG_SNAPSHOT_FILE_TARGET_BYTES; + splitMetrics.bytesPerKSecond = SERVER_KNOBS->SHARD_SPLIT_BYTES_PER_KSEC; + if (writeHot) { + splitMetrics.bytesPerKSecond = std::min(splitMetrics.bytesPerKSecond, estimated.bytesPerKSecond / 2); + splitMetrics.bytesPerKSecond = + std::max(splitMetrics.bytesPerKSecond, SERVER_KNOBS->SHARD_MIN_BYTES_PER_KSEC); + } + splitMetrics.iosPerKSecond = splitMetrics.infinity; + splitMetrics.bytesReadPerKSecond = splitMetrics.infinity; + + state PromiseStream resultStream; + state Standalone> keys; + state Future streamFuture = + bmData->db->splitStorageMetricsStream(resultStream, range, splitMetrics, estimated); + loop { + try { + Key k = waitNext(resultStream.getFuture()); + keys.push_back_deep(keys.arena(), k); + } catch (Error& e) { + if (e.code() != error_code_end_of_stream) { + throw; + } + break; + } + } + + ASSERT(keys.size() >= 2); + ASSERT(keys.front() == range.begin); + ASSERT(keys.back() == range.end); + return keys; + } else { + if (BM_DEBUG) { + printf("Not splitting range\n"); + } + Standalone> keys; + keys.push_back_deep(keys.arena(), range.begin); + keys.push_back_deep(keys.arena(), range.end); + return keys; + } + } catch (Error& e) { + if (e.code() == error_code_operation_cancelled) { + throw e; + } + // SplitStorageMetrics explicitly has a SevError if it gets an error, so no errors should propagate here + TraceEvent(SevError, "BlobManagerUnexpectedErrorSplitRange", bmData->id) + .error(e) + .detail("Epoch", bmData->epoch); + ASSERT_WE_THINK(false); + + // if not simulation, kill the BM + bmData->iAmReplaced.sendError(e); + throw e; } } @@ -335,7 +333,6 @@ ACTOR Future>> splitRange(Reference pickWorkerForAssign(Reference bmData) { // wait until there are BWs to pick from while (bmData->workerStats.size() == 0) { - // TODO REMOVE if (BM_DEBUG) { fmt::print("BM {0} waiting for blob workers before assigning granules\n", bmData->epoch); } @@ -450,8 +447,17 @@ ACTOR Future doRangeAssignment(Reference bmData, return Void(); } - // TODO confirm: using reliable delivery this should only trigger if the worker is marked as failed, right? - // So assignment needs to be retried elsewhere, and a revoke is trivially complete + if (e.code() != error_code_broken_promise && e.code() != error_code_no_more_servers) { + TraceEvent(SevWarn, "BlobManagerUnexpectedErrorDoRangeAssignment", bmData->id) + .error(e) + .detail("Epoch", bmData->epoch); + ASSERT_WE_THINK(false); + bmData->iAmReplaced.sendError(e); + throw; + } + + // We use reliable delivery (getReply), so the broken_promise means the worker is dead, and we may need to retry + // somewhere else if (assignment.isAssign) { if (BM_DEBUG) { fmt::print("BM got error {0} assigning range [{1} - {2}) to worker {3}, requeueing\n", @@ -567,8 +573,6 @@ ACTOR Future rangeAssigner(Reference bmData) { bmData->workerStats[workerId].numGranulesAssigned += 1; } - // FIXME: if range is assign, have some sort of semaphore for outstanding assignments so we don't assign - // a ton ranges at once and blow up FDB with reading initial snapshots. bmData->assignsInProgress.insert(assignment.keyRange, doRangeAssignment(bmData, assignment, workerId, seqNo)); } else { @@ -647,13 +651,6 @@ ACTOR Future writeInitialGranuleMapping(Reference bmData, tr->setOption(FDBTransactionOptions::Option::ACCESS_SYSTEM_KEYS); wait(checkManagerLock(tr, bmData)); while (i + j < boundaries.size() - 1 && j < transactionChunkSize) { - // TODO REMOVE - if (BM_DEBUG) { - fmt::print("BM {0} Persisting initial mapping for [{1} - {2})\n", - bmData->epoch, - boundaries[i + j].printable(), - boundaries[i + j + 1].printable()); - } // set to empty UID - no worker assigned yet wait(krmSetRange(tr, blobGranuleMappingKeys.begin, @@ -662,19 +659,8 @@ ACTOR Future writeInitialGranuleMapping(Reference bmData, j++; } wait(tr->commit()); - if (BM_DEBUG) { - for (int k = 0; k < j; k++) { - fmt::print("BM {0} Persisted initial mapping for [{1} - {2})\n", - bmData->epoch, - boundaries[i + k].printable(), - boundaries[i + k + 1].printable()); - } - } break; } catch (Error& e) { - if (BM_DEBUG) { - fmt::print("BM {} Persisting initial mapping got error {}\n", bmData->epoch, e.name()); - } wait(tr->onError(e)); j = 0; } @@ -684,8 +670,6 @@ ACTOR Future writeInitialGranuleMapping(Reference bmData, return Void(); } -// FIXME: this does all logic in one transaction. Adding a giant range to an existing database to blobify would -// require doing a ton of storage metrics calls, which we should split up across multiple transactions likely. ACTOR Future monitorClientRanges(Reference bmData) { state Optional lastChangeKeyValue; state bool needToCoalesce = bmData->epoch > 1; @@ -703,10 +687,20 @@ ACTOR Future monitorClientRanges(Reference bmData) { // read change key at this point along with ranges state Optional ckvBegin = wait(tr->get(blobRangeChangeKey)); - // TODO probably knobs here? This should always be pretty small though - RangeResult results = wait(krmGetRanges( - tr, blobRangeKeys.begin, KeyRange(normalKeys), 10000, GetRangeLimits::BYTE_LIMIT_UNLIMITED)); - ASSERT(!results.more && results.size() < CLIENT_KNOBS->TOO_MANY); + state RangeResult results = wait(krmGetRanges(tr, + blobRangeKeys.begin, + KeyRange(normalKeys), + CLIENT_KNOBS->TOO_MANY, + GetRangeLimits::BYTE_LIMIT_UNLIMITED)); + ASSERT_WE_THINK(!results.more && results.size() < CLIENT_KNOBS->TOO_MANY); + if (results.more || results.size() >= CLIENT_KNOBS->TOO_MANY) { + TraceEvent(SevError, "BlobManagerTooManyClientRanges", bmData->id) + .detail("Epoch", bmData->epoch) + .detail("ClientRanges", results.size() - 1); + wait(delay(600)); + bmData->iAmReplaced.sendError(internal_error()); + throw internal_error(); + } state Arena ar; ar.dependsOn(results.arena()); @@ -743,7 +737,7 @@ ACTOR Future monitorClientRanges(Reference bmData) { state std::vector>>> splitFutures; // Divide new ranges up into equal chunks by using SS byte sample for (KeyRangeRef range : rangesToAdd) { - splitFutures.push_back(splitRange(tr, range, false)); + splitFutures.push_back(splitRange(bmData, range, false)); } for (auto f : splitFutures) { @@ -847,7 +841,7 @@ ACTOR Future maybeSplitRange(Reference bmData, state int64_t newLockSeqno = -1; // first get ranges to split - Standalone> _newRanges = wait(splitRange(tr, granuleRange, writeHot)); + Standalone> _newRanges = wait(splitRange(bmData, granuleRange, writeHot)); newRanges = _newRanges; ASSERT(newRanges.size() >= 2); @@ -868,10 +862,9 @@ ACTOR Future maybeSplitRange(Reference bmData, return Void(); } - // TODO KNOB for this. - // Enforce max split fanout of 10 for performance reasons - int maxSplitFanout = 10; - if (newRanges.size() >= maxSplitFanout + 2) { // +2 because this is boundaries, so N keys would have N+1 bounaries. + // Enforce max split fanout for performance reasons. This mainly happens when a blob worker is behind. + if (newRanges.size() >= + SERVER_KNOBS->BG_MAX_SPLIT_FANOUT + 2) { // +2 because this is boundaries, so N keys would have N+1 bounaries. TEST(true); // downsampling granule split because fanout too high Standalone> coalescedRanges; coalescedRanges.arena().dependsOn(newRanges.arena()); @@ -879,15 +872,17 @@ ACTOR Future maybeSplitRange(Reference bmData, // since we include start + end boundaries here, only need maxSplitFanout-1 split boundaries to produce // maxSplitFanout granules - downsampleSplit(newRanges, coalescedRanges, 1, newRanges.size() - 1, maxSplitFanout - 1); + downsampleSplit(newRanges, coalescedRanges, 1, newRanges.size() - 1, SERVER_KNOBS->BG_MAX_SPLIT_FANOUT - 1); coalescedRanges.push_back(coalescedRanges.arena(), newRanges.back()); - ASSERT(coalescedRanges.size() == maxSplitFanout + 1); + ASSERT(coalescedRanges.size() == SERVER_KNOBS->BG_MAX_SPLIT_FANOUT + 1); if (BM_DEBUG) { - fmt::print("Downsampled split from {0} -> {1} granules", newRanges.size() - 1, maxSplitFanout); + fmt::print( + "Downsampled split from {0} -> {1} granules", newRanges.size() - 1, SERVER_KNOBS->BG_MAX_SPLIT_FANOUT); } newRanges = coalescedRanges; + ASSERT(newRanges.size() <= SERVER_KNOBS->BG_MAX_SPLIT_FANOUT + 1); } if (BM_DEBUG) { @@ -971,20 +966,9 @@ ACTOR Future maybeSplitRange(Reference bmData, // acquire granule lock so nobody else can make changes to this granule. tr->set(lockKey, blobGranuleLockValueFor(bmData->epoch, newLockSeqno, std::get<2>(prevGranuleLock))); - // set up split metadata - /*fmt::print("Persisting granule split {0} [{1} - {2})\n", - granuleID.toString().substr(0, 6), - granuleRange.begin.printable(), - granuleRange.end.printable());*/ - // set up splits in granule mapping, but point each part to the old owner (until they get reassigned) state int i; for (i = 0; i < newRanges.size() - 1; i++) { - /*fmt::print(" {0} [{1} - {2})\n", - newGranuleIDs[i].toString().substr(0, 6), - newRanges[i].printable(), - newRanges[i + 1].printable());*/ - Key splitKey = blobGranuleSplitKeyFor(granuleID, newGranuleIDs[i]); tr->atomicOp(splitKey, @@ -998,11 +982,6 @@ ACTOR Future maybeSplitRange(Reference bmData, historyValue.parentGranules.push_back(historyValue.arena(), std::pair(granuleRange, granuleStartVersion)); - /*fmt::print("Creating history entry [{0} - {1}) - [{2} - {3})\n", - newRanges[i].printable(), - newRanges[i + 1].printable(), - granuleStartVersion, - latestVersion);*/ tr->set(historyKey, blobGranuleHistoryValueFor(historyValue)); // split the assignment but still pointing to the same worker @@ -1090,11 +1069,6 @@ ACTOR Future haltBlobWorker(Reference bmData, BlobWorkerI if (e.code() == error_code_operation_cancelled) { throw; } - // TODO REMOVE - fmt::print("BM {0} got error {1} trying to halt blob worker {2}\n", - bmData->epoch, - e.name(), - bwInterf.id().toString()); if (e.code() != error_code_blob_manager_replaced) { break; } @@ -1185,8 +1159,7 @@ ACTOR Future monitorBlobWorkerStatus(Reference bmData, Bl // wait for blob manager to be done recovering, so it has initial granule mapping and worker data wait(bmData->doneRecovering.getFuture()); - // TODO knob? - state double backoff = 0.1; + state double backoff = SERVER_KNOBS->BLOB_MANAGER_STATUS_EXP_BACKOFF_MIN; loop { try { state ReplyPromiseStream statusStream = @@ -1207,7 +1180,7 @@ ACTOR Future monitorBlobWorkerStatus(Reference bmData, Bl rep.writeHotSplit ? "hot" : "normal"); } // if we get a reply from the stream, reset backoff - backoff = 0.1; + backoff = SERVER_KNOBS->BLOB_MANAGER_STATUS_EXP_BACKOFF_MIN; if (rep.epoch > bmData->epoch) { if (BM_DEBUG) { fmt::print("BM heard from BW {0} that there is a new manager with higher epoch\n", @@ -1218,7 +1191,7 @@ ACTOR Future monitorBlobWorkerStatus(Reference bmData, Bl } } - // TODO maybe this won't be true eventually, but right now the only time the blob worker reports + // This won't be true eventually, but right now the only time the blob worker reports // back is to split the range. ASSERT(rep.doSplit); @@ -1316,17 +1289,16 @@ ACTOR Future monitorBlobWorkerStatus(Reference bmData, Bl ASSERT(e.code() != error_code_end_of_stream); if (e.code() == error_code_request_maybe_delivered || e.code() == error_code_connection_failed) { wait(delay(backoff)); - backoff = std::min(backoff * 1.5, 5.0); + backoff = std::min(backoff * SERVER_KNOBS->BLOB_MANAGER_STATUS_EXP_BACKOFF_EXPONENT, + SERVER_KNOBS->BLOB_MANAGER_STATUS_EXP_BACKOFF_MAX); continue; } else { - if (BM_DEBUG) { - fmt::print( - "BM got unexpected error {0} monitoring BW {1} status\n", e.name(), bwInterf.id().toString()); - } - // TODO change back from SevError? - TraceEvent(SevError, "BWStatusMonitoringFailed", bmData->id) + TraceEvent(SevError, "BlobManagerUnexpectedErrorStatusMonitoring", bmData->id) .error(e) - .detail("BlobWorkerID", bwInterf.id()); + .detail("Epoch", bmData->epoch); + ASSERT_WE_THINK(false); + // if not simulation, kill the BM + bmData->iAmReplaced.sendError(e); throw e; } } @@ -1361,14 +1333,17 @@ ACTOR Future monitorBlobWorker(Reference bmData, BlobWork "BM {0} got monitoring error {1} from BW {2}\n", bmData->epoch, e.name(), bwInterf.id().toString()); } - // TODO: re-evaluate the expected errors here once wait failure issue is resolved // Expected errors here are: [broken_promise] if (e.code() != error_code_broken_promise) { if (BM_DEBUG) { fmt::print("BM got unexpected error {0} monitoring BW {1}\n", e.name(), bwInterf.id().toString()); } - // TODO change back from SevError? - TraceEvent(SevError, "BWMonitoringFailed", bmData->id).error(e).detail("BlobWorkerID", bwInterf.id()); + TraceEvent(SevError, "BlobManagerUnexpectedErrorMonitorBW", bmData->id) + .error(e) + .detail("Epoch", bmData->epoch); + ASSERT_WE_THINK(false); + // if not simulation, kill the BM + bmData->iAmReplaced.sendError(e); throw e; } } @@ -1536,7 +1511,7 @@ ACTOR Future recoverBlobManager(Reference bmData) { workerAssignments.insert(normalKeys, std::tuple(UID(), 0, 0)); state Reference tr = makeReference(bmData->db); - // TODO KNOB + // FIXME: use range stream instead state int rowLimit = BUGGIFY ? deterministicRandom()->randomInt(2, 10) : 10000; if (BM_DEBUG) { @@ -1559,13 +1534,11 @@ ACTOR Future recoverBlobManager(Reference bmData) { aliveAssignments.push_back(timeout(brokenPromiseToNever(it.granuleAssignmentsRequest.getReply(req)), SERVER_KNOBS->BLOB_WORKER_TIMEOUT)); } - waitForAll(aliveAssignments); state std::vector> outOfDateAssignments; state int successful = 0; state int assignIdx = 0; - // FIXME: more CPU efficient to do sorted merge of assignments? for (; assignIdx < aliveAssignments.size(); assignIdx++) { Optional reply = wait(aliveAssignments[assignIdx]); UID workerId = startingWorkers[assignIdx].id(); @@ -1595,7 +1568,7 @@ ACTOR Future recoverBlobManager(Reference bmData) { bmData->workerStats[workerId].numGranulesAssigned = reply.get().assignments.size(); } } else { - // TODO mark as failed and kill it + // SOMEDAY: mark as failed and kill it if (BM_DEBUG) { fmt::print(" Worker {}: failed\n", workerId.toString().substr(0, 5)); } @@ -1621,7 +1594,6 @@ ACTOR Future recoverBlobManager(Reference bmData) { tr->setOption(FDBTransactionOptions::ACCESS_SYSTEM_KEYS); tr->setOption(FDBTransactionOptions::PRIORITY_SYSTEM_IMMEDIATE); - // TODO: replace row limit with knob KeyRange nextRange(KeyRangeRef(beginKey, blobGranuleMappingKeys.end)); // using the krm functions can produce incorrect behavior here as it does weird stuff with beginKey state GetRangeLimits limits(rowLimit, GetRangeLimits::BYTE_LIMIT_UNLIMITED); @@ -1630,9 +1602,6 @@ ACTOR Future recoverBlobManager(Reference bmData) { // Add the mappings to our in memory key range map for (int rangeIdx = 0; rangeIdx < results.size() - 1; rangeIdx++) { - // TODO REMOVE asserts eventually - ASSERT(results[rangeIdx].key.startsWith(blobGranuleMappingKeys.begin)); - ASSERT(results[rangeIdx + 1].key.startsWith(blobGranuleMappingKeys.begin)); Key granuleStartKey = results[rangeIdx].key.removePrefix(blobGranuleMappingKeys.begin); Key granuleEndKey = results[rangeIdx + 1].key.removePrefix(blobGranuleMappingKeys.begin); if (results[rangeIdx].value.size()) { @@ -1726,9 +1695,6 @@ ACTOR Future recoverBlobManager(Reference bmData) { int64_t epoch = std::get<1>(range.value()); int64_t seqno = std::get<2>(range.value()); if (epoch == 0 && seqno == 0) { - /*if (BM_DEBUG) { - fmt::print(" [{0} - {1}) invalid\n", range.begin().printable(), range.end().printable()); - }*/ continue; } @@ -1983,7 +1949,6 @@ ACTOR Future haltBlobGranules(Reference bmData) { std::vector blobWorkers = wait(getBlobWorkers(bmData->db)); std::vector> deregisterBlobWorkers; for (auto& worker : blobWorkers) { - // TODO: send a special req to blob workers so they clean up granules/CFs bmData->addActor.send(haltBlobWorker(bmData, worker)); deregisterBlobWorkers.emplace_back(deregisterBlobWorker(bmData, worker)); } @@ -2010,7 +1975,7 @@ ACTOR Future loadHistoryFiles(Reference bmData, U /* * Deletes all files pertaining to the granule with id granuleId and * also removes the history entry for this granule from the system keyspace - * TODO ensure cannot fully delete granule that is still splitting! + * TODO: ensure cannot fully delete granule that is still splitting! */ ACTOR Future fullyDeleteGranule(Reference self, UID granuleId, KeyRef historyKey) { if (BM_DEBUG) { @@ -2194,7 +2159,6 @@ ACTOR Future pruneRange(Reference self, KeyRangeRef range } // queue of for BFS traversal of history - // TODO: consider using GranuleHistoryEntry, but that also makes it a little messy state std::queue> historyEntryQueue; // stacks of and to track which granules to delete @@ -2569,7 +2533,6 @@ ACTOR Future blobManagerExclusionSafetyCheck(Reference se TraceEvent("BMExclusionSafetyCheckNoWorkers", self->id).log(); reply.safe = false; } else { - // TODO REMOVE prints std::set remainingWorkers; for (auto& worker : self->workersById) { remainingWorkers.insert(worker.first); @@ -2627,7 +2590,6 @@ ACTOR Future blobManager(BlobManagerInterface bmInterf, self->addActor.send(chaosRangeMover(self)); } - // TODO probably other things here eventually try { loop choose { when(wait(self->iAmReplaced.getFuture())) { diff --git a/fdbserver/BlobWorker.actor.cpp b/fdbserver/BlobWorker.actor.cpp index f8afcf0e73..2197885b6b 100644 --- a/fdbserver/BlobWorker.actor.cpp +++ b/fdbserver/BlobWorker.actor.cpp @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. @@ -59,7 +59,7 @@ struct GranuleStartState { Optional history; }; -// TODO add global byte limit for pending and buffered deltas +// FIXME: add global byte limit for pending and buffered deltas struct GranuleMetadata : NonCopyable, ReferenceCounted { KeyRange keyRange; @@ -103,7 +103,6 @@ struct GranuleMetadata : NonCopyable, ReferenceCounted { } }; -// TODO: rename this struct struct GranuleRangeMetadata { int64_t lastEpoch; int64_t lastSeqno; @@ -149,6 +148,7 @@ struct GranuleHistoryEntry : NonCopyable, ReferenceCounted : range(range), granuleID(granuleID), startVersion(startVersion), endVersion(endVersion) {} }; +// TODO: add limit on active fdb initial snapshots we can do at once per worker (knob + flow lock) struct BlobWorkerData : NonCopyable, ReferenceCounted { UID id; Database db; @@ -320,10 +320,16 @@ ACTOR Future updateGranuleSplitState(Transaction* tr, BlobGranuleSplitState newState) { state KeyRange currentRange = blobGranuleSplitKeyRangeFor(parentGranuleID); - RangeResult totalState = wait(tr->getRange(currentRange, 100)); + state RangeResult totalState = wait(tr->getRange(currentRange, SERVER_KNOBS->BG_MAX_SPLIT_FANOUT + 2)); // TODO is this explicit conflit range necessary with the above read? tr->addWriteConflictRange(currentRange); - ASSERT(!totalState.more); + ASSERT_WE_THINK(!totalState.more && totalState.size() <= SERVER_KNOBS->BG_MAX_SPLIT_FANOUT + 1); + // maybe someone decreased the knob, we should gracefully handle it not in simulation + if (totalState.more || totalState.size() > SERVER_KNOBS->BG_MAX_SPLIT_FANOUT + 1) { + RangeResult tryAgain = wait(tr->getRange(currentRange, 10000)); + ASSERT(!tryAgain.more); + totalState = tryAgain; + } if (totalState.empty()) { ASSERT(newState == BlobGranuleSplitState::Done); diff --git a/fdbserver/workloads/BlobGranuleCorrectnessWorkload.actor.cpp b/fdbserver/workloads/BlobGranuleCorrectnessWorkload.actor.cpp index 352511fbb8..e6ec6013c7 100644 --- a/fdbserver/workloads/BlobGranuleCorrectnessWorkload.actor.cpp +++ b/fdbserver/workloads/BlobGranuleCorrectnessWorkload.actor.cpp @@ -191,7 +191,7 @@ struct BlobGranuleCorrectnessWorkload : TestWorkload { // set up directory with its own randomness uint32_t dirId = i * clientCount + clientId; if (BGW_DEBUG) { - printf("Client %d/%d creating directory %d\n", clientId, clientCount, dirId); + fmt::print("Client {0}/{1} creating directory {2}\n", clientId, clientCount, dirId); } directories.push_back(makeReference(dirId, targetByteRate)); targetByteRate /= skewMultiplier; @@ -306,8 +306,6 @@ struct BlobGranuleCorrectnessWorkload : TestWorkload { Version readVersion = wait(tr->getReadVersion()); return readVersion; } catch (Error& e) { - // TODO REMOVE print - printf("BGV GRV got error %s\n", e.name()); wait(tr->onError(e)); } } @@ -325,10 +323,10 @@ struct BlobGranuleCorrectnessWorkload : TestWorkload { state Version readVersion = rv; std::pair>> blob = wait(self->readFromBlob(cx, self, threadData->directoryRange, readVersion)); - printf("Directory %d got %s RV %lld\n", - threadData->directoryID, - doSetup ? "initial" : "final", - readVersion); + fmt::print("Directory {0} got {1} RV {2}\n", + threadData->directoryID, + doSetup ? "initial" : "final", + readVersion); threadData->minSuccessfulReadVersion = readVersion; return Void(); } catch (Error& e) { @@ -370,7 +368,7 @@ struct BlobGranuleCorrectnessWorkload : TestWorkload { format("%08x", endKey), readVersion); if (lastMatching.present()) { - printf(" last correct: %s\n", lastMatching.get().printable().c_str()); + fmt::print(" last correct: {}\n", lastMatching.get().printable()); } if (expectedValue.present() || blobValue.present()) { // value mismatch @@ -386,17 +384,17 @@ struct BlobGranuleCorrectnessWorkload : TestWorkload { fmt::print(" Actual Key: {0}\n", blobKey.present() ? blobKey.get().printable() : ""); } - printf("Chunks:\n"); + fmt::print("Chunks:\n"); for (auto& chunk : blob.second) { - printf("[%s - %s)\n", chunk.keyRange.begin.printable().c_str(), chunk.keyRange.end.printable().c_str()); + fmt::print("[{0} - {1})\n", chunk.keyRange.begin.printable(), chunk.keyRange.end.printable()); - printf(" SnapshotFile:\n %s\n", - chunk.snapshotFile.present() ? chunk.snapshotFile.get().toString().c_str() : ""); - printf(" DeltaFiles:\n"); + fmt::print(" SnapshotFile:\n {}\n", + chunk.snapshotFile.present() ? chunk.snapshotFile.get().toString().c_str() : ""); + fmt::print(" DeltaFiles:\n"); for (auto& df : chunk.deltaFiles) { - printf(" %s\n", df.toString().c_str()); + fmt::print(" {}\n", df.toString()); } - printf(" Deltas: (%d)", chunk.newDeltas.size()); + fmt::print(" Deltas: ({})", chunk.newDeltas.size()); if (chunk.newDeltas.size() > 0) { fmt::print(" with version [{0} - {1}]", chunk.newDeltas[0].version, @@ -410,9 +408,6 @@ struct BlobGranuleCorrectnessWorkload : TestWorkload { Value genVal(uint32_t val, uint16_t valLen) { std::string v(valLen, 'x'); auto valFormatted = format("%08x", val); - if (valFormatted.size() > v.size()) { - printf("valFormatted=%d, v.size=%d\n", valFormatted.size(), v.size()); - } ASSERT(valFormatted.size() <= v.size()); for (int i = 0; i < valFormatted.size(); i++) { From c8c97e0256bf3a78d5c3554cd475ebdb9921ee03 Mon Sep 17 00:00:00 2001 From: Josh Slocum Date: Thu, 10 Mar 2022 09:22:53 -0600 Subject: [PATCH 308/413] Blob Worker focused cleanup --- fdbclient/ServerKnobs.cpp | 12 +- fdbclient/ServerKnobs.h | 5 +- fdbserver/BlobManager.actor.cpp | 21 +- fdbserver/BlobWorker.actor.cpp | 565 ++++++++++++-------------------- 4 files changed, 231 insertions(+), 372 deletions(-) diff --git a/fdbclient/ServerKnobs.cpp b/fdbclient/ServerKnobs.cpp index e8ddd6b7d9..bc6f181c79 100644 --- a/fdbclient/ServerKnobs.cpp +++ b/fdbclient/ServerKnobs.cpp @@ -825,15 +825,19 @@ void ServerKnobs::initialize(Randomize randomize, ClientKnobs* clientKnobs, IsSi init( BG_SNAPSHOT_FILE_TARGET_BYTES, 10000000 ); if( buggifySmallShards || (randomize && BUGGIFY) ) { deterministicRandom()->random01() < 0.1 ? BG_SNAPSHOT_FILE_TARGET_BYTES /= 100 : BG_SNAPSHOT_FILE_TARGET_BYTES /= 10; } init( BG_DELTA_BYTES_BEFORE_COMPACT, BG_SNAPSHOT_FILE_TARGET_BYTES/2 ); init( BG_DELTA_FILE_TARGET_BYTES, BG_DELTA_BYTES_BEFORE_COMPACT/10 ); + init( BG_MAX_SPLIT_FANOUT, 10 ); if( randomize && BUGGIFY ) BG_MAX_SPLIT_FANOUT = deterministicRandom()->randomInt(5, 15); + init( BG_HOT_SNAPSHOT_VERSIONS, 5000000 ); + init( BLOB_WORKER_INITIAL_SNAPSHOT_PARALLELISM, 8 ); if( randomize && BUGGIFY ) BLOB_WORKER_INITIAL_SNAPSHOT_PARALLELISM = 1; 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 ); - init( BG_MAX_SPLIT_FANOUT, 10 ); if( randomize && BUGGIFY ) BLOB_WORKER_REQUEST_TIMEOUT = deterministicRandom()->randomInt(5, 15); + init( BLOB_WORKER_BATCH_GRV_INTERVAL, 0.1 ); + - init( BLOB_MANAGER_STATUS_EXP_BACKOFF_MIN, 0.1 ); - init( BLOB_MANAGER_STATUS_EXP_BACKOFF_MAX, 5.0 ); - init( BLOB_MANAGER_STATUS_EXP_BACKOFF_EXPONENT, 1.5 ); + init( BLOB_MANAGER_STATUS_EXP_BACKOFF_MIN, 0.1 ); + init( BLOB_MANAGER_STATUS_EXP_BACKOFF_MAX, 5.0 ); + init( BLOB_MANAGER_STATUS_EXP_BACKOFF_EXPONENT, 1.5 ); // clang-format on diff --git a/fdbclient/ServerKnobs.h b/fdbclient/ServerKnobs.h index 0a0023719c..a707580ae8 100644 --- a/fdbclient/ServerKnobs.h +++ b/fdbclient/ServerKnobs.h @@ -776,11 +776,14 @@ public: int BG_SNAPSHOT_FILE_TARGET_BYTES; int BG_DELTA_FILE_TARGET_BYTES; int BG_DELTA_BYTES_BEFORE_COMPACT; + int BG_MAX_SPLIT_FANOUT; + int BG_HOT_SNAPSHOT_VERSIONS; + int BLOB_WORKER_INITIAL_SNAPSHOT_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 double BLOB_WORKERLIST_FETCH_INTERVAL; - int BG_MAX_SPLIT_FANOUT; // Decreasing this knob can be unsafe + double BLOB_WORKER_BATCH_GRV_INTERVAL; double BLOB_MANAGER_STATUS_EXP_BACKOFF_MIN; double BLOB_MANAGER_STATUS_EXP_BACKOFF_MAX; diff --git a/fdbserver/BlobManager.actor.cpp b/fdbserver/BlobManager.actor.cpp index 4f94391dd2..c914b13ae7 100644 --- a/fdbserver/BlobManager.actor.cpp +++ b/fdbserver/BlobManager.actor.cpp @@ -323,7 +323,9 @@ ACTOR Future>> splitRange(ReferenceiAmReplaced.sendError(e); + if (bmData->iAmReplaced.canBeSet()) { + bmData->iAmReplaced.sendError(e); + } throw e; } } @@ -452,7 +454,9 @@ ACTOR Future doRangeAssignment(Reference bmData, .error(e) .detail("Epoch", bmData->epoch); ASSERT_WE_THINK(false); - bmData->iAmReplaced.sendError(e); + if (bmData->iAmReplaced.canBeSet()) { + bmData->iAmReplaced.sendError(e); + } throw; } @@ -698,7 +702,9 @@ ACTOR Future monitorClientRanges(Reference bmData) { .detail("Epoch", bmData->epoch) .detail("ClientRanges", results.size() - 1); wait(delay(600)); - bmData->iAmReplaced.sendError(internal_error()); + if (bmData->iAmReplaced.canBeSet()) { + bmData->iAmReplaced.sendError(internal_error()); + } throw internal_error(); } @@ -1041,6 +1047,7 @@ ACTOR Future deregisterBlobWorker(Reference bmData, BlobW try { wait(checkManagerLock(tr, bmData)); Key blobWorkerListKey = blobWorkerListKeyFor(interf.id()); + // FIXME: should be able to remove this conflict range tr->addReadConflictRange(singleKeyRange(blobWorkerListKey)); tr->clear(blobWorkerListKey); @@ -1298,7 +1305,9 @@ ACTOR Future monitorBlobWorkerStatus(Reference bmData, Bl .detail("Epoch", bmData->epoch); ASSERT_WE_THINK(false); // if not simulation, kill the BM - bmData->iAmReplaced.sendError(e); + if (bmData->iAmReplaced.canBeSet()) { + bmData->iAmReplaced.sendError(e); + } throw e; } } @@ -1343,7 +1352,9 @@ ACTOR Future monitorBlobWorker(Reference bmData, BlobWork .detail("Epoch", bmData->epoch); ASSERT_WE_THINK(false); // if not simulation, kill the BM - bmData->iAmReplaced.sendError(e); + if (bmData->iAmReplaced.canBeSet()) { + bmData->iAmReplaced.sendError(e); + } throw e; } } diff --git a/fdbserver/BlobWorker.actor.cpp b/fdbserver/BlobWorker.actor.cpp index 2197885b6b..4ec0c18886 100644 --- a/fdbserver/BlobWorker.actor.cpp +++ b/fdbserver/BlobWorker.actor.cpp @@ -44,10 +44,17 @@ #include "flow/IRandom.h" #include "flow/Trace.h" #include "flow/actorcompiler.h" // has to be last include +#include "flow/network.h" #define BW_DEBUG false #define BW_REQUEST_DEBUG false +/* + * The Blob Worker is a stateless role assigned a set of granules by the Blob Manager. + * It is responsible for managing the change feeds for those granules, and for consuming the mutations from those change + * feeds and writing them out as files to blob storage. + */ + struct GranuleStartState { UID granuleID; Version changeFeedStartVersion; @@ -93,9 +100,6 @@ struct GranuleMetadata : NonCopyable, ReferenceCounted { AssignBlobRangeRequest originalReq; - // TODO FOR DEBUGGING, REMOVE - Version waitForVersionReturned = invalidVersion; - void resume() { if (resumeSnapshot.canBeSet()) { resumeSnapshot.send(Void()); @@ -124,9 +128,6 @@ struct GranuleRangeMetadata { GranuleRangeMetadata() : lastEpoch(0), lastSeqno(0) {} GranuleRangeMetadata(int64_t epoch, int64_t seqno, Reference activeMetadata) : lastEpoch(epoch), lastSeqno(seqno), activeMetadata(activeMetadata) {} - - // TODO REMOVE - // ~GranuleRangeMetadata() { printf("Destroying granule metadata\n"); } }; // represents a previous version of a granule, and optionally the files that compose it @@ -137,6 +138,7 @@ struct GranuleHistoryEntry : NonCopyable, ReferenceCounted Version endVersion; // version of the last delta file // load files lazily, and allows for clearing old cold-queried files to save memory + // FIXME: add memory limit and evictor for old cached files Future files; // FIXME: do skip pointers with single back-pointer and neighbor pointers @@ -148,7 +150,6 @@ struct GranuleHistoryEntry : NonCopyable, ReferenceCounted : range(range), granuleID(granuleID), startVersion(startVersion), endVersion(endVersion) {} }; -// TODO: add limit on active fdb initial snapshots we can do at once per worker (knob + flow lock) struct BlobWorkerData : NonCopyable, ReferenceCounted { UID id; Database db; @@ -177,15 +178,15 @@ struct BlobWorkerData : NonCopyable, ReferenceCounted { Promise doGRVCheck; NotifiedVersion grvVersion; + Promise fatalError; + + FlowLock initialSnapshotLock; int changeFeedStreamReplyBufferSize = SERVER_KNOBS->BG_DELTA_FILE_TARGET_BYTES / 2; - BlobWorkerData(UID id, Database db) : id(id), db(db), stats(id, SERVER_KNOBS->WORKER_LOGGING_INTERVAL) {} - ~BlobWorkerData() { - if (BW_DEBUG) { - printf("Destroying BW %s data\n", id.toString().c_str()); - } - } + BlobWorkerData(UID id, Database db) + : id(id), db(db), stats(id, SERVER_KNOBS->WORKER_LOGGING_INTERVAL), + initialSnapshotLock(SERVER_KNOBS->BLOB_WORKER_INITIAL_SNAPSHOT_PARALLELISM) {} bool managerEpochOk(int64_t epoch) { if (epoch < currentManagerEpoch) { @@ -208,6 +209,18 @@ struct BlobWorkerData : NonCopyable, ReferenceCounted { } }; +// serialize change feed key as UID bytes, to use 16 bytes on disk +static Key granuleIDToCFKey(UID granuleID) { + BinaryWriter wr(Unversioned()); + wr << granuleID; + return wr.toValue(); +} + +// parse change feed key back to UID, to be human-readable +static UID cfKeyToGranuleID(Key cfKey) { + return BinaryReader::fromStringRef(cfKey, Unversioned()); +} + // returns true if we can acquire it static void acquireGranuleLock(int64_t epoch, int64_t seqno, int64_t prevOwnerEpoch, int64_t prevOwnerSeqno) { // returns true if our lock (E, S) >= (Eprev, Sprev) @@ -225,11 +238,6 @@ static void acquireGranuleLock(int64_t epoch, int64_t seqno, int64_t prevOwnerEp static void checkGranuleLock(int64_t epoch, int64_t seqno, int64_t ownerEpoch, int64_t ownerSeqno) { // sanity check - lock value should never go backwards because of acquireGranuleLock - /* - printf( - "Checking granule lock: \n mine: (%lld, %lld)\n owner: (%lld, %lld)\n", epoch, seqno, ownerEpoch, - ownerSeqno); - */ ASSERT(epoch <= ownerEpoch); ASSERT(epoch < ownerEpoch || (epoch == ownerEpoch && seqno <= ownerSeqno)); @@ -258,6 +266,7 @@ ACTOR Future readAndCheckGranuleLock(Reference checkGranuleLock(epoch, seqno, std::get<0>(currentOwner), std::get<1>(currentOwner)); // if we still own the lock, add a conflict range in case anybody else takes it over while we add this file + // FIXME: we don't need these conflict ranges tr->addReadConflictRange(singleKeyRange(lockKey)); return Void(); @@ -321,7 +330,7 @@ ACTOR Future updateGranuleSplitState(Transaction* tr, state KeyRange currentRange = blobGranuleSplitKeyRangeFor(parentGranuleID); state RangeResult totalState = wait(tr->getRange(currentRange, SERVER_KNOBS->BG_MAX_SPLIT_FANOUT + 2)); - // TODO is this explicit conflit range necessary with the above read? + // FIXME: remove above conflict range? tr->addWriteConflictRange(currentRange); ASSERT_WE_THINK(!totalState.more && totalState.size() <= SERVER_KNOBS->BG_MAX_SPLIT_FANOUT + 1); // maybe someone decreased the knob, we should gracefully handle it not in simulation @@ -387,7 +396,7 @@ ACTOR Future updateGranuleSplitState(Transaction* tr, } // FIXME: appears change feed destroy isn't working! ADD BACK - // wait(updateChangeFeed(tr, KeyRef(parentGranuleID.toString()), ChangeFeedStatus::CHANGE_FEED_DESTROY)); + // wait(updateChangeFeed(tr, granuleIDToCFKey(parentGranuleID), ChangeFeedStatus::CHANGE_FEED_DESTROY)); Key oldGranuleLockKey = blobGranuleLockKeyFor(parentGranuleRange); // FIXME: deleting granule lock can cause races where another granule with the same range starts way later @@ -398,6 +407,7 @@ ACTOR Future updateGranuleSplitState(Transaction* tr, // tr->clear(singleKeyRange(oldGranuleLockKey)); tr->clear(currentRange); + TEST(true); // Granule split cleanup on last delta file persisted } else { tr->atomicOp(myStateKey, blobGranuleSplitValueFor(newState), MutationRef::SetVersionstampedValue); if (newState == BlobGranuleSplitState::Assigned && currentState == BlobGranuleSplitState::Initialized && @@ -409,10 +419,13 @@ ACTOR Future updateGranuleSplitState(Transaction* tr, parentGranuleID.toString().c_str()); } - wait(updateChangeFeed(tr, KeyRef(parentGranuleID.toString()), ChangeFeedStatus::CHANGE_FEED_STOP)); + wait(updateChangeFeed( + tr, KeyRef(granuleIDToCFKey(parentGranuleID)), ChangeFeedStatus::CHANGE_FEED_STOP)); } + TEST(true); // Granule split stopping change feed } } else if (BW_DEBUG) { + TEST(true); // Out of order granule split state updates ignored fmt::print("Ignoring granule {0} split state from {1} {2} -> {3}\n", currentGranuleID.toString(), parentGranuleID.toString(), @@ -481,7 +494,6 @@ ACTOR Future writeDeltaFile(Reference bwData, state int numIterations = 0; try { // before updating FDB, wait for the delta file version to be committed and previous delta files to finish - // TODO fix file leak here on error pre-transaction. wait(waitCommitted); BlobFileIndex prev = wait(previousDeltaFileFuture); wait(delay(0, TaskPriority::BlobWorkerUpdateFDB)); @@ -532,11 +544,13 @@ ACTOR Future writeDeltaFile(Reference bwData, // commit a transaction, we can and want to safely delete the file we wrote. Otherwise, we may have updated FDB // with file and cannot safely delete it. if (numIterations > 0) { + TEST(true); // Granule potentially leaving orphaned delta file throw e; } if (BW_DEBUG) { fmt::print("deleting delta file {0} after error {1}\n", fname, e.name()); } + TEST(true); // Granule cleaning up delta file after error ++bwData->stats.s3DeleteReqs; bwData->addActor.send(bwData->bstore->deleteFile(fname)); throw e; @@ -583,18 +597,19 @@ ACTOR Future writeSnapshot(Reference bwData, snapshot.size()); } - // TODO REMOVE sanity checks! - if (snapshot.size() > 0) { - ASSERT(keyRange.begin <= snapshot[0].key); - ASSERT(keyRange.end > snapshot[snapshot.size() - 1].key); - } - for (int i = 0; i < snapshot.size() - 1; i++) { - if (snapshot[i].key >= snapshot[i + 1].key) { - fmt::print("SORT ORDER VIOLATION IN SNAPSHOT FILE: {0}, {1}\n", - snapshot[i].key.printable(), - snapshot[i + 1].key.printable()); + if (g_network->isSimulated()) { + if (snapshot.size() > 0) { + ASSERT(keyRange.begin <= snapshot[0].key); + ASSERT(keyRange.end > snapshot[snapshot.size() - 1].key); + } + for (int i = 0; i < snapshot.size() - 1; i++) { + if (snapshot[i].key >= snapshot[i + 1].key) { + fmt::print("SORT ORDER VIOLATION IN SNAPSHOT FILE: {0}, {1}\n", + snapshot[i].key.printable(), + snapshot[i + 1].key.printable()); + } + ASSERT(snapshot[i].key < snapshot[i + 1].key); } - ASSERT(snapshot[i].key < snapshot[i + 1].key); } state Value serialized = ObjectWriter::toValue(snapshot, Unversioned()); @@ -610,7 +625,6 @@ ACTOR Future writeSnapshot(Reference bwData, ++bwData->stats.snapshotFilesWritten; bwData->stats.snapshotBytesWritten += serializedSize; - // TODO: inject write error wait(objectFile->append(serialized.begin(), serializedSize)); wait(objectFile->finish()); @@ -650,11 +664,13 @@ ACTOR Future writeSnapshot(Reference bwData, // commit a transaction, we can and want to safely delete the file we wrote. Otherwise, we may have updated FDB // with file and cannot safely delete it. if (numIterations > 0) { + TEST(true); // Granule potentially leaving orphaned snapshot file throw e; } if (BW_DEBUG) { fmt::print("deleting snapshot file {0} after error {1}\n", fname, e.name()); } + TEST(true); // Granule deleting snapshot file after error ++bwData->stats.s3DeleteReqs; bwData->addActor.send(bwData->bstore->deleteFile(fname)); throw e; @@ -684,6 +700,9 @@ ACTOR Future dumpInitialSnapshotFromFDB(Reference metadata->keyRange.begin.printable(), metadata->keyRange.end.printable()); } + wait(bwData->initialSnapshotLock.take()); + state FlowLock::Releaser holdingDVL(bwData->initialSnapshotLock); + state Reference tr = makeReference(bwData->db); state int64_t bytesRead = 0; state int retries = 0; @@ -731,6 +750,7 @@ ACTOR Future dumpInitialSnapshotFromFDB(Reference state Error err = e; wait(tr->onError(e)); retries++; + TEST(true); // Granule initial snapshot failed TraceEvent(SevWarn, "BlobGranuleInitialSnapshotRetry", bwData->id) .error(err) .detail("Granule", metadata->keyRange) @@ -749,7 +769,6 @@ ACTOR Future dumpInitialSnapshotFromFDB(Reference // files might not be the current set of files in metadata, in the case of doing the initial snapshot of a granule that // was split. -// FIXME: only pass metadata->keyRange ACTOR Future compactFromBlob(Reference bwData, Reference metadata, UID granuleID, @@ -797,12 +816,6 @@ ACTOR Future compactFromBlob(Reference bwData, metadata->keyRange.begin.printable(), metadata->keyRange.end.printable(), version); - - /*printf(" SnapshotFile:\n %s\n", chunk.snapshotFile.get().toString().c_str()); - printf(" DeltaFiles:\n"); - for (auto& df : chunk.deltaFiles) { - printf(" %s\n", df.toString().c_str()); - }*/ } loop { @@ -819,8 +832,6 @@ ACTOR Future compactFromBlob(Reference bwData, RangeResult newGranule = wait(readBlobGranule(chunk, metadata->keyRange, version, bwData->bstore, &bwData->stats)); - // TODO: inject read error - bwData->stats.bytesReadFromS3ForCompaction += compactBytesRead; rowsStream.send(std::move(newGranule)); rowsStream.sendError(end_of_stream()); @@ -873,8 +884,11 @@ ACTOR Future checkSplitAndReSnapshot(Reference bw metadata->resumeSnapshot.reset(); state int64_t statusEpoch = metadata->continueEpoch; state int64_t statusSeqno = metadata->continueSeqno; - // TODO its own knob or something better? This is wrong in case of rollbacks - state bool writeHot = versionsSinceLastSnapshot <= SERVER_KNOBS->MAX_READ_TRANSACTION_LIFE_VERSIONS; + + // If two snapshots happen without a split within a low time interval, this granule is "write-hot" + // FIXME: If a rollback happens, this could incorrectly identify a hot granule as not hot. This should be rare + // though and is just less efficient. + state bool writeHot = versionsSinceLastSnapshot <= SERVER_KNOBS->BG_HOT_SNAPSHOT_VERSIONS; loop { loop { try { @@ -897,6 +911,7 @@ ACTOR Future checkSplitAndReSnapshot(Reference bw if (e.code() == error_code_operation_cancelled) { throw e; } + TEST(true); // Blob worker re-sending split evaluation to manager after not error/not hearing back // if we got broken promise while waiting, the old stream was killed, so we don't need to wait on // change, just retry if (e.code() == error_code_broken_promise) { @@ -930,8 +945,6 @@ ACTOR Future checkSplitAndReSnapshot(Reference bw TraceEvent("BlobGranuleSnapshotFile", bwData->id) .detail("Granule", metadata->keyRange) .detail("Version", metadata->durableDeltaVersion.get()); - // TODO: this could read from FDB instead if it knew there was a large range clear at the end or - // it knew the granule was small, or something // wait for file updater to make sure that last delta file is in the metadata before while (metadata->files.deltaFiles.empty() || metadata->files.deltaFiles.back().version < reSnapshotVersion) { @@ -977,7 +990,6 @@ ACTOR Future handleCompletedDeltaFile(Reference bwData, } // if we get an i/o error updating files, or a rollback, reassign the granule to ourselves and start fresh -// FIXME: is this the correct set of errors? static bool granuleCanRetry(const Error& e) { switch (e.code()) { case error_code_please_reboot: @@ -1000,12 +1012,6 @@ struct InFlightFile { : future(future), version(version), bytes(bytes), snapshot(snapshot) {} }; -// TODO REMOVE once correctness clean -#define DEBUG_BW_START_VERSION invalidVersion -#define DEBUG_BW_END_VERSION invalidVersion -#define DEBUG_BW_WAIT_VERSION invalidVersion -#define DEBUG_BW_VERSION(v) DEBUG_BW_START_VERSION <= v&& v <= DEBUG_BW_END_VERSION - static Version doGranuleRollback(Reference metadata, Version mutationVersion, Version rollbackVersion, @@ -1024,6 +1030,7 @@ static Version doGranuleRollback(Reference metadata, for (auto& f : inFlightFiles) { if (f.snapshot) { if (f.version > rollbackVersion) { + TEST(true); // Granule rollback cancelling snapshot file if (BW_DEBUG) { fmt::print("[{0} - {1}) rollback cancelling snapshot file @ {2}\n", metadata->keyRange.begin.printable(), @@ -1044,6 +1051,7 @@ static Version doGranuleRollback(Reference metadata, metadata->bytesInNewDeltaFiles -= f.bytes; } toPop++; + TEST(true); // Granule rollback cancelling delta file if (BW_DEBUG) { fmt::print("[{0} - {1}) rollback cancelling delta file @ {2}\n", metadata->keyRange.begin.printable(), @@ -1093,6 +1101,7 @@ static Version doGranuleRollback(Reference metadata, } else { // No pending delta files to discard, just in-memory mutations + TEST(true); // Granule rollback discarding in memory mutations // FIXME: could binary search? int mIdx = metadata->currentDeltas.size() - 1; @@ -1135,44 +1144,18 @@ static Version doGranuleRollback(Reference metadata, return cfRollbackVersion; } -#define DEBUG_WAIT_VERSION_COMMITTED false -ACTOR Future waitOnCFVersion(Reference metadata, - Version original /*TODO REMOVE, just for debugging*/, - Version waitVersion) { - if (DEBUG_BW_VERSION(original) && DEBUG_WAIT_VERSION_COMMITTED) { - fmt::print("WVC {0}: waiting for {1} \n", original, waitVersion); - } +ACTOR Future waitOnCFVersion(Reference metadata, Version waitVersion) { loop { try { - if (DEBUG_BW_VERSION(original) && DEBUG_WAIT_VERSION_COMMITTED) { - if (metadata->activeCFData.get().isValid()) { - fmt::print( - "WVC {0}: WAL (currently {1})\n", original, metadata->activeCFData.get()->getVersion()); - } else { - fmt::print("WVC {0}: invalid\n", original, metadata->activeCFData.get()->getVersion()); - } - } // if not valid, we're about to be cancelled anyway state Future atLeast = metadata->activeCFData.get().isValid() ? metadata->activeCFData.get()->whenAtLeast(waitVersion) : Never(); choose { - when(wait(atLeast)) { - if (DEBUG_BW_VERSION(original) && DEBUG_WAIT_VERSION_COMMITTED) { - fmt::print("WVC {0}: got at least {1} \n", original, waitVersion); - } - break; - } - when(wait(metadata->activeCFData.onChange())) { - if (DEBUG_BW_VERSION(original) && DEBUG_WAIT_VERSION_COMMITTED) { - fmt::print("WVC {0}: cfOnChange \n", original); - } - } + when(wait(atLeast)) { break; } + when(wait(metadata->activeCFData.onChange())) {} } } catch (Error& e) { - if (DEBUG_BW_VERSION(original) && DEBUG_WAIT_VERSION_COMMITTED) { - fmt::print("WVC {0}: got error {1} \n", original, e.name()); - } if (e.code() == error_code_operation_cancelled || e.code() == error_code_change_feed_popped) { throw e; } @@ -1180,25 +1163,10 @@ ACTOR Future waitOnCFVersion(Reference metadata, // if waiting on a parent granule change feed and we change to the child, the parent will get end_of_stream, // which could cause this waiting whenAtLeast to get change_feed_cancelled. We should simply retry and wait // a bit, as blobGranuleUpdateFiles will switch to the new change feed - if (DEBUG_BW_VERSION(original) && DEBUG_WAIT_VERSION_COMMITTED) { - if (BW_DEBUG) { - fmt::print("WVC {0}: unexpected error {1}\n", original, e.name()); - } - throw e; - } wait(delay(0.05)); } } - if (DEBUG_BW_VERSION(original) && DEBUG_WAIT_VERSION_COMMITTED) { - fmt::print("WVC {0}: got \n", original); - } - - // sanity check to make sure whenAtLeast didn't return early - if (waitVersion > metadata->waitForVersionReturned) { - metadata->waitForVersionReturned = waitVersion; - } - // stop after change feed callback wait(delay(0, TaskPriority::BlobWorkerReadChangeFeed)); @@ -1208,17 +1176,7 @@ ACTOR Future waitOnCFVersion(Reference metadata, ACTOR Future waitCommittedGrv(Reference bwData, Reference metadata, Version version) { - if (DEBUG_BW_VERSION(version) && DEBUG_WAIT_VERSION_COMMITTED) { - fmt::print("WVC {0}: grv start\n", version); - } - // TODO REMOVE debugs if (version > bwData->grvVersion.get()) { - if (DEBUG_BW_VERSION(version) && DEBUG_WAIT_VERSION_COMMITTED) { - fmt::print("WVC {0}: getting grv\n", version); - } - /*if (BW_DEBUG) { - fmt::print("waitVersionCommitted waiting {0}\n", version); - }*/ // this order is important, since we need to register a waiter on the notified version before waking the GRV // actor Future grvAtLeast = bwData->grvVersion.whenAtLeast(version); @@ -1230,10 +1188,7 @@ ACTOR Future waitCommittedGrv(Reference bwData, } Version grvVersion = bwData->grvVersion.get(); - if (DEBUG_BW_VERSION(version) && DEBUG_WAIT_VERSION_COMMITTED) { - fmt::print("WVC {0}: got grv\n", version); - } - wait(waitOnCFVersion(metadata, version, grvVersion)); + wait(waitOnCFVersion(metadata, grvVersion)); return Void(); } @@ -1242,14 +1197,8 @@ ACTOR Future waitVersionCommitted(Reference bwData, Version version) { // If GRV is way in the future, we know we can't roll back more than 5 seconds (or whatever this knob is set to) // worth of versions - if (DEBUG_BW_VERSION(version) && DEBUG_WAIT_VERSION_COMMITTED) { - fmt::print("WVC {0}: starting\n", version); - } wait(waitCommittedGrv(bwData, metadata, version) || - waitOnCFVersion(metadata, version, version + SERVER_KNOBS->MAX_READ_TRANSACTION_LIFE_VERSIONS)); - if (DEBUG_BW_VERSION(version) && DEBUG_WAIT_VERSION_COMMITTED) { - fmt::print("WVC {0}: done\n", version); - } + waitOnCFVersion(metadata, version + SERVER_KNOBS->MAX_READ_TRANSACTION_LIFE_VERSIONS)); if (version > metadata->knownCommittedVersion) { metadata->knownCommittedVersion = version; } @@ -1258,7 +1207,6 @@ ACTOR Future waitVersionCommitted(Reference bwData, // updater for a single granule // TODO: this is getting kind of large. Should try to split out this actor if it continues to grow? -// FIXME: handle errors here (forward errors) ACTOR Future blobGranuleUpdateFiles(Reference bwData, Reference metadata, Future assignFuture) { @@ -1288,9 +1236,9 @@ ACTOR Future blobGranuleUpdateFiles(Reference bwData, wait(delay(0, TaskPriority::BlobWorkerUpdateStorage)); - cfKey = StringRef(startState.granuleID.toString()); + cfKey = granuleIDToCFKey(startState.granuleID); if (startState.parentGranule.present()) { - oldCFKey = StringRef(startState.parentGranule.get().second.toString()); + oldCFKey = granuleIDToCFKey(startState.parentGranule.get().second); } if (BW_DEBUG) { @@ -1329,6 +1277,7 @@ ACTOR Future blobGranuleUpdateFiles(Reference bwData, } if (!startState.doSnapshot) { + TEST(true); // Granule moved without split startVersion = startState.previousDurableVersion; ASSERT(!metadata->files.snapshotFiles.empty()); metadata->pendingSnapshotVersion = metadata->files.snapshotFiles.back().version; @@ -1363,7 +1312,7 @@ ACTOR Future blobGranuleUpdateFiles(Reference bwData, metadata->bufferedDeltaVersion = startVersion; metadata->knownCommittedVersion = startVersion; - Reference newCFData = makeReference(); + Reference cfData = makeReference(); if (startState.parentGranule.present() && startVersion < startState.changeFeedStartVersion) { // read from parent change feed up until our new change feed is started @@ -1374,7 +1323,7 @@ ACTOR Future blobGranuleUpdateFiles(Reference bwData, // this actor readOldChangeFeed = true; - oldChangeFeedFuture = bwData->db->getChangeFeedStream(newCFData, + oldChangeFeedFuture = bwData->db->getChangeFeedStream(cfData, oldCFKey.get(), startVersion + 1, startState.changeFeedStartVersion, @@ -1384,7 +1333,7 @@ ACTOR Future blobGranuleUpdateFiles(Reference bwData, } else { readOldChangeFeed = false; - changeFeedFuture = bwData->db->getChangeFeedStream(newCFData, + changeFeedFuture = bwData->db->getChangeFeedStream(cfData, cfKey, startVersion + 1, MAX_VERSION, @@ -1395,7 +1344,7 @@ ACTOR Future blobGranuleUpdateFiles(Reference bwData, // Start actors BEFORE setting new change feed data to ensure the change feed data is properly initialized by // the client - metadata->activeCFData.set(newCFData); + metadata->activeCFData.set(cfData); ASSERT(metadata->readable.canBeSet()); metadata->readable.send(Void()); @@ -1441,22 +1390,12 @@ ACTOR Future blobGranuleUpdateFiles(Reference bwData, state Standalone> mutations; try { - /*if (DEBUG_BW_VERSION(metadata->bufferedDeltaVersion)) { - fmt::print("BW waiting mutations after ({0})\n", metadata->bufferedDeltaVersion); - }*/ // Even if there are no new mutations, there still might be readers waiting on durableDeltaVersion // to advance. We need to check whether any outstanding files have finished so we don't wait on // mutations forever choose { when(Standalone> _mutations = waitNext(metadata->activeCFData.get()->mutations.getFuture())) { - /*if (DEBUG_BW_VERSION(metadata->bufferedDeltaVersion)) { - fmt::print("BW got mutations after ({0}): {1} - {2} ({3})\n", - metadata->bufferedDeltaVersion, - _mutations.front().version, - _mutations.back().version, - _mutations.size()); - }*/ mutations = _mutations; ASSERT(!mutations.empty()); if (readOldChangeFeed) { @@ -1475,28 +1414,10 @@ ACTOR Future blobGranuleUpdateFiles(Reference bwData, } ASSERT(mutations.front().version > metadata->bufferedDeltaVersion); - // if we just got mutations, we haven't buffered them yet, so waitForVersion can't have returned - // this version yet - if (mutations.front().version <= metadata->waitForVersionReturned) { - fmt::print("ERROR: WaitForVersion returned early for granule [{0} - {1}). " - "waitForVersionReturned={2}, mutationVersion={3} !!!\n", - metadata->keyRange.begin.printable(), - metadata->keyRange.end.printable(), - metadata->waitForVersionReturned, - mutations.front().version); - } - ASSERT(mutations.front().version > metadata->waitForVersionReturned); - // If this assert trips we should have gotten change_feed_popped from SS and didn't ASSERT(mutations.front().version >= metadata->activeCFData.get()->popVersion); } - when(wait(inFlightFiles.empty() ? Never() : success(inFlightFiles.front().future))) { - // TODO REMOVE - /*if (DEBUG_BW_VERSION(metadata->bufferedDeltaVersion)) { - fmt::print("BW got file before waiting for mutations after {0}\n", - metadata->bufferedDeltaVersion); - }*/ - } + when(wait(inFlightFiles.empty() ? Never() : success(inFlightFiles.front().future))) {} } } catch (Error& e) { // only error we should expect here is when we finish consuming old change feed @@ -1517,9 +1438,9 @@ ACTOR Future blobGranuleUpdateFiles(Reference bwData, metadata->bufferedDeltaVersion); } - Reference newCFData = makeReference(); + Reference cfData = makeReference(); - changeFeedFuture = bwData->db->getChangeFeedStream(newCFData, + changeFeedFuture = bwData->db->getChangeFeedStream(cfData, cfKey, startState.changeFeedStartVersion, MAX_VERSION, @@ -1529,7 +1450,7 @@ ACTOR Future blobGranuleUpdateFiles(Reference bwData, // Start actors BEFORE setting new change feed data to ensure the change feed data is properly // initialized by the client - metadata->activeCFData.set(newCFData); + metadata->activeCFData.set(cfData); } // process mutations @@ -1557,20 +1478,7 @@ ACTOR Future blobGranuleUpdateFiles(Reference bwData, ASSERT(rollbackVersion >= metadata->durableDeltaVersion.get()); if (!rollbacksInProgress.empty()) { - // TODO REMOVE, for debugging - if (rollbacksInProgress.front().first != rollbackVersion) { - fmt::print("Found out of order rollbacks! Current in progress: {0}, mutation " - "version: {1}\n", - rollbacksInProgress.front().first, - rollbackVersion); - } ASSERT(rollbacksInProgress.front().first == rollbackVersion); - if (rollbacksInProgress.front().first != rollbackVersion) { - fmt::print("Found out of order rollbacks! Current in progress: {0}, rollback " - "version: {1}\n", - rollbacksInProgress.front().second, - deltas.version); - } ASSERT(rollbacksInProgress.front().second == deltas.version); if (BW_DEBUG) { fmt::print("Passed rollback {0} -> {1}\n", deltas.version, rollbackVersion); @@ -1586,6 +1494,7 @@ ACTOR Future blobGranuleUpdateFiles(Reference bwData, metadata->currentDeltas.back().version <= rollbackVersion)) { if (BW_DEBUG) { + TEST(true); // Granule ignoring rollback fmt::print( "Granule [{0} - {1}) on BW {2} skipping rollback {3} -> {4} completely\n", metadata->keyRange.begin.printable().c_str(), @@ -1600,6 +1509,7 @@ ACTOR Future blobGranuleUpdateFiles(Reference bwData, // stream. rollbacksCompleted.push_back(std::pair(rollbackVersion, deltas.version)); } else { + TEST(true); // Granule processing rollback if (BW_DEBUG) { fmt::print("[{0} - {1}) on BW {2} ROLLBACK @ {3} -> {4}\n", metadata->keyRange.begin.printable(), @@ -1620,15 +1530,7 @@ ACTOR Future blobGranuleUpdateFiles(Reference bwData, rollbacksInProgress, rollbacksCompleted); - // Reset change feeds to cfRollbackVersion - if (cfRollbackVersion < metadata->waitForVersionReturned) { - fmt::print("Rollback resetting waitForVersionReturned {0} -> {1}\n", - metadata->waitForVersionReturned, - cfRollbackVersion); - metadata->waitForVersionReturned = cfRollbackVersion; - } - - Reference newCFData = makeReference(); + Reference cfData = makeReference(); if (!readOldChangeFeed && cfRollbackVersion < startState.changeFeedStartVersion) { // It isn't possible to roll back across the parent/child feed boundary, but as @@ -1645,7 +1547,7 @@ ACTOR Future blobGranuleUpdateFiles(Reference bwData, if (readOldChangeFeed) { ASSERT(cfRollbackVersion < startState.changeFeedStartVersion); oldChangeFeedFuture = - bwData->db->getChangeFeedStream(newCFData, + bwData->db->getChangeFeedStream(cfData, oldCFKey.get(), cfRollbackVersion + 1, startState.changeFeedStartVersion, @@ -1662,7 +1564,7 @@ ACTOR Future blobGranuleUpdateFiles(Reference bwData, ASSERT(cfRollbackVersion >= startState.changeFeedStartVersion); changeFeedFuture = - bwData->db->getChangeFeedStream(newCFData, + bwData->db->getChangeFeedStream(cfData, cfKey, cfRollbackVersion + 1, MAX_VERSION, @@ -1673,7 +1575,7 @@ ACTOR Future blobGranuleUpdateFiles(Reference bwData, // Start actors BEFORE setting new change feed data to ensure the change feed data // is properly initialized by the client - metadata->activeCFData.set(newCFData); + metadata->activeCFData.set(cfData); justDidRollback = true; break; @@ -1681,6 +1583,7 @@ ACTOR Future blobGranuleUpdateFiles(Reference bwData, } } else if (!rollbacksInProgress.empty() && rollbacksInProgress.front().first < deltas.version && rollbacksInProgress.front().second > deltas.version) { + TEST(true); // Granule skipping mutations b/c prior rollback if (BW_DEBUG) { fmt::print("Skipping mutations @ {} b/c prior rollback\n", deltas.version); } @@ -1695,9 +1598,6 @@ ACTOR Future blobGranuleUpdateFiles(Reference bwData, .detail("ChangeFeedID", readOldChangeFeed ? oldCFKey.get() : cfKey) .detail("OldChangeFeed", readOldChangeFeed ? "T" : "F"); } - if (DEBUG_BW_VERSION(deltas.version)) { - fmt::print("BWB {0}: ({1})\n", deltas.version, deltas.mutations.size()); - } metadata->currentDeltas.push_back_deep(metadata->currentDeltas.arena(), deltas); processedAnyMutations = true; @@ -1845,6 +1745,7 @@ ACTOR Future blobGranuleUpdateFiles(Reference bwData, idx++; } while (waitIdx > 0) { + TEST(true); // Granule blocking on previous snapshot // TODO don't duplicate code BlobFileIndex completedFile = wait(inFlightFiles.front().future); if (inFlightFiles.front().snapshot) { @@ -1877,6 +1778,7 @@ ACTOR Future blobGranuleUpdateFiles(Reference bwData, // queue too many files in parallel, and slow down change feed consuming to let file writing // catch up + TEST(true); // Granule processing long tail of old change feed if (inFlightFiles.size() > 10 && inFlightFiles.front().version <= metadata->knownCommittedVersion) { if (BW_DEBUG) { fmt::print("[{0} - {1}) Waiting on delta file b/c old change feed\n", @@ -1893,14 +1795,6 @@ ACTOR Future blobGranuleUpdateFiles(Reference bwData, } } } catch (Error& e) { - // TODO REMOVE - if (BW_DEBUG) { - fmt::print("BGUF {0} [{1} - {2}) got error {3}\n", - startState.granuleID.toString(), - metadata->keyRange.begin.printable(), - metadata->keyRange.end.printable(), - e.name()); - } // Free last change feed data metadata->activeCFData.set(Reference()); @@ -1915,32 +1809,47 @@ ACTOR Future blobGranuleUpdateFiles(Reference bwData, if (e.code() == error_code_granule_assignment_conflict) { TraceEvent(SevInfo, "GranuleAssignmentConflict", bwData->id).detail("Granule", metadata->keyRange); return Void(); - } else { - ++bwData->stats.granuleUpdateErrors; - if (BW_DEBUG) { - fmt::print("Granule file updater for [{0} - {1}) got error {2}, exiting\n", - metadata->keyRange.begin.printable(), - metadata->keyRange.end.printable(), - e.name()); + } + ++bwData->stats.granuleUpdateErrors; + if (BW_DEBUG) { + fmt::print("Granule file updater for [{0} - {1}) got error {2}, exiting\n", + metadata->keyRange.begin.printable(), + metadata->keyRange.end.printable(), + e.name()); + } + + if (granuleCanRetry(e)) { + TEST(true); // Granule close and re-open on error + TraceEvent("GranuleFileUpdaterRetriableError", bwData->id) + .error(e) + .detail("Granule", metadata->keyRange) + .detail("GranuleID", startState.granuleID); + // explicitly cancel all outstanding write futures BEFORE updating promise stream, to ensure they + // can't update files after the re-assigned granule acquires the lock + // do it backwards though because future depends on previous one, so it could cause a cascade + for (int i = inFlightFiles.size() - 1; i >= 0; i--) { + inFlightFiles[i].future.cancel(); } - TraceEvent(SevWarn, "GranuleFileUpdaterError", bwData->id).error(e).detail("Granule", metadata->keyRange); - if (granuleCanRetry(e)) { - // explicitly cancel all outstanding write futures BEFORE updating promise stream, to ensure they - // can't update files after the re-assigned granule acquires the lock - // do it backwards though because future depends on previous one, so it could cause a cascade - for (int i = inFlightFiles.size() - 1; i >= 0; i--) { - inFlightFiles[i].future.cancel(); - } + // if we retry and re-open, we need to use a normal request (no continue) and update the + // seqno + metadata->originalReq.managerEpoch = metadata->continueEpoch; + metadata->originalReq.managerSeqno = metadata->continueSeqno; + metadata->originalReq.type = AssignRequestType::Normal; - // if we retry and re-open, we need to use a normal request (no continue) and update the - // seqno - metadata->originalReq.managerEpoch = metadata->continueEpoch; - metadata->originalReq.managerSeqno = metadata->continueSeqno; - metadata->originalReq.type = AssignRequestType::Normal; + bwData->granuleUpdateErrors.send(metadata->originalReq); + throw e; + } - bwData->granuleUpdateErrors.send(metadata->originalReq); - } + TraceEvent(SevError, "GranuleFileUpdaterUnexpectedError", bwData->id) + .error(e) + .detail("Granule", metadata->keyRange) + .detail("GranuleID", startState.granuleID); + ASSERT_WE_THINK(false); + + // if not simulation, kill the BW + if (bwData->fatalError.canBeSet()) { + bwData->fatalError.sendError(e); } throw e; } @@ -1997,10 +1906,6 @@ ACTOR Future blobGranuleLoadHistory(Reference bwData, if (!historyEntryStack.empty()) { Version oldestStartVersion = historyEntryStack.back()->startVersion; - // TODO REMOVE eventually, for debugging - if (stopVersion != oldestStartVersion && stopVersion != invalidVersion) { - fmt::print("Finished, stopVersion={0}, curHistory.version={1}\n", stopVersion, oldestStartVersion); - } ASSERT(stopVersion == oldestStartVersion || stopVersion == invalidVersion); } else { ASSERT(stopVersion == invalidVersion); @@ -2016,6 +1921,7 @@ ACTOR Future blobGranuleLoadHistory(Reference bwData, int i = historyEntryStack.size() - 1; while (i >= 0 && historyEntryStack[i]->startVersion <= stopVersion) { + TEST(true); // Granule skipping history entries loaded by parallel reader i--; } int skipped = historyEntryStack.size() - 1 - i; @@ -2050,11 +1956,14 @@ ACTOR Future blobGranuleLoadHistory(Reference bwData, if (e.code() == error_code_granule_assignment_conflict) { return Void(); } - if (BW_DEBUG) { - fmt::print("Loading blob granule history got unexpected error {}\n", e.name()); + // SplitStorageMetrics explicitly has a SevError if it gets an error, so no errors should propagate here + TraceEvent(SevError, "BlobWorkerUnexpectedErrorLoadGranuleHistory", bwData->id).error(e); + ASSERT_WE_THINK(false); + + // if not simulation, kill the BW + if (bwData->fatalError.canBeSet()) { + bwData->fatalError.sendError(e); } - // TODO this should never happen? - ASSERT(false); throw e; } } @@ -2083,52 +1992,19 @@ ACTOR Future waitForVersion(Reference metadata, Version v ASSERT(metadata->activeCFData.get().isValid()); - if (v == DEBUG_BW_WAIT_VERSION) { - fmt::print("{0}) [{1} - {2}) waiting for {3}\n readable:{4}\n bufferedDelta={5}\n pendingDelta={6}\n " - "durableDelta={7}\n pendingSnapshot={8}\n durableSnapshot={9}\n", - v, - metadata->keyRange.begin.printable().c_str(), - metadata->keyRange.end.printable().c_str(), - v, - metadata->readable.isSet() ? "T" : "F", - metadata->activeCFData.get()->getVersion(), - metadata->pendingDeltaVersion, - metadata->durableDeltaVersion.get(), - metadata->pendingSnapshotVersion, - metadata->durableSnapshotVersion.get()); - } - if (v <= metadata->activeCFData.get()->getVersion() && (v <= metadata->durableDeltaVersion.get() || metadata->durableDeltaVersion.get() == metadata->pendingDeltaVersion) && (v <= metadata->durableSnapshotVersion.get() || metadata->durableSnapshotVersion.get() == metadata->pendingSnapshotVersion)) { - // TODO REMOVE debugging - if (v > metadata->waitForVersionReturned) { - metadata->waitForVersionReturned = v; - } - if (v == DEBUG_BW_WAIT_VERSION) { - fmt::print("{0}) already done\n", v); - } + TEST(true); // Granule read not waiting return Void(); } // wait for change feed version to catch up to ensure we have all data if (metadata->activeCFData.get()->getVersion() < v) { - if (v == DEBUG_BW_WAIT_VERSION) { - fmt::print("{0}) waiting for CF version (currently {1})\n", v, metadata->activeCFData.get()->getVersion()); - } - wait(metadata->activeCFData.get()->whenAtLeast(v)); ASSERT(metadata->activeCFData.get()->getVersion() >= v); - - if (v == DEBUG_BW_WAIT_VERSION) { - fmt::print("{0}) got CF version {1}\n", v, metadata->activeCFData.get()->getVersion()); - } - // TODO REMOVE debugging - if (v > metadata->waitForVersionReturned) { - metadata->waitForVersionReturned = v; - } } // wait for any pending delta and snapshot files as of the moment the change feed version caught up. @@ -2138,31 +2014,17 @@ ACTOR Future waitForVersion(Reference metadata, Version v // If there are mutations that are no longer buffered but have not been // persisted to a delta file that are necessary for the query, wait for them if (pendingDeltaV > metadata->durableDeltaVersion.get() && v > metadata->durableDeltaVersion.get()) { - if (v == DEBUG_BW_WAIT_VERSION) { - fmt::print("{0}) waiting for DDV {1} < {2}\n", v, metadata->durableDeltaVersion.get(), pendingDeltaV); - } - + TEST(true); // Granule read waiting for pending delta wait(metadata->durableDeltaVersion.whenAtLeast(pendingDeltaV)); ASSERT(metadata->durableDeltaVersion.get() >= pendingDeltaV); - - if (v == DEBUG_BW_WAIT_VERSION) { - fmt::print("{0}) got DDV {1} >= {2}\n", v, metadata->durableDeltaVersion.get(), pendingDeltaV); - } } // This isn't strictly needed, but if we're in the process of re-snapshotting, we'd likely rather // return that snapshot file than the previous snapshot file and all its delta files. if (pendingSnapshotV > metadata->durableSnapshotVersion.get() && v > metadata->durableSnapshotVersion.get()) { - if (v == DEBUG_BW_WAIT_VERSION) { - fmt::print("{0}) waiting for DSV {1} < {2}\n", v, metadata->durableSnapshotVersion.get(), pendingSnapshotV); - } - + TEST(true); // Granule read waiting for pending snapshot wait(metadata->durableSnapshotVersion.whenAtLeast(pendingSnapshotV)); ASSERT(metadata->durableSnapshotVersion.get() >= pendingSnapshotV); - - if (v == DEBUG_BW_WAIT_VERSION) { - fmt::print("{0}) got DSV {1} >= {2}\n", v, metadata->durableSnapshotVersion.get(), pendingSnapshotV); - } } // There is a race here - we wait for pending delta files before this to finish, but while we do, we @@ -2170,27 +2032,16 @@ ACTOR Future waitForVersion(Reference metadata, Version v // file instead of in memory mutations, so we wait for that delta file to complete if (metadata->pendingDeltaVersion >= v) { - if (v == DEBUG_BW_WAIT_VERSION) { - fmt::print("{0}) waiting for DDV again {1} < {2}\n", v, metadata->durableDeltaVersion.get(), v); - } - + TEST(true); // Granule mutations flushed while waiting for files to complete wait(metadata->durableDeltaVersion.whenAtLeast(v)); ASSERT(metadata->durableDeltaVersion.get() >= v); - - if (v == DEBUG_BW_WAIT_VERSION) { - fmt::print("{0}) got DDV again {1} >= {2}\n", v, metadata->durableDeltaVersion.get(), v); - } - } - - if (v == DEBUG_BW_WAIT_VERSION) { - fmt::print("{0}) done\n", v); } return Void(); } ACTOR Future doBlobGranuleFileRequest(Reference bwData, BlobGranuleFileRequest req) { - if (BW_REQUEST_DEBUG || DEBUG_BW_WAIT_VERSION == req.readVersion) { + if (BW_REQUEST_DEBUG) { fmt::print("BW {0} processing blobGranuleFileRequest for range [{1} - {2}) @ {3}\n", bwData->id.toString(), req.keyRange.begin.printable(), @@ -2212,7 +2063,7 @@ ACTOR Future doBlobGranuleFileRequest(Reference bwData, Bl for (auto& r : checkRanges) { bool isValid = r.value().activeMetadata.isValid(); if (lastRangeEnd < r.begin() || !isValid) { - if (BW_REQUEST_DEBUG || DEBUG_BW_WAIT_VERSION == req.readVersion) { + if (BW_REQUEST_DEBUG) { fmt::print("No {0} blob data for [{1} - {2}) in request range [{3} - {4}), skipping request\n", isValid ? "" : "valid", lastRangeEnd.printable(), @@ -2227,7 +2078,7 @@ ACTOR Future doBlobGranuleFileRequest(Reference bwData, Bl lastRangeEnd = r.end(); } if (lastRangeEnd < req.keyRange.end) { - if (BW_REQUEST_DEBUG || DEBUG_BW_WAIT_VERSION == req.readVersion) { + if (BW_REQUEST_DEBUG) { fmt::print("No blob data for [{0} - {1}) in request range [{2} - {3}), skipping request\n", lastRangeEnd.printable(), req.keyRange.end.printable(), @@ -2263,6 +2114,7 @@ ACTOR Future doBlobGranuleFileRequest(Reference bwData, Bl state GranuleFiles chunkFiles; if (metadata->initialSnapshotVersion > req.readVersion) { + TEST(true); // Granule Time Travel Read // this is a time travel query, find previous granule if (metadata->historyLoaded.canBeSet()) { choose { @@ -2279,20 +2131,6 @@ ACTOR Future doBlobGranuleFileRequest(Reference bwData, Bl // FIXME: use skip pointers here Version expectedEndVersion = metadata->initialSnapshotVersion; if (cur.isValid()) { - // TODO REMOVE, useful for debugging for now - if (cur->endVersion != expectedEndVersion) { - fmt::print("Active granule [{0} - {1}) does not have history ancestor!!. Start is {2}, " - "ancestor is [{3} - {4}) ({5}) V[{6} - {7}). SearchKey={8}\n", - metadata->keyRange.begin.printable(), - metadata->keyRange.end.printable(), - expectedEndVersion, - cur->range.begin.printable(), - cur->range.end.printable(), - cur->granuleID.toString(), - cur->startVersion, - cur->endVersion, - historySearchKey.printable()); - } ASSERT(cur->endVersion == expectedEndVersion); } while (cur.isValid() && req.readVersion < cur->startVersion) { @@ -2307,7 +2145,7 @@ ACTOR Future doBlobGranuleFileRequest(Reference bwData, Bl throw blob_granule_transaction_too_old(); } - if (BW_REQUEST_DEBUG || DEBUG_BW_WAIT_VERSION == req.readVersion) { + if (BW_REQUEST_DEBUG) { fmt::print("[{0} - {1}) @ {2} time traveled back to {3} [{4} - {5}) @ [{6} - {7})\n", req.keyRange.begin.printable(), req.keyRange.end.printable(), @@ -2335,25 +2173,10 @@ ACTOR Future doBlobGranuleFileRequest(Reference bwData, Bl ASSERT(!chunkFiles.snapshotFiles.empty()); ASSERT(!chunkFiles.deltaFiles.empty()); - // TODO remove eventually, for help debugging asserts - if (chunkFiles.deltaFiles.back().version <= req.readVersion || - chunkFiles.snapshotFiles.front().version > req.readVersion) { - fmt::print("Time Travel read version {0} out of bounds!\n current granule initial version: {1}\n " - "snapshot files ({2}):\n", - req.readVersion, - metadata->initialSnapshotVersion, - chunkFiles.snapshotFiles.size()); - for (auto& f : chunkFiles.snapshotFiles) { - fmt::print(" {0}}\n", f.version); - } - fmt::print(" delta files {0}:\n", chunkFiles.deltaFiles.size()); - for (auto& f : chunkFiles.deltaFiles) { - fmt::print(" {0}\n", f.version); - } - } ASSERT(chunkFiles.deltaFiles.back().version > req.readVersion); ASSERT(chunkFiles.snapshotFiles.front().version <= req.readVersion); } else { + TEST(true); // Granule Active Read // this is an active granule query loop { if (!metadata->activeCFData.get().isValid() || !metadata->cancelled.canBeSet()) { @@ -2373,18 +2196,20 @@ ACTOR Future doBlobGranuleFileRequest(Reference bwData, Bl when(wait(metadata->cancelled.getFuture())) { throw wrong_shard_server(); } } } catch (Error& e) { - // we can get change feed cancelled from whenAtLeast. This is effectively + // We can get change feed cancelled from whenAtLeast. This means the change feed may retry, or + // may be cancelled. Wait a bit and try again to see if (e.code() == error_code_change_feed_popped) { + TEST(true); // Change feed popped while read waiting throw wrong_shard_server(); } if (e.code() != error_code_change_feed_cancelled) { throw e; } + TEST(true); // Change feed switched while read waiting // wait 1ms and try again wait(delay(0.001)); } - if ((BW_REQUEST_DEBUG || DEBUG_BW_WAIT_VERSION == req.readVersion) && - metadata->activeCFData.get().isValid()) { + if ((BW_REQUEST_DEBUG) && metadata->activeCFData.get().isValid()) { fmt::print("{0} - {1}) @ {2} hit CF change, restarting waitForVersion\n", req.keyRange.begin.printable().c_str(), req.keyRange.end.printable().c_str(), @@ -2540,6 +2365,7 @@ ACTOR Future handleBlobGranuleFileRequest(Reference bwData return Void(); } +// FIXME: move this up by other granule state stuff like BGUF ACTOR Future openGranule(Reference bwData, AssignBlobRangeRequest req) { ASSERT(req.type != AssignRequestType::Continue); state Transaction tr(bwData->db); @@ -2561,12 +2387,13 @@ ACTOR Future openGranule(Reference bwData, As info.changeFeedStartVersion = invalidVersion; state Future> fLockValue = tr.get(lockKey); - state Future> fHistory = getLatestGranuleHistory(&tr, req.keyRange); + Future> fHistory = getLatestGranuleHistory(&tr, req.keyRange); Optional history = wait(fHistory); info.history = history; Optional prevLockValue = wait(fLockValue); state bool hasPrevOwner = prevLockValue.present(); if (hasPrevOwner) { + TEST(true); // Granule open found previous owner std::tuple prevOwner = decodeBlobGranuleLockValue(prevLockValue.get()); acquireGranuleLock(req.managerEpoch, req.managerSeqno, std::get<0>(prevOwner), std::get<1>(prevOwner)); info.granuleID = std::get<2>(prevOwner); @@ -2601,9 +2428,6 @@ ACTOR Future openGranule(Reference bwData, As info.changeFeedStartVersion = info.previousDurableVersion; } else { // else we are first, no need to check for owner conflict - // FIXME: use actual 16 bytes of UID instead of converting it to 32 character string and - // then that to bytes - if (info.history.present()) { // if this granule is derived from a split or merge, this history entry is already // present (written by the blob manager) @@ -2614,7 +2438,7 @@ ACTOR Future openGranule(Reference bwData, As info.granuleID = deterministicRandom()->randomUniqueID(); } wait(updateChangeFeed( - &tr, StringRef(info.granuleID.toString()), ChangeFeedStatus::CHANGE_FEED_CREATE, req.keyRange)); + &tr, granuleIDToCFKey(info.granuleID), ChangeFeedStatus::CHANGE_FEED_CREATE, req.keyRange)); info.doSnapshot = true; info.previousDurableVersion = invalidVersion; } @@ -2626,6 +2450,7 @@ ACTOR Future openGranule(Reference bwData, As // ret.previousChangeFeedId, and the previous durable version will come from the previous // granules if (info.history.present() && info.history.get().value.parentGranules.size() > 0) { + TEST(true); // Granule open found parent // TODO change this for merge ASSERT(info.history.get().value.parentGranules.size() == 1); state KeyRange parentGranuleRange = info.history.get().value.parentGranules[0].first; @@ -2650,10 +2475,12 @@ ACTOR Future openGranule(Reference bwData, As } if (granuleSplitState.first == BlobGranuleSplitState::Assigned) { + TEST(true); // Granule open found granule in assign state // was already assigned, use change feed start version ASSERT(granuleSplitState.second > 0); info.changeFeedStartVersion = granuleSplitState.second; } else if (granuleSplitState.first == BlobGranuleSplitState::Initialized) { + TEST(true); // Granule open found granule in initialized state wait(updateGranuleSplitState(&tr, info.parentGranule.get().first, info.parentGranule.get().second, @@ -2662,6 +2489,7 @@ ACTOR Future openGranule(Reference bwData, As // change feed was created as part of this transaction, changeFeedStartVersion // will be set later } else { + TEST(true); // Granule open found granule in done state // this sub-granule is done splitting, no need for split logic. info.parentGranule.reset(); } @@ -2763,8 +2591,6 @@ static bool changeBlobRange(Reference bwData, bool selfReassign, std::vector>& toWaitOut, Optional assignType = Optional()) { - // since changeBlobRange is used for assigns and revokes, - // we assert that assign type is specified iff this is an ASSERT(active == assignType.present()); if (BW_DEBUG) { @@ -2907,31 +2733,6 @@ static bool resumeBlobRange(Reference bwData, KeyRange keyRange, return true; } -ACTOR Future registerBlobWorker(Reference bwData, BlobWorkerInterface interf) { - state Reference tr = makeReference(bwData->db); - loop { - tr->setOption(FDBTransactionOptions::ACCESS_SYSTEM_KEYS); - tr->setOption(FDBTransactionOptions::PRIORITY_SYSTEM_IMMEDIATE); - try { - Key blobWorkerListKey = blobWorkerListKeyFor(interf.id()); - tr->addReadConflictRange(singleKeyRange(blobWorkerListKey)); - tr->set(blobWorkerListKey, blobWorkerListValue(interf)); - - wait(tr->commit()); - - if (BW_DEBUG) { - fmt::print("Registered blob worker {}\n", interf.id().toString()); - } - return Void(); - } catch (Error& e) { - if (BW_DEBUG) { - fmt::print("Registering blob worker {0} got error {1}\n", interf.id().toString(), e.name()); - } - wait(tr->onError(e)); - } - } -} - // the contract of handleRangeAssign and handleRangeRevoke is that they change the mapping before doing any waiting. // This ensures GetGranuleAssignment returns an up-to-date set of ranges ACTOR Future handleRangeAssign(Reference bwData, @@ -2968,6 +2769,9 @@ ACTOR Future handleRangeAssign(Reference bwData, } return Void(); } catch (Error& e) { + if (e.code() == error_code_operation_cancelled) { + throw e; + } if (BW_DEBUG) { fmt::print("AssignRange [{0} - {1}) ({2}, {3}) in BW {4} got error {5}\n", req.keyRange.begin.printable().c_str(), @@ -2987,11 +2791,21 @@ ACTOR Future handleRangeAssign(Reference bwData, if (canReplyWith(e)) { req.reply.sendError(e); - // TODO: should we just return here rather than throw and kill BW } } - throw; + TraceEvent(SevError, "BlobWorkerUnexpectedErrorRangeAssign", bwData->id) + .error(e) + .detail("Range", req.keyRange) + .detail("ManagerEpoch", req.managerEpoch) + .detail("SeqNo", req.managerSeqno); + ASSERT_WE_THINK(false); + + // if not simulation, kill the BW + if (bwData->fatalError.canBeSet()) { + bwData->fatalError.sendError(e); + } + throw e; } } @@ -3019,6 +2833,32 @@ ACTOR Future handleRangeRevoke(Reference bwData, RevokeBlo } } +ACTOR Future registerBlobWorker(Reference bwData, BlobWorkerInterface interf) { + state Reference tr = makeReference(bwData->db); + loop { + tr->setOption(FDBTransactionOptions::ACCESS_SYSTEM_KEYS); + tr->setOption(FDBTransactionOptions::PRIORITY_SYSTEM_IMMEDIATE); + try { + Key blobWorkerListKey = blobWorkerListKeyFor(interf.id()); + // FIXME: should be able to remove this conflict range + tr->addReadConflictRange(singleKeyRange(blobWorkerListKey)); + tr->set(blobWorkerListKey, blobWorkerListValue(interf)); + + wait(tr->commit()); + + if (BW_DEBUG) { + fmt::print("Registered blob worker {}\n", interf.id().toString()); + } + return Void(); + } catch (Error& e) { + if (BW_DEBUG) { + fmt::print("Registering blob worker {0} got error {1}\n", interf.id().toString(), e.name()); + } + wait(tr->onError(e)); + } + } +} + ACTOR Future monitorRemoval(Reference bwData) { state Key blobWorkerListKey = blobWorkerListKeyFor(bwData->id); loop { @@ -3030,6 +2870,7 @@ ACTOR Future monitorRemoval(Reference bwData) { Optional val = wait(tr.get(blobWorkerListKey)); if (!val.present()) { + TEST(true); // Blob worker found out BM killed it from reading DB return Void(); } @@ -3054,21 +2895,17 @@ ACTOR Future runGRVChecks(Reference bwData) { loop { // only do grvs to get committed version if we need it to persist delta files while (bwData->grvVersion.numWaiting() == 0) { - // printf("GRV checker sleeping\n"); wait(bwData->doGRVCheck.getFuture()); bwData->doGRVCheck = Promise(); - // printf("GRV checker waking: %d pending\n", bwData->grvVersion.numWaiting()); } // batch potentially multiple delta files into one GRV, and also rate limit GRVs for this worker - wait(delay(0.1)); // TODO KNOB? - // printf("GRV checker doing grv @ %.2f\n", now()); + wait(delay(SERVER_KNOBS->BLOB_WORKER_BATCH_GRV_INTERVAL)); tr.reset(); try { Version readVersion = wait(tr.getReadVersion()); ASSERT(readVersion >= bwData->grvVersion.get()); - // printf("GRV checker got GRV %lld\n", readVersion); bwData->grvVersion.set(readVersion); ++bwData->stats.commitVersionChecks; @@ -3257,6 +3094,10 @@ ACTOR Future blobWorker(BlobWorkerInterface bwInterf, TraceEvent("BlobWorkerRemoved", self->id); break; } + when(wait(self->fatalError.getFuture())) { + TraceEvent(SevError, "BlobWorkerActorCollectionFatalErrorNotError", self->id); + ASSERT(false); + } } } catch (Error& e) { if (e.code() == error_code_operation_cancelled) { From 4b254d259ceaa02feea2befa8e0e22deaf509b1d Mon Sep 17 00:00:00 2001 From: Josh Slocum Date: Thu, 10 Mar 2022 11:54:57 -0600 Subject: [PATCH 309/413] Ensuring BM split retry is idempotent --- fdbserver/BlobManager.actor.cpp | 82 ++++++++++++++++++++++----- fdbserver/BlobWorker.actor.cpp | 6 +- fdbserver/ClusterController.actor.cpp | 1 - 3 files changed, 71 insertions(+), 18 deletions(-) diff --git a/fdbserver/BlobManager.actor.cpp b/fdbserver/BlobManager.actor.cpp index c914b13ae7..a20a6a40eb 100644 --- a/fdbserver/BlobManager.actor.cpp +++ b/fdbserver/BlobManager.actor.cpp @@ -883,24 +883,15 @@ ACTOR Future maybeSplitRange(Reference bmData, coalescedRanges.push_back(coalescedRanges.arena(), newRanges.back()); ASSERT(coalescedRanges.size() == SERVER_KNOBS->BG_MAX_SPLIT_FANOUT + 1); if (BM_DEBUG) { - fmt::print( - "Downsampled split from {0} -> {1} granules", newRanges.size() - 1, SERVER_KNOBS->BG_MAX_SPLIT_FANOUT); + fmt::print("Downsampled split from {0} -> {1} granules\n", + newRanges.size() - 1, + SERVER_KNOBS->BG_MAX_SPLIT_FANOUT); } newRanges = coalescedRanges; ASSERT(newRanges.size() <= SERVER_KNOBS->BG_MAX_SPLIT_FANOUT + 1); } - if (BM_DEBUG) { - fmt::print("Splitting range [{0} - {1}) into {2} granules @ {3}:\n", - granuleRange.begin.printable(), - granuleRange.end.printable(), - newRanges.size() - 1, - latestVersion); - for (int i = 0; i < newRanges.size(); i++) { - fmt::print(" {}\n", newRanges[i].printable()); - } - } ASSERT(granuleRange.begin == newRanges.front()); ASSERT(granuleRange.end == newRanges.back()); @@ -913,6 +904,19 @@ ACTOR Future maybeSplitRange(Reference bmData, newGranuleIDs.push_back(deterministicRandom()->randomUniqueID()); } + if (BM_DEBUG) { + fmt::print("Splitting range [{0} - {1}) into {2} granules @ {3}:\n", + granuleRange.begin.printable(), + granuleRange.end.printable(), + newRanges.size() - 1, + latestVersion); + for (int i = 0; i < newRanges.size(); i++) { + fmt::print(" {}:{}\n", + (i < newGranuleIDs.size() ? newGranuleIDs[i] : UID()).toString().substr(0, 6).c_str(), + newRanges[i].printable()); + } + } + // Need to split range. Persist intent to split and split metadata to DB BEFORE sending split assignments to blob // workers, so that nothing is lost on blob manager recovery loop { @@ -925,6 +929,47 @@ ACTOR Future maybeSplitRange(Reference bmData, // make sure we're still manager when this transaction gets committed wait(checkManagerLock(tr, bmData)); + // TODO can do this + lock in parallel + // Read splitState to see if anything was committed instead of reading granule mapping because we don't want + // to conflict with mapping changes/reassignments + state RangeResult existingState = + wait(tr->getRange(blobGranuleSplitKeyRangeFor(granuleID), SERVER_KNOBS->BG_MAX_SPLIT_FANOUT + 2)); + ASSERT_WE_THINK(!existingState.more && existingState.size() <= SERVER_KNOBS->BG_MAX_SPLIT_FANOUT + 1); + // maybe someone decreased the knob, we should gracefully handle it not in simulation + if (existingState.more || existingState.size() > SERVER_KNOBS->BG_MAX_SPLIT_FANOUT) { + RangeResult tryAgain = wait(tr->getRange(blobGranuleSplitKeyRangeFor(granuleID), 10000)); + ASSERT(!tryAgain.more); + existingState = tryAgain; + } + if (!existingState.empty()) { + // Something was previously committed, we must go with that decision. + // Read its boundaries and override our planned split boundaries + TEST(true); // Overriding split ranges with existing ones from DB + RangeResult existingBoundaries = wait(tr->getRange( + KeyRangeRef(granuleRange.begin.withPrefix(blobGranuleMappingKeys.begin), + keyAfter(granuleRange.end).withPrefix(blobGranuleMappingKeys.begin)), + existingState.size() + 1)); // +1 because this is boundaries and existingState was granules + ASSERT(!existingBoundaries.more); + ASSERT(existingBoundaries.size() == existingState.size() + 1); + newRanges.clear(); + newRanges.arena().dependsOn(existingBoundaries.arena()); + for (auto& it : existingBoundaries) { + newRanges.push_back(newRanges.arena(), it.key.removePrefix(blobGranuleMappingKeys.begin)); + } + ASSERT(newRanges.front() == granuleRange.begin); + ASSERT(newRanges.back() == granuleRange.end); + if (BM_DEBUG) { + fmt::print("Replaced old range splits for [{0} - {1}) with {2}:\n", + granuleRange.begin.printable(), + granuleRange.end.printable(), + newRanges.size() - 1); + for (int i = 0; i < newRanges.size(); i++) { + fmt::print(" {}\n", newRanges[i].printable()); + } + } + break; + } + // acquire lock for old granule to make sure nobody else modifies it state Key lockKey = blobGranuleLockKeyFor(granuleRange); Optional lockValue = wait(tr->get(lockKey)); @@ -955,8 +1000,8 @@ ACTOR Future maybeSplitRange(Reference bmData, if (!(bmData->epoch > ownerEpoch || (bmData->epoch == ownerEpoch && newLockSeqno > ownerSeqno))) { fmt::print("BM seqno for granule [{0} - {1}) out of order for lock! manager: ({2}, {3}), owner: " "({4}, {5}})\n", - granuleRange.begin.printable().c_str(), - granuleRange.end.printable().c_str(), + granuleRange.begin.printable(), + granuleRange.end.printable(), bmData->epoch, newLockSeqno, ownerEpoch, @@ -966,6 +1011,11 @@ ACTOR Future maybeSplitRange(Reference bmData, } else if (bmData->epoch == ownerEpoch && newLockSeqno < ownerSeqno) { // we retried, and between retries we reassigned this range elsewhere. Cancel this split TEST(true); // BM maybe split cancelled by subsequent move + if (BM_DEBUG) { + fmt::print("Splitting range [{0} - {1}) cancelled by move elsewhere!\n", + granuleRange.begin.printable(), + granuleRange.end.printable()); + } return Void(); } @@ -1036,6 +1086,10 @@ ACTOR Future maybeSplitRange(Reference bmData, bmData->rangesToAssign.send(raAssignSplit); } + // ensure the new assignments actually got processed and the split boundaries are reflected in the granule mapping + // before returning + wait(bmData->rangesToAssign.onEmpty()); + return Void(); } diff --git a/fdbserver/BlobWorker.actor.cpp b/fdbserver/BlobWorker.actor.cpp index 4ec0c18886..97e86fdc4d 100644 --- a/fdbserver/BlobWorker.actor.cpp +++ b/fdbserver/BlobWorker.actor.cpp @@ -329,12 +329,12 @@ ACTOR Future updateGranuleSplitState(Transaction* tr, BlobGranuleSplitState newState) { state KeyRange currentRange = blobGranuleSplitKeyRangeFor(parentGranuleID); - state RangeResult totalState = wait(tr->getRange(currentRange, SERVER_KNOBS->BG_MAX_SPLIT_FANOUT + 2)); + state RangeResult totalState = wait(tr->getRange(currentRange, SERVER_KNOBS->BG_MAX_SPLIT_FANOUT + 1)); // FIXME: remove above conflict range? tr->addWriteConflictRange(currentRange); - ASSERT_WE_THINK(!totalState.more && totalState.size() <= SERVER_KNOBS->BG_MAX_SPLIT_FANOUT + 1); + ASSERT_WE_THINK(!totalState.more && totalState.size() <= SERVER_KNOBS->BG_MAX_SPLIT_FANOUT); // maybe someone decreased the knob, we should gracefully handle it not in simulation - if (totalState.more || totalState.size() > SERVER_KNOBS->BG_MAX_SPLIT_FANOUT + 1) { + if (totalState.more || totalState.size() > SERVER_KNOBS->BG_MAX_SPLIT_FANOUT) { RangeResult tryAgain = wait(tr->getRange(currentRange, 10000)); ASSERT(!tryAgain.more); totalState = tryAgain; diff --git a/fdbserver/ClusterController.actor.cpp b/fdbserver/ClusterController.actor.cpp index 36a0cd3228..899522f229 100644 --- a/fdbserver/ClusterController.actor.cpp +++ b/fdbserver/ClusterController.actor.cpp @@ -2125,7 +2125,6 @@ ACTOR Future getNextBMEpoch(ClusterControllerData* self) { wait(tr->commit()); return newEpoch; } catch (Error& e) { - printf("Acquiring blob manager lock got error %s\n", e.name()); wait(tr->onError(e)); } } From d7367dad99b965e1b6f1a370bd4157cb41dc93e5 Mon Sep 17 00:00:00 2001 From: Josh Slocum Date: Thu, 10 Mar 2022 11:56:22 -0600 Subject: [PATCH 310/413] Simplifying CF gap validation to ignore SS CF fetches --- fdbserver/storageserver.actor.cpp | 7 +++++++ fdbserver/workloads/BlobGranuleVerifier.actor.cpp | 7 ++++--- 2 files changed, 11 insertions(+), 3 deletions(-) diff --git a/fdbserver/storageserver.actor.cpp b/fdbserver/storageserver.actor.cpp index b35fe3f611..483283d470 100644 --- a/fdbserver/storageserver.actor.cpp +++ b/fdbserver/storageserver.actor.cpp @@ -2032,6 +2032,13 @@ ACTOR Future> getChangeFeedMutations(Stor // gap validation while (memoryVerifyIdx < memoryReply.mutations.size() && version > memoryReply.mutations[memoryVerifyIdx].version) { + if (req.canReadPopped) { + // There are weird cases where SS fetching mixed with SS durability and popping can mean there are + // gaps before the popped version temporarily + memoryVerifyIdx++; + continue; + } + // There is a case where this can happen - if we wait on a fetching change feed, and the feed is // popped while we wait, we could have copied the memory mutations into memoryReply before the // pop, but they may or may not have been skipped writing to disk diff --git a/fdbserver/workloads/BlobGranuleVerifier.actor.cpp b/fdbserver/workloads/BlobGranuleVerifier.actor.cpp index 04aa5e936d..c8f69f3c6f 100644 --- a/fdbserver/workloads/BlobGranuleVerifier.actor.cpp +++ b/fdbserver/workloads/BlobGranuleVerifier.actor.cpp @@ -534,9 +534,10 @@ struct BlobGranuleVerifierWorkload : TestWorkload { clients.push_back(timeout(findGranules(cx, this), testDuration, Void())); for (int i = 0; i < threads; i++) { clients.push_back( - timeout(reportErrors(verifyGranules( - cx, this, clientId == 0 && i == 0 && deterministicRandom()->random01() < 0.5), - "BlobGranuleVerifier"), + timeout(reportErrors( + verifyGranules( + cx, this, false /*clientId == 0 && i == 0 && deterministicRandom()->random01() < 0.5*/), + "BlobGranuleVerifier"), testDuration, Void())); } From ba35e7908a29ab5d1ba45cefab22ea779a76a0d5 Mon Sep 17 00:00:00 2001 From: Josh Slocum Date: Thu, 10 Mar 2022 12:11:44 -0600 Subject: [PATCH 311/413] off by one --- fdbserver/BlobManager.actor.cpp | 9 +++++---- 1 file changed, 5 insertions(+), 4 deletions(-) diff --git a/fdbserver/BlobManager.actor.cpp b/fdbserver/BlobManager.actor.cpp index a20a6a40eb..abb16038e5 100644 --- a/fdbserver/BlobManager.actor.cpp +++ b/fdbserver/BlobManager.actor.cpp @@ -945,10 +945,11 @@ ACTOR Future maybeSplitRange(Reference bmData, // Something was previously committed, we must go with that decision. // Read its boundaries and override our planned split boundaries TEST(true); // Overriding split ranges with existing ones from DB - RangeResult existingBoundaries = wait(tr->getRange( - KeyRangeRef(granuleRange.begin.withPrefix(blobGranuleMappingKeys.begin), - keyAfter(granuleRange.end).withPrefix(blobGranuleMappingKeys.begin)), - existingState.size() + 1)); // +1 because this is boundaries and existingState was granules + RangeResult existingBoundaries = + wait(tr->getRange(KeyRangeRef(granuleRange.begin.withPrefix(blobGranuleMappingKeys.begin), + keyAfter(granuleRange.end).withPrefix(blobGranuleMappingKeys.begin)), + existingState.size() + 2)); + // +2 because this is boundaries and existingState was granules, and to ensure it doesn't set more ASSERT(!existingBoundaries.more); ASSERT(existingBoundaries.size() == existingState.size() + 1); newRanges.clear(); From c35e020da73d4045ad0f432fc697f54b702c8810 Mon Sep 17 00:00:00 2001 From: Josh Slocum Date: Thu, 10 Mar 2022 12:38:23 -0600 Subject: [PATCH 312/413] SS and BM test macros --- fdbserver/BlobManager.actor.cpp | 19 +++++++++++++++++-- fdbserver/storageserver.actor.cpp | 9 +++++++++ .../workloads/BlobGranuleVerifier.actor.cpp | 1 + 3 files changed, 27 insertions(+), 2 deletions(-) diff --git a/fdbserver/BlobManager.actor.cpp b/fdbserver/BlobManager.actor.cpp index abb16038e5..7c684dacac 100644 --- a/fdbserver/BlobManager.actor.cpp +++ b/fdbserver/BlobManager.actor.cpp @@ -270,6 +270,7 @@ ACTOR Future>> splitRange(Reference SERVER_KNOBS->BG_SNAPSHOT_FILE_TARGET_BYTES || writeHot) { // only split on bytes and write rate state StorageMetrics splitMetrics; @@ -335,6 +336,7 @@ ACTOR Future>> splitRange(Reference pickWorkerForAssign(Reference bmData) { // wait until there are BWs to pick from while (bmData->workerStats.size() == 0) { + TEST(true); // BM wants to assign range, but no workers available if (BM_DEBUG) { fmt::print("BM {0} waiting for blob workers before assigning granules\n", bmData->epoch); } @@ -460,6 +462,8 @@ ACTOR Future doRangeAssignment(Reference bmData, throw; } + TEST(true); // BM retrying range assign + // We use reliable delivery (getReply), so the broken_promise means the worker is dead, and we may need to retry // somewhere else if (assignment.isAssign) { @@ -533,6 +537,7 @@ ACTOR Future rangeAssigner(Reference bmData) { if (assignment.assign.get().type == AssignRequestType::Continue) { ASSERT(assignment.worker.present()); if (i.range() != assignment.keyRange || i.cvalue() != assignment.worker.get()) { + TEST(true); // BM assignment out of date if (BM_DEBUG) { fmt::print("Out of date re-assign for ({0}, {1}). Assignment must have changed while " "checking split.\n Reassign: [{2} - {3}): {4}\n Existing: [{5} - {6}): {7}\n", @@ -1087,8 +1092,8 @@ ACTOR Future maybeSplitRange(Reference bmData, bmData->rangesToAssign.send(raAssignSplit); } - // ensure the new assignments actually got processed and the split boundaries are reflected in the granule mapping - // before returning + // Ensure the new assignments actually got processed and the split boundaries are reflected in the granule mapping + // before returning. This prevents a race with a subsequent split evaluation wait(bmData->rangesToAssign.onEmpty()); return Void(); @@ -1350,6 +1355,7 @@ ACTOR Future monitorBlobWorkerStatus(Reference bmData, Bl // if it is permanent, the failure monitor will eventually trip. ASSERT(e.code() != error_code_end_of_stream); if (e.code() == error_code_request_maybe_delivered || e.code() == error_code_connection_failed) { + TEST(true); // BM retrying BW monitoring wait(delay(backoff)); backoff = std::min(backoff * SERVER_KNOBS->BLOB_MANAGER_STATUS_EXP_BACKOFF_EXPONENT, SERVER_KNOBS->BLOB_MANAGER_STATUS_EXP_BACKOFF_MAX); @@ -1503,6 +1509,7 @@ static void addAssignment(KeyRangeMap>& map, std::get<0>(old.value()) = UID(); } if (outOfDate.empty() || outOfDate.back() != std::pair(oldWorker, KeyRange(old.range()))) { + TEST(true); // BM Recovery: DB disagrees with workers outOfDate.push_back(std::pair(oldWorker, old.range())); } } else if (oldWorker != UID() && oldWorker != newId && @@ -1510,6 +1517,7 @@ static void addAssignment(KeyRangeMap>& map, // 2 blob workers reported conflicting mappings, add old one to out of date (if not already added by a // previous intersecting range in the split case) if (outOfDate.empty() || outOfDate.back() != std::pair(oldWorker, KeyRange(old.range()))) { + TEST(true); // BM Recovery: Two workers claim ownership of same granule outOfDate.push_back(std::pair(oldWorker, old.range())); } } @@ -1555,6 +1563,8 @@ ACTOR Future recoverBlobManager(Reference bmData) { return Void(); } + TEST(true); // BM doing recovery + wait(delay(0)); // At this point, bmData->workersById is a list of all alive blob workers, but could also include some dead BWs. @@ -1634,6 +1644,7 @@ ACTOR Future recoverBlobManager(Reference bmData) { bmData->workerStats[workerId].numGranulesAssigned = reply.get().assignments.size(); } } else { + TEST(true); // BM Recovery: BW didn't respond to assignments request // SOMEDAY: mark as failed and kill it if (BM_DEBUG) { fmt::print(" Worker {}: failed\n", workerId.toString().substr(0, 5)); @@ -1804,6 +1815,7 @@ ACTOR Future chaosRangeMover(Reference bmData) { // KeyRange isn't hashable and this is only for simulation, so just use toString of range state std::unordered_set alreadyMoved; ASSERT(g_network->isSimulated()); + TEST(true); // BM chaos range mover enabled loop { wait(delay(30.0)); @@ -1906,6 +1918,7 @@ ACTOR Future initializeBlobWorker(Reference self, Recruit // if it failed in an expected way, add some delay before we try to recruit again // on this worker if (newBlobWorker.isError()) { + TEST(true); // BM got error recruiting BW TraceEvent(SevWarn, "BMRecruitmentError").error(newBlobWorker.getError()); if (!newBlobWorker.isError(error_code_recruitment_failed) && !newBlobWorker.isError(error_code_request_maybe_delivered)) { @@ -2561,6 +2574,8 @@ ACTOR Future doLockChecks(Reference bmData) { wait(check.getFuture()); wait(delay(0.5)); // don't do this too often if a lot of conflict + TEST(true); // BM doing lock checks after getting conflicts + state Reference tr = makeReference(bmData->db); loop { diff --git a/fdbserver/storageserver.actor.cpp b/fdbserver/storageserver.actor.cpp index 483283d470..8862544c10 100644 --- a/fdbserver/storageserver.actor.cpp +++ b/fdbserver/storageserver.actor.cpp @@ -2118,9 +2118,11 @@ ACTOR Future> getChangeFeedMutations(Stor // If still empty, that means disk results were filtered out, but skipped all memory results. Add an empty, // either the last version from disk if (reply.mutations.empty() && res.size()) { + TEST(true); // Change feed adding empty version after disk + memory filtered reply.mutations.push_back(reply.arena, MutationsAndVersionRef(lastVersion, lastKnownCommitted)); } } else if (reply.mutations.empty() || reply.mutations.back().version < lastVersion) { + TEST(true); // Change feed adding empty version after disk filtered reply.mutations.push_back(reply.arena, MutationsAndVersionRef(lastVersion, lastKnownCommitted)); } } else { @@ -2131,6 +2133,7 @@ ACTOR Future> getChangeFeedMutations(Stor Version finalVersion = std::min(req.end - 1, dequeVersion); if ((reply.mutations.empty() || reply.mutations.back().version < finalVersion) && remainingLimitBytes > 0 && remainingDurableBytes > 0) { + TEST(true); // Change feed adding empty version after empty results reply.mutations.push_back( reply.arena, MutationsAndVersionRef(finalVersion, finalVersion == dequeVersion ? dequeKnownCommit : 0)); // if we add empty mutation after the last thing in memory, and didn't read from disk, gotAll is true @@ -2305,6 +2308,7 @@ ACTOR Future stopChangeFeedOnMove(StorageServer* data, ChangeFeedStreamReq } return Void(); } + TEST(true); // Change feed moved away cancelling queries // DO NOT call req.reply.onReady before sending - we need to propagate this error through regardless of how far // behind client is req.reply.sendError(wrong_shard_server()); @@ -4759,6 +4763,7 @@ ACTOR Future fetchChangeFeedApplier(StorageServer* data, lastVersion = remoteVersion; versionsFetched++; } else { + TEST(true); // Change feed ignoring write on move because it was popped concurrently if (MUTATION_TRACKING_ENABLED) { for (auto& m : remoteResult[remoteLoc].mutations) { DEBUG_MUTATION("ChangeFeedWriteMoveIgnore", remoteVersion, m, data->thisServerID) @@ -4827,6 +4832,7 @@ ACTOR Future fetchChangeFeedApplier(StorageServer* data, // if we were popped or removed while fetching but it didn't pass the fetch version while writing, clean up here if (versionsFetched > 0 && startVersion < changeFeedInfo->emptyVersion) { + TEST(true); // Change feed cleaning up popped data after move ASSERT(firstVersion != invalidVersion); ASSERT(lastVersion != invalidVersion); Version endClear = std::min(lastVersion + 1, changeFeedInfo->emptyVersion); @@ -4872,6 +4878,7 @@ ACTOR Future fetchChangeFeed(StorageServer* data, auto cleanupPending = data->changeFeedCleanupDurable.find(changeFeedInfo->id); if (cleanupPending != data->changeFeedCleanupDurable.end()) { + TEST(true); // Change feed waiting for dirty previous move to finish TraceEvent(SevDebug, "FetchChangeFeedWaitCleanup", data->thisServerID) .detail("RangeID", changeFeedInfo->id.printable()) .detail("Range", changeFeedInfo->range.toString()) @@ -4964,6 +4971,7 @@ ACTOR Future> fetchChangeFeedMetadata(StorageServer* data, KeyR auto feedCleanup = data->changeFeedCleanupDurable.find(cfEntry.rangeId); if (cfEntry.stopVersion < changeFeedInfo->stopVersion) { + TEST(true); // Change feed updated stop version from fetch metadata changeFeedInfo->stopVersion = cfEntry.stopVersion; addMutationToLog = true; } @@ -5363,6 +5371,7 @@ ACTOR Future fetchKeys(StorageServer* data, AddingShard* shard) { std::unordered_set newChangeFeeds; for (auto& r : ranges) { for (auto& cfInfo : r.value()) { + TEST(true); // SS fetching new change feed that didn't exist when fetch started newChangeFeeds.insert(cfInfo->id); } } diff --git a/fdbserver/workloads/BlobGranuleVerifier.actor.cpp b/fdbserver/workloads/BlobGranuleVerifier.actor.cpp index c8f69f3c6f..09d19c6db5 100644 --- a/fdbserver/workloads/BlobGranuleVerifier.actor.cpp +++ b/fdbserver/workloads/BlobGranuleVerifier.actor.cpp @@ -535,6 +535,7 @@ struct BlobGranuleVerifierWorkload : TestWorkload { for (int i = 0; i < threads; i++) { clients.push_back( timeout(reportErrors( + // TODO change back verifyGranules( cx, this, false /*clientId == 0 && i == 0 && deterministicRandom()->random01() < 0.5*/), "BlobGranuleVerifier"), From 903f7adbc4006d557d8e331dab43301106df99cf Mon Sep 17 00:00:00 2001 From: Josh Slocum Date: Thu, 10 Mar 2022 13:53:06 -0600 Subject: [PATCH 313/413] more cleanup --- fdbclient/NativeAPI.actor.cpp | 1 + fdbserver/BlobGranuleServerCommon.actor.cpp | 17 +++------------ fdbserver/BlobGranuleServerCommon.actor.h | 9 ++------ fdbserver/BlobManager.actor.cpp | 14 ++++++------- fdbserver/BlobWorker.actor.cpp | 23 ++++++++++++++++----- fdbserver/storageserver.actor.cpp | 5 ++++- 6 files changed, 34 insertions(+), 35 deletions(-) diff --git a/fdbclient/NativeAPI.actor.cpp b/fdbclient/NativeAPI.actor.cpp index bb0d0143fc..a747ecf77c 100644 --- a/fdbclient/NativeAPI.actor.cpp +++ b/fdbclient/NativeAPI.actor.cpp @@ -6747,6 +6747,7 @@ Future>> Transaction::getRangeSplitPoints(KeyRange return ::getRangeSplitPoints(trState, keys, chunkSize); } +// TODO REMOVE when correctness clean #define BG_REQUEST_DEBUG false // the blob granule requests are a bit funky because they piggyback off the existing transaction to read from the system diff --git a/fdbserver/BlobGranuleServerCommon.actor.cpp b/fdbserver/BlobGranuleServerCommon.actor.cpp index 7700d6152b..94192bb8d7 100644 --- a/fdbserver/BlobGranuleServerCommon.actor.cpp +++ b/fdbserver/BlobGranuleServerCommon.actor.cpp @@ -45,12 +45,7 @@ ACTOR Future> getLatestGranuleHistory(Transaction* tr, // Gets the files based on the file key range [startKey, endKey) // and populates the files object accordingly -ACTOR Future readGranuleFiles(Transaction* tr, - Key* startKey, - Key endKey, - GranuleFiles* files, - UID granuleID, - bool debug) { +ACTOR Future readGranuleFiles(Transaction* tr, Key* startKey, Key endKey, GranuleFiles* files, UID granuleID) { loop { int lim = BUGGIFY ? 2 : 1000; @@ -85,18 +80,12 @@ ACTOR Future readGranuleFiles(Transaction* tr, break; } } - if (debug) { - printf("Loaded %d snapshot and %d delta files for %s\n", - files->snapshotFiles.size(), - files->deltaFiles.size(), - granuleID.toString().c_str()); - } return Void(); } // Wrapper around readGranuleFiles // Gets all files belonging to the granule with id granule ID -ACTOR Future loadHistoryFiles(Database cx, UID granuleID, bool debug) { +ACTOR Future loadHistoryFiles(Database cx, UID granuleID) { state KeyRange range = blobGranuleFileKeyRangeFor(granuleID); state Key startKey = range.begin; state GranuleFiles files; @@ -106,7 +95,7 @@ ACTOR Future loadHistoryFiles(Database cx, UID granuleID, bool deb try { tr.setOption(FDBTransactionOptions::ACCESS_SYSTEM_KEYS); tr.setOption(FDBTransactionOptions::PRIORITY_SYSTEM_IMMEDIATE); - wait(readGranuleFiles(&tr, &startKey, range.end, &files, granuleID, debug)); + wait(readGranuleFiles(&tr, &startKey, range.end, &files, granuleID)); return files; } catch (Error& e) { wait(tr.onError(e)); diff --git a/fdbserver/BlobGranuleServerCommon.actor.h b/fdbserver/BlobGranuleServerCommon.actor.h index b593cc1361..d48418c951 100644 --- a/fdbserver/BlobGranuleServerCommon.actor.h +++ b/fdbserver/BlobGranuleServerCommon.actor.h @@ -64,12 +64,7 @@ struct GranuleFiles { class Transaction; ACTOR Future> getLatestGranuleHistory(Transaction* tr, KeyRange range); -ACTOR Future readGranuleFiles(Transaction* tr, - Key* startKey, - Key endKey, - GranuleFiles* files, - UID granuleID, - bool debug); +ACTOR Future readGranuleFiles(Transaction* tr, Key* startKey, Key endKey, GranuleFiles* files, UID granuleID); -ACTOR Future loadHistoryFiles(Database cx, UID granuleID, bool debug); +ACTOR Future loadHistoryFiles(Database cx, UID granuleID); #endif diff --git a/fdbserver/BlobManager.actor.cpp b/fdbserver/BlobManager.actor.cpp index 7c684dacac..c8d8a65ea8 100644 --- a/fdbserver/BlobManager.actor.cpp +++ b/fdbserver/BlobManager.actor.cpp @@ -2043,7 +2043,7 @@ ACTOR Future loadHistoryFiles(Reference bmData, U state GranuleFiles files; loop { try { - wait(readGranuleFiles(&tr, &startKey, range.end, &files, granuleID, BM_DEBUG)); + wait(readGranuleFiles(&tr, &startKey, range.end, &files, granuleID)); return files; } catch (Error& e) { wait(tr.onError(e)); @@ -2062,7 +2062,7 @@ ACTOR Future fullyDeleteGranule(Reference self, UID granu } // get files - GranuleFiles files = wait(loadHistoryFiles(self->db, granuleId, BM_DEBUG)); + GranuleFiles files = wait(loadHistoryFiles(self->db, granuleId)); std::vector> deletions; std::vector filesToDelete; // TODO: remove, just for debugging @@ -2134,7 +2134,7 @@ ACTOR Future partiallyDeleteGranule(Reference self, UID g } // get files - GranuleFiles files = wait(loadHistoryFiles(self->db, granuleId, BM_DEBUG)); + GranuleFiles files = wait(loadHistoryFiles(self->db, granuleId)); // represents the version of the latest snapshot file in this granule with G.version < pruneVersion Version latestSnapshotVersion = invalidVersion; @@ -2605,8 +2605,8 @@ ACTOR Future doLockChecks(Reference bmData) { } } -ACTOR Future blobManagerExclusionSafetyCheck(Reference self, - BlobManagerExclusionSafetyCheckRequest req) { +static void blobManagerExclusionSafetyCheck(Reference self, + BlobManagerExclusionSafetyCheckRequest req) { TraceEvent("BMExclusionSafetyCheckBegin", self->id).log(); BlobManagerExclusionSafetyCheckReply reply(true); // make sure at least one blob worker remains after exclusions @@ -2632,8 +2632,6 @@ ACTOR Future blobManagerExclusionSafetyCheck(Reference se TraceEvent("BMExclusionSafetyCheckEnd", self->id).log(); req.reply.send(reply); - - return Void(); } ACTOR Future blobManager(BlobManagerInterface bmInterf, @@ -2692,7 +2690,7 @@ ACTOR Future blobManager(BlobManagerInterface bmInterf, } when(BlobManagerExclusionSafetyCheckRequest exclCheckReq = waitNext(bmInterf.blobManagerExclCheckReq.getFuture())) { - self->addActor.send(blobManagerExclusionSafetyCheck(self, exclCheckReq)); + blobManagerExclusionSafetyCheck(self, exclCheckReq); } when(wait(collection)) { TraceEvent("BlobManagerActorCollectionError"); diff --git a/fdbserver/BlobWorker.actor.cpp b/fdbserver/BlobWorker.actor.cpp index 97e86fdc4d..46ad75f78b 100644 --- a/fdbserver/BlobWorker.actor.cpp +++ b/fdbserver/BlobWorker.actor.cpp @@ -281,7 +281,7 @@ ACTOR Future loadHistoryFiles(Reference bwData, UI state GranuleFiles files; loop { try { - wait(readGranuleFiles(&tr, &startKey, range.end, &files, granuleID, BW_DEBUG)); + wait(readGranuleFiles(&tr, &startKey, range.end, &files, granuleID)); return files; } catch (Error& e) { wait(tr.onError(e)); @@ -296,7 +296,7 @@ ACTOR Future loadPreviousFiles(Transaction* tr, UID granuleID) { // no need to add conflict range for read b/c of granule lock state Key startKey = range.begin; state GranuleFiles files; - wait(readGranuleFiles(tr, &startKey, range.end, &files, granuleID, BW_DEBUG)); + wait(readGranuleFiles(tr, &startKey, range.end, &files, granuleID)); return files; } @@ -967,7 +967,9 @@ ACTOR Future handleCompletedDeltaFile(Reference bwData, if (completedDeltaFile.version > cfStartVersion) { if (BW_DEBUG) { - fmt::print("Popping change feed {0} at {1}\n", cfKey.printable(), completedDeltaFile.version); + fmt::print("Popping change feed {0} at {1}\n", + cfKeyToGranuleID(cfKey).toString().c_str(), + completedDeltaFile.version); } // FIXME: for a write-hot shard, we could potentially batch these and only pop the largest one after several // have completed @@ -1595,7 +1597,8 @@ ACTOR Future blobGranuleUpdateFiles(Reference bwData, DEBUG_MUTATION("BlobWorkerBuffer", deltas.version, delta, bwData->id) .detail("Granule", metadata->keyRange) - .detail("ChangeFeedID", readOldChangeFeed ? oldCFKey.get() : cfKey) + .detail("ChangeFeedID", + cfKeyToGranuleID(readOldChangeFeed ? oldCFKey.get() : cfKey)) .detail("OldChangeFeed", readOldChangeFeed ? "T" : "F"); } metadata->currentDeltas.push_back_deep(metadata->currentDeltas.arena(), deltas); @@ -1990,6 +1993,16 @@ ACTOR Future waitForVersion(Reference metadata, Version v // if we don't have to wait for change feed version to catch up or wait for any pending file writes to complete, // nothing to do + if (BW_REQUEST_DEBUG) { + printf("WFV %lld) CF=%lld, pendingD=%lld, durableD=%lld, pendingS=%lld, durableS=%lld\n", + v, + metadata->activeCFData.get()->getVersion(), + metadata->pendingDeltaVersion, + metadata->durableDeltaVersion.get(), + metadata->pendingSnapshotVersion, + metadata->durableSnapshotVersion.get()); + } + ASSERT(metadata->activeCFData.get().isValid()); if (v <= metadata->activeCFData.get()->getVersion() && @@ -2163,7 +2176,7 @@ ACTOR Future doBlobGranuleFileRequest(Reference bwData, Bl // lazily load files for old granule if not present chunkRange = cur->range; if (!cur->files.isValid() || cur->files.isError()) { - cur->files = loadHistoryFiles(bwData->db, cur->granuleID, BW_DEBUG); + cur->files = loadHistoryFiles(bwData->db, cur->granuleID); } choose { diff --git a/fdbserver/storageserver.actor.cpp b/fdbserver/storageserver.actor.cpp index 8862544c10..7b08e6548d 100644 --- a/fdbserver/storageserver.actor.cpp +++ b/fdbserver/storageserver.actor.cpp @@ -1204,7 +1204,10 @@ public: auto& clientVersions = changeFeedClientVersions[addr]; Version minVersion = version.get(); for (auto& it : clientVersions) { - // fmt::print("Blocked client {0} @ {1}\n", it.first.toString().substr(0, 8), it.second); + /*fmt::print("SS {0} Blocked client {1} @ {2}\n", + thisServerID.toString().substr(0, 4), + it.first.toString().substr(0, 8), + it.second);*/ minVersion = std::min(minVersion, it.second); } return minVersion; From 211e5b579a57eaef67a8992623f667f151253226 Mon Sep 17 00:00:00 2001 From: Josh Slocum Date: Thu, 10 Mar 2022 13:55:42 -0600 Subject: [PATCH 314/413] Adding back debug stream uid to help track down bugs --- fdbclient/NativeAPI.actor.cpp | 5 +++++ fdbclient/StorageServerInterface.h | 4 +++- fdbserver/storageserver.actor.cpp | 4 +++- 3 files changed, 11 insertions(+), 2 deletions(-) diff --git a/fdbclient/NativeAPI.actor.cpp b/fdbclient/NativeAPI.actor.cpp index a747ecf77c..4be06507e8 100644 --- a/fdbclient/NativeAPI.actor.cpp +++ b/fdbclient/NativeAPI.actor.cpp @@ -7788,6 +7788,9 @@ 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; } + // TODO REMOVE + req.debugUID = deterministicRandom()->randomUniqueID(); + results->streams.push_back(it.first.changeFeedStream.getReplyStream(req)); } @@ -7934,6 +7937,8 @@ ACTOR Future singleChangeFeedStream(Reference db, req.range = range; req.canReadPopped = canReadPopped; req.replyBufferSize = replyBufferSize; + // TODO REMOVE + req.debugUID = deterministicRandom()->randomUniqueID(); results->streams.clear(); diff --git a/fdbclient/StorageServerInterface.h b/fdbclient/StorageServerInterface.h index cbe6937f2f..794397cfb2 100644 --- a/fdbclient/StorageServerInterface.h +++ b/fdbclient/StorageServerInterface.h @@ -793,13 +793,15 @@ struct ChangeFeedStreamRequest { KeyRange range; int replyBufferSize = -1; bool canReadPopped = true; + // TODO REMOVE + UID debugUID; ReplyPromiseStream reply; ChangeFeedStreamRequest() {} template void serialize(Ar& ar) { - serializer(ar, rangeID, begin, end, range, reply, spanContext, replyBufferSize, canReadPopped, arena); + serializer(ar, rangeID, begin, end, range, reply, spanContext, replyBufferSize, canReadPopped, debugUID, arena); } }; diff --git a/fdbserver/storageserver.actor.cpp b/fdbserver/storageserver.actor.cpp index 7b08e6548d..86fba6190b 100644 --- a/fdbserver/storageserver.actor.cpp +++ b/fdbserver/storageserver.actor.cpp @@ -7884,7 +7884,9 @@ ACTOR Future serveChangeFeedStreamRequests(StorageServer* self, ChangeFeedStreamRequest req = waitNext(changeFeedStream); // must notify change feed that its shard is moved away ASAP - UID streamUID = deterministicRandom()->randomUniqueID(); + // TODO CHANGE BACK + // UID streamUID = deterministicRandom()->randomUniqueID(); + UID streamUID = req.debugUID; self->actors.add(changeFeedStreamQ(self, req, streamUID) || stopChangeFeedOnMove(self, req, streamUID)); } } From 4f88e0040408667b2a4eb1c5c057492dbb83207e Mon Sep 17 00:00:00 2001 From: Josh Slocum Date: Thu, 10 Mar 2022 14:37:32 -0600 Subject: [PATCH 315/413] Adding change feed pop to set of expected granule errors --- fdbserver/BlobWorker.actor.cpp | 10 +++++++++- 1 file changed, 9 insertions(+), 1 deletion(-) diff --git a/fdbserver/BlobWorker.actor.cpp b/fdbserver/BlobWorker.actor.cpp index 46ad75f78b..fccb73b5a8 100644 --- a/fdbserver/BlobWorker.actor.cpp +++ b/fdbserver/BlobWorker.actor.cpp @@ -1810,7 +1810,15 @@ ACTOR Future blobGranuleUpdateFiles(Reference bwData, } if (e.code() == error_code_granule_assignment_conflict) { - TraceEvent(SevInfo, "GranuleAssignmentConflict", bwData->id).detail("Granule", metadata->keyRange); + TraceEvent(SevInfo, "GranuleAssignmentConflict", bwData->id) + .detail("Granule", metadata->keyRange) + .detail("GranuleID", startState.granuleID); + return Void(); + } + if (e.code() == error_code_change_feed_popped) { + TraceEvent(SevInfo, "GranuleGotChangeFeedPopped", bwData->id) + .detail("Granule", metadata->keyRange) + .detail("GranuleID", startState.granuleID); return Void(); } ++bwData->stats.granuleUpdateErrors; From edb65f4aa9b1a9672fb2bfefbdecbe885c694920 Mon Sep 17 00:00:00 2001 From: Josh Slocum Date: Thu, 10 Mar 2022 16:33:41 -0600 Subject: [PATCH 316/413] Exclude redwood from BGVerifySmall because it can make a root page too large with massive KV pairs in change feeds --- tests/fast/BlobGranuleVerifySmall.toml | 1 + tests/fast/BlobGranuleVerifySmallClean.toml | 1 + 2 files changed, 2 insertions(+) diff --git a/tests/fast/BlobGranuleVerifySmall.toml b/tests/fast/BlobGranuleVerifySmall.toml index b3852c428f..22a4b15ae6 100644 --- a/tests/fast/BlobGranuleVerifySmall.toml +++ b/tests/fast/BlobGranuleVerifySmall.toml @@ -1,5 +1,6 @@ [configuration] blobGranulesEnabled = true +storageEngineExcludeTypes = [3] # FIXME: exclude redwood because WriteDuringRead can write massive KV pairs and we don't chunk change feed data on disk yet [[test]] testTitle = 'BlobGranuleVerifySmall' diff --git a/tests/fast/BlobGranuleVerifySmallClean.toml b/tests/fast/BlobGranuleVerifySmallClean.toml index 9378ead90c..0a7d2a95d6 100644 --- a/tests/fast/BlobGranuleVerifySmallClean.toml +++ b/tests/fast/BlobGranuleVerifySmallClean.toml @@ -1,5 +1,6 @@ [configuration] blobGranulesEnabled = true +storageEngineExcludeTypes = [3] # FIXME: exclude redwood because WriteDuringRead can write massive KV pairs and we don't chunk change feed data on disk yet [[test]] testTitle = 'BlobGranuleVerifySmallClean' From 479ac313ca3a19469d5f40b16e5e175cfb32f25f Mon Sep 17 00:00:00 2001 From: Josh Slocum Date: Thu, 10 Mar 2022 16:19:57 -0600 Subject: [PATCH 317/413] Fixing conflict handling in BM recovery --- fdbserver/BlobManager.actor.cpp | 11 ++++++++--- fdbserver/BlobWorker.actor.cpp | 16 +++++++++------- 2 files changed, 17 insertions(+), 10 deletions(-) diff --git a/fdbserver/BlobManager.actor.cpp b/fdbserver/BlobManager.actor.cpp index c8d8a65ea8..8d1b839e38 100644 --- a/fdbserver/BlobManager.actor.cpp +++ b/fdbserver/BlobManager.actor.cpp @@ -1476,7 +1476,6 @@ ACTOR Future checkBlobWorkerList(Reference bmData, Promis // Resolves these conflicts by comparing the epoch + seqno for the range // Special epoch/seqnos: // (0,0): range is not mapped -// (0,1): range is mapped, but worker is unknown static void addAssignment(KeyRangeMap>& map, const KeyRangeRef& newRange, UID newId, @@ -1493,6 +1492,10 @@ static void addAssignment(KeyRangeMap>& map, int64_t oldSeqno = std::get<2>(old.value()); if (oldEpoch > newEpoch || (oldEpoch == newEpoch && oldSeqno > newSeqno)) { newer.push_back(std::pair(old.range(), std::tuple(oldWorker, oldEpoch, oldSeqno))); + if (old.range() != newRange) { + TEST(true); // BM Recovery: BWs disagree on range boundaries + anyConflicts = true; + } } else { allExistingNewer = false; if (newId != UID() && newEpoch != std::numeric_limits::max()) { @@ -1500,6 +1503,7 @@ static void addAssignment(KeyRangeMap>& map, ASSERT(oldEpoch != newEpoch || oldSeqno != newSeqno); } if (newEpoch == std::numeric_limits::max() && (oldWorker != newId || old.range() != newRange)) { + TEST(true); // BM Recovery: DB disagrees with workers // new one is from DB (source of truth on boundaries) and existing mapping disagrees on boundary or // assignment, do explicit revoke and re-assign to converge anyConflicts = true; @@ -1508,8 +1512,9 @@ static void addAssignment(KeyRangeMap>& map, if (old.range() != newRange) { std::get<0>(old.value()) = UID(); } - if (outOfDate.empty() || outOfDate.back() != std::pair(oldWorker, KeyRange(old.range()))) { - TEST(true); // BM Recovery: DB disagrees with workers + if (oldWorker != UID() && + (outOfDate.empty() || outOfDate.back() != std::pair(oldWorker, KeyRange(old.range())))) { + outOfDate.push_back(std::pair(oldWorker, old.range())); } } else if (oldWorker != UID() && oldWorker != newId && diff --git a/fdbserver/BlobWorker.actor.cpp b/fdbserver/BlobWorker.actor.cpp index fccb73b5a8..066f311163 100644 --- a/fdbserver/BlobWorker.actor.cpp +++ b/fdbserver/BlobWorker.actor.cpp @@ -979,13 +979,15 @@ ACTOR Future handleCompletedDeltaFile(Reference bwData, wait(popFuture); } while (!rollbacksCompleted->empty() && completedDeltaFile.version >= rollbacksCompleted->front().second) { - fmt::print("Granule [{0} - {1}) on BW {2} completed rollback {3} -> {4} with delta file {5}\n", - metadata->keyRange.begin.printable().c_str(), - metadata->keyRange.end.printable().c_str(), - bwData->id.toString().substr(0, 5).c_str(), - rollbacksCompleted->front().second, - rollbacksCompleted->front().first, - completedDeltaFile.version); + if (BW_DEBUG) { + fmt::print("Granule [{0} - {1}) on BW {2} completed rollback {3} -> {4} with delta file {5}\n", + metadata->keyRange.begin.printable().c_str(), + metadata->keyRange.end.printable().c_str(), + bwData->id.toString().substr(0, 5).c_str(), + rollbacksCompleted->front().second, + rollbacksCompleted->front().first, + completedDeltaFile.version); + } rollbacksCompleted->pop_front(); } return Void(); From 2a506f2dff9b7274fa4d7b6d9c30fd251de3f83a Mon Sep 17 00:00:00 2001 From: Josh Slocum Date: Thu, 10 Mar 2022 20:11:37 -0600 Subject: [PATCH 318/413] Updating SS desired every mutation to avoid CF merge deadlock --- fdbclient/NativeAPI.actor.cpp | 20 ++++++++++---------- 1 file changed, 10 insertions(+), 10 deletions(-) diff --git a/fdbclient/NativeAPI.actor.cpp b/fdbclient/NativeAPI.actor.cpp index 4be06507e8..19bdaac6a2 100644 --- a/fdbclient/NativeAPI.actor.cpp +++ b/fdbclient/NativeAPI.actor.cpp @@ -7608,33 +7608,33 @@ ACTOR Future partialChangeFeedStream(StorageServerInterface interf, wait(results.onEmpty()); if (rep.mutations[resultLoc].version >= nextVersion) { results.send(rep.mutations[resultLoc]); + + // check refresh.canBeSet so that, if we are killed after calling one of these callbacks, we + // just skip to the next wait and get actor_cancelled + // FIXME: this is somewhat expensive to do every mutation. + for (auto& it : feedData->storageData) { + if (refresh.canBeSet() && rep.mutations[resultLoc].version > it->desired.get()) { + it->desired.set(rep.mutations[resultLoc].version); + } + } } else { ASSERT(rep.mutations[resultLoc].mutations.empty()); } resultLoc++; } + // if we got the empty version that went backwards, don't decrease nextVersion if (rep.mutations.back().version + 1 > nextVersion) { nextVersion = rep.mutations.back().version + 1; } - // check refresh.canBeSet so that, if we are killed after calling one of these callbacks, we just - // skip to the next wait and get actor_cancelled - if (refresh.canBeSet() && !atLatestVersion && rep.atLatestVersion) { atLatestVersion = true; feedData->notAtLatest.set(feedData->notAtLatest.get() - 1); } - if (refresh.canBeSet() && rep.minStreamVersion > storageData->version.get()) { storageData->version.set(rep.minStreamVersion); } - - for (auto& it : feedData->storageData) { - if (refresh.canBeSet() && rep.mutations.back().version > it->desired.get()) { - it->desired.set(rep.mutations.back().version); - } - } } when(wait(atLatestVersion && replyStream.isEmpty() && results.isEmpty() ? storageData->version.whenAtLeast(nextVersion) From 01e00f3c7d0e81001b2a9de93078f0a9cae0ec81 Mon Sep 17 00:00:00 2001 From: sfc-gh-tclinkenbeard Date: Thu, 10 Mar 2022 23:35:56 -0400 Subject: [PATCH 319/413] Make DDTeamCollection::addTeam private --- fdbserver/DDTeamCollection.h | 52 ++++++++++++++++++------------------ 1 file changed, 26 insertions(+), 26 deletions(-) diff --git a/fdbserver/DDTeamCollection.h b/fdbserver/DDTeamCollection.h index 86cd92e4cf..3b7b47243a 100644 --- a/fdbserver/DDTeamCollection.h +++ b/fdbserver/DDTeamCollection.h @@ -553,6 +553,32 @@ class DDTeamCollection : public ReferenceCounted { void setCheckTeamDelay() { this->checkTeamDelay = Void(); } + // Assume begin to end is sorted by std::sort + // Assume InputIt is iterator to UID + // Note: We must allow creating empty teams because empty team is created when a remote DB is initialized. + // The empty team is used as the starting point to move data to the remote DB + // begin : the start of the team member ID + // end : end of the team member ID + // isIntialTeam : False when the team is added by addTeamsBestOf(); True otherwise, e.g., + // when the team added at init() when we recreate teams by looking up DB + template + void addTeam(InputIt begin, InputIt end, bool isInitialTeam) { + std::vector> newTeamServers; + for (auto i = begin; i != end; ++i) { + if (server_info.find(*i) != server_info.end()) { + newTeamServers.push_back(server_info[*i]); + } + } + + addTeam(newTeamServers, isInitialTeam); + } + + void addTeam(const std::vector>& newTeamServers, + bool isInitialTeam, + bool redundantTeam = false); + + void addTeam(std::set const& team, bool isInitialTeam) { addTeam(team.begin(), team.end(), isInitialTeam); } + public: Database cx; @@ -601,32 +627,6 @@ public: Future init(Reference initTeams, DDEnabledState const& ddEnabledState); - // Assume begin to end is sorted by std::sort - // Assume InputIt is iterator to UID - // Note: We must allow creating empty teams because empty team is created when a remote DB is initialized. - // The empty team is used as the starting point to move data to the remote DB - // begin : the start of the team member ID - // end : end of the team member ID - // isIntialTeam : False when the team is added by addTeamsBestOf(); True otherwise, e.g., - // when the team added at init() when we recreate teams by looking up DB - template - void addTeam(InputIt begin, InputIt end, bool isInitialTeam) { - std::vector> newTeamServers; - for (auto i = begin; i != end; ++i) { - if (server_info.find(*i) != server_info.end()) { - newTeamServers.push_back(server_info[*i]); - } - } - - addTeam(newTeamServers, isInitialTeam); - } - - void addTeam(const std::vector>& newTeamServers, - bool isInitialTeam, - bool redundantTeam = false); - - void addTeam(std::set const& team, bool isInitialTeam) { addTeam(team.begin(), team.end(), isInitialTeam); } - // Create server teams based on machine teams // Before the number of machine teams reaches the threshold, build a machine team for each server team // When it reaches the threshold, first try to build a server team with existing machine teams; if failed, From 7a5105a9d57e9dd7b821383307ffcfbf2abdb119 Mon Sep 17 00:00:00 2001 From: sfc-gh-tclinkenbeard Date: Fri, 11 Mar 2022 00:10:12 -0400 Subject: [PATCH 320/413] Add type safety to boolean parameters in DDTeamCollection --- fdbserver/DDTeamCollection.actor.cpp | 74 +++++++++++++++------------- fdbserver/DDTeamCollection.h | 24 +++++---- 2 files changed, 55 insertions(+), 43 deletions(-) diff --git a/fdbserver/DDTeamCollection.actor.cpp b/fdbserver/DDTeamCollection.actor.cpp index 75747c9c65..7165f3d652 100644 --- a/fdbserver/DDTeamCollection.actor.cpp +++ b/fdbserver/DDTeamCollection.actor.cpp @@ -23,6 +23,10 @@ #include "flow/actorcompiler.h" // This must be the last #include. FDB_DEFINE_BOOLEAN_PARAM(IsPrimary); +FDB_DEFINE_BOOLEAN_PARAM(IsInitialTeam); +FDB_DEFINE_BOOLEAN_PARAM(IsRedundantTeam); +FDB_DEFINE_BOOLEAN_PARAM(IsBadTeam); +FDB_DEFINE_BOOLEAN_PARAM(WaitWiggle); namespace { @@ -396,7 +400,7 @@ public: if (servers.size() == self->configuration.storageTeamSize || self->satisfiesPolicy(servers, self->configuration.storageTeamSize)) { servers.resize(self->configuration.storageTeamSize); - self->addTeam(servers, true); + self->addTeam(servers, IsInitialTeam::True); // self->traceTeamCollectionInfo(); // Trace at the end of the function } else { tempSet->clear(); @@ -414,7 +418,7 @@ public: serverIds.push_back(*tempMap->getObject(it)); } std::sort(serverIds.begin(), serverIds.end()); - self->addTeam(serverIds.begin(), serverIds.end(), true); + self->addTeam(serverIds.begin(), serverIds.end(), IsInitialTeam::True); } } else { serverIds.clear(); @@ -463,7 +467,7 @@ public: state std::set>::iterator teamIterEnd = self->primary ? initTeams->primaryTeams.end() : initTeams->remoteTeams.end(); for (; teamIter != teamIterEnd; ++teamIter) { - self->addTeam(teamIter->begin(), teamIter->end(), true); + self->addTeam(teamIter->begin(), teamIter->end(), IsInitialTeam::True); wait(yield()); } @@ -608,8 +612,8 @@ public: // A badTeam can be unhealthy or just a redundantTeam removed by machineTeamRemover() or serverTeamRemover() ACTOR static Future teamTracker(DDTeamCollection* self, Reference team, - bool badTeam, - bool redundantTeam) { + IsBadTeam badTeam, + IsRedundantTeam redundantTeam) { state int lastServersLeft = team->size(); state bool lastAnyUndesired = false; state bool lastAnyWigglingServer = false; @@ -1268,7 +1272,7 @@ public: bool addedNewBadTeam = false; for (auto it : newBadTeams) { if (self->removeTeam(it)) { - self->addTeam(it->getServers(), true); + self->addTeam(it->getServers(), IsInitialTeam::True); addedNewBadTeam = true; } } @@ -1387,7 +1391,7 @@ public: // NOTE: this actor returns when the cluster is healthy and stable (no server is expected to be removed in a period) // processingWiggle and processingUnhealthy indicate that some servers are going to be removed. - ACTOR static Future waitUntilHealthy(DDTeamCollection const* self, double extraDelay, bool waitWiggle) { + ACTOR static Future waitUntilHealthy(DDTeamCollection const* self, double extraDelay, WaitWiggle waitWiggle) { state int waitCount = 0; loop { while (self->zeroHealthyTeams->get() || self->processingUnhealthy->get() || @@ -1683,7 +1687,7 @@ public: // removeTeam() has side effect of swapping the last element to the current pos // in the serverTeams vector in the machine team. --teamIndex; - self->addTeam(team->getServers(), true, true); + self->addTeam(team->getServers(), IsInitialTeam::True, IsRedundantTeam::True); TEST(true); // Removed machine team } @@ -1767,7 +1771,7 @@ public: // The team will be marked as a bad team bool foundTeam = self->removeTeam(st); ASSERT(foundTeam); - self->addTeam(st->getServers(), true, true); + self->addTeam(st->getServers(), IsInitialTeam::True, IsRedundantTeam::True); TEST(true); // Marked team as a bad team self->doBuildTeams = true; @@ -3341,8 +3345,10 @@ Future DDTeamCollection::buildTeams() { return DDTeamCollectionImpl::buildTeams(this); } -Future DDTeamCollection::teamTracker(Reference team, bool badTeam, bool redundantTeam) { - return DDTeamCollectionImpl::teamTracker(this, team, badTeam, redundantTeam); +Future DDTeamCollection::teamTracker(Reference team, + IsBadTeam isBadTeam, + IsRedundantTeam isRedundantTeam) { + return DDTeamCollectionImpl::teamTracker(this, team, isBadTeam, isRedundantTeam); } Future DDTeamCollection::storageServerTracker( @@ -3359,7 +3365,7 @@ Future DDTeamCollection::removeWrongStoreType() { return DDTeamCollectionImpl::removeWrongStoreType(this); } -Future DDTeamCollection::waitUntilHealthy(double extraDelay, bool waitWiggle) const { +Future DDTeamCollection::waitUntilHealthy(double extraDelay, WaitWiggle waitWiggle) const { return DDTeamCollectionImpl::waitUntilHealthy(this, extraDelay, waitWiggle); } @@ -3783,13 +3789,13 @@ int DDTeamCollection::overlappingMachineMembers(std::vector>& newTeamServers, - bool isInitialTeam, - bool redundantTeam) { + IsInitialTeam isInitialTeam, + IsRedundantTeam redundantTeam) { auto teamInfo = makeReference(newTeamServers); // Move satisfiesPolicy to the end for performance benefit - bool badTeam = - redundantTeam || teamInfo->size() != configuration.storageTeamSize || !satisfiesPolicy(teamInfo->getServers()); + auto badTeam = IsBadTeam{ redundantTeam || teamInfo->size() != configuration.storageTeamSize || + !satisfiesPolicy(teamInfo->getServers()) }; teamInfo->tracker = teamTracker(teamInfo, badTeam, redundantTeam); // ASSERT( teamInfo->serverIDs.size() > 0 ); //team can be empty at DB initialization @@ -4534,7 +4540,7 @@ int DDTeamCollection::addTeamsBestOf(int teamsToBuild, int desiredTeams, int max } // Step 4: Add the server team - addTeam(bestServerTeam.begin(), bestServerTeam.end(), false); + addTeam(bestServerTeam.begin(), bestServerTeam.end(), IsInitialTeam::False); addedTeams++; } @@ -5235,8 +5241,8 @@ public: // state int targetTeamsPerServer = SERVER_KNOBS->DESIRED_TEAMS_PER_SERVER * (teamSize + 1) / 2; state std::unique_ptr collection = testTeamCollection(teamSize, policy, processSize); - collection->addTeam(std::set({ UID(1, 0), UID(2, 0), UID(3, 0) }), true); - collection->addTeam(std::set({ UID(1, 0), UID(3, 0), UID(4, 0) }), true); + collection->addTeam(std::set({ UID(1, 0), UID(2, 0), UID(3, 0) }), IsInitialTeam::True); + collection->addTeam(std::set({ UID(1, 0), UID(3, 0), UID(4, 0) }), IsInitialTeam::True); state int result = collection->addTeamsBestOf(8, desiredTeams, maxTeams); @@ -5265,8 +5271,8 @@ public: state int teamSize = 3; state std::unique_ptr collection = testTeamCollection(teamSize, policy, processSize); - collection->addTeam(std::set({ UID(1, 0), UID(2, 0), UID(3, 0) }), true); - collection->addTeam(std::set({ UID(1, 0), UID(3, 0), UID(4, 0) }), true); + collection->addTeam(std::set({ UID(1, 0), UID(2, 0), UID(3, 0) }), IsInitialTeam::True); + collection->addTeam(std::set({ UID(1, 0), UID(3, 0), UID(4, 0) }), IsInitialTeam::True); collection->addBestMachineTeams(10); int result = collection->addTeamsBestOf(10, desiredTeams, maxTeams); @@ -5311,8 +5317,8 @@ public: high_avail.available.bytes = 800 * 1024 * 1024; high_avail.load.bytes = 90 * 1024 * 1024; - collection->addTeam(std::set({ UID(1, 0), UID(2, 0), UID(3, 0) }), true); - collection->addTeam(std::set({ UID(2, 0), UID(3, 0), UID(4, 0) }), true); + collection->addTeam(std::set({ UID(1, 0), UID(2, 0), UID(3, 0) }), IsInitialTeam::True); + collection->addTeam(std::set({ UID(2, 0), UID(3, 0), UID(4, 0) }), IsInitialTeam::True); collection->disableBuildingTeams(); collection->setCheckTeamDelay(); @@ -5367,8 +5373,8 @@ public: high_avail.available.bytes = 800 * 1024 * 1024; high_avail.load.bytes = 90 * 1024 * 1024; - collection->addTeam(std::set({ UID(1, 0), UID(2, 0), UID(3, 0) }), true); - collection->addTeam(std::set({ UID(2, 0), UID(3, 0), UID(4, 0) }), true); + collection->addTeam(std::set({ UID(1, 0), UID(2, 0), UID(3, 0) }), IsInitialTeam::True); + collection->addTeam(std::set({ UID(2, 0), UID(3, 0), UID(4, 0) }), IsInitialTeam::True); collection->disableBuildingTeams(); collection->setCheckTeamDelay(); @@ -5425,8 +5431,8 @@ public: high_avail.available.bytes = 800 * 1024 * 1024; high_avail.load.bytes = 90 * 1024 * 1024; - collection->addTeam(std::set({ UID(1, 0), UID(2, 0), UID(3, 0) }), true); - collection->addTeam(std::set({ UID(2, 0), UID(3, 0), UID(4, 0) }), true); + collection->addTeam(std::set({ UID(1, 0), UID(2, 0), UID(3, 0) }), IsInitialTeam::True); + collection->addTeam(std::set({ UID(2, 0), UID(3, 0), UID(4, 0) }), IsInitialTeam::True); collection->disableBuildingTeams(); collection->setCheckTeamDelay(); @@ -5480,8 +5486,8 @@ public: high_avail.available.bytes = 800 * 1024 * 1024; high_avail.load.bytes = 90 * 1024 * 1024; - collection->addTeam(std::set({ UID(1, 0), UID(2, 0), UID(3, 0) }), true); - collection->addTeam(std::set({ UID(2, 0), UID(3, 0), UID(4, 0) }), true); + collection->addTeam(std::set({ UID(1, 0), UID(2, 0), UID(3, 0) }), IsInitialTeam::True); + collection->addTeam(std::set({ UID(2, 0), UID(3, 0), UID(4, 0) }), IsInitialTeam::True); collection->disableBuildingTeams(); collection->setCheckTeamDelay(); @@ -5534,8 +5540,8 @@ public: high_avail.available.bytes = 800 * 1024 * 1024; high_avail.load.bytes = 90 * 1024 * 1024; - collection->addTeam(std::set({ UID(1, 0), UID(2, 0), UID(3, 0) }), true); - collection->addTeam(std::set({ UID(2, 0), UID(3, 0), UID(4, 0) }), true); + collection->addTeam(std::set({ UID(1, 0), UID(2, 0), UID(3, 0) }), IsInitialTeam::True); + collection->addTeam(std::set({ UID(2, 0), UID(3, 0), UID(4, 0) }), IsInitialTeam::True); collection->disableBuildingTeams(); collection->setCheckTeamDelay(); @@ -5593,9 +5599,9 @@ public: high_avail.available.bytes = 800 * 1024 * 1024; high_avail.load.bytes = 90 * 1024 * 1024; - collection->addTeam(std::set({ UID(1, 0), UID(2, 0), UID(3, 0) }), true); - collection->addTeam(std::set({ UID(2, 0), UID(3, 0), UID(4, 0) }), true); - collection->addTeam(std::set({ UID(3, 0), UID(4, 0), UID(5, 0) }), true); + collection->addTeam(std::set({ UID(1, 0), UID(2, 0), UID(3, 0) }), IsInitialTeam::True); + collection->addTeam(std::set({ UID(2, 0), UID(3, 0), UID(4, 0) }), IsInitialTeam::True); + collection->addTeam(std::set({ UID(3, 0), UID(4, 0), UID(5, 0) }), IsInitialTeam::True); collection->disableBuildingTeams(); collection->setCheckTeamDelay(); diff --git a/fdbserver/DDTeamCollection.h b/fdbserver/DDTeamCollection.h index 3b7b47243a..c7827c8c3f 100644 --- a/fdbserver/DDTeamCollection.h +++ b/fdbserver/DDTeamCollection.h @@ -53,8 +53,6 @@ class TCTeamInfo; class TCMachineInfo; class TCMachineTeamInfo; -FDB_DECLARE_BOOLEAN_PARAM(IsPrimary); - // All state that represents an ongoing tss pair recruitment struct TSSPairState : ReferenceCounted, NonCopyable { Promise>> @@ -169,6 +167,12 @@ public: }; typedef AsyncMap ServerStatusMap; +FDB_DECLARE_BOOLEAN_PARAM(IsPrimary); +FDB_DECLARE_BOOLEAN_PARAM(IsInitialTeam); +FDB_DECLARE_BOOLEAN_PARAM(IsRedundantTeam); +FDB_DECLARE_BOOLEAN_PARAM(IsBadTeam); +FDB_DECLARE_BOOLEAN_PARAM(WaitWiggle); + class DDTeamCollection : public ReferenceCounted { friend class DDTeamCollectionImpl; friend class DDTeamCollectionUnitTest; @@ -402,8 +406,8 @@ class DDTeamCollection : public ReferenceCounted { void includeStorageServersForWiggle(); // Track a team and issue RelocateShards when the level of degradation changes - // A badTeam can be unhealthy or just a redundantTeam removed by machineTeamRemover() or serverTeamRemover() - Future teamTracker(Reference team, bool badTeam, bool redundantTeam); + // A bad team can be unhealthy or just a redundant team removed by machineTeamRemover() or serverTeamRemover() + Future teamTracker(Reference team, IsBadTeam, IsRedundantTeam); // Check the status of a storage server. // Apply all requirements to the server and mark it as excluded if it fails to satisfies these requirements @@ -418,7 +422,7 @@ class DDTeamCollection : public ReferenceCounted { // NOTE: this actor returns when the cluster is healthy and stable (no server is expected to be removed in a period) // processingWiggle and processingUnhealthy indicate that some servers are going to be removed. - Future waitUntilHealthy(double extraDelay = 0, bool waitWiggle = false) const; + Future waitUntilHealthy(double extraDelay = 0, WaitWiggle = WaitWiggle::False) const; bool isCorrectDC(TCServerInfo const& server) const; @@ -562,7 +566,7 @@ class DDTeamCollection : public ReferenceCounted { // isIntialTeam : False when the team is added by addTeamsBestOf(); True otherwise, e.g., // when the team added at init() when we recreate teams by looking up DB template - void addTeam(InputIt begin, InputIt end, bool isInitialTeam) { + void addTeam(InputIt begin, InputIt end, IsInitialTeam isInitialTeam) { std::vector> newTeamServers; for (auto i = begin; i != end; ++i) { if (server_info.find(*i) != server_info.end()) { @@ -574,10 +578,12 @@ class DDTeamCollection : public ReferenceCounted { } void addTeam(const std::vector>& newTeamServers, - bool isInitialTeam, - bool redundantTeam = false); + IsInitialTeam, + IsRedundantTeam = IsRedundantTeam::False); - void addTeam(std::set const& team, bool isInitialTeam) { addTeam(team.begin(), team.end(), isInitialTeam); } + void addTeam(std::set const& team, IsInitialTeam isInitialTeam) { + addTeam(team.begin(), team.end(), isInitialTeam); + } public: Database cx; From 0f8435e407b7bf6efc0581bc205b2ba6cbe49ef4 Mon Sep 17 00:00:00 2001 From: sfc-gh-tclinkenbeard Date: Fri, 11 Mar 2022 00:11:51 -0400 Subject: [PATCH 321/413] Make DDTeamCollection::addTeamsBestOf private --- fdbserver/DDTeamCollection.h | 12 ++++++------ 1 file changed, 6 insertions(+), 6 deletions(-) diff --git a/fdbserver/DDTeamCollection.h b/fdbserver/DDTeamCollection.h index c7827c8c3f..2f8cc6bdf1 100644 --- a/fdbserver/DDTeamCollection.h +++ b/fdbserver/DDTeamCollection.h @@ -585,6 +585,12 @@ class DDTeamCollection : public ReferenceCounted { addTeam(team.begin(), team.end(), isInitialTeam); } + // Create server teams based on machine teams + // Before the number of machine teams reaches the threshold, build a machine team for each server team + // When it reaches the threshold, first try to build a server team with existing machine teams; if failed, + // build an extra machine team and record the event in trace + int addTeamsBestOf(int teamsToBuild, int desiredTeams, int maxTeams); + public: Database cx; @@ -633,12 +639,6 @@ public: Future init(Reference initTeams, DDEnabledState const& ddEnabledState); - // Create server teams based on machine teams - // Before the number of machine teams reaches the threshold, build a machine team for each server team - // When it reaches the threshold, first try to build a server team with existing machine teams; if failed, - // build an extra machine team and record the event in trace - int addTeamsBestOf(int teamsToBuild, int desiredTeams, int maxTeams); - void addServer(StorageServerInterface newServer, ProcessClass processClass, Promise errorOut, From ccbbaa07f2f6a3abc83123b4cc6179c5d84bb254 Mon Sep 17 00:00:00 2001 From: sfc-gh-tclinkenbeard Date: Wed, 2 Mar 2022 16:45:34 -0800 Subject: [PATCH 322/413] Remove redundant UID field from TagThrottlerImpl::autoThrottleTag --- fdbserver/TagThrottler.actor.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/fdbserver/TagThrottler.actor.cpp b/fdbserver/TagThrottler.actor.cpp index 2e215a7c9b..b531c6d102 100644 --- a/fdbserver/TagThrottler.actor.cpp +++ b/fdbserver/TagThrottler.actor.cpp @@ -495,7 +495,7 @@ class TagThrottlerImpl { } } - Optional autoThrottleTag(UID id, TransactionTag tag, double busyness) { + Optional autoThrottleTag(TransactionTag tag, double busyness) { return throttledTags.autoThrottleTag(id, tag, busyness); } @@ -504,7 +504,7 @@ class TagThrottlerImpl { // currently MIN_TAG_PAGES_RATE > MIN_TAG_COST in our default knobs. if (busyness > SERVER_KNOBS->AUTO_THROTTLE_TARGET_TAG_BUSYNESS && rate > SERVER_KNOBS->MIN_TAG_COST) { TEST(true); // Transaction tag auto-throttled - Optional clientRate = autoThrottleTag(id, tag, busyness); + Optional clientRate = autoThrottleTag(tag, busyness); if (clientRate.present()) { TagSet tags; tags.addTag(tag); From e0006ea957b5accb8fc133e3b8fa6e87efa3f207 Mon Sep 17 00:00:00 2001 From: sfc-gh-tclinkenbeard Date: Wed, 2 Mar 2022 16:54:21 -0800 Subject: [PATCH 323/413] Mark TransactionTagCounter::costFunction const --- 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 d4760dd982..597d22a1d3 100644 --- a/fdbserver/storageserver.actor.cpp +++ b/fdbserver/storageserver.actor.cpp @@ -768,7 +768,7 @@ public: : thisServerID(thisServerID), busiestReadTagEventHolder(makeReference(thisServerID.toString() + "/BusiestReadTag")) {} - int64_t costFunction(int64_t bytes) { return bytes / SERVER_KNOBS->READ_COST_BYTE_FACTOR + 1; } + int64_t costFunction(int64_t bytes) const { return bytes / SERVER_KNOBS->READ_COST_BYTE_FACTOR + 1; } void addRequest(Optional const& tags, int64_t bytes) { if (tags.present()) { From 91d1a172d8c72ac005458cc51fb68c3e53fb01f9 Mon Sep 17 00:00:00 2001 From: sfc-gh-tclinkenbeard Date: Wed, 2 Mar 2022 17:02:40 -0800 Subject: [PATCH 324/413] Move TransactionTagCounter into separate file --- fdbserver/CMakeLists.txt | 1 + fdbserver/TransactionTagCounter.h | 93 +++++++++++++++++++++++++++++++ fdbserver/storageserver.actor.cpp | 79 ++------------------------ 3 files changed, 98 insertions(+), 75 deletions(-) create mode 100644 fdbserver/TransactionTagCounter.h diff --git a/fdbserver/CMakeLists.txt b/fdbserver/CMakeLists.txt index 728b4f9ab1..cc9bcaaf19 100644 --- a/fdbserver/CMakeLists.txt +++ b/fdbserver/CMakeLists.txt @@ -142,6 +142,7 @@ set(FDBSERVER_SRCS TesterInterface.actor.h TLogInterface.h TLogServer.actor.cpp + TransactionTagCounter.h TSSMappingUtil.actor.h TSSMappingUtil.actor.cpp VersionedBTree.actor.cpp diff --git a/fdbserver/TransactionTagCounter.h b/fdbserver/TransactionTagCounter.h new file mode 100644 index 0000000000..4a76705712 --- /dev/null +++ b/fdbserver/TransactionTagCounter.h @@ -0,0 +1,93 @@ +/* + * TransactionTagCounter.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 + +struct TransactionTagCounter { + struct TagInfo { + TransactionTag tag; + double rate; + double fractionalBusyness; + + TagInfo(TransactionTag const& tag, double rate, double fractionalBusyness) + : tag(tag), rate(rate), fractionalBusyness(fractionalBusyness) {} + }; + + TransactionTagMap intervalCounts; + int64_t intervalTotalSampledCount = 0; + TransactionTag busiestTag; + int64_t busiestTagCount = 0; + double intervalStart = 0; + + Optional previousBusiestTag; + + UID thisServerID; + + Reference busiestReadTagEventHolder; + + TransactionTagCounter(UID thisServerID) + : thisServerID(thisServerID), + busiestReadTagEventHolder(makeReference(thisServerID.toString() + "/BusiestReadTag")) {} + + int64_t costFunction(int64_t bytes) const { return bytes / SERVER_KNOBS->READ_COST_BYTE_FACTOR + 1; } + + void addRequest(Optional const& tags, int64_t bytes) { + if (tags.present()) { + TEST(true); // Tracking tag on storage server + double cost = costFunction(bytes); + for (auto& tag : tags.get()) { + int64_t& count = intervalCounts[TransactionTag(tag, tags.get().getArena())]; + count += cost; + if (count > busiestTagCount) { + busiestTagCount = count; + busiestTag = tag; + } + } + + intervalTotalSampledCount += cost; + } + } + + void startNewInterval() { + double elapsed = now() - intervalStart; + previousBusiestTag.reset(); + if (intervalStart > 0 && CLIENT_KNOBS->READ_TAG_SAMPLE_RATE > 0 && elapsed > 0) { + double rate = busiestTagCount / CLIENT_KNOBS->READ_TAG_SAMPLE_RATE / elapsed; + if (rate > SERVER_KNOBS->MIN_TAG_READ_PAGES_RATE) { + previousBusiestTag = TagInfo(busiestTag, rate, (double)busiestTagCount / intervalTotalSampledCount); + } + + TraceEvent("BusiestReadTag", thisServerID) + .detail("Elapsed", elapsed) + .detail("Tag", printable(busiestTag)) + .detail("TagCost", busiestTagCount) + .detail("TotalSampledCost", intervalTotalSampledCount) + .detail("Reported", previousBusiestTag.present()) + .trackLatest(busiestReadTagEventHolder->trackingKey); + } + + intervalCounts.clear(); + intervalTotalSampledCount = 0; + busiestTagCount = 0; + intervalStart = now(); + } + + Optional getBusiestTag() const { return previousBusiestTag; } +}; diff --git a/fdbserver/storageserver.actor.cpp b/fdbserver/storageserver.actor.cpp index 597d22a1d3..8701ecdce2 100644 --- a/fdbserver/storageserver.actor.cpp +++ b/fdbserver/storageserver.actor.cpp @@ -60,6 +60,7 @@ #include "fdbserver/StorageMetrics.h" #include "fdbserver/ServerDBInfo.h" #include "fdbserver/TLogInterface.h" +#include "fdbserver/TransactionTagCounter.h" #include "fdbserver/WaitFailure.h" #include "fdbserver/WorkerInterface.actor.h" #include "fdbrpc/sim_validation.h" @@ -742,78 +743,6 @@ public: return val; } - struct TransactionTagCounter { - struct TagInfo { - TransactionTag tag; - double rate; - double fractionalBusyness; - - TagInfo(TransactionTag const& tag, double rate, double fractionalBusyness) - : tag(tag), rate(rate), fractionalBusyness(fractionalBusyness) {} - }; - - TransactionTagMap intervalCounts; - int64_t intervalTotalSampledCount = 0; - TransactionTag busiestTag; - int64_t busiestTagCount = 0; - double intervalStart = 0; - - Optional previousBusiestTag; - - UID thisServerID; - - Reference busiestReadTagEventHolder; - - TransactionTagCounter(UID thisServerID) - : thisServerID(thisServerID), - busiestReadTagEventHolder(makeReference(thisServerID.toString() + "/BusiestReadTag")) {} - - int64_t costFunction(int64_t bytes) const { return bytes / SERVER_KNOBS->READ_COST_BYTE_FACTOR + 1; } - - void addRequest(Optional const& tags, int64_t bytes) { - if (tags.present()) { - TEST(true); // Tracking tag on storage server - double cost = costFunction(bytes); - for (auto& tag : tags.get()) { - int64_t& count = intervalCounts[TransactionTag(tag, tags.get().getArena())]; - count += cost; - if (count > busiestTagCount) { - busiestTagCount = count; - busiestTag = tag; - } - } - - intervalTotalSampledCount += cost; - } - } - - void startNewInterval() { - double elapsed = now() - intervalStart; - previousBusiestTag.reset(); - if (intervalStart > 0 && CLIENT_KNOBS->READ_TAG_SAMPLE_RATE > 0 && elapsed > 0) { - double rate = busiestTagCount / CLIENT_KNOBS->READ_TAG_SAMPLE_RATE / elapsed; - if (rate > SERVER_KNOBS->MIN_TAG_READ_PAGES_RATE) { - previousBusiestTag = TagInfo(busiestTag, rate, (double)busiestTagCount / intervalTotalSampledCount); - } - - TraceEvent("BusiestReadTag", thisServerID) - .detail("Elapsed", elapsed) - .detail("Tag", printable(busiestTag)) - .detail("TagCost", busiestTagCount) - .detail("TotalSampledCost", intervalTotalSampledCount) - .detail("Reported", previousBusiestTag.present()) - .trackLatest(busiestReadTagEventHolder->trackingKey); - } - - intervalCounts.clear(); - intervalTotalSampledCount = 0; - busiestTagCount = 0; - intervalStart = now(); - } - - Optional getBusiestTag() const { return previousBusiestTag; } - }; - TransactionTagCounter transactionTagCounter; Optional latencyBandConfig; @@ -3645,9 +3574,9 @@ void getQueuingMetrics(StorageServer* self, StorageQueuingMetricsRequest const& reply.diskUsage = self->diskUsage; reply.durableVersion = self->durableVersion.get(); - Optional busiestTag = self->transactionTagCounter.getBusiestTag(); - reply.busiestTag = busiestTag.map( - [](StorageServer::TransactionTagCounter::TagInfo tagInfo) { return tagInfo.tag; }); + Optional busiestTag = self->transactionTagCounter.getBusiestTag(); + reply.busiestTag = + busiestTag.map([](TransactionTagCounter::TagInfo tagInfo) { return tagInfo.tag; }); reply.busiestTagFractionalBusyness = busiestTag.present() ? busiestTag.get().fractionalBusyness : 0.0; reply.busiestTagRate = busiestTag.present() ? busiestTag.get().rate : 0.0; From e61c26758c637498630b72f0cd35ab0229c372c7 Mon Sep 17 00:00:00 2001 From: sfc-gh-tclinkenbeard Date: Wed, 2 Mar 2022 17:16:57 -0800 Subject: [PATCH 325/413] Move TransactionTagCounter implementation into .cpp file --- fdbserver/CMakeLists.txt | 1 + fdbserver/TransactionTagCounter.cpp | 67 +++++++++++++++++++++++++++++ fdbserver/TransactionTagCounter.h | 50 +++------------------ 3 files changed, 75 insertions(+), 43 deletions(-) create mode 100644 fdbserver/TransactionTagCounter.cpp diff --git a/fdbserver/CMakeLists.txt b/fdbserver/CMakeLists.txt index cc9bcaaf19..c745ddbf33 100644 --- a/fdbserver/CMakeLists.txt +++ b/fdbserver/CMakeLists.txt @@ -142,6 +142,7 @@ set(FDBSERVER_SRCS TesterInterface.actor.h TLogInterface.h TLogServer.actor.cpp + TransactionTagCounter.cpp TransactionTagCounter.h TSSMappingUtil.actor.h TSSMappingUtil.actor.cpp diff --git a/fdbserver/TransactionTagCounter.cpp b/fdbserver/TransactionTagCounter.cpp new file mode 100644 index 0000000000..ecebaf5fad --- /dev/null +++ b/fdbserver/TransactionTagCounter.cpp @@ -0,0 +1,67 @@ +/* + * TransactionTagCounter.cpp + * + * This source file is part of the FoundationDB open source project + * + * Copyright 2013-2022 Apple Inc. and the FoundationDB project authors + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +#include "fdbserver/TransactionTagCounter.h" +#include "flow/Trace.h" + +TransactionTagCounter::TransactionTagCounter(UID thisServerID) + : thisServerID(thisServerID), + busiestReadTagEventHolder(makeReference(thisServerID.toString() + "/BusiestReadTag")) {} + +void TransactionTagCounter::addRequest(Optional const& tags, int64_t bytes) { + if (tags.present()) { + TEST(true); // Tracking tag on storage server + double cost = costFunction(bytes); + for (auto& tag : tags.get()) { + int64_t& count = intervalCounts[TransactionTag(tag, tags.get().getArena())]; + count += cost; + if (count > busiestTagCount) { + busiestTagCount = count; + busiestTag = tag; + } + } + + intervalTotalSampledCount += cost; + } +} + +void TransactionTagCounter::startNewInterval() { + double elapsed = now() - intervalStart; + previousBusiestTag.reset(); + if (intervalStart > 0 && CLIENT_KNOBS->READ_TAG_SAMPLE_RATE > 0 && elapsed > 0) { + double rate = busiestTagCount / CLIENT_KNOBS->READ_TAG_SAMPLE_RATE / elapsed; + if (rate > SERVER_KNOBS->MIN_TAG_READ_PAGES_RATE) { + previousBusiestTag = TagInfo(busiestTag, rate, (double)busiestTagCount / intervalTotalSampledCount); + } + + TraceEvent("BusiestReadTag", thisServerID) + .detail("Elapsed", elapsed) + .detail("Tag", printable(busiestTag)) + .detail("TagCost", busiestTagCount) + .detail("TotalSampledCost", intervalTotalSampledCount) + .detail("Reported", previousBusiestTag.present()) + .trackLatest(busiestReadTagEventHolder->trackingKey); + } + + intervalCounts.clear(); + intervalTotalSampledCount = 0; + busiestTagCount = 0; + intervalStart = now(); +} diff --git a/fdbserver/TransactionTagCounter.h b/fdbserver/TransactionTagCounter.h index 4a76705712..1e4354b2b0 100644 --- a/fdbserver/TransactionTagCounter.h +++ b/fdbserver/TransactionTagCounter.h @@ -20,6 +20,9 @@ #pragma once +#include "fdbclient/TagThrottle.actor.h" +#include "fdbserver/Knobs.h" + struct TransactionTagCounter { struct TagInfo { TransactionTag tag; @@ -42,52 +45,13 @@ struct TransactionTagCounter { Reference busiestReadTagEventHolder; - TransactionTagCounter(UID thisServerID) - : thisServerID(thisServerID), - busiestReadTagEventHolder(makeReference(thisServerID.toString() + "/BusiestReadTag")) {} + TransactionTagCounter(UID thisServerID); - int64_t costFunction(int64_t bytes) const { return bytes / SERVER_KNOBS->READ_COST_BYTE_FACTOR + 1; } + static int64_t costFunction(int64_t bytes) { return bytes / SERVER_KNOBS->READ_COST_BYTE_FACTOR + 1; } - void addRequest(Optional const& tags, int64_t bytes) { - if (tags.present()) { - TEST(true); // Tracking tag on storage server - double cost = costFunction(bytes); - for (auto& tag : tags.get()) { - int64_t& count = intervalCounts[TransactionTag(tag, tags.get().getArena())]; - count += cost; - if (count > busiestTagCount) { - busiestTagCount = count; - busiestTag = tag; - } - } + void addRequest(Optional const& tags, int64_t bytes); - intervalTotalSampledCount += cost; - } - } - - void startNewInterval() { - double elapsed = now() - intervalStart; - previousBusiestTag.reset(); - if (intervalStart > 0 && CLIENT_KNOBS->READ_TAG_SAMPLE_RATE > 0 && elapsed > 0) { - double rate = busiestTagCount / CLIENT_KNOBS->READ_TAG_SAMPLE_RATE / elapsed; - if (rate > SERVER_KNOBS->MIN_TAG_READ_PAGES_RATE) { - previousBusiestTag = TagInfo(busiestTag, rate, (double)busiestTagCount / intervalTotalSampledCount); - } - - TraceEvent("BusiestReadTag", thisServerID) - .detail("Elapsed", elapsed) - .detail("Tag", printable(busiestTag)) - .detail("TagCost", busiestTagCount) - .detail("TotalSampledCost", intervalTotalSampledCount) - .detail("Reported", previousBusiestTag.present()) - .trackLatest(busiestReadTagEventHolder->trackingKey); - } - - intervalCounts.clear(); - intervalTotalSampledCount = 0; - busiestTagCount = 0; - intervalStart = now(); - } + void startNewInterval(); Optional getBusiestTag() const { return previousBusiestTag; } }; From 96983fdd7af3d7e67bc009068705873cbbb1aec4 Mon Sep 17 00:00:00 2001 From: sfc-gh-tclinkenbeard Date: Wed, 2 Mar 2022 17:23:34 -0800 Subject: [PATCH 326/413] Improve TransactionTagCounter encapsulation --- fdbserver/TransactionTagCounter.h | 11 ++++------- 1 file changed, 4 insertions(+), 7 deletions(-) diff --git a/fdbserver/TransactionTagCounter.h b/fdbserver/TransactionTagCounter.h index 1e4354b2b0..9fb00c6146 100644 --- a/fdbserver/TransactionTagCounter.h +++ b/fdbserver/TransactionTagCounter.h @@ -23,7 +23,8 @@ #include "fdbclient/TagThrottle.actor.h" #include "fdbserver/Knobs.h" -struct TransactionTagCounter { +class TransactionTagCounter { +public: struct TagInfo { TransactionTag tag; double rate; @@ -33,6 +34,7 @@ struct TransactionTagCounter { : tag(tag), rate(rate), fractionalBusyness(fractionalBusyness) {} }; +private: TransactionTagMap intervalCounts; int64_t intervalTotalSampledCount = 0; TransactionTag busiestTag; @@ -40,18 +42,13 @@ struct TransactionTagCounter { double intervalStart = 0; Optional previousBusiestTag; - UID thisServerID; - Reference busiestReadTagEventHolder; +public: TransactionTagCounter(UID thisServerID); - static int64_t costFunction(int64_t bytes) { return bytes / SERVER_KNOBS->READ_COST_BYTE_FACTOR + 1; } - void addRequest(Optional const& tags, int64_t bytes); - void startNewInterval(); - Optional getBusiestTag() const { return previousBusiestTag; } }; From cad106f9ebdb94c7eca5d514f036a670cec53ab8 Mon Sep 17 00:00:00 2001 From: sfc-gh-tclinkenbeard Date: Wed, 2 Mar 2022 18:35:13 -0800 Subject: [PATCH 327/413] Support sending multiple busy tags from storage server to ratekeeper --- fdbclient/StorageServerInterface.h | 24 ++++++++++++++++++------ fdbserver/Ratekeeper.actor.cpp | 14 +++++--------- fdbserver/Ratekeeper.h | 5 +---- fdbserver/TagThrottler.actor.cpp | 18 +++++++++--------- fdbserver/TransactionTagCounter.cpp | 6 +++--- fdbserver/TransactionTagCounter.h | 16 +++------------- fdbserver/storageserver.actor.cpp | 6 +----- 7 files changed, 40 insertions(+), 49 deletions(-) diff --git a/fdbclient/StorageServerInterface.h b/fdbclient/StorageServerInterface.h index 75c9411f18..d03676fc53 100644 --- a/fdbclient/StorageServerInterface.h +++ b/fdbclient/StorageServerInterface.h @@ -921,6 +921,22 @@ struct GetStorageMetricsRequest { }; struct StorageQueuingMetricsReply { + struct TagInfo { + constexpr static FileIdentifier file_identifier = 4528694; + TransactionTag tag; + double rate{ 0.0 }; + double fractionalBusyness{ 0.0 }; + + TagInfo() = default; + TagInfo(TransactionTag const& tag, double rate, double fractionalBusyness) + : tag(tag), rate(rate), fractionalBusyness(fractionalBusyness) {} + + template + void serialize(Ar& ar) { + serializer(ar, tag, rate, fractionalBusyness); + } + }; + constexpr static FileIdentifier file_identifier = 7633366; double localTime; int64_t instanceID; // changes if bytesDurable and bytesInput reset @@ -931,9 +947,7 @@ struct StorageQueuingMetricsReply { double cpuUsage; double diskUsage; double localRateLimit; - Optional busiestTag; - double busiestTagFractionalBusyness; - double busiestTagRate; + std::vector busiestTags; template void serialize(Ar& ar) { @@ -948,9 +962,7 @@ struct StorageQueuingMetricsReply { cpuUsage, diskUsage, localRateLimit, - busiestTag, - busiestTagFractionalBusyness, - busiestTagRate); + busiestTags); } }; diff --git a/fdbserver/Ratekeeper.actor.cpp b/fdbserver/Ratekeeper.actor.cpp index 8edb87a0fb..1d12c6f1d8 100644 --- a/fdbserver/Ratekeeper.actor.cpp +++ b/fdbserver/Ratekeeper.actor.cpp @@ -180,9 +180,7 @@ public: myQueueInfo->value.smoothLatestVersion.setTotal(reply.get().version); } - myQueueInfo->value.busiestReadTag = reply.get().busiestTag; - myQueueInfo->value.busiestReadTagFractionalBusyness = reply.get().busiestTagFractionalBusyness; - myQueueInfo->value.busiestReadTagRate = reply.get().busiestTagRate; + myQueueInfo->value.busiestReadTags = reply.get().busiestTags; } else { if (myQueueInfo->value.valid) { TraceEvent("RkStorageServerDidNotRespond", self->id).detail("StorageServer", ssi.id()); @@ -974,7 +972,7 @@ Future Ratekeeper::refreshStorageServerCommitCost() { double elapsed = now() - lastBusiestCommitTagPick; // for each SS, select the busiest commit tag from ssTrTagCommitCost for (auto it = storageQueueInfo.begin(); it != storageQueueInfo.end(); ++it) { - it->value.busiestWriteTag.reset(); + it->value.busiestWriteTags.clear(); TransactionTag busiestTag; TransactionCommitCostEstimation maxCost; double maxRate = 0, maxBusyness = 0; @@ -987,12 +985,10 @@ Future Ratekeeper::refreshStorageServerCommitCost() { } } if (maxRate > SERVER_KNOBS->MIN_TAG_WRITE_PAGES_RATE) { - it->value.busiestWriteTag = busiestTag; // TraceEvent("RefreshSSCommitCost").detail("TotalWriteCost", it->value.totalWriteCost).detail("TotalWriteOps",it->value.totalWriteOps); - ASSERT(it->value.totalWriteCosts > 0); + ASSERT_GT(it->value.totalWriteCosts, 0); maxBusyness = double(maxCost.getCostSum()) / it->value.totalWriteCosts; - it->value.busiestWriteTagFractionalBusyness = maxBusyness; - it->value.busiestWriteTagRate = maxRate; + it->value.busiestWriteTags.emplace_back(busiestTag, maxBusyness, maxRate); } TraceEvent("BusiestWriteTag", it->key) @@ -1001,7 +997,7 @@ Future Ratekeeper::refreshStorageServerCommitCost() { .detail("TagOps", maxCost.getOpsSum()) .detail("TagCost", maxCost.getCostSum()) .detail("TotalCost", it->value.totalWriteCosts) - .detail("Reported", it->value.busiestWriteTag.present()) + .detail("Reported", !it->value.busiestWriteTags.empty()) .trackLatest(it->value.busiestWriteTagEventHolder->trackingKey); // reset statistics diff --git a/fdbserver/Ratekeeper.h b/fdbserver/Ratekeeper.h index 8552eeb521..d463ac0e1d 100644 --- a/fdbserver/Ratekeeper.h +++ b/fdbserver/Ratekeeper.h @@ -58,10 +58,7 @@ struct StorageQueueInfo { Smoother smoothTotalSpace; limitReason_t limitReason; - Optional busiestReadTag, busiestWriteTag; - double busiestReadTagFractionalBusyness = 0, busiestWriteTagFractionalBusyness = 0; - double busiestReadTagRate = 0, busiestWriteTagRate = 0; - + std::vector busiestReadTags, busiestWriteTags; Reference busiestWriteTagEventHolder; // refresh periodically diff --git a/fdbserver/TagThrottler.actor.cpp b/fdbserver/TagThrottler.actor.cpp index b531c6d102..0f48695ae6 100644 --- a/fdbserver/TagThrottler.actor.cpp +++ b/fdbserver/TagThrottler.actor.cpp @@ -544,16 +544,16 @@ public: // the future if (storageQueue > SERVER_KNOBS->AUTO_TAG_THROTTLE_STORAGE_QUEUE_BYTES || storageDurabilityLag > SERVER_KNOBS->AUTO_TAG_THROTTLE_DURABILITY_LAG_VERSIONS) { - if (ss.busiestWriteTag.present()) { - return tryAutoThrottleTag(ss.busiestWriteTag.get(), - ss.busiestWriteTagRate, - ss.busiestWriteTagFractionalBusyness, - TagThrottledReason::BUSY_WRITE); + for (const auto& busiestWriteTag : ss.busiestWriteTags) { + return tryAutoThrottleTag(busiestWriteTag.tag, + busiestWriteTag.rate, + busiestWriteTag.fractionalBusyness, + TagThrottledReason::BUSY_READ); } - if (ss.busiestReadTag.present()) { - return tryAutoThrottleTag(ss.busiestReadTag.get(), - ss.busiestReadTagRate, - ss.busiestReadTagFractionalBusyness, + for (const auto& busiestReadTag : ss.busiestReadTags) { + return tryAutoThrottleTag(busiestReadTag.tag, + busiestReadTag.rate, + busiestReadTag.fractionalBusyness, TagThrottledReason::BUSY_READ); } } diff --git a/fdbserver/TransactionTagCounter.cpp b/fdbserver/TransactionTagCounter.cpp index ecebaf5fad..f9215ea7cb 100644 --- a/fdbserver/TransactionTagCounter.cpp +++ b/fdbserver/TransactionTagCounter.cpp @@ -44,11 +44,11 @@ void TransactionTagCounter::addRequest(Optional const& tags, int64_t byt void TransactionTagCounter::startNewInterval() { double elapsed = now() - intervalStart; - previousBusiestTag.reset(); + previousBusiestTags.clear(); if (intervalStart > 0 && CLIENT_KNOBS->READ_TAG_SAMPLE_RATE > 0 && elapsed > 0) { double rate = busiestTagCount / CLIENT_KNOBS->READ_TAG_SAMPLE_RATE / elapsed; if (rate > SERVER_KNOBS->MIN_TAG_READ_PAGES_RATE) { - previousBusiestTag = TagInfo(busiestTag, rate, (double)busiestTagCount / intervalTotalSampledCount); + previousBusiestTags.emplace_back(busiestTag, rate, (double)busiestTagCount / intervalTotalSampledCount); } TraceEvent("BusiestReadTag", thisServerID) @@ -56,7 +56,7 @@ void TransactionTagCounter::startNewInterval() { .detail("Tag", printable(busiestTag)) .detail("TagCost", busiestTagCount) .detail("TotalSampledCost", intervalTotalSampledCount) - .detail("Reported", previousBusiestTag.present()) + .detail("Reported", !previousBusiestTags.empty()) .trackLatest(busiestReadTagEventHolder->trackingKey); } diff --git a/fdbserver/TransactionTagCounter.h b/fdbserver/TransactionTagCounter.h index 9fb00c6146..d520259c5c 100644 --- a/fdbserver/TransactionTagCounter.h +++ b/fdbserver/TransactionTagCounter.h @@ -20,28 +20,18 @@ #pragma once +#include "fdbclient/StorageServerInterface.h" #include "fdbclient/TagThrottle.actor.h" #include "fdbserver/Knobs.h" class TransactionTagCounter { -public: - struct TagInfo { - TransactionTag tag; - double rate; - double fractionalBusyness; - - TagInfo(TransactionTag const& tag, double rate, double fractionalBusyness) - : tag(tag), rate(rate), fractionalBusyness(fractionalBusyness) {} - }; - -private: TransactionTagMap intervalCounts; int64_t intervalTotalSampledCount = 0; TransactionTag busiestTag; int64_t busiestTagCount = 0; double intervalStart = 0; - Optional previousBusiestTag; + std::vector previousBusiestTags; UID thisServerID; Reference busiestReadTagEventHolder; @@ -50,5 +40,5 @@ public: static int64_t costFunction(int64_t bytes) { return bytes / SERVER_KNOBS->READ_COST_BYTE_FACTOR + 1; } void addRequest(Optional const& tags, int64_t bytes); void startNewInterval(); - Optional getBusiestTag() const { return previousBusiestTag; } + std::vector const& getBusiestTags() const { return previousBusiestTags; } }; diff --git a/fdbserver/storageserver.actor.cpp b/fdbserver/storageserver.actor.cpp index 8701ecdce2..783f122bd1 100644 --- a/fdbserver/storageserver.actor.cpp +++ b/fdbserver/storageserver.actor.cpp @@ -3574,11 +3574,7 @@ void getQueuingMetrics(StorageServer* self, StorageQueuingMetricsRequest const& reply.diskUsage = self->diskUsage; reply.durableVersion = self->durableVersion.get(); - Optional busiestTag = self->transactionTagCounter.getBusiestTag(); - reply.busiestTag = - busiestTag.map([](TransactionTagCounter::TagInfo tagInfo) { return tagInfo.tag; }); - reply.busiestTagFractionalBusyness = busiestTag.present() ? busiestTag.get().fractionalBusyness : 0.0; - reply.busiestTagRate = busiestTag.present() ? busiestTag.get().rate : 0.0; + reply.busiestTags = self->transactionTagCounter.getBusiestTags(); req.reply.send(reply); } From 455b75abca3b8f27853ca2c0206227bbae0c5c1d Mon Sep 17 00:00:00 2001 From: sfc-gh-tclinkenbeard Date: Wed, 2 Mar 2022 18:43:49 -0800 Subject: [PATCH 328/413] Use structured bindings for for loop in Ratekeeper::refreshStorageServerCommitCost --- fdbserver/Ratekeeper.actor.cpp | 26 +++++++++++++------------- 1 file changed, 13 insertions(+), 13 deletions(-) diff --git a/fdbserver/Ratekeeper.actor.cpp b/fdbserver/Ratekeeper.actor.cpp index 1d12c6f1d8..9eddcfc2ec 100644 --- a/fdbserver/Ratekeeper.actor.cpp +++ b/fdbserver/Ratekeeper.actor.cpp @@ -971,12 +971,12 @@ Future Ratekeeper::refreshStorageServerCommitCost() { } double elapsed = now() - lastBusiestCommitTagPick; // for each SS, select the busiest commit tag from ssTrTagCommitCost - for (auto it = storageQueueInfo.begin(); it != storageQueueInfo.end(); ++it) { - it->value.busiestWriteTags.clear(); + for (auto& [ssId, ssQueueInfo] : storageQueueInfo) { + ssQueueInfo.busiestWriteTags.clear(); TransactionTag busiestTag; TransactionCommitCostEstimation maxCost; double maxRate = 0, maxBusyness = 0; - for (const auto& [tag, cost] : it->value.tagCostEst) { + for (const auto& [tag, cost] : ssQueueInfo.tagCostEst) { double rate = cost.getCostSum() / elapsed; if (rate > maxRate) { busiestTag = tag; @@ -986,24 +986,24 @@ Future Ratekeeper::refreshStorageServerCommitCost() { } if (maxRate > SERVER_KNOBS->MIN_TAG_WRITE_PAGES_RATE) { // TraceEvent("RefreshSSCommitCost").detail("TotalWriteCost", it->value.totalWriteCost).detail("TotalWriteOps",it->value.totalWriteOps); - ASSERT_GT(it->value.totalWriteCosts, 0); - maxBusyness = double(maxCost.getCostSum()) / it->value.totalWriteCosts; - it->value.busiestWriteTags.emplace_back(busiestTag, maxBusyness, maxRate); + ASSERT_GT(ssQueueInfo.totalWriteCosts, 0); + maxBusyness = double(maxCost.getCostSum()) / ssQueueInfo.totalWriteCosts; + ssQueueInfo.busiestWriteTags.emplace_back(busiestTag, maxBusyness, maxRate); } - TraceEvent("BusiestWriteTag", it->key) + TraceEvent("BusiestWriteTag", ssId) .detail("Elapsed", elapsed) .detail("Tag", printable(busiestTag)) .detail("TagOps", maxCost.getOpsSum()) .detail("TagCost", maxCost.getCostSum()) - .detail("TotalCost", it->value.totalWriteCosts) - .detail("Reported", !it->value.busiestWriteTags.empty()) - .trackLatest(it->value.busiestWriteTagEventHolder->trackingKey); + .detail("TotalCost", ssQueueInfo.totalWriteCosts) + .detail("Reported", !ssQueueInfo.busiestWriteTags.empty()) + .trackLatest(ssQueueInfo.busiestWriteTagEventHolder->trackingKey); // reset statistics - it->value.tagCostEst.clear(); - it->value.totalWriteOps = 0; - it->value.totalWriteCosts = 0; + ssQueueInfo.tagCostEst.clear(); + ssQueueInfo.totalWriteOps = 0; + ssQueueInfo.totalWriteCosts = 0; } lastBusiestCommitTagPick = now(); return Void(); From 229f0cca8ba5f9fc4694d45e19c54b09e5f8ecd1 Mon Sep 17 00:00:00 2001 From: sfc-gh-tclinkenbeard Date: Wed, 2 Mar 2022 19:16:33 -0800 Subject: [PATCH 329/413] Add StorageQueueInfo::refreshCommitCost method --- fdbserver/Ratekeeper.actor.cpp | 97 ++++++++++++++++++---------------- fdbserver/Ratekeeper.h | 6 +-- 2 files changed, 54 insertions(+), 49 deletions(-) diff --git a/fdbserver/Ratekeeper.actor.cpp b/fdbserver/Ratekeeper.actor.cpp index 9eddcfc2ec..538bca997a 100644 --- a/fdbserver/Ratekeeper.actor.cpp +++ b/fdbserver/Ratekeeper.actor.cpp @@ -288,9 +288,7 @@ public: self.addActor.send(traceRole(Role::RATEKEEPER, rkInterf.id())); self.addActor.send(self.monitorThrottlingChanges()); - Ratekeeper* selfPtr = &self; // let flow compiler capture self - self.addActor.send(recurring([selfPtr]() { selfPtr->refreshStorageServerCommitCost(); }, - SERVER_KNOBS->TAG_MEASUREMENT_INTERVAL)); + self.addActor.send(self.refreshStorageServerCommitCosts()); TraceEvent("RkTLogQueueSizeParameters", rkInterf.id()) .detail("Target", SERVER_KNOBS->TARGET_BYTES_PER_TLOG) @@ -410,6 +408,19 @@ public: return Void(); } + ACTOR static Future refreshStorageServerCommitCosts(Ratekeeper* self) { + state double lastBusiestCommitTagPick; + loop { + lastBusiestCommitTagPick = now(); + wait(delay(SERVER_KNOBS->TAG_MEASUREMENT_INTERVAL)); + double elapsed = now() - lastBusiestCommitTagPick; + // for each SS, select the busiest commit tag from ssTrTagCommitCost + for (auto& [ssId, ssQueueInfo] : self->storageQueueInfo) { + ssQueueInfo.refreshCommitCost(elapsed); + } + } + } + }; // class RatekeeperImpl Future Ratekeeper::configurationMonitor() { @@ -964,52 +975,46 @@ void Ratekeeper::updateRate(RatekeeperLimits* limits) { } } -Future Ratekeeper::refreshStorageServerCommitCost() { - if (lastBusiestCommitTagPick == 0) { // the first call should be skipped - lastBusiestCommitTagPick = now(); - return Void(); - } - double elapsed = now() - lastBusiestCommitTagPick; - // for each SS, select the busiest commit tag from ssTrTagCommitCost - for (auto& [ssId, ssQueueInfo] : storageQueueInfo) { - ssQueueInfo.busiestWriteTags.clear(); - TransactionTag busiestTag; - TransactionCommitCostEstimation maxCost; - double maxRate = 0, maxBusyness = 0; - for (const auto& [tag, cost] : ssQueueInfo.tagCostEst) { - double rate = cost.getCostSum() / elapsed; - if (rate > maxRate) { - busiestTag = tag; - maxRate = rate; - maxCost = cost; - } - } - if (maxRate > SERVER_KNOBS->MIN_TAG_WRITE_PAGES_RATE) { - // TraceEvent("RefreshSSCommitCost").detail("TotalWriteCost", it->value.totalWriteCost).detail("TotalWriteOps",it->value.totalWriteOps); - ASSERT_GT(ssQueueInfo.totalWriteCosts, 0); - maxBusyness = double(maxCost.getCostSum()) / ssQueueInfo.totalWriteCosts; - ssQueueInfo.busiestWriteTags.emplace_back(busiestTag, maxBusyness, maxRate); - } - - TraceEvent("BusiestWriteTag", ssId) - .detail("Elapsed", elapsed) - .detail("Tag", printable(busiestTag)) - .detail("TagOps", maxCost.getOpsSum()) - .detail("TagCost", maxCost.getCostSum()) - .detail("TotalCost", ssQueueInfo.totalWriteCosts) - .detail("Reported", !ssQueueInfo.busiestWriteTags.empty()) - .trackLatest(ssQueueInfo.busiestWriteTagEventHolder->trackingKey); - - // reset statistics - ssQueueInfo.tagCostEst.clear(); - ssQueueInfo.totalWriteOps = 0; - ssQueueInfo.totalWriteCosts = 0; - } - lastBusiestCommitTagPick = now(); - return Void(); +Future Ratekeeper::refreshStorageServerCommitCosts() { + return RatekeeperImpl::refreshStorageServerCommitCosts(this); } ACTOR Future ratekeeper(RatekeeperInterface rkInterf, Reference const> dbInfo) { wait(Ratekeeper::run(rkInterf, dbInfo)); return Void(); } + +void StorageQueueInfo::refreshCommitCost(double elapsed) { + busiestWriteTags.clear(); + TransactionTag busiestTag; + TransactionCommitCostEstimation maxCost; + double maxRate = 0, maxBusyness = 0; + for (const auto& [tag, cost] : tagCostEst) { + double rate = cost.getCostSum() / elapsed; + if (rate > maxRate) { + busiestTag = tag; + maxRate = rate; + maxCost = cost; + } + } + if (maxRate > SERVER_KNOBS->MIN_TAG_WRITE_PAGES_RATE) { + // TraceEvent("RefreshSSCommitCost").detail("TotalWriteCost", totalWriteCost).detail("TotalWriteOps",totalWriteOps); + ASSERT_GT(totalWriteCosts, 0); + maxBusyness = double(maxCost.getCostSum()) / totalWriteCosts; + busiestWriteTags.emplace_back(busiestTag, maxBusyness, maxRate); + } + + TraceEvent("BusiestWriteTag", id) + .detail("Elapsed", elapsed) + .detail("Tag", printable(busiestTag)) + .detail("TagOps", maxCost.getOpsSum()) + .detail("TagCost", maxCost.getCostSum()) + .detail("TotalCost", totalWriteCosts) + .detail("Reported", !busiestWriteTags.empty()) + .trackLatest(busiestWriteTagEventHolder->trackingKey); + + // reset statistics + tagCostEst.clear(); + totalWriteOps = 0; + totalWriteCosts = 0; +} diff --git a/fdbserver/Ratekeeper.h b/fdbserver/Ratekeeper.h index d463ac0e1d..97037a2a63 100644 --- a/fdbserver/Ratekeeper.h +++ b/fdbserver/Ratekeeper.h @@ -76,6 +76,8 @@ struct StorageQueueInfo { // FIXME: this is a tacky workaround for a potential uninitialized use in trackStorageServerQueueInfo lastReply.instanceID = -1; } + + void refreshCommitCost(double elapsed); }; struct TLogQueueInfo { @@ -180,14 +182,12 @@ class Ratekeeper { Future expiredTagThrottleCleanup; - double lastBusiestCommitTagPick; - Ratekeeper(UID id, Database db); Future configurationMonitor(); void updateCommitCostEstimation(UIDTransactionTagMap const& costEstimation); void updateRate(RatekeeperLimits* limits); - Future refreshStorageServerCommitCost(); + Future refreshStorageServerCommitCosts(); Future monitorServerListChange(PromiseStream>> serverChanges); Future trackEachStorageServer(FutureStream>> serverChanges); From eb5a556c9814334e6d594e07116b57a318d577bf Mon Sep 17 00:00:00 2001 From: sfc-gh-tclinkenbeard Date: Wed, 2 Mar 2022 19:24:31 -0800 Subject: [PATCH 330/413] Move constructor implementations out of Ratekeeper.h --- fdbserver/Ratekeeper.actor.cpp | 42 +++++++++++++++++++++++++++++-- fdbserver/Ratekeeper.h | 46 ++++++---------------------------- 2 files changed, 48 insertions(+), 40 deletions(-) diff --git a/fdbserver/Ratekeeper.actor.cpp b/fdbserver/Ratekeeper.actor.cpp index 538bca997a..5657b2b7d4 100644 --- a/fdbserver/Ratekeeper.actor.cpp +++ b/fdbserver/Ratekeeper.actor.cpp @@ -473,8 +473,7 @@ Ratekeeper::Ratekeeper(UID id, Database db) SERVER_KNOBS->TARGET_BYTES_PER_TLOG_BATCH, SERVER_KNOBS->SPRING_BYTES_TLOG_BATCH, SERVER_KNOBS->MAX_TL_SS_VERSION_DIFFERENCE_BATCH, - SERVER_KNOBS->TARGET_DURABILITY_LAG_VERSIONS_BATCH), - lastBusiestCommitTagPick(0.0) { + SERVER_KNOBS->TARGET_DURABILITY_LAG_VERSIONS_BATCH) { tagThrottler = std::make_unique(db, id); expiredTagThrottleCleanup = recurring([this]() { ThrottleApi::expire(this->db.getReference()); }, SERVER_KNOBS->TAG_THROTTLE_EXPIRED_CLEANUP_INTERVAL); @@ -984,6 +983,17 @@ ACTOR Future ratekeeper(RatekeeperInterface rkInterf, ReferenceSMOOTHING_AMOUNT), + smoothInputBytes(SERVER_KNOBS->SMOOTHING_AMOUNT), verySmoothDurableBytes(SERVER_KNOBS->SLOW_SMOOTHING_AMOUNT), + smoothDurableVersion(SERVER_KNOBS->SMOOTHING_AMOUNT), smoothLatestVersion(SERVER_KNOBS->SMOOTHING_AMOUNT), + smoothFreeSpace(SERVER_KNOBS->SMOOTHING_AMOUNT), smoothTotalSpace(SERVER_KNOBS->SMOOTHING_AMOUNT), + limitReason(limitReason_t::unlimited), + busiestWriteTagEventHolder(makeReference(id.toString() + "/BusiestWriteTag")) { + // FIXME: this is a tacky workaround for a potential uninitialized use in trackStorageServerQueueInfo + lastReply.instanceID = -1; +} + void StorageQueueInfo::refreshCommitCost(double elapsed) { busiestWriteTags.clear(); TransactionTag busiestTag; @@ -1018,3 +1028,31 @@ void StorageQueueInfo::refreshCommitCost(double elapsed) { totalWriteOps = 0; totalWriteCosts = 0; } + +TLogQueueInfo::TLogQueueInfo(UID id) + : valid(false), id(id), smoothDurableBytes(SERVER_KNOBS->SMOOTHING_AMOUNT), + smoothInputBytes(SERVER_KNOBS->SMOOTHING_AMOUNT), verySmoothDurableBytes(SERVER_KNOBS->SLOW_SMOOTHING_AMOUNT), + smoothFreeSpace(SERVER_KNOBS->SMOOTHING_AMOUNT), smoothTotalSpace(SERVER_KNOBS->SMOOTHING_AMOUNT) { + // FIXME: this is a tacky workaround for a potential uninitialized use in trackTLogQueueInfo (copied from + // storageQueueInfO) + lastReply.instanceID = -1; +} + +RatekeeperLimits::RatekeeperLimits(TransactionPriority priority, + std::string context, + int64_t storageTargetBytes, + int64_t storageSpringBytes, + int64_t logTargetBytes, + int64_t logSpringBytes, + double maxVersionDifference, + int64_t durabilityLagTargetVersions) + : tpsLimit(std::numeric_limits::infinity()), tpsLimitMetric(StringRef("Ratekeeper.TPSLimit" + context)), + reasonMetric(StringRef("Ratekeeper.Reason" + context)), storageTargetBytes(storageTargetBytes), + storageSpringBytes(storageSpringBytes), logTargetBytes(logTargetBytes), logSpringBytes(logSpringBytes), + maxVersionDifference(maxVersionDifference), + durabilityLagTargetVersions( + durabilityLagTargetVersions + + SERVER_KNOBS->MAX_READ_TRANSACTION_LIFE_VERSIONS), // The read transaction life versions are expected to not + // be durable on the storage servers + lastDurabilityLag(0), durabilityLagLimit(std::numeric_limits::infinity()), priority(priority), + context(context), rkUpdateEventCacheHolder(makeReference("RkUpdate" + context)) {} diff --git a/fdbserver/Ratekeeper.h b/fdbserver/Ratekeeper.h index 97037a2a63..2ac8b4c449 100644 --- a/fdbserver/Ratekeeper.h +++ b/fdbserver/Ratekeeper.h @@ -66,16 +66,7 @@ struct StorageQueueInfo { uint64_t totalWriteCosts = 0; int totalWriteOps = 0; - StorageQueueInfo(UID id, LocalityData locality) - : valid(false), id(id), locality(locality), smoothDurableBytes(SERVER_KNOBS->SMOOTHING_AMOUNT), - smoothInputBytes(SERVER_KNOBS->SMOOTHING_AMOUNT), verySmoothDurableBytes(SERVER_KNOBS->SLOW_SMOOTHING_AMOUNT), - smoothDurableVersion(SERVER_KNOBS->SMOOTHING_AMOUNT), smoothLatestVersion(SERVER_KNOBS->SMOOTHING_AMOUNT), - smoothFreeSpace(SERVER_KNOBS->SMOOTHING_AMOUNT), smoothTotalSpace(SERVER_KNOBS->SMOOTHING_AMOUNT), - limitReason(limitReason_t::unlimited), - busiestWriteTagEventHolder(makeReference(id.toString() + "/BusiestWriteTag")) { - // FIXME: this is a tacky workaround for a potential uninitialized use in trackStorageServerQueueInfo - lastReply.instanceID = -1; - } + StorageQueueInfo(UID id, LocalityData locality); void refreshCommitCost(double elapsed); }; @@ -88,14 +79,7 @@ struct TLogQueueInfo { Smoother smoothDurableBytes, smoothInputBytes, verySmoothDurableBytes; Smoother smoothFreeSpace; Smoother smoothTotalSpace; - TLogQueueInfo(UID id) - : valid(false), id(id), smoothDurableBytes(SERVER_KNOBS->SMOOTHING_AMOUNT), - smoothInputBytes(SERVER_KNOBS->SMOOTHING_AMOUNT), verySmoothDurableBytes(SERVER_KNOBS->SLOW_SMOOTHING_AMOUNT), - smoothFreeSpace(SERVER_KNOBS->SMOOTHING_AMOUNT), smoothTotalSpace(SERVER_KNOBS->SMOOTHING_AMOUNT) { - // FIXME: this is a tacky workaround for a potential uninitialized use in trackTLogQueueInfo (copied from - // storageQueueInfO) - lastReply.instanceID = -1; - } + TLogQueueInfo(UID id); }; struct RatekeeperLimits { @@ -125,17 +109,7 @@ struct RatekeeperLimits { int64_t logTargetBytes, int64_t logSpringBytes, double maxVersionDifference, - int64_t durabilityLagTargetVersions) - : tpsLimit(std::numeric_limits::infinity()), tpsLimitMetric(StringRef("Ratekeeper.TPSLimit" + context)), - reasonMetric(StringRef("Ratekeeper.Reason" + context)), storageTargetBytes(storageTargetBytes), - storageSpringBytes(storageSpringBytes), logTargetBytes(logTargetBytes), logSpringBytes(logSpringBytes), - maxVersionDifference(maxVersionDifference), - durabilityLagTargetVersions( - durabilityLagTargetVersions + - SERVER_KNOBS->MAX_READ_TRANSACTION_LIFE_VERSIONS), // The read transaction life versions are expected to not - // be durable on the storage servers - lastDurabilityLag(0), durabilityLagLimit(std::numeric_limits::infinity()), priority(priority), - context(context), rkUpdateEventCacheHolder(makeReference("RkUpdate" + context)) {} + int64_t durabilityLagTargetVersions); }; class Ratekeeper { @@ -143,16 +117,12 @@ class Ratekeeper { // Differentiate from GrvProxyInfo in DatabaseContext.h struct GrvProxyInfo { - int64_t totalTransactions; - int64_t batchTransactions; - uint64_t lastThrottledTagChangeId; + int64_t totalTransactions{ 0 }; + int64_t batchTransactions{ 0 }; + uint64_t lastThrottledTagChangeId{ 0 }; - double lastUpdateTime; - double lastTagPushTime; - - GrvProxyInfo() - : totalTransactions(0), batchTransactions(0), lastThrottledTagChangeId(0), lastUpdateTime(0), - lastTagPushTime(0) {} + double lastUpdateTime{ 0.0 }; + double lastTagPushTime{ 0.0 }; }; UID id; From 5c053c6c3743da27d4ecf9817edd070908f40241 Mon Sep 17 00:00:00 2001 From: sfc-gh-tclinkenbeard Date: Wed, 2 Mar 2022 19:28:41 -0800 Subject: [PATCH 331/413] Move expiredTagThrottleCleanup into TagThrottler class --- fdbserver/Ratekeeper.actor.cpp | 2 -- fdbserver/Ratekeeper.h | 2 -- fdbserver/TagThrottler.actor.cpp | 6 +++++- 3 files changed, 5 insertions(+), 5 deletions(-) diff --git a/fdbserver/Ratekeeper.actor.cpp b/fdbserver/Ratekeeper.actor.cpp index 5657b2b7d4..89c0f0d231 100644 --- a/fdbserver/Ratekeeper.actor.cpp +++ b/fdbserver/Ratekeeper.actor.cpp @@ -475,8 +475,6 @@ Ratekeeper::Ratekeeper(UID id, Database db) SERVER_KNOBS->MAX_TL_SS_VERSION_DIFFERENCE_BATCH, SERVER_KNOBS->TARGET_DURABILITY_LAG_VERSIONS_BATCH) { tagThrottler = std::make_unique(db, id); - expiredTagThrottleCleanup = recurring([this]() { ThrottleApi::expire(this->db.getReference()); }, - SERVER_KNOBS->TAG_THROTTLE_EXPIRED_CLEANUP_INTERVAL); } void Ratekeeper::updateCommitCostEstimation( diff --git a/fdbserver/Ratekeeper.h b/fdbserver/Ratekeeper.h index 2ac8b4c449..28b93266fc 100644 --- a/fdbserver/Ratekeeper.h +++ b/fdbserver/Ratekeeper.h @@ -150,8 +150,6 @@ class Ratekeeper { Deque actualTpsHistory; Optional remoteDC; - Future expiredTagThrottleCleanup; - Ratekeeper(UID id, Database db); Future configurationMonitor(); diff --git a/fdbserver/TagThrottler.actor.cpp b/fdbserver/TagThrottler.actor.cpp index 0f48695ae6..2172051685 100644 --- a/fdbserver/TagThrottler.actor.cpp +++ b/fdbserver/TagThrottler.actor.cpp @@ -383,6 +383,7 @@ class TagThrottlerImpl { RkTagThrottleCollection throttledTags; uint64_t throttledTagChangeId{ 0 }; bool autoThrottlingEnabled{ false }; + Future expiredTagThrottleCleanup; ACTOR static Future monitorThrottlingChanges(TagThrottlerImpl* self) { state bool committed = false; @@ -524,7 +525,10 @@ class TagThrottlerImpl { } public: - TagThrottlerImpl(Database db, UID id) : db(db), id(id) {} + TagThrottlerImpl(Database db, UID id) : db(db), id(id) { + expiredTagThrottleCleanup = recurring([this]() { ThrottleApi::expire(this->db.getReference()); }, + SERVER_KNOBS->TAG_THROTTLE_EXPIRED_CLEANUP_INTERVAL); + } Future monitorThrottlingChanges() { return monitorThrottlingChanges(this); } void addRequests(TransactionTag tag, int count) { throttledTags.addRequests(tag, count); } From af6905859672a63fcb4257a1765451b091c034c7 Mon Sep 17 00:00:00 2001 From: sfc-gh-tclinkenbeard Date: Sun, 6 Mar 2022 22:09:17 -0400 Subject: [PATCH 332/413] Move RkTagThrottleCollection class into its own files --- fdbserver/CMakeLists.txt | 2 + fdbserver/RkTagThrottleCollection.cpp | 356 +++++++++++++++++++++++++ fdbserver/RkTagThrottleCollection.h | 88 ++++++ fdbserver/TagThrottler.actor.cpp | 369 +------------------------- 4 files changed, 451 insertions(+), 364 deletions(-) create mode 100644 fdbserver/RkTagThrottleCollection.cpp create mode 100644 fdbserver/RkTagThrottleCollection.h diff --git a/fdbserver/CMakeLists.txt b/fdbserver/CMakeLists.txt index c745ddbf33..e740e98b85 100644 --- a/fdbserver/CMakeLists.txt +++ b/fdbserver/CMakeLists.txt @@ -112,6 +112,8 @@ set(FDBSERVER_SRCS RestoreWorkerInterface.actor.h Resolver.actor.cpp ResolverInterface.h + RkTagThrottleCollection.cpp + RkTagThrottleCollection.h RoleLineage.actor.h RoleLineage.actor.cpp ServerDBInfo.actor.h diff --git a/fdbserver/RkTagThrottleCollection.cpp b/fdbserver/RkTagThrottleCollection.cpp new file mode 100644 index 0000000000..df76dcb21b --- /dev/null +++ b/fdbserver/RkTagThrottleCollection.cpp @@ -0,0 +1,356 @@ +/* + * RkTagThrottleCollection.cpp + * + * This source file is part of the FoundationDB open source project + * + * Copyright 2013-2022 Apple Inc. and the FoundationDB project authors + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +#include "fdbserver/Knobs.h" +#include "fdbserver/RkTagThrottleCollection.h" + +double RkTagThrottleCollection::RkTagThrottleData::getTargetRate(Optional requestRate) { + if (limits.tpsRate == 0.0 || !requestRate.present() || requestRate.get() == 0.0 || !rateSet) { + return limits.tpsRate; + } else { + return std::min(limits.tpsRate, (limits.tpsRate / requestRate.get()) * clientRate.smoothTotal()); + } +} + +Optional RkTagThrottleCollection::RkTagThrottleData::updateAndGetClientRate(Optional requestRate) { + if (limits.expiration > now()) { + double targetRate = getTargetRate(requestRate); + if (targetRate == std::numeric_limits::max()) { + rateSet = false; + return targetRate; + } + if (!rateSet) { + rateSet = true; + clientRate.reset(targetRate); + } else { + clientRate.setTotal(targetRate); + } + + double rate = clientRate.smoothTotal(); + ASSERT(rate >= 0); + return rate; + } else { + TEST(true); // Get throttle rate for expired throttle + rateSet = false; + return Optional(); + } +} + +RkTagThrottleCollection::RkTagThrottleCollection(RkTagThrottleCollection&& other) { + autoThrottledTags = std::move(other.autoThrottledTags); + manualThrottledTags = std::move(other.manualThrottledTags); + tagData = std::move(other.tagData); +} + +RkTagThrottleCollection& RkTagThrottleCollection::RkTagThrottleCollection::operator=(RkTagThrottleCollection&& other) { + autoThrottledTags = std::move(other.autoThrottledTags); + manualThrottledTags = std::move(other.manualThrottledTags); + tagData = std::move(other.tagData); + return *this; +} + +double RkTagThrottleCollection::computeTargetTpsRate(double currentBusyness, + double targetBusyness, + double requestRate) const { + ASSERT(currentBusyness > 0); + + if (targetBusyness < 1) { + double targetFraction = targetBusyness * (1 - currentBusyness) / ((1 - targetBusyness) * currentBusyness); + return requestRate * targetFraction; + } else { + return std::numeric_limits::max(); + } +} + +Optional RkTagThrottleCollection::autoThrottleTag(UID id, + TransactionTag const& tag, + double fractionalBusyness, + Optional tpsRate, + Optional expiration) { + ASSERT(!tpsRate.present() || tpsRate.get() >= 0); + ASSERT(!expiration.present() || expiration.get() > now()); + + auto itr = autoThrottledTags.find(tag); + bool present = (itr != autoThrottledTags.end()); + if (!present) { + if (autoThrottledTags.size() >= SERVER_KNOBS->MAX_AUTO_THROTTLED_TRANSACTION_TAGS) { + TEST(true); // Reached auto-throttle limit + return Optional(); + } + + itr = autoThrottledTags.try_emplace(tag).first; + initializeTag(tag); + } else if (itr->second.limits.expiration <= now()) { + TEST(true); // Re-throttling expired tag that hasn't been cleaned up + present = false; + itr->second = RkTagThrottleData(); + } + + auto& throttle = itr->second; + + if (!tpsRate.present()) { + if (now() <= throttle.created + SERVER_KNOBS->AUTO_TAG_THROTTLE_START_AGGREGATION_TIME) { + tpsRate = std::numeric_limits::max(); + if (present) { + return Optional(); + } + } else if (now() <= throttle.lastUpdated + SERVER_KNOBS->AUTO_TAG_THROTTLE_UPDATE_FREQUENCY) { + TEST(true); // Tag auto-throttled too quickly + return Optional(); + } else { + tpsRate = computeTargetTpsRate(fractionalBusyness, + SERVER_KNOBS->AUTO_THROTTLE_TARGET_TAG_BUSYNESS, + tagData[tag].requestRate.smoothRate()); + + if (throttle.limits.expiration > now() && tpsRate.get() >= throttle.limits.tpsRate) { + TEST(true); // Tag auto-throttle rate increase attempt while active + return Optional(); + } + + throttle.lastUpdated = now(); + if (tpsRate.get() < throttle.limits.tpsRate) { + throttle.lastReduced = now(); + } + } + } + if (!expiration.present()) { + expiration = now() + SERVER_KNOBS->AUTO_TAG_THROTTLE_DURATION; + } + + ASSERT(tpsRate.present() && tpsRate.get() >= 0); + + throttle.limits.tpsRate = tpsRate.get(); + throttle.limits.expiration = expiration.get(); + + Optional clientRate = throttle.updateAndGetClientRate(getRequestRate(tag)); + + TraceEvent("RkSetAutoThrottle", id) + .detail("Tag", tag) + .detail("TargetRate", tpsRate.get()) + .detail("Expiration", expiration.get() - now()) + .detail("ClientRate", clientRate) + .detail("Created", now() - throttle.created) + .detail("LastUpdate", now() - throttle.lastUpdated) + .detail("LastReduced", now() - throttle.lastReduced); + + if (tpsRate.get() != std::numeric_limits::max()) { + return tpsRate.get(); + } else { + return Optional(); + } +} + +void RkTagThrottleCollection::manualThrottleTag(UID id, + TransactionTag const& tag, + TransactionPriority priority, + double tpsRate, + double expiration, + Optional const& oldLimits) { + ASSERT(tpsRate >= 0); + ASSERT(expiration > now()); + + auto& priorityThrottleMap = manualThrottledTags[tag]; + auto result = priorityThrottleMap.try_emplace(priority); + initializeTag(tag); + ASSERT(result.second); // Updating to the map is done by copying the whole map + + result.first->second.limits.tpsRate = tpsRate; + result.first->second.limits.expiration = expiration; + + if (!oldLimits.present()) { + TEST(true); // Transaction tag manually throttled + TraceEvent("RatekeeperAddingManualThrottle", id) + .detail("Tag", tag) + .detail("Rate", tpsRate) + .detail("Priority", transactionPriorityToString(priority)) + .detail("SecondsToExpiration", expiration - now()); + } else if (oldLimits.get().tpsRate != tpsRate || oldLimits.get().expiration != expiration) { + TEST(true); // Manual transaction tag throttle updated + TraceEvent("RatekeeperUpdatingManualThrottle", id) + .detail("Tag", tag) + .detail("Rate", tpsRate) + .detail("Priority", transactionPriorityToString(priority)) + .detail("SecondsToExpiration", expiration - now()); + } + + Optional clientRate = result.first->second.updateAndGetClientRate(getRequestRate(tag)); + ASSERT(clientRate.present()); +} + +Optional RkTagThrottleCollection::getManualTagThrottleLimits(TransactionTag const& tag, + TransactionPriority priority) { + auto itr = manualThrottledTags.find(tag); + if (itr != manualThrottledTags.end()) { + auto priorityItr = itr->second.find(priority); + if (priorityItr != itr->second.end()) { + return priorityItr->second.limits; + } + } + + return Optional(); +} + +PrioritizedTransactionTagMap RkTagThrottleCollection::getClientRates( + bool autoThrottlingEnabled) { + PrioritizedTransactionTagMap clientRates; + + for (auto tagItr = tagData.begin(); tagItr != tagData.end();) { + bool tagPresent = false; + + double requestRate = tagItr->second.requestRate.smoothRate(); + auto manualItr = manualThrottledTags.find(tagItr->first); + if (manualItr != manualThrottledTags.end()) { + Optional manualClientRate; + for (auto priority = allTransactionPriorities.rbegin(); !(priority == allTransactionPriorities.rend()); + ++priority) { + auto priorityItr = manualItr->second.find(*priority); + if (priorityItr != manualItr->second.end()) { + Optional priorityClientRate = priorityItr->second.updateAndGetClientRate(requestRate); + if (!priorityClientRate.present()) { + TEST(true); // Manual priority throttle expired + priorityItr = manualItr->second.erase(priorityItr); + } else { + if (!manualClientRate.present() || manualClientRate.get().tpsRate > priorityClientRate.get()) { + manualClientRate = ClientTagThrottleLimits(priorityClientRate.get(), + priorityItr->second.limits.expiration); + } else { + TEST(true); // Manual throttle overriden by higher priority + } + + ++priorityItr; + } + } + + if (manualClientRate.present()) { + tagPresent = true; + TEST(true); // Using manual throttle + clientRates[*priority][tagItr->first] = manualClientRate.get(); + } + } + + if (manualItr->second.empty()) { + TEST(true); // All manual throttles expired + manualThrottledTags.erase(manualItr); + break; + } + } + + auto autoItr = autoThrottledTags.find(tagItr->first); + if (autoItr != autoThrottledTags.end()) { + Optional autoClientRate = autoItr->second.updateAndGetClientRate(requestRate); + if (autoClientRate.present()) { + double adjustedRate = autoClientRate.get(); + double rampStartTime = autoItr->second.lastReduced + SERVER_KNOBS->AUTO_TAG_THROTTLE_DURATION - + SERVER_KNOBS->AUTO_TAG_THROTTLE_RAMP_UP_TIME; + if (now() >= rampStartTime && adjustedRate != std::numeric_limits::max()) { + TEST(true); // Tag auto-throttle ramping up + + double targetBusyness = SERVER_KNOBS->AUTO_THROTTLE_TARGET_TAG_BUSYNESS; + if (targetBusyness == 0) { + targetBusyness = 0.01; + } + + double rampLocation = (now() - rampStartTime) / SERVER_KNOBS->AUTO_TAG_THROTTLE_RAMP_UP_TIME; + adjustedRate = + computeTargetTpsRate(targetBusyness, pow(targetBusyness, 1 - rampLocation), adjustedRate); + } + + tagPresent = true; + if (autoThrottlingEnabled) { + auto result = clientRates[TransactionPriority::DEFAULT].try_emplace( + tagItr->first, adjustedRate, autoItr->second.limits.expiration); + if (!result.second && result.first->second.tpsRate > adjustedRate) { + result.first->second = ClientTagThrottleLimits(adjustedRate, autoItr->second.limits.expiration); + } else { + TEST(true); // Auto throttle overriden by manual throttle + } + clientRates[TransactionPriority::BATCH][tagItr->first] = + ClientTagThrottleLimits(0, autoItr->second.limits.expiration); + } + } else { + ASSERT(autoItr->second.limits.expiration <= now()); + TEST(true); // Auto throttle expired + if (BUGGIFY) { // Temporarily extend the window between expiration and cleanup + tagPresent = true; + } else { + autoThrottledTags.erase(autoItr); + } + } + } + + if (!tagPresent) { + TEST(true); // All tag throttles expired + tagItr = tagData.erase(tagItr); + } else { + ++tagItr; + } + } + + return clientRates; +} + +void RkTagThrottleCollection::addRequests(TransactionTag const& tag, int requests) { + if (requests > 0) { + TEST(true); // Requests reported for throttled tag + + auto tagItr = tagData.try_emplace(tag); + tagItr.first->second.requestRate.addDelta(requests); + + double requestRate = tagItr.first->second.requestRate.smoothRate(); + + auto autoItr = autoThrottledTags.find(tag); + if (autoItr != autoThrottledTags.end()) { + autoItr->second.updateAndGetClientRate(requestRate); + } + + auto manualItr = manualThrottledTags.find(tag); + if (manualItr != manualThrottledTags.end()) { + for (auto priorityItr = manualItr->second.begin(); priorityItr != manualItr->second.end(); ++priorityItr) { + priorityItr->second.updateAndGetClientRate(requestRate); + } + } + } +} + +Optional RkTagThrottleCollection::getRequestRate(TransactionTag const& tag) { + auto itr = tagData.find(tag); + if (itr != tagData.end()) { + return itr->second.requestRate.smoothRate(); + } + return Optional(); +} + +int64_t RkTagThrottleCollection::manualThrottleCount() const { + int64_t count = 0; + for (auto itr = manualThrottledTags.begin(); itr != manualThrottledTags.end(); ++itr) { + count += itr->second.size(); + } + + return count; +} + +void RkTagThrottleCollection::updateBusyTagCount(TagThrottledReason reason) { + if (reason == TagThrottledReason::BUSY_READ) { + ++busyReadTagCount; + } else if (reason == TagThrottledReason::BUSY_WRITE) { + ++busyWriteTagCount; + } +} diff --git a/fdbserver/RkTagThrottleCollection.h b/fdbserver/RkTagThrottleCollection.h new file mode 100644 index 0000000000..dbad4e8fb7 --- /dev/null +++ b/fdbserver/RkTagThrottleCollection.h @@ -0,0 +1,88 @@ +/* + * RkTagThrottleCollection.h + * + * This source file is part of the FoundationDB open source project + * + * Copyright 2013-2022 Apple Inc. and the FoundationDB project authors + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +#pragma once + +#include "fdbclient/Knobs.h" +#include "fdbclient/TagThrottle.actor.h" +#include "fdbrpc/Smoother.h" + +class RkTagThrottleCollection : NonCopyable { + struct RkTagData { + Smoother requestRate; + RkTagData() : requestRate(CLIENT_KNOBS->TAG_THROTTLE_SMOOTHING_WINDOW) {} + }; + + struct RkTagThrottleData { + ClientTagThrottleLimits limits; + Smoother clientRate; + + // Only used by auto-throttles + double created = now(); + double lastUpdated = 0; + double lastReduced = now(); + bool rateSet = false; + + RkTagThrottleData() : clientRate(CLIENT_KNOBS->TAG_THROTTLE_SMOOTHING_WINDOW) {} + double getTargetRate(Optional requestRate); + Optional updateAndGetClientRate(Optional requestRate); + }; + + TransactionTagMap autoThrottledTags; + TransactionTagMap> manualThrottledTags; + TransactionTagMap tagData; + uint32_t busyReadTagCount = 0, busyWriteTagCount = 0; + + void initializeTag(TransactionTag const& tag) { tagData.try_emplace(tag); } + +public: + RkTagThrottleCollection() = default; + RkTagThrottleCollection(RkTagThrottleCollection&& other); + RkTagThrottleCollection& operator=(RkTagThrottleCollection&& other); + + double computeTargetTpsRate(double currentBusyness, double targetBusyness, double requestRate) const; + + // Returns the TPS rate if the throttle is updated, otherwise returns an empty optional + Optional autoThrottleTag(UID id, + TransactionTag const& tag, + double fractionalBusyness, + Optional tpsRate = Optional(), + Optional expiration = Optional()); + + void manualThrottleTag(UID id, + TransactionTag const& tag, + TransactionPriority priority, + double tpsRate, + double expiration, + Optional const& oldLimits); + + Optional getManualTagThrottleLimits(TransactionTag const& tag, + TransactionPriority priority); + + PrioritizedTransactionTagMap getClientRates(bool autoThrottlingEnabled); + void addRequests(TransactionTag const& tag, int requests); + Optional getRequestRate(TransactionTag const& tag); + int64_t autoThrottleCount() const { return autoThrottledTags.size(); } + int64_t manualThrottleCount() const; + void updateBusyTagCount(TagThrottledReason); + auto getBusyReadTagCount() const { return busyReadTagCount; } + auto getBusyWriteTagCount() const { return busyWriteTagCount; } +}; diff --git a/fdbserver/TagThrottler.actor.cpp b/fdbserver/TagThrottler.actor.cpp index 2172051685..e7623ad1ca 100644 --- a/fdbserver/TagThrottler.actor.cpp +++ b/fdbserver/TagThrottler.actor.cpp @@ -13,369 +13,14 @@ * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, + * * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ #include "fdbserver/TagThrottler.h" - -class RkTagThrottleCollection : NonCopyable { - struct RkTagData { - Smoother requestRate; - RkTagData() : requestRate(CLIENT_KNOBS->TAG_THROTTLE_SMOOTHING_WINDOW) {} - }; - - struct RkTagThrottleData { - ClientTagThrottleLimits limits; - Smoother clientRate; - - // Only used by auto-throttles - double created = now(); - double lastUpdated = 0; - double lastReduced = now(); - bool rateSet = false; - - RkTagThrottleData() : clientRate(CLIENT_KNOBS->TAG_THROTTLE_SMOOTHING_WINDOW) {} - - double getTargetRate(Optional requestRate) { - if (limits.tpsRate == 0.0 || !requestRate.present() || requestRate.get() == 0.0 || !rateSet) { - return limits.tpsRate; - } else { - return std::min(limits.tpsRate, (limits.tpsRate / requestRate.get()) * clientRate.smoothTotal()); - } - } - - Optional updateAndGetClientRate(Optional requestRate) { - if (limits.expiration > now()) { - double targetRate = getTargetRate(requestRate); - if (targetRate == std::numeric_limits::max()) { - rateSet = false; - return targetRate; - } - if (!rateSet) { - rateSet = true; - clientRate.reset(targetRate); - } else { - clientRate.setTotal(targetRate); - } - - double rate = clientRate.smoothTotal(); - ASSERT(rate >= 0); - return rate; - } else { - TEST(true); // Get throttle rate for expired throttle - rateSet = false; - return Optional(); - } - } - }; - - void initializeTag(TransactionTag const& tag) { tagData.try_emplace(tag); } - -public: - RkTagThrottleCollection() {} - - RkTagThrottleCollection(RkTagThrottleCollection&& other) { - autoThrottledTags = std::move(other.autoThrottledTags); - manualThrottledTags = std::move(other.manualThrottledTags); - tagData = std::move(other.tagData); - } - - void operator=(RkTagThrottleCollection&& other) { - autoThrottledTags = std::move(other.autoThrottledTags); - manualThrottledTags = std::move(other.manualThrottledTags); - tagData = std::move(other.tagData); - } - - double computeTargetTpsRate(double currentBusyness, double targetBusyness, double requestRate) { - ASSERT(currentBusyness > 0); - - if (targetBusyness < 1) { - double targetFraction = targetBusyness * (1 - currentBusyness) / ((1 - targetBusyness) * currentBusyness); - return requestRate * targetFraction; - } else { - return std::numeric_limits::max(); - } - } - - // Returns the TPS rate if the throttle is updated, otherwise returns an empty optional - Optional autoThrottleTag(UID id, - TransactionTag const& tag, - double fractionalBusyness, - Optional tpsRate = Optional(), - Optional expiration = Optional()) { - ASSERT(!tpsRate.present() || tpsRate.get() >= 0); - ASSERT(!expiration.present() || expiration.get() > now()); - - auto itr = autoThrottledTags.find(tag); - bool present = (itr != autoThrottledTags.end()); - if (!present) { - if (autoThrottledTags.size() >= SERVER_KNOBS->MAX_AUTO_THROTTLED_TRANSACTION_TAGS) { - TEST(true); // Reached auto-throttle limit - return Optional(); - } - - itr = autoThrottledTags.try_emplace(tag).first; - initializeTag(tag); - } else if (itr->second.limits.expiration <= now()) { - TEST(true); // Re-throttling expired tag that hasn't been cleaned up - present = false; - itr->second = RkTagThrottleData(); - } - - auto& throttle = itr->second; - - if (!tpsRate.present()) { - if (now() <= throttle.created + SERVER_KNOBS->AUTO_TAG_THROTTLE_START_AGGREGATION_TIME) { - tpsRate = std::numeric_limits::max(); - if (present) { - return Optional(); - } - } else if (now() <= throttle.lastUpdated + SERVER_KNOBS->AUTO_TAG_THROTTLE_UPDATE_FREQUENCY) { - TEST(true); // Tag auto-throttled too quickly - return Optional(); - } else { - tpsRate = computeTargetTpsRate(fractionalBusyness, - SERVER_KNOBS->AUTO_THROTTLE_TARGET_TAG_BUSYNESS, - tagData[tag].requestRate.smoothRate()); - - if (throttle.limits.expiration > now() && tpsRate.get() >= throttle.limits.tpsRate) { - TEST(true); // Tag auto-throttle rate increase attempt while active - return Optional(); - } - - throttle.lastUpdated = now(); - if (tpsRate.get() < throttle.limits.tpsRate) { - throttle.lastReduced = now(); - } - } - } - if (!expiration.present()) { - expiration = now() + SERVER_KNOBS->AUTO_TAG_THROTTLE_DURATION; - } - - ASSERT(tpsRate.present() && tpsRate.get() >= 0); - - throttle.limits.tpsRate = tpsRate.get(); - throttle.limits.expiration = expiration.get(); - - Optional clientRate = throttle.updateAndGetClientRate(getRequestRate(tag)); - - TraceEvent("RkSetAutoThrottle", id) - .detail("Tag", tag) - .detail("TargetRate", tpsRate.get()) - .detail("Expiration", expiration.get() - now()) - .detail("ClientRate", clientRate) - .detail("Created", now() - throttle.created) - .detail("LastUpdate", now() - throttle.lastUpdated) - .detail("LastReduced", now() - throttle.lastReduced); - - if (tpsRate.get() != std::numeric_limits::max()) { - return tpsRate.get(); - } else { - return Optional(); - } - } - - void manualThrottleTag(UID id, - TransactionTag const& tag, - TransactionPriority priority, - double tpsRate, - double expiration, - Optional const& oldLimits) { - ASSERT(tpsRate >= 0); - ASSERT(expiration > now()); - - auto& priorityThrottleMap = manualThrottledTags[tag]; - auto result = priorityThrottleMap.try_emplace(priority); - initializeTag(tag); - ASSERT(result.second); // Updating to the map is done by copying the whole map - - result.first->second.limits.tpsRate = tpsRate; - result.first->second.limits.expiration = expiration; - - if (!oldLimits.present()) { - TEST(true); // Transaction tag manually throttled - TraceEvent("RatekeeperAddingManualThrottle", id) - .detail("Tag", tag) - .detail("Rate", tpsRate) - .detail("Priority", transactionPriorityToString(priority)) - .detail("SecondsToExpiration", expiration - now()); - } else if (oldLimits.get().tpsRate != tpsRate || oldLimits.get().expiration != expiration) { - TEST(true); // Manual transaction tag throttle updated - TraceEvent("RatekeeperUpdatingManualThrottle", id) - .detail("Tag", tag) - .detail("Rate", tpsRate) - .detail("Priority", transactionPriorityToString(priority)) - .detail("SecondsToExpiration", expiration - now()); - } - - Optional clientRate = result.first->second.updateAndGetClientRate(getRequestRate(tag)); - ASSERT(clientRate.present()); - } - - Optional getManualTagThrottleLimits(TransactionTag const& tag, - TransactionPriority priority) { - auto itr = manualThrottledTags.find(tag); - if (itr != manualThrottledTags.end()) { - auto priorityItr = itr->second.find(priority); - if (priorityItr != itr->second.end()) { - return priorityItr->second.limits; - } - } - - return Optional(); - } - - PrioritizedTransactionTagMap getClientRates(bool autoThrottlingEnabled) { - PrioritizedTransactionTagMap clientRates; - - for (auto tagItr = tagData.begin(); tagItr != tagData.end();) { - bool tagPresent = false; - - double requestRate = tagItr->second.requestRate.smoothRate(); - auto manualItr = manualThrottledTags.find(tagItr->first); - if (manualItr != manualThrottledTags.end()) { - Optional manualClientRate; - for (auto priority = allTransactionPriorities.rbegin(); !(priority == allTransactionPriorities.rend()); - ++priority) { - auto priorityItr = manualItr->second.find(*priority); - if (priorityItr != manualItr->second.end()) { - Optional priorityClientRate = priorityItr->second.updateAndGetClientRate(requestRate); - if (!priorityClientRate.present()) { - TEST(true); // Manual priority throttle expired - priorityItr = manualItr->second.erase(priorityItr); - } else { - if (!manualClientRate.present() || - manualClientRate.get().tpsRate > priorityClientRate.get()) { - manualClientRate = ClientTagThrottleLimits(priorityClientRate.get(), - priorityItr->second.limits.expiration); - } else { - TEST(true); // Manual throttle overriden by higher priority - } - - ++priorityItr; - } - } - - if (manualClientRate.present()) { - tagPresent = true; - TEST(true); // Using manual throttle - clientRates[*priority][tagItr->first] = manualClientRate.get(); - } - } - - if (manualItr->second.empty()) { - TEST(true); // All manual throttles expired - manualThrottledTags.erase(manualItr); - break; - } - } - - auto autoItr = autoThrottledTags.find(tagItr->first); - if (autoItr != autoThrottledTags.end()) { - Optional autoClientRate = autoItr->second.updateAndGetClientRate(requestRate); - if (autoClientRate.present()) { - double adjustedRate = autoClientRate.get(); - double rampStartTime = autoItr->second.lastReduced + SERVER_KNOBS->AUTO_TAG_THROTTLE_DURATION - - SERVER_KNOBS->AUTO_TAG_THROTTLE_RAMP_UP_TIME; - if (now() >= rampStartTime && adjustedRate != std::numeric_limits::max()) { - TEST(true); // Tag auto-throttle ramping up - - double targetBusyness = SERVER_KNOBS->AUTO_THROTTLE_TARGET_TAG_BUSYNESS; - if (targetBusyness == 0) { - targetBusyness = 0.01; - } - - double rampLocation = (now() - rampStartTime) / SERVER_KNOBS->AUTO_TAG_THROTTLE_RAMP_UP_TIME; - adjustedRate = - computeTargetTpsRate(targetBusyness, pow(targetBusyness, 1 - rampLocation), adjustedRate); - } - - tagPresent = true; - if (autoThrottlingEnabled) { - auto result = clientRates[TransactionPriority::DEFAULT].try_emplace( - tagItr->first, adjustedRate, autoItr->second.limits.expiration); - if (!result.second && result.first->second.tpsRate > adjustedRate) { - result.first->second = - ClientTagThrottleLimits(adjustedRate, autoItr->second.limits.expiration); - } else { - TEST(true); // Auto throttle overriden by manual throttle - } - clientRates[TransactionPriority::BATCH][tagItr->first] = - ClientTagThrottleLimits(0, autoItr->second.limits.expiration); - } - } else { - ASSERT(autoItr->second.limits.expiration <= now()); - TEST(true); // Auto throttle expired - if (BUGGIFY) { // Temporarily extend the window between expiration and cleanup - tagPresent = true; - } else { - autoThrottledTags.erase(autoItr); - } - } - } - - if (!tagPresent) { - TEST(true); // All tag throttles expired - tagItr = tagData.erase(tagItr); - } else { - ++tagItr; - } - } - - return clientRates; - } - - void addRequests(TransactionTag const& tag, int requests) { - if (requests > 0) { - TEST(true); // Requests reported for throttled tag - - auto tagItr = tagData.try_emplace(tag); - tagItr.first->second.requestRate.addDelta(requests); - - double requestRate = tagItr.first->second.requestRate.smoothRate(); - - auto autoItr = autoThrottledTags.find(tag); - if (autoItr != autoThrottledTags.end()) { - autoItr->second.updateAndGetClientRate(requestRate); - } - - auto manualItr = manualThrottledTags.find(tag); - if (manualItr != manualThrottledTags.end()) { - for (auto priorityItr = manualItr->second.begin(); priorityItr != manualItr->second.end(); - ++priorityItr) { - priorityItr->second.updateAndGetClientRate(requestRate); - } - } - } - } - - Optional getRequestRate(TransactionTag const& tag) { - auto itr = tagData.find(tag); - if (itr != tagData.end()) { - return itr->second.requestRate.smoothRate(); - } - return Optional(); - } - - int64_t autoThrottleCount() const { return autoThrottledTags.size(); } - - int64_t manualThrottleCount() const { - int64_t count = 0; - for (auto itr = manualThrottledTags.begin(); itr != manualThrottledTags.end(); ++itr) { - count += itr->second.size(); - } - - return count; - } - - TransactionTagMap autoThrottledTags; - TransactionTagMap> manualThrottledTags; - TransactionTagMap tagData; - uint32_t busyReadTagCount = 0, busyWriteTagCount = 0; -}; +#include "fdbserver/RkTagThrottleCollection.h" class TagThrottlerImpl { Database db; @@ -461,11 +106,7 @@ class TagThrottlerImpl { if (tagKey.throttleType == TagThrottleType::AUTO) { updatedTagThrottles.autoThrottleTag( self->id, tag, 0, tagValue.tpsRate, tagValue.expirationTime); - if (tagValue.reason == TagThrottledReason::BUSY_READ) { - updatedTagThrottles.busyReadTagCount++; - } else if (tagValue.reason == TagThrottledReason::BUSY_WRITE) { - updatedTagThrottles.busyWriteTagCount++; - } + updatedTagThrottles.updateBusyTagCount(tagValue.reason); } else { updatedTagThrottles.manualThrottleTag(self->id, tag, @@ -537,8 +178,8 @@ public: return throttledTags.getClientRates(autoThrottlingEnabled); } int64_t autoThrottleCount() const { return throttledTags.autoThrottleCount(); } - uint32_t busyReadTagCount() const { return throttledTags.busyReadTagCount; } - uint32_t busyWriteTagCount() const { return throttledTags.busyWriteTagCount; } + uint32_t busyReadTagCount() const { return throttledTags.getBusyReadTagCount(); } + uint32_t busyWriteTagCount() const { return throttledTags.getBusyWriteTagCount(); } int64_t manualThrottleCount() const { return throttledTags.manualThrottleCount(); } bool isAutoThrottlingEnabled() const { return autoThrottlingEnabled; } From 49e4aa8e2673e37910d421e523dad5a6194cc315 Mon Sep 17 00:00:00 2001 From: sfc-gh-tclinkenbeard Date: Sun, 6 Mar 2022 22:18:05 -0400 Subject: [PATCH 333/413] Add encapsulation to Smoother class --- fdbrpc/Smoother.h | 44 +++++++++++++++++++++++--------------------- 1 file changed, 23 insertions(+), 21 deletions(-) diff --git a/fdbrpc/Smoother.h b/fdbrpc/Smoother.h index 667c4c60ed..1b6996fa27 100644 --- a/fdbrpc/Smoother.h +++ b/fdbrpc/Smoother.h @@ -25,9 +25,21 @@ #include "flow/flow.h" #include -struct Smoother { +class Smoother { // Times (t) are expected to be nondecreasing + double eFoldingTime; + double time, total, estimate; + + void update(double t) { + double elapsed = t - time; + if (elapsed) { + time = t; + estimate += (total - estimate) * (1 - exp(-elapsed / eFoldingTime)); + } + } + +public: explicit Smoother(double eFoldingTime) : eFoldingTime(eFoldingTime) { reset(0); } void reset(double value) { time = 0; @@ -50,22 +62,21 @@ struct Smoother { update(t); return (total - estimate) / eFoldingTime; } - - void update(double t) { - double elapsed = t - time; - if (elapsed) { - time = t; - estimate += (total - estimate) * (1 - exp(-elapsed / eFoldingTime)); - } - } - - double eFoldingTime; - double time, total, estimate; }; struct TimerSmoother { // Times (t) are expected to be nondecreasing + double eFoldingTime; + double time, total, estimate; + + void update(double t) { + double elapsed = t - time; + time = t; + estimate += (total - estimate) * (1 - exp(-elapsed / eFoldingTime)); + } + +public: explicit TimerSmoother(double eFoldingTime) : eFoldingTime(eFoldingTime) { reset(0); } void reset(double value) { time = 0; @@ -88,15 +99,6 @@ struct TimerSmoother { update(t); return (total - estimate) / eFoldingTime; } - - void update(double t) { - double elapsed = t - time; - time = t; - estimate += (total - estimate) * (1 - exp(-elapsed / eFoldingTime)); - } - - double eFoldingTime; - double time, total, estimate; }; #endif From ea9dd4b86c3e4d196d6eb42b5aa100e13aeb44ef Mon Sep 17 00:00:00 2001 From: sfc-gh-tclinkenbeard Date: Sun, 6 Mar 2022 22:43:30 -0400 Subject: [PATCH 334/413] Make some RkTagThrottleCollection methods private --- fdbserver/RkTagThrottleCollection.h | 5 ++--- 1 file changed, 2 insertions(+), 3 deletions(-) diff --git a/fdbserver/RkTagThrottleCollection.h b/fdbserver/RkTagThrottleCollection.h index dbad4e8fb7..5fa0b122aa 100644 --- a/fdbserver/RkTagThrottleCollection.h +++ b/fdbserver/RkTagThrottleCollection.h @@ -52,14 +52,14 @@ class RkTagThrottleCollection : NonCopyable { uint32_t busyReadTagCount = 0, busyWriteTagCount = 0; void initializeTag(TransactionTag const& tag) { tagData.try_emplace(tag); } + double computeTargetTpsRate(double currentBusyness, double targetBusyness, double requestRate) const; + Optional getRequestRate(TransactionTag const& tag); public: RkTagThrottleCollection() = default; RkTagThrottleCollection(RkTagThrottleCollection&& other); RkTagThrottleCollection& operator=(RkTagThrottleCollection&& other); - double computeTargetTpsRate(double currentBusyness, double targetBusyness, double requestRate) const; - // Returns the TPS rate if the throttle is updated, otherwise returns an empty optional Optional autoThrottleTag(UID id, TransactionTag const& tag, @@ -79,7 +79,6 @@ public: PrioritizedTransactionTagMap getClientRates(bool autoThrottlingEnabled); void addRequests(TransactionTag const& tag, int requests); - Optional getRequestRate(TransactionTag const& tag); int64_t autoThrottleCount() const { return autoThrottledTags.size(); } int64_t manualThrottleCount() const; void updateBusyTagCount(TagThrottledReason); From 28cb7d956f19fe3475b7a41ec0ad8ae0e643576e Mon Sep 17 00:00:00 2001 From: sfc-gh-tclinkenbeard Date: Sun, 6 Mar 2022 22:48:42 -0400 Subject: [PATCH 335/413] Make RkTagThrottleCollection::computeTargetTpsRate static --- fdbserver/RkTagThrottleCollection.cpp | 4 ++-- fdbserver/RkTagThrottleCollection.h | 2 +- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/fdbserver/RkTagThrottleCollection.cpp b/fdbserver/RkTagThrottleCollection.cpp index df76dcb21b..87f04dce31 100644 --- a/fdbserver/RkTagThrottleCollection.cpp +++ b/fdbserver/RkTagThrottleCollection.cpp @@ -69,8 +69,8 @@ RkTagThrottleCollection& RkTagThrottleCollection::RkTagThrottleCollection::opera double RkTagThrottleCollection::computeTargetTpsRate(double currentBusyness, double targetBusyness, - double requestRate) const { - ASSERT(currentBusyness > 0); + double requestRate) { + ASSERT_GT(currentBusyness, 0); if (targetBusyness < 1) { double targetFraction = targetBusyness * (1 - currentBusyness) / ((1 - targetBusyness) * currentBusyness); diff --git a/fdbserver/RkTagThrottleCollection.h b/fdbserver/RkTagThrottleCollection.h index 5fa0b122aa..2aebec8603 100644 --- a/fdbserver/RkTagThrottleCollection.h +++ b/fdbserver/RkTagThrottleCollection.h @@ -52,7 +52,7 @@ class RkTagThrottleCollection : NonCopyable { uint32_t busyReadTagCount = 0, busyWriteTagCount = 0; void initializeTag(TransactionTag const& tag) { tagData.try_emplace(tag); } - double computeTargetTpsRate(double currentBusyness, double targetBusyness, double requestRate) const; + static double computeTargetTpsRate(double currentBusyness, double targetBusyness, double requestRate); Optional getRequestRate(TransactionTag const& tag); public: From 3efe5803fd5d5211d35b2178567359b3c90dd103 Mon Sep 17 00:00:00 2001 From: sfc-gh-tclinkenbeard Date: Mon, 7 Mar 2022 14:21:33 -0400 Subject: [PATCH 336/413] Add comments to RkTagThrottleCollection --- fdbserver/RkTagThrottleCollection.h | 2 ++ 1 file changed, 2 insertions(+) diff --git a/fdbserver/RkTagThrottleCollection.h b/fdbserver/RkTagThrottleCollection.h index 2aebec8603..35062cdb7c 100644 --- a/fdbserver/RkTagThrottleCollection.h +++ b/fdbserver/RkTagThrottleCollection.h @@ -60,6 +60,7 @@ public: RkTagThrottleCollection(RkTagThrottleCollection&& other); RkTagThrottleCollection& operator=(RkTagThrottleCollection&& other); + // Set or update an auto throttling limit for the specified tag and priority combination. // Returns the TPS rate if the throttle is updated, otherwise returns an empty optional Optional autoThrottleTag(UID id, TransactionTag const& tag, @@ -67,6 +68,7 @@ public: Optional tpsRate = Optional(), Optional expiration = Optional()); + // Set or update a manual tps rate limit for the specified tag and priority combination void manualThrottleTag(UID id, TransactionTag const& tag, TransactionPriority priority, From 5799b6cbd79ffadcb169d61ae1d384757cb8f390 Mon Sep 17 00:00:00 2001 From: sfc-gh-tclinkenbeard Date: Fri, 11 Mar 2022 12:13:34 -0400 Subject: [PATCH 337/413] Add comments and modify ITagThrottler::tryUpdateAutoThrottling method --- fdbrpc/Smoother.h | 9 ++++--- fdbserver/Ratekeeper.actor.cpp | 6 ++--- fdbserver/Ratekeeper.h | 2 ++ fdbserver/RkTagThrottleCollection.cpp | 6 ++--- fdbserver/TagThrottler.actor.cpp | 37 ++++++++++++++------------- fdbserver/TagThrottler.h | 14 +++++++++- 6 files changed, 45 insertions(+), 29 deletions(-) diff --git a/fdbrpc/Smoother.h b/fdbrpc/Smoother.h index 1b6996fa27..d96e2e4874 100644 --- a/fdbrpc/Smoother.h +++ b/fdbrpc/Smoother.h @@ -29,9 +29,10 @@ class Smoother { // Times (t) are expected to be nondecreasing double eFoldingTime; - double time, total, estimate; + double total; + mutable double time, estimate; - void update(double t) { + void update(double t) const { double elapsed = t - time; if (elapsed) { time = t; @@ -53,12 +54,12 @@ public: total += delta; } // smoothTotal() is a continuous (under)estimate of the sum of all addDeltas() - double smoothTotal(double t = now()) { + double smoothTotal(double t = now()) const { update(t); return estimate; } // smoothRate() is d/dt[smoothTotal], and is NOT continuous - double smoothRate(double t = now()) { + double smoothRate(double t = now()) const { update(t); return (total - estimate) / eFoldingTime; } diff --git a/fdbserver/Ratekeeper.actor.cpp b/fdbserver/Ratekeeper.actor.cpp index 89c0f0d231..51486fe11d 100644 --- a/fdbserver/Ratekeeper.actor.cpp +++ b/fdbserver/Ratekeeper.actor.cpp @@ -564,10 +564,10 @@ void Ratekeeper::updateRate(RatekeeperLimits* limits) { } } - int64_t storageQueue = ss.lastReply.bytesInput - ss.smoothDurableBytes.smoothTotal(); + int64_t storageQueue = ss.getStorageQueueBytes(); worstStorageQueueStorageServer = std::max(worstStorageQueueStorageServer, storageQueue); - int64_t storageDurabilityLag = ss.smoothLatestVersion.smoothTotal() - ss.smoothDurableVersion.smoothTotal(); + int64_t storageDurabilityLag = ss.getDurabilityLag(); worstDurabilityLag = std::max(worstDurabilityLag, storageDurabilityLag); storageDurabilityLagReverseIndex.insert(std::make_pair(-1 * storageDurabilityLag, &ss)); @@ -581,7 +581,7 @@ void Ratekeeper::updateRate(RatekeeperLimits* limits) { double targetRateRatio = std::min((storageQueue - targetBytes + springBytes) / (double)springBytes, 2.0); if (limits->priority == TransactionPriority::DEFAULT) { - addActor.send(tagThrottler->tryAutoThrottleTag(ss, storageQueue, storageDurabilityLag)); + addActor.send(tagThrottler->tryUpdateAutoThrottling(ss)); } double inputRate = ss.smoothInputBytes.smoothRate(); diff --git a/fdbserver/Ratekeeper.h b/fdbserver/Ratekeeper.h index 28b93266fc..a355288036 100644 --- a/fdbserver/Ratekeeper.h +++ b/fdbserver/Ratekeeper.h @@ -69,6 +69,8 @@ struct StorageQueueInfo { StorageQueueInfo(UID id, LocalityData locality); void refreshCommitCost(double elapsed); + int64_t getStorageQueueBytes() const { return lastReply.bytesInput - smoothDurableBytes.smoothTotal(); } + int64_t getDurabilityLag() const { return smoothLatestVersion.smoothTotal() - smoothDurableVersion.smoothTotal(); } }; struct TLogQueueInfo { diff --git a/fdbserver/RkTagThrottleCollection.cpp b/fdbserver/RkTagThrottleCollection.cpp index 87f04dce31..11c376f57d 100644 --- a/fdbserver/RkTagThrottleCollection.cpp +++ b/fdbserver/RkTagThrottleCollection.cpp @@ -45,7 +45,7 @@ Optional RkTagThrottleCollection::RkTagThrottleData::updateAndGetClientR } double rate = clientRate.smoothTotal(); - ASSERT(rate >= 0); + ASSERT_GE(rate, 0); return rate; } else { TEST(true); // Get throttle rate for expired throttle @@ -323,8 +323,8 @@ void RkTagThrottleCollection::addRequests(TransactionTag const& tag, int request auto manualItr = manualThrottledTags.find(tag); if (manualItr != manualThrottledTags.end()) { - for (auto priorityItr = manualItr->second.begin(); priorityItr != manualItr->second.end(); ++priorityItr) { - priorityItr->second.updateAndGetClientRate(requestRate); + for (auto& [priority, tagThrottleData] : manualItr->second) { + tagThrottleData.updateAndGetClientRate(requestRate); } } } diff --git a/fdbserver/TagThrottler.actor.cpp b/fdbserver/TagThrottler.actor.cpp index e7623ad1ca..979ddff53e 100644 --- a/fdbserver/TagThrottler.actor.cpp +++ b/fdbserver/TagThrottler.actor.cpp @@ -137,16 +137,19 @@ class TagThrottlerImpl { } } + /* Optional autoThrottleTag(TransactionTag tag, double busyness) { - return throttledTags.autoThrottleTag(id, tag, busyness); + return throttledTags.autoThrottleTag(id, tag, busyness); } + */ - Future tryAutoThrottleTag(TransactionTag tag, double rate, double busyness, TagThrottledReason reason) { + Future tryUpdateAutoThrottling(TransactionTag tag, double rate, double busyness, TagThrottledReason reason) { // NOTE: before the comparison with MIN_TAG_COST, the busiest tag rate also compares with MIN_TAG_PAGES_RATE // currently MIN_TAG_PAGES_RATE > MIN_TAG_COST in our default knobs. if (busyness > SERVER_KNOBS->AUTO_THROTTLE_TARGET_TAG_BUSYNESS && rate > SERVER_KNOBS->MIN_TAG_COST) { TEST(true); // Transaction tag auto-throttled - Optional clientRate = autoThrottleTag(tag, busyness); + Optional clientRate = + throttledTags.autoThrottleTag(id, tag, busyness); // autoThrottleTag(tag, busyness); if (clientRate.present()) { TagSet tags; tags.addTag(tag); @@ -183,23 +186,23 @@ public: int64_t manualThrottleCount() const { return throttledTags.manualThrottleCount(); } bool isAutoThrottlingEnabled() const { return autoThrottlingEnabled; } - Future tryAutoThrottleTag(StorageQueueInfo& ss, int64_t storageQueue, int64_t storageDurabilityLag) { + Future tryUpdateAutoThrottling(StorageQueueInfo const& ss) { // NOTE: we just keep it simple and don't differentiate write-saturation and read-saturation at the moment. In // most of situation, this works. More indicators besides queue size and durability lag could be investigated in // the future + auto storageQueue = ss.getStorageQueueBytes(); + auto storageDurabilityLag = ss.getDurabilityLag(); if (storageQueue > SERVER_KNOBS->AUTO_TAG_THROTTLE_STORAGE_QUEUE_BYTES || storageDurabilityLag > SERVER_KNOBS->AUTO_TAG_THROTTLE_DURABILITY_LAG_VERSIONS) { - for (const auto& busiestWriteTag : ss.busiestWriteTags) { - return tryAutoThrottleTag(busiestWriteTag.tag, - busiestWriteTag.rate, - busiestWriteTag.fractionalBusyness, - TagThrottledReason::BUSY_READ); + for (const auto& busyWriteTag : ss.busiestWriteTags) { + return tryUpdateAutoThrottling(busyWriteTag.tag, + busyWriteTag.rate, + busyWriteTag.fractionalBusyness, + TagThrottledReason::BUSY_READ); } - for (const auto& busiestReadTag : ss.busiestReadTags) { - return tryAutoThrottleTag(busiestReadTag.tag, - busiestReadTag.rate, - busiestReadTag.fractionalBusyness, - TagThrottledReason::BUSY_READ); + for (const auto& busyReadTag : ss.busiestReadTags) { + return tryUpdateAutoThrottling( + busyReadTag.tag, busyReadTag.rate, busyReadTag.fractionalBusyness, TagThrottledReason::BUSY_READ); } } return Void(); @@ -236,8 +239,6 @@ int64_t TagThrottler::manualThrottleCount() const { bool TagThrottler::isAutoThrottlingEnabled() const { return impl->isAutoThrottlingEnabled(); } -Future TagThrottler::tryAutoThrottleTag(StorageQueueInfo& ss, - int64_t storageQueue, - int64_t storageDurabilityLag) { - return impl->tryAutoThrottleTag(ss, storageQueue, storageDurabilityLag); +Future TagThrottler::tryUpdateAutoThrottling(StorageQueueInfo const& ss) { + return impl->tryUpdateAutoThrottling(ss); } diff --git a/fdbserver/TagThrottler.h b/fdbserver/TagThrottler.h index 4070162545..99ce15126e 100644 --- a/fdbserver/TagThrottler.h +++ b/fdbserver/TagThrottler.h @@ -28,15 +28,27 @@ class TagThrottler { public: TagThrottler(Database db, UID id); - ~TagThrottler(); + ~ITagThrottler(); + + // Poll the system keyspace looking for updates made through the tag throttling API Future monitorThrottlingChanges(); + + // Increment the number of known requests associated with the specified tag void addRequests(TransactionTag tag, int count); + + // This throttled tag change ID is used to coordinate updates with the uint64_t getThrottledTagChangeId() const; + + // For each tag and priority combination, return the throughput limit and expiration time PrioritizedTransactionTagMap getClientRates(); + int64_t autoThrottleCount() const; uint32_t busyReadTagCount() const; uint32_t busyWriteTagCount() const; int64_t manualThrottleCount() const; bool isAutoThrottlingEnabled() const; + + // Based on the busiest read and write tags in the provided storage queue info, update + // tag throttling limits. Future tryAutoThrottleTag(StorageQueueInfo&, int64_t storageQueue, int64_t storageDurabilityLag); }; From cc52a44df55a1a76553478e871971dba59f97802 Mon Sep 17 00:00:00 2001 From: sfc-gh-tclinkenbeard Date: Fri, 11 Mar 2022 12:14:26 -0400 Subject: [PATCH 338/413] Clean up TagThrottler* files --- fdbserver/TagThrottler.actor.cpp | 9 +-------- fdbserver/TagThrottler.h | 6 +++--- 2 files changed, 4 insertions(+), 11 deletions(-) diff --git a/fdbserver/TagThrottler.actor.cpp b/fdbserver/TagThrottler.actor.cpp index 979ddff53e..043999848a 100644 --- a/fdbserver/TagThrottler.actor.cpp +++ b/fdbserver/TagThrottler.actor.cpp @@ -137,19 +137,12 @@ class TagThrottlerImpl { } } - /* - Optional autoThrottleTag(TransactionTag tag, double busyness) { - return throttledTags.autoThrottleTag(id, tag, busyness); - } - */ - Future tryUpdateAutoThrottling(TransactionTag tag, double rate, double busyness, TagThrottledReason reason) { // NOTE: before the comparison with MIN_TAG_COST, the busiest tag rate also compares with MIN_TAG_PAGES_RATE // currently MIN_TAG_PAGES_RATE > MIN_TAG_COST in our default knobs. if (busyness > SERVER_KNOBS->AUTO_THROTTLE_TARGET_TAG_BUSYNESS && rate > SERVER_KNOBS->MIN_TAG_COST) { TEST(true); // Transaction tag auto-throttled - Optional clientRate = - throttledTags.autoThrottleTag(id, tag, busyness); // autoThrottleTag(tag, busyness); + Optional clientRate = throttledTags.autoThrottleTag(id, tag, busyness); if (clientRate.present()) { TagSet tags; tags.addTag(tag); diff --git a/fdbserver/TagThrottler.h b/fdbserver/TagThrottler.h index 99ce15126e..69e3909c7d 100644 --- a/fdbserver/TagThrottler.h +++ b/fdbserver/TagThrottler.h @@ -28,7 +28,7 @@ class TagThrottler { public: TagThrottler(Database db, UID id); - ~ITagThrottler(); + ~TagThrottler(); // Poll the system keyspace looking for updates made through the tag throttling API Future monitorThrottlingChanges(); @@ -36,7 +36,7 @@ public: // Increment the number of known requests associated with the specified tag void addRequests(TransactionTag tag, int count); - // This throttled tag change ID is used to coordinate updates with the + // This throttled tag change ID is used to coordinate updates with the GRV proxies uint64_t getThrottledTagChangeId() const; // For each tag and priority combination, return the throughput limit and expiration time @@ -50,5 +50,5 @@ public: // Based on the busiest read and write tags in the provided storage queue info, update // tag throttling limits. - Future tryAutoThrottleTag(StorageQueueInfo&, int64_t storageQueue, int64_t storageDurabilityLag); + Future tryUpdateAutoThrottling(StorageQueueInfo const&); }; From 67dd3d071e0cd027ddd4058277d168a6e9864ec5 Mon Sep 17 00:00:00 2001 From: Josh Slocum Date: Fri, 11 Mar 2022 10:49:26 -0600 Subject: [PATCH 339/413] Fixing incorrect assert in BM split handling --- fdbserver/BlobManager.actor.cpp | 48 ++++++++++++++++++++++++++------- 1 file changed, 39 insertions(+), 9 deletions(-) diff --git a/fdbserver/BlobManager.actor.cpp b/fdbserver/BlobManager.actor.cpp index 8d1b839e38..9a2e1cdb69 100644 --- a/fdbserver/BlobManager.actor.cpp +++ b/fdbserver/BlobManager.actor.cpp @@ -1074,6 +1074,14 @@ ACTOR Future maybeSplitRange(Reference bmData, } } + if (BM_DEBUG) { + fmt::print("Splitting range [{0} - {1}) into {2} granules @ {3} done, sending assignments:\n", + granuleRange.begin.printable(), + granuleRange.end.printable(), + newRanges.size() - 1, + latestVersion); + } + // transaction committed, send range assignments // range could have been moved since split eval started, so just revoke from whoever has it RangeAssignment raRevoke; @@ -1092,10 +1100,27 @@ ACTOR Future maybeSplitRange(Reference bmData, bmData->rangesToAssign.send(raAssignSplit); } + if (BM_DEBUG) { + fmt::print( + "Splitting range [{0} - {1}) into {2} granules @ {3} sent assignments, waiting for them to be processed:\n", + granuleRange.begin.printable(), + granuleRange.end.printable(), + newRanges.size() - 1, + latestVersion); + } + // Ensure the new assignments actually got processed and the split boundaries are reflected in the granule mapping // before returning. This prevents a race with a subsequent split evaluation wait(bmData->rangesToAssign.onEmpty()); + if (BM_DEBUG) { + fmt::print("Splitting range [{0} - {1}) into {2} granules @ {3} got assignments processed\n", + granuleRange.begin.printable(), + granuleRange.end.printable(), + newRanges.size() - 1, + latestVersion); + } + return Void(); } @@ -1268,18 +1293,21 @@ ACTOR Future monitorBlobWorkerStatus(Reference bmData, Bl currGranuleAssignment.end() == rep.granuleRange.end && currGranuleAssignment.cvalue() == bwInterf.id())) { if (BM_DEBUG) { - fmt::print( - "Manager {0} ignoring status from BW {1} for granule [{2} - {3}) since BW {4} owns it.\n", - bmData->epoch, - bwInterf.id().toString().substr(0, 5), - rep.granuleRange.begin.printable(), - rep.granuleRange.end.printable(), - currGranuleAssignment.cvalue().toString().substr(0, 5)); + fmt::print("Manager {0} ignoring status from BW {1} for granule [{2} - {3}) since BW {4} owns " + "[{5} - {6}).\n", + bmData->epoch, + bwInterf.id().toString().substr(0, 5), + rep.granuleRange.begin.printable(), + rep.granuleRange.end.printable(), + currGranuleAssignment.cvalue().toString().substr(0, 5), + currGranuleAssignment.begin().printable(), + currGranuleAssignment.end().printable()); } // FIXME: could send revoke request continue; } + // FIXME: We will need to go over all splits in the range once we're doing merges, instead of first one auto lastSplitEval = bmData->splitEvaluations.rangeContaining(rep.granuleRange.begin); if (rep.granuleRange.begin == lastSplitEval.begin() && rep.granuleRange.end == lastSplitEval.end() && rep.epoch == lastSplitEval.cvalue().epoch && rep.seqno == lastSplitEval.cvalue().seqno) { @@ -1298,18 +1326,20 @@ ACTOR Future monitorBlobWorkerStatus(Reference bmData, Bl // For example, one worker asked BM to split, then died, granule was moved, new worker asks to // split on recovery. We need to ensure that they are semantically the same split. // We will just rely on the in-progress split to finish - ASSERT(lastSplitEval.range() == rep.granuleRange); if (BM_DEBUG) { fmt::print("Manager {0} got split request for [{1} - {2}) @ ({3}, {4}), but already in " - "progress from ({5}, {6})\n", + "progress from [{5} - {6}) @ ({7}, {8})\n", bmData->epoch, rep.granuleRange.begin.printable().c_str(), rep.granuleRange.end.printable().c_str(), rep.epoch, rep.seqno, + lastSplitEval.begin().printable().c_str(), + lastSplitEval.end().printable().c_str(), lastSplitEval.cvalue().epoch, lastSplitEval.cvalue().seqno); } + // ignore the request, they will retry } else { if (BM_DEBUG) { fmt::print("Manager {0} evaluating [{1} - {2}) @ ({3}, {4}) for split\n", From 82eef0c7f7074ed2e70824b61875df6ce29682b0 Mon Sep 17 00:00:00 2001 From: Josh Slocum Date: Fri, 11 Mar 2022 16:37:52 -0600 Subject: [PATCH 340/413] Better CF tracing for client and server --- fdbclient/NativeAPI.actor.cpp | 39 ++++++++++++++++++++++++++++--- fdbserver/storageserver.actor.cpp | 33 +++++++++++++++++++++++++- 2 files changed, 68 insertions(+), 4 deletions(-) diff --git a/fdbclient/NativeAPI.actor.cpp b/fdbclient/NativeAPI.actor.cpp index 19bdaac6a2..1e3c70a733 100644 --- a/fdbclient/NativeAPI.actor.cpp +++ b/fdbclient/NativeAPI.actor.cpp @@ -7570,13 +7570,16 @@ Future ChangeFeedData::whenAtLeast(Version version) { return changeFeedWhenAtLatest(Reference::addRef(this), version); } +#define DEBUG_CF_CLIENT_TRACE false + ACTOR Future partialChangeFeedStream(StorageServerInterface interf, PromiseStream> results, ReplyPromiseStream replyStream, Version begin, Version end, Reference feedData, - Reference storageData) { + Reference storageData, + UID debugUID /* TODO REMOVE */) { // calling lastReturnedVersion's callbacks could cause us to be cancelled state Promise refresh = feedData->refresh; @@ -7649,6 +7652,9 @@ ACTOR Future partialChangeFeedStream(StorageServerInterface interf, } } } catch (Error& e) { + if (DEBUG_CF_CLIENT_TRACE) { + TraceEvent(SevDebug, "TraceChangeFeedClientMergeCursorError", debugUID).errorUnsuppressed(e); + } if (e.code() == error_code_actor_cancelled) { throw; } @@ -7775,6 +7781,7 @@ ACTOR Future mergeChangeFeedStream(Reference db, TEST(interfs.size() > 10); // Large change feed merge cursor TEST(interfs.size() > 100); // Very large change feed merge cursor + state std::vector debugUIDs; results->streams.clear(); for (auto& it : interfs) { ChangeFeedStreamRequest req; @@ -7790,6 +7797,17 @@ ACTOR Future mergeChangeFeedStream(Reference db, } // TODO REMOVE req.debugUID = deterministicRandom()->randomUniqueID(); + debugUIDs.push_back(req.debugUID); + + if (DEBUG_CF_CLIENT_TRACE) { + TraceEvent(SevDebug, "TraceChangeFeedClientMergeCursor", req.debugUID) + .detail("FeedID", rangeID) + .detail("MergeRange", KeyRangeRef(interfs.front().second.begin, interfs.back().second.end)) + .detail("PartialRange", it.second) + .detail("Begin", *begin) + .detail("End", end) + .detail("CanReadPopped", true); + } results->streams.push_back(it.first.changeFeedStream.getReplyStream(req)); } @@ -7811,8 +7829,14 @@ ACTOR Future mergeChangeFeedStream(Reference db, for (int i = 0; i < interfs.size(); i++) { onErrors[i] = results->streams[i].onError(); - fetchers[i] = partialChangeFeedStream( - interfs[i].first, streams[i].results, results->streams[i], *begin, end, results, results->storageData[i]); + fetchers[i] = partialChangeFeedStream(interfs[i].first, + streams[i].results, + results->streams[i], + *begin, + end, + results, + results->storageData[i], + debugUIDs[i]); } wait(onCFErrors(onErrors) || mergeChangeFeedStreamInternal(results, interfs, streams, begin, end)); @@ -7940,6 +7964,15 @@ ACTOR Future singleChangeFeedStream(Reference db, // TODO REMOVE req.debugUID = deterministicRandom()->randomUniqueID(); + if (DEBUG_CF_CLIENT_TRACE) { + TraceEvent(SevDebug, "TraceChangeFeedClientSingleCursor", req.debugUID) + .detail("FeedID", rangeID) + .detail("Range", range) + .detail("Begin", *begin) + .detail("End", end) + .detail("CanReadPopped", true); + } + results->streams.clear(); for (auto& it : results->storageData) { diff --git a/fdbserver/storageserver.actor.cpp b/fdbserver/storageserver.actor.cpp index 86fba6190b..a4f5b9033c 100644 --- a/fdbserver/storageserver.actor.cpp +++ b/fdbserver/storageserver.actor.cpp @@ -2333,7 +2333,7 @@ ACTOR Future changeFeedStreamQ(StorageServer* data, ChangeFeedStreamReques try { if (DEBUG_CF_TRACE) { - TraceEvent(SevDebug, "TraceChangeFeedMutationsBegin", data->thisServerID) + TraceEvent(SevDebug, "TraceChangeFeedStreamStart", data->thisServerID) .detail("FeedID", req.rangeID) .detail("StreamUID", streamUID) .detail("Range", req.range) @@ -2352,11 +2352,32 @@ ACTOR Future changeFeedStreamQ(StorageServer* data, ChangeFeedStreamReques ASSERT(emptyInitialReply.minStreamVersion == invalidVersion); req.reply.send(emptyInitialReply); + if (DEBUG_CF_TRACE) { + TraceEvent(SevDebug, "TraceChangeFeedStreamSentInitialEmpty", data->thisServerID) + .detail("FeedID", req.rangeID) + .detail("StreamUID", streamUID) + .detail("Range", req.range) + .detail("Begin", req.begin) + .detail("End", req.end) + .detail("CanReadPopped", req.canReadPopped) + .detail("Version", req.begin - 1); + } + loop { Future onReady = req.reply.onReady(); if (atLatest && !onReady.isReady() && !removeUID) { data->changeFeedClientVersions[req.reply.getEndpoint().getPrimaryAddress()][streamUID] = blockedVersion.present() ? blockedVersion.get() : data->prevVersion; + if (DEBUG_CF_TRACE) { + TraceEvent(SevDebug, "TraceChangeFeedStreamBlockedOnReady", data->thisServerID) + .detail("FeedID", req.rangeID) + .detail("StreamUID", streamUID) + .detail("Range", req.range) + .detail("Begin", req.begin) + .detail("End", req.end) + .detail("CanReadPopped", req.canReadPopped) + .detail("Version", blockedVersion.present() ? blockedVersion.get() : data->prevVersion); + } removeUID = true; } wait(onReady); @@ -2367,6 +2388,16 @@ ACTOR Future changeFeedStreamQ(StorageServer* data, ChangeFeedStreamReques data->changeFeedClientVersions[req.reply.getEndpoint().getPrimaryAddress()][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("Range", req.range) + .detail("Begin", req.begin) + .detail("End", req.end) + .detail("CanReadPopped", req.canReadPopped) + .detail("Version", blockedVersion.present() ? blockedVersion.get() : data->prevVersion); + } } std::pair _feedReply = wait(feedReplyFuture); ChangeFeedStreamReply feedReply = _feedReply.first; From a2bbb188c3636dd42ee8a5d1c4c3d1a8f2beba84 Mon Sep 17 00:00:00 2001 From: Josh Slocum Date: Fri, 11 Mar 2022 16:50:23 -0600 Subject: [PATCH 341/413] Fixing delayed health updater to mark address failed when too many connections triggers before new connection starts, instead of after --- fdbrpc/FlowTransport.actor.cpp | 31 ++++++++++++++++++------------- 1 file changed, 18 insertions(+), 13 deletions(-) diff --git a/fdbrpc/FlowTransport.actor.cpp b/fdbrpc/FlowTransport.actor.cpp index 3e3b13709a..f4fbf461f0 100644 --- a/fdbrpc/FlowTransport.actor.cpp +++ b/fdbrpc/FlowTransport.actor.cpp @@ -545,28 +545,20 @@ ACTOR Future connectionWriter(Reference self, Reference } } -ACTOR Future delayedHealthUpdate(NetworkAddress address) { +ACTOR Future delayedHealthUpdate(NetworkAddress address, bool* tooManyConnectionsClosed) { state double start = now(); - state bool delayed = false; loop { if (FLOW_KNOBS->HEALTH_MONITOR_MARK_FAILED_UNSTABLE_CONNECTIONS && FlowTransport::transport().healthMonitor()->tooManyConnectionsClosed(address) && address.isPublic()) { - if (!delayed) { - TraceEvent("TooManyConnectionsClosedMarkFailed") - .detail("Dest", address) - .detail("StartTime", start) - .detail("ClosedCount", FlowTransport::transport().healthMonitor()->closedConnectionsCount(address)); - IFailureMonitor::failureMonitor().setStatus(address, FailureStatus(true)); - } - delayed = true; wait(delayJittered(FLOW_KNOBS->MAX_RECONNECTION_TIME * 2.0)); } else { - if (delayed) { + if (*tooManyConnectionsClosed) { TraceEvent("TooManyConnectionsClosedMarkAvailable") .detail("Dest", address) .detail("StartTime", start) .detail("TimeElapsed", now() - start) .detail("ClosedCount", FlowTransport::transport().healthMonitor()->closedConnectionsCount(address)); + *tooManyConnectionsClosed = false; } IFailureMonitor::failureMonitor().setStatus(address, FailureStatus(false)); break; @@ -586,6 +578,7 @@ ACTOR Future connectionKeeper(Reference self, state Future delayedHealthUpdateF; state Optional firstConnFailedTime = Optional(); state int retryConnect = false; + state bool tooManyConnectionsClosed = false; loop { try { @@ -635,7 +628,8 @@ ACTOR Future connectionKeeper(Reference self, IFailureMonitor::failureMonitor().setStatus(self->destination, FailureStatus(false)); } if (self->unsent.empty()) { - delayedHealthUpdateF = delayedHealthUpdate(self->destination); + delayedHealthUpdateF = + delayedHealthUpdate(self->destination, &tooManyConnectionsClosed); choose { when(wait(delayedHealthUpdateF)) { conn->close(); @@ -675,7 +669,7 @@ ACTOR Future connectionKeeper(Reference self, try { self->transport->countConnEstablished++; if (!delayedHealthUpdateF.isValid()) - delayedHealthUpdateF = delayedHealthUpdate(self->destination); + delayedHealthUpdateF = delayedHealthUpdate(self->destination, &tooManyConnectionsClosed); wait(connectionWriter(self, conn) || reader || connectionMonitor(self) || self->resetConnection.onTrigger()); TraceEvent("ConnectionReset", conn ? conn->getDebugID() : UID()) @@ -761,6 +755,17 @@ ACTOR Future connectionKeeper(Reference self, if (conn) { if (self->destination.isPublic() && e.code() == error_code_connection_failed) { FlowTransport::transport().healthMonitor()->reportPeerClosed(self->destination); + if (FLOW_KNOBS->HEALTH_MONITOR_MARK_FAILED_UNSTABLE_CONNECTIONS && + FlowTransport::transport().healthMonitor()->tooManyConnectionsClosed(self->destination) && + self->destination.isPublic()) { + TraceEvent("TooManyConnectionsClosedMarkFailed") + .detail("Dest", self->destination) + .detail( + "ClosedCount", + FlowTransport::transport().healthMonitor()->closedConnectionsCount(self->destination)); + tooManyConnectionsClosed = true; + IFailureMonitor::failureMonitor().setStatus(self->destination, FailureStatus(true)); + } } conn->close(); From 324a678e39053350f4186b313f67720da64b7164 Mon Sep 17 00:00:00 2001 From: sfc-gh-tclinkenbeard Date: Sat, 12 Mar 2022 13:32:41 -0400 Subject: [PATCH 342/413] Minor bug fix in TagThrottlerImpl::tryUpdateAutoThrottling --- fdbserver/TagThrottler.actor.cpp | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/fdbserver/TagThrottler.actor.cpp b/fdbserver/TagThrottler.actor.cpp index 043999848a..d85e021544 100644 --- a/fdbserver/TagThrottler.actor.cpp +++ b/fdbserver/TagThrottler.actor.cpp @@ -187,11 +187,14 @@ public: auto storageDurabilityLag = ss.getDurabilityLag(); if (storageQueue > SERVER_KNOBS->AUTO_TAG_THROTTLE_STORAGE_QUEUE_BYTES || storageDurabilityLag > SERVER_KNOBS->AUTO_TAG_THROTTLE_DURABILITY_LAG_VERSIONS) { + // TODO: Update once size is potentially > 1 + ASSERT_WE_THINK(ss.busiestWriteTags.size() <= 1); + ASSERT_WE_THINK(ss.busiestReadTags.size() <= 1); for (const auto& busyWriteTag : ss.busiestWriteTags) { return tryUpdateAutoThrottling(busyWriteTag.tag, busyWriteTag.rate, busyWriteTag.fractionalBusyness, - TagThrottledReason::BUSY_READ); + TagThrottledReason::BUSY_WRITE); } for (const auto& busyReadTag : ss.busiestReadTags) { return tryUpdateAutoThrottling( From 1ef010297435cd4441e15721f848b7701d9d84f1 Mon Sep 17 00:00:00 2001 From: sfc-gh-tclinkenbeard Date: Sat, 12 Mar 2022 14:04:10 -0400 Subject: [PATCH 343/413] Add TLogQueueInfo::update method --- fdbserver/Ratekeeper.actor.cpp | 43 ++++++++++++++++++---------------- fdbserver/Ratekeeper.h | 6 +++-- 2 files changed, 27 insertions(+), 22 deletions(-) diff --git a/fdbserver/Ratekeeper.actor.cpp b/fdbserver/Ratekeeper.actor.cpp index 51486fe11d..6b1f8e615d 100644 --- a/fdbserver/Ratekeeper.actor.cpp +++ b/fdbserver/Ratekeeper.actor.cpp @@ -208,24 +208,7 @@ public: ErrorOr reply = wait(tli.getQueuingMetrics.getReplyUnlessFailedFor( TLogQueuingMetricsRequest(), 0, 0)); // SOMEDAY: or tryGetReply? if (reply.present()) { - myQueueInfo->value.valid = true; - myQueueInfo->value.prevReply = myQueueInfo->value.lastReply; - myQueueInfo->value.lastReply = reply.get(); - if (myQueueInfo->value.prevReply.instanceID != reply.get().instanceID) { - myQueueInfo->value.smoothDurableBytes.reset(reply.get().bytesDurable); - myQueueInfo->value.verySmoothDurableBytes.reset(reply.get().bytesDurable); - myQueueInfo->value.smoothInputBytes.reset(reply.get().bytesInput); - myQueueInfo->value.smoothFreeSpace.reset(reply.get().storageBytes.available); - myQueueInfo->value.smoothTotalSpace.reset(reply.get().storageBytes.total); - } else { - self->smoothTotalDurableBytes.addDelta(reply.get().bytesDurable - - myQueueInfo->value.prevReply.bytesDurable); - myQueueInfo->value.smoothDurableBytes.setTotal(reply.get().bytesDurable); - myQueueInfo->value.verySmoothDurableBytes.setTotal(reply.get().bytesDurable); - myQueueInfo->value.smoothInputBytes.setTotal(reply.get().bytesInput); - myQueueInfo->value.smoothFreeSpace.setTotal(reply.get().storageBytes.available); - myQueueInfo->value.smoothTotalSpace.setTotal(reply.get().storageBytes.total); - } + myQueueInfo->value.update(reply.get(), self->smoothTotalDurableBytes); } else { if (myQueueInfo->value.valid) { TraceEvent("RkTLogDidNotRespond", self->id).detail("TransactionLog", tli.id()); @@ -769,11 +752,11 @@ void Ratekeeper::updateRate(RatekeeperLimits* limits) { auto& tl = it.value; if (!tl.valid) continue; - maxTLVer = std::max(maxTLVer, tl.lastReply.v); + maxTLVer = std::max(maxTLVer, tl.getLastCommittedVersion()); } if (minSSVer != std::numeric_limits::max() && maxTLVer != std::numeric_limits::min()) { - // writeToReadLatencyLimit: 0 = infinte speed; 1 = TL durable speed ; 2 = half TL durable speed + // writeToReadLatencyLimit: 0 = infinite speed; 1 = TL durable speed ; 2 = half TL durable speed writeToReadLatencyLimit = ((maxTLVer - minLimitingSSVer) - limits->maxVersionDifference / 2) / (limits->maxVersionDifference / 4); worstVersionLag = std::max((Version)0, maxTLVer - minSSVer); @@ -1036,6 +1019,26 @@ TLogQueueInfo::TLogQueueInfo(UID id) lastReply.instanceID = -1; } +void TLogQueueInfo::update(TLogQueuingMetricsReply const& reply, Smoother& smoothTotalDurableBytes) { + valid = true; + auto prevReply = std::move(lastReply); + lastReply = reply; + if (prevReply.instanceID != reply.instanceID) { + smoothDurableBytes.reset(reply.bytesDurable); + verySmoothDurableBytes.reset(reply.bytesDurable); + smoothInputBytes.reset(reply.bytesInput); + smoothFreeSpace.reset(reply.storageBytes.available); + smoothTotalSpace.reset(reply.storageBytes.total); + } else { + smoothTotalDurableBytes.addDelta(reply.bytesDurable - prevReply.bytesDurable); + smoothDurableBytes.setTotal(reply.bytesDurable); + verySmoothDurableBytes.setTotal(reply.bytesDurable); + smoothInputBytes.setTotal(reply.bytesInput); + smoothFreeSpace.setTotal(reply.storageBytes.available); + smoothTotalSpace.setTotal(reply.storageBytes.total); + } +} + RatekeeperLimits::RatekeeperLimits(TransactionPriority priority, std::string context, int64_t storageTargetBytes, diff --git a/fdbserver/Ratekeeper.h b/fdbserver/Ratekeeper.h index a355288036..79ecc359d4 100644 --- a/fdbserver/Ratekeeper.h +++ b/fdbserver/Ratekeeper.h @@ -74,14 +74,16 @@ struct StorageQueueInfo { }; struct TLogQueueInfo { + TLogQueuingMetricsReply lastReply; bool valid; UID id; - TLogQueuingMetricsReply lastReply; - TLogQueuingMetricsReply prevReply; Smoother smoothDurableBytes, smoothInputBytes, verySmoothDurableBytes; Smoother smoothFreeSpace; Smoother smoothTotalSpace; + TLogQueueInfo(UID id); + Version getLastCommittedVersion() const { return lastReply.v; } + void update(TLogQueuingMetricsReply const& reply, Smoother& smoothTotalDurableBytes); }; struct RatekeeperLimits { From cdc099b1925db1d8657880dd199c0dc39d3dec2b Mon Sep 17 00:00:00 2001 From: sfc-gh-tclinkenbeard Date: Sat, 12 Mar 2022 14:17:44 -0400 Subject: [PATCH 344/413] Add StorageQueueInfo::update method --- fdbserver/Ratekeeper.actor.cpp | 51 ++++++++++++++++++---------------- fdbserver/Ratekeeper.h | 2 +- 2 files changed, 28 insertions(+), 25 deletions(-) diff --git a/fdbserver/Ratekeeper.actor.cpp b/fdbserver/Ratekeeper.actor.cpp index 6b1f8e615d..3a42268475 100644 --- a/fdbserver/Ratekeeper.actor.cpp +++ b/fdbserver/Ratekeeper.actor.cpp @@ -157,30 +157,7 @@ public: ErrorOr reply = wait(ssi.getQueuingMetrics.getReplyUnlessFailedFor( StorageQueuingMetricsRequest(), 0, 0)); // SOMEDAY: or tryGetReply? if (reply.present()) { - myQueueInfo->value.valid = true; - myQueueInfo->value.prevReply = myQueueInfo->value.lastReply; - myQueueInfo->value.lastReply = reply.get(); - if (myQueueInfo->value.prevReply.instanceID != reply.get().instanceID) { - myQueueInfo->value.smoothDurableBytes.reset(reply.get().bytesDurable); - myQueueInfo->value.verySmoothDurableBytes.reset(reply.get().bytesDurable); - myQueueInfo->value.smoothInputBytes.reset(reply.get().bytesInput); - myQueueInfo->value.smoothFreeSpace.reset(reply.get().storageBytes.available); - myQueueInfo->value.smoothTotalSpace.reset(reply.get().storageBytes.total); - myQueueInfo->value.smoothDurableVersion.reset(reply.get().durableVersion); - myQueueInfo->value.smoothLatestVersion.reset(reply.get().version); - } else { - self->smoothTotalDurableBytes.addDelta(reply.get().bytesDurable - - myQueueInfo->value.prevReply.bytesDurable); - myQueueInfo->value.smoothDurableBytes.setTotal(reply.get().bytesDurable); - myQueueInfo->value.verySmoothDurableBytes.setTotal(reply.get().bytesDurable); - myQueueInfo->value.smoothInputBytes.setTotal(reply.get().bytesInput); - myQueueInfo->value.smoothFreeSpace.setTotal(reply.get().storageBytes.available); - myQueueInfo->value.smoothTotalSpace.setTotal(reply.get().storageBytes.total); - myQueueInfo->value.smoothDurableVersion.setTotal(reply.get().durableVersion); - myQueueInfo->value.smoothLatestVersion.setTotal(reply.get().version); - } - - myQueueInfo->value.busiestReadTags = reply.get().busiestTags; + myQueueInfo->value.update(reply.get(), self->smoothTotalDurableBytes); } else { if (myQueueInfo->value.valid) { TraceEvent("RkStorageServerDidNotRespond", self->id).detail("StorageServer", ssi.id()); @@ -975,6 +952,32 @@ StorageQueueInfo::StorageQueueInfo(UID id, LocalityData locality) lastReply.instanceID = -1; } +void StorageQueueInfo::update(StorageQueuingMetricsReply const& reply, Smoother& smoothTotalDurableBytes) { + valid = true; + auto prevReply = std::move(lastReply); + lastReply = reply; + if (prevReply.instanceID != reply.instanceID) { + smoothDurableBytes.reset(reply.bytesDurable); + verySmoothDurableBytes.reset(reply.bytesDurable); + smoothInputBytes.reset(reply.bytesInput); + smoothFreeSpace.reset(reply.storageBytes.available); + smoothTotalSpace.reset(reply.storageBytes.total); + smoothDurableVersion.reset(reply.durableVersion); + smoothLatestVersion.reset(reply.version); + } else { + smoothTotalDurableBytes.addDelta(reply.bytesDurable - prevReply.bytesDurable); + smoothDurableBytes.setTotal(reply.bytesDurable); + verySmoothDurableBytes.setTotal(reply.bytesDurable); + smoothInputBytes.setTotal(reply.bytesInput); + smoothFreeSpace.setTotal(reply.storageBytes.available); + smoothTotalSpace.setTotal(reply.storageBytes.total); + smoothDurableVersion.setTotal(reply.durableVersion); + smoothLatestVersion.setTotal(reply.version); + } + + busiestReadTags = reply.busiestTags; +} + void StorageQueueInfo::refreshCommitCost(double elapsed) { busiestWriteTags.clear(); TransactionTag busiestTag; diff --git a/fdbserver/Ratekeeper.h b/fdbserver/Ratekeeper.h index 79ecc359d4..b96299b420 100644 --- a/fdbserver/Ratekeeper.h +++ b/fdbserver/Ratekeeper.h @@ -51,7 +51,6 @@ struct StorageQueueInfo { UID id; LocalityData locality; StorageQueuingMetricsReply lastReply; - StorageQueuingMetricsReply prevReply; Smoother smoothDurableBytes, smoothInputBytes, verySmoothDurableBytes; Smoother smoothDurableVersion, smoothLatestVersion; Smoother smoothFreeSpace; @@ -71,6 +70,7 @@ struct StorageQueueInfo { void refreshCommitCost(double elapsed); int64_t getStorageQueueBytes() const { return lastReply.bytesInput - smoothDurableBytes.smoothTotal(); } int64_t getDurabilityLag() const { return smoothLatestVersion.smoothTotal() - smoothDurableVersion.smoothTotal(); } + void update(StorageQueuingMetricsReply const&, Smoother& smoothTotalDurableBytes); }; struct TLogQueueInfo { From a13b40879377239c660e5ab6f956828dc646f005 Mon Sep 17 00:00:00 2001 From: sfc-gh-tclinkenbeard Date: Sat, 12 Mar 2022 14:28:50 -0400 Subject: [PATCH 345/413] Add StorageQueueInfo::addCommitCost method --- fdbserver/Ratekeeper.actor.cpp | 16 ++++++++++------ fdbserver/Ratekeeper.h | 19 ++++++++++--------- 2 files changed, 20 insertions(+), 15 deletions(-) diff --git a/fdbserver/Ratekeeper.actor.cpp b/fdbserver/Ratekeeper.actor.cpp index 3a42268475..4100005c1d 100644 --- a/fdbserver/Ratekeeper.actor.cpp +++ b/fdbserver/Ratekeeper.actor.cpp @@ -444,9 +444,7 @@ void Ratekeeper::updateCommitCostEstimation( if (tagCostIt == costEstimation.end()) continue; for (const auto& [tagName, cost] : tagCostIt->second) { - it->value.tagCostEst[tagName] += cost; - it->value.totalWriteCosts += cost.getCostSum(); - it->value.totalWriteOps += cost.getOpsSum(); + it->value.addCommitCost(tagName, cost); } } } @@ -942,16 +940,22 @@ ACTOR Future ratekeeper(RatekeeperInterface rkInterf, ReferenceSMOOTHING_AMOUNT), + : busiestWriteTagEventHolder(makeReference(id.toString() + "/BusiestWriteTag")), valid(false), + id(id), locality(locality), smoothDurableBytes(SERVER_KNOBS->SMOOTHING_AMOUNT), smoothInputBytes(SERVER_KNOBS->SMOOTHING_AMOUNT), verySmoothDurableBytes(SERVER_KNOBS->SLOW_SMOOTHING_AMOUNT), smoothDurableVersion(SERVER_KNOBS->SMOOTHING_AMOUNT), smoothLatestVersion(SERVER_KNOBS->SMOOTHING_AMOUNT), smoothFreeSpace(SERVER_KNOBS->SMOOTHING_AMOUNT), smoothTotalSpace(SERVER_KNOBS->SMOOTHING_AMOUNT), - limitReason(limitReason_t::unlimited), - busiestWriteTagEventHolder(makeReference(id.toString() + "/BusiestWriteTag")) { + limitReason(limitReason_t::unlimited) { // FIXME: this is a tacky workaround for a potential uninitialized use in trackStorageServerQueueInfo lastReply.instanceID = -1; } +void StorageQueueInfo::addCommitCost(TransactionTagRef tagName, TransactionCommitCostEstimation const& cost) { + tagCostEst[tagName] += cost; + totalWriteCosts += cost.getCostSum(); + totalWriteOps += cost.getOpsSum(); +} + void StorageQueueInfo::update(StorageQueuingMetricsReply const& reply, Smoother& smoothTotalDurableBytes) { valid = true; auto prevReply = std::move(lastReply); diff --git a/fdbserver/Ratekeeper.h b/fdbserver/Ratekeeper.h index b96299b420..1fe0031c15 100644 --- a/fdbserver/Ratekeeper.h +++ b/fdbserver/Ratekeeper.h @@ -46,7 +46,15 @@ enum limitReason_t { limitReason_t_end }; -struct StorageQueueInfo { +class StorageQueueInfo { + uint64_t totalWriteCosts{ 0 }; + int totalWriteOps{ 0 }; + Reference busiestWriteTagEventHolder; + + // refresh periodically + TransactionTagMap tagCostEst; + +public: bool valid; UID id; LocalityData locality; @@ -56,21 +64,14 @@ struct StorageQueueInfo { Smoother smoothFreeSpace; Smoother smoothTotalSpace; limitReason_t limitReason; - std::vector busiestReadTags, busiestWriteTags; - Reference busiestWriteTagEventHolder; - - // refresh periodically - TransactionTagMap tagCostEst; - uint64_t totalWriteCosts = 0; - int totalWriteOps = 0; StorageQueueInfo(UID id, LocalityData locality); - void refreshCommitCost(double elapsed); int64_t getStorageQueueBytes() const { return lastReply.bytesInput - smoothDurableBytes.smoothTotal(); } int64_t getDurabilityLag() const { return smoothLatestVersion.smoothTotal() - smoothDurableVersion.smoothTotal(); } void update(StorageQueuingMetricsReply const&, Smoother& smoothTotalDurableBytes); + void addCommitCost(TransactionTagRef tagName, TransactionCommitCostEstimation const& cost); }; struct TLogQueueInfo { From 26cbe6863d7f37d71bb613e000843abc7dfda7c5 Mon Sep 17 00:00:00 2001 From: Josh Slocum Date: Mon, 14 Mar 2022 11:48:15 -0500 Subject: [PATCH 346/413] Adjusting simulation knobs to reduce change feed memory overhead for large merge cases --- fdbclient/ServerKnobs.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/fdbclient/ServerKnobs.cpp b/fdbclient/ServerKnobs.cpp index bc6f181c79..620b24cd2f 100644 --- a/fdbclient/ServerKnobs.cpp +++ b/fdbclient/ServerKnobs.cpp @@ -290,7 +290,7 @@ void ServerKnobs::initialize(Randomize randomize, ClientKnobs* clientKnobs, IsSi init( SQLITE_CHUNK_SIZE_PAGES_SIM, 1024 ); // 4MB init( SQLITE_READER_THREADS, 64 ); // number of read threads init( SQLITE_WRITE_WINDOW_SECONDS, -1 ); - init( SQLITE_CURSOR_MAX_LIFETIME_BYTES, 1e6 ); if( randomize && BUGGIFY ) SQLITE_CURSOR_MAX_LIFETIME_BYTES = 0; + init( SQLITE_CURSOR_MAX_LIFETIME_BYTES, 1e6 ); if (buggifySmallShards || simulationMediumShards) SQLITE_CURSOR_MAX_LIFETIME_BYTES = MIN_SHARD_BYTES; if( randomize && BUGGIFY ) SQLITE_CURSOR_MAX_LIFETIME_BYTES = 0; init( SQLITE_WRITE_WINDOW_LIMIT, -1 ); if( randomize && BUGGIFY ) { // Choose an window between .01 and 1.01 seconds. @@ -822,7 +822,7 @@ void ServerKnobs::initialize(Randomize randomize, ClientKnobs* clientKnobs, IsSi // Blob granlues init( BG_URL, isSimulated ? "file://fdbblob/" : "" ); // TODO: store in system key space or something, eventually - init( BG_SNAPSHOT_FILE_TARGET_BYTES, 10000000 ); if( buggifySmallShards || (randomize && BUGGIFY) ) { deterministicRandom()->random01() < 0.1 ? BG_SNAPSHOT_FILE_TARGET_BYTES /= 100 : BG_SNAPSHOT_FILE_TARGET_BYTES /= 10; } + init( BG_SNAPSHOT_FILE_TARGET_BYTES, 10000000 ); if( buggifySmallShards ) BG_SNAPSHOT_FILE_TARGET_BYTES = 100000; else if (simulationMediumShards || (randomize && BUGGIFY) ) BG_SNAPSHOT_FILE_TARGET_BYTES = 1000000; init( BG_DELTA_BYTES_BEFORE_COMPACT, BG_SNAPSHOT_FILE_TARGET_BYTES/2 ); init( BG_DELTA_FILE_TARGET_BYTES, BG_DELTA_BYTES_BEFORE_COMPACT/10 ); init( BG_MAX_SPLIT_FANOUT, 10 ); if( randomize && BUGGIFY ) BG_MAX_SPLIT_FANOUT = deterministicRandom()->randomInt(5, 15); From 98eeaac1b7c2595f7f30510436c5b6873517ad41 Mon Sep 17 00:00:00 2001 From: Josh Slocum Date: Mon, 14 Mar 2022 11:57:43 -0500 Subject: [PATCH 347/413] More debugging and fixing a stuck merge cursor --- fdbclient/DatabaseContext.h | 2 + fdbclient/NativeAPI.actor.cpp | 133 ++++++++++++++++++++++++++++++---- 2 files changed, 121 insertions(+), 14 deletions(-) diff --git a/fdbclient/DatabaseContext.h b/fdbclient/DatabaseContext.h index 485617c102..65b47f95f9 100644 --- a/fdbclient/DatabaseContext.h +++ b/fdbclient/DatabaseContext.h @@ -170,6 +170,8 @@ public: struct MutationAndVersionStream { Standalone next; PromiseStream> results; + // TODO REMOVE, for debugging + int idx = -1; bool operator<(MutationAndVersionStream const& rhs) const { return next.version > rhs.next.version; } }; diff --git a/fdbclient/NativeAPI.actor.cpp b/fdbclient/NativeAPI.actor.cpp index 1e3c70a733..e82c767dcd 100644 --- a/fdbclient/NativeAPI.actor.cpp +++ b/fdbclient/NativeAPI.actor.cpp @@ -7479,6 +7479,9 @@ Version ChangeFeedData::getVersion() { return lastReturnedVersion.get(); } +// TODO REMOVE +#define DEBUG_CF_WAIT_VERSION invalidVersion + // This function is essentially bubbling the information about what has been processed from the server through the // change feed client. First it makes sure the server has returned all mutations up through the target version, the // native api has consumed and processed, them, and then the fdb client has consumed all of the mutations. @@ -7486,11 +7489,28 @@ ACTOR Future changeFeedWaitLatest(Reference self, Version // wait on SS to have sent up through version int desired = 0; int waiting = 0; + if (version == DEBUG_CF_WAIT_VERSION) { + fmt::print("WFV {}) waitLatest\n", version); + } std::vector> allAtLeast; for (auto& it : self->storageData) { if (it->version.get() < version) { waiting++; + if (version == DEBUG_CF_WAIT_VERSION) { + fmt::print("WFV {0}) SS {1} {2} < {3}\n", + version, + it->id.toString().substr(0, 4), + it->version.get(), + version); + } if (version > it->desired.get()) { + if (version == DEBUG_CF_WAIT_VERSION) { + fmt::print("WFV {0}) SS {1} desired {2} -> {3}\n", + version, + it->id.toString().substr(0, 4), + it->desired.get(), + version); + } it->desired.set(version); desired++; } @@ -7498,6 +7518,10 @@ ACTOR Future changeFeedWaitLatest(Reference self, Version } } + if (version == DEBUG_CF_WAIT_VERSION) { + fmt::print("WFV {0}) waiting for {1}/{2} ({3} desired)\n", version, waiting, self->storageData.size(), desired); + } + wait(waitForAll(allAtLeast)); // then, wait on ss streams to have processed up through version @@ -7547,7 +7571,9 @@ ACTOR Future changeFeedWhenAtLatest(Reference self, Versio return Void(); } state Future lastReturned = self->lastReturnedVersion.whenAtLeast(version); - + if (version == DEBUG_CF_WAIT_VERSION) { + fmt::print("WFV {0}) notAtLatest={1}\n", version, self->notAtLatest.get()); + } loop { // only allowed to use empty versions if you're caught up Future waitEmptyVersion = (self->notAtLatest.get() == 0) ? changeFeedWaitLatest(self, version) : Never(); @@ -7570,7 +7596,7 @@ Future ChangeFeedData::whenAtLeast(Version version) { return changeFeedWhenAtLatest(Reference::addRef(this), version); } -#define DEBUG_CF_CLIENT_TRACE false +#define DEBUG_CF_CLIENT_TRACE true ACTOR Future partialChangeFeedStream(StorageServerInterface interf, PromiseStream> results, @@ -7585,6 +7611,12 @@ ACTOR Future partialChangeFeedStream(StorageServerInterface interf, state Promise refresh = feedData->refresh; state bool atLatestVersion = false; state Version nextVersion = begin; + // We don't need to force every other partial stream to do an empty if we get an empty, but if we get actual + // mutations back after sending an empty, we may need the other partial streams to get an empty, to advance the + // merge cursor, so we can send the mutations we just got. + // if lastEmpty != invalidVersion, we need to update the desired versions of the other streams BEFORE waiting + // onReady once getting a reply + state Version lastEmpty = invalidVersion; try { loop { if (nextVersion >= end) { @@ -7599,6 +7631,18 @@ ACTOR Future partialChangeFeedStream(StorageServerInterface interf, continue; } + if (DEBUG_CF_CLIENT_TRACE) { + TraceEvent(SevDebug, "TraceChangeFeedClientMergeCursorReply", debugUID) + .detail("SSID", storageData->id) + .detail("AtLatest", atLatestVersion) + .detail("FirstVersion", rep.mutations.front().version) + .detail("LastVersion", rep.mutations.back().version) + .detail("Count", rep.mutations.size()) + .detail("MinStreamVersion", rep.minStreamVersion) + .detail("PopVersion", rep.popVersion) + .detail("RepAtLatest", rep.atLatestVersion); + } + if (rep.mutations.back().version > feedData->maxSeenVersion) { feedData->maxSeenVersion = rep.mutations.back().version; } @@ -7606,12 +7650,27 @@ ACTOR Future partialChangeFeedStream(StorageServerInterface interf, feedData->popVersion = rep.popVersion; } + if (lastEmpty != invalidVersion && !results.isEmpty()) { + for (auto& it : feedData->storageData) { + if (refresh.canBeSet() && lastEmpty > it->desired.get()) { + it->desired.set(lastEmpty); + } + } + lastEmpty = invalidVersion; + } + state int resultLoc = 0; while (resultLoc < rep.mutations.size()) { wait(results.onEmpty()); if (rep.mutations[resultLoc].version >= nextVersion) { results.send(rep.mutations[resultLoc]); + if (DEBUG_CF_CLIENT_TRACE) { + TraceEvent(SevDebug, "TraceChangeFeedClientMergeCursorSend", debugUID) + .detail("Version", rep.mutations[resultLoc].version) + .detail("Size", rep.mutations[resultLoc].mutations.size()); + } + // check refresh.canBeSet so that, if we are killed after calling one of these callbacks, we // just skip to the next wait and get actor_cancelled // FIXME: this is somewhat expensive to do every mutation. @@ -7638,6 +7697,10 @@ ACTOR Future partialChangeFeedStream(StorageServerInterface interf, if (refresh.canBeSet() && rep.minStreamVersion > storageData->version.get()) { storageData->version.set(rep.minStreamVersion); } + if (DEBUG_CF_CLIENT_TRACE) { + TraceEvent(SevDebug, "TraceChangeFeedClientMergeCursorReplyDone", debugUID) + .detail("AtLatestNow", atLatestVersion); + } } when(wait(atLatestVersion && replyStream.isEmpty() && results.isEmpty() ? storageData->version.whenAtLeast(nextVersion) @@ -7646,6 +7709,11 @@ ACTOR Future partialChangeFeedStream(StorageServerInterface interf, empty.version = storageData->version.get(); results.send(empty); nextVersion = storageData->version.get() + 1; + if (DEBUG_CF_CLIENT_TRACE) { + TraceEvent(SevDebug, "TraceChangeFeedClientMergeCursorSendEmpty", debugUID) + .detail("Version", empty.version); + } + lastEmpty = empty.version; } when(wait(atLatestVersion && replyStream.isEmpty() && !results.isEmpty() ? results.onEmpty() : Future(Never()))) {} @@ -7667,18 +7735,30 @@ ACTOR Future mergeChangeFeedStreamInternal(Reference resul std::vector> interfs, std::vector streams, Version* begin, - Version end) { + Version end, + UID mergeCursorUID) { state Promise refresh = results->refresh; // with empty version handling in the partial cursor, all streams will always have a next element with version >= // the minimum version of any stream's next element state std::priority_queue> mutations; + if (DEBUG_CF_CLIENT_TRACE) { + TraceEvent(SevDebug, "TraceChangeFeedClientMergeCursorStart", mergeCursorUID) + .detail("StreamCount", interfs.size()) + .detail("Begin", *begin) + .detail("End", end); + } + // previous version of change feed may have put a mutation in the promise stream and then immediately died. Wait for // that mutation first, so the promise stream always starts empty wait(results->mutations.onEmpty()); wait(delay(0)); ASSERT(results->mutations.isEmpty()); + if (DEBUG_CF_CLIENT_TRACE) { + TraceEvent(SevDebug, "TraceChangeFeedClientMergeCursorGotEmpty", mergeCursorUID); + } + // update lastReturned once the previous mutation has been consumed if (*begin - 1 > results->lastReturnedVersion.get()) { results->lastReturnedVersion.set(*begin - 1); @@ -7698,13 +7778,26 @@ ACTOR Future mergeChangeFeedStreamInternal(Reference resul interfNum = 0; while (interfNum < streamsUsed.size()) { try { + if (DEBUG_CF_CLIENT_TRACE) { + TraceEvent(SevDebug, "TraceChangeFeedClientMergeCursorWait", mergeCursorUID) + .detail("Idx", streamsUsed[interfNum].idx); + } Standalone res = waitNext(streamsUsed[interfNum].results.getFuture()); streamsUsed[interfNum].next = res; mutations.push(streamsUsed[interfNum]); + if (DEBUG_CF_CLIENT_TRACE) { + TraceEvent(SevDebug, "TraceChangeFeedClientMergeCursorGot", mergeCursorUID) + .detail("Idx", streamsUsed[interfNum].idx) + .detail("Version", res.version); + } } catch (Error& e) { if (e.code() != error_code_end_of_stream) { throw e; } + if (DEBUG_CF_CLIENT_TRACE) { + TraceEvent(SevDebug, "TraceChangeFeedClientMergeCursorEOS", mergeCursorUID) + .detail("Idx", streamsUsed[interfNum].idx); + } } interfNum++; } @@ -7742,6 +7835,12 @@ ACTOR Future mergeChangeFeedStreamInternal(Reference resul *begin = nextVersion + 1; + if (DEBUG_CF_CLIENT_TRACE) { + TraceEvent(SevDebug, "TraceChangeFeedClientMergeCursorSending", mergeCursorUID) + .detail("Count", streamsUsed.size()) + .detail("Version", nextVersion); + } + // send mutations at nextVersion to the client if (nextOut.back().mutations.empty()) { ASSERT(results->mutations.isEmpty()); @@ -7781,6 +7880,8 @@ ACTOR Future mergeChangeFeedStream(Reference db, TEST(interfs.size() > 10); // Large change feed merge cursor TEST(interfs.size() > 100); // Very large change feed merge cursor + // TODO REMOVE eventually, useful for debugging for now + state UID mergeCursorUID = UID(); state std::vector debugUIDs; results->streams.clear(); for (auto& it : interfs) { @@ -7798,16 +7899,8 @@ ACTOR Future mergeChangeFeedStream(Reference db, // TODO REMOVE req.debugUID = deterministicRandom()->randomUniqueID(); debugUIDs.push_back(req.debugUID); - - if (DEBUG_CF_CLIENT_TRACE) { - TraceEvent(SevDebug, "TraceChangeFeedClientMergeCursor", req.debugUID) - .detail("FeedID", rangeID) - .detail("MergeRange", KeyRangeRef(interfs.front().second.begin, interfs.back().second.end)) - .detail("PartialRange", it.second) - .detail("Begin", *begin) - .detail("End", end) - .detail("CanReadPopped", true); - } + mergeCursorUID = + UID(mergeCursorUID.first() ^ req.debugUID.first(), mergeCursorUID.second() ^ req.debugUID.second()); results->streams.push_back(it.first.changeFeedStream.getReplyStream(req)); } @@ -7828,6 +7921,18 @@ ACTOR Future mergeChangeFeedStream(Reference db, refresh.send(Void()); for (int i = 0; i < interfs.size(); i++) { + if (DEBUG_CF_CLIENT_TRACE) { + TraceEvent(SevDebug, "TraceChangeFeedClientMergeCursorInit", debugUIDs[i]) + .detail("CursorDebugUID", mergeCursorUID) + .detail("Idx", i) + .detail("FeedID", rangeID) + .detail("MergeRange", KeyRangeRef(interfs.front().second.begin, interfs.back().second.end)) + .detail("PartialRange", interfs[i].second) + .detail("Begin", *begin) + .detail("End", end) + .detail("CanReadPopped", canReadPopped); + } + streams[i].idx = i; onErrors[i] = results->streams[i].onError(); fetchers[i] = partialChangeFeedStream(interfs[i].first, streams[i].results, @@ -7839,7 +7944,7 @@ ACTOR Future mergeChangeFeedStream(Reference db, debugUIDs[i]); } - wait(onCFErrors(onErrors) || mergeChangeFeedStreamInternal(results, interfs, streams, begin, end)); + wait(onCFErrors(onErrors) || mergeChangeFeedStreamInternal(results, interfs, streams, begin, end, mergeCursorUID)); return Void(); } From 8d88b7ca41c50ead42f9333b30c5050f971b1d4b Mon Sep 17 00:00:00 2001 From: Josh Slocum Date: Mon, 14 Mar 2022 14:39:00 -0500 Subject: [PATCH 348/413] making range assigner boundary changes fully synchronous to ensure no races with things reading assignments --- fdbclient/NativeAPI.actor.cpp | 2 +- fdbserver/BlobManager.actor.cpp | 95 ++++++++++++++++++++------------- 2 files changed, 60 insertions(+), 37 deletions(-) diff --git a/fdbclient/NativeAPI.actor.cpp b/fdbclient/NativeAPI.actor.cpp index e82c767dcd..07d983cd64 100644 --- a/fdbclient/NativeAPI.actor.cpp +++ b/fdbclient/NativeAPI.actor.cpp @@ -7596,7 +7596,7 @@ Future ChangeFeedData::whenAtLeast(Version version) { return changeFeedWhenAtLatest(Reference::addRef(this), version); } -#define DEBUG_CF_CLIENT_TRACE true +#define DEBUG_CF_CLIENT_TRACE false ACTOR Future partialChangeFeedStream(StorageServerInterface interf, PromiseStream> results, diff --git a/fdbserver/BlobManager.actor.cpp b/fdbserver/BlobManager.actor.cpp index 9a2e1cdb69..712e069dee 100644 --- a/fdbserver/BlobManager.actor.cpp +++ b/fdbserver/BlobManager.actor.cpp @@ -342,6 +342,8 @@ ACTOR Future pickWorkerForAssign(Reference bmData) { } bmData->restartRecruiting.trigger(); wait(bmData->recruitingStream.onChange() || bmData->foundBlobWorkers.getFuture()); + // FIXME: may want to have some buffer here so zero-worker recruiting case doesn't assign every single pending + // range to the first worker recruited } int minGranulesAssigned = INT_MAX; @@ -374,8 +376,23 @@ ACTOR Future pickWorkerForAssign(Reference bmData) { ACTOR Future doRangeAssignment(Reference bmData, RangeAssignment assignment, - UID workerID, + Optional workerID, int64_t seqNo) { + // WorkerId is set, except in case of assigning to any worker. Then we pick the worker to assign to in here + + // inject delay into range assignments + if (BUGGIFY_WITH_PROB(0.05)) { + wait(delay(deterministicRandom()->random01())); + } + + if (!workerID.present()) { + ASSERT(assignment.isAssign); + UID _workerId = wait(pickWorkerForAssign(bmData)); + if (BM_DEBUG) { + fmt::print("Chose BW {0} for seqno {1} in BM {2}\n", _workerId.toString(), seqNo, bmData->epoch); + } + workerID = _workerId; + } if (BM_DEBUG) { fmt::print("BM {0} {1} range [{2} - {3}) @ ({4}, {5}) to {6}\n", @@ -385,7 +402,7 @@ ACTOR Future doRangeAssignment(Reference bmData, assignment.keyRange.end.printable(), bmData->epoch, seqNo, - workerID.toString()); + workerID.get().toString()); } try { @@ -401,10 +418,10 @@ ACTOR Future doRangeAssignment(Reference bmData, req.type = assignment.assign.get().type; // if that worker isn't alive anymore, add the range back into the stream - if (bmData->workersById.count(workerID) == 0) { + if (bmData->workersById.count(workerID.get()) == 0) { throw no_more_servers(); } - wait(bmData->workersById[workerID].assignBlobRangeRequest.getReply(req)); + wait(bmData->workersById[workerID.get()].assignBlobRangeRequest.getReply(req)); } else { ASSERT(!assignment.assign.present()); ASSERT(assignment.revoke.present()); @@ -417,8 +434,8 @@ ACTOR Future doRangeAssignment(Reference bmData, req.dispose = assignment.revoke.get().dispose; // if that worker isn't alive anymore, this is a noop - if (bmData->workersById.count(workerID)) { - wait(bmData->workersById[workerID].revokeBlobRangeRequest.getReply(req)); + if (bmData->workersById.count(workerID.get())) { + wait(bmData->workersById[workerID.get()].revokeBlobRangeRequest.getReply(req)); } else { return Void(); } @@ -443,7 +460,7 @@ ACTOR Future doRangeAssignment(Reference bmData, bmData->epoch, assignment.keyRange.begin.printable(), assignment.keyRange.end.printable(), - workerID.toString()); + workerID.get().toString()); } if (bmData->doLockCheck.canBeSet()) { bmData->doLockCheck.send(Void()); @@ -472,7 +489,7 @@ ACTOR Future doRangeAssignment(Reference bmData, e.name(), assignment.keyRange.begin.printable(), assignment.keyRange.end.printable(), - workerID.toString()); + workerID.get().toString()); } // re-send revoke to queue to handle range being un-assigned from that worker before the new one @@ -518,10 +535,7 @@ ACTOR Future doRangeAssignment(Reference bmData, ACTOR Future rangeAssigner(Reference bmData) { loop { - // inject delay into range assignments - if (BUGGIFY_WITH_PROB(0.05)) { - wait(delay(deterministicRandom()->random01())); - } + state RangeAssignment assignment = waitNext(bmData->rangesToAssign.getFuture()); state int64_t seqNo = bmData->seqNo; bmData->seqNo++; @@ -568,22 +582,22 @@ ACTOR Future rangeAssigner(Reference bmData) { bmData->id.toString()); } workerId = assignment.worker.get(); + + bmData->workerAssignments.insert(assignment.keyRange, workerId); + bmData->assignsInProgress.insert(assignment.keyRange, + doRangeAssignment(bmData, assignment, workerId, seqNo)); } else { - UID _workerId = wait(pickWorkerForAssign(bmData)); - if (BM_DEBUG) { - fmt::print("Chose BW {0} for seqno {1} in BM {2}\n", _workerId.toString(), seqNo, bmData->epoch); - } - workerId = _workerId; + // Ensure the key boundaries are updated before we pick a worker + bmData->workerAssignments.insert(assignment.keyRange, UID()); + bmData->assignsInProgress.insert(assignment.keyRange, + doRangeAssignment(bmData, assignment, Optional(), seqNo)); } - bmData->workerAssignments.insert(assignment.keyRange, workerId); // If we know about the worker and this is not a continue, then this is a new range for the worker if (bmData->workerStats.count(workerId) && assignment.assign.get().type != AssignRequestType::Continue) { bmData->workerStats[workerId].numGranulesAssigned += 1; } - bmData->assignsInProgress.insert(assignment.keyRange, - doRangeAssignment(bmData, assignment, workerId, seqNo)); } else { if (assignment.worker.present()) { // revoke this specific range from this specific worker. Either part of recovery or failing a worker @@ -778,7 +792,7 @@ ACTOR Future monitorClientRanges(Reference bmData) { ra.assign = RangeAssignmentData(); // type=normal bmData->rangesToAssign.send(ra); } - wait(bmData->rangesToAssign.onEmpty()); + ASSERT(bmData->rangesToAssign.isEmpty()); } lastChangeKeyValue = @@ -956,6 +970,23 @@ ACTOR Future maybeSplitRange(Reference bmData, existingState.size() + 2)); // +2 because this is boundaries and existingState was granules, and to ensure it doesn't set more ASSERT(!existingBoundaries.more); + // TODO remove debugging printf + if (existingBoundaries.size() != existingState.size() + 1) { + printf("DBG: EB=%d, ES=%d for [%s - %s)\n", + existingBoundaries.size(), + existingState.size(), + granuleRange.begin.printable().c_str(), + granuleRange.end.printable().c_str()); + printf("Boundaries:\n"); + for (auto& it : existingBoundaries) { + printf(" %s\n", it.key.removePrefix(blobGranuleMappingKeys.begin).printable().c_str()); + } + printf("State:\n"); + for (auto& it : existingState) { + std::pair k = decodeBlobGranuleSplitKey(it.key); + printf(" %s\n", k.second.toString().substr(0, 6).c_str()); + } + } ASSERT(existingBoundaries.size() == existingState.size() + 1); newRanges.clear(); newRanges.arena().dependsOn(existingBoundaries.arena()); @@ -1100,18 +1131,9 @@ ACTOR Future maybeSplitRange(Reference bmData, bmData->rangesToAssign.send(raAssignSplit); } - if (BM_DEBUG) { - fmt::print( - "Splitting range [{0} - {1}) into {2} granules @ {3} sent assignments, waiting for them to be processed:\n", - granuleRange.begin.printable(), - granuleRange.end.printable(), - newRanges.size() - 1, - latestVersion); - } - // Ensure the new assignments actually got processed and the split boundaries are reflected in the granule mapping // before returning. This prevents a race with a subsequent split evaluation - wait(bmData->rangesToAssign.onEmpty()); + ASSERT(bmData->rangesToAssign.isEmpty()); if (BM_DEBUG) { fmt::print("Splitting range [{0} - {1}) into {2} granules @ {3} got assignments processed\n", @@ -1221,6 +1243,8 @@ ACTOR Future killBlobWorker(Reference bmData, BlobWorkerI bmData->rangesToAssign.send(raAssign); } + ASSERT(bmData->rangesToAssign.isEmpty()); + // Send halt to blob worker, with no expectation of hearing back if (BM_DEBUG) { fmt::print("Sending halt to BW {}\n", bwId.toString()); @@ -1229,10 +1253,7 @@ ACTOR Future killBlobWorker(Reference bmData, BlobWorkerI // wait for blob worker to be removed from DB and in-memory mapping to have reassigned all shards from this worker // before removing it from deadWorkers, to avoid a race with checkBlobWorkerList - wait(deregister && bmData->rangesToAssign.onEmpty()); - // delay(0) after onEmpty to yield back to the range assigner on the final pop to ensure it gets processed before - // deadWorkers.erase - wait(delay(0)); + wait(deregister); // restart recruiting to replace the dead blob worker bmData->restartRecruiting.trigger(); @@ -2686,9 +2707,12 @@ ACTOR Future blobManager(BlobManagerInterface bmInterf, self->epoch = epoch; + // start rangeAssigner first since other actors can send messages to it + self->addActor.send(rangeAssigner(self)); // although we start the recruiter, we wait until existing workers are ack'd auto recruitBlobWorker = IAsyncListener>::create( dbInfo, [](auto const& info) { return info.clusterInterface.recruitBlobWorker; }); + self->addActor.send(blobWorkerRecruiter(self, recruitBlobWorker)); // we need to recover the old blob manager's state (e.g. granule assignments) before @@ -2697,7 +2721,6 @@ ACTOR Future blobManager(BlobManagerInterface bmInterf, self->addActor.send(doLockChecks(self)); self->addActor.send(monitorClientRanges(self)); - self->addActor.send(rangeAssigner(self)); self->addActor.send(monitorPruneKeys(self)); if (BUGGIFY) { From e7aa534cae1a5d69f607020ab74ef8d837155c57 Mon Sep 17 00:00:00 2001 From: Josh Slocum Date: Mon, 14 Mar 2022 15:04:43 -0500 Subject: [PATCH 349/413] Forgot to update worker mapping after picking worker --- fdbserver/BlobManager.actor.cpp | 3 +++ 1 file changed, 3 insertions(+) diff --git a/fdbserver/BlobManager.actor.cpp b/fdbserver/BlobManager.actor.cpp index 712e069dee..fb1293e3dc 100644 --- a/fdbserver/BlobManager.actor.cpp +++ b/fdbserver/BlobManager.actor.cpp @@ -392,6 +392,9 @@ ACTOR Future doRangeAssignment(Reference bmData, fmt::print("Chose BW {0} for seqno {1} in BM {2}\n", _workerId.toString(), seqNo, bmData->epoch); } workerID = _workerId; + // We don't have to check for races with an overlapping assignment because it would insert over us in the actor + // map, cancelling this actor before it got here + bmData->workerAssignments.insert(assignment.keyRange, workerID.get()); } if (BM_DEBUG) { From 3e278e8580b0acf131ee30f1bc97c57af30c1cd6 Mon Sep 17 00:00:00 2001 From: Josh Slocum Date: Mon, 14 Mar 2022 15:53:09 -0500 Subject: [PATCH 350/413] asserts for assign is empty is false if BM dies and rangeAssigner actor is dead --- fdbserver/BlobManager.actor.cpp | 7 ------- 1 file changed, 7 deletions(-) diff --git a/fdbserver/BlobManager.actor.cpp b/fdbserver/BlobManager.actor.cpp index fb1293e3dc..916c9b25a3 100644 --- a/fdbserver/BlobManager.actor.cpp +++ b/fdbserver/BlobManager.actor.cpp @@ -795,7 +795,6 @@ ACTOR Future monitorClientRanges(Reference bmData) { ra.assign = RangeAssignmentData(); // type=normal bmData->rangesToAssign.send(ra); } - ASSERT(bmData->rangesToAssign.isEmpty()); } lastChangeKeyValue = @@ -1134,10 +1133,6 @@ ACTOR Future maybeSplitRange(Reference bmData, bmData->rangesToAssign.send(raAssignSplit); } - // Ensure the new assignments actually got processed and the split boundaries are reflected in the granule mapping - // before returning. This prevents a race with a subsequent split evaluation - ASSERT(bmData->rangesToAssign.isEmpty()); - if (BM_DEBUG) { fmt::print("Splitting range [{0} - {1}) into {2} granules @ {3} got assignments processed\n", granuleRange.begin.printable(), @@ -1246,8 +1241,6 @@ ACTOR Future killBlobWorker(Reference bmData, BlobWorkerI bmData->rangesToAssign.send(raAssign); } - ASSERT(bmData->rangesToAssign.isEmpty()); - // Send halt to blob worker, with no expectation of hearing back if (BM_DEBUG) { fmt::print("Sending halt to BW {}\n", bwId.toString()); From 1c14d4674dd6223dfd622c894b2c4852592f0fae Mon Sep 17 00:00:00 2001 From: Josh Slocum Date: Mon, 14 Mar 2022 16:11:17 -0500 Subject: [PATCH 351/413] Fixing more BM races with granule splitting --- fdbserver/BlobManager.actor.cpp | 36 +++++++-------------------------- 1 file changed, 7 insertions(+), 29 deletions(-) diff --git a/fdbserver/BlobManager.actor.cpp b/fdbserver/BlobManager.actor.cpp index 916c9b25a3..7943f948d8 100644 --- a/fdbserver/BlobManager.actor.cpp +++ b/fdbserver/BlobManager.actor.cpp @@ -865,7 +865,6 @@ ACTOR Future maybeSplitRange(Reference bmData, bool writeHot) { state Reference tr = makeReference(bmData->db); state Standalone> newRanges; - state int64_t newLockSeqno = -1; // first get ranges to split Standalone> _newRanges = wait(splitRange(bmData, granuleRange, writeHot)); @@ -1015,7 +1014,6 @@ ACTOR Future maybeSplitRange(Reference bmData, ASSERT(lockValue.present()); std::tuple prevGranuleLock = decodeBlobGranuleLockValue(lockValue.get()); int64_t ownerEpoch = std::get<0>(prevGranuleLock); - int64_t ownerSeqno = std::get<1>(prevGranuleLock); if (ownerEpoch > bmData->epoch) { if (BM_DEBUG) { @@ -1033,33 +1031,13 @@ ACTOR Future maybeSplitRange(Reference bmData, return Void(); } - if (newLockSeqno == -1) { - newLockSeqno = bmData->seqNo; - bmData->seqNo++; - if (!(bmData->epoch > ownerEpoch || (bmData->epoch == ownerEpoch && newLockSeqno > ownerSeqno))) { - fmt::print("BM seqno for granule [{0} - {1}) out of order for lock! manager: ({2}, {3}), owner: " - "({4}, {5}})\n", - granuleRange.begin.printable(), - granuleRange.end.printable(), - bmData->epoch, - newLockSeqno, - ownerEpoch, - ownerSeqno); - } - ASSERT(bmData->epoch > ownerEpoch || (bmData->epoch == ownerEpoch && newLockSeqno > ownerSeqno)); - } else if (bmData->epoch == ownerEpoch && newLockSeqno < ownerSeqno) { - // we retried, and between retries we reassigned this range elsewhere. Cancel this split - TEST(true); // BM maybe split cancelled by subsequent move - if (BM_DEBUG) { - fmt::print("Splitting range [{0} - {1}) cancelled by move elsewhere!\n", - granuleRange.begin.printable(), - granuleRange.end.printable()); - } - return Void(); - } - - // acquire granule lock so nobody else can make changes to this granule. - tr->set(lockKey, blobGranuleLockValueFor(bmData->epoch, newLockSeqno, std::get<2>(prevGranuleLock))); + // Set lock to max value for this manager, so other reassignments can't race with this transaction + // and existing owner can't modify it further. + // FIXME: Implementing merging may require us to make lock go backwards if we later merge other granules + // back to this same range, but I think that's fine + tr->set(lockKey, + blobGranuleLockValueFor( + bmData->epoch, std::numeric_limits::max(), std::get<2>(prevGranuleLock))); // set up splits in granule mapping, but point each part to the old owner (until they get reassigned) state int i; From 8c35ceb68462fa58bd4e7e74a7d1649a1154dc08 Mon Sep 17 00:00:00 2001 From: Josh Slocum Date: Mon, 14 Mar 2022 17:38:31 -0500 Subject: [PATCH 352/413] switching bg file keys to sort on version then type, and using that to fix race where 2 blob workers try to split the same granule at different versions --- fdbclient/BlobWorkerInterface.h | 9 +-- fdbclient/SystemData.cpp | 12 ++-- fdbclient/SystemData.h | 4 +- fdbserver/BlobGranuleServerCommon.actor.cpp | 2 +- fdbserver/BlobManager.actor.cpp | 68 +++++++++++++-------- fdbserver/BlobWorker.actor.cpp | 13 ++-- 6 files changed, 64 insertions(+), 44 deletions(-) diff --git a/fdbclient/BlobWorkerInterface.h b/fdbclient/BlobWorkerInterface.h index 29318d6a1a..97d97e547a 100644 --- a/fdbclient/BlobWorkerInterface.h +++ b/fdbclient/BlobWorkerInterface.h @@ -152,7 +152,6 @@ struct GranuleStatusReply : public ReplyPromiseStreamReply { int64_t seqno; UID granuleID; Version startVersion; - Version latestVersion; GranuleStatusReply() {} explicit GranuleStatusReply(KeyRange range, @@ -161,10 +160,9 @@ struct GranuleStatusReply : public ReplyPromiseStreamReply { int64_t epoch, int64_t seqno, UID granuleID, - Version startVersion, - Version latestVersion) + Version startVersion) : granuleRange(range), doSplit(doSplit), writeHotSplit(writeHotSplit), epoch(epoch), seqno(seqno), - granuleID(granuleID), startVersion(startVersion), latestVersion(latestVersion) {} + granuleID(granuleID), startVersion(startVersion) {} int expectedSize() const { return sizeof(GranuleStatusReply) + granuleRange.expectedSize(); } @@ -179,8 +177,7 @@ struct GranuleStatusReply : public ReplyPromiseStreamReply { epoch, seqno, granuleID, - startVersion, - latestVersion); + startVersion); } }; diff --git a/fdbclient/SystemData.cpp b/fdbclient/SystemData.cpp index 2ed6872d57..21aea18d11 100644 --- a/fdbclient/SystemData.cpp +++ b/fdbclient/SystemData.cpp @@ -1133,26 +1133,26 @@ const KeyRef blobGranulePruneChangeKey = LiteralStringRef("\xff\x02/bgpChange"); const uint8_t BG_FILE_TYPE_DELTA = 'D'; const uint8_t BG_FILE_TYPE_SNAPSHOT = 'S'; -const Key blobGranuleFileKeyFor(UID granuleID, uint8_t fileType, Version fileVersion) { +const Key blobGranuleFileKeyFor(UID granuleID, Version fileVersion, uint8_t fileType) { ASSERT(fileType == 'D' || fileType == 'S'); BinaryWriter wr(AssumeVersion(ProtocolVersion::withBlobGranule())); wr.serializeBytes(blobGranuleFileKeys.begin); wr << granuleID; - wr << fileType; wr << bigEndian64(fileVersion); + wr << fileType; return wr.toValue(); } -std::tuple decodeBlobGranuleFileKey(KeyRef const& key) { +std::tuple decodeBlobGranuleFileKey(KeyRef const& key) { UID granuleID; - uint8_t fileType; Version fileVersion; + uint8_t fileType; BinaryReader reader(key.removePrefix(blobGranuleFileKeys.begin), AssumeVersion(ProtocolVersion::withBlobGranule())); reader >> granuleID; - reader >> fileType; reader >> fileVersion; + reader >> fileType; ASSERT(fileType == 'D' || fileType == 'S'); - return std::tuple(granuleID, fileType, bigEndian64(fileVersion)); + return std::tuple(granuleID, bigEndian64(fileVersion), fileType); } const KeyRange blobGranuleFileKeyRangeFor(UID granuleID) { diff --git a/fdbclient/SystemData.h b/fdbclient/SystemData.h index ca21eca61f..0facf79a0a 100644 --- a/fdbclient/SystemData.h +++ b/fdbclient/SystemData.h @@ -561,8 +561,8 @@ extern const KeyRangeRef blobGranulePruneKeys; extern const KeyRangeRef blobGranuleVersionKeys; extern const KeyRef blobGranulePruneChangeKey; -const Key blobGranuleFileKeyFor(UID granuleID, uint8_t fileType, Version fileVersion); -std::tuple decodeBlobGranuleFileKey(KeyRef const& key); +const Key blobGranuleFileKeyFor(UID granuleID, Version fileVersion, uint8_t fileType); +std::tuple decodeBlobGranuleFileKey(KeyRef const& key); const KeyRange blobGranuleFileKeyRangeFor(UID granuleID); const Value blobGranuleFileValueFor(StringRef const& filename, int64_t offset, int64_t length); diff --git a/fdbserver/BlobGranuleServerCommon.actor.cpp b/fdbserver/BlobGranuleServerCommon.actor.cpp index 94192bb8d7..c47ed13199 100644 --- a/fdbserver/BlobGranuleServerCommon.actor.cpp +++ b/fdbserver/BlobGranuleServerCommon.actor.cpp @@ -59,7 +59,7 @@ ACTOR Future readGranuleFiles(Transaction* tr, Key* startKey, Key endKey, int64_t offset; int64_t length; - std::tie(gid, fileType, version) = decodeBlobGranuleFileKey(it.key); + std::tie(gid, version, fileType) = decodeBlobGranuleFileKey(it.key); ASSERT(gid == granuleID); std::tie(filename, offset, length) = decodeBlobGranuleFileValue(it.value); diff --git a/fdbserver/BlobManager.actor.cpp b/fdbserver/BlobManager.actor.cpp index 7943f948d8..5568ede056 100644 --- a/fdbserver/BlobManager.actor.cpp +++ b/fdbserver/BlobManager.actor.cpp @@ -47,7 +47,7 @@ * The Blob Manager is responsible for managing range granules, and recruiting and monitoring Blob Workers. */ -#define BM_DEBUG false +#define BM_DEBUG true void handleClientBlobRange(KeyRangeMap* knownBlobRanges, Arena& ar, @@ -204,12 +204,11 @@ struct BlobWorkerStats { struct SplitEvaluation { int64_t epoch; int64_t seqno; - Version version; Future inProgress; - SplitEvaluation() : epoch(0), seqno(0), version(invalidVersion) {} - SplitEvaluation(int64_t epoch, int64_t seqno, int64_t version, Future inProgress) - : epoch(epoch), seqno(seqno), version(version), inProgress(inProgress) {} + SplitEvaluation() : epoch(0), seqno(0) {} + SplitEvaluation(int64_t epoch, int64_t seqno, Future inProgress) + : epoch(epoch), seqno(seqno), inProgress(inProgress) {} }; struct BlobManagerData : NonCopyable, ReferenceCounted { @@ -386,7 +385,7 @@ ACTOR Future doRangeAssignment(Reference bmData, } if (!workerID.present()) { - ASSERT(assignment.isAssign); + ASSERT(assignment.isAssign && assignment.assign.get().type != AssignRequestType::Continue); UID _workerId = wait(pickWorkerForAssign(bmData)); if (BM_DEBUG) { fmt::print("Chose BW {0} for seqno {1} in BM {2}\n", _workerId.toString(), seqNo, bmData->epoch); @@ -395,6 +394,10 @@ ACTOR Future doRangeAssignment(Reference bmData, // We don't have to check for races with an overlapping assignment because it would insert over us in the actor // map, cancelling this actor before it got here bmData->workerAssignments.insert(assignment.keyRange, workerID.get()); + + if (bmData->workerStats.count(workerID.get())) { + bmData->workerStats[workerID.get()].numGranulesAssigned += 1; + } } if (BM_DEBUG) { @@ -589,6 +592,11 @@ ACTOR Future rangeAssigner(Reference bmData) { bmData->workerAssignments.insert(assignment.keyRange, workerId); bmData->assignsInProgress.insert(assignment.keyRange, doRangeAssignment(bmData, assignment, workerId, seqNo)); + // If we know about the worker and this is not a continue, then this is a new range for the worker + if (bmData->workerStats.count(workerId) && + assignment.assign.get().type != AssignRequestType::Continue) { + bmData->workerStats[workerId].numGranulesAssigned += 1; + } } else { // Ensure the key boundaries are updated before we pick a worker bmData->workerAssignments.insert(assignment.keyRange, UID()); @@ -596,11 +604,6 @@ ACTOR Future rangeAssigner(Reference bmData) { doRangeAssignment(bmData, assignment, Optional(), seqNo)); } - // If we know about the worker and this is not a continue, then this is a new range for the worker - if (bmData->workerStats.count(workerId) && assignment.assign.get().type != AssignRequestType::Continue) { - bmData->workerStats[workerId].numGranulesAssigned += 1; - } - } else { if (assignment.worker.present()) { // revoke this specific range from this specific worker. Either part of recovery or failing a worker @@ -861,7 +864,6 @@ ACTOR Future maybeSplitRange(Reference bmData, KeyRange granuleRange, UID granuleID, Version granuleStartVersion, - Version latestVersion, bool writeHot) { state Reference tr = makeReference(bmData->db); state Standalone> newRanges; @@ -925,11 +927,10 @@ ACTOR Future maybeSplitRange(Reference bmData, } if (BM_DEBUG) { - fmt::print("Splitting range [{0} - {1}) into {2} granules @ {3}:\n", + fmt::print("Splitting range [{0} - {1}) into {2} granules:\n", granuleRange.begin.printable(), granuleRange.end.printable(), - newRanges.size() - 1, - latestVersion); + newRanges.size() - 1); for (int i = 0; i < newRanges.size(); i++) { fmt::print(" {}:{}\n", (i < newGranuleIDs.size() ? newGranuleIDs[i] : UID()).toString().substr(0, 6).c_str(), @@ -937,6 +938,8 @@ ACTOR Future maybeSplitRange(Reference bmData, } } + state Version splitVersion; + // Need to split range. Persist intent to split and split metadata to DB BEFORE sending split assignments to blob // workers, so that nothing is lost on blob manager recovery loop { @@ -1018,7 +1021,7 @@ ACTOR Future maybeSplitRange(Reference bmData, if (ownerEpoch > bmData->epoch) { if (BM_DEBUG) { fmt::print("BM {0} found a higher epoch {1} than {2} for granule lock of [{3} - {4})\n", - bmData->id.toString(), + bmData->epoch, ownerEpoch, bmData->epoch, granuleRange.begin.printable(), @@ -1039,6 +1042,23 @@ ACTOR Future maybeSplitRange(Reference bmData, blobGranuleLockValueFor( bmData->epoch, std::numeric_limits::max(), std::get<2>(prevGranuleLock))); + // get last delta file version written, to make that the split version + RangeResult lastDeltaFile = + wait(tr->getRange(blobGranuleFileKeyRangeFor(granuleID), 1, Snapshot::False, Reverse::True)); + ASSERT(lastDeltaFile.size() == 1); + std::tuple k = decodeBlobGranuleFileKey(lastDeltaFile[0].key); + ASSERT(std::get<0>(k) == granuleID); + ASSERT(std::get<2>(k) == 'D'); + splitVersion = std::get<1>(k); + + if (BM_DEBUG) { + fmt::print("BM {0} found version {1} for splitting [{2} - {3})\n", + bmData->epoch, + splitVersion, + granuleRange.begin.printable(), + granuleRange.end.printable()); + } + // set up splits in granule mapping, but point each part to the old owner (until they get reassigned) state int i; for (i = 0; i < newRanges.size() - 1; i++) { @@ -1048,7 +1068,7 @@ ACTOR Future maybeSplitRange(Reference bmData, blobGranuleSplitValueFor(BlobGranuleSplitState::Initialized), MutationRef::SetVersionstampedValue); - Key historyKey = blobGranuleHistoryKeyFor(KeyRangeRef(newRanges[i], newRanges[i + 1]), latestVersion); + Key historyKey = blobGranuleHistoryKeyFor(KeyRangeRef(newRanges[i], newRanges[i + 1]), splitVersion); Standalone historyValue; historyValue.granuleID = newGranuleIDs[i]; @@ -1090,7 +1110,7 @@ ACTOR Future maybeSplitRange(Reference bmData, granuleRange.begin.printable(), granuleRange.end.printable(), newRanges.size() - 1, - latestVersion); + splitVersion); } // transaction committed, send range assignments @@ -1116,7 +1136,7 @@ ACTOR Future maybeSplitRange(Reference bmData, granuleRange.begin.printable(), granuleRange.end.printable(), newRanges.size() - 1, - latestVersion); + splitVersion); } return Void(); @@ -1306,7 +1326,6 @@ ACTOR Future monitorBlobWorkerStatus(Reference bmData, Bl auto lastSplitEval = bmData->splitEvaluations.rangeContaining(rep.granuleRange.begin); if (rep.granuleRange.begin == lastSplitEval.begin() && rep.granuleRange.end == lastSplitEval.end() && rep.epoch == lastSplitEval.cvalue().epoch && rep.seqno == lastSplitEval.cvalue().seqno) { - ASSERT(lastSplitEval.cvalue().version == rep.latestVersion); if (BM_DEBUG) { fmt::print("Manager {0} received repeat status for the same granule [{1} - {2}), ignoring.\n", bmData->epoch, @@ -1349,10 +1368,9 @@ ACTOR Future monitorBlobWorkerStatus(Reference bmData, Bl rep.granuleRange, rep.granuleID, rep.startVersion, - rep.latestVersion, rep.writeHotSplit); - bmData->splitEvaluations.insert( - rep.granuleRange, SplitEvaluation(rep.epoch, rep.seqno, rep.latestVersion, doSplitEval)); + bmData->splitEvaluations.insert(rep.granuleRange, + SplitEvaluation(rep.epoch, rep.seqno, doSplitEval)); } } } @@ -2179,7 +2197,7 @@ ACTOR Future partiallyDeleteGranule(Reference self, UID g if (latestSnapshotVersion != invalidVersion) { std::string fname = files.snapshotFiles[idx].filename; deletions.emplace_back(self->bstore->deleteFile(fname)); - deletedFileKeys.emplace_back(blobGranuleFileKeyFor(granuleId, 'S', files.snapshotFiles[idx].version)); + deletedFileKeys.emplace_back(blobGranuleFileKeyFor(granuleId, files.snapshotFiles[idx].version, 'S')); filesToDelete.emplace_back(fname); } else if (files.snapshotFiles[idx].version <= pruneVersion) { // otherwise if this is the FIRST snapshot file with version < pruneVersion, @@ -2202,7 +2220,7 @@ ACTOR Future partiallyDeleteGranule(Reference self, UID g // == should also be deleted because the last delta file before a snapshot would have the same version std::string fname = deltaFile.filename; deletions.emplace_back(self->bstore->deleteFile(fname)); - deletedFileKeys.emplace_back(blobGranuleFileKeyFor(granuleId, 'D', deltaFile.version)); + deletedFileKeys.emplace_back(blobGranuleFileKeyFor(granuleId, deltaFile.version, 'D')); filesToDelete.emplace_back(fname); } diff --git a/fdbserver/BlobWorker.actor.cpp b/fdbserver/BlobWorker.actor.cpp index 066f311163..a87d722d16 100644 --- a/fdbserver/BlobWorker.actor.cpp +++ b/fdbserver/BlobWorker.actor.cpp @@ -506,7 +506,7 @@ ACTOR Future writeDeltaFile(Reference bwData, wait(readAndCheckGranuleLock(tr, keyRange, epoch, seqno)); numIterations++; - Key dfKey = blobGranuleFileKeyFor(granuleID, 'D', currentDeltaVersion); + Key dfKey = blobGranuleFileKeyFor(granuleID, currentDeltaVersion, 'D'); Value dfValue = blobGranuleFileValueFor(fname, 0, serializedSize); tr->set(dfKey, dfValue); @@ -643,7 +643,7 @@ ACTOR Future writeSnapshot(Reference bwData, try { wait(readAndCheckGranuleLock(tr, keyRange, epoch, seqno)); numIterations++; - Key snapshotFileKey = blobGranuleFileKeyFor(granuleID, 'S', version); + Key snapshotFileKey = blobGranuleFileKeyFor(granuleID, version, 'S'); Key snapshotFileValue = blobGranuleFileValueFor(fname, 0, serializedSize); tr->set(snapshotFileKey, snapshotFileValue); // create granule history at version if this is a new granule with the initial dump from FDB @@ -904,8 +904,7 @@ ACTOR Future checkSplitAndReSnapshot(Reference bw statusEpoch, statusSeqno, granuleID, - metadata->initialSnapshotVersion, - reSnapshotVersion)); + metadata->initialSnapshotVersion)); break; } catch (Error& e) { if (e.code() == error_code_operation_cancelled) { @@ -1900,6 +1899,12 @@ ACTOR Future blobGranuleLoadHistory(Reference bwData, try { Optional v = wait(tr.get(blobGranuleHistoryKeyFor( curHistory.value.parentGranules[0].first, curHistory.value.parentGranules[0].second))); + if (!v.present()) { + printf("No granule history present for [%s - %s) @ %lld!!\n", + curHistory.value.parentGranules[0].first.begin.printable().c_str(), + curHistory.value.parentGranules[0].first.end.printable().c_str(), + curHistory.value.parentGranules[0].first); + } ASSERT(v.present()); next = GranuleHistory(curHistory.value.parentGranules[0].first, curHistory.value.parentGranules[0].second, From 9d9b70abd4fc1085bea3f378bfffc320da2b83ed Mon Sep 17 00:00:00 2001 From: Josh Slocum Date: Mon, 14 Mar 2022 18:56:36 -0500 Subject: [PATCH 353/413] Fixing another BM recovery and granule splitting race --- fdbserver/BlobManager.actor.cpp | 7 ++++++- 1 file changed, 6 insertions(+), 1 deletion(-) diff --git a/fdbserver/BlobManager.actor.cpp b/fdbserver/BlobManager.actor.cpp index 5568ede056..c48f864245 100644 --- a/fdbserver/BlobManager.actor.cpp +++ b/fdbserver/BlobManager.actor.cpp @@ -1550,7 +1550,7 @@ static void addAssignment(KeyRangeMap>& map, // new one is from DB (source of truth on boundaries) and existing mapping disagrees on boundary or // assignment, do explicit revoke and re-assign to converge anyConflicts = true; - // if ranges don't match, need to explicitly reaassign all parts of old range, as it could be from a + // if ranges don't match, need to explicitly reassign all parts of old range, as it could be from a // yet-unassigned split if (old.range() != newRange) { std::get<0>(old.value()) = UID(); @@ -1564,6 +1564,11 @@ static void addAssignment(KeyRangeMap>& map, (oldEpoch < newEpoch || (oldEpoch == newEpoch && oldSeqno < newSeqno))) { // 2 blob workers reported conflicting mappings, add old one to out of date (if not already added by a // previous intersecting range in the split case) + // if ranges don't match, need to explicitly reassign all parts of old range, as it could be from a + // partially-assigned split + if (old.range() != newRange) { + std::get<0>(old.value()) = UID(); + } if (outOfDate.empty() || outOfDate.back() != std::pair(oldWorker, KeyRange(old.range()))) { TEST(true); // BM Recovery: Two workers claim ownership of same granule outOfDate.push_back(std::pair(oldWorker, old.range())); From b27d6bbfab7244dadebb94655763b2592fdc5a02 Mon Sep 17 00:00:00 2001 From: Josh Slocum Date: Tue, 15 Mar 2022 10:10:29 -0500 Subject: [PATCH 354/413] Have BW read manager lock on startup to avoid races with old BM --- fdbserver/BlobWorker.actor.cpp | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/fdbserver/BlobWorker.actor.cpp b/fdbserver/BlobWorker.actor.cpp index a87d722d16..d53f5ce264 100644 --- a/fdbserver/BlobWorker.actor.cpp +++ b/fdbserver/BlobWorker.actor.cpp @@ -2872,6 +2872,12 @@ ACTOR Future registerBlobWorker(Reference bwData, BlobWork tr->addReadConflictRange(singleKeyRange(blobWorkerListKey)); tr->set(blobWorkerListKey, blobWorkerListValue(interf)); + // Get manager lock from DB + Optional currentLockValue = wait(tr->get(blobManagerEpochKey)); + ASSERT(currentLockValue.present()); + int64_t currentEpoch = decodeBlobManagerEpochValue(currentLockValue.get()); + bwData->managerEpochOk(currentEpoch); + wait(tr->commit()); if (BW_DEBUG) { From ce5b567971f830233ee42465590d7dc4d8a8f1dd Mon Sep 17 00:00:00 2001 From: Josh Slocum Date: Tue, 15 Mar 2022 14:32:25 -0500 Subject: [PATCH 355/413] Cleanup the cleanup --- fdbclient/DatabaseContext.h | 2 - fdbclient/NativeAPI.actor.cpp | 57 +------------------ fdbclient/StorageServerInterface.h | 5 +- fdbserver/BlobManager.actor.cpp | 19 +------ fdbserver/BlobWorker.actor.cpp | 2 +- fdbserver/storageserver.actor.cpp | 6 +- .../BlobGranuleCorrectnessWorkload.actor.cpp | 3 +- 7 files changed, 10 insertions(+), 84 deletions(-) diff --git a/fdbclient/DatabaseContext.h b/fdbclient/DatabaseContext.h index 65b47f95f9..485617c102 100644 --- a/fdbclient/DatabaseContext.h +++ b/fdbclient/DatabaseContext.h @@ -170,8 +170,6 @@ public: struct MutationAndVersionStream { Standalone next; PromiseStream> results; - // TODO REMOVE, for debugging - int idx = -1; bool operator<(MutationAndVersionStream const& rhs) const { return next.version > rhs.next.version; } }; diff --git a/fdbclient/NativeAPI.actor.cpp b/fdbclient/NativeAPI.actor.cpp index 07d983cd64..6207455adb 100644 --- a/fdbclient/NativeAPI.actor.cpp +++ b/fdbclient/NativeAPI.actor.cpp @@ -6747,7 +6747,6 @@ Future>> Transaction::getRangeSplitPoints(KeyRange return ::getRangeSplitPoints(trState, keys, chunkSize); } -// TODO REMOVE when correctness clean #define BG_REQUEST_DEBUG false // the blob granule requests are a bit funky because they piggyback off the existing transaction to read from the system @@ -7479,9 +7478,6 @@ Version ChangeFeedData::getVersion() { return lastReturnedVersion.get(); } -// TODO REMOVE -#define DEBUG_CF_WAIT_VERSION invalidVersion - // This function is essentially bubbling the information about what has been processed from the server through the // change feed client. First it makes sure the server has returned all mutations up through the target version, the // native api has consumed and processed, them, and then the fdb client has consumed all of the mutations. @@ -7489,28 +7485,11 @@ ACTOR Future changeFeedWaitLatest(Reference self, Version // wait on SS to have sent up through version int desired = 0; int waiting = 0; - if (version == DEBUG_CF_WAIT_VERSION) { - fmt::print("WFV {}) waitLatest\n", version); - } std::vector> allAtLeast; for (auto& it : self->storageData) { if (it->version.get() < version) { waiting++; - if (version == DEBUG_CF_WAIT_VERSION) { - fmt::print("WFV {0}) SS {1} {2} < {3}\n", - version, - it->id.toString().substr(0, 4), - it->version.get(), - version); - } if (version > it->desired.get()) { - if (version == DEBUG_CF_WAIT_VERSION) { - fmt::print("WFV {0}) SS {1} desired {2} -> {3}\n", - version, - it->id.toString().substr(0, 4), - it->desired.get(), - version); - } it->desired.set(version); desired++; } @@ -7518,10 +7497,6 @@ ACTOR Future changeFeedWaitLatest(Reference self, Version } } - if (version == DEBUG_CF_WAIT_VERSION) { - fmt::print("WFV {0}) waiting for {1}/{2} ({3} desired)\n", version, waiting, self->storageData.size(), desired); - } - wait(waitForAll(allAtLeast)); // then, wait on ss streams to have processed up through version @@ -7571,9 +7546,6 @@ ACTOR Future changeFeedWhenAtLatest(Reference self, Versio return Void(); } state Future lastReturned = self->lastReturnedVersion.whenAtLeast(version); - if (version == DEBUG_CF_WAIT_VERSION) { - fmt::print("WFV {0}) notAtLatest={1}\n", version, self->notAtLatest.get()); - } loop { // only allowed to use empty versions if you're caught up Future waitEmptyVersion = (self->notAtLatest.get() == 0) ? changeFeedWaitLatest(self, version) : Never(); @@ -7605,7 +7577,7 @@ ACTOR Future partialChangeFeedStream(StorageServerInterface interf, Version end, Reference feedData, Reference storageData, - UID debugUID /* TODO REMOVE */) { + UID debugUID) { // calling lastReturnedVersion's callbacks could cause us to be cancelled state Promise refresh = feedData->refresh; @@ -7778,26 +7750,13 @@ ACTOR Future mergeChangeFeedStreamInternal(Reference resul interfNum = 0; while (interfNum < streamsUsed.size()) { try { - if (DEBUG_CF_CLIENT_TRACE) { - TraceEvent(SevDebug, "TraceChangeFeedClientMergeCursorWait", mergeCursorUID) - .detail("Idx", streamsUsed[interfNum].idx); - } Standalone res = waitNext(streamsUsed[interfNum].results.getFuture()); streamsUsed[interfNum].next = res; mutations.push(streamsUsed[interfNum]); - if (DEBUG_CF_CLIENT_TRACE) { - TraceEvent(SevDebug, "TraceChangeFeedClientMergeCursorGot", mergeCursorUID) - .detail("Idx", streamsUsed[interfNum].idx) - .detail("Version", res.version); - } } catch (Error& e) { if (e.code() != error_code_end_of_stream) { throw e; } - if (DEBUG_CF_CLIENT_TRACE) { - TraceEvent(SevDebug, "TraceChangeFeedClientMergeCursorEOS", mergeCursorUID) - .detail("Idx", streamsUsed[interfNum].idx); - } } interfNum++; } @@ -7858,13 +7817,6 @@ ACTOR Future mergeChangeFeedStreamInternal(Reference resul } } -ACTOR Future onCFErrors(std::vector> onErrors) { - wait(waitForAny(onErrors)); - // TODO: just do waitForAny and remove onCFErrors if this doesn't trip - ASSERT(false); - return Void(); -} - ACTOR Future mergeChangeFeedStream(Reference db, std::vector> interfs, Reference results, @@ -7880,7 +7832,6 @@ ACTOR Future mergeChangeFeedStream(Reference db, TEST(interfs.size() > 10); // Large change feed merge cursor TEST(interfs.size() > 100); // Very large change feed merge cursor - // TODO REMOVE eventually, useful for debugging for now state UID mergeCursorUID = UID(); state std::vector debugUIDs; results->streams.clear(); @@ -7896,7 +7847,6 @@ 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; } - // TODO REMOVE req.debugUID = deterministicRandom()->randomUniqueID(); debugUIDs.push_back(req.debugUID); mergeCursorUID = @@ -7932,7 +7882,6 @@ ACTOR Future mergeChangeFeedStream(Reference db, .detail("End", end) .detail("CanReadPopped", canReadPopped); } - streams[i].idx = i; onErrors[i] = results->streams[i].onError(); fetchers[i] = partialChangeFeedStream(interfs[i].first, streams[i].results, @@ -7944,7 +7893,8 @@ ACTOR Future mergeChangeFeedStream(Reference db, debugUIDs[i]); } - wait(onCFErrors(onErrors) || mergeChangeFeedStreamInternal(results, interfs, streams, begin, end, mergeCursorUID)); + wait(waitForAny(onErrors) || + mergeChangeFeedStreamInternal(results, interfs, streams, begin, end, mergeCursorUID)); return Void(); } @@ -8066,7 +8016,6 @@ ACTOR Future singleChangeFeedStream(Reference db, req.range = range; req.canReadPopped = canReadPopped; req.replyBufferSize = replyBufferSize; - // TODO REMOVE req.debugUID = deterministicRandom()->randomUniqueID(); if (DEBUG_CF_CLIENT_TRACE) { diff --git a/fdbclient/StorageServerInterface.h b/fdbclient/StorageServerInterface.h index 794397cfb2..d4d44ba7bf 100644 --- a/fdbclient/StorageServerInterface.h +++ b/fdbclient/StorageServerInterface.h @@ -793,8 +793,9 @@ struct ChangeFeedStreamRequest { KeyRange range; int replyBufferSize = -1; bool canReadPopped = true; - // TODO REMOVE - UID debugUID; + 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 ReplyPromiseStream reply; diff --git a/fdbserver/BlobManager.actor.cpp b/fdbserver/BlobManager.actor.cpp index c48f864245..4934e355be 100644 --- a/fdbserver/BlobManager.actor.cpp +++ b/fdbserver/BlobManager.actor.cpp @@ -47,7 +47,7 @@ * The Blob Manager is responsible for managing range granules, and recruiting and monitoring Blob Workers. */ -#define BM_DEBUG true +#define BM_DEBUG false void handleClientBlobRange(KeyRangeMap* knownBlobRanges, Arena& ar, @@ -974,23 +974,6 @@ ACTOR Future maybeSplitRange(Reference bmData, existingState.size() + 2)); // +2 because this is boundaries and existingState was granules, and to ensure it doesn't set more ASSERT(!existingBoundaries.more); - // TODO remove debugging printf - if (existingBoundaries.size() != existingState.size() + 1) { - printf("DBG: EB=%d, ES=%d for [%s - %s)\n", - existingBoundaries.size(), - existingState.size(), - granuleRange.begin.printable().c_str(), - granuleRange.end.printable().c_str()); - printf("Boundaries:\n"); - for (auto& it : existingBoundaries) { - printf(" %s\n", it.key.removePrefix(blobGranuleMappingKeys.begin).printable().c_str()); - } - printf("State:\n"); - for (auto& it : existingState) { - std::pair k = decodeBlobGranuleSplitKey(it.key); - printf(" %s\n", k.second.toString().substr(0, 6).c_str()); - } - } ASSERT(existingBoundaries.size() == existingState.size() + 1); newRanges.clear(); newRanges.arena().dependsOn(existingBoundaries.arena()); diff --git a/fdbserver/BlobWorker.actor.cpp b/fdbserver/BlobWorker.actor.cpp index d53f5ce264..1e08ab3837 100644 --- a/fdbserver/BlobWorker.actor.cpp +++ b/fdbserver/BlobWorker.actor.cpp @@ -1495,9 +1495,9 @@ ACTOR Future blobGranuleUpdateFiles(Reference bwData, if (metadata->pendingDeltaVersion <= rollbackVersion && (metadata->currentDeltas.empty() || metadata->currentDeltas.back().version <= rollbackVersion)) { + TEST(true); // Granule ignoring rollback if (BW_DEBUG) { - TEST(true); // Granule ignoring rollback fmt::print( "Granule [{0} - {1}) on BW {2} skipping rollback {3} -> {4} completely\n", metadata->keyRange.begin.printable().c_str(), diff --git a/fdbserver/storageserver.actor.cpp b/fdbserver/storageserver.actor.cpp index a4f5b9033c..c089764b1b 100644 --- a/fdbserver/storageserver.actor.cpp +++ b/fdbserver/storageserver.actor.cpp @@ -7914,11 +7914,7 @@ ACTOR Future serveChangeFeedStreamRequests(StorageServer* self, loop { ChangeFeedStreamRequest req = waitNext(changeFeedStream); // must notify change feed that its shard is moved away ASAP - - // TODO CHANGE BACK - // UID streamUID = deterministicRandom()->randomUniqueID(); - UID streamUID = req.debugUID; - self->actors.add(changeFeedStreamQ(self, req, streamUID) || stopChangeFeedOnMove(self, req, streamUID)); + self->actors.add(changeFeedStreamQ(self, req, req.debugUID) || stopChangeFeedOnMove(self, req, req.debugUID)); } } diff --git a/fdbserver/workloads/BlobGranuleCorrectnessWorkload.actor.cpp b/fdbserver/workloads/BlobGranuleCorrectnessWorkload.actor.cpp index e6ec6013c7..ea43cebbb7 100644 --- a/fdbserver/workloads/BlobGranuleCorrectnessWorkload.actor.cpp +++ b/fdbserver/workloads/BlobGranuleCorrectnessWorkload.actor.cpp @@ -121,8 +121,7 @@ struct ThreadData : ReferenceCounted, NonCopyable { Key getKey(uint32_t key, uint32_t id) { return StringRef(format("%08x/%08x/%08x", directoryID, key, id)); } }; -// TODO REMOVE eventually? -// For debugging mismatches on what data should be and why +// For debugging mismatches on what data should be and why // set mismatch to true, dir id and key id to the directory and key id that are wrong, and rv to read version that read // the wrong value #define DEBUG_MISMATCH false From e8e69e7c564fab714a44225bfce7dcb925a86beb Mon Sep 17 00:00:00 2001 From: Josh Slocum Date: Tue, 15 Mar 2022 15:15:16 -0500 Subject: [PATCH 356/413] Fix formatting --- fdbclient/NativeAPI.actor.cpp | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/fdbclient/NativeAPI.actor.cpp b/fdbclient/NativeAPI.actor.cpp index 6207455adb..7bf27d619b 100644 --- a/fdbclient/NativeAPI.actor.cpp +++ b/fdbclient/NativeAPI.actor.cpp @@ -7893,8 +7893,7 @@ ACTOR Future mergeChangeFeedStream(Reference db, debugUIDs[i]); } - wait(waitForAny(onErrors) || - mergeChangeFeedStreamInternal(results, interfs, streams, begin, end, mergeCursorUID)); + wait(waitForAny(onErrors) || mergeChangeFeedStreamInternal(results, interfs, streams, begin, end, mergeCursorUID)); return Void(); } From 2c88a189a9c9225baeee7593aa53972209ec5dda Mon Sep 17 00:00:00 2001 From: Evan Tschannen Date: Tue, 15 Mar 2022 13:34:59 -0700 Subject: [PATCH 357/413] more pruning bug fixes --- fdbserver/BlobManager.actor.cpp | 11 ++--- fdbserver/BlobWorker.actor.cpp | 41 +++++++++++++------ .../workloads/BlobGranuleVerifier.actor.cpp | 39 ++++++++++-------- 3 files changed, 56 insertions(+), 35 deletions(-) diff --git a/fdbserver/BlobManager.actor.cpp b/fdbserver/BlobManager.actor.cpp index 13cd7acd38..7b97199b54 100644 --- a/fdbserver/BlobManager.actor.cpp +++ b/fdbserver/BlobManager.actor.cpp @@ -2092,7 +2092,7 @@ ACTOR Future loadHistoryFiles(Reference bmData, U * also removes the history entry for this granule from the system keyspace * TODO: ensure cannot fully delete granule that is still splitting! */ -ACTOR Future fullyDeleteGranule(Reference self, UID granuleId, KeyRef historyKey) { +ACTOR Future fullyDeleteGranule(Reference self, UID granuleId, Key historyKey) { if (BM_DEBUG) { fmt::print("Fully deleting granule {0}: init\n", granuleId.toString()); } @@ -2194,8 +2194,9 @@ ACTOR Future partiallyDeleteGranule(Reference self, UID g } } - // we would have only partially deleted the granule if such a snapshot existed - ASSERT(latestSnapshotVersion != invalidVersion); + if (latestSnapshotVersion == invalidVersion) { + return Void(); + } // delete all delta files older than latestSnapshotVersion for (auto deltaFile : files.deltaFiles) { @@ -2277,7 +2278,7 @@ ACTOR Future pruneRange(Reference self, KeyRangeRef range state std::queue> historyEntryQueue; // stacks of and to track which granules to delete - state std::vector> toFullyDelete; + state std::vector> toFullyDelete; state std::vector toPartiallyDelete; // track which granules we have already added to traversal @@ -2443,7 +2444,7 @@ ACTOR Future pruneRange(Reference self, KeyRangeRef range } for (i = toFullyDelete.size() - 1; i >= 0; --i) { UID granuleId; - KeyRef historyKey; + Key historyKey; std::tie(granuleId, historyKey) = toFullyDelete[i]; // FIXME: consider batching into a single txn (need to take care of txn size limit) if (BM_DEBUG) { diff --git a/fdbserver/BlobWorker.actor.cpp b/fdbserver/BlobWorker.actor.cpp index a0af488edf..eef6558050 100644 --- a/fdbserver/BlobWorker.actor.cpp +++ b/fdbserver/BlobWorker.actor.cpp @@ -1889,31 +1889,34 @@ ACTOR Future blobGranuleLoadHistory(Reference bwData, stopVersion = prev.value().isValid() ? prev.value()->startVersion : invalidVersion; state std::vector> historyEntryStack; + state bool foundHistory = true; // while the start version of the current granule's parent not past the last known start version, // walk backwards while (curHistory.value.parentGranules.size() > 0 && curHistory.value.parentGranules[0].second >= stopVersion) { state GranuleHistory next; + loop { try { Optional v = wait(tr.get(blobGranuleHistoryKeyFor( curHistory.value.parentGranules[0].first, curHistory.value.parentGranules[0].second))); if (!v.present()) { - printf("No granule history present for [%s - %s) @ %lld!!\n", - curHistory.value.parentGranules[0].first.begin.printable().c_str(), - curHistory.value.parentGranules[0].first.end.printable().c_str(), - curHistory.value.parentGranules[0].first); + foundHistory = false; + } else { + next = GranuleHistory(curHistory.value.parentGranules[0].first, + curHistory.value.parentGranules[0].second, + decodeBlobGranuleHistoryValue(v.get())); } - ASSERT(v.present()); - next = GranuleHistory(curHistory.value.parentGranules[0].first, - curHistory.value.parentGranules[0].second, - decodeBlobGranuleHistoryValue(v.get())); + break; } catch (Error& e) { wait(tr.onError(e)); } } + if (!foundHistory) { + break; + } ASSERT(next.version != invalidVersion); // granule next.granuleID goes from the version range [next.version, curHistory.version] @@ -1924,8 +1927,14 @@ ACTOR Future blobGranuleLoadHistory(Reference bwData, if (!historyEntryStack.empty()) { Version oldestStartVersion = historyEntryStack.back()->startVersion; + if (!foundHistory && stopVersion != invalidVersion) { + stopVersion = oldestStartVersion; + } ASSERT(stopVersion == oldestStartVersion || stopVersion == invalidVersion); } else { + if (!foundHistory && stopVersion != invalidVersion) { + stopVersion = invalidVersion; + } ASSERT(stopVersion == invalidVersion); } @@ -1947,11 +1956,13 @@ ACTOR Future blobGranuleLoadHistory(Reference bwData, while (i >= 0) { auto prevRanges = bwData->granuleHistory.rangeContaining(historyEntryStack[i]->range.begin); - // sanity check - ASSERT(!prevRanges.value().isValid() || - prevRanges.value()->endVersion == historyEntryStack[i]->startVersion); + if (prevRanges.value().isValid() && + prevRanges.value()->endVersion != historyEntryStack[i]->startVersion) { + historyEntryStack[i]->parentGranule = Reference(); + } else { + historyEntryStack[i]->parentGranule = prevRanges.value(); + } - historyEntryStack[i]->parentGranule = prevRanges.value(); bwData->granuleHistory.insert(historyEntryStack[i]->range, historyEntryStack[i]); i--; } @@ -2199,7 +2210,11 @@ ACTOR Future doBlobGranuleFileRequest(Reference bwData, Bl when(wait(metadata->cancelled.getFuture())) { throw wrong_shard_server(); } } - ASSERT(!chunkFiles.snapshotFiles.empty()); + if (chunkFiles.snapshotFiles.empty()) { + // a snapshot file must have been pruned + throw blob_granule_transaction_too_old(); + } + ASSERT(!chunkFiles.deltaFiles.empty()); ASSERT(chunkFiles.deltaFiles.back().version > req.readVersion); if (chunkFiles.snapshotFiles.front().version > req.readVersion) { diff --git a/fdbserver/workloads/BlobGranuleVerifier.actor.cpp b/fdbserver/workloads/BlobGranuleVerifier.actor.cpp index 88fff6ef47..0b3981e705 100644 --- a/fdbserver/workloads/BlobGranuleVerifier.actor.cpp +++ b/fdbserver/workloads/BlobGranuleVerifier.actor.cpp @@ -39,8 +39,6 @@ #define BGV_DEBUG true -Version dbgPruneVersion = 0; - /* * This workload is designed to verify the correctness of the blob data produced by the blob workers. * As a read-only validation workload, it can piggyback off of other write or read/write workloads. @@ -64,6 +62,7 @@ struct BlobGranuleVerifierWorkload : TestWorkload { int64_t rowsRead = 0; int64_t bytesRead = 0; std::vector> clients; + bool enablePruning; DatabaseConfiguration config; @@ -79,6 +78,7 @@ struct BlobGranuleVerifierWorkload : TestWorkload { timeTravelLimit = getOption(options, LiteralStringRef("timeTravelLimit"), testDuration); timeTravelBufferSize = getOption(options, LiteralStringRef("timeTravelBufferSize"), 100000000); threads = getOption(options, LiteralStringRef("threads"), 1); + enablePruning = getOption(options, LiteralStringRef("enablePruning"), false /*sharedRandomNumber % 2 == 0*/); ASSERT(threads >= 1); if (BGV_DEBUG) { @@ -454,13 +454,19 @@ struct BlobGranuleVerifierWorkload : TestWorkload { try { state Version newPruneVersion = 0; - state bool doPruning = - allowPruning && prevPruneVersion < oldRead.v && deterministicRandom()->random01() < 0.5; + state bool doPruning = allowPruning && deterministicRandom()->random01() < 0.5; if (doPruning) { - newPruneVersion = deterministicRandom()->randomInt64(prevPruneVersion, oldRead.v); - prevPruneVersion = std::max(prevPruneVersion, newPruneVersion); - dbgPruneVersion = prevPruneVersion; - wait(self->pruneAtVersion(cx, normalKeys, newPruneVersion, false)); + Version maxPruneVersion = oldRead.v; + for (auto& it : timeTravelChecks) { + maxPruneVersion = std::min(it.second.v, maxPruneVersion); + } + if (prevPruneVersion < maxPruneVersion) { + newPruneVersion = deterministicRandom()->randomInt64(prevPruneVersion, maxPruneVersion); + prevPruneVersion = std::max(prevPruneVersion, newPruneVersion); + wait(self->pruneAtVersion(cx, normalKeys, newPruneVersion, false)); + } else { + doPruning = false; + } } std::pair>> reReadResult = wait(self->readFromBlob(cx, self, oldRead.range, oldRead.v)); @@ -487,7 +493,7 @@ struct BlobGranuleVerifierWorkload : TestWorkload { } } } catch (Error& e) { - if (e.code() == error_code_blob_granule_transaction_too_old && oldRead.v >= dbgPruneVersion) { + if (e.code() == error_code_blob_granule_transaction_too_old) { self->timeTravelTooOld++; // TODO: add debugging info for when this is a failure } @@ -532,15 +538,14 @@ struct BlobGranuleVerifierWorkload : TestWorkload { Future start(Database const& cx) override { clients.reserve(threads + 1); clients.push_back(timeout(findGranules(cx, this), testDuration, Void())); - for (int i = 0; i < threads; i++) { + if (enablePruning && clientId == 0) { clients.push_back( - timeout(reportErrors( - // TODO change back - verifyGranules( - cx, this, false /*clientId == 0 && i == 0 && deterministicRandom()->random01() < 0.5*/), - "BlobGranuleVerifier"), - testDuration, - Void())); + timeout(reportErrors(verifyGranules(cx, this, true), "BlobGranuleVerifier"), testDuration, Void())); + } else if (!enablePruning) { + for (int i = 0; i < threads; i++) { + clients.push_back(timeout( + reportErrors(verifyGranules(cx, this, false), "BlobGranuleVerifier"), testDuration, Void())); + } } return delay(testDuration); } From 1aabbb07d2045a23fffe863b7db2274bc8a0a67e Mon Sep 17 00:00:00 2001 From: sfc-gh-tclinkenbeard Date: Wed, 16 Mar 2022 11:24:31 -0700 Subject: [PATCH 358/413] Add missing copyright headers --- fdbclient/JsonBuilder.cpp | 20 +++++++++++++++++++ fdbclient/VersionedMap.cpp | 20 +++++++++++++++++++ fdbserver/FDBExecHelper.actor.cpp | 20 +++++++++++++++++++ fdbserver/KeyValueStoreRocksDB.actor.cpp | 20 +++++++++++++++++++ fdbserver/SigStack.cpp | 20 +++++++++++++++++++ fdbserver/workloads/Cache.actor.cpp | 20 +++++++++++++++++++ fdbserver/workloads/Mako.actor.cpp | 20 +++++++++++++++++++ fdbserver/workloads/SnapTest.actor.cpp | 20 +++++++++++++++++++ .../workloads/SuspendProcesses.actor.cpp | 20 +++++++++++++++++++ fdbserver/workloads/TriggerRecovery.actor.cpp | 20 +++++++++++++++++++ flow/IThreadPoolTest.actor.cpp | 20 +++++++++++++++++++ 11 files changed, 220 insertions(+) diff --git a/fdbclient/JsonBuilder.cpp b/fdbclient/JsonBuilder.cpp index 0097ca7e1c..05f5dda36e 100644 --- a/fdbclient/JsonBuilder.cpp +++ b/fdbclient/JsonBuilder.cpp @@ -1,3 +1,23 @@ +/* + * JsonBuilder.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/JsonBuilder.h" #include diff --git a/fdbclient/VersionedMap.cpp b/fdbclient/VersionedMap.cpp index be4589fac1..8dba1e4ad4 100644 --- a/fdbclient/VersionedMap.cpp +++ b/fdbclient/VersionedMap.cpp @@ -1,3 +1,23 @@ +/* + * VersionedMap.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/VersionedMap.h" #include "flow/TreeBenchmark.h" #include "flow/UnitTest.h" diff --git a/fdbserver/FDBExecHelper.actor.cpp b/fdbserver/FDBExecHelper.actor.cpp index 6dc359b2df..9b690d4d35 100644 --- a/fdbserver/FDBExecHelper.actor.cpp +++ b/fdbserver/FDBExecHelper.actor.cpp @@ -1,3 +1,23 @@ +/* + * FDBExecHelper.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. + */ + #if !defined(_WIN32) && !defined(__APPLE__) && !defined(__INTEL_COMPILER) #define BOOST_SYSTEM_NO_LIB #define BOOST_DATE_TIME_NO_LIB diff --git a/fdbserver/KeyValueStoreRocksDB.actor.cpp b/fdbserver/KeyValueStoreRocksDB.actor.cpp index c157458d81..b2bdd20b0b 100644 --- a/fdbserver/KeyValueStoreRocksDB.actor.cpp +++ b/fdbserver/KeyValueStoreRocksDB.actor.cpp @@ -1,3 +1,23 @@ +/* + * KeyValueStoreRocksDB.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. + */ + #ifdef SSD_ROCKSDB_EXPERIMENTAL #include diff --git a/fdbserver/SigStack.cpp b/fdbserver/SigStack.cpp index f00ec5eebc..12522ea9ef 100644 --- a/fdbserver/SigStack.cpp +++ b/fdbserver/SigStack.cpp @@ -1,3 +1,23 @@ +/* + * SigStack.cpp + * + * This source file is part of the FoundationDB open source project + * + * Copyright 2013-2018 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 "flow/flow.h" #include "fdbclient/StackLineage.h" #include diff --git a/fdbserver/workloads/Cache.actor.cpp b/fdbserver/workloads/Cache.actor.cpp index 9e32281d54..2b107a07e1 100644 --- a/fdbserver/workloads/Cache.actor.cpp +++ b/fdbserver/workloads/Cache.actor.cpp @@ -1,3 +1,23 @@ +/* + * Cache.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/ManagementAPI.actor.h" #include "fdbserver/TesterInterface.actor.h" #include "fdbserver/workloads/workloads.actor.h" diff --git a/fdbserver/workloads/Mako.actor.cpp b/fdbserver/workloads/Mako.actor.cpp index 0db3bc62ec..e1ec2ba9f7 100644 --- a/fdbserver/workloads/Mako.actor.cpp +++ b/fdbserver/workloads/Mako.actor.cpp @@ -1,3 +1,23 @@ +/* + * Mako.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/NativeAPI.actor.h" #include "fdbserver/TesterInterface.actor.h" #include "fdbserver/workloads/workloads.actor.h" diff --git a/fdbserver/workloads/SnapTest.actor.cpp b/fdbserver/workloads/SnapTest.actor.cpp index 718ab09b51..2629666cfa 100644 --- a/fdbserver/workloads/SnapTest.actor.cpp +++ b/fdbserver/workloads/SnapTest.actor.cpp @@ -1,3 +1,23 @@ +/* + * SnapTest.actor.cpp + * + * This source file is part of the FoundationDB open source project + * + * Copyright 2013-2022 Apple Inc. and the FoundationDB project authors + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + #include #include "fdbclient/ManagementAPI.actor.h" #include "fdbclient/NativeAPI.actor.h" diff --git a/fdbserver/workloads/SuspendProcesses.actor.cpp b/fdbserver/workloads/SuspendProcesses.actor.cpp index 1a8d4af1b3..ce1a5c05e2 100644 --- a/fdbserver/workloads/SuspendProcesses.actor.cpp +++ b/fdbserver/workloads/SuspendProcesses.actor.cpp @@ -1,3 +1,23 @@ +/* + * SuspendProcesses.actor.cpp + * + * This source file is part of the FoundationDB open source project + * + * Copyright 2013-2022 Apple Inc. and the FoundationDB project authors + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + #include #include "fdbserver/workloads/workloads.actor.h" diff --git a/fdbserver/workloads/TriggerRecovery.actor.cpp b/fdbserver/workloads/TriggerRecovery.actor.cpp index e3bd9240ff..ee5fc2fd9a 100644 --- a/fdbserver/workloads/TriggerRecovery.actor.cpp +++ b/fdbserver/workloads/TriggerRecovery.actor.cpp @@ -1,3 +1,23 @@ +/* + * TriggerRecovery.actor.cpp + * + * This source file is part of the FoundationDB open source project + * + * Copyright 2013-2022 Apple Inc. and the FoundationDB project authors + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + #include "fdbserver/workloads/workloads.actor.h" #include "fdbserver/ServerDBInfo.h" #include "fdbclient/Status.h" diff --git a/flow/IThreadPoolTest.actor.cpp b/flow/IThreadPoolTest.actor.cpp index 2e0c1341f6..ed18c17b6f 100644 --- a/flow/IThreadPoolTest.actor.cpp +++ b/flow/IThreadPoolTest.actor.cpp @@ -1,3 +1,23 @@ +/* + * IThreadPoolTest.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. + */ + // Thread naming only works on Linux. #if defined(__linux__) From 1ab18aae6a0909ff1e45c05664ca5d40d37a7df0 Mon Sep 17 00:00:00 2001 From: sfc-gh-tclinkenbeard Date: Wed, 16 Mar 2022 14:21:58 -0700 Subject: [PATCH 359/413] Fix calls to DDTeamCollection::addTeam --- fdbserver/DDTeamCollection.actor.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/fdbserver/DDTeamCollection.actor.cpp b/fdbserver/DDTeamCollection.actor.cpp index 7e059499f5..d83ded0edc 100644 --- a/fdbserver/DDTeamCollection.actor.cpp +++ b/fdbserver/DDTeamCollection.actor.cpp @@ -5674,8 +5674,8 @@ public: high_avail.available.bytes = 800 * 1024 * 1024; high_avail.load.bytes = 90 * 1024 * 1024; - collection->addTeam(std::set({ UID(1, 0), UID(2, 0), UID(3, 0) }), true); - collection->addTeam(std::set({ UID(2, 0), UID(3, 0), UID(4, 0) }), true); + collection->addTeam(std::set({ UID(1, 0), UID(2, 0), UID(3, 0) }), IsInitialTeam::True); + collection->addTeam(std::set({ UID(2, 0), UID(3, 0), UID(4, 0) }), IsInitialTeam::True); collection->disableBuildingTeams(); collection->setCheckTeamDelay(); From 7908ea54f324f5f9a20cb79cda5cebf76d4adb46 Mon Sep 17 00:00:00 2001 From: Evan Tschannen Date: Thu, 17 Mar 2022 09:57:46 -0700 Subject: [PATCH 360/413] fix: NetNotifiedQueueWithAcknowledgements could miss disconnects that happen during the delay(0) in deliver() --- fdbrpc/FlowTransport.actor.cpp | 36 +++++++++++++++++++++++++++------- fdbrpc/FlowTransport.h | 2 ++ fdbrpc/fdbrpc.h | 4 +--- 3 files changed, 32 insertions(+), 10 deletions(-) diff --git a/fdbrpc/FlowTransport.actor.cpp b/fdbrpc/FlowTransport.actor.cpp index f4fbf461f0..908af910fa 100644 --- a/fdbrpc/FlowTransport.actor.cpp +++ b/fdbrpc/FlowTransport.actor.cpp @@ -45,6 +45,7 @@ #include "flow/actorcompiler.h" // This must be the last #include. static NetworkAddressList g_currentDeliveryPeerAddress = NetworkAddressList(); +static Future g_currentDeliveryPeerDisconnect; constexpr int PACKET_LEN_WIDTH = sizeof(uint32_t); const uint64_t TOKEN_STREAM_FLAG = 1; @@ -781,6 +782,9 @@ ACTOR Future connectionKeeper(Reference self, // Clients might send more packets in response, which needs to go out on the next connection IFailureMonitor::failureMonitor().notifyDisconnect(self->destination); + Promise disconnect = self->disconnect; + self->disconnect = Promise(); + disconnect.send(Void()); if (e.code() == error_code_actor_cancelled) throw; @@ -923,7 +927,8 @@ ACTOR static void deliver(TransportData* self, Endpoint destination, TaskPriority priority, ArenaReader reader, - bool inReadSocket) { + bool inReadSocket, + Future disconnect) { // We want to run the task at the right priority. If the priority is higher than the current priority (which is // ReadSocket) we can just upgrade. Otherwise we'll context switch so that we don't block other tasks that might run // with a higher priority. ReplyPromiseStream needs to guarentee that messages are recieved in the order they were @@ -942,13 +947,16 @@ ACTOR static void deliver(TransportData* self, } try { g_currentDeliveryPeerAddress = destination.addresses; + g_currentDeliveryPeerDisconnect = disconnect; StringRef data = reader.arenaReadAll(); ASSERT(data.size() > 8); ArenaObjectReader objReader(reader.arena(), reader.arenaReadAll(), AssumeVersion(reader.protocolVersion())); receiver->receive(objReader); g_currentDeliveryPeerAddress = { NetworkAddress() }; + g_currentDeliveryPeerDisconnect = Future(); } catch (Error& e) { g_currentDeliveryPeerAddress = { NetworkAddress() }; + g_currentDeliveryPeerDisconnect = Future(); TraceEvent(SevError, "ReceiverError") .error(e) .detail("Token", destination.token.toString()) @@ -982,7 +990,8 @@ static void scanPackets(TransportData* transport, const uint8_t* e, Arena& arena, NetworkAddress const& peerAddress, - ProtocolVersion peerProtocolVersion) { + ProtocolVersion peerProtocolVersion, + Future disconnect) { // Find each complete packet in the given byte range and queue a ready task to deliver it. // Remove the complete packets from the range by increasing unprocessed_begin. // There won't be more than 64K of data plus one packet, so this shouldn't take a long time. @@ -1095,7 +1104,7 @@ static void scanPackets(TransportData* transport, // we have many messages to UnknownEndpoint we want to optimize earlier. As deliver is an actor it // will allocate some state on the heap and this prevents it from doing that. if (priority != TaskPriority::UnknownEndpoint || (token.first() & TOKEN_STREAM_FLAG) != 0) { - deliver(transport, Endpoint({ peerAddress }, token), priority, std::move(reader), true); + deliver(transport, Endpoint({ peerAddress }, token), priority, std::move(reader), true, disconnect); } unprocessed_begin = p = p + packetLen; @@ -1290,8 +1299,13 @@ ACTOR static Future connectionReader(TransportData* transport, if (!expectConnectPacket) { if (compatible || peerProtocolVersion.hasStableInterfaces()) { - scanPackets( - transport, unprocessed_begin, unprocessed_end, arena, peerAddress, peerProtocolVersion); + scanPackets(transport, + unprocessed_begin, + unprocessed_end, + arena, + peerAddress, + peerProtocolVersion, + peer->disconnect.getFuture()); } else { unprocessed_begin = unprocessed_end; peer->resetPing.trigger(); @@ -1488,6 +1502,10 @@ Endpoint FlowTransport::loadedEndpoint(const UID& token) { return Endpoint(g_currentDeliveryPeerAddress, token); } +Future FlowTransport::loadedDisconnect() { + return g_currentDeliveryPeerDisconnect; +} + void FlowTransport::addPeerReference(const Endpoint& endpoint, bool isStream) { if (!isStream || !endpoint.getPrimaryAddress().isValid() || !endpoint.getPrimaryAddress().isPublic()) return; @@ -1561,8 +1579,12 @@ static void sendLocal(TransportData* self, ISerializeSource const& what, const E ASSERT(copy.size() > 0); TaskPriority priority = self->endpoints.getPriority(destination.token); if (priority != TaskPriority::UnknownEndpoint || (destination.token.first() & TOKEN_STREAM_FLAG) != 0) { - deliver( - self, destination, priority, ArenaReader(copy.arena(), copy, AssumeVersion(currentProtocolVersion)), false); + deliver(self, + destination, + priority, + ArenaReader(copy.arena(), copy, AssumeVersion(currentProtocolVersion)), + false, + Never()); } } diff --git a/fdbrpc/FlowTransport.h b/fdbrpc/FlowTransport.h index 24daae400a..3dce811078 100644 --- a/fdbrpc/FlowTransport.h +++ b/fdbrpc/FlowTransport.h @@ -169,6 +169,7 @@ struct Peer : public ReferenceCounted { int connectIncomingCount; int connectFailedCount; ContinuousSample connectLatencies; + Promise disconnect; explicit Peer(TransportData* transport, NetworkAddress const& destination); @@ -269,6 +270,7 @@ public: static NetworkAddressList getGlobalLocalAddresses() { return transport().getLocalAddresses(); } Endpoint loadedEndpoint(const UID& token); + Future loadedDisconnect(); HealthMonitor* healthMonitor(); diff --git a/fdbrpc/fdbrpc.h b/fdbrpc/fdbrpc.h index 7490542138..9e1ddc7919 100644 --- a/fdbrpc/fdbrpc.h +++ b/fdbrpc/fdbrpc.h @@ -333,9 +333,7 @@ struct NetNotifiedQueueWithAcknowledgements final : NotifiedQueue, NetNotifiedQueueWithAcknowledgements(int futures, int promises, const Endpoint& remoteEndpoint) : NotifiedQueue(futures, promises), FlowReceiver(remoteEndpoint, true), onConnect(nullptr) { // A ReplyPromiseStream will be terminated on the server side if the network connection with the client breaks - acknowledgements.failures = tagError( - makeDependent(IFailureMonitor::failureMonitor()).onDisconnect(remoteEndpoint.getPrimaryAddress()), - operation_obsolete()); + acknowledgements.failures = tagError(FlowTransport::transport().loadedDisconnect(), operation_obsolete()); } void destroy() override { delete this; } From f8cddd3842855cab954e3604f10d90ff372a5051 Mon Sep 17 00:00:00 2001 From: "A.J. Beamon" Date: Sat, 19 Feb 2022 15:25:51 -0800 Subject: [PATCH 361/413] Add fdbcli support for tenants --- fdbcli/BlobRangeCommand.actor.cpp | 23 ++++- fdbcli/CMakeLists.txt | 1 + fdbcli/ChangeFeedCommand.actor.cpp | 16 ++- fdbcli/ConfigureCommand.actor.cpp | 5 +- fdbcli/TenantCommands.actor.cpp | 143 +++++++++++++++++++++++++++ fdbcli/fdbcli.actor.cpp | 152 ++++++++++++++++++++++++----- fdbcli/fdbcli.actor.h | 17 +++- 7 files changed, 325 insertions(+), 32 deletions(-) create mode 100644 fdbcli/TenantCommands.actor.cpp diff --git a/fdbcli/BlobRangeCommand.actor.cpp b/fdbcli/BlobRangeCommand.actor.cpp index 33a34e7e7b..97bfc98872 100644 --- a/fdbcli/BlobRangeCommand.actor.cpp +++ b/fdbcli/BlobRangeCommand.actor.cpp @@ -62,12 +62,27 @@ ACTOR Future setBlobRange(Database db, Key startKey, Key endKey, Value val namespace fdb_cli { -ACTOR Future blobRangeCommandActor(Database localDb, std::vector tokens) { +ACTOR Future blobRangeCommandActor(Database localDb, + Optional tenantEntry, + std::vector tokens) { // enables blob writing for the given range if (tokens.size() != 4) { printUsage(tokens[0]); return false; - } else if (tokens[3] > LiteralStringRef("\xff")) { + } + + Key begin; + Key end; + + if (tenantEntry.present()) { + begin = tokens[2].withPrefix(tenantEntry.get().prefix); + end = tokens[3].withPrefix(tenantEntry.get().prefix); + } else { + begin = tokens[2]; + end = tokens[3]; + } + + if (end > LiteralStringRef("\xff")) { // TODO is this something we want? printf("Cannot blobbify system keyspace! Problematic End Key: %s\n", tokens[3].printable().c_str()); return false; @@ -78,12 +93,12 @@ ACTOR Future blobRangeCommandActor(Database localDb, std::vector "); diff --git a/fdbcli/CMakeLists.txt b/fdbcli/CMakeLists.txt index c54338fa95..29e831f859 100644 --- a/fdbcli/CMakeLists.txt +++ b/fdbcli/CMakeLists.txt @@ -24,6 +24,7 @@ set(FDBCLI_SRCS SnapshotCommand.actor.cpp StatusCommand.actor.cpp SuspendCommand.actor.cpp + TenantCommands.actor.cpp ThrottleCommand.actor.cpp TriggerDDTeamInfoLogCommand.actor.cpp TssqCommand.actor.cpp diff --git a/fdbcli/ChangeFeedCommand.actor.cpp b/fdbcli/ChangeFeedCommand.actor.cpp index 0cebc508ec..796be8f337 100644 --- a/fdbcli/ChangeFeedCommand.actor.cpp +++ b/fdbcli/ChangeFeedCommand.actor.cpp @@ -75,7 +75,10 @@ ACTOR Future requestVersionUpdate(Database localDb, Reference changeFeedCommandActor(Database localDb, std::vector tokens, Future warn) { +ACTOR Future changeFeedCommandActor(Database localDb, + Optional tenantEntry, + std::vector tokens, + Future warn) { if (tokens.size() == 1) { printUsage(tokens[0]); return false; @@ -92,8 +95,15 @@ ACTOR Future changeFeedCommandActor(Database localDb, std::vector|" "commit_proxies=|grv_proxies=|logs=|resolvers=>*|" "count=|perpetual_storage_wiggle=|perpetual_storage_wiggle_locality=" - "<:|0>|storage_migration_type={disabled|gradual|aggressive}", + "<:|0>|storage_migration_type={disabled|gradual|aggressive}" + "|tenant_mode={optional|required}", "change the database configuration", "The `new' option, if present, initializes a new database with the given configuration rather than changing " "the configuration of an existing one. When used, both a redundancy mode and a storage engine must be " @@ -295,6 +296,8 @@ CommandFactory configureFactory( "perpetual_storage_wiggle_locality=<:|0>: Set the process filter for wiggling. " "The processes that match the given locality key and locality value are only wiggled. The value 0 will disable " "the locality filter and matches all the processes for wiggling.\n\n" + "tenant_mode=: Sets the tenant mode for the cluster. If optional, then transactions can be " + "run with or without specifying tenants. If required, all data must be accessed using tenants.\n\n" "See the FoundationDB Administration Guide for more information.")); } // namespace fdb_cli diff --git a/fdbcli/TenantCommands.actor.cpp b/fdbcli/TenantCommands.actor.cpp new file mode 100644 index 0000000000..fb9e051b52 --- /dev/null +++ b/fdbcli/TenantCommands.actor.cpp @@ -0,0 +1,143 @@ +/* + * TenantCommands.actor.cpp + * + * This source file is part of the FoundationDB open source project + * + * Copyright 2013-2022 Apple Inc. and the FoundationDB project authors + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +#include "fdbcli/fdbcli.actor.h" + +#include "fdbclient/FDBOptions.g.h" +#include "fdbclient/IClientApi.h" +#include "fdbclient/Knobs.h" +#include "fdbclient/ManagementAPI.actor.h" +#include "fdbclient/Schemas.h" + +#include "flow/Arena.h" +#include "flow/FastRef.h" +#include "flow/ThreadHelper.actor.h" +#include "flow/actorcompiler.h" // This must be the last #include. + +namespace fdb_cli { +// createtenant command +ACTOR Future createTenantCommandActor(Reference db, std::vector tokens) { + if (tokens.size() != 2) { + printUsage(tokens[0]); + return false; + } + + wait(ManagementAPI::createTenant(db, tokens[1])); + printf("The tenant `%s' has been created.\n", printable(tokens[1]).c_str()); + return true; +} + +CommandFactory createTenantFactory("createtenant", + CommandHelp("createtenant ", + "creates a new tenant in the cluster", + "Creates a new tenant in the cluster with the specified name.")); + +// deletetenant command +ACTOR Future deleteTenantCommandActor(Reference db, std::vector tokens) { + if (tokens.size() != 2) { + printUsage(tokens[0]); + return false; + } + + wait(ManagementAPI::deleteTenant(db, tokens[1])); + printf("The tenant `%s' has been deleted.\n", printable(tokens[1]).c_str()); + return true; +} + +CommandFactory deleteTenantFactory( + "deletetenant", + CommandHelp( + "deletetenant ", + "deletes a tenant from the cluster", + "Deletes a tenant from the cluster. Deletion will be allowed only if the specified tenant contains no data.")); + +// listtenants command +ACTOR Future listTenantsCommandActor(Reference db, std::vector tokens) { + if (tokens.size() > 4) { + printUsage(tokens[0]); + return false; + } + + StringRef begin = ""_sr; + StringRef end = "\xff\xff"_sr; + state int limit = 100; + + if (tokens.size() >= 2) { + begin = tokens[1]; + } + if (tokens.size() >= 3) { + end = tokens[2]; + if (end <= begin) { + fprintf(stderr, "ERROR: end must be larger than begin"); + return false; + } + } + if (tokens.size() == 4) { + int n = 0; + if (sscanf(tokens[3].toString().c_str(), "%d%n", &limit, &n) != 1 || n != tokens[4].size()) { + fprintf(stderr, "ERROR: invalid limit %s\n", tokens[3].toString().c_str()); + return false; + } + } + + Standalone> tenants = wait(ManagementAPI::listTenants(db, begin, end, limit)); + + if (tenants.empty()) { + if (tokens.size() == 1) { + printf("The cluster has no tenants.\n"); + } else { + printf("The cluster has no tenants in the specified range.\n"); + } + } + + int index = 0; + for (auto tenant : tenants) { + printf(" %d. %s\n", ++index, printable(tenant).c_str()); + } + + return true; +} + +CommandFactory listTenantsFactory( + "listtenants", + CommandHelp("listtenants [BEGIN] [END] [LIMIT]", + "print a list of tenants in the cluster", + "Print a list of tenants in the cluster. Only tenants in the range [BEGIN] - [END] will be printed. " + "The number of tenants to print can be specified using the [LIMIT] parameter, which defaults to 100.")); + +// gettenant command +ACTOR Future getTenantCommandActor(Reference db, std::vector tokens) { + if (tokens.size() != 2) { + printUsage(tokens[0]); + return false; + } + + TenantMapEntry tenant = wait(ManagementAPI::getTenant(db, tokens[1])); + printf(" ID: %" PRId64 "\n", tenant.id); + printf(" Prefix: %s\n", printable(tenant.prefix).c_str()); + + return true; +} + +CommandFactory getTenantFactory("gettenant", + CommandHelp("gettenant ", + "prints the metadata for a tenant", + "Prints the metadata for a tenant.")); +} // namespace fdb_cli diff --git a/fdbcli/fdbcli.actor.cpp b/fdbcli/fdbcli.actor.cpp index 069692346f..ad09b13a97 100644 --- a/fdbcli/fdbcli.actor.cpp +++ b/fdbcli/fdbcli.actor.cpp @@ -55,6 +55,7 @@ #include "fdbcli/fdbcli.actor.h" #include +#include #include #include @@ -530,6 +531,19 @@ void initHelp() { helpMap["writemode"] = CommandHelp("writemode ", "enables or disables sets and clears", "Setting or clearing keys from the CLI is not recommended."); + helpMap["usetenant"] = + CommandHelp("usetenant [NAME]", + "prints or configures the tenant used for transactions", + "If no name is given, prints the name of the current active tenant. Otherwise, all commands that " + "are used to read or write keys are done inside the tenant with the specified NAME. By default, " + "no tenant is configured and operations are performed on the raw key-space. The tenant cannot be " + "configured while a transaction started with `begin' is open."); + helpMap["defaulttenant"] = + CommandHelp("defaulttenant", + "configures transactions to not use a named tenant", + "All commands that are used to read or write keys will be done without a tenant and will operate " + "on the raw key-space. This is the default behavior. The tenant cannot be configured while a " + "transaction started with `begin' is open."); } void printVersion() { @@ -670,13 +684,18 @@ ACTOR Future createSnapshot(Database db, std::vector tokens) { // TODO: Update the function to get rid of the Database after refactoring Reference getTransaction(Reference db, + Reference tenant, Reference& tr, FdbOptions* options, bool intrans) { // Update "tr" to point to a brand new transaction object when it's not initialized or "intrans" flag is "false", // which indicates we need a new transaction object if (!tr || !intrans) { - tr = db->createTransaction(); + if (tenant) { + tr = tenant->createTransaction(); + } else { + tr = db->createTransaction(); + } options->apply(tr); } @@ -769,7 +788,8 @@ void configureGenerator(const char* text, const char* line, std::vectorgetReadVersion())))); + wait(makeInterruptable(success(safeThreadFutureToFuture( + getTransaction(db, managementTenant, tr, options, intrans)->getReadVersion())))); continue; } @@ -1478,14 +1505,14 @@ ACTOR Future cli(CLIOptions opt, LineNoise* plinenoise) { } if (tokencmp(tokens[0], "changefeed")) { - bool _result = wait(makeInterruptable(changeFeedCommandActor(localDb, tokens, warn))); + bool _result = wait(makeInterruptable(changeFeedCommandActor(localDb, tenantEntry, tokens, warn))); if (!_result) is_error = true; continue; } if (tokencmp(tokens[0], "blobrange")) { - bool _result = wait(makeInterruptable(blobRangeCommandActor(localDb, tokens))); + bool _result = wait(makeInterruptable(blobRangeCommandActor(localDb, tenantEntry, tokens))); if (!_result) is_error = true; continue; @@ -1536,7 +1563,7 @@ ACTOR Future cli(CLIOptions opt, LineNoise* plinenoise) { } else { activeOptions = FdbOptions(globalOptions); options = &activeOptions; - getTransaction(db, tr, options, false); + getTransaction(db, tenant, tr, options, false); intrans = true; printf("Transaction started\n"); } @@ -1597,7 +1624,7 @@ ACTOR Future cli(CLIOptions opt, LineNoise* plinenoise) { is_error = true; } else { state ThreadFuture> valueF = - getTransaction(db, tr, options, intrans)->get(tokens[1]); + getTransaction(db, tenant, tr, options, intrans)->get(tokens[1]); Optional> v = wait(makeInterruptable(safeThreadFutureToFuture(valueF))); if (v.present()) @@ -1613,8 +1640,8 @@ ACTOR Future cli(CLIOptions opt, LineNoise* plinenoise) { printUsage(tokens[0]); is_error = true; } else { - Version v = wait(makeInterruptable( - safeThreadFutureToFuture(getTransaction(db, tr, options, intrans)->getReadVersion()))); + Version v = wait(makeInterruptable(safeThreadFutureToFuture( + getTransaction(db, tenant, tr, options, intrans)->getReadVersion()))); fmt::print("{}\n", v); } continue; @@ -1628,7 +1655,7 @@ ACTOR Future cli(CLIOptions opt, LineNoise* plinenoise) { } if (tokencmp(tokens[0], "kill")) { - getTransaction(db, tr, options, intrans); + getTransaction(db, managementTenant, tr, options, intrans); bool _result = wait(makeInterruptable(killCommandActor(db, tr, tokens, &address_interface))); if (!_result) is_error = true; @@ -1636,7 +1663,7 @@ ACTOR Future cli(CLIOptions opt, LineNoise* plinenoise) { } if (tokencmp(tokens[0], "suspend")) { - getTransaction(db, tr, options, intrans); + getTransaction(db, managementTenant, tr, options, intrans); bool _result = wait(makeInterruptable(suspendCommandActor(db, tr, tokens, &address_interface))); if (!_result) is_error = true; @@ -1658,7 +1685,7 @@ ACTOR Future cli(CLIOptions opt, LineNoise* plinenoise) { } if (tokencmp(tokens[0], "consistencycheck")) { - getTransaction(db, tr, options, intrans); + getTransaction(db, managementTenant, tr, options, intrans); bool _result = wait(makeInterruptable(consistencyCheckCommandActor(tr, tokens, intrans))); if (!_result) is_error = true; @@ -1666,7 +1693,7 @@ ACTOR Future cli(CLIOptions opt, LineNoise* plinenoise) { } if (tokencmp(tokens[0], "profile")) { - getTransaction(db, tr, options, intrans); + getTransaction(db, managementTenant, tr, options, intrans); bool _result = wait(makeInterruptable(profileCommandActor(tr, tokens, intrans))); if (!_result) is_error = true; @@ -1674,7 +1701,7 @@ ACTOR Future cli(CLIOptions opt, LineNoise* plinenoise) { } if (tokencmp(tokens[0], "expensive_data_check")) { - getTransaction(db, tr, options, intrans); + getTransaction(db, managementTenant, tr, options, intrans); bool _result = wait(makeInterruptable(expensiveDataCheckCommandActor(db, tr, tokens, &address_interface))); if (!_result) @@ -1734,8 +1761,8 @@ ACTOR Future cli(CLIOptions opt, LineNoise* plinenoise) { endKey = strinc(tokens[1]); } - state ThreadFuture kvsF = - getTransaction(db, tr, options, intrans)->getRange(KeyRangeRef(tokens[1], endKey), limit); + getTransaction(db, tenant, tr, options, intrans); + state ThreadFuture kvsF = tr->getRange(KeyRangeRef(tokens[1], endKey), limit); RangeResult kvs = wait(makeInterruptable(safeThreadFutureToFuture(kvsF))); printf("\nRange limited to %d keys\n", limit); @@ -1779,7 +1806,7 @@ ACTOR Future cli(CLIOptions opt, LineNoise* plinenoise) { printUsage(tokens[0]); is_error = true; } else { - getTransaction(db, tr, options, intrans); + getTransaction(db, tenant, tr, options, intrans); tr->set(tokens[1], tokens[2]); if (!intrans) { @@ -1800,7 +1827,7 @@ ACTOR Future cli(CLIOptions opt, LineNoise* plinenoise) { printUsage(tokens[0]); is_error = true; } else { - getTransaction(db, tr, options, intrans); + getTransaction(db, tenant, tr, options, intrans); tr->clear(tokens[1]); if (!intrans) { @@ -1821,7 +1848,7 @@ ACTOR Future cli(CLIOptions opt, LineNoise* plinenoise) { printUsage(tokens[0]); is_error = true; } else { - getTransaction(db, tr, options, intrans); + getTransaction(db, tenant, tr, options, intrans); tr->clear(KeyRangeRef(tokens[1], tokens[2])); if (!intrans) { @@ -1910,6 +1937,81 @@ ACTOR Future cli(CLIOptions opt, LineNoise* plinenoise) { continue; } + if (tokencmp(tokens[0], "usetenant")) { + if (tokens.size() > 2) { + printUsage(tokens[0]); + is_error = true; + } else if (intrans && tokens.size() == 2) { + fprintf(stderr, "ERROR: Tenant cannot be changed while a transaction is open\n"); + is_error = true; + } else if (tokens.size() == 1) { + if (!tenantName.present()) { + printf("Using the default tenant\n"); + } else { + printf("Using tenant `%s'\n", printable(tenantName.get()).c_str()); + } + } else { + Optional entry = + wait(makeInterruptable(ManagementAPI::tryGetTenant(db, tokens[1]))); + if (!entry.present()) { + fprintf(stderr, "ERROR: Tenant `%s' does not exist\n", printable(tokens[1]).c_str()); + is_error = true; + } else { + tenant = db->openTenant(tokens[1]); + tenantName = tokens[1]; + tenantEntry = entry; + printf("Using tenant `%s'\n", printable(tokens[1]).c_str()); + } + } + + continue; + } + + if (tokencmp(tokens[0], "defaulttenant")) { + if (tokens.size() != 1) { + printUsage(tokens[0]); + is_error = true; + } else if (intrans) { + fprintf(stderr, "ERROR: Tenant cannot be changed while a transaction is open\n"); + is_error = true; + } else { + tenant = Reference(); + tenantName = Optional>(); + tenantEntry = Optional(); + printf("Using the default tenant\n"); + } + + continue; + } + + if (tokencmp(tokens[0], "createtenant")) { + bool _result = wait(makeInterruptable(createTenantCommandActor(db, tokens))); + if (!_result) + is_error = true; + continue; + } + + if (tokencmp(tokens[0], "deletetenant")) { + bool _result = wait(makeInterruptable(deleteTenantCommandActor(db, tokens))); + if (!_result) + is_error = true; + continue; + } + + if (tokencmp(tokens[0], "listtenants")) { + bool _result = wait(makeInterruptable(listTenantsCommandActor(db, tokens))); + if (!_result) + is_error = true; + continue; + } + + if (tokencmp(tokens[0], "gettenant")) { + bool _result = wait(makeInterruptable(getTenantCommandActor(db, tokens))); + if (!_result) + is_error = true; + continue; + } + fprintf(stderr, "ERROR: Unknown command `%s'. Try `help'?\n", formatStringRef(tokens[0]).c_str()); is_error = true; } @@ -1917,8 +2019,14 @@ ACTOR Future cli(CLIOptions opt, LineNoise* plinenoise) { TraceEvent(SevInfo, "CLICommandLog", randomID).detail("Command", line).detail("IsError", is_error); } catch (Error& e) { - if (e.code() != error_code_actor_cancelled) + if (e.code() == error_code_tenant_name_required) { + printAtCol("ERROR: tenant name required. Use the `usetenant' command to select a tenant or enable the " + "`RAW_ACCESS' option to read raw keys.", + 80, + stderr); + } else if (e.code() != error_code_actor_cancelled) { fprintf(stderr, "ERROR: %s (%d)\n", e.what(), e.code()); + } is_error = true; if (intrans) { printf("Rolling back current transaction\n"); diff --git a/fdbcli/fdbcli.actor.h b/fdbcli/fdbcli.actor.h index 0a8cbf8796..afd48bcb3c 100644 --- a/fdbcli/fdbcli.actor.h +++ b/fdbcli/fdbcli.actor.h @@ -138,8 +138,12 @@ ACTOR Future consistencyCheckCommandActor(Reference tr, bool intrans); // coordinators command ACTOR Future coordinatorsCommandActor(Reference db, std::vector tokens); +// createtenant command +ACTOR Future createTenantCommandActor(Reference db, std::vector tokens); // datadistribution command ACTOR Future dataDistributionCommandActor(Reference db, std::vector tokens); +// deletetenant command +ACTOR Future deleteTenantCommandActor(Reference db, std::vector tokens); // exclude command ACTOR Future excludeCommandActor(Reference db, std::vector tokens, Future warn); // expensive_data_check command @@ -155,6 +159,8 @@ ACTOR Future fileConfigureCommandActor(Reference db, bool force); // force_recovery_with_data_loss command ACTOR Future forceRecoveryWithDataLossCommandActor(Reference db, std::vector tokens); +// gettenant command +ACTOR Future getTenantCommandActor(Reference db, std::vector tokens); // include command ACTOR Future includeCommandActor(Reference db, std::vector tokens); // kill command @@ -162,13 +168,20 @@ ACTOR Future killCommandActor(Reference db, Reference tr, std::vector tokens, std::map>* address_interface); +// listtenants command +ACTOR Future listTenantsCommandActor(Reference db, std::vector tokens); // lock/unlock command ACTOR Future lockCommandActor(Reference db, std::vector tokens); ACTOR Future unlockDatabaseActor(Reference db, UID uid); // changefeed command -ACTOR Future changeFeedCommandActor(Database localDb, std::vector tokens, Future warn); +ACTOR Future changeFeedCommandActor(Database localDb, + Optional tenantEntry, + std::vector tokens, + Future warn); // blobrange command -ACTOR Future blobRangeCommandActor(Database localDb, std::vector tokens); +ACTOR Future blobRangeCommandActor(Database localDb, + Optional tenantEntry, + std::vector tokens); // maintenance command ACTOR Future setHealthyZone(Reference db, StringRef zoneId, double seconds, bool printWarning = false); ACTOR Future clearHealthyZone(Reference db, From 4aa053d3424aff8f4de043db005055dda967d6e0 Mon Sep 17 00:00:00 2001 From: "A.J. Beamon" Date: Thu, 3 Mar 2022 14:47:50 -0800 Subject: [PATCH 362/413] Update tenant mode options in fdbcli --- fdbcli/ConfigureCommand.actor.cpp | 8 +++++--- 1 file changed, 5 insertions(+), 3 deletions(-) diff --git a/fdbcli/ConfigureCommand.actor.cpp b/fdbcli/ConfigureCommand.actor.cpp index 30b38888d7..03368a6f9e 100644 --- a/fdbcli/ConfigureCommand.actor.cpp +++ b/fdbcli/ConfigureCommand.actor.cpp @@ -265,7 +265,7 @@ CommandFactory configureFactory( "commit_proxies=|grv_proxies=|logs=|resolvers=>*|" "count=|perpetual_storage_wiggle=|perpetual_storage_wiggle_locality=" "<:|0>|storage_migration_type={disabled|gradual|aggressive}" - "|tenant_mode={optional|required}", + "|tenant_mode={disabled|optional_experimental|required_experimental}", "change the database configuration", "The `new' option, if present, initializes a new database with the given configuration rather than changing " "the configuration of an existing one. When used, both a redundancy mode and a storage engine must be " @@ -296,8 +296,10 @@ CommandFactory configureFactory( "perpetual_storage_wiggle_locality=<:|0>: Set the process filter for wiggling. " "The processes that match the given locality key and locality value are only wiggled. The value 0 will disable " "the locality filter and matches all the processes for wiggling.\n\n" - "tenant_mode=: Sets the tenant mode for the cluster. If optional, then transactions can be " - "run with or without specifying tenants. If required, all data must be accessed using tenants.\n\n" + "tenant_mode=: Sets the tenant mode for the cluster. If " + "optional, then transactions can be run with or without specifying tenants. If required, all data must be " + "accessed using tenants.\n\n" + "See the FoundationDB Administration Guide for more information.")); } // namespace fdb_cli From bd64781ad9b5904f144e401f19e282307629ca68 Mon Sep 17 00:00:00 2001 From: "A.J. Beamon" Date: Sun, 6 Mar 2022 21:22:44 -0800 Subject: [PATCH 363/413] Use special keys to manage tenants --- fdbcli/TenantCommands.actor.cpp | 144 ++++++++++++++++++++++++++------ 1 file changed, 120 insertions(+), 24 deletions(-) diff --git a/fdbcli/TenantCommands.actor.cpp b/fdbcli/TenantCommands.actor.cpp index fb9e051b52..b69ea8ebde 100644 --- a/fdbcli/TenantCommands.actor.cpp +++ b/fdbcli/TenantCommands.actor.cpp @@ -32,6 +32,10 @@ #include "flow/actorcompiler.h" // This must be the last #include. namespace fdb_cli { + +const KeyRangeRef tenantSpecialKeyRange(LiteralStringRef("\xff\xff/management/tenant_map/"), + LiteralStringRef("\xff\xff/management/tenant_map0")); + // createtenant command ACTOR Future createTenantCommandActor(Reference db, std::vector tokens) { if (tokens.size() != 2) { @@ -39,7 +43,35 @@ ACTOR Future createTenantCommandActor(Reference db, std::vector return false; } - wait(ManagementAPI::createTenant(db, tokens[1])); + state Key tenantNameKey = fdb_cli::tenantSpecialKeyRange.begin.withSuffix(tokens[1]); + state Reference tr = db->createTransaction(); + state bool doneExistenceCheck = false; + + loop { + tr->setOption(FDBTransactionOptions::SPECIAL_KEY_SPACE_ENABLE_WRITES); + try { + if (!doneExistenceCheck) { + Optional existingTenant = wait(safeThreadFutureToFuture(tr->get(tenantNameKey))); + if (existingTenant.present()) { + throw tenant_already_exists(); + } + doneExistenceCheck = true; + } + + tr->set(tenantNameKey, ValueRef()); + wait(safeThreadFutureToFuture(tr->commit())); + break; + } catch (Error& e) { + state Error err(e); + if (e.code() == error_code_special_keys_api_failure) { + std::string errorMsgStr = wait(fdb_cli::getSpecialKeysFailureErrorMessage(tr)); + fprintf(stderr, "ERROR: %s\n", errorMsgStr.c_str()); + return false; + } + wait(safeThreadFutureToFuture(tr->onError(err))); + } + } + printf("The tenant `%s' has been created.\n", printable(tokens[1]).c_str()); return true; } @@ -56,7 +88,35 @@ ACTOR Future deleteTenantCommandActor(Reference db, std::vector return false; } - wait(ManagementAPI::deleteTenant(db, tokens[1])); + state Key tenantNameKey = fdb_cli::tenantSpecialKeyRange.begin.withSuffix(tokens[1]); + state Reference tr = db->createTransaction(); + state bool doneExistenceCheck = false; + + loop { + tr->setOption(FDBTransactionOptions::SPECIAL_KEY_SPACE_ENABLE_WRITES); + try { + if (!doneExistenceCheck) { + Optional existingTenant = wait(safeThreadFutureToFuture(tr->get(tenantNameKey))); + if (!existingTenant.present()) { + throw tenant_not_found(); + } + doneExistenceCheck = true; + } + + tr->clear(tenantNameKey); + wait(safeThreadFutureToFuture(tr->commit())); + break; + } catch (Error& e) { + state Error err(e); + if (e.code() == error_code_special_keys_api_failure) { + std::string errorMsgStr = wait(fdb_cli::getSpecialKeysFailureErrorMessage(tr)); + fprintf(stderr, "ERROR: %s\n", errorMsgStr.c_str()); + return false; + } + wait(safeThreadFutureToFuture(tr->onError(err))); + } + } + printf("The tenant `%s' has been deleted.\n", printable(tokens[1]).c_str()); return true; } @@ -75,16 +135,16 @@ ACTOR Future listTenantsCommandActor(Reference db, std::vector< return false; } - StringRef begin = ""_sr; - StringRef end = "\xff\xff"_sr; + StringRef beginTenant = ""_sr; + StringRef endTenant = "\xff\xff"_sr; state int limit = 100; if (tokens.size() >= 2) { - begin = tokens[1]; + beginTenant = tokens[1]; } if (tokens.size() >= 3) { - end = tokens[2]; - if (end <= begin) { + endTenant = tokens[2]; + if (endTenant <= beginTenant) { fprintf(stderr, "ERROR: end must be larger than begin"); return false; } @@ -97,22 +157,41 @@ ACTOR Future listTenantsCommandActor(Reference db, std::vector< } } - Standalone> tenants = wait(ManagementAPI::listTenants(db, begin, end, limit)); + state Key beginTenantKey = fdb_cli::tenantSpecialKeyRange.begin.withSuffix(beginTenant); + state Key endTenantKey = fdb_cli::tenantSpecialKeyRange.begin.withSuffix(endTenant); + state Reference tr = db->createTransaction(); - if (tenants.empty()) { - if (tokens.size() == 1) { - printf("The cluster has no tenants.\n"); - } else { - printf("The cluster has no tenants in the specified range.\n"); + loop { + try { + RangeResult tenants = wait(safeThreadFutureToFuture( + tr->getRange(firstGreaterOrEqual(beginTenantKey), firstGreaterOrEqual(endTenantKey), limit))); + + if (tenants.empty()) { + if (tokens.size() == 1) { + printf("The cluster has no tenants.\n"); + } else { + printf("The cluster has no tenants in the specified range.\n"); + } + } + + int index = 0; + for (auto tenant : tenants) { + printf(" %d. %s\n", + ++index, + printable(tenant.key.removePrefix(fdb_cli::tenantSpecialKeyRange.begin)).c_str()); + } + + return true; + } catch (Error& e) { + state Error err(e); + if (e.code() == error_code_special_keys_api_failure) { + std::string errorMsgStr = wait(fdb_cli::getSpecialKeysFailureErrorMessage(tr)); + fprintf(stderr, "ERROR: %s\n", errorMsgStr.c_str()); + return false; + } + wait(safeThreadFutureToFuture(tr->onError(err))); } } - - int index = 0; - for (auto tenant : tenants) { - printf(" %d. %s\n", ++index, printable(tenant).c_str()); - } - - return true; } CommandFactory listTenantsFactory( @@ -129,11 +208,28 @@ ACTOR Future getTenantCommandActor(Reference db, std::vector tr = db->createTransaction(); - return true; + loop { + try { + Optional tenant = wait(safeThreadFutureToFuture(tr->get(tenantNameKey))); + if (!tenant.present()) { + throw tenant_not_found(); + } + + printf(" %s\n", tenant.get().toString().c_str()); + return true; + } catch (Error& e) { + state Error err(e); + if (e.code() == error_code_special_keys_api_failure) { + std::string errorMsgStr = wait(fdb_cli::getSpecialKeysFailureErrorMessage(tr)); + fprintf(stderr, "ERROR: %s\n", errorMsgStr.c_str()); + return false; + } + wait(safeThreadFutureToFuture(tr->onError(err))); + } + } } CommandFactory getTenantFactory("gettenant", From a23add6bc45c372f286083feb7f33f0d4aa5dbfb Mon Sep 17 00:00:00 2001 From: "A.J. Beamon" Date: Mon, 14 Mar 2022 16:04:23 -0700 Subject: [PATCH 364/413] Add fdbcli test for tenants. Add documentation for new fdbcli tenant commands. Various output cleanup. Fix limit parsing bug in listtenants command. Update gettenant output format. --- bindings/python/tests/fdbcli_tests.py | 116 ++++++++++++++++++ .../sphinx/source/command-line-interface.rst | 46 +++++++ fdbcli/TenantCommands.actor.cpp | 22 +++- fdbcli/fdbcli.actor.cpp | 5 + tests/TestRunner/local_cluster.py | 2 +- 5 files changed, 184 insertions(+), 7 deletions(-) diff --git a/bindings/python/tests/fdbcli_tests.py b/bindings/python/tests/fdbcli_tests.py index bb2746df13..29ac474eff 100755 --- a/bindings/python/tests/fdbcli_tests.py +++ b/bindings/python/tests/fdbcli_tests.py @@ -542,6 +542,121 @@ def triggerddteaminfolog(logger): output = run_fdbcli_command('triggerddteaminfolog') assert output == 'Triggered team info logging in data distribution.' +@enable_logging() +def tenants(logger): + output = run_fdbcli_command('listtenants') + print(output) + assert output == 'The cluster has no tenants' + + output = run_fdbcli_command('createtenant tenant') + print(output) + assert output == 'The tenant `tenant\' has been created' + + output = run_fdbcli_command('createtenant tenant2') + print(output) + assert output == 'The tenant `tenant2\' has been created' + + output = run_fdbcli_command('listtenants') + print(output) + assert output == '1. tenant\n 2. tenant2' + + output = run_fdbcli_command('listtenants a z 1') + print(output) + assert output == '1. tenant' + + output = run_fdbcli_command('listtenants a tenant2') + print(output) + assert output == '1. tenant' + + output = run_fdbcli_command('listtenants tenant2 z') + print(output) + assert output == '1. tenant2' + + output = run_fdbcli_command('gettenant tenant') + print(output) + lines = output.split('\n') + assert len(lines) == 2 + assert lines[0].strip().startswith('id: ') + assert lines[1].strip().startswith('prefix: ') + + output = run_fdbcli_command('usetenant') + print(output) + assert output == 'Using the default tenant' + + output = run_fdbcli_command_and_get_error('usetenant tenant3') + print(output) + assert output == 'ERROR: Tenant `tenant3\' does not exist' + + # Test writing keys to different tenants and make sure they all work correctly + run_fdbcli_command('writemode on; set tenant_test default_tenant') + output = run_fdbcli_command('get tenant_test') + print(output) + assert output == '`tenant_test\' is `default_tenant\'' + + process = subprocess.Popen(command_template[:-1], stdin=subprocess.PIPE, stdout=subprocess.PIPE, env=fdbcli_env) + cmd_sequence = ['writemode on', 'usetenant tenant', 'get tenant_test', 'set tenant_test tenant'] + output, _ = process.communicate(input='\n'.join(cmd_sequence).encode()) + + lines = output.decode().strip().split('\n')[-3:] + print(lines) + assert lines[0] == 'Using tenant `tenant\'' + assert lines[1] == '`tenant_test\': not found' + assert lines[2].startswith('Committed') + + process = subprocess.Popen(command_template[:-1], stdin=subprocess.PIPE, stdout=subprocess.PIPE, env=fdbcli_env) + cmd_sequence = ['writemode on', 'usetenant tenant2', 'get tenant_test', 'set tenant_test tenant2', 'get tenant_test'] + output, _ = process.communicate(input='\n'.join(cmd_sequence).encode()) + + lines = output.decode().strip().split('\n')[-4:] + print(lines) + assert lines[0] == 'Using tenant `tenant2\'' + assert lines[1] == '`tenant_test\': not found' + assert lines[2].startswith('Committed') + assert lines[3] == '`tenant_test\' is `tenant2\'' + + process = subprocess.Popen(command_template[:-1], stdin=subprocess.PIPE, stdout=subprocess.PIPE, env=fdbcli_env) + cmd_sequence = ['usetenant tenant', 'get tenant_test', 'defaulttenant', 'get tenant_test'] + output, _ = process.communicate(input='\n'.join(cmd_sequence).encode()) + + lines = output.decode().strip().split('\n')[-4:] + print(lines) + assert lines[0] == 'Using tenant `tenant\'' + assert lines[1] == '`tenant_test\' is `tenant\'' + assert lines[2] == 'Using the default tenant' + assert lines[3] == '`tenant_test\' is `default_tenant\'' + + process = subprocess.Popen(command_template[:-1], stdin=subprocess.PIPE, stdout=subprocess.PIPE, stderr=subprocess.PIPE, env=fdbcli_env) + cmd_sequence = ['writemode on', 'usetenant tenant', 'clear tenant_test', 'deletetenant tenant', 'get tenant_test', 'defaulttenant', 'usetenant tenant'] + output, error_output = process.communicate(input='\n'.join(cmd_sequence).encode()) + + lines = output.decode().strip().split('\n')[-7:] + error_lines = error_output.decode().strip().split('\n')[-2:] + print(lines) + print(error_lines) + assert lines[0] == 'Using tenant `tenant\'' + assert lines[1].startswith('Committed') + assert lines[2] == 'The tenant `tenant\' has been deleted' + assert lines[3] == 'WARNING: the active tenant was deleted. Use the `usetenant\' or `defaulttenant\'' + assert lines[4] == 'command to choose a new tenant.' + assert error_lines[0] == 'ERROR: Tenant does not exist (2131)' + assert lines[6] == 'Using the default tenant' + assert error_lines[1] == 'ERROR: Tenant `tenant\' does not exist' + + process = subprocess.Popen(command_template[:-1], stdin=subprocess.PIPE, stdout=subprocess.PIPE, stderr=subprocess.PIPE, env=fdbcli_env) + cmd_sequence = ['writemode on', 'deletetenant tenant2', 'usetenant tenant2', 'clear tenant_test', 'defaulttenant', 'deletetenant tenant2'] + output, error_output = process.communicate(input='\n'.join(cmd_sequence).encode()) + + lines = output.decode().strip().split('\n')[-4:] + error_lines = error_output.decode().strip().split('\n')[-1:] + print(lines) + print(error_lines) + assert error_lines[0] == 'ERROR: Cannot delete a non-empty tenant (2133)' + assert lines[0] == 'Using tenant `tenant2\'' + assert lines[1].startswith('Committed') + assert lines[2] == 'Using the default tenant' + assert lines[3] == 'The tenant `tenant2\' has been deleted' + + run_fdbcli_command('writemode on; clear tenant_test') if __name__ == '__main__': parser = ArgumentParser(formatter_class=RawDescriptionHelpFormatter, @@ -586,6 +701,7 @@ if __name__ == '__main__': transaction() throttle() triggerddteaminfolog() + tenants() else: assert args.process_number > 1, "Process number should be positive" coordinators() diff --git a/documentation/sphinx/source/command-line-interface.rst b/documentation/sphinx/source/command-line-interface.rst index 6d2243f02f..72d7da54ef 100644 --- a/documentation/sphinx/source/command-line-interface.rst +++ b/documentation/sphinx/source/command-line-interface.rst @@ -153,6 +153,27 @@ If ``description=`` is specified, the description field in the cluster fil For more information on setting the cluster description, see :ref:`configuration-setting-cluster-description`. +createtenant +------------ + +The ``createtenant`` command is used to create new tenants in the cluster. Its syntax is ``createtenant ``. + +The tenant name can be any byte string that does not begin with the ``\xff`` byte. If the tenant already exists, ``fdbcli`` will report an error. + +defaulttenant +------------- + +The ``defaulttenant`` command configures ``fdbcli`` to run its commands without a tenant. This is the default behavior. + +The active tenant cannot be changed while a transaction (using ``begin``) is open. + +deletetenant +------------ + +The ``deletetenant`` command is used to delete tenants from the cluster. Its syntax is ``deletetenant ``. + +In order to delete a tenant, it must be empty. To delete a tenant with data, first clear that data using the ``clear`` command. If the tenant does not exist, ``fdbcli`` will report an error. + exclude ------- @@ -210,6 +231,13 @@ The ``getrangekeys`` command fetches keys in a range. Its syntax is ``getrangeke Note that :ref:`characters can be escaped ` when specifying keys (or values) in ``fdbcli``. +gettenant +--------- + +The ``gettenant`` command fetches metadata for a given tenant and displays it. Its syntax is ``gettenant ``. + +Included in the output of this command are the ``id`` and ``prefix`` assigned to the tenant. If the tenant does not exist, ``fdbcli`` will report an error. + getversion ---------- @@ -300,6 +328,13 @@ Attempts to kill all specified processes. Each address should include the IP and Attempts to kill all known processes in the cluster. +listtenants +----------- + +The ``listtenants`` command prints the names of tenants in the cluster. Its syntax is ``listtenants [BEGIN] [END] [LIMIT]``. + +By default, the ``listtenants`` command will print up to 100 entries from the entire range of tenants. A narrower sub-range can be printed using the optional ``[BEGIN]`` and ``[END]`` parameters, and the limit can be changed by specifying an integer ``[LIMIT]`` parameter. + lock ---- @@ -512,6 +547,17 @@ unlock The ``unlock`` command unlocks the database with the specified lock UID. Because this is a potentially dangerous operation, users must copy a passphrase before the unlock command is executed. +usetenant +--------- + +The ``usetenant`` command configures ``fdbcli`` to run transactions within the specified tenant. Its syntax is ``usetenant ``. + +When configured, transactions will read and write keys from the key-space associated with the specified tenant. By default, ``fdbcli`` runs without a tenant. Management operations that modify keys (e.g. ``exclude``) will not operate within the tenant. + +If the tenant chosen does not exist, ``fdbcli`` will report an error. + +The active tenant cannot be changed while a transaction (using ``begin``) is open. + writemode --------- diff --git a/fdbcli/TenantCommands.actor.cpp b/fdbcli/TenantCommands.actor.cpp index b69ea8ebde..c03bb17c88 100644 --- a/fdbcli/TenantCommands.actor.cpp +++ b/fdbcli/TenantCommands.actor.cpp @@ -72,7 +72,7 @@ ACTOR Future createTenantCommandActor(Reference db, std::vector } } - printf("The tenant `%s' has been created.\n", printable(tokens[1]).c_str()); + printf("The tenant `%s' has been created\n", printable(tokens[1]).c_str()); return true; } @@ -117,7 +117,7 @@ ACTOR Future deleteTenantCommandActor(Reference db, std::vector } } - printf("The tenant `%s' has been deleted.\n", printable(tokens[1]).c_str()); + printf("The tenant `%s' has been deleted\n", printable(tokens[1]).c_str()); return true; } @@ -151,7 +151,7 @@ ACTOR Future listTenantsCommandActor(Reference db, std::vector< } if (tokens.size() == 4) { int n = 0; - if (sscanf(tokens[3].toString().c_str(), "%d%n", &limit, &n) != 1 || n != tokens[4].size()) { + if (sscanf(tokens[3].toString().c_str(), "%d%n", &limit, &n) != 1 || n != tokens[3].size()) { fprintf(stderr, "ERROR: invalid limit %s\n", tokens[3].toString().c_str()); return false; } @@ -168,9 +168,9 @@ ACTOR Future listTenantsCommandActor(Reference db, std::vector< if (tenants.empty()) { if (tokens.size() == 1) { - printf("The cluster has no tenants.\n"); + printf("The cluster has no tenants\n"); } else { - printf("The cluster has no tenants in the specified range.\n"); + printf("The cluster has no tenants in the specified range\n"); } } @@ -218,7 +218,17 @@ ACTOR Future getTenantCommandActor(Reference db, std::vector cli(CLIOptions opt, LineNoise* plinenoise) { bool _result = wait(makeInterruptable(deleteTenantCommandActor(db, tokens))); if (!_result) is_error = true; + else if (tenantName.present() && tokens[1] == tenantName.get()) { + printAtCol("WARNING: the active tenant was deleted. Use the `usetenant' or `defaulttenant' " + "command to choose a new tenant.\n", + 80); + } continue; } diff --git a/tests/TestRunner/local_cluster.py b/tests/TestRunner/local_cluster.py index 1f733aa4d6..30162147fe 100644 --- a/tests/TestRunner/local_cluster.py +++ b/tests/TestRunner/local_cluster.py @@ -136,5 +136,5 @@ logdir = {logdir} def create_database(self, storage='ssd'): args = [self.fdbcli_binary, '-C', self.etc.joinpath('fdb.cluster'), '--exec', - 'configure new single {}'.format(storage)] + 'configure new single {} tenant_mode=optional_experimental'.format(storage)] subprocess.run(args) From 051d09159f48d05a990c490847b20d03b3232696 Mon Sep 17 00:00:00 2001 From: Josh Slocum Date: Thu, 17 Mar 2022 14:39:52 -0500 Subject: [PATCH 365/413] Changing pops to asynchronous to remove performance bottleneck --- fdbserver/BlobWorker.actor.cpp | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/fdbserver/BlobWorker.actor.cpp b/fdbserver/BlobWorker.actor.cpp index eef6558050..739ed5360b 100644 --- a/fdbserver/BlobWorker.actor.cpp +++ b/fdbserver/BlobWorker.actor.cpp @@ -972,10 +972,10 @@ ACTOR Future handleCompletedDeltaFile(Reference bwData, } // FIXME: for a write-hot shard, we could potentially batch these and only pop the largest one after several // have completed - // FIXME: also have these be async, have each pop change feed wait on the prior one, wait on them before - // re-snapshotting Future popFuture = bwData->db->popChangeFeedMutations(cfKey, completedDeltaFile.version); - wait(popFuture); + // Do pop asynchronously + // FIXME: this means we could have permanently unpopped data in the case of blob worker failure + bwData->addActor.send(popFuture); } while (!rollbacksCompleted->empty() && completedDeltaFile.version >= rollbacksCompleted->front().second) { if (BW_DEBUG) { From a85b578d2b8c3e14091bc689a3ff703769f6e83b Mon Sep 17 00:00:00 2001 From: Josh Slocum Date: Thu, 17 Mar 2022 13:14:47 -0500 Subject: [PATCH 366/413] Added simulation validation that multiple blob maangers don't have same epoch --- fdbserver/BlobManager.actor.cpp | 14 ++++++++++++++ 1 file changed, 14 insertions(+) diff --git a/fdbserver/BlobManager.actor.cpp b/fdbserver/BlobManager.actor.cpp index 7b97199b54..28fcab7856 100644 --- a/fdbserver/BlobManager.actor.cpp +++ b/fdbserver/BlobManager.actor.cpp @@ -2678,9 +2678,23 @@ static void blobManagerExclusionSafetyCheck(Reference self, req.reply.send(reply); } +// Simulation validation that multiple blob managers aren't started with the same epoch +static std::map managerEpochsSeen; + ACTOR Future blobManager(BlobManagerInterface bmInterf, Reference const> dbInfo, int64_t epoch) { + if (g_network->isSimulated()) { + bool managerEpochAlreadySeen = managerEpochsSeen.count(epoch); + if (managerEpochAlreadySeen) { + TraceEvent(SevError, "DuplicateBlobManagersAtEpoch") + .detail("Epoch", epoch) + .detail("BMID1", bmInterf.id()) + .detail("BMID2", managerEpochsSeen.at(epoch)); + } + ASSERT(!managerEpochAlreadySeen); + managerEpochsSeen[epoch] = bmInterf.id(); + } state Reference self = makeReference(deterministicRandom()->randomUniqueID(), openDBOnServer(dbInfo, TaskPriority::DefaultEndpoint, LockAware::True), From 0f9e88572a262e73ee745f05fa1c1a1bc5fa5698 Mon Sep 17 00:00:00 2001 From: Josh Slocum Date: Thu, 17 Mar 2022 14:40:34 -0500 Subject: [PATCH 367/413] Cleaning up debugging and fixing race in blob manager recruitment --- fdbserver/BlobManager.actor.cpp | 7 +-- fdbserver/ClusterController.actor.cpp | 1 + fdbserver/worker.actor.cpp | 74 +++++++++++++++++---------- 3 files changed, 52 insertions(+), 30 deletions(-) diff --git a/fdbserver/BlobManager.actor.cpp b/fdbserver/BlobManager.actor.cpp index 28fcab7856..2b6a4da2bf 100644 --- a/fdbserver/BlobManager.actor.cpp +++ b/fdbserver/BlobManager.actor.cpp @@ -2735,17 +2735,18 @@ ACTOR Future blobManager(BlobManagerInterface bmInterf, if (BM_DEBUG) { fmt::print("BM {} exiting because it is replaced\n", self->epoch); } + TraceEvent("BlobManagerReplaced", bmInterf.id()).detail("Epoch", epoch); break; } when(HaltBlobManagerRequest req = waitNext(bmInterf.haltBlobManager.getFuture())) { req.reply.send(Void()); - TraceEvent("BlobManagerHalted", bmInterf.id()).detail("ReqID", req.requesterID); + TraceEvent("BlobManagerHalted", bmInterf.id()).detail("Epoch", epoch).detail("ReqID", req.requesterID); break; } when(state HaltBlobGranulesRequest req = waitNext(bmInterf.haltBlobGranules.getFuture())) { wait(haltBlobGranules(self)); req.reply.send(Void()); - TraceEvent("BlobGranulesHalted", bmInterf.id()).detail("ReqID", req.requesterID); + TraceEvent("BlobGranulesHalted", bmInterf.id()).detail("Epoch", epoch).detail("ReqID", req.requesterID); break; } when(BlobManagerExclusionSafetyCheckRequest exclCheckReq = @@ -2753,7 +2754,7 @@ ACTOR Future blobManager(BlobManagerInterface bmInterf, blobManagerExclusionSafetyCheck(self, exclCheckReq); } when(wait(collection)) { - TraceEvent("BlobManagerActorCollectionError"); + TraceEvent(SevError, "BlobManagerActorCollectionError"); ASSERT(false); throw internal_error(); } diff --git a/fdbserver/ClusterController.actor.cpp b/fdbserver/ClusterController.actor.cpp index 899522f229..a7169cc259 100644 --- a/fdbserver/ClusterController.actor.cpp +++ b/fdbserver/ClusterController.actor.cpp @@ -2123,6 +2123,7 @@ ACTOR Future getNextBMEpoch(ClusterControllerData* self) { tr->set(blobManagerEpochKey, blobManagerEpochValueFor(newEpoch)); wait(tr->commit()); + TraceEvent(SevDebug, "CCNextBlobManagerEpoch", self->id).detail("Epoch", newEpoch); return newEpoch; } catch (Error& e) { wait(tr->onError(e)); diff --git a/fdbserver/worker.actor.cpp b/fdbserver/worker.actor.cpp index ff6adde754..913ed139fb 100644 --- a/fdbserver/worker.actor.cpp +++ b/fdbserver/worker.actor.cpp @@ -524,20 +524,21 @@ std::vector getDiskStores(std::string folder) { // Register the worker interf to cluster controller (cc) and // re-register the worker when key roles interface, e.g., cc, dd, ratekeeper, change. -ACTOR Future registrationClient(Reference> const> ccInterface, - WorkerInterface interf, - Reference> asyncPriorityInfo, - ProcessClass initialClass, - Reference> const> ddInterf, - Reference> const> rkInterf, - Reference> const> bmInterf, - Reference> const> ekpInterf, - Reference const> degraded, - Reference connRecord, - Reference> const> issues, - Reference configNode, - Reference localConfig, - Reference> dbInfo) { +ACTOR Future registrationClient( + Reference> const> ccInterface, + WorkerInterface interf, + Reference> asyncPriorityInfo, + ProcessClass initialClass, + Reference> const> ddInterf, + Reference> const> rkInterf, + Reference>> const> bmInterf, + Reference> const> ekpInterf, + Reference const> degraded, + Reference connRecord, + Reference> const> issues, + Reference configNode, + Reference localConfig, + Reference> dbInfo) { // 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. @@ -567,7 +568,8 @@ ACTOR Future registrationClient(Referenceget(), rkInterf->get(), - bmInterf->get(), + bmInterf->get().present() ? bmInterf->get().get().second + : Optional(), ekpInterf->get(), degraded->get(), localConfig->lastSeenVersion(), @@ -1374,6 +1376,24 @@ ACTOR Future chaosMetricsLogger() { } } +// like genericactors setWhenDoneOrError, but we need to take into account the bm epoch. We don't want to reset it if +// this manager was replaced by a later manager (with a higher epoch) on this worker +ACTOR Future resetBlobManagerWhenDoneOrError( + Future blobManagerProcess, + Reference>>> var, + int64_t epoch) { + try { + wait(blobManagerProcess); + } catch (Error& e) { + if (e.code() == error_code_actor_cancelled) + throw; + } + if (var->get().present() && var->get().get().first == epoch) { + var->set(Optional>()); + } + return Void(); +} + ACTOR Future workerServer(Reference connRecord, Reference> const> ccInterface, LocalityData locality, @@ -1395,8 +1415,8 @@ ACTOR Future workerServer(Reference connRecord, state Reference>> ddInterf( new AsyncVar>()); state Reference>> rkInterf(new AsyncVar>()); - state Reference>> bmInterf(new AsyncVar>()); - state int64_t myBMEpoch = -1; + state Reference>>> bmEpochAndInterf( + new AsyncVar>>()); state Reference>> ekpInterf( new AsyncVar>()); state Future handleErrors = workerHandleErrors(errors.getFuture()); // Needs to be stopped last @@ -1672,7 +1692,7 @@ ACTOR Future workerServer(Reference connRecord, initialClass, ddInterf, rkInterf, - bmInterf, + bmEpochAndInterf, ekpInterf, degraded, connRecord, @@ -1874,8 +1894,8 @@ ACTOR Future workerServer(Reference connRecord, BlobManagerInterface recruited(locality, req.reqId); recruited.initEndpoints(); - if (bmInterf->get().present() && myBMEpoch == req.epoch) { - recruited = bmInterf->get().get(); + if (bmEpochAndInterf->get().present() && bmEpochAndInterf->get().get().first == req.epoch) { + recruited = bmEpochAndInterf->get().get().second; TEST(true); // Recruited while already a blob manager. } else { @@ -1884,7 +1904,6 @@ ACTOR Future workerServer(Reference connRecord, // Also, not halting lets us handle the case here where the last BM had a higher // epoch and somehow the epochs got out of order by a delayed initialize request. The one we start // here will just halt on the lock check. - myBMEpoch = req.epoch; startRole(Role::BLOB_MANAGER, recruited.id(), interf.id()); DUMPTOKEN(recruited.waitFailure); DUMPTOKEN(recruited.haltBlobManager); @@ -1892,12 +1911,13 @@ ACTOR Future workerServer(Reference connRecord, DUMPTOKEN(recruited.blobManagerExclCheckReq); Future blobManagerProcess = blobManager(recruited, dbInfo, req.epoch); - errorForwarders.add(forwardError( - errors, - Role::BLOB_MANAGER, - recruited.id(), - setWhenDoneOrError(blobManagerProcess, bmInterf, Optional()))); - bmInterf->set(Optional(recruited)); + errorForwarders.add( + forwardError(errors, + Role::BLOB_MANAGER, + recruited.id(), + resetBlobManagerWhenDoneOrError(blobManagerProcess, bmEpochAndInterf, req.epoch))); + bmEpochAndInterf->set( + Optional>(std::pair(req.epoch, recruited))); } TraceEvent("BlobManagerReceived", req.reqId).detail("BlobManagerId", recruited.id()); req.reply.send(recruited); From a04ca65ad55cf45018b3f2a9a3b43f2b3ccc887c Mon Sep 17 00:00:00 2001 From: sfc-gh-tclinkenbeard Date: Thu, 17 Mar 2022 23:27:33 -0700 Subject: [PATCH 368/413] Change signature of rebalanceTeams --- fdbserver/DataDistributionQueue.actor.cpp | 17 ++++++++--------- 1 file changed, 8 insertions(+), 9 deletions(-) diff --git a/fdbserver/DataDistributionQueue.actor.cpp b/fdbserver/DataDistributionQueue.actor.cpp index f89244c5b7..812b7aa0f1 100644 --- a/fdbserver/DataDistributionQueue.actor.cpp +++ b/fdbserver/DataDistributionQueue.actor.cpp @@ -1396,12 +1396,12 @@ ACTOR Future dataDistributionRelocator(DDQueueData* self, RelocateData rd, } // Move a random shard of sourceTeam's to destTeam if sourceTeam has much more data than destTeam -ACTOR Future rebalanceTeams(DDQueueData* self, - int priority, - Reference sourceTeam, - Reference destTeam, - bool primary, - TraceEvent* traceEvent) { +ACTOR static Future rebalanceTeams(DDQueueData* self, + int priority, + Reference sourceTeam, + int64_t destBytes, + bool primary, + TraceEvent* traceEvent) { if (g_network->isSimulated() && g_simulator.speedUpSimulation) { traceEvent->detail("CancelingDueToSimulationSpeedup", true); return false; @@ -1437,7 +1437,6 @@ ACTOR Future rebalanceTeams(DDQueueData* self, } int64_t sourceBytes = sourceTeam->getLoadBytes(false); - int64_t destBytes = destTeam->getLoadBytes(); bool sourceAndDestTooSimilar = sourceBytes - destBytes <= 3 * std::max(SERVER_KNOBS->MIN_SHARD_BYTES, metrics.bytes); @@ -1530,7 +1529,7 @@ ACTOR Future BgDDMountainChopper(DDQueueData* self, int teamCollectionInde bool _moved = wait(rebalanceTeams(self, SERVER_KNOBS->PRIORITY_REBALANCE_OVERUTILIZED_TEAM, loadedTeam.first.get(), - randomTeam.first.get(), + randomTeam.first.get()->getLoadBytes(), teamCollectionIndex == 0, &traceEvent)); moved = _moved; @@ -1636,7 +1635,7 @@ ACTOR Future BgDDValleyFiller(DDQueueData* self, int teamCollectionIndex) bool _moved = wait(rebalanceTeams(self, SERVER_KNOBS->PRIORITY_REBALANCE_UNDERUTILIZED_TEAM, randomTeam.first.get(), - unloadedTeam.first.get(), + unloadedTeam.first.get()->getLoadBytes(), teamCollectionIndex == 0, &traceEvent)); moved = _moved; From bf9bf19797b5dfdd8f358846545476ace38639c3 Mon Sep 17 00:00:00 2001 From: Josh Slocum Date: Fri, 18 Mar 2022 10:30:54 -0500 Subject: [PATCH 369/413] Disabling tenants for BlobGranule tests for now --- fdbserver/SimulatedCluster.actor.cpp | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/fdbserver/SimulatedCluster.actor.cpp b/fdbserver/SimulatedCluster.actor.cpp index 5c574012eb..d9afb5ac54 100644 --- a/fdbserver/SimulatedCluster.actor.cpp +++ b/fdbserver/SimulatedCluster.actor.cpp @@ -2324,8 +2324,9 @@ ACTOR void setupAndRun(std::string dataFolder, // Disable the default tenant in backup and DR tests for now. This is because backup does not currently duplicate // the tenant map and related state. - // TODO: reenable when backup/DR supports tenants. - if (std::string_view(testFile).find("Backup") != std::string_view::npos || testConfig.extraDB != 0) { + // TODO: reenable when backup/DR or BlobGranule supports tenants. + if (std::string_view(testFile).find("Backup") != std::string_view::npos || + std::string_view(testFile).find("BlobGranule") != std::string_view::npos || testConfig.extraDB != 0) { allowDefaultTenant = false; } From 5fbdbb5c4ac917496c14affc1f41e3258c4aa846 Mon Sep 17 00:00:00 2001 From: Josh Slocum Date: Fri, 25 Feb 2022 09:33:46 -0600 Subject: [PATCH 370/413] Better cancelling logic that reflects whether move has actually started --- fdbserver/DataDistributionQueue.actor.cpp | 25 ++++++++++++++++------- 1 file changed, 18 insertions(+), 7 deletions(-) diff --git a/fdbserver/DataDistributionQueue.actor.cpp b/fdbserver/DataDistributionQueue.actor.cpp index f89244c5b7..796575ebeb 100644 --- a/fdbserver/DataDistributionQueue.actor.cpp +++ b/fdbserver/DataDistributionQueue.actor.cpp @@ -50,11 +50,12 @@ struct RelocateData { std::vector completeSources; std::vector completeDests; bool wantsNewServers; + bool cancellable; TraceInterval interval; RelocateData() : priority(-1), boundaryPriority(-1), healthPriority(-1), startTime(-1), workFactor(0), wantsNewServers(false), - interval("QueuedRelocation") {} + cancellable(false), interval("QueuedRelocation") {} explicit RelocateData(RelocateShard const& rs) : keys(rs.keys), priority(rs.priority), boundaryPriority(isBoundaryPriority(rs.priority) ? rs.priority : -1), healthPriority(isHealthPriority(rs.priority) ? rs.priority : -1), startTime(now()), @@ -63,7 +64,7 @@ struct RelocateData { rs.priority == SERVER_KNOBS->PRIORITY_REBALANCE_UNDERUTILIZED_TEAM || rs.priority == SERVER_KNOBS->PRIORITY_SPLIT_SHARD || rs.priority == SERVER_KNOBS->PRIORITY_TEAM_REDUNDANT), - interval("QueuedRelocation") {} + cancellable(true), interval("QueuedRelocation") {} static bool isHealthPriority(int priority) { return priority == SERVER_KNOBS->PRIORITY_POPULATE_REGION || @@ -610,19 +611,23 @@ struct DDQueueData { .detail( "Problem", "the key range in the inFlight map matches the key range in the RelocateData message"); + } else if (it->value().cancellable) { + TraceEvent(SevError, "DDQueueValidateError13") + .detail("Problem", "key range is cancellable but not in flight!") + .detail("Range", it->range()); } } for (auto it = busymap.begin(); it != busymap.end(); ++it) { for (int i = 0; i < it->second.ledger.size() - 1; i++) { if (it->second.ledger[i] < it->second.ledger[i + 1]) - TraceEvent(SevError, "DDQueueValidateError13") + TraceEvent(SevError, "DDQueueValidateError14") .detail("Problem", "ascending ledger problem") .detail("LedgerLevel", i) .detail("LedgerValueA", it->second.ledger[i]) .detail("LedgerValueB", it->second.ledger[i + 1]); if (it->second.ledger[i] < 0.0) - TraceEvent(SevError, "DDQueueValidateError14") + TraceEvent(SevError, "DDQueueValidateError15") .detail("Problem", "negative ascending problem") .detail("LedgerLevel", i) .detail("LedgerValue", it->second.ledger[i]); @@ -632,13 +637,13 @@ struct DDQueueData { for (auto it = destBusymap.begin(); it != destBusymap.end(); ++it) { for (int i = 0; i < it->second.ledger.size() - 1; i++) { if (it->second.ledger[i] < it->second.ledger[i + 1]) - TraceEvent(SevError, "DDQueueValidateError15") + TraceEvent(SevError, "DDQueueValidateError16") .detail("Problem", "ascending ledger problem") .detail("LedgerLevel", i) .detail("LedgerValueA", it->second.ledger[i]) .detail("LedgerValueB", it->second.ledger[i + 1]); if (it->second.ledger[i] < 0.0) - TraceEvent(SevError, "DDQueueValidateError16") + TraceEvent(SevError, "DDQueueValidateError17") .detail("Problem", "negative ascending problem") .detail("LedgerLevel", i) .detail("LedgerValue", it->second.ledger[i]); @@ -954,7 +959,7 @@ struct DDQueueData { auto containedRanges = inFlight.containedRanges(rd.keys); std::vector cancellableRelocations; for (auto it = containedRanges.begin(); it != containedRanges.end(); ++it) { - if (inFlightActors.liveActorAt(it->range().begin)) { + if (it.value().cancellable) { cancellableRelocations.push_back(it->value()); } } @@ -1180,6 +1185,12 @@ ACTOR Future dataDistributionRelocator(DDQueueData* self, RelocateData rd, // TODO different trace event + knob for overloaded? Could wait on an async var for done moves } + // set cancellable to false on inFlight's entry for this key range + auto inFlightRange = self->inFlight.rangeContaining(rd.keys.begin); + ASSERT(inFlightRange.range() == rd.keys); + ASSERT(inFlightRange.value().randomId == rd.randomId); + inFlightRange.value().cancellable = false; + destIds.clear(); state std::vector healthyIds; state std::vector extraIds; From 327ae95819ccbc29a419f1e4a50a0ff1fc27c178 Mon Sep 17 00:00:00 2001 From: Josh Slocum Date: Fri, 18 Mar 2022 11:19:00 -0500 Subject: [PATCH 371/413] fix warnings --- fdbserver/BlobWorker.actor.cpp | 55 +++++++++++++++++----------------- 1 file changed, 28 insertions(+), 27 deletions(-) diff --git a/fdbserver/BlobWorker.actor.cpp b/fdbserver/BlobWorker.actor.cpp index d1d8756d9a..dcbb458f15 100644 --- a/fdbserver/BlobWorker.actor.cpp +++ b/fdbserver/BlobWorker.actor.cpp @@ -954,12 +954,12 @@ ACTOR Future checkSplitAndReSnapshot(Reference bw return reSnapshotIdx; } -ACTOR Future handleCompletedDeltaFile(Reference bwData, - Reference metadata, - BlobFileIndex completedDeltaFile, - Key cfKey, - Version cfStartVersion, - std::deque>* rollbacksCompleted) { +static void handleCompletedDeltaFile(Reference bwData, + Reference metadata, + BlobFileIndex completedDeltaFile, + Key cfKey, + Version cfStartVersion, + std::deque>* rollbacksCompleted) { metadata->files.deltaFiles.push_back(completedDeltaFile); ASSERT(metadata->durableDeltaVersion.get() < completedDeltaFile.version); metadata->durableDeltaVersion.set(completedDeltaFile.version); @@ -972,9 +972,11 @@ ACTOR Future handleCompletedDeltaFile(Reference bwData, } // FIXME: for a write-hot shard, we could potentially batch these and only pop the largest one after several // have completed + // FIXME: we actually want to pop at this version + 1 because pop is exclusive? + // FIXME: since this is async, and worker could die, new blob worker that opens granule should probably kick off + // an async pop at its previousDurableVersion after opening the granule to guarantee it is eventually popped? Future popFuture = bwData->db->popChangeFeedMutations(cfKey, completedDeltaFile.version); // Do pop asynchronously - // FIXME: this means we could have permanently unpopped data in the case of blob worker failure bwData->addActor.send(popFuture); } while (!rollbacksCompleted->empty() && completedDeltaFile.version >= rollbacksCompleted->front().second) { @@ -989,7 +991,6 @@ ACTOR Future handleCompletedDeltaFile(Reference bwData, } rollbacksCompleted->pop_front(); } - return Void(); } // if we get an i/o error updating files, or a rollback, reassign the granule to ourselves and start fresh @@ -1368,12 +1369,12 @@ ACTOR Future blobGranuleUpdateFiles(Reference bwData, metadata->durableSnapshotVersion.set(completedFile.version); pendingSnapshots--; } else { - wait(handleCompletedDeltaFile(bwData, - metadata, - completedFile, - cfKey, - startState.changeFeedStartVersion, - &rollbacksCompleted)); + handleCompletedDeltaFile(bwData, + metadata, + completedFile, + cfKey, + startState.changeFeedStartVersion, + &rollbacksCompleted); } inFlightFiles.pop_front(); @@ -1762,12 +1763,12 @@ ACTOR Future blobGranuleUpdateFiles(Reference bwData, metadata->durableSnapshotVersion.set(completedFile.version); pendingSnapshots--; } else { - wait(handleCompletedDeltaFile(bwData, - metadata, - completedFile, - cfKey, - startState.changeFeedStartVersion, - &rollbacksCompleted)); + handleCompletedDeltaFile(bwData, + metadata, + completedFile, + cfKey, + startState.changeFeedStartVersion, + &rollbacksCompleted); } inFlightFiles.pop_front(); @@ -2020,13 +2021,13 @@ ACTOR Future waitForVersion(Reference metadata, Version v // nothing to do if (BW_REQUEST_DEBUG) { - printf("WFV %lld) CF=%lld, pendingD=%lld, durableD=%lld, pendingS=%lld, durableS=%lld\n", - v, - metadata->activeCFData.get()->getVersion(), - metadata->pendingDeltaVersion, - metadata->durableDeltaVersion.get(), - metadata->pendingSnapshotVersion, - metadata->durableSnapshotVersion.get()); + fmt::print("WFV {0}) CF={1}, pendingD={2}, durableD={3}, pendingS={4}, durableS={5}\n", + v, + metadata->activeCFData.get()->getVersion(), + metadata->pendingDeltaVersion, + metadata->durableDeltaVersion.get(), + metadata->pendingSnapshotVersion, + metadata->durableSnapshotVersion.get()); } ASSERT(metadata->activeCFData.get().isValid()); From 4a0720e8ad1ac999e512e0f93414eb60ee5b19a0 Mon Sep 17 00:00:00 2001 From: sfc-gh-tclinkenbeard Date: Fri, 18 Mar 2022 10:20:04 -0700 Subject: [PATCH 372/413] Provide destTeam parameter to rebalanceTeams again. There is a wait before the load bytes of the dest team is used, so the last commit inadvertently caused a behaviour change. Instead, update the comment for the function, and pass const IDataDistributionTeam references. --- fdbserver/DataDistributionQueue.actor.cpp | 11 ++++++----- 1 file changed, 6 insertions(+), 5 deletions(-) diff --git a/fdbserver/DataDistributionQueue.actor.cpp b/fdbserver/DataDistributionQueue.actor.cpp index 812b7aa0f1..4e7ef2f59c 100644 --- a/fdbserver/DataDistributionQueue.actor.cpp +++ b/fdbserver/DataDistributionQueue.actor.cpp @@ -1395,11 +1395,11 @@ ACTOR Future dataDistributionRelocator(DDQueueData* self, RelocateData rd, } } -// Move a random shard of sourceTeam's to destTeam if sourceTeam has much more data than destTeam +// Move a random shard from sourceTeam if sourceTeam has much more data than provided destTeam ACTOR static Future rebalanceTeams(DDQueueData* self, int priority, - Reference sourceTeam, - int64_t destBytes, + Reference sourceTeam, + Reference destTeam, bool primary, TraceEvent* traceEvent) { if (g_network->isSimulated() && g_simulator.speedUpSimulation) { @@ -1437,6 +1437,7 @@ ACTOR static Future rebalanceTeams(DDQueueData* self, } int64_t sourceBytes = sourceTeam->getLoadBytes(false); + int64_t destBytes = destTeam->getLoadBytes(); bool sourceAndDestTooSimilar = sourceBytes - destBytes <= 3 * std::max(SERVER_KNOBS->MIN_SHARD_BYTES, metrics.bytes); @@ -1529,7 +1530,7 @@ ACTOR Future BgDDMountainChopper(DDQueueData* self, int teamCollectionInde bool _moved = wait(rebalanceTeams(self, SERVER_KNOBS->PRIORITY_REBALANCE_OVERUTILIZED_TEAM, loadedTeam.first.get(), - randomTeam.first.get()->getLoadBytes(), + randomTeam.first.get(), teamCollectionIndex == 0, &traceEvent)); moved = _moved; @@ -1635,7 +1636,7 @@ ACTOR Future BgDDValleyFiller(DDQueueData* self, int teamCollectionIndex) bool _moved = wait(rebalanceTeams(self, SERVER_KNOBS->PRIORITY_REBALANCE_UNDERUTILIZED_TEAM, randomTeam.first.get(), - unloadedTeam.first.get()->getLoadBytes(), + unloadedTeam.first.get(), teamCollectionIndex == 0, &traceEvent)); moved = _moved; From 8205771e8f8cca61b1cc8cb1ab311edb1aa307da Mon Sep 17 00:00:00 2001 From: Josh Slocum Date: Fri, 18 Mar 2022 17:05:12 -0500 Subject: [PATCH 373/413] ReplyPromiseStreams need synchronous disconnect to avoid change feed races --- fdbclient/NativeAPI.actor.cpp | 2 +- fdbrpc/genericactors.actor.h | 3 +++ 2 files changed, 4 insertions(+), 1 deletion(-) diff --git a/fdbclient/NativeAPI.actor.cpp b/fdbclient/NativeAPI.actor.cpp index 081e0a8fb5..60c1430e61 100644 --- a/fdbclient/NativeAPI.actor.cpp +++ b/fdbclient/NativeAPI.actor.cpp @@ -8724,7 +8724,7 @@ ACTOR Future singleChangeFeedStream(Reference db, .detail("Range", range) .detail("Begin", *begin) .detail("End", end) - .detail("CanReadPopped", true); + .detail("CanReadPopped", canReadPopped); } results->streams.clear(); diff --git a/fdbrpc/genericactors.actor.h b/fdbrpc/genericactors.actor.h index e93825b43b..a61ea0d25e 100644 --- a/fdbrpc/genericactors.actor.h +++ b/fdbrpc/genericactors.actor.h @@ -213,6 +213,9 @@ void endStreamOnDisconnect(Future signal, try { choose { when(wait(signal)) { stream.sendError(connection_failed()); } + when(wait(peer.isValid() ? peer->disconnect.getFuture() : Never())) { + stream.sendError(connection_failed()); + } when(wait(stream.getErrorFutureAndDelPromiseRef())) {} } } catch (Error& e) { From 6be992cc3988348f81906d5eabb40867c73c3e59 Mon Sep 17 00:00:00 2001 From: Evan Tschannen Date: Fri, 18 Mar 2022 15:24:31 -0700 Subject: [PATCH 374/413] fix: do not reply to a txnStateRequest for the final sequence unless we have processes the txnStateStore --- fdbserver/CommitProxyServer.actor.cpp | 8 +++++++- 1 file changed, 7 insertions(+), 1 deletion(-) diff --git a/fdbserver/CommitProxyServer.actor.cpp b/fdbserver/CommitProxyServer.actor.cpp index 13f3729ef0..04184c430a 100644 --- a/fdbserver/CommitProxyServer.actor.cpp +++ b/fdbserver/CommitProxyServer.actor.cpp @@ -1955,6 +1955,8 @@ struct TransactionStateResolveContext { // Pointer to transaction state store, shortcut for commitData.txnStateStore IKeyValueStore* pTxnStateStore = nullptr; + Future txnRecovery; + // Actor streams PromiseStream>* pActors = nullptr; @@ -2070,6 +2072,9 @@ ACTOR Future processTransactionStateRequestPart(TransactionStateResolveCon ASSERT(pContext->pActors != nullptr); if (pContext->receivedSequences.count(request.sequence)) { + if (pContext->receivedSequences.size() == pContext->maxSequence) { + wait(pContext->txnRecovery); + } // This part is already received. Still we will re-broadcast it to other CommitProxies pContext->pActors->send(broadcastTxnRequest(request, SERVER_KNOBS->TXN_STATE_SEND_AMOUNT, true)); wait(yield()); @@ -2095,7 +2100,8 @@ ACTOR Future processTransactionStateRequestPart(TransactionStateResolveCon if (pContext->receivedSequences.size() == pContext->maxSequence) { // Received all components of the txnStateRequest ASSERT(!pContext->processed); - wait(processCompleteTransactionStateRequest(pContext)); + pContext->txnRecovery = processCompleteTransactionStateRequest(pContext); + wait(pContext->txnRecovery); pContext->processed = true; } From f7f2c9e979fe0a1722d071ee7066f52bbbbbf9ab Mon Sep 17 00:00:00 2001 From: Josh Slocum Date: Sun, 20 Mar 2022 08:13:38 -0500 Subject: [PATCH 375/413] Fixing BW erroneously asking BM to split even after it recieved a reply to continue without splitting --- fdbserver/BlobWorker.actor.cpp | 16 ++++++++++++++++ 1 file changed, 16 insertions(+) diff --git a/fdbserver/BlobWorker.actor.cpp b/fdbserver/BlobWorker.actor.cpp index dcbb458f15..5919a36dac 100644 --- a/fdbserver/BlobWorker.actor.cpp +++ b/fdbserver/BlobWorker.actor.cpp @@ -889,15 +889,22 @@ ACTOR Future checkSplitAndReSnapshot(Reference bw // FIXME: If a rollback happens, this could incorrectly identify a hot granule as not hot. This should be rare // though and is just less efficient. state bool writeHot = versionsSinceLastSnapshot <= SERVER_KNOBS->BG_HOT_SNAPSHOT_VERSIONS; + // FIXME: could probably refactor all of this logic into one large choose/when state machine that's less complex loop { loop { try { + // wait for manager stream to become ready, and send a message loop { choose { when(wait(bwData->currentManagerStatusStream.get().onReady())) { break; } when(wait(bwData->currentManagerStatusStream.onChange())) {} + when(wait(metadata->resumeSnapshot.getFuture())) { break; } } } + if (metadata->resumeSnapshot.isSet()) { + break; + } + bwData->currentManagerStatusStream.get().send(GranuleStatusReply(metadata->keyRange, true, writeHot, @@ -921,6 +928,8 @@ ACTOR Future checkSplitAndReSnapshot(Reference bw } } + // wait for manager reply (which will either cancel this future or call resumeSnapshot), or re-send on manager + // change/no response choose { when(wait(bwData->currentManagerStatusStream.onChange())) {} when(wait(metadata->resumeSnapshot.getFuture())) { break; } @@ -3061,9 +3070,16 @@ ACTOR Future blobWorker(BlobWorkerInterface bwInterf, // endpoint as failed self->currentManagerStatusStream.get().sendError(connection_failed()); + // hold a copy of the previous stream if it exists, so any waiting send calls don't get + // proken_promise before onChange + ReplyPromiseStream copy; + if (self->statusStreamInitialized) { + copy = self->currentManagerStatusStream.get(); + } // TODO: pick a reasonable byte limit instead of just piggy-backing req.reply.setByteLimit(SERVER_KNOBS->BLOBWORKERSTATUSSTREAM_LIMIT_BYTES); self->statusStreamInitialized = true; + self->currentManagerStatusStream.set(req.reply); } else { req.reply.sendError(blob_manager_replaced()); From bed6d428fb9985b0193fa22dc886f2e68384e16c Mon Sep 17 00:00:00 2001 From: Josh Slocum Date: Sun, 20 Mar 2022 14:10:52 -0500 Subject: [PATCH 376/413] Found another change feed fetch and remove race --- fdbserver/storageserver.actor.cpp | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/fdbserver/storageserver.actor.cpp b/fdbserver/storageserver.actor.cpp index 25a969f684..759e332a19 100644 --- a/fdbserver/storageserver.actor.cpp +++ b/fdbserver/storageserver.actor.cpp @@ -4817,13 +4817,14 @@ ACTOR Future fetchChangeFeedApplier(StorageServer* data, ASSERT(startVersion >= 0); - if (startVersion >= endVersion) { + if (startVersion >= endVersion || (changeFeedInfo->removing)) { TEST(true); // Change Feed popped before fetch TraceEvent(SevDebug, "FetchChangeFeedNoOp", data->thisServerID) .detail("RangeID", rangeId.printable()) .detail("Range", range.toString()) .detail("StartVersion", startVersion) - .detail("EndVersion", endVersion); + .detail("EndVersion", endVersion) + .detail("Removing", changeFeedInfo->removing); return invalidVersion; } From f4c2a789c6f09d659aeacd9bb8cabe6e415635c8 Mon Sep 17 00:00:00 2001 From: Josh Slocum Date: Mon, 21 Mar 2022 11:47:40 -0500 Subject: [PATCH 377/413] Explicitly cancelling file updater future --- fdbserver/BlobWorker.actor.cpp | 1 + 1 file changed, 1 insertion(+) diff --git a/fdbserver/BlobWorker.actor.cpp b/fdbserver/BlobWorker.actor.cpp index 5919a36dac..1e3e4232cf 100644 --- a/fdbserver/BlobWorker.actor.cpp +++ b/fdbserver/BlobWorker.actor.cpp @@ -123,6 +123,7 @@ struct GranuleRangeMetadata { activeMetadata.clear(); assignFuture.cancel(); historyLoaderFuture.cancel(); + fileUpdaterFuture.cancel(); } GranuleRangeMetadata() : lastEpoch(0), lastSeqno(0) {} From 3f7365c433b170fdb3bfd2f1c608a2e711a1587a Mon Sep 17 00:00:00 2001 From: "A.J. Beamon" Date: Mon, 21 Mar 2022 09:21:03 -0700 Subject: [PATCH 378/413] Remove test debugging lines --- bindings/python/tests/fdbcli_tests.py | 18 ------------------ 1 file changed, 18 deletions(-) diff --git a/bindings/python/tests/fdbcli_tests.py b/bindings/python/tests/fdbcli_tests.py index 29ac474eff..d24ddb876f 100755 --- a/bindings/python/tests/fdbcli_tests.py +++ b/bindings/python/tests/fdbcli_tests.py @@ -545,52 +545,41 @@ def triggerddteaminfolog(logger): @enable_logging() def tenants(logger): output = run_fdbcli_command('listtenants') - print(output) assert output == 'The cluster has no tenants' output = run_fdbcli_command('createtenant tenant') - print(output) assert output == 'The tenant `tenant\' has been created' output = run_fdbcli_command('createtenant tenant2') - print(output) assert output == 'The tenant `tenant2\' has been created' output = run_fdbcli_command('listtenants') - print(output) assert output == '1. tenant\n 2. tenant2' output = run_fdbcli_command('listtenants a z 1') - print(output) assert output == '1. tenant' output = run_fdbcli_command('listtenants a tenant2') - print(output) assert output == '1. tenant' output = run_fdbcli_command('listtenants tenant2 z') - print(output) assert output == '1. tenant2' output = run_fdbcli_command('gettenant tenant') - print(output) lines = output.split('\n') assert len(lines) == 2 assert lines[0].strip().startswith('id: ') assert lines[1].strip().startswith('prefix: ') output = run_fdbcli_command('usetenant') - print(output) assert output == 'Using the default tenant' output = run_fdbcli_command_and_get_error('usetenant tenant3') - print(output) assert output == 'ERROR: Tenant `tenant3\' does not exist' # Test writing keys to different tenants and make sure they all work correctly run_fdbcli_command('writemode on; set tenant_test default_tenant') output = run_fdbcli_command('get tenant_test') - print(output) assert output == '`tenant_test\' is `default_tenant\'' process = subprocess.Popen(command_template[:-1], stdin=subprocess.PIPE, stdout=subprocess.PIPE, env=fdbcli_env) @@ -598,7 +587,6 @@ def tenants(logger): output, _ = process.communicate(input='\n'.join(cmd_sequence).encode()) lines = output.decode().strip().split('\n')[-3:] - print(lines) assert lines[0] == 'Using tenant `tenant\'' assert lines[1] == '`tenant_test\': not found' assert lines[2].startswith('Committed') @@ -608,7 +596,6 @@ def tenants(logger): output, _ = process.communicate(input='\n'.join(cmd_sequence).encode()) lines = output.decode().strip().split('\n')[-4:] - print(lines) assert lines[0] == 'Using tenant `tenant2\'' assert lines[1] == '`tenant_test\': not found' assert lines[2].startswith('Committed') @@ -619,7 +606,6 @@ def tenants(logger): output, _ = process.communicate(input='\n'.join(cmd_sequence).encode()) lines = output.decode().strip().split('\n')[-4:] - print(lines) assert lines[0] == 'Using tenant `tenant\'' assert lines[1] == '`tenant_test\' is `tenant\'' assert lines[2] == 'Using the default tenant' @@ -631,8 +617,6 @@ def tenants(logger): lines = output.decode().strip().split('\n')[-7:] error_lines = error_output.decode().strip().split('\n')[-2:] - print(lines) - print(error_lines) assert lines[0] == 'Using tenant `tenant\'' assert lines[1].startswith('Committed') assert lines[2] == 'The tenant `tenant\' has been deleted' @@ -648,8 +632,6 @@ def tenants(logger): lines = output.decode().strip().split('\n')[-4:] error_lines = error_output.decode().strip().split('\n')[-1:] - print(lines) - print(error_lines) assert error_lines[0] == 'ERROR: Cannot delete a non-empty tenant (2133)' assert lines[0] == 'Using tenant `tenant2\'' assert lines[1].startswith('Committed') From 58c0f3d772cb66c3a8bdc2b086da69458760790f Mon Sep 17 00:00:00 2001 From: Josh Slocum Date: Mon, 21 Mar 2022 12:46:43 -0500 Subject: [PATCH 379/413] Handling change feed create mutation getting rolled back properly --- fdbserver/storageserver.actor.cpp | 28 +++++++++++++++++++++++++--- 1 file changed, 25 insertions(+), 3 deletions(-) diff --git a/fdbserver/storageserver.actor.cpp b/fdbserver/storageserver.actor.cpp index 759e332a19..f03f2e5e78 100644 --- a/fdbserver/storageserver.actor.cpp +++ b/fdbserver/storageserver.actor.cpp @@ -430,6 +430,10 @@ struct ChangeFeedInfo : ReferenceCounted { // stopVersion = MAX_VERSION means the feed has not been stopped Version stopVersion = MAX_VERSION; + // We need to track the version the change feed metadata was created by private mutation, so that if it is rolled + // back, we can avoid notifying other SS of change feeds that don't durably exist + Version metadataCreateVersion = invalidVersion; + bool removing = false; KeyRangeMap>> moveTriggers; @@ -1945,9 +1949,11 @@ ACTOR Future overlappingChangeFeedsQ(StorageServer* data, OverlappingChang std::map> rangeIds; for (auto r : ranges) { for (auto& it : r.value()) { - // Can't tell other SS about a stopVersion that may get rolled back, and we only need to tell it about the - // stopVersion if req.minVersion > stopVersion, since it will get the information from its own private - // mutations if it hasn't processed up to stopVersion yet + // Can't tell other SS about a change feed create or stopVersion that may get rolled back, and we only need + // to tell it about the metadata if req.minVersion > metadataVersion, since it will get the information from + // its own private mutations if it hasn't processed up that version yet + knownCommittedRequired = std::max(knownCommittedRequired, it->metadataCreateVersion); + Version stopVersion; if (it->stopVersion != MAX_VERSION && req.minVersion > it->stopVersion) { stopVersion = it->stopVersion; @@ -1955,6 +1961,7 @@ ACTOR Future overlappingChangeFeedsQ(StorageServer* data, OverlappingChang } else { stopVersion = MAX_VERSION; } + rangeIds[it->id] = std::tuple(it->range, it->emptyVersion, stopVersion); } } @@ -4884,6 +4891,7 @@ ACTOR Future fetchChangeFeedApplier(StorageServer* data, Version localVersion = localResult.version; Version remoteVersion = remoteResult[remoteLoc].version; + if (remoteVersion <= localVersion) { if (remoteVersion > changeFeedInfo->emptyVersion) { // merge if same version @@ -4967,6 +4975,11 @@ ACTOR Future fetchChangeFeedApplier(StorageServer* data, wait(yield()); } } catch (Error& e) { + TraceEvent(SevDebug, "FetchChangeFeedError", data->thisServerID) + .errorUnsuppressed(e) + .detail("RangeID", rangeId.printable()) + .detail("Range", range.toString()) + .detail("EndVersion", endVersion); if (e.code() != error_code_end_of_stream) { throw; } @@ -6102,9 +6115,17 @@ private: .detail("FromVersion", fromVersion) .detail("ToVersion", rollbackVersion) .detail("AtVersion", currentVersion) + .detail("RestoredVersion", restoredVersion) .detail("StorageVersion", data->storageVersion()); ASSERT(rollbackVersion >= data->storageVersion()); rollback(data, rollbackVersion, currentVersion); + } else { + TraceEvent(SevDebug, "RollbackSkip", data->thisServerID) + .detail("FromVersion", fromVersion) + .detail("ToVersion", rollbackVersion) + .detail("AtVersion", currentVersion) + .detail("RestoredVersion", restoredVersion) + .detail("StorageVersion", data->storageVersion()); } for (auto& it : data->uidChangeFeed) { if (!it.second->removing && currentVersion < it.second->stopVersion) { @@ -6185,6 +6206,7 @@ private: TEST(true); // SS got non-create change feed private mutation before move created its metadata changeFeedInfo->emptyVersion = invalidVersion; } + changeFeedInfo->metadataCreateVersion = currentVersion; data->uidChangeFeed[changeFeedId] = changeFeedInfo; feed = data->uidChangeFeed.find(changeFeedId); From 074698cdb12b03e2360081bef84b32db0038f405 Mon Sep 17 00:00:00 2001 From: Steve Atherton Date: Mon, 21 Mar 2022 13:30:27 -0700 Subject: [PATCH 380/413] Added 16k magazine size to memory stats. (#6639) --- flow/SystemMonitor.cpp | 3 +++ 1 file changed, 3 insertions(+) diff --git a/flow/SystemMonitor.cpp b/flow/SystemMonitor.cpp index 46a99cf439..b0ff90c163 100644 --- a/flow/SystemMonitor.cpp +++ b/flow/SystemMonitor.cpp @@ -160,6 +160,7 @@ SystemStatistics customSystemMonitor(std::string const& eventName, StatisticsSta .DETAILALLOCATORMEMUSAGE(2048) .DETAILALLOCATORMEMUSAGE(4096) .DETAILALLOCATORMEMUSAGE(8192) + .DETAILALLOCATORMEMUSAGE(16384) .detail("HugeArenaMemory", g_hugeArenaMemory.load()) .detail("DCID", machineState.dcId) .detail("ZoneID", machineState.zoneId) @@ -177,6 +178,7 @@ SystemStatistics customSystemMonitor(std::string const& eventName, StatisticsSta total_memory += FastAllocator<2048>::getTotalMemory(); total_memory += FastAllocator<4096>::getTotalMemory(); total_memory += FastAllocator<8192>::getTotalMemory(); + total_memory += FastAllocator<16384>::getTotalMemory(); uint64_t unused_memory = 0; unused_memory += FastAllocator<16>::getApproximateMemoryUnused(); @@ -190,6 +192,7 @@ SystemStatistics customSystemMonitor(std::string const& eventName, StatisticsSta unused_memory += FastAllocator<2048>::getApproximateMemoryUnused(); unused_memory += FastAllocator<4096>::getApproximateMemoryUnused(); unused_memory += FastAllocator<8192>::getApproximateMemoryUnused(); + unused_memory += FastAllocator<16384>::getApproximateMemoryUnused(); if (total_memory > 0) { TraceEvent("FastAllocMemoryUsage") From a71099471b07ef7f387d64c5633640385cbc58ef Mon Sep 17 00:00:00 2001 From: sfc-gh-tclinkenbeard Date: Mon, 21 Mar 2022 13:36:23 -0700 Subject: [PATCH 381/413] Update copyright header dates --- FDBLibTLS/FDBLibTLSPlugin.cpp | 2 +- FDBLibTLS/FDBLibTLSPlugin.h | 2 +- FDBLibTLS/FDBLibTLSPolicy.cpp | 2 +- FDBLibTLS/FDBLibTLSPolicy.h | 2 +- FDBLibTLS/FDBLibTLSSession.cpp | 2 +- FDBLibTLS/FDBLibTLSSession.h | 2 +- FDBLibTLS/FDBLibTLSVerify.cpp | 2 +- FDBLibTLS/FDBLibTLSVerify.h | 2 +- FDBLibTLS/plugin-test.cpp | 2 +- FDBLibTLS/scripts/make-test-certs.sh | 2 +- FDBLibTLS/verify-test.cpp | 2 +- bindings/c/fdb_c.cpp | 2 +- bindings/c/foundationdb/ClientWorkload.h | 2 +- bindings/c/foundationdb/fdb_c.h | 2 +- bindings/c/test/test.h | 2 +- bindings/c/test/unit/disconnected_timeout_tests.cpp | 2 +- bindings/c/test/unit/fdb_api.cpp | 2 +- bindings/c/test/unit/setup_tests.cpp | 2 +- bindings/c/test/unit/trace_partial_file_suffix_test.cpp | 2 +- bindings/c/test/unit/unit_tests.cpp | 2 +- bindings/c/test/unit/unit_tests_version_510.cpp | 2 +- bindings/c/test/workloads/SimpleWorkload.cpp | 2 +- bindings/c/test/workloads/workloads.cpp | 2 +- bindings/c/test/workloads/workloads.h | 2 +- bindings/flow/DirectoryLayer.actor.cpp | 2 +- bindings/flow/DirectoryLayer.h | 2 +- bindings/flow/DirectoryPartition.h | 2 +- bindings/flow/DirectorySubspace.cpp | 2 +- bindings/flow/DirectorySubspace.h | 2 +- bindings/flow/FDBLoanerTypes.h | 2 +- bindings/flow/HighContentionAllocator.actor.cpp | 2 +- bindings/flow/HighContentionAllocator.h | 2 +- bindings/flow/IDirectory.h | 2 +- bindings/flow/Node.actor.cpp | 2 +- bindings/flow/Subspace.cpp | 2 +- bindings/flow/Subspace.h | 2 +- bindings/flow/Tuple.cpp | 2 +- bindings/flow/Tuple.h | 2 +- bindings/flow/fdb_flow.actor.cpp | 2 +- bindings/flow/fdb_flow.h | 2 +- bindings/flow/tester/DirectoryTester.actor.cpp | 2 +- bindings/flow/tester/Tester.actor.cpp | 2 +- bindings/flow/tester/Tester.actor.h | 2 +- bindings/java/JavaWorkload.cpp | 2 +- bindings/java/fdbJNI.cpp | 2 +- documentation/tutorial/tutorial.actor.cpp | 2 +- fdbbackup/BackupTLSConfig.cpp | 2 +- fdbbackup/BackupTLSConfig.h | 2 +- fdbbackup/FileConverter.actor.cpp | 2 +- fdbbackup/FileConverter.h | 2 +- fdbbackup/FileDecoder.actor.cpp | 2 +- fdbbackup/backup.actor.cpp | 2 +- fdbcli/AdvanceVersionCommand.actor.cpp | 2 +- fdbcli/BlobRangeCommand.actor.cpp | 2 +- fdbcli/CacheRangeCommand.actor.cpp | 2 +- fdbcli/ChangeFeedCommand.actor.cpp | 2 +- fdbcli/ConfigureCommand.actor.cpp | 2 +- fdbcli/ConsistencyCheckCommand.actor.cpp | 2 +- fdbcli/CoordinatorsCommand.actor.cpp | 2 +- fdbcli/DataDistributionCommand.actor.cpp | 2 +- fdbcli/ExcludeCommand.actor.cpp | 2 +- fdbcli/ExpensiveDataCheckCommand.actor.cpp | 2 +- fdbcli/FileConfigureCommand.actor.cpp | 2 +- fdbcli/FlowLineNoise.actor.cpp | 2 +- fdbcli/FlowLineNoise.h | 2 +- fdbcli/ForceRecoveryWithDataLossCommand.actor.cpp | 2 +- fdbcli/IncludeCommand.actor.cpp | 2 +- fdbcli/KillCommand.actor.cpp | 2 +- fdbcli/LockCommand.actor.cpp | 2 +- fdbcli/MaintenanceCommand.actor.cpp | 2 +- fdbcli/ProfileCommand.actor.cpp | 2 +- fdbcli/SetClassCommand.actor.cpp | 2 +- fdbcli/SnapshotCommand.actor.cpp | 2 +- fdbcli/StatusCommand.actor.cpp | 2 +- fdbcli/SuspendCommand.actor.cpp | 2 +- fdbcli/ThrottleCommand.actor.cpp | 2 +- fdbcli/TriggerDDTeamInfoLogCommand.actor.cpp | 2 +- fdbcli/TssqCommand.actor.cpp | 2 +- fdbcli/Util.actor.cpp | 2 +- fdbcli/fdbcli.actor.cpp | 2 +- fdbcli/fdbcli.actor.h | 2 +- fdbclient/ActorLineageProfiler.cpp | 2 +- fdbclient/ActorLineageProfiler.h | 2 +- fdbclient/AnnotateActor.cpp | 2 +- fdbclient/AnnotateActor.h | 2 +- fdbclient/AsyncFileS3BlobStore.actor.cpp | 2 +- fdbclient/AsyncFileS3BlobStore.actor.h | 2 +- fdbclient/AsyncTaskThread.actor.cpp | 2 +- fdbclient/AsyncTaskThread.h | 2 +- fdbclient/Atomic.h | 2 +- fdbclient/AutoPublicAddress.cpp | 2 +- fdbclient/BackupAgent.actor.h | 2 +- fdbclient/BackupAgentBase.actor.cpp | 2 +- fdbclient/BackupContainer.actor.cpp | 2 +- fdbclient/BackupContainer.h | 2 +- fdbclient/BackupContainerAzureBlobStore.actor.cpp | 2 +- fdbclient/BackupContainerAzureBlobStore.h | 2 +- fdbclient/BackupContainerFileSystem.actor.cpp | 2 +- fdbclient/BackupContainerFileSystem.h | 2 +- fdbclient/BackupContainerLocalDirectory.actor.cpp | 2 +- fdbclient/BackupContainerLocalDirectory.h | 2 +- fdbclient/BackupContainerS3BlobStore.actor.cpp | 2 +- fdbclient/BackupContainerS3BlobStore.h | 2 +- fdbclient/BlobGranuleCommon.h | 2 +- fdbclient/BlobGranuleFiles.cpp | 2 +- fdbclient/BlobGranuleFiles.h | 2 +- fdbclient/BlobGranuleReader.actor.cpp | 2 +- fdbclient/BlobGranuleReader.actor.h | 2 +- fdbclient/BlobWorkerCommon.h | 2 +- fdbclient/BlobWorkerInterface.h | 2 +- fdbclient/ClientBooleanParams.cpp | 2 +- fdbclient/ClientBooleanParams.h | 2 +- fdbclient/ClientKnobCollection.cpp | 2 +- fdbclient/ClientKnobCollection.h | 2 +- fdbclient/ClientKnobs.cpp | 2 +- fdbclient/ClientKnobs.h | 2 +- fdbclient/ClientLogEvents.h | 2 +- fdbclient/ClientVersion.h | 2 +- fdbclient/ClientWorkerInterface.h | 2 +- fdbclient/ClusterConnectionFile.actor.cpp | 2 +- fdbclient/ClusterConnectionFile.h | 2 +- fdbclient/ClusterConnectionKey.actor.cpp | 2 +- fdbclient/ClusterConnectionKey.actor.h | 2 +- fdbclient/ClusterConnectionMemoryRecord.actor.cpp | 2 +- fdbclient/ClusterConnectionMemoryRecord.h | 2 +- fdbclient/ClusterInterface.h | 2 +- fdbclient/CommitProxyInterface.h | 2 +- fdbclient/CommitTransaction.h | 2 +- fdbclient/ConfigKnobs.cpp | 2 +- fdbclient/ConfigKnobs.h | 2 +- fdbclient/ConfigTransactionInterface.cpp | 2 +- fdbclient/ConfigTransactionInterface.h | 2 +- fdbclient/CoordinationInterface.h | 2 +- fdbclient/DatabaseBackupAgent.actor.cpp | 2 +- fdbclient/DatabaseConfiguration.cpp | 2 +- fdbclient/DatabaseConfiguration.h | 2 +- fdbclient/DatabaseContext.h | 2 +- fdbclient/EventTypes.actor.h | 2 +- fdbclient/FDBOptions.h | 2 +- fdbclient/FDBTypes.cpp | 2 +- fdbclient/FDBTypes.h | 2 +- fdbclient/FileBackupAgent.actor.cpp | 2 +- fdbclient/FluentDSampleIngestor.cpp | 2 +- fdbclient/GlobalConfig.actor.cpp | 2 +- fdbclient/GlobalConfig.actor.h | 2 +- fdbclient/GlobalConfig.h | 2 +- fdbclient/GrvProxyInterface.h | 2 +- fdbclient/HTTP.actor.cpp | 2 +- fdbclient/HTTP.h | 2 +- fdbclient/IClientApi.h | 2 +- fdbclient/IConfigTransaction.cpp | 2 +- fdbclient/IConfigTransaction.h | 2 +- fdbclient/IKnobCollection.cpp | 2 +- fdbclient/IKnobCollection.h | 2 +- fdbclient/ISingleThreadTransaction.cpp | 2 +- fdbclient/ISingleThreadTransaction.h | 2 +- fdbclient/JSONDoc.h | 2 +- fdbclient/KeyBackedTypes.h | 2 +- fdbclient/KeyRangeMap.actor.cpp | 2 +- fdbclient/KeyRangeMap.h | 2 +- fdbclient/Knobs.h | 2 +- fdbclient/LocalClientAPI.cpp | 2 +- fdbclient/LocalClientAPI.h | 2 +- fdbclient/ManagementAPI.actor.cpp | 2 +- fdbclient/ManagementAPI.actor.h | 2 +- fdbclient/MonitorLeader.actor.cpp | 2 +- fdbclient/MonitorLeader.h | 2 +- fdbclient/MultiVersionAssignmentVars.h | 2 +- fdbclient/MultiVersionTransaction.actor.cpp | 2 +- fdbclient/MultiVersionTransaction.h | 2 +- fdbclient/MutationList.h | 2 +- fdbclient/MutationLogReader.actor.cpp | 2 +- fdbclient/MutationLogReader.actor.h | 2 +- fdbclient/NameLineage.cpp | 2 +- fdbclient/NameLineage.h | 2 +- fdbclient/NativeAPI.actor.cpp | 2 +- fdbclient/NativeAPI.actor.h | 2 +- fdbclient/Notified.h | 2 +- fdbclient/PImpl.h | 2 +- fdbclient/ParallelStream.actor.cpp | 2 +- fdbclient/ParallelStream.actor.h | 2 +- fdbclient/PaxosConfigTransaction.actor.cpp | 2 +- fdbclient/PaxosConfigTransaction.h | 2 +- fdbclient/ProcessInterface.h | 2 +- fdbclient/RYWIterator.cpp | 2 +- fdbclient/RYWIterator.h | 2 +- fdbclient/ReadYourWrites.actor.cpp | 2 +- fdbclient/ReadYourWrites.h | 2 +- fdbclient/RestoreInterface.cpp | 2 +- fdbclient/RestoreInterface.h | 2 +- fdbclient/RunTransaction.actor.h | 2 +- fdbclient/S3BlobStore.actor.cpp | 2 +- fdbclient/S3BlobStore.h | 2 +- fdbclient/Schemas.cpp | 2 +- fdbclient/Schemas.h | 2 +- fdbclient/ServerKnobCollection.cpp | 2 +- fdbclient/ServerKnobCollection.h | 2 +- fdbclient/ServerKnobs.cpp | 2 +- fdbclient/ServerKnobs.h | 2 +- fdbclient/SimpleConfigTransaction.actor.cpp | 2 +- fdbclient/SimpleConfigTransaction.h | 2 +- fdbclient/SnapshotCache.h | 2 +- fdbclient/SpecialKeySpace.actor.cpp | 2 +- fdbclient/SpecialKeySpace.actor.h | 2 +- fdbclient/StackLineage.cpp | 2 +- fdbclient/StackLineage.h | 2 +- fdbclient/Status.h | 2 +- fdbclient/StatusClient.actor.cpp | 2 +- fdbclient/StatusClient.h | 2 +- fdbclient/StorageServerInterface.cpp | 2 +- fdbclient/StorageServerInterface.h | 2 +- fdbclient/Subspace.cpp | 2 +- fdbclient/Subspace.h | 2 +- fdbclient/SystemData.cpp | 2 +- fdbclient/SystemData.h | 2 +- fdbclient/TagThrottle.actor.cpp | 2 +- fdbclient/TagThrottle.actor.h | 2 +- fdbclient/TaskBucket.actor.cpp | 2 +- fdbclient/TaskBucket.h | 2 +- fdbclient/TestKnobCollection.cpp | 2 +- fdbclient/TestKnobCollection.h | 2 +- fdbclient/ThreadSafeTransaction.cpp | 2 +- fdbclient/ThreadSafeTransaction.h | 2 +- fdbclient/TransactionLineage.cpp | 2 +- fdbclient/TransactionLineage.h | 2 +- fdbclient/Tuple.cpp | 2 +- fdbclient/Tuple.h | 2 +- fdbclient/VersionedMap.actor.h | 2 +- fdbclient/VersionedMap.h | 2 +- fdbclient/WellKnownEndpoints.h | 2 +- fdbclient/WriteMap.h | 2 +- fdbmonitor/fdbmonitor.cpp | 2 +- fdbrpc/ActorFuzz.actor.cpp | 2 +- fdbrpc/ActorFuzz.h | 2 +- fdbrpc/AsyncFileCached.actor.cpp | 2 +- fdbrpc/AsyncFileCached.actor.h | 2 +- fdbrpc/AsyncFileChaos.actor.h | 2 +- fdbrpc/AsyncFileEIO.actor.h | 2 +- fdbrpc/AsyncFileEncrypted.actor.cpp | 2 +- fdbrpc/AsyncFileEncrypted.h | 2 +- fdbrpc/AsyncFileKAIO.actor.h | 2 +- fdbrpc/AsyncFileNonDurable.actor.cpp | 2 +- fdbrpc/AsyncFileNonDurable.actor.h | 2 +- fdbrpc/AsyncFileReadAhead.actor.h | 2 +- fdbrpc/AsyncFileWinASIO.actor.h | 2 +- fdbrpc/AsyncFileWriteChecker.cpp | 2 +- fdbrpc/AsyncFileWriteChecker.h | 2 +- fdbrpc/ContinuousSample.h | 2 +- fdbrpc/FailureMonitor.actor.cpp | 2 +- fdbrpc/FailureMonitor.h | 2 +- fdbrpc/FlowTests.actor.cpp | 2 +- fdbrpc/FlowTransport.actor.cpp | 2 +- fdbrpc/FlowTransport.h | 2 +- fdbrpc/HealthMonitor.actor.cpp | 2 +- fdbrpc/HealthMonitor.h | 2 +- fdbrpc/IAsyncFile.actor.cpp | 2 +- fdbrpc/IAsyncFile.h | 2 +- fdbrpc/IRateControl.h | 2 +- fdbrpc/LoadBalance.actor.cpp | 2 +- fdbrpc/LoadBalance.actor.h | 2 +- fdbrpc/LoadBalance.h | 2 +- fdbrpc/LoadPlugin.h | 2 +- fdbrpc/Locality.cpp | 2 +- fdbrpc/Locality.h | 2 +- fdbrpc/MultiInterface.h | 2 +- fdbrpc/Net2FileSystem.cpp | 2 +- fdbrpc/Net2FileSystem.h | 2 +- fdbrpc/PerfMetric.cpp | 2 +- fdbrpc/PerfMetric.h | 2 +- fdbrpc/QueueModel.cpp | 2 +- fdbrpc/QueueModel.h | 2 +- fdbrpc/RangeMap.h | 2 +- fdbrpc/Replication.cpp | 2 +- fdbrpc/Replication.h | 2 +- fdbrpc/ReplicationPolicy.cpp | 2 +- fdbrpc/ReplicationPolicy.h | 2 +- fdbrpc/ReplicationTypes.cpp | 2 +- fdbrpc/ReplicationTypes.h | 2 +- fdbrpc/ReplicationUtils.cpp | 2 +- fdbrpc/ReplicationUtils.h | 2 +- fdbrpc/SimExternalConnection.actor.cpp | 2 +- fdbrpc/SimExternalConnection.h | 2 +- fdbrpc/Smoother.h | 2 +- fdbrpc/Stats.actor.cpp | 2 +- fdbrpc/Stats.h | 2 +- fdbrpc/TSSComparison.h | 2 +- fdbrpc/TimedRequest.h | 2 +- fdbrpc/TraceFileIO.cpp | 2 +- fdbrpc/TraceFileIO.h | 2 +- fdbrpc/dsltest.actor.cpp | 2 +- fdbrpc/fdbrpc.h | 2 +- fdbrpc/genericactors.actor.cpp | 2 +- fdbrpc/genericactors.actor.h | 2 +- fdbrpc/linux_kaio.h | 2 +- fdbrpc/networksender.actor.h | 2 +- fdbrpc/sim2.actor.cpp | 2 +- fdbrpc/sim_validation.cpp | 2 +- fdbrpc/sim_validation.h | 2 +- fdbrpc/simulator.h | 2 +- fdbserver/ApplyMetadataMutation.cpp | 2 +- fdbserver/ApplyMetadataMutation.h | 2 +- fdbserver/ArtMutationBuffer.h | 2 +- fdbserver/BackupInterface.h | 2 +- fdbserver/BackupProgress.actor.cpp | 2 +- fdbserver/BackupProgress.actor.h | 2 +- fdbserver/BackupWorker.actor.cpp | 2 +- fdbserver/BlobManager.actor.cpp | 2 +- fdbserver/BlobManagerInterface.h | 2 +- fdbserver/BlobWorker.actor.cpp | 2 +- fdbserver/ClusterController.actor.cpp | 2 +- fdbserver/ClusterController.actor.h | 2 +- fdbserver/ClusterRecovery.actor.cpp | 2 +- fdbserver/ClusterRecovery.actor.h | 2 +- fdbserver/CommitProxyServer.actor.cpp | 2 +- fdbserver/CompactMap.cpp | 2 +- fdbserver/ConfigBroadcastInterface.h | 2 +- fdbserver/ConfigBroadcaster.actor.cpp | 2 +- fdbserver/ConfigBroadcaster.h | 2 +- fdbserver/ConfigDatabaseUnitTests.actor.cpp | 2 +- fdbserver/ConfigFollowerInterface.cpp | 2 +- fdbserver/ConfigFollowerInterface.h | 2 +- fdbserver/ConfigNode.actor.cpp | 2 +- fdbserver/ConfigNode.h | 2 +- fdbserver/ConflictSet.h | 2 +- fdbserver/CoordinatedState.actor.cpp | 2 +- fdbserver/CoordinatedState.h | 2 +- fdbserver/Coordination.actor.cpp | 2 +- fdbserver/CoordinationInterface.h | 2 +- fdbserver/CoroFlow.actor.cpp | 2 +- fdbserver/CoroFlow.h | 2 +- fdbserver/CoroFlowCoro.actor.cpp | 2 +- fdbserver/DBCoreState.h | 2 +- fdbserver/DataDistribution.actor.cpp | 2 +- fdbserver/DataDistribution.actor.h | 2 +- fdbserver/DataDistributionQueue.actor.cpp | 2 +- fdbserver/DataDistributionTracker.actor.cpp | 2 +- fdbserver/DataDistributorInterface.h | 2 +- fdbserver/DeltaTree.h | 2 +- fdbserver/DiskQueue.actor.cpp | 2 +- fdbserver/EncryptKeyProxy.actor.cpp | 2 +- fdbserver/EncryptKeyProxyInterface.h | 2 +- fdbserver/GrvProxyServer.actor.cpp | 2 +- fdbserver/IConfigConsumer.cpp | 2 +- fdbserver/IConfigConsumer.h | 2 +- fdbserver/IDiskQueue.h | 2 +- fdbserver/IKeyValueContainer.h | 2 +- fdbserver/IKeyValueStore.h | 2 +- fdbserver/IPager.h | 2 +- fdbserver/KeyValueStoreCompressTestData.actor.cpp | 2 +- fdbserver/KeyValueStoreMemory.actor.cpp | 2 +- fdbserver/KeyValueStoreSQLite.actor.cpp | 2 +- fdbserver/Knobs.h | 2 +- fdbserver/LatencyBandConfig.cpp | 2 +- fdbserver/LatencyBandConfig.h | 2 +- fdbserver/LeaderElection.actor.cpp | 2 +- fdbserver/LeaderElection.h | 2 +- fdbserver/LocalConfiguration.actor.cpp | 2 +- fdbserver/LocalConfiguration.h | 2 +- fdbserver/LogProtocolMessage.h | 2 +- fdbserver/LogRouter.actor.cpp | 2 +- fdbserver/LogSystem.cpp | 2 +- fdbserver/LogSystem.h | 2 +- fdbserver/LogSystemConfig.cpp | 2 +- fdbserver/LogSystemConfig.h | 2 +- fdbserver/LogSystemDiskQueueAdapter.actor.cpp | 2 +- fdbserver/LogSystemDiskQueueAdapter.h | 2 +- fdbserver/LogSystemPeekCursor.actor.cpp | 2 +- fdbserver/MasterInterface.h | 2 +- fdbserver/MetricLogger.actor.cpp | 2 +- fdbserver/MetricLogger.actor.h | 2 +- fdbserver/MoveKeys.actor.cpp | 2 +- fdbserver/MoveKeys.actor.h | 2 +- fdbserver/MutationTracking.cpp | 2 +- fdbserver/MutationTracking.h | 2 +- fdbserver/NetworkTest.h | 2 +- fdbserver/OldTLogServer_4_6.actor.cpp | 2 +- fdbserver/OldTLogServer_6_0.actor.cpp | 2 +- fdbserver/OldTLogServer_6_2.actor.cpp | 2 +- fdbserver/OnDemandStore.actor.cpp | 2 +- fdbserver/OnDemandStore.h | 2 +- fdbserver/PaxosConfigConsumer.actor.cpp | 2 +- fdbserver/PaxosConfigConsumer.h | 2 +- fdbserver/ProxyCommitData.actor.h | 2 +- fdbserver/QuietDatabase.actor.cpp | 2 +- fdbserver/QuietDatabase.h | 2 +- fdbserver/RadixTree.h | 2 +- fdbserver/RatekeeperInterface.h | 2 +- fdbserver/RecoveryState.h | 2 +- fdbserver/Resolver.actor.cpp | 2 +- fdbserver/ResolverInterface.h | 2 +- fdbserver/RestoreApplier.actor.cpp | 2 +- fdbserver/RestoreApplier.actor.h | 2 +- fdbserver/RestoreCommon.actor.cpp | 2 +- fdbserver/RestoreCommon.actor.h | 2 +- fdbserver/RestoreController.actor.cpp | 2 +- fdbserver/RestoreController.actor.h | 2 +- fdbserver/RestoreLoader.actor.cpp | 2 +- fdbserver/RestoreLoader.actor.h | 2 +- fdbserver/RestoreRoleCommon.actor.cpp | 2 +- fdbserver/RestoreRoleCommon.actor.h | 2 +- fdbserver/RestoreUtil.actor.cpp | 2 +- fdbserver/RestoreUtil.h | 2 +- fdbserver/RestoreWorker.actor.cpp | 2 +- fdbserver/RestoreWorker.actor.h | 2 +- fdbserver/RestoreWorkerInterface.actor.cpp | 2 +- fdbserver/RestoreWorkerInterface.actor.h | 2 +- fdbserver/RoleLineage.actor.cpp | 2 +- fdbserver/RoleLineage.actor.h | 2 +- fdbserver/ServerDBInfo.actor.h | 2 +- fdbserver/ServerDBInfo.h | 2 +- fdbserver/SigStack.cpp | 2 +- fdbserver/SimEncryptVaultProxy.actor.cpp | 2 +- fdbserver/SimEncryptVaultProxy.actor.h | 2 +- fdbserver/SimpleConfigConsumer.actor.cpp | 2 +- fdbserver/SimpleConfigConsumer.h | 2 +- fdbserver/SimulatedCluster.actor.cpp | 2 +- fdbserver/SimulatedCluster.h | 2 +- fdbserver/SkipList.cpp | 2 +- fdbserver/SpanContextMessage.h | 2 +- fdbserver/Status.actor.cpp | 2 +- fdbserver/Status.h | 2 +- fdbserver/StorageCache.actor.cpp | 2 +- fdbserver/StorageMetrics.actor.h | 2 +- fdbserver/StorageMetrics.h | 2 +- fdbserver/TLogInterface.h | 2 +- fdbserver/TLogServer.actor.cpp | 2 +- fdbserver/TSSMappingUtil.actor.cpp | 2 +- fdbserver/TSSMappingUtil.actor.h | 2 +- fdbserver/TagPartitionedLogSystem.actor.cpp | 2 +- fdbserver/TagPartitionedLogSystem.actor.h | 2 +- fdbserver/TesterInterface.actor.h | 2 +- fdbserver/VFSAsync.cpp | 2 +- fdbserver/VFSAsync.h | 2 +- fdbserver/VersionedBTree.actor.cpp | 2 +- fdbserver/WaitFailure.actor.cpp | 2 +- fdbserver/WaitFailure.h | 2 +- fdbserver/WorkerInterface.actor.h | 2 +- fdbserver/art.h | 2 +- fdbserver/art_impl.h | 2 +- fdbserver/fdbserver.actor.cpp | 2 +- fdbserver/masterserver.actor.cpp | 2 +- fdbserver/networktest.actor.cpp | 2 +- fdbserver/pubsub.actor.cpp | 2 +- fdbserver/pubsub.h | 2 +- fdbserver/storageserver.actor.cpp | 2 +- fdbserver/template_fdb.h | 2 +- fdbserver/tester.actor.cpp | 2 +- fdbserver/worker.actor.cpp | 2 +- fdbserver/workloads/ApiCorrectness.actor.cpp | 2 +- fdbserver/workloads/ApiWorkload.actor.cpp | 2 +- fdbserver/workloads/ApiWorkload.h | 2 +- fdbserver/workloads/AsyncFile.actor.h | 2 +- fdbserver/workloads/AsyncFile.cpp | 2 +- fdbserver/workloads/AsyncFileCorrectness.actor.cpp | 2 +- fdbserver/workloads/AsyncFileRead.actor.cpp | 2 +- fdbserver/workloads/AsyncFileWrite.actor.cpp | 2 +- fdbserver/workloads/AtomicOps.actor.cpp | 2 +- fdbserver/workloads/AtomicOpsApiCorrectness.actor.cpp | 2 +- fdbserver/workloads/AtomicRestore.actor.cpp | 2 +- fdbserver/workloads/AtomicSwitchover.actor.cpp | 2 +- fdbserver/workloads/BackgroundSelectors.actor.cpp | 2 +- .../workloads/BackupAndParallelRestoreCorrectness.actor.cpp | 2 +- fdbserver/workloads/BackupCorrectness.actor.cpp | 2 +- fdbserver/workloads/BackupToBlob.actor.cpp | 2 +- fdbserver/workloads/BackupToDBAbort.actor.cpp | 2 +- fdbserver/workloads/BackupToDBCorrectness.actor.cpp | 2 +- fdbserver/workloads/BackupToDBUpgrade.actor.cpp | 2 +- fdbserver/workloads/BlobGranuleVerifier.actor.cpp | 2 +- fdbserver/workloads/BlobStoreWorkload.h | 2 +- fdbserver/workloads/BulkLoad.actor.cpp | 2 +- fdbserver/workloads/BulkSetup.actor.h | 2 +- fdbserver/workloads/ChangeConfig.actor.cpp | 2 +- fdbserver/workloads/ChangeFeeds.actor.cpp | 2 +- fdbserver/workloads/ClearSingleRange.actor.cpp | 2 +- .../workloads/ClientTransactionProfileCorrectness.actor.cpp | 2 +- fdbserver/workloads/ClogSingleConnection.actor.cpp | 2 +- fdbserver/workloads/CommitBugCheck.actor.cpp | 2 +- fdbserver/workloads/ConfigIncrement.actor.cpp | 2 +- fdbserver/workloads/ConfigureDatabase.actor.cpp | 2 +- fdbserver/workloads/ConflictRange.actor.cpp | 2 +- fdbserver/workloads/ConsistencyCheck.actor.cpp | 2 +- fdbserver/workloads/CpuProfiler.actor.cpp | 2 +- fdbserver/workloads/Cycle.actor.cpp | 2 +- fdbserver/workloads/DDBalance.actor.cpp | 2 +- fdbserver/workloads/DDMetrics.actor.cpp | 2 +- fdbserver/workloads/DDMetricsExclude.actor.cpp | 2 +- fdbserver/workloads/DataDistributionMetrics.actor.cpp | 2 +- fdbserver/workloads/DataLossRecovery.actor.cpp | 2 +- fdbserver/workloads/DifferentClustersSameRV.actor.cpp | 2 +- fdbserver/workloads/DiskDurability.actor.cpp | 2 +- fdbserver/workloads/DiskDurabilityTest.actor.cpp | 2 +- fdbserver/workloads/DiskFailureInjection.actor.cpp | 2 +- fdbserver/workloads/DummyWorkload.actor.cpp | 2 +- fdbserver/workloads/EncryptionOps.actor.cpp | 2 +- fdbserver/workloads/ExternalWorkload.actor.cpp | 2 +- fdbserver/workloads/FastTriggeredWatches.actor.cpp | 2 +- fdbserver/workloads/FileSystem.actor.cpp | 2 +- fdbserver/workloads/Fuzz.cpp | 2 +- fdbserver/workloads/FuzzApiCorrectness.actor.cpp | 2 +- fdbserver/workloads/GetRangeStream.actor.cpp | 2 +- fdbserver/workloads/HealthMetricsApi.actor.cpp | 2 +- fdbserver/workloads/Increment.actor.cpp | 2 +- fdbserver/workloads/IncrementalBackup.actor.cpp | 2 +- fdbserver/workloads/IndexScan.actor.cpp | 2 +- fdbserver/workloads/Inventory.actor.cpp | 2 +- fdbserver/workloads/KVStoreTest.actor.cpp | 2 +- fdbserver/workloads/KillRegion.actor.cpp | 2 +- fdbserver/workloads/LocalRatekeeper.actor.cpp | 2 +- fdbserver/workloads/LockDatabase.actor.cpp | 2 +- fdbserver/workloads/LockDatabaseFrequently.actor.cpp | 2 +- fdbserver/workloads/LogMetrics.actor.cpp | 2 +- fdbserver/workloads/LowLatency.actor.cpp | 2 +- fdbserver/workloads/MachineAttrition.actor.cpp | 2 +- fdbserver/workloads/MemoryKeyValueStore.cpp | 2 +- fdbserver/workloads/MemoryKeyValueStore.h | 2 +- fdbserver/workloads/MemoryLifetime.actor.cpp | 2 +- fdbserver/workloads/MetricLogging.actor.cpp | 2 +- fdbserver/workloads/MiniCycle.actor.cpp | 2 +- fdbserver/workloads/MutationLogReaderCorrectness.actor.cpp | 2 +- fdbserver/workloads/ParallelRestore.actor.cpp | 2 +- fdbserver/workloads/Performance.actor.cpp | 2 +- fdbserver/workloads/PhysicalShardMove.actor.cpp | 2 +- fdbserver/workloads/Ping.actor.cpp | 2 +- fdbserver/workloads/PopulateTPCC.actor.cpp | 2 +- fdbserver/workloads/ProtocolVersion.actor.cpp | 2 +- fdbserver/workloads/PubSubMultiples.actor.cpp | 2 +- fdbserver/workloads/QueuePush.actor.cpp | 2 +- fdbserver/workloads/RYWDisable.actor.cpp | 2 +- fdbserver/workloads/RYWPerformance.actor.cpp | 2 +- fdbserver/workloads/RandomClogging.actor.cpp | 2 +- fdbserver/workloads/RandomMoveKeys.actor.cpp | 2 +- fdbserver/workloads/RandomSelector.actor.cpp | 2 +- fdbserver/workloads/ReadAfterWrite.actor.cpp | 2 +- fdbserver/workloads/ReadHotDetection.actor.cpp | 2 +- fdbserver/workloads/ReadWrite.actor.cpp | 2 +- fdbserver/workloads/RemoveServersSafely.actor.cpp | 2 +- fdbserver/workloads/ReportConflictingKeys.actor.cpp | 2 +- fdbserver/workloads/RestoreBackup.actor.cpp | 2 +- fdbserver/workloads/RestoreFromBlob.actor.cpp | 2 +- fdbserver/workloads/Rollback.actor.cpp | 2 +- fdbserver/workloads/RyowCorrectness.actor.cpp | 2 +- fdbserver/workloads/SaveAndKill.actor.cpp | 2 +- fdbserver/workloads/SelectorCorrectness.actor.cpp | 2 +- fdbserver/workloads/Serializability.actor.cpp | 2 +- fdbserver/workloads/Sideband.actor.cpp | 2 +- fdbserver/workloads/SimpleAtomicAdd.actor.cpp | 2 +- fdbserver/workloads/SlowTaskWorkload.actor.cpp | 2 +- fdbserver/workloads/SpecialKeySpaceCorrectness.actor.cpp | 2 +- fdbserver/workloads/StatusWorkload.actor.cpp | 2 +- fdbserver/workloads/Storefront.actor.cpp | 2 +- fdbserver/workloads/StreamingRangeRead.actor.cpp | 2 +- fdbserver/workloads/StreamingRead.actor.cpp | 2 +- fdbserver/workloads/SubmitBackup.actor.cpp | 2 +- fdbserver/workloads/TPCC.actor.cpp | 2 +- fdbserver/workloads/TPCCWorkload.h | 2 +- fdbserver/workloads/TagThrottleApi.actor.cpp | 2 +- fdbserver/workloads/TargetedKill.actor.cpp | 2 +- fdbserver/workloads/TaskBucketCorrectness.actor.cpp | 2 +- fdbserver/workloads/ThreadSafety.actor.cpp | 2 +- fdbserver/workloads/Throttling.actor.cpp | 2 +- fdbserver/workloads/Throughput.actor.cpp | 2 +- fdbserver/workloads/TimeKeeperCorrectness.actor.cpp | 2 +- fdbserver/workloads/UDPWorkload.actor.cpp | 2 +- fdbserver/workloads/UnitPerf.actor.cpp | 2 +- fdbserver/workloads/UnitTests.actor.cpp | 2 +- fdbserver/workloads/Unreadable.actor.cpp | 2 +- fdbserver/workloads/VersionStamp.actor.cpp | 2 +- fdbserver/workloads/WatchAndWait.actor.cpp | 2 +- fdbserver/workloads/Watches.actor.cpp | 2 +- fdbserver/workloads/WatchesSameKeyCorrectness.actor.cpp | 2 +- fdbserver/workloads/WorkerErrors.actor.cpp | 2 +- fdbserver/workloads/WriteBandwidth.actor.cpp | 2 +- fdbserver/workloads/WriteDuringRead.actor.cpp | 2 +- fdbserver/workloads/WriteTagThrottling.actor.cpp | 2 +- fdbserver/workloads/workloads.actor.h | 2 +- fdbservice/FDBService.cpp | 2 +- flow/ActorCollection.actor.cpp | 2 +- flow/ActorCollection.h | 2 +- flow/Arena.cpp | 2 +- flow/Arena.h | 2 +- flow/ArgParseUtil.h | 2 +- flow/AsioReactor.h | 2 +- flow/BooleanParam.h | 2 +- flow/CompressedInt.actor.cpp | 2 +- flow/CompressedInt.h | 2 +- flow/Deque.cpp | 2 +- flow/Deque.h | 2 +- flow/DeterministicRandom.cpp | 2 +- flow/DeterministicRandom.h | 2 +- flow/Error.cpp | 2 +- flow/Error.h | 2 +- flow/EventTypes.actor.h | 2 +- flow/FastAlloc.cpp | 2 +- flow/FastAlloc.h | 2 +- flow/FastRef.h | 2 +- flow/FaultInjection.cpp | 2 +- flow/FaultInjection.h | 2 +- flow/FileIdentifier.h | 2 +- flow/Histogram.cpp | 2 +- flow/Histogram.h | 2 +- flow/IDispatched.h | 2 +- flow/IRandom.h | 2 +- flow/IThreadPool.cpp | 2 +- flow/IThreadPool.h | 2 +- flow/IndexedSet.actor.h | 2 +- flow/IndexedSet.cpp | 2 +- flow/IndexedSet.h | 2 +- flow/Knobs.cpp | 2 +- flow/Knobs.h | 2 +- flow/MetricSample.h | 2 +- flow/Net2.actor.cpp | 2 +- flow/Net2Packet.cpp | 2 +- flow/Net2Packet.h | 2 +- flow/ObjectSerializer.h | 2 +- flow/ObjectSerializerTraits.h | 2 +- flow/Platform.actor.cpp | 2 +- flow/Platform.actor.h | 2 +- flow/Platform.h | 2 +- flow/Profiler.actor.cpp | 2 +- flow/Profiler.h | 2 +- flow/ProtocolVersion.h | 2 +- flow/SendBufferIterator.h | 2 +- flow/SignalSafeUnwind.cpp | 2 +- flow/SignalSafeUnwind.h | 2 +- flow/StreamCipher.cpp | 2 +- flow/StreamCipher.h | 2 +- flow/SystemMonitor.cpp | 2 +- flow/SystemMonitor.h | 2 +- flow/TDMetric.actor.h | 2 +- flow/TDMetric.cpp | 2 +- flow/TLSConfig.actor.cpp | 2 +- flow/TLSConfig.actor.h | 2 +- flow/ThreadHelper.actor.cpp | 2 +- flow/ThreadHelper.actor.h | 2 +- flow/ThreadPrimitives.cpp | 2 +- flow/ThreadPrimitives.h | 2 +- flow/Trace.cpp | 2 +- flow/Trace.h | 2 +- flow/Tracing.actor.cpp | 2 +- flow/Tracing.h | 2 +- flow/TypeTraits.h | 2 +- flow/UnitTest.cpp | 2 +- flow/UnitTest.h | 2 +- flow/Util.h | 2 +- flow/WriteOnlySet.actor.cpp | 2 +- flow/WriteOnlySet.h | 2 +- flow/actorcompiler.h | 2 +- flow/error_definitions.h | 2 +- flow/flat_buffers.cpp | 2 +- flow/flat_buffers.h | 2 +- flow/flow.cpp | 2 +- flow/flow.h | 2 +- flow/folly_memcpy.h | 2 +- flow/genericactors.actor.cpp | 2 +- flow/genericactors.actor.h | 2 +- flow/network.cpp | 2 +- flow/network.h | 2 +- flow/serialize.cpp | 2 +- flow/serialize.h | 2 +- flow/unactorcompiler.h | 2 +- flow/version.cpp | 2 +- flowbench/BenchCallback.actor.cpp | 2 +- flowbench/BenchEncrypt.cpp | 2 +- flowbench/BenchHash.cpp | 2 +- flowbench/BenchIONet2.actor.cpp | 2 +- flowbench/BenchIterate.cpp | 2 +- flowbench/BenchMem.cpp | 2 +- flowbench/BenchMetadataCheck.cpp | 2 +- flowbench/BenchNet2.actor.cpp | 2 +- flowbench/BenchPopulate.cpp | 2 +- flowbench/BenchRandom.cpp | 2 +- flowbench/BenchRef.cpp | 2 +- flowbench/BenchStream.actor.cpp | 2 +- flowbench/BenchTimer.cpp | 2 +- flowbench/GlobalData.cpp | 2 +- flowbench/GlobalData.h | 2 +- flowbench/flowbench.actor.cpp | 2 +- packaging/docker/fdb.bash | 2 +- packaging/docker/samples/golang/app/start.bash | 2 +- packaging/docker/samples/local/start.bash | 2 +- packaging/docker/samples/local/stop.bash | 2 +- packaging/docker/samples/python/app/start.bash | 2 +- 682 files changed, 682 insertions(+), 682 deletions(-) diff --git a/FDBLibTLS/FDBLibTLSPlugin.cpp b/FDBLibTLS/FDBLibTLSPlugin.cpp index 83940b0806..bc6474ec81 100644 --- a/FDBLibTLS/FDBLibTLSPlugin.cpp +++ b/FDBLibTLS/FDBLibTLSPlugin.cpp @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/FDBLibTLS/FDBLibTLSPlugin.h b/FDBLibTLS/FDBLibTLSPlugin.h index 98f11d63e1..d47ff41c58 100644 --- a/FDBLibTLS/FDBLibTLSPlugin.h +++ b/FDBLibTLS/FDBLibTLSPlugin.h @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/FDBLibTLS/FDBLibTLSPolicy.cpp b/FDBLibTLS/FDBLibTLSPolicy.cpp index 2e3142165d..9eeb9df833 100644 --- a/FDBLibTLS/FDBLibTLSPolicy.cpp +++ b/FDBLibTLS/FDBLibTLSPolicy.cpp @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/FDBLibTLS/FDBLibTLSPolicy.h b/FDBLibTLS/FDBLibTLSPolicy.h index ad874ddd3e..aef3c31ca8 100644 --- a/FDBLibTLS/FDBLibTLSPolicy.h +++ b/FDBLibTLS/FDBLibTLSPolicy.h @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/FDBLibTLS/FDBLibTLSSession.cpp b/FDBLibTLS/FDBLibTLSSession.cpp index 75c60dd049..a7193a61a4 100644 --- a/FDBLibTLS/FDBLibTLSSession.cpp +++ b/FDBLibTLS/FDBLibTLSSession.cpp @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/FDBLibTLS/FDBLibTLSSession.h b/FDBLibTLS/FDBLibTLSSession.h index 83f0e4d1c9..0044a78a01 100644 --- a/FDBLibTLS/FDBLibTLSSession.h +++ b/FDBLibTLS/FDBLibTLSSession.h @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/FDBLibTLS/FDBLibTLSVerify.cpp b/FDBLibTLS/FDBLibTLSVerify.cpp index 506f04c734..a885584a7b 100644 --- a/FDBLibTLS/FDBLibTLSVerify.cpp +++ b/FDBLibTLS/FDBLibTLSVerify.cpp @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/FDBLibTLS/FDBLibTLSVerify.h b/FDBLibTLS/FDBLibTLSVerify.h index 8cf3330112..02e72dbc2d 100644 --- a/FDBLibTLS/FDBLibTLSVerify.h +++ b/FDBLibTLS/FDBLibTLSVerify.h @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/FDBLibTLS/plugin-test.cpp b/FDBLibTLS/plugin-test.cpp index 3567ec83a8..b7156ca4ba 100644 --- a/FDBLibTLS/plugin-test.cpp +++ b/FDBLibTLS/plugin-test.cpp @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/FDBLibTLS/scripts/make-test-certs.sh b/FDBLibTLS/scripts/make-test-certs.sh index 266e1cd185..9ce8843261 100755 --- a/FDBLibTLS/scripts/make-test-certs.sh +++ b/FDBLibTLS/scripts/make-test-certs.sh @@ -4,7 +4,7 @@ # # This source file is part of the FoundationDB open source project # -# Copyright 2013-2018 Apple Inc. and the FoundationDB project authors +# 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. diff --git a/FDBLibTLS/verify-test.cpp b/FDBLibTLS/verify-test.cpp index 59592aecf5..9932e88329 100644 --- a/FDBLibTLS/verify-test.cpp +++ b/FDBLibTLS/verify-test.cpp @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/bindings/c/fdb_c.cpp b/bindings/c/fdb_c.cpp index f5a428384d..7b6a9f089d 100644 --- a/bindings/c/fdb_c.cpp +++ b/bindings/c/fdb_c.cpp @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/bindings/c/foundationdb/ClientWorkload.h b/bindings/c/foundationdb/ClientWorkload.h index 66fb7979f0..1234140755 100644 --- a/bindings/c/foundationdb/ClientWorkload.h +++ b/bindings/c/foundationdb/ClientWorkload.h @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2019 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/bindings/c/foundationdb/fdb_c.h b/bindings/c/foundationdb/fdb_c.h index 214a9bd124..626faabb21 100644 --- a/bindings/c/foundationdb/fdb_c.h +++ b/bindings/c/foundationdb/fdb_c.h @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/bindings/c/test/test.h b/bindings/c/test/test.h index 0b79e232c6..8c73d209ef 100644 --- a/bindings/c/test/test.h +++ b/bindings/c/test/test.h @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/bindings/c/test/unit/disconnected_timeout_tests.cpp b/bindings/c/test/unit/disconnected_timeout_tests.cpp index 633fe0dea5..829b6d750f 100644 --- a/bindings/c/test/unit/disconnected_timeout_tests.cpp +++ b/bindings/c/test/unit/disconnected_timeout_tests.cpp @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2021 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/bindings/c/test/unit/fdb_api.cpp b/bindings/c/test/unit/fdb_api.cpp index 9ce9dd5d9e..4fc715dbc5 100644 --- a/bindings/c/test/unit/fdb_api.cpp +++ b/bindings/c/test/unit/fdb_api.cpp @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2020 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/bindings/c/test/unit/setup_tests.cpp b/bindings/c/test/unit/setup_tests.cpp index 602af99845..01884124ef 100644 --- a/bindings/c/test/unit/setup_tests.cpp +++ b/bindings/c/test/unit/setup_tests.cpp @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2020 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/bindings/c/test/unit/trace_partial_file_suffix_test.cpp b/bindings/c/test/unit/trace_partial_file_suffix_test.cpp index 4ccf9ff6d2..5a74b28c77 100644 --- a/bindings/c/test/unit/trace_partial_file_suffix_test.cpp +++ b/bindings/c/test/unit/trace_partial_file_suffix_test.cpp @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2021 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/bindings/c/test/unit/unit_tests.cpp b/bindings/c/test/unit/unit_tests.cpp index b4bc11f9d9..9cb2b3ae5a 100644 --- a/bindings/c/test/unit/unit_tests.cpp +++ b/bindings/c/test/unit/unit_tests.cpp @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2020 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/bindings/c/test/unit/unit_tests_version_510.cpp b/bindings/c/test/unit/unit_tests_version_510.cpp index fbc4143011..ff369316ee 100644 --- a/bindings/c/test/unit/unit_tests_version_510.cpp +++ b/bindings/c/test/unit/unit_tests_version_510.cpp @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2021 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/bindings/c/test/workloads/SimpleWorkload.cpp b/bindings/c/test/workloads/SimpleWorkload.cpp index 5cf7cf8344..0aa9f1093e 100644 --- a/bindings/c/test/workloads/SimpleWorkload.cpp +++ b/bindings/c/test/workloads/SimpleWorkload.cpp @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2019 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/bindings/c/test/workloads/workloads.cpp b/bindings/c/test/workloads/workloads.cpp index 098f40ef28..09a2f078ca 100644 --- a/bindings/c/test/workloads/workloads.cpp +++ b/bindings/c/test/workloads/workloads.cpp @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2019 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/bindings/c/test/workloads/workloads.h b/bindings/c/test/workloads/workloads.h index b9666d80c2..688ad5e451 100644 --- a/bindings/c/test/workloads/workloads.h +++ b/bindings/c/test/workloads/workloads.h @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2019 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/bindings/flow/DirectoryLayer.actor.cpp b/bindings/flow/DirectoryLayer.actor.cpp index 750ba85daf..056b203a2e 100644 --- a/bindings/flow/DirectoryLayer.actor.cpp +++ b/bindings/flow/DirectoryLayer.actor.cpp @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/bindings/flow/DirectoryLayer.h b/bindings/flow/DirectoryLayer.h index f5b87de263..ca5806f49c 100644 --- a/bindings/flow/DirectoryLayer.h +++ b/bindings/flow/DirectoryLayer.h @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/bindings/flow/DirectoryPartition.h b/bindings/flow/DirectoryPartition.h index 8ec80e3288..84bf235359 100644 --- a/bindings/flow/DirectoryPartition.h +++ b/bindings/flow/DirectoryPartition.h @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/bindings/flow/DirectorySubspace.cpp b/bindings/flow/DirectorySubspace.cpp index 63b26b5765..e519c1a982 100644 --- a/bindings/flow/DirectorySubspace.cpp +++ b/bindings/flow/DirectorySubspace.cpp @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/bindings/flow/DirectorySubspace.h b/bindings/flow/DirectorySubspace.h index 8c02fdd9e2..a46cae20c4 100644 --- a/bindings/flow/DirectorySubspace.h +++ b/bindings/flow/DirectorySubspace.h @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/bindings/flow/FDBLoanerTypes.h b/bindings/flow/FDBLoanerTypes.h index 79af2f96f5..01000f6e27 100644 --- a/bindings/flow/FDBLoanerTypes.h +++ b/bindings/flow/FDBLoanerTypes.h @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/bindings/flow/HighContentionAllocator.actor.cpp b/bindings/flow/HighContentionAllocator.actor.cpp index 41d3481bcd..f7b383d0a5 100644 --- a/bindings/flow/HighContentionAllocator.actor.cpp +++ b/bindings/flow/HighContentionAllocator.actor.cpp @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/bindings/flow/HighContentionAllocator.h b/bindings/flow/HighContentionAllocator.h index bd4ed2cb97..9784f382c3 100644 --- a/bindings/flow/HighContentionAllocator.h +++ b/bindings/flow/HighContentionAllocator.h @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/bindings/flow/IDirectory.h b/bindings/flow/IDirectory.h index e629b7cf82..923ca5648a 100644 --- a/bindings/flow/IDirectory.h +++ b/bindings/flow/IDirectory.h @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/bindings/flow/Node.actor.cpp b/bindings/flow/Node.actor.cpp index 901376aa68..8a9123bce1 100644 --- a/bindings/flow/Node.actor.cpp +++ b/bindings/flow/Node.actor.cpp @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/bindings/flow/Subspace.cpp b/bindings/flow/Subspace.cpp index 896e67f551..f56b50c1f0 100644 --- a/bindings/flow/Subspace.cpp +++ b/bindings/flow/Subspace.cpp @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/bindings/flow/Subspace.h b/bindings/flow/Subspace.h index bdbf0e4d04..bf33f7baa3 100644 --- a/bindings/flow/Subspace.h +++ b/bindings/flow/Subspace.h @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/bindings/flow/Tuple.cpp b/bindings/flow/Tuple.cpp index 337792e508..fde784f418 100644 --- a/bindings/flow/Tuple.cpp +++ b/bindings/flow/Tuple.cpp @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/bindings/flow/Tuple.h b/bindings/flow/Tuple.h index e26b489c61..03c08a6a02 100644 --- a/bindings/flow/Tuple.h +++ b/bindings/flow/Tuple.h @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/bindings/flow/fdb_flow.actor.cpp b/bindings/flow/fdb_flow.actor.cpp index bba7da080f..2a2b04a1f8 100644 --- a/bindings/flow/fdb_flow.actor.cpp +++ b/bindings/flow/fdb_flow.actor.cpp @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/bindings/flow/fdb_flow.h b/bindings/flow/fdb_flow.h index f1b87c16ba..aafa75d07d 100644 --- a/bindings/flow/fdb_flow.h +++ b/bindings/flow/fdb_flow.h @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/bindings/flow/tester/DirectoryTester.actor.cpp b/bindings/flow/tester/DirectoryTester.actor.cpp index 35b95dabd6..b21da1097c 100644 --- a/bindings/flow/tester/DirectoryTester.actor.cpp +++ b/bindings/flow/tester/DirectoryTester.actor.cpp @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/bindings/flow/tester/Tester.actor.cpp b/bindings/flow/tester/Tester.actor.cpp index 958ff1a0be..ee0f499536 100644 --- a/bindings/flow/tester/Tester.actor.cpp +++ b/bindings/flow/tester/Tester.actor.cpp @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/bindings/flow/tester/Tester.actor.h b/bindings/flow/tester/Tester.actor.h index df1d0f5551..bc426913e7 100644 --- a/bindings/flow/tester/Tester.actor.h +++ b/bindings/flow/tester/Tester.actor.h @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/bindings/java/JavaWorkload.cpp b/bindings/java/JavaWorkload.cpp index 9a7c07a5bc..7003a9e60a 100644 --- a/bindings/java/JavaWorkload.cpp +++ b/bindings/java/JavaWorkload.cpp @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2019 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/bindings/java/fdbJNI.cpp b/bindings/java/fdbJNI.cpp index 1032c418e2..d2164b9887 100644 --- a/bindings/java/fdbJNI.cpp +++ b/bindings/java/fdbJNI.cpp @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/documentation/tutorial/tutorial.actor.cpp b/documentation/tutorial/tutorial.actor.cpp index a42677f0ee..646a078ed4 100644 --- a/documentation/tutorial/tutorial.actor.cpp +++ b/documentation/tutorial/tutorial.actor.cpp @@ -4,7 +4,7 @@ * * This source file is part of the FoundationDB open source project * -* Copyright 2013-2018 Apple Inc. and the FoundationDB project authors +* 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. diff --git a/fdbbackup/BackupTLSConfig.cpp b/fdbbackup/BackupTLSConfig.cpp index 99ad0640cc..8d7c771ee4 100644 --- a/fdbbackup/BackupTLSConfig.cpp +++ b/fdbbackup/BackupTLSConfig.cpp @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2021 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbbackup/BackupTLSConfig.h b/fdbbackup/BackupTLSConfig.h index 4222c0c25f..503f819fcf 100644 --- a/fdbbackup/BackupTLSConfig.h +++ b/fdbbackup/BackupTLSConfig.h @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2021 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbbackup/FileConverter.actor.cpp b/fdbbackup/FileConverter.actor.cpp index 49bbe260ed..1e48bd523d 100644 --- a/fdbbackup/FileConverter.actor.cpp +++ b/fdbbackup/FileConverter.actor.cpp @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2019 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbbackup/FileConverter.h b/fdbbackup/FileConverter.h index 8c0f8fba39..a33032b183 100644 --- a/fdbbackup/FileConverter.h +++ b/fdbbackup/FileConverter.h @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2019 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbbackup/FileDecoder.actor.cpp b/fdbbackup/FileDecoder.actor.cpp index 2c570ee7b0..e28b00b2f7 100644 --- a/fdbbackup/FileDecoder.actor.cpp +++ b/fdbbackup/FileDecoder.actor.cpp @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2019 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbbackup/backup.actor.cpp b/fdbbackup/backup.actor.cpp index 4c980e172e..219d9ab820 100644 --- a/fdbbackup/backup.actor.cpp +++ b/fdbbackup/backup.actor.cpp @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbcli/AdvanceVersionCommand.actor.cpp b/fdbcli/AdvanceVersionCommand.actor.cpp index 248c742c88..d76402b88b 100644 --- a/fdbcli/AdvanceVersionCommand.actor.cpp +++ b/fdbcli/AdvanceVersionCommand.actor.cpp @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2021 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbcli/BlobRangeCommand.actor.cpp b/fdbcli/BlobRangeCommand.actor.cpp index 97bfc98872..02b922c4a8 100644 --- a/fdbcli/BlobRangeCommand.actor.cpp +++ b/fdbcli/BlobRangeCommand.actor.cpp @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2021 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbcli/CacheRangeCommand.actor.cpp b/fdbcli/CacheRangeCommand.actor.cpp index b0878d278a..947fa9ce15 100644 --- a/fdbcli/CacheRangeCommand.actor.cpp +++ b/fdbcli/CacheRangeCommand.actor.cpp @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2021 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbcli/ChangeFeedCommand.actor.cpp b/fdbcli/ChangeFeedCommand.actor.cpp index 796be8f337..88114d323b 100644 --- a/fdbcli/ChangeFeedCommand.actor.cpp +++ b/fdbcli/ChangeFeedCommand.actor.cpp @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2021 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbcli/ConfigureCommand.actor.cpp b/fdbcli/ConfigureCommand.actor.cpp index 03368a6f9e..acb02843b2 100644 --- a/fdbcli/ConfigureCommand.actor.cpp +++ b/fdbcli/ConfigureCommand.actor.cpp @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2021 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbcli/ConsistencyCheckCommand.actor.cpp b/fdbcli/ConsistencyCheckCommand.actor.cpp index 5f3f2e2cf1..2e14e71fcc 100644 --- a/fdbcli/ConsistencyCheckCommand.actor.cpp +++ b/fdbcli/ConsistencyCheckCommand.actor.cpp @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2021 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbcli/CoordinatorsCommand.actor.cpp b/fdbcli/CoordinatorsCommand.actor.cpp index a62848861c..b68d5ab3d3 100644 --- a/fdbcli/CoordinatorsCommand.actor.cpp +++ b/fdbcli/CoordinatorsCommand.actor.cpp @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2021 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbcli/DataDistributionCommand.actor.cpp b/fdbcli/DataDistributionCommand.actor.cpp index a7dc171ef1..7b6fe75c48 100644 --- a/fdbcli/DataDistributionCommand.actor.cpp +++ b/fdbcli/DataDistributionCommand.actor.cpp @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2021 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbcli/ExcludeCommand.actor.cpp b/fdbcli/ExcludeCommand.actor.cpp index 7a53337a4a..9cdeab276e 100644 --- a/fdbcli/ExcludeCommand.actor.cpp +++ b/fdbcli/ExcludeCommand.actor.cpp @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2021 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbcli/ExpensiveDataCheckCommand.actor.cpp b/fdbcli/ExpensiveDataCheckCommand.actor.cpp index 46866b711b..58c50396d6 100644 --- a/fdbcli/ExpensiveDataCheckCommand.actor.cpp +++ b/fdbcli/ExpensiveDataCheckCommand.actor.cpp @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2021 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbcli/FileConfigureCommand.actor.cpp b/fdbcli/FileConfigureCommand.actor.cpp index e612002eb6..e35114c429 100644 --- a/fdbcli/FileConfigureCommand.actor.cpp +++ b/fdbcli/FileConfigureCommand.actor.cpp @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2021 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbcli/FlowLineNoise.actor.cpp b/fdbcli/FlowLineNoise.actor.cpp index 1cb8044279..1029de30b4 100644 --- a/fdbcli/FlowLineNoise.actor.cpp +++ b/fdbcli/FlowLineNoise.actor.cpp @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbcli/FlowLineNoise.h b/fdbcli/FlowLineNoise.h index 87f88c69de..86dac21a1a 100644 --- a/fdbcli/FlowLineNoise.h +++ b/fdbcli/FlowLineNoise.h @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbcli/ForceRecoveryWithDataLossCommand.actor.cpp b/fdbcli/ForceRecoveryWithDataLossCommand.actor.cpp index de03af3359..e294bcee1b 100644 --- a/fdbcli/ForceRecoveryWithDataLossCommand.actor.cpp +++ b/fdbcli/ForceRecoveryWithDataLossCommand.actor.cpp @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2021 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbcli/IncludeCommand.actor.cpp b/fdbcli/IncludeCommand.actor.cpp index 2a72a214e2..a463772960 100644 --- a/fdbcli/IncludeCommand.actor.cpp +++ b/fdbcli/IncludeCommand.actor.cpp @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2021 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbcli/KillCommand.actor.cpp b/fdbcli/KillCommand.actor.cpp index d8b93f2957..4ce89b5919 100644 --- a/fdbcli/KillCommand.actor.cpp +++ b/fdbcli/KillCommand.actor.cpp @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2021 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbcli/LockCommand.actor.cpp b/fdbcli/LockCommand.actor.cpp index 7ae69be35c..1ed988ee34 100644 --- a/fdbcli/LockCommand.actor.cpp +++ b/fdbcli/LockCommand.actor.cpp @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2021 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbcli/MaintenanceCommand.actor.cpp b/fdbcli/MaintenanceCommand.actor.cpp index abc8fc355e..cf466b2c95 100644 --- a/fdbcli/MaintenanceCommand.actor.cpp +++ b/fdbcli/MaintenanceCommand.actor.cpp @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2021 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbcli/ProfileCommand.actor.cpp b/fdbcli/ProfileCommand.actor.cpp index b58c87c4dd..43d3d7a966 100644 --- a/fdbcli/ProfileCommand.actor.cpp +++ b/fdbcli/ProfileCommand.actor.cpp @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2021 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbcli/SetClassCommand.actor.cpp b/fdbcli/SetClassCommand.actor.cpp index a7b3cf2789..27ad221383 100644 --- a/fdbcli/SetClassCommand.actor.cpp +++ b/fdbcli/SetClassCommand.actor.cpp @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2021 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbcli/SnapshotCommand.actor.cpp b/fdbcli/SnapshotCommand.actor.cpp index f4413d60a0..5bc7302f0c 100644 --- a/fdbcli/SnapshotCommand.actor.cpp +++ b/fdbcli/SnapshotCommand.actor.cpp @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2021 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbcli/StatusCommand.actor.cpp b/fdbcli/StatusCommand.actor.cpp index a1f0f5faba..b9159acd1e 100644 --- a/fdbcli/StatusCommand.actor.cpp +++ b/fdbcli/StatusCommand.actor.cpp @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2021 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbcli/SuspendCommand.actor.cpp b/fdbcli/SuspendCommand.actor.cpp index 65b01e88a7..813f6bc0a0 100644 --- a/fdbcli/SuspendCommand.actor.cpp +++ b/fdbcli/SuspendCommand.actor.cpp @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2021 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbcli/ThrottleCommand.actor.cpp b/fdbcli/ThrottleCommand.actor.cpp index ee8f777b70..bb8b3e778c 100644 --- a/fdbcli/ThrottleCommand.actor.cpp +++ b/fdbcli/ThrottleCommand.actor.cpp @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2021 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbcli/TriggerDDTeamInfoLogCommand.actor.cpp b/fdbcli/TriggerDDTeamInfoLogCommand.actor.cpp index f7ff8657a6..0af97a4d8a 100644 --- a/fdbcli/TriggerDDTeamInfoLogCommand.actor.cpp +++ b/fdbcli/TriggerDDTeamInfoLogCommand.actor.cpp @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2021 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbcli/TssqCommand.actor.cpp b/fdbcli/TssqCommand.actor.cpp index 0e0f86e653..654cbe73c5 100644 --- a/fdbcli/TssqCommand.actor.cpp +++ b/fdbcli/TssqCommand.actor.cpp @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2021 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbcli/Util.actor.cpp b/fdbcli/Util.actor.cpp index 489001feee..d40a5dcaeb 100644 --- a/fdbcli/Util.actor.cpp +++ b/fdbcli/Util.actor.cpp @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2021 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbcli/fdbcli.actor.cpp b/fdbcli/fdbcli.actor.cpp index 7b6b4b9834..d875be6f9e 100644 --- a/fdbcli/fdbcli.actor.cpp +++ b/fdbcli/fdbcli.actor.cpp @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbcli/fdbcli.actor.h b/fdbcli/fdbcli.actor.h index afd48bcb3c..f7898fa9da 100644 --- a/fdbcli/fdbcli.actor.h +++ b/fdbcli/fdbcli.actor.h @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2021 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbclient/ActorLineageProfiler.cpp b/fdbclient/ActorLineageProfiler.cpp index ff097af198..0e16405dde 100644 --- a/fdbclient/ActorLineageProfiler.cpp +++ b/fdbclient/ActorLineageProfiler.cpp @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2021 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbclient/ActorLineageProfiler.h b/fdbclient/ActorLineageProfiler.h index 07b7c30966..6219a5428b 100644 --- a/fdbclient/ActorLineageProfiler.h +++ b/fdbclient/ActorLineageProfiler.h @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2021 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbclient/AnnotateActor.cpp b/fdbclient/AnnotateActor.cpp index 80b9a8cec4..f02d109d38 100644 --- a/fdbclient/AnnotateActor.cpp +++ b/fdbclient/AnnotateActor.cpp @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2021 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbclient/AnnotateActor.h b/fdbclient/AnnotateActor.h index 757410b18a..ff4af5052c 100644 --- a/fdbclient/AnnotateActor.h +++ b/fdbclient/AnnotateActor.h @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2021 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbclient/AsyncFileS3BlobStore.actor.cpp b/fdbclient/AsyncFileS3BlobStore.actor.cpp index a584144766..b97b95cef7 100644 --- a/fdbclient/AsyncFileS3BlobStore.actor.cpp +++ b/fdbclient/AsyncFileS3BlobStore.actor.cpp @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbclient/AsyncFileS3BlobStore.actor.h b/fdbclient/AsyncFileS3BlobStore.actor.h index db436755b3..5c11d74f18 100644 --- a/fdbclient/AsyncFileS3BlobStore.actor.h +++ b/fdbclient/AsyncFileS3BlobStore.actor.h @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbclient/AsyncTaskThread.actor.cpp b/fdbclient/AsyncTaskThread.actor.cpp index 665d6f923e..dc024755ee 100644 --- a/fdbclient/AsyncTaskThread.actor.cpp +++ b/fdbclient/AsyncTaskThread.actor.cpp @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbclient/AsyncTaskThread.h b/fdbclient/AsyncTaskThread.h index 223a434257..94fbb9f7fe 100644 --- a/fdbclient/AsyncTaskThread.h +++ b/fdbclient/AsyncTaskThread.h @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbclient/Atomic.h b/fdbclient/Atomic.h index af6d1ccbc0..6643bcdafc 100644 --- a/fdbclient/Atomic.h +++ b/fdbclient/Atomic.h @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbclient/AutoPublicAddress.cpp b/fdbclient/AutoPublicAddress.cpp index 3e62b622fa..c77414ded3 100644 --- a/fdbclient/AutoPublicAddress.cpp +++ b/fdbclient/AutoPublicAddress.cpp @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbclient/BackupAgent.actor.h b/fdbclient/BackupAgent.actor.h index 5669f5d9d7..94cb10d290 100644 --- a/fdbclient/BackupAgent.actor.h +++ b/fdbclient/BackupAgent.actor.h @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbclient/BackupAgentBase.actor.cpp b/fdbclient/BackupAgentBase.actor.cpp index 74f40743d2..6033f80992 100644 --- a/fdbclient/BackupAgentBase.actor.cpp +++ b/fdbclient/BackupAgentBase.actor.cpp @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbclient/BackupContainer.actor.cpp b/fdbclient/BackupContainer.actor.cpp index ab511a7393..37b2eae015 100644 --- a/fdbclient/BackupContainer.actor.cpp +++ b/fdbclient/BackupContainer.actor.cpp @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbclient/BackupContainer.h b/fdbclient/BackupContainer.h index e8fd41c9a2..312e3b8ac7 100644 --- a/fdbclient/BackupContainer.h +++ b/fdbclient/BackupContainer.h @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbclient/BackupContainerAzureBlobStore.actor.cpp b/fdbclient/BackupContainerAzureBlobStore.actor.cpp index 3405473b9e..1ec987fc56 100644 --- a/fdbclient/BackupContainerAzureBlobStore.actor.cpp +++ b/fdbclient/BackupContainerAzureBlobStore.actor.cpp @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbclient/BackupContainerAzureBlobStore.h b/fdbclient/BackupContainerAzureBlobStore.h index 3e860e8116..77285ced16 100644 --- a/fdbclient/BackupContainerAzureBlobStore.h +++ b/fdbclient/BackupContainerAzureBlobStore.h @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbclient/BackupContainerFileSystem.actor.cpp b/fdbclient/BackupContainerFileSystem.actor.cpp index 22c5087bc0..7acbd227f2 100644 --- a/fdbclient/BackupContainerFileSystem.actor.cpp +++ b/fdbclient/BackupContainerFileSystem.actor.cpp @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbclient/BackupContainerFileSystem.h b/fdbclient/BackupContainerFileSystem.h index 8f458df010..52c5d3fc54 100644 --- a/fdbclient/BackupContainerFileSystem.h +++ b/fdbclient/BackupContainerFileSystem.h @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbclient/BackupContainerLocalDirectory.actor.cpp b/fdbclient/BackupContainerLocalDirectory.actor.cpp index f3082e9d81..45e7ef29ab 100644 --- a/fdbclient/BackupContainerLocalDirectory.actor.cpp +++ b/fdbclient/BackupContainerLocalDirectory.actor.cpp @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbclient/BackupContainerLocalDirectory.h b/fdbclient/BackupContainerLocalDirectory.h index f7c77e4636..e6e949b0d8 100644 --- a/fdbclient/BackupContainerLocalDirectory.h +++ b/fdbclient/BackupContainerLocalDirectory.h @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbclient/BackupContainerS3BlobStore.actor.cpp b/fdbclient/BackupContainerS3BlobStore.actor.cpp index b915701a3f..2240cc6741 100644 --- a/fdbclient/BackupContainerS3BlobStore.actor.cpp +++ b/fdbclient/BackupContainerS3BlobStore.actor.cpp @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbclient/BackupContainerS3BlobStore.h b/fdbclient/BackupContainerS3BlobStore.h index 9e47483adf..133313661b 100644 --- a/fdbclient/BackupContainerS3BlobStore.h +++ b/fdbclient/BackupContainerS3BlobStore.h @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbclient/BlobGranuleCommon.h b/fdbclient/BlobGranuleCommon.h index c9b4d11d43..b8ddfe2851 100644 --- a/fdbclient/BlobGranuleCommon.h +++ b/fdbclient/BlobGranuleCommon.h @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbclient/BlobGranuleFiles.cpp b/fdbclient/BlobGranuleFiles.cpp index f30e8190bf..a7e5dda5a3 100644 --- a/fdbclient/BlobGranuleFiles.cpp +++ b/fdbclient/BlobGranuleFiles.cpp @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbclient/BlobGranuleFiles.h b/fdbclient/BlobGranuleFiles.h index 448767b41d..44ec7dec75 100644 --- a/fdbclient/BlobGranuleFiles.h +++ b/fdbclient/BlobGranuleFiles.h @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2021 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbclient/BlobGranuleReader.actor.cpp b/fdbclient/BlobGranuleReader.actor.cpp index bd723df2f3..e69d1ad94f 100644 --- a/fdbclient/BlobGranuleReader.actor.cpp +++ b/fdbclient/BlobGranuleReader.actor.cpp @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbclient/BlobGranuleReader.actor.h b/fdbclient/BlobGranuleReader.actor.h index 55a7d4c18a..1b168ebc5d 100644 --- a/fdbclient/BlobGranuleReader.actor.h +++ b/fdbclient/BlobGranuleReader.actor.h @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbclient/BlobWorkerCommon.h b/fdbclient/BlobWorkerCommon.h index bc4c2a687c..71d1f579d4 100644 --- a/fdbclient/BlobWorkerCommon.h +++ b/fdbclient/BlobWorkerCommon.h @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2021 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbclient/BlobWorkerInterface.h b/fdbclient/BlobWorkerInterface.h index f95bb87b74..48fd92ddb3 100644 --- a/fdbclient/BlobWorkerInterface.h +++ b/fdbclient/BlobWorkerInterface.h @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbclient/ClientBooleanParams.cpp b/fdbclient/ClientBooleanParams.cpp index 1027fdece6..43febb31ba 100644 --- a/fdbclient/ClientBooleanParams.cpp +++ b/fdbclient/ClientBooleanParams.cpp @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbclient/ClientBooleanParams.h b/fdbclient/ClientBooleanParams.h index c078c6575e..7cf86b865d 100644 --- a/fdbclient/ClientBooleanParams.h +++ b/fdbclient/ClientBooleanParams.h @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbclient/ClientKnobCollection.cpp b/fdbclient/ClientKnobCollection.cpp index d53d982b6b..385c8d08d8 100644 --- a/fdbclient/ClientKnobCollection.cpp +++ b/fdbclient/ClientKnobCollection.cpp @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbclient/ClientKnobCollection.h b/fdbclient/ClientKnobCollection.h index 8baaa92a61..d1597f4c6e 100644 --- a/fdbclient/ClientKnobCollection.h +++ b/fdbclient/ClientKnobCollection.h @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbclient/ClientKnobs.cpp b/fdbclient/ClientKnobs.cpp index 1eb46cfe1b..a092a57285 100644 --- a/fdbclient/ClientKnobs.cpp +++ b/fdbclient/ClientKnobs.cpp @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbclient/ClientKnobs.h b/fdbclient/ClientKnobs.h index 1c4a9ca874..82b73464ad 100644 --- a/fdbclient/ClientKnobs.h +++ b/fdbclient/ClientKnobs.h @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbclient/ClientLogEvents.h b/fdbclient/ClientLogEvents.h index 53b338f569..49fbc0f298 100644 --- a/fdbclient/ClientLogEvents.h +++ b/fdbclient/ClientLogEvents.h @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbclient/ClientVersion.h b/fdbclient/ClientVersion.h index 40727e512b..fe3068affc 100644 --- a/fdbclient/ClientVersion.h +++ b/fdbclient/ClientVersion.h @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2021 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbclient/ClientWorkerInterface.h b/fdbclient/ClientWorkerInterface.h index 28caa48739..c91736d8db 100644 --- a/fdbclient/ClientWorkerInterface.h +++ b/fdbclient/ClientWorkerInterface.h @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbclient/ClusterConnectionFile.actor.cpp b/fdbclient/ClusterConnectionFile.actor.cpp index 42fe438620..01ecba277c 100644 --- a/fdbclient/ClusterConnectionFile.actor.cpp +++ b/fdbclient/ClusterConnectionFile.actor.cpp @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2021 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbclient/ClusterConnectionFile.h b/fdbclient/ClusterConnectionFile.h index e64ae6a659..623c04cd0e 100644 --- a/fdbclient/ClusterConnectionFile.h +++ b/fdbclient/ClusterConnectionFile.h @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2021 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbclient/ClusterConnectionKey.actor.cpp b/fdbclient/ClusterConnectionKey.actor.cpp index 8577e790d9..d2cdff7fd2 100644 --- a/fdbclient/ClusterConnectionKey.actor.cpp +++ b/fdbclient/ClusterConnectionKey.actor.cpp @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2021 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbclient/ClusterConnectionKey.actor.h b/fdbclient/ClusterConnectionKey.actor.h index 40cad40b44..ea6a629ec7 100644 --- a/fdbclient/ClusterConnectionKey.actor.h +++ b/fdbclient/ClusterConnectionKey.actor.h @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2021 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbclient/ClusterConnectionMemoryRecord.actor.cpp b/fdbclient/ClusterConnectionMemoryRecord.actor.cpp index d86544bdbc..6e6c040e67 100644 --- a/fdbclient/ClusterConnectionMemoryRecord.actor.cpp +++ b/fdbclient/ClusterConnectionMemoryRecord.actor.cpp @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2021 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbclient/ClusterConnectionMemoryRecord.h b/fdbclient/ClusterConnectionMemoryRecord.h index 5a556391d9..47f62eb5e8 100644 --- a/fdbclient/ClusterConnectionMemoryRecord.h +++ b/fdbclient/ClusterConnectionMemoryRecord.h @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2021 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbclient/ClusterInterface.h b/fdbclient/ClusterInterface.h index 6b86322bb6..14935f1700 100644 --- a/fdbclient/ClusterInterface.h +++ b/fdbclient/ClusterInterface.h @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbclient/CommitProxyInterface.h b/fdbclient/CommitProxyInterface.h index 2cb913b7a3..ac187a0f55 100644 --- a/fdbclient/CommitProxyInterface.h +++ b/fdbclient/CommitProxyInterface.h @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbclient/CommitTransaction.h b/fdbclient/CommitTransaction.h index bdd4a1c4d6..ed4a45f289 100644 --- a/fdbclient/CommitTransaction.h +++ b/fdbclient/CommitTransaction.h @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbclient/ConfigKnobs.cpp b/fdbclient/ConfigKnobs.cpp index 9318c949a3..03b4b09fbd 100644 --- a/fdbclient/ConfigKnobs.cpp +++ b/fdbclient/ConfigKnobs.cpp @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbclient/ConfigKnobs.h b/fdbclient/ConfigKnobs.h index d3cf74c5b8..536bca16f3 100644 --- a/fdbclient/ConfigKnobs.h +++ b/fdbclient/ConfigKnobs.h @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbclient/ConfigTransactionInterface.cpp b/fdbclient/ConfigTransactionInterface.cpp index 65f403a869..fab26f3498 100644 --- a/fdbclient/ConfigTransactionInterface.cpp +++ b/fdbclient/ConfigTransactionInterface.cpp @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbclient/ConfigTransactionInterface.h b/fdbclient/ConfigTransactionInterface.h index 4d02633f3f..7489594ad0 100644 --- a/fdbclient/ConfigTransactionInterface.h +++ b/fdbclient/ConfigTransactionInterface.h @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbclient/CoordinationInterface.h b/fdbclient/CoordinationInterface.h index bd37fb3f55..e105619e90 100644 --- a/fdbclient/CoordinationInterface.h +++ b/fdbclient/CoordinationInterface.h @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbclient/DatabaseBackupAgent.actor.cpp b/fdbclient/DatabaseBackupAgent.actor.cpp index 808707f91f..17c588b26f 100644 --- a/fdbclient/DatabaseBackupAgent.actor.cpp +++ b/fdbclient/DatabaseBackupAgent.actor.cpp @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbclient/DatabaseConfiguration.cpp b/fdbclient/DatabaseConfiguration.cpp index 5c44db08d8..2508991919 100644 --- a/fdbclient/DatabaseConfiguration.cpp +++ b/fdbclient/DatabaseConfiguration.cpp @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbclient/DatabaseConfiguration.h b/fdbclient/DatabaseConfiguration.h index 398c4c5b91..9d7012fc18 100644 --- a/fdbclient/DatabaseConfiguration.h +++ b/fdbclient/DatabaseConfiguration.h @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbclient/DatabaseContext.h b/fdbclient/DatabaseContext.h index 2fd7bfffb5..2911137058 100644 --- a/fdbclient/DatabaseContext.h +++ b/fdbclient/DatabaseContext.h @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbclient/EventTypes.actor.h b/fdbclient/EventTypes.actor.h index 628cae5adc..39a75e09dc 100644 --- a/fdbclient/EventTypes.actor.h +++ b/fdbclient/EventTypes.actor.h @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbclient/FDBOptions.h b/fdbclient/FDBOptions.h index 284d9ab0f9..b3660461e3 100644 --- a/fdbclient/FDBOptions.h +++ b/fdbclient/FDBOptions.h @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbclient/FDBTypes.cpp b/fdbclient/FDBTypes.cpp index 17390365ed..85b90313e5 100644 --- a/fdbclient/FDBTypes.cpp +++ b/fdbclient/FDBTypes.cpp @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbclient/FDBTypes.h b/fdbclient/FDBTypes.h index a10949aadc..14fd1b023b 100644 --- a/fdbclient/FDBTypes.h +++ b/fdbclient/FDBTypes.h @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbclient/FileBackupAgent.actor.cpp b/fdbclient/FileBackupAgent.actor.cpp index 17e7e6c2cb..fc1dc558c7 100644 --- a/fdbclient/FileBackupAgent.actor.cpp +++ b/fdbclient/FileBackupAgent.actor.cpp @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbclient/FluentDSampleIngestor.cpp b/fdbclient/FluentDSampleIngestor.cpp index 3ecfc40d30..d60700ffb5 100644 --- a/fdbclient/FluentDSampleIngestor.cpp +++ b/fdbclient/FluentDSampleIngestor.cpp @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2021 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbclient/GlobalConfig.actor.cpp b/fdbclient/GlobalConfig.actor.cpp index 039123e69c..4a2e4aec56 100644 --- a/fdbclient/GlobalConfig.actor.cpp +++ b/fdbclient/GlobalConfig.actor.cpp @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2021 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbclient/GlobalConfig.actor.h b/fdbclient/GlobalConfig.actor.h index 746226a045..91a78921cd 100644 --- a/fdbclient/GlobalConfig.actor.h +++ b/fdbclient/GlobalConfig.actor.h @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2021 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbclient/GlobalConfig.h b/fdbclient/GlobalConfig.h index f68ea2361e..cd458b3ac1 100644 --- a/fdbclient/GlobalConfig.h +++ b/fdbclient/GlobalConfig.h @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2021 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbclient/GrvProxyInterface.h b/fdbclient/GrvProxyInterface.h index d4b3b78bcb..10dd81546d 100644 --- a/fdbclient/GrvProxyInterface.h +++ b/fdbclient/GrvProxyInterface.h @@ -4,7 +4,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2020 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbclient/HTTP.actor.cpp b/fdbclient/HTTP.actor.cpp index 6ac53ecbbe..70964e7dbc 100644 --- a/fdbclient/HTTP.actor.cpp +++ b/fdbclient/HTTP.actor.cpp @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbclient/HTTP.h b/fdbclient/HTTP.h index c382ba9a56..d012c30ce8 100644 --- a/fdbclient/HTTP.h +++ b/fdbclient/HTTP.h @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbclient/IClientApi.h b/fdbclient/IClientApi.h index 8d224fe3d8..f726e1b3ba 100644 --- a/fdbclient/IClientApi.h +++ b/fdbclient/IClientApi.h @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbclient/IConfigTransaction.cpp b/fdbclient/IConfigTransaction.cpp index f91483eb76..334f56f220 100644 --- a/fdbclient/IConfigTransaction.cpp +++ b/fdbclient/IConfigTransaction.cpp @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbclient/IConfigTransaction.h b/fdbclient/IConfigTransaction.h index 8d6d5decd8..7b96f48e94 100644 --- a/fdbclient/IConfigTransaction.h +++ b/fdbclient/IConfigTransaction.h @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbclient/IKnobCollection.cpp b/fdbclient/IKnobCollection.cpp index 8748280784..41d0441904 100644 --- a/fdbclient/IKnobCollection.cpp +++ b/fdbclient/IKnobCollection.cpp @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbclient/IKnobCollection.h b/fdbclient/IKnobCollection.h index 266988d415..d37955ce55 100644 --- a/fdbclient/IKnobCollection.h +++ b/fdbclient/IKnobCollection.h @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbclient/ISingleThreadTransaction.cpp b/fdbclient/ISingleThreadTransaction.cpp index da387efa6a..e37213c149 100644 --- a/fdbclient/ISingleThreadTransaction.cpp +++ b/fdbclient/ISingleThreadTransaction.cpp @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbclient/ISingleThreadTransaction.h b/fdbclient/ISingleThreadTransaction.h index cc4054535f..cdbb6bf02c 100644 --- a/fdbclient/ISingleThreadTransaction.h +++ b/fdbclient/ISingleThreadTransaction.h @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbclient/JSONDoc.h b/fdbclient/JSONDoc.h index 10e1c40b4c..2fdeb7ba66 100644 --- a/fdbclient/JSONDoc.h +++ b/fdbclient/JSONDoc.h @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbclient/KeyBackedTypes.h b/fdbclient/KeyBackedTypes.h index 73b113d1ac..3733380ec2 100644 --- a/fdbclient/KeyBackedTypes.h +++ b/fdbclient/KeyBackedTypes.h @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbclient/KeyRangeMap.actor.cpp b/fdbclient/KeyRangeMap.actor.cpp index 152377cb95..c736c714bf 100644 --- a/fdbclient/KeyRangeMap.actor.cpp +++ b/fdbclient/KeyRangeMap.actor.cpp @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbclient/KeyRangeMap.h b/fdbclient/KeyRangeMap.h index 6fd9bfe5b7..88cce027a8 100644 --- a/fdbclient/KeyRangeMap.h +++ b/fdbclient/KeyRangeMap.h @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbclient/Knobs.h b/fdbclient/Knobs.h index b143c8a063..edca80d053 100644 --- a/fdbclient/Knobs.h +++ b/fdbclient/Knobs.h @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbclient/LocalClientAPI.cpp b/fdbclient/LocalClientAPI.cpp index 0bdb5177cc..6e67744741 100644 --- a/fdbclient/LocalClientAPI.cpp +++ b/fdbclient/LocalClientAPI.cpp @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2021 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbclient/LocalClientAPI.h b/fdbclient/LocalClientAPI.h index 98df61bfb8..705d833c2c 100644 --- a/fdbclient/LocalClientAPI.h +++ b/fdbclient/LocalClientAPI.h @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2021 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbclient/ManagementAPI.actor.cpp b/fdbclient/ManagementAPI.actor.cpp index 044ac79395..1b37a9ec4e 100644 --- a/fdbclient/ManagementAPI.actor.cpp +++ b/fdbclient/ManagementAPI.actor.cpp @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbclient/ManagementAPI.actor.h b/fdbclient/ManagementAPI.actor.h index a2adbb7025..64c54447a7 100644 --- a/fdbclient/ManagementAPI.actor.h +++ b/fdbclient/ManagementAPI.actor.h @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbclient/MonitorLeader.actor.cpp b/fdbclient/MonitorLeader.actor.cpp index e63824466f..509953599b 100644 --- a/fdbclient/MonitorLeader.actor.cpp +++ b/fdbclient/MonitorLeader.actor.cpp @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbclient/MonitorLeader.h b/fdbclient/MonitorLeader.h index 4aa3e5ff7f..6ceed69a63 100644 --- a/fdbclient/MonitorLeader.h +++ b/fdbclient/MonitorLeader.h @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbclient/MultiVersionAssignmentVars.h b/fdbclient/MultiVersionAssignmentVars.h index 58b68713de..0cdd67e1be 100644 --- a/fdbclient/MultiVersionAssignmentVars.h +++ b/fdbclient/MultiVersionAssignmentVars.h @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbclient/MultiVersionTransaction.actor.cpp b/fdbclient/MultiVersionTransaction.actor.cpp index 8986d0817e..a340eb3a5f 100644 --- a/fdbclient/MultiVersionTransaction.actor.cpp +++ b/fdbclient/MultiVersionTransaction.actor.cpp @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbclient/MultiVersionTransaction.h b/fdbclient/MultiVersionTransaction.h index 58bfbfd4c7..c6b793a047 100644 --- a/fdbclient/MultiVersionTransaction.h +++ b/fdbclient/MultiVersionTransaction.h @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbclient/MutationList.h b/fdbclient/MutationList.h index a4de13a18c..d78865bfc0 100644 --- a/fdbclient/MutationList.h +++ b/fdbclient/MutationList.h @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbclient/MutationLogReader.actor.cpp b/fdbclient/MutationLogReader.actor.cpp index 454f86e735..5919fdc66b 100644 --- a/fdbclient/MutationLogReader.actor.cpp +++ b/fdbclient/MutationLogReader.actor.cpp @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2021 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbclient/MutationLogReader.actor.h b/fdbclient/MutationLogReader.actor.h index 7d6d0fd29e..671d461e84 100644 --- a/fdbclient/MutationLogReader.actor.h +++ b/fdbclient/MutationLogReader.actor.h @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2021 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbclient/NameLineage.cpp b/fdbclient/NameLineage.cpp index 5f98cf73c4..a88026e52f 100644 --- a/fdbclient/NameLineage.cpp +++ b/fdbclient/NameLineage.cpp @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2021 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbclient/NameLineage.h b/fdbclient/NameLineage.h index f949aeb13a..d21631f38e 100644 --- a/fdbclient/NameLineage.h +++ b/fdbclient/NameLineage.h @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2021 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbclient/NativeAPI.actor.cpp b/fdbclient/NativeAPI.actor.cpp index aa0045fe35..694b2dc569 100644 --- a/fdbclient/NativeAPI.actor.cpp +++ b/fdbclient/NativeAPI.actor.cpp @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbclient/NativeAPI.actor.h b/fdbclient/NativeAPI.actor.h index a3c8c9e856..bb6999b7c3 100644 --- a/fdbclient/NativeAPI.actor.h +++ b/fdbclient/NativeAPI.actor.h @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbclient/Notified.h b/fdbclient/Notified.h index 55fd1a4450..fe9e40e59a 100644 --- a/fdbclient/Notified.h +++ b/fdbclient/Notified.h @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbclient/PImpl.h b/fdbclient/PImpl.h index 1aa65169db..424761efe5 100644 --- a/fdbclient/PImpl.h +++ b/fdbclient/PImpl.h @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbclient/ParallelStream.actor.cpp b/fdbclient/ParallelStream.actor.cpp index a9c3785350..ffc1a12850 100644 --- a/fdbclient/ParallelStream.actor.cpp +++ b/fdbclient/ParallelStream.actor.cpp @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbclient/ParallelStream.actor.h b/fdbclient/ParallelStream.actor.h index efe677ade3..6dd06c832d 100644 --- a/fdbclient/ParallelStream.actor.h +++ b/fdbclient/ParallelStream.actor.h @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbclient/PaxosConfigTransaction.actor.cpp b/fdbclient/PaxosConfigTransaction.actor.cpp index dd715ee9e7..b6fc75fec9 100644 --- a/fdbclient/PaxosConfigTransaction.actor.cpp +++ b/fdbclient/PaxosConfigTransaction.actor.cpp @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbclient/PaxosConfigTransaction.h b/fdbclient/PaxosConfigTransaction.h index 64192e26de..67487fff32 100644 --- a/fdbclient/PaxosConfigTransaction.h +++ b/fdbclient/PaxosConfigTransaction.h @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbclient/ProcessInterface.h b/fdbclient/ProcessInterface.h index 2c471d28da..b18a2267d2 100644 --- a/fdbclient/ProcessInterface.h +++ b/fdbclient/ProcessInterface.h @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2021 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbclient/RYWIterator.cpp b/fdbclient/RYWIterator.cpp index cfd233e3a2..3e7e18c1e8 100644 --- a/fdbclient/RYWIterator.cpp +++ b/fdbclient/RYWIterator.cpp @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbclient/RYWIterator.h b/fdbclient/RYWIterator.h index fb5b4768bd..9e3e8879d6 100644 --- a/fdbclient/RYWIterator.h +++ b/fdbclient/RYWIterator.h @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbclient/ReadYourWrites.actor.cpp b/fdbclient/ReadYourWrites.actor.cpp index 734e33665d..c7b9629196 100644 --- a/fdbclient/ReadYourWrites.actor.cpp +++ b/fdbclient/ReadYourWrites.actor.cpp @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbclient/ReadYourWrites.h b/fdbclient/ReadYourWrites.h index 887fd89c6c..907cf29023 100644 --- a/fdbclient/ReadYourWrites.h +++ b/fdbclient/ReadYourWrites.h @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbclient/RestoreInterface.cpp b/fdbclient/RestoreInterface.cpp index 4bf390f706..9e957684d7 100644 --- a/fdbclient/RestoreInterface.cpp +++ b/fdbclient/RestoreInterface.cpp @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2021 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbclient/RestoreInterface.h b/fdbclient/RestoreInterface.h index ceaece557f..bdb2499298 100644 --- a/fdbclient/RestoreInterface.h +++ b/fdbclient/RestoreInterface.h @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2021 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbclient/RunTransaction.actor.h b/fdbclient/RunTransaction.actor.h index fb381d7b64..91530a02cd 100644 --- a/fdbclient/RunTransaction.actor.h +++ b/fdbclient/RunTransaction.actor.h @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbclient/S3BlobStore.actor.cpp b/fdbclient/S3BlobStore.actor.cpp index 0abcbe441a..a9bd360cde 100644 --- a/fdbclient/S3BlobStore.actor.cpp +++ b/fdbclient/S3BlobStore.actor.cpp @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbclient/S3BlobStore.h b/fdbclient/S3BlobStore.h index fab21225c1..c259d6a4da 100644 --- a/fdbclient/S3BlobStore.h +++ b/fdbclient/S3BlobStore.h @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbclient/Schemas.cpp b/fdbclient/Schemas.cpp index 0a40b7832c..79f7974f71 100644 --- a/fdbclient/Schemas.cpp +++ b/fdbclient/Schemas.cpp @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbclient/Schemas.h b/fdbclient/Schemas.h index 88424d6a87..980122c2aa 100644 --- a/fdbclient/Schemas.h +++ b/fdbclient/Schemas.h @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbclient/ServerKnobCollection.cpp b/fdbclient/ServerKnobCollection.cpp index bbe99f8ed9..15e6eb9129 100644 --- a/fdbclient/ServerKnobCollection.cpp +++ b/fdbclient/ServerKnobCollection.cpp @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbclient/ServerKnobCollection.h b/fdbclient/ServerKnobCollection.h index bf7ac4e414..79f4576088 100644 --- a/fdbclient/ServerKnobCollection.h +++ b/fdbclient/ServerKnobCollection.h @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbclient/ServerKnobs.cpp b/fdbclient/ServerKnobs.cpp index 8693b1bd95..94a5559119 100644 --- a/fdbclient/ServerKnobs.cpp +++ b/fdbclient/ServerKnobs.cpp @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbclient/ServerKnobs.h b/fdbclient/ServerKnobs.h index df5e0b6cae..4575740ff8 100644 --- a/fdbclient/ServerKnobs.h +++ b/fdbclient/ServerKnobs.h @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbclient/SimpleConfigTransaction.actor.cpp b/fdbclient/SimpleConfigTransaction.actor.cpp index a070b8a53d..ecb8380d4c 100644 --- a/fdbclient/SimpleConfigTransaction.actor.cpp +++ b/fdbclient/SimpleConfigTransaction.actor.cpp @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbclient/SimpleConfigTransaction.h b/fdbclient/SimpleConfigTransaction.h index bb013f2c51..83d8411518 100644 --- a/fdbclient/SimpleConfigTransaction.h +++ b/fdbclient/SimpleConfigTransaction.h @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbclient/SnapshotCache.h b/fdbclient/SnapshotCache.h index f4e110edc4..7944faa0dc 100644 --- a/fdbclient/SnapshotCache.h +++ b/fdbclient/SnapshotCache.h @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbclient/SpecialKeySpace.actor.cpp b/fdbclient/SpecialKeySpace.actor.cpp index b3b3d97baf..21797cbd67 100644 --- a/fdbclient/SpecialKeySpace.actor.cpp +++ b/fdbclient/SpecialKeySpace.actor.cpp @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2020 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbclient/SpecialKeySpace.actor.h b/fdbclient/SpecialKeySpace.actor.h index df5e1d3431..acf4db480c 100644 --- a/fdbclient/SpecialKeySpace.actor.h +++ b/fdbclient/SpecialKeySpace.actor.h @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2020 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbclient/StackLineage.cpp b/fdbclient/StackLineage.cpp index b1731b879c..74195ae1f2 100644 --- a/fdbclient/StackLineage.cpp +++ b/fdbclient/StackLineage.cpp @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2021 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbclient/StackLineage.h b/fdbclient/StackLineage.h index a94d20a836..cbcc741ed0 100644 --- a/fdbclient/StackLineage.h +++ b/fdbclient/StackLineage.h @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2021 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbclient/Status.h b/fdbclient/Status.h index aa32f2f0c4..90f69ebd83 100644 --- a/fdbclient/Status.h +++ b/fdbclient/Status.h @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbclient/StatusClient.actor.cpp b/fdbclient/StatusClient.actor.cpp index aaa3970f47..6b329ba1b9 100644 --- a/fdbclient/StatusClient.actor.cpp +++ b/fdbclient/StatusClient.actor.cpp @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbclient/StatusClient.h b/fdbclient/StatusClient.h index 6b780163a4..0fe8f19b58 100755 --- a/fdbclient/StatusClient.h +++ b/fdbclient/StatusClient.h @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbclient/StorageServerInterface.cpp b/fdbclient/StorageServerInterface.cpp index ed71b1be96..b7efbfbc83 100644 --- a/fdbclient/StorageServerInterface.cpp +++ b/fdbclient/StorageServerInterface.cpp @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbclient/StorageServerInterface.h b/fdbclient/StorageServerInterface.h index 592d2dd167..4417d36800 100644 --- a/fdbclient/StorageServerInterface.h +++ b/fdbclient/StorageServerInterface.h @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbclient/Subspace.cpp b/fdbclient/Subspace.cpp index c585ffb886..b71c5a0739 100644 --- a/fdbclient/Subspace.cpp +++ b/fdbclient/Subspace.cpp @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbclient/Subspace.h b/fdbclient/Subspace.h index fd60ca4b13..ef88fc3855 100644 --- a/fdbclient/Subspace.h +++ b/fdbclient/Subspace.h @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbclient/SystemData.cpp b/fdbclient/SystemData.cpp index 9d1329f98b..c044559e75 100644 --- a/fdbclient/SystemData.cpp +++ b/fdbclient/SystemData.cpp @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbclient/SystemData.h b/fdbclient/SystemData.h index 228c058d77..bc926c8227 100644 --- a/fdbclient/SystemData.h +++ b/fdbclient/SystemData.h @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbclient/TagThrottle.actor.cpp b/fdbclient/TagThrottle.actor.cpp index 613cfa3d90..16c0b0489c 100644 --- a/fdbclient/TagThrottle.actor.cpp +++ b/fdbclient/TagThrottle.actor.cpp @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2020 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbclient/TagThrottle.actor.h b/fdbclient/TagThrottle.actor.h index d8d4f50d0c..127dac79fd 100644 --- a/fdbclient/TagThrottle.actor.h +++ b/fdbclient/TagThrottle.actor.h @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2020 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbclient/TaskBucket.actor.cpp b/fdbclient/TaskBucket.actor.cpp index a1f6526fd3..dc3d75259a 100644 --- a/fdbclient/TaskBucket.actor.cpp +++ b/fdbclient/TaskBucket.actor.cpp @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbclient/TaskBucket.h b/fdbclient/TaskBucket.h index cfd4cd50ea..c87a353797 100644 --- a/fdbclient/TaskBucket.h +++ b/fdbclient/TaskBucket.h @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbclient/TestKnobCollection.cpp b/fdbclient/TestKnobCollection.cpp index 86c4e2863a..086ed083d2 100644 --- a/fdbclient/TestKnobCollection.cpp +++ b/fdbclient/TestKnobCollection.cpp @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbclient/TestKnobCollection.h b/fdbclient/TestKnobCollection.h index 10877ba6dc..af8e5790d0 100644 --- a/fdbclient/TestKnobCollection.h +++ b/fdbclient/TestKnobCollection.h @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbclient/ThreadSafeTransaction.cpp b/fdbclient/ThreadSafeTransaction.cpp index fb0cb5b578..146951b313 100644 --- a/fdbclient/ThreadSafeTransaction.cpp +++ b/fdbclient/ThreadSafeTransaction.cpp @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbclient/ThreadSafeTransaction.h b/fdbclient/ThreadSafeTransaction.h index b3c46ea6d4..f37cf84bf1 100644 --- a/fdbclient/ThreadSafeTransaction.h +++ b/fdbclient/ThreadSafeTransaction.h @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbclient/TransactionLineage.cpp b/fdbclient/TransactionLineage.cpp index 9ef0f21e1b..ab338a1c3f 100644 --- a/fdbclient/TransactionLineage.cpp +++ b/fdbclient/TransactionLineage.cpp @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2021 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbclient/TransactionLineage.h b/fdbclient/TransactionLineage.h index a24501c6b0..6eed26b805 100644 --- a/fdbclient/TransactionLineage.h +++ b/fdbclient/TransactionLineage.h @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2021 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbclient/Tuple.cpp b/fdbclient/Tuple.cpp index ab1fcb0314..ec74abaeac 100644 --- a/fdbclient/Tuple.cpp +++ b/fdbclient/Tuple.cpp @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbclient/Tuple.h b/fdbclient/Tuple.h index 62feba307b..5c357af712 100644 --- a/fdbclient/Tuple.h +++ b/fdbclient/Tuple.h @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbclient/VersionedMap.actor.h b/fdbclient/VersionedMap.actor.h index be506daf08..bc7f1c9035 100644 --- a/fdbclient/VersionedMap.actor.h +++ b/fdbclient/VersionedMap.actor.h @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbclient/VersionedMap.h b/fdbclient/VersionedMap.h index 5744d90743..ad5d9f32f0 100644 --- a/fdbclient/VersionedMap.h +++ b/fdbclient/VersionedMap.h @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbclient/WellKnownEndpoints.h b/fdbclient/WellKnownEndpoints.h index 6452e62958..5db3f34cea 100644 --- a/fdbclient/WellKnownEndpoints.h +++ b/fdbclient/WellKnownEndpoints.h @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2021 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbclient/WriteMap.h b/fdbclient/WriteMap.h index 129509b1b4..05793313f0 100644 --- a/fdbclient/WriteMap.h +++ b/fdbclient/WriteMap.h @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbmonitor/fdbmonitor.cpp b/fdbmonitor/fdbmonitor.cpp index a54dcd6736..0018e89d79 100644 --- a/fdbmonitor/fdbmonitor.cpp +++ b/fdbmonitor/fdbmonitor.cpp @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbrpc/ActorFuzz.actor.cpp b/fdbrpc/ActorFuzz.actor.cpp index 349a4c7e2a..b8807a2ccb 100644 --- a/fdbrpc/ActorFuzz.actor.cpp +++ b/fdbrpc/ActorFuzz.actor.cpp @@ -4,7 +4,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbrpc/ActorFuzz.h b/fdbrpc/ActorFuzz.h index 33796a42a0..996f51563c 100644 --- a/fdbrpc/ActorFuzz.h +++ b/fdbrpc/ActorFuzz.h @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbrpc/AsyncFileCached.actor.cpp b/fdbrpc/AsyncFileCached.actor.cpp index f91e4ab10d..1ea7c36aa4 100644 --- a/fdbrpc/AsyncFileCached.actor.cpp +++ b/fdbrpc/AsyncFileCached.actor.cpp @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbrpc/AsyncFileCached.actor.h b/fdbrpc/AsyncFileCached.actor.h index f152cc3db3..8c986db9b5 100644 --- a/fdbrpc/AsyncFileCached.actor.h +++ b/fdbrpc/AsyncFileCached.actor.h @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbrpc/AsyncFileChaos.actor.h b/fdbrpc/AsyncFileChaos.actor.h index 2e72a826e2..2e5cb4ff47 100644 --- a/fdbrpc/AsyncFileChaos.actor.h +++ b/fdbrpc/AsyncFileChaos.actor.h @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbrpc/AsyncFileEIO.actor.h b/fdbrpc/AsyncFileEIO.actor.h index ea1d8d416e..21360fe7e1 100644 --- a/fdbrpc/AsyncFileEIO.actor.h +++ b/fdbrpc/AsyncFileEIO.actor.h @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbrpc/AsyncFileEncrypted.actor.cpp b/fdbrpc/AsyncFileEncrypted.actor.cpp index 88a54faa3c..bf1c44da59 100644 --- a/fdbrpc/AsyncFileEncrypted.actor.cpp +++ b/fdbrpc/AsyncFileEncrypted.actor.cpp @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbrpc/AsyncFileEncrypted.h b/fdbrpc/AsyncFileEncrypted.h index bc345d043c..a01c32f8cf 100644 --- a/fdbrpc/AsyncFileEncrypted.h +++ b/fdbrpc/AsyncFileEncrypted.h @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbrpc/AsyncFileKAIO.actor.h b/fdbrpc/AsyncFileKAIO.actor.h index 68568527bd..4294ee4724 100644 --- a/fdbrpc/AsyncFileKAIO.actor.h +++ b/fdbrpc/AsyncFileKAIO.actor.h @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbrpc/AsyncFileNonDurable.actor.cpp b/fdbrpc/AsyncFileNonDurable.actor.cpp index 389caa8003..f7b41c5ae2 100644 --- a/fdbrpc/AsyncFileNonDurable.actor.cpp +++ b/fdbrpc/AsyncFileNonDurable.actor.cpp @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbrpc/AsyncFileNonDurable.actor.h b/fdbrpc/AsyncFileNonDurable.actor.h index ffbe7746a6..6396f16e50 100644 --- a/fdbrpc/AsyncFileNonDurable.actor.h +++ b/fdbrpc/AsyncFileNonDurable.actor.h @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbrpc/AsyncFileReadAhead.actor.h b/fdbrpc/AsyncFileReadAhead.actor.h index 5bd951e7c4..48be5f92f3 100644 --- a/fdbrpc/AsyncFileReadAhead.actor.h +++ b/fdbrpc/AsyncFileReadAhead.actor.h @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbrpc/AsyncFileWinASIO.actor.h b/fdbrpc/AsyncFileWinASIO.actor.h index 6fdd165869..168c257e00 100644 --- a/fdbrpc/AsyncFileWinASIO.actor.h +++ b/fdbrpc/AsyncFileWinASIO.actor.h @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbrpc/AsyncFileWriteChecker.cpp b/fdbrpc/AsyncFileWriteChecker.cpp index c421520419..8e8a3a55d9 100644 --- a/fdbrpc/AsyncFileWriteChecker.cpp +++ b/fdbrpc/AsyncFileWriteChecker.cpp @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbrpc/AsyncFileWriteChecker.h b/fdbrpc/AsyncFileWriteChecker.h index be94f29e9f..d474e2ca92 100644 --- a/fdbrpc/AsyncFileWriteChecker.h +++ b/fdbrpc/AsyncFileWriteChecker.h @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbrpc/ContinuousSample.h b/fdbrpc/ContinuousSample.h index b8cc25aee0..11a9d35d46 100644 --- a/fdbrpc/ContinuousSample.h +++ b/fdbrpc/ContinuousSample.h @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbrpc/FailureMonitor.actor.cpp b/fdbrpc/FailureMonitor.actor.cpp index 31ad94d4f9..dcb4052cf5 100644 --- a/fdbrpc/FailureMonitor.actor.cpp +++ b/fdbrpc/FailureMonitor.actor.cpp @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbrpc/FailureMonitor.h b/fdbrpc/FailureMonitor.h index 795a6e9953..280f74d8cf 100644 --- a/fdbrpc/FailureMonitor.h +++ b/fdbrpc/FailureMonitor.h @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbrpc/FlowTests.actor.cpp b/fdbrpc/FlowTests.actor.cpp index e08963073f..5c592ae0cb 100644 --- a/fdbrpc/FlowTests.actor.cpp +++ b/fdbrpc/FlowTests.actor.cpp @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbrpc/FlowTransport.actor.cpp b/fdbrpc/FlowTransport.actor.cpp index 2c59be9af2..fe32bd1adb 100644 --- a/fdbrpc/FlowTransport.actor.cpp +++ b/fdbrpc/FlowTransport.actor.cpp @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbrpc/FlowTransport.h b/fdbrpc/FlowTransport.h index 24daae400a..3a84b4162a 100644 --- a/fdbrpc/FlowTransport.h +++ b/fdbrpc/FlowTransport.h @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbrpc/HealthMonitor.actor.cpp b/fdbrpc/HealthMonitor.actor.cpp index 4ea1712e8b..db30979d0a 100644 --- a/fdbrpc/HealthMonitor.actor.cpp +++ b/fdbrpc/HealthMonitor.actor.cpp @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2020 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbrpc/HealthMonitor.h b/fdbrpc/HealthMonitor.h index 1275c3ac7e..d9e2bc8ae1 100644 --- a/fdbrpc/HealthMonitor.h +++ b/fdbrpc/HealthMonitor.h @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2020 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbrpc/IAsyncFile.actor.cpp b/fdbrpc/IAsyncFile.actor.cpp index bfe955b635..591b7adc2a 100644 --- a/fdbrpc/IAsyncFile.actor.cpp +++ b/fdbrpc/IAsyncFile.actor.cpp @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbrpc/IAsyncFile.h b/fdbrpc/IAsyncFile.h index 033b9292aa..31067fb637 100644 --- a/fdbrpc/IAsyncFile.h +++ b/fdbrpc/IAsyncFile.h @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbrpc/IRateControl.h b/fdbrpc/IRateControl.h index 83ad94b478..28a2408537 100644 --- a/fdbrpc/IRateControl.h +++ b/fdbrpc/IRateControl.h @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbrpc/LoadBalance.actor.cpp b/fdbrpc/LoadBalance.actor.cpp index 8934a36357..1d02e35cbe 100644 --- a/fdbrpc/LoadBalance.actor.cpp +++ b/fdbrpc/LoadBalance.actor.cpp @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2021 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbrpc/LoadBalance.actor.h b/fdbrpc/LoadBalance.actor.h index cb9321d3d1..76326c48a4 100644 --- a/fdbrpc/LoadBalance.actor.h +++ b/fdbrpc/LoadBalance.actor.h @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbrpc/LoadBalance.h b/fdbrpc/LoadBalance.h index 3824c373f7..cf5a1f4816 100644 --- a/fdbrpc/LoadBalance.h +++ b/fdbrpc/LoadBalance.h @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbrpc/LoadPlugin.h b/fdbrpc/LoadPlugin.h index 390823ddef..fd70f96ce7 100644 --- a/fdbrpc/LoadPlugin.h +++ b/fdbrpc/LoadPlugin.h @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbrpc/Locality.cpp b/fdbrpc/Locality.cpp index 4deec7f517..bd4c3a9461 100644 --- a/fdbrpc/Locality.cpp +++ b/fdbrpc/Locality.cpp @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbrpc/Locality.h b/fdbrpc/Locality.h index 25c3eeb22a..3e8d07f6f4 100644 --- a/fdbrpc/Locality.h +++ b/fdbrpc/Locality.h @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbrpc/MultiInterface.h b/fdbrpc/MultiInterface.h index 739b7285fa..4f15dbf087 100644 --- a/fdbrpc/MultiInterface.h +++ b/fdbrpc/MultiInterface.h @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbrpc/Net2FileSystem.cpp b/fdbrpc/Net2FileSystem.cpp index 8815fe8e9b..b6460b172c 100644 --- a/fdbrpc/Net2FileSystem.cpp +++ b/fdbrpc/Net2FileSystem.cpp @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbrpc/Net2FileSystem.h b/fdbrpc/Net2FileSystem.h index adb1b2139d..09a9a1ad2c 100644 --- a/fdbrpc/Net2FileSystem.h +++ b/fdbrpc/Net2FileSystem.h @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbrpc/PerfMetric.cpp b/fdbrpc/PerfMetric.cpp index 3122166c5a..c3caa1009e 100644 --- a/fdbrpc/PerfMetric.cpp +++ b/fdbrpc/PerfMetric.cpp @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2021 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbrpc/PerfMetric.h b/fdbrpc/PerfMetric.h index 2f1967ecda..256dbf50f6 100644 --- a/fdbrpc/PerfMetric.h +++ b/fdbrpc/PerfMetric.h @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbrpc/QueueModel.cpp b/fdbrpc/QueueModel.cpp index e9c2ddca4e..4a0a02fef6 100644 --- a/fdbrpc/QueueModel.cpp +++ b/fdbrpc/QueueModel.cpp @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbrpc/QueueModel.h b/fdbrpc/QueueModel.h index f7eedc6fa8..8733e3b811 100644 --- a/fdbrpc/QueueModel.h +++ b/fdbrpc/QueueModel.h @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbrpc/RangeMap.h b/fdbrpc/RangeMap.h index f60c8f0a92..09f5163b74 100644 --- a/fdbrpc/RangeMap.h +++ b/fdbrpc/RangeMap.h @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbrpc/Replication.cpp b/fdbrpc/Replication.cpp index 9fc4d46542..04d75a3672 100644 --- a/fdbrpc/Replication.cpp +++ b/fdbrpc/Replication.cpp @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbrpc/Replication.h b/fdbrpc/Replication.h index c562128651..a3a9806495 100644 --- a/fdbrpc/Replication.h +++ b/fdbrpc/Replication.h @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbrpc/ReplicationPolicy.cpp b/fdbrpc/ReplicationPolicy.cpp index 4c822d689c..a31cd89190 100644 --- a/fdbrpc/ReplicationPolicy.cpp +++ b/fdbrpc/ReplicationPolicy.cpp @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbrpc/ReplicationPolicy.h b/fdbrpc/ReplicationPolicy.h index c2401a4bc0..6de8d701c5 100644 --- a/fdbrpc/ReplicationPolicy.h +++ b/fdbrpc/ReplicationPolicy.h @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbrpc/ReplicationTypes.cpp b/fdbrpc/ReplicationTypes.cpp index ffb24e94dc..c0a105cf12 100644 --- a/fdbrpc/ReplicationTypes.cpp +++ b/fdbrpc/ReplicationTypes.cpp @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbrpc/ReplicationTypes.h b/fdbrpc/ReplicationTypes.h index dab5c9f0d0..ba0715d4ab 100644 --- a/fdbrpc/ReplicationTypes.h +++ b/fdbrpc/ReplicationTypes.h @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbrpc/ReplicationUtils.cpp b/fdbrpc/ReplicationUtils.cpp index 55f252ad44..cd84eea44f 100644 --- a/fdbrpc/ReplicationUtils.cpp +++ b/fdbrpc/ReplicationUtils.cpp @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbrpc/ReplicationUtils.h b/fdbrpc/ReplicationUtils.h index a7d5507a41..911c912ff8 100644 --- a/fdbrpc/ReplicationUtils.h +++ b/fdbrpc/ReplicationUtils.h @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbrpc/SimExternalConnection.actor.cpp b/fdbrpc/SimExternalConnection.actor.cpp index 70a97eb34e..38f6df5c72 100644 --- a/fdbrpc/SimExternalConnection.actor.cpp +++ b/fdbrpc/SimExternalConnection.actor.cpp @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbrpc/SimExternalConnection.h b/fdbrpc/SimExternalConnection.h index de62fff2bf..0b5a071155 100644 --- a/fdbrpc/SimExternalConnection.h +++ b/fdbrpc/SimExternalConnection.h @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbrpc/Smoother.h b/fdbrpc/Smoother.h index 80b9a6bbc3..d806a3d2aa 100644 --- a/fdbrpc/Smoother.h +++ b/fdbrpc/Smoother.h @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbrpc/Stats.actor.cpp b/fdbrpc/Stats.actor.cpp index ced47ea76d..274a4ec92e 100644 --- a/fdbrpc/Stats.actor.cpp +++ b/fdbrpc/Stats.actor.cpp @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbrpc/Stats.h b/fdbrpc/Stats.h index 704c2b8704..f8a15e7c16 100644 --- a/fdbrpc/Stats.h +++ b/fdbrpc/Stats.h @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbrpc/TSSComparison.h b/fdbrpc/TSSComparison.h index fcf3286784..3c0765c948 100644 --- a/fdbrpc/TSSComparison.h +++ b/fdbrpc/TSSComparison.h @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbrpc/TimedRequest.h b/fdbrpc/TimedRequest.h index 6b2b48f26d..07a8c0bfd3 100644 --- a/fdbrpc/TimedRequest.h +++ b/fdbrpc/TimedRequest.h @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2019 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbrpc/TraceFileIO.cpp b/fdbrpc/TraceFileIO.cpp index de18fada7c..fc5552f67d 100644 --- a/fdbrpc/TraceFileIO.cpp +++ b/fdbrpc/TraceFileIO.cpp @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbrpc/TraceFileIO.h b/fdbrpc/TraceFileIO.h index d5a97ccaa2..75a31a21c9 100644 --- a/fdbrpc/TraceFileIO.h +++ b/fdbrpc/TraceFileIO.h @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbrpc/dsltest.actor.cpp b/fdbrpc/dsltest.actor.cpp index 9fd43e02c7..fa33301145 100644 --- a/fdbrpc/dsltest.actor.cpp +++ b/fdbrpc/dsltest.actor.cpp @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbrpc/fdbrpc.h b/fdbrpc/fdbrpc.h index fd80249034..e97eb82acf 100644 --- a/fdbrpc/fdbrpc.h +++ b/fdbrpc/fdbrpc.h @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbrpc/genericactors.actor.cpp b/fdbrpc/genericactors.actor.cpp index e99d951fc8..8eb9ba49b1 100644 --- a/fdbrpc/genericactors.actor.cpp +++ b/fdbrpc/genericactors.actor.cpp @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbrpc/genericactors.actor.h b/fdbrpc/genericactors.actor.h index 46a79d29cf..9bf08ceac9 100644 --- a/fdbrpc/genericactors.actor.h +++ b/fdbrpc/genericactors.actor.h @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbrpc/linux_kaio.h b/fdbrpc/linux_kaio.h index 112e234016..f7e6e892f1 100644 --- a/fdbrpc/linux_kaio.h +++ b/fdbrpc/linux_kaio.h @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbrpc/networksender.actor.h b/fdbrpc/networksender.actor.h index 3c438bfa85..ad805c14f3 100644 --- a/fdbrpc/networksender.actor.h +++ b/fdbrpc/networksender.actor.h @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbrpc/sim2.actor.cpp b/fdbrpc/sim2.actor.cpp index 2de43a3bf7..ba3a247bb6 100644 --- a/fdbrpc/sim2.actor.cpp +++ b/fdbrpc/sim2.actor.cpp @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbrpc/sim_validation.cpp b/fdbrpc/sim_validation.cpp index d2344dbe70..292c5255f3 100644 --- a/fdbrpc/sim_validation.cpp +++ b/fdbrpc/sim_validation.cpp @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbrpc/sim_validation.h b/fdbrpc/sim_validation.h index 975b6a424f..38eca0884b 100644 --- a/fdbrpc/sim_validation.h +++ b/fdbrpc/sim_validation.h @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbrpc/simulator.h b/fdbrpc/simulator.h index 86c0b8c6b2..464e473ae0 100644 --- a/fdbrpc/simulator.h +++ b/fdbrpc/simulator.h @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbserver/ApplyMetadataMutation.cpp b/fdbserver/ApplyMetadataMutation.cpp index 5afc862b92..ef53e278f2 100644 --- a/fdbserver/ApplyMetadataMutation.cpp +++ b/fdbserver/ApplyMetadataMutation.cpp @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2019 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbserver/ApplyMetadataMutation.h b/fdbserver/ApplyMetadataMutation.h index bdceaf8d8e..1a364888d5 100644 --- a/fdbserver/ApplyMetadataMutation.h +++ b/fdbserver/ApplyMetadataMutation.h @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbserver/ArtMutationBuffer.h b/fdbserver/ArtMutationBuffer.h index 803511bc01..21d7a68ef3 100644 --- a/fdbserver/ArtMutationBuffer.h +++ b/fdbserver/ArtMutationBuffer.h @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2020 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbserver/BackupInterface.h b/fdbserver/BackupInterface.h index 12b189ccdb..ce413563af 100644 --- a/fdbserver/BackupInterface.h +++ b/fdbserver/BackupInterface.h @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2019 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbserver/BackupProgress.actor.cpp b/fdbserver/BackupProgress.actor.cpp index 28af84eaac..1055cae265 100644 --- a/fdbserver/BackupProgress.actor.cpp +++ b/fdbserver/BackupProgress.actor.cpp @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2020 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbserver/BackupProgress.actor.h b/fdbserver/BackupProgress.actor.h index aec6f8c5b9..7a56732345 100644 --- a/fdbserver/BackupProgress.actor.h +++ b/fdbserver/BackupProgress.actor.h @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2019 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbserver/BackupWorker.actor.cpp b/fdbserver/BackupWorker.actor.cpp index 895497fff7..9d94ba27b5 100644 --- a/fdbserver/BackupWorker.actor.cpp +++ b/fdbserver/BackupWorker.actor.cpp @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2019 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbserver/BlobManager.actor.cpp b/fdbserver/BlobManager.actor.cpp index 24601dab8b..82e71eb914 100644 --- a/fdbserver/BlobManager.actor.cpp +++ b/fdbserver/BlobManager.actor.cpp @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbserver/BlobManagerInterface.h b/fdbserver/BlobManagerInterface.h index 4f6efcd8c5..fc030ec8ca 100644 --- a/fdbserver/BlobManagerInterface.h +++ b/fdbserver/BlobManagerInterface.h @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbserver/BlobWorker.actor.cpp b/fdbserver/BlobWorker.actor.cpp index 7e497fd3cc..e1cc499db4 100644 --- a/fdbserver/BlobWorker.actor.cpp +++ b/fdbserver/BlobWorker.actor.cpp @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbserver/ClusterController.actor.cpp b/fdbserver/ClusterController.actor.cpp index b5c5c67348..ebdf4c8b1e 100644 --- a/fdbserver/ClusterController.actor.cpp +++ b/fdbserver/ClusterController.actor.cpp @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2019 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbserver/ClusterController.actor.h b/fdbserver/ClusterController.actor.h index 8a1fa1e3cf..2ce2f0a23e 100644 --- a/fdbserver/ClusterController.actor.h +++ b/fdbserver/ClusterController.actor.h @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2020 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbserver/ClusterRecovery.actor.cpp b/fdbserver/ClusterRecovery.actor.cpp index 512539a607..ac8fee9b9e 100644 --- a/fdbserver/ClusterRecovery.actor.cpp +++ b/fdbserver/ClusterRecovery.actor.cpp @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2019 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbserver/ClusterRecovery.actor.h b/fdbserver/ClusterRecovery.actor.h index d53e136484..36dcb1bed9 100644 --- a/fdbserver/ClusterRecovery.actor.h +++ b/fdbserver/ClusterRecovery.actor.h @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2020 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbserver/CommitProxyServer.actor.cpp b/fdbserver/CommitProxyServer.actor.cpp index 04184c430a..137084c640 100644 --- a/fdbserver/CommitProxyServer.actor.cpp +++ b/fdbserver/CommitProxyServer.actor.cpp @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbserver/CompactMap.cpp b/fdbserver/CompactMap.cpp index 8db6ab6727..1854cd4d6b 100644 --- a/fdbserver/CompactMap.cpp +++ b/fdbserver/CompactMap.cpp @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbserver/ConfigBroadcastInterface.h b/fdbserver/ConfigBroadcastInterface.h index 71ba71fdf9..cb75e1a251 100644 --- a/fdbserver/ConfigBroadcastInterface.h +++ b/fdbserver/ConfigBroadcastInterface.h @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbserver/ConfigBroadcaster.actor.cpp b/fdbserver/ConfigBroadcaster.actor.cpp index 0ba9e4d6cf..faf0eb0fa0 100644 --- a/fdbserver/ConfigBroadcaster.actor.cpp +++ b/fdbserver/ConfigBroadcaster.actor.cpp @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbserver/ConfigBroadcaster.h b/fdbserver/ConfigBroadcaster.h index 0e45c63c41..bddc1f5209 100644 --- a/fdbserver/ConfigBroadcaster.h +++ b/fdbserver/ConfigBroadcaster.h @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbserver/ConfigDatabaseUnitTests.actor.cpp b/fdbserver/ConfigDatabaseUnitTests.actor.cpp index 26c1a3e925..dba90dcbbd 100644 --- a/fdbserver/ConfigDatabaseUnitTests.actor.cpp +++ b/fdbserver/ConfigDatabaseUnitTests.actor.cpp @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbserver/ConfigFollowerInterface.cpp b/fdbserver/ConfigFollowerInterface.cpp index f294af52b3..1bb9b424be 100644 --- a/fdbserver/ConfigFollowerInterface.cpp +++ b/fdbserver/ConfigFollowerInterface.cpp @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbserver/ConfigFollowerInterface.h b/fdbserver/ConfigFollowerInterface.h index 5b0d264c56..9b901874f7 100644 --- a/fdbserver/ConfigFollowerInterface.h +++ b/fdbserver/ConfigFollowerInterface.h @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbserver/ConfigNode.actor.cpp b/fdbserver/ConfigNode.actor.cpp index 7152c8ce9b..c76ee882cf 100644 --- a/fdbserver/ConfigNode.actor.cpp +++ b/fdbserver/ConfigNode.actor.cpp @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbserver/ConfigNode.h b/fdbserver/ConfigNode.h index 7d4513b44e..333652893b 100644 --- a/fdbserver/ConfigNode.h +++ b/fdbserver/ConfigNode.h @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbserver/ConflictSet.h b/fdbserver/ConflictSet.h index f68e2d8aac..645e11e016 100644 --- a/fdbserver/ConflictSet.h +++ b/fdbserver/ConflictSet.h @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbserver/CoordinatedState.actor.cpp b/fdbserver/CoordinatedState.actor.cpp index 4ffdd057a2..7a2c5e028c 100644 --- a/fdbserver/CoordinatedState.actor.cpp +++ b/fdbserver/CoordinatedState.actor.cpp @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbserver/CoordinatedState.h b/fdbserver/CoordinatedState.h index b657d96138..da88be04eb 100644 --- a/fdbserver/CoordinatedState.h +++ b/fdbserver/CoordinatedState.h @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbserver/Coordination.actor.cpp b/fdbserver/Coordination.actor.cpp index eac75e9f8b..851007fef7 100644 --- a/fdbserver/Coordination.actor.cpp +++ b/fdbserver/Coordination.actor.cpp @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbserver/CoordinationInterface.h b/fdbserver/CoordinationInterface.h index 634ee1b14e..87998e20bb 100644 --- a/fdbserver/CoordinationInterface.h +++ b/fdbserver/CoordinationInterface.h @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbserver/CoroFlow.actor.cpp b/fdbserver/CoroFlow.actor.cpp index 23f08b66cf..1af12ffda2 100644 --- a/fdbserver/CoroFlow.actor.cpp +++ b/fdbserver/CoroFlow.actor.cpp @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbserver/CoroFlow.h b/fdbserver/CoroFlow.h index 8928f012e9..d074b4ec90 100644 --- a/fdbserver/CoroFlow.h +++ b/fdbserver/CoroFlow.h @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbserver/CoroFlowCoro.actor.cpp b/fdbserver/CoroFlowCoro.actor.cpp index bf9c626725..fbf27daf36 100644 --- a/fdbserver/CoroFlowCoro.actor.cpp +++ b/fdbserver/CoroFlowCoro.actor.cpp @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbserver/DBCoreState.h b/fdbserver/DBCoreState.h index d024e2a081..7c06432498 100644 --- a/fdbserver/DBCoreState.h +++ b/fdbserver/DBCoreState.h @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbserver/DataDistribution.actor.cpp b/fdbserver/DataDistribution.actor.cpp index 36036e34ac..9227cef3e4 100644 --- a/fdbserver/DataDistribution.actor.cpp +++ b/fdbserver/DataDistribution.actor.cpp @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbserver/DataDistribution.actor.h b/fdbserver/DataDistribution.actor.h index e9bd7da53e..50290a4f52 100644 --- a/fdbserver/DataDistribution.actor.h +++ b/fdbserver/DataDistribution.actor.h @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbserver/DataDistributionQueue.actor.cpp b/fdbserver/DataDistributionQueue.actor.cpp index f89244c5b7..8faa08e3f9 100644 --- a/fdbserver/DataDistributionQueue.actor.cpp +++ b/fdbserver/DataDistributionQueue.actor.cpp @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbserver/DataDistributionTracker.actor.cpp b/fdbserver/DataDistributionTracker.actor.cpp index d20cc65bec..e68f3df2a0 100644 --- a/fdbserver/DataDistributionTracker.actor.cpp +++ b/fdbserver/DataDistributionTracker.actor.cpp @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbserver/DataDistributorInterface.h b/fdbserver/DataDistributorInterface.h index 5db806eb46..b858278f98 100644 --- a/fdbserver/DataDistributorInterface.h +++ b/fdbserver/DataDistributorInterface.h @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2019 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbserver/DeltaTree.h b/fdbserver/DeltaTree.h index 0c0af23794..aa2097464d 100644 --- a/fdbserver/DeltaTree.h +++ b/fdbserver/DeltaTree.h @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbserver/DiskQueue.actor.cpp b/fdbserver/DiskQueue.actor.cpp index adfd682c1a..2ea74508d0 100644 --- a/fdbserver/DiskQueue.actor.cpp +++ b/fdbserver/DiskQueue.actor.cpp @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbserver/EncryptKeyProxy.actor.cpp b/fdbserver/EncryptKeyProxy.actor.cpp index 4fca28d2f3..1fbd3ede93 100644 --- a/fdbserver/EncryptKeyProxy.actor.cpp +++ b/fdbserver/EncryptKeyProxy.actor.cpp @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2021 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbserver/EncryptKeyProxyInterface.h b/fdbserver/EncryptKeyProxyInterface.h index ad2da2f8ba..030d7870ae 100644 --- a/fdbserver/EncryptKeyProxyInterface.h +++ b/fdbserver/EncryptKeyProxyInterface.h @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2021 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbserver/GrvProxyServer.actor.cpp b/fdbserver/GrvProxyServer.actor.cpp index 1523acbd39..9d3e4bfde8 100644 --- a/fdbserver/GrvProxyServer.actor.cpp +++ b/fdbserver/GrvProxyServer.actor.cpp @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2020 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbserver/IConfigConsumer.cpp b/fdbserver/IConfigConsumer.cpp index 5ac97bf2b2..9d4c7d7e70 100644 --- a/fdbserver/IConfigConsumer.cpp +++ b/fdbserver/IConfigConsumer.cpp @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbserver/IConfigConsumer.h b/fdbserver/IConfigConsumer.h index cf3b578c2c..f485d9f698 100644 --- a/fdbserver/IConfigConsumer.h +++ b/fdbserver/IConfigConsumer.h @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbserver/IDiskQueue.h b/fdbserver/IDiskQueue.h index e8275071b4..ad5deec930 100644 --- a/fdbserver/IDiskQueue.h +++ b/fdbserver/IDiskQueue.h @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbserver/IKeyValueContainer.h b/fdbserver/IKeyValueContainer.h index 9d0522c8c5..942f4c2973 100644 --- a/fdbserver/IKeyValueContainer.h +++ b/fdbserver/IKeyValueContainer.h @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbserver/IKeyValueStore.h b/fdbserver/IKeyValueStore.h index a295a55004..479a7c544b 100644 --- a/fdbserver/IKeyValueStore.h +++ b/fdbserver/IKeyValueStore.h @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbserver/IPager.h b/fdbserver/IPager.h index ca3000dd33..0625650086 100644 --- a/fdbserver/IPager.h +++ b/fdbserver/IPager.h @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbserver/KeyValueStoreCompressTestData.actor.cpp b/fdbserver/KeyValueStoreCompressTestData.actor.cpp index d416707b9b..3121232b58 100644 --- a/fdbserver/KeyValueStoreCompressTestData.actor.cpp +++ b/fdbserver/KeyValueStoreCompressTestData.actor.cpp @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbserver/KeyValueStoreMemory.actor.cpp b/fdbserver/KeyValueStoreMemory.actor.cpp index 563e7db736..4ded4f6749 100644 --- a/fdbserver/KeyValueStoreMemory.actor.cpp +++ b/fdbserver/KeyValueStoreMemory.actor.cpp @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbserver/KeyValueStoreSQLite.actor.cpp b/fdbserver/KeyValueStoreSQLite.actor.cpp index 46bca480cf..47c69e9764 100644 --- a/fdbserver/KeyValueStoreSQLite.actor.cpp +++ b/fdbserver/KeyValueStoreSQLite.actor.cpp @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbserver/Knobs.h b/fdbserver/Knobs.h index 19977b08dc..701a7488c6 100644 --- a/fdbserver/Knobs.h +++ b/fdbserver/Knobs.h @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbserver/LatencyBandConfig.cpp b/fdbserver/LatencyBandConfig.cpp index 6bafdd9d4f..7519198b4e 100644 --- a/fdbserver/LatencyBandConfig.cpp +++ b/fdbserver/LatencyBandConfig.cpp @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2019 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbserver/LatencyBandConfig.h b/fdbserver/LatencyBandConfig.h index 304cde2872..93eaddbef9 100644 --- a/fdbserver/LatencyBandConfig.h +++ b/fdbserver/LatencyBandConfig.h @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2019 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbserver/LeaderElection.actor.cpp b/fdbserver/LeaderElection.actor.cpp index 582e61ad01..1c968a515c 100644 --- a/fdbserver/LeaderElection.actor.cpp +++ b/fdbserver/LeaderElection.actor.cpp @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbserver/LeaderElection.h b/fdbserver/LeaderElection.h index e73adfe356..ad5d959ce5 100644 --- a/fdbserver/LeaderElection.h +++ b/fdbserver/LeaderElection.h @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbserver/LocalConfiguration.actor.cpp b/fdbserver/LocalConfiguration.actor.cpp index 0e102c6d93..e3c6a74ad9 100644 --- a/fdbserver/LocalConfiguration.actor.cpp +++ b/fdbserver/LocalConfiguration.actor.cpp @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbserver/LocalConfiguration.h b/fdbserver/LocalConfiguration.h index d020526462..5458e9b928 100644 --- a/fdbserver/LocalConfiguration.h +++ b/fdbserver/LocalConfiguration.h @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbserver/LogProtocolMessage.h b/fdbserver/LogProtocolMessage.h index 7d712ce30e..7c3d558b92 100644 --- a/fdbserver/LogProtocolMessage.h +++ b/fdbserver/LogProtocolMessage.h @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbserver/LogRouter.actor.cpp b/fdbserver/LogRouter.actor.cpp index 43abdac623..e595dc315f 100644 --- a/fdbserver/LogRouter.actor.cpp +++ b/fdbserver/LogRouter.actor.cpp @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbserver/LogSystem.cpp b/fdbserver/LogSystem.cpp index c91f88067a..1e4118fedd 100644 --- a/fdbserver/LogSystem.cpp +++ b/fdbserver/LogSystem.cpp @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbserver/LogSystem.h b/fdbserver/LogSystem.h index 30df4d04cc..f725dd26cb 100644 --- a/fdbserver/LogSystem.h +++ b/fdbserver/LogSystem.h @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbserver/LogSystemConfig.cpp b/fdbserver/LogSystemConfig.cpp index 16bbd46551..60c35e8e4e 100644 --- a/fdbserver/LogSystemConfig.cpp +++ b/fdbserver/LogSystemConfig.cpp @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2021 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbserver/LogSystemConfig.h b/fdbserver/LogSystemConfig.h index 1be515131d..3ff9b41e92 100644 --- a/fdbserver/LogSystemConfig.h +++ b/fdbserver/LogSystemConfig.h @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbserver/LogSystemDiskQueueAdapter.actor.cpp b/fdbserver/LogSystemDiskQueueAdapter.actor.cpp index 1178e7359b..36aedd0d15 100644 --- a/fdbserver/LogSystemDiskQueueAdapter.actor.cpp +++ b/fdbserver/LogSystemDiskQueueAdapter.actor.cpp @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbserver/LogSystemDiskQueueAdapter.h b/fdbserver/LogSystemDiskQueueAdapter.h index c05686124c..3a995cba3b 100644 --- a/fdbserver/LogSystemDiskQueueAdapter.h +++ b/fdbserver/LogSystemDiskQueueAdapter.h @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbserver/LogSystemPeekCursor.actor.cpp b/fdbserver/LogSystemPeekCursor.actor.cpp index 8ee250093a..3f604e12a6 100644 --- a/fdbserver/LogSystemPeekCursor.actor.cpp +++ b/fdbserver/LogSystemPeekCursor.actor.cpp @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbserver/MasterInterface.h b/fdbserver/MasterInterface.h index a092226f75..90d49e9492 100644 --- a/fdbserver/MasterInterface.h +++ b/fdbserver/MasterInterface.h @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbserver/MetricLogger.actor.cpp b/fdbserver/MetricLogger.actor.cpp index f072957c98..933c1f5d85 100644 --- a/fdbserver/MetricLogger.actor.cpp +++ b/fdbserver/MetricLogger.actor.cpp @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbserver/MetricLogger.actor.h b/fdbserver/MetricLogger.actor.h index 1fd9ca09f5..3dcd943db9 100644 --- a/fdbserver/MetricLogger.actor.h +++ b/fdbserver/MetricLogger.actor.h @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbserver/MoveKeys.actor.cpp b/fdbserver/MoveKeys.actor.cpp index 707fdf952e..068338d05c 100644 --- a/fdbserver/MoveKeys.actor.cpp +++ b/fdbserver/MoveKeys.actor.cpp @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbserver/MoveKeys.actor.h b/fdbserver/MoveKeys.actor.h index 220a940f25..92a14b1eb6 100644 --- a/fdbserver/MoveKeys.actor.h +++ b/fdbserver/MoveKeys.actor.h @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbserver/MutationTracking.cpp b/fdbserver/MutationTracking.cpp index 0acae5cdac..9ec17299d5 100644 --- a/fdbserver/MutationTracking.cpp +++ b/fdbserver/MutationTracking.cpp @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2020 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbserver/MutationTracking.h b/fdbserver/MutationTracking.h index 44be11145b..c29ff79de3 100644 --- a/fdbserver/MutationTracking.h +++ b/fdbserver/MutationTracking.h @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2020 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbserver/NetworkTest.h b/fdbserver/NetworkTest.h index 5106167b33..086aaf238d 100644 --- a/fdbserver/NetworkTest.h +++ b/fdbserver/NetworkTest.h @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbserver/OldTLogServer_4_6.actor.cpp b/fdbserver/OldTLogServer_4_6.actor.cpp index 47b79cc740..7fa6573a53 100644 --- a/fdbserver/OldTLogServer_4_6.actor.cpp +++ b/fdbserver/OldTLogServer_4_6.actor.cpp @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbserver/OldTLogServer_6_0.actor.cpp b/fdbserver/OldTLogServer_6_0.actor.cpp index 23d2a67d48..e52ab4955c 100644 --- a/fdbserver/OldTLogServer_6_0.actor.cpp +++ b/fdbserver/OldTLogServer_6_0.actor.cpp @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbserver/OldTLogServer_6_2.actor.cpp b/fdbserver/OldTLogServer_6_2.actor.cpp index c2e5bbaa86..35818053f9 100644 --- a/fdbserver/OldTLogServer_6_2.actor.cpp +++ b/fdbserver/OldTLogServer_6_2.actor.cpp @@ -4,7 +4,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbserver/OnDemandStore.actor.cpp b/fdbserver/OnDemandStore.actor.cpp index ffec8c2ed2..99e094fc59 100644 --- a/fdbserver/OnDemandStore.actor.cpp +++ b/fdbserver/OnDemandStore.actor.cpp @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbserver/OnDemandStore.h b/fdbserver/OnDemandStore.h index 0c58bfc151..8e97c836a4 100644 --- a/fdbserver/OnDemandStore.h +++ b/fdbserver/OnDemandStore.h @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbserver/PaxosConfigConsumer.actor.cpp b/fdbserver/PaxosConfigConsumer.actor.cpp index 437a0db090..4ada8cf701 100644 --- a/fdbserver/PaxosConfigConsumer.actor.cpp +++ b/fdbserver/PaxosConfigConsumer.actor.cpp @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbserver/PaxosConfigConsumer.h b/fdbserver/PaxosConfigConsumer.h index 0bd5926b2e..fc9e424d23 100644 --- a/fdbserver/PaxosConfigConsumer.h +++ b/fdbserver/PaxosConfigConsumer.h @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbserver/ProxyCommitData.actor.h b/fdbserver/ProxyCommitData.actor.h index 75b84e5c38..a560ed18d1 100644 --- a/fdbserver/ProxyCommitData.actor.h +++ b/fdbserver/ProxyCommitData.actor.h @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2020 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbserver/QuietDatabase.actor.cpp b/fdbserver/QuietDatabase.actor.cpp index 0c8718513d..8e6b847d5f 100644 --- a/fdbserver/QuietDatabase.actor.cpp +++ b/fdbserver/QuietDatabase.actor.cpp @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbserver/QuietDatabase.h b/fdbserver/QuietDatabase.h index eacd277f37..3cd406fe49 100644 --- a/fdbserver/QuietDatabase.h +++ b/fdbserver/QuietDatabase.h @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbserver/RadixTree.h b/fdbserver/RadixTree.h index eabe927c91..9939cae02f 100644 --- a/fdbserver/RadixTree.h +++ b/fdbserver/RadixTree.h @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbserver/RatekeeperInterface.h b/fdbserver/RatekeeperInterface.h index 7c83a0f90c..9d90a980ef 100644 --- a/fdbserver/RatekeeperInterface.h +++ b/fdbserver/RatekeeperInterface.h @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2019 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbserver/RecoveryState.h b/fdbserver/RecoveryState.h index a5b1287f15..e1f20c22ae 100644 --- a/fdbserver/RecoveryState.h +++ b/fdbserver/RecoveryState.h @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbserver/Resolver.actor.cpp b/fdbserver/Resolver.actor.cpp index 44576d3465..523e07c7b4 100644 --- a/fdbserver/Resolver.actor.cpp +++ b/fdbserver/Resolver.actor.cpp @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbserver/ResolverInterface.h b/fdbserver/ResolverInterface.h index f3ad222811..10b69fbe7f 100644 --- a/fdbserver/ResolverInterface.h +++ b/fdbserver/ResolverInterface.h @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbserver/RestoreApplier.actor.cpp b/fdbserver/RestoreApplier.actor.cpp index b8ff926358..8049aba133 100644 --- a/fdbserver/RestoreApplier.actor.cpp +++ b/fdbserver/RestoreApplier.actor.cpp @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbserver/RestoreApplier.actor.h b/fdbserver/RestoreApplier.actor.h index 0f3cdafa9e..464b8fea8a 100644 --- a/fdbserver/RestoreApplier.actor.h +++ b/fdbserver/RestoreApplier.actor.h @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbserver/RestoreCommon.actor.cpp b/fdbserver/RestoreCommon.actor.cpp index 31e60edc21..fbaaaf8ef5 100644 --- a/fdbserver/RestoreCommon.actor.cpp +++ b/fdbserver/RestoreCommon.actor.cpp @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbserver/RestoreCommon.actor.h b/fdbserver/RestoreCommon.actor.h index 1520151612..a1a4acf335 100644 --- a/fdbserver/RestoreCommon.actor.h +++ b/fdbserver/RestoreCommon.actor.h @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbserver/RestoreController.actor.cpp b/fdbserver/RestoreController.actor.cpp index 441ad49050..64d7d3d785 100644 --- a/fdbserver/RestoreController.actor.cpp +++ b/fdbserver/RestoreController.actor.cpp @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbserver/RestoreController.actor.h b/fdbserver/RestoreController.actor.h index 1a91eb0b55..5c9a271f7a 100644 --- a/fdbserver/RestoreController.actor.h +++ b/fdbserver/RestoreController.actor.h @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbserver/RestoreLoader.actor.cpp b/fdbserver/RestoreLoader.actor.cpp index ec15168433..9aa1aadee3 100644 --- a/fdbserver/RestoreLoader.actor.cpp +++ b/fdbserver/RestoreLoader.actor.cpp @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2020 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbserver/RestoreLoader.actor.h b/fdbserver/RestoreLoader.actor.h index d41ebe697d..b16e4c11fa 100644 --- a/fdbserver/RestoreLoader.actor.h +++ b/fdbserver/RestoreLoader.actor.h @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbserver/RestoreRoleCommon.actor.cpp b/fdbserver/RestoreRoleCommon.actor.cpp index 6cef2b1a3e..e52dd1ee7a 100644 --- a/fdbserver/RestoreRoleCommon.actor.cpp +++ b/fdbserver/RestoreRoleCommon.actor.cpp @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbserver/RestoreRoleCommon.actor.h b/fdbserver/RestoreRoleCommon.actor.h index 11890d9e6c..7fb8422a10 100644 --- a/fdbserver/RestoreRoleCommon.actor.h +++ b/fdbserver/RestoreRoleCommon.actor.h @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbserver/RestoreUtil.actor.cpp b/fdbserver/RestoreUtil.actor.cpp index ad0963e9e7..e25b665fd2 100644 --- a/fdbserver/RestoreUtil.actor.cpp +++ b/fdbserver/RestoreUtil.actor.cpp @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbserver/RestoreUtil.h b/fdbserver/RestoreUtil.h index 9d3548ab8d..e198e39e01 100644 --- a/fdbserver/RestoreUtil.h +++ b/fdbserver/RestoreUtil.h @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbserver/RestoreWorker.actor.cpp b/fdbserver/RestoreWorker.actor.cpp index d3dc7b2098..bca11b8ce8 100644 --- a/fdbserver/RestoreWorker.actor.cpp +++ b/fdbserver/RestoreWorker.actor.cpp @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbserver/RestoreWorker.actor.h b/fdbserver/RestoreWorker.actor.h index 8552b21310..d0c6ed5aa6 100644 --- a/fdbserver/RestoreWorker.actor.h +++ b/fdbserver/RestoreWorker.actor.h @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbserver/RestoreWorkerInterface.actor.cpp b/fdbserver/RestoreWorkerInterface.actor.cpp index 56e2a36833..a93bafa440 100644 --- a/fdbserver/RestoreWorkerInterface.actor.cpp +++ b/fdbserver/RestoreWorkerInterface.actor.cpp @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbserver/RestoreWorkerInterface.actor.h b/fdbserver/RestoreWorkerInterface.actor.h index cc052d18f8..065b22c468 100644 --- a/fdbserver/RestoreWorkerInterface.actor.h +++ b/fdbserver/RestoreWorkerInterface.actor.h @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbserver/RoleLineage.actor.cpp b/fdbserver/RoleLineage.actor.cpp index b54282f5f0..344fc59453 100644 --- a/fdbserver/RoleLineage.actor.cpp +++ b/fdbserver/RoleLineage.actor.cpp @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2020 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbserver/RoleLineage.actor.h b/fdbserver/RoleLineage.actor.h index 5f85ce2c70..a933b62e20 100644 --- a/fdbserver/RoleLineage.actor.h +++ b/fdbserver/RoleLineage.actor.h @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2020 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbserver/ServerDBInfo.actor.h b/fdbserver/ServerDBInfo.actor.h index df9fcca967..9a1bd30de6 100644 --- a/fdbserver/ServerDBInfo.actor.h +++ b/fdbserver/ServerDBInfo.actor.h @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbserver/ServerDBInfo.h b/fdbserver/ServerDBInfo.h index e2eb4df0c4..7424885116 100644 --- a/fdbserver/ServerDBInfo.h +++ b/fdbserver/ServerDBInfo.h @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbserver/SigStack.cpp b/fdbserver/SigStack.cpp index 12522ea9ef..66d39e26c1 100644 --- a/fdbserver/SigStack.cpp +++ b/fdbserver/SigStack.cpp @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbserver/SimEncryptVaultProxy.actor.cpp b/fdbserver/SimEncryptVaultProxy.actor.cpp index ad9ffdbcd0..6f3e8cfea3 100644 --- a/fdbserver/SimEncryptVaultProxy.actor.cpp +++ b/fdbserver/SimEncryptVaultProxy.actor.cpp @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2021 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbserver/SimEncryptVaultProxy.actor.h b/fdbserver/SimEncryptVaultProxy.actor.h index 169cc351cf..f997f1128e 100644 --- a/fdbserver/SimEncryptVaultProxy.actor.h +++ b/fdbserver/SimEncryptVaultProxy.actor.h @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2021 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbserver/SimpleConfigConsumer.actor.cpp b/fdbserver/SimpleConfigConsumer.actor.cpp index eef4c3e120..a2cbfba9db 100644 --- a/fdbserver/SimpleConfigConsumer.actor.cpp +++ b/fdbserver/SimpleConfigConsumer.actor.cpp @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbserver/SimpleConfigConsumer.h b/fdbserver/SimpleConfigConsumer.h index e74ef99486..8701f51d60 100644 --- a/fdbserver/SimpleConfigConsumer.h +++ b/fdbserver/SimpleConfigConsumer.h @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbserver/SimulatedCluster.actor.cpp b/fdbserver/SimulatedCluster.actor.cpp index 599173ee67..d55ba58c24 100644 --- a/fdbserver/SimulatedCluster.actor.cpp +++ b/fdbserver/SimulatedCluster.actor.cpp @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbserver/SimulatedCluster.h b/fdbserver/SimulatedCluster.h index 60919e9f74..d92d3dbb98 100644 --- a/fdbserver/SimulatedCluster.h +++ b/fdbserver/SimulatedCluster.h @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbserver/SkipList.cpp b/fdbserver/SkipList.cpp index 60a041fa81..180dc13dfa 100644 --- a/fdbserver/SkipList.cpp +++ b/fdbserver/SkipList.cpp @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbserver/SpanContextMessage.h b/fdbserver/SpanContextMessage.h index 3d091500c0..d6afa6b49b 100644 --- a/fdbserver/SpanContextMessage.h +++ b/fdbserver/SpanContextMessage.h @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2020 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbserver/Status.actor.cpp b/fdbserver/Status.actor.cpp index 8d3981feff..427c57a65d 100644 --- a/fdbserver/Status.actor.cpp +++ b/fdbserver/Status.actor.cpp @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbserver/Status.h b/fdbserver/Status.h index 6cc79a6d68..63e78f49f0 100644 --- a/fdbserver/Status.h +++ b/fdbserver/Status.h @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbserver/StorageCache.actor.cpp b/fdbserver/StorageCache.actor.cpp index 7ffe845445..a97931a2a8 100644 --- a/fdbserver/StorageCache.actor.cpp +++ b/fdbserver/StorageCache.actor.cpp @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2019 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbserver/StorageMetrics.actor.h b/fdbserver/StorageMetrics.actor.h index c50b5539d5..fa35fffdca 100644 --- a/fdbserver/StorageMetrics.actor.h +++ b/fdbserver/StorageMetrics.actor.h @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbserver/StorageMetrics.h b/fdbserver/StorageMetrics.h index 703c34850c..9791c0b113 100644 --- a/fdbserver/StorageMetrics.h +++ b/fdbserver/StorageMetrics.h @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbserver/TLogInterface.h b/fdbserver/TLogInterface.h index cfbd06b99b..e890a19661 100644 --- a/fdbserver/TLogInterface.h +++ b/fdbserver/TLogInterface.h @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbserver/TLogServer.actor.cpp b/fdbserver/TLogServer.actor.cpp index e3066e9d11..cb865c368c 100644 --- a/fdbserver/TLogServer.actor.cpp +++ b/fdbserver/TLogServer.actor.cpp @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbserver/TSSMappingUtil.actor.cpp b/fdbserver/TSSMappingUtil.actor.cpp index 4dc0c1b48b..47d81d3672 100644 --- a/fdbserver/TSSMappingUtil.actor.cpp +++ b/fdbserver/TSSMappingUtil.actor.cpp @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbserver/TSSMappingUtil.actor.h b/fdbserver/TSSMappingUtil.actor.h index 0ebd76f22d..d9d0743147 100644 --- a/fdbserver/TSSMappingUtil.actor.h +++ b/fdbserver/TSSMappingUtil.actor.h @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbserver/TagPartitionedLogSystem.actor.cpp b/fdbserver/TagPartitionedLogSystem.actor.cpp index c6e6d568b7..1152ab0fd7 100644 --- a/fdbserver/TagPartitionedLogSystem.actor.cpp +++ b/fdbserver/TagPartitionedLogSystem.actor.cpp @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2021 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbserver/TagPartitionedLogSystem.actor.h b/fdbserver/TagPartitionedLogSystem.actor.h index 628dcd3647..f4df560cea 100644 --- a/fdbserver/TagPartitionedLogSystem.actor.h +++ b/fdbserver/TagPartitionedLogSystem.actor.h @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbserver/TesterInterface.actor.h b/fdbserver/TesterInterface.actor.h index b50a2223fe..a52b11673e 100644 --- a/fdbserver/TesterInterface.actor.h +++ b/fdbserver/TesterInterface.actor.h @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbserver/VFSAsync.cpp b/fdbserver/VFSAsync.cpp index 950c2f60e9..ac00876bd1 100644 --- a/fdbserver/VFSAsync.cpp +++ b/fdbserver/VFSAsync.cpp @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbserver/VFSAsync.h b/fdbserver/VFSAsync.h index aee3544233..6cc98fe736 100644 --- a/fdbserver/VFSAsync.h +++ b/fdbserver/VFSAsync.h @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbserver/VersionedBTree.actor.cpp b/fdbserver/VersionedBTree.actor.cpp index 1538af100a..8b34c35161 100644 --- a/fdbserver/VersionedBTree.actor.cpp +++ b/fdbserver/VersionedBTree.actor.cpp @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbserver/WaitFailure.actor.cpp b/fdbserver/WaitFailure.actor.cpp index b16a41f126..d1ff0f0a4e 100644 --- a/fdbserver/WaitFailure.actor.cpp +++ b/fdbserver/WaitFailure.actor.cpp @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbserver/WaitFailure.h b/fdbserver/WaitFailure.h index d89446a4a4..2fa03a2de4 100644 --- a/fdbserver/WaitFailure.h +++ b/fdbserver/WaitFailure.h @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbserver/WorkerInterface.actor.h b/fdbserver/WorkerInterface.actor.h index 371e2e3d6e..695990aa7a 100644 --- a/fdbserver/WorkerInterface.actor.h +++ b/fdbserver/WorkerInterface.actor.h @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbserver/art.h b/fdbserver/art.h index faba082b36..b318ac2547 100644 --- a/fdbserver/art.h +++ b/fdbserver/art.h @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2020 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbserver/art_impl.h b/fdbserver/art_impl.h index d5a41d6484..b8e6816876 100644 --- a/fdbserver/art_impl.h +++ b/fdbserver/art_impl.h @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2020 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbserver/fdbserver.actor.cpp b/fdbserver/fdbserver.actor.cpp index 2148004fdb..17f1bcf9d2 100644 --- a/fdbserver/fdbserver.actor.cpp +++ b/fdbserver/fdbserver.actor.cpp @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbserver/masterserver.actor.cpp b/fdbserver/masterserver.actor.cpp index 023a7c2da0..d9e860eb52 100644 --- a/fdbserver/masterserver.actor.cpp +++ b/fdbserver/masterserver.actor.cpp @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbserver/networktest.actor.cpp b/fdbserver/networktest.actor.cpp index 371f61dd25..a768051f56 100644 --- a/fdbserver/networktest.actor.cpp +++ b/fdbserver/networktest.actor.cpp @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbserver/pubsub.actor.cpp b/fdbserver/pubsub.actor.cpp index e342a0d4dd..49a52b49ae 100644 --- a/fdbserver/pubsub.actor.cpp +++ b/fdbserver/pubsub.actor.cpp @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbserver/pubsub.h b/fdbserver/pubsub.h index 5f9a1f2b66..42742e9971 100644 --- a/fdbserver/pubsub.h +++ b/fdbserver/pubsub.h @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbserver/storageserver.actor.cpp b/fdbserver/storageserver.actor.cpp index 421304c716..561b6747f6 100644 --- a/fdbserver/storageserver.actor.cpp +++ b/fdbserver/storageserver.actor.cpp @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbserver/template_fdb.h b/fdbserver/template_fdb.h index d9e24cff71..a09a7e419d 100644 --- a/fdbserver/template_fdb.h +++ b/fdbserver/template_fdb.h @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbserver/tester.actor.cpp b/fdbserver/tester.actor.cpp index 00b4d02113..1596da1362 100644 --- a/fdbserver/tester.actor.cpp +++ b/fdbserver/tester.actor.cpp @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbserver/worker.actor.cpp b/fdbserver/worker.actor.cpp index faad491488..d4ae7de299 100644 --- a/fdbserver/worker.actor.cpp +++ b/fdbserver/worker.actor.cpp @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbserver/workloads/ApiCorrectness.actor.cpp b/fdbserver/workloads/ApiCorrectness.actor.cpp index 15fe5eb0e4..55eebfd0f6 100644 --- a/fdbserver/workloads/ApiCorrectness.actor.cpp +++ b/fdbserver/workloads/ApiCorrectness.actor.cpp @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbserver/workloads/ApiWorkload.actor.cpp b/fdbserver/workloads/ApiWorkload.actor.cpp index 86c4809b79..e326029b46 100644 --- a/fdbserver/workloads/ApiWorkload.actor.cpp +++ b/fdbserver/workloads/ApiWorkload.actor.cpp @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbserver/workloads/ApiWorkload.h b/fdbserver/workloads/ApiWorkload.h index adc5324290..751cbafc37 100644 --- a/fdbserver/workloads/ApiWorkload.h +++ b/fdbserver/workloads/ApiWorkload.h @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbserver/workloads/AsyncFile.actor.h b/fdbserver/workloads/AsyncFile.actor.h index 504de6da14..89ad2f4d56 100644 --- a/fdbserver/workloads/AsyncFile.actor.h +++ b/fdbserver/workloads/AsyncFile.actor.h @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbserver/workloads/AsyncFile.cpp b/fdbserver/workloads/AsyncFile.cpp index fd8de70131..e7f942aa53 100644 --- a/fdbserver/workloads/AsyncFile.cpp +++ b/fdbserver/workloads/AsyncFile.cpp @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbserver/workloads/AsyncFileCorrectness.actor.cpp b/fdbserver/workloads/AsyncFileCorrectness.actor.cpp index 9132894b3d..70215ad961 100644 --- a/fdbserver/workloads/AsyncFileCorrectness.actor.cpp +++ b/fdbserver/workloads/AsyncFileCorrectness.actor.cpp @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbserver/workloads/AsyncFileRead.actor.cpp b/fdbserver/workloads/AsyncFileRead.actor.cpp index 7af34ba416..b048f9864a 100644 --- a/fdbserver/workloads/AsyncFileRead.actor.cpp +++ b/fdbserver/workloads/AsyncFileRead.actor.cpp @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbserver/workloads/AsyncFileWrite.actor.cpp b/fdbserver/workloads/AsyncFileWrite.actor.cpp index bd5d6945d9..9bafc4373a 100644 --- a/fdbserver/workloads/AsyncFileWrite.actor.cpp +++ b/fdbserver/workloads/AsyncFileWrite.actor.cpp @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbserver/workloads/AtomicOps.actor.cpp b/fdbserver/workloads/AtomicOps.actor.cpp index de11f0519f..2cc2f1020f 100644 --- a/fdbserver/workloads/AtomicOps.actor.cpp +++ b/fdbserver/workloads/AtomicOps.actor.cpp @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbserver/workloads/AtomicOpsApiCorrectness.actor.cpp b/fdbserver/workloads/AtomicOpsApiCorrectness.actor.cpp index 81ac2789bf..4acdb7eddc 100644 --- a/fdbserver/workloads/AtomicOpsApiCorrectness.actor.cpp +++ b/fdbserver/workloads/AtomicOpsApiCorrectness.actor.cpp @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbserver/workloads/AtomicRestore.actor.cpp b/fdbserver/workloads/AtomicRestore.actor.cpp index 3a0fa54dc2..4c3c2703f9 100644 --- a/fdbserver/workloads/AtomicRestore.actor.cpp +++ b/fdbserver/workloads/AtomicRestore.actor.cpp @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbserver/workloads/AtomicSwitchover.actor.cpp b/fdbserver/workloads/AtomicSwitchover.actor.cpp index aedd4f0dae..0337da51b4 100644 --- a/fdbserver/workloads/AtomicSwitchover.actor.cpp +++ b/fdbserver/workloads/AtomicSwitchover.actor.cpp @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbserver/workloads/BackgroundSelectors.actor.cpp b/fdbserver/workloads/BackgroundSelectors.actor.cpp index 377af31633..702b40054a 100644 --- a/fdbserver/workloads/BackgroundSelectors.actor.cpp +++ b/fdbserver/workloads/BackgroundSelectors.actor.cpp @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbserver/workloads/BackupAndParallelRestoreCorrectness.actor.cpp b/fdbserver/workloads/BackupAndParallelRestoreCorrectness.actor.cpp index 3a1eac9982..890bdf6a3a 100644 --- a/fdbserver/workloads/BackupAndParallelRestoreCorrectness.actor.cpp +++ b/fdbserver/workloads/BackupAndParallelRestoreCorrectness.actor.cpp @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbserver/workloads/BackupCorrectness.actor.cpp b/fdbserver/workloads/BackupCorrectness.actor.cpp index 1eeb1171ce..92550a23bf 100644 --- a/fdbserver/workloads/BackupCorrectness.actor.cpp +++ b/fdbserver/workloads/BackupCorrectness.actor.cpp @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbserver/workloads/BackupToBlob.actor.cpp b/fdbserver/workloads/BackupToBlob.actor.cpp index bf5bc0db42..ee27e1a480 100644 --- a/fdbserver/workloads/BackupToBlob.actor.cpp +++ b/fdbserver/workloads/BackupToBlob.actor.cpp @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbserver/workloads/BackupToDBAbort.actor.cpp b/fdbserver/workloads/BackupToDBAbort.actor.cpp index 1a6a23eca6..a505d8e02e 100644 --- a/fdbserver/workloads/BackupToDBAbort.actor.cpp +++ b/fdbserver/workloads/BackupToDBAbort.actor.cpp @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbserver/workloads/BackupToDBCorrectness.actor.cpp b/fdbserver/workloads/BackupToDBCorrectness.actor.cpp index b80952f352..415744a677 100644 --- a/fdbserver/workloads/BackupToDBCorrectness.actor.cpp +++ b/fdbserver/workloads/BackupToDBCorrectness.actor.cpp @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbserver/workloads/BackupToDBUpgrade.actor.cpp b/fdbserver/workloads/BackupToDBUpgrade.actor.cpp index 1f00eacd22..6ccb691c9d 100644 --- a/fdbserver/workloads/BackupToDBUpgrade.actor.cpp +++ b/fdbserver/workloads/BackupToDBUpgrade.actor.cpp @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbserver/workloads/BlobGranuleVerifier.actor.cpp b/fdbserver/workloads/BlobGranuleVerifier.actor.cpp index 3f27864831..1af8c5e9d4 100644 --- a/fdbserver/workloads/BlobGranuleVerifier.actor.cpp +++ b/fdbserver/workloads/BlobGranuleVerifier.actor.cpp @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbserver/workloads/BlobStoreWorkload.h b/fdbserver/workloads/BlobStoreWorkload.h index 1c96065988..e8b7ee12b5 100644 --- a/fdbserver/workloads/BlobStoreWorkload.h +++ b/fdbserver/workloads/BlobStoreWorkload.h @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbserver/workloads/BulkLoad.actor.cpp b/fdbserver/workloads/BulkLoad.actor.cpp index 8d503dc08a..d10a909a04 100644 --- a/fdbserver/workloads/BulkLoad.actor.cpp +++ b/fdbserver/workloads/BulkLoad.actor.cpp @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbserver/workloads/BulkSetup.actor.h b/fdbserver/workloads/BulkSetup.actor.h index a900842b88..4d1acdbb44 100644 --- a/fdbserver/workloads/BulkSetup.actor.h +++ b/fdbserver/workloads/BulkSetup.actor.h @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbserver/workloads/ChangeConfig.actor.cpp b/fdbserver/workloads/ChangeConfig.actor.cpp index c19a97782d..8f9949ec52 100644 --- a/fdbserver/workloads/ChangeConfig.actor.cpp +++ b/fdbserver/workloads/ChangeConfig.actor.cpp @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbserver/workloads/ChangeFeeds.actor.cpp b/fdbserver/workloads/ChangeFeeds.actor.cpp index edc3444340..530aa54e33 100644 --- a/fdbserver/workloads/ChangeFeeds.actor.cpp +++ b/fdbserver/workloads/ChangeFeeds.actor.cpp @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbserver/workloads/ClearSingleRange.actor.cpp b/fdbserver/workloads/ClearSingleRange.actor.cpp index 62e7671a7c..bc4ab66d34 100644 --- a/fdbserver/workloads/ClearSingleRange.actor.cpp +++ b/fdbserver/workloads/ClearSingleRange.actor.cpp @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbserver/workloads/ClientTransactionProfileCorrectness.actor.cpp b/fdbserver/workloads/ClientTransactionProfileCorrectness.actor.cpp index 77f42df6b9..64da60cf7a 100644 --- a/fdbserver/workloads/ClientTransactionProfileCorrectness.actor.cpp +++ b/fdbserver/workloads/ClientTransactionProfileCorrectness.actor.cpp @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbserver/workloads/ClogSingleConnection.actor.cpp b/fdbserver/workloads/ClogSingleConnection.actor.cpp index 24e0b6e32f..0fe828ca6a 100644 --- a/fdbserver/workloads/ClogSingleConnection.actor.cpp +++ b/fdbserver/workloads/ClogSingleConnection.actor.cpp @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbserver/workloads/CommitBugCheck.actor.cpp b/fdbserver/workloads/CommitBugCheck.actor.cpp index 5a61c67d5b..4da04cd7bf 100644 --- a/fdbserver/workloads/CommitBugCheck.actor.cpp +++ b/fdbserver/workloads/CommitBugCheck.actor.cpp @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbserver/workloads/ConfigIncrement.actor.cpp b/fdbserver/workloads/ConfigIncrement.actor.cpp index da15fceba6..6f1d1babb2 100644 --- a/fdbserver/workloads/ConfigIncrement.actor.cpp +++ b/fdbserver/workloads/ConfigIncrement.actor.cpp @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbserver/workloads/ConfigureDatabase.actor.cpp b/fdbserver/workloads/ConfigureDatabase.actor.cpp index 10832a9785..5b12f7e3c6 100644 --- a/fdbserver/workloads/ConfigureDatabase.actor.cpp +++ b/fdbserver/workloads/ConfigureDatabase.actor.cpp @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbserver/workloads/ConflictRange.actor.cpp b/fdbserver/workloads/ConflictRange.actor.cpp index 9b53b503cd..8509482a8f 100644 --- a/fdbserver/workloads/ConflictRange.actor.cpp +++ b/fdbserver/workloads/ConflictRange.actor.cpp @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbserver/workloads/ConsistencyCheck.actor.cpp b/fdbserver/workloads/ConsistencyCheck.actor.cpp index 4176740f52..431de4de5c 100644 --- a/fdbserver/workloads/ConsistencyCheck.actor.cpp +++ b/fdbserver/workloads/ConsistencyCheck.actor.cpp @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbserver/workloads/CpuProfiler.actor.cpp b/fdbserver/workloads/CpuProfiler.actor.cpp index 0f796aad70..bc7cbfea15 100644 --- a/fdbserver/workloads/CpuProfiler.actor.cpp +++ b/fdbserver/workloads/CpuProfiler.actor.cpp @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbserver/workloads/Cycle.actor.cpp b/fdbserver/workloads/Cycle.actor.cpp index 4fb5bd3590..4e17e82a88 100644 --- a/fdbserver/workloads/Cycle.actor.cpp +++ b/fdbserver/workloads/Cycle.actor.cpp @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbserver/workloads/DDBalance.actor.cpp b/fdbserver/workloads/DDBalance.actor.cpp index ecb7a98640..62b9a03ccd 100644 --- a/fdbserver/workloads/DDBalance.actor.cpp +++ b/fdbserver/workloads/DDBalance.actor.cpp @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbserver/workloads/DDMetrics.actor.cpp b/fdbserver/workloads/DDMetrics.actor.cpp index 8e8cd5c359..360ec4e0ac 100644 --- a/fdbserver/workloads/DDMetrics.actor.cpp +++ b/fdbserver/workloads/DDMetrics.actor.cpp @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbserver/workloads/DDMetricsExclude.actor.cpp b/fdbserver/workloads/DDMetricsExclude.actor.cpp index a9d6a0ccb0..ff35a73f21 100644 --- a/fdbserver/workloads/DDMetricsExclude.actor.cpp +++ b/fdbserver/workloads/DDMetricsExclude.actor.cpp @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbserver/workloads/DataDistributionMetrics.actor.cpp b/fdbserver/workloads/DataDistributionMetrics.actor.cpp index 73c2af5ee6..4cb59c5768 100644 --- a/fdbserver/workloads/DataDistributionMetrics.actor.cpp +++ b/fdbserver/workloads/DataDistributionMetrics.actor.cpp @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbserver/workloads/DataLossRecovery.actor.cpp b/fdbserver/workloads/DataLossRecovery.actor.cpp index 38e6ede9ce..915e40577d 100644 --- a/fdbserver/workloads/DataLossRecovery.actor.cpp +++ b/fdbserver/workloads/DataLossRecovery.actor.cpp @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2021 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbserver/workloads/DifferentClustersSameRV.actor.cpp b/fdbserver/workloads/DifferentClustersSameRV.actor.cpp index 197f924b91..9598607830 100644 --- a/fdbserver/workloads/DifferentClustersSameRV.actor.cpp +++ b/fdbserver/workloads/DifferentClustersSameRV.actor.cpp @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbserver/workloads/DiskDurability.actor.cpp b/fdbserver/workloads/DiskDurability.actor.cpp index 1f23af5392..f5b0dc2c3e 100644 --- a/fdbserver/workloads/DiskDurability.actor.cpp +++ b/fdbserver/workloads/DiskDurability.actor.cpp @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbserver/workloads/DiskDurabilityTest.actor.cpp b/fdbserver/workloads/DiskDurabilityTest.actor.cpp index d74c17e9ae..e7cbc6ce58 100644 --- a/fdbserver/workloads/DiskDurabilityTest.actor.cpp +++ b/fdbserver/workloads/DiskDurabilityTest.actor.cpp @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbserver/workloads/DiskFailureInjection.actor.cpp b/fdbserver/workloads/DiskFailureInjection.actor.cpp index ad978b6f7e..8f3338eb28 100644 --- a/fdbserver/workloads/DiskFailureInjection.actor.cpp +++ b/fdbserver/workloads/DiskFailureInjection.actor.cpp @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbserver/workloads/DummyWorkload.actor.cpp b/fdbserver/workloads/DummyWorkload.actor.cpp index d1727a3d4c..99af0a2947 100644 --- a/fdbserver/workloads/DummyWorkload.actor.cpp +++ b/fdbserver/workloads/DummyWorkload.actor.cpp @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbserver/workloads/EncryptionOps.actor.cpp b/fdbserver/workloads/EncryptionOps.actor.cpp index bb85b57e4f..cad228dfa0 100644 --- a/fdbserver/workloads/EncryptionOps.actor.cpp +++ b/fdbserver/workloads/EncryptionOps.actor.cpp @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbserver/workloads/ExternalWorkload.actor.cpp b/fdbserver/workloads/ExternalWorkload.actor.cpp index eae51cafe1..2c1786e43f 100644 --- a/fdbserver/workloads/ExternalWorkload.actor.cpp +++ b/fdbserver/workloads/ExternalWorkload.actor.cpp @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2019 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbserver/workloads/FastTriggeredWatches.actor.cpp b/fdbserver/workloads/FastTriggeredWatches.actor.cpp index 0cd0ed294d..8d1e635a47 100644 --- a/fdbserver/workloads/FastTriggeredWatches.actor.cpp +++ b/fdbserver/workloads/FastTriggeredWatches.actor.cpp @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbserver/workloads/FileSystem.actor.cpp b/fdbserver/workloads/FileSystem.actor.cpp index 7fe60af5d0..a5fa90af4e 100644 --- a/fdbserver/workloads/FileSystem.actor.cpp +++ b/fdbserver/workloads/FileSystem.actor.cpp @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbserver/workloads/Fuzz.cpp b/fdbserver/workloads/Fuzz.cpp index e5a84d3fab..d26726c3e6 100644 --- a/fdbserver/workloads/Fuzz.cpp +++ b/fdbserver/workloads/Fuzz.cpp @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbserver/workloads/FuzzApiCorrectness.actor.cpp b/fdbserver/workloads/FuzzApiCorrectness.actor.cpp index 5e1f39e5b5..7f3e5d41e1 100644 --- a/fdbserver/workloads/FuzzApiCorrectness.actor.cpp +++ b/fdbserver/workloads/FuzzApiCorrectness.actor.cpp @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbserver/workloads/GetRangeStream.actor.cpp b/fdbserver/workloads/GetRangeStream.actor.cpp index fbe235a13f..8d476069f1 100644 --- a/fdbserver/workloads/GetRangeStream.actor.cpp +++ b/fdbserver/workloads/GetRangeStream.actor.cpp @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2020 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbserver/workloads/HealthMetricsApi.actor.cpp b/fdbserver/workloads/HealthMetricsApi.actor.cpp index 17fb506c8e..29fb0a0b64 100644 --- a/fdbserver/workloads/HealthMetricsApi.actor.cpp +++ b/fdbserver/workloads/HealthMetricsApi.actor.cpp @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbserver/workloads/Increment.actor.cpp b/fdbserver/workloads/Increment.actor.cpp index c37396a690..ee65c19a62 100644 --- a/fdbserver/workloads/Increment.actor.cpp +++ b/fdbserver/workloads/Increment.actor.cpp @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbserver/workloads/IncrementalBackup.actor.cpp b/fdbserver/workloads/IncrementalBackup.actor.cpp index 0697c14c60..688387023e 100644 --- a/fdbserver/workloads/IncrementalBackup.actor.cpp +++ b/fdbserver/workloads/IncrementalBackup.actor.cpp @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbserver/workloads/IndexScan.actor.cpp b/fdbserver/workloads/IndexScan.actor.cpp index 774dc79c3f..ca844c980c 100644 --- a/fdbserver/workloads/IndexScan.actor.cpp +++ b/fdbserver/workloads/IndexScan.actor.cpp @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbserver/workloads/Inventory.actor.cpp b/fdbserver/workloads/Inventory.actor.cpp index 9c81c95016..c5f8074625 100644 --- a/fdbserver/workloads/Inventory.actor.cpp +++ b/fdbserver/workloads/Inventory.actor.cpp @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbserver/workloads/KVStoreTest.actor.cpp b/fdbserver/workloads/KVStoreTest.actor.cpp index aa3ca8a420..85eb2f0931 100644 --- a/fdbserver/workloads/KVStoreTest.actor.cpp +++ b/fdbserver/workloads/KVStoreTest.actor.cpp @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbserver/workloads/KillRegion.actor.cpp b/fdbserver/workloads/KillRegion.actor.cpp index 6490c2c4bd..d15255bd7b 100644 --- a/fdbserver/workloads/KillRegion.actor.cpp +++ b/fdbserver/workloads/KillRegion.actor.cpp @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbserver/workloads/LocalRatekeeper.actor.cpp b/fdbserver/workloads/LocalRatekeeper.actor.cpp index dad958296b..97f8af04bc 100644 --- a/fdbserver/workloads/LocalRatekeeper.actor.cpp +++ b/fdbserver/workloads/LocalRatekeeper.actor.cpp @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbserver/workloads/LockDatabase.actor.cpp b/fdbserver/workloads/LockDatabase.actor.cpp index a5ba9f07fe..85cfdc6837 100644 --- a/fdbserver/workloads/LockDatabase.actor.cpp +++ b/fdbserver/workloads/LockDatabase.actor.cpp @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbserver/workloads/LockDatabaseFrequently.actor.cpp b/fdbserver/workloads/LockDatabaseFrequently.actor.cpp index 9fab3f27fe..bfab394031 100644 --- a/fdbserver/workloads/LockDatabaseFrequently.actor.cpp +++ b/fdbserver/workloads/LockDatabaseFrequently.actor.cpp @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2020 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbserver/workloads/LogMetrics.actor.cpp b/fdbserver/workloads/LogMetrics.actor.cpp index 1101aaaa6f..1ebb9c8e98 100644 --- a/fdbserver/workloads/LogMetrics.actor.cpp +++ b/fdbserver/workloads/LogMetrics.actor.cpp @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbserver/workloads/LowLatency.actor.cpp b/fdbserver/workloads/LowLatency.actor.cpp index 800da48e05..dc7dacc7b5 100644 --- a/fdbserver/workloads/LowLatency.actor.cpp +++ b/fdbserver/workloads/LowLatency.actor.cpp @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbserver/workloads/MachineAttrition.actor.cpp b/fdbserver/workloads/MachineAttrition.actor.cpp index 9411eb33eb..42f951dd37 100644 --- a/fdbserver/workloads/MachineAttrition.actor.cpp +++ b/fdbserver/workloads/MachineAttrition.actor.cpp @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbserver/workloads/MemoryKeyValueStore.cpp b/fdbserver/workloads/MemoryKeyValueStore.cpp index ac966f7621..8ac088f210 100644 --- a/fdbserver/workloads/MemoryKeyValueStore.cpp +++ b/fdbserver/workloads/MemoryKeyValueStore.cpp @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbserver/workloads/MemoryKeyValueStore.h b/fdbserver/workloads/MemoryKeyValueStore.h index c90aa8f1a4..ecb52d77a7 100644 --- a/fdbserver/workloads/MemoryKeyValueStore.h +++ b/fdbserver/workloads/MemoryKeyValueStore.h @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbserver/workloads/MemoryLifetime.actor.cpp b/fdbserver/workloads/MemoryLifetime.actor.cpp index b294c003e4..3b0ca269a5 100644 --- a/fdbserver/workloads/MemoryLifetime.actor.cpp +++ b/fdbserver/workloads/MemoryLifetime.actor.cpp @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbserver/workloads/MetricLogging.actor.cpp b/fdbserver/workloads/MetricLogging.actor.cpp index e10d1a5d47..866ba522c7 100644 --- a/fdbserver/workloads/MetricLogging.actor.cpp +++ b/fdbserver/workloads/MetricLogging.actor.cpp @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbserver/workloads/MiniCycle.actor.cpp b/fdbserver/workloads/MiniCycle.actor.cpp index 84d6adeda2..b071902a8c 100644 --- a/fdbserver/workloads/MiniCycle.actor.cpp +++ b/fdbserver/workloads/MiniCycle.actor.cpp @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2021 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbserver/workloads/MutationLogReaderCorrectness.actor.cpp b/fdbserver/workloads/MutationLogReaderCorrectness.actor.cpp index 69a9e9667a..fc6db8ae22 100644 --- a/fdbserver/workloads/MutationLogReaderCorrectness.actor.cpp +++ b/fdbserver/workloads/MutationLogReaderCorrectness.actor.cpp @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbserver/workloads/ParallelRestore.actor.cpp b/fdbserver/workloads/ParallelRestore.actor.cpp index 22de9e13f5..af685cb5f5 100644 --- a/fdbserver/workloads/ParallelRestore.actor.cpp +++ b/fdbserver/workloads/ParallelRestore.actor.cpp @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbserver/workloads/Performance.actor.cpp b/fdbserver/workloads/Performance.actor.cpp index ff11124cef..0cfc16497a 100644 --- a/fdbserver/workloads/Performance.actor.cpp +++ b/fdbserver/workloads/Performance.actor.cpp @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbserver/workloads/PhysicalShardMove.actor.cpp b/fdbserver/workloads/PhysicalShardMove.actor.cpp index feb2ebd8e8..752a4a734b 100644 --- a/fdbserver/workloads/PhysicalShardMove.actor.cpp +++ b/fdbserver/workloads/PhysicalShardMove.actor.cpp @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2021 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbserver/workloads/Ping.actor.cpp b/fdbserver/workloads/Ping.actor.cpp index 3c37da98fd..f939d2525b 100644 --- a/fdbserver/workloads/Ping.actor.cpp +++ b/fdbserver/workloads/Ping.actor.cpp @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbserver/workloads/PopulateTPCC.actor.cpp b/fdbserver/workloads/PopulateTPCC.actor.cpp index bb8acaabbc..eb0b7c4ea4 100644 --- a/fdbserver/workloads/PopulateTPCC.actor.cpp +++ b/fdbserver/workloads/PopulateTPCC.actor.cpp @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2020 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbserver/workloads/ProtocolVersion.actor.cpp b/fdbserver/workloads/ProtocolVersion.actor.cpp index 21c8cc72c9..45b14d1a98 100644 --- a/fdbserver/workloads/ProtocolVersion.actor.cpp +++ b/fdbserver/workloads/ProtocolVersion.actor.cpp @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2019 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbserver/workloads/PubSubMultiples.actor.cpp b/fdbserver/workloads/PubSubMultiples.actor.cpp index 4a0e35ca4b..a3c27ae512 100644 --- a/fdbserver/workloads/PubSubMultiples.actor.cpp +++ b/fdbserver/workloads/PubSubMultiples.actor.cpp @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbserver/workloads/QueuePush.actor.cpp b/fdbserver/workloads/QueuePush.actor.cpp index 010e0885c7..bc9c192c6d 100644 --- a/fdbserver/workloads/QueuePush.actor.cpp +++ b/fdbserver/workloads/QueuePush.actor.cpp @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbserver/workloads/RYWDisable.actor.cpp b/fdbserver/workloads/RYWDisable.actor.cpp index 2504001ad9..2586118cb6 100644 --- a/fdbserver/workloads/RYWDisable.actor.cpp +++ b/fdbserver/workloads/RYWDisable.actor.cpp @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbserver/workloads/RYWPerformance.actor.cpp b/fdbserver/workloads/RYWPerformance.actor.cpp index 655925937a..22f9cffb7c 100644 --- a/fdbserver/workloads/RYWPerformance.actor.cpp +++ b/fdbserver/workloads/RYWPerformance.actor.cpp @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbserver/workloads/RandomClogging.actor.cpp b/fdbserver/workloads/RandomClogging.actor.cpp index 0428b8892a..7afa89e93b 100644 --- a/fdbserver/workloads/RandomClogging.actor.cpp +++ b/fdbserver/workloads/RandomClogging.actor.cpp @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbserver/workloads/RandomMoveKeys.actor.cpp b/fdbserver/workloads/RandomMoveKeys.actor.cpp index 6d17e4b0d9..de6255c000 100644 --- a/fdbserver/workloads/RandomMoveKeys.actor.cpp +++ b/fdbserver/workloads/RandomMoveKeys.actor.cpp @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbserver/workloads/RandomSelector.actor.cpp b/fdbserver/workloads/RandomSelector.actor.cpp index c41385943d..1573e6f848 100644 --- a/fdbserver/workloads/RandomSelector.actor.cpp +++ b/fdbserver/workloads/RandomSelector.actor.cpp @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbserver/workloads/ReadAfterWrite.actor.cpp b/fdbserver/workloads/ReadAfterWrite.actor.cpp index 046960872b..ecbd97410a 100644 --- a/fdbserver/workloads/ReadAfterWrite.actor.cpp +++ b/fdbserver/workloads/ReadAfterWrite.actor.cpp @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2020 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbserver/workloads/ReadHotDetection.actor.cpp b/fdbserver/workloads/ReadHotDetection.actor.cpp index 2bba376132..e5b15ef04d 100644 --- a/fdbserver/workloads/ReadHotDetection.actor.cpp +++ b/fdbserver/workloads/ReadHotDetection.actor.cpp @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbserver/workloads/ReadWrite.actor.cpp b/fdbserver/workloads/ReadWrite.actor.cpp index 669d0d4265..4741a01371 100644 --- a/fdbserver/workloads/ReadWrite.actor.cpp +++ b/fdbserver/workloads/ReadWrite.actor.cpp @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbserver/workloads/RemoveServersSafely.actor.cpp b/fdbserver/workloads/RemoveServersSafely.actor.cpp index 4151153737..d2550394d1 100644 --- a/fdbserver/workloads/RemoveServersSafely.actor.cpp +++ b/fdbserver/workloads/RemoveServersSafely.actor.cpp @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbserver/workloads/ReportConflictingKeys.actor.cpp b/fdbserver/workloads/ReportConflictingKeys.actor.cpp index 629d19b412..33ac14584a 100644 --- a/fdbserver/workloads/ReportConflictingKeys.actor.cpp +++ b/fdbserver/workloads/ReportConflictingKeys.actor.cpp @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2020 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbserver/workloads/RestoreBackup.actor.cpp b/fdbserver/workloads/RestoreBackup.actor.cpp index 70f8937c91..c7122bc107 100644 --- a/fdbserver/workloads/RestoreBackup.actor.cpp +++ b/fdbserver/workloads/RestoreBackup.actor.cpp @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbserver/workloads/RestoreFromBlob.actor.cpp b/fdbserver/workloads/RestoreFromBlob.actor.cpp index 01ebbede76..482f22ded4 100644 --- a/fdbserver/workloads/RestoreFromBlob.actor.cpp +++ b/fdbserver/workloads/RestoreFromBlob.actor.cpp @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbserver/workloads/Rollback.actor.cpp b/fdbserver/workloads/Rollback.actor.cpp index 8f3c8833dc..3c3ea8efa2 100644 --- a/fdbserver/workloads/Rollback.actor.cpp +++ b/fdbserver/workloads/Rollback.actor.cpp @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbserver/workloads/RyowCorrectness.actor.cpp b/fdbserver/workloads/RyowCorrectness.actor.cpp index eb68fea0c1..dfaa5bfc8c 100644 --- a/fdbserver/workloads/RyowCorrectness.actor.cpp +++ b/fdbserver/workloads/RyowCorrectness.actor.cpp @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbserver/workloads/SaveAndKill.actor.cpp b/fdbserver/workloads/SaveAndKill.actor.cpp index 43d3e0d2b3..7488db5597 100644 --- a/fdbserver/workloads/SaveAndKill.actor.cpp +++ b/fdbserver/workloads/SaveAndKill.actor.cpp @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbserver/workloads/SelectorCorrectness.actor.cpp b/fdbserver/workloads/SelectorCorrectness.actor.cpp index c6e0a5c719..5cb94e8f7d 100644 --- a/fdbserver/workloads/SelectorCorrectness.actor.cpp +++ b/fdbserver/workloads/SelectorCorrectness.actor.cpp @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbserver/workloads/Serializability.actor.cpp b/fdbserver/workloads/Serializability.actor.cpp index fbf564bbef..a191ae01ed 100644 --- a/fdbserver/workloads/Serializability.actor.cpp +++ b/fdbserver/workloads/Serializability.actor.cpp @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbserver/workloads/Sideband.actor.cpp b/fdbserver/workloads/Sideband.actor.cpp index 86f5df1952..15a8f09122 100644 --- a/fdbserver/workloads/Sideband.actor.cpp +++ b/fdbserver/workloads/Sideband.actor.cpp @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbserver/workloads/SimpleAtomicAdd.actor.cpp b/fdbserver/workloads/SimpleAtomicAdd.actor.cpp index 7a23df4000..503bcba2a3 100644 --- a/fdbserver/workloads/SimpleAtomicAdd.actor.cpp +++ b/fdbserver/workloads/SimpleAtomicAdd.actor.cpp @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbserver/workloads/SlowTaskWorkload.actor.cpp b/fdbserver/workloads/SlowTaskWorkload.actor.cpp index ce48f1770f..3eec25fdec 100644 --- a/fdbserver/workloads/SlowTaskWorkload.actor.cpp +++ b/fdbserver/workloads/SlowTaskWorkload.actor.cpp @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbserver/workloads/SpecialKeySpaceCorrectness.actor.cpp b/fdbserver/workloads/SpecialKeySpaceCorrectness.actor.cpp index b4e9e3c3a6..03ddb4e615 100644 --- a/fdbserver/workloads/SpecialKeySpaceCorrectness.actor.cpp +++ b/fdbserver/workloads/SpecialKeySpaceCorrectness.actor.cpp @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2020 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbserver/workloads/StatusWorkload.actor.cpp b/fdbserver/workloads/StatusWorkload.actor.cpp index afd8b5c65e..76964d213d 100644 --- a/fdbserver/workloads/StatusWorkload.actor.cpp +++ b/fdbserver/workloads/StatusWorkload.actor.cpp @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbserver/workloads/Storefront.actor.cpp b/fdbserver/workloads/Storefront.actor.cpp index cab329e501..cd036074c5 100644 --- a/fdbserver/workloads/Storefront.actor.cpp +++ b/fdbserver/workloads/Storefront.actor.cpp @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbserver/workloads/StreamingRangeRead.actor.cpp b/fdbserver/workloads/StreamingRangeRead.actor.cpp index 76400f72c8..a461fc4fb3 100644 --- a/fdbserver/workloads/StreamingRangeRead.actor.cpp +++ b/fdbserver/workloads/StreamingRangeRead.actor.cpp @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbserver/workloads/StreamingRead.actor.cpp b/fdbserver/workloads/StreamingRead.actor.cpp index 9f39676923..71b3df8bbb 100644 --- a/fdbserver/workloads/StreamingRead.actor.cpp +++ b/fdbserver/workloads/StreamingRead.actor.cpp @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbserver/workloads/SubmitBackup.actor.cpp b/fdbserver/workloads/SubmitBackup.actor.cpp index 55b7b86cef..50759bf014 100644 --- a/fdbserver/workloads/SubmitBackup.actor.cpp +++ b/fdbserver/workloads/SubmitBackup.actor.cpp @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbserver/workloads/TPCC.actor.cpp b/fdbserver/workloads/TPCC.actor.cpp index 6826892089..bab468fcc4 100644 --- a/fdbserver/workloads/TPCC.actor.cpp +++ b/fdbserver/workloads/TPCC.actor.cpp @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2020 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbserver/workloads/TPCCWorkload.h b/fdbserver/workloads/TPCCWorkload.h index 471d587f4f..466ec21666 100644 --- a/fdbserver/workloads/TPCCWorkload.h +++ b/fdbserver/workloads/TPCCWorkload.h @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2020 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbserver/workloads/TagThrottleApi.actor.cpp b/fdbserver/workloads/TagThrottleApi.actor.cpp index d2bd47a158..b24ead205f 100644 --- a/fdbserver/workloads/TagThrottleApi.actor.cpp +++ b/fdbserver/workloads/TagThrottleApi.actor.cpp @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2020 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbserver/workloads/TargetedKill.actor.cpp b/fdbserver/workloads/TargetedKill.actor.cpp index e1ce5bc1e9..f9c39e499a 100644 --- a/fdbserver/workloads/TargetedKill.actor.cpp +++ b/fdbserver/workloads/TargetedKill.actor.cpp @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbserver/workloads/TaskBucketCorrectness.actor.cpp b/fdbserver/workloads/TaskBucketCorrectness.actor.cpp index 2c8c44e00a..7a14a29652 100644 --- a/fdbserver/workloads/TaskBucketCorrectness.actor.cpp +++ b/fdbserver/workloads/TaskBucketCorrectness.actor.cpp @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbserver/workloads/ThreadSafety.actor.cpp b/fdbserver/workloads/ThreadSafety.actor.cpp index 06d10a104f..eb7cba378d 100644 --- a/fdbserver/workloads/ThreadSafety.actor.cpp +++ b/fdbserver/workloads/ThreadSafety.actor.cpp @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbserver/workloads/Throttling.actor.cpp b/fdbserver/workloads/Throttling.actor.cpp index 51b7da07fc..fc906c8e55 100644 --- a/fdbserver/workloads/Throttling.actor.cpp +++ b/fdbserver/workloads/Throttling.actor.cpp @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbserver/workloads/Throughput.actor.cpp b/fdbserver/workloads/Throughput.actor.cpp index f31f757547..099154a171 100644 --- a/fdbserver/workloads/Throughput.actor.cpp +++ b/fdbserver/workloads/Throughput.actor.cpp @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbserver/workloads/TimeKeeperCorrectness.actor.cpp b/fdbserver/workloads/TimeKeeperCorrectness.actor.cpp index 5a4d08cb6c..bfbbd5fe1c 100644 --- a/fdbserver/workloads/TimeKeeperCorrectness.actor.cpp +++ b/fdbserver/workloads/TimeKeeperCorrectness.actor.cpp @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbserver/workloads/UDPWorkload.actor.cpp b/fdbserver/workloads/UDPWorkload.actor.cpp index cd707c61d8..d9ea607d93 100644 --- a/fdbserver/workloads/UDPWorkload.actor.cpp +++ b/fdbserver/workloads/UDPWorkload.actor.cpp @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2020 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbserver/workloads/UnitPerf.actor.cpp b/fdbserver/workloads/UnitPerf.actor.cpp index 55c31ef13a..fce0866f2c 100644 --- a/fdbserver/workloads/UnitPerf.actor.cpp +++ b/fdbserver/workloads/UnitPerf.actor.cpp @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbserver/workloads/UnitTests.actor.cpp b/fdbserver/workloads/UnitTests.actor.cpp index 8127b0ae87..f337b96514 100644 --- a/fdbserver/workloads/UnitTests.actor.cpp +++ b/fdbserver/workloads/UnitTests.actor.cpp @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbserver/workloads/Unreadable.actor.cpp b/fdbserver/workloads/Unreadable.actor.cpp index 26ac28a5b8..ee9dce883c 100644 --- a/fdbserver/workloads/Unreadable.actor.cpp +++ b/fdbserver/workloads/Unreadable.actor.cpp @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbserver/workloads/VersionStamp.actor.cpp b/fdbserver/workloads/VersionStamp.actor.cpp index 4b0403b0a2..89ee83c254 100644 --- a/fdbserver/workloads/VersionStamp.actor.cpp +++ b/fdbserver/workloads/VersionStamp.actor.cpp @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbserver/workloads/WatchAndWait.actor.cpp b/fdbserver/workloads/WatchAndWait.actor.cpp index 722df7f589..0d227923d3 100644 --- a/fdbserver/workloads/WatchAndWait.actor.cpp +++ b/fdbserver/workloads/WatchAndWait.actor.cpp @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbserver/workloads/Watches.actor.cpp b/fdbserver/workloads/Watches.actor.cpp index 08a904994b..47a030b061 100644 --- a/fdbserver/workloads/Watches.actor.cpp +++ b/fdbserver/workloads/Watches.actor.cpp @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbserver/workloads/WatchesSameKeyCorrectness.actor.cpp b/fdbserver/workloads/WatchesSameKeyCorrectness.actor.cpp index 2fe36518b1..44ac1e5892 100644 --- a/fdbserver/workloads/WatchesSameKeyCorrectness.actor.cpp +++ b/fdbserver/workloads/WatchesSameKeyCorrectness.actor.cpp @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2021 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbserver/workloads/WorkerErrors.actor.cpp b/fdbserver/workloads/WorkerErrors.actor.cpp index 3981e290d5..2ab8c3e255 100644 --- a/fdbserver/workloads/WorkerErrors.actor.cpp +++ b/fdbserver/workloads/WorkerErrors.actor.cpp @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbserver/workloads/WriteBandwidth.actor.cpp b/fdbserver/workloads/WriteBandwidth.actor.cpp index 22f5d1e60a..516f153761 100644 --- a/fdbserver/workloads/WriteBandwidth.actor.cpp +++ b/fdbserver/workloads/WriteBandwidth.actor.cpp @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbserver/workloads/WriteDuringRead.actor.cpp b/fdbserver/workloads/WriteDuringRead.actor.cpp index 66f65b38cd..9bcbf22898 100644 --- a/fdbserver/workloads/WriteDuringRead.actor.cpp +++ b/fdbserver/workloads/WriteDuringRead.actor.cpp @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbserver/workloads/WriteTagThrottling.actor.cpp b/fdbserver/workloads/WriteTagThrottling.actor.cpp index e2ef81597c..ad639b2a96 100644 --- a/fdbserver/workloads/WriteTagThrottling.actor.cpp +++ b/fdbserver/workloads/WriteTagThrottling.actor.cpp @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbserver/workloads/workloads.actor.h b/fdbserver/workloads/workloads.actor.h index bf70332f04..0b93e0ee5b 100644 --- a/fdbserver/workloads/workloads.actor.h +++ b/fdbserver/workloads/workloads.actor.h @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/fdbservice/FDBService.cpp b/fdbservice/FDBService.cpp index 61de5bb5d9..26b862ae6f 100644 --- a/fdbservice/FDBService.cpp +++ b/fdbservice/FDBService.cpp @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/flow/ActorCollection.actor.cpp b/flow/ActorCollection.actor.cpp index 907043bb8b..e19419b09d 100644 --- a/flow/ActorCollection.actor.cpp +++ b/flow/ActorCollection.actor.cpp @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/flow/ActorCollection.h b/flow/ActorCollection.h index 08adc83b61..ae4567f178 100644 --- a/flow/ActorCollection.h +++ b/flow/ActorCollection.h @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/flow/Arena.cpp b/flow/Arena.cpp index ac6043a0fe..2ab44eab85 100644 --- a/flow/Arena.cpp +++ b/flow/Arena.cpp @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2020 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/flow/Arena.h b/flow/Arena.h index 4f7e5cddff..59923d67a8 100644 --- a/flow/Arena.h +++ b/flow/Arena.h @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/flow/ArgParseUtil.h b/flow/ArgParseUtil.h index cc54be02d6..918877f6a5 100644 --- a/flow/ArgParseUtil.h +++ b/flow/ArgParseUtil.h @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2021 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/flow/AsioReactor.h b/flow/AsioReactor.h index 940539152a..7543792377 100644 --- a/flow/AsioReactor.h +++ b/flow/AsioReactor.h @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/flow/BooleanParam.h b/flow/BooleanParam.h index f6ef5744ce..d5491ac608 100644 --- a/flow/BooleanParam.h +++ b/flow/BooleanParam.h @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/flow/CompressedInt.actor.cpp b/flow/CompressedInt.actor.cpp index 36f563795e..ef889cf23f 100644 --- a/flow/CompressedInt.actor.cpp +++ b/flow/CompressedInt.actor.cpp @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/flow/CompressedInt.h b/flow/CompressedInt.h index ebd8056f8b..d37ce7e534 100644 --- a/flow/CompressedInt.h +++ b/flow/CompressedInt.h @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/flow/Deque.cpp b/flow/Deque.cpp index 2eb02ef9f6..5781198eff 100644 --- a/flow/Deque.cpp +++ b/flow/Deque.cpp @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/flow/Deque.h b/flow/Deque.h index 2037b56b31..4def1c8335 100644 --- a/flow/Deque.h +++ b/flow/Deque.h @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/flow/DeterministicRandom.cpp b/flow/DeterministicRandom.cpp index 6357feea40..648ec10642 100644 --- a/flow/DeterministicRandom.cpp +++ b/flow/DeterministicRandom.cpp @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/flow/DeterministicRandom.h b/flow/DeterministicRandom.h index b4f45b08bd..3f5115c371 100644 --- a/flow/DeterministicRandom.h +++ b/flow/DeterministicRandom.h @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/flow/Error.cpp b/flow/Error.cpp index 16776661c5..e756fb6d86 100644 --- a/flow/Error.cpp +++ b/flow/Error.cpp @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/flow/Error.h b/flow/Error.h index 590d2a747a..1ebb8620ef 100644 --- a/flow/Error.h +++ b/flow/Error.h @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/flow/EventTypes.actor.h b/flow/EventTypes.actor.h index 9fafb1ffe6..95abffff0f 100644 --- a/flow/EventTypes.actor.h +++ b/flow/EventTypes.actor.h @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/flow/FastAlloc.cpp b/flow/FastAlloc.cpp index 26a1df97d7..262d1b017c 100644 --- a/flow/FastAlloc.cpp +++ b/flow/FastAlloc.cpp @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/flow/FastAlloc.h b/flow/FastAlloc.h index a20ae056f7..935f00e358 100644 --- a/flow/FastAlloc.h +++ b/flow/FastAlloc.h @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/flow/FastRef.h b/flow/FastRef.h index 06221e4cc0..3c65cc6002 100644 --- a/flow/FastRef.h +++ b/flow/FastRef.h @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/flow/FaultInjection.cpp b/flow/FaultInjection.cpp index 5ba346efc5..d309574ae5 100644 --- a/flow/FaultInjection.cpp +++ b/flow/FaultInjection.cpp @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/flow/FaultInjection.h b/flow/FaultInjection.h index fa8f521076..0bbc6799c5 100644 --- a/flow/FaultInjection.h +++ b/flow/FaultInjection.h @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/flow/FileIdentifier.h b/flow/FileIdentifier.h index ff21643897..0110d06952 100644 --- a/flow/FileIdentifier.h +++ b/flow/FileIdentifier.h @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/flow/Histogram.cpp b/flow/Histogram.cpp index 4b5b851b04..1b6f154ff0 100644 --- a/flow/Histogram.cpp +++ b/flow/Histogram.cpp @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/flow/Histogram.h b/flow/Histogram.h index cf0156caca..b4bea7ab66 100644 --- a/flow/Histogram.h +++ b/flow/Histogram.h @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/flow/IDispatched.h b/flow/IDispatched.h index 395c3555d7..b3c8640b18 100644 --- a/flow/IDispatched.h +++ b/flow/IDispatched.h @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/flow/IRandom.h b/flow/IRandom.h index eb04a318b7..87f7f42424 100644 --- a/flow/IRandom.h +++ b/flow/IRandom.h @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/flow/IThreadPool.cpp b/flow/IThreadPool.cpp index 4ea43382a8..a898d30ee8 100644 --- a/flow/IThreadPool.cpp +++ b/flow/IThreadPool.cpp @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/flow/IThreadPool.h b/flow/IThreadPool.h index f9cf1d5616..0ef1bf1969 100644 --- a/flow/IThreadPool.h +++ b/flow/IThreadPool.h @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/flow/IndexedSet.actor.h b/flow/IndexedSet.actor.h index 1d7598f1d4..af4f88082f 100644 --- a/flow/IndexedSet.actor.h +++ b/flow/IndexedSet.actor.h @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/flow/IndexedSet.cpp b/flow/IndexedSet.cpp index 85e2b4098e..7f1337279c 100644 --- a/flow/IndexedSet.cpp +++ b/flow/IndexedSet.cpp @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/flow/IndexedSet.h b/flow/IndexedSet.h index 753abdfb11..197fe2acf6 100644 --- a/flow/IndexedSet.h +++ b/flow/IndexedSet.h @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/flow/Knobs.cpp b/flow/Knobs.cpp index 9f5d43436d..b0fbe613ee 100644 --- a/flow/Knobs.cpp +++ b/flow/Knobs.cpp @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/flow/Knobs.h b/flow/Knobs.h index 7877c7ebca..1bf2bd09cf 100644 --- a/flow/Knobs.h +++ b/flow/Knobs.h @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/flow/MetricSample.h b/flow/MetricSample.h index c332f08c5f..f8f9250a61 100644 --- a/flow/MetricSample.h +++ b/flow/MetricSample.h @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/flow/Net2.actor.cpp b/flow/Net2.actor.cpp index 15086f0b65..1e65e828c3 100644 --- a/flow/Net2.actor.cpp +++ b/flow/Net2.actor.cpp @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/flow/Net2Packet.cpp b/flow/Net2Packet.cpp index 673e97daac..fe1b5e4bf1 100644 --- a/flow/Net2Packet.cpp +++ b/flow/Net2Packet.cpp @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/flow/Net2Packet.h b/flow/Net2Packet.h index 8724233f24..5031a924b8 100644 --- a/flow/Net2Packet.h +++ b/flow/Net2Packet.h @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/flow/ObjectSerializer.h b/flow/ObjectSerializer.h index efb5dc0fd7..84937e31de 100644 --- a/flow/ObjectSerializer.h +++ b/flow/ObjectSerializer.h @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/flow/ObjectSerializerTraits.h b/flow/ObjectSerializerTraits.h index b191eba232..979db897e3 100644 --- a/flow/ObjectSerializerTraits.h +++ b/flow/ObjectSerializerTraits.h @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/flow/Platform.actor.cpp b/flow/Platform.actor.cpp index 52a5b1504f..9602fc431e 100644 --- a/flow/Platform.actor.cpp +++ b/flow/Platform.actor.cpp @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/flow/Platform.actor.h b/flow/Platform.actor.h index 25c3af6a84..1a2ef050a4 100644 --- a/flow/Platform.actor.h +++ b/flow/Platform.actor.h @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2020 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/flow/Platform.h b/flow/Platform.h index 3a8d75a3f5..dae2a63a08 100644 --- a/flow/Platform.h +++ b/flow/Platform.h @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/flow/Profiler.actor.cpp b/flow/Profiler.actor.cpp index 715d288ea7..26ba5999eb 100644 --- a/flow/Profiler.actor.cpp +++ b/flow/Profiler.actor.cpp @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/flow/Profiler.h b/flow/Profiler.h index 05d1213a9f..8fef792679 100644 --- a/flow/Profiler.h +++ b/flow/Profiler.h @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/flow/ProtocolVersion.h b/flow/ProtocolVersion.h index 81deef5a4e..b89dac1f8b 100644 --- a/flow/ProtocolVersion.h +++ b/flow/ProtocolVersion.h @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2019 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/flow/SendBufferIterator.h b/flow/SendBufferIterator.h index 001d546265..81fa8aa822 100644 --- a/flow/SendBufferIterator.h +++ b/flow/SendBufferIterator.h @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/flow/SignalSafeUnwind.cpp b/flow/SignalSafeUnwind.cpp index f815801163..a43dd81277 100644 --- a/flow/SignalSafeUnwind.cpp +++ b/flow/SignalSafeUnwind.cpp @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/flow/SignalSafeUnwind.h b/flow/SignalSafeUnwind.h index 736ece98eb..2d17b6637a 100644 --- a/flow/SignalSafeUnwind.h +++ b/flow/SignalSafeUnwind.h @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/flow/StreamCipher.cpp b/flow/StreamCipher.cpp index 92edff9d7a..cf5f8d7b2f 100644 --- a/flow/StreamCipher.cpp +++ b/flow/StreamCipher.cpp @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2020 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/flow/StreamCipher.h b/flow/StreamCipher.h index d963472d08..98db51fe8d 100644 --- a/flow/StreamCipher.h +++ b/flow/StreamCipher.h @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2020 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/flow/SystemMonitor.cpp b/flow/SystemMonitor.cpp index 46a99cf439..6127fa7eb4 100644 --- a/flow/SystemMonitor.cpp +++ b/flow/SystemMonitor.cpp @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/flow/SystemMonitor.h b/flow/SystemMonitor.h index f52508609b..37424a4826 100644 --- a/flow/SystemMonitor.h +++ b/flow/SystemMonitor.h @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/flow/TDMetric.actor.h b/flow/TDMetric.actor.h index f2d30d1739..732a3f9ea8 100644 --- a/flow/TDMetric.actor.h +++ b/flow/TDMetric.actor.h @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/flow/TDMetric.cpp b/flow/TDMetric.cpp index 002d96c0ca..0d66e286f0 100644 --- a/flow/TDMetric.cpp +++ b/flow/TDMetric.cpp @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/flow/TLSConfig.actor.cpp b/flow/TLSConfig.actor.cpp index 192f3558d8..4f1d385b9a 100644 --- a/flow/TLSConfig.actor.cpp +++ b/flow/TLSConfig.actor.cpp @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2020 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/flow/TLSConfig.actor.h b/flow/TLSConfig.actor.h index 14415f8547..723424da00 100644 --- a/flow/TLSConfig.actor.h +++ b/flow/TLSConfig.actor.h @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2020 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/flow/ThreadHelper.actor.cpp b/flow/ThreadHelper.actor.cpp index 06645f8d3e..5b5cf83896 100644 --- a/flow/ThreadHelper.actor.cpp +++ b/flow/ThreadHelper.actor.cpp @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2021 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/flow/ThreadHelper.actor.h b/flow/ThreadHelper.actor.h index 3b5315753a..6627b9e25e 100644 --- a/flow/ThreadHelper.actor.h +++ b/flow/ThreadHelper.actor.h @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/flow/ThreadPrimitives.cpp b/flow/ThreadPrimitives.cpp index a4d66b73e0..f87dc27e1e 100644 --- a/flow/ThreadPrimitives.cpp +++ b/flow/ThreadPrimitives.cpp @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/flow/ThreadPrimitives.h b/flow/ThreadPrimitives.h index 7ee3cc90f1..f15b39c564 100644 --- a/flow/ThreadPrimitives.h +++ b/flow/ThreadPrimitives.h @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/flow/Trace.cpp b/flow/Trace.cpp index 1b3eefc5fd..86d572d4fe 100644 --- a/flow/Trace.cpp +++ b/flow/Trace.cpp @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/flow/Trace.h b/flow/Trace.h index cf89ce6b18..2d686d30f3 100644 --- a/flow/Trace.h +++ b/flow/Trace.h @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/flow/Tracing.actor.cpp b/flow/Tracing.actor.cpp index 0b3f443df5..43d1610b8e 100644 --- a/flow/Tracing.actor.cpp +++ b/flow/Tracing.actor.cpp @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2020 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/flow/Tracing.h b/flow/Tracing.h index e9aff85593..d74f0a760a 100644 --- a/flow/Tracing.h +++ b/flow/Tracing.h @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2020 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/flow/TypeTraits.h b/flow/TypeTraits.h index 261bb2e860..b8a2e5b406 100644 --- a/flow/TypeTraits.h +++ b/flow/TypeTraits.h @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/flow/UnitTest.cpp b/flow/UnitTest.cpp index ccb5fed38a..67feee60f4 100644 --- a/flow/UnitTest.cpp +++ b/flow/UnitTest.cpp @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/flow/UnitTest.h b/flow/UnitTest.h index fe2017720a..cb2daf3331 100644 --- a/flow/UnitTest.h +++ b/flow/UnitTest.h @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/flow/Util.h b/flow/Util.h index ed4c3eefc2..727065a4cf 100644 --- a/flow/Util.h +++ b/flow/Util.h @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/flow/WriteOnlySet.actor.cpp b/flow/WriteOnlySet.actor.cpp index b9f7c17363..46077e4aa2 100644 --- a/flow/WriteOnlySet.actor.cpp +++ b/flow/WriteOnlySet.actor.cpp @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2021 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/flow/WriteOnlySet.h b/flow/WriteOnlySet.h index e64c44cab0..6c8cfcda8f 100644 --- a/flow/WriteOnlySet.h +++ b/flow/WriteOnlySet.h @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2021 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/flow/actorcompiler.h b/flow/actorcompiler.h index e783900e45..1ed8c5391c 100644 --- a/flow/actorcompiler.h +++ b/flow/actorcompiler.h @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/flow/error_definitions.h b/flow/error_definitions.h index b0a5a25a57..7214df4c3d 100755 --- a/flow/error_definitions.h +++ b/flow/error_definitions.h @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/flow/flat_buffers.cpp b/flow/flat_buffers.cpp index 277575064c..eb6daad787 100644 --- a/flow/flat_buffers.cpp +++ b/flow/flat_buffers.cpp @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/flow/flat_buffers.h b/flow/flat_buffers.h index cc5c453dc2..04f3a881bd 100644 --- a/flow/flat_buffers.h +++ b/flow/flat_buffers.h @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/flow/flow.cpp b/flow/flow.cpp index 46e6a2a60b..285de7e904 100644 --- a/flow/flow.cpp +++ b/flow/flow.cpp @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/flow/flow.h b/flow/flow.h index 6f40a2be8b..23fced7bf4 100644 --- a/flow/flow.h +++ b/flow/flow.h @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/flow/folly_memcpy.h b/flow/folly_memcpy.h index e6bdd17946..fd9d4860ef 100644 --- a/flow/folly_memcpy.h +++ b/flow/folly_memcpy.h @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/flow/genericactors.actor.cpp b/flow/genericactors.actor.cpp index 9b7f906713..6bb1e3fd8d 100644 --- a/flow/genericactors.actor.cpp +++ b/flow/genericactors.actor.cpp @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/flow/genericactors.actor.h b/flow/genericactors.actor.h index fd7b88d8d4..f30ef772e5 100644 --- a/flow/genericactors.actor.h +++ b/flow/genericactors.actor.h @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/flow/network.cpp b/flow/network.cpp index 6dd84869fa..c0515eda16 100644 --- a/flow/network.cpp +++ b/flow/network.cpp @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/flow/network.h b/flow/network.h index 6ab9e655ed..967a145b7e 100644 --- a/flow/network.h +++ b/flow/network.h @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/flow/serialize.cpp b/flow/serialize.cpp index a39461d3dd..d51416a308 100644 --- a/flow/serialize.cpp +++ b/flow/serialize.cpp @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2019 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/flow/serialize.h b/flow/serialize.h index 06824feeb3..7609ccf678 100644 --- a/flow/serialize.h +++ b/flow/serialize.h @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/flow/unactorcompiler.h b/flow/unactorcompiler.h index 2d366b93ff..d48f7001ab 100644 --- a/flow/unactorcompiler.h +++ b/flow/unactorcompiler.h @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/flow/version.cpp b/flow/version.cpp index 2b2ffe8f68..1c4f1eb21b 100644 --- a/flow/version.cpp +++ b/flow/version.cpp @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/flowbench/BenchCallback.actor.cpp b/flowbench/BenchCallback.actor.cpp index a7c7fd207e..15ec207d10 100644 --- a/flowbench/BenchCallback.actor.cpp +++ b/flowbench/BenchCallback.actor.cpp @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2020 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/flowbench/BenchEncrypt.cpp b/flowbench/BenchEncrypt.cpp index 4b4cc6159c..e99e80a209 100644 --- a/flowbench/BenchEncrypt.cpp +++ b/flowbench/BenchEncrypt.cpp @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2020 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/flowbench/BenchHash.cpp b/flowbench/BenchHash.cpp index e5a2fd8401..b5e0dc5507 100644 --- a/flowbench/BenchHash.cpp +++ b/flowbench/BenchHash.cpp @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2020 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/flowbench/BenchIONet2.actor.cpp b/flowbench/BenchIONet2.actor.cpp index 9572ccf9ff..fb98cafd4e 100644 --- a/flowbench/BenchIONet2.actor.cpp +++ b/flowbench/BenchIONet2.actor.cpp @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2020 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/flowbench/BenchIterate.cpp b/flowbench/BenchIterate.cpp index 19a3219287..793c3fc307 100644 --- a/flowbench/BenchIterate.cpp +++ b/flowbench/BenchIterate.cpp @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2020 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/flowbench/BenchMem.cpp b/flowbench/BenchMem.cpp index 5373353fb0..8696461efd 100644 --- a/flowbench/BenchMem.cpp +++ b/flowbench/BenchMem.cpp @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2020 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/flowbench/BenchMetadataCheck.cpp b/flowbench/BenchMetadataCheck.cpp index adf4ea6e3c..c028e3deec 100644 --- a/flowbench/BenchMetadataCheck.cpp +++ b/flowbench/BenchMetadataCheck.cpp @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2020 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/flowbench/BenchNet2.actor.cpp b/flowbench/BenchNet2.actor.cpp index 44b527e472..8eddc476eb 100644 --- a/flowbench/BenchNet2.actor.cpp +++ b/flowbench/BenchNet2.actor.cpp @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2020 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/flowbench/BenchPopulate.cpp b/flowbench/BenchPopulate.cpp index 0223b4e98e..35213b4171 100644 --- a/flowbench/BenchPopulate.cpp +++ b/flowbench/BenchPopulate.cpp @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2020 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/flowbench/BenchRandom.cpp b/flowbench/BenchRandom.cpp index e2d114b93c..47d607cffe 100644 --- a/flowbench/BenchRandom.cpp +++ b/flowbench/BenchRandom.cpp @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2020 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/flowbench/BenchRef.cpp b/flowbench/BenchRef.cpp index facf09db68..297e32a66b 100644 --- a/flowbench/BenchRef.cpp +++ b/flowbench/BenchRef.cpp @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2020 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/flowbench/BenchStream.actor.cpp b/flowbench/BenchStream.actor.cpp index 6f77f7ca6f..bc8f892dd8 100644 --- a/flowbench/BenchStream.actor.cpp +++ b/flowbench/BenchStream.actor.cpp @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2020 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/flowbench/BenchTimer.cpp b/flowbench/BenchTimer.cpp index c97bf66304..90a542a3f9 100644 --- a/flowbench/BenchTimer.cpp +++ b/flowbench/BenchTimer.cpp @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2020 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/flowbench/GlobalData.cpp b/flowbench/GlobalData.cpp index eef9c718e5..8cdd38a844 100644 --- a/flowbench/GlobalData.cpp +++ b/flowbench/GlobalData.cpp @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2020 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/flowbench/GlobalData.h b/flowbench/GlobalData.h index 11b956c029..3f2d1f53db 100644 --- a/flowbench/GlobalData.h +++ b/flowbench/GlobalData.h @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2020 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/flowbench/flowbench.actor.cpp b/flowbench/flowbench.actor.cpp index 951bfb789f..f61583cb1c 100644 --- a/flowbench/flowbench.actor.cpp +++ b/flowbench/flowbench.actor.cpp @@ -3,7 +3,7 @@ * * This source file is part of the FoundationDB open source project * - * Copyright 2013-2020 Apple Inc. and the FoundationDB project authors + * 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. diff --git a/packaging/docker/fdb.bash b/packaging/docker/fdb.bash index 5d0f3e0ff7..d0d05e466c 100755 --- a/packaging/docker/fdb.bash +++ b/packaging/docker/fdb.bash @@ -5,7 +5,7 @@ # # This source file is part of the FoundationDB open source project # -# Copyright 2013-2021 Apple Inc. and the FoundationDB project authors +# 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. diff --git a/packaging/docker/samples/golang/app/start.bash b/packaging/docker/samples/golang/app/start.bash index 08e215ffb9..468f1426c3 100755 --- a/packaging/docker/samples/golang/app/start.bash +++ b/packaging/docker/samples/golang/app/start.bash @@ -4,7 +4,7 @@ # # This source file is part of the FoundationDB open source project # -# Copyright 2013-2019 Apple Inc. and the FoundationDB project authors +# 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. diff --git a/packaging/docker/samples/local/start.bash b/packaging/docker/samples/local/start.bash index 64def42f51..0abc55abae 100755 --- a/packaging/docker/samples/local/start.bash +++ b/packaging/docker/samples/local/start.bash @@ -5,7 +5,7 @@ # # This source file is part of the FoundationDB open source project # -# Copyright 2013-2018 Apple Inc. and the FoundationDB project authors +# 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. diff --git a/packaging/docker/samples/local/stop.bash b/packaging/docker/samples/local/stop.bash index 55acc50953..7f99f986f5 100755 --- a/packaging/docker/samples/local/stop.bash +++ b/packaging/docker/samples/local/stop.bash @@ -5,7 +5,7 @@ # # This source file is part of the FoundationDB open source project # -# Copyright 2013-2018 Apple Inc. and the FoundationDB project authors +# 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. diff --git a/packaging/docker/samples/python/app/start.bash b/packaging/docker/samples/python/app/start.bash index e16b1828cd..6ed2b8341a 100644 --- a/packaging/docker/samples/python/app/start.bash +++ b/packaging/docker/samples/python/app/start.bash @@ -4,7 +4,7 @@ # # This source file is part of the FoundationDB open source project # -# Copyright 2013-2018 Apple Inc. and the FoundationDB project authors +# 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. From 7b3a65676be9072bb2ad9c79c2ceefd5aa55ab82 Mon Sep 17 00:00:00 2001 From: Josh Slocum Date: Mon, 21 Mar 2022 17:06:47 -0500 Subject: [PATCH 382/413] Fixing another change feed fetch and move race --- fdbserver/storageserver.actor.cpp | 21 +++++++++++++++------ 1 file changed, 15 insertions(+), 6 deletions(-) diff --git a/fdbserver/storageserver.actor.cpp b/fdbserver/storageserver.actor.cpp index f03f2e5e78..20f337dd5e 100644 --- a/fdbserver/storageserver.actor.cpp +++ b/fdbserver/storageserver.actor.cpp @@ -2452,7 +2452,16 @@ ACTOR Future localChangeFeedStream(StorageServer* data, } } } catch (Error& e) { - TraceEvent(SevError, "LocalChangeFeedError", data->thisServerID).error(e).detail("CFID", rangeID.printable()); + if (e.code() == error_code_unknown_change_feed) { + TEST(true); // CF was moved away, no more local data to merge with + // Send endVersion so local stream is effectively done. We couldn't have send that already, because that + // would mean the stream would have finished without error + results.send(MutationsAndVersionRef(end, invalidVersion)); + } else { + TraceEvent(SevError, "LocalChangeFeedError", data->thisServerID) + .error(e) + .detail("CFID", rangeID.printable()); + } throw; } } @@ -4975,12 +4984,12 @@ ACTOR Future fetchChangeFeedApplier(StorageServer* data, wait(yield()); } } catch (Error& e) { - TraceEvent(SevDebug, "FetchChangeFeedError", data->thisServerID) - .errorUnsuppressed(e) - .detail("RangeID", rangeId.printable()) - .detail("Range", range.toString()) - .detail("EndVersion", endVersion); if (e.code() != error_code_end_of_stream) { + TraceEvent(SevDebug, "FetchChangeFeedError", data->thisServerID) + .errorUnsuppressed(e) + .detail("RangeID", rangeId.printable()) + .detail("Range", range.toString()) + .detail("EndVersion", endVersion); throw; } } From 917c7acca54ff4b6667a04f0dc421225eb939b0d Mon Sep 17 00:00:00 2001 From: Josh Slocum Date: Mon, 21 Mar 2022 22:21:40 -0500 Subject: [PATCH 383/413] Using desiredOldest for when change feed metadata is safe --- fdbserver/BlobWorker.actor.cpp | 7 ++++--- fdbserver/storageserver.actor.cpp | 14 +++++++------- 2 files changed, 11 insertions(+), 10 deletions(-) diff --git a/fdbserver/BlobWorker.actor.cpp b/fdbserver/BlobWorker.actor.cpp index 1e3e4232cf..07000fb4e7 100644 --- a/fdbserver/BlobWorker.actor.cpp +++ b/fdbserver/BlobWorker.actor.cpp @@ -1261,7 +1261,7 @@ ACTOR Future blobGranuleUpdateFiles(Reference bwData, metadata->keyRange.end.printable(), metadata->originalEpoch, metadata->originalSeqno); - fmt::print(" CFID: {}\n", startState.granuleID.toString()); + fmt::print(" CFID: {} ({})\n", startState.granuleID.toString(), cfKey.printable()); fmt::print(" CF Start Version: {}\n", startState.changeFeedStartVersion); fmt::print(" Previous Durable Version: {}\n", startState.previousDurableVersion); fmt::print(" doSnapshot={}\n", startState.doSnapshot ? "T" : "F"); @@ -2427,6 +2427,7 @@ ACTOR Future openGranule(Reference bwData, As ASSERT(req.type != AssignRequestType::Continue); state Transaction tr(bwData->db); state Key lockKey = blobGranuleLockKeyFor(req.keyRange); + state UID newGranuleID = deterministicRandom()->randomUniqueID(); if (BW_DEBUG) { fmt::print("{0} [{1} - {2}) opening\n", @@ -2491,8 +2492,8 @@ ACTOR Future openGranule(Reference bwData, As info.granuleID = info.history.get().value.granuleID; } else { // FIXME: could avoid max uid for granule ids here - // if this granule is not derived from a split or merge, create the granule id here - info.granuleID = deterministicRandom()->randomUniqueID(); + // if this granule is not derived from a split or merge, use new granule id + info.granuleID = newGranuleID; } wait(updateChangeFeed( &tr, granuleIDToCFKey(info.granuleID), ChangeFeedStatus::CHANGE_FEED_CREATE, req.keyRange)); diff --git a/fdbserver/storageserver.actor.cpp b/fdbserver/storageserver.actor.cpp index 20f337dd5e..c06dfb8951 100644 --- a/fdbserver/storageserver.actor.cpp +++ b/fdbserver/storageserver.actor.cpp @@ -1943,7 +1943,7 @@ ACTOR Future overlappingChangeFeedsQ(StorageServer* data, OverlappingChang return Void(); } - Version knownCommittedRequired = invalidVersion; + Version metadataVersion = invalidVersion; auto ranges = data->keyChangeFeed.intersectingRanges(req.range); std::map> rangeIds; @@ -1952,12 +1952,12 @@ ACTOR Future overlappingChangeFeedsQ(StorageServer* data, OverlappingChang // Can't tell other SS about a change feed create or stopVersion that may get rolled back, and we only need // to tell it about the metadata if req.minVersion > metadataVersion, since it will get the information from // its own private mutations if it hasn't processed up that version yet - knownCommittedRequired = std::max(knownCommittedRequired, it->metadataCreateVersion); + metadataVersion = std::max(metadataVersion, it->metadataCreateVersion); Version stopVersion; if (it->stopVersion != MAX_VERSION && req.minVersion > it->stopVersion) { stopVersion = it->stopVersion; - knownCommittedRequired = std::max(knownCommittedRequired, stopVersion); + metadataVersion = std::max(metadataVersion, stopVersion); } else { stopVersion = MAX_VERSION; } @@ -1971,10 +1971,10 @@ ACTOR Future overlappingChangeFeedsQ(StorageServer* data, OverlappingChang it.first, std::get<0>(it.second), std::get<1>(it.second), std::get<2>(it.second))); } - // Make sure all of the stop versions we are sending aren't going to get rolled back - if (knownCommittedRequired != invalidVersion && knownCommittedRequired > data->knownCommittedVersion.get()) { - TEST(true); // overlapping change feeds waiting for stop version to be committed - wait(data->knownCommittedVersion.whenAtLeast(knownCommittedRequired)); + // Make sure all of the metadata we are sending won't get rolled back + if (metadataVersion != invalidVersion && metadataVersion > data->knownCommittedVersion.get()) { + TEST(true); // overlapping change feeds waiting for metadata version to be committed + wait(data->desiredOldestVersion.whenAtLeast(metadataVersion)); } req.reply.send(reply); return Void(); From 790ef9ff36a5c281b2dfa39fae3253a10e1310e3 Mon Sep 17 00:00:00 2001 From: Neethu Haneesha Bingi Date: Tue, 15 Mar 2022 00:38:07 -0700 Subject: [PATCH 384/413] Adding rocksdb compaction readahead size option. --- fdbclient/ServerKnobs.cpp | 1 + fdbclient/ServerKnobs.h | 1 + fdbserver/KeyValueStoreRocksDB.actor.cpp | 3 +++ 3 files changed, 5 insertions(+) diff --git a/fdbclient/ServerKnobs.cpp b/fdbclient/ServerKnobs.cpp index 94a5559119..f50aca30a9 100644 --- a/fdbclient/ServerKnobs.cpp +++ b/fdbclient/ServerKnobs.cpp @@ -380,6 +380,7 @@ void ServerKnobs::initialize(Randomize randomize, ClientKnobs* clientKnobs, IsSi // Set ROCKSDB_CAN_COMMIT_DELAY_TIMES_ON_OVERLOAD to 0, to disable init( ROCKSDB_CAN_COMMIT_DELAY_ON_OVERLOAD, 1 ); init( ROCKSDB_CAN_COMMIT_DELAY_TIMES_ON_OVERLOAD, 5 ); + init( ROCKSDB_COMPACTION_READAHEAD_SIZE, 32768 ); // 32 KB, performs bigger reads when doing compaction. // Leader election bool longLeaderElection = randomize && BUGGIFY; diff --git a/fdbclient/ServerKnobs.h b/fdbclient/ServerKnobs.h index 4575740ff8..f636f91cd4 100644 --- a/fdbclient/ServerKnobs.h +++ b/fdbclient/ServerKnobs.h @@ -307,6 +307,7 @@ public: int64_t ROCKSDB_CAN_COMMIT_COMPACT_BYTES_LIMIT; int ROCKSDB_CAN_COMMIT_DELAY_ON_OVERLOAD; int ROCKSDB_CAN_COMMIT_DELAY_TIMES_ON_OVERLOAD; + int64_t ROCKSDB_COMPACTION_READAHEAD_SIZE; // Leader election int MAX_NOTIFICATIONS; diff --git a/fdbserver/KeyValueStoreRocksDB.actor.cpp b/fdbserver/KeyValueStoreRocksDB.actor.cpp index b2bdd20b0b..7069c9de42 100644 --- a/fdbserver/KeyValueStoreRocksDB.actor.cpp +++ b/fdbserver/KeyValueStoreRocksDB.actor.cpp @@ -306,6 +306,9 @@ rocksdb::Options getOptions() { if (SERVER_KNOBS->ROCKSDB_MAX_SUBCOMPACTIONS > 0) { options.max_subcompactions = SERVER_KNOBS->ROCKSDB_MAX_SUBCOMPACTIONS; } + if (SERVER_KNOBS->ROCKSDB_COMPACTION_READAHEAD_SIZE > 0) { + options.compaction_readahead_size = SERVER_KNOBS->ROCKSDB_COMPACTION_READAHEAD_SIZE; + } options.statistics = rocksdb::CreateDBStatistics(); options.statistics->set_stats_level(rocksdb::kExceptHistogramOrTimers); From 4e20da76fbeb11414dc5c9bc7ed14cc01a730a1c Mon Sep 17 00:00:00 2001 From: Josh Slocum Date: Tue, 22 Mar 2022 15:37:16 -0500 Subject: [PATCH 385/413] bump --- fdbserver/BlobWorker.actor.cpp | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/fdbserver/BlobWorker.actor.cpp b/fdbserver/BlobWorker.actor.cpp index 07000fb4e7..4d87ccbe7f 100644 --- a/fdbserver/BlobWorker.actor.cpp +++ b/fdbserver/BlobWorker.actor.cpp @@ -71,7 +71,8 @@ struct GranuleMetadata : NonCopyable, ReferenceCounted { KeyRange keyRange; GranuleFiles files; - Standalone currentDeltas; // only contain deltas in pendingDeltaVersion + 1, bufferedDeltaVersion + Standalone + currentDeltas; // only contain deltas in pendingDeltaVersion + 1 through bufferedDeltaVersion uint64_t bytesInNewDeltaFiles = 0; uint64_t bufferedDeltaBytes = 0; From cff1bc24fccb2cddfb4c5958e68963c3f6f827f3 Mon Sep 17 00:00:00 2001 From: Jingyu Zhou Date: Tue, 22 Mar 2022 16:04:56 -0700 Subject: [PATCH 386/413] Fix decoder for missing ClearRanges If the ClearRange mutation happens within the given key prefix space, previously the mutation is not printed. Fix by checking the prefix range overlaps with the ClearRange mutation. --- fdbbackup/FileDecoder.actor.cpp | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/fdbbackup/FileDecoder.actor.cpp b/fdbbackup/FileDecoder.actor.cpp index e28b00b2f7..93c6cdc084 100644 --- a/fdbbackup/FileDecoder.actor.cpp +++ b/fdbbackup/FileDecoder.actor.cpp @@ -457,7 +457,8 @@ ACTOR Future process_file(Reference container, LogFile f print = m.param1.startsWith(StringRef(params.prefix)); } else if (m.type == MutationRef::ClearRange) { KeyRange range(KeyRangeRef(m.param1, m.param2)); - print = range.contains(StringRef(params.prefix)); + KeyRange range2 = prefixRange(StringRef(params.prefix)); + print = range.intersects(range2); } else { ASSERT(false); } From af9deeabc2ced523f362baf3c5970829dfd28eb3 Mon Sep 17 00:00:00 2001 From: Chaoguang Lin Date: Tue, 22 Mar 2022 14:54:45 -0700 Subject: [PATCH 387/413] Move the Promise before the Future vector to be destroyed after the vector --- fdbserver/PaxosConfigConsumer.actor.cpp | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/fdbserver/PaxosConfigConsumer.actor.cpp b/fdbserver/PaxosConfigConsumer.actor.cpp index 4ada8cf701..934efccfa3 100644 --- a/fdbserver/PaxosConfigConsumer.actor.cpp +++ b/fdbserver/PaxosConfigConsumer.actor.cpp @@ -40,6 +40,9 @@ struct QuorumVersion { }; class GetCommittedVersionQuorum { + // Set to the versions a quorum of + // ConfigNodes agree on, otherwise unset. + Promise quorumVersion; std::vector> actors; std::vector cfis; std::map> replies; @@ -52,9 +55,6 @@ class GetCommittedVersionQuorum { Version lastSeenVersion; size_t totalRepliesReceived{ 0 }; size_t maxAgreement{ 0 }; - // Set to the versions a quorum of - // ConfigNodes agree on, otherwise unset. - Promise quorumVersion; // Stores the largest committed version out of all responses. Version largestCommitted{ 0 }; From 30651bf2c6123b5f2974ccdd7f5100d677e9755e Mon Sep 17 00:00:00 2001 From: sfc-gh-tclinkenbeard Date: Tue, 22 Mar 2022 17:06:33 -0700 Subject: [PATCH 388/413] Fix order of TagInfo constructor arguments --- fdbserver/Ratekeeper.actor.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/fdbserver/Ratekeeper.actor.cpp b/fdbserver/Ratekeeper.actor.cpp index 275716f7ac..94e41d0f52 100644 --- a/fdbserver/Ratekeeper.actor.cpp +++ b/fdbserver/Ratekeeper.actor.cpp @@ -999,7 +999,7 @@ void StorageQueueInfo::refreshCommitCost(double elapsed) { // TraceEvent("RefreshSSCommitCost").detail("TotalWriteCost", totalWriteCost).detail("TotalWriteOps",totalWriteOps); ASSERT_GT(totalWriteCosts, 0); maxBusyness = double(maxCost.getCostSum()) / totalWriteCosts; - busiestWriteTags.emplace_back(busiestTag, maxBusyness, maxRate); + busiestWriteTags.emplace_back(busiestTag, maxRate, maxBusyness); } TraceEvent("BusiestWriteTag", id) From c72197122dcdf2fdd59595309dc5dcb4ecd1a77c Mon Sep 17 00:00:00 2001 From: Josh Slocum Date: Wed, 23 Mar 2022 08:07:49 -0500 Subject: [PATCH 389/413] bump --- fdbserver/BlobWorker.actor.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/fdbserver/BlobWorker.actor.cpp b/fdbserver/BlobWorker.actor.cpp index 4d87ccbe7f..e939d71ec6 100644 --- a/fdbserver/BlobWorker.actor.cpp +++ b/fdbserver/BlobWorker.actor.cpp @@ -132,7 +132,7 @@ struct GranuleRangeMetadata { : lastEpoch(epoch), lastSeqno(seqno), activeMetadata(activeMetadata) {} }; -// represents a previous version of a granule, and optionally the files that compose it +// represents a previous version of a granule, and optionally the files that compose it. struct GranuleHistoryEntry : NonCopyable, ReferenceCounted { KeyRange range; UID granuleID; From 3379f1e9742e574ce5979f454110fb49c761728d Mon Sep 17 00:00:00 2001 From: Jingyu Zhou Date: Fri, 18 Mar 2022 15:14:02 -0700 Subject: [PATCH 390/413] Move resolutionBalancing() back to master This revert the behavior done by a recent refactor on master recovery in PR #6191. --- fdbserver/ClusterRecovery.actor.cpp | 151 +----------------- fdbserver/ClusterRecovery.actor.h | 16 +- fdbserver/MasterInterface.h | 19 ++- fdbserver/masterserver.actor.cpp | 188 ++++++++++++++++++++--- fdbserver/workloads/KillRegion.actor.cpp | 5 +- 5 files changed, 188 insertions(+), 191 deletions(-) diff --git a/fdbserver/ClusterRecovery.actor.cpp b/fdbserver/ClusterRecovery.actor.cpp index ac8fee9b9e..22b6807a00 100644 --- a/fdbserver/ClusterRecovery.actor.cpp +++ b/fdbserver/ClusterRecovery.actor.cpp @@ -470,7 +470,6 @@ ACTOR Future trackTlogRecovery(Reference self, self->dbgid) .detail("StatusCode", RecoveryStatus::fully_recovered) .detail("Status", RecoveryStatus::names[RecoveryStatus::fully_recovered]) - .detail("FullyRecoveredAtVersion", self->version) .detail("ClusterId", self->clusterId) .trackLatest(self->clusterRecoveryStateEventHolder->trackingKey); @@ -511,144 +510,6 @@ ACTOR Future trackTlogRecovery(Reference self, } } -std::pair findRange(CoalescedKeyRangeMap& key_resolver, - Standalone>& movedRanges, - int src, - int dest) { - auto ranges = key_resolver.ranges(); - auto prev = ranges.begin(); - auto it = ranges.begin(); - ++it; - if (it == ranges.end()) { - if (ranges.begin().value() != src || - std::find(movedRanges.begin(), movedRanges.end(), ResolverMoveRef(ranges.begin()->range(), dest)) != - movedRanges.end()) - throw operation_failed(); - return std::make_pair(ranges.begin().range(), true); - } - - std::set borders; - // If possible expand an existing boundary between the two resolvers - for (; it != ranges.end(); ++it) { - if (it->value() == src && prev->value() == dest && - std::find(movedRanges.begin(), movedRanges.end(), ResolverMoveRef(it->range(), dest)) == - movedRanges.end()) { - return std::make_pair(it->range(), true); - } - if (it->value() == dest && prev->value() == src && - std::find(movedRanges.begin(), movedRanges.end(), ResolverMoveRef(prev->range(), dest)) == - movedRanges.end()) { - return std::make_pair(prev->range(), false); - } - if (it->value() == dest) - borders.insert(prev->value()); - if (prev->value() == dest) - borders.insert(it->value()); - ++prev; - } - - prev = ranges.begin(); - it = ranges.begin(); - ++it; - // If possible create a new boundry which doesn't exist yet - for (; it != ranges.end(); ++it) { - if (it->value() == src && !borders.count(prev->value()) && - std::find(movedRanges.begin(), movedRanges.end(), ResolverMoveRef(it->range(), dest)) == - movedRanges.end()) { - return std::make_pair(it->range(), true); - } - if (prev->value() == src && !borders.count(it->value()) && - std::find(movedRanges.begin(), movedRanges.end(), ResolverMoveRef(prev->range(), dest)) == - movedRanges.end()) { - return std::make_pair(prev->range(), false); - } - ++prev; - } - - it = ranges.begin(); - for (; it != ranges.end(); ++it) { - if (it->value() == src && - std::find(movedRanges.begin(), movedRanges.end(), ResolverMoveRef(it->range(), dest)) == - movedRanges.end()) { - return std::make_pair(it->range(), true); - } - } - throw operation_failed(); // we are already attempting to move all of the data one resolver is assigned, so do not - // move anything -} - -ACTOR Future resolutionBalancing(Reference self) { - state CoalescedKeyRangeMap key_resolver; - key_resolver.insert(allKeys, 0); - loop { - wait(delay(SERVER_KNOBS->MIN_BALANCE_TIME, TaskPriority::ResolutionMetrics)); - while (self->resolverChanges.get().size()) - wait(self->resolverChanges.onChange()); - state std::vector> futures; - for (auto& p : self->resolvers) - futures.push_back( - brokenPromiseToNever(p.metrics.getReply(ResolutionMetricsRequest(), TaskPriority::ResolutionMetrics))); - wait(waitForAll(futures)); - state IndexedSet, NoMetric> metrics; - - int64_t total = 0; - for (int i = 0; i < futures.size(); i++) { - total += futures[i].get().value; - metrics.insert(std::make_pair(futures[i].get().value, i), NoMetric()); - //TraceEvent("ResolverMetric").detail("I", i).detail("Metric", futures[i].get()); - } - if (metrics.lastItem()->first - metrics.begin()->first > SERVER_KNOBS->MIN_BALANCE_DIFFERENCE) { - try { - state int src = metrics.lastItem()->second; - state int dest = metrics.begin()->second; - state int64_t amount = std::min(metrics.lastItem()->first - total / self->resolvers.size(), - total / self->resolvers.size() - metrics.begin()->first) / - 2; - state Standalone> movedRanges; - - loop { - state std::pair range = findRange(key_resolver, movedRanges, src, dest); - - ResolutionSplitRequest req; - req.front = range.second; - req.offset = amount; - req.range = range.first; - - ResolutionSplitReply split = - wait(brokenPromiseToNever(self->resolvers[metrics.lastItem()->second].split.getReply( - req, TaskPriority::ResolutionMetrics))); - KeyRangeRef moveRange = range.second ? KeyRangeRef(range.first.begin, split.key) - : KeyRangeRef(split.key, range.first.end); - movedRanges.push_back_deep(movedRanges.arena(), ResolverMoveRef(moveRange, dest)); - TraceEvent("MovingResolutionRange") - .detail("Src", src) - .detail("Dest", dest) - .detail("Amount", amount) - .detail("StartRange", range.first) - .detail("MoveRange", moveRange) - .detail("Used", split.used) - .detail("KeyResolverRanges", key_resolver.size()); - amount -= split.used; - if (moveRange != range.first || amount <= 0) - break; - } - for (auto& it : movedRanges) - key_resolver.insert(it.range, it.dest); - // for(auto& it : key_resolver.ranges()) - // TraceEvent("KeyResolver").detail("Range", it.range()).detail("Value", it.value()); - - self->resolverChangesVersion = self->version + 1; - for (auto& p : self->commitProxies) - self->resolverNeedingChanges.insert(p.id()); - self->resolverChanges.set(movedRanges); - } catch (Error& e) { - if (e.code() != error_code_operation_failed) - throw; - } - } - } -} - ACTOR Future changeCoordinators(Reference self) { loop { ChangeCoordinatorsRequest req = waitNext(self->clusterController.changeCoordinators.getFuture()); @@ -1127,8 +988,8 @@ ACTOR Future>> recruitEverything( newTLogServers(self, recruits, oldLogSystem, &confChanges)); // Update recovery related information to the newly elected sequencer (master) process. - wait(brokenPromiseToNever(self->masterInterface.updateRecoveryData.getReply( - UpdateRecoveryDataRequest(self->recoveryTransactionVersion, self->lastEpochEnd, self->commitProxies)))); + wait(brokenPromiseToNever(self->masterInterface.updateRecoveryData.getReply(UpdateRecoveryDataRequest( + self->recoveryTransactionVersion, self->lastEpochEnd, self->commitProxies, self->resolvers)))); return confChanges; } @@ -1802,14 +1663,6 @@ ACTOR Future clusterRecoveryCore(Reference self) { .detail("RecoveryDuration", recoveryDuration) .trackLatest(self->clusterRecoveryStateEventHolder->trackingKey); - TraceEvent(getRecoveryEventName(ClusterRecoveryEventType::CLUSTER_RECOVERY_AVAILABLE_EVENT_NAME).c_str(), - self->dbgid) - .detail("AvailableAtVersion", self->version) - .trackLatest(self->clusterRecoveryAvailableEventHolder->trackingKey); - - if (self->resolvers.size() > 1) - self->addActor.send(resolutionBalancing(self)); - self->addActor.send(changeCoordinators(self)); Database cx = openDBOnServer(self->dbInfo, TaskPriority::DefaultEndpoint, LockAware::True); self->addActor.send(configurationMonitor(self, cx)); diff --git a/fdbserver/ClusterRecovery.actor.h b/fdbserver/ClusterRecovery.actor.h index 36dcb1bed9..19911fdd02 100644 --- a/fdbserver/ClusterRecovery.actor.h +++ b/fdbserver/ClusterRecovery.actor.h @@ -185,7 +185,6 @@ struct ClusterRecoveryData : NonCopyable, ReferenceCounted ServerCoordinators coordinators; Reference logSystem; - Version version; // The last version assigned to a proxy by getVersion() double lastVersionTime; LogSystemDiskQueueAdapter* txnStateLogAdapter; IKeyValueStore* txnStateStore; @@ -225,10 +224,6 @@ struct ClusterRecoveryData : NonCopyable, ReferenceCounted RecoveryState recoveryState; - AsyncVar>> resolverChanges; - Version resolverChangesVersion; - std::set resolverNeedingChanges; - PromiseStream> addActor; Reference> recruitmentStalled; bool forceRecovery; @@ -266,12 +261,11 @@ struct ClusterRecoveryData : NonCopyable, ReferenceCounted : controllerData(controllerData), dbgid(masterInterface.id()), lastEpochEnd(invalidVersion), recoveryTransactionVersion(invalidVersion), lastCommitTime(0), liveCommittedVersion(invalidVersion), databaseLocked(false), minKnownCommittedVersion(invalidVersion), hasConfiguration(false), - coordinators(coordinators), version(invalidVersion), lastVersionTime(0), txnStateStore(nullptr), - memoryLimit(2e9), dbId(dbId), masterInterface(masterInterface), masterLifetime(masterLifetimeToken), - clusterController(clusterController), cstate(coordinators, addActor, dbgid), dbInfo(dbInfo), - registrationCount(0), addActor(addActor), recruitmentStalled(makeReference>(false)), - forceRecovery(forceRecovery), neverCreated(false), safeLocality(tagLocalityInvalid), - primaryLocality(tagLocalityInvalid), cc("Master", dbgid.toString()), + coordinators(coordinators), lastVersionTime(0), txnStateStore(nullptr), memoryLimit(2e9), dbId(dbId), + masterInterface(masterInterface), masterLifetime(masterLifetimeToken), clusterController(clusterController), + cstate(coordinators, addActor, dbgid), dbInfo(dbInfo), registrationCount(0), addActor(addActor), + recruitmentStalled(makeReference>(false)), forceRecovery(forceRecovery), neverCreated(false), + safeLocality(tagLocalityInvalid), primaryLocality(tagLocalityInvalid), cc("Master", dbgid.toString()), changeCoordinatorsRequests("ChangeCoordinatorsRequests", cc), getCommitVersionRequests("GetCommitVersionRequests", cc), backupWorkerDoneRequests("BackupWorkerDoneRequests", cc), diff --git a/fdbserver/MasterInterface.h b/fdbserver/MasterInterface.h index 90d49e9492..1b7918a583 100644 --- a/fdbserver/MasterInterface.h +++ b/fdbserver/MasterInterface.h @@ -23,14 +23,15 @@ #pragma once #include "fdbclient/CommitProxyInterface.h" -#include "fdbclient/FDBTypes.h" -#include "fdbclient/StorageServerInterface.h" #include "fdbclient/CommitTransaction.h" #include "fdbclient/DatabaseConfiguration.h" -#include "fdbserver/TLogInterface.h" +#include "fdbclient/FDBTypes.h" #include "fdbclient/Notified.h" +#include "fdbclient/StorageServerInterface.h" +#include "fdbserver/ResolverInterface.h" +#include "fdbserver/TLogInterface.h" -typedef uint64_t DBRecoveryCount; +using DBRecoveryCount = uint64_t; struct MasterInterface { constexpr static FileIdentifier file_identifier = 5979145; @@ -155,18 +156,20 @@ struct UpdateRecoveryDataRequest { Version recoveryTransactionVersion; Version lastEpochEnd; std::vector commitProxies; + std::vector resolvers; ReplyPromise reply; - UpdateRecoveryDataRequest() {} + UpdateRecoveryDataRequest() = default; UpdateRecoveryDataRequest(Version recoveryTransactionVersion, Version lastEpochEnd, - std::vector commitProxies) + const std::vector& commitProxies, + const std::vector& resolvers) : recoveryTransactionVersion(recoveryTransactionVersion), lastEpochEnd(lastEpochEnd), - commitProxies(commitProxies) {} + commitProxies(commitProxies), resolvers(resolvers) {} template void serialize(Ar& ar) { - serializer(ar, recoveryTransactionVersion, lastEpochEnd, commitProxies, reply); + serializer(ar, recoveryTransactionVersion, lastEpochEnd, commitProxies, resolvers, reply); } }; diff --git a/fdbserver/masterserver.actor.cpp b/fdbserver/masterserver.actor.cpp index d9e860eb52..5a43adf927 100644 --- a/fdbserver/masterserver.actor.cpp +++ b/fdbserver/masterserver.actor.cpp @@ -67,6 +67,9 @@ struct MasterData : NonCopyable, ReferenceCounted { std::vector commitProxies; std::map lastCommitProxyVersionReplies; + std::vector resolvers; + + PromiseStream> addActor; MasterInterface myInterface; @@ -94,7 +97,7 @@ struct MasterData : NonCopyable, ReferenceCounted { : dbgid(myInterface.id()), lastEpochEnd(invalidVersion), recoveryTransactionVersion(invalidVersion), liveCommittedVersion(invalidVersion), databaseLocked(false), minKnownCommittedVersion(invalidVersion), coordinators(coordinators), version(invalidVersion), lastVersionTime(0), txnStateStore(nullptr), - myInterface(myInterface), forceRecovery(forceRecovery), cc("Master", dbgid.toString()), + addActor(addActor), myInterface(myInterface), forceRecovery(forceRecovery), cc("Master", dbgid.toString()), getCommitVersionRequests("GetCommitVersionRequests", cc), getLiveCommittedVersionRequests("GetLiveCommittedVersionRequests", cc), reportLiveCommittedVersionRequests("ReportLiveCommittedVersionRequests", cc) { @@ -110,6 +113,145 @@ struct MasterData : NonCopyable, ReferenceCounted { } }; +static std::pair findRange(CoalescedKeyRangeMap& key_resolver, + Standalone>& movedRanges, + int src, + int dest) { + auto ranges = key_resolver.ranges(); + auto prev = ranges.begin(); + auto it = ranges.begin(); + ++it; + if (it == ranges.end()) { + if (ranges.begin().value() != src || + std::find(movedRanges.begin(), movedRanges.end(), ResolverMoveRef(ranges.begin()->range(), dest)) != + movedRanges.end()) + throw operation_failed(); + return std::make_pair(ranges.begin().range(), true); + } + + std::set borders; + // If possible expand an existing boundary between the two resolvers + for (; it != ranges.end(); ++it) { + if (it->value() == src && prev->value() == dest && + std::find(movedRanges.begin(), movedRanges.end(), ResolverMoveRef(it->range(), dest)) == + movedRanges.end()) { + return std::make_pair(it->range(), true); + } + if (it->value() == dest && prev->value() == src && + std::find(movedRanges.begin(), movedRanges.end(), ResolverMoveRef(prev->range(), dest)) == + movedRanges.end()) { + return std::make_pair(prev->range(), false); + } + if (it->value() == dest) + borders.insert(prev->value()); + if (prev->value() == dest) + borders.insert(it->value()); + ++prev; + } + + prev = ranges.begin(); + it = ranges.begin(); + ++it; + // If possible create a new boundry which doesn't exist yet + for (; it != ranges.end(); ++it) { + if (it->value() == src && !borders.count(prev->value()) && + std::find(movedRanges.begin(), movedRanges.end(), ResolverMoveRef(it->range(), dest)) == + movedRanges.end()) { + return std::make_pair(it->range(), true); + } + if (prev->value() == src && !borders.count(it->value()) && + std::find(movedRanges.begin(), movedRanges.end(), ResolverMoveRef(prev->range(), dest)) == + movedRanges.end()) { + return std::make_pair(prev->range(), false); + } + ++prev; + } + + it = ranges.begin(); + for (; it != ranges.end(); ++it) { + if (it->value() == src && + std::find(movedRanges.begin(), movedRanges.end(), ResolverMoveRef(it->range(), dest)) == + movedRanges.end()) { + return std::make_pair(it->range(), true); + } + } + throw operation_failed(); // we are already attempting to move all of the data one resolver is assigned, so do not + // move anything +} + +// Balance key ranges among resolvers so that their load are evenly distributed. +ACTOR Future resolutionBalancing(Reference self) { + state CoalescedKeyRangeMap key_resolver; + key_resolver.insert(allKeys, 0); + loop { + wait(delay(SERVER_KNOBS->MIN_BALANCE_TIME, TaskPriority::ResolutionMetrics)); + while (self->resolverChanges.get().size()) + wait(self->resolverChanges.onChange()); + state std::vector> futures; + for (auto& p : self->resolvers) + futures.push_back( + brokenPromiseToNever(p.metrics.getReply(ResolutionMetricsRequest(), TaskPriority::ResolutionMetrics))); + wait(waitForAll(futures)); + state IndexedSet, NoMetric> metrics; + + int64_t total = 0; + for (int i = 0; i < futures.size(); i++) { + total += futures[i].get().value; + metrics.insert(std::make_pair(futures[i].get().value, i), NoMetric()); + //TraceEvent("ResolverMetric").detail("I", i).detail("Metric", futures[i].get()); + } + if (metrics.lastItem()->first - metrics.begin()->first > SERVER_KNOBS->MIN_BALANCE_DIFFERENCE) { + try { + state int src = metrics.lastItem()->second; + state int dest = metrics.begin()->second; + state int64_t amount = std::min(metrics.lastItem()->first - total / self->resolvers.size(), + total / self->resolvers.size() - metrics.begin()->first) / + 2; + state Standalone> movedRanges; + + loop { + state std::pair range = findRange(key_resolver, movedRanges, src, dest); + + ResolutionSplitRequest req; + req.front = range.second; + req.offset = amount; + req.range = range.first; + + ResolutionSplitReply split = + wait(brokenPromiseToNever(self->resolvers[metrics.lastItem()->second].split.getReply( + req, TaskPriority::ResolutionMetrics))); + KeyRangeRef moveRange = range.second ? KeyRangeRef(range.first.begin, split.key) + : KeyRangeRef(split.key, range.first.end); + movedRanges.push_back_deep(movedRanges.arena(), ResolverMoveRef(moveRange, dest)); + TraceEvent("MovingResolutionRange") + .detail("Src", src) + .detail("Dest", dest) + .detail("Amount", amount) + .detail("StartRange", range.first) + .detail("MoveRange", moveRange) + .detail("Used", split.used) + .detail("KeyResolverRanges", key_resolver.size()); + amount -= split.used; + if (moveRange != range.first || amount <= 0) + break; + } + for (auto& it : movedRanges) + key_resolver.insert(it.range, it.dest); + // for(auto& it : key_resolver.ranges()) + // TraceEvent("KeyResolver").detail("Range", it.range()).detail("Value", it.value()); + + self->resolverChangesVersion = self->version + 1; + for (auto& p : self->commitProxies) + self->resolverNeedingChanges.insert(p.id()); + self->resolverChanges.set(movedRanges); + } catch (Error& e) { + if (e.code() != error_code_operation_failed) + throw; + } + } + } +} + ACTOR Future getVersion(Reference self, GetCommitVersionRequest req) { state Span span("M:getVersion"_loc, { req.spanContext }); state std::map::iterator proxyItr = @@ -244,31 +386,33 @@ ACTOR Future serveLiveCommittedVersion(Reference self) { ACTOR Future updateRecoveryData(Reference self) { loop { - choose { - when(UpdateRecoveryDataRequest req = waitNext(self->myInterface.updateRecoveryData.getFuture())) { - TraceEvent("UpdateRecoveryData", self->dbgid) - .detail("RecoveryTxnVersion", req.recoveryTransactionVersion) - .detail("LastEpochEnd", req.lastEpochEnd) - .detail("NumCommitProxies", req.commitProxies.size()); + UpdateRecoveryDataRequest req = waitNext(self->myInterface.updateRecoveryData.getFuture()); + TraceEvent("UpdateRecoveryData", self->dbgid) + .detail("RecoveryTxnVersion", req.recoveryTransactionVersion) + .detail("LastEpochEnd", req.lastEpochEnd) + .detail("NumCommitProxies", req.commitProxies.size()); - if (self->recoveryTransactionVersion == invalidVersion || - req.recoveryTransactionVersion > self->recoveryTransactionVersion) { - self->recoveryTransactionVersion = req.recoveryTransactionVersion; - } - if (self->lastEpochEnd == invalidVersion || req.lastEpochEnd > self->lastEpochEnd) { - self->lastEpochEnd = req.lastEpochEnd; - } - if (req.commitProxies.size() > 0) { - self->commitProxies = req.commitProxies; - self->lastCommitProxyVersionReplies.clear(); + if (self->recoveryTransactionVersion == invalidVersion || + req.recoveryTransactionVersion > self->recoveryTransactionVersion) { + self->recoveryTransactionVersion = req.recoveryTransactionVersion; + } + if (self->lastEpochEnd == invalidVersion || req.lastEpochEnd > self->lastEpochEnd) { + self->lastEpochEnd = req.lastEpochEnd; + } + if (req.commitProxies.size() > 0) { + self->commitProxies = req.commitProxies; + self->lastCommitProxyVersionReplies.clear(); - for (auto& p : self->commitProxies) { - self->lastCommitProxyVersionReplies[p.id()] = CommitProxyVersionReplies(); - } - } - req.reply.send(Void()); + for (auto& p : self->commitProxies) { + self->lastCommitProxyVersionReplies[p.id()] = CommitProxyVersionReplies(); } } + + self->resolvers = req.resolvers; + if (req.resolvers.size() > 1) + self->addActor.send(resolutionBalancing(self)); + + req.reply.send(Void()); } } diff --git a/fdbserver/workloads/KillRegion.actor.cpp b/fdbserver/workloads/KillRegion.actor.cpp index d15255bd7b..08b3f62946 100644 --- a/fdbserver/workloads/KillRegion.actor.cpp +++ b/fdbserver/workloads/KillRegion.actor.cpp @@ -107,7 +107,10 @@ struct KillRegionWorkload : TestWorkload { DatabaseConfiguration conf = wait(getDatabaseConfiguration(cx)); - TraceEvent("ForceRecovery_GotConfig").detail("Conf", conf.toString()); + TraceEvent("ForceRecovery_GotConfig") + .setMaxEventLength(11000) + .setMaxFieldLength(10000) + .detail("Conf", conf.toString()); if (conf.usableRegions > 1) { loop { From 213e37191c7baaae9c45fbd5f8298b450340f0c3 Mon Sep 17 00:00:00 2001 From: Jingyu Zhou Date: Fri, 18 Mar 2022 15:27:31 -0700 Subject: [PATCH 391/413] Add code coverage macro for resolution balancing --- fdbserver/masterserver.actor.cpp | 1 + 1 file changed, 1 insertion(+) diff --git a/fdbserver/masterserver.actor.cpp b/fdbserver/masterserver.actor.cpp index 5a43adf927..f9898073c2 100644 --- a/fdbserver/masterserver.actor.cpp +++ b/fdbserver/masterserver.actor.cpp @@ -306,6 +306,7 @@ ACTOR Future getVersion(Reference self, GetCommitVersionReques rep.resolverChangesVersion = self->resolverChangesVersion; self->resolverNeedingChanges.erase(req.requestingProxy); + TEST(!rep.resolverChanges.empty()); // resolution balancing moves keyranges if (self->resolverNeedingChanges.empty()) self->resolverChanges.set(Standalone>()); } From 7736ea87b05047f6d64b8eb6a9be97e0700482cd Mon Sep 17 00:00:00 2001 From: Jingyu Zhou Date: Fri, 18 Mar 2022 15:57:34 -0700 Subject: [PATCH 392/413] Fix code format --- fdbserver/masterserver.actor.cpp | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/fdbserver/masterserver.actor.cpp b/fdbserver/masterserver.actor.cpp index f9898073c2..33dacd45ad 100644 --- a/fdbserver/masterserver.actor.cpp +++ b/fdbserver/masterserver.actor.cpp @@ -389,12 +389,12 @@ ACTOR Future updateRecoveryData(Reference self) { loop { UpdateRecoveryDataRequest req = waitNext(self->myInterface.updateRecoveryData.getFuture()); TraceEvent("UpdateRecoveryData", self->dbgid) - .detail("RecoveryTxnVersion", req.recoveryTransactionVersion) - .detail("LastEpochEnd", req.lastEpochEnd) - .detail("NumCommitProxies", req.commitProxies.size()); + .detail("RecoveryTxnVersion", req.recoveryTransactionVersion) + .detail("LastEpochEnd", req.lastEpochEnd) + .detail("NumCommitProxies", req.commitProxies.size()); if (self->recoveryTransactionVersion == invalidVersion || - req.recoveryTransactionVersion > self->recoveryTransactionVersion) { + req.recoveryTransactionVersion > self->recoveryTransactionVersion) { self->recoveryTransactionVersion = req.recoveryTransactionVersion; } if (self->lastEpochEnd == invalidVersion || req.lastEpochEnd > self->lastEpochEnd) { From 6e8d16538dcb4b0dd0e9c5f3ec71f64306b4bde4 Mon Sep 17 00:00:00 2001 From: Jingyu Zhou Date: Mon, 21 Mar 2022 13:48:09 -0700 Subject: [PATCH 393/413] Remove an unused variable from MasterData --- fdbserver/masterserver.actor.cpp | 10 +++------- 1 file changed, 3 insertions(+), 7 deletions(-) diff --git a/fdbserver/masterserver.actor.cpp b/fdbserver/masterserver.actor.cpp index 33dacd45ad..f79680c888 100644 --- a/fdbserver/masterserver.actor.cpp +++ b/fdbserver/masterserver.actor.cpp @@ -63,7 +63,6 @@ struct MasterData : NonCopyable, ReferenceCounted { Version version; // The last version assigned to a proxy by getVersion() double lastVersionTime; - IKeyValueStore* txnStateStore; std::vector commitProxies; std::map lastCommitProxyVersionReplies; @@ -96,8 +95,8 @@ struct MasterData : NonCopyable, ReferenceCounted { : dbgid(myInterface.id()), lastEpochEnd(invalidVersion), recoveryTransactionVersion(invalidVersion), liveCommittedVersion(invalidVersion), databaseLocked(false), minKnownCommittedVersion(invalidVersion), - coordinators(coordinators), version(invalidVersion), lastVersionTime(0), txnStateStore(nullptr), - addActor(addActor), myInterface(myInterface), forceRecovery(forceRecovery), cc("Master", dbgid.toString()), + coordinators(coordinators), version(invalidVersion), lastVersionTime(0), addActor(addActor), + myInterface(myInterface), forceRecovery(forceRecovery), cc("Master", dbgid.toString()), getCommitVersionRequests("GetCommitVersionRequests", cc), getLiveCommittedVersionRequests("GetLiveCommittedVersionRequests", cc), reportLiveCommittedVersionRequests("ReportLiveCommittedVersionRequests", cc) { @@ -107,10 +106,7 @@ struct MasterData : NonCopyable, ReferenceCounted { forceRecovery = false; } } - ~MasterData() { - if (txnStateStore) - txnStateStore->close(); - } + ~MasterData() = default; }; static std::pair findRange(CoalescedKeyRangeMap& key_resolver, From 437e7d27c643407ba57365eca38f050c8e557da1 Mon Sep 17 00:00:00 2001 From: Jingyu Zhou Date: Mon, 21 Mar 2022 16:38:23 -0700 Subject: [PATCH 394/413] Use trigger to start resolutionBalancing Trigger when there are more than one resolvers. This also avoids the problem of receiving multiple UpdateRecoveryDataRequests. --- fdbserver/masterserver.actor.cpp | 35 +++++++++----------------------- 1 file changed, 10 insertions(+), 25 deletions(-) diff --git a/fdbserver/masterserver.actor.cpp b/fdbserver/masterserver.actor.cpp index f79680c888..4e7c0b1653 100644 --- a/fdbserver/masterserver.actor.cpp +++ b/fdbserver/masterserver.actor.cpp @@ -20,31 +20,15 @@ #include -#include "fdbclient/NativeAPI.actor.h" -#include "fdbclient/Notified.h" -#include "fdbclient/SystemData.h" -#include "fdbrpc/FailureMonitor.h" -#include "fdbrpc/PerfMetric.h" #include "fdbrpc/sim_validation.h" -#include "fdbrpc/simulator.h" -#include "fdbserver/ApplyMetadataMutation.h" -#include "fdbserver/BackupProgress.actor.h" -#include "fdbserver/ConflictSet.h" #include "fdbserver/CoordinatedState.h" #include "fdbserver/CoordinationInterface.h" // copy constructors for ServerCoordinators class -#include "fdbserver/DBCoreState.h" -#include "fdbserver/DataDistribution.actor.h" -#include "fdbserver/IKeyValueStore.h" #include "fdbserver/Knobs.h" -#include "fdbserver/LogSystemDiskQueueAdapter.h" #include "fdbserver/MasterInterface.h" -#include "fdbserver/ProxyCommitData.actor.h" -#include "fdbserver/RecoveryState.h" #include "fdbserver/ServerDBInfo.h" -#include "fdbserver/WaitFailure.h" -#include "fdbserver/WorkerInterface.actor.h" #include "flow/ActorCollection.h" #include "flow/Trace.h" +#include "flow/genericactors.actor.h" #include "flow/actorcompiler.h" // This must be the last #include. @@ -68,13 +52,12 @@ struct MasterData : NonCopyable, ReferenceCounted { std::map lastCommitProxyVersionReplies; std::vector resolvers; - PromiseStream> addActor; - MasterInterface myInterface; AsyncVar>> resolverChanges; Version resolverChangesVersion; std::set resolverNeedingChanges; + AsyncTrigger triggerResolution; bool forceRecovery; @@ -90,13 +73,12 @@ struct MasterData : NonCopyable, ReferenceCounted { ServerCoordinators const& coordinators, ClusterControllerFullInterface const& clusterController, Standalone const& dbId, - PromiseStream> const& addActor, bool forceRecovery) : dbgid(myInterface.id()), lastEpochEnd(invalidVersion), recoveryTransactionVersion(invalidVersion), liveCommittedVersion(invalidVersion), databaseLocked(false), minKnownCommittedVersion(invalidVersion), - coordinators(coordinators), version(invalidVersion), lastVersionTime(0), addActor(addActor), - myInterface(myInterface), forceRecovery(forceRecovery), cc("Master", dbgid.toString()), + coordinators(coordinators), version(invalidVersion), lastVersionTime(0), myInterface(myInterface), + forceRecovery(forceRecovery), cc("Master", dbgid.toString()), getCommitVersionRequests("GetCommitVersionRequests", cc), getLiveCommittedVersionRequests("GetLiveCommittedVersionRequests", cc), reportLiveCommittedVersionRequests("ReportLiveCommittedVersionRequests", cc) { @@ -177,6 +159,8 @@ static std::pair findRange(CoalescedKeyRangeMap& key_res // Balance key ranges among resolvers so that their load are evenly distributed. ACTOR Future resolutionBalancing(Reference self) { + wait(self->triggerResolution.onTrigger()); + state CoalescedKeyRangeMap key_resolver; key_resolver.insert(allKeys, 0); loop { @@ -407,7 +391,7 @@ ACTOR Future updateRecoveryData(Reference self) { self->resolvers = req.resolvers; if (req.resolvers.size() > 1) - self->addActor.send(resolutionBalancing(self)); + self->triggerResolution.trigger(); req.reply.send(Void()); } @@ -454,14 +438,15 @@ ACTOR Future masterServer(MasterInterface mi, state Future onDBChange = Void(); state PromiseStream> addActor; - state Reference self(new MasterData( - db, mi, coordinators, db->get().clusterInterface, LiteralStringRef(""), addActor, forceRecovery)); + state Reference self( + new MasterData(db, mi, coordinators, db->get().clusterInterface, LiteralStringRef(""), forceRecovery)); state Future collection = actorCollection(addActor.getFuture()); addActor.send(traceRole(Role::MASTER, mi.id())); addActor.send(provideVersions(self)); addActor.send(serveLiveCommittedVersion(self)); addActor.send(updateRecoveryData(self)); + addActor.send(resolutionBalancing(self)); TEST(!lifetime.isStillValid(db->get().masterLifetime, mi.id() == db->get().master.id())); // Master born doomed TraceEvent("MasterLifetime", self->dbgid).detail("LifetimeToken", lifetime.toString()); From 0c88be03931656d1c9e25c35413891cab0bea876 Mon Sep 17 00:00:00 2001 From: Jingyu Zhou Date: Mon, 21 Mar 2022 21:35:48 -0700 Subject: [PATCH 395/413] Refactor resolution balancing into separate files --- fdbserver/CMakeLists.txt | 2 + fdbserver/ResolutionBalancer.actor.cpp | 187 +++++++++++++++++++++++++ fdbserver/ResolutionBalancer.actor.h | 64 +++++++++ fdbserver/masterserver.actor.cpp | 186 ++---------------------- 4 files changed, 266 insertions(+), 173 deletions(-) create mode 100644 fdbserver/ResolutionBalancer.actor.cpp create mode 100644 fdbserver/ResolutionBalancer.actor.h diff --git a/fdbserver/CMakeLists.txt b/fdbserver/CMakeLists.txt index d16d65b51d..98d9d3eeae 100644 --- a/fdbserver/CMakeLists.txt +++ b/fdbserver/CMakeLists.txt @@ -96,6 +96,8 @@ set(FDBSERVER_SRCS Ratekeeper.h RatekeeperInterface.h RecoveryState.h + ResolutionBalancer.actor.cpp + ResolutionBalancer.actor.h Resolver.actor.cpp ResolverInterface.h RestoreApplier.actor.cpp diff --git a/fdbserver/ResolutionBalancer.actor.cpp b/fdbserver/ResolutionBalancer.actor.cpp new file mode 100644 index 0000000000..6e569d7058 --- /dev/null +++ b/fdbserver/ResolutionBalancer.actor.cpp @@ -0,0 +1,187 @@ +/* + * ResolutionBalancer.actor.cpp + * + * This source file is part of the FoundationDB open source project + * + * Copyright 2013-2022 Apple Inc. and the FoundationDB project authors + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +#include "fdbserver/ResolutionBalancer.actor.h" + +#include "fdbclient/KeyRangeMap.h" +#include "fdbserver/MasterInterface.h" +#include "fdbserver/Knobs.h" +#include "flow/flow.h" + +#include "flow/actorcompiler.h" // This must be the last #include. + +void ResolutionBalancer::setResolvers(const std::vector& v) { + resolvers = v; + if (resolvers.size() > 1) + triggerResolution.trigger(); +} + +void ResolutionBalancer::setChangesInReply(UID requestingProxy, GetCommitVersionReply& rep) { + if (resolverNeedingChanges.count(requestingProxy)) { + rep.resolverChanges = resolverChanges.get(); + rep.resolverChangesVersion = resolverChangesVersion; + resolverNeedingChanges.erase(requestingProxy); + + TEST(!rep.resolverChanges.empty()); // resolution balancing moves keyranges + if (resolverNeedingChanges.empty()) + resolverChanges.set(Standalone>()); + } +} + +static std::pair findRange(CoalescedKeyRangeMap& key_resolver, + Standalone>& movedRanges, + int src, + int dest) { + auto ranges = key_resolver.ranges(); + auto prev = ranges.begin(); + auto it = ranges.begin(); + ++it; + if (it == ranges.end()) { + if (ranges.begin().value() != src || + std::find(movedRanges.begin(), movedRanges.end(), ResolverMoveRef(ranges.begin()->range(), dest)) != + movedRanges.end()) + throw operation_failed(); + return std::make_pair(ranges.begin().range(), true); + } + + std::set borders; + // If possible expand an existing boundary between the two resolvers + for (; it != ranges.end(); ++it) { + if (it->value() == src && prev->value() == dest && + std::find(movedRanges.begin(), movedRanges.end(), ResolverMoveRef(it->range(), dest)) == + movedRanges.end()) { + return std::make_pair(it->range(), true); + } + if (it->value() == dest && prev->value() == src && + std::find(movedRanges.begin(), movedRanges.end(), ResolverMoveRef(prev->range(), dest)) == + movedRanges.end()) { + return std::make_pair(prev->range(), false); + } + if (it->value() == dest) + borders.insert(prev->value()); + if (prev->value() == dest) + borders.insert(it->value()); + ++prev; + } + + prev = ranges.begin(); + it = ranges.begin(); + ++it; + // If possible create a new boundry which doesn't exist yet + for (; it != ranges.end(); ++it) { + if (it->value() == src && !borders.count(prev->value()) && + std::find(movedRanges.begin(), movedRanges.end(), ResolverMoveRef(it->range(), dest)) == + movedRanges.end()) { + return std::make_pair(it->range(), true); + } + if (prev->value() == src && !borders.count(it->value()) && + std::find(movedRanges.begin(), movedRanges.end(), ResolverMoveRef(prev->range(), dest)) == + movedRanges.end()) { + return std::make_pair(prev->range(), false); + } + ++prev; + } + + it = ranges.begin(); + for (; it != ranges.end(); ++it) { + if (it->value() == src && + std::find(movedRanges.begin(), movedRanges.end(), ResolverMoveRef(it->range(), dest)) == + movedRanges.end()) { + return std::make_pair(it->range(), true); + } + } + throw operation_failed(); // we are already attempting to move all of the data one resolver is assigned, so do not + // move anything +} + +// Balance key ranges among resolvers so that their load are evenly distributed. +ACTOR Future ResolutionBalancer::resolutionBalancing_impl(ResolutionBalancer* self) { + wait(self->triggerResolution.onTrigger()); + + state CoalescedKeyRangeMap key_resolver; + key_resolver.insert(allKeys, 0); + loop { + wait(delay(SERVER_KNOBS->MIN_BALANCE_TIME, TaskPriority::ResolutionMetrics)); + while (self->resolverChanges.get().size()) + wait(self->resolverChanges.onChange()); + state std::vector> futures; + for (auto& p : self->resolvers) + futures.push_back( + brokenPromiseToNever(p.metrics.getReply(ResolutionMetricsRequest(), TaskPriority::ResolutionMetrics))); + wait(waitForAll(futures)); + state IndexedSet, NoMetric> metrics; + + int64_t total = 0; + for (int i = 0; i < futures.size(); i++) { + total += futures[i].get().value; + metrics.insert(std::make_pair(futures[i].get().value, i), NoMetric()); + //TraceEvent("ResolverMetric").detail("I", i).detail("Metric", futures[i].get()); + } + if (metrics.lastItem()->first - metrics.begin()->first > SERVER_KNOBS->MIN_BALANCE_DIFFERENCE) { + try { + state int src = metrics.lastItem()->second; + state int dest = metrics.begin()->second; + state int64_t amount = std::min(metrics.lastItem()->first - total / self->resolvers.size(), + total / self->resolvers.size() - metrics.begin()->first) / + 2; + state Standalone> movedRanges; + + loop { + state std::pair range = findRange(key_resolver, movedRanges, src, dest); + + ResolutionSplitRequest req; + req.front = range.second; + req.offset = amount; + req.range = range.first; + + ResolutionSplitReply split = + wait(brokenPromiseToNever(self->resolvers[metrics.lastItem()->second].split.getReply( + req, TaskPriority::ResolutionMetrics))); + KeyRangeRef moveRange = range.second ? KeyRangeRef(range.first.begin, split.key) + : KeyRangeRef(split.key, range.first.end); + movedRanges.push_back_deep(movedRanges.arena(), ResolverMoveRef(moveRange, dest)); + TraceEvent("MovingResolutionRange") + .detail("Src", src) + .detail("Dest", dest) + .detail("Amount", amount) + .detail("StartRange", range.first) + .detail("MoveRange", moveRange) + .detail("Used", split.used) + .detail("KeyResolverRanges", key_resolver.size()); + amount -= split.used; + if (moveRange != range.first || amount <= 0) + break; + } + for (auto& it : movedRanges) + key_resolver.insert(it.range, it.dest); + // for(auto& it : key_resolver.ranges()) + // TraceEvent("KeyResolver").detail("Range", it.range()).detail("Value", it.value()); + + self->resolverChangesVersion = *self->pVersion + 1; + for (auto& p : self->commitProxies) + self->resolverNeedingChanges.insert(p.id()); + self->resolverChanges.set(movedRanges); + } catch (Error& e) { + if (e.code() != error_code_operation_failed) + throw; + } + } + } +} diff --git a/fdbserver/ResolutionBalancer.actor.h b/fdbserver/ResolutionBalancer.actor.h new file mode 100644 index 0000000000..263195f0e5 --- /dev/null +++ b/fdbserver/ResolutionBalancer.actor.h @@ -0,0 +1,64 @@ +/* + * ResolutionBalancer.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. + */ + +#include "fdbclient/CommitProxyInterface.h" +#include "fdbserver/ResolverInterface.h" +#if defined(NO_INTELLISENSE) && !defined(FDBSERVER_RESOLUTION_BALANCER_G_H) +#define FDBSERVER_RESOLUTION_BALANCER_G_H +#include "fdbserver/ResolutionBalancer.actor.g.h" +#elif !defined(FDBSERVER_RESOLUTION_BALANCER_H) +#define FDBSERVER_RESOLUTION_BALANCER_H + +#include + +#include "fdbclient/FDBTypes.h" +#include "fdbserver/MasterInterface.h" +#include "flow/Arena.h" +#include "flow/IRandom.h" +#include "flow/genericactors.actor.h" + +struct ResolutionBalancer { + AsyncVar>> resolverChanges; + Version resolverChangesVersion = invalidVersion; + std::set resolverNeedingChanges; + + Version* pVersion; // points to MasterData::version + + std::vector commitProxies; + std::vector resolvers; + AsyncTrigger triggerResolution; + + ResolutionBalancer(Version* version) : pVersion(version) {} + + Future resolutionBalancing() { return resolutionBalancing_impl(this); } + + ACTOR static Future resolutionBalancing_impl(ResolutionBalancer* self); + + // Sets resolver interfaces. Trigger resolutionBalancing() actor if more + // than one resolvers are present. + void setResolvers(const std::vector& resolvers); + + void setCommitProxies(const std::vector& proxies) { commitProxies = proxies; } + + void setChangesInReply(UID requestingProxy, GetCommitVersionReply& rep); +}; + +#include "flow/unactorcompiler.h" +#endif diff --git a/fdbserver/masterserver.actor.cpp b/fdbserver/masterserver.actor.cpp index 4e7c0b1653..7a2ba7b2de 100644 --- a/fdbserver/masterserver.actor.cpp +++ b/fdbserver/masterserver.actor.cpp @@ -25,10 +25,10 @@ #include "fdbserver/CoordinationInterface.h" // copy constructors for ServerCoordinators class #include "fdbserver/Knobs.h" #include "fdbserver/MasterInterface.h" +#include "fdbserver/ResolutionBalancer.actor.h" #include "fdbserver/ServerDBInfo.h" #include "flow/ActorCollection.h" #include "flow/Trace.h" -#include "flow/genericactors.actor.h" #include "flow/actorcompiler.h" // This must be the last #include. @@ -48,16 +48,11 @@ struct MasterData : NonCopyable, ReferenceCounted { Version version; // The last version assigned to a proxy by getVersion() double lastVersionTime; - std::vector commitProxies; std::map lastCommitProxyVersionReplies; - std::vector resolvers; MasterInterface myInterface; - AsyncVar>> resolverChanges; - Version resolverChangesVersion; - std::set resolverNeedingChanges; - AsyncTrigger triggerResolution; + ResolutionBalancer resolutionBalancer; bool forceRecovery; @@ -67,6 +62,7 @@ struct MasterData : NonCopyable, ReferenceCounted { Counter reportLiveCommittedVersionRequests; Future logger; + Future balancer; MasterData(Reference const> const& dbInfo, MasterInterface const& myInterface, @@ -78,7 +74,7 @@ struct MasterData : NonCopyable, ReferenceCounted { : dbgid(myInterface.id()), lastEpochEnd(invalidVersion), recoveryTransactionVersion(invalidVersion), liveCommittedVersion(invalidVersion), databaseLocked(false), minKnownCommittedVersion(invalidVersion), coordinators(coordinators), version(invalidVersion), lastVersionTime(0), myInterface(myInterface), - forceRecovery(forceRecovery), cc("Master", dbgid.toString()), + resolutionBalancer(&version), forceRecovery(forceRecovery), cc("Master", dbgid.toString()), getCommitVersionRequests("GetCommitVersionRequests", cc), getLiveCommittedVersionRequests("GetLiveCommittedVersionRequests", cc), reportLiveCommittedVersionRequests("ReportLiveCommittedVersionRequests", cc) { @@ -87,151 +83,11 @@ struct MasterData : NonCopyable, ReferenceCounted { TraceEvent(SevError, "ForcedRecoveryRequiresDcID").log(); forceRecovery = false; } + balancer = resolutionBalancer.resolutionBalancing(); } ~MasterData() = default; }; -static std::pair findRange(CoalescedKeyRangeMap& key_resolver, - Standalone>& movedRanges, - int src, - int dest) { - auto ranges = key_resolver.ranges(); - auto prev = ranges.begin(); - auto it = ranges.begin(); - ++it; - if (it == ranges.end()) { - if (ranges.begin().value() != src || - std::find(movedRanges.begin(), movedRanges.end(), ResolverMoveRef(ranges.begin()->range(), dest)) != - movedRanges.end()) - throw operation_failed(); - return std::make_pair(ranges.begin().range(), true); - } - - std::set borders; - // If possible expand an existing boundary between the two resolvers - for (; it != ranges.end(); ++it) { - if (it->value() == src && prev->value() == dest && - std::find(movedRanges.begin(), movedRanges.end(), ResolverMoveRef(it->range(), dest)) == - movedRanges.end()) { - return std::make_pair(it->range(), true); - } - if (it->value() == dest && prev->value() == src && - std::find(movedRanges.begin(), movedRanges.end(), ResolverMoveRef(prev->range(), dest)) == - movedRanges.end()) { - return std::make_pair(prev->range(), false); - } - if (it->value() == dest) - borders.insert(prev->value()); - if (prev->value() == dest) - borders.insert(it->value()); - ++prev; - } - - prev = ranges.begin(); - it = ranges.begin(); - ++it; - // If possible create a new boundry which doesn't exist yet - for (; it != ranges.end(); ++it) { - if (it->value() == src && !borders.count(prev->value()) && - std::find(movedRanges.begin(), movedRanges.end(), ResolverMoveRef(it->range(), dest)) == - movedRanges.end()) { - return std::make_pair(it->range(), true); - } - if (prev->value() == src && !borders.count(it->value()) && - std::find(movedRanges.begin(), movedRanges.end(), ResolverMoveRef(prev->range(), dest)) == - movedRanges.end()) { - return std::make_pair(prev->range(), false); - } - ++prev; - } - - it = ranges.begin(); - for (; it != ranges.end(); ++it) { - if (it->value() == src && - std::find(movedRanges.begin(), movedRanges.end(), ResolverMoveRef(it->range(), dest)) == - movedRanges.end()) { - return std::make_pair(it->range(), true); - } - } - throw operation_failed(); // we are already attempting to move all of the data one resolver is assigned, so do not - // move anything -} - -// Balance key ranges among resolvers so that their load are evenly distributed. -ACTOR Future resolutionBalancing(Reference self) { - wait(self->triggerResolution.onTrigger()); - - state CoalescedKeyRangeMap key_resolver; - key_resolver.insert(allKeys, 0); - loop { - wait(delay(SERVER_KNOBS->MIN_BALANCE_TIME, TaskPriority::ResolutionMetrics)); - while (self->resolverChanges.get().size()) - wait(self->resolverChanges.onChange()); - state std::vector> futures; - for (auto& p : self->resolvers) - futures.push_back( - brokenPromiseToNever(p.metrics.getReply(ResolutionMetricsRequest(), TaskPriority::ResolutionMetrics))); - wait(waitForAll(futures)); - state IndexedSet, NoMetric> metrics; - - int64_t total = 0; - for (int i = 0; i < futures.size(); i++) { - total += futures[i].get().value; - metrics.insert(std::make_pair(futures[i].get().value, i), NoMetric()); - //TraceEvent("ResolverMetric").detail("I", i).detail("Metric", futures[i].get()); - } - if (metrics.lastItem()->first - metrics.begin()->first > SERVER_KNOBS->MIN_BALANCE_DIFFERENCE) { - try { - state int src = metrics.lastItem()->second; - state int dest = metrics.begin()->second; - state int64_t amount = std::min(metrics.lastItem()->first - total / self->resolvers.size(), - total / self->resolvers.size() - metrics.begin()->first) / - 2; - state Standalone> movedRanges; - - loop { - state std::pair range = findRange(key_resolver, movedRanges, src, dest); - - ResolutionSplitRequest req; - req.front = range.second; - req.offset = amount; - req.range = range.first; - - ResolutionSplitReply split = - wait(brokenPromiseToNever(self->resolvers[metrics.lastItem()->second].split.getReply( - req, TaskPriority::ResolutionMetrics))); - KeyRangeRef moveRange = range.second ? KeyRangeRef(range.first.begin, split.key) - : KeyRangeRef(split.key, range.first.end); - movedRanges.push_back_deep(movedRanges.arena(), ResolverMoveRef(moveRange, dest)); - TraceEvent("MovingResolutionRange") - .detail("Src", src) - .detail("Dest", dest) - .detail("Amount", amount) - .detail("StartRange", range.first) - .detail("MoveRange", moveRange) - .detail("Used", split.used) - .detail("KeyResolverRanges", key_resolver.size()); - amount -= split.used; - if (moveRange != range.first || amount <= 0) - break; - } - for (auto& it : movedRanges) - key_resolver.insert(it.range, it.dest); - // for(auto& it : key_resolver.ranges()) - // TraceEvent("KeyResolver").detail("Range", it.range()).detail("Value", it.value()); - - self->resolverChangesVersion = self->version + 1; - for (auto& p : self->commitProxies) - self->resolverNeedingChanges.insert(p.id()); - self->resolverChanges.set(movedRanges); - } catch (Error& e) { - if (e.code() != error_code_operation_failed) - throw; - } - } - } -} - ACTOR Future getVersion(Reference self, GetCommitVersionRequest req) { state Span span("M:getVersion"_loc, { req.spanContext }); state std::map::iterator proxyItr = @@ -281,15 +137,7 @@ ACTOR Future getVersion(Reference self, GetCommitVersionReques TEST(maxVersionGap); // Maximum possible version gap self->lastVersionTime = t1; - if (self->resolverNeedingChanges.count(req.requestingProxy)) { - rep.resolverChanges = self->resolverChanges.get(); - rep.resolverChangesVersion = self->resolverChangesVersion; - self->resolverNeedingChanges.erase(req.requestingProxy); - - TEST(!rep.resolverChanges.empty()); // resolution balancing moves keyranges - if (self->resolverNeedingChanges.empty()) - self->resolverChanges.set(Standalone>()); - } + self->resolutionBalancer.setChangesInReply(req.requestingProxy, rep); } rep.version = self->version; @@ -311,16 +159,11 @@ ACTOR Future getVersion(Reference self, GetCommitVersionReques ACTOR Future provideVersions(Reference self) { state ActorCollection versionActors(false); - for (auto& p : self->commitProxies) - self->lastCommitProxyVersionReplies[p.id()] = CommitProxyVersionReplies(); - - loop { - choose { - when(GetCommitVersionRequest req = waitNext(self->myInterface.getCommitVersion.getFuture())) { - versionActors.add(getVersion(self, req)); - } - when(wait(versionActors.getResult())) {} + loop choose { + when(GetCommitVersionRequest req = waitNext(self->myInterface.getCommitVersion.getFuture())) { + versionActors.add(getVersion(self, req)); } + when(wait(versionActors.getResult())) {} } } @@ -381,17 +224,15 @@ ACTOR Future updateRecoveryData(Reference self) { self->lastEpochEnd = req.lastEpochEnd; } if (req.commitProxies.size() > 0) { - self->commitProxies = req.commitProxies; self->lastCommitProxyVersionReplies.clear(); - for (auto& p : self->commitProxies) { + for (auto& p : req.commitProxies) { self->lastCommitProxyVersionReplies[p.id()] = CommitProxyVersionReplies(); } } - self->resolvers = req.resolvers; - if (req.resolvers.size() > 1) - self->triggerResolution.trigger(); + self->resolutionBalancer.setCommitProxies(req.commitProxies); + self->resolutionBalancer.setResolvers(req.resolvers); req.reply.send(Void()); } @@ -446,7 +287,6 @@ ACTOR Future masterServer(MasterInterface mi, addActor.send(provideVersions(self)); addActor.send(serveLiveCommittedVersion(self)); addActor.send(updateRecoveryData(self)); - addActor.send(resolutionBalancing(self)); TEST(!lifetime.isStillValid(db->get().masterLifetime, mi.id() == db->get().master.id())); // Master born doomed TraceEvent("MasterLifetime", self->dbgid).detail("LifetimeToken", lifetime.toString()); From 930cd7f9c7758161fc99975684ff416fd254e3da Mon Sep 17 00:00:00 2001 From: Bala Namasivayam Date: Wed, 23 Mar 2022 11:03:49 -0700 Subject: [PATCH 396/413] Add backup documentation to select between v2 and v4 signature protocol. --- documentation/sphinx/source/backups.rst | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/documentation/sphinx/source/backups.rst b/documentation/sphinx/source/backups.rst index 6d09954998..3bd668a974 100644 --- a/documentation/sphinx/source/backups.rst +++ b/documentation/sphinx/source/backups.rst @@ -155,6 +155,12 @@ Here is a complete list of valid parameters: **Example**: The URL parameter *header=x-amz-storage-class:REDUCED_REDUNDANCY* would send the HTTP header required to use the reduced redundancy storage option in the S3 API. +Signing Protocol +================= + +AWS signature version 4 is the default signing protocol choice. This boolean knob ``--knob_http_request_aws_v4_header`` can be used to select between v4 style and v2 style signatures. +If the knob is set to ``true`` then v4 signature will be used and if set to ``false`` then v2 signature will be used. + .. _blob-credential-files: Blob Credential Files From be57fd5b2670dfae139b0c89b83246226d93532e Mon Sep 17 00:00:00 2001 From: Trevor Clinkenbeard Date: Wed, 23 Mar 2022 11:07:06 -0700 Subject: [PATCH 397/413] Fix `TEST` macro comment Co-authored-by: A.J. Beamon --- fdbserver/TransactionTagCounter.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/fdbserver/TransactionTagCounter.cpp b/fdbserver/TransactionTagCounter.cpp index f9215ea7cb..1f0a25c2cc 100644 --- a/fdbserver/TransactionTagCounter.cpp +++ b/fdbserver/TransactionTagCounter.cpp @@ -27,7 +27,7 @@ TransactionTagCounter::TransactionTagCounter(UID thisServerID) void TransactionTagCounter::addRequest(Optional const& tags, int64_t bytes) { if (tags.present()) { - TEST(true); // Tracking tag on storage server + TEST(true); // Tracking transaction tag in counter double cost = costFunction(bytes); for (auto& tag : tags.get()) { int64_t& count = intervalCounts[TransactionTag(tag, tags.get().getArena())]; From 383b0cf0e10766552a9a844eb463f6afef8b85be Mon Sep 17 00:00:00 2001 From: Zhe Wang Date: Wed, 23 Mar 2022 15:51:55 -0400 Subject: [PATCH 398/413] add-rocksdb-memory-usage-metrics --- fdbserver/KeyValueStoreRocksDB.actor.cpp | 2 ++ 1 file changed, 2 insertions(+) diff --git a/fdbserver/KeyValueStoreRocksDB.actor.cpp b/fdbserver/KeyValueStoreRocksDB.actor.cpp index 7069c9de42..9c4f925854 100644 --- a/fdbserver/KeyValueStoreRocksDB.actor.cpp +++ b/fdbserver/KeyValueStoreRocksDB.actor.cpp @@ -784,6 +784,8 @@ ACTOR Future rocksDBMetricLogger(std::shared_ptr stat { "EstimateLiveDataSize", rocksdb::DB::Properties::kEstimateLiveDataSize }, { "BaseLevel", rocksdb::DB::Properties::kBaseLevel }, { "EstPendCompactBytes", rocksdb::DB::Properties::kEstimatePendingCompactionBytes }, + { "BlockCacheUsage", rocksdb::DB::Properties::kBlockCacheUsage }, + { "BlockCachePinnedUsage", rocksdb::DB::Properties::kBlockCachePinnedUsage }, }; state std::unordered_map readIteratorPoolStats = { From ca653c77eedad003b312722e9505058511dd4928 Mon Sep 17 00:00:00 2001 From: "A.J. Beamon" Date: Wed, 23 Mar 2022 13:57:45 -0700 Subject: [PATCH 399/413] Tenant binding tester support (#6642) * Add binding tester support for tenants * Configure tenant_mode=optional_experimental in the binding tester * Use the special key-space to delete tenants in between binding tester runs. * Separate tenant tester spec into its own file. --- bindings/bindingtester/bindingtester.py | 14 ++++ bindings/bindingtester/known_testers.py | 3 +- bindings/bindingtester/spec/tenantTester.md | 77 +++++++++++++++++++++ bindings/bindingtester/tests/api.py | 38 +++++++++- contrib/Joshua/scripts/localClusterStart.sh | 2 +- 5 files changed, 129 insertions(+), 5 deletions(-) create mode 100644 bindings/bindingtester/spec/tenantTester.md diff --git a/bindings/bindingtester/bindingtester.py b/bindings/bindingtester/bindingtester.py index 9c178a09d5..d914e9d9dc 100755 --- a/bindings/bindingtester/bindingtester.py +++ b/bindings/bindingtester/bindingtester.py @@ -202,6 +202,7 @@ class TestRunner(object): self.args.types = list(reduce(lambda t1, t2: filter(t1.__contains__, t2), map(lambda tester: tester.types, self.testers))) self.args.no_directory_snapshot_ops = self.args.no_directory_snapshot_ops or any([not tester.directory_snapshot_ops_enabled for tester in self.testers]) + self.args.no_tenants = self.args.no_tenants or any([not tester.tenants_enabled for tester in self.testers]) def print_test(self): test_instructions = self._generate_test() @@ -282,6 +283,17 @@ class TestRunner(object): def _insert_instructions(self, test_instructions): util.get_logger().info('\nInserting test into database...') del self.db[:] + + while True: + tr = self.db.create_transaction() + try: + tr.options.set_special_key_space_enable_writes() + del tr[b'\xff\xff/management/tenant_map/' : b'\xff\xff/management/tenant_map0'] + tr.commit().wait() + break + except fdb.FDBError as e: + tr.on_error(e).wait() + for subspace, thread in test_instructions.items(): thread.insert_operations(self.db, subspace) @@ -445,6 +457,8 @@ def parse_args(argv): parser.add_argument('--no-directory-snapshot-ops', action='store_true', help='Disables snapshot operations for directory instructions.') + parser.add_argument('--no-tenants', action='store_true', help='Disables tenant operations.') + return parser.parse_args(argv) diff --git a/bindings/bindingtester/known_testers.py b/bindings/bindingtester/known_testers.py index 0fe5ad638f..2cff6c3cbc 100644 --- a/bindings/bindingtester/known_testers.py +++ b/bindings/bindingtester/known_testers.py @@ -26,7 +26,7 @@ ALL_TYPES = COMMON_TYPES + ['versionstamp'] class Tester: - def __init__(self, name, cmd, max_int_bits=64, min_api_version=0, max_api_version=MAX_API_VERSION, threads_enabled=True, types=COMMON_TYPES, directory_snapshot_ops_enabled=True): + def __init__(self, name, cmd, max_int_bits=64, min_api_version=0, max_api_version=MAX_API_VERSION, threads_enabled=True, types=COMMON_TYPES, directory_snapshot_ops_enabled=True, tenants_enabled=False): self.name = name self.cmd = cmd self.max_int_bits = max_int_bits @@ -35,6 +35,7 @@ class Tester: self.threads_enabled = threads_enabled self.types = types self.directory_snapshot_ops_enabled = directory_snapshot_ops_enabled + self.tenants_enabled = tenants_enabled def supports_api_version(self, api_version): return api_version >= self.min_api_version and api_version <= self.max_api_version diff --git a/bindings/bindingtester/spec/tenantTester.md b/bindings/bindingtester/spec/tenantTester.md new file mode 100644 index 0000000000..2ba54a74c4 --- /dev/null +++ b/bindings/bindingtester/spec/tenantTester.md @@ -0,0 +1,77 @@ +Overview +-------- + +Tenant testing is an optional extension to the core binding tester that enables +testing of the tenant API. This testing is enabled by adding some additional +instructions and modifying the behavior of some existing instructions. + +Additional State and Initialization +----------------------------------- + +Your tester should store an additional piece of state tracking the active tenant +that is to be used to create transactions. This tenant must support an unset +state, in which case transactions will be created directly on the database. + +New Instructions +---------------- + +The tenant API introduces some new operations: + +#### TENANT_CREATE + + Pops the top item off of the stack as TENANT_NAME. Creates a new tenant + in the database with the name TENANT_NAME. May optionally push a future + onto the stack. + +#### TENANT_DELETE + + Pops the top item off of the stack as TENANT_NAME. Deletes the tenant with + the name TENANT_NAME from the database. May optionally push a future onto + the stack. + +#### TENANT_SET_ACTIVE + + Pops the top item off of the stack as TENANT_NAME. Opens the tenant with + name TENANT_NAME and stores it as the active tenant. + +#### TENANT_CLEAR_ACTIVE + + Unsets the active tenant. + +Updates to Existing Instructions +-------------------------------- + +Some existing operations in the binding tester will have slightly modified +behavior when tenants are enabled. + +#### NEW_TRANSACTION + + When creating a new transaction, the active tenant should be used. If no active + tenant is set, then the transaction should be created as normal using the + database. + +#### _TENANT suffix + + Similar to the _DATABASE suffix, an operation with the _TENANT suffix indicates + that the operation should be performed on the current active tenant object. If + there is no active tenant, then the operation should be performed on the database + as if _DATABASE was specified. In any case where the operation suffixed with + _DATABASE is allowed to push a future onto the stack, the same operation suffixed + with _TENANT is also allowed to push a future onto the stack. + + If your binding does not support operations directly on a tenant object, you should + simulate it using an anonymous transaction. Remember that set and clear operations + must immediately commit (with appropriate retry behavior!). + + Operations that can include the _TENANT prefix are: + + GET_TENANT + GET_KEY_TENANT + GET_RANGE_TENANT + GET_RANGE_STARTS_WITH_TENANT + GET_RANGE_SELECTOR_TENANT + SET_TENANT + CLEAR_TENANT + CLEAR_RANGE_TENANT + CLEAR_RANGE_STARTS_WITH_TENANT + ATOMIC_OP_TENANT diff --git a/bindings/bindingtester/tests/api.py b/bindings/bindingtester/tests/api.py index df90adf890..fd495fac76 100644 --- a/bindings/bindingtester/tests/api.py +++ b/bindings/bindingtester/tests/api.py @@ -58,6 +58,7 @@ class ApiTest(Test): self.outstanding_ops = [] self.random = test_util.RandomGenerator(args.max_int_bits, args.api_version, args.types) self.api_version = args.api_version + self.allocated_tenants = set() def add_stack_items(self, num): self.stack_size += num @@ -137,6 +138,12 @@ class ApiTest(Test): test_util.to_front(instructions, self.stack_size - read[0]) instructions.append('WAIT_FUTURE') + def choose_tenant(self, new_tenant_probability): + if len(self.allocated_tenants) == 0 or random.random() < new_tenant_probability: + return self.random.random_string(random.randint(0, 30)) + else: + return random.choice(list(self.allocated_tenants)) + def generate(self, args, thread_number): instructions = InstructionSet() @@ -158,6 +165,7 @@ class ApiTest(Test): write_conflicts = ['WRITE_CONFLICT_RANGE', 'WRITE_CONFLICT_KEY', 'DISABLE_WRITE_CONFLICT'] txn_sizes = ['GET_APPROXIMATE_SIZE'] storage_metrics = ['GET_ESTIMATED_RANGE_SIZE', 'GET_RANGE_SPLIT_POINTS'] + tenants = ['TENANT_CREATE', 'TENANT_DELETE', 'TENANT_SET_ACTIVE', 'TENANT_CLEAR_ACTIVE'] op_choices += reads op_choices += mutations @@ -173,6 +181,9 @@ class ApiTest(Test): op_choices += txn_sizes op_choices += storage_metrics + if not args.no_tenants: + op_choices += tenants + idempotent_atomic_ops = ['BIT_AND', 'BIT_OR', 'MAX', 'MIN', 'BYTE_MIN', 'BYTE_MAX'] atomic_ops = idempotent_atomic_ops + ['ADD', 'BIT_XOR', 'APPEND_IF_FITS'] @@ -195,7 +206,7 @@ class ApiTest(Test): # print 'Adding instruction %s at %d' % (op, index) - if args.concurrency == 1 and (op in database_mutations): + if args.concurrency == 1 and (op in database_mutations or op in ['TENANT_CREATE', 'TENANT_DELETE']): self.wait_for_reads(instructions) test_util.blocking_commit(instructions) self.can_get_commit_version = False @@ -570,18 +581,39 @@ class ApiTest(Test): instructions.push_args(key1, key2, chunkSize) instructions.append(op) self.add_strings(1) - + elif op == 'TENANT_CREATE': + tenant_name = self.choose_tenant(0.8) + self.allocated_tenants.add(tenant_name) + instructions.push_args(tenant_name) + instructions.append(op) + self.add_strings(1) + elif op == 'TENANT_DELETE': + tenant_name = self.choose_tenant(0.2) + if tenant_name in self.allocated_tenants: + self.allocated_tenants.remove(tenant_name) + instructions.push_args(tenant_name) + instructions.append(op) + self.add_strings(1) + elif op == 'TENANT_SET_ACTIVE': + tenant_name = self.choose_tenant(0.8) + instructions.push_args(tenant_name) + instructions.append(op) + elif op == 'TENANT_CLEAR_ACTIVE': + instructions.append(op) else: assert False, 'Unknown operation: ' + op if read_performed and op not in database_reads: self.outstanding_ops.append((self.stack_size, len(instructions) - 1)) - if args.concurrency == 1 and (op in database_reads or op in database_mutations): + if args.concurrency == 1 and (op in database_reads or op in database_mutations or op in ['TENANT_CREATE', 'TENANT_DELETE']): instructions.append('WAIT_FUTURE') instructions.begin_finalization() + if not args.no_tenants: + instructions.append('TENANT_CLEAR_ACTIVE') + if args.concurrency == 1: self.wait_for_reads(instructions) test_util.blocking_commit(instructions) diff --git a/contrib/Joshua/scripts/localClusterStart.sh b/contrib/Joshua/scripts/localClusterStart.sh index 7cbca6e41b..abbf93abc5 100644 --- a/contrib/Joshua/scripts/localClusterStart.sh +++ b/contrib/Joshua/scripts/localClusterStart.sh @@ -346,7 +346,7 @@ function createDatabase # Configure the database. else - "${BINDIR}/fdbcli" -C "${FDBCONF}" --exec 'configure new single memory; status' --timeout "${CONFIGUREWAIT}" --log --log-dir "${LOGDIR}" &>> "${LOGDIR}/fdbclient.log" + "${BINDIR}/fdbcli" -C "${FDBCONF}" --exec 'configure new single memory tenant_mode=optional_experimental; status' --timeout "${CONFIGUREWAIT}" --log --log-dir "${LOGDIR}" &>> "${LOGDIR}/fdbclient.log" if ! displayMessage "Checking if config succeeded" then From ce03f5783d7cb44c29f7cf8d975558e301b6b44f Mon Sep 17 00:00:00 2001 From: "A.J. Beamon" Date: Sat, 19 Feb 2022 15:40:32 -0800 Subject: [PATCH 400/413] Add tenant support to Python --- bindings/bindingtester/known_testers.py | 4 +- bindings/python/fdb/__init__.py | 1 + bindings/python/fdb/impl.py | 170 +++++++++++++++--------- bindings/python/tests/tester.py | 24 +++- 4 files changed, 133 insertions(+), 66 deletions(-) diff --git a/bindings/bindingtester/known_testers.py b/bindings/bindingtester/known_testers.py index 2cff6c3cbc..fbae72d36c 100644 --- a/bindings/bindingtester/known_testers.py +++ b/bindings/bindingtester/known_testers.py @@ -58,8 +58,8 @@ _java_cmd = 'java -ea -cp %s:%s com.apple.foundationdb.test.' % ( # We could set min_api_version lower on some of these if the testers were updated to support them testers = { - 'python': Tester('python', 'python ' + _absolute_path('python/tests/tester.py'), 2040, 23, MAX_API_VERSION, types=ALL_TYPES), - 'python3': Tester('python3', 'python3 ' + _absolute_path('python/tests/tester.py'), 2040, 23, MAX_API_VERSION, types=ALL_TYPES), + 'python': Tester('python', 'python ' + _absolute_path('python/tests/tester.py'), 2040, 23, MAX_API_VERSION, types=ALL_TYPES, tenants_enabled=True), + 'python3': Tester('python3', 'python3 ' + _absolute_path('python/tests/tester.py'), 2040, 23, MAX_API_VERSION, types=ALL_TYPES, tenants_enabled=True), 'ruby': Tester('ruby', _absolute_path('ruby/tests/tester.rb'), 2040, 23, MAX_API_VERSION), 'java': Tester('java', _java_cmd + 'StackTester', 2040, 510, MAX_API_VERSION, types=ALL_TYPES), 'java_async': Tester('java', _java_cmd + 'AsyncStackTester', 2040, 510, MAX_API_VERSION, types=ALL_TYPES), diff --git a/bindings/python/fdb/__init__.py b/bindings/python/fdb/__init__.py index 0054e72808..413c81249a 100644 --- a/bindings/python/fdb/__init__.py +++ b/bindings/python/fdb/__init__.py @@ -88,6 +88,7 @@ def api_version(ver): 'predicates', 'Future', 'Database', + 'Tenant', 'Transaction', 'KeyValue', 'KeySelector', diff --git a/bindings/python/fdb/impl.py b/bindings/python/fdb/impl.py index 837d3937c6..3a7ea07c9e 100644 --- a/bindings/python/fdb/impl.py +++ b/bindings/python/fdb/impl.py @@ -198,9 +198,10 @@ def transactional(*tr_args, **tr_kwargs): one of two actions, depending on the type of the parameter passed to the function at call time. - If given a Database, a Transaction will be created and passed into - the wrapped code in place of the Database. After the function is - complete, the newly created transaction will be committed. + If given a Database or Tenant, a Transaction will be created and + passed into the wrapped code in place of the Database or Tenant. + After the function is complete, the newly created transaction + will be committed. It is important to note that the wrapped method may be called multiple times in the event of a commit failure, until the commit @@ -943,128 +944,114 @@ class FormerFuture(_FDBBase): except: pass - -class Database(_FDBBase): - def __init__(self, dpointer): - self.dpointer = dpointer - self.options = _DatabaseOptions(self) - - def __del__(self): - # print('Destroying database 0x%x' % self.dpointer) - self.capi.fdb_database_destroy(self.dpointer) - +class _TransactionCreator(_FDBBase): def get(self, key): - return Database.__database_getitem(self, key) + return _TransactionCreator.__creator_getitem(self, key) def __getitem__(self, key): if isinstance(key, slice): return self.get_range(key.start, key.stop, reverse=(key.step == -1)) - return Database.__database_getitem(self, key) + return _TransactionCreator.__creator_getitem(self, key) def get_key(self, key_selector): - return Database.__database_get_key(self, key_selector) + return _TransactionCreator.__creator_get_key(self, key_selector) def get_range(self, begin, end, limit=0, reverse=False, streaming_mode=StreamingMode.want_all): - return Database.__database_get_range(self, begin, end, limit, reverse, streaming_mode) + return _TransactionCreator.__creator_get_range(self, begin, end, limit, reverse, streaming_mode) def get_range_startswith(self, prefix, *args, **kwargs): - return Database.__database_get_range_startswith(self, prefix, *args, **kwargs) + return _TransactionCreator.__creator_get_range_startswith(self, prefix, *args, **kwargs) def set(self, key, value): - Database.__database_setitem(self, key, value) + _TransactionCreator.__creator_setitem(self, key, value) def __setitem__(self, key, value): - Database.__database_setitem(self, key, value) + _TransactionCreator.__creator_setitem(self, key, value) def clear(self, key): - Database.__database_delitem(self, key) + _TransactionCreator.__creator_delitem(self, key) def clear_range(self, begin, end): - Database.__database_delitem(self, slice(begin, end)) + _TransactionCreator.__creator_delitem(self, slice(begin, end)) def __delitem__(self, key_or_slice): - Database.__database_delitem(self, key_or_slice) + _TransactionCreator.__creator_delitem(self, key_or_slice) def clear_range_startswith(self, prefix): - Database.__database_clear_range_startswith(self, prefix) + _TransactionCreator.__creator_clear_range_startswith(self, prefix) def get_and_watch(self, key): - return Database.__database_get_and_watch(self, key) + return _TransactionCreator.__creator_get_and_watch(self, key) def set_and_watch(self, key, value): - return Database.__database_set_and_watch(self, key, value) + return _TransactionCreator.__creator_set_and_watch(self, key, value) def clear_and_watch(self, key): - return Database.__database_clear_and_watch(self, key) + return _TransactionCreator.__creator_clear_and_watch(self, key) def create_transaction(self): - pointer = ctypes.c_void_p() - self.capi.fdb_database_create_transaction(self.dpointer, ctypes.byref(pointer)) - return Transaction(pointer.value, self) - - def _set_option(self, option, param, length): - self.capi.fdb_database_set_option(self.dpointer, option, param, length) + pass def _atomic_operation(self, opcode, key, param): - Database.__database_atomic_operation(self, opcode, key, param) + _TransactionCreator.__creator_atomic_operation(self, opcode, key, param) #### Transaction implementations #### @staticmethod @transactional - def __database_getitem(tr, key): + def __creator_getitem(tr, key): return tr[key].value @staticmethod @transactional - def __database_get_key(tr, key_selector): + def __creator_get_key(tr, key_selector): return tr.get_key(key_selector).value @staticmethod @transactional - def __database_get_range(tr, begin, end, limit, reverse, streaming_mode): + def __creator_get_range(tr, begin, end, limit, reverse, streaming_mode): return tr.get_range(begin, end, limit, reverse, streaming_mode).to_list() @staticmethod @transactional - def __database_get_range_startswith(tr, prefix, *args, **kwargs): + def __creator_get_range_startswith(tr, prefix, *args, **kwargs): return tr.get_range_startswith(prefix, *args, **kwargs).to_list() @staticmethod @transactional - def __database_setitem(tr, key, value): + def __creator_setitem(tr, key, value): tr[key] = value @staticmethod @transactional - def __database_clear_range_startswith(tr, prefix): + def __creator_clear_range_startswith(tr, prefix): tr.clear_range_startswith(prefix) @staticmethod @transactional - def __database_get_and_watch(tr, key): + def __creator_get_and_watch(tr, key): v = tr.get(key) return v, tr.watch(key) @staticmethod @transactional - def __database_set_and_watch(tr, key, value): + def __creator_set_and_watch(tr, key, value): tr.set(key, value) return tr.watch(key) @staticmethod @transactional - def __database_clear_and_watch(tr, key): + def __creator_clear_and_watch(tr, key): del tr[key] return tr.watch(key) @staticmethod @transactional - def __database_delitem(tr, key_or_slice): + def __creator_delitem(tr, key_or_slice): del tr[key_or_slice] @staticmethod @transactional - def __database_atomic_operation(tr, opcode, key, param): + def __creator_atomic_operation(tr, opcode, key, param): tr._atomic_operation(opcode, key, param) # Asynchronous transactions @@ -1074,11 +1061,11 @@ class Database(_FDBBase): From = asyncio.From coroutine = asyncio.coroutine - class Database: + class TransactionCreator: @staticmethod @transactional @coroutine - def __database_getitem(tr, key): + def __creator_getitem(tr, key): # raise Return(( yield From( tr[key] ) )) raise Return(tr[key]) yield None @@ -1086,26 +1073,26 @@ class Database(_FDBBase): @staticmethod @transactional @coroutine - def __database_get_key(tr, key_selector): + def __creator_get_key(tr, key_selector): raise Return(tr.get_key(key_selector)) yield None @staticmethod @transactional @coroutine - def __database_get_range(tr, begin, end, limit, reverse, streaming_mode): + def __creator_get_range(tr, begin, end, limit, reverse, streaming_mode): raise Return((yield From(tr.get_range(begin, end, limit, reverse, streaming_mode).to_list()))) @staticmethod @transactional @coroutine - def __database_get_range_startswith(tr, prefix, *args, **kwargs): + def __creator_get_range_startswith(tr, prefix, *args, **kwargs): raise Return((yield From(tr.get_range_startswith(prefix, *args, **kwargs).to_list()))) @staticmethod @transactional @coroutine - def __database_setitem(tr, key, value): + def __creator_setitem(tr, key, value): tr[key] = value raise Return() yield None @@ -1113,7 +1100,7 @@ class Database(_FDBBase): @staticmethod @transactional @coroutine - def __database_clear_range_startswith(tr, prefix): + def __creator_clear_range_startswith(tr, prefix): tr.clear_range_startswith(prefix) raise Return() yield None @@ -1121,7 +1108,7 @@ class Database(_FDBBase): @staticmethod @transactional @coroutine - def __database_get_and_watch(tr, key): + def __creator_get_and_watch(tr, key): v = tr.get(key) raise Return(v, tr.watch(key)) yield None @@ -1129,7 +1116,7 @@ class Database(_FDBBase): @staticmethod @transactional @coroutine - def __database_set_and_watch(tr, key, value): + def __creator_set_and_watch(tr, key, value): tr.set(key, value) raise Return(tr.watch(key)) yield None @@ -1137,7 +1124,7 @@ class Database(_FDBBase): @staticmethod @transactional @coroutine - def __database_clear_and_watch(tr, key): + def __creator_clear_and_watch(tr, key): del tr[key] raise Return(tr.watch(key)) yield None @@ -1145,7 +1132,7 @@ class Database(_FDBBase): @staticmethod @transactional @coroutine - def __database_delitem(tr, key_or_slice): + def __creator_delitem(tr, key_or_slice): del tr[key_or_slice] raise Return() yield None @@ -1153,11 +1140,55 @@ class Database(_FDBBase): @staticmethod @transactional @coroutine - def __database_atomic_operation(tr, opcode, key, param): + def __creator_atomic_operation(tr, opcode, key, param): tr._atomic_operation(opcode, key, param) raise Return() yield None - return Database + return TransactionCreator + + +class Database(_TransactionCreator): + def __init__(self, dpointer): + self.dpointer = dpointer + self.options = _DatabaseOptions(self) + + def __del__(self): + # print('Destroying database 0x%x' % self.dpointer) + self.capi.fdb_database_destroy(self.dpointer) + + def _set_option(self, option, param, length): + self.capi.fdb_database_set_option(self.dpointer, option, param, length) + + def open_tenant(self, name): + if not isinstance(name, bytes): + raise TypeError('Tenant name must be of type ' + bytes.__name__) + pointer = ctypes.c_void_p() + self.capi.fdb_database_open_tenant(self.dpointer, name, len(name), ctypes.byref(pointer)) + return Tenant(pointer.value) + + def create_transaction(self): + pointer = ctypes.c_void_p() + self.capi.fdb_database_create_transaction(self.dpointer, ctypes.byref(pointer)) + return Transaction(pointer.value, self) + + def allocate_tenant(self, name): + return FutureVoid(self.capi.fdb_database_allocate_tenant(self.dpointer, name, len(name))) + + def delete_tenant(self, name): + return FutureVoid(self.capi.fdb_database_remove_tenant(self.dpointer, name, len(name))) + + +class Tenant(_TransactionCreator): + def __init__(self, tpointer): + self.tpointer = tpointer + + def __del__(self): + self.capi.fdb_tenant_destroy(self.tpointer) + + def create_transaction(self): + pointer = ctypes.c_void_p() + self.capi.fdb_tenant_create_transaction(self.tpointer, ctypes.byref(pointer)) + return Transaction(pointer.value, self) fill_operations() @@ -1458,6 +1489,10 @@ def init_c_api(): _capi.fdb_database_destroy.argtypes = [ctypes.c_void_p] _capi.fdb_database_destroy.restype = None + _capi.fdb_database_open_tenant.argtypes = [ctypes.c_void_p, ctypes.c_void_p, ctypes.c_int, ctypes.POINTER(ctypes.c_void_p)] + _capi.fdb_database_open_tenant.restype = ctypes.c_int + _capi.fdb_database_open_tenant.errcheck = check_error_code + _capi.fdb_database_create_transaction.argtypes = [ctypes.c_void_p, ctypes.POINTER(ctypes.c_void_p)] _capi.fdb_database_create_transaction.restype = ctypes.c_int _capi.fdb_database_create_transaction.errcheck = check_error_code @@ -1466,6 +1501,19 @@ def init_c_api(): _capi.fdb_database_set_option.restype = ctypes.c_int _capi.fdb_database_set_option.errcheck = check_error_code + _capi.fdb_database_allocate_tenant.argtypes = [ctypes.c_void_p, ctypes.c_void_p, ctypes.c_int] + _capi.fdb_database_allocate_tenant.restype = ctypes.c_void_p + + _capi.fdb_database_remove_tenant.argtypes = [ctypes.c_void_p, ctypes.c_void_p, ctypes.c_int] + _capi.fdb_database_remove_tenant.restype = ctypes.c_void_p + + _capi.fdb_tenant_destroy.argtypes = [ctypes.c_void_p] + _capi.fdb_tenant_destroy.restype = None + + _capi.fdb_tenant_create_transaction.argtypes = [ctypes.c_void_p, ctypes.POINTER(ctypes.c_void_p)] + _capi.fdb_tenant_create_transaction.restype = ctypes.c_int + _capi.fdb_tenant_create_transaction.errcheck = check_error_code + _capi.fdb_transaction_destroy.argtypes = [ctypes.c_void_p] _capi.fdb_transaction_destroy.restype = None @@ -1686,10 +1734,10 @@ def init(event_model=None): raise asyncio.Return(self) return it() FDBRange.iterate = iterate - AT = Database.declare_asynchronous_transactions() + AT = _TransactionCreator.declare_asynchronous_transactions() for name in dir(AT): - if name.startswith("_Database__database_"): - setattr(Database, name, getattr(AT, name)) + if name.startswith("__TransactionCreator__creator_"): + setattr(_TransactionCreator, name, getattr(AT, name)) def to_list(self): if self._mode == StreamingMode.iterator: diff --git a/bindings/python/tests/tester.py b/bindings/python/tests/tester.py index 6aa41dea4a..e7ce61d766 100644 --- a/bindings/python/tests/tester.py +++ b/bindings/python/tests/tester.py @@ -112,12 +112,13 @@ class Stack: class Instruction: - def __init__(self, tr, stack, op, index, isDatabase=False, isSnapshot=False): + def __init__(self, tr, stack, op, index, isDatabase=False, isTenant=False, isSnapshot=False): self.tr = tr self.stack = stack self.op = op self.index = index self.isDatabase = isDatabase + self.isTenant = isTenant self.isSnapshot = isSnapshot def pop(self, count=None, with_idx=False): @@ -277,6 +278,7 @@ class Tester: def __init__(self, db, prefix): self.db = db + self.tenant = None self.instructions = self.db[fdb.tuple.range((prefix,))] @@ -317,7 +319,8 @@ class Tester: def new_transaction(self): with Tester.tr_map_lock: - Tester.tr_map[self.tr_name] = self.db.create_transaction() + tr_source = self.tenant if self.tenant is not None else self.db + Tester.tr_map[self.tr_name] = tr_source.create_transaction() def switch_transaction(self, name): self.tr_name = name @@ -335,18 +338,22 @@ class Tester: # print("%d. Instruction is %s" % (idx, op)) isDatabase = op.endswith(six.u('_DATABASE')) + isTenant = op.endswith(six.u('_TENANT')) isSnapshot = op.endswith(six.u('_SNAPSHOT')) if isDatabase: op = op[:-9] obj = self.db + elif isTenant: + op = op[:-7] + obj = self.tenant if self.tenant else self.db elif isSnapshot: op = op[:-9] obj = self.current_transaction().snapshot else: obj = self.current_transaction() - inst = Instruction(obj, self.stack, op, idx, isDatabase, isSnapshot) + inst = Instruction(obj, self.stack, op, idx, isDatabase, isTenant, isSnapshot) try: if inst.op == six.u("PUSH"): @@ -583,6 +590,17 @@ class Tester: prefix = inst.pop() Tester.wait_empty(self.db, prefix) inst.push(b"WAITED_FOR_EMPTY") + elif inst.op == six.u("TENANT_CREATE"): + name = inst.pop() + inst.push(self.db.allocate_tenant(name)) + elif inst.op == six.u("TENANT_DELETE"): + name = inst.pop() + inst.push(self.db.delete_tenant(name)) + elif inst.op == six.u("TENANT_SET_ACTIVE"): + name = inst.pop() + self.tenant = self.db.open_tenant(name) + elif inst.op == six.u("TENANT_CLEAR_ACTIVE"): + self.tenant = None elif inst.op == six.u("UNIT_TESTS"): try: test_db_options(db) From 8e848f16df33b5a128ad1280d2fde4be8df49991 Mon Sep 17 00:00:00 2001 From: Jon Fu Date: Mon, 28 Feb 2022 18:15:10 -0500 Subject: [PATCH 401/413] Support tuples in python tenants --- bindings/python/fdb/impl.py | 19 ++++++-- .../python/tests/tenant_tuple_name_tests.py | 47 +++++++++++++++++++ bindings/python/tests/tester.py | 3 ++ 3 files changed, 64 insertions(+), 5 deletions(-) create mode 100644 bindings/python/tests/tenant_tuple_name_tests.py diff --git a/bindings/python/fdb/impl.py b/bindings/python/fdb/impl.py index 3a7ea07c9e..47e22f043d 100644 --- a/bindings/python/fdb/impl.py +++ b/bindings/python/fdb/impl.py @@ -34,6 +34,7 @@ import traceback import fdb from fdb import six +from fdb.tuple import pack, unpack _network_thread = None _network_thread_reentrant_lock = threading.RLock() @@ -1146,6 +1147,13 @@ class _TransactionCreator(_FDBBase): yield None return TransactionCreator +def process_tenant_name(name): + if isinstance(name, tuple): + return pack(name) + elif isinstance(name, bytes): + return name + else: + raise TypeError('Tenant name must be of type ' + bytes.__name__ + ' or of type ' + tuple.__name__) class Database(_TransactionCreator): def __init__(self, dpointer): @@ -1160,10 +1168,9 @@ class Database(_TransactionCreator): self.capi.fdb_database_set_option(self.dpointer, option, param, length) def open_tenant(self, name): - if not isinstance(name, bytes): - raise TypeError('Tenant name must be of type ' + bytes.__name__) + tname = process_tenant_name(name) pointer = ctypes.c_void_p() - self.capi.fdb_database_open_tenant(self.dpointer, name, len(name), ctypes.byref(pointer)) + self.capi.fdb_database_open_tenant(self.dpointer, tname, len(tname), ctypes.byref(pointer)) return Tenant(pointer.value) def create_transaction(self): @@ -1172,10 +1179,12 @@ class Database(_TransactionCreator): return Transaction(pointer.value, self) def allocate_tenant(self, name): - return FutureVoid(self.capi.fdb_database_allocate_tenant(self.dpointer, name, len(name))) + tname = process_tenant_name(name) + return FutureVoid(self.capi.fdb_database_allocate_tenant(self.dpointer, tname, len(tname))) def delete_tenant(self, name): - return FutureVoid(self.capi.fdb_database_remove_tenant(self.dpointer, name, len(name))) + tname = process_tenant_name(name) + return FutureVoid(self.capi.fdb_database_remove_tenant(self.dpointer, tname, len(tname))) class Tenant(_TransactionCreator): diff --git a/bindings/python/tests/tenant_tuple_name_tests.py b/bindings/python/tests/tenant_tuple_name_tests.py new file mode 100644 index 0000000000..3b64675c68 --- /dev/null +++ b/bindings/python/tests/tenant_tuple_name_tests.py @@ -0,0 +1,47 @@ +#!/usr/bin/python +# +# tenant_tuple_name_tests.py +# +# 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. +# +import fdb +import sys +from fdb.tuple import pack + +if __name__ == '__main__': + fdb.api_version(710) + +def test_tenant_tuple_name(db): + tuplename=(b'test', b'level', b'hierarchy', 3, 1.24, 'str') + db.allocate_tenant(tuplename).wait() + + tenant=db.open_tenant(tuplename) + tenant[b'foo'] = b'bar' + + assert tenant[b'foo'] == b'bar' + + del tenant[b'foo'] + db.delete_tenant(tuplename).wait() + +# Expect a cluster file as input. This test will write to the FDB cluster, so +# be aware of potential side effects. +if __name__ == '__main__': + clusterFile = sys.argv[1] + db = fdb.open(clusterFile) + db.options.set_transaction_timeout(2000) # 2 seconds + db.options.set_transaction_retry_limit(3) + test_tenant_tuple_name(db) diff --git a/bindings/python/tests/tester.py b/bindings/python/tests/tester.py index e7ce61d766..3f1f3f10d9 100644 --- a/bindings/python/tests/tester.py +++ b/bindings/python/tests/tester.py @@ -49,6 +49,7 @@ from cancellation_timeout_tests import test_db_retry_limits from cancellation_timeout_tests import test_combinations from size_limit_tests import test_size_limit_option, test_get_approximate_size +from tenant_tuple_name_tests import test_tenant_tuple_name random.seed(0) @@ -618,6 +619,8 @@ class Tester: test_size_limit_option(db) test_get_approximate_size(db) + test_tenant_tuple_name(db) + except fdb.FDBError as e: print("Unit tests failed: %s" % e.description) traceback.print_exc() From 8b92d3fccdc2237fadd838b659f58706c23df463 Mon Sep 17 00:00:00 2001 From: "A.J. Beamon" Date: Sun, 6 Mar 2022 21:09:20 -0800 Subject: [PATCH 402/413] Use special keys to create/delete tenants --- bindings/python/fdb/impl.py | 36 +++++++++++++------ .../python/tests/tenant_tuple_name_tests.py | 4 +-- bindings/python/tests/tester.py | 6 ++-- 3 files changed, 32 insertions(+), 14 deletions(-) diff --git a/bindings/python/fdb/impl.py b/bindings/python/fdb/impl.py index 47e22f043d..5fcefb73f3 100644 --- a/bindings/python/fdb/impl.py +++ b/bindings/python/fdb/impl.py @@ -1179,12 +1179,34 @@ class Database(_TransactionCreator): return Transaction(pointer.value, self) def allocate_tenant(self, name): - tname = process_tenant_name(name) - return FutureVoid(self.capi.fdb_database_allocate_tenant(self.dpointer, tname, len(tname))) + Database.__database_allocate_tenant(self, process_tenant_name(name), []) def delete_tenant(self, name): - tname = process_tenant_name(name) - return FutureVoid(self.capi.fdb_database_remove_tenant(self.dpointer, tname, len(tname))) + Database.__database_delete_tenant(self, process_tenant_name(name), []) + + @staticmethod + @transactional + def __database_allocate_tenant(tr, name, existence_check_marker): + tr.options.set_special_key_space_enable_writes() + key = b'\xff\xff/management/tenant_map/%s' % name + if not existence_check_marker: + existing_tenant = tr[key].wait() + existence_check_marker.append(None) + if existing_tenant != None: + raise fdb.FDBError(2132) # tenant_already_exists + tr[key] = b'' + + @staticmethod + @transactional + def __database_delete_tenant(tr, name, existence_check_marker): + tr.options.set_special_key_space_enable_writes() + key = b'\xff\xff/management/tenant_map/%s' % name + if not existence_check_marker: + existing_tenant = tr[key].wait() + existence_check_marker.append(None) + if existing_tenant == None: + raise fdb.FDBError(2131) # tenant_not_found + del tr[key] class Tenant(_TransactionCreator): @@ -1510,12 +1532,6 @@ def init_c_api(): _capi.fdb_database_set_option.restype = ctypes.c_int _capi.fdb_database_set_option.errcheck = check_error_code - _capi.fdb_database_allocate_tenant.argtypes = [ctypes.c_void_p, ctypes.c_void_p, ctypes.c_int] - _capi.fdb_database_allocate_tenant.restype = ctypes.c_void_p - - _capi.fdb_database_remove_tenant.argtypes = [ctypes.c_void_p, ctypes.c_void_p, ctypes.c_int] - _capi.fdb_database_remove_tenant.restype = ctypes.c_void_p - _capi.fdb_tenant_destroy.argtypes = [ctypes.c_void_p] _capi.fdb_tenant_destroy.restype = None diff --git a/bindings/python/tests/tenant_tuple_name_tests.py b/bindings/python/tests/tenant_tuple_name_tests.py index 3b64675c68..13af513953 100644 --- a/bindings/python/tests/tenant_tuple_name_tests.py +++ b/bindings/python/tests/tenant_tuple_name_tests.py @@ -27,7 +27,7 @@ if __name__ == '__main__': def test_tenant_tuple_name(db): tuplename=(b'test', b'level', b'hierarchy', 3, 1.24, 'str') - db.allocate_tenant(tuplename).wait() + db.allocate_tenant(tuplename) tenant=db.open_tenant(tuplename) tenant[b'foo'] = b'bar' @@ -35,7 +35,7 @@ def test_tenant_tuple_name(db): assert tenant[b'foo'] == b'bar' del tenant[b'foo'] - db.delete_tenant(tuplename).wait() + db.delete_tenant(tuplename) # Expect a cluster file as input. This test will write to the FDB cluster, so # be aware of potential side effects. diff --git a/bindings/python/tests/tester.py b/bindings/python/tests/tester.py index 3f1f3f10d9..e1a559b396 100644 --- a/bindings/python/tests/tester.py +++ b/bindings/python/tests/tester.py @@ -593,10 +593,12 @@ class Tester: inst.push(b"WAITED_FOR_EMPTY") elif inst.op == six.u("TENANT_CREATE"): name = inst.pop() - inst.push(self.db.allocate_tenant(name)) + self.db.allocate_tenant(name) + inst.push(b"RESULT_NOT_PRESENT") elif inst.op == six.u("TENANT_DELETE"): name = inst.pop() - inst.push(self.db.delete_tenant(name)) + self.db.delete_tenant(name) + inst.push(b"RESULT_NOT_PRESENT") elif inst.op == six.u("TENANT_SET_ACTIVE"): name = inst.pop() self.tenant = self.db.open_tenant(name) From 1d0a3669df84985fbdb39bdee39e7b2d1c7c5eba Mon Sep 17 00:00:00 2001 From: "A.J. Beamon" Date: Mon, 21 Mar 2022 12:01:30 -0700 Subject: [PATCH 403/413] Add documentation for the new Python tenant APIs. --- documentation/sphinx/source/api-python.rst | 33 +++++++++++++++++++++- 1 file changed, 32 insertions(+), 1 deletion(-) diff --git a/documentation/sphinx/source/api-python.rst b/documentation/sphinx/source/api-python.rst index f3af667e0c..2e9934564b 100644 --- a/documentation/sphinx/source/api-python.rst +++ b/documentation/sphinx/source/api-python.rst @@ -7,7 +7,7 @@ .. |database-type| replace:: ``Database`` .. |database-class| replace:: :class:`Database` .. |database-auto| replace:: the :func:`@fdb.transactional ` decorator -.. |tenant-type| replace:: FIXME +.. |tenant-type| replace:: :class:`Tenant` .. |transaction-class| replace:: :class:`Transaction` .. |get-key-func| replace:: :func:`Transaction.get_key` .. |get-range-func| replace:: :func:`Transaction.get_range` @@ -316,9 +316,29 @@ A |database-blurb1| |database-blurb2| Returns a new :class:`Transaction` object. Consider using the :func:`@fdb.transactional ` decorator to create transactions instead, since it will automatically provide you with appropriate retry behavior. +.. method:: Database.open_tenant(tenant_name) + + Opens an existing tenant to be used for running transactions and returns it as a :class`Tenant` object. + + The tenant name can be either a byte string or a tuple. If a tuple is provided, the tuple will be packed using the tuple layer to generate the byte string tenant name. + .. |sync-read| replace:: This read is fully synchronous. .. |sync-write| replace:: This change will be committed immediately, and is fully synchronous. +.. method:: Database.allocate_tenant(tenant_name): + + Creates a new tenant in the cluster. |sync-write| + + The tenant name can be either a byte string or a tuple and cannot start with the ``\xff`` byte. If a tuple is provided, the tuple will be packed using the tuple layer to generate the byte string tenant name. + +.. method:: Database.delete_tenant(tenant_name): + + Delete a tenant from the cluster. |sync-write| + + The tenant name can be either a byte string or a tuple. If a tuple is provided, the tuple will be packed using the tuple layer to generate the byte string tenant name. + + It is an error to delete a tenant that still has data. To delete a non-empty tenant, first clear all of the keys in the tenant. + .. method:: Database.get(key) Returns the value associated with the specified key in the database (or ``None`` if the key does not exist). |sync-read| @@ -460,6 +480,17 @@ Database options .. method:: Database.options.set_snapshot_ryw_disable() |option-db-snapshot-ryw-disable-blurb| + +Tenant objects +============== + +.. class:: Tenant + +|tenant-blurb1| + +.. method:: Tenant.create_transaction() + + Returns a new :class:`Transaction` object. Consider using the :func:`@fdb.transactional ` decorator to create transactions instead, since it will automatically provide you with appropriate retry behavior. .. _api-python-transactional-decorator: From ded4d046f07cf6338b44fe9385ea0c702157ffc5 Mon Sep 17 00:00:00 2001 From: "A.J. Beamon" Date: Mon, 21 Mar 2022 12:06:36 -0700 Subject: [PATCH 404/413] Update transactional decorator documentation to reference tenants --- documentation/sphinx/source/api-python.rst | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/documentation/sphinx/source/api-python.rst b/documentation/sphinx/source/api-python.rst index 2e9934564b..5dab3e49c6 100644 --- a/documentation/sphinx/source/api-python.rst +++ b/documentation/sphinx/source/api-python.rst @@ -510,9 +510,9 @@ Transactional decoration The ``@fdb.transactional`` decorator makes ``simple_function`` a transactional function. All functions using this decorator must have an argument **named** ``tr``. This specially named argument is passed a transaction that the function can use to do reads and writes. - A caller of a transactionally decorated function can pass a :class:`Database` instead of a transaction for the ``tr`` parameter. Then a transaction will be created automatically, and automatically committed before returning to the caller. The decorator will retry calling the decorated function until the transaction successfully commits. + A caller of a transactionally decorated function can pass a :class:`Database` or :class:`Tenant` instead of a transaction for the ``tr`` parameter. Then a transaction will be created automatically, and automatically committed before returning to the caller. The decorator will retry calling the decorated function until the transaction successfully commits. - If ``db`` is a :class:`Database`, a call like :: + If ``db`` is a :class:`Database` or :class:`Tenant`, a call like :: simple_function(db, 'a', 'b') @@ -775,7 +775,7 @@ Committing .. decorator:: transactional() - The ``transactional`` decorator makes it easy to write transactional functions which accept either a :class:`Database` or a :class:`Transaction` as a parameter and automatically commit. See :func:`@fdb.transactional ` for explanation and examples. + The ``transactional`` decorator makes it easy to write transactional functions which accept a :class:`Database`, :class`Tenant`, or :class:`Transaction` as a parameter and automatically commit. See :func:`@fdb.transactional ` for explanation and examples. .. method :: Transaction.commit() @@ -785,7 +785,7 @@ Committing |commit-outstanding-reads-blurb| - .. note :: Consider using the :func:`@fdb.transactional ` decorator, which not only calls :meth:`Database.create_transaction` and :meth:`Transaction.commit()` for you but also implements the required error handling and retry logic for transactions. + .. note :: Consider using the :func:`@fdb.transactional ` decorator, which not only calls :meth:`Database.create_transaction` or :meth`Tenant.create_transaction` and :meth:`Transaction.commit()` for you but also implements the required error handling and retry logic for transactions. .. warning :: |used-during-commit-blurb| From 77ce0f4fc7cb8f038d6345f15d41a7e453f902bd Mon Sep 17 00:00:00 2001 From: "A.J. Beamon" Date: Wed, 23 Mar 2022 15:50:06 -0700 Subject: [PATCH 405/413] Add a unit test in Python to exercise some of the tenant code. Add some comments to the allocate and delete tenant implementations. --- bindings/python/fdb/impl.py | 16 +++ bindings/python/tests/tenant_tests.py | 123 ++++++++++++++++++ .../python/tests/tenant_tuple_name_tests.py | 47 ------- bindings/python/tests/tester.py | 4 +- 4 files changed, 141 insertions(+), 49 deletions(-) create mode 100755 bindings/python/tests/tenant_tests.py delete mode 100644 bindings/python/tests/tenant_tuple_name_tests.py diff --git a/bindings/python/fdb/impl.py b/bindings/python/fdb/impl.py index 5fcefb73f3..023e85ae95 100644 --- a/bindings/python/fdb/impl.py +++ b/bindings/python/fdb/impl.py @@ -1184,6 +1184,14 @@ class Database(_TransactionCreator): def delete_tenant(self, name): Database.__database_delete_tenant(self, process_tenant_name(name), []) + # Attempt to allocate a tenant in the cluster. If the tenant already exists, + # this function will return a tenant_already_exists error. If the tenant is created + # concurrently, then this function may return success even if another caller creates + # it. + # + # The existence_check_marker is expected to be an empty list. This function will + # modify the list after completing the existence check to avoid checking for existence + # on retries. This allows the operation to be idempotent. @staticmethod @transactional def __database_allocate_tenant(tr, name, existence_check_marker): @@ -1196,6 +1204,14 @@ class Database(_TransactionCreator): raise fdb.FDBError(2132) # tenant_already_exists tr[key] = b'' + # Attempt to remove a tenant in the cluster. If the tenant doesn't exist, this + # function will return a tenant_not_found error. If the tenant is deleted + # concurrently, then this function may return success even if another caller deletes + # it. + # + # The existence_check_marker is expected to be an empty list. This function will + # modify the list after completing the existence check to avoid checking for existence + # on retries. This allows the operation to be idempotent. @staticmethod @transactional def __database_delete_tenant(tr, name, existence_check_marker): diff --git a/bindings/python/tests/tenant_tests.py b/bindings/python/tests/tenant_tests.py new file mode 100755 index 0000000000..9f35620b6a --- /dev/null +++ b/bindings/python/tests/tenant_tests.py @@ -0,0 +1,123 @@ +#!/usr/bin/python +# +# tenant_tests.py +# +# 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. +# +import fdb +import sys +import json +from fdb.tuple import pack + +if __name__ == '__main__': + fdb.api_version(710) + +def test_tenant_tuple_name(db): + tuplename=(b'test', b'level', b'hierarchy', 3, 1.24, 'str') + db.allocate_tenant(tuplename) + + tenant=db.open_tenant(tuplename) + tenant[b'foo'] = b'bar' + + assert tenant[b'foo'] == b'bar' + + del tenant[b'foo'] + db.delete_tenant(tuplename) + +def cleanup_tenant(db, tenant_name): + try: + tenant = db.open_tenant(tenant_name) + del tenant[:] + db.delete_tenant(tenant_name) + except fdb.FDBError as e: + if e.code == 2131: # tenant not found + pass + else: + raise + +def test_tenant_operations(db): + cleanup_tenant(db, b'tenant1') + cleanup_tenant(db, b'tenant2') + + db.allocate_tenant(b'tenant1') + db.allocate_tenant(b'tenant2') + + tenant1 = db.open_tenant(b'tenant1') + tenant2 = db.open_tenant(b'tenant2') + + db[b'tenant_test_key'] = b'no_tenant' + tenant1[b'tenant_test_key'] = b'tenant1' + tenant2[b'tenant_test_key'] = b'tenant2' + + tenant1_entry = db[b'\xff\xff/management/tenant_map/tenant1'] + tenant1_json = json.loads(tenant1_entry) + prefix1 = tenant1_json['prefix'].encode('utf8') + + tenant2_entry = db[b'\xff\xff/management/tenant_map/tenant2'] + tenant2_json = json.loads(tenant2_entry) + prefix2 = tenant2_json['prefix'].encode('utf8') + + assert tenant1[b'tenant_test_key'] == b'tenant1' + assert db[prefix1 + b'tenant_test_key'] == b'tenant1' + assert tenant2[b'tenant_test_key'] == b'tenant2' + assert db[prefix2 + b'tenant_test_key'] == b'tenant2' + assert db[b'tenant_test_key'] == b'no_tenant' + + tr1 = tenant1.create_transaction() + try: + del tr1[:] + tr1.commit().wait() + except fdb.FDBError as e: + tr.on_error(e).wait() + + assert tenant1[b'tenant_test_key'] == None + assert db[prefix1 + b'tenant_test_key'] == None + assert tenant2[b'tenant_test_key'] == b'tenant2' + assert db[prefix2 + b'tenant_test_key'] == b'tenant2' + assert db[b'tenant_test_key'] == b'no_tenant' + + db.delete_tenant(b'tenant1') + try: + tenant1[b'tenant_test_key'] + assert False + except fdb.FDBError as e: + assert e.code == 2131 # tenant not found + + del tenant2[:] + db.delete_tenant(b'tenant2') + + assert db[prefix1 + b'tenant_test_key'] == None + assert db[prefix2 + b'tenant_test_key'] == None + assert db[b'tenant_test_key'] == b'no_tenant' + + del db[b'tenant_test_key'] + + assert db[b'tenant_test_key'] == None + +def test_tenants(db): + test_tenant_tuple_name(db) + test_tenant_operations(db) + +# Expect a cluster file as input. This test will write to the FDB cluster, so +# be aware of potential side effects. +if __name__ == '__main__': + clusterFile = sys.argv[1] + db = fdb.open(clusterFile) + db.options.set_transaction_timeout(2000) # 2 seconds + db.options.set_transaction_retry_limit(3) + + test_tenants(db) diff --git a/bindings/python/tests/tenant_tuple_name_tests.py b/bindings/python/tests/tenant_tuple_name_tests.py deleted file mode 100644 index 13af513953..0000000000 --- a/bindings/python/tests/tenant_tuple_name_tests.py +++ /dev/null @@ -1,47 +0,0 @@ -#!/usr/bin/python -# -# tenant_tuple_name_tests.py -# -# 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. -# -import fdb -import sys -from fdb.tuple import pack - -if __name__ == '__main__': - fdb.api_version(710) - -def test_tenant_tuple_name(db): - tuplename=(b'test', b'level', b'hierarchy', 3, 1.24, 'str') - db.allocate_tenant(tuplename) - - tenant=db.open_tenant(tuplename) - tenant[b'foo'] = b'bar' - - assert tenant[b'foo'] == b'bar' - - del tenant[b'foo'] - db.delete_tenant(tuplename) - -# Expect a cluster file as input. This test will write to the FDB cluster, so -# be aware of potential side effects. -if __name__ == '__main__': - clusterFile = sys.argv[1] - db = fdb.open(clusterFile) - db.options.set_transaction_timeout(2000) # 2 seconds - db.options.set_transaction_retry_limit(3) - test_tenant_tuple_name(db) diff --git a/bindings/python/tests/tester.py b/bindings/python/tests/tester.py index e1a559b396..7f8d794207 100644 --- a/bindings/python/tests/tester.py +++ b/bindings/python/tests/tester.py @@ -49,7 +49,7 @@ from cancellation_timeout_tests import test_db_retry_limits from cancellation_timeout_tests import test_combinations from size_limit_tests import test_size_limit_option, test_get_approximate_size -from tenant_tuple_name_tests import test_tenant_tuple_name +from tenant_tests import test_tenants random.seed(0) @@ -621,7 +621,7 @@ class Tester: test_size_limit_option(db) test_get_approximate_size(db) - test_tenant_tuple_name(db) + test_tenants(db) except fdb.FDBError as e: print("Unit tests failed: %s" % e.description) From 977fab2089f76283ef4250041746445f37c2802c Mon Sep 17 00:00:00 2001 From: Andrew Noyes Date: Wed, 23 Mar 2022 17:15:28 -0700 Subject: [PATCH 406/413] Make overwriteProxiesCount a function of conf keys (#6645) * Improve assert diagnostics * Make overwriteProxiesCount a deterministic function of conf keys Previously, the final values for grv_proxy count, commit_proxy count, and proxy count were derived from already-derived values from overwriteProxiesCount. Instead, we should only look at the conf keys. Also treat these keys as set to -1 if absent * Add unit test Illustrates how overwriteCommitProxy is not a function of conf keys. Passes after change and fails before --- fdbclient/DatabaseConfiguration.cpp | 37 ++++++++++++++++++++++------- fdbserver/ClusterRecovery.actor.cpp | 12 +++++----- 2 files changed, 34 insertions(+), 15 deletions(-) diff --git a/fdbclient/DatabaseConfiguration.cpp b/fdbclient/DatabaseConfiguration.cpp index e0205b8628..e915950361 100644 --- a/fdbclient/DatabaseConfiguration.cpp +++ b/fdbclient/DatabaseConfiguration.cpp @@ -23,6 +23,7 @@ #include "flow/ITrace.h" #include "flow/Trace.h" #include "flow/genericactors.actor.h" +#include "flow/UnitTest.h" DatabaseConfiguration::DatabaseConfiguration() { resetInternal(); @@ -490,9 +491,9 @@ void DatabaseConfiguration::overwriteProxiesCount() { Optional optGrvProxies = DatabaseConfiguration::get(grvProxiesKey); Optional optProxies = DatabaseConfiguration::get(proxiesKey); - const int mutableGrvProxyCount = optGrvProxies.present() ? toInt(optGrvProxies.get()) : 0; - const int mutableCommitProxyCount = optCommitProxies.present() ? toInt(optCommitProxies.get()) : 0; - const int mutableProxiesCount = optProxies.present() ? toInt(optProxies.get()) : 0; + const int mutableGrvProxyCount = optGrvProxies.present() ? toInt(optGrvProxies.get()) : -1; + const int mutableCommitProxyCount = optCommitProxies.present() ? toInt(optCommitProxies.get()) : -1; + const int mutableProxiesCount = optProxies.present() ? toInt(optProxies.get()) : -1; if (mutableProxiesCount > 1) { TraceEvent(SevDebug, "OverwriteProxiesCount") @@ -502,23 +503,23 @@ void DatabaseConfiguration::overwriteProxiesCount() { .detail("MutableGrvCPCount", mutableGrvProxyCount) .detail("MutableProxiesCount", mutableProxiesCount); - if (grvProxyCount == -1 && commitProxyCount > 0) { - if (mutableProxiesCount > commitProxyCount) { - grvProxyCount = mutableProxiesCount - commitProxyCount; + if (mutableGrvProxyCount == -1 && mutableCommitProxyCount > 0) { + if (mutableProxiesCount > mutableCommitProxyCount) { + grvProxyCount = mutableProxiesCount - mutableCommitProxyCount; } else { // invalid configuration; provision min GrvProxies grvProxyCount = 1; commitProxyCount = mutableProxiesCount - 1; } - } else if (grvProxyCount > 0 && commitProxyCount == -1) { - if (mutableProxiesCount > grvProxyCount) { + } else if (mutableGrvProxyCount > 0 && mutableCommitProxyCount == -1) { + if (mutableProxiesCount > mutableGrvProxyCount) { commitProxyCount = mutableProxiesCount - grvProxyCount; } else { // invalid configuration; provision min CommitProxies commitProxyCount = 1; grvProxyCount = mutableProxiesCount - 1; } - } else if (grvProxyCount == -1 && commitProxyCount == -1) { + } else if (mutableGrvProxyCount == -1 && mutableCommitProxyCount == -1) { // Use DEFAULT_COMMIT_GRV_PROXIES_RATIO to split proxies between Grv & Commit proxies const int derivedGrvProxyCount = std::max(1, @@ -825,3 +826,21 @@ bool DatabaseConfiguration::isOverridden(std::string key) const { return false; } + +TEST_CASE("/fdbclient/databaseConfiguration/overwriteCommitProxy") { + DatabaseConfiguration conf1; + conf1.applyMutation(MutationRef(MutationRef::SetValue, "\xff/conf/grv_proxies"_sr, "5"_sr)); + conf1.applyMutation(MutationRef(MutationRef::SetValue, "\xff/conf/proxies"_sr, "10"_sr)); + conf1.applyMutation(MutationRef(MutationRef::SetValue, "\xff/conf/grv_proxies"_sr, "-1"_sr)); + conf1.applyMutation(MutationRef(MutationRef::SetValue, "\xff/conf/commit_proxies"_sr, "-1"_sr)); + + DatabaseConfiguration conf2; + conf2.applyMutation(MutationRef(MutationRef::SetValue, "\xff/conf/proxies"_sr, "10"_sr)); + conf2.applyMutation(MutationRef(MutationRef::SetValue, "\xff/conf/grv_proxies"_sr, "-1"_sr)); + conf2.applyMutation(MutationRef(MutationRef::SetValue, "\xff/conf/commit_proxies"_sr, "-1"_sr)); + + ASSERT(conf1 == conf2); + ASSERT(conf1.getDesiredCommitProxies() == conf2.getDesiredCommitProxies()); + + return Void(); +} \ No newline at end of file diff --git a/fdbserver/ClusterRecovery.actor.cpp b/fdbserver/ClusterRecovery.actor.cpp index 22b6807a00..180673f850 100644 --- a/fdbserver/ClusterRecovery.actor.cpp +++ b/fdbserver/ClusterRecovery.actor.cpp @@ -1477,12 +1477,12 @@ ACTOR Future clusterRecoveryCore(Reference self) { recoverAndEndEpoch.cancel(); - ASSERT(self->commitProxies.size() <= self->configuration.getDesiredCommitProxies()); - ASSERT(self->commitProxies.size() >= 1); - ASSERT(self->grvProxies.size() <= self->configuration.getDesiredGrvProxies()); - ASSERT(self->grvProxies.size() >= 1); - ASSERT(self->resolvers.size() <= self->configuration.getDesiredResolvers()); - ASSERT(self->resolvers.size() >= 1); + ASSERT_LE(self->commitProxies.size(), self->configuration.getDesiredCommitProxies()); + ASSERT_GE(self->commitProxies.size(), 1); + ASSERT_LE(self->grvProxies.size(), self->configuration.getDesiredGrvProxies()); + ASSERT_GE(self->grvProxies.size(), 1); + ASSERT_LE(self->resolvers.size(), self->configuration.getDesiredResolvers()); + ASSERT_GE(self->resolvers.size(), 1); self->recoveryState = RecoveryState::RECOVERY_TRANSACTION; TraceEvent(getRecoveryEventName(ClusterRecoveryEventType::CLUSTER_RECOVERY_STATE_EVENT_NAME).c_str(), self->dbgid) From 017709aec64e73665642e6c6dab497a7c2d67321 Mon Sep 17 00:00:00 2001 From: Ata E Husain Bohra Date: Thu, 24 Mar 2022 07:31:49 -0700 Subject: [PATCH 407/413] Introduce BlobCipher interface and cipher caching interface (#6391) * Introduce BlobCipher interface and cipher caching interface diff-3: Update the code to avoid deriving encryption key periodically. Implement EncyrptBuf interface to limit memcpys. Improve both unit test and simulation to better code coverage. diff-2: Add specific error code for OpenSSL AES call failures diff-1: Update encryption scheme to AES-256-CTR. Minor updates to Header to capture more information. Major changes proposed are: 1. Introduce encyrption header format. 2. Introduce a BlobCipher cipher key representation encoding following information: baseCipher details, derived encryption cipher details, creationTime and random salt. 3. Introduce interface to support block cipher encrytion and decrytion operations. Encyrption populates encryption header allowing client to persist them on-disk, this header is then read allowing decryption on reads. 4. Introduce interface to allow in-memory caching of cipher keys. The cache allowing mapping of "encryption domain" -> "base cipher id" -> "derived cipher keys" (3D hash map). This cache interface will be used by FDB processes participating in encryption to cache recently used ciphers (performance optimization). Testing: 1. Unit test to validate caching interface. 2. Update EncryptionOps simulation test to validate block cipher operations. --- fdbclient/ServerKnobs.cpp | 1 + fdbclient/ServerKnobs.h | 3 +- .../workloads/ConsistencyCheck.actor.cpp | 4 +- fdbserver/workloads/EncryptionOps.actor.cpp | 215 ++++-- fdbserver/workloads/UnitTests.actor.cpp | 2 + flow/BlobCipher.cpp | 652 ++++++++++++++++++ flow/BlobCipher.h | 321 +++++++++ flow/CMakeLists.txt | 2 + flow/Platform.actor.cpp | 2 + flow/error_definitions.h | 9 + 10 files changed, 1160 insertions(+), 51 deletions(-) create mode 100644 flow/BlobCipher.cpp create mode 100644 flow/BlobCipher.h diff --git a/fdbclient/ServerKnobs.cpp b/fdbclient/ServerKnobs.cpp index 0951498d52..f53efac786 100644 --- a/fdbclient/ServerKnobs.cpp +++ b/fdbclient/ServerKnobs.cpp @@ -828,6 +828,7 @@ void ServerKnobs::initialize(Randomize randomize, ClientKnobs* clientKnobs, IsSi // encrypt key proxy init( ENABLE_ENCRYPTION, false ); + init( ENCRYPTION_MODE, "AES-256-CTR"); // Blob granlues init( BG_URL, isSimulated ? "file://fdbblob/" : "" ); // TODO: store in system key space or something, eventually diff --git a/fdbclient/ServerKnobs.h b/fdbclient/ServerKnobs.h index d120ed3986..de69ef43dc 100644 --- a/fdbclient/ServerKnobs.h +++ b/fdbclient/ServerKnobs.h @@ -775,8 +775,9 @@ public: // Cluster recovery std::string CLUSTER_RECOVERY_EVENT_NAME_PREFIX; - // encrypt key proxy + // Encryption bool ENABLE_ENCRYPTION; + std::string ENCRYPTION_MODE; // blob granule stuff // FIXME: configure url with database configuration instead of knob eventually diff --git a/fdbserver/workloads/ConsistencyCheck.actor.cpp b/fdbserver/workloads/ConsistencyCheck.actor.cpp index 27a8e6f4cf..16d39a7f1e 100644 --- a/fdbserver/workloads/ConsistencyCheck.actor.cpp +++ b/fdbserver/workloads/ConsistencyCheck.actor.cpp @@ -2378,9 +2378,9 @@ struct ConsistencyCheckWorkload : TestWorkload { (!nonExcludedWorkerProcessMap.count(db.encryptKeyProxy.get().address()) || nonExcludedWorkerProcessMap[db.encryptKeyProxy.get().address()].processClass.machineClassFitness( ProcessClass::EncryptKeyProxy) > fitnessLowerBound)) { - TraceEvent("ConsistencyCheck_EncyrptKeyProxyNotBest") + TraceEvent("ConsistencyCheck_EncryptKeyProxyNotBest") .detail("BestEncryptKeyProxyFitness", fitnessLowerBound) - .detail("ExistingEncyrptKeyProxyFitness", + .detail("ExistingEncryptKeyProxyFitness", nonExcludedWorkerProcessMap.count(db.encryptKeyProxy.get().address()) ? nonExcludedWorkerProcessMap[db.encryptKeyProxy.get().address()] .processClass.machineClassFitness(ProcessClass::EncryptKeyProxy) diff --git a/fdbserver/workloads/EncryptionOps.actor.cpp b/fdbserver/workloads/EncryptionOps.actor.cpp index cad228dfa0..487491048b 100644 --- a/fdbserver/workloads/EncryptionOps.actor.cpp +++ b/fdbserver/workloads/EncryptionOps.actor.cpp @@ -21,16 +21,19 @@ #include "fdbclient/DatabaseContext.h" #include "fdbclient/NativeAPI.actor.h" #include "flow/IRandom.h" -#include "flow/StreamCipher.h" +#include "flow/BlobCipher.h" #include "fdbserver/workloads/workloads.actor.h" +#include "flow/ITrace.h" #include "flow/Trace.h" -#include "flow/actorcompiler.h" // This must be the last #include. - -#if ENCRYPTION_ENABLED #include #include #include +#include + +#include "flow/actorcompiler.h" // This must be the last #include. + +#if ENCRYPTION_ENABLED #define MEGA_BYTES (1024 * 1024) #define NANO_SECOND (1000 * 1000 * 1000) @@ -78,45 +81,69 @@ struct WorkloadMetrics { } }; +// Workload generator for encryption/decryption operations. +// 1. For every client run, it generate unique random encryptionDomainId range and simulate encryption of +// either fixed size or variable size payload. +// 2. For each encryption run, it would interact with BlobCipherKeyCache to fetch the desired encryption key, +// which then is used for encrypting the plaintext payload. +// 3. Encryption operation generates 'encryption header', it is leveraged to decrypt the ciphertext obtained from +// step#2 (simulate real-world scenario) +// +// Correctness validations: +// ----------------------- +// Correctness invariants are validated at various steps: +// 1. Encryption key correctness: as part of performing decryption, BlobCipherKeyCache lookup is done to procure +// desired encrytion key based on: {encryptionDomainId, baseCipherId}; the obtained key is validated against +// the encryption key used for encrypting the data. +// 2. After encryption, generated 'encryption header' fields are validated, encrypted buffer size and contents are +// validated. +// 3. After decryption, the obtained deciphertext is validated against the orginal plaintext payload. +// +// Performance metrics: +// ------------------- +// The workload generator profiles below operations across the iterations and logs the details at the end, they are: +// 1. Time spent in encryption key fetch (and derivation) operations. +// 2. Time spent encrypting the buffer (doesn't incude key lookup time); also records the throughput in MB/sec. +// 3. Time spent decrypting the buffer (doesn't incude key lookup time); also records the throughput in MB/sec. + struct EncryptionOpsWorkload : TestWorkload { int mode; int64_t numIterations; int pageSize; int maxBufSize; std::unique_ptr buff; - std::unique_ptr validationBuff; - StreamCipher::IV iv; - std::unique_ptr hmacGenerator; - std::unique_ptr parentKey; Arena arena; std::unique_ptr metrics; + BlobCipherDomainId minDomainId; + BlobCipherDomainId maxDomainId; + BlobCipherBaseKeyId minBaseCipherId; + EncryptionOpsWorkload(WorkloadContext const& wcx) : TestWorkload(wcx) { mode = getOption(options, LiteralStringRef("fixedSize"), 1); numIterations = getOption(options, LiteralStringRef("numIterations"), 10); pageSize = getOption(options, LiteralStringRef("pageSize"), 4096); maxBufSize = getOption(options, LiteralStringRef("maxBufSize"), 512 * 1024); buff = std::make_unique(maxBufSize); - validationBuff = std::make_unique(maxBufSize); - iv = getRandomIV(); - hmacGenerator = std::make_unique(); - parentKey = std::make_unique(AES_256_KEY_LENGTH); - generateRandomData(parentKey.get(), AES_256_KEY_LENGTH); + // assign unique encryptionDomainId range per workload clients + minDomainId = wcx.clientId * 100 + mode * 30 + 1; + maxDomainId = deterministicRandom()->randomInt(minDomainId, minDomainId + 10) + 5; + minBaseCipherId = 100; metrics = std::make_unique(); - TraceEvent("EncryptionOpsWorkload").detail("Mode", getModeStr()); + TraceEvent("EncryptionOpsWorkload") + .detail("Mode", getModeStr()) + .detail("MinDomainId", minDomainId) + .detail("MaxDomainId", maxDomainId); } + ~EncryptionOpsWorkload() { TraceEvent("EncryptionOpsWorkload_Done").log(); } + bool isFixedSizePayload() { return mode == 1; } - StreamCipher::IV getRandomIV() { - generateRandomData(iv.data(), iv.size()); - return iv; - } - std::string getModeStr() const { if (mode == 1) { return "FixedSize"; @@ -127,47 +154,97 @@ struct EncryptionOpsWorkload : TestWorkload { throw internal_error(); } - void updateEncryptionKey(StreamCipherKey* cipherKey) { - auto start = std::chrono::high_resolution_clock::now(); - applyHmacKeyDerivationFunc(cipherKey, hmacGenerator.get(), arena); - auto end = std::chrono::high_resolution_clock::now(); - - metrics->updateKeyDerivationTime(std::chrono::duration(end - start).count()); + void generateRandomBaseCipher(const int maxLen, uint8_t* buff, int* retLen) { + memset(buff, 0, maxLen); + *retLen = deterministicRandom()->randomInt(maxLen / 2, maxLen); + generateRandomData(buff, *retLen); } - StringRef doEncryption(const StreamCipherKey* key, uint8_t* payload, int len) { - EncryptionStreamCipher encryptor(key, iv); + void setupCipherEssentials() { + auto& cipherKeyCache = BlobCipherKeyCache::getInstance(); + + TraceEvent("SetupCipherEssentials_Start").detail("MinDomainId", minDomainId).detail("MaxDomainId", maxDomainId); + + uint8_t buff[AES_256_KEY_LENGTH]; + std::vector> cipherKeys; + for (BlobCipherDomainId id = minDomainId; id <= maxDomainId; id++) { + int cipherLen = 0; + generateRandomBaseCipher(AES_256_KEY_LENGTH, &buff[0], &cipherLen); + cipherKeyCache.insertCipherKey(id, minBaseCipherId, buff, cipherLen); + + ASSERT(cipherLen > 0 && cipherLen <= AES_256_KEY_LENGTH); + + cipherKeys = cipherKeyCache.getAllCiphers(id); + ASSERT(cipherKeys.size() == 1); + } + + TraceEvent("SetupCipherEssentials_Done").detail("MinDomainId", minDomainId).detail("MaxDomainId", maxDomainId); + } + + void resetCipherEssentials() { + auto& cipherKeyCache = BlobCipherKeyCache::getInstance(); + cipherKeyCache.cleanup(); + + TraceEvent("ResetCipherEssentials_Done").log(); + } + + void updateLatestBaseCipher(const BlobCipherDomainId encryptDomainId, + uint8_t* baseCipher, + int* baseCipherLen, + BlobCipherBaseKeyId* nextBaseCipherId) { + auto& cipherKeyCache = BlobCipherKeyCache::getInstance(); + Reference cipherKey = cipherKeyCache.getLatestCipherKey(encryptDomainId); + *nextBaseCipherId = cipherKey->getBaseCipherId() + 1; + + generateRandomBaseCipher(AES_256_KEY_LENGTH, baseCipher, baseCipherLen); + + ASSERT(*baseCipherLen > 0 && *baseCipherLen <= AES_256_KEY_LENGTH); + TraceEvent("UpdateBaseCipher").detail("DomainId", encryptDomainId).detail("BaseCipherId", *nextBaseCipherId); + } + + Reference doEncryption(Reference key, + uint8_t* payload, + int len, + BlobCipherEncryptHeader* header) { + uint8_t iv[AES_256_IV_LENGTH]; + generateRandomData(&iv[0], AES_256_IV_LENGTH); + EncryptBlobCipherAes265Ctr encryptor(key, &iv[0], AES_256_IV_LENGTH); auto start = std::chrono::high_resolution_clock::now(); - auto encrypted = encryptor.encrypt(buff.get(), len, arena); - encryptor.finish(arena); + Reference encrypted = encryptor.encrypt(payload, len, header, arena); auto end = std::chrono::high_resolution_clock::now(); // validate encrypted buffer size and contents (not matching with plaintext) - ASSERT(encrypted.size() == len); - std::copy(encrypted.begin(), encrypted.end(), validationBuff.get()); - ASSERT(memcmp(validationBuff.get(), buff.get(), len) != 0); + ASSERT(encrypted->getLogicalSize() == len); + ASSERT(memcmp(encrypted->begin(), payload, len) != 0); + ASSERT(header->flags.headerVersion == EncryptBlobCipherAes265Ctr::ENCRYPT_HEADER_VERSION); metrics->updateEncryptionTime(std::chrono::duration(end - start).count()); return encrypted; } - void doDecryption(const StreamCipherKey* key, - const StringRef& encrypted, + void doDecryption(Reference encrypted, int len, + const BlobCipherEncryptHeader& header, uint8_t* originalPayload, - uint8_t* validationBuff) { - DecryptionStreamCipher decryptor(key, iv); + Reference orgCipherKey) { + ASSERT(header.flags.headerVersion == EncryptBlobCipherAes265Ctr::ENCRYPT_HEADER_VERSION); + ASSERT(header.flags.encryptMode == BLOB_CIPHER_ENCRYPT_MODE_AES_256_CTR); + + auto& cipherKeyCache = BlobCipherKeyCache::getInstance(); + Reference cipherKey = cipherKeyCache.getCipherKey(header.encryptDomainId, header.baseCipherId); + ASSERT(cipherKey.isValid()); + ASSERT(cipherKey->isEqual(orgCipherKey)); + + DecryptBlobCipherAes256Ctr decryptor(cipherKey, &header.iv[0]); auto start = std::chrono::high_resolution_clock::now(); - Standalone decrypted = decryptor.decrypt(encrypted.begin(), len, arena); - decryptor.finish(arena); + Reference decrypted = decryptor.decrypt(encrypted->begin(), len, header, arena); auto end = std::chrono::high_resolution_clock::now(); // validate decrypted buffer size and contents (matching with original plaintext) - ASSERT(decrypted.size() == len); - std::copy(decrypted.begin(), decrypted.end(), validationBuff); - ASSERT(memcmp(validationBuff, originalPayload, len) == 0); + ASSERT(decrypted->getLogicalSize() == len); + ASSERT(memcmp(decrypted->begin(), originalPayload, len) == 0); metrics->updateDecryptionTime(std::chrono::duration(end - start).count()); } @@ -177,22 +254,64 @@ struct EncryptionOpsWorkload : TestWorkload { std::string description() const override { return "EncryptionOps"; } Future start(Database const& cx) override { + uint8_t baseCipher[AES_256_KEY_LENGTH]; + int baseCipherLen = 0; + BlobCipherBaseKeyId nextBaseCipherId; + + // Setup encryptDomainIds and corresponding baseCipher details + setupCipherEssentials(); + for (int i = 0; i < numIterations; i++) { - StreamCipherKey key(AES_256_KEY_LENGTH); - // derive the encryption key - updateEncryptionKey(&key); + bool updateBaseCipher = deterministicRandom()->randomInt(1, 100) < 5; + + // Step-1: Encryption key derivation, caching the cipher for later use + auto& cipherKeyCache = BlobCipherKeyCache::getInstance(); + + // randomly select a domainId + const BlobCipherDomainId encryptDomainId = deterministicRandom()->randomInt(minDomainId, maxDomainId); + ASSERT(encryptDomainId >= minDomainId && encryptDomainId <= maxDomainId); + + if (updateBaseCipher) { + // simulate baseCipherId getting refreshed/updated + updateLatestBaseCipher(encryptDomainId, &baseCipher[0], &baseCipherLen, &nextBaseCipherId); + cipherKeyCache.insertCipherKey(encryptDomainId, nextBaseCipherId, &baseCipher[0], baseCipherLen); + } + + auto start = std::chrono::high_resolution_clock::now(); + Reference cipherKey = cipherKeyCache.getLatestCipherKey(encryptDomainId); + auto end = std::chrono::high_resolution_clock::now(); + metrics->updateKeyDerivationTime(std::chrono::duration(end - start).count()); + + // Validate sanity of "getLatestCipher", especially when baseCipher gets updated + if (updateBaseCipher) { + ASSERT(cipherKey->getBaseCipherId() == nextBaseCipherId); + ASSERT(cipherKey->getBaseCipherLen() == baseCipherLen); + ASSERT(memcmp(cipherKey->rawBaseCipher(), baseCipher, baseCipherLen) == 0); + } int dataLen = isFixedSizePayload() ? pageSize : deterministicRandom()->randomInt(100, maxBufSize); generateRandomData(buff.get(), dataLen); - // encrypt the payload - const auto& encrypted = doEncryption(&key, buff.get(), dataLen); + // Encrypt the payload - generates BlobCipherEncryptHeader to assist decryption later + BlobCipherEncryptHeader header; + try { + Reference encrypted = doEncryption(cipherKey, buff.get(), dataLen, &header); - // decrypt the payload - doDecryption(&key, encrypted, dataLen, buff.get(), validationBuff.get()); + // Decrypt the payload - parses the BlobCipherEncryptHeader, fetch corresponding cipherKey and + // decrypt + doDecryption(encrypted, dataLen, header, buff.get(), cipherKey); + } catch (Error& e) { + TraceEvent("Failed") + .detail("DomainId", encryptDomainId) + .detail("BaseCipherId", cipherKey->getBaseCipherId()); + throw; + } metrics->updateBytes(dataLen); } + + // Cleanup cipherKeys + resetCipherEssentials(); return Void(); } diff --git a/fdbserver/workloads/UnitTests.actor.cpp b/fdbserver/workloads/UnitTests.actor.cpp index f337b96514..3b2bf42fd7 100644 --- a/fdbserver/workloads/UnitTests.actor.cpp +++ b/fdbserver/workloads/UnitTests.actor.cpp @@ -30,6 +30,7 @@ void forceLinkMemcpyTests(); void forceLinkMemcpyPerfTests(); #if (!defined(TLS_DISABLED) && !defined(_WIN32)) void forceLinkStreamCipherTests(); +void forceLinkBLockCiherTests(); #endif void forceLinkParallelStreamTests(); void forceLinkSimExternalConnectionTests(); @@ -76,6 +77,7 @@ struct UnitTestWorkload : TestWorkload { forceLinkMemcpyPerfTests(); #if (!defined(TLS_DISABLED) && !defined(_WIN32)) forceLinkStreamCipherTests(); + void forceLinkBlobCipherTests(); #endif forceLinkParallelStreamTests(); forceLinkSimExternalConnectionTests(); diff --git a/flow/BlobCipher.cpp b/flow/BlobCipher.cpp new file mode 100644 index 0000000000..a909b783a6 --- /dev/null +++ b/flow/BlobCipher.cpp @@ -0,0 +1,652 @@ +/* + * BlobCipher.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 "flow/BlobCipher.h" +#include "flow/Error.h" +#include "flow/FastRef.h" +#include "flow/IRandom.h" +#include "flow/ITrace.h" +#include "flow/network.h" +#include "flow/Trace.h" +#include "flow/UnitTest.h" + +#include +#include + +#if ENCRYPTION_ENABLED + +// BlobCipherEncryptHeader +BlobCipherEncryptHeader::BlobCipherEncryptHeader() { + flags.encryptMode = BLOB_CIPHER_ENCRYPT_MODE_NONE; +} + +// BlobCipherKey class methods + +BlobCipherKey::BlobCipherKey(const BlobCipherDomainId& domainId, + const BlobCipherBaseKeyId& baseCiphId, + const uint8_t* baseCiph, + int baseCiphLen) { + BlobCipherRandomSalt salt; + if (g_network->isSimulated()) { + salt = deterministicRandom()->randomUInt64(); + } else { + salt = nondeterministicRandom()->randomUInt64(); + } + initKey(domainId, baseCiph, baseCiphLen, baseCiphId, salt); + /*TraceEvent("BlobCipherKey") + .detail("DomainId", domainId) + .detail("BaseCipherId", baseCipherId) + .detail("BaseCipherLen", baseCipherLen) + .detail("RandomSalt", randomSalt) + .detail("CreationTime", creationTime);*/ +} + +void BlobCipherKey::initKey(const BlobCipherDomainId& domainId, + const uint8_t* baseCiph, + int baseCiphLen, + const BlobCipherBaseKeyId& baseCiphId, + const BlobCipherRandomSalt& salt) { + // Set the base encryption key properties + baseCipher = std::make_unique(AES_256_KEY_LENGTH); + memset(baseCipher.get(), 0, AES_256_KEY_LENGTH); + memcpy(baseCipher.get(), baseCiph, std::min(baseCiphLen, AES_256_KEY_LENGTH)); + baseCipherLen = baseCiphLen; + baseCipherId = baseCiphId; + // Set the encryption domain for the base encryption key + encryptDomainId = domainId; + randomSalt = salt; + // derive the encryption key + cipher = std::make_unique(AES_256_KEY_LENGTH); + memset(cipher.get(), 0, AES_256_KEY_LENGTH); + applyHmacSha256Derivation(); + // update the key creation time + creationTime = now(); +} + +void BlobCipherKey::applyHmacSha256Derivation() { + Arena arena; + uint8_t buf[baseCipherLen + sizeof(BlobCipherRandomSalt)]; + memcpy(&buf[0], baseCipher.get(), baseCipherLen); + memcpy(&buf[0] + baseCipherLen, &randomSalt, sizeof(BlobCipherRandomSalt)); + HmacSha256DigestGen hmacGen(baseCipher.get(), baseCipherLen); + StringRef digest = hmacGen.digest(&buf[0], baseCipherLen + sizeof(BlobCipherRandomSalt), arena); + std::copy(digest.begin(), digest.end(), cipher.get()); + if (digest.size() < AES_256_KEY_LENGTH) { + memcpy(cipher.get() + digest.size(), buf, AES_256_KEY_LENGTH - digest.size()); + } +} + +void BlobCipherKey::reset() { + memset(baseCipher.get(), 0, baseCipherLen); + memset(cipher.get(), 0, AES_256_KEY_LENGTH); +} + +// BlobKeyIdCache class methods + +BlobCipherKeyIdCache::BlobCipherKeyIdCache() + : domainId(INVALID_DOMAIN_ID), latestBaseCipherKeyId(INVALID_CIPHER_KEY_ID) {} + +BlobCipherKeyIdCache::BlobCipherKeyIdCache(BlobCipherDomainId dId) + : domainId(dId), latestBaseCipherKeyId(INVALID_CIPHER_KEY_ID) { + TraceEvent("Init_BlobCipherKeyIdCache").detail("DomainId", domainId); +} + +Reference BlobCipherKeyIdCache::getLatestCipherKey() { + return getCipherByBaseCipherId(latestBaseCipherKeyId); +} + +Reference BlobCipherKeyIdCache::getCipherByBaseCipherId(BlobCipherBaseKeyId baseCipherKeyId) { + BlobCipherKeyIdCacheMapCItr itr = keyIdCache.find(baseCipherKeyId); + if (itr == keyIdCache.end()) { + throw encrypt_key_not_found(); + } + return itr->second; +} + +void BlobCipherKeyIdCache::insertBaseCipherKey(BlobCipherBaseKeyId baseCipherId, + const uint8_t* baseCipher, + int baseCipherLen) { + ASSERT(baseCipherId > INVALID_CIPHER_KEY_ID); + + // BaseCipherKeys are immutable, ensure that cached value doesn't get updated. + BlobCipherKeyIdCacheMapCItr itr = keyIdCache.find(baseCipherId); + if (itr != keyIdCache.end()) { + if (memcmp(itr->second->rawBaseCipher(), baseCipher, baseCipherLen) == 0) { + TraceEvent("InsertBaseCipherKey_AlreadyPresent") + .detail("BaseCipherKeyId", baseCipherId) + .detail("DomainId", domainId); + // Key is already present; nothing more to do. + return; + } else { + TraceEvent("InsertBaseCipherKey_UpdateCipher") + .detail("BaseCipherKeyId", baseCipherId) + .detail("DomainId", domainId); + throw encrypt_update_cipher(); + } + } + + keyIdCache.emplace(baseCipherId, makeReference(domainId, baseCipherId, baseCipher, baseCipherLen)); + // Update the latest BaseCipherKeyId for the given encryption domain + latestBaseCipherKeyId = baseCipherId; +} + +void BlobCipherKeyIdCache::cleanup() { + for (auto& keyItr : keyIdCache) { + keyItr.second->reset(); + } + + keyIdCache.clear(); +} + +std::vector> BlobCipherKeyIdCache::getAllCipherKeys() { + std::vector> cipherKeys; + for (auto& keyItr : keyIdCache) { + cipherKeys.push_back(keyItr.second); + } + return cipherKeys; +} + +// BlobCipherKeyCache class methods + +void BlobCipherKeyCache::insertCipherKey(const BlobCipherDomainId& domainId, + const BlobCipherBaseKeyId& baseCipherId, + const uint8_t* baseCipher, + int baseCipherLen) { + if (domainId == INVALID_DOMAIN_ID || baseCipherId == INVALID_CIPHER_KEY_ID) { + throw encrypt_invalid_id(); + } + + try { + auto domainItr = domainCacheMap.find(domainId); + if (domainItr == domainCacheMap.end()) { + // Add mapping to track new encryption domain + Reference keyIdCache = makeReference(domainId); + keyIdCache->insertBaseCipherKey(baseCipherId, baseCipher, baseCipherLen); + domainCacheMap.emplace(domainId, keyIdCache); + } else { + // Track new baseCipher keys + Reference keyIdCache = domainItr->second; + keyIdCache->insertBaseCipherKey(baseCipherId, baseCipher, baseCipherLen); + } + + TraceEvent("InsertCipherKey").detail("DomainId", domainId).detail("BaseCipherKeyId", baseCipherId); + } catch (Error& e) { + TraceEvent("InsertCipherKey_Failed").detail("BaseCipherKeyId", baseCipherId).detail("DomainId", domainId); + throw; + } +} + +Reference BlobCipherKeyCache::getLatestCipherKey(const BlobCipherDomainId& domainId) { + auto domainItr = domainCacheMap.find(domainId); + if (domainItr == domainCacheMap.end()) { + TraceEvent("GetLatestCipherKey_DomainNotFound").detail("DomainId", domainId); + throw encrypt_key_not_found(); + } + + Reference keyIdCache = domainItr->second; + Reference cipherKey = keyIdCache->getLatestCipherKey(); + if ((now() - cipherKey->getCreationTime()) > BlobCipherKeyCache::CIPHER_KEY_CACHE_TTL_SEC) { + TraceEvent("GetLatestCipherKey_ExpiredTTL") + .detail("DomainId", domainId) + .detail("BaseCipherId", cipherKey->getBaseCipherId()); + throw encrypt_key_ttl_expired(); + } + + return cipherKey; +} + +Reference BlobCipherKeyCache::getCipherKey(const BlobCipherDomainId& domainId, + const BlobCipherBaseKeyId& baseCipherId) { + auto domainItr = domainCacheMap.find(domainId); + if (domainItr == domainCacheMap.end()) { + throw encrypt_key_not_found(); + } + + Reference keyIdCache = domainItr->second; + return keyIdCache->getCipherByBaseCipherId(baseCipherId); +} + +void BlobCipherKeyCache::resetEncyrptDomainId(const BlobCipherDomainId domainId) { + auto domainItr = domainCacheMap.find(domainId); + if (domainItr == domainCacheMap.end()) { + throw encrypt_key_not_found(); + } + + Reference keyIdCache = domainItr->second; + keyIdCache->cleanup(); + TraceEvent("ResetEncryptDomainId").detail("DomainId", domainId); +} + +void BlobCipherKeyCache::cleanup() noexcept { + BlobCipherKeyCache& instance = BlobCipherKeyCache::getInstance(); + for (auto& domainItr : instance.domainCacheMap) { + Reference keyIdCache = domainItr.second; + keyIdCache->cleanup(); + TraceEvent("BlobCipherKeyCache_Cleanup").detail("DomainId", domainItr.first); + } + + instance.domainCacheMap.clear(); +} + +std::vector> BlobCipherKeyCache::getAllCiphers(const BlobCipherDomainId& domainId) { + auto domainItr = domainCacheMap.find(domainId); + if (domainItr == domainCacheMap.end()) { + return {}; + } + + Reference keyIdCache = domainItr->second; + return keyIdCache->getAllCipherKeys(); +} + +// EncryptBlobCipher class methods + +EncryptBlobCipherAes265Ctr::EncryptBlobCipherAes265Ctr(Reference key, + const uint8_t* cipherIV, + const int ivLen) + : ctx(EVP_CIPHER_CTX_new()), cipherKey(key) { + ASSERT(ivLen == AES_256_IV_LENGTH); + memcpy(&iv[0], cipherIV, ivLen); + + if (ctx == nullptr) { + throw encrypt_ops_error(); + } + if (EVP_EncryptInit_ex(ctx, EVP_aes_256_ctr(), nullptr, nullptr, nullptr) != 1) { + throw encrypt_ops_error(); + } + if (EVP_EncryptInit_ex(ctx, nullptr, nullptr, key.getPtr()->data(), cipherIV) != 1) { + throw encrypt_ops_error(); + } +} + +Reference EncryptBlobCipherAes265Ctr::encrypt(const uint8_t* plaintext, + const int plaintextLen, + BlobCipherEncryptHeader* header, + Arena& arena) { + TEST(true); // Encrypting data with BlobCipher + + Reference encryptBuf = makeReference(plaintextLen + AES_BLOCK_SIZE, arena); + uint8_t* ciphertext = encryptBuf->begin(); + int bytes{ 0 }; + if (EVP_EncryptUpdate(ctx, ciphertext, &bytes, plaintext, plaintextLen) != 1) { + TraceEvent("Encrypt_UpdateFailed") + .detail("BaseCipherId", cipherKey->getBaseCipherId()) + .detail("EncryptDomainId", cipherKey->getDomainId()); + throw encrypt_ops_error(); + } + + int finalBytes{ 0 }; + if (EVP_EncryptFinal_ex(ctx, ciphertext + bytes, &finalBytes) != 1) { + TraceEvent("Encrypt_FinalFailed") + .detail("BaseCipherId", cipherKey->getBaseCipherId()) + .detail("EncryptDomainId", cipherKey->getDomainId()); + throw encrypt_ops_error(); + } + + if ((bytes + finalBytes) != plaintextLen) { + TraceEvent("Encrypt_UnexpectedCipherLen") + .detail("PlaintextLen", plaintextLen) + .detail("EncryptedBufLen", bytes + finalBytes); + throw encrypt_ops_error(); + } + + // populate header details for the encrypted blob. + header->flags.size = sizeof(BlobCipherEncryptHeader); + header->flags.headerVersion = EncryptBlobCipherAes265Ctr::ENCRYPT_HEADER_VERSION; + header->flags.encryptMode = BLOB_CIPHER_ENCRYPT_MODE_AES_256_CTR; + header->baseCipherId = cipherKey->getBaseCipherId(); + header->encryptDomainId = cipherKey->getDomainId(); + header->salt = cipherKey->getSalt(); + memcpy(&header->iv[0], &iv[0], AES_256_IV_LENGTH); + + // Preserve checksum of encrypted bytes in the header; approach protects against disk induced bit-rot/flip + // scenarios. AES CTR mode doesn't generate 'tag' by default as with schemes such as: AES 256 GCM. + + header->ciphertextChecksum = computeEncryptChecksum(ciphertext, bytes + finalBytes, cipherKey->getSalt(), arena); + + encryptBuf->setLogicalSize(plaintextLen); + return encryptBuf; +} + +EncryptBlobCipherAes265Ctr::~EncryptBlobCipherAes265Ctr() { + if (ctx != nullptr) { + EVP_CIPHER_CTX_free(ctx); + } +} + +// DecryptBlobCipher class methods + +DecryptBlobCipherAes256Ctr::DecryptBlobCipherAes256Ctr(Reference key, const uint8_t* iv) + : ctx(EVP_CIPHER_CTX_new()) { + if (ctx == nullptr) { + throw encrypt_ops_error(); + } + if (!EVP_DecryptInit_ex(ctx, EVP_aes_256_ctr(), nullptr, nullptr, nullptr)) { + throw encrypt_ops_error(); + } + if (!EVP_DecryptInit_ex(ctx, nullptr, nullptr, key.getPtr()->data(), iv)) { + throw encrypt_ops_error(); + } +} + +void DecryptBlobCipherAes256Ctr::verifyEncryptBlobHeader(const uint8_t* ciphertext, + const int ciphertextLen, + const BlobCipherEncryptHeader& header, + Arena& arena) { + // validate header flag sanity + if (header.flags.headerVersion != EncryptBlobCipherAes265Ctr::ENCRYPT_HEADER_VERSION || + header.flags.encryptMode != BLOB_CIPHER_ENCRYPT_MODE_AES_256_CTR) { + TraceEvent("VerifyEncryptBlobHeader") + .detail("HeaderVersion", header.flags.headerVersion) + .detail("HeaderMode", header.flags.encryptMode) + .detail("ExpectedVersion", EncryptBlobCipherAes265Ctr::ENCRYPT_HEADER_VERSION) + .detail("ExpectedMode", BLOB_CIPHER_ENCRYPT_MODE_AES_256_CTR); + throw encrypt_header_metadata_mismatch(); + } + + // encrypted byte checksum sanity; protection against data bit-rot/flip. + BlobCipherChecksum computed = computeEncryptChecksum(ciphertext, ciphertextLen, header.salt, arena); + if (computed != header.ciphertextChecksum) { + TraceEvent("VerifyEncryptBlobHeader_ChecksumMismatch") + .detail("HeaderVersion", header.flags.headerVersion) + .detail("HeaderMode", header.flags.encryptMode) + .detail("CiphertextChecksum", header.ciphertextChecksum) + .detail("ComputedCiphertextChecksum", computed); + throw encrypt_header_checksum_mismatch(); + } +} + +Reference DecryptBlobCipherAes256Ctr::decrypt(const uint8_t* ciphertext, + const int ciphertextLen, + const BlobCipherEncryptHeader& header, + Arena& arena) { + TEST(true); // Decrypting data with BlobCipher + + verifyEncryptBlobHeader(ciphertext, ciphertextLen, header, arena); + + Reference decrypted = makeReference(ciphertextLen + AES_BLOCK_SIZE, arena); + uint8_t* plaintext = decrypted->begin(); + int bytesDecrypted{ 0 }; + if (!EVP_DecryptUpdate(ctx, plaintext, &bytesDecrypted, ciphertext, ciphertextLen)) { + TraceEvent("Decrypt_UpdateFailed") + .detail("BaseCipherId", header.baseCipherId) + .detail("EncryptDomainId", header.encryptDomainId); + throw encrypt_ops_error(); + } + + int finalBlobBytes{ 0 }; + if (EVP_DecryptFinal_ex(ctx, plaintext + bytesDecrypted, &finalBlobBytes) <= 0) { + TraceEvent("Decrypt_FinalFailed") + .detail("BaseCipherId", header.baseCipherId) + .detail("EncryptDomainId", header.encryptDomainId); + throw encrypt_ops_error(); + } + + if ((bytesDecrypted + finalBlobBytes) != ciphertextLen) { + TraceEvent("Encrypt_UnexpectedPlaintextLen") + .detail("CiphertextLen", ciphertextLen) + .detail("DecryptedBufLen", bytesDecrypted + finalBlobBytes); + throw encrypt_ops_error(); + } + + decrypted->setLogicalSize(ciphertextLen); + return decrypted; +} + +DecryptBlobCipherAes256Ctr::~DecryptBlobCipherAes256Ctr() { + if (ctx != nullptr) { + EVP_CIPHER_CTX_free(ctx); + } +} + +// HmacSha256DigestGen class methods + +HmacSha256DigestGen::HmacSha256DigestGen(const unsigned char* key, size_t len) : ctx(HMAC_CTX_new()) { + if (!HMAC_Init_ex(ctx, key, len, EVP_sha256(), nullptr)) { + throw encrypt_ops_error(); + } +} + +HmacSha256DigestGen::~HmacSha256DigestGen() { + if (ctx != nullptr) { + HMAC_CTX_free(ctx); + } +} + +StringRef HmacSha256DigestGen::digest(const unsigned char* data, size_t len, Arena& arena) { + TEST(true); // Digest generation + unsigned int digestLen = HMAC_size(ctx); + auto digest = new (arena) unsigned char[digestLen]; + if (HMAC_Update(ctx, data, len) != 1) { + throw encrypt_ops_error(); + } + + if (HMAC_Final(ctx, digest, &digestLen) != 1) { + throw encrypt_ops_error(); + } + return StringRef(digest, digestLen); +} + +// Only used to link unit tests +void forceLinkBlobCipherTests() {} + +// Tests cases includes: +// 1. Populate cache by inserting 'baseCipher' details for new encryptionDomainIds +// 2. Random lookup for cipherKeys and content validation +// 3. Inserting of 'identical' cipherKey (already cached) more than once works as desired. +// 4. Inserting of 'non-identical' cipherKey (already cached) more than once works as desired. +// 5. Validation encryption ops (correctness): +// 5.1. Encyrpt a buffer followed by decryption of the buffer, validate the contents. +// 5.2. Simulate anomolies such as: EncyrptionHeader corruption, checkSum mismatch / encryptionMode mismatch etc. +// 6. Cache cleanup +// 6.1 cleanup cipherKeys by given encryptDomainId +// 6.2. Cleanup all cached cipherKeys +TEST_CASE("flow/BlobCipher") { + TraceEvent("BlobCipherTest_Start").log(); + // Construct a dummy External Key Manager representation and populate with some keys + class BaseCipher : public ReferenceCounted, NonCopyable { + public: + BlobCipherDomainId domainId; + int len; + BlobCipherBaseKeyId keyId; + std::unique_ptr key; + + BaseCipher(const BlobCipherDomainId& dId, const BlobCipherBaseKeyId& kId) + : domainId(dId), len(deterministicRandom()->randomInt(AES_256_KEY_LENGTH / 2, AES_256_KEY_LENGTH + 1)), + keyId(kId), key(std::make_unique(len)) { + generateRandomData(key.get(), len); + } + }; + + using BaseKeyMap = std::unordered_map>; + using DomainKeyMap = std::unordered_map; + DomainKeyMap domainKeyMap; + const BlobCipherDomainId minDomainId = 1; + const BlobCipherDomainId maxDomainId = deterministicRandom()->randomInt(minDomainId, minDomainId + 10) + 5; + const BlobCipherBaseKeyId minBaseCipherKeyId = 100; + const BlobCipherBaseKeyId maxBaseCipherKeyId = + deterministicRandom()->randomInt(minBaseCipherKeyId, minBaseCipherKeyId + 50) + 15; + for (int dId = minDomainId; dId <= maxDomainId; dId++) { + for (int kId = minBaseCipherKeyId; kId <= maxBaseCipherKeyId; kId++) { + domainKeyMap[dId].emplace(kId, makeReference(dId, kId)); + } + } + ASSERT(domainKeyMap.size() == maxDomainId); + + // insert BlobCipher keys into BlobCipherKeyCache map and validate + TraceEvent("BlobCipherTest_InsertKeys").log(); + BlobCipherKeyCache& cipherKeyCache = BlobCipherKeyCache::getInstance(); + for (auto& domainItr : domainKeyMap) { + for (auto& baseKeyItr : domainItr.second) { + Reference baseCipher = baseKeyItr.second; + + cipherKeyCache.insertCipherKey( + baseCipher->domainId, baseCipher->keyId, baseCipher->key.get(), baseCipher->len); + } + } + TraceEvent("BlobCipherTest_InsertKeysDone").log(); + + // validate the cipherKey lookups work as desired + for (auto& domainItr : domainKeyMap) { + for (auto& baseKeyItr : domainItr.second) { + Reference baseCipher = baseKeyItr.second; + Reference cipherKey = cipherKeyCache.getCipherKey(baseCipher->domainId, baseCipher->keyId); + ASSERT(cipherKey.isValid()); + // validate common cipher properties - domainId, baseCipherId, baseCipherLen, rawBaseCipher + ASSERT(cipherKey->getBaseCipherId() == baseCipher->keyId); + ASSERT(cipherKey->getDomainId() == baseCipher->domainId); + ASSERT(cipherKey->getBaseCipherLen() == baseCipher->len); + // ensure that baseCipher matches with the cached information + ASSERT(std::memcmp(cipherKey->rawBaseCipher(), baseCipher->key.get(), cipherKey->getBaseCipherLen()) == 0); + // validate the encryption derivation + ASSERT(std::memcmp(cipherKey->rawCipher(), baseCipher->key.get(), cipherKey->getBaseCipherLen()) != 0); + } + } + TraceEvent("BlobCipherTest_LooksupDone").log(); + + // Ensure attemtping to insert existing cipherKey (identical) more than once is treated as a NOP + try { + Reference baseCipher = domainKeyMap[minDomainId][minBaseCipherKeyId]; + cipherKeyCache.insertCipherKey(baseCipher->domainId, baseCipher->keyId, baseCipher->key.get(), baseCipher->len); + } catch (Error& e) { + throw; + } + TraceEvent("BlobCipherTest_ReinsertIdempotentKeyDone").log(); + + // Ensure attemtping to insert an existing cipherKey (modified) fails with appropriate error + try { + Reference baseCipher = domainKeyMap[minDomainId][minBaseCipherKeyId]; + uint8_t rawCipher[baseCipher->len]; + memcpy(rawCipher, baseCipher->key.get(), baseCipher->len); + // modify few bytes in the cipherKey + for (int i = 2; i < 5; i++) { + rawCipher[i]++; + } + cipherKeyCache.insertCipherKey(baseCipher->domainId, baseCipher->keyId, &rawCipher[0], baseCipher->len); + } catch (Error& e) { + if (e.code() != error_code_encrypt_update_cipher) { + throw; + } + } + TraceEvent("BlobCipherTest_ReinsertNonIdempotentKeyDone").log(); + + // Validate Encyrption ops + Reference cipherKey = cipherKeyCache.getLatestCipherKey(minDomainId); + const int bufLen = deterministicRandom()->randomInt(786, 2127) + 512; + uint8_t orgData[bufLen]; + generateRandomData(&orgData[0], bufLen); + + Arena arena; + uint8_t iv[AES_256_IV_LENGTH]; + generateRandomData(&iv[0], AES_256_IV_LENGTH); + + // validate basic encrypt followed by decrypt operation + EncryptBlobCipherAes265Ctr encryptor(cipherKey, iv, AES_256_IV_LENGTH); + BlobCipherEncryptHeader header; + Reference encrypted = encryptor.encrypt(&orgData[0], bufLen, &header, arena); + + ASSERT(encrypted->getLogicalSize() == bufLen); + ASSERT(memcmp(&orgData[0], encrypted->begin(), bufLen) != 0); + ASSERT(header.flags.headerVersion == EncryptBlobCipherAes265Ctr::ENCRYPT_HEADER_VERSION); + ASSERT(header.flags.encryptMode == BLOB_CIPHER_ENCRYPT_MODE_AES_256_CTR); + + TraceEvent("BlobCipherTest_EncryptDone") + .detail("HeaderVersion", header.flags.headerVersion) + .detail("HeaderEncryptMode", header.flags.encryptMode) + .detail("DomainId", header.encryptDomainId) + .detail("BaseCipherId", header.baseCipherId) + .detail("HeaderChecksum", header.ciphertextChecksum); + + Reference encyrptKey = cipherKeyCache.getCipherKey(header.encryptDomainId, header.baseCipherId); + ASSERT(encyrptKey->isEqual(cipherKey)); + DecryptBlobCipherAes256Ctr decryptor(encyrptKey, &header.iv[0]); + Reference decrypted = decryptor.decrypt(encrypted->begin(), bufLen, header, arena); + + ASSERT(decrypted->getLogicalSize() == bufLen); + ASSERT(memcmp(decrypted->begin(), &orgData[0], bufLen) == 0); + + TraceEvent("BlobCipherTest_DecryptDone").log(); + + // induce encryption header corruption - headerVersion corrupted + header.flags.headerVersion += 1; + try { + decrypted = decryptor.decrypt(encrypted->begin(), bufLen, header, arena); + } catch (Error& e) { + if (e.code() != error_code_encrypt_header_metadata_mismatch) { + throw; + } + header.flags.headerVersion -= 1; + } + + // induce encryption header corruption - encryptionMode corrupted + header.flags.encryptMode += 1; + try { + decrypted = decryptor.decrypt(encrypted->begin(), bufLen, header, arena); + } catch (Error& e) { + if (e.code() != error_code_encrypt_header_metadata_mismatch) { + throw; + } + header.flags.encryptMode -= 1; + } + + // induce encryption header corruption - checksum mismatch + header.ciphertextChecksum += 1; + try { + decrypted = decryptor.decrypt(encrypted->begin(), bufLen, header, arena); + } catch (Error& e) { + if (e.code() != error_code_encrypt_header_checksum_mismatch) { + throw; + } + header.ciphertextChecksum -= 1; + } + + // Validate dropping encyrptDomainId cached keys + const BlobCipherDomainId candidate = deterministicRandom()->randomInt(minDomainId, maxDomainId); + cipherKeyCache.resetEncyrptDomainId(candidate); + std::vector> cachedKeys = cipherKeyCache.getAllCiphers(candidate); + ASSERT(cachedKeys.empty()); + + // Validate dropping all cached cipherKeys + cipherKeyCache.cleanup(); + for (int dId = minDomainId; dId < maxDomainId; dId++) { + std::vector> cachedKeys = cipherKeyCache.getAllCiphers(dId); + ASSERT(cachedKeys.empty()); + } + + TraceEvent("BlobCipherTest_Done").log(); + return Void(); +} + +BlobCipherChecksum computeEncryptChecksum(const uint8_t* payload, + const int payloadLen, + const BlobCipherRandomSalt& salt, + Arena& arena) { + // FIPS compliance recommendation is to leverage cryptographic digest mechanism to generate checksum + // Leverage HMAC_SHA256 using header.randomSalt as the initialization 'key' for the hmac digest. + + HmacSha256DigestGen hmacGenerator((const uint8_t*)&salt, sizeof(salt)); + StringRef digest = hmacGenerator.digest(payload, payloadLen, arena); + ASSERT(digest.size() >= sizeof(BlobCipherChecksum)); + + BlobCipherChecksum checksum; + memcpy((uint8_t*)&checksum, digest.begin(), sizeof(BlobCipherChecksum)); + return checksum; +} + +#endif // ENCRYPTION_ENABLED diff --git a/flow/BlobCipher.h b/flow/BlobCipher.h new file mode 100644 index 0000000000..151e60efd0 --- /dev/null +++ b/flow/BlobCipher.h @@ -0,0 +1,321 @@ +/* + * BlobCipher.h + * + * This source file is part of the FoundationDB open source project + * + * Copyright 2013-2022 Apple Inc. and the FoundationDB project authors + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +#pragma once + +#include +#include +#include +#include + +#if (!defined(TLS_DISABLED) && !defined(_WIN32)) +#define ENCRYPTION_ENABLED 1 +#else +#define ENCRYPTION_ENABLED 0 +#endif + +#if ENCRYPTION_ENABLED + +#include "flow/Arena.h" +#include "flow/FastRef.h" +#include "flow/flow.h" +#include "flow/xxhash.h" + +#include +#include +#include +#include +#include + +#define AES_256_KEY_LENGTH 32 +#define AES_256_IV_LENGTH 16 +#define INVALID_DOMAIN_ID 0 +#define INVALID_CIPHER_KEY_ID 0 + +using BlobCipherDomainId = uint64_t; +using BlobCipherRandomSalt = uint64_t; +using BlobCipherBaseKeyId = uint64_t; +using BlobCipherChecksum = uint64_t; + +typedef enum { BLOB_CIPHER_ENCRYPT_MODE_NONE = 0, BLOB_CIPHER_ENCRYPT_MODE_AES_256_CTR = 1 } BlockCipherEncryptMode; + +// Encryption operations buffer management +// Approach limits number of copies needed during encryption or decryption operations. +// For encryption EncryptBuf is allocated using client supplied Arena and provided to AES library to capture +// the ciphertext. Similarly, on decryption EncryptBuf is allocated using client supplied Arena and provided +// to the AES library to capture decipher text and passed back to the clients. Given the object passed around +// is reference-counted, it gets freed once refrenceCount goes to 0. + +class EncryptBuf : public ReferenceCounted, NonCopyable { +public: + EncryptBuf(int size, Arena& arena) : allocSize(size), logicalSize(size) { + if (size > 0) { + buffer = new (arena) uint8_t[size]; + } else { + buffer = nullptr; + } + } + + int getLogicalSize() { return logicalSize; } + void setLogicalSize(int value) { + ASSERT(value <= allocSize); + logicalSize = value; + } + uint8_t* begin() { return buffer; } + +private: + int allocSize; + int logicalSize; + uint8_t* buffer; +}; + +// BlobCipher Encryption header format +// This header is persisted along with encrypted buffer, it contains information necessary +// to assist decrypting the buffers to serve read requests. +// +// The total space overhead is 56 bytes. + +#pragma pack(push, 1) // exact fit - no padding +typedef struct BlobCipherEncryptHeader { + union { + struct { + uint8_t size; // reading first byte is sufficient to determine header + // length. ALWAYS THE FIRST HEADER ELEMENT. + uint8_t headerVersion{}; + uint8_t encryptMode{}; + uint8_t _reserved[5]{}; + } flags; + uint64_t _padding{}; + }; + // Encyrption domain boundary identifier. + BlobCipherDomainId encryptDomainId{}; + // BaseCipher encryption key identifier + BlobCipherBaseKeyId baseCipherId{}; + // Random salt + BlobCipherRandomSalt salt{}; + // Checksum of the encrypted buffer. It protects against 'tampering' of ciphertext as well 'bit rots/flips'. + BlobCipherChecksum ciphertextChecksum{}; + // Initialization vector used to encrypt the payload. + uint8_t iv[AES_256_IV_LENGTH]; + + BlobCipherEncryptHeader(); +} BlobCipherEncryptHeader; +#pragma pack(pop) + +// This interface is in-memory representation of CipherKey used for encryption/decryption information. +// It caches base encryption key properties as well as caches the 'derived encryption' key obtained by applying +// HMAC-SHA-256 derivation technique. + +class BlobCipherKey : public ReferenceCounted, NonCopyable { +public: + BlobCipherKey(const BlobCipherDomainId& domainId, + const BlobCipherBaseKeyId& baseCiphId, + const uint8_t* baseCiph, + int baseCiphLen); + + uint8_t* data() const { return cipher.get(); } + uint64_t getCreationTime() const { return creationTime; } + BlobCipherDomainId getDomainId() const { return encryptDomainId; } + BlobCipherRandomSalt getSalt() const { return randomSalt; } + BlobCipherBaseKeyId getBaseCipherId() const { return baseCipherId; } + int getBaseCipherLen() const { return baseCipherLen; } + uint8_t* rawCipher() const { return cipher.get(); } + uint8_t* rawBaseCipher() const { return baseCipher.get(); } + bool isEqual(const Reference toCompare) { + return encryptDomainId == toCompare->getDomainId() && baseCipherId == toCompare->getBaseCipherId() && + randomSalt == toCompare->getSalt() && baseCipherLen == toCompare->getBaseCipherLen() && + memcmp(cipher.get(), toCompare->rawCipher(), AES_256_KEY_LENGTH) == 0 && + memcmp(baseCipher.get(), toCompare->rawBaseCipher(), baseCipherLen) == 0; + } + void reset(); + +private: + // Encryption domain boundary identifier + BlobCipherDomainId encryptDomainId; + // Base encryption cipher key properties + std::unique_ptr baseCipher; + int baseCipherLen; + BlobCipherBaseKeyId baseCipherId; + // Random salt used for encryption cipher key derivation + BlobCipherRandomSalt randomSalt; + // Creation timestamp for the derived encryption cipher key + uint64_t creationTime; + // Derived encryption cipher key + std::unique_ptr cipher; + + void initKey(const BlobCipherDomainId& domainId, + const uint8_t* baseCiph, + int baseCiphLen, + const BlobCipherBaseKeyId& baseCiphId, + const BlobCipherRandomSalt& salt); + void applyHmacSha256Derivation(); +}; + +// This interface allows FDB processes participating in encryption to store and +// index recently used encyption cipher keys. FDB encryption has two dimensions: +// 1. Mapping on cipher encryption keys per "encryption domains" +// 2. Per encryption domain, the cipher keys are index using "baseCipherKeyId". +// +// The design supports NIST recommendation of limiting lifetime of an encryption +// key. For details refer to: +// https://csrc.nist.gov/publications/detail/sp/800-57-part-1/rev-3/archive/2012-07-10 +// +// Below gives a pictoral representation of in-memory datastructure implemented +// to index encryption keys: +// { encryptionDomain -> { baseCipherId -> cipherKey } } +// +// Supported cache lookups schemes: +// 1. Lookup cipher based on { encryptionDomainId, baseCipherKeyId } tuple. +// 2. Lookup latest cipher key for a given encryptionDomainId. +// +// Client is responsible to handle cache-miss usecase, the corrective operation +// might vary based on the calling process, for instance: EncryptKeyServer +// cache-miss shall invoke RPC to external Encryption Key Manager to fetch the +// required encryption key, however, CPs/SSs cache-miss would result in RPC to +// EncryptKeyServer to refresh the desired encryption key. + +using BlobCipherKeyIdCacheMap = std::unordered_map>; +using BlobCipherKeyIdCacheMapCItr = std::unordered_map>::const_iterator; + +struct BlobCipherKeyIdCache : ReferenceCounted { +public: + BlobCipherKeyIdCache(); + explicit BlobCipherKeyIdCache(BlobCipherDomainId dId); + + // API returns the last inserted cipherKey. + // If none exists, 'encrypt_key_not_found' is thrown. + Reference getLatestCipherKey(); + // API returns cipherKey corresponding to input 'baseCipherKeyId'. + // If none exists, 'encrypt_key_not_found' is thrown. + Reference getCipherByBaseCipherId(BlobCipherBaseKeyId baseCipherKeyId); + // API enables inserting base encryption cipher details to the BlobCipherKeyIdCache. + // Given cipherKeys are immutable, attempting to re-insert same 'identical' cipherKey + // is treated as a NOP (success), however, an attempt to update cipherKey would throw + // 'encrypt_update_cipher' exception. + void insertBaseCipherKey(BlobCipherBaseKeyId baseCipherId, const uint8_t* baseCipher, int baseCipherLen); + // API cleanup the cache by dropping all cached cipherKeys + void cleanup(); + // API returns list of all 'cached' cipherKeys + std::vector> getAllCipherKeys(); + +private: + BlobCipherDomainId domainId; + BlobCipherKeyIdCacheMap keyIdCache; + BlobCipherBaseKeyId latestBaseCipherKeyId; +}; + +using BlobCipherDomainCacheMap = std::unordered_map>; + +class BlobCipherKeyCache : NonCopyable { +public: + // Enable clients to insert base encryption cipher details to the BlobCipherKeyCache. + // The cipherKeys are indexed using 'baseCipherId', given cipherKeys are immutable, + // attempting to re-insert same 'identical' cipherKey is treated as a NOP (success), + // however, an attempt to update cipherKey would throw 'encrypt_update_cipher' exception. + void insertCipherKey(const BlobCipherDomainId& domainId, + const BlobCipherBaseKeyId& baseCipherId, + const uint8_t* baseCipher, + int baseCipherLen); + // API returns the last insert cipherKey for a given encyryption domain Id. + // If none exists, it would throw 'encrypt_key_not_found' exception. + Reference getLatestCipherKey(const BlobCipherDomainId& domainId); + // API returns cipherKey corresponding to {encryptionDomainId, baseCipherId} tuple. + // If none exists, it would throw 'encrypt_key_not_found' exception. + Reference getCipherKey(const BlobCipherDomainId& domainId, const BlobCipherBaseKeyId& baseCipherId); + // API returns point in time list of all 'cached' cipherKeys for a given encryption domainId. + std::vector> getAllCiphers(const BlobCipherDomainId& domainId); + // API enables dropping all 'cached' cipherKeys for a given encryption domain Id. + // Useful to cleanup cache if an encryption domain gets removed/destroyed etc. + void resetEncyrptDomainId(const BlobCipherDomainId domainId); + + static BlobCipherKeyCache& getInstance() { + static BlobCipherKeyCache instance; + return instance; + } + // Ensures cached encryption key(s) (plaintext) never gets persisted as part + // of FDB process/core dump. + static void cleanup() noexcept; + +private: + BlobCipherDomainCacheMap domainCacheMap; + static constexpr uint64_t CIPHER_KEY_CACHE_TTL_SEC = 10 * 60L; + + BlobCipherKeyCache() {} +}; + +// This interface enables data block encryption. An invocation to encrypt() will +// do two things: +// 1) generate encrypted ciphertext for given plaintext input. +// 2) generate BlobCipherEncryptHeader (including the 'header checksum') and persit for decryption on reads. + +class EncryptBlobCipherAes265Ctr final : NonCopyable, public ReferenceCounted { +public: + static constexpr uint8_t ENCRYPT_HEADER_VERSION = 1; + + EncryptBlobCipherAes265Ctr(Reference key, const uint8_t* iv, const int ivLen); + ~EncryptBlobCipherAes265Ctr(); + Reference encrypt(const uint8_t* plaintext, + const int plaintextLen, + BlobCipherEncryptHeader* header, + Arena&); + +private: + EVP_CIPHER_CTX* ctx; + Reference cipherKey; + uint8_t iv[AES_256_IV_LENGTH]; +}; + +// This interface enable data block decryption. An invocation to decrypt() would generate +// 'plaintext' for a given 'ciphertext' input, the caller needs to supply BlobCipherEncryptHeader. + +class DecryptBlobCipherAes256Ctr final : NonCopyable, public ReferenceCounted { +public: + DecryptBlobCipherAes256Ctr(Reference key, const uint8_t* iv); + ~DecryptBlobCipherAes256Ctr(); + Reference decrypt(const uint8_t* ciphertext, + const int ciphertextLen, + const BlobCipherEncryptHeader& header, + Arena&); + +private: + EVP_CIPHER_CTX* ctx; + + void verifyEncryptBlobHeader(const uint8_t* cipherText, + const int ciphertextLen, + const BlobCipherEncryptHeader& header, + Arena& arena); +}; + +class HmacSha256DigestGen final : NonCopyable { +public: + HmacSha256DigestGen(const unsigned char* key, size_t len); + ~HmacSha256DigestGen(); + HMAC_CTX* getCtx() const { return ctx; } + StringRef digest(unsigned char const* data, size_t len, Arena&); + +private: + HMAC_CTX* ctx; +}; + +BlobCipherChecksum computeEncryptChecksum(const uint8_t* payload, + const int payloadLen, + const BlobCipherRandomSalt& salt, + Arena& arena); + +#endif // ENCRYPTION_ENABLED diff --git a/flow/CMakeLists.txt b/flow/CMakeLists.txt index 96a8842bbf..6884e5af78 100644 --- a/flow/CMakeLists.txt +++ b/flow/CMakeLists.txt @@ -8,6 +8,8 @@ set(FLOW_SRCS ArgParseUtil.h AsioReactor.h BooleanParam.h + BlobCipher.h + BlobCipher.cpp CompressedInt.actor.cpp CompressedInt.h Deque.cpp diff --git a/flow/Platform.actor.cpp b/flow/Platform.actor.cpp index 9602fc431e..20a13ac8c7 100644 --- a/flow/Platform.actor.cpp +++ b/flow/Platform.actor.cpp @@ -33,6 +33,7 @@ #if (!defined(TLS_DISABLED) && !defined(_WIN32)) #include "flow/StreamCipher.h" +#include "flow/BlobCipher.h" #endif #include "flow/Trace.h" #include "flow/Error.h" @@ -3501,6 +3502,7 @@ void crashHandler(int sig) { #if (!defined(TLS_DISABLED) && !defined(_WIN32)) StreamCipherKey::cleanup(); StreamCipher::cleanup(); + BlobCipherKeyCache::cleanup(); #endif fflush(stdout); diff --git a/flow/error_definitions.h b/flow/error_definitions.h index 1f82b0be46..7710bca9ca 100755 --- a/flow/error_definitions.h +++ b/flow/error_definitions.h @@ -284,6 +284,15 @@ ERROR( snap_log_anti_quorum_unsupported, 2507, "Unsupported when log anti quorum ERROR( snap_with_recovery_unsupported, 2508, "Cluster recovery during snapshot operation not supported") ERROR( snap_invalid_uid_string, 2509, "The given uid string is not a 32-length hex string") +// 3XXX - Encryption operations errors +ERROR( encrypt_ops_error, 3000, "Encryption operation error") +ERROR( encrypt_header_metadata_mismatch, 3001, "Encryption header metadata mismatch") +ERROR( encrypt_key_not_found, 3002, "Expected encryption key is missing") +ERROR( encrypt_key_ttl_expired, 3003, "Expected encryption key TTL has expired") +ERROR( encrypt_header_checksum_mismatch, 3004, "Encryption header checksum mismatch") +ERROR( encrypt_update_cipher, 3005, "Attempt to update encryption cipher key") +ERROR( encrypt_invalid_id, 3006, "Invalid encryption domainId or encryption cipher key id") + // 4xxx Internal errors (those that should be generated only by bugs) are decimal 4xxx ERROR( unknown_error, 4000, "An unknown error occurred" ) // C++ exception not of type Error ERROR( internal_error, 4100, "An internal error occurred" ) From 2532ac5882eb9e4a85b30ff704f2573596ea6ff4 Mon Sep 17 00:00:00 2001 From: Jingyu Zhou Date: Wed, 23 Mar 2022 11:23:10 -0700 Subject: [PATCH 408/413] Add knob value changes to fdbdecode E.g., HTTP_VERBOSE_LEVEL, and HTTP_REQUEST_AWS_V4_HEADER knobs. --- fdbbackup/FileConverter.h | 2 ++ fdbbackup/FileDecoder.actor.cpp | 58 +++++++++++++++++++++++++++++++-- fdbclient/JSONDoc.h | 1 + 3 files changed, 58 insertions(+), 3 deletions(-) diff --git a/fdbbackup/FileConverter.h b/fdbbackup/FileConverter.h index a33032b183..5ad5c53b1b 100644 --- a/fdbbackup/FileConverter.h +++ b/fdbbackup/FileConverter.h @@ -46,6 +46,7 @@ enum { OPT_HEX_KEY_PREFIX, OPT_BEGIN_VERSION_FILTER, OPT_END_VERSION_FILTER, + OPT_KNOB, OPT_HELP }; @@ -72,6 +73,7 @@ CSimpleOpt::SOption gConverterOptions[] = { { OPT_CONTAINER, "-r", SO_REQ_SEP }, { OPT_HEX_KEY_PREFIX, "--hex-prefix", SO_REQ_SEP }, { OPT_BEGIN_VERSION_FILTER, "--begin-version-filter", SO_REQ_SEP }, { OPT_END_VERSION_FILTER, "--end-version-filter", SO_REQ_SEP }, + { OPT_KNOB, "--knob-", SO_REQ_SEP }, { OPT_HELP, "-?", SO_NONE }, { OPT_HELP, "-h", SO_NONE }, { OPT_HELP, "--help", SO_NONE }, diff --git a/fdbbackup/FileDecoder.actor.cpp b/fdbbackup/FileDecoder.actor.cpp index 93c6cdc084..7e851bf6e0 100644 --- a/fdbbackup/FileDecoder.actor.cpp +++ b/fdbbackup/FileDecoder.actor.cpp @@ -26,17 +26,21 @@ #include #include "fdbbackup/BackupTLSConfig.h" +#include "fdbclient/BuildFlags.h" +#include "fdbbackup/FileConverter.h" #include "fdbclient/BackupAgent.actor.h" #include "fdbclient/BackupContainer.h" -#include "fdbbackup/FileConverter.h" #include "fdbclient/CommitTransaction.h" #include "fdbclient/FDBTypes.h" +#include "fdbclient/IKnobCollection.h" +#include "fdbclient/Knobs.h" #include "fdbclient/MutationList.h" +#include "flow/ArgParseUtil.h" #include "flow/IRandom.h" #include "flow/Trace.h" #include "flow/flow.h" #include "flow/serialize.h" -#include "fdbclient/BuildFlags.h" + #include "flow/actorcompiler.h" // has to be last include #define SevDecodeInfo SevVerbose @@ -73,11 +77,13 @@ void printDecodeUsage() { " --list-only Print file list and exit.\n" " -k KEY_PREFIX Use the prefix for filtering mutations\n" " --hex-prefix HEX_PREFIX\n" - " The prefix specified in HEX format, e.g., \\x05\\x01.\n" + " The prefix specified in HEX format, e.g., \"\\\\x05\\\\x01\".\n" " --begin-version-filter BEGIN_VERSION\n" " The version range's begin version (inclusive) for filtering.\n" " --end-version-filter END_VERSION\n" " The version range's end version (exclusive) for filtering.\n" + " --knob-KNOBNAME KNOBVALUE\n" + " Changes a knob value. KNOBNAME should be lowercase." "\n"; return; } @@ -97,6 +103,8 @@ struct DecodeParams { Version beginVersionFilter = 0; Version endVersionFilter = std::numeric_limits::max(); + std::vector> knobs; + // Returns if [begin, end) overlap with the filter range bool overlap(Version begin, Version end) const { // Filter [100, 200), [50,75) [200, 300) @@ -130,8 +138,39 @@ struct DecodeParams { if (!prefix.empty()) { s.append(", KeyPrefix: ").append(printable(KeyRef(prefix))); } + for (const auto& [knob, value] : knobs) { + s.append(", KNOB-").append(knob).append(" = ").append(value); + } return s; } + + void updateKnobs() { + auto& g_knobs = IKnobCollection::getMutableGlobalKnobCollection(); + for (const auto& [knobName, knobValueString] : knobs) { + try { + auto knobValue = g_knobs.parseKnobValue(knobName, knobValueString); + g_knobs.setKnob(knobName, knobValue); + } catch (Error& e) { + if (e.code() == error_code_invalid_option_value) { + std::cerr << "WARNING: Invalid value '" << knobValueString << "' for knob option '" << knobName + << "'\n"; + TraceEvent(SevWarnAlways, "InvalidKnobValue") + .detail("Knob", printable(knobName)) + .detail("Value", printable(knobValueString)); + } else { + std::cerr << "ERROR: Failed to set knob option '" << knobName << "': " << e.what() << "\n"; + TraceEvent(SevError, "FailedToSetKnob") + .errorUnsuppressed(e) + .detail("Knob", printable(knobName)) + .detail("Value", printable(knobValueString)); + throw; + } + } + } + + // Reinitialize knobs in order to update knobs that are dependent on explicitly set knobs + g_knobs.initialize(Randomize::True, IsSimulated::False); + } }; // Decode an ASCII string, e.g., "\x15\x1b\x19\x04\xaf\x0c\x28\x0a", @@ -256,6 +295,16 @@ int parseDecodeCommandLine(DecodeParams* param, CSimpleOpt* args) { param->tlsConfig.blobCredentials.push_back(args->OptionArg()); break; + case OPT_KNOB: { + Optional knobName = extractPrefixedArgument("--knob", args->OptionSyntax()); + if (!knobName.present()) { + std::cerr << "ERROR: unable to parse knob option '" << args->OptionSyntax() << "'\n"; + return FDB_EXIT_ERROR; + } + param->knobs.emplace_back(knobName.get(), args->OptionArg()); + break; + } + #ifndef TLS_DISABLED case TLSConfig::OPT_TLS_PLUGIN: args->OptionArg(); @@ -552,6 +601,9 @@ int main(int argc, char** argv) { StringRef url(param.container_url); setupNetwork(0, UseMetrics::True); + // Must be called after setupNetwork() to be effective + param.updateKnobs(); + TraceEvent::setNetworkThread(); openTraceFile(NetworkAddress(), 10 << 20, 500 << 20, param.log_dir, "decode", param.trace_log_group); param.tlsConfig.setupBlobCredentials(); diff --git a/fdbclient/JSONDoc.h b/fdbclient/JSONDoc.h index 2fdeb7ba66..39a1b388ee 100644 --- a/fdbclient/JSONDoc.h +++ b/fdbclient/JSONDoc.h @@ -22,6 +22,7 @@ #include "fdbclient/json_spirit/json_spirit_writer_template.h" #include "fdbclient/json_spirit/json_spirit_reader_template.h" +#include "flow/Error.h" // JSONDoc is a convenient reader/writer class for manipulating JSON documents using "paths". // Access is done using a "path", which is a string of dot-separated From 902c10c2996c78b0e5564249df21a5a66ccdb53f Mon Sep 17 00:00:00 2001 From: Jingyu Zhou Date: Wed, 23 Mar 2022 16:16:20 -0700 Subject: [PATCH 409/413] Fix missing blob access key in credentials This causes failures of backup_auth_missing: ErrorDescription="Cannot find authentication details (such as a password or secret key) for the specified Backup Container URL" This is found by manually running "fdbdecode". --- fdbclient/S3BlobStore.actor.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/fdbclient/S3BlobStore.actor.cpp b/fdbclient/S3BlobStore.actor.cpp index a9bd360cde..0104bfa241 100644 --- a/fdbclient/S3BlobStore.actor.cpp +++ b/fdbclient/S3BlobStore.actor.cpp @@ -538,7 +538,7 @@ ACTOR Future updateSecret_impl(Reference b) { JSONDoc accounts(doc.last().get_obj()); if (accounts.has(credentialsFileKey, false) && accounts.last().type() == json_spirit::obj_type) { JSONDoc account(accounts.last()); - S3BlobStoreEndpoint::Credentials creds; + S3BlobStoreEndpoint::Credentials creds = b->credentials.get(); if (b->lookupKey) { std::string apiKey; if (account.tryGet("api_key", apiKey)) From ad6a63c16d5b77b9e90506e8c83e7f5e11a00c1c Mon Sep 17 00:00:00 2001 From: Aaron Molitor Date: Thu, 24 Mar 2022 13:13:07 -0500 Subject: [PATCH 410/413] used fixed Jinja2 version for documentation --- documentation/sphinx/requirements.txt | 1 + 1 file changed, 1 insertion(+) diff --git a/documentation/sphinx/requirements.txt b/documentation/sphinx/requirements.txt index 67ca207628..06e23ea6d3 100644 --- a/documentation/sphinx/requirements.txt +++ b/documentation/sphinx/requirements.txt @@ -3,3 +3,4 @@ setuptools>=20.10.0,<=57.4.0 sphinx==1.5.6 sphinx-bootstrap-theme==0.4.8 docutils==0.16 +Jinja2==3.0.3 From 1b1182f414c5e33ba88eac143638df9393b005fd Mon Sep 17 00:00:00 2001 From: Josh Slocum Date: Wed, 16 Mar 2022 15:11:29 -0500 Subject: [PATCH 411/413] Blob Worker side of beginVersion done, added unit test --- fdbclient/BlobGranuleFiles.cpp | 5 +- fdbclient/BlobGranuleReader.actor.cpp | 10 +- fdbclient/BlobWorkerCommon.h | 3 + fdbclient/BlobWorkerInterface.h | 3 +- fdbclient/ReadYourWrites.actor.cpp | 2 - fdbserver/BlobGranuleServerCommon.actor.cpp | 251 ++++++++++++++++++++ fdbserver/BlobGranuleServerCommon.actor.h | 14 +- fdbserver/BlobManager.actor.cpp | 2 +- fdbserver/BlobWorker.actor.cpp | 109 ++++----- tests/BGServerCommonUnit.toml | 9 + tests/CMakeLists.txt | 3 +- 11 files changed, 333 insertions(+), 78 deletions(-) create mode 100644 tests/BGServerCommonUnit.toml diff --git a/fdbclient/BlobGranuleFiles.cpp b/fdbclient/BlobGranuleFiles.cpp index a7e5dda5a3..6c2cd83a5f 100644 --- a/fdbclient/BlobGranuleFiles.cpp +++ b/fdbclient/BlobGranuleFiles.cpp @@ -172,7 +172,7 @@ RangeResult materializeBlobGranule(const BlobGranuleChunkRef& chunk, Version readVersion, Optional snapshotData, StringRef deltaFileData[]) { - // TODO REMOVE with V2 of protocol + // TODO REMOVE with early replying ASSERT(readVersion == chunk.includedVersion); ASSERT(chunk.snapshotFile.present()); ASSERT(snapshotData.present()); @@ -278,8 +278,7 @@ ErrorOr loadAndMaterializeBlobGranules(const Standalone> readFile(Reference(dataRef, arena); } catch (Error& e) { - printf("Reading file %s got error %s\n", f.toString().c_str(), e.name()); throw e; } } @@ -68,7 +67,7 @@ ACTOR Future readBlobGranule(BlobGranuleChunkRef chunk, Reference bstore, Optional stats) { - // TODO REMOVE with V2 of protocol + // TODO REMOVE with early replying ASSERT(readVersion == chunk.includedVersion); ASSERT(chunk.snapshotFile.present()); @@ -106,7 +105,6 @@ ACTOR Future readBlobGranule(BlobGranuleChunkRef chunk, return materializeBlobGranule(chunk, keyRange, readVersion, snapshotData, deltaData); } catch (Error& e) { - printf("Reading blob granule got error %s\n", e.name()); throw e; } } @@ -121,18 +119,12 @@ ACTOR Future readBlobGranules(BlobGranuleFileRequest request, try { state int i; for (i = 0; i < reply.chunks.size(); i++) { - /*printf("ReadBlobGranules processing chunk %d [%s - %s)\n", - i, - reply.chunks[i].keyRange.begin.printable().c_str(), - reply.chunks[i].keyRange.end.printable().c_str());*/ RangeResult chunkResult = wait(readBlobGranule(reply.chunks[i], request.keyRange, request.readVersion, bstore)); results.send(std::move(chunkResult)); } - // printf("ReadBlobGranules done, sending EOS\n"); results.sendError(end_of_stream()); } catch (Error& e) { - printf("ReadBlobGranules got error %s\n", e.name()); results.sendError(e); } diff --git a/fdbclient/BlobWorkerCommon.h b/fdbclient/BlobWorkerCommon.h index 2898412e73..49aed17985 100644 --- a/fdbclient/BlobWorkerCommon.h +++ b/fdbclient/BlobWorkerCommon.h @@ -38,6 +38,8 @@ struct BlobWorkerStats { Counter commitVersionChecks; Counter granuleUpdateErrors; Counter granuleRequestTimeouts; + Counter readRequestsWithBegin; + Counter readRequestsCollapsed; int numRangesAssigned; int mutationBytesBuffered; @@ -59,6 +61,7 @@ struct BlobWorkerStats { readReqTotalFilesReturned("ReadReqTotalFilesReturned", cc), readReqDeltaBytesReturned("ReadReqDeltaBytesReturned", cc), commitVersionChecks("CommitVersionChecks", cc), granuleUpdateErrors("GranuleUpdateErrors", cc), granuleRequestTimeouts("GranuleRequestTimeouts", cc), + readRequestsWithBegin("ReadRequestsWithBegin", cc), readRequestsCollapsed("ReadRequestsCollapsed", cc), numRangesAssigned(0), mutationBytesBuffered(0), activeReadRequests(0) { specialCounter(cc, "NumRangesAssigned", [this]() { return this->numRangesAssigned; }); specialCounter(cc, "MutationBytesBuffered", [this]() { return this->mutationBytesBuffered; }); diff --git a/fdbclient/BlobWorkerInterface.h b/fdbclient/BlobWorkerInterface.h index f69b73e1bc..5dd36b7128 100644 --- a/fdbclient/BlobWorkerInterface.h +++ b/fdbclient/BlobWorkerInterface.h @@ -86,13 +86,14 @@ struct BlobGranuleFileRequest { KeyRangeRef keyRange; Version beginVersion = 0; Version readVersion; + bool canCollapseBegin = true; ReplyPromise reply; BlobGranuleFileRequest() {} template void serialize(Ar& ar) { - serializer(ar, keyRange, beginVersion, readVersion, reply, arena); + serializer(ar, keyRange, beginVersion, readVersion, canCollapseBegin, reply, arena); } }; diff --git a/fdbclient/ReadYourWrites.actor.cpp b/fdbclient/ReadYourWrites.actor.cpp index c7b9629196..a16034963b 100644 --- a/fdbclient/ReadYourWrites.actor.cpp +++ b/fdbclient/ReadYourWrites.actor.cpp @@ -1791,8 +1791,6 @@ Future>> ReadYourWritesTransaction::re Version begin, Optional readVersion, Version* readVersionOut) { - // Remove in V2 of API - ASSERT(begin == 0); if (!options.readYourWritesDisabled) { return blob_granule_no_ryw(); diff --git a/fdbserver/BlobGranuleServerCommon.actor.cpp b/fdbserver/BlobGranuleServerCommon.actor.cpp index c47ed13199..28b4c1db44 100644 --- a/fdbserver/BlobGranuleServerCommon.actor.cpp +++ b/fdbserver/BlobGranuleServerCommon.actor.cpp @@ -18,6 +18,7 @@ * limitations under the License. */ +#include "contrib/fmt-8.1.1/include/fmt/format.h" #include "fdbclient/SystemData.h" #include "fdbclient/BlobGranuleCommon.h" #include "fdbserver/BlobGranuleServerCommon.actor.h" @@ -25,6 +26,7 @@ #include "fdbclient/FDBTypes.h" #include "fdbclient/ReadYourWrites.h" #include "flow/Arena.h" +#include "flow/UnitTest.h" #include "flow/actorcompiler.h" // has to be last include // Gets the latest granule history node for range that was persisted @@ -102,3 +104,252 @@ ACTOR Future loadHistoryFiles(Database cx, UID granuleID) { } } } + +// Normally a beginVersion != 0 means the caller wants all mutations between beginVersion and readVersion, instead of +// the latest snapshot before readVersion + deltas after the snapshot. When canCollapse is set, the beginVersion is +// essentially just an optimization hint. The caller is still concerned with reconstructing rows at readVersion, it just +// knows it doesn't need anything before beginVersion. +// Normally this can eliminate the need for a snapshot and just return a small amount of deltas. But in a highly active +// key range, the granule may have a snapshot file at version X, where beginVersion < X <= readVersion. In this case, if +// the number of bytes in delta files between beginVersion and X is larger than the snapshot file at version X, it is +// strictly more efficient (in terms of files and bytes read) to just use the snapshot file at version X instead. +void GranuleFiles::getFiles(Version beginVersion, + Version readVersion, + bool canCollapse, + BlobGranuleChunkRef& chunk, + Arena& replyArena) const { + BlobFileIndex dummyIndex; // for searching + + auto snapshotF = snapshotFiles.end(); + if (beginVersion == 0 || canCollapse) { + dummyIndex.version = readVersion; + snapshotF = std::lower_bound(snapshotFiles.begin(), snapshotFiles.end(), dummyIndex); + if (snapshotF == snapshotFiles.end() || snapshotF->version > readVersion) { + ASSERT(snapshotF != snapshotFiles.begin()); + snapshotF--; + } + ASSERT(snapshotF != snapshotFiles.end()); + ASSERT(snapshotF->version <= readVersion); + } + + auto deltaF = deltaFiles.end(); + if (beginVersion > 0) { + dummyIndex.version = beginVersion; + deltaF = std::lower_bound(deltaFiles.begin(), deltaFiles.end(), dummyIndex); + if (canCollapse) { + ASSERT(snapshotF != snapshotFiles.end()); + // see if delta files up to snapshotVersion are smaller or larger than snapshotBytes in total + auto deltaFCopy = deltaF; + int64_t snapshotBytes = snapshotF->length; + while (deltaFCopy != deltaFiles.end() && deltaFCopy->version <= snapshotF->version && snapshotBytes > 0) { + snapshotBytes -= deltaFCopy->length; + deltaFCopy++; + } + // if delta files contain the same or more bytes as the snapshot with collapse, do the collapse + if (snapshotBytes > 0) { + // don't collapse, clear snapshotF and just do delta files + snapshotF = snapshotFiles.end(); + } else { + // do snapshot instead of previous deltas + dummyIndex.version = snapshotF->version; + deltaF = std::upper_bound(deltaFiles.begin(), deltaFiles.end(), dummyIndex); + ASSERT(deltaF == deltaFiles.end() || deltaF->version > snapshotF->version); + } + } + } else { + dummyIndex.version = snapshotF->version; + deltaF = std::upper_bound(deltaFiles.begin(), deltaFiles.end(), dummyIndex); + ASSERT(deltaF == deltaFiles.end() || deltaF->version > snapshotF->version); + } + + Version lastIncluded = invalidVersion; + if (snapshotF != snapshotFiles.end()) { + chunk.snapshotVersion = snapshotF->version; + chunk.snapshotFile = BlobFilePointerRef(replyArena, snapshotF->filename, snapshotF->offset, snapshotF->length); + lastIncluded = chunk.snapshotVersion; + } else { + chunk.snapshotVersion = invalidVersion; + } + + int64_t deltaBytes = 0; + while (deltaF != deltaFiles.end() && deltaF->version < readVersion) { + chunk.deltaFiles.emplace_back_deep(replyArena, deltaF->filename, deltaF->offset, deltaF->length); + deltaBytes += deltaF->length; + ASSERT(lastIncluded < deltaF->version); + lastIncluded = deltaF->version; + deltaF++; + } + // include last delta file that passes readVersion, if it exists + if (deltaF != deltaFiles.end() && lastIncluded < readVersion) { + chunk.deltaFiles.emplace_back_deep(replyArena, deltaF->filename, deltaF->offset, deltaF->length); + deltaBytes += deltaF->length; + lastIncluded = deltaF->version; + } + + // TODO wire this up, + // bwData->stats.readReqDeltaBytesReturned += deltaBytes; +} + +static std::string makeTestFileName(Version v) { + return "test" + std::to_string(v); +} + +static BlobFileIndex makeTestFile(Version v, int64_t len) { + return BlobFileIndex(v, makeTestFileName(v), 0, len); +} + +static void checkFile(int expectedVersion, const BlobFilePointerRef& actualFile) { + ASSERT(makeTestFileName(expectedVersion) == actualFile.filename.toString()); +} + +static void checkFiles(const GranuleFiles& f, + Version beginVersion, + Version readVersion, + bool canCollapse, + Optional expectedSnapshotVersion, + std::vector expectedDeltaVersions) { + Arena a; + BlobGranuleChunkRef chunk; + f.getFiles(beginVersion, readVersion, canCollapse, chunk, a); + fmt::print("results({0}, {1}, {2}):\nEXPECTED: snapshot={3}\n deltas ({4}):\n", + beginVersion, + readVersion, + canCollapse ? "T" : "F", + expectedSnapshotVersion.present() ? makeTestFileName(expectedSnapshotVersion.get()).c_str() : "", + expectedDeltaVersions.size()); + for (int d : expectedDeltaVersions) { + fmt::print(" {}\n", makeTestFileName(d)); + } + fmt::print("ACTUAL:\n snapshot={0}\n deltas ({1}):\n", + chunk.snapshotFile.present() ? chunk.snapshotFile.get().filename.toString().c_str() : "", + chunk.deltaFiles.size()); + for (auto& it : chunk.deltaFiles) { + fmt::print(" {}\n", it.filename.toString()); + } + printf("\n\n\n"); + ASSERT(expectedSnapshotVersion.present() == chunk.snapshotFile.present()); + if (expectedSnapshotVersion.present()) { + checkFile(expectedSnapshotVersion.get(), chunk.snapshotFile.get()); + } + ASSERT(expectedDeltaVersions.size() == chunk.deltaFiles.size()); + for (int i = 0; i < expectedDeltaVersions.size(); i++) { + checkFile(expectedDeltaVersions[i], chunk.deltaFiles[i]); + } +} + +/* + * Files: + * S @ 100 (10 bytes) + * D @ 150 (5 bytes) + * D @ 200 (6 bytes) + * S @ 200 (15 bytes) + * D @ 250 (7 bytes) + * D @ 300 (8 bytes) + * S @ 300 (10 bytes) + * D @ 350 (4 bytes) + */ +TEST_CASE("/blobgranule/server/common/granulefiles") { + // simple cases first + + // single snapshot file, no deltas + GranuleFiles files; + files.snapshotFiles.push_back(makeTestFile(100, 10)); + + printf("Just snapshot\n"); + + checkFiles(files, 0, 100, false, 100, {}); + checkFiles(files, 0, 200, false, 100, {}); + + printf("Small test\n"); + // add delta files with re-snapshot at end + files.deltaFiles.push_back(makeTestFile(150, 5)); + files.deltaFiles.push_back(makeTestFile(200, 6)); + files.snapshotFiles.push_back(makeTestFile(200, 15)); + + // check different read versions with beginVersion=0 + checkFiles(files, 0, 100, false, 100, {}); + checkFiles(files, 0, 101, false, 100, { 150 }); + checkFiles(files, 0, 149, false, 100, { 150 }); + checkFiles(files, 0, 150, false, 100, { 150 }); + checkFiles(files, 0, 151, false, 100, { 150, 200 }); + checkFiles(files, 0, 199, false, 100, { 150, 200 }); + checkFiles(files, 0, 200, false, 200, {}); + checkFiles(files, 0, 300, false, 200, {}); + + // Test all cases of beginVersion + readVersion. Because delta files are smaller than snapshot at 200, this should + // be the same with and without collapse + checkFiles(files, 100, 200, false, Optional(), { 150, 200 }); + checkFiles(files, 100, 300, false, Optional(), { 150, 200 }); + checkFiles(files, 101, 199, false, Optional(), { 150, 200 }); + checkFiles(files, 149, 151, false, Optional(), { 150, 200 }); + checkFiles(files, 149, 150, false, Optional(), { 150 }); + checkFiles(files, 150, 151, false, Optional(), { 150, 200 }); + checkFiles(files, 151, 200, false, Optional(), { 200 }); + + checkFiles(files, 100, 200, true, Optional(), { 150, 200 }); + checkFiles(files, 100, 300, true, Optional(), { 150, 200 }); + checkFiles(files, 101, 199, true, Optional(), { 150, 200 }); + checkFiles(files, 149, 151, true, Optional(), { 150, 200 }); + checkFiles(files, 149, 150, true, Optional(), { 150 }); + checkFiles(files, 150, 151, true, Optional(), { 150, 200 }); + checkFiles(files, 151, 200, true, Optional(), { 200 }); + + printf("Larger test\n"); + // add more delta files and snapshots to check collapse logic + files.deltaFiles.push_back(makeTestFile(250, 7)); + files.deltaFiles.push_back(makeTestFile(300, 8)); + files.snapshotFiles.push_back(makeTestFile(300, 10)); + files.deltaFiles.push_back(makeTestFile(350, 4)); + + checkFiles(files, 0, 300, false, 300, {}); + checkFiles(files, 0, 301, false, 300, { 350 }); + checkFiles(files, 0, 400, false, 300, { 350 }); + + // check delta files without collapse + + checkFiles(files, 100, 301, false, Optional(), { 150, 200, 250, 300, 350 }); + checkFiles(files, 100, 300, false, Optional(), { 150, 200, 250, 300 }); + checkFiles(files, 100, 251, false, Optional(), { 150, 200, 250, 300 }); + checkFiles(files, 100, 250, false, Optional(), { 150, 200, 250 }); + + checkFiles(files, 151, 300, false, Optional(), { 200, 250, 300 }); + checkFiles(files, 151, 301, false, Optional(), { 200, 250, 300, 350 }); + checkFiles(files, 151, 400, false, Optional(), { 200, 250, 300, 350 }); + + checkFiles(files, 201, 300, false, Optional(), { 250, 300 }); + checkFiles(files, 201, 301, false, Optional(), { 250, 300, 350 }); + checkFiles(files, 201, 400, false, Optional(), { 250, 300, 350 }); + + checkFiles(files, 251, 300, false, Optional(), { 300 }); + checkFiles(files, 251, 301, false, Optional(), { 300, 350 }); + checkFiles(files, 251, 400, false, Optional(), { 300, 350 }); + checkFiles(files, 301, 400, false, Optional(), { 350 }); + checkFiles(files, 351, 400, false, Optional(), {}); + + // check with collapse + // these 2 collapse because the delta files at 150+200+250+300 are larger than the snapshot at 300 + checkFiles(files, 100, 301, true, 300, { 350 }); + checkFiles(files, 100, 300, true, 300, {}); + // these 2 don't collapse because 150+200 delta files are smaller than the snapshot at 200 + checkFiles(files, 100, 251, true, Optional(), { 150, 200, 250, 300 }); + checkFiles(files, 100, 250, true, Optional(), { 150, 200, 250 }); + + // these 3 do collapse because the delta files at 200+250+300 are larger than the snapshot at 300 + checkFiles(files, 151, 300, true, 300, {}); + checkFiles(files, 151, 301, true, 300, { 350 }); + checkFiles(files, 151, 400, true, 300, { 350 }); + + // these 3 do collapse because the delta files at 250+300 are larger than the snapshot at 300 + checkFiles(files, 201, 300, true, 300, {}); + checkFiles(files, 201, 301, true, 300, { 350 }); + checkFiles(files, 201, 400, true, 300, { 350 }); + + // these don't collapse because the delta file at 300 is smaller than the snapshot at 300 + checkFiles(files, 251, 300, true, Optional(), { 300 }); + checkFiles(files, 251, 301, true, Optional(), { 300, 350 }); + checkFiles(files, 251, 400, true, Optional(), { 300, 350 }); + checkFiles(files, 301, 400, true, Optional(), { 350 }); + checkFiles(files, 351, 400, true, Optional(), {}); + + return Void(); +} \ No newline at end of file diff --git a/fdbserver/BlobGranuleServerCommon.actor.h b/fdbserver/BlobGranuleServerCommon.actor.h index d48418c951..b088903967 100644 --- a/fdbserver/BlobGranuleServerCommon.actor.h +++ b/fdbserver/BlobGranuleServerCommon.actor.h @@ -54,12 +54,22 @@ struct BlobFileIndex { BlobFileIndex(Version version, std::string filename, int64_t offset, int64_t length) : version(version), filename(filename), offset(offset), length(length) {} + + // compare on version + bool operator<(const BlobFileIndex& r) const { return version < r.version; } }; +// FIXME: initialize these to smaller default sizes to save a bit of memory, particularly snapshotFiles // Stores the files that comprise a blob granule struct GranuleFiles { - std::deque snapshotFiles; - std::deque deltaFiles; + std::vector snapshotFiles; + std::vector deltaFiles; + + void getFiles(Version beginVersion, + Version readVersion, + bool canCollapse, + BlobGranuleChunkRef& chunk, + Arena& replyArena) const; }; class Transaction; diff --git a/fdbserver/BlobManager.actor.cpp b/fdbserver/BlobManager.actor.cpp index 2b6a4da2bf..b81fee7d70 100644 --- a/fdbserver/BlobManager.actor.cpp +++ b/fdbserver/BlobManager.actor.cpp @@ -2778,7 +2778,7 @@ ACTOR Future blobManager(BlobManagerInterface bmInterf, // DB has [A - B) and [C - D). They should show up in knownBlobRanges, and [B - C) should be in removed. // DB has [B - C). It should show up in knownBlobRanges, [B - C) should be in added, and [A - B) and [C - D) // should be in removed. -TEST_CASE(":/blobmanager/updateranges") { +TEST_CASE("/blobmanager/updateranges") { KeyRangeMap knownBlobRanges(false, normalKeys.end); Arena ar; diff --git a/fdbserver/BlobWorker.actor.cpp b/fdbserver/BlobWorker.actor.cpp index e939d71ec6..5b6d22c2b1 100644 --- a/fdbserver/BlobWorker.actor.cpp +++ b/fdbserver/BlobWorker.actor.cpp @@ -18,6 +18,7 @@ * limitations under the License. */ +#include #include #include #include @@ -43,9 +44,10 @@ #include "flow/Error.h" #include "flow/IRandom.h" #include "flow/Trace.h" -#include "flow/actorcompiler.h" // has to be last include #include "flow/network.h" +#include "flow/actorcompiler.h" // has to be last include + #define BW_DEBUG false #define BW_REQUEST_DEBUG false @@ -2100,9 +2102,13 @@ ACTOR Future doBlobGranuleFileRequest(Reference bwData, Bl req.readVersion); } + state bool didCollapse = false; try { - // TODO REMOVE in api V2 - ASSERT(req.beginVersion == 0); + // TODO remove requirement for canCollapseBegin once we implement early replying + ASSERT(req.beginVersion == 0 || req.canCollapseBegin); + if (req.beginVersion != 0) { + ASSERT(req.beginVersion > 0); + } state BlobGranuleFileReply rep; state std::vector> granules; @@ -2150,6 +2156,7 @@ ACTOR Future doBlobGranuleFileRequest(Reference bwData, Bl continue; } state Reference metadata = m; + state Version granuleBeginVersion = req.beginVersion; choose { when(wait(metadata->readable.getFuture())) {} @@ -2290,67 +2297,25 @@ ACTOR Future doBlobGranuleFileRequest(Reference bwData, Bl // granule is up to date, do read ASSERT(metadata->cancelled.canBeSet()); + // Right now we force a collapse if the version range crosses granule boundaries, for simplicity + if (chunkFiles.snapshotFiles.front().version < granuleBeginVersion) { + didCollapse = true; + granuleBeginVersion = 0; + } BlobGranuleChunkRef chunk; - // TODO change in V2 + // TODO change with early reply chunk.includedVersion = req.readVersion; chunk.keyRange = KeyRangeRef(StringRef(rep.arena, chunkRange.begin), StringRef(rep.arena, chunkRange.end)); - // handle snapshot files - // TODO refactor the "find snapshot file" logic to GranuleFiles? - // FIXME: binary search instead of linear search, especially when file count is large - int i = chunkFiles.snapshotFiles.size() - 1; - while (i >= 0 && chunkFiles.snapshotFiles[i].version > req.readVersion) { - i--; - } - // because of granule history, we should always be able to find the desired snapshot - // version, and have thrown blob_granule_transaction_too_old earlier if not possible. - if (i < 0) { - fmt::print("req @ {0} >= initial snapshot {1} but can't find snapshot in ({2}) files:\n", - req.readVersion, - metadata->initialSnapshotVersion, - chunkFiles.snapshotFiles.size()); - for (auto& f : chunkFiles.snapshotFiles) { - fmt::print(" {0}", f.version); - } - } - ASSERT(i >= 0); - - BlobFileIndex snapshotF = chunkFiles.snapshotFiles[i]; - chunk.snapshotFile = BlobFilePointerRef(rep.arena, snapshotF.filename, snapshotF.offset, snapshotF.length); - Version snapshotVersion = chunkFiles.snapshotFiles[i].version; - chunk.snapshotVersion = snapshotVersion; - - // handle delta files - // cast this to an int so i going to -1 still compares properly - int lastDeltaFileIdx = chunkFiles.deltaFiles.size() - 1; - i = lastDeltaFileIdx; - // skip delta files that are too new - while (i >= 0 && chunkFiles.deltaFiles[i].version > req.readVersion) { - i--; - } - if (i < lastDeltaFileIdx) { - // we skipped one file at the end with a larger read version, this will actually contain - // our query version, so add it back. - i++; - } - // only include delta files after the snapshot file - int j = i; - while (j >= 0 && chunkFiles.deltaFiles[j].version > snapshotVersion) { - j--; - } - j++; - while (j <= i) { - BlobFileIndex deltaF = chunkFiles.deltaFiles[j]; - chunk.deltaFiles.emplace_back_deep(rep.arena, deltaF.filename, deltaF.offset, deltaF.length); - bwData->stats.readReqDeltaBytesReturned += deltaF.length; - j++; + chunkFiles.getFiles(granuleBeginVersion, req.readVersion, req.canCollapseBegin, chunk, rep.arena); + if (granuleBeginVersion > 0 && chunk.snapshotFile.present()) { + didCollapse = true; } // new deltas (if version is larger than version of last delta file) // FIXME: do trivial key bounds here if key range is not fully contained in request key // range - - if (req.readVersion > metadata->durableDeltaVersion.get()) { + if (req.readVersion > metadata->durableDeltaVersion.get() && metadata->currentDeltas.size()) { if (metadata->durableDeltaVersion.get() != metadata->pendingDeltaVersion) { fmt::print("real-time read [{0} - {1}) @ {2} doesn't have mutations!! durable={3}, pending={4}\n", metadata->keyRange.begin.printable(), @@ -2359,13 +2324,31 @@ ACTOR Future doBlobGranuleFileRequest(Reference bwData, Bl metadata->durableDeltaVersion.get(), metadata->pendingDeltaVersion); } + + // prune mutations based on begin version, if possible ASSERT(metadata->durableDeltaVersion.get() == metadata->pendingDeltaVersion); rep.arena.dependsOn(metadata->currentDeltas.arena()); - for (auto& delta : metadata->currentDeltas) { - if (delta.version > req.readVersion) { + MutationsAndVersionRef* mutationIt = metadata->currentDeltas.begin(); + if (granuleBeginVersion > metadata->currentDeltas.back().version) { + TEST(true); // beginVersion pruning all in-memory mutations + mutationIt = metadata->currentDeltas.end(); + } else if (granuleBeginVersion > metadata->currentDeltas.front().version) { + // binary search for beginVersion + TEST(true); // beginVersion pruning some in-memory mutations + mutationIt = std::lower_bound(metadata->currentDeltas.begin(), + metadata->currentDeltas.end(), + MutationsAndVersionRef(granuleBeginVersion, 0), + MutationsAndVersionRef::OrderByVersion()); + } + + // add mutations to response + while (mutationIt != metadata->currentDeltas.end()) { + if (mutationIt->version > req.readVersion) { + TEST(true); // readVersion pruning some in-memory mutations break; } - chunk.newDeltas.push_back_deep(rep.arena, delta); + chunk.newDeltas.push_back_deep(rep.arena, *mutationIt); + mutationIt++; } } @@ -2376,11 +2359,19 @@ ACTOR Future doBlobGranuleFileRequest(Reference bwData, Bl wait(yield(TaskPriority::DefaultEndpoint)); } + // do these together to keep them synchronous + if (req.beginVersion != 0) { + ++bwData->stats.readRequestsWithBegin; + } + if (didCollapse) { + ++bwData->stats.readRequestsCollapsed; + } ASSERT(!req.reply.isSet()); req.reply.send(rep); --bwData->stats.activeReadRequests; } catch (Error& e) { - // fmt::print("Error in BGFRequest {0}\n", e.name()); + // TODO REMOVE + fmt::print("Error in BGFRequest {0}\n", e.name()); if (e.code() == error_code_operation_cancelled) { req.reply.sendError(wrong_shard_server()); throw; diff --git a/tests/BGServerCommonUnit.toml b/tests/BGServerCommonUnit.toml new file mode 100644 index 0000000000..d7a5eba2ca --- /dev/null +++ b/tests/BGServerCommonUnit.toml @@ -0,0 +1,9 @@ +[[test]] +testTitle = 'BGServerCommonUnit' +useDB = false +startDelay = 0 + + [[test.workload]] + testName = 'UnitTests' + maxTestCases = 0 + testsMatching = /blobgranule/server/common/ \ No newline at end of file diff --git a/tests/CMakeLists.txt b/tests/CMakeLists.txt index 42713ad6cb..d9a3eca683 100644 --- a/tests/CMakeLists.txt +++ b/tests/CMakeLists.txt @@ -50,7 +50,8 @@ if(WITH_PYTHON) add_fdb_test(TEST_FILES s3VersionHeaders.txt IGNORE) add_fdb_test(TEST_FILES BandwidthThrottle.txt IGNORE) add_fdb_test(TEST_FILES BigInsert.txt IGNORE) - add_fdb_test(TEST_FILES BlobGranuleFileUnit.txt) + add_fdb_test(TEST_FILES BGServerCommonUnit.toml) + add_fdb_test(TEST_FILES BlobGranuleFileUnit.txt) # TODO change these to toml add_fdb_test(TEST_FILES BlobManagerUnit.txt) add_fdb_test(TEST_FILES ConsistencyCheck.txt IGNORE) add_fdb_test(TEST_FILES DDMetricsExclude.txt IGNORE) From 989dd8d7ebdb5677d2995b30597e19a57af6ff70 Mon Sep 17 00:00:00 2001 From: Josh Slocum Date: Thu, 17 Mar 2022 09:28:53 -0500 Subject: [PATCH 412/413] Finished implementing beginVersion --- fdbclient/BlobGranuleFiles.cpp | 51 +++++++---- fdbclient/BlobGranuleFiles.h | 1 + fdbclient/BlobGranuleReader.actor.cpp | 28 ++++--- fdbclient/BlobGranuleReader.actor.h | 1 + fdbclient/NativeAPI.actor.cpp | 1 + fdbserver/BlobWorker.actor.cpp | 16 +++- .../BlobGranuleCorrectnessWorkload.actor.cpp | 84 ++++++++++++++++--- .../workloads/BlobGranuleVerifier.actor.cpp | 2 +- 8 files changed, 143 insertions(+), 41 deletions(-) diff --git a/fdbclient/BlobGranuleFiles.cpp b/fdbclient/BlobGranuleFiles.cpp index 6c2cd83a5f..67b119531b 100644 --- a/fdbclient/BlobGranuleFiles.cpp +++ b/fdbclient/BlobGranuleFiles.cpp @@ -119,29 +119,50 @@ static void applyDelta(KeyRangeRef keyRange, MutationRef m, std::map& dataMap) { - if (!deltas.empty()) { - // check that consecutive delta file versions are disjoint - ASSERT(lastFileEndVersion < deltas.front().version); + if (deltas.empty()) { + return; } - for (const MutationsAndVersionRef& delta : deltas) { - if (delta.version > readVersion) { + // check that consecutive delta file versions are disjoint + ASSERT(lastFileEndVersion < deltas.front().version); + + const MutationsAndVersionRef* mutationIt = deltas.begin(); + // prune beginVersion if necessary + if (beginVersion > deltas.front().version) { + if (beginVersion > deltas.back().version) { + printf("beginVersion=%lld, deltas.front=%lld, deltas.back=%lld, deltas.size=%d\n", + beginVersion, + deltas.front().version, + deltas.back().version, + deltas.size()); + } + ASSERT(beginVersion <= deltas.back().version); + // binary search for beginVersion + mutationIt = std::lower_bound(deltas.begin(), + deltas.end(), + MutationsAndVersionRef(beginVersion, 0), + MutationsAndVersionRef::OrderByVersion()); + } + + while (mutationIt != deltas.end()) { + if (mutationIt->version > readVersion) { lastFileEndVersion = readVersion; return; } - for (auto& m : delta.mutations) { + for (auto& m : mutationIt->mutations) { applyDelta(keyRange, m, dataMap); } + mutationIt++; } - if (!deltas.empty()) { - lastFileEndVersion = deltas.back().version; - } + lastFileEndVersion = deltas.back().version; } static Arena loadDeltaFile(StringRef deltaData, KeyRangeRef keyRange, + Version beginVersion, Version readVersion, Version& lastFileEndVersion, std::map& dataMap) { @@ -163,19 +184,18 @@ static Arena loadDeltaFile(StringRef deltaData, ASSERT(deltas[i].version <= deltas[i + 1].version); } - applyDeltas(deltas, keyRange, readVersion, lastFileEndVersion, dataMap); + applyDeltas(deltas, keyRange, beginVersion, readVersion, lastFileEndVersion, dataMap); return parseArena; } RangeResult materializeBlobGranule(const BlobGranuleChunkRef& chunk, KeyRangeRef keyRange, + Version beginVersion, Version readVersion, Optional snapshotData, StringRef deltaFileData[]) { // TODO REMOVE with early replying ASSERT(readVersion == chunk.includedVersion); - ASSERT(chunk.snapshotFile.present()); - ASSERT(snapshotData.present()); // Arena to hold all allocations for applying deltas. Most of it, and the arenas produced by reading the files, // will likely be tossed if there are a significant number of mutations, so we copy at the end instead of doing a @@ -195,13 +215,14 @@ RangeResult materializeBlobGranule(const BlobGranuleChunkRef& chunk, fmt::print("Applying {} delta files\n", chunk.deltaFiles.size()); } for (int deltaIdx = 0; deltaIdx < chunk.deltaFiles.size(); deltaIdx++) { - Arena deltaArena = loadDeltaFile(deltaFileData[deltaIdx], keyRange, readVersion, lastFileEndVersion, dataMap); + Arena deltaArena = + loadDeltaFile(deltaFileData[deltaIdx], keyRange, beginVersion, readVersion, lastFileEndVersion, dataMap); arena.dependsOn(deltaArena); } if (BG_READ_DEBUG) { fmt::print("Applying {} memory deltas\n", chunk.newDeltas.size()); } - applyDeltas(chunk.newDeltas, keyRange, readVersion, lastFileEndVersion, dataMap); + applyDeltas(chunk.newDeltas, keyRange, beginVersion, readVersion, lastFileEndVersion, dataMap); RangeResult ret; for (auto& it : dataMap) { @@ -262,7 +283,7 @@ ErrorOr loadAndMaterializeBlobGranules(const Standalone loadAndMaterializeBlobGranules(const Standalone snapshotData, StringRef deltaFileData[]); diff --git a/fdbclient/BlobGranuleReader.actor.cpp b/fdbclient/BlobGranuleReader.actor.cpp index 0b9a4fba30..2fde30be0d 100644 --- a/fdbclient/BlobGranuleReader.actor.cpp +++ b/fdbclient/BlobGranuleReader.actor.cpp @@ -28,6 +28,7 @@ #include "fdbclient/BlobGranuleReader.actor.h" #include "fdbclient/BlobWorkerCommon.h" #include "fdbclient/BlobWorkerInterface.h" +#include "fdbclient/FDBTypes.h" #include "flow/actorcompiler.h" // This must be the last #include. // TODO more efficient data structure besides std::map? PTree is unnecessary since this isn't versioned, but some other @@ -63,22 +64,25 @@ ACTOR Future> readFile(Reference readBlobGranule(BlobGranuleChunkRef chunk, KeyRangeRef keyRange, + Version beginVersion, Version readVersion, Reference bstore, Optional stats) { // TODO REMOVE with early replying ASSERT(readVersion == chunk.includedVersion); - ASSERT(chunk.snapshotFile.present()); state Arena arena; try { - Future> readSnapshotFuture = readFile(bstore, chunk.snapshotFile.get()); - state std::vector>> readDeltaFutures; - if (stats.present()) { - ++stats.get()->s3GetReqs; + Future> readSnapshotFuture; + if (chunk.snapshotFile.present()) { + readSnapshotFuture = readFile(bstore, chunk.snapshotFile.get()); + if (stats.present()) { + ++stats.get()->s3GetReqs; + } } + state std::vector>> readDeltaFutures; readDeltaFutures.reserve(chunk.deltaFiles.size()); for (BlobFilePointerRef deltaFile : chunk.deltaFiles) { @@ -88,8 +92,12 @@ ACTOR Future readBlobGranule(BlobGranuleChunkRef chunk, } } - state Standalone snapshotData = wait(readSnapshotFuture); - arena.dependsOn(snapshotData.arena()); + state Optional snapshotData; // not present if snapshotFile isn't present + if (chunk.snapshotFile.present()) { + state Standalone s = wait(readSnapshotFuture); + arena.dependsOn(s.arena()); + snapshotData = s; + } state int numDeltaFiles = chunk.deltaFiles.size(); state StringRef* deltaData = new (arena) StringRef[numDeltaFiles]; @@ -102,7 +110,7 @@ ACTOR Future readBlobGranule(BlobGranuleChunkRef chunk, arena.dependsOn(data.arena()); } - return materializeBlobGranule(chunk, keyRange, readVersion, snapshotData, deltaData); + return materializeBlobGranule(chunk, keyRange, beginVersion, readVersion, snapshotData, deltaData); } catch (Error& e) { throw e; @@ -119,8 +127,8 @@ ACTOR Future readBlobGranules(BlobGranuleFileRequest request, try { state int i; for (i = 0; i < reply.chunks.size(); i++) { - RangeResult chunkResult = - wait(readBlobGranule(reply.chunks[i], request.keyRange, request.readVersion, bstore)); + RangeResult chunkResult = wait( + readBlobGranule(reply.chunks[i], request.keyRange, request.beginVersion, request.readVersion, bstore)); results.send(std::move(chunkResult)); } results.sendError(end_of_stream()); diff --git a/fdbclient/BlobGranuleReader.actor.h b/fdbclient/BlobGranuleReader.actor.h index 1b168ebc5d..958dc817e8 100644 --- a/fdbclient/BlobGranuleReader.actor.h +++ b/fdbclient/BlobGranuleReader.actor.h @@ -40,6 +40,7 @@ // the request ACTOR Future readBlobGranule(BlobGranuleChunkRef chunk, KeyRangeRef keyRange, + Version beginVersion, Version readVersion, Reference bstore, Optional stats = Optional()); diff --git a/fdbclient/NativeAPI.actor.cpp b/fdbclient/NativeAPI.actor.cpp index a46c16641d..af465df0c9 100644 --- a/fdbclient/NativeAPI.actor.cpp +++ b/fdbclient/NativeAPI.actor.cpp @@ -7442,6 +7442,7 @@ ACTOR Future>> readBlobGranulesActor( req.keyRange = KeyRangeRef(StringRef(req.arena, granuleStartKey), StringRef(req.arena, granuleEndKey)); req.beginVersion = begin; req.readVersion = rv; + req.canCollapseBegin = true; // TODO make this a parameter once we support it std::vector>> v; v.push_back( diff --git a/fdbserver/BlobWorker.actor.cpp b/fdbserver/BlobWorker.actor.cpp index 5b6d22c2b1..5c6bba8a8e 100644 --- a/fdbserver/BlobWorker.actor.cpp +++ b/fdbserver/BlobWorker.actor.cpp @@ -834,7 +834,7 @@ ACTOR Future compactFromBlob(Reference bwData, rowsStream, false); RangeResult newGranule = - wait(readBlobGranule(chunk, metadata->keyRange, version, bwData->bstore, &bwData->stats)); + wait(readBlobGranule(chunk, metadata->keyRange, 0, version, bwData->bstore, &bwData->stats)); bwData->stats.bytesReadFromS3ForCompaction += compactBytesRead; rowsStream.send(std::move(newGranule)); @@ -2095,11 +2095,16 @@ ACTOR Future waitForVersion(Reference metadata, Version v ACTOR Future doBlobGranuleFileRequest(Reference bwData, BlobGranuleFileRequest req) { if (BW_REQUEST_DEBUG) { - fmt::print("BW {0} processing blobGranuleFileRequest for range [{1} - {2}) @ {3}\n", + fmt::print("BW {0} processing blobGranuleFileRequest for range [{1} - {2}) @ ", bwData->id.toString(), req.keyRange.begin.printable(), req.keyRange.end.printable(), req.readVersion); + if (req.beginVersion > 0) { + fmt::print("{0} - {1}\n", req.beginVersion, req.readVersion); + } else { + fmt::print("{}", req.readVersion); + } } state bool didCollapse = false; @@ -2298,7 +2303,8 @@ ACTOR Future doBlobGranuleFileRequest(Reference bwData, Bl ASSERT(metadata->cancelled.canBeSet()); // Right now we force a collapse if the version range crosses granule boundaries, for simplicity - if (chunkFiles.snapshotFiles.front().version < granuleBeginVersion) { + if (granuleBeginVersion <= chunkFiles.snapshotFiles.front().version) { + TEST(true); // collapsed begin version request because of boundaries didCollapse = true; granuleBeginVersion = 0; } @@ -2309,13 +2315,14 @@ ACTOR Future doBlobGranuleFileRequest(Reference bwData, Bl chunkFiles.getFiles(granuleBeginVersion, req.readVersion, req.canCollapseBegin, chunk, rep.arena); if (granuleBeginVersion > 0 && chunk.snapshotFile.present()) { + TEST(true); // collapsed begin version request for efficiency didCollapse = true; } // new deltas (if version is larger than version of last delta file) // FIXME: do trivial key bounds here if key range is not fully contained in request key // range - if (req.readVersion > metadata->durableDeltaVersion.get() && metadata->currentDeltas.size()) { + if (req.readVersion > metadata->durableDeltaVersion.get() && !metadata->currentDeltas.empty()) { if (metadata->durableDeltaVersion.get() != metadata->pendingDeltaVersion) { fmt::print("real-time read [{0} - {1}) @ {2} doesn't have mutations!! durable={3}, pending={4}\n", metadata->keyRange.begin.printable(), @@ -2327,6 +2334,7 @@ ACTOR Future doBlobGranuleFileRequest(Reference bwData, Bl // prune mutations based on begin version, if possible ASSERT(metadata->durableDeltaVersion.get() == metadata->pendingDeltaVersion); + // FIXME: I think we can remove this dependsOn since we are doing push_back_deep rep.arena.dependsOn(metadata->currentDeltas.arena()); MutationsAndVersionRef* mutationIt = metadata->currentDeltas.begin(); if (granuleBeginVersion > metadata->currentDeltas.back().version) { diff --git a/fdbserver/workloads/BlobGranuleCorrectnessWorkload.actor.cpp b/fdbserver/workloads/BlobGranuleCorrectnessWorkload.actor.cpp index ea43cebbb7..e0d309954e 100644 --- a/fdbserver/workloads/BlobGranuleCorrectnessWorkload.actor.cpp +++ b/fdbserver/workloads/BlobGranuleCorrectnessWorkload.actor.cpp @@ -272,15 +272,20 @@ struct BlobGranuleCorrectnessWorkload : TestWorkload { } // FIXME: typedef this pair type and/or chunk list - ACTOR Future>>> - readFromBlob(Database cx, BlobGranuleCorrectnessWorkload* self, KeyRange range, Version version) { + ACTOR Future>>> readFromBlob( + Database cx, + BlobGranuleCorrectnessWorkload* self, + KeyRange range, + Version beginVersion, + Version readVersion) { state RangeResult out; state Standalone> chunks; state Transaction tr(cx); loop { try { - Standalone> chunks_ = wait(tr.readBlobGranules(range, 0, version)); + Standalone> chunks_ = + wait(tr.readBlobGranules(range, beginVersion, readVersion)); chunks = chunks_; break; } catch (Error& e) { @@ -289,7 +294,7 @@ struct BlobGranuleCorrectnessWorkload : TestWorkload { } for (const BlobGranuleChunkRef& chunk : chunks) { - RangeResult chunkRows = wait(readBlobGranule(chunk, range, version, self->bstore)); + RangeResult chunkRows = wait(readBlobGranule(chunk, range, beginVersion, readVersion, self->bstore)); out.arena().dependsOn(chunkRows.arena()); out.append(out.arena(), chunkRows.begin(), chunkRows.size()); } @@ -321,7 +326,7 @@ struct BlobGranuleCorrectnessWorkload : TestWorkload { Version rv = wait(self->doGrv(&tr)); state Version readVersion = rv; std::pair>> blob = - wait(self->readFromBlob(cx, self, threadData->directoryRange, readVersion)); + wait(self->readFromBlob(cx, self, threadData->directoryRange, 0, readVersion)); fmt::print("Directory {0} got {1} RV {2}\n", threadData->directoryID, doSetup ? "initial" : "final", @@ -349,6 +354,7 @@ struct BlobGranuleCorrectnessWorkload : TestWorkload { const Optional& blobValue, uint32_t startKey, uint32_t endKey, + Version beginVersion, Version readVersion, const std::pair>>& blob) { threadData->mismatches++; @@ -360,11 +366,13 @@ struct BlobGranuleCorrectnessWorkload : TestWorkload { ev.detail("DirectoryID", format("%08x", threadData->directoryID)) .detail("RangeStart", format("%08x", startKey)) .detail("RangeEnd", format("%08x", endKey)) + .detail("BeginVersion", beginVersion) .detail("Version", readVersion); - fmt::print("Found mismatch! Request for dir {0} [{1} - {2}) @ {3}\n", + fmt::print("Found mismatch! Request for dir {0} [{1} - {2}) @ {3} - {4}\n", format("%08x", threadData->directoryID), format("%08x", startKey), format("%08x", endKey), + beginVersion, readVersion); if (lastMatching.present()) { fmt::print(" last correct: {}\n", lastMatching.get().printable()); @@ -456,6 +464,29 @@ struct BlobGranuleCorrectnessWorkload : TestWorkload { readVersion); } + // because each chunk could be separately collapsed or not if we set beginVersion, we have to track it by chunk + KeyRangeMap beginVersionByChunk; + beginVersionByChunk.insert(normalKeys, 0); + int beginCollapsed = 0; + int beginNotCollapsed = 0; + for (auto& chunk : blob.second) { + if (!chunk.snapshotFile.present()) { + ASSERT(beginVersion > 0); + ASSERT(chunk.snapshotVersion == invalidVersion); + beginCollapsed++; + beginVersionByChunk.insert(chunk.keyRange, beginVersion); + } else { + ASSERT(chunk.snapshotVersion != invalidVersion); + if (beginVersion > 0) { + beginNotCollapsed++; + } + } + } + TEST(beginCollapsed > 0); // BGCorrectness got collapsed request with beginVersion > 0 + TEST(beginNotCollapsed > 0); // BGCorrectness got un-collapsed request with beginVersion > 0 + TEST(beginCollapsed > 0 && + beginNotCollapsed > 0); // BGCorrectness got both collapsed and uncollapsed in the same request! + while (checkIt != threadData->keyData.end() && checkIt->first < endKeyExclusive) { uint32_t key = checkIt->first; if (DEBUG_READ_OP(threadData->directoryID, readVersion)) { @@ -475,6 +506,16 @@ struct BlobGranuleCorrectnessWorkload : TestWorkload { for (; idIdx < checkIt->second.writes.size() && checkIt->second.writes[idIdx].writeVersion <= readVersion; idIdx++) { Key nextKeyShouldBe = threadData->getKey(key, idIdx); + Version keyBeginVersion = beginVersionByChunk.rangeContaining(nextKeyShouldBe).cvalue(); + if (keyBeginVersion > checkIt->second.writes[idIdx].writeVersion) { + if (DEBUG_READ_OP(threadData->directoryID, readVersion)) { + fmt::print("DBG READ: Skip ID {0} written @ {1} < beginVersion {2}\n", + idIdx, + checkIt->second.writes[idIdx].clearVersion, + keyBeginVersion); + } + continue; + } if (DEBUG_READ_OP(threadData->directoryID, readVersion)) { fmt::print("DBG READ: Checking ID {0} ({1}) written @ {2}\n", format("%08x", idIdx), @@ -491,6 +532,7 @@ struct BlobGranuleCorrectnessWorkload : TestWorkload { Optional(), startKeyInclusive, endKeyExclusive, + beginVersion, readVersion, blob); return false; @@ -509,6 +551,7 @@ struct BlobGranuleCorrectnessWorkload : TestWorkload { Optional(), startKeyInclusive, endKeyExclusive, + beginVersion, readVersion, blob); return false; @@ -523,6 +566,7 @@ struct BlobGranuleCorrectnessWorkload : TestWorkload { blob.first[resultIdx].value, startKeyInclusive, endKeyExclusive, + beginVersion, readVersion, blob); return false; @@ -545,6 +589,7 @@ struct BlobGranuleCorrectnessWorkload : TestWorkload { Optional(), startKeyInclusive, endKeyExclusive, + beginVersion, readVersion, blob); return false; @@ -565,6 +610,7 @@ struct BlobGranuleCorrectnessWorkload : TestWorkload { state double targetReadBytesPerSec = threadData->targetByteRate * 4; ASSERT(targetReadBytesPerSec > 0); + state Version beginVersion; state Version readVersion; TraceEvent("BlobGranuleCorrectnessReaderStart").log(); @@ -610,26 +656,42 @@ struct BlobGranuleCorrectnessWorkload : TestWorkload { state KeyRange range = KeyRangeRef(threadData->getKey(startKey, 0), threadData->getKey(endKey, 0)); // pick read version - // TODO could also pick begin version here ASSERT(threadData->writeVersions.back() >= threadData->minSuccessfulReadVersion); + size_t readVersionIdx; // randomly choose up to date vs time travel read if (deterministicRandom()->random01() < 0.5) { threadData->reads++; + readVersionIdx = threadData->writeVersions.size() - 1; readVersion = threadData->writeVersions.back(); } else { threadData->timeTravelReads++; + size_t startIdx = 0; loop { - int readVersionIdx = deterministicRandom()->randomInt(0, threadData->writeVersions.size()); + readVersionIdx = deterministicRandom()->randomInt(startIdx, threadData->writeVersions.size()); readVersion = threadData->writeVersions[readVersionIdx]; if (readVersion >= threadData->minSuccessfulReadVersion) { break; + } else { + startIdx = readVersionIdx + 1; } } } + // randomly choose begin version or not + beginVersion = 0; + if (deterministicRandom()->random01() < 0.5) { + int startIdx = 0; + int endIdxExclusive = readVersionIdx + 1; + // Choose skewed towards later versions. It's ok if beginVersion isn't readable though because it + // will collapse + size_t beginVersionIdx = (size_t)std::sqrt( + deterministicRandom()->randomInt(startIdx * startIdx, endIdxExclusive * endIdxExclusive)); + beginVersion = threadData->writeVersions[beginVersionIdx]; + } + std::pair>> blob = - wait(self->readFromBlob(cx, self, range, readVersion)); - self->validateResult(threadData, blob, startKey, endKey, 0, readVersion); + wait(self->readFromBlob(cx, self, range, beginVersion, readVersion)); + self->validateResult(threadData, blob, startKey, endKey, beginVersion, readVersion); int resultBytes = blob.first.expectedSize(); threadData->rowsRead += blob.first.size(); @@ -822,7 +884,7 @@ struct BlobGranuleCorrectnessWorkload : TestWorkload { fmt::print("Directory {0} doing final data check @ {1}\n", threadData->directoryID, readVersion); } std::pair>> blob = - wait(self->readFromBlob(cx, self, threadData->directoryRange, readVersion)); + wait(self->readFromBlob(cx, self, threadData->directoryRange, 0, readVersion)); result = self->validateResult(threadData, blob, 0, std::numeric_limits::max(), 0, readVersion); finalRowsValidated = blob.first.size(); diff --git a/fdbserver/workloads/BlobGranuleVerifier.actor.cpp b/fdbserver/workloads/BlobGranuleVerifier.actor.cpp index d4264058ca..cd97b1960b 100644 --- a/fdbserver/workloads/BlobGranuleVerifier.actor.cpp +++ b/fdbserver/workloads/BlobGranuleVerifier.actor.cpp @@ -225,7 +225,7 @@ struct BlobGranuleVerifierWorkload : TestWorkload { } for (const BlobGranuleChunkRef& chunk : chunks) { - RangeResult chunkRows = wait(readBlobGranule(chunk, range, version, self->bstore)); + RangeResult chunkRows = wait(readBlobGranule(chunk, range, 0, version, self->bstore)); out.arena().dependsOn(chunkRows.arena()); out.append(out.arena(), chunkRows.begin(), chunkRows.size()); } From 3cd1e5599ea45a9dd5c3a1ef519474f4441b1515 Mon Sep 17 00:00:00 2001 From: Josh Slocum Date: Thu, 24 Mar 2022 09:53:42 -0500 Subject: [PATCH 413/413] Cleanup --- fdbclient/BlobGranuleFiles.cpp | 9 +-------- fdbserver/BlobGranuleServerCommon.actor.cpp | 20 ++++++++++---------- fdbserver/BlobGranuleServerCommon.actor.h | 3 ++- fdbserver/BlobWorker.actor.cpp | 7 ++++--- tests/BGServerCommonUnit.toml | 4 ++-- tests/BlobGranuleFileUnit.toml | 10 ++++++++++ tests/BlobGranuleFileUnit.txt | 7 ------- tests/BlobManagerUnit.toml | 9 +++++++++ tests/BlobManagerUnit.txt | 7 ------- tests/CMakeLists.txt | 4 ++-- 10 files changed, 40 insertions(+), 40 deletions(-) create mode 100644 tests/BlobGranuleFileUnit.toml delete mode 100644 tests/BlobGranuleFileUnit.txt create mode 100644 tests/BlobManagerUnit.toml delete mode 100644 tests/BlobManagerUnit.txt diff --git a/fdbclient/BlobGranuleFiles.cpp b/fdbclient/BlobGranuleFiles.cpp index 67b119531b..af09907ad4 100644 --- a/fdbclient/BlobGranuleFiles.cpp +++ b/fdbclient/BlobGranuleFiles.cpp @@ -132,13 +132,6 @@ static void applyDeltas(const GranuleDeltas& deltas, const MutationsAndVersionRef* mutationIt = deltas.begin(); // prune beginVersion if necessary if (beginVersion > deltas.front().version) { - if (beginVersion > deltas.back().version) { - printf("beginVersion=%lld, deltas.front=%lld, deltas.back=%lld, deltas.size=%d\n", - beginVersion, - deltas.front().version, - deltas.back().version, - deltas.size()); - } ASSERT(beginVersion <= deltas.back().version); // binary search for beginVersion mutationIt = std::lower_bound(deltas.begin(), @@ -172,7 +165,7 @@ static Arena loadDeltaFile(StringRef deltaData, reader.deserialize(FileIdentifierFor::value, deltas, parseArena); if (BG_READ_DEBUG) { - fmt::print("Parsed {}} deltas from file\n", deltas.size()); + fmt::print("Parsed {} deltas from file\n", deltas.size()); } // TODO REMOVE sanity check diff --git a/fdbserver/BlobGranuleServerCommon.actor.cpp b/fdbserver/BlobGranuleServerCommon.actor.cpp index 28b4c1db44..4792984d62 100644 --- a/fdbserver/BlobGranuleServerCommon.actor.cpp +++ b/fdbserver/BlobGranuleServerCommon.actor.cpp @@ -117,9 +117,11 @@ void GranuleFiles::getFiles(Version beginVersion, Version readVersion, bool canCollapse, BlobGranuleChunkRef& chunk, - Arena& replyArena) const { + Arena& replyArena, + int64_t& deltaBytesCounter) const { BlobFileIndex dummyIndex; // for searching + // if beginVersion == 0 or we can collapse, find the latest snapshot <= readVersion auto snapshotF = snapshotFiles.end(); if (beginVersion == 0 || canCollapse) { dummyIndex.version = readVersion; @@ -138,7 +140,8 @@ void GranuleFiles::getFiles(Version beginVersion, deltaF = std::lower_bound(deltaFiles.begin(), deltaFiles.end(), dummyIndex); if (canCollapse) { ASSERT(snapshotF != snapshotFiles.end()); - // see if delta files up to snapshotVersion are smaller or larger than snapshotBytes in total + // If we can collapse, see if delta files up to snapshotVersion are smaller or larger than snapshotBytes in + // total auto deltaFCopy = deltaF; int64_t snapshotBytes = snapshotF->length; while (deltaFCopy != deltaFiles.end() && deltaFCopy->version <= snapshotF->version && snapshotBytes > 0) { @@ -171,10 +174,9 @@ void GranuleFiles::getFiles(Version beginVersion, chunk.snapshotVersion = invalidVersion; } - int64_t deltaBytes = 0; while (deltaF != deltaFiles.end() && deltaF->version < readVersion) { chunk.deltaFiles.emplace_back_deep(replyArena, deltaF->filename, deltaF->offset, deltaF->length); - deltaBytes += deltaF->length; + deltaBytesCounter += deltaF->length; ASSERT(lastIncluded < deltaF->version); lastIncluded = deltaF->version; deltaF++; @@ -182,12 +184,9 @@ void GranuleFiles::getFiles(Version beginVersion, // include last delta file that passes readVersion, if it exists if (deltaF != deltaFiles.end() && lastIncluded < readVersion) { chunk.deltaFiles.emplace_back_deep(replyArena, deltaF->filename, deltaF->offset, deltaF->length); - deltaBytes += deltaF->length; + deltaBytesCounter += deltaF->length; lastIncluded = deltaF->version; } - - // TODO wire this up, - // bwData->stats.readReqDeltaBytesReturned += deltaBytes; } static std::string makeTestFileName(Version v) { @@ -210,8 +209,9 @@ static void checkFiles(const GranuleFiles& f, std::vector expectedDeltaVersions) { Arena a; BlobGranuleChunkRef chunk; - f.getFiles(beginVersion, readVersion, canCollapse, chunk, a); - fmt::print("results({0}, {1}, {2}):\nEXPECTED: snapshot={3}\n deltas ({4}):\n", + int64_t deltaBytes = 0; + f.getFiles(beginVersion, readVersion, canCollapse, chunk, a, deltaBytes); + fmt::print("results({0}, {1}, {2}):\nEXPECTED:\n snapshot={3}\n deltas ({4}):\n", beginVersion, readVersion, canCollapse ? "T" : "F", diff --git a/fdbserver/BlobGranuleServerCommon.actor.h b/fdbserver/BlobGranuleServerCommon.actor.h index b088903967..399ae5b7b0 100644 --- a/fdbserver/BlobGranuleServerCommon.actor.h +++ b/fdbserver/BlobGranuleServerCommon.actor.h @@ -69,7 +69,8 @@ struct GranuleFiles { Version readVersion, bool canCollapse, BlobGranuleChunkRef& chunk, - Arena& replyArena) const; + Arena& replyArena, + int64_t& deltaBytesCounter) const; }; class Transaction; diff --git a/fdbserver/BlobWorker.actor.cpp b/fdbserver/BlobWorker.actor.cpp index 5c6bba8a8e..791ee7a05a 100644 --- a/fdbserver/BlobWorker.actor.cpp +++ b/fdbserver/BlobWorker.actor.cpp @@ -2313,7 +2313,10 @@ ACTOR Future doBlobGranuleFileRequest(Reference bwData, Bl chunk.includedVersion = req.readVersion; chunk.keyRange = KeyRangeRef(StringRef(rep.arena, chunkRange.begin), StringRef(rep.arena, chunkRange.end)); - chunkFiles.getFiles(granuleBeginVersion, req.readVersion, req.canCollapseBegin, chunk, rep.arena); + int64_t deltaBytes = 0; + chunkFiles.getFiles( + granuleBeginVersion, req.readVersion, req.canCollapseBegin, chunk, rep.arena, deltaBytes); + bwData->stats.readReqDeltaBytesReturned += deltaBytes; if (granuleBeginVersion > 0 && chunk.snapshotFile.present()) { TEST(true); // collapsed begin version request for efficiency didCollapse = true; @@ -2378,8 +2381,6 @@ ACTOR Future doBlobGranuleFileRequest(Reference bwData, Bl req.reply.send(rep); --bwData->stats.activeReadRequests; } catch (Error& e) { - // TODO REMOVE - fmt::print("Error in BGFRequest {0}\n", e.name()); if (e.code() == error_code_operation_cancelled) { req.reply.sendError(wrong_shard_server()); throw; diff --git a/tests/BGServerCommonUnit.toml b/tests/BGServerCommonUnit.toml index d7a5eba2ca..6d37b51e6b 100644 --- a/tests/BGServerCommonUnit.toml +++ b/tests/BGServerCommonUnit.toml @@ -1,9 +1,9 @@ [[test]] -testTitle = 'BGServerCommonUnit' +testTitle = 'BlobGranuleServerCommonUnit' useDB = false startDelay = 0 [[test.workload]] testName = 'UnitTests' maxTestCases = 0 - testsMatching = /blobgranule/server/common/ \ No newline at end of file + testsMatching = /blobgranule/server/common/ diff --git a/tests/BlobGranuleFileUnit.toml b/tests/BlobGranuleFileUnit.toml new file mode 100644 index 0000000000..d725be4d67 --- /dev/null +++ b/tests/BlobGranuleFileUnit.toml @@ -0,0 +1,10 @@ +[[test]] +testTitle = 'BlobGranuleFileUnit' +useDB = false +startDelay = 0 + + [[test.workload]] + testName = 'UnitTests' + maxTestCases = 0 + testsMatching = /blobgranule/files/ + diff --git a/tests/BlobGranuleFileUnit.txt b/tests/BlobGranuleFileUnit.txt deleted file mode 100644 index efdeffe085..0000000000 --- a/tests/BlobGranuleFileUnit.txt +++ /dev/null @@ -1,7 +0,0 @@ -testTitle=UnitTests -startDelay=0 -useDB=false - - testName=UnitTests - maxTestCases=0 - testsMatching=/blobgranule/ diff --git a/tests/BlobManagerUnit.toml b/tests/BlobManagerUnit.toml new file mode 100644 index 0000000000..9e151c94e8 --- /dev/null +++ b/tests/BlobManagerUnit.toml @@ -0,0 +1,9 @@ +[[test]] +testTitle = 'BlobManagerUnit' +useDB = false +startDelay = 0 + + [[test.workload]] + testName = 'UnitTests' + maxTestCases = 0 + testsMatching = /blobmanager/ diff --git a/tests/BlobManagerUnit.txt b/tests/BlobManagerUnit.txt deleted file mode 100644 index c0fc3da21d..0000000000 --- a/tests/BlobManagerUnit.txt +++ /dev/null @@ -1,7 +0,0 @@ -testTitle=UnitTests -startDelay=0 -useDB=false - - testName=UnitTests - maxTestCases=0 - testsMatching=/blobmanager/ diff --git a/tests/CMakeLists.txt b/tests/CMakeLists.txt index d9a3eca683..44f0cf4d4e 100644 --- a/tests/CMakeLists.txt +++ b/tests/CMakeLists.txt @@ -51,8 +51,8 @@ if(WITH_PYTHON) add_fdb_test(TEST_FILES BandwidthThrottle.txt IGNORE) add_fdb_test(TEST_FILES BigInsert.txt IGNORE) add_fdb_test(TEST_FILES BGServerCommonUnit.toml) - add_fdb_test(TEST_FILES BlobGranuleFileUnit.txt) # TODO change these to toml - add_fdb_test(TEST_FILES BlobManagerUnit.txt) + add_fdb_test(TEST_FILES BlobGranuleFileUnit.toml) + add_fdb_test(TEST_FILES BlobManagerUnit.toml) add_fdb_test(TEST_FILES ConsistencyCheck.txt IGNORE) add_fdb_test(TEST_FILES DDMetricsExclude.txt IGNORE) add_fdb_test(TEST_FILES DataDistributionMetrics.txt IGNORE)