summaryrefslogtreecommitdiff
path: root/src/mongo
diff options
context:
space:
mode:
authorGeert Bosch <geert@mongodb.com>2015-04-27 18:02:49 -0400
committerGeert Bosch <geert@mongodb.com>2015-05-07 17:00:50 -0400
commit465ba933e8d6f5ad9173c4c806686b915bfffe1c (patch)
tree80874f0de721267d18a399144f4cef691dc2329e /src/mongo
parent97b5712600ada8439f024f6bf446172f0fc9a7aa (diff)
downloadmongo-465ba933e8d6f5ad9173c4c806686b915bfffe1c.tar.gz
SERVER-18190: Make ParallelBatchWriterMode use a LockManager managed lock
Diffstat (limited to 'src/mongo')
-rw-r--r--src/mongo/db/concurrency/d_concurrency.cpp47
-rw-r--r--src/mongo/db/concurrency/d_concurrency.h102
-rw-r--r--src/mongo/db/concurrency/lock_manager_defs.h45
-rw-r--r--src/mongo/db/concurrency/lock_state.cpp62
-rw-r--r--src/mongo/db/concurrency/lock_state.h7
-rw-r--r--src/mongo/db/concurrency/locker.h9
-rw-r--r--src/mongo/db/concurrency/locker_noop.h5
-rw-r--r--src/mongo/db/repl/sync_tail.cpp2
-rw-r--r--src/mongo/db/stats/fill_locker_info.cpp28
-rw-r--r--src/mongo/db/stats/fill_locker_info.h1
10 files changed, 168 insertions, 140 deletions
diff --git a/src/mongo/db/concurrency/d_concurrency.cpp b/src/mongo/db/concurrency/d_concurrency.cpp
index b1eaa7ddc30..ddee7e74e33 100644
--- a/src/mongo/db/concurrency/d_concurrency.cpp
+++ b/src/mongo/db/concurrency/d_concurrency.cpp
@@ -44,31 +44,8 @@ namespace {
// SERVER-14668: Remove or invert sense once MMAPv1 CLL can be default
MONGO_EXPORT_STARTUP_SERVER_PARAMETER(enableCollectionLocking, bool, true);
-
-
- class AcquiringParallelWriter {
- public:
-
- AcquiringParallelWriter(Locker* ls)
- : _ls(ls) {
-
- _ls->setLockPendingParallelWriter(true);
- }
-
- ~AcquiringParallelWriter() {
- _ls->setLockPendingParallelWriter(false);
- }
-
- private:
- Locker* const _ls;
- };
-
} // namespace
-
- RWLockRecursive Lock::ParallelBatchWriterMode::_batchLock("special");
-
-
Lock::TempRelease::TempRelease(Locker* lockState)
: _lockState(lockState),
_lockSnapshot(),
@@ -83,11 +60,23 @@ namespace {
}
}
+ Lock::GlobalLock::GlobalLock(Locker* locker)
+ : _locker(locker),
+ _result(LOCK_INVALID),
+ _pbwm(locker, resourceIdParallelBatchWriterMode) { }
+
+ Lock::GlobalLock::GlobalLock(Locker* locker, LockMode lockMode, unsigned timeoutMs)
+ : _locker(locker),
+ _result(LOCK_INVALID),
+ _pbwm(locker, resourceIdParallelBatchWriterMode) {
+ _lock(lockMode, timeoutMs);
+ }
+
+
void Lock::GlobalLock::_lock(LockMode lockMode, unsigned timeoutMs) {
if (!_locker->isBatchWriter()) {
- AcquiringParallelWriter a(_locker);
- _pbws_lk.reset(new RWLockRecursive::Shared(ParallelBatchWriterMode::_batchLock));
+ _pbwm.lock(MODE_IS);
}
_result = _locker->lockGlobalBegin(lockMode);
@@ -95,15 +84,14 @@ namespace {
_result = _locker->lockGlobalComplete(timeoutMs);
}
- if (_result != LOCK_OK) {
- _pbws_lk.reset();
+ if (_result != LOCK_OK && !_locker->isBatchWriter()) {
+ _pbwm.unlock();
}
}
void Lock::GlobalLock::_unlock() {
if (isLocked()) {
_locker->unlockAll();
- _pbws_lk.reset();
_result = LOCK_INVALID;
}
}
@@ -218,10 +206,13 @@ namespace {
}
}
+ Lock::ParallelBatchWriterMode::ParallelBatchWriterMode(Locker* lockState)
+ : _pbwm(lockState, resourceIdParallelBatchWriterMode, MODE_X) { }
void Lock::ResourceLock::lock(LockMode mode) {
invariant(_result == LOCK_INVALID);
_result = _locker->lock(_rid, mode);
+ invariant(_result == LOCK_OK);
}
void Lock::ResourceLock::unlock() {
diff --git a/src/mongo/db/concurrency/d_concurrency.h b/src/mongo/db/concurrency/d_concurrency.h
index 6569aae2b10..2bfe80b844d 100644
--- a/src/mongo/db/concurrency/d_concurrency.h
+++ b/src/mongo/db/concurrency/d_concurrency.h
@@ -64,20 +64,44 @@ namespace mongo {
};
- /** turn on "parallel batch writer mode". blocks all other threads. this mode is off
- by default. note only one thread creates a ParallelBatchWriterMode object; the rest just
- call iAmABatchParticipant(). Note that this lock is not released on a temprelease, just
- the normal lock things below.
- */
- class ParallelBatchWriterMode {
- MONGO_DISALLOW_COPYING(ParallelBatchWriterMode);
+ /**
+ * General purpose RAII wrapper for a resource managed by the lock manager
+ *
+ * See LockMode for the supported modes. Unlike DBLock/Collection lock, this will not do
+ * any additional checks/upgrades or global locking. Use ResourceLock for locking
+ * resources other than RESOURCE_GLOBAL, RESOURCE_DATABASE and RESOURCE_COLLECTION.
+ */
+ class ResourceLock {
+ MONGO_DISALLOW_COPYING(ResourceLock);
+
public:
- ParallelBatchWriterMode() : _lk(_batchLock) { }
+ ResourceLock(Locker* locker, ResourceId rid)
+ : _rid(rid),
+ _locker(locker),
+ _result(LOCK_INVALID) {
+ }
+
+ ResourceLock(Locker* locker, ResourceId rid, LockMode mode)
+ : _rid(rid),
+ _locker(locker),
+ _result(LOCK_INVALID) {
+ lock(mode);
+ }
+
+ ~ResourceLock() {
+ unlock();
+ }
- static RWLockRecursive _batchLock;
+ void lock(LockMode mode);
+ void unlock();
+
+ bool isLocked() const { return _result == LOCK_OK; }
private:
- RWLockRecursive::Exclusive _lk;
+ const ResourceId _rid;
+ Locker* const _locker;
+
+ LockResult _result;
};
@@ -90,14 +114,8 @@ namespace mongo {
*/
class GlobalLock {
public:
- explicit GlobalLock(Locker* locker) : _locker(locker), _result(LOCK_INVALID) { }
-
- GlobalLock(Locker* locker, LockMode lockMode, unsigned timeoutMs)
- : _locker(locker),
- _result(LOCK_INVALID) {
-
- _lock(lockMode, timeoutMs);
- }
+ explicit GlobalLock(Locker* locker);
+ GlobalLock(Locker* locker, LockMode lockMode, unsigned timeoutMs);
~GlobalLock() {
_unlock();
@@ -112,8 +130,7 @@ namespace mongo {
Locker* const _locker;
LockResult _result;
-
- boost::scoped_ptr<RWLockRecursive::Shared> _pbws_lk;
+ ResourceLock _pbwm;
};
@@ -249,46 +266,21 @@ namespace mongo {
bool _serialized;
};
+
/**
- * General purpose RAII wrapper for a resource managed by the lock manager
- *
- * See LockMode for the supported modes. Unlike DBLock/Collection lock, this will not do
- * any additional checks/upgrades or global locking. Use ResourceLock for locking
- * resources other than RESOURCE_GLOBAL, RESOURCE_DATABASE and RESOURCE_COLLECTION.
+ * Turn on "parallel batch writer mode" by locking the global ParallelBatchWriterMode
+ * resource in exclusive mode. This mode is off by default.
+ * Note that only one thread creates a ParallelBatchWriterMode object; the other batch
+ * writers just call setIsBatchWriter().
*/
- class ResourceLock {
- MONGO_DISALLOW_COPYING(ResourceLock);
- public:
- ResourceLock(Locker* locker, ResourceId rid)
- : _rid(rid),
- _locker(locker),
- _result(LOCK_INVALID) {
-
- }
-
- ResourceLock(Locker* locker, ResourceId rid, LockMode mode)
- : _rid(rid),
- _locker(locker),
- _result(LOCK_INVALID) {
-
- lock(mode);
- }
-
- ~ResourceLock() {
- unlock();
- }
-
- void lock(LockMode mode);
- void unlock();
+ class ParallelBatchWriterMode {
+ MONGO_DISALLOW_COPYING(ParallelBatchWriterMode);
- bool isLocked() const { return _result == LOCK_OK; }
+ public:
+ explicit ParallelBatchWriterMode(Locker* lockState);
private:
- const ResourceId _rid;
- Locker* const _locker;
-
- LockResult _result;
+ ResourceLock _pbwm;
};
-
};
}
diff --git a/src/mongo/db/concurrency/lock_manager_defs.h b/src/mongo/db/concurrency/lock_manager_defs.h
index 9c646fafd47..33cc0279ea9 100644
--- a/src/mongo/db/concurrency/lock_manager_defs.h
+++ b/src/mongo/db/concurrency/lock_manager_defs.h
@@ -136,16 +136,21 @@ namespace mongo {
/**
* Hierarchy of resource types. The lock manager knows nothing about this hierarchy, it is
- * purely logical. Resources of different types will never conflict with each other. While the
- * lock manager does not know or care about ordering, the general policy is that resources are
- * acquired in the order below. For example, one might first acquire a RESOURCE_GLOBAL and then
- * the desired RESOURCE_DATABASE, both using intent modes, and finally a RESOURCE_COLLECTION
- * in exclusive mode.
+ * purely logical. Resources of different types will never conflict with each other.
+ *
+ * While the lock manager does not know or care about ordering, the general policy is that
+ * resources are acquired in the order below. For example, one might first acquire a
+ * RESOURCE_GLOBAL and then the desired RESOURCE_DATABASE, both using intent modes, and
+ * finally a RESOURCE_COLLECTION in exclusive mode. When locking multiple resources of the
+ * same type, the canonical order is by resourceId order.
+ *
+ * It is OK to lock resources out of order, but it is the users responsibility to ensure
+ * ordering is consistent so deadlock cannot occur.
*/
enum ResourceType {
- // Special (singleton) resources
+ // Types used for special resources, use with a hash id from ResourceId::SingletonHashIds.
RESOURCE_INVALID = 0,
- RESOURCE_GLOBAL,
+ RESOURCE_GLOBAL, // Used for mode changes or global exclusive operations
RESOURCE_MMAPV1_FLUSH, // Necessary only for the MMAPv1 engine
// Generic resources
@@ -171,6 +176,17 @@ namespace mongo {
BOOST_STATIC_ASSERT(ResourceTypesCount <= (1 << resourceTypeBits));
public:
+ /**
+ * Assign hash ids for special resources to avoid accidental reuse of ids. For ids used
+ * with the same ResourceType, the order here must be the same as the locking order.
+ */
+ enum SingletonHashIds {
+ SINGLETON_INVALID = 0,
+ SINGLETON_PARALLEL_BATCH_WRITER_MODE,
+ SINGLETON_GLOBAL,
+ SINGLETON_MMAPV1_FLUSH
+ };
+
ResourceId() : _fullHash(0) { }
ResourceId(ResourceType type, StringData ns);
ResourceId(ResourceType type, const std::string& ns);
@@ -182,6 +198,11 @@ namespace mongo {
return _fullHash;
}
+ // This defines the canonical locking order, first by type and then hash id
+ bool operator<(const ResourceId& rhs) const {
+ return _fullHash < rhs._fullHash;
+ }
+
ResourceType getType() const {
return static_cast<ResourceType>(_fullHash >> (64 - resourceTypeBits));
}
@@ -193,7 +214,6 @@ namespace mongo {
std::string toString() const;
private:
-
/**
* The top 'resourceTypeBits' bits of '_fullHash' represent the resource type,
* while the remaining bits contain the bottom bits of the hashId. This avoids false
@@ -229,6 +249,15 @@ namespace mongo {
// are serialized (see SERVER-16092)
extern const ResourceId resourceIdAdminDB;
+ // Hardcoded resource id for ParallelBatchWriterMode. We use the same resource type
+ // as resourceIdGlobal. This will also ensure the waits are reported as global, which
+ // is appropriate. The lock will never be contended unless the parallel batch writers
+ // must stop all other accesses globally. This resource must be locked before all other
+ // resources (including resourceIdGlobal). Replication applier threads don't take this
+ // lock.
+ // TODO: Merge this with resourceIdGlobal
+ extern const ResourceId resourceIdParallelBatchWriterMode;
+
/**
* Interface on which granted lock requests will be notified. See the contract for the notify
* method for more information and also the LockManager::lock call.
diff --git a/src/mongo/db/concurrency/lock_state.cpp b/src/mongo/db/concurrency/lock_state.cpp
index 423887d4f44..d5d5654fbc5 100644
--- a/src/mongo/db/concurrency/lock_state.cpp
+++ b/src/mongo/db/concurrency/lock_state.cpp
@@ -32,6 +32,8 @@
#include "mongo/db/concurrency/lock_state.h"
+#include <vector>
+
#include "mongo/db/service_context.h"
#include "mongo/db/namespace_string.h"
#include "mongo/platform/compiler.h"
@@ -101,29 +103,19 @@ namespace {
};
- /**
- * Used to sort locks by granularity when snapshotting lock state. We must report and reacquire
- * locks in the same granularity in which they are acquired (i.e. global, flush, database,
- * collection, etc).
- */
- struct SortByGranularity {
- inline bool operator()(const Locker::OneLock& lhs, const Locker::OneLock& rhs) const {
- return lhs.resourceId.getType() < rhs.resourceId.getType();
- }
- };
-
-
// Global lock manager instance.
LockManager globalLockManager;
// Global lock. Every server operation, which uses the Locker must acquire this lock at least
// once. See comments in the header file (begin/endTransaction) for more information.
- const ResourceId resourceIdGlobal = ResourceId(RESOURCE_GLOBAL, 1ULL);
+ const ResourceId resourceIdGlobal = ResourceId(RESOURCE_GLOBAL,
+ ResourceId::SINGLETON_GLOBAL);
// Flush lock. This is only used for the MMAP V1 storage engine and synchronizes journal writes
// to the shared view and remaps. See the comments in the header for information on how MMAP V1
// concurrency control works.
- const ResourceId resourceIdMMAPV1Flush = ResourceId(RESOURCE_MMAPV1_FLUSH, 2ULL);
+ const ResourceId resourceIdMMAPV1Flush = ResourceId(RESOURCE_MMAPV1_FLUSH,
+ ResourceId::SINGLETON_MMAPV1_FLUSH);
// How often (in millis) to check for deadlock if a lock has not been granted for some time
const unsigned DeadlockTimeoutMs = 500;
@@ -143,7 +135,7 @@ namespace {
bool shouldDelayUnlock(ResourceId resId, LockMode mode) {
// Global and flush lock are not used to protect transactional resources and as such, they
// need to be acquired and released when requested.
- if (resId == resourceIdGlobal) {
+ if (resId.getType() == RESOURCE_GLOBAL) {
return false;
}
@@ -264,9 +256,7 @@ namespace {
: _id(idCounter.addAndFetch(1)),
_requestStartTime(0),
_wuowNestingLevel(0),
- _batchWriter(false),
- _lockPendingParallelWriter(false) {
-
+ _batchWriter(false) {
}
template<bool IsForMMAPV1>
@@ -353,7 +343,12 @@ namespace {
// If we're here we should only have one reference to any lock. It is a programming
// error for any lock to have more references than the global lock, because every
// scope starts by calling lockGlobal.
- invariant(_unlockImpl(it));
+ if (it.key().getType() == RESOURCE_GLOBAL) {
+ it.next();
+ }
+ else {
+ invariant(_unlockImpl(it));
+ }
}
return true;
@@ -515,7 +510,7 @@ namespace {
}
_lock.unlock();
- std::sort(lockerInfo->locks.begin(), lockerInfo->locks.end(), SortByGranularity());
+ std::sort(lockerInfo->locks.begin(), lockerInfo->locks.end());
lockerInfo->waitingResource = getWaitingResource();
lockerInfo->stats.append(_stats);
@@ -557,7 +552,8 @@ namespace {
// We should never have to save and restore metadata locks.
invariant((IsForMMAPV1 && (resourceIdMMAPV1Flush == resId)) ||
RESOURCE_DATABASE == resId.getType() ||
- RESOURCE_COLLECTION == resId.getType());
+ RESOURCE_COLLECTION == resId.getType() ||
+ (RESOURCE_GLOBAL == resId.getType() && isSharedLockMode(it->mode)));
// And, stuff the info into the out parameter.
OneLock info;
@@ -569,8 +565,8 @@ namespace {
invariant(unlock(resId));
}
- // Sort locks from coarsest to finest. They'll later be acquired in this order.
- std::sort(stateOut->locks.begin(), stateOut->locks.end(), SortByGranularity());
+ // Sort locks by ResourceId. They'll later be acquired in this canonical locking order.
+ std::sort(stateOut->locks.begin(), stateOut->locks.end());
return true;
}
@@ -580,9 +576,14 @@ namespace {
// We shouldn't be saving and restoring lock state from inside a WriteUnitOfWork.
invariant(!inAWriteUnitOfWork());
- invariant(LOCK_OK == lockGlobal(state.globalMode));
-
std::vector<OneLock>::const_iterator it = state.locks.begin();
+ // If we locked the PBWM, it must be locked before the resourceIdGlobal resource.
+ if (it != state.locks.end() && it->resourceId == resourceIdParallelBatchWriterMode) {
+ invariant(LOCK_OK == lock(it->resourceId, it->mode));
+ it++;
+ }
+
+ invariant(LOCK_OK == lockGlobal(state.globalMode));
for (; it != state.locks.end(); it++) {
// This is a sanity check that lockGlobal restored the MMAP V1 flush lock in the
// expected mode.
@@ -619,9 +620,10 @@ namespace {
globalStats.recordAcquisition(_id, resId, mode);
_stats.recordAcquisition(resId, mode);
- // Give priority to the full modes for global and flush lock so we don't stall global
- // operations such as shutdown or flush.
- if (resId == resourceIdGlobal || (IsForMMAPV1 && resId == resourceIdMMAPV1Flush)) {
+ // Give priority to the full modes for global, parallel batch writer mode,
+ // and flush lock so we don't stall global operations such as shutdown or flush.
+ const ResourceType resType = resId.getType();
+ if (resType == RESOURCE_GLOBAL || (IsForMMAPV1 && resId == resourceIdMMAPV1Flush)) {
if (mode == MODE_S || mode == MODE_X) {
request->enqueueAtFront = true;
request->compatibleFirst = true;
@@ -669,7 +671,7 @@ namespace {
// correct to do if not in a write unit of work.
const bool yieldFlushLock = IsForMMAPV1 &&
!inAWriteUnitOfWork() &&
- resId != resourceIdGlobal &&
+ resId.getType() != RESOURCE_GLOBAL &&
resId != resourceIdMMAPV1Flush;
if (yieldFlushLock) {
invariant(unlock(resourceIdMMAPV1Flush));
@@ -915,5 +917,7 @@ namespace {
const ResourceId resourceIdOplog =
ResourceId(RESOURCE_COLLECTION, StringData("local.oplog.rs"));
const ResourceId resourceIdAdminDB = ResourceId(RESOURCE_DATABASE, StringData("admin"));
+ const ResourceId resourceIdParallelBatchWriterMode =
+ ResourceId(RESOURCE_GLOBAL, ResourceId::SINGLETON_PARALLEL_BATCH_WRITER_MODE);
} // namespace mongo
diff --git a/src/mongo/db/concurrency/lock_state.h b/src/mongo/db/concurrency/lock_state.h
index 67ec8113310..8ea914ffbeb 100644
--- a/src/mongo/db/concurrency/lock_state.h
+++ b/src/mongo/db/concurrency/lock_state.h
@@ -236,20 +236,15 @@ namespace mongo {
virtual void assertEmptyAndReset();
- virtual bool hasLockPending() const { return getWaitingResource().isValid() || _lockPendingParallelWriter; }
+ virtual bool hasLockPending() const { return getWaitingResource().isValid(); }
virtual void setIsBatchWriter(bool newValue) { _batchWriter = newValue; }
virtual bool isBatchWriter() const { return _batchWriter; }
- virtual void setLockPendingParallelWriter(bool newValue) {
- _lockPendingParallelWriter = newValue;
- }
virtual bool hasStrongLocks() const;
private:
-
bool _batchWriter;
- bool _lockPendingParallelWriter;
};
typedef LockerImpl<false> DefaultLockerImpl;
diff --git a/src/mongo/db/concurrency/locker.h b/src/mongo/db/concurrency/locker.h
index d88042f0d69..304b92bea9c 100644
--- a/src/mongo/db/concurrency/locker.h
+++ b/src/mongo/db/concurrency/locker.h
@@ -190,6 +190,11 @@ namespace mongo {
// In what mode is it held?
LockMode mode;
+
+ // Reporting/serialization order is by resourceId, which is the canonical locking order
+ bool operator<(const OneLock& rhs) const {
+ return resourceId < rhs.resourceId;
+ }
};
/**
@@ -199,8 +204,7 @@ namespace mongo {
* reused.
*/
struct LockerInfo {
- // List of high-level locks held by this locker, sorted by hierarchy in the order
- // Global, Flush (MMAP V1 only), Database, Collection.
+ // List of high-level locks held by this locker, sorted by ResourceId
std::vector<OneLock> locks;
// If isValid(), then what lock this particular locker is sleeping on
@@ -278,7 +282,6 @@ namespace mongo {
// Used for the replication parallel log op application threads
virtual void setIsBatchWriter(bool newValue) = 0;
virtual bool isBatchWriter() const = 0;
- virtual void setLockPendingParallelWriter(bool newValue) = 0;
/**
* A string lock is MODE_X or MODE_S.
diff --git a/src/mongo/db/concurrency/locker_noop.h b/src/mongo/db/concurrency/locker_noop.h
index be834cbc71f..9bfcbc93227 100644
--- a/src/mongo/db/concurrency/locker_noop.h
+++ b/src/mongo/db/concurrency/locker_noop.h
@@ -166,14 +166,9 @@ namespace mongo {
invariant(false);
}
- virtual void setLockPendingParallelWriter(bool newValue) {
- invariant(false);
- }
-
virtual bool hasStrongLocks() const {
return false;
}
-
};
} // namespace mongo
diff --git a/src/mongo/db/repl/sync_tail.cpp b/src/mongo/db/repl/sync_tail.cpp
index f5e3551a0ac..aa239b0490c 100644
--- a/src/mongo/db/repl/sync_tail.cpp
+++ b/src/mongo/db/repl/sync_tail.cpp
@@ -295,7 +295,7 @@ namespace repl {
SimpleMutex::scoped_lock fsynclk(filesLockedFsync);
// stop all readers until we're done
- Lock::ParallelBatchWriterMode pbwm;
+ Lock::ParallelBatchWriterMode pbwm(txn->lockState());
ReplicationCoordinator* replCoord = getGlobalReplicationCoordinator();
if (replCoord->getMemberState().primary() &&
diff --git a/src/mongo/db/stats/fill_locker_info.cpp b/src/mongo/db/stats/fill_locker_info.cpp
index 1bdcb2762f1..6193f91637c 100644
--- a/src/mongo/db/stats/fill_locker_info.cpp
+++ b/src/mongo/db/stats/fill_locker_info.cpp
@@ -26,24 +26,42 @@
* it in the license file.
*/
+#include "mongo/db/stats/fill_locker_info.h"
+
+#include <algorithm>
+
#include "mongo/db/concurrency/locker.h"
#include "mongo/db/jsobj.h"
-#include "mongo/db/stats/fill_locker_info.h"
namespace mongo {
void fillLockerInfo(const Locker::LockerInfo& lockerInfo, BSONObjBuilder& infoBuilder) {
// "locks" section
BSONObjBuilder locks(infoBuilder.subobjStart("locks"));
- for (size_t i = 0; i < lockerInfo.locks.size(); i++) {
+ const size_t locksSize = lockerInfo.locks.size();
+
+ // Only add the last lock of each type, and use the largest mode encountered
+ LockMode modeForType[LockModesCount] = { }; // default initialize to zero (min value)
+ for (size_t i = 0; i < locksSize; i++) {
const Locker::OneLock& lock = lockerInfo.locks[i];
+ const ResourceType lockType = lock.resourceId.getType();
+ const LockMode lockMode = std::max(lock.mode, modeForType[lockType]);
- if (resourceIdLocalDB == lock.resourceId) {
+ // Check that lockerInfo is sorted on resource type
+ invariant(i == 0 || lockType >= lockerInfo.locks[i - 1].resourceId.getType());
+
+ if (lock.resourceId == resourceIdLocalDB) {
locks.append("local", legacyModeName(lock.mode));
+ continue;
+ }
+
+ modeForType[lockType] = lockMode;
+
+ if (i + 1 < locksSize && lockerInfo.locks[i + 1].resourceId.getType() == lockType) {
+ continue; // skip this lock as it is not the last one of its type
}
else {
- locks.append(resourceTypeName(lock.resourceId.getType()),
- legacyModeName(lock.mode));
+ locks.append(resourceTypeName(lockType), legacyModeName(lockMode));
}
}
locks.done();
diff --git a/src/mongo/db/stats/fill_locker_info.h b/src/mongo/db/stats/fill_locker_info.h
index b4743764e25..440b91b2816 100644
--- a/src/mongo/db/stats/fill_locker_info.h
+++ b/src/mongo/db/stats/fill_locker_info.h
@@ -34,6 +34,7 @@ namespace mongo {
/**
* Constructs a human-readable BSON from the specified LockerInfo structure.
+ * The lockerInfo must be sorted.
*/
void fillLockerInfo(const Locker::LockerInfo& lockerInfo, BSONObjBuilder& infoBuilder);