summaryrefslogtreecommitdiff
diff options
context:
space:
mode:
authorNathan Myers <ncm@cantrip.org>2017-08-18 13:42:27 -0400
committerNathan Myers <nathan.myers@10gen.com>2017-08-25 13:39:56 -0400
commit529d5de71344fda500802fa4a8671c5745ad62fa (patch)
treee607a426dcc576ecab61c9e7437fc031d9986de0
parent67aaaf319b637024ef053ca81b3f945224a88759 (diff)
downloadmongo-529d5de71344fda500802fa4a8671c5745ad62fa.tar.gz
SERVER-30748 WithLock to replace _inlock
-rw-r--r--src/mongo/db/s/balancer/migration_manager.cpp26
-rw-r--r--src/mongo/db/s/balancer/migration_manager.h17
-rw-r--r--src/mongo/db/s/collection_range_deleter.cpp3
-rw-r--r--src/mongo/db/s/metadata_manager.cpp70
-rw-r--r--src/mongo/db/s/metadata_manager.h29
-rw-r--r--src/mongo/db/s/migration_destination_manager.cpp6
-rw-r--r--src/mongo/db/s/migration_destination_manager.h5
-rw-r--r--src/mongo/util/concurrency/with_lock.h98
8 files changed, 179 insertions, 75 deletions
diff --git a/src/mongo/db/s/balancer/migration_manager.cpp b/src/mongo/db/s/balancer/migration_manager.cpp
index 9552c959323..cd2d1baa30e 100644
--- a/src/mongo/db/s/balancer/migration_manager.cpp
+++ b/src/mongo/db/s/balancer/migration_manager.cpp
@@ -399,7 +399,7 @@ void MigrationManager::interruptAndDisableMigrations() {
}
}
- _checkDrained_inlock();
+ _checkDrained(lock);
}
void MigrationManager::drainActiveMigrations() {
@@ -470,14 +470,15 @@ shared_ptr<Notification<RemoteCommandResponse>> MigrationManager::_schedule(
auto retVal = migration.completionNotification;
- _schedule_inlock(opCtx, fromHostStatus.getValue(), std::move(migration));
+ _schedule(lock, opCtx, fromHostStatus.getValue(), std::move(migration));
return retVal;
}
-void MigrationManager::_schedule_inlock(OperationContext* opCtx,
- const HostAndPort& targetHost,
- Migration migration) {
+void MigrationManager::_schedule(WithLock lock,
+ OperationContext* opCtx,
+ const HostAndPort& targetHost,
+ Migration migration) {
executor::TaskExecutor* const executor =
Grid::get(opCtx)->getExecutorPool()->getFixedExecutor();
@@ -526,7 +527,7 @@ void MigrationManager::_schedule_inlock(OperationContext* opCtx,
auto opCtx = cc().makeOperationContext();
stdx::lock_guard<stdx::mutex> lock(_mutex);
- _complete_inlock(opCtx.get(), itMigration, args.response);
+ _complete(lock, opCtx.get(), itMigration, args.response);
});
if (callbackHandleWithStatus.isOK()) {
@@ -534,12 +535,13 @@ void MigrationManager::_schedule_inlock(OperationContext* opCtx,
return;
}
- _complete_inlock(opCtx, itMigration, std::move(callbackHandleWithStatus.getStatus()));
+ _complete(lock, opCtx, itMigration, std::move(callbackHandleWithStatus.getStatus()));
}
-void MigrationManager::_complete_inlock(OperationContext* opCtx,
- MigrationsList::iterator itMigration,
- const RemoteCommandResponse& remoteCommandResponse) {
+void MigrationManager::_complete(WithLock lock,
+ OperationContext* opCtx,
+ MigrationsList::iterator itMigration,
+ const RemoteCommandResponse& remoteCommandResponse) {
const NamespaceString nss(itMigration->nss);
// Make sure to signal the notification last, after the distributed lock is freed, so that we
@@ -557,13 +559,13 @@ void MigrationManager::_complete_inlock(OperationContext* opCtx,
Grid::get(opCtx)->catalogClient()->getDistLockManager()->unlock(
opCtx, _lockSessionID, nss.ns());
_activeMigrations.erase(it);
- _checkDrained_inlock();
+ _checkDrained(lock);
}
notificationToSignal->set(remoteCommandResponse);
}
-void MigrationManager::_checkDrained_inlock() {
+void MigrationManager::_checkDrained(WithLock) {
if (_state == State::kEnabled || _state == State::kRecovering) {
return;
}
diff --git a/src/mongo/db/s/balancer/migration_manager.h b/src/mongo/db/s/balancer/migration_manager.h
index 26da3f057f7..a99b5696d80 100644
--- a/src/mongo/db/s/balancer/migration_manager.h
+++ b/src/mongo/db/s/balancer/migration_manager.h
@@ -44,6 +44,7 @@
#include "mongo/stdx/mutex.h"
#include "mongo/stdx/unordered_map.h"
#include "mongo/util/concurrency/notification.h"
+#include "mongo/util/concurrency/with_lock.h"
namespace mongo {
@@ -194,9 +195,10 @@ private:
* The distributed lock is acquired before scheduling the first migration for the collection and
* is only released when all active migrations on the collection have finished.
*/
- void _schedule_inlock(OperationContext* opCtx,
- const HostAndPort& targetHost,
- Migration migration);
+ void _schedule(WithLock,
+ OperationContext* opCtx,
+ const HostAndPort& targetHost,
+ Migration migration);
/**
* Used internally for migrations scheduled with the distributed lock acquired by the config
@@ -204,15 +206,16 @@ private:
* passed iterator and if this is the last migration for the collection will free the collection
* distributed lock.
*/
- void _complete_inlock(OperationContext* opCtx,
- MigrationsList::iterator itMigration,
- const executor::RemoteCommandResponse& remoteCommandResponse);
+ void _complete(WithLock,
+ OperationContext* opCtx,
+ MigrationsList::iterator itMigration,
+ const executor::RemoteCommandResponse& remoteCommandResponse);
/**
* If the state of the migration manager is kStopping, checks whether there are any outstanding
* scheduled requests and if there aren't any signals the class condition variable.
*/
- void _checkDrained_inlock();
+ void _checkDrained(WithLock);
/**
* Blocking call, which waits for the migration manager to leave the recovering state (if it is
diff --git a/src/mongo/db/s/collection_range_deleter.cpp b/src/mongo/db/s/collection_range_deleter.cpp
index cafc540ebb3..2e17d047d5d 100644
--- a/src/mongo/db/s/collection_range_deleter.cpp
+++ b/src/mongo/db/s/collection_range_deleter.cpp
@@ -105,7 +105,7 @@ auto CollectionRangeDeleter::cleanUpNextRange(OperationContext* opCtx,
<< nss.ns();
}
stdx::lock_guard<stdx::mutex> lk(css->_metadataManager->_managerLock);
- css->_metadataManager->_clearAllCleanups();
+ css->_metadataManager->_clearAllCleanups(lk);
return boost::none;
}
if (!forTestOnly && scopedCollectionMetadata->getCollVersion().epoch() != epoch) {
@@ -160,6 +160,7 @@ auto CollectionRangeDeleter::cleanUpNextRange(OperationContext* opCtx,
} catch (DBException const& e) {
stdx::lock_guard<stdx::mutex> scopedLock(css->_metadataManager->_managerLock);
css->_metadataManager->_clearAllCleanups(
+ scopedLock,
{e.code(),
str::stream() << "cannot push startRangeDeletion record to Op Log,"
" abandoning scheduled range deletions: " << e.what()});
diff --git a/src/mongo/db/s/metadata_manager.cpp b/src/mongo/db/s/metadata_manager.cpp
index cb0c6259bc8..c7d5c74052e 100644
--- a/src/mongo/db/s/metadata_manager.cpp
+++ b/src/mongo/db/s/metadata_manager.cpp
@@ -126,18 +126,19 @@ MetadataManager::MetadataManager(ServiceContext* sc, NamespaceString nss, TaskEx
MetadataManager::~MetadataManager() {
stdx::lock_guard<stdx::mutex> scopedLock(_managerLock);
- _clearAllCleanups();
+ _clearAllCleanups(scopedLock);
auto metadata = std::move(_metadata);
}
-void MetadataManager::_clearAllCleanups() {
+void MetadataManager::_clearAllCleanups(WithLock lock) {
_clearAllCleanups(
+ lock,
{ErrorCodes::InterruptedDueToReplStateChange,
str::stream() << "Range deletions in " << _nss.ns()
<< " abandoned because collection was dropped or became unsharded"});
}
-void MetadataManager::_clearAllCleanups(Status status) {
+void MetadataManager::_clearAllCleanups(WithLock, Status status) {
for (auto& metadata : _metadata) {
std::ignore = _rangesToClean.add(std::move(metadata->_tracker.orphans));
}
@@ -147,7 +148,7 @@ void MetadataManager::_clearAllCleanups(Status status) {
ScopedCollectionMetadata MetadataManager::getActiveMetadata(std::shared_ptr<MetadataManager> self) {
stdx::lock_guard<stdx::mutex> scopedLock(_managerLock);
if (!_metadata.empty()) {
- return ScopedCollectionMetadata(std::move(self), _metadata.back());
+ return ScopedCollectionMetadata(scopedLock, std::move(self), _metadata.back());
}
return ScopedCollectionMetadata();
}
@@ -175,7 +176,7 @@ void MetadataManager::refreshActiveMetadata(std::unique_ptr<CollectionMetadata>
<< " as no longer sharded";
_receivingChunks.clear();
- _clearAllCleanups();
+ _clearAllCleanups(scopedLock);
_metadata.clear();
return;
}
@@ -192,7 +193,7 @@ void MetadataManager::refreshActiveMetadata(std::unique_ptr<CollectionMetadata>
invariant(_receivingChunks.empty());
invariant(_rangesToClean.isEmpty());
- _setActiveMetadata_inlock(std::move(remoteMetadata));
+ _setActiveMetadata(scopedLock, std::move(remoteMetadata));
return;
}
@@ -206,8 +207,8 @@ void MetadataManager::refreshActiveMetadata(std::unique_ptr<CollectionMetadata>
<< " due to epoch change";
_receivingChunks.clear();
- _setActiveMetadata_inlock(std::move(remoteMetadata));
- _clearAllCleanups();
+ _setActiveMetadata(scopedLock, std::move(remoteMetadata));
+ _clearAllCleanups(scopedLock);
return;
}
@@ -240,16 +241,17 @@ void MetadataManager::refreshActiveMetadata(std::unique_ptr<CollectionMetadata>
it = _receivingChunks.begin();
}
- _setActiveMetadata_inlock(std::move(remoteMetadata));
+ _setActiveMetadata(scopedLock, std::move(remoteMetadata));
}
-void MetadataManager::_setActiveMetadata_inlock(std::unique_ptr<CollectionMetadata> newMetadata) {
+void MetadataManager::_setActiveMetadata(WithLock lock,
+ std::unique_ptr<CollectionMetadata> newMetadata) {
invariant(newMetadata);
_metadata.push_back(std::move(newMetadata));
- _retireExpiredMetadata();
+ _retireExpiredMetadata(lock);
}
-void MetadataManager::_retireExpiredMetadata() {
+void MetadataManager::_retireExpiredMetadata(WithLock lock) {
if (_metadata.empty()) {
return; // The collection was dropped, or went unsharded, before the query was cleaned up.
}
@@ -260,7 +262,7 @@ void MetadataManager::_retireExpiredMetadata() {
<< " range(s) finished; scheduling ranges for deletion";
// It is safe to push orphan ranges from _metadata.back(), even though new queries might
// start any time, because any request to delete a range it maps is rejected.
- _pushListToClean(std::move(_metadata.front()->_tracker.orphans));
+ _pushListToClean(lock, std::move(_metadata.front()->_tracker.orphans));
}
if (&_metadata.front() == &_metadata.back())
break; // do not pop the active chunk mapping!
@@ -269,8 +271,8 @@ void MetadataManager::_retireExpiredMetadata() {
// ScopedCollectionMetadata members
-// call with MetadataManager locked
-ScopedCollectionMetadata::ScopedCollectionMetadata(std::shared_ptr<MetadataManager> manager,
+ScopedCollectionMetadata::ScopedCollectionMetadata(WithLock,
+ std::shared_ptr<MetadataManager> manager,
std::shared_ptr<CollectionMetadata> metadata)
: _metadata(std::move(metadata)), _manager(std::move(manager)) {
invariant(_metadata);
@@ -318,7 +320,7 @@ void ScopedCollectionMetadata::_clear() {
// zero, some go to zero but can't be expired yet.) Note that new instances of
// ScopedCollectionMetadata may get attached to _metadata.back(), so its usage count can
// increase from zero, unlike other reference counts.
- _manager->_retireExpiredMetadata();
+ _manager->_retireExpiredMetadata(managerLock);
}
_metadata.reset();
_manager.reset();
@@ -413,17 +415,17 @@ void scheduleCleanup(executor::TaskExecutor* executor,
} // namespace
-auto MetadataManager::_pushRangeToClean(ChunkRange const& range, Date_t when)
+auto MetadataManager::_pushRangeToClean(WithLock lock, ChunkRange const& range, Date_t when)
-> CleanupNotification {
std::list<Deletion> ranges;
auto ownedRange = ChunkRange{range.getMin().getOwned(), range.getMax().getOwned()};
ranges.emplace_back(Deletion{std::move(ownedRange), when});
auto& notifn = ranges.back().notification;
- _pushListToClean(std::move(ranges));
+ _pushListToClean(lock, std::move(ranges));
return notifn;
}
-void MetadataManager::_pushListToClean(std::list<Deletion> ranges) {
+void MetadataManager::_pushListToClean(WithLock, std::list<Deletion> ranges) {
auto when = _rangesToClean.add(std::move(ranges));
if (when) {
auto epoch = _metadata.back()->getCollVersion().epoch();
@@ -432,7 +434,7 @@ void MetadataManager::_pushListToClean(std::list<Deletion> ranges) {
invariant(ranges.empty());
}
-void MetadataManager::_addToReceiving(ChunkRange const& range) {
+void MetadataManager::_addToReceiving(WithLock, ChunkRange const& range) {
_receivingChunks.insert(
std::make_pair(range.getMin().getOwned(),
CachedChunkInfo(range.getMax().getOwned(), ChunkVersion::IGNORED())));
@@ -442,17 +444,17 @@ auto MetadataManager::beginReceive(ChunkRange const& range) -> CleanupNotificati
stdx::unique_lock<stdx::mutex> scopedLock(_managerLock);
invariant(!_metadata.empty());
- if (_overlapsInUseChunk(range)) {
+ if (_overlapsInUseChunk(scopedLock, range)) {
return Status{ErrorCodes::RangeOverlapConflict,
"Documents in target range may still be in use on the destination shard."};
}
- _addToReceiving(range);
+ _addToReceiving(scopedLock, range);
log() << "Scheduling deletion of any documents in " << _nss.ns() << " range "
<< redact(range.toString()) << " before migrating in a chunk covering the range";
- return _pushRangeToClean(range, Date_t{});
+ return _pushRangeToClean(scopedLock, range, Date_t{});
}
-void MetadataManager::_removeFromReceiving(ChunkRange const& range) {
+void MetadataManager::_removeFromReceiving(WithLock, ChunkRange const& range) {
auto it = _receivingChunks.find(range.getMin());
invariant(it != _receivingChunks.end());
_receivingChunks.erase(it);
@@ -467,9 +469,9 @@ void MetadataManager::forgetReceive(ChunkRange const& range) {
log() << "Abandoning in-migration of " << _nss.ns() << " range " << range
<< "; scheduling deletion of any documents already copied";
- invariant(!_overlapsInUseChunk(range));
- _removeFromReceiving(range);
- _pushRangeToClean(range, Date_t{}).abandon();
+ invariant(!_overlapsInUseChunk(scopedLock, range));
+ _removeFromReceiving(scopedLock, range);
+ _pushRangeToClean(scopedLock, range, Date_t{}).abandon();
}
auto MetadataManager::cleanUpRange(ChunkRange const& range, Date_t whenToDelete)
@@ -490,11 +492,11 @@ auto MetadataManager::cleanUpRange(ChunkRange const& range, Date_t whenToDelete)
}
StringData whenStr = (whenToDelete == Date_t{}) ? "immediate"_sd : "deferred"_sd;
- if (!_overlapsInUseChunk(range)) {
+ if (!_overlapsInUseChunk(scopedLock, range)) {
// No running queries can depend on it, so queue it for deletion immediately.
log() << "Scheduling " << whenStr << " deletion of " << _nss.ns() << " range "
<< redact(range.toString());
- return _pushRangeToClean(range, whenToDelete);
+ return _pushRangeToClean(scopedLock, range, whenToDelete);
}
auto ownedRange = ChunkRange{range.getMin().getOwned(), range.getMax().getOwned()};
activeMetadata->_tracker.orphans.emplace_back(Deletion{std::move(ownedRange), whenToDelete});
@@ -515,13 +517,13 @@ auto MetadataManager::overlappingMetadata(std::shared_ptr<MetadataManager> const
auto it = _metadata.crbegin(); // start with the current active chunk mapping
if ((*it)->rangeOverlapsChunk(range)) {
// We ignore the refcount of the active mapping; effectively, we assume it is in use.
- result.push_back(ScopedCollectionMetadata(self, *it));
+ result.push_back(ScopedCollectionMetadata(scopedLock, self, *it));
}
++it; // step to snapshots
for (auto end = _metadata.crend(); it != end; ++it) {
// We want all the overlapping snapshot mappings still possibly in use by a query.
if ((*it)->_tracker.usageCounter > 0 && (*it)->rangeOverlapsChunk(range)) {
- result.push_back(ScopedCollectionMetadata(self, *it));
+ result.push_back(ScopedCollectionMetadata(scopedLock, self, *it));
}
}
return result;
@@ -544,14 +546,14 @@ size_t MetadataManager::numberOfRangesToClean() {
auto MetadataManager::trackOrphanedDataCleanup(ChunkRange const& range)
-> boost::optional<CleanupNotification> {
stdx::unique_lock<stdx::mutex> scopedLock(_managerLock);
- auto overlaps = _overlapsInUseCleanups(range);
+ auto overlaps = _overlapsInUseCleanups(scopedLock, range);
if (overlaps) {
return overlaps;
}
return _rangesToClean.overlaps(range);
}
-bool MetadataManager::_overlapsInUseChunk(ChunkRange const& range) {
+bool MetadataManager::_overlapsInUseChunk(WithLock, ChunkRange const& range) {
invariant(!_metadata.empty());
for (auto it = _metadata.begin(), end = --_metadata.end(); it != end; ++it) {
if (((*it)->_tracker.usageCounter != 0) && (*it)->rangeOverlapsChunk(range)) {
@@ -564,7 +566,7 @@ bool MetadataManager::_overlapsInUseChunk(ChunkRange const& range) {
return false;
}
-auto MetadataManager::_overlapsInUseCleanups(ChunkRange const& range)
+auto MetadataManager::_overlapsInUseCleanups(WithLock, ChunkRange const& range)
-> boost::optional<CleanupNotification> {
invariant(!_metadata.empty());
diff --git a/src/mongo/db/s/metadata_manager.h b/src/mongo/db/s/metadata_manager.h
index 48cd08df125..42210d59154 100644
--- a/src/mongo/db/s/metadata_manager.h
+++ b/src/mongo/db/s/metadata_manager.h
@@ -41,6 +41,7 @@
#include "mongo/s/catalog/type_chunk.h"
#include "mongo/stdx/memory.h"
#include "mongo/util/concurrency/notification.h"
+#include "mongo/util/concurrency/with_lock.h"
namespace mongo {
@@ -141,65 +142,64 @@ public:
boost::optional<KeyRange> getNextOrphanRange(BSONObj const& from);
private:
- // All of the following functions must be called while holding _managerLock.
-
/**
* Cancels all scheduled deletions of orphan ranges, notifying listeners with specified status.
*/
- void _clearAllCleanups(Status);
+ void _clearAllCleanups(WithLock, Status);
/**
* Cancels all scheduled deletions of orphan ranges, notifying listeners with status
* InterruptedDueToReplStateChange.
*/
- void _clearAllCleanups();
+ void _clearAllCleanups(WithLock);
/**
* Retires any metadata that has fallen out of use, and pushes any orphan ranges found in them
* to the list of ranges actively being cleaned up.
*/
- void _retireExpiredMetadata();
+ void _retireExpiredMetadata(WithLock);
/**
* Pushes current set of chunks, if any, to _metadataInUse, replaces it with newMetadata.
*/
- void _setActiveMetadata_inlock(std::unique_ptr<CollectionMetadata> newMetadata);
+ void _setActiveMetadata(WithLock, std::unique_ptr<CollectionMetadata> newMetadata);
/**
* Returns true if the specified range overlaps any chunk that might be currently in use by a
* running query.
*/
- bool _overlapsInUseChunk(ChunkRange const& range);
+ bool _overlapsInUseChunk(WithLock, ChunkRange const& range);
/**
* Returns a notification if any range (possibly) still in use, but scheduled for cleanup,
* overlaps the argument range.
*/
- auto _overlapsInUseCleanups(ChunkRange const& range) -> boost::optional<CleanupNotification>;
+ auto _overlapsInUseCleanups(WithLock, ChunkRange const& range)
+ -> boost::optional<CleanupNotification>;
/**
* Copies the argument range to the list of ranges scheduled for immediate deletion, and
* schedules a a background task to perform the work.
*/
- auto _pushRangeToClean(ChunkRange const& range, Date_t when) -> CleanupNotification;
+ auto _pushRangeToClean(WithLock, ChunkRange const& range, Date_t when) -> CleanupNotification;
/**
* Splices the argument list elements to the list of ranges scheduled for immediate deletion,
* and schedules a a background task to perform the work.
*/
- void _pushListToClean(std::list<Deletion> range);
+ void _pushListToClean(WithLock, std::list<Deletion> range);
/**
* Adds a range from the receiving map, so getNextOrphanRange will skip ranges migrating in.
*/
- void _addToReceiving(ChunkRange const& range);
+ void _addToReceiving(WithLock, ChunkRange const& range);
/**
* Removes a range from the receiving map after a migration failure. The range must
* exactly match an element of _receivingChunks.
*/
- void _removeFromReceiving(ChunkRange const& range);
+ void _removeFromReceiving(WithLock, ChunkRange const& range);
// data members
@@ -288,13 +288,12 @@ private:
*
* Must be called with manager->_managerLock held. Arguments must be non-null.
*/
- ScopedCollectionMetadata(std::shared_ptr<MetadataManager> manager,
+ ScopedCollectionMetadata(WithLock,
+ std::shared_ptr<MetadataManager> manager,
std::shared_ptr<CollectionMetadata> metadata);
/**
* Disconnect from the CollectionMetadata, possibly triggering GC of unused CollectionMetadata.
- *
- * Must be called with manager->_managerLock held.
*/
void _clear();
diff --git a/src/mongo/db/s/migration_destination_manager.cpp b/src/mongo/db/s/migration_destination_manager.cpp
index d1e9b919f96..ca2c736a6b1 100644
--- a/src/mongo/db/s/migration_destination_manager.cpp
+++ b/src/mongo/db/s/migration_destination_manager.cpp
@@ -244,10 +244,10 @@ void MigrationDestinationManager::setStateFailWarn(std::string msg) {
bool MigrationDestinationManager::isActive() const {
stdx::lock_guard<stdx::mutex> lk(_mutex);
- return _isActive_inlock();
+ return _isActive(lk);
}
-bool MigrationDestinationManager::_isActive_inlock() const {
+bool MigrationDestinationManager::_isActive(WithLock) const {
return _sessionId.is_initialized();
}
@@ -283,7 +283,7 @@ void MigrationDestinationManager::report(BSONObjBuilder& b) {
BSONObj MigrationDestinationManager::getMigrationStatusReport() {
stdx::lock_guard<stdx::mutex> lk(_mutex);
- if (_isActive_inlock()) {
+ if (_isActive(lk)) {
return migrationutil::makeMigrationStatusDocument(
_nss, _fromShard, _toShard, false, _min, _max);
} else {
diff --git a/src/mongo/db/s/migration_destination_manager.h b/src/mongo/db/s/migration_destination_manager.h
index e88ad70f61f..610f70196c1 100644
--- a/src/mongo/db/s/migration_destination_manager.h
+++ b/src/mongo/db/s/migration_destination_manager.h
@@ -43,6 +43,7 @@
#include "mongo/stdx/condition_variable.h"
#include "mongo/stdx/mutex.h"
#include "mongo/stdx/thread.h"
+#include "mongo/util/concurrency/with_lock.h"
#include "mongo/util/timer.h"
namespace mongo {
@@ -174,10 +175,8 @@ private:
/**
* Checks whether the MigrationDestinationManager is currently handling a migration by checking
* that the migration "_sessionId" is initialized.
- *
- * Expects the caller to have the class _mutex locked!
*/
- bool _isActive_inlock() const;
+ bool _isActive(WithLock) const;
// Mutex to guard all fields
mutable stdx::mutex _mutex;
diff --git a/src/mongo/util/concurrency/with_lock.h b/src/mongo/util/concurrency/with_lock.h
new file mode 100644
index 00000000000..4bad7346637
--- /dev/null
+++ b/src/mongo/util/concurrency/with_lock.h
@@ -0,0 +1,98 @@
+/** Copyright 2017 MongoDB, Inc.
+ *
+ * This program is free software: you can redistribute it and/or modify
+ * it under the terms of the GNU Affero General Public License, version 3,
+ * as published by the Free Software Foundation.
+ *
+ * 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
+ * GNU Affero General Public License for more details.
+ *
+ * You should have received a copy of the GNU Affero General Public License
+ * along with this program. If not, see <http://www.gnu.org/licenses/>.
+ *
+ * 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 GNU Affero General 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/stdx/mutex.h"
+#include "mongo/util/assert_util.h"
+
+#include <utility>
+
+namespace mongo {
+
+/**
+ * WithLock is an attestation to pass as an argument to functions that must be called only while
+ * holding a lock, as a rigorous alternative to an unchecked naming convention and/or stern
+ * comments. It helps prevent a common usage error.
+ *
+ * It may be used to modernize code from (something like) this
+ *
+ * // Member _mutex MUST be held when calling this:
+ * void _clobber_inlock(OperationContext* opCtx) {
+ * _stuff = makeStuff(opCtx);
+ * }
+ *
+ * into
+ *
+ * void _clobber(WithLock, OperationContext* opCtx) {
+ * _stuff = makeStuff(opCtx);
+ * }
+ *
+ * A call to such a function looks like this:
+ *
+ * stdx::lock_guard<stdx::mutex> lk(_mutex);
+ * _clobber(lk, opCtx); // instead of _clobber_inlock(opCtx)
+ *
+ * Note that the formal argument need not (and should not) be named unless it is needed to pass
+ * the attestation along to another function:
+ *
+ * void _clobber(WithLock lock, OperationContext* opCtx) {
+ * _really_clobber(lock, opCtx);
+ * }
+ *
+ */
+struct WithLock {
+ template <typename Mutex>
+ WithLock(stdx::lock_guard<Mutex> const&) noexcept {}
+
+ template <typename Mutex>
+ WithLock(stdx::unique_lock<Mutex> const& lock) noexcept {
+ invariant(lock.owns_lock());
+ }
+
+ // Pass by value is OK.
+ WithLock(WithLock const&) noexcept {}
+ WithLock(WithLock&&) noexcept {}
+
+ WithLock() = delete;
+
+ // No assigning WithLocks.
+ void operator=(WithLock const&) = delete;
+ void operator=(WithLock&&) = delete;
+
+ // No (accidentally) moving a unique_lock<> in.
+ template <typename Mutex>
+ WithLock(stdx::unique_lock<Mutex>&&) = delete;
+};
+
+} // namespace mongo
+
+namespace std {
+// No moving a WithLock:
+template <>
+mongo::WithLock&& move<mongo::WithLock>(mongo::WithLock&&) noexcept = delete;
+} // namespace std