summaryrefslogtreecommitdiff
path: root/src/mongo/db/s
diff options
context:
space:
mode:
authorMartin Neupauer <martin.neupauer@10gen.com>2018-03-26 19:30:26 -0400
committerMartin Neupauer <martin.neupauer@mongodb.com>2018-04-05 12:52:02 -0400
commita6d486b8a6e0c81771bd771cc0237236791d635d (patch)
tree61b49fae8cf316c033b2ac707202e50b281cc72a /src/mongo/db/s
parent6fbc1bbfcd5ffcfb451c300a6ef523f19d5edb55 (diff)
downloadmongo-a6d486b8a6e0c81771bd771cc0237236791d635d.tar.gz
SERVER-33523 Add timestamp support to CollectionMetadata
Diffstat (limited to 'src/mongo/db/s')
-rw-r--r--src/mongo/db/s/cleanup_orphaned_cmd.cpp2
-rw-r--r--src/mongo/db/s/collection_range_deleter.cpp2
-rw-r--r--src/mongo/db/s/collection_sharding_state.cpp24
-rw-r--r--src/mongo/db/s/collection_sharding_state.h8
-rw-r--r--src/mongo/db/s/collection_sharding_state_test.cpp8
-rw-r--r--src/mongo/db/s/get_shard_version_command.cpp2
-rw-r--r--src/mongo/db/s/merge_chunks_command.cpp4
-rw-r--r--src/mongo/db/s/metadata_manager.cpp29
-rw-r--r--src/mongo/db/s/metadata_manager.h15
-rw-r--r--src/mongo/db/s/migration_destination_manager.cpp4
-rw-r--r--src/mongo/db/s/migration_source_manager.cpp10
-rw-r--r--src/mongo/db/s/set_shard_version_command.cpp6
-rw-r--r--src/mongo/db/s/shard_filtering_metadata_refresh.cpp27
-rw-r--r--src/mongo/db/s/shard_server_op_observer.cpp6
-rw-r--r--src/mongo/db/s/split_chunk.cpp2
15 files changed, 99 insertions, 50 deletions
diff --git a/src/mongo/db/s/cleanup_orphaned_cmd.cpp b/src/mongo/db/s/cleanup_orphaned_cmd.cpp
index bd4716401a9..1177a8dbf71 100644
--- a/src/mongo/db/s/cleanup_orphaned_cmd.cpp
+++ b/src/mongo/db/s/cleanup_orphaned_cmd.cpp
@@ -79,7 +79,7 @@ CleanupResult cleanupOrphanedData(OperationContext* opCtx,
{
AutoGetCollection autoColl(opCtx, ns, MODE_IX);
const auto css = CollectionShardingState::get(opCtx, ns);
- auto metadata = css->getMetadata();
+ auto metadata = css->getMetadata(opCtx);
if (!metadata) {
log() << "skipping orphaned data cleanup for " << ns.toString()
<< ", collection is not sharded";
diff --git a/src/mongo/db/s/collection_range_deleter.cpp b/src/mongo/db/s/collection_range_deleter.cpp
index b575538ecc8..9f90ed44e02 100644
--- a/src/mongo/db/s/collection_range_deleter.cpp
+++ b/src/mongo/db/s/collection_range_deleter.cpp
@@ -109,7 +109,7 @@ boost::optional<Date_t> CollectionRangeDeleter::cleanUpNextRange(
auto* const css = CollectionShardingState::get(opCtx, nss);
auto* const self = forTestOnly ? forTestOnly : &css->_metadataManager->_rangesToClean;
- auto scopedCollectionMetadata = css->getMetadata();
+ auto scopedCollectionMetadata = css->getMetadata(opCtx);
if (!forTestOnly && (!collection || !scopedCollectionMetadata)) {
if (!collection) {
diff --git a/src/mongo/db/s/collection_sharding_state.cpp b/src/mongo/db/s/collection_sharding_state.cpp
index 9fd8c6c887e..5bafa1f37fd 100644
--- a/src/mongo/db/s/collection_sharding_state.cpp
+++ b/src/mongo/db/s/collection_sharding_state.cpp
@@ -125,14 +125,14 @@ public:
return *it->second;
}
- void report(BSONObjBuilder* builder) {
+ void report(OperationContext* opCtx, BSONObjBuilder* builder) {
BSONObjBuilder versionB(builder->subobjStart("versions"));
{
stdx::lock_guard<stdx::mutex> lg(_mutex);
for (auto& coll : _collections) {
- ScopedCollectionMetadata metadata = coll.second->getMetadata();
+ ScopedCollectionMetadata metadata = coll.second->getMetadata(opCtx);
if (metadata) {
versionB.appendTimestamp(coll.first, metadata->getShardVersion().toLong());
} else {
@@ -182,11 +182,14 @@ CollectionShardingState* CollectionShardingState::get(OperationContext* opCtx,
void CollectionShardingState::report(OperationContext* opCtx, BSONObjBuilder* builder) {
auto& collectionsMap = getCollectionShardingStateMap(opCtx->getServiceContext());
- collectionsMap.report(builder);
+ collectionsMap.report(opCtx, builder);
}
-ScopedCollectionMetadata CollectionShardingState::getMetadata() {
- return _metadataManager->getActiveMetadata(_metadataManager);
+ScopedCollectionMetadata CollectionShardingState::getMetadata(OperationContext* opCtx) {
+ // TODO: SERVER-34276 - find an alternative to get the atClusterTime.
+ auto atClusterTime = repl::ReadConcernArgs::get(opCtx).getArgsAtClusterTime();
+ return atClusterTime ? _metadataManager->createMetadataAt(opCtx, atClusterTime.get())
+ : _metadataManager->getActiveMetadata(_metadataManager);
}
void CollectionShardingState::refreshMetadata(OperationContext* opCtx,
@@ -261,8 +264,8 @@ void CollectionShardingState::checkShardVersionOrThrow(OperationContext* opCtx)
}
}
-bool CollectionShardingState::collectionIsSharded() {
- auto metadata = getMetadata().getMetadata();
+bool CollectionShardingState::collectionIsSharded(OperationContext* opCtx) {
+ auto metadata = getMetadata(opCtx).getMetadata();
if (metadata && (metadata->getCollVersion().isStrictlyEqualTo(ChunkVersion::UNSHARDED()))) {
return false;
}
@@ -352,8 +355,9 @@ void CollectionShardingState::onUpdateOp(OperationContext* opCtx,
}
}
-auto CollectionShardingState::makeDeleteState(BSONObj const& doc) -> DeleteState {
- return {getMetadata().extractDocumentKey(doc).getOwned(),
+auto CollectionShardingState::makeDeleteState(OperationContext* opCtx, BSONObj const& doc)
+ -> DeleteState {
+ return {getMetadata(opCtx).extractDocumentKey(doc).getOwned(),
_sourceMgr && _sourceMgr->getCloner()->isDocumentInMigratingChunk(doc)};
}
@@ -407,7 +411,7 @@ bool CollectionShardingState::_checkShardVersionOk(OperationContext* opCtx,
}
// Set this for error messaging purposes before potentially returning false.
- auto metadata = getMetadata();
+ auto metadata = getMetadata(opCtx);
*actualShardVersion = metadata ? metadata->getShardVersion() : ChunkVersion::UNSHARDED();
auto criticalSectionSignal = _critSec.getSignal(opCtx->lockState()->isWriteLocked()
diff --git a/src/mongo/db/s/collection_sharding_state.h b/src/mongo/db/s/collection_sharding_state.h
index 51ff7d26435..6aa2463dc9d 100644
--- a/src/mongo/db/s/collection_sharding_state.h
+++ b/src/mongo/db/s/collection_sharding_state.h
@@ -75,7 +75,7 @@ public:
bool isMigrating;
};
- DeleteState makeDeleteState(BSONObj const& doc);
+ DeleteState makeDeleteState(OperationContext* opCtx, BSONObj const& doc);
/**
* Obtains the sharding state for the specified collection. If it does not exist, it will be
@@ -93,8 +93,10 @@ public:
* Returns the chunk metadata for the collection. The metadata it represents lives as long as
* the object itself, and the collection, exist. After dropping the collection lock, the
* collection may no longer exist, but it is still safe to destroy the object.
+ * The metadata is tied to a specific point in time (atClusterTime) and the time is retrieved
+ * from the operation context (opCtx).
*/
- ScopedCollectionMetadata getMetadata();
+ ScopedCollectionMetadata getMetadata(OperationContext* opCtx);
/**
* BSON output of the pending metadata into a BSONArray
@@ -198,7 +200,7 @@ public:
* Returns whether this collection is sharded. Valid only if mongoD is primary.
* TODO SERVER-24960: This method may return a false positive until SERVER-24960 is fixed.
*/
- bool collectionIsSharded();
+ bool collectionIsSharded(OperationContext* opCtx);
/**
* Tracks deletion of any documents within the range, returning when deletion is complete.
diff --git a/src/mongo/db/s/collection_sharding_state_test.cpp b/src/mongo/db/s/collection_sharding_state_test.cpp
index 63db860c50d..72f01c98f53 100644
--- a/src/mongo/db/s/collection_sharding_state_test.cpp
+++ b/src/mongo/db/s/collection_sharding_state_test.cpp
@@ -156,7 +156,7 @@ TEST_F(DeleteStateTest, MakeDeleteStateUnsharded) {
// First, check that an order for deletion from an unsharded collection (where css has not been
// "refreshed" with chunk metadata) extracts just the "_id" field:
- auto deleteState = css->makeDeleteState(doc);
+ auto deleteState = css->makeDeleteState(operationContext(), doc);
ASSERT_BSONOBJ_EQ(deleteState.documentKey,
BSON("_id"
<< "hello"));
@@ -181,7 +181,7 @@ TEST_F(DeleteStateTest, MakeDeleteStateShardedWithoutIdInShardKey) {
<< true);
// Verify the shard key is extracted, in correct order, followed by the "_id" field.
- auto deleteState = css->makeDeleteState(doc);
+ auto deleteState = css->makeDeleteState(operationContext(), doc);
ASSERT_BSONOBJ_EQ(deleteState.documentKey,
BSON("key" << 100 << "key3"
<< "abc"
@@ -207,7 +207,7 @@ TEST_F(DeleteStateTest, MakeDeleteStateShardedWithIdInShardKey) {
<< 100);
// Verify the shard key is extracted with "_id" in the right place.
- auto deleteState = css->makeDeleteState(doc);
+ auto deleteState = css->makeDeleteState(operationContext(), doc);
ASSERT_BSONOBJ_EQ(deleteState.documentKey,
BSON("key" << 100 << "_id"
<< "hello"
@@ -231,7 +231,7 @@ TEST_F(DeleteStateTest, MakeDeleteStateShardedWithIdHashInShardKey) {
<< 100);
// Verify the shard key is extracted with "_id" in the right place, not hashed.
- auto deleteState = css->makeDeleteState(doc);
+ auto deleteState = css->makeDeleteState(operationContext(), doc);
ASSERT_BSONOBJ_EQ(deleteState.documentKey,
BSON("_id"
<< "hello"));
diff --git a/src/mongo/db/s/get_shard_version_command.cpp b/src/mongo/db/s/get_shard_version_command.cpp
index 325a7fbc5fc..4fd8a6d2261 100644
--- a/src/mongo/db/s/get_shard_version_command.cpp
+++ b/src/mongo/db/s/get_shard_version_command.cpp
@@ -109,7 +109,7 @@ public:
AutoGetCollection autoColl(opCtx, nss, MODE_IS);
CollectionShardingState* const css = CollectionShardingState::get(opCtx, nss);
- const auto metadata = css->getMetadata();
+ const auto metadata = css->getMetadata(opCtx);
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 2405dbc5620..5c1a3fbc321 100644
--- a/src/mongo/db/s/merge_chunks_command.cpp
+++ b/src/mongo/db/s/merge_chunks_command.cpp
@@ -62,7 +62,7 @@ bool checkMetadataForSuccess(OperationContext* opCtx,
const BSONObj& maxKey) {
const auto metadataAfterMerge = [&] {
AutoGetCollection autoColl(opCtx, nss, MODE_IS);
- return CollectionShardingState::get(opCtx, nss)->getMetadata();
+ return CollectionShardingState::get(opCtx, nss)->getMetadata(opCtx);
}();
uassert(ErrorCodes::StaleEpoch,
@@ -107,7 +107,7 @@ Status mergeChunks(OperationContext* opCtx,
const auto metadata = [&] {
AutoGetCollection autoColl(opCtx, nss, MODE_IS);
- return CollectionShardingState::get(opCtx, nss)->getMetadata();
+ return CollectionShardingState::get(opCtx, nss)->getMetadata(opCtx);
}();
if (!metadata) {
diff --git a/src/mongo/db/s/metadata_manager.cpp b/src/mongo/db/s/metadata_manager.cpp
index 60b8edca338..070b8b0710a 100644
--- a/src/mongo/db/s/metadata_manager.cpp
+++ b/src/mongo/db/s/metadata_manager.cpp
@@ -40,6 +40,8 @@
#include "mongo/db/range_arithmetic.h"
#include "mongo/db/s/collection_sharding_state.h"
#include "mongo/db/s/sharding_state.h"
+#include "mongo/s/catalog_cache.h"
+#include "mongo/s/grid.h"
#include "mongo/stdx/memory.h"
#include "mongo/util/assert_util.h"
#include "mongo/util/fail_point_service.h"
@@ -190,6 +192,27 @@ ScopedCollectionMetadata MetadataManager::getActiveMetadata(std::shared_ptr<Meta
return ScopedCollectionMetadata();
}
+ScopedCollectionMetadata MetadataManager::createMetadataAt(OperationContext* opCtx,
+ LogicalTime atClusterTime) {
+ auto cache = Grid::get(opCtx)->catalogCache();
+ if (!cache) {
+ return ScopedCollectionMetadata();
+ }
+
+ auto routingTable = cache->getCollectionRoutingTableHistoryNoRefresh(_nss);
+ if (!routingTable) {
+ return ScopedCollectionMetadata();
+ }
+ auto cm = std::make_shared<ChunkManager>(routingTable, atClusterTime.asTimestamp());
+
+ CollectionMetadata metadata(std::move(cm), ShardingState::get(opCtx)->getShardName());
+
+ auto metadataTracker =
+ std::make_shared<MetadataManager::CollectionMetadataTracker>(std::move(metadata));
+
+ return ScopedCollectionMetadata(std::move(metadataTracker));
+}
+
size_t MetadataManager::numberOfMetadataSnapshots() const {
stdx::lock_guard<stdx::mutex> lg(_managerLock);
if (_metadata.empty())
@@ -540,6 +563,12 @@ ScopedCollectionMetadata::ScopedCollectionMetadata(
++_metadataTracker->usageCounter;
}
+ScopedCollectionMetadata::ScopedCollectionMetadata(
+ std::shared_ptr<MetadataManager::CollectionMetadataTracker> metadataTracker)
+ : _metadataTracker(std::move(metadataTracker)) {
+ invariant(_metadataTracker);
+}
+
ScopedCollectionMetadata::ScopedCollectionMetadata(ScopedCollectionMetadata&& other) {
*this = std::move(other);
}
diff --git a/src/mongo/db/s/metadata_manager.h b/src/mongo/db/s/metadata_manager.h
index c1ececdfae7..9fe3a813dea 100644
--- a/src/mongo/db/s/metadata_manager.h
+++ b/src/mongo/db/s/metadata_manager.h
@@ -69,6 +69,12 @@ public:
ScopedCollectionMetadata getActiveMetadata(std::shared_ptr<MetadataManager> self);
/**
+ * Creates the metadata on demand for a specific point in time. The object is not tracked by
+ * the metadata manager.
+ */
+ ScopedCollectionMetadata createMetadataAt(OperationContext* opCtx, LogicalTime atCusterTime);
+
+ /**
* Returns the number of CollectionMetadata objects being maintained on behalf of running
* queries. The actual number may vary after it returns, so this is really only useful for unit
* tests.
@@ -291,6 +297,9 @@ private:
friend ScopedCollectionMetadata MetadataManager::getActiveMetadata(
std::shared_ptr<MetadataManager>);
+ friend ScopedCollectionMetadata MetadataManager::createMetadataAt(OperationContext*,
+ LogicalTime);
+
friend std::vector<ScopedCollectionMetadata> MetadataManager::overlappingMetadata(
std::shared_ptr<MetadataManager> const&, ChunkRange const&);
@@ -311,6 +320,12 @@ private:
std::shared_ptr<MetadataManager::CollectionMetadataTracker> metadataTracker);
/**
+ * Metadata not tracked by the manager - created on demand for a specific point in time.
+ */
+ ScopedCollectionMetadata(
+ std::shared_ptr<MetadataManager::CollectionMetadataTracker> metadataTracker);
+
+ /**
* Disconnect from the CollectionMetadata, possibly triggering GC of unused CollectionMetadata.
*/
void _clear();
diff --git a/src/mongo/db/s/migration_destination_manager.cpp b/src/mongo/db/s/migration_destination_manager.cpp
index 81541e3daaf..1f311cec6d1 100644
--- a/src/mongo/db/s/migration_destination_manager.cpp
+++ b/src/mongo/db/s/migration_destination_manager.cpp
@@ -1112,7 +1112,7 @@ CollectionShardingState::CleanupNotification MigrationDestinationManager::_noteP
AutoGetCollection autoColl(opCtx, nss, MODE_IX, MODE_X);
auto css = CollectionShardingState::get(opCtx, nss);
- auto metadata = css->getMetadata();
+ auto metadata = css->getMetadata(opCtx);
// This can currently happen because drops aren't synchronized with in-migrations. The idea for
// checking this here is that in the future we shouldn't have this problem.
@@ -1146,7 +1146,7 @@ void MigrationDestinationManager::_forgetPending(OperationContext* opCtx,
UninterruptibleLockGuard noInterrupt(opCtx->lockState());
AutoGetCollection autoColl(opCtx, nss, MODE_IX, MODE_X);
auto css = CollectionShardingState::get(opCtx, nss);
- auto metadata = css->getMetadata();
+ auto metadata = css->getMetadata(opCtx);
// This can currently happen because drops aren't synchronized with in-migrations. The idea for
// checking this here is that in the future we shouldn't have this problem.
diff --git a/src/mongo/db/s/migration_source_manager.cpp b/src/mongo/db/s/migration_source_manager.cpp
index bb0dc85c807..253496d2832 100644
--- a/src/mongo/db/s/migration_source_manager.cpp
+++ b/src/mongo/db/s/migration_source_manager.cpp
@@ -163,7 +163,7 @@ MigrationSourceManager::MigrationSourceManager(OperationContext* opCtx,
collectionUUID = autoColl.getCollection()->uuid().value();
}
- auto metadata = CollectionShardingState::get(opCtx, getNss())->getMetadata();
+ auto metadata = CollectionShardingState::get(opCtx, getNss())->getMetadata(opCtx);
uassert(ErrorCodes::IncompatibleShardingMetadata,
str::stream() << "cannot move chunks for an unsharded collection",
metadata);
@@ -238,7 +238,7 @@ Status MigrationSourceManager::startClone(OperationContext* opCtx) {
AutoGetCollection autoColl(opCtx, getNss(), MODE_IX, MODE_X);
auto css = CollectionShardingState::get(opCtx, getNss());
- const auto metadata = css->getMetadata();
+ const auto metadata = css->getMetadata(opCtx);
Status status = checkCollectionEpochMatches(metadata, _collectionEpoch);
if (!status.isOK())
return status;
@@ -293,7 +293,7 @@ Status MigrationSourceManager::enterCriticalSection(OperationContext* opCtx) {
const auto metadata = [&] {
UninterruptibleLockGuard noInterrupt(opCtx->lockState());
AutoGetCollection autoColl(opCtx, _args.getNss(), MODE_IS);
- return CollectionShardingState::get(opCtx, _args.getNss())->getMetadata();
+ return CollectionShardingState::get(opCtx, _args.getNss())->getMetadata(opCtx);
}();
Status status = checkCollectionEpochMatches(metadata, _collectionEpoch);
@@ -390,7 +390,7 @@ Status MigrationSourceManager::commitChunkMetadataOnConfig(OperationContext* opC
const auto metadata = [&] {
UninterruptibleLockGuard noInterrupt(opCtx->lockState());
AutoGetCollection autoColl(opCtx, _args.getNss(), MODE_IS);
- return CollectionShardingState::get(opCtx, _args.getNss())->getMetadata();
+ return CollectionShardingState::get(opCtx, _args.getNss())->getMetadata(opCtx);
}();
Status status = checkCollectionEpochMatches(metadata, _collectionEpoch);
@@ -545,7 +545,7 @@ Status MigrationSourceManager::commitChunkMetadataOnConfig(OperationContext* opC
auto refreshedMetadata = [&] {
UninterruptibleLockGuard noInterrupt(opCtx->lockState());
AutoGetCollection autoColl(opCtx, getNss(), MODE_IS);
- return CollectionShardingState::get(opCtx, getNss())->getMetadata();
+ return CollectionShardingState::get(opCtx, getNss())->getMetadata(opCtx);
}();
if (!refreshedMetadata) {
diff --git a/src/mongo/db/s/set_shard_version_command.cpp b/src/mongo/db/s/set_shard_version_command.cpp
index 63818e7fe89..6dca94efffd 100644
--- a/src/mongo/db/s/set_shard_version_command.cpp
+++ b/src/mongo/db/s/set_shard_version_command.cpp
@@ -229,8 +229,8 @@ public:
auto css = CollectionShardingState::get(opCtx, nss);
const ChunkVersion collectionShardVersion =
- (css->getMetadata() ? css->getMetadata()->getShardVersion()
- : ChunkVersion::UNSHARDED());
+ (css->getMetadata(opCtx) ? css->getMetadata(opCtx)->getShardVersion()
+ : ChunkVersion::UNSHARDED());
if (requestedVersion.isWriteCompatibleWith(collectionShardVersion)) {
// MongoS and MongoD agree on what is the collection's shard version
@@ -341,7 +341,7 @@ public:
AutoGetCollection autoColl(opCtx, nss, MODE_IS);
ChunkVersion currVersion = ChunkVersion::UNSHARDED();
- auto collMetadata = CollectionShardingState::get(opCtx, nss)->getMetadata();
+ auto collMetadata = CollectionShardingState::get(opCtx, nss)->getMetadata(opCtx);
if (collMetadata) {
currVersion = collMetadata->getShardVersion();
}
diff --git a/src/mongo/db/s/shard_filtering_metadata_refresh.cpp b/src/mongo/db/s/shard_filtering_metadata_refresh.cpp
index eed8ca38df0..e641d43578e 100644
--- a/src/mongo/db/s/shard_filtering_metadata_refresh.cpp
+++ b/src/mongo/db/s/shard_filtering_metadata_refresh.cpp
@@ -72,7 +72,7 @@ Status onShardVersionMismatch(OperationContext* opCtx,
const auto currentShardVersion = [&] {
AutoGetCollection autoColl(opCtx, nss, MODE_IS);
- const auto currentMetadata = CollectionShardingState::get(opCtx, nss)->getMetadata();
+ const auto currentMetadata = CollectionShardingState::get(opCtx, nss)->getMetadata(opCtx);
if (currentMetadata) {
return currentMetadata->getShardVersion();
}
@@ -122,15 +122,14 @@ ChunkVersion forceShardFilteringMetadataRefresh(OperationContext* opCtx,
{
AutoGetCollection autoColl(opCtx, nss, MODE_IS);
- auto css = CollectionShardingState::get(opCtx, nss);
+ auto metadata = CollectionShardingState::get(opCtx, nss)->getMetadata(opCtx);
// We already have newer version
- if (css->getMetadata() &&
- css->getMetadata()->getCollVersion().epoch() == cm->getVersion().epoch() &&
- css->getMetadata()->getCollVersion() >= cm->getVersion()) {
+ if (metadata && metadata->getCollVersion().epoch() == cm->getVersion().epoch() &&
+ metadata->getCollVersion() >= cm->getVersion()) {
LOG(1) << "Skipping refresh of metadata for " << nss << " "
- << css->getMetadata()->getCollVersion() << " with an older " << cm->getVersion();
- return css->getMetadata()->getShardVersion();
+ << metadata->getCollVersion() << " with an older " << cm->getVersion();
+ return metadata->getShardVersion();
}
}
@@ -138,14 +137,14 @@ ChunkVersion forceShardFilteringMetadataRefresh(OperationContext* opCtx,
AutoGetCollection autoColl(opCtx, nss, MODE_IX, MODE_X);
auto css = CollectionShardingState::get(opCtx, nss);
+ auto metadata = css->getMetadata(opCtx);
// We already have newer version
- if (css->getMetadata() &&
- css->getMetadata()->getCollVersion().epoch() == cm->getVersion().epoch() &&
- css->getMetadata()->getCollVersion() >= cm->getVersion()) {
- LOG(1) << "Skipping refresh of metadata for " << nss << " "
- << css->getMetadata()->getCollVersion() << " with an older " << cm->getVersion();
- return css->getMetadata()->getShardVersion();
+ if (metadata && metadata->getCollVersion().epoch() == cm->getVersion().epoch() &&
+ metadata->getCollVersion() >= cm->getVersion()) {
+ LOG(1) << "Skipping refresh of metadata for " << nss << " " << metadata->getCollVersion()
+ << " with an older " << cm->getVersion();
+ return metadata->getShardVersion();
}
std::unique_ptr<CollectionMetadata> newCollectionMetadata =
@@ -153,7 +152,7 @@ ChunkVersion forceShardFilteringMetadataRefresh(OperationContext* opCtx,
css->refreshMetadata(opCtx, std::move(newCollectionMetadata));
- return css->getMetadata()->getShardVersion();
+ return css->getMetadata(opCtx)->getShardVersion();
}
void onDbVersionMismatch(OperationContext* opCtx,
diff --git a/src/mongo/db/s/shard_server_op_observer.cpp b/src/mongo/db/s/shard_server_op_observer.cpp
index aabaeebbd77..73083621d78 100644
--- a/src/mongo/db/s/shard_server_op_observer.cpp
+++ b/src/mongo/db/s/shard_server_op_observer.cpp
@@ -202,7 +202,7 @@ void ShardServerOpObserver::onInserts(OperationContext* opCtx,
std::vector<InsertStatement>::const_iterator end,
bool fromMigrate) {
auto const css = CollectionShardingState::get(opCtx, nss);
- const auto metadata = css->getMetadata();
+ const auto metadata = css->getMetadata(opCtx);
for (auto it = begin; it != end; ++it) {
const auto& insertedDoc = it->doc;
@@ -228,7 +228,7 @@ void ShardServerOpObserver::onInserts(OperationContext* opCtx,
void ShardServerOpObserver::onUpdate(OperationContext* opCtx, const OplogUpdateEntryArgs& args) {
auto const css = CollectionShardingState::get(opCtx, args.nss);
- const auto metadata = css->getMetadata();
+ const auto metadata = css->getMetadata(opCtx);
if (args.nss.ns() == NamespaceString::kShardConfigCollectionsCollectionName) {
// Notification of routing table changes are only needed on secondaries
@@ -293,7 +293,7 @@ void ShardServerOpObserver::aboutToDelete(OperationContext* opCtx,
BSONObj const& doc) {
auto& deleteState = getDeleteState(opCtx);
auto* css = CollectionShardingState::get(opCtx, nss.ns());
- deleteState = css->makeDeleteState(doc);
+ deleteState = css->makeDeleteState(opCtx, doc);
}
void ShardServerOpObserver::onDelete(OperationContext* opCtx,
diff --git a/src/mongo/db/s/split_chunk.cpp b/src/mongo/db/s/split_chunk.cpp
index ca9da994e73..0b7e287a774 100644
--- a/src/mongo/db/s/split_chunk.cpp
+++ b/src/mongo/db/s/split_chunk.cpp
@@ -98,7 +98,7 @@ bool checkMetadataForSuccessfulSplitChunk(OperationContext* opCtx,
const std::vector<BSONObj>& splitKeys) {
const auto metadataAfterSplit = [&] {
AutoGetCollection autoColl(opCtx, nss, MODE_IS);
- return CollectionShardingState::get(opCtx, nss)->getMetadata();
+ return CollectionShardingState::get(opCtx, nss)->getMetadata(opCtx);
}();
uassert(ErrorCodes::StaleEpoch,