summaryrefslogtreecommitdiff
diff options
context:
space:
mode:
authorSpencer T Brody <spencer@mongodb.com>2018-09-20 15:48:46 -0400
committerSpencer T Brody <spencer@mongodb.com>2018-09-28 13:15:09 -0400
commitf96903979ac329a760e7b6f1bb5d8695d3daf2a7 (patch)
tree53ee2d48499a84fa4ea280bff0eff8cc7aa6c645
parent9406af079a894bae80fbbec4703b04974bf84476 (diff)
downloadmongo-f96903979ac329a760e7b6f1bb5d8695d3daf2a7.tar.gz
SERVER-35870 Allow stepdown to work with prepared transactions
-rw-r--r--jstests/replsets/stepdown_with_prepared_transaction.js49
-rw-r--r--src/mongo/db/concurrency/lock_state.cpp32
-rw-r--r--src/mongo/db/concurrency/lock_state.h5
-rw-r--r--src/mongo/db/concurrency/locker.h7
-rw-r--r--src/mongo/db/concurrency/locker_noop.h4
-rw-r--r--src/mongo/db/concurrency/replication_lock_manager_manipulator.cpp10
-rw-r--r--src/mongo/db/kill_sessions_local.cpp11
-rw-r--r--src/mongo/db/kill_sessions_local.h10
-rw-r--r--src/mongo/db/repl/replication_coordinator_impl.cpp4
-rw-r--r--src/mongo/db/repl/replication_state_transition_lock_guard.cpp45
-rw-r--r--src/mongo/db/repl/replication_state_transition_lock_guard.h44
-rw-r--r--src/mongo/db/transaction_participant.cpp16
-rw-r--r--src/mongo/db/transaction_participant.h17
13 files changed, 228 insertions, 26 deletions
diff --git a/jstests/replsets/stepdown_with_prepared_transaction.js b/jstests/replsets/stepdown_with_prepared_transaction.js
new file mode 100644
index 00000000000..f1b738af4f9
--- /dev/null
+++ b/jstests/replsets/stepdown_with_prepared_transaction.js
@@ -0,0 +1,49 @@
+/**
+ * Tests that it is possible to step down a primary while there are transactions in prepare.
+ *
+ * @tags: [uses_transactions]
+ */
+(function() {
+ "use strict";
+ load("jstests/core/txns/libs/prepare_helpers.js");
+ load("jstests/replsets/rslib.js"); // For reconnect()
+
+ const replTest = new ReplSetTest({nodes: 1});
+ replTest.startSet();
+ replTest.initiate();
+
+ const priConn = replTest.getPrimary();
+ const lsid = UUID();
+ const dbName = jsTest.name();
+ const collName = jsTest.name();
+ const testDB = priConn.getDB(dbName);
+
+ assert.commandWorked(testDB.runCommand({create: collName, writeConcern: {w: "majority"}}));
+
+ jsTestLog("Starting basic transaction");
+
+ const session = priConn.startSession({causalConsistency: false});
+ const sessionDB = session.getDatabase(dbName);
+ session.startTransaction();
+
+ assert.commandWorked(sessionDB.getCollection(collName).insert({a: 1}));
+
+ jsTestLog("Putting transaction into prepare");
+ PrepareHelpers.prepareTransaction(session);
+
+ replTest.awaitReplication();
+
+ jsTestLog("Stepping down primary");
+
+ // Force stepdown primary since there are no secondaries.
+ assert.throws(function() {
+ priConn.adminCommand({replSetStepDown: 60, force: true});
+ });
+
+ reconnect(priConn);
+ assert(!assert.commandWorked(priConn.adminCommand({ismaster: 1})).isMaster);
+ assert.eq(ReplSetTest.State.SECONDARY,
+ assert.commandWorked(priConn.adminCommand({replSetGetStatus: 1})).myState);
+
+ replTest.stopSet(null /*signal*/, false /*forRestart*/, {skipValidation: true});
+})();
diff --git a/src/mongo/db/concurrency/lock_state.cpp b/src/mongo/db/concurrency/lock_state.cpp
index 576d476abae..a8c5cd22707 100644
--- a/src/mongo/db/concurrency/lock_state.cpp
+++ b/src/mongo/db/concurrency/lock_state.cpp
@@ -130,6 +130,10 @@ OperationContext::Decoration<LockManager::TemporaryResourceQueue*> globalResourc
} // namespace
bool LockerImpl::_shouldDelayUnlock(ResourceId resId, LockMode mode) const {
+ if (_prepareModeForLockYields) {
+ return false;
+ }
+
switch (resId.getType()) {
case RESOURCE_MUTEX:
return false;
@@ -587,9 +591,18 @@ boost::optional<Locker::LockerInfo> LockerImpl::getLockerInfo(
return std::move(lockerInfo);
}
+bool LockerImpl::saveLockStateAndUnlockForPrepare(Locker::LockSnapshot* stateOut) {
+ invariant(!_prepareModeForLockYields);
+ _prepareModeForLockYields = true;
+ ON_BLOCK_EXIT([&] { _prepareModeForLockYields = false; });
+ return saveLockStateAndUnlock(stateOut);
+}
+
bool LockerImpl::saveLockStateAndUnlock(Locker::LockSnapshot* stateOut) {
- // We shouldn't be saving and restoring lock state from inside a WriteUnitOfWork.
- invariant(!inAWriteUnitOfWork());
+ // We shouldn't be saving and restoring lock state from inside a WriteUnitOfWork, excepting the
+ // special behavior for saving/restoring locks for prepared transactions during repl state
+ // transitions.
+ invariant(!inAWriteUnitOfWork() || _prepareModeForLockYields);
// Clear out whatever is in stateOut.
stateOut->locks.clear();
@@ -646,8 +659,10 @@ bool LockerImpl::saveLockStateAndUnlock(Locker::LockSnapshot* stateOut) {
}
void LockerImpl::restoreLockState(OperationContext* opCtx, const Locker::LockSnapshot& state) {
- // We shouldn't be saving and restoring lock state from inside a WriteUnitOfWork.
- invariant(!inAWriteUnitOfWork());
+ // We shouldn't be saving and restoring lock state from inside a WriteUnitOfWork, excepting the
+ // special behavior for saving/restoring locks for prepared transactions during repl state
+ // transitions.
+ invariant(!inAWriteUnitOfWork() || _prepareModeForLockYields);
invariant(_modeForTicket == MODE_NONE);
std::vector<OneLock>::const_iterator it = state.locks.begin();
@@ -670,9 +685,14 @@ void LockerImpl::restoreLockStateWithTemporaryGlobalResource(
LockManager::TemporaryResourceQueue* tempGlobalResource) {
invariant(tempGlobalResource->getResourceId().getType() == ResourceType::RESOURCE_GLOBAL);
invariant(globalResourceShadow(opCtx) == nullptr);
+ invariant(!_prepareModeForLockYields);
globalResourceShadow(opCtx) = tempGlobalResource;
- ON_BLOCK_EXIT([&] { globalResourceShadow(opCtx) = nullptr; });
+ _prepareModeForLockYields = true;
+ ON_BLOCK_EXIT([&] {
+ globalResourceShadow(opCtx) = nullptr;
+ _prepareModeForLockYields = false;
+ });
restoreLockState(opCtx, state);
}
@@ -756,7 +776,7 @@ LockResult LockerImpl::lockBegin(OperationContext* opCtx, ResourceId resId, Lock
_notify.clear();
LockResult result{LockResult::LOCK_INVALID};
- if (resType == RESOURCE_GLOBAL && opCtx && globalResourceShadow(opCtx)) {
+ if (resId == resourceIdGlobal && opCtx && globalResourceShadow(opCtx)) {
// If we're trying to lock the global resource and we have a temporary global resource
// installed, use the temporary resource instead of letting the LockManager look up the
// true resource for the global lock.
diff --git a/src/mongo/db/concurrency/lock_state.h b/src/mongo/db/concurrency/lock_state.h
index 8eb9ec4053c..ca8c1591277 100644
--- a/src/mongo/db/concurrency/lock_state.h
+++ b/src/mongo/db/concurrency/lock_state.h
@@ -187,6 +187,7 @@ public:
const boost::optional<SingleThreadedLockStats> lockStatsBase) const final;
virtual bool saveLockStateAndUnlock(LockSnapshot* stateOut);
+ virtual bool saveLockStateAndUnlockForPrepare(LockSnapshot* stateOut);
virtual void restoreLockState(OperationContext* opCtx, const LockSnapshot& stateToRestore);
virtual void restoreLockState(const LockSnapshot& stateToRestore) {
@@ -339,6 +340,10 @@ private:
// If true, shared locks will participate in two-phase locking.
bool _sharedLocksShouldTwoPhaseLock = false;
+ // When true it means we are in the process of saving/restoring locks for prepared transactions.
+ // Two-phase locking gets disabled in this mode to allow yielding locks from within a WUOW.
+ bool _prepareModeForLockYields = false;
+
// If this is set, dictates the max number of milliseconds that we will wait for lock
// acquisition. Effectively resets lock acquisition deadlines to time out sooner. If set to 0,
// for example, lock attempts will time out immediately if the lock is not immediately
diff --git a/src/mongo/db/concurrency/locker.h b/src/mongo/db/concurrency/locker.h
index 07e45d052c5..be27a4a8c28 100644
--- a/src/mongo/db/concurrency/locker.h
+++ b/src/mongo/db/concurrency/locker.h
@@ -359,6 +359,12 @@ public:
virtual bool saveLockStateAndUnlock(LockSnapshot* stateOut) = 0;
/**
+ * Like saveLockStateAndUnlock but allows saving locks from within a WUOW. Used during
+ * replication state transitions for yielding locks held by prepared transactions.
+ */
+ virtual bool saveLockStateAndUnlockForPrepare(LockSnapshot* stateOut) = 0;
+
+ /**
* Re-locks all locks whose state was stored in 'stateToRestore'.
* @param opCtx An operation context that enables the restoration to be interrupted.
*/
@@ -371,6 +377,7 @@ public:
* restores the global locks into the TemporaryResourceQueue for the global resource that is
* provided. Locks on resources other than the global lock are restored to their true
* LockManager-owned resource objects.
+ * Also allows restoring locks from within a WUOW.
*/
virtual void restoreLockStateWithTemporaryGlobalResource(
OperationContext* opCtx,
diff --git a/src/mongo/db/concurrency/locker_noop.h b/src/mongo/db/concurrency/locker_noop.h
index 957627c8124..7167863f29a 100644
--- a/src/mongo/db/concurrency/locker_noop.h
+++ b/src/mongo/db/concurrency/locker_noop.h
@@ -171,6 +171,10 @@ public:
MONGO_UNREACHABLE;
}
+ virtual bool saveLockStateAndUnlockForPrepare(LockSnapshot* stateOut) {
+ MONGO_UNREACHABLE;
+ }
+
virtual void restoreLockState(OperationContext* opCtx, const LockSnapshot& stateToRestore) {
MONGO_UNREACHABLE;
}
diff --git a/src/mongo/db/concurrency/replication_lock_manager_manipulator.cpp b/src/mongo/db/concurrency/replication_lock_manager_manipulator.cpp
index 009bbbe3c20..9faa8636ad6 100644
--- a/src/mongo/db/concurrency/replication_lock_manager_manipulator.cpp
+++ b/src/mongo/db/concurrency/replication_lock_manager_manipulator.cpp
@@ -26,7 +26,7 @@
* it in the license file.
*/
-#define MONGO_LOG_DEFAULT_COMPONENT ::mongo::logger::LogComponent::kDefault
+#define MONGO_LOG_DEFAULT_COMPONENT ::mongo::logger::LogComponent::kReplication
#include "mongo/platform/basic.h"
@@ -44,6 +44,9 @@ void ReplicationLockManagerManipulator::lockUncontestedTemporaryGlobalResource(
invariant(request->recursiveCount == 1);
invariant(!request->partitioned);
invariant(tempGlobalResource->_lockHead.resourceId.getType() == ResourceType::RESOURCE_GLOBAL);
+ invariant(mode == MODE_IX,
+ str::stream() << "Locking temporary global resource must happen in MODE_IX, found: "
+ << mode);
request->mode = mode;
const auto lockResult = tempGlobalResource->_lockHead.newRequest(request);
@@ -81,7 +84,10 @@ void ReplicationLockManagerManipulator::replaceGlobalLocksWithLocksFromTemporary
for (LockRequest* it = tempGlobalLockHead->grantedList._front; it != nullptr;) {
LockRequest* next = it->next;
- invariant(it->mode == MODE_IX);
+ invariant(it->mode == MODE_IX,
+ str::stream() << "Expected granted requests from temporary global resource to be "
+ "in MODE_IX but found: "
+ << it->mode);
invariant(it->status == LockRequest::Status::STATUS_GRANTED);
invariant(it->lock == tempGlobalLockHead);
diff --git a/src/mongo/db/kill_sessions_local.cpp b/src/mongo/db/kill_sessions_local.cpp
index 0406a818284..e76b2b8db65 100644
--- a/src/mongo/db/kill_sessions_local.cpp
+++ b/src/mongo/db/kill_sessions_local.cpp
@@ -97,4 +97,15 @@ void killSessionsLocalShutdownAllTransactions(OperationContext* opCtx) {
});
}
+void killSessionsLocalAbortOrYieldAllTransactions(
+ OperationContext* opCtx, std::vector<std::pair<Locker*, Locker::LockSnapshot>>* yieldedLocks) {
+ SessionKiller::Matcher matcherAllSessions(
+ KillAllSessionsByPatternSet{makeKillAllSessionsByPattern(opCtx)});
+ SessionCatalog::get(opCtx)->scanSessions(
+ opCtx, matcherAllSessions, [yieldedLocks](OperationContext* opCtx, Session* session) {
+ TransactionParticipant::getFromNonCheckedOutSession(session)
+ ->abortOrYieldArbitraryTransaction(yieldedLocks);
+ });
+}
+
} // namespace mongo
diff --git a/src/mongo/db/kill_sessions_local.h b/src/mongo/db/kill_sessions_local.h
index ea8f442b325..e7609355a08 100644
--- a/src/mongo/db/kill_sessions_local.h
+++ b/src/mongo/db/kill_sessions_local.h
@@ -28,6 +28,9 @@
#pragma once
+#include <vector>
+
+#include "mongo/db/concurrency/d_concurrency.h"
#include "mongo/db/session_killer.h"
/**
@@ -58,4 +61,11 @@ void killAllExpiredTransactions(OperationContext* opCtx);
*/
void killSessionsLocalShutdownAllTransactions(OperationContext* opCtx);
+/**
+ * Run during replication state transitions. Aborts all unprepared transactions and causes all
+ * prepared transactions to yield their locks into 'yieldedLocks'.
+ */
+void killSessionsLocalAbortOrYieldAllTransactions(
+ OperationContext* opCtx, std::vector<std::pair<Locker*, Locker::LockSnapshot>>* yieldedLocks);
+
} // namespace mongo
diff --git a/src/mongo/db/repl/replication_coordinator_impl.cpp b/src/mongo/db/repl/replication_coordinator_impl.cpp
index 2c16966e3dd..557e27e6d9d 100644
--- a/src/mongo/db/repl/replication_coordinator_impl.cpp
+++ b/src/mongo/db/repl/replication_coordinator_impl.cpp
@@ -1762,7 +1762,7 @@ void ReplicationCoordinatorImpl::stepDown(OperationContext* opCtx,
// The stepdown attempt failed. We now release the global lock to allow secondaries
// to read the oplog, then wait until enough secondaries are caught up for us to
// finish stepdown.
- transitionGuard.releaseGlobalLock();
+ transitionGuard.releaseGlobalLockForStepdownAttempt();
invariant(!opCtx->lockState()->isLocked());
// Make sure we re-acquire the global lock before returning so that we're always holding
@@ -1779,7 +1779,7 @@ void ReplicationCoordinatorImpl::stepDown(OperationContext* opCtx,
// clean up a failed stepdown attempt, we might as well spend whatever time we need
// to acquire it now. For the same reason, we also disable lock acquisition
// interruption, to guarantee that we get the lock eventually.
- transitionGuard.reacquireGlobalLock();
+ transitionGuard.reacquireGlobalLockForStepdownAttempt();
invariant(opCtx->lockState()->isW());
lk.lock();
});
diff --git a/src/mongo/db/repl/replication_state_transition_lock_guard.cpp b/src/mongo/db/repl/replication_state_transition_lock_guard.cpp
index c47f1f77a76..910bc221333 100644
--- a/src/mongo/db/repl/replication_state_transition_lock_guard.cpp
+++ b/src/mongo/db/repl/replication_state_transition_lock_guard.cpp
@@ -33,6 +33,9 @@
#include "mongo/db/repl/replication_state_transition_lock_guard.h"
#include "mongo/db/kill_sessions_local.h"
+#include "mongo/db/session_catalog.h"
+#include "mongo/db/session_killer.h"
+#include "mongo/db/transaction_participant.h"
namespace mongo {
namespace repl {
@@ -40,22 +43,33 @@ namespace repl {
ReplicationStateTransitionLockGuard::ReplicationStateTransitionLockGuard(OperationContext* opCtx,
const Args& args)
: _opCtx(opCtx), _args(args) {
+
+ // First enqueue the request for the global X lock.
_globalLock.emplace(opCtx,
MODE_X,
args.lockDeadline,
Lock::InterruptBehavior::kThrow,
Lock::GlobalLock::EnqueueOnly());
+ // Next prevent any Sessions from being created or checked out.
+ _preventCheckingOutSessions.emplace(SessionCatalog::get(opCtx));
+
+ // If we're going to be killing all user operations do it before waiting for the global lock
+ // and for all Sessions to be checked in as killing all running user ops may make those things
+ // happen faster.
if (args.killUserOperations) {
ServiceContext* environment = opCtx->getServiceContext();
environment->killAllUserOperations(opCtx, ErrorCodes::InterruptedDueToStepDown);
-
- // Destroy all stashed transaction resources, in order to release locks.
- SessionKiller::Matcher matcherAllSessions(
- KillAllSessionsByPatternSet{makeKillAllSessionsByPattern(opCtx)});
- killSessionsLocalKillTransactions(opCtx, matcherAllSessions);
}
+ // Now wait for all Sessions to be checked in so we can iterate over all of them and abort
+ // any in-progress transactions and yield and gather the LockSnapshots for all prepared
+ // transactions.
+ _preventCheckingOutSessions->waitForAllSessionsToBeCheckedIn(opCtx);
+ killSessionsLocalAbortOrYieldAllTransactions(opCtx, &_yieldedLocks);
+
+ // Now that all transactions have either aborted or yielded their locks, we can wait for the
+ // global X lock to be taken successfully.
_globalLock->waitForLockUntil(args.lockDeadline);
uassert(ErrorCodes::ExceededTimeLimit,
"Could not acquire the global lock before the deadline",
@@ -64,14 +78,31 @@ ReplicationStateTransitionLockGuard::ReplicationStateTransitionLockGuard(Operati
ReplicationStateTransitionLockGuard::~ReplicationStateTransitionLockGuard() {
invariant(_globalLock->isLocked());
+
+ // Restore the locks for the prepared transactions, but put all requests for the global lock
+ // into a TemporaryResourceQueue for the global resource.
+ const ResourceId globalResId(RESOURCE_GLOBAL, ResourceId::SINGLETON_GLOBAL);
+ LockManager::TemporaryResourceQueue tempGlobalResource(globalResId);
+ for (auto&& pair : _yieldedLocks) {
+ auto locker = pair.first;
+ auto lockSnapshot = pair.second;
+
+ locker->restoreLockStateWithTemporaryGlobalResource(
+ _opCtx, lockSnapshot, &tempGlobalResource);
+ }
+
+ // Now atomically release the global X lock and restore the locks on the global resource from
+ // the TemporaryResourceQueue that was populated with the Global lock requests from the yielded
+ // locks from prepared transactions.
+ _opCtx->lockState()->replaceGlobalLockStateWithTemporaryGlobalResource(&tempGlobalResource);
}
-void ReplicationStateTransitionLockGuard::releaseGlobalLock() {
+void ReplicationStateTransitionLockGuard::releaseGlobalLockForStepdownAttempt() {
invariant(_globalLock->isLocked());
_globalLock.reset();
}
-void ReplicationStateTransitionLockGuard::reacquireGlobalLock() {
+void ReplicationStateTransitionLockGuard::reacquireGlobalLockForStepdownAttempt() {
invariant(!_globalLock);
UninterruptibleLockGuard noInterrupt(_opCtx->lockState());
diff --git a/src/mongo/db/repl/replication_state_transition_lock_guard.h b/src/mongo/db/repl/replication_state_transition_lock_guard.h
index 32fc41d3d0b..4094459c60a 100644
--- a/src/mongo/db/repl/replication_state_transition_lock_guard.h
+++ b/src/mongo/db/repl/replication_state_transition_lock_guard.h
@@ -32,6 +32,7 @@
#include "mongo/base/disallow_copying.h"
#include "mongo/db/concurrency/d_concurrency.h"
+#include "mongo/db/session_catalog.h"
#include "mongo/util/time_support.h"
namespace mongo {
@@ -40,7 +41,9 @@ namespace repl {
/**
* This object handles acquiring the global exclusive lock for replication state transitions, as
* well as any actions that need to happen in between enqueuing the global lock request and waiting
- * for it to be granted.
+ * for it to be granted. One of the main such actions is aborting all in-progress transactions and
+ * causing all prepared transaction to yield their locks during the transition and restoring them
+ * when the transition is complete.
*/
class ReplicationStateTransitionLockGuard {
MONGO_DISALLOW_COPYING(ReplicationStateTransitionLockGuard);
@@ -56,27 +59,50 @@ public:
};
/**
- * Acquires the global X lock and performs any other required actions accoriding to the Args
- * provided.
+ * Acquires the global X lock while yielding the locks held by any prepared transactions.
+ * Also performs any other actions required according to the Args provided.
*/
ReplicationStateTransitionLockGuard(OperationContext* opCtx, const Args& args);
+
+ /**
+ * Releases the global X lock and atomically restores the locks for prepared transactions that
+ * were yielded in the constructor.
+ */
~ReplicationStateTransitionLockGuard();
/**
- * Temporarily releases the global X lock. Must be followed by a call to reacquireGlobalLock().
+ * Temporarily releases the global X lock. Must be followed by a call to
+ * reacquireGlobalLockForStepdownAttempt().
*/
- void releaseGlobalLock();
+ void releaseGlobalLockForStepdownAttempt();
/**
- * Requires the global X lock after it was released via a call to releaseGlobalLock. Ignores
- * the configured 'lockDeadline' and instead waits forever for the lock to be acquired.
+ * Requires the global X lock after it was released via a call to
+ * releaseGlobalLockForStepdownAttempt(). Ignores the configured 'lockDeadline' and instead
+ * waits forever for the lock to be acquired.
*/
- void reacquireGlobalLock();
+ void reacquireGlobalLockForStepdownAttempt();
private:
+ // OperationContext of the thread driving the state transition.
OperationContext* const _opCtx;
+
+ // Args to configure what behaviors need to be taken while acquiring the global X lock for the
+ // state transition.
Args _args;
- boost::optional<Lock::GlobalLock> _globalLock = boost::none;
+
+ // The global X lock that this object is responsible for acquiring as part of the state
+ // transition.
+ boost::optional<Lock::GlobalLock> _globalLock;
+
+ // Used to prevent Sessions from being checked out, so that we can wait for all sessions to be
+ // checked in and iterate over all Sessions to get Sessions with prepared transactions to yield
+ // their locks.
+ boost::optional<SessionCatalog::PreventCheckingOutSessionsBlock> _preventCheckingOutSessions;
+
+ // Locks that were held by prepared transactions and were yielded in order to allow taking the
+ // global X lock.
+ std::vector<std::pair<Locker*, Locker::LockSnapshot>> _yieldedLocks;
};
} // namespace repl
diff --git a/src/mongo/db/transaction_participant.cpp b/src/mongo/db/transaction_participant.cpp
index a2c967afbe4..144dec8c0cc 100644
--- a/src/mongo/db/transaction_participant.cpp
+++ b/src/mongo/db/transaction_participant.cpp
@@ -868,6 +868,22 @@ void TransactionParticipant::abortActiveUnpreparedOrStashPreparedTransaction(
std::terminate();
}
+void TransactionParticipant::abortOrYieldArbitraryTransaction(
+ std::vector<std::pair<Locker*, Locker::LockSnapshot>>* yieldedLocks) {
+ stdx::lock_guard<stdx::mutex> lk(_mutex);
+
+ if (_txnState.isInProgress(lk)) {
+ _abortTransactionOnSession(lk);
+ return;
+ }
+
+ if (_txnState.isPrepared(lk)) {
+ Locker::LockSnapshot locks;
+ _txnResourceStash->locker()->saveLockStateAndUnlockForPrepare(&locks);
+ yieldedLocks->push_back(std::make_pair(_txnResourceStash->locker(), std::move(locks)));
+ }
+}
+
void TransactionParticipant::_abortActiveTransaction(stdx::unique_lock<stdx::mutex> lock,
OperationContext* opCtx,
TransactionState::StateSet expectedStates) {
diff --git a/src/mongo/db/transaction_participant.h b/src/mongo/db/transaction_participant.h
index 22b3d1485f1..9bdefa70dec 100644
--- a/src/mongo/db/transaction_participant.h
+++ b/src/mongo/db/transaction_participant.h
@@ -93,6 +93,13 @@ public:
}
/**
+ * Same as above but non-const.
+ */
+ Locker* locker() {
+ return _locker.get();
+ }
+
+ /**
* Releases stashed transaction state onto 'opCtx'. Must only be called once.
* Ephemerally holds the Client lock associated with opCtx.
*/
@@ -327,6 +334,16 @@ public:
*/
void abortActiveUnpreparedOrStashPreparedTransaction(OperationContext* opCtx);
+ /**
+ * If the transaction is not prepared, aborts the transaction and releases its resources.
+ * If the transaction is prepared, yields the transaction's locks and adds the Locker and
+ * LockSnapshot of the yielded locks to the end of the 'yieldedLocks' output vector.
+ *
+ * Not called with session checked out.
+ */
+ void abortOrYieldArbitraryTransaction(
+ std::vector<std::pair<Locker*, Locker::LockSnapshot>>* yieldedLocks);
+
void addMultikeyPathInfo(MultikeyPathInfo info) {
_multikeyPathInfo.push_back(std::move(info));
}