From 7074dee1fbf6763c0d463c377c2e47d8ef2c4f6f Mon Sep 17 00:00:00 2001 From: Kevin Pulo Date: Tue, 19 May 2020 14:29:26 +1000 Subject: SERVER-47914 Move clusterTime from LogicalClock to VectorClock --- src/mongo/dbtests/SConscript | 11 +- src/mongo/dbtests/storage_timestamp_tests.cpp | 269 ++++++++++++++++++++------ 2 files changed, 213 insertions(+), 67 deletions(-) (limited to 'src/mongo/dbtests') diff --git a/src/mongo/dbtests/SConscript b/src/mongo/dbtests/SConscript index 3337f065559..02939bd9933 100644 --- a/src/mongo/dbtests/SConscript +++ b/src/mongo/dbtests/SConscript @@ -143,8 +143,6 @@ if not has_option('noshell') and usemozjs: "$BUILD_DIR/mongo/db/exec/document_value/document_value_test_util", "$BUILD_DIR/mongo/db/index/index_access_methods", "$BUILD_DIR/mongo/db/index/index_build_interceptor", - "$BUILD_DIR/mongo/db/logical_clock", - "$BUILD_DIR/mongo/db/logical_session_id_helpers", "$BUILD_DIR/mongo/db/logical_time_metadata_hook", "$BUILD_DIR/mongo/db/mongohasher", "$BUILD_DIR/mongo/db/query/collation/collator_interface_mock", @@ -173,15 +171,8 @@ if not has_option('noshell') and usemozjs: "$BUILD_DIR/mongo/db/storage/wiredtiger/storage_wiredtiger" if wiredtiger else [], "$BUILD_DIR/mongo/db/traffic_reader", "$BUILD_DIR/mongo/db/transaction", - "$BUILD_DIR/mongo/executor/network_interface_factory", - "$BUILD_DIR/mongo/executor/network_interface_thread_pool", - "$BUILD_DIR/mongo/executor/thread_pool_task_executor", - "$BUILD_DIR/mongo/rpc/protocol", - "$BUILD_DIR/mongo/scripting/scripting", - "$BUILD_DIR/mongo/shell/benchrun", - "$BUILD_DIR/mongo/shell/mongojs", + "$BUILD_DIR/mongo/db/vector_clock", "$BUILD_DIR/mongo/shell/shell_utils", - "$BUILD_DIR/mongo/transport/message_compressor", "$BUILD_DIR/mongo/transport/message_compressor_options_server", "$BUILD_DIR/mongo/transport/transport_layer_manager", "$BUILD_DIR/mongo/util/clock_source_mock", diff --git a/src/mongo/dbtests/storage_timestamp_tests.cpp b/src/mongo/dbtests/storage_timestamp_tests.cpp index 3181b1cb0c0..19603ffdbad 100644 --- a/src/mongo/dbtests/storage_timestamp_tests.cpp +++ b/src/mongo/dbtests/storage_timestamp_tests.cpp @@ -52,7 +52,6 @@ #include "mongo/db/index/index_build_interceptor.h" #include "mongo/db/index/index_descriptor.h" #include "mongo/db/index_builds_coordinator.h" -#include "mongo/db/logical_clock.h" #include "mongo/db/multi_key_path_tracker.h" #include "mongo/db/op_observer_registry.h" #include "mongo/db/repl/apply_ops.h" @@ -83,6 +82,7 @@ #include "mongo/db/storage/storage_engine_impl.h" #include "mongo/db/transaction_participant.h" #include "mongo/db/transaction_participant_gen.h" +#include "mongo/db/vector_clock_mutable.h" #include "mongo/dbtests/dbtests.h" #include "mongo/logv2/log.h" #include "mongo/stdx/future.h" @@ -159,12 +159,13 @@ class StorageTimestampTest { public: ServiceContext::UniqueOperationContext _opCtxRaii = cc().makeOperationContext(); OperationContext* _opCtx = _opCtxRaii.get(); - LogicalClock* _clock = LogicalClock::get(_opCtx); + VectorClockMutable* _clock = VectorClockMutable::get(_opCtx); // Set up Timestamps in the past, present, and future. - const LogicalTime pastLt = _clock->reserveTicks(1); + static constexpr auto ClusterTime = VectorClock::Component::ClusterTime; + const LogicalTime pastLt = _clock->tick(ClusterTime, 1); const Timestamp pastTs = pastLt.asTimestamp(); - const LogicalTime presentLt = _clock->reserveTicks(1); + const LogicalTime presentLt = _clock->tick(ClusterTime, 1); const Timestamp presentTs = presentLt.asTimestamp(); const LogicalTime futureLt = presentLt.addTicks(1); const Timestamp futureTs = futureLt.asTimestamp(); @@ -211,7 +212,7 @@ public: repl::setOplogCollectionName(getGlobalServiceContext()); repl::createOplog(_opCtx); - ASSERT_OK(_clock->advanceClusterTime(LogicalTime(Timestamp(1, 0)))); + _clock->tickTo(ClusterTime, LogicalTime(Timestamp(1, 0))); ASSERT_EQUALS(presentTs, pastLt.addTicks(1).asTimestamp()); setReplCoordAppliedOpTime(repl::OpTime(presentTs, presentTerm)); @@ -716,7 +717,7 @@ public: AutoGetCollection autoColl(_opCtx, nss, LockMode::MODE_IX); const std::int32_t docsToInsert = 10; - const LogicalTime firstInsertTime = _clock->reserveTicks(docsToInsert); + const LogicalTime firstInsertTime = _clock->tick(ClusterTime, docsToInsert); for (std::int32_t idx = 0; idx < docsToInsert; ++idx) { BSONObjBuilder result; ASSERT_OK(applyOps( @@ -765,7 +766,7 @@ public: AutoGetCollection autoColl(_opCtx, nss, LockMode::MODE_IX); const std::int32_t docsToInsert = 10; - const LogicalTime firstInsertTime = _clock->reserveTicks(docsToInsert); + const LogicalTime firstInsertTime = _clock->tick(ClusterTime, docsToInsert); BSONObjBuilder oplogCommonBuilder; oplogCommonBuilder << "v" << 2 << "op" @@ -824,7 +825,7 @@ public: // Insert some documents. const std::int32_t docsToInsert = 10; - const LogicalTime firstInsertTime = _clock->reserveTicks(docsToInsert); + const LogicalTime firstInsertTime = _clock->tick(ClusterTime, docsToInsert); const LogicalTime lastInsertTime = firstInsertTime.addTicks(docsToInsert - 1); WriteUnitOfWork wunit(_opCtx); for (std::int32_t num = 0; num < docsToInsert; ++num) { @@ -837,7 +838,7 @@ public: ASSERT_EQ(docsToInsert, itCount(autoColl.getCollection())); // Delete all documents one at a time. - const LogicalTime startDeleteTime = _clock->reserveTicks(docsToInsert); + const LogicalTime startDeleteTime = _clock->tick(ClusterTime, docsToInsert); for (std::int32_t num = 0; num < docsToInsert; ++num) { ASSERT_OK(doNonAtomicApplyOps( nss.db().toString(), @@ -871,7 +872,7 @@ public: AutoGetCollection autoColl(_opCtx, nss, LockMode::MODE_IX); // Insert one document that will go through a series of updates. - const LogicalTime insertTime = _clock->reserveTicks(1); + const LogicalTime insertTime = _clock->tick(ClusterTime, 1); WriteUnitOfWork wunit(_opCtx); insertDocument(autoColl.getCollection(), InsertStatement(BSON("_id" << 0), insertTime.asTimestamp(), 0LL)); @@ -896,7 +897,7 @@ public: << "theOtherSet")), BSON("_id" << 0 << "theMap" << BSON("val" << 1) << "theOtherSet" << BSONArray())}}; - const LogicalTime firstUpdateTime = _clock->reserveTicks(updates.size()); + const LogicalTime firstUpdateTime = _clock->tick(ClusterTime, updates.size()); for (std::size_t idx = 0; idx < updates.size(); ++idx) { ASSERT_OK(doNonAtomicApplyOps( nss.db().toString(), @@ -933,7 +934,7 @@ public: AutoGetCollection autoColl(_opCtx, nss, LockMode::MODE_IX); - const LogicalTime insertTime = _clock->reserveTicks(2); + const LogicalTime insertTime = _clock->tick(ClusterTime, 2); // This applyOps runs into an insert of `{_id: 0, field: 0}` followed by a second insert // on the same collection with `{_id: 0}`. It's expected for this second insert to be @@ -985,7 +986,7 @@ public: AutoGetCollection autoColl(_opCtx, nss, LockMode::MODE_IX); // Reserve a timestamp before the inserts should happen. - const LogicalTime preInsertTimestamp = _clock->reserveTicks(1); + const LogicalTime preInsertTimestamp = _clock->tick(ClusterTime, 1); auto swResult = doAtomicApplyOps(nss.db().toString(), {BSON("op" @@ -1037,7 +1038,7 @@ public: AutoGetCollection autoColl(_opCtx, nss, LockMode::MODE_IX); - const LogicalTime preInsertTimestamp = _clock->reserveTicks(1); + const LogicalTime preInsertTimestamp = _clock->tick(ClusterTime, 1); auto swResult = doAtomicApplyOps(nss.db().toString(), {BSON("op" @@ -1293,10 +1294,10 @@ public: _coordinatorMock->alwaysAllowWrites(false); - const LogicalTime pastTime = _clock->reserveTicks(1); - const LogicalTime insertTime0 = _clock->reserveTicks(1); - const LogicalTime insertTime1 = _clock->reserveTicks(1); - const LogicalTime insertTime2 = _clock->reserveTicks(1); + const LogicalTime pastTime = _clock->tick(ClusterTime, 1); + const LogicalTime insertTime0 = _clock->tick(ClusterTime, 1); + const LogicalTime insertTime1 = _clock->tick(ClusterTime, 1); + const LogicalTime insertTime2 = _clock->tick(ClusterTime, 1); BSONObj doc0 = BSON("_id" << 0 << "a" << 3); BSONObj doc1 = BSON("_id" << 1 << "a" << BSON_ARRAY(1 << 2)); @@ -1364,11 +1365,11 @@ public: _coordinatorMock->alwaysAllowWrites(false); ASSERT_OK(_coordinatorMock->setFollowerMode({repl::MemberState::MS::RS_STARTUP2})); - const LogicalTime pastTime = _clock->reserveTicks(1); - const LogicalTime insertTime0 = _clock->reserveTicks(1); - const LogicalTime indexBuildTime = _clock->reserveTicks(1); - const LogicalTime insertTime1 = _clock->reserveTicks(1); - const LogicalTime insertTime2 = _clock->reserveTicks(1); + const LogicalTime pastTime = _clock->tick(ClusterTime, 1); + const LogicalTime insertTime0 = _clock->tick(ClusterTime, 1); + const LogicalTime indexBuildTime = _clock->tick(ClusterTime, 1); + const LogicalTime insertTime1 = _clock->tick(ClusterTime, 1); + const LogicalTime insertTime2 = _clock->tick(ClusterTime, 1); BSONObj doc0 = BSON("_id" << 0 << "a" << 3); BSONObj doc1 = BSON("_id" << 1 << "a" << BSON_ARRAY(1 << 2)); @@ -1455,7 +1456,7 @@ public: << static_cast(kIndexVersion)); ASSERT_OK(dbtests::createIndexFromSpec(_opCtx, nss.ns(), indexSpec)); - const LogicalTime pastTime = _clock->reserveTicks(1); + const LogicalTime pastTime = _clock->tick(ClusterTime, 1); const LogicalTime insertTime = pastTime.addTicks(1); BSONObj doc = BSON("_id" << 1 << "a" << BSON_ARRAY(1 << 2)); @@ -1484,7 +1485,7 @@ public: << static_cast(kIndexVersion)); ASSERT_OK(dbtests::createIndexFromSpec(_opCtx, nss.ns(), indexSpec)); - const LogicalTime pastTime = _clock->reserveTicks(1); + const LogicalTime pastTime = _clock->tick(ClusterTime, 1); const LogicalTime insertTime = pastTime.addTicks(1); BSONObj doc = BSON("_id" << 1 << "a" << BSON_ARRAY(1 << 2)); @@ -1521,14 +1522,14 @@ public: ASSERT_OK(dbtests::createIndexFromSpec(_opCtx, nss.ns(), indexSpec)); } - auto presentTs = _clock->getClusterTime().asTimestamp(); + auto presentTs = _clock->getTime()[ClusterTime].asTimestamp(); // This test does not run a real ReplicationCoordinator, so must advance the snapshot // manager manually. auto storageEngine = cc().getServiceContext()->getStorageEngine(); storageEngine->getSnapshotManager()->setLastApplied(presentTs); - const auto beforeTxnTime = _clock->reserveTicks(1); + const auto beforeTxnTime = _clock->tick(ClusterTime, 1); auto beforeTxnTs = beforeTxnTime.asTimestamp(); const auto multikeyNoopTime = beforeTxnTime.addTicks(1); auto multikeyNoopTs = multikeyNoopTime.asTimestamp(); @@ -1748,7 +1749,7 @@ public: auto durableCatalog = storageEngine->getCatalog(); // Declare the database to be in a "synced" state, i.e: in steady-state replication. - Timestamp syncTime = _clock->reserveTicks(1).asTimestamp(); + Timestamp syncTime = _clock->tick(ClusterTime, 1).asTimestamp(); invariant(!syncTime.isNull()); storageEngine->setInitialDataTimestamp(syncTime); @@ -1777,7 +1778,7 @@ public: // side-effect of not timestamping the collection creation. repl::UnreplicatedWritesBlock notReplicated(_opCtx); if (nss.isReplicated()) { - TimestampBlock tsBlock(_opCtx, _clock->reserveTicks(1).asTimestamp()); + TimestampBlock tsBlock(_opCtx, _clock->tick(ClusterTime, 1).asTimestamp()); reset(nss); } else { reset(nss); @@ -1801,7 +1802,7 @@ public: // Reserve a tick, this represents a time after the rename in which the `kvDropDatabase` // ident for `kvDropDatabase` still exists. - const Timestamp postRenameTime = _clock->reserveTicks(1).asTimestamp(); + const Timestamp postRenameTime = _clock->tick(ClusterTime, 1).asTimestamp(); // If the storage engine is managing drops internally, the ident should not be visible after // a drop. @@ -1812,7 +1813,7 @@ public: assertIdentsExistAtTimestamp(durableCatalog, collIdent, indexIdent, postRenameTime); } - const Timestamp dropTime = _clock->reserveTicks(1).asTimestamp(); + const Timestamp dropTime = _clock->tick(ClusterTime, 1).asTimestamp(); if (SimulatePrimary) { ASSERT_OK(dropDatabaseForApplyOps(_opCtx, nss.db().toString())); } else { @@ -1869,7 +1870,7 @@ public: AutoGetCollection autoColl(_opCtx, nss, LockMode::MODE_X); RecordId catalogId = autoColl.getCollection()->getCatalogId(); - const LogicalTime insertTimestamp = _clock->reserveTicks(1); + const LogicalTime insertTimestamp = _clock->tick(ClusterTime, 1); { WriteUnitOfWork wuow(_opCtx); insertDocument(autoColl.getCollection(), @@ -1890,7 +1891,7 @@ public: indexer.abortIndexBuild( _opCtx, autoColl.getCollection(), MultiIndexBlock::kNoopOnCleanUpFn); }); - const LogicalTime beforeIndexBuild = _clock->reserveTicks(2); + const LogicalTime beforeIndexBuild = _clock->tick(ClusterTime, 2); BSONObj indexInfoObj; { // Primaries do not have a wrapping `TimestampBlock`; secondaries do. @@ -1916,7 +1917,7 @@ public: indexInfoObj = std::move(swIndexInfoObj.getValue()[0]); } - const LogicalTime afterIndexInit = _clock->reserveTicks(2); + const LogicalTime afterIndexInit = _clock->tick(ClusterTime, 2); // Inserting all the documents has the side-effect of setting internal state on the index // builder that the index is multikey. @@ -1938,7 +1939,7 @@ public: _opCtx, nss, autoColl.getCollection()->uuid(), indexSpec, false); } else { ASSERT_OK(_opCtx->recoveryUnit()->setTimestamp( - _clock->getClusterTime().asTimestamp())); + _clock->getTime()[ClusterTime].asTimestamp())); } }, MultiIndexBlock::kNoopOnCommitFn)); @@ -1946,7 +1947,7 @@ public: } abortOnExit.dismiss(); - const Timestamp afterIndexBuild = _clock->reserveTicks(1).asTimestamp(); + const Timestamp afterIndexBuild = _clock->tick(ClusterTime, 1).asTimestamp(); const std::string indexIdent = getNewIndexIdentAtTime(durableCatalog, origIdents, Timestamp::min()); @@ -1981,6 +1982,160 @@ public: } }; +template +class TimestampIndexBuildDrain : public StorageTimestampTest { +public: + void run() { + const bool SimulateSecondary = !SimulatePrimary; + if (SimulateSecondary) { + // The MemberState is inspected during index builds to use a "ghost" write to timestamp + // index completion. + ASSERT_OK(_coordinatorMock->setFollowerMode({repl::MemberState::MS::RS_SECONDARY})); + } + + NamespaceString nss("unittests.timestampIndexBuildDrain"); + reset(nss); + + AutoGetCollection autoColl(_opCtx, nss, LockMode::MODE_X); + + // Build an index on `{a: 1}`. + MultiIndexBlock indexer; + auto abortOnExit = makeGuard([&] { + indexer.abortIndexBuild( + _opCtx, autoColl.getCollection(), MultiIndexBlock::kNoopOnCleanUpFn); + }); + const LogicalTime beforeIndexBuild = _clock->tick(ClusterTime, 2); + BSONObj indexInfoObj; + { + // Primaries do not have a wrapping `TimestampBlock`; secondaries do. + const Timestamp commitTimestamp = + SimulatePrimary ? Timestamp::min() : beforeIndexBuild.addTicks(1).asTimestamp(); + TimestampBlock tsBlock(_opCtx, commitTimestamp); + + // Secondaries will also be in an `UnreplicatedWritesBlock` that prevents the `logOp` + // from making creating an entry. + boost::optional unreplicated; + if (SimulateSecondary) { + unreplicated.emplace(_opCtx); + } + + auto swIndexInfoObj = indexer.init( + _opCtx, + autoColl.getCollection(), + {BSON("v" << 2 << "unique" << true << "name" + << "a_1" + << "ns" << nss.ns() << "key" << BSON("a" << 1))}, + MultiIndexBlock::makeTimestampedIndexOnInitFn(_opCtx, autoColl.getCollection())); + ASSERT_OK(swIndexInfoObj.getStatus()); + indexInfoObj = std::move(swIndexInfoObj.getValue()[0]); + } + + const LogicalTime afterIndexInit = _clock->tick(ClusterTime, 1); + + // Insert a document that will be intercepted and need to be drained. This timestamp will + // become the lastApplied time. + const LogicalTime firstInsert = _clock->tick(ClusterTime, 1); + { + WriteUnitOfWork wuow(_opCtx); + insertDocument(autoColl.getCollection(), + InsertStatement(BSON("_id" << 0 << "a" << 1), + firstInsert.asTimestamp(), + presentTerm)); + wuow.commit(); + ASSERT_EQ(1, itCount(autoColl.getCollection())); + } + + // Index build drain will timestamp writes from the side table into the index with the + // lastApplied timestamp. This is because these writes are not associated with any specific + // oplog entry. + ASSERT_EQ(repl::ReplicationCoordinator::get(getGlobalServiceContext()) + ->getMyLastAppliedOpTime() + .getTimestamp(), + firstInsert.asTimestamp()); + + ASSERT_OK(indexer.drainBackgroundWrites(_opCtx, + RecoveryUnit::ReadSource::kUnset, + IndexBuildInterceptor::DrainYieldPolicy::kNoYield)); + + auto indexCatalog = autoColl.getCollection()->getIndexCatalog(); + const IndexCatalogEntry* buildingIndex = indexCatalog->getEntry( + indexCatalog->findIndexByName(_opCtx, "a_1", /* includeUnfinished */ true)); + ASSERT(buildingIndex); + + { + // Before the drain, there are no writes to apply. + OneOffRead oor(_opCtx, afterIndexInit.asTimestamp()); + ASSERT_TRUE(buildingIndex->indexBuildInterceptor()->areAllWritesApplied(_opCtx)); + } + + // Note: In this case, we can't observe a state where all writes are not applied, because + // the index build drain effectively rewrites history by retroactively committing the drain + // at the same time as the first insert, meaning there is no point-in-time with undrained + // writes. This is fine, as long as the drain does not commit at a time before this insert. + + { + // At time of the first insert, all writes are applied. + OneOffRead oor(_opCtx, firstInsert.asTimestamp()); + ASSERT_TRUE(buildingIndex->indexBuildInterceptor()->areAllWritesApplied(_opCtx)); + } + + // Insert a second document that will be intercepted and need to be drained. + const LogicalTime secondInsert = _clock->tick(ClusterTime, 1); + { + WriteUnitOfWork wuow(_opCtx); + insertDocument(autoColl.getCollection(), + InsertStatement(BSON("_id" << 1 << "a" << 2), + secondInsert.asTimestamp(), + presentTerm)); + wuow.commit(); + ASSERT_EQ(2, itCount(autoColl.getCollection())); + } + + // Advance the lastApplied optime to observe a point before the drain where there are + // un-drained writes. + const LogicalTime afterSecondInsert = _clock->tick(ClusterTime, 1); + setReplCoordAppliedOpTime(repl::OpTime(afterSecondInsert.asTimestamp(), presentTerm)); + + ASSERT_OK(indexer.drainBackgroundWrites(_opCtx, + RecoveryUnit::ReadSource::kUnset, + IndexBuildInterceptor::DrainYieldPolicy::kNoYield)); + + { + // At time of the second insert, there are un-drained writes. + OneOffRead oor(_opCtx, secondInsert.asTimestamp()); + ASSERT_FALSE(buildingIndex->indexBuildInterceptor()->areAllWritesApplied(_opCtx)); + } + + { + // After the second insert, also the lastApplied time, all writes are applied. + OneOffRead oor(_opCtx, afterSecondInsert.asTimestamp()); + ASSERT_TRUE(buildingIndex->indexBuildInterceptor()->areAllWritesApplied(_opCtx)); + } + + ASSERT_OK(indexer.checkConstraints(_opCtx)); + + { + WriteUnitOfWork wuow(_opCtx); + ASSERT_OK(indexer.commit( + _opCtx, + autoColl.getCollection(), + [&](const BSONObj& indexSpec) { + if (SimulatePrimary) { + // The timestamping responsibility for each index is placed on the caller. + _opCtx->getServiceContext()->getOpObserver()->onCreateIndex( + _opCtx, nss, autoColl.getCollection()->uuid(), indexSpec, false); + } else { + ASSERT_OK(_opCtx->recoveryUnit()->setTimestamp( + _clock->getTime()[ClusterTime].asTimestamp())); + } + }, + MultiIndexBlock::kNoopOnCommitFn)); + wuow.commit(); + } + abortOnExit.dismiss(); + } +}; + class TimestampMultiIndexBuilds : public StorageTimestampTest { public: void run() { @@ -2010,7 +2165,7 @@ public: { AutoGetCollection autoColl(_opCtx, nss, LockMode::MODE_X); - const LogicalTime insertTimestamp = _clock->reserveTicks(1); + const LogicalTime insertTimestamp = _clock->tick(ClusterTime, 1); WriteUnitOfWork wuow(_opCtx); insertDocument(autoColl.getCollection(), @@ -2117,7 +2272,7 @@ public: { AutoGetCollection autoColl(_opCtx, nss, LockMode::MODE_X); - const LogicalTime insertTimestamp = _clock->reserveTicks(1); + const LogicalTime insertTimestamp = _clock->tick(ClusterTime, 1); WriteUnitOfWork wuow(_opCtx); insertDocument(autoColl.getCollection(), @@ -2235,8 +2390,8 @@ public: { AutoGetCollection autoColl(_opCtx, nss, LockMode::MODE_X); - auto insertTimestamp1 = _clock->reserveTicks(1); - auto insertTimestamp2 = _clock->reserveTicks(1); + auto insertTimestamp1 = _clock->tick(ClusterTime, 1); + auto insertTimestamp2 = _clock->tick(ClusterTime, 1); // Insert two documents with the same value for field 'a' so that // we will fail to create a unique index. @@ -2334,7 +2489,7 @@ public: AutoGetCollection autoColl(_opCtx, nss, LockMode::MODE_X); - const LogicalTime insertTimestamp = _clock->reserveTicks(1); + const LogicalTime insertTimestamp = _clock->tick(ClusterTime, 1); { WriteUnitOfWork wuow(_opCtx); insertDocument(autoColl.getCollection(), @@ -2346,7 +2501,7 @@ public: } - const Timestamp beforeIndexBuild = _clock->reserveTicks(1).asTimestamp(); + const Timestamp beforeIndexBuild = _clock->tick(ClusterTime, 1).asTimestamp(); // Save the pre-state idents so we can capture the specific ident related to index // creation. @@ -2359,7 +2514,7 @@ public: createIndex(autoColl.getCollection(), str::stream() << key << "_1", BSON(key << 1)); // Timestamps at the completion of each index build. - afterCreateTimestamps.push_back(_clock->reserveTicks(1).asTimestamp()); + afterCreateTimestamps.push_back(_clock->tick(ClusterTime, 1).asTimestamp()); // Add the new ident to the vector and reset the current idents. indexIdents.push_back( @@ -2375,7 +2530,7 @@ public: durableCatalog, "", indexIdents[i], afterCreateTimestamps[i]); } - const LogicalTime beforeDropTs = _clock->getClusterTime(); + const LogicalTime beforeDropTs = _clock->getTime()[ClusterTime]; // Drop all of the indexes. BSONObjBuilder result; @@ -2490,7 +2645,7 @@ public: BSON("_id" << 2 << "a" << BSON_ARRAY(4 << 5) << "b" << BSON_ARRAY(4 << 5)); // NOTE: This test does not test any timestamp reads. - const LogicalTime insert1 = _clock->reserveTicks(1); + const LogicalTime insert1 = _clock->tick(ClusterTime, 1); { LOGV2(22505, "inserting {badDoc1}", "badDoc1"_attr = badDoc1); WriteUnitOfWork wuow(_opCtx); @@ -2499,7 +2654,7 @@ public: wuow.commit(); } - const LogicalTime insert2 = _clock->reserveTicks(1); + const LogicalTime insert2 = _clock->tick(ClusterTime, 1); { LOGV2(22506, "inserting {badDoc2}", "badDoc2"_attr = badDoc2); WriteUnitOfWork wuow(_opCtx); @@ -2518,7 +2673,7 @@ public: const auto buildUUID = UUID::gen(); indexer.setTwoPhaseBuildUUID(buildUUID); - const LogicalTime indexInit = _clock->reserveTicks(3); + const LogicalTime indexInit = _clock->tick(ClusterTime, 3); // First, simulate being a secondary. Indexing errors are ignored. { @@ -2717,7 +2872,7 @@ public: NamespaceString nss(dbName, "oplogApplicationOnPrimary"); BSONObj doc = BSON("_id" << 1 << "field" << 1); - const LogicalTime setupStart = _clock->reserveTicks(1); + const LogicalTime setupStart = _clock->tick(ClusterTime, 1); UUID collUUID = UUID::gen(); { @@ -2738,7 +2893,7 @@ public: auto coll = autoColl.getCollection(); ASSERT(coll); - const auto presentTs = _clock->getClusterTime().asTimestamp(); + const auto presentTs = _clock->getTime()[ClusterTime].asTimestamp(); assertDocumentAtTimestamp(coll, presentTs, doc); } @@ -2747,7 +2902,7 @@ public: // the applyOps command no longer allows createIndexes (see SERVER-41554). _coordinatorMock->alwaysAllowWrites(false); { - const auto beforeBuildTime = _clock->reserveTicks(2); + const auto beforeBuildTime = _clock->tick(ClusterTime, 2); const auto startBuildTs = beforeBuildTime.addTicks(1).asTimestamp(); // Grab the existing idents to identify the ident created by the index build. @@ -2965,12 +3120,12 @@ public: ui = coll->uuid(); } - presentTs = _clock->getClusterTime().asTimestamp(); + presentTs = _clock->getTime()[ClusterTime].asTimestamp(); // This test does not run a real ReplicationCoordinator, so must advance the snapshot // manager manually. auto storageEngine = cc().getServiceContext()->getStorageEngine(); storageEngine->getSnapshotManager()->setLastApplied(presentTs); - const auto beforeTxnTime = _clock->reserveTicks(1); + const auto beforeTxnTime = _clock->tick(ClusterTime, 1); beforeTxnTs = beforeTxnTime.asTimestamp(); commitEntryTs = beforeTxnTime.addTicks(1).asTimestamp(); @@ -3096,7 +3251,7 @@ private: class MultiOplogEntryTransaction : public MultiDocumentTransactionTest { public: MultiOplogEntryTransaction() : MultiDocumentTransactionTest("multiOplogEntryTransaction") { - const auto currentTime = _clock->getClusterTime(); + const auto currentTime = _clock->getTime()[ClusterTime]; firstOplogEntryTs = currentTime.addTicks(1).asTimestamp(); commitEntryTs = currentTime.addTicks(2).asTimestamp(); } @@ -3200,7 +3355,7 @@ class CommitPreparedMultiOplogEntryTransaction : public MultiDocumentTransaction public: CommitPreparedMultiOplogEntryTransaction() : MultiDocumentTransactionTest("preparedMultiOplogEntryTransaction") { - const auto currentTime = _clock->getClusterTime(); + const auto currentTime = _clock->getTime()[ClusterTime]; firstOplogEntryTs = currentTime.addTicks(1).asTimestamp(); prepareEntryTs = currentTime.addTicks(2).asTimestamp(); commitEntryTs = currentTime.addTicks(3).asTimestamp(); @@ -3395,7 +3550,7 @@ class AbortPreparedMultiOplogEntryTransaction : public MultiDocumentTransactionT public: AbortPreparedMultiOplogEntryTransaction() : MultiDocumentTransactionTest("preparedMultiOplogEntryTransaction") { - const auto currentTime = _clock->getClusterTime(); + const auto currentTime = _clock->getTime()[ClusterTime]; prepareEntryTs = currentTime.addTicks(1).asTimestamp(); abortEntryTs = currentTime.addTicks(2).asTimestamp(); } @@ -3504,7 +3659,7 @@ public: auto txnParticipant = TransactionParticipant::get(_opCtx); ASSERT(txnParticipant); - const auto currentTime = _clock->getClusterTime(); + const auto currentTime = _clock->getTime()[ClusterTime]; const auto prepareTs = currentTime.addTicks(1).asTimestamp(); commitEntryTs = currentTime.addTicks(2).asTimestamp(); LOGV2(22514, "Prepare TS: {prepareTs}", "prepareTs"_attr = prepareTs); @@ -3605,7 +3760,7 @@ public: auto txnParticipant = TransactionParticipant::get(_opCtx); ASSERT(txnParticipant); - const auto currentTime = _clock->getClusterTime(); + const auto currentTime = _clock->getTime()[ClusterTime]; const auto prepareTs = currentTime.addTicks(1).asTimestamp(); const auto abortEntryTs = currentTime.addTicks(2).asTimestamp(); LOGV2(22515, "Prepare TS: {prepareTs}", "prepareTs"_attr = prepareTs); -- cgit v1.2.1