summaryrefslogtreecommitdiff
path: root/src
diff options
context:
space:
mode:
authorKaloian Manassiev <kaloian.manassiev@mongodb.com>2018-10-24 11:43:57 +0200
committerKaloian Manassiev <kaloian.manassiev@mongodb.com>2018-10-25 18:56:12 +0200
commit58cfddf1bdc0aca99a9dcb5666279349621fd156 (patch)
tree3df5cf832bf81f87cb4ce2a1bfceab4328cccb7b /src
parent1b897361cdf0119bec00400e4ca287809e87c79b (diff)
downloadmongo-58cfddf1bdc0aca99a9dcb5666279349621fd156.tar.gz
SERVER-37711 Revert commits related to lock yielding for prepared transactions on step down
This change reverts the following commits: * SERVER-35870 Allow stepdown to work with prepared transactions (f96903979ac329a760e7b6f1bb5d8695d3daf2a7) * SERVER-36913 Add functionality to LockManager for repl state transitions with prepared transactions (e65ff57e108ed69c46cc0b0ccbdd675663de2469) * SERVER-35870 Allow more than one thread to block Session checkout at a time (9406af079a894bae80fbbec4703b04974bf84476) * SERVER-35870 Add functionality to prevent Session checkouts & wait for all Sessions to be checked in (c6d90316d6b694e12426274c713a4a078e004fc5)
Diffstat (limited to 'src')
-rw-r--r--src/mongo/db/concurrency/SConscript1
-rw-r--r--src/mongo/db/concurrency/lock_manager.cpp317
-rw-r--r--src/mongo/db/concurrency/lock_manager.h150
-rw-r--r--src/mongo/db/concurrency/lock_state.cpp95
-rw-r--r--src/mongo/db/concurrency/lock_state.h13
-rw-r--r--src/mongo/db/concurrency/lock_state_test.cpp99
-rw-r--r--src/mongo/db/concurrency/locker.h27
-rw-r--r--src/mongo/db/concurrency/locker_noop.h16
-rw-r--r--src/mongo/db/concurrency/replication_lock_manager_manipulator.cpp113
-rw-r--r--src/mongo/db/concurrency/replication_lock_manager_manipulator.h73
-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/session_catalog.cpp39
-rw-r--r--src/mongo/db/session_catalog.h39
-rw-r--r--src/mongo/db/session_catalog_test.cpp103
-rw-r--r--src/mongo/db/transaction_participant.cpp16
-rw-r--r--src/mongo/db/transaction_participant.h17
20 files changed, 230 insertions, 1002 deletions
diff --git a/src/mongo/db/concurrency/SConscript b/src/mongo/db/concurrency/SConscript
index 9ddfa508f36..0ca5123d60b 100644
--- a/src/mongo/db/concurrency/SConscript
+++ b/src/mongo/db/concurrency/SConscript
@@ -38,7 +38,6 @@ env.Library(
'lock_manager.cpp',
'lock_state.cpp',
'lock_stats.cpp',
- 'replication_lock_manager_manipulator.cpp',
],
LIBDEPS=[
'$BUILD_DIR/mongo/base',
diff --git a/src/mongo/db/concurrency/lock_manager.cpp b/src/mongo/db/concurrency/lock_manager.cpp
index 981a46e7465..11f2eac150c 100644
--- a/src/mongo/db/concurrency/lock_manager.cpp
+++ b/src/mongo/db/concurrency/lock_manager.cpp
@@ -131,6 +131,208 @@ MONGO_STATIC_ASSERT((sizeof(LockRequestStatusNames) / sizeof(LockRequestStatusNa
} // namespace
/**
+ * There is one of these objects for each resource that has a lock request. Empty objects (i.e.
+ * LockHead with no requests) are allowed to exist on the lock manager's hash table.
+ *
+ * The memory and lifetime is controlled entirely by the LockManager class.
+ *
+ * Not thread-safe and should only be accessed under the LockManager's bucket lock. Must be locked
+ * before locking a partition, not after.
+ */
+struct LockHead {
+
+ /**
+ * Used for initialization of a LockHead, which might have been retrieved from cache and also in
+ * order to keep the LockHead structure a POD.
+ */
+ void initNew(ResourceId resId) {
+ resourceId = resId;
+
+ grantedList.reset();
+ memset(grantedCounts, 0, sizeof(grantedCounts));
+ grantedModes = 0;
+
+ conflictList.reset();
+ memset(conflictCounts, 0, sizeof(conflictCounts));
+ conflictModes = 0;
+
+ conversionsCount = 0;
+ compatibleFirstCount = 0;
+ }
+
+ /**
+ * True iff there may be partitions with granted requests for this resource.
+ */
+ bool partitioned() const {
+ return !partitions.empty();
+ }
+
+ /**
+ * Locates the request corresponding to the particular locker or returns nullptr. Must be called
+ * with the bucket holding this lock head locked.
+ */
+ LockRequest* findRequest(LockerId lockerId) const {
+ // Check the granted queue first
+ for (LockRequest* it = grantedList._front; it != nullptr; it = it->next) {
+ if (it->locker->getId() == lockerId) {
+ return it;
+ }
+ }
+
+ // Check the conflict queue second
+ for (LockRequest* it = conflictList._front; it != nullptr; it = it->next) {
+ if (it->locker->getId() == lockerId) {
+ return it;
+ }
+ }
+
+ return nullptr;
+ }
+
+ /**
+ * Finish creation of request and put it on the LockHead's conflict or granted queues. Returns
+ * LOCK_WAITING for conflict case and LOCK_OK otherwise.
+ */
+ LockResult newRequest(LockRequest* request) {
+ invariant(!request->partitionedLock);
+ request->lock = this;
+
+ // We cannot set request->partitioned to false, as this might be a migration, in which case
+ // access to that field is not protected. The 'partitioned' member instead indicates if a
+ // request was initially partitioned.
+
+ // New lock request. Queue after all granted modes and after any already requested
+ // conflicting modes
+ if (conflicts(request->mode, grantedModes) ||
+ (!compatibleFirstCount && conflicts(request->mode, conflictModes))) {
+ request->status = LockRequest::STATUS_WAITING;
+
+ // Put it on the conflict queue. Conflicts are granted front to back.
+ if (request->enqueueAtFront) {
+ conflictList.push_front(request);
+ } else {
+ conflictList.push_back(request);
+ }
+
+ incConflictModeCount(request->mode);
+
+ return LOCK_WAITING;
+ }
+
+ // No conflict, new request
+ request->status = LockRequest::STATUS_GRANTED;
+
+ grantedList.push_back(request);
+ incGrantedModeCount(request->mode);
+
+ if (request->compatibleFirst) {
+ compatibleFirstCount++;
+ }
+
+ return LOCK_OK;
+ }
+
+ /**
+ * Lock each partitioned LockHead in turn, and move any (granted) intent mode requests for
+ * lock->resourceId to lock, which must itself already be locked.
+ */
+ void migratePartitionedLockHeads();
+
+ // Methods to maintain the granted queue
+ void incGrantedModeCount(LockMode mode) {
+ invariant(grantedCounts[mode] >= 0);
+ if (++grantedCounts[mode] == 1) {
+ invariant((grantedModes & modeMask(mode)) == 0);
+ grantedModes |= modeMask(mode);
+ }
+ }
+
+ void decGrantedModeCount(LockMode mode) {
+ invariant(grantedCounts[mode] >= 1);
+ if (--grantedCounts[mode] == 0) {
+ invariant((grantedModes & modeMask(mode)) == modeMask(mode));
+ grantedModes &= ~modeMask(mode);
+ }
+ }
+
+ // Methods to maintain the conflict queue
+ void incConflictModeCount(LockMode mode) {
+ invariant(conflictCounts[mode] >= 0);
+ if (++conflictCounts[mode] == 1) {
+ invariant((conflictModes & modeMask(mode)) == 0);
+ conflictModes |= modeMask(mode);
+ }
+ }
+
+ void decConflictModeCount(LockMode mode) {
+ invariant(conflictCounts[mode] >= 1);
+ if (--conflictCounts[mode] == 0) {
+ invariant((conflictModes & modeMask(mode)) == modeMask(mode));
+ conflictModes &= ~modeMask(mode);
+ }
+ }
+
+ // Id of the resource which is protected by this lock. Initialized at construction time and does
+ // not change.
+ ResourceId resourceId;
+
+ //
+ // Granted queue
+ //
+
+ // Doubly-linked list of requests, which have been granted. Newly granted requests go to
+ // the end of the queue. Conversion requests are granted from the beginning forward.
+ LockRequestList grantedList;
+
+ // Counts the grants and conversion counts for each of the supported lock modes. These
+ // counts should exactly match the aggregated modes on the granted list.
+ uint32_t grantedCounts[LockModesCount];
+
+ // Bit-mask of the granted + converting modes on the granted queue. Maintained in lock-step
+ // with the grantedCounts array.
+ uint32_t grantedModes;
+
+ //
+ // Conflict queue
+ //
+
+ // Doubly-linked list of requests, which have not been granted yet because they conflict
+ // with the set of granted modes. Requests are queued at the end of the queue and are
+ // granted from the beginning forward, which gives these locks FIFO ordering. Exceptions
+ // are high-priority locks, such as the MMAP V1 flush lock.
+ LockRequestList conflictList;
+
+ // Counts the conflicting requests for each of the lock modes. These counts should exactly
+ // match the aggregated modes on the conflicts list.
+ uint32_t conflictCounts[LockModesCount];
+
+ // Bit-mask of the conflict modes on the conflict queue. Maintained in lock-step with the
+ // conflictCounts array.
+ uint32_t conflictModes;
+
+ // References partitions that may have PartitionedLockHeads for this LockHead.
+ // Non-empty implies the lock has no conflicts and only has intent modes as grantedModes.
+ // TODO: Remove this vector and make LockHead a POD
+ std::vector<LockManager::Partition*> partitions;
+
+ //
+ // Conversion
+ //
+
+ // Counts the number of requests on the granted queue, which have requested any kind of
+ // conflicting conversion and are blocked (i.e. all requests which are currently
+ // STATUS_CONVERTING). This is an optimization for unlocking in that we do not need to
+ // check the granted queue for requests in STATUS_CONVERTING if this count is zero. This
+ // saves cycles in the regular case and only burdens the less-frequent lock upgrade case.
+ uint32_t conversionsCount;
+
+ // Counts the number of requests on the granted queue, which have requested that the policy
+ // be switched to compatible-first. As long as this value is > 0, the policy will stay
+ // compatible-first.
+ uint32_t compatibleFirstCount;
+};
+
+/**
* The PartitionedLockHead allows optimizing the case where requests overwhelmingly use
* the intent lock modes MODE_IS and MODE_IX, which are compatible with each other.
* Having to use a single LockHead causes contention where none would be needed.
@@ -170,85 +372,6 @@ struct PartitionedLockHead {
LockRequestList grantedList;
};
-//
-// LockHead
-//
-void LockHead::initNew(ResourceId resId) {
- resourceId = resId;
-
- grantedList.reset();
- memset(grantedCounts, 0, sizeof(grantedCounts));
- grantedModes = 0;
-
- conflictList.reset();
- memset(conflictCounts, 0, sizeof(conflictCounts));
- conflictModes = 0;
-
- conversionsCount = 0;
- compatibleFirstCount = 0;
-}
-
-bool LockHead::partitioned() const {
- return !partitions.empty();
-}
-
-LockRequest* LockHead::findRequest(LockerId lockerId) const {
- // Check the granted queue first
- for (LockRequest* it = grantedList._front; it != nullptr; it = it->next) {
- if (it->locker->getId() == lockerId) {
- return it;
- }
- }
-
- // Check the conflict queue second
- for (LockRequest* it = conflictList._front; it != nullptr; it = it->next) {
- if (it->locker->getId() == lockerId) {
- return it;
- }
- }
-
- return nullptr;
-}
-
-LockResult LockHead::newRequest(LockRequest* request) {
- invariant(!request->partitionedLock);
- request->lock = this;
-
- // We cannot set request->partitioned to false, as this might be a migration, in which case
- // access to that field is not protected. The 'partitioned' member instead indicates if a
- // request was initially partitioned.
-
- // New lock request. Queue after all granted modes and after any already requested
- // conflicting modes
- if (conflicts(request->mode, grantedModes) ||
- (!compatibleFirstCount && conflicts(request->mode, conflictModes))) {
- request->status = LockRequest::STATUS_WAITING;
-
- // Put it on the conflict queue. Conflicts are granted front to back.
- if (request->enqueueAtFront) {
- conflictList.push_front(request);
- } else {
- conflictList.push_back(request);
- }
-
- incConflictModeCount(request->mode);
-
- return LOCK_WAITING;
- }
-
- // No conflict, new request
- request->status = LockRequest::STATUS_GRANTED;
-
- grantedList.push_back(request);
- incGrantedModeCount(request->mode);
-
- if (request->compatibleFirst) {
- compatibleFirstCount++;
- }
-
- return LOCK_OK;
-}
-
void LockHead::migratePartitionedLockHeads() {
invariant(partitioned());
@@ -282,38 +405,6 @@ void LockHead::migratePartitionedLockHeads() {
}
}
-void LockHead::incGrantedModeCount(LockMode mode) {
- invariant(grantedCounts[mode] >= 0);
- if (++grantedCounts[mode] == 1) {
- invariant((grantedModes & modeMask(mode)) == 0);
- grantedModes |= modeMask(mode);
- }
-}
-
-void LockHead::decGrantedModeCount(LockMode mode) {
- invariant(grantedCounts[mode] >= 1);
- if (--grantedCounts[mode] == 0) {
- invariant((grantedModes & modeMask(mode)) == modeMask(mode));
- grantedModes &= ~modeMask(mode);
- }
-}
-
-void LockHead::incConflictModeCount(LockMode mode) {
- invariant(conflictCounts[mode] >= 0);
- if (++conflictCounts[mode] == 1) {
- invariant((conflictModes & modeMask(mode)) == 0);
- conflictModes |= modeMask(mode);
- }
-}
-
-void LockHead::decConflictModeCount(LockMode mode) {
- invariant(conflictCounts[mode] >= 1);
- if (--conflictCounts[mode] == 0) {
- invariant((conflictModes & modeMask(mode)) == modeMask(mode));
- conflictModes &= ~modeMask(mode);
- }
-}
-
//
// LockManager
//
@@ -1115,8 +1206,4 @@ const char* lockRequestStatusName(LockRequest::Status status) {
return LockRequestStatusNames[status];
}
-LockManager::TemporaryResourceQueue::TemporaryResourceQueue(ResourceId resourceId) {
- _lockHead.initNew(resourceId);
-}
-
} // namespace mongo
diff --git a/src/mongo/db/concurrency/lock_manager.h b/src/mongo/db/concurrency/lock_manager.h
index 69c180d0701..7b137715412 100644
--- a/src/mongo/db/concurrency/lock_manager.h
+++ b/src/mongo/db/concurrency/lock_manager.h
@@ -59,8 +59,6 @@ public:
LockManager();
~LockManager();
- class TemporaryResourceQueue;
-
/**
* Acquires lock on the specified resource in the specified mode and returns the outcome
* of the operation. See the details for LockResult for more information on what the
@@ -142,10 +140,6 @@ private:
// The lockheads need access to the partitions
friend struct LockHead;
- // ReplicationLockManagerManipulator manipulates LockManager state directly in order to
- // encapsulate specific logic for replication state transitions.
- friend class ReplicationLockManagerManipulator;
-
// These types describe the locks hash table
struct LockBucket {
@@ -208,7 +202,7 @@ private:
*
* @param lock Lock whose grant state should be recalculated.
* @param checkConflictQueue Whether to go through the conflict queue. This is an
- * optimization in that we only need to check the conflict queue if one of the
+ * optimisation in that we only need to check the conflict queue if one of the
* granted modes, which was conflicting before became zero.
*/
void _onLockModeChanged(LockHead* lock, bool checkConflictQueue);
@@ -321,146 +315,4 @@ private:
bool _foundCycle;
};
-/**
- * There is one of these objects for each resource that has a lock request. Empty objects (i.e.
- * LockHead with no requests) are allowed to exist on the lock manager's hash table.
- *
- * The memory and lifetime is controlled entirely by the LockManager class.
- *
- * Not thread-safe and should only be accessed under the LockManager's bucket lock. Must be locked
- * before locking a partition, not after.
- *
- * This struct *should* just be a private sub-class of LockManager since it is effectively an
- * implementation detail of LockManager. The reason it isn't is because it is also used by
- * ReplicationLockManagerManipulator and because it is forward-declared in lock_manager_defs.h.
- * Nevertheless this struct should be thought of as an implementation detail of LockManager and
- * should not be used directly except by LockManager and friend classes of LockManager that serve to
- * extend LockManager functionality.
- */
-struct LockHead {
-private:
- friend class DeadlockDetector;
- friend class LockManager;
- friend class ReplicationLockManagerManipulator;
-
- /**
- * Used for initialization of a LockHead, which might have been retrieved from cache and also in
- * order to keep the LockHead structure a POD.
- */
- void initNew(ResourceId resId);
-
- /**
- * True iff there may be partitions with granted requests for this resource.
- */
- bool partitioned() const;
-
- /**
- * Locates the request corresponding to the particular locker or returns nullptr. Must be called
- * with the bucket holding this lock head locked.
- */
- LockRequest* findRequest(LockerId lockerId) const;
-
- /**
- * Finish creation of request and put it on the LockHead's conflict or granted queues. Returns
- * LOCK_WAITING for conflict case and LOCK_OK otherwise.
- */
- LockResult newRequest(LockRequest* request);
-
- /**
- * Lock each partitioned LockHead in turn, and move any (granted) intent mode requests for
- * lock->resourceId to lock, which must itself already be locked.
- */
- void migratePartitionedLockHeads();
-
- // Methods to maintain the granted queue
- void incGrantedModeCount(LockMode mode);
-
- void decGrantedModeCount(LockMode mode);
-
- // Methods to maintain the conflict queue
- void incConflictModeCount(LockMode mode);
-
- void decConflictModeCount(LockMode mode);
-
- // Id of the resource which is protected by this lock. Initialized at construction time and does
- // not change.
- ResourceId resourceId;
-
- //
- // Granted queue
- //
-
- // Doubly-linked list of requests, which have been granted. Newly granted requests go to
- // the end of the queue. Conversion requests are granted from the beginning forward.
- LockRequestList grantedList;
-
- // Counts the grants and conversion counts for each of the supported lock modes. These
- // counts should exactly match the aggregated modes on the granted list.
- uint32_t grantedCounts[LockModesCount];
-
- // Bit-mask of the granted + converting modes on the granted queue. Maintained in lock-step
- // with the grantedCounts array.
- uint32_t grantedModes;
-
- //
- // Conflict queue
- //
-
- // Doubly-linked list of requests, which have not been granted yet because they conflict
- // with the set of granted modes. Requests are queued at the end of the queue and are
- // granted from the beginning forward, which gives these locks FIFO ordering. Exceptions
- // are high-priority locks, such as the MMAP V1 flush lock.
- LockRequestList conflictList;
-
- // Counts the conflicting requests for each of the lock modes. These counts should exactly
- // match the aggregated modes on the conflicts list.
- uint32_t conflictCounts[LockModesCount];
-
- // Bit-mask of the conflict modes on the conflict queue. Maintained in lock-step with the
- // conflictCounts array.
- uint32_t conflictModes;
-
- // References partitions that may have PartitionedLockHeads for this LockHead.
- // Non-empty implies the lock has no conflicts and only has intent modes as grantedModes.
- // TODO: Remove this vector and make LockHead a POD
- std::vector<LockManager::Partition*> partitions;
-
- //
- // Conversion
- //
-
- // Counts the number of requests on the granted queue, which have requested any kind of
- // conflicting conversion and are blocked (i.e. all requests which are currently
- // STATUS_CONVERTING). This is an optimization for unlocking in that we do not need to
- // check the granted queue for requests in STATUS_CONVERTING if this count is zero. This
- // saves cycles in the regular case and only burdens the less-frequent lock upgrade case.
- uint32_t conversionsCount;
-
- // Counts the number of requests on the granted queue, which have requested that the policy
- // be switched to compatible-first. As long as this value is > 0, the policy will stay
- // compatible-first.
- uint32_t compatibleFirstCount;
-};
-
-/**
- * This class wraps a LockHead and is used to represent a temporary state for a resource managed by
- * the LockManager. This allows us to prepare lock requests against a resource without those
- * requests actually being present in the "true" version of the resource which is actually being
- * managed by the LockManager. We use this to avoid exposing LockHead, which is an implementation
- * detail of the LockManager, while still providing a handle to state for a LockManager resource.
- */
-class LockManager::TemporaryResourceQueue {
-public:
- explicit TemporaryResourceQueue(ResourceId resourceId);
-
- ResourceId getResourceId() const {
- return _lockHead.resourceId;
- }
-
-private:
- friend class ReplicationLockManagerManipulator;
-
- LockHead _lockHead;
-};
-
} // namespace mongo
diff --git a/src/mongo/db/concurrency/lock_state.cpp b/src/mongo/db/concurrency/lock_state.cpp
index c97430363c1..e4df0cb2528 100644
--- a/src/mongo/db/concurrency/lock_state.cpp
+++ b/src/mongo/db/concurrency/lock_state.cpp
@@ -36,7 +36,6 @@
#include <vector>
-#include "mongo/db/concurrency/replication_lock_manager_manipulator.h"
#include "mongo/db/namespace_string.h"
#include "mongo/db/service_context.h"
#include "mongo/platform/compiler.h"
@@ -123,19 +122,9 @@ AtomicUInt64 idCounter(0);
// Partitioned global lock statistics, so we don't hit the same bucket
PartitionedInstanceWideLockStats globalStats;
-// When this is set, lock acquisitions for the global resource go into the TemporaryRequestQueue
-// stored in this decoration instead of letting the LockManager look up and put the requests into
-// the true global resource state.
-OperationContext::Decoration<LockManager::TemporaryResourceQueue*> globalResourceShadow =
- OperationContext::declareDecoration<LockManager::TemporaryResourceQueue*>();
-
} // namespace
bool LockerImpl::_shouldDelayUnlock(ResourceId resId, LockMode mode) const {
- if (_prepareModeForLockYields) {
- return false;
- }
-
switch (resId.getType()) {
case RESOURCE_MUTEX:
return false;
@@ -276,8 +265,7 @@ LockerImpl::~LockerImpl() {
invariant(!inAWriteUnitOfWork());
invariant(_numResourcesToUnlockAtEndUnitOfWork == 0);
invariant(_requests.empty());
- invariant(_modeForTicket == MODE_NONE,
- str::stream() << "_modeForTicket found: " << _modeForTicket);
+ invariant(_modeForTicket == MODE_NONE);
// Reset the locking statistics so the object can be reused
_stats.reset();
@@ -363,7 +351,7 @@ LockResult LockerImpl::_lockGlobalBegin(OperationContext* opCtx, LockMode mode,
}
LockMode actualLockMode = mode;
- if (opCtx && opCtx->getServiceContext()) {
+ if (opCtx) {
auto storageEngine = opCtx->getServiceContext()->getStorageEngine();
if (storageEngine && !storageEngine->supportsDBLocking()) {
actualLockMode = isSharedLockMode(mode) ? MODE_S : MODE_X;
@@ -375,7 +363,7 @@ LockResult LockerImpl::_lockGlobalBegin(OperationContext* opCtx, LockMode mode,
// Currently, deadlock detection does not happen inline with lock acquisition so the only
// unsuccessful result that the lock manager would return is LOCK_WAITING.
- invariant(result == LOCK_WAITING, str::stream() << "Unexpected lock result: " << result);
+ invariant(result == LOCK_WAITING);
return result;
}
@@ -593,18 +581,9 @@ 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, excepting the
- // special behavior for saving/restoring locks for prepared transactions during repl state
- // transitions.
- invariant(!inAWriteUnitOfWork() || _prepareModeForLockYields);
+ // We shouldn't be saving and restoring lock state from inside a WriteUnitOfWork.
+ invariant(!inAWriteUnitOfWork());
// Clear out whatever is in stateOut.
stateOut->locks.clear();
@@ -661,10 +640,8 @@ 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, excepting the
- // special behavior for saving/restoring locks for prepared transactions during repl state
- // transitions.
- invariant(!inAWriteUnitOfWork() || _prepareModeForLockYields);
+ // We shouldn't be saving and restoring lock state from inside a WriteUnitOfWork.
+ invariant(!inAWriteUnitOfWork());
invariant(_modeForTicket == MODE_NONE);
std::vector<OneLock>::const_iterator it = state.locks.begin();
@@ -681,47 +658,6 @@ void LockerImpl::restoreLockState(OperationContext* opCtx, const Locker::LockSna
invariant(_modeForTicket != MODE_NONE);
}
-void LockerImpl::restoreLockStateWithTemporaryGlobalResource(
- OperationContext* opCtx,
- const LockSnapshot& state,
- LockManager::TemporaryResourceQueue* tempGlobalResource) {
- invariant(tempGlobalResource->getResourceId().getType() == ResourceType::RESOURCE_GLOBAL);
- invariant(globalResourceShadow(opCtx) == nullptr);
- invariant(!_prepareModeForLockYields);
-
- globalResourceShadow(opCtx) = tempGlobalResource;
- _prepareModeForLockYields = true;
- ON_BLOCK_EXIT([&] {
- globalResourceShadow(opCtx) = nullptr;
- _prepareModeForLockYields = false;
- });
-
- restoreLockState(opCtx, state);
-}
-
-void LockerImpl::replaceGlobalLockStateWithTemporaryGlobalResource(
- LockManager::TemporaryResourceQueue* tempGlobalResource) {
- invariant(tempGlobalResource->getResourceId().getType() == ResourceType::RESOURCE_GLOBAL);
-
- // Transfer the LockRequests from tempGlobalResource into the true resource for the global lock
- // that is managed by the LockManager. This also removes the existing MODE_X LockRequest from
- // the granted list for the true global resource, but we still need to delete that LockRequest
- // and remove it from this Locker's _requests list.
- ReplicationLockManagerManipulator(&globalLockManager)
- .replaceGlobalLocksWithLocksFromTemporaryGlobalResource(resourceIdGlobal,
- tempGlobalResource);
-
- // Release the ticket that this Locker was holding for the global X lock.
- invariant(_modeForTicket == MODE_X);
- _releaseTicket();
- _modeForTicket = MODE_NONE;
-
- // Now fully delete the LockRequest.
- auto it = _requests.find(resourceIdGlobal);
- scoped_spinlock scopedLock(_lock);
- it.remove();
-}
-
LockResult LockerImpl::lockBegin(OperationContext* opCtx, ResourceId resId, LockMode mode) {
dassert(!getWaitingResource().isValid());
@@ -777,21 +713,8 @@ LockResult LockerImpl::lockBegin(OperationContext* opCtx, ResourceId resId, Lock
// otherwise we might reset state if the lock becomes granted very fast.
_notify.clear();
- LockResult result{LockResult::LOCK_INVALID};
- 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.
- invariant(isNew);
- ReplicationLockManagerManipulator(&globalLockManager)
- .lockUncontestedTemporaryGlobalResource(globalResourceShadow(opCtx), request, mode);
- // lockUncontestedTemporaryGlobalResource can't fail.
- result = LockResult::LOCK_OK;
- } else {
- // Normal case using the true global lock head.
- result = isNew ? globalLockManager.lock(resId, request, mode)
- : globalLockManager.convert(resId, request, mode);
- }
+ LockResult result = isNew ? globalLockManager.lock(resId, request, mode)
+ : globalLockManager.convert(resId, request, mode);
if (result == LOCK_WAITING) {
globalStats.recordWait(_id, resId, mode);
diff --git a/src/mongo/db/concurrency/lock_state.h b/src/mongo/db/concurrency/lock_state.h
index fce45afc56c..4532e0e9830 100644
--- a/src/mongo/db/concurrency/lock_state.h
+++ b/src/mongo/db/concurrency/lock_state.h
@@ -189,21 +189,12 @@ 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) {
restoreLockState(nullptr, stateToRestore);
}
- void restoreLockStateWithTemporaryGlobalResource(
- OperationContext* opCtx,
- const LockSnapshot& stateToRestore,
- LockManager::TemporaryResourceQueue* tempGlobalResource) override;
-
- void replaceGlobalLockStateWithTemporaryGlobalResource(
- LockManager::TemporaryResourceQueue* tempGlobalResource) override;
-
virtual void releaseTicket();
virtual void reacquireTicket(OperationContext* opCtx);
@@ -342,10 +333,6 @@ 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/lock_state_test.cpp b/src/mongo/db/concurrency/lock_state_test.cpp
index 356454d8be9..ba3c3e32079 100644
--- a/src/mongo/db/concurrency/lock_state_test.cpp
+++ b/src/mongo/db/concurrency/lock_state_test.cpp
@@ -39,7 +39,6 @@
#include "mongo/config.h"
#include "mongo/db/concurrency/lock_manager_test_help.h"
#include "mongo/db/concurrency/locker.h"
-#include "mongo/db/operation_context_noop.h"
#include "mongo/unittest/unittest.h"
#include "mongo/util/log.h"
#include "mongo/util/timer.h"
@@ -660,102 +659,4 @@ TEST(LockerImpl, ConvertLockPendingUnlockAndUnlock) {
locker.unlockGlobal();
}
-/**
- * This test mimics the process that replica set state transitions go through.
- * First a request for the global X lock is enqueued, then the existing locks held by various
- * ongoing prepared transactions are yielded so that the global X lock request can succeed.
- * When the state transition is complete we then need to atomically restore the locks that were
- * yielded while dropping the global X lock that was held for the transition.
- */
-TEST(LockerImpl, AtomicLockRestoreAndGlobalLockReleaseForReplStateTransitions) {
- const ResourceId globalResId(RESOURCE_GLOBAL, ResourceId::SINGLETON_GLOBAL);
- const ResourceId resIdDatabase(RESOURCE_DATABASE, "TestDB"_sd);
- const ResourceId resIdCollection1(RESOURCE_COLLECTION, "TestDB.collection1"_sd);
- const ResourceId resIdCollection2(RESOURCE_COLLECTION, "TestDB.collection2"_sd);
- const ResourceId resIdCollection3(RESOURCE_COLLECTION, "TestDB.collection2"_sd);
-
- Locker::LockSnapshot lockInfo1, lockInfo2, lockInfo3;
- LockerImpl stepUpLocker, txnLocker1, txnLocker2, txnLocker3, randomOpLocker1, randomOpLocker2;
- OperationContextNoop opCtx1, opCtx2, opCtx3;
- ON_BLOCK_EXIT([&] {
- // clean up locks on test completion.
- stepUpLocker.unlockGlobal();
- txnLocker1.unlockGlobal();
- txnLocker2.unlockGlobal();
- txnLocker3.unlockGlobal();
- randomOpLocker1.unlockGlobal();
- randomOpLocker2.unlockGlobal();
- });
-
- // Take some locks, mimicking the locks that would be held by ongoing prepared transactions.
- txnLocker1.lockGlobal(&opCtx1, MODE_IX);
- ASSERT_EQUALS(LOCK_OK, txnLocker1.lock(&opCtx1, resIdDatabase, MODE_IX));
- ASSERT_EQUALS(LOCK_OK, txnLocker1.lock(&opCtx1, resIdCollection1, MODE_IX));
- ASSERT_EQUALS(LOCK_OK, txnLocker1.lock(&opCtx1, resIdCollection2, MODE_IX));
-
- txnLocker2.lockGlobal(&opCtx2, MODE_IX);
- ASSERT_EQUALS(LOCK_OK, txnLocker2.lock(&opCtx2, resIdDatabase, MODE_IX));
- ASSERT_EQUALS(LOCK_OK, txnLocker2.lock(&opCtx2, resIdCollection2, MODE_IX));
-
- txnLocker3.lockGlobal(&opCtx3, MODE_IX);
- ASSERT_EQUALS(LOCK_OK, txnLocker3.lock(&opCtx3, resIdDatabase, MODE_IX));
- ASSERT_EQUALS(LOCK_OK, txnLocker3.lock(&opCtx3, resIdCollection3, MODE_IX));
-
- // Enqueue request for global X lock in stepUpLocker, mimicking the lock held by the thread
- // performing the state transition.
- ASSERT_EQUALS(LockResult::LOCK_WAITING, stepUpLocker.lockGlobalBegin(MODE_X, Date_t::max()));
-
- // Enqueue a lock request behind the pending global X request to ensure that it gets granted
- // later when the global X lock is released.
- ASSERT_EQUALS(LockResult::LOCK_WAITING,
- randomOpLocker1.lockGlobalBegin(MODE_IS, Date_t::max()));
-
- // Yield locks on all txn threads.
- ASSERT_TRUE(txnLocker1.saveLockStateAndUnlock(&lockInfo1));
- ASSERT_TRUE(txnLocker2.saveLockStateAndUnlock(&lockInfo2));
- ASSERT_TRUE(txnLocker3.saveLockStateAndUnlock(&lockInfo3));
-
- // Ensure that stepUpLocker is now able to acquire the global X lock.
- ASSERT_EQUALS(LockResult::LOCK_OK, stepUpLocker.lockGlobalComplete(Date_t::max()));
-
- // Enqueue a lock request behind the global X lock to ensure that it gets granted
- // later when the global X lock is released.
- ASSERT_EQUALS(LockResult::LOCK_WAITING,
- randomOpLocker2.lockGlobalBegin(MODE_IX, Date_t::max()));
-
- // Now we need to atomically release the global X lock from stepUpLocker and restore the lock
- // state for the txn threads. We start by restoring the lock state from the txn threads with
- // the global locks placed into a temporary LockHead.
- LockManager::TemporaryResourceQueue tempGlobalResource(globalResId);
-
- txnLocker1.restoreLockStateWithTemporaryGlobalResource(&opCtx1, lockInfo1, &tempGlobalResource);
- txnLocker2.restoreLockStateWithTemporaryGlobalResource(&opCtx2, lockInfo2, &tempGlobalResource);
- txnLocker3.restoreLockStateWithTemporaryGlobalResource(&opCtx3, lockInfo3, &tempGlobalResource);
-
- // Atomically release the global X lock from stepUpLocker and move the global locks for the
- // txn threads from the temporary LockHead into the true LockHead for the global lock.
- stepUpLocker.replaceGlobalLockStateWithTemporaryGlobalResource(&tempGlobalResource);
-
- // Make sure all the locks were acquired and released appropriately.
- ASSERT_EQUALS(MODE_NONE, stepUpLocker.getLockMode(globalResId));
-
- ASSERT_EQUALS(MODE_IX, txnLocker1.getLockMode(globalResId));
- ASSERT_EQUALS(MODE_IX, txnLocker1.getLockMode(resIdDatabase));
- ASSERT_EQUALS(MODE_IX, txnLocker1.getLockMode(resIdCollection1));
- ASSERT_EQUALS(MODE_IX, txnLocker1.getLockMode(resIdCollection2));
-
- ASSERT_EQUALS(MODE_IX, txnLocker2.getLockMode(globalResId));
- ASSERT_EQUALS(MODE_IX, txnLocker2.getLockMode(resIdDatabase));
- ASSERT_EQUALS(MODE_IX, txnLocker2.getLockMode(resIdCollection2));
-
- ASSERT_EQUALS(MODE_IX, txnLocker3.getLockMode(globalResId));
- ASSERT_EQUALS(MODE_IX, txnLocker3.getLockMode(resIdDatabase));
- ASSERT_EQUALS(MODE_IX, txnLocker3.getLockMode(resIdCollection3));
-
- // Make sure the pending global lock requests got granted when the global X lock was released.
- ASSERT_EQUALS(LockResult::LOCK_OK, randomOpLocker1.lockGlobalComplete(Date_t::now()));
- ASSERT_EQUALS(LockResult::LOCK_OK, randomOpLocker2.lockGlobalComplete(Date_t::now()));
- ASSERT_EQUALS(MODE_IS, randomOpLocker1.getLockMode(globalResId));
- ASSERT_EQUALS(MODE_IX, randomOpLocker2.getLockMode(globalResId));
-}
} // namespace mongo
diff --git a/src/mongo/db/concurrency/locker.h b/src/mongo/db/concurrency/locker.h
index 4677f2b1fc2..03b4d871c5c 100644
--- a/src/mongo/db/concurrency/locker.h
+++ b/src/mongo/db/concurrency/locker.h
@@ -361,12 +361,6 @@ 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.
*/
@@ -374,27 +368,6 @@ public:
virtual void restoreLockState(const LockSnapshot& stateToRestore) = 0;
/**
- * Works like restoreLockState but for any global locks in the state to restore, rather than
- * restoring them into the true global lock resource owned by the LockManager,
- * 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,
- const LockSnapshot& stateToRestore,
- LockManager::TemporaryResourceQueue* tempGlobalResource) = 0;
-
- /**
- * Atomically releases the global X lock from the true global resource managed by the
- * LockManager and transfers the locks from the 'tempGlobalResource' into the true global
- * resource.
- */
- virtual void replaceGlobalLockStateWithTemporaryGlobalResource(
- LockManager::TemporaryResourceQueue* tempGlobalResource) = 0;
-
- /**
* Releases the ticket associated with the Locker. This allows locks to be held without
* contributing to reader/writer throttling.
*/
diff --git a/src/mongo/db/concurrency/locker_noop.h b/src/mongo/db/concurrency/locker_noop.h
index 56498de53f2..95f426b2e15 100644
--- a/src/mongo/db/concurrency/locker_noop.h
+++ b/src/mongo/db/concurrency/locker_noop.h
@@ -173,10 +173,6 @@ public:
MONGO_UNREACHABLE;
}
- virtual bool saveLockStateAndUnlockForPrepare(LockSnapshot* stateOut) {
- MONGO_UNREACHABLE;
- }
-
virtual void restoreLockState(OperationContext* opCtx, const LockSnapshot& stateToRestore) {
MONGO_UNREACHABLE;
}
@@ -184,18 +180,6 @@ public:
MONGO_UNREACHABLE;
}
- void restoreLockStateWithTemporaryGlobalResource(
- OperationContext* opCtx,
- const LockSnapshot& stateToRestore,
- LockManager::TemporaryResourceQueue* tempGlobalResource) override {
- MONGO_UNREACHABLE;
- }
-
- void replaceGlobalLockStateWithTemporaryGlobalResource(
- LockManager::TemporaryResourceQueue* tempGlobalResource) override {
- MONGO_UNREACHABLE;
- }
-
virtual void releaseTicket() {
MONGO_UNREACHABLE;
}
diff --git a/src/mongo/db/concurrency/replication_lock_manager_manipulator.cpp b/src/mongo/db/concurrency/replication_lock_manager_manipulator.cpp
deleted file mode 100644
index 4a64ce57d38..00000000000
--- a/src/mongo/db/concurrency/replication_lock_manager_manipulator.cpp
+++ /dev/null
@@ -1,113 +0,0 @@
-
-/**
- * Copyright (C) 2018-present MongoDB, Inc.
- *
- * This program is free software: you can redistribute it and/or modify
- * it under the terms of the Server Side Public License, version 1,
- * as published by MongoDB, Inc.
- *
- * This program is distributed in the hope that it will be useful,
- * but WITHOUT ANY WARRANTY; without even the implied warranty of
- * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
- * Server Side Public License for more details.
- *
- * You should have received a copy of the Server Side Public License
- * along with this program. If not, see
- * <http://www.mongodb.com/licensing/server-side-public-license>.
- *
- * As a special exception, the copyright holders give permission to link the
- * code of portions of this program with the OpenSSL library under certain
- * conditions as described in each individual source file and distribute
- * linked combinations including the program with the OpenSSL library. You
- * must comply with the Server Side Public License in all respects for
- * all of the code used other than as permitted herein. If you modify file(s)
- * with this exception, you may extend this exception to your version of the
- * file(s), but you are not obligated to do so. If you do not wish to do so,
- * delete this exception statement from your version. If you delete this
- * exception statement from all source files in the program, then also delete
- * it in the license file.
- */
-
-#define MONGO_LOG_DEFAULT_COMPONENT ::mongo::logger::LogComponent::kReplication
-
-#include "mongo/platform/basic.h"
-
-#include "mongo/db/concurrency/replication_lock_manager_manipulator.h"
-
-namespace mongo {
-
-ReplicationLockManagerManipulator::ReplicationLockManagerManipulator(LockManager* lockManager)
- : _lockManager(lockManager) {}
-
-void ReplicationLockManagerManipulator::lockUncontestedTemporaryGlobalResource(
- LockManager::TemporaryResourceQueue* tempGlobalResource, LockRequest* request, LockMode mode) {
- // Sanity check that requests are not being reused without proper cleanup
- invariant(request->status == LockRequest::STATUS_NEW);
- 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);
- invariant(lockResult == LockResult::LOCK_OK);
-}
-
-void ReplicationLockManagerManipulator::replaceGlobalLocksWithLocksFromTemporaryGlobalResource(
- ResourceId resId, LockManager::TemporaryResourceQueue* tempGlobalResource) {
- invariant(resId.getType() == ResourceType::RESOURCE_GLOBAL);
- invariant(tempGlobalResource->_lockHead.resourceId == resId);
-
- LockManager::LockBucket* bucket = _lockManager->_getBucket(resId);
- stdx::lock_guard<SimpleMutex> scopedLock(bucket->mutex);
- LockHead* trueGlobalLockHead = bucket->findOrInsert(resId);
-
- LockHead* tempGlobalLockHead = &tempGlobalResource->_lockHead;
-
- invariant(trueGlobalLockHead->grantedCounts[MODE_X] == 1);
- invariant(trueGlobalLockHead->compatibleFirstCount == 1);
- invariant(tempGlobalLockHead->conflictList.empty());
-
- LockRequest* existingGlobalLockRequest = trueGlobalLockHead->grantedList._front;
- invariant(!existingGlobalLockRequest->next);
- invariant(existingGlobalLockRequest->mode == MODE_X);
- invariant(existingGlobalLockRequest->status == LockRequest::Status::STATUS_GRANTED);
-
- // Remove the existing granted MODE_X lock from the trueGlobalLockHead so it can be replaced
- // by the locks from tempGlobalLockHead.
- trueGlobalLockHead->grantedList.remove(existingGlobalLockRequest);
- trueGlobalLockHead->decGrantedModeCount(existingGlobalLockRequest->mode);
- trueGlobalLockHead->compatibleFirstCount--;
-
- // Now iterate over the granted LockRequests in the tempGlobalLockHead and transfer them over
- // to the trueGlobalLockHead.
- for (LockRequest* it = tempGlobalLockHead->grantedList._front; it != nullptr;) {
- LockRequest* next = it->next;
-
- 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);
-
- it->lock = trueGlobalLockHead;
- tempGlobalLockHead->grantedList.remove(it);
- tempGlobalLockHead->decGrantedModeCount(it->mode);
- trueGlobalLockHead->grantedList.push_back(it);
- trueGlobalLockHead->incGrantedModeCount(it->mode);
-
- it = next;
- }
- invariant(tempGlobalLockHead->grantedList.empty());
- invariant(tempGlobalLockHead->grantedCounts[MODE_X] == 0);
- invariant(tempGlobalLockHead->grantedModes == 0);
-
- // Grant any pending requests against the true global lock head that can proceed now that the
- // global X lock has been released.
- _lockManager->_onLockModeChanged(trueGlobalLockHead, true);
-}
-
-} // namespace mongo
diff --git a/src/mongo/db/concurrency/replication_lock_manager_manipulator.h b/src/mongo/db/concurrency/replication_lock_manager_manipulator.h
deleted file mode 100644
index 4760771ba79..00000000000
--- a/src/mongo/db/concurrency/replication_lock_manager_manipulator.h
+++ /dev/null
@@ -1,73 +0,0 @@
-
-/**
- * Copyright (C) 2018-present MongoDB, Inc.
- *
- * This program is free software: you can redistribute it and/or modify
- * it under the terms of the Server Side Public License, version 1,
- * as published by MongoDB, Inc.
- *
- * This program is distributed in the hope that it will be useful,
- * but WITHOUT ANY WARRANTY; without even the implied warranty of
- * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
- * Server Side Public License for more details.
- *
- * You should have received a copy of the Server Side Public License
- * along with this program. If not, see
- * <http://www.mongodb.com/licensing/server-side-public-license>.
- *
- * As a special exception, the copyright holders give permission to link the
- * code of portions of this program with the OpenSSL library under certain
- * conditions as described in each individual source file and distribute
- * linked combinations including the program with the OpenSSL library. You
- * must comply with the Server Side Public License in all respects for
- * all of the code used other than as permitted herein. If you modify file(s)
- * with this exception, you may extend this exception to your version of the
- * file(s), but you are not obligated to do so. If you do not wish to do so,
- * delete this exception statement from your version. If you delete this
- * exception statement from all source files in the program, then also delete
- * it in the license file.
- */
-
-#pragma once
-
-#include "mongo/db/concurrency/lock_manager.h"
-
-namespace mongo {
-
-/**
- * This friend class to the LockManager extends LockManager functionality to enable behaviors
- * required for replication state transitions, which must atomically release the global X lock and
- * restore locks for prepared transactions into their individual Lockers.
- */
-class ReplicationLockManagerManipulator {
- MONGO_DISALLOW_COPYING(ReplicationLockManagerManipulator);
-
-public:
- explicit ReplicationLockManagerManipulator(LockManager* lockManager);
- ~ReplicationLockManagerManipulator() = default;
-
- /**
- * Works like LockManager::lock() except that it only works for the Global lock and rather than
- * looking up the true LockHead for the global resource ID, it puts the LockRequest into the
- * given TemporaryResourceQueue, which is guaranteed not to have any conflicting locks for the
- * given request.
- */
- void lockUncontestedTemporaryGlobalResource(
- LockManager::TemporaryResourceQueue* tempGlobalResource,
- LockRequest* request,
- LockMode mode);
-
- /**
- * Takes the locks from a given TemporaryResourceQueue for the Global resource and moves them
- * into the true LockHead for the global resource, atomically releasing the global X lock from
- * the true LockHead for the global resource in the process.
- */
- void replaceGlobalLocksWithLocksFromTemporaryGlobalResource(
- ResourceId resId, LockManager::TemporaryResourceQueue* tempGlobalResource);
-
-
-private:
- LockManager* _lockManager;
-};
-
-} // namespace mongo
diff --git a/src/mongo/db/kill_sessions_local.cpp b/src/mongo/db/kill_sessions_local.cpp
index 02b1212a463..991bc179e59 100644
--- a/src/mongo/db/kill_sessions_local.cpp
+++ b/src/mongo/db/kill_sessions_local.cpp
@@ -110,15 +110,4 @@ void killSessionsLocalShutdownAllTransactions(OperationContext* opCtx) {
});
}
-void killSessionsLocalAbortOrYieldAllTransactions(
- OperationContext* opCtx, std::vector<std::pair<Locker*, Locker::LockSnapshot>>* yieldedLocks) {
- SessionKiller::Matcher matcherAllSessions(
- KillAllSessionsByPatternSet{makeKillAllSessionsByPattern(opCtx)});
- killSessionsAction(
- 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 79fd6f06c33..3d6f0699cba 100644
--- a/src/mongo/db/kill_sessions_local.h
+++ b/src/mongo/db/kill_sessions_local.h
@@ -30,9 +30,6 @@
#pragma once
-#include <vector>
-
-#include "mongo/db/concurrency/d_concurrency.h"
#include "mongo/db/session_killer.h"
/**
@@ -63,11 +60,4 @@ 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 99129b4c14d..e036ddbb975 100644
--- a/src/mongo/db/repl/replication_coordinator_impl.cpp
+++ b/src/mongo/db/repl/replication_coordinator_impl.cpp
@@ -1765,7 +1765,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.releaseGlobalLockForStepdownAttempt();
+ transitionGuard.releaseGlobalLock();
invariant(!opCtx->lockState()->isLocked());
// Make sure we re-acquire the global lock before returning so that we're always holding
@@ -1782,7 +1782,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.reacquireGlobalLockForStepdownAttempt();
+ transitionGuard.reacquireGlobalLock();
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 c7728b265eb..b78fb7f290e 100644
--- a/src/mongo/db/repl/replication_state_transition_lock_guard.cpp
+++ b/src/mongo/db/repl/replication_state_transition_lock_guard.cpp
@@ -35,9 +35,6 @@
#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 {
@@ -45,33 +42,22 @@ 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);
- }
- // 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);
+ // Destroy all stashed transaction resources, in order to release locks.
+ SessionKiller::Matcher matcherAllSessions(
+ KillAllSessionsByPatternSet{makeKillAllSessionsByPattern(opCtx)});
+ killSessionsLocalKillTransactions(opCtx, matcherAllSessions);
+ }
- // 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",
@@ -80,31 +66,14 @@ 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::releaseGlobalLockForStepdownAttempt() {
+void ReplicationStateTransitionLockGuard::releaseGlobalLock() {
invariant(_globalLock->isLocked());
_globalLock.reset();
}
-void ReplicationStateTransitionLockGuard::reacquireGlobalLockForStepdownAttempt() {
+void ReplicationStateTransitionLockGuard::reacquireGlobalLock() {
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 80d3e99e125..a4a30e7381d 100644
--- a/src/mongo/db/repl/replication_state_transition_lock_guard.h
+++ b/src/mongo/db/repl/replication_state_transition_lock_guard.h
@@ -34,7 +34,6 @@
#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 {
@@ -43,9 +42,7 @@ 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. 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.
+ * for it to be granted.
*/
class ReplicationStateTransitionLockGuard {
MONGO_DISALLOW_COPYING(ReplicationStateTransitionLockGuard);
@@ -61,50 +58,27 @@ public:
};
/**
- * 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.
+ * Acquires the global X lock and performs any other required actions accoriding 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
- * reacquireGlobalLockForStepdownAttempt().
+ * Temporarily releases the global X lock. Must be followed by a call to reacquireGlobalLock().
*/
- void releaseGlobalLockForStepdownAttempt();
+ void releaseGlobalLock();
/**
- * 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.
+ * 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.
*/
- void reacquireGlobalLockForStepdownAttempt();
+ void reacquireGlobalLock();
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;
-
- // 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;
+ boost::optional<Lock::GlobalLock> _globalLock = boost::none;
};
} // namespace repl
diff --git a/src/mongo/db/session_catalog.cpp b/src/mongo/db/session_catalog.cpp
index cc1616626de..059c6639f39 100644
--- a/src/mongo/db/session_catalog.cpp
+++ b/src/mongo/db/session_catalog.cpp
@@ -83,10 +83,6 @@ ScopedCheckedOutSession SessionCatalog::checkOutSession(OperationContext* opCtx)
stdx::unique_lock<stdx::mutex> ul(_mutex);
- while (!_isSessionCheckoutAllowed()) {
- opCtx->waitForConditionOrInterrupt(_checkingOutSessionsAllowedCond, ul);
- }
-
auto sri = _getOrCreateSessionRuntimeInfo(ul, opCtx, lsid);
// Wait until the session is no longer checked out
@@ -95,7 +91,6 @@ ScopedCheckedOutSession SessionCatalog::checkOutSession(OperationContext* opCtx)
invariant(!sri->checkedOut);
sri->checkedOut = true;
- ++_numCheckedOutSessions;
return ScopedCheckedOutSession(opCtx, ScopedSession(std::move(sri)));
}
@@ -174,7 +169,6 @@ void SessionCatalog::scanSessions(OperationContext* opCtx,
std::shared_ptr<SessionCatalog::SessionRuntimeInfo> SessionCatalog::_getOrCreateSessionRuntimeInfo(
WithLock, OperationContext* opCtx, const LogicalSessionId& lsid) {
invariant(!opCtx->lockState()->inAWriteUnitOfWork());
- invariant(_isSessionCheckoutAllowed());
auto it = _sessions.find(lsid);
if (it == _sessions.end()) {
@@ -195,39 +189,6 @@ void SessionCatalog::_releaseSession(const LogicalSessionId& lsid) {
sri->checkedOut = false;
sri->availableCondVar.notify_one();
- --_numCheckedOutSessions;
- if (_numCheckedOutSessions == 0) {
- _allSessionsCheckedInCond.notify_all();
- }
-}
-
-SessionCatalog::PreventCheckingOutSessionsBlock::PreventCheckingOutSessionsBlock(
- SessionCatalog* sessionCatalog)
- : _sessionCatalog(sessionCatalog) {
- invariant(sessionCatalog);
-
- stdx::lock_guard<stdx::mutex> lg(sessionCatalog->_mutex);
- ++sessionCatalog->_preventSessionCheckoutRequests;
-}
-
-SessionCatalog::PreventCheckingOutSessionsBlock::~PreventCheckingOutSessionsBlock() {
- stdx::lock_guard<stdx::mutex> lg(_sessionCatalog->_mutex);
-
- invariant(_sessionCatalog->_preventSessionCheckoutRequests > 0);
- --_sessionCatalog->_preventSessionCheckoutRequests;
- if (_sessionCatalog->_preventSessionCheckoutRequests == 0) {
- _sessionCatalog->_checkingOutSessionsAllowedCond.notify_all();
- }
-}
-
-void SessionCatalog::PreventCheckingOutSessionsBlock::waitForAllSessionsToBeCheckedIn(
- OperationContext* opCtx) {
- stdx::unique_lock<stdx::mutex> ul(_sessionCatalog->_mutex);
-
- invariant(!_sessionCatalog->_isSessionCheckoutAllowed());
- while (_sessionCatalog->_numCheckedOutSessions > 0) {
- opCtx->waitForConditionOrInterrupt(_sessionCatalog->_allSessionsCheckedInCond, ul);
- }
}
OperationContextSession::OperationContextSession(OperationContext* opCtx, bool checkOutSession)
diff --git a/src/mongo/db/session_catalog.h b/src/mongo/db/session_catalog.h
index 2da89c2fb7a..addbb3c313c 100644
--- a/src/mongo/db/session_catalog.h
+++ b/src/mongo/db/session_catalog.h
@@ -58,8 +58,6 @@ class SessionCatalog {
friend class ScopedCheckedOutSession;
public:
- class PreventCheckingOutSessionsBlock;
-
SessionCatalog() = default;
~SessionCatalog();
@@ -152,47 +150,10 @@ private:
*/
void _releaseSession(const LogicalSessionId& lsid);
- bool _isSessionCheckoutAllowed() const {
- return _preventSessionCheckoutRequests == 0;
- };
-
- // Protects members below.
stdx::mutex _mutex;
// Owns the Session objects for all current Sessions.
LogicalSessionIdMap<std::shared_ptr<SessionRuntimeInfo>> _sessions;
-
- // Count of the number of Sessions that are currently checked out.
- uint32_t _numCheckedOutSessions{0};
-
- // When >0 all Session checkout or creation requests will block.
- uint32_t _preventSessionCheckoutRequests{0};
-
- // Condition that is signaled when the number of checked out sessions goes to 0.
- stdx::condition_variable _allSessionsCheckedInCond;
-
- // Condition that is signaled when checking out Sessions becomes legal again after having
- // previously been forbidden.
- stdx::condition_variable _checkingOutSessionsAllowedCond;
-};
-
-/**
- * While this object is in scope, all requests to check out a Session will block.
- */
-class SessionCatalog::PreventCheckingOutSessionsBlock {
- MONGO_DISALLOW_COPYING(PreventCheckingOutSessionsBlock);
-
-public:
- explicit PreventCheckingOutSessionsBlock(SessionCatalog* sessionCatalog);
- ~PreventCheckingOutSessionsBlock();
-
- /**
- * Waits until there are no Sessions checked out in the SessionCatalog.
- */
- void waitForAllSessionsToBeCheckedIn(OperationContext* opCtx);
-
-private:
- SessionCatalog* _sessionCatalog{nullptr};
};
/**
diff --git a/src/mongo/db/session_catalog_test.cpp b/src/mongo/db/session_catalog_test.cpp
index 678c44cfa2b..59b332b3989 100644
--- a/src/mongo/db/session_catalog_test.cpp
+++ b/src/mongo/db/session_catalog_test.cpp
@@ -202,108 +202,5 @@ TEST_F(SessionCatalogTest, ScanSessions) {
ASSERT_EQ(lsids.front(), lsid2);
}
-TEST_F(SessionCatalogTest, PreventCheckout) {
- const auto lsid = makeLogicalSessionIdForTest();
- opCtx()->setLogicalSessionId(lsid);
- opCtx()->setDeadlineAfterNowBy(Milliseconds(10), ErrorCodes::MaxTimeMSExpired);
-
- {
- SessionCatalog::PreventCheckingOutSessionsBlock preventCheckoutBlock(catalog());
-
- ASSERT_THROWS_CODE(
- catalog()->checkOutSession(opCtx()), AssertionException, ErrorCodes::MaxTimeMSExpired);
- }
-
- auto scopedSession = catalog()->checkOutSession(opCtx());
- ASSERT(scopedSession.get());
- ASSERT_EQ(lsid, scopedSession->getSessionId());
-}
-
-TEST_F(SessionCatalogTest, WaitForAllSessions) {
- const auto lsid1 = makeLogicalSessionIdForTest();
- const auto lsid2 = makeLogicalSessionIdForTest();
- opCtx()->setLogicalSessionId(lsid1);
-
- // Check out a Session.
- boost::optional<OperationContextSession> ocs;
- ocs.emplace(opCtx(), true);
- ASSERT_EQ(lsid1, ocs->get(opCtx())->getSessionId());
-
- // Prevent new Sessions from being checked out.
- boost::optional<SessionCatalog::PreventCheckingOutSessionsBlock> preventCheckoutBlock;
- preventCheckoutBlock.emplace(catalog());
-
- // Enqueue a request to check out a Session.
- auto future = stdx::async(stdx::launch::async, [&] {
- ON_BLOCK_EXIT([&] { Client::destroy(); });
- Client::initThreadIfNotAlready();
- auto sideOpCtx = Client::getCurrent()->makeOperationContext();
- sideOpCtx->setLogicalSessionId(lsid2);
- auto asyncScopedSession =
- SessionCatalog::get(sideOpCtx.get())->checkOutSession(sideOpCtx.get());
-
- ASSERT(asyncScopedSession.get());
- ASSERT_EQ(lsid2, asyncScopedSession->getSessionId());
- });
-
- // Ensure that waitForAllSessionsToBeCheckedIn() times out since we are holding a Session
- // checked out.
- opCtx()->setDeadlineAfterNowBy(Milliseconds(10), ErrorCodes::MaxTimeMSExpired);
- ASSERT_THROWS_CODE(preventCheckoutBlock->waitForAllSessionsToBeCheckedIn(opCtx()),
- AssertionException,
- ErrorCodes::MaxTimeMSExpired);
-
- ASSERT(stdx::future_status::ready != future.wait_for(Milliseconds(10).toSystemDuration()));
-
- // Release the Session we have checked out.
- ocs.reset();
-
- // Now ensure that waitForAllSessionsToBeCheckedIn() can complete.
- preventCheckoutBlock->waitForAllSessionsToBeCheckedIn(opCtx());
-
- // Ensure that the async thread trying to check out a Session is still blocked.
- ASSERT(stdx::future_status::ready != future.wait_for(Milliseconds(10).toSystemDuration()));
-
- // Allow checking out Sessions to proceed.
- preventCheckoutBlock.reset();
-
- // Ensure that the async thread can now proceed and successfully check out a Session.
- future.get();
-}
-
-TEST_F(SessionCatalogTest, MultiplePreventCheckingOutSessionsBlocks) {
- const auto lsid1 = makeLogicalSessionIdForTest();
- opCtx()->setLogicalSessionId(lsid1);
- opCtx()->setDeadlineAfterNowBy(Milliseconds(10), ErrorCodes::MaxTimeMSExpired);
-
- boost::optional<OperationContextSession> ocs;
-
- // Prevent new Sessions from being checked out.
- boost::optional<SessionCatalog::PreventCheckingOutSessionsBlock> preventCheckoutBlock1,
- preventCheckoutBlock2;
- preventCheckoutBlock1.emplace(catalog());
-
- // Ensure that checking out a Session fails
- ASSERT_THROWS_CODE(
- ocs.emplace(opCtx(), true), AssertionException, ErrorCodes::MaxTimeMSExpired);
-
- // A second request to prevent checking out Sessions is legal.
- preventCheckoutBlock2.emplace(catalog());
- ASSERT_THROWS_CODE(
- ocs.emplace(opCtx(), true), AssertionException, ErrorCodes::MaxTimeMSExpired);
-
- // The first request completing before the second is valid and doesn't start allowing checkouts.
- preventCheckoutBlock1.reset();
- ASSERT_THROWS_CODE(
- ocs.emplace(opCtx(), true), AssertionException, ErrorCodes::MaxTimeMSExpired);
-
- // Releasing the last PreventCheckingOutSessionsBlock allows Session checkout to proceed.
- preventCheckoutBlock2.reset();
-
- ASSERT_TRUE(ocs == boost::none);
- ocs.emplace(opCtx(), true);
- ASSERT_EQ(lsid1, ocs->get(opCtx())->getSessionId());
-}
-
} // namespace
} // namespace mongo
diff --git a/src/mongo/db/transaction_participant.cpp b/src/mongo/db/transaction_participant.cpp
index 512c0ce7070..eb5af24bcd2 100644
--- a/src/mongo/db/transaction_participant.cpp
+++ b/src/mongo/db/transaction_participant.cpp
@@ -1111,22 +1111,6 @@ 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 1874926122c..451f502b5bb 100644
--- a/src/mongo/db/transaction_participant.h
+++ b/src/mongo/db/transaction_participant.h
@@ -98,13 +98,6 @@ 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.
*/
@@ -350,16 +343,6 @@ 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));
}