summaryrefslogtreecommitdiff
diff options
context:
space:
mode:
-rw-r--r--jstests/noPassthrough/set_user_write_block_mode.js101
-rw-r--r--src/mongo/db/commands/set_user_write_block_mode_command.cpp19
-rw-r--r--src/mongo/db/index_builds_coordinator.cpp44
-rw-r--r--src/mongo/db/index_builds_coordinator.h10
-rw-r--r--src/mongo/db/index_builds_coordinator_mongod.cpp26
-rw-r--r--src/mongo/db/s/SConscript1
-rw-r--r--src/mongo/db/s/global_user_write_block_state.cpp17
-rw-r--r--src/mongo/db/s/global_user_write_block_state.h19
-rw-r--r--src/mongo/db/s/shardsvr_set_user_write_block_mode_command.cpp20
-rw-r--r--src/mongo/db/user_write_block_mode_op_observer.cpp9
-rw-r--r--src/mongo/db/user_write_block_mode_op_observer.h18
11 files changed, 240 insertions, 44 deletions
diff --git a/jstests/noPassthrough/set_user_write_block_mode.js b/jstests/noPassthrough/set_user_write_block_mode.js
index bc8ebead019..4ee7b109508 100644
--- a/jstests/noPassthrough/set_user_write_block_mode.js
+++ b/jstests/noPassthrough/set_user_write_block_mode.js
@@ -206,34 +206,97 @@ function runTest(fixture) {
fixture.asAdmin(({conn}) => testCheckedOps(conn, true));
fixture.asUser(({conn}) => testCheckedOps(conn, true));
- // Test that enabling write blocking while there is an active index build will cause the index
- // build to fail.
+ // Test that enabling write blocking while there is an active index build on a user collection
+ // (i.e. non-internal) will cause the index build to fail.
fixture.asUser(({conn}) => {
const db = conn.getDB(jsTestName());
assert.commandWorked(db.createCollection("test"));
assert.commandWorked(db.test.insert({"a": 2}));
});
- const fp = fixture.setFailPoint('hangAfterInitializingIndexBuild');
- // This createIndex should hang at setup, and when it resumes, userWriteBlockMode will be
- // enabled and it should eventually fail.
- const parallelShell = fixture.runInParallelShell(false /* asAdmin */,
- `({conn}) => {
- assert.commandFailedWithCode(
- conn.getDB(jsTestName()).test.createIndex({"a": 1}, {"name": "index"}),
- ErrorCodes.UserWritesBlocked);
+ fixture.asAdmin(({conn}) => {
+ // We use config.system.sessions because it is a collection in an internal DB (config) which
+ // is sharded, meaning index builds will be handled by the shard servers. Indexes on
+ // non-sharded collections in internal DBs are built by the config server, which doesn't
+ // have the UserWriteBlockModeOpObserver installed.
+ const config = conn.getDB('config');
+ assert.commandWorked(config.createCollection("system.sessions"));
+ assert.commandWorked(config.system.sessions.insert({"a": 2}));
+ });
+
+ const testParallelShellWithFailpoint = makeParallelShell => {
+ const fp = fixture.setFailPoint('hangAfterInitializingIndexBuild');
+ const shell = makeParallelShell();
+ fp.wait();
+ fixture.enableWriteBlockMode();
+ fp.off();
+ shell();
+ fixture.disableWriteBlockMode();
+ };
+
+ const indexName = "testIndex";
+
+ // Test that index builds on user collections spawned by both non-privileged and privileged
+ // users will be aborted on enableWriteBlockMode.
+ testParallelShellWithFailpoint(() => fixture.runInParallelShell(false /* asAdmin */,
+ `({conn}) => {
+ assert.commandFailedWithCode(
+ conn.getDB(jsTestName()).test.createIndex({"a": 1}, {"name": "${indexName}"}),
+ ErrorCodes.IndexBuildAborted);
+ }`));
+ testParallelShellWithFailpoint(() => fixture.runInParallelShell(true /* asAdmin */,
+ `({conn}) => {
+ assert.commandFailedWithCode(
+ conn.getDB(jsTestName()).test.createIndex({"a": 1}, {"name": "${indexName}"}),
+ ErrorCodes.IndexBuildAborted);
+ }`));
+
+ // Test that index builds on non-user (internal collections) won't be aborted on
+ // enableWriteBlockMode.
+ testParallelShellWithFailpoint(() => fixture.runInParallelShell(true /* asAdmin */,
+ `({conn}) => {
+ assert.commandWorked(
+ conn.getDB('config').system.sessions.createIndex(
+ {"a": 1}, {"name": "${indexName}"}));
+ }`));
+
+ // Ensure index was not successfully created on user db, but was on internal db.
+ fixture.asAdmin(({conn}) => {
+ assert.eq(undefined,
+ conn.getDB(jsTestName()).test.getIndexes().find(i => i.name === indexName));
+ assert.neq(
+ undefined,
+ conn.getDB('config').system.sessions.getIndexes().find(i => i.name === indexName));
+ });
+
+ // Test that index builds which hang before commit will block activation of
+ // enableWriteBlockMode.
+ {
+ const fp = fixture.setFailPoint("hangIndexBuildBeforeCommit");
+ const waitIndexBuild = fixture.runInParallelShell(true /* asAdmin */,
+ `({conn}) => {
+ assert.commandWorked(
+ conn.getDB(jsTestName()).test.createIndex({"a": 1}, {"name": "${indexName}"}));
}`);
+ fp.wait();
- // Let index build progress to the point where it hits the failpoint.
- fp.wait();
- fixture.enableWriteBlockMode();
- fp.off();
- parallelShell();
+ const waitWriteBlock = fixture.runInParallelShell(true /* asAdmin */,
+ `({conn}) => {
+ assert.commandWorked(
+ conn.getDB("admin").runCommand({setUserWriteBlockMode: 1, global: true}));
+ }`);
+ // Wait, and ensure that the setUserWriteBlockMode has not finished yet (it must wait for
+ // the index build to finish).
+ sleep(3000);
+ fixture.assertWriteBlockMode(UserWriteBlockHelpers.WriteBlockState.DISABLED);
+
+ fp.off();
+ waitIndexBuild();
+ waitWriteBlock();
+ fixture.assertWriteBlockMode(UserWriteBlockHelpers.WriteBlockState.ENABLED);
- // Ensure index was not created.
- fixture.asAdmin(
- ({conn}) => assert.eq(
- undefined, conn.getDB(jsTestName()).test.getIndexes().find(i => i.name === "index")));
+ fixture.disableWriteBlockMode();
+ }
if (fixture.takeGlobalLock) {
// Test that serverStatus will produce WriteBlockState.UNKNOWN when the global lock is held.
diff --git a/src/mongo/db/commands/set_user_write_block_mode_command.cpp b/src/mongo/db/commands/set_user_write_block_mode_command.cpp
index b1d645dda4b..f4f6cb1d61f 100644
--- a/src/mongo/db/commands/set_user_write_block_mode_command.cpp
+++ b/src/mongo/db/commands/set_user_write_block_mode_command.cpp
@@ -35,8 +35,10 @@
#include "mongo/db/commands.h"
#include "mongo/db/commands/feature_compatibility_version.h"
#include "mongo/db/commands/set_user_write_block_mode_gen.h"
+#include "mongo/db/index_builds_coordinator.h"
#include "mongo/db/repl/repl_client_info.h"
#include "mongo/db/repl/replication_coordinator.h"
+#include "mongo/db/s/global_user_write_block_state.h"
#include "mongo/db/s/user_writes_recoverable_critical_section_service.h"
#include "mongo/db/server_feature_flags_gen.h"
#include "mongo/logv2/log.h"
@@ -74,6 +76,9 @@ public:
repl::ReplicationCoordinator::get(opCtx)->getReplicationMode() !=
repl::ReplicationCoordinator::modeNone);
+ // Only one attempt to change write block mode may make progress at once, because the
+ // way we enable/disable user index build blocking is not concurrency-safe.
+ stdx::lock_guard lock(_mutex);
{
// TODO SERVER-65010 Remove FCV guard once 6.0 has branched out
FixedFCVRegion fixedFcvRegion(opCtx);
@@ -83,6 +88,18 @@ public:
serverGlobalParams.featureCompatibility));
if (request().getGlobal()) {
+ // Enabling write block mode on a replicaset requires several steps
+ // First, we must prevent new index builds from starting
+ auto writeBlockState = GlobalUserWriteBlockState::get(opCtx);
+ writeBlockState->enableUserIndexBuildBlocking(opCtx);
+ // Ensure that we eventually restore index build state.
+ ScopeGuard guard(
+ [&]() { writeBlockState->disableUserIndexBuildBlocking(opCtx); });
+ // Abort and wait for ongoing index builds to finish.
+ IndexBuildsCoordinator::get(opCtx)->abortUserIndexBuildsForUserWriteBlocking(
+ opCtx);
+
+ // Engage write blocking
UserWritesRecoverableCriticalSectionService::get(opCtx)
->acquireRecoverableCriticalSectionBlockingUserWrites(
opCtx,
@@ -122,6 +139,8 @@ public:
->isAuthorizedForPrivilege(Privilege{ResourcePattern::forClusterResource(),
ActionType::setUserWriteBlockMode}));
}
+
+ Mutex _mutex = MONGO_MAKE_LATCH("SetUserWriteBlockModeCommand::_mutex");
};
} setUserWriteBlockModeCommand;
} // namespace
diff --git a/src/mongo/db/index_builds_coordinator.cpp b/src/mongo/db/index_builds_coordinator.cpp
index 87b48b874eb..6b224824e43 100644
--- a/src/mongo/db/index_builds_coordinator.cpp
+++ b/src/mongo/db/index_builds_coordinator.cpp
@@ -847,6 +847,50 @@ void IndexBuildsCoordinator::abortAllIndexBuildsForInitialSync(OperationContext*
}
}
+void IndexBuildsCoordinator::abortUserIndexBuildsForUserWriteBlocking(OperationContext* opCtx) {
+ LOGV2(6511600,
+ "About to abort index builders running on user databases for user write blocking");
+
+ auto builds = [&]() -> std::vector<std::shared_ptr<ReplIndexBuildState>> {
+ auto indexBuildFilter = [](const auto& replState) {
+ return !NamespaceString(replState.dbName).isOnInternalDb();
+ };
+ return activeIndexBuilds.filterIndexBuilds(indexBuildFilter);
+ }();
+
+ std::vector<std::shared_ptr<ReplIndexBuildState>> buildsWaitingToFinish;
+
+ for (const auto& replState : builds) {
+ if (!abortIndexBuildByBuildUUID(opCtx,
+ replState->buildUUID,
+ IndexBuildAction::kPrimaryAbort,
+ "User write blocking")) {
+ // If the index build is already finishing and thus can't be aborted, we must wait on
+ // it.
+ LOGV2(6511601,
+ "Index build: failed to abort index build for write blocking, will wait for "
+ "completion instead",
+ "buildUUID"_attr = replState->buildUUID,
+ "db"_attr = replState->dbName,
+ "collectionUUID"_attr = replState->collectionUUID);
+ buildsWaitingToFinish.push_back(replState);
+ }
+ }
+
+ // Before returning, we must wait on all index builds which could not be aborted to finish.
+ // Otherwise, index builds started before enabling user write block mode could commit after
+ // enabling it.
+ for (const auto& replState : buildsWaitingToFinish) {
+ LOGV2(6511602,
+ "Waiting on index build to finish for user write blocking",
+ "buildUUID"_attr = replState->buildUUID,
+ "db"_attr = replState->dbName,
+ "collectionUUID"_attr = replState->collectionUUID);
+ awaitNoIndexBuildInProgressForCollection(
+ opCtx, replState->collectionUUID, replState->protocol);
+ }
+}
+
namespace {
NamespaceString getNsFromUUID(OperationContext* opCtx, const UUID& uuid) {
auto catalog = CollectionCatalog::get(opCtx);
diff --git a/src/mongo/db/index_builds_coordinator.h b/src/mongo/db/index_builds_coordinator.h
index 1f01a38b06b..425c511a1c2 100644
--- a/src/mongo/db/index_builds_coordinator.h
+++ b/src/mongo/db/index_builds_coordinator.h
@@ -276,6 +276,16 @@ public:
void abortAllIndexBuildsForInitialSync(OperationContext* opCtx, const std::string& reason);
/**
+ * Signals all index builds on non-internal databases to abort and waits until they are no
+ * longer running.
+ *
+ * Does not require holding locks.
+ *
+ * Does not stop new index builds from starting. Caller must make that guarantee.
+ */
+ void abortUserIndexBuildsForUserWriteBlocking(OperationContext* opCtx);
+
+ /**
* Aborts an index build by index build UUID. Returns when the index build thread exits.
*
* Returns true if the index build was aborted or the index build is already aborted.
diff --git a/src/mongo/db/index_builds_coordinator_mongod.cpp b/src/mongo/db/index_builds_coordinator_mongod.cpp
index 1b98e9c5f67..efa563e6b03 100644
--- a/src/mongo/db/index_builds_coordinator_mongod.cpp
+++ b/src/mongo/db/index_builds_coordinator_mongod.cpp
@@ -46,6 +46,7 @@
#include "mongo/db/operation_context.h"
#include "mongo/db/repl/tenant_migration_access_blocker_util.h"
#include "mongo/db/s/forwardable_operation_metadata.h"
+#include "mongo/db/s/global_user_write_block_state.h"
#include "mongo/db/s/operation_sharding_state.h"
#include "mongo/db/service_context.h"
#include "mongo/db/stats/resource_consumption_metrics.h"
@@ -163,6 +164,12 @@ IndexBuildsCoordinatorMongod::_startIndexBuild(OperationContext* opCtx,
const boost::optional<ResumeIndexInfo>& resumeInfo) {
const NamespaceStringOrUUID nssOrUuid{dbName, collectionUUID};
+ auto writeBlockState = GlobalUserWriteBlockState::get(opCtx);
+
+ invariant(!opCtx->lockState()->isRSTLExclusive(), buildUUID.toString());
+
+ const auto nss = CollectionCatalog::get(opCtx)->resolveNamespaceStringOrUUID(opCtx, nssOrUuid);
+
{
// Only operations originating from user connections need to wait while there are more than
// 'maxNumActiveUserIndexBuilds' index builds currently running.
@@ -185,10 +192,11 @@ IndexBuildsCoordinatorMongod::_startIndexBuild(OperationContext* opCtx,
replCoord->canAcceptWritesFor(opCtx, nssOrUuid));
}
- // The check here catches empty index builds and also allows us to stop index
+ // The checks here catch empty index builds and also allow us to stop index
// builds before waiting for throttling. It may race with the abort at the start
// of migration so we do check again later.
uassertStatusOK(tenant_migration_access_blocker::checkIfCanBuildIndex(opCtx, dbName));
+ uassertStatusOK(writeBlockState->checkIfIndexBuildAllowedToStart(opCtx, nss));
stdx::unique_lock<Latch> lk(_throttlingMutex);
bool messageLogged = false;
@@ -273,13 +281,21 @@ IndexBuildsCoordinatorMongod::_startIndexBuild(OperationContext* opCtx,
invariant(_getIndexBuild(buildUUID)));
return migrationStatus;
}
+
+ auto buildBlockedStatus = writeBlockState->checkIfIndexBuildAllowedToStart(opCtx, nss);
+ if (!buildBlockedStatus.isOK()) {
+ LOGV2(6511603,
+ "Aborted index build due to user index builds being blocked",
+ "error"_attr = buildBlockedStatus,
+ "buildUUID"_attr = buildUUID,
+ "collectionUUID"_attr = collectionUUID);
+ activeIndexBuilds.unregisterIndexBuild(&_indexBuildsManager,
+ invariant(_getIndexBuild(buildUUID)));
+ return buildBlockedStatus;
+ }
}
}
- invariant(!opCtx->lockState()->isRSTLExclusive(), buildUUID.toString());
-
- const auto nss = CollectionCatalog::get(opCtx)->resolveNamespaceStringOrUUID(opCtx, nssOrUuid);
-
auto& oss = OperationShardingState::get(opCtx);
// Task in thread pool should have similar CurOp representation to the caller so that it can be
diff --git a/src/mongo/db/s/SConscript b/src/mongo/db/s/SConscript
index 892a2fee04c..e1becf4ed5d 100644
--- a/src/mongo/db/s/SConscript
+++ b/src/mongo/db/s/SConscript
@@ -446,6 +446,7 @@ env.Library(
'$BUILD_DIR/mongo/db/commands/test_commands_enabled',
'$BUILD_DIR/mongo/db/commands/txn_cmd_request',
'$BUILD_DIR/mongo/db/fle_crud',
+ '$BUILD_DIR/mongo/db/index_builds_coordinator_interface',
'$BUILD_DIR/mongo/db/internal_transactions_feature_flag',
'$BUILD_DIR/mongo/db/multitenancy',
'$BUILD_DIR/mongo/db/repl/primary_only_service',
diff --git a/src/mongo/db/s/global_user_write_block_state.cpp b/src/mongo/db/s/global_user_write_block_state.cpp
index d53159a626d..a4a1ab933fd 100644
--- a/src/mongo/db/s/global_user_write_block_state.cpp
+++ b/src/mongo/db/s/global_user_write_block_state.cpp
@@ -88,4 +88,21 @@ void GlobalUserWriteBlockState::checkShardedDDLAllowedToStart(OperationContext*
WriteBlockBypass::get(opCtx).isWriteBlockBypassEnabled() || nss.isOnInternalDb());
}
+void GlobalUserWriteBlockState::enableUserIndexBuildBlocking(OperationContext* opCtx) {
+ _userIndexBuildsBlocked.store(true);
+}
+
+void GlobalUserWriteBlockState::disableUserIndexBuildBlocking(OperationContext* opCtx) {
+ _userIndexBuildsBlocked.store(false);
+}
+
+Status GlobalUserWriteBlockState::checkIfIndexBuildAllowedToStart(
+ OperationContext* opCtx, const NamespaceString& nss) const {
+ if (_userIndexBuildsBlocked.load() &&
+ !WriteBlockBypass::get(opCtx).isWriteBlockBypassEnabled() && !nss.isOnInternalDb()) {
+ return Status(ErrorCodes::UserWritesBlocked, "User writes blocked");
+ }
+ return Status::OK();
+}
+
} // namespace mongo
diff --git a/src/mongo/db/s/global_user_write_block_state.h b/src/mongo/db/s/global_user_write_block_state.h
index 66c44ea90aa..3ed998d7953 100644
--- a/src/mongo/db/s/global_user_write_block_state.h
+++ b/src/mongo/db/s/global_user_write_block_state.h
@@ -49,7 +49,7 @@ public:
/**
* Checks that user writes are allowed on the specified namespace. Callers must hold the
- * GlobalLock in any mode. Throws OperationFailed if user writes are disallowed.
+ * GlobalLock in any mode. Throws UserWritesBlocked if user writes are disallowed.
*/
void checkUserWritesAllowed(OperationContext* opCtx, const NamespaceString& nss) const;
@@ -66,14 +66,29 @@ public:
void disableUserShardedDDLBlocking(OperationContext* opCtx);
/**
- * Checks that new sharded DDL operations are allowed to start. Throws OperationFailed if
+ * Checks that new sharded DDL operations are allowed to start. Throws UserWritesBlocked if
* starting new sharded DDL operations is disallowed.
*/
void checkShardedDDLAllowedToStart(OperationContext* opCtx, const NamespaceString& nss) const;
+ /**
+ * Methods to enable/disable blocking new user index builds.
+ */
+ void enableUserIndexBuildBlocking(OperationContext* opCtx);
+ void disableUserIndexBuildBlocking(OperationContext* opCtx);
+
+ /**
+ * Checks that an index build is allowed to start on the specified namespace. Returns
+ * UserWritesBlocked if user index builds are disallowed, OK otherwise.
+ */
+ Status checkIfIndexBuildAllowedToStart(OperationContext* opCtx,
+ const NamespaceString& nss) const;
+
+
private:
bool _globalUserWritesBlocked{false};
AtomicWord<bool> _userShardedDDLBlocked{false};
+ AtomicWord<bool> _userIndexBuildsBlocked{false};
};
} // namespace mongo
diff --git a/src/mongo/db/s/shardsvr_set_user_write_block_mode_command.cpp b/src/mongo/db/s/shardsvr_set_user_write_block_mode_command.cpp
index 5ecfdc73bd6..0ab0b54e0a9 100644
--- a/src/mongo/db/s/shardsvr_set_user_write_block_mode_command.cpp
+++ b/src/mongo/db/s/shardsvr_set_user_write_block_mode_command.cpp
@@ -34,7 +34,9 @@
#include "mongo/db/auth/authorization_session.h"
#include "mongo/db/commands.h"
#include "mongo/db/dbdirectclient.h"
+#include "mongo/db/index_builds_coordinator.h"
#include "mongo/db/repl/repl_client_info.h"
+#include "mongo/db/s/global_user_write_block_state.h"
#include "mongo/db/s/sharding_ddl_coordinator.h"
#include "mongo/db/s/sharding_ddl_coordinator_service.h"
#include "mongo/db/s/user_writes_recoverable_critical_section_service.h"
@@ -119,13 +121,25 @@ public:
->waitForOngoingCoordinatorsToFinish(opCtx, shouldWaitPred);
}
break;
- case ShardsvrSetUserWriteBlockModePhaseEnum::kComplete:
+ case ShardsvrSetUserWriteBlockModePhaseEnum::kComplete: {
+ // The way we enable/disable user index build blocking is not
+ // concurrency-safe, so use a mutex to make this a critical section
+ stdx::lock_guard lock(_mutex);
+ auto writeBlockState = GlobalUserWriteBlockState::get(opCtx);
+ writeBlockState->enableUserIndexBuildBlocking(opCtx);
+ // Ensure that we eventually restore index build state.
+ ScopeGuard guard(
+ [&]() { writeBlockState->disableUserIndexBuildBlocking(opCtx); });
+ // Abort and wait for ongoing index builds to finish.
+ IndexBuildsCoordinator::get(opCtx)
+ ->abortUserIndexBuildsForUserWriteBlocking(opCtx);
+
UserWritesRecoverableCriticalSectionService::get(opCtx)
->promoteRecoverableCriticalSectionToBlockUserWrites(
opCtx,
UserWritesRecoverableCriticalSectionService::
kGlobalUserWritesNamespace);
- break;
+ } break;
default:
MONGO_UNREACHABLE;
}
@@ -166,6 +180,8 @@ public:
->isAuthorizedForActionsOnResource(ResourcePattern::forClusterResource(),
ActionType::internal));
}
+
+ Mutex _mutex = MONGO_MAKE_LATCH("ShardsvrSetUserWriteBlockCommand::_mutex");
};
std::string help() const override {
diff --git a/src/mongo/db/user_write_block_mode_op_observer.cpp b/src/mongo/db/user_write_block_mode_op_observer.cpp
index 2ef15e1ef20..9ebdd053f21 100644
--- a/src/mongo/db/user_write_block_mode_op_observer.cpp
+++ b/src/mongo/db/user_write_block_mode_op_observer.cpp
@@ -192,15 +192,6 @@ void UserWriteBlockModeOpObserver::onStartIndexBuild(OperationContext* opCtx,
_checkWriteAllowed(opCtx, nss);
}
-void UserWriteBlockModeOpObserver::onCommitIndexBuild(OperationContext* opCtx,
- const NamespaceString& nss,
- const UUID& collUUID,
- const UUID& indexBuildUUID,
- const std::vector<BSONObj>& indexes,
- bool fromMigrate) {
- _checkWriteAllowed(opCtx, nss);
-}
-
void UserWriteBlockModeOpObserver::onStartIndexBuildSinglePhase(OperationContext* opCtx,
const NamespaceString& nss) {
_checkWriteAllowed(opCtx, nss);
diff --git a/src/mongo/db/user_write_block_mode_op_observer.h b/src/mongo/db/user_write_block_mode_op_observer.h
index 5c3e21f12b9..330779d0cd6 100644
--- a/src/mongo/db/user_write_block_mode_op_observer.h
+++ b/src/mongo/db/user_write_block_mode_op_observer.h
@@ -79,13 +79,6 @@ public:
const std::vector<BSONObj>& indexes,
bool fromMigrate) final;
- void onCommitIndexBuild(OperationContext* opCtx,
- const NamespaceString& nss,
- const UUID& collUUID,
- const UUID& indexBuildUUID,
- const std::vector<BSONObj>& indexes,
- bool fromMigrate) final;
-
void onStartIndexBuildSinglePhase(OperationContext* opCtx, const NamespaceString& nss) final;
void onCreateCollection(OperationContext* opCtx,
@@ -155,6 +148,17 @@ public:
// Noop operations (don't perform any check).
+ // Index builds committing can be left unchecked since we kill any active index builds before
+ // enabling write blocking. This means any index build which gets to the commit phase while
+ // write blocking is active was started and hit the onStartIndexBuild hook with write blocking
+ // active, and thus must be allowed under user write blocking.
+ void onCommitIndexBuild(OperationContext* opCtx,
+ const NamespaceString& nss,
+ const UUID& collUUID,
+ const UUID& indexBuildUUID,
+ const std::vector<BSONObj>& indexes,
+ bool fromMigrate) final {}
+
// At the moment we are leaving the onAbortIndexBuilds as unchecked. This is because they can be
// called from both user and internal codepaths, and we don't want to risk throwing an assert
// for the internal paths.