summaryrefslogtreecommitdiff
diff options
context:
space:
mode:
authorLeon Zaruvinsky <leon@mongodb.com>2016-06-22 17:34:05 -0400
committerLeon Zaruvinsky <leon@mongodb.com>2016-07-07 17:44:19 -0400
commitabaa59f3990f42aadba3b04e32dabbf3c1e47ce3 (patch)
treebf88ab49e7ec53cea70626b83e8020db75631389
parent419a2e4eaf791a8d217050dbf0ca63149f261e0f (diff)
downloadmongo-abaa59f3990f42aadba3b04e32dabbf3c1e47ce3.tar.gz
SERVER-24415 Replace instances of shared_ptr<CollectionMetadata> with ScopedCollectionMetadata
-rw-r--r--src/mongo/db/commands/mr.cpp3
-rw-r--r--src/mongo/db/exec/shard_filter.cpp6
-rw-r--r--src/mongo/db/exec/shard_filter.h7
-rw-r--r--src/mongo/db/exec/update.cpp3
-rw-r--r--src/mongo/db/query/get_executor.cpp3
-rw-r--r--src/mongo/db/s/cleanup_orphaned_cmd.cpp5
-rw-r--r--src/mongo/db/s/collection_sharding_state.cpp20
-rw-r--r--src/mongo/db/s/collection_sharding_state.h15
-rw-r--r--src/mongo/db/s/get_shard_version_command.cpp6
-rw-r--r--src/mongo/db/s/merge_chunks_command.cpp3
-rw-r--r--src/mongo/db/s/metadata_manager.cpp13
-rw-r--r--src/mongo/db/s/metadata_manager.h13
-rw-r--r--src/mongo/db/s/migration_source_manager.cpp14
-rw-r--r--src/mongo/db/s/migration_source_manager.h15
-rw-r--r--src/mongo/db/s/move_chunk_command.cpp3
-rw-r--r--src/mongo/db/s/sharding_state.cpp31
-rw-r--r--src/mongo/db/s/sharding_state.h4
-rw-r--r--src/mongo/db/s/split_chunk_command.cpp4
18 files changed, 95 insertions, 73 deletions
diff --git a/src/mongo/db/commands/mr.cpp b/src/mongo/db/commands/mr.cpp
index ade9ae9225e..423cfeeec29 100644
--- a/src/mongo/db/commands/mr.cpp
+++ b/src/mongo/db/commands/mr.cpp
@@ -1358,10 +1358,9 @@ public:
uassert(16149, "cannot run map reduce without the js engine", globalScriptEngine);
- shared_ptr<CollectionMetadata> collMetadata;
-
// Prevent sharding state from changing during the MR.
unique_ptr<RangePreserver> rangePreserver;
+ ScopedCollectionMetadata collMetadata;
{
AutoGetCollectionForRead ctx(txn, config.ns);
diff --git a/src/mongo/db/exec/shard_filter.cpp b/src/mongo/db/exec/shard_filter.cpp
index 8388689cbc6..9c2f142622b 100644
--- a/src/mongo/db/exec/shard_filter.cpp
+++ b/src/mongo/db/exec/shard_filter.cpp
@@ -35,7 +35,7 @@
#include "mongo/db/exec/filter.h"
#include "mongo/db/exec/scoped_timer.h"
#include "mongo/db/exec/working_set_common.h"
-#include "mongo/db/s/collection_metadata.h"
+#include "mongo/db/s/metadata_manager.h"
#include "mongo/s/shard_key_pattern.h"
#include "mongo/stdx/memory.h"
#include "mongo/util/log.h"
@@ -51,10 +51,10 @@ using stdx::make_unique;
const char* ShardFilterStage::kStageType = "SHARDING_FILTER";
ShardFilterStage::ShardFilterStage(OperationContext* opCtx,
- const shared_ptr<CollectionMetadata>& metadata,
+ ScopedCollectionMetadata metadata,
WorkingSet* ws,
PlanStage* child)
- : PlanStage(kStageType, opCtx), _ws(ws), _metadata(metadata) {
+ : PlanStage(kStageType, opCtx), _ws(ws), _metadata(std::move(metadata)) {
_children.emplace_back(child);
}
diff --git a/src/mongo/db/exec/shard_filter.h b/src/mongo/db/exec/shard_filter.h
index 44d7bc08055..0a180afd2e6 100644
--- a/src/mongo/db/exec/shard_filter.h
+++ b/src/mongo/db/exec/shard_filter.h
@@ -29,11 +29,10 @@
#pragma once
#include "mongo/db/exec/plan_stage.h"
+#include "mongo/db/s/metadata_manager.h"
namespace mongo {
-class CollectionMetadata;
-
/**
* This stage drops documents that didn't belong to the shard we're executing on at the time of
* construction. This matches the contract for sharded cursorids which guarantees that a
@@ -72,7 +71,7 @@ class CollectionMetadata;
class ShardFilterStage final : public PlanStage {
public:
ShardFilterStage(OperationContext* opCtx,
- const std::shared_ptr<CollectionMetadata>& metadata,
+ ScopedCollectionMetadata metadata,
WorkingSet* ws,
PlanStage* child);
~ShardFilterStage();
@@ -98,7 +97,7 @@ private:
// Note: it is important that this is the metadata from the time this stage is constructed.
// See class comment for details.
- const std::shared_ptr<CollectionMetadata> _metadata;
+ ScopedCollectionMetadata _metadata;
};
} // namespace mongo
diff --git a/src/mongo/db/exec/update.cpp b/src/mongo/db/exec/update.cpp
index 7981c55cc23..a8f58613a5b 100644
--- a/src/mongo/db/exec/update.cpp
+++ b/src/mongo/db/exec/update.cpp
@@ -436,8 +436,7 @@ bool shouldRestartUpdateIfNoLongerMatches(const UpdateStageParams& params) {
};
const std::vector<FieldRef*>* getImmutableFields(OperationContext* txn, const NamespaceString& ns) {
- std::shared_ptr<CollectionMetadata> metadata =
- CollectionShardingState::get(txn, ns)->getMetadata();
+ auto metadata = CollectionShardingState::get(txn, ns)->getMetadata();
if (metadata) {
const std::vector<FieldRef*>& fields = metadata->getKeyPatternFields();
// Return shard-keys as immutable for the update system.
diff --git a/src/mongo/db/query/get_executor.cpp b/src/mongo/db/query/get_executor.cpp
index b4e9ca6de89..2e9189c87e3 100644
--- a/src/mongo/db/query/get_executor.cpp
+++ b/src/mongo/db/query/get_executor.cpp
@@ -175,8 +175,7 @@ void fillOutPlannerParams(OperationContext* txn,
// If the caller wants a shard filter, make sure we're actually sharded.
if (plannerParams->options & QueryPlannerParams::INCLUDE_SHARD_FILTER) {
- std::shared_ptr<CollectionMetadata> collMetadata =
- CollectionShardingState::get(txn, canonicalQuery->nss())->getMetadata();
+ auto collMetadata = CollectionShardingState::get(txn, canonicalQuery->nss())->getMetadata();
if (collMetadata) {
plannerParams->shardKey = collMetadata->getKeyPattern();
} else {
diff --git a/src/mongo/db/s/cleanup_orphaned_cmd.cpp b/src/mongo/db/s/cleanup_orphaned_cmd.cpp
index ff27007fb77..ea04f994a59 100644
--- a/src/mongo/db/s/cleanup_orphaned_cmd.cpp
+++ b/src/mongo/db/s/cleanup_orphaned_cmd.cpp
@@ -80,7 +80,7 @@ CleanupResult cleanupOrphanedData(OperationContext* txn,
string* errMsg) {
BSONObj startingFromKey = startingFromKeyConst;
- std::shared_ptr<CollectionMetadata> metadata;
+ ScopedCollectionMetadata metadata;
{
AutoGetCollection autoColl(txn, ns, MODE_IS);
metadata = CollectionShardingState::get(txn, ns.toString())->getMetadata();
@@ -116,9 +116,6 @@ CleanupResult cleanupOrphanedData(OperationContext* txn,
orphanRange.ns = ns.ns();
*stoppedAtKey = orphanRange.maxKey;
- // We're done with this metadata now, no matter what happens
- metadata.reset();
-
LOG(1) << "orphaned data cleanup requested for " << ns.toString() << " starting from "
<< startingFromKey << ", removing next orphan range"
<< " [" << orphanRange.minKey << "," << orphanRange.maxKey << ")";
diff --git a/src/mongo/db/s/collection_sharding_state.cpp b/src/mongo/db/s/collection_sharding_state.cpp
index 29606ca8640..af9e67fdbfe 100644
--- a/src/mongo/db/s/collection_sharding_state.cpp
+++ b/src/mongo/db/s/collection_sharding_state.cpp
@@ -78,7 +78,9 @@ using std::string;
CollectionShardingState::CollectionShardingState(
NamespaceString nss, std::unique_ptr<CollectionMetadata> initialMetadata)
- : _nss(std::move(nss)), _metadata(std::move(initialMetadata)) {}
+ : _nss(std::move(nss)), _metadataManager{} {
+ _metadataManager.setActiveMetadata(std::move(initialMetadata));
+}
CollectionShardingState::~CollectionShardingState() {
invariant(!_sourceMgr);
@@ -98,13 +100,16 @@ CollectionShardingState* CollectionShardingState::get(OperationContext* txn,
return shardingState->getNS(ns);
}
-void CollectionShardingState::setMetadata(std::shared_ptr<CollectionMetadata> newMetadata) {
+ScopedCollectionMetadata CollectionShardingState::getMetadata() {
+ return _metadataManager.getActiveMetadata();
+}
+
+void CollectionShardingState::setMetadata(std::unique_ptr<CollectionMetadata> newMetadata) {
if (newMetadata) {
invariant(!newMetadata->getCollVersion().isWriteCompatibleWith(ChunkVersion::UNSHARDED()));
invariant(!newMetadata->getShardVersion().isWriteCompatibleWith(ChunkVersion::UNSHARDED()));
}
-
- _metadata = std::move(newMetadata);
+ _metadataManager.setActiveMetadata(std::move(newMetadata));
}
MigrationSourceManager* CollectionShardingState::getMigrationSourceManager() {
@@ -127,7 +132,7 @@ void CollectionShardingState::clearMigrationSourceManager(OperationContext* txn)
_sourceMgr = nullptr;
}
-void CollectionShardingState::checkShardVersionOrThrow(OperationContext* txn) const {
+void CollectionShardingState::checkShardVersionOrThrow(OperationContext* txn) {
string errmsg;
ChunkVersion received;
ChunkVersion wanted;
@@ -204,7 +209,7 @@ void CollectionShardingState::onDeleteOp(OperationContext* txn, const BSONObj& d
bool CollectionShardingState::_checkShardVersionOk(OperationContext* txn,
string* errmsg,
ChunkVersion* expectedShardVersion,
- ChunkVersion* actualShardVersion) const {
+ ChunkVersion* actualShardVersion) {
Client* client = txn->getClient();
// Operations using the DBDirectClient are unversioned.
@@ -240,7 +245,8 @@ bool CollectionShardingState::_checkShardVersionOk(OperationContext* txn,
}
// Set this for error messaging purposes before potentially returning false.
- *actualShardVersion = (_metadata ? _metadata->getShardVersion() : ChunkVersion::UNSHARDED());
+ auto metadata = getMetadata();
+ *actualShardVersion = metadata ? metadata->getShardVersion() : ChunkVersion::UNSHARDED();
if (_sourceMgr && _sourceMgr->getMigrationCriticalSectionSignal()) {
*errmsg = str::stream() << "migration commit in progress for " << _nss.ns();
diff --git a/src/mongo/db/s/collection_sharding_state.h b/src/mongo/db/s/collection_sharding_state.h
index 449c8b4ba6b..ba482b15c74 100644
--- a/src/mongo/db/s/collection_sharding_state.h
+++ b/src/mongo/db/s/collection_sharding_state.h
@@ -34,6 +34,7 @@
#include "mongo/base/disallow_copying.h"
#include "mongo/base/string_data.h"
#include "mongo/db/namespace_string.h"
+#include "mongo/db/s/metadata_manager.h"
namespace mongo {
@@ -75,14 +76,12 @@ public:
/**
* Returns the chunk metadata for the collection.
*/
- std::shared_ptr<CollectionMetadata> getMetadata() const {
- return _metadata;
- }
+ ScopedCollectionMetadata getMetadata();
/**
* Set a new metadata to be used for this collection.
*/
- void setMetadata(std::shared_ptr<CollectionMetadata> newMetadata);
+ void setMetadata(std::unique_ptr<CollectionMetadata> newMetadata);
/**
* Returns the active migration source manager, if one is available.
@@ -112,7 +111,7 @@ public:
* response is constructed, this function should be the only means of checking for shard version
* match.
*/
- void checkShardVersionOrThrow(OperationContext* txn) const;
+ void checkShardVersionOrThrow(OperationContext* txn);
// Replication subsystem hooks. If this collection is serving as a source for migration, these
// methods inform it of any changes to its contents.
@@ -142,14 +141,12 @@ private:
bool _checkShardVersionOk(OperationContext* txn,
std::string* errmsg,
ChunkVersion* expectedShardVersion,
- ChunkVersion* actualShardVersion) const;
+ ChunkVersion* actualShardVersion);
// Namespace to which this state belongs.
const NamespaceString _nss;
- // Contains all the chunks associated with this collection. This value will be null if the
- // collection is not sharded.
- std::shared_ptr<CollectionMetadata> _metadata;
+ MetadataManager _metadataManager;
// If this collection is serving as a source shard for chunk migration, this value will be
// non-null. To write this value there needs to be X-lock on the collection in order to
diff --git a/src/mongo/db/s/get_shard_version_command.cpp b/src/mongo/db/s/get_shard_version_command.cpp
index 1331bbde061..5e5e1edef97 100644
--- a/src/mongo/db/s/get_shard_version_command.cpp
+++ b/src/mongo/db/s/get_shard_version_command.cpp
@@ -114,7 +114,11 @@ public:
AutoGetCollection autoColl(txn, nss, MODE_IS);
CollectionShardingState* const css = CollectionShardingState::get(txn, nss);
- shared_ptr<CollectionMetadata> metadata(css ? css->getMetadata() : nullptr);
+ ScopedCollectionMetadata metadata;
+ if (css) {
+ metadata = css->getMetadata();
+ }
+
if (metadata) {
result.appendTimestamp("global", metadata->getShardVersion().toLong());
} else {
diff --git a/src/mongo/db/s/merge_chunks_command.cpp b/src/mongo/db/s/merge_chunks_command.cpp
index dd688648be8..f996fda63e9 100644
--- a/src/mongo/db/s/merge_chunks_command.cpp
+++ b/src/mongo/db/s/merge_chunks_command.cpp
@@ -39,6 +39,7 @@
#include "mongo/db/namespace_string.h"
#include "mongo/db/s/collection_metadata.h"
#include "mongo/db/s/collection_sharding_state.h"
+#include "mongo/db/s/metadata_manager.h"
#include "mongo/db/s/sharding_state.h"
#include "mongo/s/catalog/type_chunk.h"
#include "mongo/s/grid.h"
@@ -196,7 +197,7 @@ bool mergeChunks(OperationContext* txn,
return false;
}
- std::shared_ptr<CollectionMetadata> metadata;
+ ScopedCollectionMetadata metadata;
{
AutoGetCollection autoColl(txn, nss, MODE_IS);
diff --git a/src/mongo/db/s/metadata_manager.cpp b/src/mongo/db/s/metadata_manager.cpp
index cf2cb76def5..457d1e9ae29 100644
--- a/src/mongo/db/s/metadata_manager.cpp
+++ b/src/mongo/db/s/metadata_manager.cpp
@@ -43,7 +43,9 @@ MetadataManager::~MetadataManager() {
ScopedCollectionMetadata MetadataManager::getActiveMetadata() {
stdx::lock_guard<stdx::mutex> scopedLock(_managerLock);
- invariant(_activeMetadataTracker);
+ if (!_activeMetadataTracker) {
+ return ScopedCollectionMetadata();
+ }
return ScopedCollectionMetadata(this, _activeMetadataTracker.get());
}
@@ -72,6 +74,8 @@ MetadataManager::CollectionMetadataTracker::CollectionMetadataTracker(
std::unique_ptr<CollectionMetadata> m)
: metadata(std::move(m)), usageCounter(0){};
+ScopedCollectionMetadata::ScopedCollectionMetadata() = default;
+
// called in lock
ScopedCollectionMetadata::ScopedCollectionMetadata(
MetadataManager* manager, MetadataManager::CollectionMetadataTracker* tracker)
@@ -80,6 +84,9 @@ ScopedCollectionMetadata::ScopedCollectionMetadata(
}
ScopedCollectionMetadata::~ScopedCollectionMetadata() {
+ if (!_tracker)
+ return;
+
stdx::lock_guard<stdx::mutex> scopedLock(_manager->_managerLock);
invariant(_tracker->usageCounter > 0);
if (--_tracker->usageCounter == 0) {
@@ -110,6 +117,10 @@ ScopedCollectionMetadata& ScopedCollectionMetadata::operator=(ScopedCollectionMe
return *this;
}
+ScopedCollectionMetadata::operator bool() const {
+ return _tracker && _tracker->metadata.get();
+}
+
std::map<BSONObj, ChunkRange> MetadataManager::getCopyOfRanges() {
stdx::lock_guard<stdx::mutex> scopedLock(_managerLock);
return _rangesToClean;
diff --git a/src/mongo/db/s/metadata_manager.h b/src/mongo/db/s/metadata_manager.h
index 73f91f7e44f..04135cef840 100644
--- a/src/mongo/db/s/metadata_manager.h
+++ b/src/mongo/db/s/metadata_manager.h
@@ -123,6 +123,12 @@ class ScopedCollectionMetadata {
public:
/**
+ * Creates an empty ScopedCollectionMetadata. Using the default constructor means that no
+ * metadata is available.
+ */
+ ScopedCollectionMetadata();
+
+ /**
* Decrements the usageCounter and conditionally makes a call to _removeMetadata on
* the tracker if the count has reached zero.
*/
@@ -137,6 +143,9 @@ public:
CollectionMetadata* operator->();
CollectionMetadata* getMetadata();
+ /** True if the ScopedCollectionMetadata stores a metadata (is not empty) */
+ operator bool() const;
+
private:
friend ScopedCollectionMetadata MetadataManager::getActiveMetadata();
@@ -146,8 +155,8 @@ private:
ScopedCollectionMetadata(MetadataManager* manager,
MetadataManager::CollectionMetadataTracker* tracker);
- MetadataManager* _manager;
- MetadataManager::CollectionMetadataTracker* _tracker;
+ MetadataManager* _manager{nullptr};
+ MetadataManager::CollectionMetadataTracker* _tracker{nullptr};
};
} // namespace mongo
diff --git a/src/mongo/db/s/migration_source_manager.cpp b/src/mongo/db/s/migration_source_manager.cpp
index 61a4399a7c5..cdc188dcd46 100644
--- a/src/mongo/db/s/migration_source_manager.cpp
+++ b/src/mongo/db/s/migration_source_manager.cpp
@@ -120,8 +120,8 @@ MigrationSourceManager::MigrationSourceManager(OperationContext* txn, MoveChunkR
ScopedTransaction scopedXact(txn, MODE_IS);
AutoGetCollection autoColl(txn, _args.getNss(), MODE_IS);
- auto css = CollectionShardingState::get(txn, _args.getNss());
- _committedMetadata = css->getMetadata();
+ _committedMetadata = CollectionShardingState::get(txn, _args.getNss())->getMetadata();
+ _keyPattern = _committedMetadata->getKeyPattern();
}
const ChunkVersion collectionVersion = _committedMetadata->getCollVersion();
@@ -357,10 +357,10 @@ Status MigrationSourceManager::commitDonateChunk(OperationContext* txn) {
ChunkType migratingChunkToForget;
migratingChunkToForget.setMin(_args.getMinKey());
migratingChunkToForget.setMax(_args.getMaxKey());
- _committedMetadata =
- _committedMetadata->cloneMigrate(migratingChunkToForget, uncommittedCollVersion);
auto css = CollectionShardingState::get(txn, _args.getNss().ns());
- css->setMetadata(_committedMetadata);
+ css->setMetadata(
+ _committedMetadata->cloneMigrate(migratingChunkToForget, uncommittedCollVersion));
+ _committedMetadata = css->getMetadata();
} else {
// This could be an unrelated error (e.g. network error). Check whether the metadata update
// succeeded by refreshing the collection metadata from the config server and checking that
@@ -415,8 +415,8 @@ Status MigrationSourceManager::commitDonateChunk(OperationContext* txn) {
AutoGetCollection autoColl(txn, _args.getNss(), MODE_IS);
ChunkVersion previousMetadataCollVersion = _committedMetadata->getCollVersion();
- auto css = CollectionShardingState::get(txn, _args.getNss());
- std::shared_ptr<CollectionMetadata> refreshedMetadata = css->getMetadata();
+ auto refreshedMetadata =
+ CollectionShardingState::get(txn, _args.getNss())->getMetadata();
if (refreshedMetadata->keyBelongsToMe(_args.getMinKey())) {
invariant(refreshedMetadata->getCollVersion() == previousMetadataCollVersion);
diff --git a/src/mongo/db/s/migration_source_manager.h b/src/mongo/db/s/migration_source_manager.h
index 16eddc2351b..a5f1260c098 100644
--- a/src/mongo/db/s/migration_source_manager.h
+++ b/src/mongo/db/s/migration_source_manager.h
@@ -31,6 +31,7 @@
#include <string>
#include "mongo/base/disallow_copying.h"
+#include "mongo/db/s/metadata_manager.h"
#include "mongo/s/move_chunk_request.h"
#include "mongo/s/shard_key_pattern.h"
#include "mongo/util/concurrency/notification.h"
@@ -38,7 +39,6 @@
namespace mongo {
-class CollectionMetadata;
class MigrationChunkClonerSource;
class OperationContext;
@@ -145,12 +145,10 @@ public:
void cleanupOnError(OperationContext* txn);
/**
- * Retrieves the last known committed collection metadata. What gets returned by this call may
- * change before and after the critical section has started. It should only be used for
- * diagnostics purposes and not relied on for any routing/consistency checking decisions.
+ * Returns the key pattern object for the stored committed metadata.
*/
- std::shared_ptr<CollectionMetadata> getCommittedMetadata() const {
- return _committedMetadata;
+ BSONObj getKeyPattern() const {
+ return _keyPattern;
}
/**
@@ -197,7 +195,10 @@ private:
// The cached collection metadata from just after the collection distributed lock was acquired.
// This metadata is guaranteed to not change until either failure or successful completion,
// because the distributed lock is being held. Available after stabilize stage has completed.
- std::shared_ptr<CollectionMetadata> _committedMetadata;
+ ScopedCollectionMetadata _committedMetadata;
+
+ // The key pattern of the collection whose chunks are being moved.
+ BSONObj _keyPattern;
// The chunk cloner source. Only available if there is an active migration going on. To set and
// remove it, global S lock needs to be acquired first in order to block all logOp calls and
diff --git a/src/mongo/db/s/move_chunk_command.cpp b/src/mongo/db/s/move_chunk_command.cpp
index e7f332fe9f3..493bf35ad3d 100644
--- a/src/mongo/db/s/move_chunk_command.cpp
+++ b/src/mongo/db/s/move_chunk_command.cpp
@@ -224,8 +224,7 @@ private:
MigrationSourceManager migrationSourceManager(txn, moveChunkRequest);
- shardKeyPattern =
- migrationSourceManager.getCommittedMetadata()->getKeyPattern().getOwned();
+ shardKeyPattern = migrationSourceManager.getKeyPattern().getOwned();
moveTimingHelper.done(2);
MONGO_FAIL_POINT_PAUSE_WHILE_SET(moveChunkHangAtStep2);
diff --git a/src/mongo/db/s/sharding_state.cpp b/src/mongo/db/s/sharding_state.cpp
index b810a710d0b..f54eafc6ae4 100644
--- a/src/mongo/db/s/sharding_state.cpp
+++ b/src/mongo/db/s/sharding_state.cpp
@@ -270,8 +270,7 @@ void ShardingState::clearCollectionMetadata() {
}
ChunkVersion ShardingState::getVersion(const string& ns) {
- shared_ptr<CollectionMetadata> p;
-
+ ScopedCollectionMetadata p;
{
stdx::lock_guard<stdx::mutex> lk(_mutex);
CollectionShardingStateMap::const_iterator it = _collections.find(ns);
@@ -319,8 +318,7 @@ Status ShardingState::onStaleShardVersion(OperationContext* txn,
{
AutoGetCollection autoColl(txn, nss, MODE_IS);
- shared_ptr<CollectionMetadata> storedMetadata =
- CollectionShardingState::get(txn, nss)->getMetadata();
+ auto storedMetadata = CollectionShardingState::get(txn, nss)->getMetadata();
if (storedMetadata) {
collectionShardVersion = storedMetadata->getShardVersion();
}
@@ -647,7 +645,7 @@ Status ShardingState::_refreshMetadata(OperationContext* txn,
// The idea here is that we're going to reload the metadata from the config server, but we need
// to do so outside any locks. When we get our result back, if the current metadata has
// changed, we may not be able to install the new metadata.
- shared_ptr<CollectionMetadata> beforeMetadata;
+ ScopedCollectionMetadata beforeMetadata;
{
ScopedTransaction transaction(txn, MODE_IS);
AutoGetCollection autoColl(txn, nss, MODE_IS);
@@ -698,12 +696,13 @@ Status ShardingState::_refreshMetadata(OperationContext* txn,
long long refreshMillis;
{
- Status status = mdLoader.makeCollectionMetadata(txn,
- grid.catalogClient(txn),
- ns,
- getShardName(),
- fullReload ? nullptr : beforeMetadata.get(),
- remoteMetadata.get());
+ Status status =
+ mdLoader.makeCollectionMetadata(txn,
+ grid.catalogClient(txn),
+ ns,
+ getShardName(),
+ fullReload ? nullptr : beforeMetadata.getMetadata(),
+ remoteMetadata.get());
refreshMillis = refreshTimer.millis();
if (status.code() == ErrorCodes::NamespaceNotFound) {
@@ -727,7 +726,7 @@ Status ShardingState::_refreshMetadata(OperationContext* txn,
// Get ready to install loaded metadata if needed
//
- shared_ptr<CollectionMetadata> afterMetadata;
+ ScopedCollectionMetadata afterMetadata;
ChunkVersion afterShardVersion;
ChunkVersion afterCollVersion;
VersionChoice choice;
@@ -750,6 +749,7 @@ Status ShardingState::_refreshMetadata(OperationContext* txn,
// Get the metadata now that the load has completed
auto css = CollectionShardingState::get(txn, nss);
afterMetadata = css->getMetadata();
+
if (afterMetadata) {
afterShardVersion = afterMetadata->getShardVersion();
afterCollVersion = afterMetadata->getCollVersion();
@@ -761,7 +761,8 @@ Status ShardingState::_refreshMetadata(OperationContext* txn,
// Resolve newer pending chunks with the remote metadata, finish construction
//
- Status status = mdLoader.promotePendingChunks(afterMetadata.get(), remoteMetadata.get());
+ Status status =
+ mdLoader.promotePendingChunks(afterMetadata.getMetadata(), remoteMetadata.get());
if (!status.isOK()) {
warning() << "remote metadata for " << ns
<< " is inconsistent with current pending chunks"
@@ -894,7 +895,7 @@ void ShardingState::appendInfo(OperationContext* txn, BSONObjBuilder& builder) {
for (CollectionShardingStateMap::const_iterator it = _collections.begin();
it != _collections.end();
++it) {
- shared_ptr<CollectionMetadata> metadata = it->second->getMetadata();
+ ScopedCollectionMetadata metadata = it->second->getMetadata();
if (metadata) {
versionB.appendTimestamp(it->first, metadata->getShardVersion().toLong());
} else {
@@ -917,7 +918,7 @@ bool ShardingState::needCollectionMetadata(OperationContext* txn, const string&
OperationShardingState::get(txn).hasShardVersion();
}
-shared_ptr<CollectionMetadata> ShardingState::getCollectionMetadata(const string& ns) {
+ScopedCollectionMetadata ShardingState::getCollectionMetadata(const string& ns) {
stdx::lock_guard<stdx::mutex> lk(_mutex);
CollectionShardingStateMap::const_iterator it = _collections.find(ns);
diff --git a/src/mongo/db/s/sharding_state.h b/src/mongo/db/s/sharding_state.h
index 5b12c44d19d..4a53906370a 100644
--- a/src/mongo/db/s/sharding_state.h
+++ b/src/mongo/db/s/sharding_state.h
@@ -48,10 +48,10 @@ namespace mongo {
class BSONObj;
class BSONObjBuilder;
struct ChunkVersion;
-class CollectionMetadata;
class CollectionShardingState;
class ConnectionString;
class OperationContext;
+class ScopedCollectionMetadata;
class ServiceContext;
class ShardIdentityType;
class Status;
@@ -200,7 +200,7 @@ public:
bool needCollectionMetadata(OperationContext* txn, const std::string& ns);
- std::shared_ptr<CollectionMetadata> getCollectionMetadata(const std::string& ns);
+ ScopedCollectionMetadata getCollectionMetadata(const std::string& ns);
/**
* Updates the config server field of the shardIdentity document with the given connection
diff --git a/src/mongo/db/s/split_chunk_command.cpp b/src/mongo/db/s/split_chunk_command.cpp
index f9c1a56710a..90dc2c96427 100644
--- a/src/mongo/db/s/split_chunk_command.cpp
+++ b/src/mongo/db/s/split_chunk_command.cpp
@@ -282,7 +282,7 @@ public:
}
}
- std::shared_ptr<CollectionMetadata> collMetadata;
+ ScopedCollectionMetadata collMetadata;
{
AutoGetCollection autoColl(txn, nss, MODE_IS);
@@ -291,7 +291,7 @@ public:
}
// With nonzero shard version, we must have metadata
- invariant(NULL != collMetadata);
+ invariant(collMetadata);
ChunkVersion collVersion = collMetadata->getCollVersion();
// With nonzero shard version, we must have a coll version >= our shard version