summaryrefslogtreecommitdiff
diff options
context:
space:
mode:
authorYu Jin Kang Park <yujin.kang@mongodb.com>2023-04-03 14:32:27 +0000
committerEvergreen Agent <no-reply@evergreen.mongodb.com>2023-04-03 16:32:29 +0000
commit8283c6988081633a991b86037ef7937af0f5b41e (patch)
treefd4aa4add9a7f96a39d8c8073b7a8d98bd253df4
parent3e855d6995eab89df943d24c80622ad793bc7319 (diff)
downloadmongo-8283c6988081633a991b86037ef7937af0f5b41e.tar.gz
SERVER-75083 Make index build step-up checks resilient to concurrent commit
-rw-r--r--jstests/noPassthrough/index_build_step_up_retry_while_commit.js71
-rw-r--r--src/mongo/db/index_builds_coordinator.cpp80
-rw-r--r--src/mongo/db/index_builds_coordinator.h4
-rw-r--r--src/mongo/db/index_builds_coordinator_mongod.cpp7
-rw-r--r--src/mongo/db/index_builds_coordinator_mongod.h2
-rw-r--r--src/mongo/db/repl_index_build_state.cpp72
-rw-r--r--src/mongo/db/repl_index_build_state.h11
-rw-r--r--src/mongo/embedded/index_builds_coordinator_embedded.cpp2
-rw-r--r--src/mongo/embedded/index_builds_coordinator_embedded.h2
9 files changed, 175 insertions, 76 deletions
diff --git a/jstests/noPassthrough/index_build_step_up_retry_while_commit.js b/jstests/noPassthrough/index_build_step_up_retry_while_commit.js
new file mode 100644
index 00000000000..0389ff7765b
--- /dev/null
+++ b/jstests/noPassthrough/index_build_step_up_retry_while_commit.js
@@ -0,0 +1,71 @@
+/**
+ * Tests that the step-up skipped record tracker check skips builds that have been concurrently
+ * committed.
+ *
+ * @tags: [
+ * requires_replication,
+ * ]
+ *
+ */
+load('jstests/noPassthrough/libs/index_build.js');
+load("jstests/libs/fail_point_util.js");
+load("jstests/replsets/rslib.js");
+
+(function() {
+
+"use strict";
+
+const dbName = "test";
+const collName = "coll";
+
+const rst = new ReplSetTest({nodes: 1});
+rst.startSet();
+rst.initiate();
+
+const primary = rst.getPrimary();
+
+const primaryDB = primary.getDB(dbName);
+const primaryColl = primaryDB[collName];
+
+jsTestLog("Do a document write");
+assert.commandWorked(primaryColl.insert({_id: 1, x: 1}, {"writeConcern": {"w": 1}}));
+
+// Clear the log.
+assert.commandWorked(primary.adminCommand({clearLog: 'global'}));
+
+// Enable fail point which makes the index build to hang before unregistering after a commit.
+const hangBeforeUnregisteringAfterCommit =
+ configureFailPoint(primary, 'hangBeforeUnregisteringAfterCommit');
+
+const indexThread = IndexBuildTest.startIndexBuild(
+ primary, primaryColl.getFullName(), {x: 1}, {}, ErrorCodes.InterruptedDueToReplStateChange);
+
+jsTestLog("Waiting for index build to hit failpoint");
+hangBeforeUnregisteringAfterCommit.wait();
+
+const stepDownThread = startParallelShell(() => {
+ jsTestLog("Make primary step down");
+ assert.commandWorked(db.adminCommand({"replSetStepDown": 60 * 60, "force": true}));
+}, primary.port);
+
+jsTestLog("Waiting for stepdown to complete");
+indexThread();
+stepDownThread();
+
+waitForState(primary, ReplSetTest.State.SECONDARY);
+// Allow the primary to be re-elected, and wait for it.
+assert.commandWorked(primary.adminCommand({replSetFreeze: 0}));
+rst.getPrimary();
+
+// Wait for the step-up check to be done.
+// "Finished performing asynchronous step-up checks on index builds"
+checkLog.containsJson(primary, 7508300);
+
+hangBeforeUnregisteringAfterCommit.off();
+
+IndexBuildTest.waitForIndexBuildToStop(primaryDB, primaryColl.getFullName(), "x_1");
+
+IndexBuildTest.assertIndexes(primaryColl, 2, ["_id_", "x_1"], []);
+
+rst.stopSet();
+})();
diff --git a/src/mongo/db/index_builds_coordinator.cpp b/src/mongo/db/index_builds_coordinator.cpp
index 120d847d76d..2212b975414 100644
--- a/src/mongo/db/index_builds_coordinator.cpp
+++ b/src/mongo/db/index_builds_coordinator.cpp
@@ -89,6 +89,7 @@ MONGO_FAIL_POINT_DEFINE(hangIndexBuildBeforeCommit);
MONGO_FAIL_POINT_DEFINE(hangBeforeBuildingIndex);
MONGO_FAIL_POINT_DEFINE(hangBeforeBuildingIndexSecond);
MONGO_FAIL_POINT_DEFINE(hangIndexBuildBeforeWaitingUntilMajorityOpTime);
+MONGO_FAIL_POINT_DEFINE(hangBeforeUnregisteringAfterCommit);
MONGO_FAIL_POINT_DEFINE(failSetUpResumeIndexBuild);
MONGO_FAIL_POINT_DEFINE(failIndexBuildWithError);
@@ -1595,23 +1596,6 @@ void IndexBuildsCoordinator::onStepUp(OperationContext* opCtx) {
// with FCV 4.2, and then upgraded FCV 4.4.
indexbuildentryhelpers::ensureIndexBuildEntriesNamespaceExists(opCtx);
- auto indexBuilds = _getIndexBuilds();
- auto onIndexBuild = [this, opCtx](const std::shared_ptr<ReplIndexBuildState>& replState) {
- if (IndexBuildProtocol::kTwoPhase != replState->protocol) {
- return;
- }
-
- if (!_signalIfCommitQuorumNotEnabled(opCtx, replState)) {
- // This reads from system.indexBuilds collection to see if commit quorum got satisfied.
- try {
- _signalIfCommitQuorumIsSatisfied(opCtx, replState);
- } catch (DBException& ex) {
- fassert(31440, ex.toStatus());
- }
- }
- };
- forEachIndexBuild(indexBuilds, "IndexBuildsCoordinator::onStepUp"_sd, onIndexBuild);
-
if (_stepUpThread.joinable()) {
// Under normal circumstances this should not result in a wait. The thread's opCtx should
// be interrupted on replication state change, or finish while being primary. If this
@@ -1639,27 +1623,45 @@ void IndexBuildsCoordinator::onStepUp(OperationContext* opCtx) {
void IndexBuildsCoordinator::_onStepUpAsyncTaskFn(OperationContext* opCtx) {
auto indexBuilds = _getIndexBuilds();
- const auto retrySkippedRecords = [this, opCtx](
- const std::shared_ptr<ReplIndexBuildState>& replState) {
- if (replState->protocol == IndexBuildProtocol::kTwoPhase) {
- try {
- // We don't need to check if we are primary because the opCtx is interrupted at
- // stepdown, so it is guaranteed that if taking the locks succeeds, we are primary.
- // Take an intent lock, the actual index build should keep running in parallel.
- const NamespaceStringOrUUID dbAndUUID(replState->dbName, replState->collectionUUID);
- AutoGetCollection autoColl(opCtx, dbAndUUID, MODE_IX);
+ const auto signalCommitQuorumAndRetrySkippedRecords =
+ [this, opCtx](const std::shared_ptr<ReplIndexBuildState>& replState) {
+ if (replState->protocol != IndexBuildProtocol::kTwoPhase) {
+ return;
+ }
+
+ // We don't need to check if we are primary because the opCtx is interrupted at
+ // stepdown, so it is guaranteed that if taking the locks succeeds, we are primary.
+ // Take an intent lock, the actual index build should keep running in parallel.
+ // This also prevents the concurrent index build from aborting or committing
+ // while we check if the commit quorum has to be signaled or check the skipped records.
+ const NamespaceStringOrUUID dbAndUUID(replState->dbName, replState->collectionUUID);
+ AutoGetCollection autoColl(opCtx, dbAndUUID, MODE_IX);
- // The index build might have committed or aborted while looping and not holding the
- // collection lock. Re-check it is still active after taking locks.
- auto indexBuilds = activeIndexBuilds.filterIndexBuilds(
- [&replState](const ReplIndexBuildState& filterState) {
- return filterState.buildUUID == replState->buildUUID;
- });
+ // The index build might have committed or aborted while looping and not holding the
+ // collection lock. Re-checking if it is still active after taking locks would not solve
+ // the issue, as build can still be registered as active, even if it is in an aborted or
+ // committed state.
+ if (replState->isAborting() || replState->isAborted() || replState->isCommitted()) {
+ return;
+ }
- if (indexBuilds.empty()) {
- return;
+ if (!_signalIfCommitQuorumNotEnabled(opCtx, replState)) {
+ // This reads from system.indexBuilds collection to see if commit quorum got
+ // satisfied.
+ try {
+ if (_signalIfCommitQuorumIsSatisfied(opCtx, replState)) {
+ // The index build has been signalled to commit. As retrying skipped records
+ // during step-up is done to prevent waiting until commit time, if the build
+ // has already been signalled to commit, we may skip the retry during
+ // step-up.
+ return;
+ }
+ } catch (DBException& ex) {
+ fassert(31440, ex.toStatus());
}
+ }
+ try {
// Only checks if key generation is valid, does not actually insert.
uassertStatusOK(_indexBuildsManager.retrySkippedRecords(
opCtx,
@@ -1679,18 +1681,19 @@ void IndexBuildsCoordinator::_onStepUpAsyncTaskFn(OperationContext* opCtx) {
abortIndexBuildByBuildUUID(
opCtx, replState->buildUUID, IndexBuildAction::kPrimaryAbort, status.reason());
}
- }
- };
+ };
try {
- forEachIndexBuild(
- indexBuilds, "IndexBuildsCoordinator::_onStepUpAsyncTaskFn"_sd, retrySkippedRecords);
+ forEachIndexBuild(indexBuilds,
+ "IndexBuildsCoordinator::_onStepUpAsyncTaskFn"_sd,
+ signalCommitQuorumAndRetrySkippedRecords);
} catch (const DBException& ex) {
LOGV2_DEBUG(7333100,
1,
"Step-up retry of skipped records for all index builds interrupted",
"exception"_attr = ex);
}
+ LOGV2(7508300, "Finished performing asynchronous step-up checks on index builds");
}
IndexBuilds IndexBuildsCoordinator::stopIndexBuildsForRollback(OperationContext* opCtx) {
@@ -2483,6 +2486,7 @@ void IndexBuildsCoordinator::_runIndexBuild(
// Ensure the index build is unregistered from the Coordinator and the Promise is set with
// the build's result so that callers are notified of the outcome.
if (status.isOK()) {
+ hangBeforeUnregisteringAfterCommit.pauseWhileSet();
// Unregister first so that when we fulfill the future, the build is not observed as active.
activeIndexBuilds.unregisterIndexBuild(&_indexBuildsManager, replState);
replState->sharedPromise.emplaceValue(replState->stats);
diff --git a/src/mongo/db/index_builds_coordinator.h b/src/mongo/db/index_builds_coordinator.h
index b006d92ce20..144a61c2bc8 100644
--- a/src/mongo/db/index_builds_coordinator.h
+++ b/src/mongo/db/index_builds_coordinator.h
@@ -793,8 +793,10 @@ protected:
* commit the index build if the number of voters have satisfied the commit quorum for that
* index build. Sets the ReplIndexBuildState::waitForNextAction promise value to be
* IndexBuildAction::kCommitQuorumSatisfied.
+ *
+ * Returns true when the index build has been signalled, false otherwise.
*/
- virtual void _signalIfCommitQuorumIsSatisfied(
+ virtual bool _signalIfCommitQuorumIsSatisfied(
OperationContext* opCtx, std::shared_ptr<ReplIndexBuildState> replState) = 0;
/**
diff --git a/src/mongo/db/index_builds_coordinator_mongod.cpp b/src/mongo/db/index_builds_coordinator_mongod.cpp
index e3ca806ecb0..48d582dafb8 100644
--- a/src/mongo/db/index_builds_coordinator_mongod.cpp
+++ b/src/mongo/db/index_builds_coordinator_mongod.cpp
@@ -609,7 +609,7 @@ void IndexBuildsCoordinatorMongod::_sendCommitQuorumSatisfiedSignal(
replState->setCommitQuorumSatisfied(opCtx);
}
-void IndexBuildsCoordinatorMongod::_signalIfCommitQuorumIsSatisfied(
+bool IndexBuildsCoordinatorMongod::_signalIfCommitQuorumIsSatisfied(
OperationContext* opCtx, std::shared_ptr<ReplIndexBuildState> replState) {
// Acquire the commitQuorumLk in shared mode to make sure commit quorum value did not change
@@ -625,17 +625,18 @@ void IndexBuildsCoordinatorMongod::_signalIfCommitQuorumIsSatisfied(
// This can occur when no vote got received and stepup tries to check if commit quorum is
// satisfied.
if (!voteMemberList)
- return;
+ return false;
bool commitQuorumSatisfied = repl::ReplicationCoordinator::get(opCtx)->isCommitQuorumSatisfied(
indexBuildEntry.getCommitQuorum(), voteMemberList.value());
if (!commitQuorumSatisfied)
- return;
+ return false;
LOGV2(
3856201, "Index build: commit quorum satisfied", "indexBuildEntry"_attr = indexBuildEntry);
_sendCommitQuorumSatisfiedSignal(opCtx, replState);
+ return true;
}
bool IndexBuildsCoordinatorMongod::_signalIfCommitQuorumNotEnabled(
diff --git a/src/mongo/db/index_builds_coordinator_mongod.h b/src/mongo/db/index_builds_coordinator_mongod.h
index 48087664a5e..a56900578c5 100644
--- a/src/mongo/db/index_builds_coordinator_mongod.h
+++ b/src/mongo/db/index_builds_coordinator_mongod.h
@@ -150,7 +150,7 @@ private:
std::shared_ptr<ReplIndexBuildState> replState);
- void _signalIfCommitQuorumIsSatisfied(OperationContext* opCtx,
+ bool _signalIfCommitQuorumIsSatisfied(OperationContext* opCtx,
std::shared_ptr<ReplIndexBuildState> replState) override;
diff --git a/src/mongo/db/repl_index_build_state.cpp b/src/mongo/db/repl_index_build_state.cpp
index a2014b8a7eb..ebc1d0e72f0 100644
--- a/src/mongo/db/repl_index_build_state.cpp
+++ b/src/mongo/db/repl_index_build_state.cpp
@@ -199,18 +199,18 @@ ReplIndexBuildState::ReplIndexBuildState(const UUID& indexBuildUUID,
}
void ReplIndexBuildState::onThreadScheduled(OperationContext* opCtx) {
- stdx::unique_lock<Latch> lk(_mutex);
+ stdx::lock_guard lk(_mutex);
_opId = opCtx->getOpID();
}
void ReplIndexBuildState::completeSetup() {
- stdx::unique_lock<Latch> lk(_mutex);
+ stdx::lock_guard lk(_mutex);
_indexBuildState.setState(IndexBuildState::kPostSetup, false /* skipCheck */);
_cleanUpRequired = true;
}
Status ReplIndexBuildState::tryStart(OperationContext* opCtx) {
- stdx::unique_lock<Latch> lk(_mutex);
+ stdx::lock_guard lk(_mutex);
// The index build might have been aborted/interrupted before reaching this point. Trying to
// transtion to kInProgress would be an error.
auto interruptCheck = opCtx->checkForInterruptNoAssert();
@@ -224,14 +224,14 @@ void ReplIndexBuildState::commit(OperationContext* opCtx) {
auto skipCheck = _shouldSkipIndexBuildStateTransitionCheck(opCtx);
opCtx->recoveryUnit()->onCommit(
[this, skipCheck](OperationContext*, boost::optional<Timestamp>) {
- stdx::unique_lock<Latch> lk(_mutex);
+ stdx::lock_guard lk(_mutex);
_indexBuildState.setState(IndexBuildState::kCommitted, skipCheck);
});
}
void ReplIndexBuildState::requestAbortFromPrimary(const Status& abortStatus) {
invariant(protocol == IndexBuildProtocol::kTwoPhase);
- stdx::unique_lock<Latch> lk(_mutex);
+ stdx::lock_guard lk(_mutex);
// It is possible that a 'commitIndexBuild' oplog entry is applied while the index builder is
// transitioning to an abort, or even to have been in a state where the oplog applier is already
@@ -250,12 +250,12 @@ void ReplIndexBuildState::requestAbortFromPrimary(const Status& abortStatus) {
}
Timestamp ReplIndexBuildState::getCommitTimestamp() const {
- stdx::unique_lock<Latch> lk(_mutex);
+ stdx::lock_guard lk(_mutex);
return _indexBuildState.getTimestamp().value_or(Timestamp());
}
void ReplIndexBuildState::onOplogCommit(bool isPrimary) const {
- stdx::unique_lock<Latch> lk(_mutex);
+ stdx::lock_guard lk(_mutex);
invariant(!isPrimary && _indexBuildState.isApplyingCommitOplogEntry(),
str::stream() << "Index build: " << buildUUID
<< ", index build state: " << _indexBuildState.toString());
@@ -263,13 +263,13 @@ void ReplIndexBuildState::onOplogCommit(bool isPrimary) const {
void ReplIndexBuildState::abortSelf(OperationContext* opCtx) {
auto skipCheck = _shouldSkipIndexBuildStateTransitionCheck(opCtx);
- stdx::unique_lock<Latch> lk(_mutex);
+ stdx::lock_guard lk(_mutex);
_indexBuildState.setState(IndexBuildState::kAborted, skipCheck);
}
void ReplIndexBuildState::abortForShutdown(OperationContext* opCtx) {
// Promise should be set at least once before it's getting destroyed.
- stdx::unique_lock<Latch> lk(_mutex);
+ stdx::lock_guard lk(_mutex);
if (!_waitForNextAction->getFuture().isReady()) {
_waitForNextAction->emplaceValue(IndexBuildAction::kNoAction);
}
@@ -283,7 +283,7 @@ void ReplIndexBuildState::onOplogAbort(OperationContext* opCtx, const NamespaceS
!replCoord->getSettings().shouldRecoverFromOplogAsStandalone();
invariant(!isPrimary, str::stream() << "Index build: " << buildUUID);
- stdx::unique_lock<Latch> lk(_mutex);
+ stdx::lock_guard lk(_mutex);
invariant(_indexBuildState.isAborted(),
str::stream() << "Index build: " << buildUUID
<< ", index build state: " << _indexBuildState.toString());
@@ -298,23 +298,33 @@ void ReplIndexBuildState::onOplogAbort(OperationContext* opCtx, const NamespaceS
}
bool ReplIndexBuildState::isAbortCleanUpRequired() const {
- stdx::unique_lock<Latch> lk(_mutex);
+ stdx::lock_guard lk(_mutex);
// Cleanup is required for external aborts if setup stage completed at some point in the past.
return _cleanUpRequired;
}
bool ReplIndexBuildState::isAborted() const {
- stdx::unique_lock<Latch> lk(_mutex);
+ stdx::lock_guard lk(_mutex);
return _indexBuildState.isAborted();
}
+bool ReplIndexBuildState::isAborting() const {
+ stdx::lock_guard lk(_mutex);
+ return _indexBuildState.isAwaitingPrimaryAbort() || _indexBuildState.isForceSelfAbort();
+}
+
+bool ReplIndexBuildState::isCommitted() const {
+ stdx::lock_guard lk(_mutex);
+ return _indexBuildState.isCommitted();
+}
+
bool ReplIndexBuildState::isSettingUp() const {
- stdx::unique_lock<Latch> lk(_mutex);
+ stdx::lock_guard lk(_mutex);
return _indexBuildState.isSettingUp();
}
std::string ReplIndexBuildState::getAbortReason() const {
- stdx::unique_lock<Latch> lk(_mutex);
+ stdx::lock_guard lk(_mutex);
invariant(_indexBuildState.isAborted() || _indexBuildState.isAwaitingPrimaryAbort(),
str::stream() << "Index build: " << buildUUID
<< ", index build state: " << _indexBuildState.toString());
@@ -324,12 +334,12 @@ std::string ReplIndexBuildState::getAbortReason() const {
}
Status ReplIndexBuildState::getAbortStatus() const {
- stdx::unique_lock<Latch> lk(_mutex);
+ stdx::lock_guard lk(_mutex);
return _indexBuildState.getAbortStatus();
}
void ReplIndexBuildState::setCommitQuorumSatisfied(OperationContext* opCtx) {
- stdx::unique_lock<Latch> lk(_mutex);
+ stdx::lock_guard lk(_mutex);
if (!_waitForNextAction->getFuture().isReady()) {
_setSignalAndCancelVoteRequestCbkIfActive(
lk, opCtx, IndexBuildAction::kCommitQuorumSatisfied);
@@ -352,7 +362,7 @@ void ReplIndexBuildState::setCommitQuorumSatisfied(OperationContext* opCtx) {
}
void ReplIndexBuildState::setSinglePhaseCommit(OperationContext* opCtx) {
- stdx::unique_lock<Latch> lk(_mutex);
+ stdx::lock_guard lk(_mutex);
if (_waitForNextAction->getFuture().isReady()) {
// If the signal action has been set, it should only be because a concurrent operation
// already aborted the index build.
@@ -369,7 +379,7 @@ void ReplIndexBuildState::setSinglePhaseCommit(OperationContext* opCtx) {
}
bool ReplIndexBuildState::tryCommit(OperationContext* opCtx) {
- stdx::unique_lock<Latch> lk(_mutex);
+ stdx::lock_guard lk(_mutex);
if (_indexBuildState.isSettingUp() || _indexBuildState.isPostSetup()) {
// It's possible that the index build thread has not reached the point where it can be
// committed yet.
@@ -410,7 +420,7 @@ bool ReplIndexBuildState::tryCommit(OperationContext* opCtx) {
ReplIndexBuildState::TryAbortResult ReplIndexBuildState::tryAbort(OperationContext* opCtx,
IndexBuildAction signalAction,
std::string reason) {
- stdx::unique_lock<Latch> lk(_mutex);
+ stdx::lock_guard lk(_mutex);
// Wait until the build is done setting up. This indicates that all required state is
// initialized to attempt an abort.
if (_indexBuildState.isSettingUp()) {
@@ -492,7 +502,7 @@ ReplIndexBuildState::TryAbortResult ReplIndexBuildState::tryAbort(OperationConte
}
bool ReplIndexBuildState::forceSelfAbort(OperationContext* opCtx, const Status& error) {
- stdx::unique_lock<Latch> lk(_mutex);
+ stdx::lock_guard lk(_mutex);
if (_indexBuildState.isSettingUp() || _indexBuildState.isAborted() ||
_indexBuildState.isCommitted() || _indexBuildState.isAwaitingPrimaryAbort() ||
_indexBuildState.isApplyingCommitOplogEntry()) {
@@ -523,7 +533,7 @@ bool ReplIndexBuildState::forceSelfAbort(OperationContext* opCtx, const Status&
void ReplIndexBuildState::onVoteRequestScheduled(OperationContext* opCtx,
executor::TaskExecutor::CallbackHandle handle) {
- stdx::unique_lock<Latch> lk(_mutex);
+ stdx::lock_guard lk(_mutex);
if (_waitForNextAction->getFuture().isReady()) {
auto replCoord = repl::ReplicationCoordinator::get(opCtx);
replCoord->cancelCbkHandle(handle);
@@ -534,23 +544,23 @@ void ReplIndexBuildState::onVoteRequestScheduled(OperationContext* opCtx,
}
void ReplIndexBuildState::clearVoteRequestCbk() {
- stdx::unique_lock<Latch> lk(_mutex);
+ stdx::lock_guard lk(_mutex);
_voteCmdCbkHandle = executor::TaskExecutor::CallbackHandle();
}
void ReplIndexBuildState::resetNextActionPromise() {
- stdx::unique_lock<Latch> lk(_mutex);
+ stdx::lock_guard lk(_mutex);
_waitForNextAction = std::make_unique<SharedPromise<IndexBuildAction>>();
}
SharedSemiFuture<IndexBuildAction> ReplIndexBuildState::getNextActionFuture() const {
- stdx::unique_lock<Latch> lk(_mutex);
+ stdx::lock_guard lk(_mutex);
invariant(_waitForNextAction, str::stream() << buildUUID);
return _waitForNextAction->getFuture();
}
boost::optional<IndexBuildAction> ReplIndexBuildState::getNextActionNoWait() const {
- stdx::unique_lock<Latch> lk(_mutex);
+ stdx::lock_guard lk(_mutex);
auto future = _waitForNextAction->getFuture();
if (!future.isReady()) {
return boost::none;
@@ -570,7 +580,7 @@ Status ReplIndexBuildState::onConflictWithNewIndexBuild(const ReplIndexBuildStat
IndexBuildState existingIndexBuildState;
{
// We have to lock the mutex in order to read the committed/aborted state.
- stdx::unique_lock<Latch> lk(_mutex);
+ stdx::lock_guard lk(_mutex);
existingIndexBuildState = _indexBuildState;
}
ss << " index build state: " << existingIndexBuildState.toString();
@@ -598,27 +608,27 @@ Status ReplIndexBuildState::onConflictWithNewIndexBuild(const ReplIndexBuildStat
}
bool ReplIndexBuildState::isResumable() const {
- stdx::unique_lock<Latch> lk(_mutex);
+ stdx::lock_guard lk(_mutex);
return !_lastOpTimeBeforeInterceptors.isNull();
}
repl::OpTime ReplIndexBuildState::getLastOpTimeBeforeInterceptors() const {
- stdx::unique_lock<Latch> lk(_mutex);
+ stdx::lock_guard lk(_mutex);
return _lastOpTimeBeforeInterceptors;
}
void ReplIndexBuildState::setLastOpTimeBeforeInterceptors(repl::OpTime opTime) {
- stdx::unique_lock<Latch> lk(_mutex);
+ stdx::lock_guard lk(_mutex);
_lastOpTimeBeforeInterceptors = std::move(opTime);
}
void ReplIndexBuildState::clearLastOpTimeBeforeInterceptors() {
- stdx::unique_lock<Latch> lk(_mutex);
+ stdx::lock_guard lk(_mutex);
_lastOpTimeBeforeInterceptors = {};
}
void ReplIndexBuildState::appendBuildInfo(BSONObjBuilder* builder) const {
- stdx::unique_lock<Latch> lk(_mutex);
+ stdx::lock_guard lk(_mutex);
// This allows listIndexes callers to identify how to kill the index build.
// Previously, users have to locate the index build in the currentOp command output
diff --git a/src/mongo/db/repl_index_build_state.h b/src/mongo/db/repl_index_build_state.h
index 04efcd49730..eff7b0c976e 100644
--- a/src/mongo/db/repl_index_build_state.h
+++ b/src/mongo/db/repl_index_build_state.h
@@ -29,6 +29,7 @@
#pragma once
+#include "mongo/stdx/mutex.h"
#include <algorithm>
#include <list>
#include <string>
@@ -352,6 +353,16 @@ public:
bool isAborted() const;
/**
+ * Returns true if this index is in the process of aborting.
+ */
+ bool isAborting() const;
+
+ /**
+ * Returns true if this index build has been committed.
+ */
+ bool isCommitted() const;
+
+ /**
* Returns true if this index build is being set up.
*/
bool isSettingUp() const;
diff --git a/src/mongo/embedded/index_builds_coordinator_embedded.cpp b/src/mongo/embedded/index_builds_coordinator_embedded.cpp
index 0204e80cb99..5dd97bff40a 100644
--- a/src/mongo/embedded/index_builds_coordinator_embedded.cpp
+++ b/src/mongo/embedded/index_builds_coordinator_embedded.cpp
@@ -118,7 +118,7 @@ Status IndexBuildsCoordinatorEmbedded::setCommitQuorum(OperationContext* opCtx,
MONGO_UNREACHABLE;
}
-void IndexBuildsCoordinatorEmbedded::_signalIfCommitQuorumIsSatisfied(
+bool IndexBuildsCoordinatorEmbedded::_signalIfCommitQuorumIsSatisfied(
OperationContext* opCtx, std::shared_ptr<ReplIndexBuildState> replState) {
MONGO_UNREACHABLE;
}
diff --git a/src/mongo/embedded/index_builds_coordinator_embedded.h b/src/mongo/embedded/index_builds_coordinator_embedded.h
index 959d268c349..11e77083e11 100644
--- a/src/mongo/embedded/index_builds_coordinator_embedded.h
+++ b/src/mongo/embedded/index_builds_coordinator_embedded.h
@@ -89,7 +89,7 @@ public:
const CommitQuorumOptions& newCommitQuorum) override;
private:
- void _signalIfCommitQuorumIsSatisfied(OperationContext* opCtx,
+ bool _signalIfCommitQuorumIsSatisfied(OperationContext* opCtx,
std::shared_ptr<ReplIndexBuildState> replState) override;
bool _signalIfCommitQuorumNotEnabled(OperationContext* opCtx,