diff options
Diffstat (limited to 'src/mongo')
-rw-r--r-- | src/mongo/db/catalog/index_build_block.cpp | 71 | ||||
-rw-r--r-- | src/mongo/db/catalog/index_builds_manager.cpp | 9 | ||||
-rw-r--r-- | src/mongo/db/catalog/multi_index_block.cpp | 19 | ||||
-rw-r--r-- | src/mongo/db/commands/create_indexes.cpp | 25 | ||||
-rw-r--r-- | src/mongo/db/index/index_access_method.cpp | 6 | ||||
-rw-r--r-- | src/mongo/db/index_builds_coordinator.cpp | 23 | ||||
-rw-r--r-- | src/mongo/db/index_builds_coordinator.h | 2 | ||||
-rw-r--r-- | src/mongo/db/index_builds_coordinator_mongod.cpp | 9 |
8 files changed, 92 insertions, 72 deletions
diff --git a/src/mongo/db/catalog/index_build_block.cpp b/src/mongo/db/catalog/index_build_block.cpp index 2c470a9d794..1efc8d28c7b 100644 --- a/src/mongo/db/catalog/index_build_block.cpp +++ b/src/mongo/db/catalog/index_build_block.cpp @@ -165,42 +165,47 @@ void IndexBuildBlock::success(OperationContext* opCtx, Collection* collection) { invariant(_indexBuildInterceptor->areAllWritesApplied(opCtx)); } - LOGV2(20345, - "index build: done building index {indexName} on ns {nss}", - "index build: done building", - "namespace"_attr = _nss, - "index"_attr = _indexName); - collection->indexBuildSuccess(opCtx, _indexCatalogEntry); auto svcCtx = opCtx->getClient()->getServiceContext(); - opCtx->recoveryUnit()->onCommit([svcCtx, - indexName = _indexName, - spec = _spec, - entry = _indexCatalogEntry, - coll = collection](boost::optional<Timestamp> commitTime) { - // Note: this runs after the WUOW commits but before we release our X lock on the - // collection. This means that any snapshot created after this must include the full - // index, and no one can try to read this index before we set the visibility. - if (!commitTime) { - // The end of background index builds on secondaries does not get a commit - // timestamp. We use the cluster time since it's guaranteed to be greater than the - // time of the index build. It is possible the cluster time could be in the future, - // and we will need to do another write to reach the minimum visible snapshot. - commitTime = LogicalClock::getClusterTimeForReplicaSet(svcCtx).asTimestamp(); - } - entry->setMinimumVisibleSnapshot(commitTime.get()); - // We must also set the minimum visible snapshot on the collection like during init(). - // This prevents reads in the past from reading inconsistent metadata. We should be - // able to remove this when the catalog is versioned. - coll->setMinimumVisibleSnapshot(commitTime.get()); - - // Add the index to the TTLCollectionCache upon successfully committing the index build. - if (spec.hasField(IndexDescriptor::kExpireAfterSecondsFieldName)) { - TTLCollectionCache::get(svcCtx).registerTTLInfo( - std::make_pair(coll->uuid(), indexName)); - } - }); + opCtx->recoveryUnit()->onCommit( + [svcCtx, + indexName = _indexName, + spec = _spec, + entry = _indexCatalogEntry, + coll = collection, + buildUUID = _buildUUID](boost::optional<Timestamp> commitTime) { + // Note: this runs after the WUOW commits but before we release our X lock on the + // collection. This means that any snapshot created after this must include the full + // index, and no one can try to read this index before we set the visibility. + if (!commitTime) { + // The end of background index builds on secondaries does not get a commit + // timestamp. We use the cluster time since it's guaranteed to be greater than the + // time of the index build. It is possible the cluster time could be in the future, + // and we will need to do another write to reach the minimum visible snapshot. + commitTime = LogicalClock::getClusterTimeForReplicaSet(svcCtx).asTimestamp(); + } + + LOGV2(20345, + "Index build: done building index {indexName} on ns {nss}", + "Index build: done building", + "buildUUID"_attr = buildUUID, + "namespace"_attr = coll->ns(), + "index"_attr = indexName, + "commitTimestamp"_attr = commitTime); + + entry->setMinimumVisibleSnapshot(commitTime.get()); + // We must also set the minimum visible snapshot on the collection like during init(). + // This prevents reads in the past from reading inconsistent metadata. We should be + // able to remove this when the catalog is versioned. + coll->setMinimumVisibleSnapshot(commitTime.get()); + + // Add the index to the TTLCollectionCache upon successfully committing the index build. + if (spec.hasField(IndexDescriptor::kExpireAfterSecondsFieldName)) { + TTLCollectionCache::get(svcCtx).registerTTLInfo( + std::make_pair(coll->uuid(), indexName)); + } + }); } } // namespace mongo diff --git a/src/mongo/db/catalog/index_builds_manager.cpp b/src/mongo/db/catalog/index_builds_manager.cpp index 9edcbbb2834..a0c2cb8a399 100644 --- a/src/mongo/db/catalog/index_builds_manager.cpp +++ b/src/mongo/db/catalog/index_builds_manager.cpp @@ -114,15 +114,6 @@ Status IndexBuildsManager::setUpIndexBuild(OperationContext* opCtx, return ex.toStatus(); } - LOGV2( - 20346, - "Index build initialized: {buildUUID}: {nss} ({collection_uuid} ): indexes: {indexes_size}", - "Index build initialized", - "indexBuildUUID"_attr = buildUUID, - "namespace"_attr = nss, - "collectionUuid"_attr = collection->uuid(), - "numIndexes"_attr = indexes.size()); - return Status::OK(); } diff --git a/src/mongo/db/catalog/multi_index_block.cpp b/src/mongo/db/catalog/multi_index_block.cpp index a288db41248..a298ffb9428 100644 --- a/src/mongo/db/catalog/multi_index_block.cpp +++ b/src/mongo/db/catalog/multi_index_block.cpp @@ -293,6 +293,7 @@ StatusWith<std::vector<BSONObj>> MultiIndexBlock::init(OperationContext* opCtx, logv2::DynamicAttributes attrs; attrs.add("namespace", ns); + attrs.add("buildUUID", _buildUUID); attrs.add("properties", *descriptor); attrs.add("method", _method); if (index.bulk) @@ -300,9 +301,9 @@ StatusWith<std::vector<BSONObj>> MultiIndexBlock::init(OperationContext* opCtx, eachIndexBuildMaxMemoryUsageBytes / 1024 / 1024); LOGV2(20384, - "index build: starting on {namespace} properties: {properties} using method: " + "Index build: starting on {namespace} properties: {properties} using method: " "{method}", - "index build: starting", + "Index build: starting", attrs); @@ -323,6 +324,17 @@ StatusWith<std::vector<BSONObj>> MultiIndexBlock::init(OperationContext* opCtx, return status; } + opCtx->recoveryUnit()->onCommit([ns, this](auto commitTs) { + LOGV2(20346, + "Index build initialized: {buildUUID}: {nss} ({collection_uuid}): indexes: " + "{indexes_size}", + "Index build: initialized", + "buildUUID"_attr = _buildUUID, + "namespace"_attr = ns, + "collectionUUID"_attr = _collectionUUID, + "initializationTimestamp"_attr = commitTs); + }); + wunit.commit(); return indexInfoObjs; // Avoid converting WCE to Status @@ -539,8 +551,9 @@ Status MultiIndexBlock::insertAllDocumentsInCollection(OperationContext* opCtx, progress->finished(); LOGV2(20391, - "index build: collection scan done. scanned {n} total records in {t_seconds} seconds", + "Index build: collection scan done. scanned {n} total records in {t_seconds} seconds", "Index build: collection scan done", + "buildUUID"_attr = _buildUUID, "totalRecords"_attr = n, "duration"_attr = duration_cast<Milliseconds>(Seconds(t.seconds()))); diff --git a/src/mongo/db/commands/create_indexes.cpp b/src/mongo/db/commands/create_indexes.cpp index 0b8206aa661..a3a160ee01c 100644 --- a/src/mongo/db/commands/create_indexes.cpp +++ b/src/mongo/db/commands/create_indexes.cpp @@ -583,7 +583,7 @@ bool runCreateIndexesWithCoordinator(OperationContext* opCtx, IndexBuildsCoordinator::IndexBuildOptions indexBuildOptions = {commitQuorum}; LOGV2(20438, - "Registering index build", + "Index build: registering", "buildUUID"_attr = buildUUID, "namespace"_attr = ns, "collectionUUID"_attr = *collectionUUID, @@ -597,7 +597,7 @@ bool runCreateIndexesWithCoordinator(OperationContext* opCtx, auto deadline = opCtx->getDeadline(); LOGV2(20440, - "Waiting for index build to complete", + "Index build: waiting for index build to complete", "buildUUID"_attr = buildUUID, "deadline"_attr = deadline); @@ -606,7 +606,7 @@ bool runCreateIndexesWithCoordinator(OperationContext* opCtx, stats = buildIndexFuture.get(opCtx); } catch (const ExceptionForCat<ErrorCategory::Interruption>& interruptionEx) { LOGV2(20441, - "Index build received interrupt signal", + "Index build: received interrupt signal", "buildUUID"_attr = buildUUID, "signal"_attr = interruptionEx); @@ -618,7 +618,7 @@ bool runCreateIndexesWithCoordinator(OperationContext* opCtx, // entry from the new primary. if (ErrorCodes::InterruptedDueToReplStateChange == interruptionEx.code()) { LOGV2(20442, - "Index build ignoring interrupt and continuing in background", + "Index build: ignoring interrupt and continuing in background", "buildUUID"_attr = buildUUID); throw; } @@ -640,13 +640,14 @@ bool runCreateIndexesWithCoordinator(OperationContext* opCtx, << ": " << interruptionEx.toString()); indexBuildsCoord->abortIndexBuildByBuildUUID( abortCtx.get(), buildUUID, IndexBuildAction::kPrimaryAbort, abortReason); - LOGV2( - 20443, "Index build aborted due to interruption", "buildUUID"_attr = buildUUID); + LOGV2(20443, + "Index build: aborted due to interruption", + "buildUUID"_attr = buildUUID); } throw; } catch (const ExceptionForCat<ErrorCategory::NotMasterError>& ex) { LOGV2(20444, - "Index build received interrupt signal due to change in replication state", + "Index build: received interrupt signal due to change in replication state", "buildUUID"_attr = buildUUID, "ex"_attr = ex); @@ -655,7 +656,7 @@ bool runCreateIndexesWithCoordinator(OperationContext* opCtx, // entry from the new primary. if (IndexBuildProtocol::kTwoPhase == protocol) { LOGV2(20445, - "Index build ignoring interrupt and continuing in background", + "Index build: ignoring interrupt and continuing in background", "buildUUID"_attr = buildUUID); throw; } @@ -665,11 +666,11 @@ bool runCreateIndexesWithCoordinator(OperationContext* opCtx, indexBuildsCoord->abortIndexBuildByBuildUUID( opCtx, buildUUID, IndexBuildAction::kPrimaryAbort, abortReason); LOGV2( - 20446, "Index build aborted due to NotMaster error", "buildUUID"_attr = buildUUID); + 20446, "Index build: aborted due to NotMaster error", "buildUUID"_attr = buildUUID); throw; } - LOGV2(20447, "Index build completed", "buildUUID"_attr = buildUUID); + LOGV2(20447, "Index build: completed", "buildUUID"_attr = buildUUID); } catch (DBException& ex) { // If the collection is dropped after the initial checks in this function (before the // AutoStatsTracker is created), the IndexBuildsCoordinator (either startIndexBuild() or @@ -677,7 +678,7 @@ bool runCreateIndexesWithCoordinator(OperationContext* opCtx, // considered an error and the command should return success. if (ErrorCodes::NamespaceNotFound == ex.code()) { LOGV2(20448, - "Index build failed: collection dropped", + "Index build: failed because collection dropped", "buildUUID"_attr = buildUUID, "namespace"_attr = ns, "collectionUUID"_attr = *collectionUUID, @@ -686,7 +687,7 @@ bool runCreateIndexesWithCoordinator(OperationContext* opCtx, } // All other errors should be forwarded to the caller with index build information included. - LOGV2(20449, "Index build failed", "buildUUID"_attr = buildUUID, "error"_attr = ex); + LOGV2(20449, "Index build: failed", "buildUUID"_attr = buildUUID, "error"_attr = ex); ex.addContext(str::stream() << "Index build failed: " << buildUUID << ": Collection " << ns << " ( " << *collectionUUID << " )"); diff --git a/src/mongo/db/index/index_access_method.cpp b/src/mongo/db/index/index_access_method.cpp index fa4924ed2ca..d412a3ff8fc 100644 --- a/src/mongo/db/index/index_access_method.cpp +++ b/src/mongo/db/index/index_access_method.cpp @@ -685,9 +685,11 @@ Status AbstractIndexAccessMethod::commitBulk(OperationContext* opCtx, pm.finished(); LOGV2(20685, - "index build: inserted {bulk_getKeysInserted} keys from external sorter into index in " + "Index build: inserted {bulk_getKeysInserted} keys from external sorter into index in " "{timer_seconds} seconds", - message, + "Index build: inserted keys from external sorter into index", + "namespace"_attr = _descriptor->parentNS(), + "index"_attr = _descriptor->indexName(), "keysInserted"_attr = bulk->getKeysInserted(), "duration"_attr = Milliseconds(Seconds(timer.seconds()))); diff --git a/src/mongo/db/index_builds_coordinator.cpp b/src/mongo/db/index_builds_coordinator.cpp index 6a0e4cec184..586364fed55 100644 --- a/src/mongo/db/index_builds_coordinator.cpp +++ b/src/mongo/db/index_builds_coordinator.cpp @@ -763,7 +763,7 @@ void IndexBuildsCoordinator::applyCommitIndexBuild(OperationContext* opCtx, auto fut = replState->sharedPromise.getFuture(); LOGV2(20654, - "Index build joined after commit", + "Index build: joined after commit", "buildUUID"_attr = buildUUID, "result"_attr = fut.waitNoThrow(opCtx)); @@ -1602,7 +1602,10 @@ void IndexBuildsCoordinator::_unregisterIndexBuild( invariant(_allIndexBuilds.erase(replIndexBuildState->buildUUID)); - LOGV2(4656004, "Unregistering index build", "buildUUID"_attr = replIndexBuildState->buildUUID); + LOGV2_DEBUG(4656004, + 1, + "Index build: Unregistering", + "buildUUID"_attr = replIndexBuildState->buildUUID); _indexBuildsManager.unregisterIndexBuild(replIndexBuildState->buildUUID); _indexBuildsCondVar.notify_all(); } @@ -2110,6 +2113,10 @@ void IndexBuildsCoordinator::_buildIndex(OperationContext* opCtx, _waitForNextIndexBuildActionAndCommit(opCtx, replState, indexBuildOptions); } +/* + * First phase is doing a collection scan and inserting keys into sorter. + * Second phase is extracting the sorted keys and writing them into the new index table. + */ void IndexBuildsCoordinator::_scanCollectionAndInsertKeysIntoSorter( OperationContext* opCtx, std::shared_ptr<ReplIndexBuildState> replState) { // Collection scan and insert into index. @@ -2148,8 +2155,8 @@ void IndexBuildsCoordinator::_scanCollectionAndInsertKeysIntoSorter( } } -/** - * Second phase is extracting the sorted keys and writing them into the new index table. +/* + * Third phase is catching up on all the writes that occurred during the first two phases. */ void IndexBuildsCoordinator::_insertKeysFromSideTablesWithoutBlockingWrites( OperationContext* opCtx, std::shared_ptr<ReplIndexBuildState> replState) { @@ -2199,7 +2206,7 @@ void IndexBuildsCoordinator::_insertKeysFromSideTablesBlockingWrites( } /** - * Third phase is catching up on all the writes that occurred during the first two phases. + * Continue the third phase of catching up on all remaining writes that occurred and then commit. * Accepts a commit timestamp for the index (null if not available). */ IndexBuildsCoordinator::CommitResult IndexBuildsCoordinator::_insertKeysFromSideTablesAndCommit( @@ -2265,8 +2272,8 @@ IndexBuildsCoordinator::CommitResult IndexBuildsCoordinator::_insertKeysFromSide << replState->buildUUID << ", collection UUID: " << replState->collectionUUID); - // Perform the third and final drain after releasing a shared lock and reacquiring an - // exclusive lock on the database. + // Perform the third and final drain after releasing a shared lock and reacquiring an exclusive + // lock on the collection. uassertStatusOK(_indexBuildsManager.drainBackgroundWrites( opCtx, replState->buildUUID, @@ -2388,7 +2395,7 @@ IndexBuildsCoordinator::CommitResult IndexBuildsCoordinator::_insertKeysFromSide removeIndexBuildEntryAfterCommitOrAbort(opCtx, dbAndUUID, *replState); replState->stats.numIndexesAfter = getNumIndexesTotal(opCtx, collection); LOGV2(20663, - "Index build completed successfully", + "Index build: completed successfully", "buildUUID"_attr = replState->buildUUID, "namespace"_attr = collection->ns(), "uuid"_attr = replState->collectionUUID, diff --git a/src/mongo/db/index_builds_coordinator.h b/src/mongo/db/index_builds_coordinator.h index 941fb93b0ce..e6c2e01388a 100644 --- a/src/mongo/db/index_builds_coordinator.h +++ b/src/mongo/db/index_builds_coordinator.h @@ -637,7 +637,7 @@ protected: /** * Signals the primary to commit the index build by sending "voteCommitIndexBuild" command * request to it with write concern 'majority', then waits for that command's response. And, - * command gets retried on error. This function gets called after the second draining phase of + * command gets retried on error. This function gets called after the first draining phase of * index build. */ virtual void _signalPrimaryForCommitReadiness( diff --git a/src/mongo/db/index_builds_coordinator_mongod.cpp b/src/mongo/db/index_builds_coordinator_mongod.cpp index 701392aec45..f4796c556de 100644 --- a/src/mongo/db/index_builds_coordinator_mongod.cpp +++ b/src/mongo/db/index_builds_coordinator_mongod.cpp @@ -428,7 +428,8 @@ void IndexBuildsCoordinatorMongod::_signalIfCommitQuorumIsSatisfied( if (!commitQuorumSatisfied) return; - LOGV2(3856201, "Index build commit quorum satisfied", "indexBuildEntry"_attr = indexBuildEntry); + LOGV2( + 3856201, "Index build: commit quorum satisfied", "indexBuildEntry"_attr = indexBuildEntry); _sendCommitQuorumSatisfiedSignal(opCtx, replState); } @@ -634,7 +635,7 @@ void IndexBuildsCoordinatorMongod::_waitForNextIndexBuildActionAndCommit( std::shared_ptr<ReplIndexBuildState> replState, const IndexBuildOptions& indexBuildOptions) { LOGV2(3856203, - "Index build waiting for next action before completing final phase", + "Index build: waiting for next action before completing final phase", "buildUUID"_attr = replState->buildUUID); while (true) { @@ -646,7 +647,7 @@ void IndexBuildsCoordinatorMongod::_waitForNextIndexBuildActionAndCommit( const auto nextAction = _drainSideWritesUntilNextActionIsAvailable(opCtx, replState); LOGV2(3856204, - "Index build received signal", + "Index build: received signal", "buildUUID"_attr = replState->buildUUID, "action"_attr = _indexBuildActionToString(nextAction)); @@ -663,7 +664,7 @@ void IndexBuildsCoordinatorMongod::_waitForNextIndexBuildActionAndCommit( invariant(replState->indexBuildState.getTimestamp(), replState->buildUUID.toString()); LOGV2(3856205, - "Committing index build from oplog entry", + "Index build: committing from oplog entry", "buildUUID"_attr = replState->buildUUID, "commitTimestamp"_attr = replState->indexBuildState.getTimestamp().get(), "collectionUUID"_attr = replState->collectionUUID); |