summaryrefslogtreecommitdiff
diff options
context:
space:
mode:
-rw-r--r--jstests/noPassthrough/commit_quorum_does_not_hang_with_initial_sync.js135
-rw-r--r--jstests/noPassthrough/initial_sync_aborts_two_phase_index_builds.js96
-rw-r--r--jstests/noPassthrough/libs/index_initial_sync.js2
-rw-r--r--src/mongo/client/dbclient_base.cpp20
-rw-r--r--src/mongo/client/dbclient_base.h19
-rw-r--r--src/mongo/db/catalog/create_collection.cpp9
-rw-r--r--src/mongo/db/catalog/database_impl.cpp17
-rw-r--r--src/mongo/db/cloner.cpp4
-rw-r--r--src/mongo/db/index_builds_coordinator.cpp76
-rw-r--r--src/mongo/db/index_builds_coordinator.h14
-rw-r--r--src/mongo/db/index_builds_coordinator_mongod.cpp8
-rw-r--r--src/mongo/db/repl/SConscript1
-rw-r--r--src/mongo/db/repl/collection_cloner.cpp94
-rw-r--r--src/mongo/db/repl/collection_cloner.h17
-rw-r--r--src/mongo/db/repl/collection_cloner_test.cpp8
-rw-r--r--src/mongo/db/repl/oplog.cpp58
-rw-r--r--src/mongo/db/repl_index_build_state.h4
-rw-r--r--src/mongo/db/s/migration_destination_manager.cpp4
-rw-r--r--src/mongo/db/s/shard_key_util.cpp4
-rw-r--r--src/mongo/db/sessions_collection_rs.cpp5
-rw-r--r--src/mongo/db/sessions_collection_standalone.cpp5
-rw-r--r--src/mongo/dbtests/clienttests.cpp22
-rw-r--r--src/mongo/dbtests/querytests.cpp19
-rw-r--r--src/mongo/embedded/index_builds_coordinator_embedded.cpp2
24 files changed, 549 insertions, 94 deletions
diff --git a/jstests/noPassthrough/commit_quorum_does_not_hang_with_initial_sync.js b/jstests/noPassthrough/commit_quorum_does_not_hang_with_initial_sync.js
new file mode 100644
index 00000000000..78dc6792e01
--- /dev/null
+++ b/jstests/noPassthrough/commit_quorum_does_not_hang_with_initial_sync.js
@@ -0,0 +1,135 @@
+/**
+ * Initial syncing a node with two phase index builds should immediately build all ready indexes
+ * from the sync source and only setup the index builder threads for any unfinished index builds
+ * grouped by their buildUUID.
+ *
+ * Previously, an initial syncing node would start and finish the index build when it applied the
+ * "commitIndexBuild" oplog entry, but the primary will no longer send that oplog entry until the
+ * commit quorum is satisfied, which may depend on the initial syncing nodes vote.
+ *
+ * Take into consideration the following scenario where the primary could not achieve the commit
+ * quorum without the initial syncing nodes vote:
+ * 1. Node A (primary) starts a two-phase index build "x_1" with commit quorum "votingMembers".
+ * 2. Node B (secondary) shuts down while building the "x_1" index, preventing the node from sending
+ * its vote to the primary.
+ * 3. Node A cannot achieve the commit quorum and is stuck. The "commitIndexBuild" oplog entry does
+ * not get sent to any other nodes.
+ *
+ * @tags: [requires_replication]
+ */
+(function() {
+"use strict";
+
+load("jstests/noPassthrough/libs/index_build.js");
+
+const dbName = jsTest.name();
+const collName = "commitQuorumWithInitialSync";
+
+const rst = new ReplSetTest({
+ nodes: [
+ {},
+ {
+ // Disallow elections on secondary.
+ rsConfig: {
+ priority: 0,
+ },
+ },
+ ]
+});
+
+rst.startSet();
+rst.initiate();
+
+const primary = rst.getPrimary();
+const db = primary.getDB(dbName);
+const coll = db.getCollection(collName);
+
+if (!(IndexBuildTest.supportsTwoPhaseIndexBuild(primary) &&
+ IndexBuildTest.indexBuildCommitQuorumEnabled(primary))) {
+ jsTestLog(
+ 'Skipping test because two phase index build and index build commit quorum are not supported.');
+ rst.stopSet();
+ return;
+}
+
+assert.commandWorked(coll.insert({a: 1, b: 1, c: 1, d: 1, e: 1, f: 1, g: 1}));
+assert.commandWorked(coll.createIndex({a: 1}, {}, "votingMembers"));
+rst.awaitReplication();
+
+// Start multiple index builds using a commit quorum of "votingMembers", but pause the index build
+// on the secondary, preventing it from voting to commit the index build.
+jsTest.log("Pausing index builds on the secondary");
+let secondary = rst.getSecondary();
+IndexBuildTest.pauseIndexBuilds(secondary);
+
+TestData.dbName = dbName;
+TestData.collName = collName;
+const awaitFirstIndexBuild = startParallelShell(() => {
+ const coll = db.getSiblingDB(TestData.dbName).getCollection(TestData.collName);
+ assert.commandWorked(coll.createIndex({b: 1}, {}, "votingMembers"));
+}, primary.port);
+
+const awaitSecondIndexBuild = startParallelShell(() => {
+ const coll = db.getSiblingDB(TestData.dbName).getCollection(TestData.collName);
+ assert.commandWorked(coll.createIndexes([{c: 1}, {d: 1}], {}, "votingMembers"));
+}, primary.port);
+
+const awaitThirdIndexBuild = startParallelShell(() => {
+ const coll = db.getSiblingDB(TestData.dbName).getCollection(TestData.collName);
+ assert.commandWorked(coll.createIndexes([{e: 1}, {f: 1}, {g: 1}], {}, "votingMembers"));
+}, primary.port);
+
+// Wait for all the indexes to start building on the primary.
+IndexBuildTest.waitForIndexBuildToStart(db, collName, "b_1");
+IndexBuildTest.waitForIndexBuildToStart(db, collName, "c_1");
+IndexBuildTest.waitForIndexBuildToStart(db, collName, "d_1");
+IndexBuildTest.waitForIndexBuildToStart(db, collName, "e_1");
+IndexBuildTest.waitForIndexBuildToStart(db, collName, "f_1");
+IndexBuildTest.waitForIndexBuildToStart(db, collName, "g_1");
+
+// Restart the secondary with a clean data directory to start the initial sync process.
+secondary = rst.restart(1, {
+ startClean: true,
+ setParameter: 'failpoint.initialSyncHangAfterDataCloning=' + tojson({mode: 'alwaysOn'}),
+});
+
+// The secondary node will start any in-progress two-phase index builds from the primary before
+// starting the oplog replay phase. This ensures that the secondary will send its vote to the
+// primary when it is ready to commit the index build. The index build on the secondary will get
+// committed once the primary sends the "commitIndexBuild" oplog entry after the commit quorum is
+// satisfied with the secondaries vote.
+checkLog.containsJson(secondary, 21184);
+
+// Cannot use IndexBuildTest helper functions on the secondary during initial sync.
+function checkForIndexes(indexes) {
+ for (let i = 0; i < indexes.length; i++) {
+ checkLog.containsJson(secondary, 20384, {
+ "properties": function(obj) {
+ return obj.name === indexes[i];
+ }
+ });
+ }
+}
+checkForIndexes(["b_1", "c_1", "d_1", "e_1", "f_1", "g_1"]);
+
+// Checks that the index specs have the proper grouping by ensuring that we only start 3 index
+// builder threads.
+checkLog.containsWithCount(secondary, "Index build initialized", 3);
+
+assert.commandWorked(
+ secondary.adminCommand({configureFailPoint: "initialSyncHangAfterDataCloning", mode: "off"}));
+
+rst.awaitReplication();
+rst.awaitSecondaryNodes();
+
+awaitFirstIndexBuild();
+awaitSecondIndexBuild();
+awaitThirdIndexBuild();
+
+let indexes = secondary.getDB(dbName).getCollection(collName).getIndexes();
+assert.eq(8, indexes.length);
+
+indexes = coll.getIndexes();
+assert.eq(8, indexes.length);
+rst.stopSet();
+}());
diff --git a/jstests/noPassthrough/initial_sync_aborts_two_phase_index_builds.js b/jstests/noPassthrough/initial_sync_aborts_two_phase_index_builds.js
new file mode 100644
index 00000000000..77c2f37ba47
--- /dev/null
+++ b/jstests/noPassthrough/initial_sync_aborts_two_phase_index_builds.js
@@ -0,0 +1,96 @@
+/**
+ * Verifies that an initial syncing node can abort in-progress two phase index builds during the
+ * oplog replay phase.
+ *
+ * @tags: [requires_replication]
+ */
+(function() {
+"use strict";
+
+load("jstests/noPassthrough/libs/index_build.js");
+
+const dbName = jsTest.name();
+const collName = "test";
+
+const rst = new ReplSetTest({
+ nodes: [
+ {},
+ {
+ // Disallow elections on secondary.
+ rsConfig: {
+ priority: 0,
+ },
+ },
+ ]
+});
+
+rst.startSet();
+rst.initiate();
+
+const primary = rst.getPrimary();
+
+if (!(IndexBuildTest.supportsTwoPhaseIndexBuild(primary) &&
+ IndexBuildTest.indexBuildCommitQuorumEnabled(primary))) {
+ jsTestLog(
+ 'Skipping test because two phase index build and index build commit quorum are not supported.');
+ rst.stopSet();
+ return;
+}
+
+const db = primary.getDB(dbName);
+const coll = db.getCollection(collName);
+
+assert.commandWorked(coll.insert({a: 1}));
+assert.commandWorked(coll.createIndex({a: 1}, {}, "votingMembers"));
+rst.awaitReplication();
+
+// Forcefully re-sync the secondary.
+let secondary = rst.restart(1, {
+ startClean: true,
+ setParameter: {
+ 'failpoint.initialSyncHangDuringCollectionClone': tojson(
+ {mode: 'alwaysOn', data: {namespace: "admin.system.version", numDocsToClone: 0}}),
+ }
+});
+
+// Wait until we block on cloning 'admin.system.version'.
+checkLog.containsJson(secondary, 21138);
+
+assert.commandWorked(coll.insert({a: 2}));
+assert.commandWorked(coll.dropIndex({a: 1}));
+
+IndexBuildTest.pauseIndexBuilds(secondary);
+
+// Start an index build on the primary, so that when initial sync is cloning the user collection it
+// sees an in-progress two phase index build.
+TestData.dbName = dbName;
+TestData.collName = collName;
+const awaitIndexBuild = startParallelShell(() => {
+ const coll = db.getSiblingDB(TestData.dbName).getCollection(TestData.collName);
+ assert.commandWorked(coll.createIndex({a: 1}, {}, "votingMembers"));
+}, primary.port);
+
+IndexBuildTest.waitForIndexBuildToStart(db, collName, "a_1");
+
+assert.commandWorked(secondary.adminCommand(
+ {configureFailPoint: "initialSyncHangDuringCollectionClone", mode: "off"}));
+
+rst.awaitReplication();
+rst.awaitSecondaryNodes();
+
+// Check the that secondary hit the background operation in progress error.
+checkLog.containsJson(secondary, 23879, {reason: "Aborting index builds during initial sync"});
+
+IndexBuildTest.resumeIndexBuilds(secondary);
+
+awaitIndexBuild();
+
+let indexes = secondary.getDB(dbName).getCollection(collName).getIndexes();
+assert.eq(2, indexes.length);
+
+indexes = coll.getIndexes();
+assert.eq(2, indexes.length);
+
+rst.stopSet();
+return;
+}());
diff --git a/jstests/noPassthrough/libs/index_initial_sync.js b/jstests/noPassthrough/libs/index_initial_sync.js
index ecd2322d6d9..d22ec50fcd6 100644
--- a/jstests/noPassthrough/libs/index_initial_sync.js
+++ b/jstests/noPassthrough/libs/index_initial_sync.js
@@ -55,7 +55,7 @@ var IndexInitialSyncTest = function(options) {
try {
if (IndexBuildTest.supportsTwoPhaseIndexBuild(primary)) {
IndexBuildTest.assertIndexes(
- secondaryColl, 1, ['_id_'], [], {includeBuildUUIDs: true});
+ secondaryColl, 2, ['_id_'], ['a_1'], {includeBuildUUIDs: true});
} else {
IndexBuildTest.assertIndexes(secondaryColl, 2, ['_id_', 'a_1']);
}
diff --git a/src/mongo/client/dbclient_base.cpp b/src/mongo/client/dbclient_base.cpp
index 4e2d1b36a19..8405348b843 100644
--- a/src/mongo/client/dbclient_base.cpp
+++ b/src/mongo/client/dbclient_base.cpp
@@ -932,22 +932,10 @@ BSONObj makeListIndexesCommand(const NamespaceStringOrUUID& nsOrUuid, bool inclu
} // namespace
-list<BSONObj> DBClientBase::getIndexSpecs(const NamespaceStringOrUUID& nsOrUuid, int options) {
- return _getIndexSpecs(nsOrUuid, makeListIndexesCommand(nsOrUuid, false), options);
-}
-
-std::list<BSONObj> DBClientBase::getReadyIndexSpecs(const NamespaceStringOrUUID& nsOrUuid,
- int options) {
- auto specsWithBuildUUIDs =
- _getIndexSpecs(nsOrUuid, makeListIndexesCommand(nsOrUuid, true), options);
- list<BSONObj> specs;
- for (const auto& spec : specsWithBuildUUIDs) {
- if (spec["buildUUID"]) {
- continue;
- }
- specs.push_back(spec);
- }
- return specs;
+std::list<BSONObj> DBClientBase::getIndexSpecs(const NamespaceStringOrUUID& nsOrUuid,
+ bool includeBuildUUIDs,
+ int options) {
+ return _getIndexSpecs(nsOrUuid, makeListIndexesCommand(nsOrUuid, includeBuildUUIDs), options);
}
std::list<BSONObj> DBClientBase::_getIndexSpecs(const NamespaceStringOrUUID& nsOrUuid,
diff --git a/src/mongo/client/dbclient_base.h b/src/mongo/client/dbclient_base.h
index 5db09d9412e..06b8f20a8d1 100644
--- a/src/mongo/client/dbclient_base.h
+++ b/src/mongo/client/dbclient_base.h
@@ -531,15 +531,20 @@ public:
/**
* Lists indexes on the collection 'nsOrUuid'.
* Includes in-progress indexes.
+ *
+ * If 'includeBuildUUIDs' is true, in-progress index specs will have the following format:
+ * {
+ * spec: <BSONObj>
+ * buildUUID: <UUID>
+ * }
+ * and ready index specs will only list the spec.
+ *
+ * If 'includeBuildUUIDs' is false, only the index spec will be returned without a way to
+ * distinguish between ready and in-progress index specs.
*/
virtual std::list<BSONObj> getIndexSpecs(const NamespaceStringOrUUID& nsOrUuid,
- int options = 0);
-
- /**
- * Lists completed indexes on the collection 'nsOrUuid'.
- */
- virtual std::list<BSONObj> getReadyIndexSpecs(const NamespaceStringOrUUID& nsOrUuid,
- int options = 0);
+ bool includeBuildUUIDs,
+ int options);
virtual void dropIndex(const std::string& ns,
BSONObj keys,
diff --git a/src/mongo/db/catalog/create_collection.cpp b/src/mongo/db/catalog/create_collection.cpp
index 1a863962c26..4728ca234bd 100644
--- a/src/mongo/db/catalog/create_collection.cpp
+++ b/src/mongo/db/catalog/create_collection.cpp
@@ -41,6 +41,7 @@
#include "mongo/db/concurrency/write_conflict_exception.h"
#include "mongo/db/curop.h"
#include "mongo/db/db_raii.h"
+#include "mongo/db/index_builds_coordinator.h"
#include "mongo/db/namespace_string.h"
#include "mongo/db/op_observer.h"
#include "mongo/db/operation_context.h"
@@ -333,6 +334,14 @@ Status createCollectionForApplyOps(OperationContext* opCtx,
/*dropTargetUUID*/ {},
/*numRecords*/ 0U,
stayTemp);
+
+ // Abort any remaining index builds on the temporary collection.
+ IndexBuildsCoordinator::get(opCtx)->abortCollectionIndexBuilds(
+ opCtx,
+ tmpName,
+ futureColl->uuid(),
+ "Aborting index builds on temporary collection");
+
// The existing collection has been successfully moved out of the way.
needsRenaming = false;
}
diff --git a/src/mongo/db/catalog/database_impl.cpp b/src/mongo/db/catalog/database_impl.cpp
index 4e0d4caed1a..1f0de2724b0 100644
--- a/src/mongo/db/catalog/database_impl.cpp
+++ b/src/mongo/db/catalog/database_impl.cpp
@@ -56,6 +56,7 @@
#include "mongo/db/concurrency/write_conflict_exception.h"
#include "mongo/db/curop.h"
#include "mongo/db/index/index_access_method.h"
+#include "mongo/db/index_builds_coordinator.h"
#include "mongo/db/introspect.h"
#include "mongo/db/op_observer.h"
#include "mongo/db/query/collation/collator_factory_interface.h"
@@ -525,13 +526,17 @@ Status DatabaseImpl::renameCollection(OperationContext* opCtx,
if (!collToRename) {
return Status(ErrorCodes::NamespaceNotFound, "collection not found to rename");
}
- invariant(!collToRename->getIndexCatalog()->haveAnyIndexesInProgress(),
- str::stream() << "cannot perform operation: an index build is currently running for "
- "collection "
- << fromNss);
- Collection* toColl = CollectionCatalog::get(opCtx).lookupCollectionByNamespace(opCtx, toNss);
- if (toColl) {
+ // Renaming a collection under the same database name is permitted for two-phase index builds.
+ if (!IndexBuildsCoordinator::supportsTwoPhaseIndexBuild()) {
+ invariant(
+ !collToRename->getIndexCatalog()->haveAnyIndexesInProgress(),
+ str::stream() << "cannot perform operation: an index build is currently running for "
+ "collection "
+ << fromNss);
+
+ Collection* toColl =
+ CollectionCatalog::get(opCtx).lookupCollectionByNamespace(opCtx, toNss);
invariant(
!toColl->getIndexCatalog()->haveAnyIndexesInProgress(),
str::stream() << "cannot perform operation: an index build is currently running for "
diff --git a/src/mongo/db/cloner.cpp b/src/mongo/db/cloner.cpp
index 388b3bf7e8c..0e9eeb5fdce 100644
--- a/src/mongo/db/cloner.cpp
+++ b/src/mongo/db/cloner.cpp
@@ -514,7 +514,9 @@ Status Cloner::copyDb(OperationContext* opCtx,
Lock::TempRelease tempRelease(opCtx->lockState());
for (auto&& params : createCollectionParams) {
const NamespaceString nss(dBName, params.collectionName);
- auto indexSpecs = conn->getIndexSpecs(nss);
+ const bool includeBuildUUIDs = false;
+ const int options = 0;
+ auto indexSpecs = conn->getIndexSpecs(nss, includeBuildUUIDs, options);
collectionIndexSpecs[params.collectionName] = indexSpecs;
diff --git a/src/mongo/db/index_builds_coordinator.cpp b/src/mongo/db/index_builds_coordinator.cpp
index 2d6b5707ca2..b214ebf07dd 100644
--- a/src/mongo/db/index_builds_coordinator.cpp
+++ b/src/mongo/db/index_builds_coordinator.cpp
@@ -492,6 +492,8 @@ std::string IndexBuildsCoordinator::_indexBuildActionToString(IndexBuildAction a
return "Oplog commit";
} else if (action == IndexBuildAction::kOplogAbort) {
return "Oplog abort";
+ } else if (action == IndexBuildAction::kInitialSyncAbort) {
+ return "Initial sync abort";
} else if (action == IndexBuildAction::kRollbackAbort) {
return "Rollback abort";
} else if (action == IndexBuildAction::kPrimaryAbort) {
@@ -594,17 +596,18 @@ NamespaceString getNsFromUUID(OperationContext* opCtx, const UUID& uuid) {
} // namespace
void IndexBuildsCoordinator::applyStartIndexBuild(OperationContext* opCtx,
- bool isInitialSync,
+ ApplicationMode applicationMode,
const IndexBuildOplogEntry& oplogEntry) {
const auto collUUID = oplogEntry.collUUID;
const auto nss = getNsFromUUID(opCtx, collUUID);
IndexBuildsCoordinator::IndexBuildOptions indexBuildOptions;
indexBuildOptions.replSetAndNotPrimaryAtStart = true;
+ indexBuildOptions.applicationMode = applicationMode;
// If this is an initial syncing node, drop any conflicting ready index specs prior to
// proceeding with building them.
- if (isInitialSync) {
+ if (indexBuildOptions.applicationMode == ApplicationMode::kInitialSync) {
auto dbAndUUID = NamespaceStringOrUUID(nss.db().toString(), collUUID);
writeConflictRetry(opCtx, "IndexBuildsCoordinator::applyStartIndexBuild", nss.ns(), [&] {
WriteUnitOfWork wuow(opCtx);
@@ -956,11 +959,23 @@ bool IndexBuildsCoordinator::abortIndexBuildByBuildUUID(OperationContext* opCtx,
// check the replication state and to prevent any state transitions from happening while
// aborting the index build. Once an index build is put into kAborted, the index builder
// thread will be torn down, and an oplog entry must be replicated. Single-phase builds do
- // not have this restriction and may be aborted after a stepDown.
+ // not have this restriction and may be aborted after a stepDown. Initial syncing nodes need
+ // to be able to abort two phase index builds during the oplog replay phase.
if (IndexBuildProtocol::kTwoPhase == replState->protocol) {
- // The DBLock helper takes the RSTL implictly.
+ // The DBLock helper takes the RSTL implicitly.
invariant(opCtx->lockState()->isRSTLLocked());
+
+ // Override the 'signalAction' as this is an initial syncing node.
auto replCoord = repl::ReplicationCoordinator::get(opCtx);
+ if (replCoord->getMemberState().startup2()) {
+ LOGV2_DEBUG(4665902,
+ 1,
+ "Overriding abort 'signalAction' for initial sync",
+ "from"_attr = signalAction,
+ "to"_attr = IndexBuildAction::kInitialSyncAbort);
+ signalAction = IndexBuildAction::kInitialSyncAbort;
+ }
+
if (IndexBuildAction::kPrimaryAbort == signalAction &&
!replCoord->canAcceptWritesFor(opCtx, dbAndUUID)) {
uassertStatusOK({ErrorCodes::NotMaster,
@@ -1044,6 +1059,28 @@ void IndexBuildsCoordinator::_completeAbort(OperationContext* opCtx,
break;
}
// Deletes the index from the durable catalog.
+ case IndexBuildAction::kInitialSyncAbort: {
+ invariant(replState->protocol == IndexBuildProtocol::kTwoPhase);
+ invariant(replCoord->getMemberState().startup2());
+
+ bool isMaster = replCoord->canAcceptWritesFor(opCtx, nss);
+ invariant(!isMaster, str::stream() << "Index build: " << replState->buildUUID);
+ invariant(replState->indexBuildState.isAborted(),
+ str::stream()
+ << "Index build: " << replState->buildUUID
+ << ", index build state: " << replState->indexBuildState.toString());
+ invariant(replState->indexBuildState.getAbortReason(), replState->buildUUID.toString());
+ LOGV2(4665903,
+ "Aborting index build during initial sync",
+ "buildUUID"_attr = replState->buildUUID,
+ "abortReason"_attr = replState->indexBuildState.getAbortReason().get(),
+ "collectionUUID"_attr = replState->collectionUUID);
+
+ _indexBuildsManager.abortIndexBuild(
+ opCtx, coll, replState->buildUUID, MultiIndexBlock::kNoopOnCleanUpFn);
+ break;
+ }
+ // Deletes the index from the durable catalog.
case IndexBuildAction::kOplogAbort: {
invariant(IndexBuildProtocol::kTwoPhase == replState->protocol);
// This signal can be received during primary (drain phase), secondary,
@@ -1208,10 +1245,10 @@ void IndexBuildsCoordinator::restartIndexBuildsForRecovery(OperationContext* opC
IndexBuildsCoordinator::IndexBuildOptions indexBuildOptions;
// Start the index build as if in secondary oplog application.
indexBuildOptions.replSetAndNotPrimaryAtStart = true;
- // Indicate that the intialization should not generate oplog entries or timestamps for the
+ // Indicate that the initialization should not generate oplog entries or timestamps for the
// first catalog write, and that the original durable catalog entries should be dropped and
// replaced.
- indexBuildOptions.twoPhaseRecovery = true;
+ indexBuildOptions.applicationMode = ApplicationMode::kStartupRepair;
// This spawns a new thread and returns immediately. These index builds will start and wait
// for a commit or abort to be replicated.
MONGO_COMPILER_VARIABLE_UNUSED auto fut =
@@ -1639,7 +1676,7 @@ IndexBuildsCoordinator::PostSetupAction IndexBuildsCoordinator::_setUpIndexBuild
OperationContext* opCtx,
std::shared_ptr<ReplIndexBuildState> replState,
Timestamp startTimestamp,
- boost::optional<CommitQuorumOptions> commitQuorum) {
+ const IndexBuildOptions& indexBuildOptions) {
const NamespaceStringOrUUID nssOrUuid{replState->dbName, replState->collectionUUID};
AutoGetCollection autoColl(opCtx, nssOrUuid, MODE_X);
@@ -1654,8 +1691,9 @@ IndexBuildsCoordinator::PostSetupAction IndexBuildsCoordinator::_setUpIndexBuild
// We will not have a start timestamp if we are newly a secondary (i.e. we started as
// primary but there was a stepdown). We will be unable to timestamp the initial catalog write,
- // so we must fail the index build.
- if (replSetAndNotPrimary) {
+ // so we must fail the index build. During initial sync, there is no commit timestamp set.
+ if (replSetAndNotPrimary &&
+ indexBuildOptions.applicationMode != ApplicationMode::kInitialSync) {
uassert(ErrorCodes::NotMaster,
str::stream() << "Replication state changed while setting up the index build: "
<< replState->buildUUID,
@@ -1675,7 +1713,7 @@ IndexBuildsCoordinator::PostSetupAction IndexBuildsCoordinator::_setUpIndexBuild
}
// Two phase index builds should have commit quorum set.
- invariant(commitQuorum,
+ invariant(indexBuildOptions.commitQuorum,
str::stream()
<< "Commit quorum required for two phase index build, buildUUID: "
<< replState->buildUUID
@@ -1684,7 +1722,7 @@ IndexBuildsCoordinator::PostSetupAction IndexBuildsCoordinator::_setUpIndexBuild
// Persist the commit quorum value in the config.system.indexBuilds collection.
IndexBuildEntry indexBuildEntry(replState->buildUUID,
replState->collectionUUID,
- commitQuorum.get(),
+ indexBuildOptions.commitQuorum.get(),
replState->indexNames);
uassertStatusOK(addIndexBuildEntry(opCtx, indexBuildEntry));
@@ -1720,9 +1758,14 @@ IndexBuildsCoordinator::PostSetupAction IndexBuildsCoordinator::_setUpIndexBuild
// our initial oplog entry in setUpIndexBuild().
repl::UnreplicatedWritesBlock uwb(opCtx);
- // Use the provided timestamp to write the initial catalog entry.
- invariant(!startTimestamp.isNull());
- TimestampBlock tsBlock(opCtx, startTimestamp);
+ boost::optional<TimestampBlock> tsBlock;
+ if (indexBuildOptions.applicationMode != ApplicationMode::kInitialSync) {
+ // Use the provided timestamp to write the initial catalog entry. Initial sync does
+ // not set a commit timestamp.
+ invariant(!startTimestamp.isNull());
+ tsBlock.emplace(opCtx, startTimestamp);
+ }
+
uassertStatusOK(_indexBuildsManager.setUpIndexBuild(
opCtx, collection, replState->indexSpecs, replState->buildUUID, onInitFn, options));
}
@@ -1748,12 +1791,13 @@ IndexBuildsCoordinator::PostSetupAction IndexBuildsCoordinator::_setUpIndexBuild
Status IndexBuildsCoordinator::_setUpIndexBuild(OperationContext* opCtx,
const UUID& buildUUID,
Timestamp startTimestamp,
- boost::optional<CommitQuorumOptions> commitQuorum) {
+ const IndexBuildOptions& indexBuildOptions) {
auto replState = invariant(_getIndexBuild(buildUUID));
auto postSetupAction = PostSetupAction::kContinueIndexBuild;
try {
- postSetupAction = _setUpIndexBuildInner(opCtx, replState, startTimestamp, commitQuorum);
+ postSetupAction =
+ _setUpIndexBuildInner(opCtx, replState, startTimestamp, indexBuildOptions);
} catch (const DBException& ex) {
stdx::unique_lock<Latch> lk(_mutex);
_unregisterIndexBuild(lk, replState);
diff --git a/src/mongo/db/index_builds_coordinator.h b/src/mongo/db/index_builds_coordinator.h
index c665a433659..aecb7de265b 100644
--- a/src/mongo/db/index_builds_coordinator.h
+++ b/src/mongo/db/index_builds_coordinator.h
@@ -76,12 +76,18 @@ class ServiceContext;
class IndexBuildsCoordinator {
public:
/**
+ * Represents the set of different application modes used around building indexes that differ
+ * from the default behaviour.
+ */
+ enum class ApplicationMode { kNormal, kStartupRepair, kInitialSync };
+
+ /**
* Contains additional information required by 'startIndexBuild()'.
*/
struct IndexBuildOptions {
boost::optional<CommitQuorumOptions> commitQuorum;
bool replSetAndNotPrimaryAtStart = false;
- bool twoPhaseRecovery = false;
+ ApplicationMode applicationMode = ApplicationMode::kNormal;
};
/**
@@ -173,7 +179,7 @@ public:
* performed the initial ready:false write. Throws if there were any errors building the index.
*/
void applyStartIndexBuild(OperationContext* opCtx,
- bool isInitialSync,
+ ApplicationMode applicationMode,
const IndexBuildOplogEntry& entry);
/**
@@ -491,7 +497,7 @@ protected:
Status _setUpIndexBuild(OperationContext* opCtx,
const UUID& buildUUID,
Timestamp startTimestamp,
- boost::optional<CommitQuorumOptions> commitQuorum);
+ const IndexBuildOptions& indexBuildOptions);
/**
* Acquires locks and sets up index build. Throws on error.
@@ -502,7 +508,7 @@ protected:
PostSetupAction _setUpIndexBuildInner(OperationContext* opCtx,
std::shared_ptr<ReplIndexBuildState> replState,
Timestamp startTimestamp,
- boost::optional<CommitQuorumOptions> commitQuorum);
+ const IndexBuildOptions& indexBuildOptions);
/**
* Sets up the in-memory and durable state of the index build for two-phase recovery.
diff --git a/src/mongo/db/index_builds_coordinator_mongod.cpp b/src/mongo/db/index_builds_coordinator_mongod.cpp
index 004b1ba04a6..b1206039ccf 100644
--- a/src/mongo/db/index_builds_coordinator_mongod.cpp
+++ b/src/mongo/db/index_builds_coordinator_mongod.cpp
@@ -174,7 +174,7 @@ IndexBuildsCoordinatorMongod::startIndexBuild(OperationContext* opCtx,
_indexBuildFinished.notify_one();
});
- if (indexBuildOptions.twoPhaseRecovery) {
+ if (indexBuildOptions.applicationMode == ApplicationMode::kStartupRepair) {
// Two phase index build recovery goes though a different set-up procedure because the
// original index will be dropped first.
invariant(protocol == IndexBuildProtocol::kTwoPhase);
@@ -281,9 +281,8 @@ IndexBuildsCoordinatorMongod::startIndexBuild(OperationContext* opCtx,
ShouldNotConflictWithSecondaryBatchApplicationBlock shouldNotConflictBlock(
opCtx->lockState());
- if (!indexBuildOptions.twoPhaseRecovery) {
- status = _setUpIndexBuild(
- opCtx.get(), buildUUID, startTimestamp, indexBuildOptions.commitQuorum);
+ if (indexBuildOptions.applicationMode != ApplicationMode::kStartupRepair) {
+ status = _setUpIndexBuild(opCtx.get(), buildUUID, startTimestamp, indexBuildOptions);
if (!status.isOK()) {
startPromise.setError(status);
return;
@@ -677,6 +676,7 @@ void IndexBuildsCoordinatorMongod::_waitForNextIndexBuildActionAndCommit(
break;
case IndexBuildAction::kOplogAbort:
case IndexBuildAction::kRollbackAbort:
+ case IndexBuildAction::kInitialSyncAbort:
case IndexBuildAction::kPrimaryAbort:
// The calling thread should have interrupted us before signaling an abort action.
LOGV2_FATAL(4698901, "Index build abort should have interrupted this operation");
diff --git a/src/mongo/db/repl/SConscript b/src/mongo/db/repl/SConscript
index 297dbbab2a8..628a2555a3d 100644
--- a/src/mongo/db/repl/SConscript
+++ b/src/mongo/db/repl/SConscript
@@ -971,6 +971,7 @@ env.Library(
'$BUILD_DIR/mongo/db/catalog/collection_options',
'$BUILD_DIR/mongo/db/commands/list_collections_filter',
'$BUILD_DIR/mongo/db/index_builds_coordinator_interface',
+ '$BUILD_DIR/mongo/db/index_build_entry_helpers',
'$BUILD_DIR/mongo/util/progress_meter',
]
)
diff --git a/src/mongo/db/repl/collection_cloner.cpp b/src/mongo/db/repl/collection_cloner.cpp
index 6ea2c468afc..44216ddc25c 100644
--- a/src/mongo/db/repl/collection_cloner.cpp
+++ b/src/mongo/db/repl/collection_cloner.cpp
@@ -33,6 +33,7 @@
#include "mongo/base/string_data.h"
#include "mongo/db/commands/list_collections_filter.h"
+#include "mongo/db/index_build_entry_helpers.h"
#include "mongo/db/index_builds_coordinator.h"
#include "mongo/db/repl/collection_bulk_loader.h"
#include "mongo/db/repl/collection_cloner.h"
@@ -69,6 +70,10 @@ CollectionCloner::CollectionCloner(const NamespaceString& sourceNss,
_listIndexesStage("listIndexes", this, &CollectionCloner::listIndexesStage),
_createCollectionStage("createCollection", this, &CollectionCloner::createCollectionStage),
_queryStage("query", this, &CollectionCloner::queryStage),
+ _setupIndexBuildersForUnfinishedIndexesStage(
+ "setupIndexBuildersForUnfinishedIndexes",
+ this,
+ &CollectionCloner::setupIndexBuildersForUnfinishedIndexesStage),
_progressMeter(1U, // total will be replaced with count command result.
kProgressMeterSecondsBetween,
kProgressMeterCheckInterval,
@@ -99,7 +104,11 @@ CollectionCloner::CollectionCloner(const NamespaceString& sourceNss,
}
BaseCloner::ClonerStages CollectionCloner::getStages() {
- return {&_countStage, &_listIndexesStage, &_createCollectionStage, &_queryStage};
+ return {&_countStage,
+ &_listIndexesStage,
+ &_createCollectionStage,
+ &_queryStage,
+ &_setupIndexBuildersForUnfinishedIndexesStage};
}
@@ -159,9 +168,9 @@ BaseCloner::AfterStageBehavior CollectionCloner::countStage() {
}
BaseCloner::AfterStageBehavior CollectionCloner::listIndexesStage() {
- auto indexSpecs = IndexBuildsCoordinator::supportsTwoPhaseIndexBuild()
- ? getClient()->getReadyIndexSpecs(_sourceDbAndUuid, QueryOption_SlaveOk)
- : getClient()->getIndexSpecs(_sourceDbAndUuid, QueryOption_SlaveOk);
+ const bool includeBuildUUIDs = IndexBuildsCoordinator::supportsTwoPhaseIndexBuild();
+ auto indexSpecs =
+ getClient()->getIndexSpecs(_sourceDbAndUuid, includeBuildUUIDs, QueryOption_SlaveOk);
if (indexSpecs.empty()) {
LOGV2_WARNING(21143,
"No indexes found for collection {namespace} while cloning from {source}",
@@ -169,16 +178,22 @@ BaseCloner::AfterStageBehavior CollectionCloner::listIndexesStage() {
"namespace"_attr = _sourceNss.ns(),
"source"_attr = getSource());
}
+
+ // Parse the index specs into their respective state, ready or unfinished.
for (auto&& spec : indexSpecs) {
- if (spec["name"].str() == "_id_"_sd) {
+ if (spec.hasField("buildUUID")) {
+ _unfinishedIndexSpecs.push_back(spec.getOwned());
+ } else if (spec.hasField("name") && spec.getStringField("name") == "_id_"_sd) {
_idIndexSpec = spec.getOwned();
} else {
- _indexSpecs.push_back(spec.getOwned());
+ _readyIndexSpecs.push_back(spec.getOwned());
}
}
+
{
stdx::lock_guard<Latch> lk(_mutex);
- _stats.indexes = _indexSpecs.size() + (_idIndexSpec.isEmpty() ? 0 : 1);
+ _stats.indexes = _readyIndexSpecs.size() + _unfinishedIndexSpecs.size() +
+ (_idIndexSpec.isEmpty() ? 0 : 1);
};
if (!_idIndexSpec.isEmpty() && _collectionOptions.autoIndexId == CollectionOptions::NO) {
@@ -192,8 +207,15 @@ BaseCloner::AfterStageBehavior CollectionCloner::listIndexesStage() {
}
BaseCloner::AfterStageBehavior CollectionCloner::createCollectionStage() {
+ if (!IndexBuildsCoordinator::supportsTwoPhaseIndexBuild()) {
+ // Single phase index builds should have an empty '_unfinishedIndexSpecs' vector because in
+ // the 'listIndexesStage', we only populate '_unfinishedIndexSpecs' if a buildUUID is
+ // present. A buildUUID is only present for two phase index builds.
+ invariant(_unfinishedIndexSpecs.empty());
+ }
+
auto collectionBulkLoader = getStorageInterface()->createCollectionForBulkLoading(
- _sourceNss, _collectionOptions, _idIndexSpec, _indexSpecs);
+ _sourceNss, _collectionOptions, _idIndexSpec, _readyIndexSpecs);
uassertStatusOK(collectionBulkLoader.getStatus());
_collLoader = std::move(collectionBulkLoader.getValue());
return kContinueNormally;
@@ -208,6 +230,62 @@ BaseCloner::AfterStageBehavior CollectionCloner::queryStage() {
return kContinueNormally;
}
+BaseCloner::AfterStageBehavior CollectionCloner::setupIndexBuildersForUnfinishedIndexesStage() {
+ if (!IndexBuildsCoordinator::supportsTwoPhaseIndexBuild() || _unfinishedIndexSpecs.empty()) {
+ return kContinueNormally;
+ }
+
+ // Need to group the index specs by 'buildUUID' and start all the index specs with the same
+ // 'buildUUID' on the same index builder thread.
+ stdx::unordered_map<UUID, std::vector<BSONObj>, UUID::Hash> groupedIndexSpecs;
+ for (const auto& unfinishedSpec : _unfinishedIndexSpecs) {
+ UUID buildUUID = uassertStatusOK(UUID::parse(unfinishedSpec["buildUUID"]));
+ groupedIndexSpecs[buildUUID].push_back(unfinishedSpec["spec"].Obj());
+ }
+
+ auto opCtx = cc().makeOperationContext();
+
+ for (const auto& groupedIndexSpec : groupedIndexSpecs) {
+ std::vector<std::string> indexNames;
+ std::vector<BSONObj> indexSpecs;
+ for (const auto& indexSpec : groupedIndexSpec.second) {
+ std::string indexName = indexSpec.getStringField(IndexDescriptor::kIndexNameFieldName);
+ indexNames.push_back(indexName);
+ indexSpecs.push_back(indexSpec.getOwned());
+ }
+
+ UnreplicatedWritesBlock uwb(opCtx.get());
+
+ // This spawns a new thread and returns immediately once the index build has been
+ // registered with the IndexBuildsCoordinator.
+ try {
+ IndexBuildsCoordinator::get(opCtx.get())
+ ->applyStartIndexBuild(opCtx.get(),
+ IndexBuildsCoordinator::ApplicationMode::kInitialSync,
+ {getSourceUuid(),
+ repl::OplogEntry::CommandType::kStartIndexBuild,
+ "createIndexes",
+ groupedIndexSpec.first,
+ std::move(indexNames),
+ std::move(indexSpecs),
+ boost::none});
+ } catch (const ExceptionFor<ErrorCodes::IndexAlreadyExists>&) {
+ // Suppress the IndexAlreadyExists error code.
+ // It's possible for the DBDirectClient to return duplicate index specs with different
+ // buildUUIDs from the sync source due to getMore() making multiple network calls.
+ // In these cases, we can ignore this error as the oplog replay phase will correctly
+ // abort and start the appropriate indexes.
+ // Example:
+ // - listIndexes on the sync source sees x_1 (ready: false) with buildUUID ‘x’.
+ // - Sync source aborts the index build with buildUUID ‘x’.
+ // - Sync source starts x_1 (ready: false) with buildUUID ‘y’.
+ // - getMore on listIndexes sees x_1 with buildUUID 'y'.
+ }
+ }
+
+ return kContinueNormally;
+}
+
void CollectionCloner::runQuery() {
// Non-resumable query.
Query query;
diff --git a/src/mongo/db/repl/collection_cloner.h b/src/mongo/db/repl/collection_cloner.h
index ba7c66c08ae..3954040132c 100644
--- a/src/mongo/db/repl/collection_cloner.h
+++ b/src/mongo/db/repl/collection_cloner.h
@@ -197,6 +197,11 @@ private:
AfterStageBehavior queryStage();
/**
+ * Stage function that sets up index builders for any unfinished two-phase index builds.
+ */
+ AfterStageBehavior setupIndexBuildersForUnfinishedIndexesStage();
+
+ /**
* Put all results from a query batch into a buffer to be inserted, and schedule
* it to be inserted.
*/
@@ -236,13 +241,15 @@ private:
// The size of the batches of documents returned in collection cloning.
int _collectionClonerBatchSize; // (R)
- CollectionClonerStage _countStage; // (R)
- CollectionClonerStage _listIndexesStage; // (R)
- CollectionClonerStage _createCollectionStage; // (R)
- CollectionClonerQueryStage _queryStage; // (R)
+ CollectionClonerStage _countStage; // (R)
+ CollectionClonerStage _listIndexesStage; // (R)
+ CollectionClonerStage _createCollectionStage; // (R)
+ CollectionClonerQueryStage _queryStage; // (R)
+ CollectionClonerStage _setupIndexBuildersForUnfinishedIndexesStage; // (R)
ProgressMeter _progressMeter; // (X) progress meter for this instance.
- std::vector<BSONObj> _indexSpecs; // (X) Except for _id_
+ std::vector<BSONObj> _readyIndexSpecs; // (X) Except for _id_
+ std::vector<BSONObj> _unfinishedIndexSpecs; // (X)
BSONObj _idIndexSpec; // (X)
std::unique_ptr<CollectionBulkLoader> _collLoader; // (X)
// Function for scheduling database work using the executor.
diff --git a/src/mongo/db/repl/collection_cloner_test.cpp b/src/mongo/db/repl/collection_cloner_test.cpp
index 0527c205a39..ae5477abc05 100644
--- a/src/mongo/db/repl/collection_cloner_test.cpp
+++ b/src/mongo/db/repl/collection_cloner_test.cpp
@@ -101,8 +101,12 @@ protected:
return cloner->_progressMeter;
}
- std::vector<BSONObj>& getIndexSpecs(CollectionCloner* cloner) {
- return cloner->_indexSpecs;
+ std::vector<BSONObj> getIndexSpecs(CollectionCloner* cloner) {
+ std::vector<BSONObj> indexSpecs = cloner->_readyIndexSpecs;
+ for (const auto& unfinishedSpec : cloner->_unfinishedIndexSpecs) {
+ indexSpecs.push_back(unfinishedSpec["spec"].Obj());
+ }
+ return indexSpecs;
}
BSONObj& getIdIndexSpec(CollectionCloner* cloner) {
diff --git a/src/mongo/db/repl/oplog.cpp b/src/mongo/db/repl/oplog.cpp
index a047442c447..d08874c4b84 100644
--- a/src/mongo/db/repl/oplog.cpp
+++ b/src/mongo/db/repl/oplog.cpp
@@ -155,6 +155,23 @@ bool shouldBuildInForeground(OperationContext* opCtx,
return false;
}
+void abortIndexBuilds(OperationContext* opCtx,
+ const OplogEntry::CommandType& commandType,
+ const NamespaceString& nss,
+ const std::string& reason) {
+ auto indexBuildsCoordinator = IndexBuildsCoordinator::get(opCtx);
+ if (commandType == OplogEntry::CommandType::kDropDatabase) {
+ indexBuildsCoordinator->abortDatabaseIndexBuilds(opCtx, nss.db(), reason);
+ } else if (commandType == OplogEntry::CommandType::kDrop ||
+ commandType == OplogEntry::CommandType::kDropIndexes ||
+ commandType == OplogEntry::CommandType::kRenameCollection) {
+ const boost::optional<UUID> collUUID =
+ CollectionCatalog::get(opCtx).lookupUUIDByNSS(opCtx, nss);
+ invariant(collUUID);
+
+ indexBuildsCoordinator->abortCollectionIndexBuilds(opCtx, nss, *collUUID, reason);
+ }
+}
} // namespace
@@ -750,9 +767,13 @@ const StringMap<ApplyOpMetadata> kOpsMap = {
"Error parsing 'startIndexBuild' oplog entry");
}
- const bool isInitialSync = mode == OplogApplication::Mode::kInitialSync;
+ IndexBuildsCoordinator::ApplicationMode applicationMode =
+ IndexBuildsCoordinator::ApplicationMode::kNormal;
+ if (mode == OplogApplication::Mode::kInitialSync) {
+ applicationMode = IndexBuildsCoordinator::ApplicationMode::kInitialSync;
+ }
IndexBuildsCoordinator::get(opCtx)->applyStartIndexBuild(
- opCtx, isInitialSync, swOplogEntry.getValue());
+ opCtx, applicationMode, swOplogEntry.getValue());
return Status::OK();
},
{ErrorCodes::IndexAlreadyExists,
@@ -1550,6 +1571,18 @@ Status applyCommand_inlock(OperationContext* opCtx,
throw WriteConflictException();
}
case ErrorCodes::BackgroundOperationInProgressForDatabase: {
+ if (mode == OplogApplication::Mode::kInitialSync) {
+ abortIndexBuilds(opCtx,
+ entry.getCommandType(),
+ nss,
+ "Aborting index builds during initial sync");
+ LOGV2_DEBUG(4665900,
+ 1,
+ "Conflicting DDL operation encountered during initial sync; "
+ "aborting index build and retrying",
+ "db"_attr = nss.db());
+ }
+
Lock::TempRelease release(opCtx->lockState());
BackgroundOperation::awaitNoBgOpInProgForDb(nss.db());
@@ -1568,14 +1601,25 @@ Status applyCommand_inlock(OperationContext* opCtx,
break;
}
case ErrorCodes::BackgroundOperationInProgressForNamespace: {
- Lock::TempRelease release(opCtx->lockState());
-
Command* cmd = CommandHelpers::findCommand(o.firstElement().fieldName());
invariant(cmd);
- // TODO: This parse could be expensive and not worth it.
- auto ns =
- cmd->parse(opCtx, OpMsgRequest::fromDBAndBody(nss.db(), o))->ns().toString();
+ auto ns = cmd->parse(opCtx, OpMsgRequest::fromDBAndBody(nss.db(), o))->ns();
+
+ if (mode == OplogApplication::Mode::kInitialSync) {
+ abortIndexBuilds(opCtx,
+ entry.getCommandType(),
+ ns,
+ "Aborting index builds during initial sync");
+ LOGV2_DEBUG(4665901,
+ 1,
+ "Conflicting DDL operation encountered during initial sync; "
+ "aborting index build and retrying",
+ "namespace"_attr = ns);
+ }
+
+ Lock::TempRelease release(opCtx->lockState());
+
auto swUUID = entry.getUuid();
if (!swUUID) {
LOGV2_ERROR(21261,
diff --git a/src/mongo/db/repl_index_build_state.h b/src/mongo/db/repl_index_build_state.h
index 7b1fd07bf7f..f0becd468a8 100644
--- a/src/mongo/db/repl_index_build_state.h
+++ b/src/mongo/db/repl_index_build_state.h
@@ -81,6 +81,10 @@ enum class IndexBuildAction {
*/
kRollbackAbort,
/**
+ * Abort signal set on initial sync.
+ */
+ kInitialSyncAbort,
+ /**
* Abort signal set by createIndexes cmd or by drop databases/collections/indexes cmds
*/
kPrimaryAbort,
diff --git a/src/mongo/db/s/migration_destination_manager.cpp b/src/mongo/db/s/migration_destination_manager.cpp
index 9dfc07a5bd3..4b3a57f2e54 100644
--- a/src/mongo/db/s/migration_destination_manager.cpp
+++ b/src/mongo/db/s/migration_destination_manager.cpp
@@ -651,7 +651,9 @@ void MigrationDestinationManager::cloneCollectionIndexesAndOptions(
// Determine which indexes exist on the local collection that don't exist on the donor's
// collection.
DBDirectClient client(opCtx);
- auto indexes = client.getIndexSpecs(nss);
+ const bool includeBuildUUIDs = false;
+ const int options = 0;
+ auto indexes = client.getIndexSpecs(nss, includeBuildUUIDs, options);
for (auto&& recipientIndex : indexes) {
bool dropIndex = true;
for (auto&& donorIndex : collectionOptionsAndIndexes.indexSpecs) {
diff --git a/src/mongo/db/s/shard_key_util.cpp b/src/mongo/db/s/shard_key_util.cpp
index bd0f192dd34..7bbeed2614d 100644
--- a/src/mongo/db/s/shard_key_util.cpp
+++ b/src/mongo/db/s/shard_key_util.cpp
@@ -189,7 +189,9 @@ void validateShardKeyIndexExistsOrCreateIfPossible(OperationContext* opCtx,
std::vector<BSONObj> ValidationBehaviorsShardCollection::loadIndexes(
const NamespaceString& nss) const {
- std::list<BSONObj> indexes = _localClient->getIndexSpecs(nss);
+ const bool includeBuildUUIDs = false;
+ const int options = 0;
+ std::list<BSONObj> indexes = _localClient->getIndexSpecs(nss, includeBuildUUIDs, options);
// Convert std::list to a std::vector.
return std::vector<BSONObj>{std::make_move_iterator(std::begin(indexes)),
std::make_move_iterator(std::end(indexes))};
diff --git a/src/mongo/db/sessions_collection_rs.cpp b/src/mongo/db/sessions_collection_rs.cpp
index a102478bb5c..664f34c0c8b 100644
--- a/src/mongo/db/sessions_collection_rs.cpp
+++ b/src/mongo/db/sessions_collection_rs.cpp
@@ -149,7 +149,10 @@ void SessionsCollectionRS::setupSessionsCollection(OperationContext* opCtx) {
void SessionsCollectionRS::checkSessionsCollectionExists(OperationContext* opCtx) {
DBDirectClient client(opCtx);
- auto indexes = client.getIndexSpecs(NamespaceString::kLogicalSessionsNamespace);
+ const bool includeBuildUUIDs = false;
+ const int options = 0;
+ auto indexes = client.getIndexSpecs(
+ NamespaceString::kLogicalSessionsNamespace, includeBuildUUIDs, options);
uassert(ErrorCodes::NamespaceNotFound,
str::stream() << NamespaceString::kLogicalSessionsNamespace << " does not exist",
diff --git a/src/mongo/db/sessions_collection_standalone.cpp b/src/mongo/db/sessions_collection_standalone.cpp
index 3032d521a18..477f20b28a9 100644
--- a/src/mongo/db/sessions_collection_standalone.cpp
+++ b/src/mongo/db/sessions_collection_standalone.cpp
@@ -72,7 +72,10 @@ void SessionsCollectionStandalone::setupSessionsCollection(OperationContext* opC
void SessionsCollectionStandalone::checkSessionsCollectionExists(OperationContext* opCtx) {
DBDirectClient client(opCtx);
- auto indexes = client.getIndexSpecs(NamespaceString::kLogicalSessionsNamespace);
+ const bool includeBuildUUIDs = false;
+ const int options = 0;
+ auto indexes = client.getIndexSpecs(
+ NamespaceString::kLogicalSessionsNamespace, includeBuildUUIDs, options);
uassert(ErrorCodes::NamespaceNotFound,
str::stream() << NamespaceString::kLogicalSessionsNamespace << " does not exist",
diff --git a/src/mongo/dbtests/clienttests.cpp b/src/mongo/dbtests/clienttests.cpp
index a18f4c259b7..b007e2956ca 100644
--- a/src/mongo/dbtests/clienttests.cpp
+++ b/src/mongo/dbtests/clienttests.cpp
@@ -82,20 +82,23 @@ public:
OperationContext& opCtx = *opCtxPtr;
DBDirectClient db(&opCtx);
+ const bool includeBuildUUIDs = false;
+ const int options = 0;
+
db.insert(ns(), BSON("x" << 2));
- ASSERT_EQUALS(1u, db.getIndexSpecs(nss()).size());
+ ASSERT_EQUALS(1u, db.getIndexSpecs(nss(), includeBuildUUIDs, options).size());
ASSERT_OK(dbtests::createIndex(&opCtx, ns(), BSON("x" << 1)));
- ASSERT_EQUALS(2u, db.getIndexSpecs(nss()).size());
+ ASSERT_EQUALS(2u, db.getIndexSpecs(nss(), includeBuildUUIDs, options).size());
db.dropIndex(ns(), BSON("x" << 1));
- ASSERT_EQUALS(1u, db.getIndexSpecs(nss()).size());
+ ASSERT_EQUALS(1u, db.getIndexSpecs(nss(), includeBuildUUIDs, options).size());
ASSERT_OK(dbtests::createIndex(&opCtx, ns(), BSON("x" << 1)));
- ASSERT_EQUALS(2u, db.getIndexSpecs(nss()).size());
+ ASSERT_EQUALS(2u, db.getIndexSpecs(nss(), includeBuildUUIDs, options).size());
db.dropIndexes(ns());
- ASSERT_EQUALS(1u, db.getIndexSpecs(nss()).size());
+ ASSERT_EQUALS(1u, db.getIndexSpecs(nss(), includeBuildUUIDs, options).size());
}
};
@@ -120,20 +123,23 @@ public:
ASSERT(collection);
IndexCatalog* indexCatalog = collection->getIndexCatalog();
+ const bool includeBuildUUIDs = false;
+ const int options = 0;
+
ASSERT_EQUALS(1, indexCatalog->numIndexesReady(&opCtx));
// _id index
- ASSERT_EQUALS(1U, db.getIndexSpecs(nss()).size());
+ ASSERT_EQUALS(1U, db.getIndexSpecs(nss(), includeBuildUUIDs, options).size());
ASSERT_EQUALS(ErrorCodes::DuplicateKey,
dbtests::createIndex(&opCtx, ns(), BSON("y" << 1), true));
ASSERT_EQUALS(1, indexCatalog->numIndexesReady(&opCtx));
- ASSERT_EQUALS(1U, db.getIndexSpecs(nss()).size());
+ ASSERT_EQUALS(1U, db.getIndexSpecs(nss(), includeBuildUUIDs, options).size());
ASSERT_OK(dbtests::createIndex(&opCtx, ns(), BSON("x" << 1), true));
ASSERT_EQUALS(2, indexCatalog->numIndexesReady(&opCtx));
- ASSERT_EQUALS(2U, db.getIndexSpecs(nss()).size());
+ ASSERT_EQUALS(2U, db.getIndexSpecs(nss(), includeBuildUUIDs, options).size());
}
};
diff --git a/src/mongo/dbtests/querytests.cpp b/src/mongo/dbtests/querytests.cpp
index d017dcd2311..786df7534a9 100644
--- a/src/mongo/dbtests/querytests.cpp
+++ b/src/mongo/dbtests/querytests.cpp
@@ -916,10 +916,14 @@ public:
return NamespaceString(ns());
}
void index() {
- ASSERT_EQUALS(2u, _client.getIndexSpecs(nss()).size());
+ const bool includeBuildUUIDs = false;
+ const int options = 0;
+ ASSERT_EQUALS(2u, _client.getIndexSpecs(nss(), includeBuildUUIDs, options).size());
}
void noIndex() {
- ASSERT_EQUALS(0u, _client.getIndexSpecs(nss()).size());
+ const bool includeBuildUUIDs = false;
+ const int options = 0;
+ ASSERT_EQUALS(0u, _client.getIndexSpecs(nss(), includeBuildUUIDs, options).size());
}
void checkIndex() {
ASSERT_OK(dbtests::createIndex(&_opCtx, ns(), BSON("a" << 1)));
@@ -1815,14 +1819,21 @@ public:
insert(ns(), BSON("a" << 2));
insert(ns(), BSON("a" << 3));
+ const bool includeBuildUUIDs = false;
+ const int options = 0;
+
auto specsWithIdIndexOnly =
- _client.getIndexSpecs(NamespaceStringOrUUID(nss().db().toString(), *coll_opts.uuid));
+ _client.getIndexSpecs(NamespaceStringOrUUID(nss().db().toString(), *coll_opts.uuid),
+ includeBuildUUIDs,
+ options);
ASSERT_EQUALS(1U, specsWithIdIndexOnly.size());
ASSERT_OK(dbtests::createIndex(&_opCtx, ns(), BSON("a" << 1), true));
auto specsWithBothIndexes =
- _client.getIndexSpecs(NamespaceStringOrUUID(nss().db().toString(), *coll_opts.uuid));
+ _client.getIndexSpecs(NamespaceStringOrUUID(nss().db().toString(), *coll_opts.uuid),
+ includeBuildUUIDs,
+ options);
ASSERT_EQUALS(2U, specsWithBothIndexes.size());
}
};
diff --git a/src/mongo/embedded/index_builds_coordinator_embedded.cpp b/src/mongo/embedded/index_builds_coordinator_embedded.cpp
index 1a8dddc2d39..d5dbfeab5c4 100644
--- a/src/mongo/embedded/index_builds_coordinator_embedded.cpp
+++ b/src/mongo/embedded/index_builds_coordinator_embedded.cpp
@@ -65,7 +65,7 @@ IndexBuildsCoordinatorEmbedded::startIndexBuild(OperationContext* opCtx,
return statusWithOptionalResult.getValue().get();
}
- auto status = _setUpIndexBuild(opCtx, buildUUID, Timestamp(), indexBuildOptions.commitQuorum);
+ auto status = _setUpIndexBuild(opCtx, buildUUID, Timestamp(), indexBuildOptions);
if (!status.isOK()) {
return status;
}