From b8163bb5a2ec57c3fa04816bcd0dfa8350fbd60c Mon Sep 17 00:00:00 2001 From: Gregory Noma Date: Tue, 1 Oct 2024 21:13:07 -0400 Subject: [PATCH] SERVER-95335 Remove namespace from bucket catalog interfaces (#27694) GitOrigin-RevId: 6fbbf8acf73554ff9cebab3266b71a8b5d96cfc4 --- .../db/query/write_ops/write_ops_exec.cpp | 10 +- .../bucket_catalog/bucket_catalog.cpp | 23 ++-- .../bucket_catalog/bucket_catalog.h | 8 +- .../bucket_catalog/bucket_catalog_helpers.h | 1 - .../bucket_catalog_internal.cpp | 17 +-- .../bucket_catalog/bucket_catalog_internal.h | 6 +- .../bucket_catalog/bucket_catalog_test.cpp | 130 +++++++++--------- .../bucket_state_registry_test.cpp | 4 +- .../db/timeseries/timeseries_write_util.cpp | 10 +- 9 files changed, 86 insertions(+), 123 deletions(-) diff --git a/src/mongo/db/query/write_ops/write_ops_exec.cpp b/src/mongo/db/query/write_ops/write_ops_exec.cpp index e281ae36785a7..99bde2558e3c6 100644 --- a/src/mongo/db/query/write_ops/write_ops_exec.cpp +++ b/src/mongo/db/query/write_ops/write_ops_exec.cpp @@ -2593,7 +2593,7 @@ bool commitTimeseriesBucket(OperationContext* opCtx, timeseries::bucket_catalog::GlobalBucketCatalog::get(opCtx->getServiceContext()); auto metadata = getMetadata(bucketCatalog, batch->bucketId); - auto status = prepareCommit(bucketCatalog, request.getNamespace(), batch); + auto status = prepareCommit(bucketCatalog, batch); if (!status.isOK()) { invariant(timeseries::bucket_catalog::isWriteBatchFinished(*batch)); docsToRetry->push_back(index); @@ -2656,13 +2656,12 @@ bool commitTimeseriesBucket(OperationContext* opCtx, } timeseries::getOpTimeAndElectionId(opCtx, opTime, electionId); - auto bucketsNs = request.getNamespace().makeTimeseriesBucketsNamespace(); auto closedBucket = finish(bucketCatalog, - bucketsNs, batch, timeseries::bucket_catalog::CommitInfo{*opTime, *electionId}, - timeseries::getPostCommitDebugChecks(opCtx, bucketsNs)); + timeseries::getPostCommitDebugChecks( + opCtx, request.getNamespace().makeTimeseriesBucketsNamespace())); if (closedBucket) { // If this write closed a bucket, compress the bucket @@ -2710,7 +2709,7 @@ bool commitTimeseriesBucketsAtomically(OperationContext* opCtx, for (auto batch : batchesToCommit) { auto metadata = getMetadata(bucketCatalog, batch.get()->bucketId); - auto prepareCommitStatus = prepareCommit(bucketCatalog, request.getNamespace(), batch); + auto prepareCommitStatus = prepareCommit(bucketCatalog, batch); if (!prepareCommitStatus.isOK()) { abortStatus = prepareCommitStatus; return false; @@ -2741,7 +2740,6 @@ bool commitTimeseriesBucketsAtomically(OperationContext* opCtx, for (auto batch : batchesToCommit) { auto closedBucket = finish(bucketCatalog, - bucketsNs, batch, timeseries::bucket_catalog::CommitInfo{*opTime, *electionId}, timeseries::getPostCommitDebugChecks(opCtx, bucketsNs)); diff --git a/src/mongo/db/timeseries/bucket_catalog/bucket_catalog.cpp b/src/mongo/db/timeseries/bucket_catalog/bucket_catalog.cpp index a496b07cd68e8..36d7eac8a0421 100644 --- a/src/mongo/db/timeseries/bucket_catalog/bucket_catalog.cpp +++ b/src/mongo/db/timeseries/bucket_catalog/bucket_catalog.cpp @@ -259,7 +259,6 @@ void getDetailedMemoryUsage(const BucketCatalog& catalog, BSONObjBuilder& builde } StatusWith tryInsert(BucketCatalog& catalog, - const NamespaceString& nss, const StringDataComparator* comparator, const BSONObj& doc, OperationId opId, @@ -280,7 +279,7 @@ StatusWith tryInsert(BucketCatalog& catalog, stdx::lock_guard stripeLock{stripe.mutex}; Bucket* bucket = internal::useBucket( - catalog, stripe, stripeLock, nss, insertContext, internal::AllowBucketCreation::kNo, time); + catalog, stripe, stripeLock, insertContext, internal::AllowBucketCreation::kNo, time); // If there are no open buckets for our measurement that we can use, we return a // reopeningContext to try reopening a closed bucket from disk. if (!bucket) { @@ -320,8 +319,8 @@ StatusWith tryInsert(BucketCatalog& catalog, // If we were time forward or backward, we might be able to "reopen" a bucket we still have // in memory that's set to be closed when pending operations finish. if ((*reason == RolloverReason::kTimeBackward || *reason == RolloverReason::kTimeForward)) { - if (Bucket* alternate = internal::useAlternateBucket( - catalog, stripe, stripeLock, nss, insertContext, time)) { + if (Bucket* alternate = + internal::useAlternateBucket(catalog, stripe, stripeLock, insertContext, time)) { insertionResult = insertIntoBucket(catalog, stripe, stripeLock, @@ -356,7 +355,6 @@ StatusWith tryInsert(BucketCatalog& catalog, } StatusWith insertWithReopeningContext(BucketCatalog& catalog, - const NamespaceString& nss, const StringDataComparator* comparator, const BSONObj& doc, OperationId opId, @@ -402,7 +400,6 @@ StatusWith insertWithReopeningContext(BucketCatalog& catalog, swBucket = internal::reuseExistingBucket(catalog, stripe, stripeLock, - nss, insertContext.stats, insertContext.key, *existingBucket, @@ -446,7 +443,7 @@ StatusWith insertWithReopeningContext(BucketCatalog& catalog, } Bucket* bucket = useBucket( - catalog, stripe, stripeLock, nss, insertContext, internal::AllowBucketCreation::kYes, time); + catalog, stripe, stripeLock, insertContext, internal::AllowBucketCreation::kYes, time); invariant(bucket); auto insertionResult = insertIntoBucket(catalog, @@ -466,7 +463,6 @@ StatusWith insertWithReopeningContext(BucketCatalog& catalog, } StatusWith insert(BucketCatalog& catalog, - const NamespaceString& nss, const StringDataComparator* comparator, const BSONObj& doc, OperationId opId, @@ -478,7 +474,7 @@ StatusWith insert(BucketCatalog& catalog, stdx::lock_guard stripeLock{stripe.mutex}; Bucket* bucket = useBucket( - catalog, stripe, stripeLock, nss, insertContext, internal::AllowBucketCreation::kYes, time); + catalog, stripe, stripeLock, insertContext, internal::AllowBucketCreation::kYes, time); invariant(bucket); auto insertionResult = insertIntoBucket(catalog, @@ -506,9 +502,7 @@ void waitToInsert(InsertWaiter* waiter) { } } -Status prepareCommit(BucketCatalog& catalog, - const NamespaceString& nss, - std::shared_ptr batch) { +Status prepareCommit(BucketCatalog& catalog, std::shared_ptr batch) { auto getBatchStatus = [&] { return batch->promise.getFuture().getNoThrow().getStatus(); }; @@ -542,7 +536,7 @@ Status prepareCommit(BucketCatalog& catalog, stripe, stripeLock, batch, - internal::getTimeseriesBucketClearedError(nss, batch->bucketId.oid)); + internal::getTimeseriesBucketClearedError(batch->bucketId.oid)); return getBatchStatus(); } @@ -553,7 +547,6 @@ Status prepareCommit(BucketCatalog& catalog, boost::optional finish( BucketCatalog& catalog, - const NamespaceString& nss, std::shared_ptr batch, const CommitInfo& info, const std::function& @@ -627,7 +620,7 @@ boost::optional finish( stripeLock, *bucket, nullptr, - internal::getTimeseriesBucketClearedError(nss, bucket->bucketId.oid)); + internal::getTimeseriesBucketClearedError(bucket->bucketId.oid)); } } else if (allCommitted(*bucket)) { switch (bucket->rolloverAction) { diff --git a/src/mongo/db/timeseries/bucket_catalog/bucket_catalog.h b/src/mongo/db/timeseries/bucket_catalog/bucket_catalog.h index 5913a32406613..b8132181e1601 100644 --- a/src/mongo/db/timeseries/bucket_catalog/bucket_catalog.h +++ b/src/mongo/db/timeseries/bucket_catalog/bucket_catalog.h @@ -263,7 +263,6 @@ void getDetailedMemoryUsage(const BucketCatalog& catalog, BSONObjBuilder& builde * 'insert' to insert 'doc', passing any fetched bucket back as a member of the 'ReopeningContext'. */ StatusWith tryInsert(BucketCatalog& catalog, - const NamespaceString& nss, const StringDataComparator* comparator, const BSONObj& doc, OperationId, @@ -282,7 +281,6 @@ StatusWith tryInsert(BucketCatalog& catalog, * bucket if none exists. */ StatusWith insertWithReopeningContext(BucketCatalog& catalog, - const NamespaceString& nss, const StringDataComparator* comparator, const BSONObj& doc, OperationId, @@ -300,7 +298,6 @@ StatusWith insertWithReopeningContext(BucketCatalog& catalog, * We will attempt to find a suitable open bucket, or open a new bucket if none exists. */ StatusWith insert(BucketCatalog& catalog, - const NamespaceString& nss, const StringDataComparator* comparator, const BSONObj& doc, OperationId, @@ -322,9 +319,7 @@ void waitToInsert(InsertWaiter* waiter); * on the same bucket, or there is an outstanding 'ReopeningRequest' for the same series (metaField * value), this operation will block waiting for it to complete. */ -Status prepareCommit(BucketCatalog& catalog, - const NamespaceString& nss, - std::shared_ptr batch); +Status prepareCommit(BucketCatalog& catalog, std::shared_ptr batch); /** * Records the result of a batch commit. Caller must already have commit rights on batch, and batch @@ -337,7 +332,6 @@ Status prepareCommit(BucketCatalog& catalog, */ boost::optional finish( BucketCatalog& catalog, - const NamespaceString& nss, std::shared_ptr batch, const CommitInfo& info, const std::function& diff --git a/src/mongo/db/timeseries/bucket_catalog/bucket_catalog_helpers.h b/src/mongo/db/timeseries/bucket_catalog/bucket_catalog_helpers.h index 4e83a9e8bb609..cd9bf1e1a34f9 100644 --- a/src/mongo/db/timeseries/bucket_catalog/bucket_catalog_helpers.h +++ b/src/mongo/db/timeseries/bucket_catalog/bucket_catalog_helpers.h @@ -43,7 +43,6 @@ #include "mongo/bson/bsonobjbuilder.h" #include "mongo/bson/oid.h" #include "mongo/db/catalog/collection.h" -#include "mongo/db/namespace_string.h" #include "mongo/db/storage/kv/kv_engine.h" #include "mongo/db/storage/recovery_unit.h" #include "mongo/db/timeseries/bucket_catalog/bucket.h" diff --git a/src/mongo/db/timeseries/bucket_catalog/bucket_catalog_internal.cpp b/src/mongo/db/timeseries/bucket_catalog/bucket_catalog_internal.cpp index 204622c3dfa62..bfb55cbaa5ca1 100644 --- a/src/mongo/db/timeseries/bucket_catalog/bucket_catalog_internal.cpp +++ b/src/mongo/db/timeseries/bucket_catalog/bucket_catalog_internal.cpp @@ -277,7 +277,6 @@ Bucket* useBucketAndChangePreparedState(BucketStateRegistry& registry, Bucket* useBucket(BucketCatalog& catalog, Stripe& stripe, WithLock stripeLock, - const NamespaceString& nss, InsertContext& info, AllowBucketCreation mode, const Date_t& time) { @@ -314,7 +313,7 @@ Bucket* useBucket(BucketCatalog& catalog, stripeLock, *bucket, nullptr, - getTimeseriesBucketClearedError(nss, bucket->bucketId.oid)); + getTimeseriesBucketClearedError(bucket->bucketId.oid)); return mode == AllowBucketCreation::kYes ? &allocateBucket(catalog, stripe, stripeLock, info, time) @@ -324,7 +323,6 @@ Bucket* useBucket(BucketCatalog& catalog, Bucket* useAlternateBucket(BucketCatalog& catalog, Stripe& stripe, WithLock stripeLock, - const NamespaceString& nss, InsertContext& insertContext, const Date_t& time) { auto it = stripe.openBucketsByKey.find(insertContext.key); @@ -365,7 +363,7 @@ Bucket* useAlternateBucket(BucketCatalog& catalog, stripeLock, *potentialBucket, nullptr, - getTimeseriesBucketClearedError(nss, potentialBucket->bucketId.oid)); + getTimeseriesBucketClearedError(potentialBucket->bucketId.oid)); } } @@ -601,7 +599,6 @@ StatusWith> reopenBucket(BucketCatalog& catalog, StatusWith> reuseExistingBucket(BucketCatalog& catalog, Stripe& stripe, WithLock stripeLock, - const NamespaceString& nss, ExecutionStatsController& stats, const BucketKey& key, Bucket& existingBucket, @@ -617,7 +614,7 @@ StatusWith> reuseExistingBucket(BucketCatalog& ca stripeLock, existingBucket, nullptr, - getTimeseriesBucketClearedError(nss, existingBucket.bucketId.oid)); + getTimeseriesBucketClearedError(existingBucket.bucketId.oid)); return {ErrorCodes::WriteConflict, "Bucket may be stale"}; } else if (transientlyConflictsWithReopening(state.value())) { // Avoid reusing the bucket if it conflicts with reopening. @@ -1358,13 +1355,9 @@ void mergeExecutionStatsToBucketCatalog(BucketCatalog& catalog, addCollectionExecutionStats(stats, *collStats); } -Status getTimeseriesBucketClearedError(const NamespaceString& nss, const OID& oid) { +Status getTimeseriesBucketClearedError(const OID& oid) { return {ErrorCodes::TimeseriesBucketCleared, - str::stream() << "Time-series bucket " << oid << " for collection " - << (nss.isTimeseriesBucketsCollection() - ? nss.getTimeseriesViewNamespace().toStringForErrorMsg() - : nss.toStringForErrorMsg()) - << " was cleared"}; + str::stream() << "Time-series bucket " << oid << " was cleared"}; } void closeOpenBucket(BucketCatalog& catalog, diff --git a/src/mongo/db/timeseries/bucket_catalog/bucket_catalog_internal.h b/src/mongo/db/timeseries/bucket_catalog/bucket_catalog_internal.h index 48a65eaa40187..cfd2583515546 100644 --- a/src/mongo/db/timeseries/bucket_catalog/bucket_catalog_internal.h +++ b/src/mongo/db/timeseries/bucket_catalog/bucket_catalog_internal.h @@ -44,7 +44,6 @@ #include "mongo/base/string_data.h" #include "mongo/bson/bsonobj.h" #include "mongo/bson/oid.h" -#include "mongo/db/namespace_string.h" #include "mongo/db/timeseries/bucket_catalog/bucket.h" #include "mongo/db/timeseries/bucket_catalog/bucket_catalog.h" #include "mongo/db/timeseries/bucket_catalog/bucket_identifiers.h" @@ -144,7 +143,6 @@ Bucket* useBucketAndChangePreparedState(BucketStateRegistry& registry, Bucket* useBucket(BucketCatalog& catalog, Stripe& stripe, WithLock stripeLock, - const NamespaceString& nss, InsertContext& info, AllowBucketCreation mode, const Date_t& time); @@ -156,7 +154,6 @@ Bucket* useBucket(BucketCatalog& catalog, Bucket* useAlternateBucket(BucketCatalog& catalog, Stripe& stripe, WithLock stripeLock, - const NamespaceString& nss, InsertContext& info, const Date_t& time); @@ -197,7 +194,6 @@ StatusWith> reopenBucket(BucketCatalog& catalog, StatusWith> reuseExistingBucket(BucketCatalog& catalog, Stripe& stripe, WithLock stripeLock, - const NamespaceString& nss, ExecutionStatsController& stats, const BucketKey& key, Bucket& existingBucket, @@ -392,7 +388,7 @@ void mergeExecutionStatsToBucketCatalog(BucketCatalog& catalog, /** * Generates a status with code TimeseriesBucketCleared and an appropriate error message. */ -Status getTimeseriesBucketClearedError(const NamespaceString& nss, const OID& oid); +Status getTimeseriesBucketClearedError(const OID& oid); /** * Close an open bucket, setting the state appropriately and removing it from the catalog. diff --git a/src/mongo/db/timeseries/bucket_catalog/bucket_catalog_test.cpp b/src/mongo/db/timeseries/bucket_catalog/bucket_catalog_test.cpp index fa48d6172be3c..a9d10a83404e6 100644 --- a/src/mongo/db/timeseries/bucket_catalog/bucket_catalog_test.cpp +++ b/src/mongo/db/timeseries/bucket_catalog/bucket_catalog_test.cpp @@ -242,11 +242,11 @@ void BucketCatalogTest::_commit(const NamespaceString& ns, uint16_t numPreviouslyCommittedMeasurements, size_t expectedBatchSize) { ASSERT(claimWriteBatchCommitRights(*batch)); - ASSERT_OK(prepareCommit(*_bucketCatalog, ns, batch)); + ASSERT_OK(prepareCommit(*_bucketCatalog, batch)); ASSERT_EQ(batch->measurements.size(), expectedBatchSize); ASSERT_EQ(batch->numPreviouslyCommittedMeasurements, numPreviouslyCommittedMeasurements); - finish(*_bucketCatalog, ns, batch, {}); + finish(*_bucketCatalog, batch, {}); } void BucketCatalogTest::_insertOneAndCommit(const NamespaceString& ns, @@ -260,7 +260,6 @@ void BucketCatalogTest::_insertOneAndCommit(const NamespaceString& ns, BSON(_timeField << time))); auto result = insert(*_bucketCatalog, - ns, _getCollator(ns), BSON(_timeField << time), _opCtx->getOpID(), @@ -284,7 +283,6 @@ StatusWith BucketCatalogTest::_ prepareInsert(catalog, uuid, _getCollator(nss), _getTimeseriesOptions(nss), doc)); return insert(catalog, - nss, _getCollator(nss), doc, opCtx->getOpID(), @@ -306,7 +304,6 @@ StatusWith BucketCatalogTest::_ prepareInsert(catalog, uuid, _getCollator(nss), _getTimeseriesOptions(nss), doc)); return tryInsert(catalog, - nss, _getCollator(nss), doc, _opCtx->getOpID(), @@ -330,7 +327,6 @@ BucketCatalogTest::_insertOneWithReopeningContextHelper( prepareInsert(catalog, uuid, _getCollator(nss), _getTimeseriesOptions(nss), doc)); return insertWithReopeningContext(catalog, - nss, _getCollator(nss), doc, _opCtx->getOpID(), @@ -462,7 +458,7 @@ TEST_F(BucketCatalogTest, InsertIntoSameBucket) { // The batch hasn't actually been committed yet. ASSERT(!isWriteBatchFinished(*batch1)); - ASSERT_OK(prepareCommit(*_bucketCatalog, _ns1, batch1)); + ASSERT_OK(prepareCommit(*_bucketCatalog, batch1)); // Still not finished. ASSERT(!isWriteBatchFinished(*batch1)); @@ -473,7 +469,7 @@ TEST_F(BucketCatalogTest, InsertIntoSameBucket) { ASSERT_EQ(batch1->numPreviouslyCommittedMeasurements, 0); // Once the commit has occurred, the waiter should be notified. - finish(*_bucketCatalog, _ns1, batch1, {}); + finish(*_bucketCatalog, batch1, {}); ASSERT(isWriteBatchFinished(*batch2)); auto result3 = getWriteBatchResult(*batch2); ASSERT_OK(result3.getStatus()); @@ -547,16 +543,16 @@ TEST_F(BucketCatalogTest, InsertThroughDifferentCatalogsIntoDifferentBuckets) { // Committing one bucket should only return the one document in that bucket and should not // affect the other bucket. ASSERT(claimWriteBatchCommitRights(*batch1)); - ASSERT_OK(prepareCommit(*_bucketCatalog, _ns1, batch1)); + ASSERT_OK(prepareCommit(*_bucketCatalog, batch1)); ASSERT_EQ(batch1->measurements.size(), 1); ASSERT_EQ(batch1->numPreviouslyCommittedMeasurements, 0); - finish(*_bucketCatalog, _ns1, batch1, {}); + finish(*_bucketCatalog, batch1, {}); ASSERT(claimWriteBatchCommitRights(*batch2)); - ASSERT_OK(prepareCommit(temporaryBucketCatalog, _ns1, batch2)); + ASSERT_OK(prepareCommit(temporaryBucketCatalog, batch2)); ASSERT_EQ(batch2->measurements.size(), 1); ASSERT_EQ(batch2->numPreviouslyCommittedMeasurements, 0); - finish(temporaryBucketCatalog, _ns1, batch2, {}); + finish(temporaryBucketCatalog, batch2, {}); } TEST_F(BucketCatalogTest, InsertIntoSameBucketArray) { @@ -678,7 +674,7 @@ TEST_F(BucketCatalogTest, InsertBetweenPrepareAndFinish) { _insertOneHelper(_opCtx, *_bucketCatalog, _ns1, _uuid1, BSON(_timeField << Date_t::now())); auto batch1 = get(result1.getValue()).batch; ASSERT(claimWriteBatchCommitRights(*batch1)); - ASSERT_OK(prepareCommit(*_bucketCatalog, _ns1, batch1)); + ASSERT_OK(prepareCommit(*_bucketCatalog, batch1)); ASSERT_EQ(batch1->measurements.size(), 1); ASSERT_EQ(batch1->numPreviouslyCommittedMeasurements, 0); @@ -688,7 +684,7 @@ TEST_F(BucketCatalogTest, InsertBetweenPrepareAndFinish) { auto batch2 = get(result2.getValue()).batch; ASSERT_NE(batch1, batch2); - finish(*_bucketCatalog, _ns1, batch1, {}); + finish(*_bucketCatalog, batch1, {}); ASSERT(isWriteBatchFinished(*batch1)); // Verify the second batch still commits one doc, and that the first batch only commited one. @@ -699,7 +695,7 @@ DEATH_TEST_F(BucketCatalogTest, CannotCommitWithoutRights, "invariant") { auto result = _insertOneHelper(_opCtx, *_bucketCatalog, _ns1, _uuid1, BSON(_timeField << Date_t::now())); auto& batch = get(result.getValue()).batch; - ASSERT_OK(prepareCommit(*_bucketCatalog, _ns1, batch)); + ASSERT_OK(prepareCommit(*_bucketCatalog, batch)); // BucketCatalog::prepareCommit uses dassert, so it will only invariant in debug mode. Ensure we // die here in non-debug mode as well. @@ -779,7 +775,7 @@ TEST_F(BucketCatalogTest, AbortBatchOnBucketWithPreparedCommit) { _insertOneHelper(_opCtx, *_bucketCatalog, _ns1, _uuid1, BSON(_timeField << Date_t::now())); auto batch1 = get(result1.getValue()).batch; ASSERT(claimWriteBatchCommitRights(*batch1)); - ASSERT_OK(prepareCommit(*_bucketCatalog, _ns1, batch1)); + ASSERT_OK(prepareCommit(*_bucketCatalog, batch1)); ASSERT_EQ(batch1->measurements.size(), 1); ASSERT_EQ(batch1->numPreviouslyCommittedMeasurements, 0); @@ -794,7 +790,7 @@ TEST_F(BucketCatalogTest, AbortBatchOnBucketWithPreparedCommit) { ASSERT(isWriteBatchFinished(*batch2)); ASSERT_EQ(getWriteBatchResult(*batch2).getStatus(), ErrorCodes::TimeseriesBucketCleared); - finish(*_bucketCatalog, _ns1, batch1, {}); + finish(*_bucketCatalog, batch1, {}); ASSERT(isWriteBatchFinished(*batch1)); ASSERT_OK(getWriteBatchResult(*batch1).getStatus()); } @@ -807,7 +803,7 @@ TEST_F(BucketCatalogTest, ClearNamespaceWithConcurrentWrites) { clear(*_bucketCatalog, _uuid1); - ASSERT_NOT_OK(prepareCommit(*_bucketCatalog, _ns1, batch)); + ASSERT_NOT_OK(prepareCommit(*_bucketCatalog, batch)); ASSERT(isWriteBatchFinished(*batch)); ASSERT_EQ(getWriteBatchResult(*batch).getStatus(), ErrorCodes::TimeseriesBucketCleared); @@ -815,7 +811,7 @@ TEST_F(BucketCatalogTest, ClearNamespaceWithConcurrentWrites) { _insertOneHelper(_opCtx, *_bucketCatalog, _ns1, _uuid1, BSON(_timeField << Date_t::now())); batch = get(result.getValue()).batch; ASSERT(claimWriteBatchCommitRights(*batch)); - ASSERT_OK(prepareCommit(*_bucketCatalog, _ns1, batch)); + ASSERT_OK(prepareCommit(*_bucketCatalog, batch)); ASSERT_EQ(batch->measurements.size(), 1); ASSERT_EQ(batch->numPreviouslyCommittedMeasurements, 0); @@ -826,7 +822,7 @@ TEST_F(BucketCatalogTest, ClearNamespaceWithConcurrentWrites) { // operation got the collection lock. So the write did actually happen, but is has since been // removed, and that's fine for our purposes. The finish just records the result to the batch // and updates some statistics. - finish(*_bucketCatalog, _ns1, batch, {}); + finish(*_bucketCatalog, batch, {}); ASSERT(isWriteBatchFinished(*batch)); ASSERT_OK(getWriteBatchResult(*batch).getStatus()); } @@ -837,7 +833,7 @@ TEST_F(BucketCatalogTest, ClearBucketWithPreparedBatchThrowsConflict) { _insertOneHelper(_opCtx, *_bucketCatalog, _ns1, _uuid1, BSON(_timeField << Date_t::now())); auto batch = get(result.getValue()).batch; ASSERT(claimWriteBatchCommitRights(*batch)); - ASSERT_OK(prepareCommit(*_bucketCatalog, _ns1, batch)); + ASSERT_OK(prepareCommit(*_bucketCatalog, batch)); ASSERT_EQ(batch->measurements.size(), 1); ASSERT_EQ(batch->numPreviouslyCommittedMeasurements, 0); @@ -854,7 +850,7 @@ TEST_F(BucketCatalogTest, PrepareCommitOnClearedBatchWithAlreadyPreparedBatch) { _insertOneHelper(_opCtx, *_bucketCatalog, _ns1, _uuid1, BSON(_timeField << Date_t::now())); auto batch1 = get(result1.getValue()).batch; ASSERT(claimWriteBatchCommitRights(*batch1)); - ASSERT_OK(prepareCommit(*_bucketCatalog, _ns1, batch1)); + ASSERT_OK(prepareCommit(*_bucketCatalog, batch1)); ASSERT_EQ(batch1->measurements.size(), 1); ASSERT_EQ(batch1->numPreviouslyCommittedMeasurements, 0); @@ -870,7 +866,7 @@ TEST_F(BucketCatalogTest, PrepareCommitOnClearedBatchWithAlreadyPreparedBatch) { // Now try to prepare the second batch. Ensure it aborts the batch. ASSERT(claimWriteBatchCommitRights(*batch2)); - ASSERT_NOT_OK(prepareCommit(*_bucketCatalog, _ns1, batch2)); + ASSERT_NOT_OK(prepareCommit(*_bucketCatalog, batch2)); ASSERT(isWriteBatchFinished(*batch2)); ASSERT_EQ(getWriteBatchResult(*batch2).getStatus(), ErrorCodes::TimeseriesBucketCleared); @@ -890,7 +886,7 @@ TEST_F(BucketCatalogTest, PrepareCommitOnClearedBatchWithAlreadyPreparedBatch) { abort(*_bucketCatalog, batch3, {ErrorCodes::TimeseriesBucketCleared, ""}); // Make sure we can finish the cleanly prepared batch. - finish(*_bucketCatalog, _ns1, batch1, {}); + finish(*_bucketCatalog, batch1, {}); ASSERT(isWriteBatchFinished(*batch1)); ASSERT_OK(getWriteBatchResult(*batch1).getStatus()); } @@ -905,7 +901,7 @@ TEST_F(BucketCatalogTest, PrepareCommitOnAlreadyAbortedBatch) { ASSERT(isWriteBatchFinished(*batch)); ASSERT_EQ(getWriteBatchResult(*batch).getStatus(), ErrorCodes::TimeseriesBucketCleared); - ASSERT_NOT_OK(prepareCommit(*_bucketCatalog, _ns1, batch)); + ASSERT_NOT_OK(prepareCommit(*_bucketCatalog, batch)); ASSERT(isWriteBatchFinished(*batch)); ASSERT_EQ(getWriteBatchResult(*batch).getStatus(), ErrorCodes::TimeseriesBucketCleared); } @@ -975,20 +971,20 @@ TEST_F(BucketCatalogTest, CannotConcurrentlyCommitBatchesForSameBucket) { ASSERT(claimWriteBatchCommitRights(*batch2)); // Batch 2 will not be able to commit until batch 1 has finished. - ASSERT_OK(prepareCommit(*_bucketCatalog, _ns1, batch1)); + ASSERT_OK(prepareCommit(*_bucketCatalog, batch1)); { auto task = RunBackgroundTaskAndWaitForFailpoint{ "hangTimeSeriesBatchPrepareWaitingForConflictingOperation", [&]() { - ASSERT_OK(prepareCommit(*_bucketCatalog, _ns1, batch2)); + ASSERT_OK(prepareCommit(*_bucketCatalog, batch2)); }}; // Finish the first batch. - finish(*_bucketCatalog, _ns1, batch1, {}); + finish(*_bucketCatalog, batch1, {}); ASSERT(isWriteBatchFinished(*batch1)); } - finish(*_bucketCatalog, _ns1, batch2, {}); + finish(*_bucketCatalog, batch2, {}); ASSERT(isWriteBatchFinished(*batch2)); } @@ -1025,12 +1021,12 @@ TEST_F(BucketCatalogTest, AbortingBatchEnsuresBucketIsEventuallyClosed) { ASSERT(claimWriteBatchCommitRights(*batch3)); // Batch 2 will not be able to commit until batch 1 has finished. - ASSERT_OK(prepareCommit(*_bucketCatalog, _ns1, batch1)); + ASSERT_OK(prepareCommit(*_bucketCatalog, batch1)); { auto task = RunBackgroundTaskAndWaitForFailpoint{ "hangTimeSeriesBatchPrepareWaitingForConflictingOperation", [&]() { - ASSERT_NOT_OK(prepareCommit(*_bucketCatalog, _ns1, batch2)); + ASSERT_NOT_OK(prepareCommit(*_bucketCatalog, batch2)); }}; // If we abort the third batch, it should abort the second one too, as it isn't prepared. @@ -1038,13 +1034,13 @@ TEST_F(BucketCatalogTest, AbortingBatchEnsuresBucketIsEventuallyClosed) { // can then finish the first batch, which will allow the second batch to proceed. It should // recognize it has been aborted and clean up the bucket. abort(*_bucketCatalog, batch3, Status{ErrorCodes::TimeseriesBucketCleared, "cleared"}); - finish(*_bucketCatalog, _ns1, batch1, {}); + finish(*_bucketCatalog, batch1, {}); ASSERT(isWriteBatchFinished(*batch1)); } // Wait for the batch 2 task to finish preparing commit. Since batch 1 finished, batch 2 should // be unblocked. Note that after aborting batch 3, batch 2 was not in a prepared state, so we // expect the prepareCommit() call to fail. - ASSERT_NOT_OK(prepareCommit(*_bucketCatalog, _ns1, batch2)); + ASSERT_NOT_OK(prepareCommit(*_bucketCatalog, batch2)); ASSERT(isWriteBatchFinished(*batch2)); // Make sure a new batch ends up in a new bucket. @@ -1079,12 +1075,12 @@ TEST_F(BucketCatalogTest, AbortingBatchEnsuresNewInsertsGoToNewBucket) { ASSERT_EQ(batch1->bucketId, batch2->bucketId); ASSERT(claimWriteBatchCommitRights(*batch1)); ASSERT(claimWriteBatchCommitRights(*batch2)); - ASSERT_OK(prepareCommit(*_bucketCatalog, _ns1, batch1)); + ASSERT_OK(prepareCommit(*_bucketCatalog, batch1)); // Batch 1 will be in a prepared state now. Abort the second batch so that bucket 1 will be // closed after batch 1 finishes. abort(*_bucketCatalog, batch2, Status{ErrorCodes::TimeseriesBucketCleared, "cleared"}); - finish(*_bucketCatalog, _ns1, batch1, {}); + finish(*_bucketCatalog, batch1, {}); ASSERT(isWriteBatchFinished(*batch1)); ASSERT(isWriteBatchFinished(*batch2)); @@ -1119,17 +1115,17 @@ TEST_F(BucketCatalogTest, DuplicateNewFieldNamesAcrossConcurrentBatches) { // Batch 2 is the first batch to commit the time field. ASSERT(claimWriteBatchCommitRights(*batch2)); - ASSERT_OK(prepareCommit(*_bucketCatalog, _ns1, batch2)); + ASSERT_OK(prepareCommit(*_bucketCatalog, batch2)); ASSERT_EQ(batch2->newFieldNamesToBeInserted.size(), 1); ASSERT_EQ(batch2->newFieldNamesToBeInserted.begin()->first, _timeField); - finish(*_bucketCatalog, _ns1, batch2, {}); + finish(*_bucketCatalog, batch2, {}); // Batch 1 was the first batch to insert the time field, but by commit time it was already // committed by batch 2. ASSERT(claimWriteBatchCommitRights(*batch1)); - ASSERT_OK(prepareCommit(*_bucketCatalog, _ns1, batch1)); + ASSERT_OK(prepareCommit(*_bucketCatalog, batch1)); ASSERT(batch1->newFieldNamesToBeInserted.empty()); - finish(*_bucketCatalog, _ns1, batch1, {}); + finish(*_bucketCatalog, batch1, {}); } TEST_F(BucketCatalogTest, SchemaChanges) { @@ -1364,7 +1360,7 @@ TEST_F(BucketCatalogTest, ReopenUncompressedBucketAndInsertCompatibleMeasurement auto batch = get(result.getValue()).batch; ASSERT(claimWriteBatchCommitRights(*batch)); - ASSERT_OK(prepareCommit(*_bucketCatalog, _ns1, batch)); + ASSERT_OK(prepareCommit(*_bucketCatalog, batch)); ASSERT_EQ(batch->measurements.size(), 1); // The reopened bucket already contains three committed measurements. @@ -1376,7 +1372,7 @@ TEST_F(BucketCatalogTest, ReopenUncompressedBucketAndInsertCompatibleMeasurement batch->max, BSON("u" << BSON("time" << Date_t::fromMillisSinceEpoch(1654529680000) << "b" << 100))); - finish(*_bucketCatalog, _ns1, batch, {}); + finish(*_bucketCatalog, batch, {}); } TEST_F(BucketCatalogTest, ReopenUncompressedBucketAndInsertCompatibleMeasurementWithMeta) { @@ -1411,7 +1407,7 @@ TEST_F(BucketCatalogTest, ReopenUncompressedBucketAndInsertCompatibleMeasurement auto batch = get(result.getValue()).batch; ASSERT(claimWriteBatchCommitRights(*batch)); - ASSERT_OK(prepareCommit(*_bucketCatalog, _ns1, batch)); + ASSERT_OK(prepareCommit(*_bucketCatalog, batch)); ASSERT_EQ(batch->measurements.size(), 1); // The reopened bucket already contains three committed measurements. @@ -1423,7 +1419,7 @@ TEST_F(BucketCatalogTest, ReopenUncompressedBucketAndInsertCompatibleMeasurement batch->max, BSON("u" << BSON("time" << Date_t::fromMillisSinceEpoch(1654529680000) << "b" << 100))); - finish(*_bucketCatalog, _ns1, batch, {}); + finish(*_bucketCatalog, batch, {}); } TEST_F(BucketCatalogTest, ReopenUncompressedBucketAndInsertIncompatibleMeasurement) { @@ -1464,13 +1460,13 @@ TEST_F(BucketCatalogTest, ReopenUncompressedBucketAndInsertIncompatibleMeasureme auto batch = get(result.getValue()).batch; ASSERT(claimWriteBatchCommitRights(*batch)); - ASSERT_OK(prepareCommit(*_bucketCatalog, _ns1, batch)); + ASSERT_OK(prepareCommit(*_bucketCatalog, batch)); ASSERT_EQ(batch->measurements.size(), 1); // Since the reopened bucket was incompatible, we opened a new one. ASSERT_EQ(batch->numPreviouslyCommittedMeasurements, 0); - finish(*_bucketCatalog, _ns1, batch, {}); + finish(*_bucketCatalog, batch, {}); } TEST_F(BucketCatalogTest, ReopenCompressedBucketAndInsertCompatibleMeasurement) { @@ -1517,7 +1513,7 @@ TEST_F(BucketCatalogTest, ReopenCompressedBucketAndInsertCompatibleMeasurement) auto batch = get(result.getValue()).batch; ASSERT(claimWriteBatchCommitRights(*batch)); - ASSERT_OK(prepareCommit(*_bucketCatalog, _ns1, batch)); + ASSERT_OK(prepareCommit(*_bucketCatalog, batch)); ASSERT_EQ(batch->measurements.size(), 1); // The reopened bucket already contains three committed measurements. @@ -1529,7 +1525,7 @@ TEST_F(BucketCatalogTest, ReopenCompressedBucketAndInsertCompatibleMeasurement) batch->max, BSON("u" << BSON("time" << Date_t::fromMillisSinceEpoch(1654529680000) << "b" << 100))); - finish(*_bucketCatalog, _ns1, batch, {}); + finish(*_bucketCatalog, batch, {}); } TEST_F(BucketCatalogTest, ReopenCompressedBucketAndInsertIncompatibleMeasurement) { @@ -1578,13 +1574,13 @@ TEST_F(BucketCatalogTest, ReopenCompressedBucketAndInsertIncompatibleMeasurement auto batch = get(result.getValue()).batch; ASSERT(claimWriteBatchCommitRights(*batch)); - ASSERT_OK(prepareCommit(*_bucketCatalog, _ns1, batch)); + ASSERT_OK(prepareCommit(*_bucketCatalog, batch)); ASSERT_EQ(batch->measurements.size(), 1); // Since the reopened bucket was incompatible, we opened a new one. ASSERT_EQ(batch->numPreviouslyCommittedMeasurements, 0); - finish(*_bucketCatalog, _ns1, batch, {}); + finish(*_bucketCatalog, batch, {}); } TEST_F(BucketCatalogTest, ReopenCompressedBucketFails) { @@ -1635,8 +1631,8 @@ TEST_F(BucketCatalogTest, ArchivingUnderMemoryPressure) { ASSERT_OK(result.getStatus()); auto batch = get(result.getValue()).batch; ASSERT(claimWriteBatchCommitRights(*batch)); - ASSERT_OK(prepareCommit(*_bucketCatalog, _ns1, batch)); - finish(*_bucketCatalog, _ns1, batch, {}); + ASSERT_OK(prepareCommit(*_bucketCatalog, batch)); + finish(*_bucketCatalog, batch, {}); return std::move(get(result.getValue()).closedBuckets); }; @@ -1715,9 +1711,9 @@ TEST_F(BucketCatalogTest, TryInsertWillNotCreateBucketWhenWeShouldTryToReopen) { ASSERT(batch); bucketId = batch->bucketId; ASSERT(claimWriteBatchCommitRights(*batch)); - ASSERT_OK(prepareCommit(*_bucketCatalog, _ns1, batch)); + ASSERT_OK(prepareCommit(*_bucketCatalog, batch)); ASSERT_EQ(batch->measurements.size(), 1); - finish(*_bucketCatalog, _ns1, batch, {}); + finish(*_bucketCatalog, batch, {}); } // Time backwards should hint to re-open. @@ -1769,9 +1765,9 @@ TEST_F(BucketCatalogTest, TryInsertWillNotCreateBucketWhenWeShouldTryToReopen) { ASSERT_NE(batch->bucketId, bucketId); ASSERT(batch); ASSERT(claimWriteBatchCommitRights(*batch)); - ASSERT_OK(prepareCommit(*_bucketCatalog, _ns1, batch)); + ASSERT_OK(prepareCommit(*_bucketCatalog, batch)); ASSERT_EQ(batch->measurements.size(), 1); - finish(*_bucketCatalog, _ns1, batch, {}); + finish(*_bucketCatalog, batch, {}); } // If we try to insert something that could fit in the archived bucket, we should get it back as @@ -1805,9 +1801,9 @@ TEST_F(BucketCatalogTest, TryInsertWillCreateBucketIfWeWouldCloseExistingBucket) ASSERT(batch); auto bucketId = batch->bucketId; ASSERT(claimWriteBatchCommitRights(*batch)); - ASSERT_OK(prepareCommit(*_bucketCatalog, _ns1, batch)); + ASSERT_OK(prepareCommit(*_bucketCatalog, batch)); ASSERT_EQ(batch->measurements.size(), 1); - finish(*_bucketCatalog, _ns1, batch, {}); + finish(*_bucketCatalog, batch, {}); // Incompatible schema would close the existing bucket, so we should expect to open a new bucket // and proceed to insert the document. @@ -1822,9 +1818,9 @@ TEST_F(BucketCatalogTest, TryInsertWillCreateBucketIfWeWouldCloseExistingBucket) ASSERT(batch); ASSERT_NE(batch->bucketId, bucketId); ASSERT(claimWriteBatchCommitRights(*batch)); - ASSERT_OK(prepareCommit(*_bucketCatalog, _ns1, batch)); + ASSERT_OK(prepareCommit(*_bucketCatalog, batch)); ASSERT_EQ(batch->measurements.size(), 1); - finish(*_bucketCatalog, _ns1, batch, {}); + finish(*_bucketCatalog, batch, {}); } TEST_F(BucketCatalogTest, InsertIntoReopenedUncompressedBucket) { @@ -1846,9 +1842,9 @@ TEST_F(BucketCatalogTest, InsertIntoReopenedUncompressedBucket) { ASSERT(batch); auto oldBucketId = batch->bucketId; ASSERT(claimWriteBatchCommitRights(*batch)); - ASSERT_OK(prepareCommit(*_bucketCatalog, _ns1, batch)); + ASSERT_OK(prepareCommit(*_bucketCatalog, batch)); ASSERT_EQ(batch->measurements.size(), 1); - finish(*_bucketCatalog, _ns1, batch, {}); + finish(*_bucketCatalog, batch, {}); BSONObj bucketDoc = ::mongo::fromjson( R"({"_id":{"$oid":"629e1e680958e279dc29a517"}, @@ -1882,9 +1878,9 @@ TEST_F(BucketCatalogTest, InsertIntoReopenedUncompressedBucket) { ASSERT(batch); ASSERT_EQ(batch->bucketId.oid, bucketDoc["_id"].OID()); ASSERT(claimWriteBatchCommitRights(*batch)); - ASSERT_OK(prepareCommit(*_bucketCatalog, _ns1, batch)); + ASSERT_OK(prepareCommit(*_bucketCatalog, batch)); ASSERT_EQ(batch->measurements.size(), 1); - finish(*_bucketCatalog, _ns1, batch, {}); + finish(*_bucketCatalog, batch, {}); // Verify the old bucket was soft-closed ASSERT_EQ(1, _getExecutionStat(_uuid1, kNumClosedDueToReopening)); ASSERT_EQ(1, _getExecutionStat(_uuid1, kNumBucketsReopened)); @@ -1923,9 +1919,9 @@ TEST_F(BucketCatalogTest, CannotInsertIntoOutdatedBucket) { ASSERT(batch); auto oldBucketId = batch->bucketId; ASSERT(claimWriteBatchCommitRights(*batch)); - ASSERT_OK(prepareCommit(*_bucketCatalog, _ns1, batch)); + ASSERT_OK(prepareCommit(*_bucketCatalog, batch)); ASSERT_EQ(batch->measurements.size(), 1); - finish(*_bucketCatalog, _ns1, batch, {}); + finish(*_bucketCatalog, batch, {}); BSONObj bucketDoc = ::mongo::fromjson( R"({"_id":{"$oid":"629e1e680958e279dc29a517"}, @@ -2006,7 +2002,7 @@ TEST_F(BucketCatalogTest, ReopeningConflictsWithPreparedBatch) { auto batch1 = get(result1.getValue()).batch; ASSERT(batch1); ASSERT(claimWriteBatchCommitRights(*batch1)); - ASSERT_OK(prepareCommit(*_bucketCatalog, _ns1, batch1)); + ASSERT_OK(prepareCommit(*_bucketCatalog, batch1)); ASSERT_EQ(batch1->measurements.size(), 1); // Stage and abort another insert on the same bucket, so that new inserts can't land without @@ -2065,7 +2061,7 @@ TEST_F(BucketCatalogTest, PreparingBatchConflictsWithQueryBasedReopening) { // Ensure it blocks until we resolve the reopening request. auto task = RunBackgroundTaskAndWaitForFailpoint{ "hangTimeSeriesBatchPrepareWaitingForConflictingOperation", [&]() { - ASSERT_OK(prepareCommit(*_bucketCatalog, _ns1, batch)); + ASSERT_OK(prepareCommit(*_bucketCatalog, batch)); }}; result1 = boost::none; } diff --git a/src/mongo/db/timeseries/bucket_catalog/bucket_state_registry_test.cpp b/src/mongo/db/timeseries/bucket_catalog/bucket_state_registry_test.cpp index a0360ceb747e6..35d94531657e3 100644 --- a/src/mongo/db/timeseries/bucket_catalog/bucket_state_registry_test.cpp +++ b/src/mongo/db/timeseries/bucket_catalog/bucket_state_registry_test.cpp @@ -768,7 +768,7 @@ TEST_F(BucketStateRegistryTest, ClosingBucketGoesThroughPendingCompressionState) stats, StringData{bucket.timeField.data(), bucket.timeField.size()}); ASSERT(claimWriteBatchCommitRights(*batch)); - ASSERT_OK(prepareCommit(*this, ns, batch)); + ASSERT_OK(prepareCommit(*this, batch)); ASSERT_TRUE(doesBucketStateMatch(bucketId, BucketState::kPrepared)); { @@ -776,7 +776,7 @@ TEST_F(BucketStateRegistryTest, ClosingBucketGoesThroughPendingCompressionState) // this and closes the bucket. bucket.rolloverAction = RolloverAction::kHardClose; CommitInfo commitInfo{}; - auto closedBucket = finish(*this, ns, batch, commitInfo); + auto closedBucket = finish(*this, batch, commitInfo); ASSERT(closedBucket.has_value()); ASSERT_EQ(closedBucket.value().bucketId.oid, bucketId.oid); diff --git a/src/mongo/db/timeseries/timeseries_write_util.cpp b/src/mongo/db/timeseries/timeseries_write_util.cpp index 8774e27a39c56..8076d3b60e897 100644 --- a/src/mongo/db/timeseries/timeseries_write_util.cpp +++ b/src/mongo/db/timeseries/timeseries_write_util.cpp @@ -677,7 +677,6 @@ StatusWith attemptInsertIntoBucketWithReopening( // compress-and-write-uncompressed-bucket scenario. { auto swResult = bucket_catalog::tryInsert(bucketCatalog, - bucketsColl->ns().getTimeseriesViewNamespace(), bucketsColl->getDefaultCollator(), measurementDoc, opCtx->getOpID(), @@ -722,7 +721,6 @@ StatusWith attemptInsertIntoBucketWithReopening( return bucket_catalog::insertWithReopeningContext( bucketCatalog, - bucketsColl->ns().getTimeseriesViewNamespace(), bucketsColl->getDefaultCollator(), measurementDoc, opCtx->getOpID(), @@ -1169,7 +1167,6 @@ StatusWith attemptInsertIntoBucket( // insert directly on a new bucket. return bucket_catalog::insert( bucketCatalog, - bucketsColl->ns().getTimeseriesViewNamespace(), bucketsColl->getDefaultCollator(), measurementDoc, opCtx->getOpID(), @@ -1185,7 +1182,6 @@ StatusWith attemptInsertIntoBucket( case BucketReopeningPermittance::kDisallowed: return bucket_catalog::insert( bucketCatalog, - bucketsColl->ns().getTimeseriesViewNamespace(), bucketsColl->getDefaultCollator(), measurementDoc, opCtx->getOpID(), @@ -1354,8 +1350,7 @@ void commitTimeseriesBucketsAtomically( bucket_catalog::GlobalBucketCatalog::get(opCtx->getServiceContext()); for (auto batch : batchesToCommit) { auto metadata = getMetadata(sideBucketCatalog, batch.get()->bucketId); - auto prepareCommitStatus = - prepareCommit(sideBucketCatalog, coll->ns().getTimeseriesViewNamespace(), batch); + auto prepareCommitStatus = prepareCommit(sideBucketCatalog, batch); if (!prepareCommitStatus.isOK()) { abortStatus = prepareCommitStatus; return; @@ -1382,7 +1377,6 @@ void commitTimeseriesBucketsAtomically( for (auto batch : batchesToCommit) { finish(sideBucketCatalog, - coll->ns(), batch, bucket_catalog::CommitInfo{opTime, electionId}, getPostCommitDebugChecks(opCtx, coll->ns())); @@ -1508,7 +1502,7 @@ getPostCommitDebugChecks(OperationContext* opCtx, const NamespaceString& ns) { return nullptr; } - return [opCtx, ns](const timeseries::bucket_catalog::WriteBatch& batch, StringData timeField) { + return [opCtx, &ns](const timeseries::bucket_catalog::WriteBatch& batch, StringData timeField) { // Check in-memory and disk state, caller still has commit rights. DBDirectClient client{opCtx}; BSONObj queriedBucket = client.findOne(ns, BSON("_id" << batch.bucketId.oid));