summaryrefslogtreecommitdiff
diff options
context:
space:
mode:
-rw-r--r--src/mongo/db/catalog/capped_utils.cpp8
-rw-r--r--src/mongo/db/catalog/catalog_control.cpp28
-rw-r--r--src/mongo/db/catalog/collection_catalog.cpp371
-rw-r--r--src/mongo/db/catalog/collection_catalog.h129
-rw-r--r--src/mongo/db/catalog/collection_catalog_helper.cpp15
-rw-r--r--src/mongo/db/catalog/collection_catalog_test.cpp69
-rw-r--r--src/mongo/db/catalog/collection_compact.cpp4
-rw-r--r--src/mongo/db/catalog/create_collection.cpp45
-rw-r--r--src/mongo/db/catalog/create_collection_test.cpp4
-rw-r--r--src/mongo/db/catalog/database.h3
-rw-r--r--src/mongo/db/catalog/database_holder_impl.cpp17
-rw-r--r--src/mongo/db/catalog/database_impl.cpp32
-rw-r--r--src/mongo/db/catalog/database_impl.h8
-rw-r--r--src/mongo/db/catalog/database_test.cpp17
-rw-r--r--src/mongo/db/catalog/drop_collection.cpp16
-rw-r--r--src/mongo/db/catalog/drop_database.cpp6
-rw-r--r--src/mongo/db/catalog/index_builds_manager.cpp6
-rw-r--r--src/mongo/db/catalog/multi_index_block.cpp2
-rw-r--r--src/mongo/db/catalog/rename_collection.cpp41
-rw-r--r--src/mongo/db/catalog/rename_collection_test.cpp11
-rw-r--r--src/mongo/db/catalog/uncommitted_collections.cpp11
-rw-r--r--src/mongo/db/catalog/validate_state.cpp4
-rw-r--r--src/mongo/db/catalog_raii.cpp39
-rw-r--r--src/mongo/db/catalog_raii.h29
-rw-r--r--src/mongo/db/cloner.cpp11
-rw-r--r--src/mongo/db/commands/count_cmd.cpp2
-rw-r--r--src/mongo/db/commands/create_indexes.cpp2
-rw-r--r--src/mongo/db/commands/dbcheck.cpp5
-rw-r--r--src/mongo/db/commands/dbcommands.cpp2
-rw-r--r--src/mongo/db/commands/dbcommands_d.cpp12
-rw-r--r--src/mongo/db/commands/dbhash.cpp2
-rw-r--r--src/mongo/db/commands/distinct.cpp2
-rw-r--r--src/mongo/db/commands/find_and_modify.cpp8
-rw-r--r--src/mongo/db/commands/find_cmd.cpp2
-rw-r--r--src/mongo/db/commands/getmore_cmd.cpp4
-rw-r--r--src/mongo/db/commands/killcursors_cmd.cpp2
-rw-r--r--src/mongo/db/commands/list_collections.cpp2
-rw-r--r--src/mongo/db/commands/list_databases.cpp2
-rw-r--r--src/mongo/db/commands/list_indexes.cpp2
-rw-r--r--src/mongo/db/commands/mr_test.cpp2
-rw-r--r--src/mongo/db/commands/oplog_application_checks.cpp4
-rw-r--r--src/mongo/db/commands/profile_common.cpp2
-rw-r--r--src/mongo/db/commands/run_aggregate.cpp5
-rw-r--r--src/mongo/db/commands/set_feature_compatibility_version_command.cpp8
-rw-r--r--src/mongo/db/commands/test_commands.cpp2
-rw-r--r--src/mongo/db/concurrency/d_concurrency.cpp5
-rw-r--r--src/mongo/db/concurrency/lock_manager.cpp4
-rw-r--r--src/mongo/db/curop.cpp2
-rw-r--r--src/mongo/db/curop.h2
-rw-r--r--src/mongo/db/db_raii.cpp14
-rw-r--r--src/mongo/db/db_raii.h1
-rw-r--r--src/mongo/db/dbhelpers.cpp4
-rw-r--r--src/mongo/db/exec/requires_collection_stage.cpp4
-rw-r--r--src/mongo/db/exec/requires_collection_stage.h2
-rw-r--r--src/mongo/db/free_mon/free_mon_mongod.cpp4
-rw-r--r--src/mongo/db/index_build_entry_helpers.cpp2
-rw-r--r--src/mongo/db/index_builds_coordinator.cpp16
-rw-r--r--src/mongo/db/index_builds_coordinator_mongod.cpp2
-rw-r--r--src/mongo/db/introspect.cpp5
-rw-r--r--src/mongo/db/matcher/expression_text.cpp2
-rw-r--r--src/mongo/db/mongod_main.cpp4
-rw-r--r--src/mongo/db/op_observer_impl.cpp2
-rw-r--r--src/mongo/db/ops/update.cpp2
-rw-r--r--src/mongo/db/ops/write_ops_exec.cpp8
-rw-r--r--src/mongo/db/pipeline/document_source_change_stream_test.cpp60
-rw-r--r--src/mongo/db/pipeline/document_source_cursor.cpp2
-rw-r--r--src/mongo/db/pipeline/process_interface/common_mongod_process_interface.cpp2
-rw-r--r--src/mongo/db/query/find.cpp4
-rw-r--r--src/mongo/db/query/plan_insert_listener.cpp2
-rw-r--r--src/mongo/db/repair.cpp8
-rw-r--r--src/mongo/db/repl/apply_ops.cpp5
-rw-r--r--src/mongo/db/repl/dbcheck.cpp8
-rw-r--r--src/mongo/db/repl/idempotency_test_fixture.cpp6
-rw-r--r--src/mongo/db/repl/oplog.cpp22
-rw-r--r--src/mongo/db/repl/oplog_applier_utils.cpp6
-rw-r--r--src/mongo/db/repl/replication_coordinator_impl.cpp2
-rw-r--r--src/mongo/db/repl/replication_recovery.cpp2
-rw-r--r--src/mongo/db/repl/rollback_impl.cpp12
-rw-r--r--src/mongo/db/repl/rs_rollback.cpp28
-rw-r--r--src/mongo/db/repl/rs_rollback_test.cpp36
-rw-r--r--src/mongo/db/repl/storage_interface_impl.cpp12
-rw-r--r--src/mongo/db/run_op_kill_cursors.cpp2
-rw-r--r--src/mongo/db/s/migration_destination_manager.cpp4
-rw-r--r--src/mongo/db/s/migration_util.cpp8
-rw-r--r--src/mongo/db/s/resharding/resharding_oplog_fetcher.cpp2
-rw-r--r--src/mongo/db/s/resharding_destined_recipient_test.cpp2
-rw-r--r--src/mongo/db/s/resharding_util.cpp4
-rw-r--r--src/mongo/db/s/set_shard_version_command.cpp2
-rw-r--r--src/mongo/db/startup_recovery.cpp21
-rw-r--r--src/mongo/db/storage/durable_catalog_impl.cpp4
-rw-r--r--src/mongo/db/storage/kv/durable_catalog_test.cpp5
-rw-r--r--src/mongo/db/storage/storage_engine_impl.cpp41
-rw-r--r--src/mongo/db/storage/storage_engine_test_fixture.h12
-rw-r--r--src/mongo/db/storage/storage_util.cpp98
-rw-r--r--src/mongo/db/storage/wiredtiger/oplog_stones_server_status_section.cpp2
-rw-r--r--src/mongo/db/storage/wiredtiger/wiredtiger_kv_engine.cpp2
-rw-r--r--src/mongo/db/transaction_history_iterator.cpp2
-rw-r--r--src/mongo/db/transaction_participant.cpp5
-rw-r--r--src/mongo/db/transaction_participant_test.cpp4
-rw-r--r--src/mongo/db/ttl.cpp6
-rw-r--r--src/mongo/db/views/durable_view_catalog.cpp10
-rw-r--r--src/mongo/db/views/view_catalog.cpp26
-rw-r--r--src/mongo/db/views/view_catalog_test.cpp33
-rw-r--r--src/mongo/dbtests/catalogtests.cpp2
-rw-r--r--src/mongo/dbtests/counttests.cpp2
-rw-r--r--src/mongo/dbtests/dbtests.cpp2
-rw-r--r--src/mongo/dbtests/dbtests.h2
-rw-r--r--src/mongo/dbtests/pdfiletests.cpp4
-rw-r--r--src/mongo/dbtests/plan_executor_invalidation_test.cpp4
-rw-r--r--src/mongo/dbtests/query_plan_executor.cpp4
-rw-r--r--src/mongo/dbtests/query_stage_count_scan.cpp2
-rw-r--r--src/mongo/dbtests/query_stage_fetch.cpp4
-rw-r--r--src/mongo/dbtests/query_stage_merge_sort.cpp20
-rw-r--r--src/mongo/dbtests/query_stage_sort.cpp12
-rw-r--r--src/mongo/dbtests/query_stage_update.cpp2
-rw-r--r--src/mongo/dbtests/querytests.cpp4
-rw-r--r--src/mongo/dbtests/repltests.cpp8
-rw-r--r--src/mongo/dbtests/rollbacktests.cpp16
-rw-r--r--src/mongo/dbtests/storage_timestamp_tests.cpp2
-rw-r--r--src/mongo/s/commands/cluster_profile_cmd.cpp6
120 files changed, 896 insertions, 795 deletions
diff --git a/src/mongo/db/catalog/capped_utils.cpp b/src/mongo/db/catalog/capped_utils.cpp
index c635d7cf5e9..0baba04783f 100644
--- a/src/mongo/db/catalog/capped_utils.cpp
+++ b/src/mongo/db/catalog/capped_utils.cpp
@@ -122,7 +122,7 @@ void cloneCollectionAsCapped(OperationContext* opCtx,
long long size,
bool temp) {
CollectionPtr fromCollection =
- CollectionCatalog::get(opCtx).lookupCollectionByNamespace(opCtx, fromNss);
+ CollectionCatalog::get(opCtx)->lookupCollectionByNamespace(opCtx, fromNss);
if (!fromCollection) {
uassert(ErrorCodes::CommandNotSupportedOnView,
str::stream() << "cloneCollectionAsCapped not supported for views: " << fromNss,
@@ -140,7 +140,7 @@ void cloneCollectionAsCapped(OperationContext* opCtx,
uassert(ErrorCodes::NamespaceExists,
str::stream() << "cloneCollectionAsCapped failed - destination collection " << toNss
<< " already exists. source collection: " << fromNss,
- !CollectionCatalog::get(opCtx).lookupCollectionByNamespace(opCtx, toNss));
+ !CollectionCatalog::get(opCtx)->lookupCollectionByNamespace(opCtx, toNss));
// create new collection
{
@@ -161,7 +161,7 @@ void cloneCollectionAsCapped(OperationContext* opCtx,
}
CollectionPtr toCollection =
- CollectionCatalog::get(opCtx).lookupCollectionByNamespace(opCtx, toNss);
+ CollectionCatalog::get(opCtx)->lookupCollectionByNamespace(opCtx, toNss);
invariant(toCollection); // we created above
// how much data to ignore because it won't fit anyway
@@ -279,7 +279,7 @@ void convertToCapped(OperationContext* opCtx, const NamespaceString& ns, long lo
<< " to a capped collection");
collLock.emplace(opCtx, tmpNameResult.getValue(), MODE_X);
- if (!CollectionCatalog::get(opCtx).lookupCollectionByNamespace(
+ if (!CollectionCatalog::get(opCtx)->lookupCollectionByNamespace(
opCtx, tmpNameResult.getValue())) {
return std::move(tmpNameResult.getValue());
}
diff --git a/src/mongo/db/catalog/catalog_control.cpp b/src/mongo/db/catalog/catalog_control.cpp
index 2b243b53c08..f2caab96161 100644
--- a/src/mongo/db/catalog/catalog_control.cpp
+++ b/src/mongo/db/catalog/catalog_control.cpp
@@ -57,9 +57,9 @@ MinVisibleTimestampMap closeCatalog(OperationContext* opCtx) {
opCtx->getServiceContext()->getStorageEngine()->listDatabases();
auto databaseHolder = DatabaseHolder::get(opCtx);
+ auto catalog = CollectionCatalog::get(opCtx);
for (auto&& dbName : allDbs) {
- const auto db = databaseHolder->getDb(opCtx, dbName);
- for (auto collIt = db->begin(opCtx); collIt != db->end(opCtx); ++collIt) {
+ for (auto collIt = catalog->begin(opCtx, dbName); collIt != catalog->end(opCtx); ++collIt) {
auto coll = *collIt;
if (!coll) {
break;
@@ -83,13 +83,17 @@ MinVisibleTimestampMap closeCatalog(OperationContext* opCtx) {
}
// Need to mark the CollectionCatalog as open if we our closeAll fails, dismissed if successful.
- auto reopenOnFailure =
- makeGuard([opCtx] { CollectionCatalog::get(opCtx).onOpenCatalog(opCtx); });
+ auto reopenOnFailure = makeGuard([opCtx] {
+ CollectionCatalog::write(opCtx,
+ [&](CollectionCatalog& catalog) { catalog.onOpenCatalog(opCtx); });
+ });
// Closing CollectionCatalog: only lookupNSSByUUID will fall back to using pre-closing state to
// allow authorization for currently unknown UUIDs. This is needed because authorization needs
// to work before acquiring locks, and might otherwise spuriously regard a UUID as unknown
// while reloading the catalog.
- CollectionCatalog::get(opCtx).onCloseCatalog(opCtx);
+ CollectionCatalog::write(opCtx,
+ [&](CollectionCatalog& catalog) { catalog.onCloseCatalog(opCtx); });
+
LOGV2_DEBUG(20270, 1, "closeCatalog: closing collection catalog");
// Close all databases.
@@ -154,10 +158,11 @@ void openCatalog(OperationContext* opCtx,
ino.second.emplace_back(std::move(indexesToRebuild.second.back()));
}
+ auto catalog = CollectionCatalog::get(opCtx);
for (const auto& entry : nsToIndexNameObjMap) {
NamespaceString collNss(entry.first);
- auto collection = CollectionCatalog::get(opCtx).lookupCollectionByNamespace(opCtx, collNss);
+ auto collection = catalog->lookupCollectionByNamespace(opCtx, collNss);
invariant(collection, str::stream() << "couldn't get collection " << collNss.toString());
for (const auto& indexName : entry.second.first) {
@@ -188,12 +193,10 @@ void openCatalog(OperationContext* opCtx,
23992, 1, "openCatalog: dbholder reopening database", "db"_attr = dbName);
auto db = databaseHolder->openDb(opCtx, dbName);
invariant(db, str::stream() << "failed to reopen database " << dbName);
- for (auto&& collNss :
- CollectionCatalog::get(opCtx).getAllCollectionNamesFromDb(opCtx, dbName)) {
+ for (auto&& collNss : catalog->getAllCollectionNamesFromDb(opCtx, dbName)) {
// Note that the collection name already includes the database component.
- auto collection =
- CollectionCatalog::get(opCtx).lookupCollectionByNamespaceForMetadataWrite(
- opCtx, CollectionCatalog::LifetimeMode::kInplace, collNss);
+ auto collection = catalog->lookupCollectionByNamespaceForMetadataWrite(
+ opCtx, CollectionCatalog::LifetimeMode::kInplace, collNss);
invariant(collection,
str::stream()
<< "failed to get valid collection pointer for namespace " << collNss);
@@ -224,7 +227,8 @@ void openCatalog(OperationContext* opCtx,
// Opening CollectionCatalog: The collection catalog is now in sync with the storage engine
// catalog. Clear the pre-closing state.
- CollectionCatalog::get(opCtx).onOpenCatalog(opCtx);
+ CollectionCatalog::write(opCtx,
+ [&](CollectionCatalog& catalog) { catalog.onOpenCatalog(opCtx); });
opCtx->getServiceContext()->incrementCatalogGeneration();
LOGV2(20278, "openCatalog: finished reloading collection catalog");
}
diff --git a/src/mongo/db/catalog/collection_catalog.cpp b/src/mongo/db/catalog/collection_catalog.cpp
index ac3104627a1..e6e28b0d920 100644
--- a/src/mongo/db/catalog/collection_catalog.cpp
+++ b/src/mongo/db/catalog/collection_catalog.cpp
@@ -44,8 +44,11 @@
namespace mongo {
namespace {
-const ServiceContext::Decoration<CollectionCatalog> getCatalog =
- ServiceContext::declareDecoration<CollectionCatalog>();
+struct LatestCollectionCatalog {
+ std::shared_ptr<CollectionCatalog> catalog = std::make_shared<CollectionCatalog>();
+};
+const ServiceContext::Decoration<LatestCollectionCatalog> getCatalog =
+ ServiceContext::declareDecoration<LatestCollectionCatalog>();
/**
* Decoration on OperationContext to store cloned Collections until they are committed or rolled
@@ -53,7 +56,7 @@ const ServiceContext::Decoration<CollectionCatalog> getCatalog =
*/
class UncommittedWritableCollections {
public:
- using CommitFn = std::function<void(boost::optional<Timestamp>)>;
+ using CommitFn = std::function<void(CollectionCatalog&, boost::optional<Timestamp>)>;
/**
* Lookup of Collection by UUID
@@ -155,23 +158,28 @@ const OperationContext::Decoration<UncommittedWritableCollections>
getUncommittedWritableCollections =
OperationContext::declareDecoration<UncommittedWritableCollections>();
+const OperationContext::Decoration<std::shared_ptr<const CollectionCatalog>> stashedCatalog =
+ OperationContext::declareDecoration<std::shared_ptr<const CollectionCatalog>>();
+
class FinishDropCollectionChange : public RecoveryUnit::Change {
public:
- FinishDropCollectionChange(CollectionCatalog* catalog,
+ FinishDropCollectionChange(OperationContext* opCtx,
std::shared_ptr<Collection> coll,
CollectionUUID uuid)
- : _catalog(catalog), _coll(std::move(coll)), _uuid(uuid) {}
+ : _opCtx(opCtx), _coll(std::move(coll)), _uuid(uuid) {}
void commit(boost::optional<Timestamp>) override {
_coll.reset();
}
void rollback() override {
- _catalog->registerCollection(_uuid, std::move(_coll));
+ CollectionCatalog::write(_opCtx, [&](CollectionCatalog& catalog) {
+ catalog.registerCollection(_uuid, std::move(_coll));
+ });
}
private:
- CollectionCatalog* _catalog;
+ OperationContext* _opCtx;
std::shared_ptr<Collection> _coll;
CollectionUUID _uuid;
};
@@ -180,12 +188,10 @@ private:
CollectionCatalog::iterator::iterator(OperationContext* opCtx,
StringData dbName,
- uint64_t genNum,
const CollectionCatalog& catalog)
- : _opCtx(opCtx), _dbName(dbName), _genNum(genNum), _catalog(&catalog) {
+ : _opCtx(opCtx), _dbName(dbName), _catalog(&catalog) {
auto minUuid = UUID::parse("00000000-0000-0000-0000-000000000000").getValue();
- stdx::lock_guard<Latch> lock(_catalog->_catalogLock);
_mapIter = _catalog->_orderedCollections.lower_bound(std::make_pair(_dbName, minUuid));
// Start with the first collection that is visible outside of its transaction.
@@ -201,13 +207,10 @@ CollectionCatalog::iterator::iterator(OperationContext* opCtx,
CollectionCatalog::iterator::iterator(OperationContext* opCtx,
std::map<std::pair<std::string, CollectionUUID>,
std::shared_ptr<Collection>>::const_iterator mapIter,
- uint64_t genNum,
const CollectionCatalog& catalog)
- : _opCtx(opCtx), _genNum(genNum), _mapIter(mapIter), _catalog(&catalog) {}
+ : _opCtx(opCtx), _mapIter(mapIter), _catalog(&catalog) {}
CollectionCatalog::iterator::value_type CollectionCatalog::iterator::operator*() {
- stdx::lock_guard<Latch> lock(_catalog->_catalogLock);
- _repositionIfNeeded();
if (_exhausted()) {
return CollectionPtr();
}
@@ -217,7 +220,7 @@ CollectionCatalog::iterator::value_type CollectionCatalog::iterator::operator*()
Collection* CollectionCatalog::iterator::getWritableCollection(OperationContext* opCtx,
LifetimeMode mode) {
- return CollectionCatalog::get(opCtx).lookupCollectionByUUIDForMetadataWrite(
+ return CollectionCatalog::get(opCtx)->lookupCollectionByUUIDForMetadataWrite(
opCtx, mode, operator*()->uuid());
}
@@ -226,11 +229,7 @@ boost::optional<CollectionUUID> CollectionCatalog::iterator::uuid() {
}
CollectionCatalog::iterator CollectionCatalog::iterator::operator++() {
- stdx::lock_guard<Latch> lock(_catalog->_catalogLock);
-
- if (!_repositionIfNeeded()) {
- _mapIter++; // If the position was not updated, increment iterator to next element.
- }
+ _mapIter++;
// Skip any collections that are not yet visible outside of their respective transactions.
while (!_exhausted() && !_mapIter->second->isCommitted()) {
@@ -256,8 +255,7 @@ CollectionCatalog::iterator CollectionCatalog::iterator::operator++(int) {
}
bool CollectionCatalog::iterator::operator==(const iterator& other) {
- stdx::lock_guard<Latch> lock(_catalog->_catalogLock);
-
+ invariant(_catalog == other._catalog);
if (other._mapIter == _catalog->_orderedCollections.end()) {
return _uuid == boost::none;
}
@@ -269,56 +267,158 @@ bool CollectionCatalog::iterator::operator!=(const iterator& other) {
return !(*this == other);
}
-bool CollectionCatalog::iterator::_repositionIfNeeded() {
- if (_genNum == _catalog->_generationNumber) {
- return false;
- }
+bool CollectionCatalog::iterator::_exhausted() {
+ return _mapIter == _catalog->_orderedCollections.end() || _mapIter->first.first != _dbName;
+}
- _genNum = _catalog->_generationNumber;
- // If the map has been modified, find the entry the iterator was on, or the one right after it.
- // The entry the iterator was on must have been for a collection visible outside of its
- // transaction.
- _mapIter = _catalog->_orderedCollections.lower_bound(std::make_pair(_dbName, *_uuid));
+std::shared_ptr<const CollectionCatalog> CollectionCatalog::get(ServiceContext* svcCtx) {
+ return atomic_load(&getCatalog(svcCtx).catalog);
+}
- while (!_exhausted() && !_mapIter->second->isCommitted()) {
- _mapIter++;
+std::shared_ptr<const CollectionCatalog> CollectionCatalog::get(OperationContext* opCtx) {
+ const auto& stashed = stashedCatalog(opCtx);
+ if (stashed)
+ return stashed;
+ return get(opCtx->getServiceContext());
+}
+
+void CollectionCatalog::stash(OperationContext* opCtx,
+ std::shared_ptr<const CollectionCatalog> catalog) {
+ stashedCatalog(opCtx) = std::move(catalog);
+}
+
+void CollectionCatalog::write(ServiceContext* svcCtx, CatalogWriteFn job) {
+ // It is potentially expensive to copy the collection catalog so we batch the operations by only
+ // having one concurrent thread copying the catalog and executing all the write jobs.
+
+ struct JobEntry {
+ JobEntry(CatalogWriteFn write) : job(std::move(write)) {}
+
+ CatalogWriteFn job;
+
+ struct CompletionInfo {
+ // Used to wait for job to complete by worker thread
+ Mutex mutex;
+ stdx::condition_variable cv;
+
+ // Exception storage if we threw during job execution, so we can transfer the exception
+ // back to the calling thread
+ std::exception_ptr exception;
+
+ // The job is completed when the catalog we modified has been committed back to the
+ // storage or if we threw during its execution
+ bool completed = false;
+ };
+
+ // Shared state for completion info as JobEntry's gets deleted when we are finished
+ // executing. No shared state means that this job belongs to the same thread executing them.
+ std::shared_ptr<CompletionInfo> completion;
+ };
+
+ static std::list<JobEntry> queue;
+ static bool workerExists = false;
+ static Mutex mutex =
+ MONGO_MAKE_LATCH("CollectionCatalog::write"); // Protecting the two globals above
+
+ invariant(job);
+
+ // Current batch of jobs to execute
+ std::list<JobEntry> pending;
+ {
+ stdx::unique_lock lock(mutex);
+ queue.emplace_back(std::move(job));
+
+ // If worker already exists, then wait on our condition variable until the job is completed
+ if (workerExists) {
+ auto completion = std::make_shared<JobEntry::CompletionInfo>();
+ queue.back().completion = completion;
+ lock.unlock();
+
+ stdx::unique_lock completionLock(completion->mutex);
+ const bool& completed = completion->completed;
+ completion->cv.wait(completionLock, [&completed]() { return completed; });
+
+ // Throw any exception that was caught during execution of our job
+ if (completion->exception)
+ std::rethrow_exception(completion->exception);
+ return;
+ }
+
+ // No worker existed, then we take this responsibility
+ workerExists = true;
+ pending.splice(pending.end(), queue);
}
- if (_exhausted()) {
- return true;
+ // Implementation for thread with worker responsibility below, only one thread at a time can be
+ // in here. Keep track of completed jobs so we can notify them when we've written back the
+ // catalog to storage
+ std::list<JobEntry> completed;
+ std::exception_ptr myException;
+
+ auto& storage = getCatalog(svcCtx);
+ // hold onto base so if we need to delete it we can do it outside of the lock
+ auto base = atomic_load(&storage.catalog);
+ // copy the collection catalog, this could be expensive, but we will only have one pending
+ // collection in flight at a given time
+ auto clone = std::make_shared<CollectionCatalog>(*base);
+
+ // Execute jobs until we drain the queue
+ while (true) {
+ for (auto&& current : pending) {
+ // Store any exception thrown during job execution so we can notify the calling thread
+ try {
+ current.job(*clone);
+ } catch (...) {
+ if (current.completion)
+ current.completion->exception = std::current_exception();
+ else
+ myException = std::current_exception();
+ }
+ }
+ // Transfer the jobs we just executed to the completed list
+ completed.splice(completed.end(), pending);
+
+ stdx::lock_guard lock(mutex);
+ if (queue.empty()) {
+ // Queue is empty, store catalog and relinquish responsibility of being worker thread
+ atomic_store(&storage.catalog, std::move(clone));
+ workerExists = false;
+ break;
+ }
+
+ // Transfer jobs in queue to the pending list
+ pending.splice(pending.end(), queue);
}
- // If the old pair matches the previous DB name and UUID, the iterator was not repositioned.
- auto dbUuidPair = _mapIter->first;
- bool repositioned = !(dbUuidPair.first == _dbName && dbUuidPair.second == _uuid);
- _uuid = dbUuidPair.second;
+ for (auto&& entry : completed) {
+ if (!entry.completion) {
+ continue;
+ }
- return repositioned;
+ stdx::lock_guard completionLock(entry.completion->mutex);
+ entry.completion->completed = true;
+ entry.completion->cv.notify_one();
+ }
+ LOGV2_DEBUG(
+ 5255601, 1, "Finished writing to the CollectionCatalog", "jobs"_attr = completed.size());
+ if (myException)
+ std::rethrow_exception(myException);
}
-bool CollectionCatalog::iterator::_exhausted() {
- return _mapIter == _catalog->_orderedCollections.end() || _mapIter->first.first != _dbName;
+void CollectionCatalog::write(OperationContext* opCtx,
+ std::function<void(CollectionCatalog&)> job) {
+ write(opCtx->getServiceContext(), std::move(job));
}
-CollectionCatalog& CollectionCatalog::get(ServiceContext* svcCtx) {
- return getCatalog(svcCtx);
-}
-CollectionCatalog& CollectionCatalog::get(OperationContext* opCtx) {
- return getCatalog(opCtx->getServiceContext());
-}
void CollectionCatalog::setCollectionNamespace(OperationContext* opCtx,
Collection* coll,
const NamespaceString& fromCollection,
- const NamespaceString& toCollection) {
+ const NamespaceString& toCollection) const {
// Rather than maintain, in addition to the UUID -> Collection* mapping, an auxiliary
// data structure with the UUID -> namespace mapping, the CollectionCatalog relies on
// Collection::ns() to provide UUID to namespace lookup. In addition, the CollectionCatalog
// does not require callers to hold locks.
- //
- // This means that Collection::ns() may be called while only '_catalogLock' (and no lock
- // manager locks) are held. The purpose of this function is ensure that we write to the
- // Collection's namespace string under '_catalogLock'.
invariant(coll);
coll->setNs(toCollection);
@@ -326,14 +426,15 @@ void CollectionCatalog::setCollectionNamespace(OperationContext* opCtx,
uncommittedWritableCollections.rename(
coll,
fromCollection,
- [this, coll, fromCollection, toCollection](boost::optional<Timestamp> commitTime) {
- _collections.erase(fromCollection);
+ [opCtx, coll, fromCollection, toCollection](CollectionCatalog& writableCatalog,
+ boost::optional<Timestamp> commitTime) {
+ writableCatalog._collections.erase(fromCollection);
ResourceId oldRid = ResourceId(RESOURCE_COLLECTION, fromCollection.ns());
ResourceId newRid = ResourceId(RESOURCE_COLLECTION, toCollection.ns());
- removeResource(oldRid, fromCollection.ns());
- addResource(newRid, toCollection.ns());
+ writableCatalog.removeResource(oldRid, fromCollection.ns());
+ writableCatalog.addResource(newRid, toCollection.ns());
// Ban reading from this collection on committed reads on snapshots before now.
if (commitTime) {
@@ -371,7 +472,6 @@ void CollectionCatalog::onCloseDatabase(OperationContext* opCtx, std::string dbN
void CollectionCatalog::onCloseCatalog(OperationContext* opCtx) {
invariant(opCtx->lockState()->isW());
- stdx::lock_guard<Latch> lock(_catalogLock);
invariant(!_shadowCatalog);
_shadowCatalog.emplace();
for (auto& entry : _catalog)
@@ -380,7 +480,6 @@ void CollectionCatalog::onCloseCatalog(OperationContext* opCtx) {
void CollectionCatalog::onOpenCatalog(OperationContext* opCtx) {
invariant(opCtx->lockState()->isW());
- stdx::lock_guard<Latch> lock(_catalogLock);
invariant(_shadowCatalog);
_shadowCatalog.reset();
++_epoch;
@@ -396,14 +495,13 @@ std::shared_ptr<const Collection> CollectionCatalog::lookupCollectionByUUIDForRe
return coll;
}
- stdx::lock_guard<Latch> lock(_catalogLock);
- auto coll = _lookupCollectionByUUID(lock, uuid);
+ auto coll = _lookupCollectionByUUID(uuid);
return (coll && coll->isCommitted()) ? coll : nullptr;
}
Collection* CollectionCatalog::lookupCollectionByUUIDForMetadataWrite(OperationContext* opCtx,
LifetimeMode mode,
- CollectionUUID uuid) {
+ CollectionUUID uuid) const {
if (mode == LifetimeMode::kInplace) {
return const_cast<Collection*>(lookupCollectionByUUID(opCtx, uuid).get());
}
@@ -418,14 +516,10 @@ Collection* CollectionCatalog::lookupCollectionByUUIDForMetadataWrite(OperationC
return coll.get();
}
- std::shared_ptr<Collection> coll;
- {
- stdx::lock_guard<Latch> lock(_catalogLock);
- coll = _lookupCollectionByUUID(lock, uuid);
+ std::shared_ptr<Collection> coll = _lookupCollectionByUUID(uuid);
- if (!coll || !coll->isCommitted())
- return nullptr;
- }
+ if (!coll || !coll->isCommitted())
+ return nullptr;
if (coll->ns().isOplog())
return coll.get();
@@ -436,12 +530,19 @@ Collection* CollectionCatalog::lookupCollectionByUUIDForMetadataWrite(OperationC
if (mode == LifetimeMode::kManagedInWriteUnitOfWork) {
opCtx->recoveryUnit()->onCommit(
- [this, &uncommittedWritableCollections, clonedPtr = cloned.get()](
+ [opCtx, &uncommittedWritableCollections, clonedPtr = cloned.get()](
boost::optional<Timestamp> commitTime) {
auto [collection, commitHandlers] =
uncommittedWritableCollections.remove(clonedPtr);
if (collection) {
- _commitWritableClone(std::move(collection), commitTime, commitHandlers);
+ CollectionCatalog::write(
+ opCtx,
+ [collection = std::move(collection),
+ &commitTime,
+ commitHandlers = &commitHandlers](CollectionCatalog& catalog) {
+ catalog._commitWritableClone(
+ std::move(collection), commitTime, *commitHandlers);
+ });
}
});
opCtx->recoveryUnit()->onRollback([&uncommittedWritableCollections, cloned]() {
@@ -463,27 +564,18 @@ CollectionPtr CollectionCatalog::lookupCollectionByUUID(OperationContext* opCtx,
return {opCtx, coll.get(), LookupCollectionForYieldRestore()};
}
- stdx::lock_guard<Latch> lock(_catalogLock);
- auto coll = _lookupCollectionByUUID(lock, uuid);
+ auto coll = _lookupCollectionByUUID(uuid);
return (coll && coll->isCommitted())
? CollectionPtr(opCtx, coll.get(), LookupCollectionForYieldRestore())
: CollectionPtr();
}
-void CollectionCatalog::makeCollectionVisible(CollectionUUID uuid) {
- stdx::lock_guard<Latch> lock(_catalogLock);
- auto coll = _lookupCollectionByUUID(lock, uuid);
- coll->setCommitted(true);
-}
-
bool CollectionCatalog::isCollectionAwaitingVisibility(CollectionUUID uuid) const {
- stdx::lock_guard<Latch> lock(_catalogLock);
- auto coll = _lookupCollectionByUUID(lock, uuid);
+ auto coll = _lookupCollectionByUUID(uuid);
return coll && !coll->isCommitted();
}
-std::shared_ptr<Collection> CollectionCatalog::_lookupCollectionByUUID(WithLock,
- CollectionUUID uuid) const {
+std::shared_ptr<Collection> CollectionCatalog::_lookupCollectionByUUID(CollectionUUID uuid) const {
auto foundIt = _catalog.find(uuid);
return foundIt == _catalog.end() ? nullptr : foundIt->second;
}
@@ -494,14 +586,13 @@ std::shared_ptr<const Collection> CollectionCatalog::lookupCollectionByNamespace
return coll;
}
- stdx::lock_guard<Latch> lock(_catalogLock);
auto it = _collections.find(nss);
auto coll = (it == _collections.end() ? nullptr : it->second);
return (coll && coll->isCommitted()) ? coll : nullptr;
}
Collection* CollectionCatalog::lookupCollectionByNamespaceForMetadataWrite(
- OperationContext* opCtx, LifetimeMode mode, const NamespaceString& nss) {
+ OperationContext* opCtx, LifetimeMode mode, const NamespaceString& nss) const {
if (mode == LifetimeMode::kInplace || nss.isOplog()) {
return const_cast<Collection*>(lookupCollectionByNamespace(opCtx, nss).get());
}
@@ -517,15 +608,11 @@ Collection* CollectionCatalog::lookupCollectionByNamespaceForMetadataWrite(
return coll.get();
}
- std::shared_ptr<Collection> coll;
- {
- stdx::lock_guard<Latch> lock(_catalogLock);
- auto it = _collections.find(nss);
- coll = (it == _collections.end() ? nullptr : it->second);
+ auto it = _collections.find(nss);
+ auto coll = (it == _collections.end() ? nullptr : it->second);
- if (!coll || !coll->isCommitted())
- return nullptr;
- }
+ if (!coll || !coll->isCommitted())
+ return nullptr;
invariant(opCtx->lockState()->isCollectionLockedForMode(nss, MODE_X));
auto cloned = coll->clone();
@@ -533,12 +620,19 @@ Collection* CollectionCatalog::lookupCollectionByNamespaceForMetadataWrite(
if (mode == LifetimeMode::kManagedInWriteUnitOfWork) {
opCtx->recoveryUnit()->onCommit(
- [this, &uncommittedWritableCollections, clonedPtr = cloned.get()](
+ [opCtx, &uncommittedWritableCollections, clonedPtr = cloned.get()](
boost::optional<Timestamp> commitTime) {
auto [collection, commitHandlers] =
uncommittedWritableCollections.remove(clonedPtr);
if (collection) {
- _commitWritableClone(std::move(collection), commitTime, commitHandlers);
+ CollectionCatalog::write(
+ opCtx,
+ [collection = std::move(collection),
+ &commitTime,
+ commitHandlers = &commitHandlers](CollectionCatalog& catalog) {
+ catalog._commitWritableClone(
+ std::move(collection), commitTime, *commitHandlers);
+ });
}
});
opCtx->recoveryUnit()->onRollback([&uncommittedWritableCollections, cloned]() {
@@ -561,7 +655,6 @@ CollectionPtr CollectionCatalog::lookupCollectionByNamespace(OperationContext* o
return {opCtx, coll.get(), LookupCollectionForYieldRestore()};
}
- stdx::lock_guard<Latch> lock(_catalogLock);
auto it = _collections.find(nss);
auto coll = (it == _collections.end() ? nullptr : it->second);
return (coll && coll->isCommitted())
@@ -579,7 +672,6 @@ boost::optional<NamespaceString> CollectionCatalog::lookupNSSByUUID(OperationCon
return coll->ns();
}
- stdx::lock_guard<Latch> lock(_catalogLock);
auto foundIt = _catalog.find(uuid);
if (foundIt != _catalog.end()) {
boost::optional<NamespaceString> ns = foundIt->second->ns();
@@ -611,7 +703,6 @@ boost::optional<CollectionUUID> CollectionCatalog::lookupUUIDByNSS(
return coll->uuid();
}
- stdx::lock_guard<Latch> lock(_catalogLock);
auto it = _collections.find(nss);
if (it != _collections.end()) {
boost::optional<CollectionUUID> uuid = it->second->uuid();
@@ -620,8 +711,8 @@ boost::optional<CollectionUUID> CollectionCatalog::lookupUUIDByNSS(
return boost::none;
}
-NamespaceString CollectionCatalog::resolveNamespaceStringOrUUID(OperationContext* opCtx,
- NamespaceStringOrUUID nsOrUUID) {
+NamespaceString CollectionCatalog::resolveNamespaceStringOrUUID(
+ OperationContext* opCtx, NamespaceStringOrUUID nsOrUUID) const {
if (auto& nss = nsOrUUID.nss()) {
uassert(ErrorCodes::InvalidNamespace,
str::stream() << "Namespace " << *nss << " is not a valid collection name",
@@ -647,8 +738,7 @@ bool CollectionCatalog::checkIfCollectionSatisfiable(CollectionUUID uuid,
CollectionInfoFn predicate) const {
invariant(predicate);
- stdx::lock_guard<Latch> lock(_catalogLock);
- auto collection = _lookupCollectionByUUID(lock, uuid);
+ auto collection = _lookupCollectionByUUID(uuid);
if (!collection) {
return false;
@@ -659,7 +749,6 @@ bool CollectionCatalog::checkIfCollectionSatisfiable(CollectionUUID uuid,
std::vector<CollectionUUID> CollectionCatalog::getAllCollectionUUIDsFromDb(
StringData dbName) const {
- stdx::lock_guard<Latch> lock(_catalogLock);
auto minUuid = UUID::parse("00000000-0000-0000-0000-000000000000").getValue();
auto it = _orderedCollections.lower_bound(std::make_pair(dbName.toString(), minUuid));
@@ -677,7 +766,6 @@ std::vector<NamespaceString> CollectionCatalog::getAllCollectionNamesFromDb(
OperationContext* opCtx, StringData dbName) const {
invariant(opCtx->lockState()->isDbLockedForMode(dbName, MODE_S));
- stdx::lock_guard<Latch> lock(_catalogLock);
auto minUuid = UUID::parse("00000000-0000-0000-0000-000000000000").getValue();
std::vector<NamespaceString> ret;
@@ -693,7 +781,6 @@ std::vector<NamespaceString> CollectionCatalog::getAllCollectionNamesFromDb(
std::vector<std::string> CollectionCatalog::getAllDbNames() const {
std::vector<std::string> ret;
- stdx::lock_guard<Latch> lock(_catalogLock);
auto maxUuid = UUID::parse("FFFFFFFF-FFFF-FFFF-FFFF-FFFFFFFFFFFF").getValue();
auto iter = _orderedCollections.upper_bound(std::make_pair("", maxUuid));
while (iter != _orderedCollections.end()) {
@@ -714,13 +801,11 @@ std::vector<std::string> CollectionCatalog::getAllDbNames() const {
void CollectionCatalog::setDatabaseProfileSettings(
StringData dbName, CollectionCatalog::ProfileSettings newProfileSettings) {
- stdx::lock_guard<Latch> lock(_profileSettingsLock);
_databaseProfileSettings[dbName] = newProfileSettings;
}
CollectionCatalog::ProfileSettings CollectionCatalog::getDatabaseProfileSettings(
StringData dbName) const {
- stdx::lock_guard<Latch> lock(_profileSettingsLock);
auto it = _databaseProfileSettings.find(dbName);
if (it != _databaseProfileSettings.end()) {
return it->second;
@@ -730,13 +815,11 @@ CollectionCatalog::ProfileSettings CollectionCatalog::getDatabaseProfileSettings
}
void CollectionCatalog::clearDatabaseProfileSettings(StringData dbName) {
- stdx::lock_guard<Latch> lock(_profileSettingsLock);
_databaseProfileSettings.erase(dbName);
}
void CollectionCatalog::registerCollection(CollectionUUID uuid, std::shared_ptr<Collection> coll) {
auto ns = coll->ns();
- stdx::lock_guard<Latch> lock(_catalogLock);
if (_collections.find(ns) != _collections.end()) {
LOGV2(20279,
"Conflicted creating a collection. ns: {coll_ns} ({coll_uuid}).",
@@ -772,8 +855,6 @@ void CollectionCatalog::registerCollection(CollectionUUID uuid, std::shared_ptr<
std::shared_ptr<Collection> CollectionCatalog::deregisterCollection(OperationContext* opCtx,
CollectionUUID uuid) {
- stdx::lock_guard<Latch> lock(_catalogLock);
-
invariant(_catalog.find(uuid) != _catalog.end());
auto coll = std::move(_catalog[uuid]);
@@ -800,21 +881,15 @@ std::shared_ptr<Collection> CollectionCatalog::deregisterCollection(OperationCon
auto collRid = ResourceId(RESOURCE_COLLECTION, ns.ns());
removeResource(collRid, ns.ns());
- // Removal from an ordered map will invalidate iterators and potentially references to the
- // erased element.
- _generationNumber++;
-
return coll;
}
std::unique_ptr<RecoveryUnit::Change> CollectionCatalog::makeFinishDropCollectionChange(
- std::shared_ptr<Collection> coll, CollectionUUID uuid) {
- return std::make_unique<FinishDropCollectionChange>(this, std::move(coll), uuid);
+ OperationContext* opCtx, std::shared_ptr<Collection> coll, CollectionUUID uuid) {
+ return std::make_unique<FinishDropCollectionChange>(opCtx, std::move(coll), uuid);
}
void CollectionCatalog::deregisterAllCollections() {
- stdx::lock_guard<Latch> lock(_catalogLock);
-
LOGV2(20282, "Deregistering all the collections");
for (auto& entry : _catalog) {
auto uuid = entry.first;
@@ -832,30 +907,26 @@ void CollectionCatalog::deregisterAllCollections() {
_orderedCollections.clear();
_catalog.clear();
- stdx::lock_guard<Latch> resourceLock(_resourceLock);
_resourceInformation.clear();
-
- _generationNumber++;
}
CollectionCatalog::iterator CollectionCatalog::begin(OperationContext* opCtx, StringData db) const {
- return iterator(opCtx, db, _generationNumber, *this);
+ return iterator(opCtx, db, *this);
}
CollectionCatalog::iterator CollectionCatalog::end(OperationContext* opCtx) const {
- return iterator(opCtx, _orderedCollections.end(), _generationNumber, *this);
+ return iterator(opCtx, _orderedCollections.end(), *this);
}
-boost::optional<std::string> CollectionCatalog::lookupResourceName(const ResourceId& rid) {
+boost::optional<std::string> CollectionCatalog::lookupResourceName(const ResourceId& rid) const {
invariant(rid.getType() == RESOURCE_DATABASE || rid.getType() == RESOURCE_COLLECTION);
- stdx::lock_guard<Latch> lock(_resourceLock);
auto search = _resourceInformation.find(rid);
if (search == _resourceInformation.end()) {
return boost::none;
}
- std::set<std::string>& namespaces = search->second;
+ const std::set<std::string>& namespaces = search->second;
// When there are multiple namespaces mapped to the same ResourceId, return boost::none as the
// ResourceId does not identify a single namespace.
@@ -868,7 +939,6 @@ boost::optional<std::string> CollectionCatalog::lookupResourceName(const Resourc
void CollectionCatalog::removeResource(const ResourceId& rid, const std::string& entry) {
invariant(rid.getType() == RESOURCE_DATABASE || rid.getType() == RESOURCE_COLLECTION);
- stdx::lock_guard<Latch> lock(_resourceLock);
auto search = _resourceInformation.find(rid);
if (search == _resourceInformation.end()) {
@@ -886,7 +956,6 @@ void CollectionCatalog::removeResource(const ResourceId& rid, const std::string&
void CollectionCatalog::addResource(const ResourceId& rid, const std::string& entry) {
invariant(rid.getType() == RESOURCE_DATABASE || rid.getType() == RESOURCE_COLLECTION);
- stdx::lock_guard<Latch> lock(_resourceLock);
auto search = _resourceInformation.find(rid);
if (search == _resourceInformation.end()) {
@@ -906,8 +975,8 @@ void CollectionCatalog::addResource(const ResourceId& rid, const std::string& en
void CollectionCatalog::_commitWritableClone(
std::shared_ptr<Collection> cloned,
boost::optional<Timestamp> commitTime,
- const std::vector<std::function<void(boost::optional<Timestamp>)>>& commitHandlers) {
- stdx::lock_guard<Latch> lock(_catalogLock);
+ const std::vector<std::function<void(CollectionCatalog&, boost::optional<Timestamp>)>>&
+ commitHandlers) {
_collections[cloned->ns()] = cloned;
_catalog[cloned->uuid()] = cloned;
@@ -915,7 +984,7 @@ void CollectionCatalog::_commitWritableClone(
_orderedCollections[dbIdPair] = cloned;
for (auto&& commitHandler : commitHandlers) {
- commitHandler(commitTime);
+ commitHandler(*this, commitTime);
}
}
@@ -923,7 +992,12 @@ void CollectionCatalog::commitUnmanagedClone(OperationContext* opCtx, Collection
auto& uncommittedWritableCollections = getUncommittedWritableCollections(opCtx);
auto [cloned, commitHandlers] = uncommittedWritableCollections.remove(collection);
if (cloned) {
- _commitWritableClone(std::move(cloned), boost::none, commitHandlers);
+ CollectionCatalog::write(opCtx,
+ [cloned = std::move(cloned),
+ commitHandlers = &commitHandlers](CollectionCatalog& catalog) {
+ catalog._commitWritableClone(
+ std::move(cloned), boost::none, *commitHandlers);
+ });
}
}
@@ -932,9 +1006,38 @@ void CollectionCatalog::discardUnmanagedClone(OperationContext* opCtx, Collectio
uncommittedWritableCollections.remove(collection);
}
+CollectionCatalogStasher::CollectionCatalogStasher(OperationContext* opCtx)
+ : _opCtx(opCtx), _stashed(false) {}
+CollectionCatalogStasher::CollectionCatalogStasher(OperationContext* opCtx,
+ std::shared_ptr<const CollectionCatalog> catalog)
+ : _opCtx(opCtx), _stashed(true) {
+ invariant(catalog);
+ CollectionCatalog::stash(_opCtx, std::move(catalog));
+}
+CollectionCatalogStasher::~CollectionCatalogStasher() {
+ reset();
+}
+
+CollectionCatalogStasher::CollectionCatalogStasher(CollectionCatalogStasher&& other)
+ : _opCtx(other._opCtx), _stashed(other._stashed) {
+ other._stashed = false;
+}
+
+void CollectionCatalogStasher::stash(std::shared_ptr<const CollectionCatalog> catalog) {
+ CollectionCatalog::stash(_opCtx, std::move(catalog));
+ _stashed = true;
+}
+
+void CollectionCatalogStasher::reset() {
+ if (_stashed) {
+ CollectionCatalog::stash(_opCtx, nullptr);
+ _stashed = false;
+ }
+}
+
const Collection* LookupCollectionForYieldRestore::operator()(OperationContext* opCtx,
CollectionUUID uuid) const {
- return CollectionCatalog::get(opCtx).lookupCollectionByUUID(opCtx, uuid).get();
+ return CollectionCatalog::get(opCtx)->lookupCollectionByUUID(opCtx, uuid).get();
}
} // namespace mongo
diff --git a/src/mongo/db/catalog/collection_catalog.h b/src/mongo/db/catalog/collection_catalog.h
index bf32a3c0de5..1ab526ec764 100644
--- a/src/mongo/db/catalog/collection_catalog.h
+++ b/src/mongo/db/catalog/collection_catalog.h
@@ -46,12 +46,10 @@ namespace mongo {
* collection lookup by UUID.
*/
using CollectionUUID = UUID;
+class CollectionCatalog;
class Database;
class CollectionCatalog {
- CollectionCatalog(const CollectionCatalog&) = delete;
- CollectionCatalog& operator=(const CollectionCatalog&) = delete;
-
friend class iterator;
public:
@@ -75,14 +73,10 @@ public:
public:
using value_type = CollectionPtr;
- iterator(OperationContext* opCtx,
- StringData dbName,
- uint64_t genNum,
- const CollectionCatalog& catalog);
+ iterator(OperationContext* opCtx, StringData dbName, const CollectionCatalog& catalog);
iterator(OperationContext* opCtx,
std::map<std::pair<std::string, CollectionUUID>,
std::shared_ptr<Collection>>::const_iterator mapIter,
- uint64_t genNum,
const CollectionCatalog& catalog);
value_type operator*();
iterator operator++();
@@ -99,21 +93,11 @@ public:
bool operator!=(const iterator& other);
private:
- /**
- * Check if _mapIter has been invalidated due to a change in the _orderedCollections map. If
- * it has, restart iteration through a call to lower_bound. If the element that the iterator
- * is currently pointing to has been deleted, the iterator will be repositioned to the
- * element that follows it.
- *
- * Returns true if iterator got repositioned.
- */
- bool _repositionIfNeeded();
bool _exhausted();
OperationContext* _opCtx;
std::string _dbName;
boost::optional<CollectionUUID> _uuid;
- uint64_t _genNum;
std::map<std::pair<std::string, CollectionUUID>,
std::shared_ptr<Collection>>::const_iterator _mapIter;
const CollectionCatalog* _catalog;
@@ -139,9 +123,31 @@ public:
}
};
- static CollectionCatalog& get(ServiceContext* svcCtx);
- static CollectionCatalog& get(OperationContext* opCtx);
- CollectionCatalog() = default;
+ static std::shared_ptr<const CollectionCatalog> get(ServiceContext* svcCtx);
+ static std::shared_ptr<const CollectionCatalog> get(OperationContext* opCtx);
+
+ /**
+ * Stashes provided CollectionCatalog pointer on the OperationContext.
+ * Will cause get() to return it for this OperationContext.
+ */
+ static void stash(OperationContext* opCtx, std::shared_ptr<const CollectionCatalog> catalog);
+
+ /**
+ * Perform a write to the catalog using copy-on-write. A catalog previously returned by get()
+ * will not be modified.
+ *
+ * This call will block until the modified catalog has been committed. Concurrant writes are
+ * batched together and will thus block each other. It is important to not perform blocking
+ * operations such as acquiring locks or waiting for I/O in the write job as that would also
+ * block other writers.
+ *
+ * The provided job is allowed to throw which will be propagated through this call.
+ *
+ * The write job may execute on a different thread.
+ */
+ using CatalogWriteFn = std::function<void(CollectionCatalog&)>;
+ static void write(ServiceContext* svcCtx, CatalogWriteFn job);
+ static void write(OperationContext* opCtx, CatalogWriteFn job);
/**
* This function is responsible for safely setting the namespace string inside 'coll' to the
@@ -153,7 +159,7 @@ public:
void setCollectionNamespace(OperationContext* opCtx,
Collection* coll,
const NamespaceString& fromCollection,
- const NamespaceString& toCollection);
+ const NamespaceString& toCollection) const;
void onCloseDatabase(OperationContext* opCtx, std::string dbName);
@@ -170,8 +176,8 @@ public:
/**
* Returns the RecoveryUnit's Change for dropping the collection
*/
- std::unique_ptr<RecoveryUnit::Change> makeFinishDropCollectionChange(
- std::shared_ptr<Collection>, CollectionUUID uuid);
+ static std::unique_ptr<RecoveryUnit::Change> makeFinishDropCollectionChange(
+ OperationContext* opCtx, std::shared_ptr<Collection>, CollectionUUID uuid);
/**
* Deregister all the collection objects.
@@ -187,14 +193,11 @@ public:
*/
Collection* lookupCollectionByUUIDForMetadataWrite(OperationContext* opCtx,
LifetimeMode mode,
- CollectionUUID uuid);
+ CollectionUUID uuid) const;
CollectionPtr lookupCollectionByUUID(OperationContext* opCtx, CollectionUUID uuid) const;
std::shared_ptr<const Collection> lookupCollectionByUUIDForRead(OperationContext* opCtx,
CollectionUUID uuid) const;
-
- void makeCollectionVisible(CollectionUUID uuid);
-
/**
* Returns true if the collection has been registered in the CollectionCatalog but not yet made
* visible.
@@ -210,7 +213,7 @@ public:
*/
Collection* lookupCollectionByNamespaceForMetadataWrite(OperationContext* opCtx,
LifetimeMode mode,
- const NamespaceString& nss);
+ const NamespaceString& nss) const;
CollectionPtr lookupCollectionByNamespace(OperationContext* opCtx,
const NamespaceString& nss) const;
std::shared_ptr<const Collection> lookupCollectionByNamespaceForRead(
@@ -236,7 +239,7 @@ public:
* can be resolved, but the resulting collection is in the wrong database.
*/
NamespaceString resolveNamespaceStringOrUUID(OperationContext* opCtx,
- NamespaceStringOrUUID nsOrUUID);
+ NamespaceStringOrUUID nsOrUUID) const;
/**
* Returns whether the collection with 'uuid' satisfies the provided 'predicate'. If the
@@ -338,7 +341,7 @@ public:
* the same ResourceId entry, we return the boost::none for those namespaces until there is
* only one namespace in the set. If the ResourceId is not found, boost::none is returned.
*/
- boost::optional<std::string> lookupResourceName(const ResourceId& rid);
+ boost::optional<std::string> lookupResourceName(const ResourceId& rid) const;
/**
* Removes an existing ResourceId 'rid' with namespace 'entry' from the map.
@@ -354,18 +357,18 @@ public:
* Commit unmanaged Collection that was acquired by lookupCollectionBy***ForMetadataWrite and
* lifetime mode kUnmanagedClone.
*/
- void commitUnmanagedClone(OperationContext* opCtx, Collection* collection);
+ static void commitUnmanagedClone(OperationContext* opCtx, Collection* collection);
/**
* Discard unmanaged Collection that was acquired by lookupCollectionBy***ForMetadataWrite and
* lifetime mode kUnmanagedClone.
*/
- void discardUnmanagedClone(OperationContext* opCtx, Collection* collection);
+ static void discardUnmanagedClone(OperationContext* opCtx, Collection* collection);
private:
friend class CollectionCatalog::iterator;
- std::shared_ptr<Collection> _lookupCollectionByUUID(WithLock, CollectionUUID uuid) const;
+ std::shared_ptr<Collection> _lookupCollectionByUUID(CollectionUUID uuid) const;
/**
* Helper to commit a cloned Collection into the catalog. It takes a vector of commit handlers
@@ -375,11 +378,8 @@ private:
void _commitWritableClone(
std::shared_ptr<Collection> cloned,
boost::optional<Timestamp> commitTime,
- const std::vector<std::function<void(boost::optional<Timestamp>)>>& commitHandlers);
-
- const std::vector<CollectionUUID>& _getOrdering_inlock(const StringData& db,
- const stdx::lock_guard<Latch>&);
- mutable mongo::Mutex _catalogLock;
+ const std::vector<std::function<void(CollectionCatalog&, boost::optional<Timestamp>)>>&
+ commitHandlers);
/**
* When present, indicates that the catalog is in closed state, and contains a map from UUID
@@ -401,11 +401,6 @@ private:
OrderedCollectionMap _orderedCollections; // Ordered by <dbName, collUUID> pair
NamespaceCollectionMap _collections;
- /**
- * Generation number to track changes to the catalog that could invalidate iterators.
- */
- uint64_t _generationNumber = 0;
-
// Incremented whenever the CollectionCatalog gets closed and reopened (onCloseCatalog and
// onOpenCatalog).
//
@@ -418,26 +413,54 @@ private:
// global lock in at least MODE_IS to read it.
uint64_t _epoch = 0;
- // Protects _resourceInformation.
- mutable Mutex _resourceLock = MONGO_MAKE_LATCH("CollectionCatalog::_resourceLock");
-
// Mapping from ResourceId to a set of strings that contains collection and database namespaces.
std::map<ResourceId, std::set<std::string>> _resourceInformation;
- // Protects _databaseProfileSettings.
- mutable Mutex _profileSettingsLock =
- MONGO_MAKE_LATCH("CollectionCatalog::_profileSettingsLock");
-
/**
* Contains non-default database profile settings. New collections, current collections and
* views must all be able to access the correct profile settings for the database in which they
- * reside. Simple database name to struct ProfileSettings map. Access protected by
- * _profileSettingsLock.
+ * reside. Simple database name to struct ProfileSettings map.
*/
DatabaseProfileSettingsMap _databaseProfileSettings;
};
/**
+ * RAII style object to stash a versioned CollectionCatalog on the OperationContext.
+ * Calls to CollectionCatalog::get(OperationContext*) will return this instance.
+ *
+ * Unstashes the CollectionCatalog at destruction.
+ *
+ * It is not safe to nest usages of this type.
+ */
+class CollectionCatalogStasher {
+public:
+ CollectionCatalogStasher(OperationContext* opCtx);
+ CollectionCatalogStasher(OperationContext* opCtx,
+ std::shared_ptr<const CollectionCatalog> catalog);
+ ~CollectionCatalogStasher();
+
+ CollectionCatalogStasher(const CollectionCatalogStasher&) = delete;
+ CollectionCatalogStasher(CollectionCatalogStasher&& other);
+
+ CollectionCatalogStasher& operator=(const CollectionCatalogStasher&) = delete;
+ CollectionCatalogStasher& operator=(CollectionCatalogStasher&&) = delete;
+
+ /**
+ * Stashes a new catalog on OperationContext
+ */
+ void stash(std::shared_ptr<const CollectionCatalog> catalog);
+
+ /**
+ * Resets the OperationContext so CollectionCatalog::get() returns latest catalog again
+ */
+ void reset();
+
+private:
+ OperationContext* _opCtx;
+ bool _stashed;
+};
+
+/**
* Functor for looking up Collection by UUID from the Collection Catalog. This is the default yield
* restore implementation for CollectionPtr when acquired from the catalog.
*/
diff --git a/src/mongo/db/catalog/collection_catalog_helper.cpp b/src/mongo/db/catalog/collection_catalog_helper.cpp
index 8b220b79464..60e0d125973 100644
--- a/src/mongo/db/catalog/collection_catalog_helper.cpp
+++ b/src/mongo/db/catalog/collection_catalog_helper.cpp
@@ -44,25 +44,28 @@ void forEachCollectionFromDb(OperationContext* opCtx,
CollectionCatalog::CollectionInfoFn callback,
CollectionCatalog::CollectionInfoFn predicate) {
- CollectionCatalog& catalog = CollectionCatalog::get(opCtx);
- for (auto collectionIt = catalog.begin(opCtx, dbName); collectionIt != catalog.end(opCtx);
+ auto catalogForIteration = CollectionCatalog::get(opCtx);
+ for (auto collectionIt = catalogForIteration->begin(opCtx, dbName);
+ collectionIt != catalogForIteration->end(opCtx);
++collectionIt) {
auto uuid = collectionIt.uuid().get();
- if (predicate && !catalog.checkIfCollectionSatisfiable(uuid, predicate)) {
+ if (predicate && !catalogForIteration->checkIfCollectionSatisfiable(uuid, predicate)) {
continue;
}
boost::optional<Lock::CollectionLock> clk;
CollectionPtr collection;
- while (auto nss = catalog.lookupNSSByUUID(opCtx, uuid)) {
+ auto catalog = CollectionCatalog::get(opCtx);
+ while (auto nss = catalog->lookupNSSByUUID(opCtx, uuid)) {
// Get a fresh snapshot for each locked collection to see any catalog changes.
clk.emplace(opCtx, *nss, collLockMode);
opCtx->recoveryUnit()->abandonSnapshot();
+ catalog = CollectionCatalog::get(opCtx);
- if (catalog.lookupNSSByUUID(opCtx, uuid) == nss) {
+ if (catalog->lookupNSSByUUID(opCtx, uuid) == nss) {
// Success: locked the namespace and the UUID still maps to it.
- collection = catalog.lookupCollectionByUUID(opCtx, uuid);
+ collection = catalog->lookupCollectionByUUID(opCtx, uuid);
invariant(collection);
break;
}
diff --git a/src/mongo/db/catalog/collection_catalog_test.cpp b/src/mongo/db/catalog/collection_catalog_test.cpp
index 9f5a1809f4f..0f790bcc7f2 100644
--- a/src/mongo/db/catalog/collection_catalog_test.cpp
+++ b/src/mongo/db/catalog/collection_catalog_test.cpp
@@ -294,6 +294,7 @@ public:
}
void tearDown() {
+ std::vector<CollectionUUID> collectionsToDeregister;
for (auto it = catalog.begin(&opCtx, "resourceDb"); it != catalog.end(&opCtx); ++it) {
auto coll = *it;
auto uuid = coll->uuid();
@@ -301,6 +302,10 @@ public:
break;
}
+ collectionsToDeregister.push_back(uuid);
+ }
+
+ for (auto&& uuid : collectionsToDeregister) {
catalog.deregisterCollection(&opCtx, uuid);
}
@@ -401,70 +406,6 @@ TEST_F(CollectionCatalogIterationTest, InvalidateEntry) {
}
}
-// Delete the entry pointed to by the iterator and dereference the iterator.
-TEST_F(CollectionCatalogIterationTest, InvalidateAndDereference) {
- auto it = catalog.begin(&opCtx, "bar");
- auto collsIt = collsIterator("bar");
- auto uuid = collsIt->first;
- catalog.deregisterCollection(&opCtx, uuid);
- ++collsIt;
-
- ASSERT(it != catalog.end(&opCtx));
- auto catalogColl = *it;
- ASSERT(catalogColl != nullptr);
- ASSERT_EQUALS(catalogColl->ns(), collsIt->second->ns());
-
- dropColl("bar", uuid);
-}
-
-// Delete the last entry for a database while pointing to it and dereference the iterator.
-TEST_F(CollectionCatalogIterationTest, InvalidateLastEntryAndDereference) {
- auto it = catalog.begin(&opCtx, "bar");
- NamespaceString lastNs;
- boost::optional<CollectionUUID> uuid;
- for (auto collsIt = collsIterator("bar"); collsIt != collsIteratorEnd("bar"); ++collsIt) {
- lastNs = collsIt->second->ns();
- uuid = collsIt->first;
- }
-
- // Increment until it points to the last collection.
- for (; it != catalog.end(&opCtx); ++it) {
- auto coll = *it;
- ASSERT(coll != nullptr);
- if (coll->ns() == lastNs) {
- break;
- }
- }
-
- catalog.deregisterCollection(&opCtx, *uuid);
- dropColl("bar", *uuid);
- ASSERT(*it == nullptr);
-}
-
-// Delete the last entry in the map while pointing to it and dereference the iterator.
-TEST_F(CollectionCatalogIterationTest, InvalidateLastEntryInMapAndDereference) {
- auto it = catalog.begin(&opCtx, "foo");
- NamespaceString lastNs;
- boost::optional<CollectionUUID> uuid;
- for (auto collsIt = collsIterator("foo"); collsIt != collsIteratorEnd("foo"); ++collsIt) {
- lastNs = collsIt->second->ns();
- uuid = collsIt->first;
- }
-
- // Increment until it points to the last collection.
- for (; it != catalog.end(&opCtx); ++it) {
- auto coll = *it;
- ASSERT(coll != nullptr);
- if (coll->ns() == lastNs) {
- break;
- }
- }
-
- catalog.deregisterCollection(&opCtx, *uuid);
- dropColl("foo", *uuid);
- ASSERT(*it == nullptr);
-}
-
TEST_F(CollectionCatalogIterationTest, GetUUIDWontRepositionEvenIfEntryIsDropped) {
auto it = catalog.begin(&opCtx, "bar");
auto collsIt = collsIterator("bar");
diff --git a/src/mongo/db/catalog/collection_compact.cpp b/src/mongo/db/catalog/collection_compact.cpp
index dbb1130ce2c..f907fd7832a 100644
--- a/src/mongo/db/catalog/collection_compact.cpp
+++ b/src/mongo/db/catalog/collection_compact.cpp
@@ -55,8 +55,8 @@ CollectionPtr getCollectionForCompact(OperationContext* opCtx,
const NamespaceString& collectionNss) {
invariant(opCtx->lockState()->isCollectionLockedForMode(collectionNss, MODE_IX));
- CollectionCatalog& collectionCatalog = CollectionCatalog::get(opCtx);
- CollectionPtr collection = collectionCatalog.lookupCollectionByNamespace(opCtx, collectionNss);
+ auto collectionCatalog = CollectionCatalog::get(opCtx);
+ CollectionPtr collection = collectionCatalog->lookupCollectionByNamespace(opCtx, collectionNss);
if (!collection) {
std::shared_ptr<ViewDefinition> view =
diff --git a/src/mongo/db/catalog/create_collection.cpp b/src/mongo/db/catalog/create_collection.cpp
index bb5b55c96f8..2e2288a0c71 100644
--- a/src/mongo/db/catalog/create_collection.cpp
+++ b/src/mongo/db/catalog/create_collection.cpp
@@ -54,8 +54,8 @@ namespace mongo {
namespace {
void _createSystemDotViewsIfNecessary(OperationContext* opCtx, const Database* db) {
// Create 'system.views' in a separate WUOW if it does not exist.
- if (!CollectionCatalog::get(opCtx).lookupCollectionByNamespace(opCtx,
- db->getSystemViewsName())) {
+ if (!CollectionCatalog::get(opCtx)->lookupCollectionByNamespace(opCtx,
+ db->getSystemViewsName())) {
WriteUnitOfWork wuow(opCtx);
invariant(db->createCollection(opCtx, db->getSystemViewsName()));
wuow.commit();
@@ -92,7 +92,7 @@ Status _createView(OperationContext* opCtx,
nss,
Top::LockType::NotLocked,
AutoStatsTracker::LogMode::kUpdateTopAndCurOp,
- CollectionCatalog::get(opCtx).getDatabaseProfileLevel(nss.db()));
+ CollectionCatalog::get(opCtx)->getDatabaseProfileLevel(nss.db()));
// If the view creation rolls back, ensure that the Top entry created for the view is
// deleted.
@@ -170,21 +170,20 @@ Status _createTimeseries(OperationContext* opCtx,
auto db = autoColl.ensureDbExists();
_createSystemDotViewsIfNecessary(opCtx, db);
+ auto catalog = CollectionCatalog::get(opCtx);
WriteUnitOfWork wuow(opCtx);
- AutoStatsTracker statsTracker(
- opCtx,
- ns,
- Top::LockType::NotLocked,
- AutoStatsTracker::LogMode::kUpdateTopAndCurOp,
- CollectionCatalog::get(opCtx).getDatabaseProfileLevel(ns.db()));
+ AutoStatsTracker statsTracker(opCtx,
+ ns,
+ Top::LockType::NotLocked,
+ AutoStatsTracker::LogMode::kUpdateTopAndCurOp,
+ catalog->getDatabaseProfileLevel(ns.db()));
- AutoStatsTracker bucketsStatsTracker(
- opCtx,
- bucketsNs,
- Top::LockType::NotLocked,
- AutoStatsTracker::LogMode::kUpdateTopAndCurOp,
- CollectionCatalog::get(opCtx).getDatabaseProfileLevel(ns.db()));
+ AutoStatsTracker bucketsStatsTracker(opCtx,
+ bucketsNs,
+ Top::LockType::NotLocked,
+ AutoStatsTracker::LogMode::kUpdateTopAndCurOp,
+ catalog->getDatabaseProfileLevel(ns.db()));
// If the buckets collection and time-series view creation roll back, ensure that their Top
// entries are deleted.
@@ -224,7 +223,7 @@ Status _createCollection(OperationContext* opCtx,
// This is a top-level handler for collection creation name conflicts. New commands coming
// in, or commands that generated a WriteConflict must return a NamespaceExists error here
// on conflict.
- if (CollectionCatalog::get(opCtx).lookupCollectionByNamespace(opCtx, nss)) {
+ if (CollectionCatalog::get(opCtx)->lookupCollectionByNamespace(opCtx, nss)) {
return Status(ErrorCodes::NamespaceExists,
str::stream() << "Collection already exists. NS: " << nss);
}
@@ -246,7 +245,7 @@ Status _createCollection(OperationContext* opCtx,
nss,
Top::LockType::NotLocked,
AutoStatsTracker::LogMode::kUpdateTopAndCurOp,
- CollectionCatalog::get(opCtx).getDatabaseProfileLevel(nss.db()));
+ CollectionCatalog::get(opCtx)->getDatabaseProfileLevel(nss.db()));
// If the collection creation rolls back, ensure that the Top entry created for the
// collection is deleted.
@@ -390,8 +389,8 @@ Status createCollectionForApplyOps(OperationContext* opCtx,
"Invalid UUID in applyOps create command: " + uuid.toString(),
uuid.isRFC4122v4());
- auto& catalog = CollectionCatalog::get(opCtx);
- const auto currentName = catalog.lookupNSSByUUID(opCtx, uuid);
+ auto catalog = CollectionCatalog::get(opCtx);
+ const auto currentName = catalog->lookupNSSByUUID(opCtx, uuid);
auto serviceContext = opCtx->getServiceContext();
auto opObserver = serviceContext->getOpObserver();
if (currentName && *currentName == newCollName)
@@ -415,9 +414,7 @@ Status createCollectionForApplyOps(OperationContext* opCtx,
// a random temporary name is correct: once all entries are replayed no temporary
// names will remain.
const bool stayTemp = true;
- auto futureColl = db
- ? CollectionCatalog::get(opCtx).lookupCollectionByNamespace(opCtx, newCollName)
- : nullptr;
+ auto futureColl = db ? catalog->lookupCollectionByNamespace(opCtx, newCollName) : nullptr;
bool needsRenaming = static_cast<bool>(futureColl);
invariant(!needsRenaming || allowRenameOutOfTheWay);
@@ -462,7 +459,7 @@ Status createCollectionForApplyOps(OperationContext* opCtx,
wuow.commit();
// Re-fetch collection after commit to get a valid pointer
- futureColl = CollectionCatalog::get(opCtx).lookupCollectionByUUID(opCtx, uuid);
+ futureColl = CollectionCatalog::get(opCtx)->lookupCollectionByUUID(opCtx, uuid);
return Status::OK();
});
@@ -490,7 +487,7 @@ Status createCollectionForApplyOps(OperationContext* opCtx,
// If the collection with the requested UUID already exists, but with a different
// name, just rename it to 'newCollName'.
- if (catalog.lookupCollectionByUUID(opCtx, uuid)) {
+ if (catalog->lookupCollectionByUUID(opCtx, uuid)) {
invariant(currentName);
uassert(40655,
str::stream() << "Invalid name " << newCollName << " for UUID " << uuid,
diff --git a/src/mongo/db/catalog/create_collection_test.cpp b/src/mongo/db/catalog/create_collection_test.cpp
index e8c36b94fcc..fa003fc4e99 100644
--- a/src/mongo/db/catalog/create_collection_test.cpp
+++ b/src/mongo/db/catalog/create_collection_test.cpp
@@ -214,8 +214,8 @@ TEST_F(CreateCollectionTest,
ASSERT_EQUALS(uuid, getCollectionUuid(opCtx.get(), newNss));
// Check that old collection that was renamed out of the way still exists.
- auto& catalog = CollectionCatalog::get(opCtx.get());
- auto renamedCollectionNss = catalog.lookupNSSByUUID(opCtx.get(), existingCollectionUuid);
+ auto catalog = CollectionCatalog::get(opCtx.get());
+ auto renamedCollectionNss = catalog->lookupNSSByUUID(opCtx.get(), existingCollectionUuid);
ASSERT(renamedCollectionNss);
ASSERT_TRUE(collectionExists(opCtx.get(), *renamedCollectionNss))
<< "old renamed collection with UUID " << existingCollectionUuid
diff --git a/src/mongo/db/catalog/database.h b/src/mongo/db/catalog/database.h
index 27982bbf772..f7ba622493b 100644
--- a/src/mongo/db/catalog/database.h
+++ b/src/mongo/db/catalog/database.h
@@ -74,9 +74,6 @@ public:
inline Database(Database&&) = delete;
inline Database& operator=(Database&&) = delete;
- virtual CollectionCatalog::iterator begin(OperationContext* opCtx) const = 0;
- virtual CollectionCatalog::iterator end(OperationContext* opCtx) const = 0;
-
/**
* Sets up internal memory structures.
*/
diff --git a/src/mongo/db/catalog/database_holder_impl.cpp b/src/mongo/db/catalog/database_holder_impl.cpp
index 130d84d2594..c33b8eca03c 100644
--- a/src/mongo/db/catalog/database_holder_impl.cpp
+++ b/src/mongo/db/catalog/database_holder_impl.cpp
@@ -154,7 +154,7 @@ Database* DatabaseHolderImpl::openDb(OperationContext* opCtx, StringData ns, boo
// block.
lk.unlock();
- if (CollectionCatalog::get(opCtx).getAllCollectionUUIDsFromDb(dbname).empty()) {
+ if (CollectionCatalog::get(opCtx)->getAllCollectionUUIDsFromDb(dbname).empty()) {
audit::logCreateDatabase(opCtx->getClient(), dbname);
if (justCreated)
*justCreated = true;
@@ -192,7 +192,8 @@ void DatabaseHolderImpl::dropDb(OperationContext* opCtx, Database* db) {
invariant(opCtx->lockState()->isDbLockedForMode(name, MODE_X));
- for (auto collIt = db->begin(opCtx); collIt != db->end(opCtx); ++collIt) {
+ auto catalog = CollectionCatalog::get(opCtx);
+ for (auto collIt = catalog->begin(opCtx, name); collIt != catalog->end(opCtx); ++collIt) {
auto coll = *collIt;
if (!coll) {
break;
@@ -208,7 +209,7 @@ void DatabaseHolderImpl::dropDb(OperationContext* opCtx, Database* db) {
auto const serviceContext = opCtx->getServiceContext();
- for (auto collIt = db->begin(opCtx); collIt != db->end(opCtx); ++collIt) {
+ for (auto collIt = catalog->begin(opCtx, name); collIt != catalog->end(opCtx); ++collIt) {
auto coll = *collIt;
if (!coll) {
break;
@@ -218,7 +219,8 @@ void DatabaseHolderImpl::dropDb(OperationContext* opCtx, Database* db) {
}
// Clean up the in-memory database state.
- CollectionCatalog::get(opCtx).clearDatabaseProfileSettings(name);
+ CollectionCatalog::write(
+ opCtx, [&](CollectionCatalog& catalog) { catalog.clearDatabaseProfileSettings(name); });
close(opCtx, name);
auto const storageEngine = serviceContext->getStorageEngine();
@@ -239,7 +241,9 @@ void DatabaseHolderImpl::close(OperationContext* opCtx, StringData ns) {
}
auto db = it->second;
- CollectionCatalog::get(opCtx).onCloseDatabase(opCtx, dbName.toString());
+ CollectionCatalog::write(opCtx, [&](CollectionCatalog& catalog) {
+ catalog.onCloseDatabase(opCtx, dbName.toString());
+ });
delete db;
db = nullptr;
@@ -269,7 +273,8 @@ void DatabaseHolderImpl::closeAll(OperationContext* opCtx) {
LOGV2_DEBUG(20311, 2, "DatabaseHolder::closeAll name:{name}", "name"_attr = name);
Database* db = _dbs[name];
- CollectionCatalog::get(opCtx).onCloseDatabase(opCtx, name);
+ CollectionCatalog::write(
+ opCtx, [&](CollectionCatalog& catalog) { catalog.onCloseDatabase(opCtx, name); });
delete db;
_dbs.erase(name);
diff --git a/src/mongo/db/catalog/database_impl.cpp b/src/mongo/db/catalog/database_impl.cpp
index 9274e5a4c2b..8167dd2c1a9 100644
--- a/src/mongo/db/catalog/database_impl.cpp
+++ b/src/mongo/db/catalog/database_impl.cpp
@@ -171,8 +171,8 @@ void DatabaseImpl::init(OperationContext* const opCtx) const {
uasserted(10028, status.toString());
}
- auto& catalog = CollectionCatalog::get(opCtx);
- for (const auto& uuid : catalog.getAllCollectionUUIDsFromDb(_name)) {
+ auto catalog = CollectionCatalog::get(opCtx);
+ for (const auto& uuid : catalog->getAllCollectionUUIDsFromDb(_name)) {
CollectionWriter collection(
opCtx,
uuid,
@@ -336,7 +336,7 @@ Status DatabaseImpl::dropCollection(OperationContext* opCtx,
// Cannot drop uncommitted collections.
invariant(!UncommittedCollections::getForTxn(opCtx, nss));
- if (!CollectionCatalog::get(opCtx).lookupCollectionByNamespace(opCtx, nss)) {
+ if (!CollectionCatalog::get(opCtx)->lookupCollectionByNamespace(opCtx, nss)) {
// Collection doesn't exist so don't bother validating if it can be dropped.
return Status::OK();
}
@@ -345,7 +345,7 @@ Status DatabaseImpl::dropCollection(OperationContext* opCtx,
if (nss.isSystem()) {
if (nss.isSystemDotProfile()) {
- if (CollectionCatalog::get(opCtx).getDatabaseProfileLevel(_name) != 0)
+ if (CollectionCatalog::get(opCtx)->getDatabaseProfileLevel(_name) != 0)
return Status(ErrorCodes::IllegalOperation,
"turn off profiling before dropping system.profile collection");
} else if (!(nss.isSystemDotViews() || nss.isHealthlog() ||
@@ -516,9 +516,12 @@ Status DatabaseImpl::_finishDropCollection(OperationContext* opCtx,
if (!status.isOK())
return status;
- auto removedColl = CollectionCatalog::get(opCtx).deregisterCollection(opCtx, uuid);
+ std::shared_ptr<Collection> removedColl;
+ CollectionCatalog::write(opCtx, [&](CollectionCatalog& catalog) {
+ removedColl = catalog.deregisterCollection(opCtx, uuid);
+ });
opCtx->recoveryUnit()->registerChange(
- CollectionCatalog::get(opCtx).makeFinishDropCollectionChange(std::move(removedColl), uuid));
+ CollectionCatalog::makeFinishDropCollectionChange(opCtx, std::move(removedColl), uuid));
return Status::OK();
}
@@ -534,7 +537,7 @@ Status DatabaseImpl::renameCollection(OperationContext* opCtx,
invariant(fromNss.db() == _name);
invariant(toNss.db() == _name);
- if (CollectionCatalog::get(opCtx).lookupCollectionByNamespace(opCtx, toNss)) {
+ if (CollectionCatalog::get(opCtx)->lookupCollectionByNamespace(opCtx, toNss)) {
return Status(ErrorCodes::NamespaceExists,
str::stream() << "Cannot rename '" << fromNss << "' to '" << toNss
<< "' because the destination namespace already exists");
@@ -563,7 +566,9 @@ Status DatabaseImpl::renameCollection(OperationContext* opCtx,
// because the CollectionCatalog mutex synchronizes concurrent access to the collection's
// namespace for callers that may not hold a collection lock.
auto writableCollection = collToRename.getWritableCollection();
- CollectionCatalog::get(opCtx).setCollectionNamespace(opCtx, writableCollection, fromNss, toNss);
+
+ CollectionCatalog::get(opCtx)->setCollectionNamespace(
+ opCtx, writableCollection, fromNss, toNss);
return status;
}
@@ -571,7 +576,7 @@ Status DatabaseImpl::renameCollection(OperationContext* opCtx,
void DatabaseImpl::_checkCanCreateCollection(OperationContext* opCtx,
const NamespaceString& nss,
const CollectionOptions& options) const {
- if (CollectionCatalog::get(opCtx).lookupCollectionByNamespace(opCtx, nss)) {
+ if (CollectionCatalog::get(opCtx)->lookupCollectionByNamespace(opCtx, nss)) {
if (options.isView()) {
uasserted(17399,
str::stream()
@@ -794,7 +799,7 @@ StatusWith<NamespaceString> DatabaseImpl::makeUniqueCollectionNamespace(
replacePercentSign);
NamespaceString nss(_name, collectionName);
- if (!CollectionCatalog::get(opCtx).lookupCollectionByNamespace(opCtx, nss)) {
+ if (!CollectionCatalog::get(opCtx)->lookupCollectionByNamespace(opCtx, nss)) {
return nss;
}
}
@@ -814,8 +819,8 @@ void DatabaseImpl::checkForIdIndexesAndDropPendingCollections(OperationContext*
return;
}
- for (const auto& nss :
- CollectionCatalog::get(opCtx).getAllCollectionNamesFromDb(opCtx, _name)) {
+ auto catalog = CollectionCatalog::get(opCtx);
+ for (const auto& nss : catalog->getAllCollectionNamesFromDb(opCtx, _name)) {
if (nss.isDropPendingNamespace()) {
auto dropOpTime = fassert(40459, nss.getDropPendingNamespaceOpTime());
LOGV2(20321,
@@ -830,8 +835,7 @@ void DatabaseImpl::checkForIdIndexesAndDropPendingCollections(OperationContext*
if (nss.isSystem())
continue;
- const CollectionPtr& coll =
- CollectionCatalog::get(opCtx).lookupCollectionByNamespace(opCtx, nss);
+ const CollectionPtr& coll = catalog->lookupCollectionByNamespace(opCtx, nss);
if (!coll)
continue;
diff --git a/src/mongo/db/catalog/database_impl.h b/src/mongo/db/catalog/database_impl.h
index 56bc5ec78b4..e82fd4ad0b2 100644
--- a/src/mongo/db/catalog/database_impl.h
+++ b/src/mongo/db/catalog/database_impl.h
@@ -112,14 +112,6 @@ public:
void checkForIdIndexesAndDropPendingCollections(OperationContext* opCtx) const final;
- CollectionCatalog::iterator begin(OperationContext* opCtx) const final {
- return CollectionCatalog::get(opCtx).begin(opCtx, _name);
- }
-
- CollectionCatalog::iterator end(OperationContext* opCtx) const final {
- return CollectionCatalog::get(opCtx).end(opCtx);
- }
-
private:
/**
* Throws if there is a reason 'ns' cannot be created as a user collection. Namespace pattern
diff --git a/src/mongo/db/catalog/database_test.cpp b/src/mongo/db/catalog/database_test.cpp
index 4badacce96f..19f61925c5e 100644
--- a/src/mongo/db/catalog/database_test.cpp
+++ b/src/mongo/db/catalog/database_test.cpp
@@ -187,11 +187,11 @@ void _testDropCollection(OperationContext* opCtx,
WriteUnitOfWork wuow(opCtx);
if (!createCollectionBeforeDrop) {
- ASSERT_FALSE(CollectionCatalog::get(opCtx).lookupCollectionByNamespace(opCtx, nss));
+ ASSERT_FALSE(CollectionCatalog::get(opCtx)->lookupCollectionByNamespace(opCtx, nss));
}
ASSERT_OK(db->dropCollection(opCtx, nss, dropOpTime));
- ASSERT_FALSE(CollectionCatalog::get(opCtx).lookupCollectionByNamespace(opCtx, nss));
+ ASSERT_FALSE(CollectionCatalog::get(opCtx)->lookupCollectionByNamespace(opCtx, nss));
wuow.commit();
});
}
@@ -355,15 +355,15 @@ TEST_F(DatabaseTest, RenameCollectionPreservesUuidOfSourceCollectionAndUpdatesUu
ASSERT_TRUE(db);
auto fromUuid = UUID::gen();
- auto& catalog = CollectionCatalog::get(opCtx);
writeConflictRetry(opCtx, "create", fromNss.ns(), [&] {
- ASSERT_EQUALS(boost::none, catalog.lookupNSSByUUID(opCtx, fromUuid));
+ auto catalog = CollectionCatalog::get(opCtx);
+ ASSERT_EQUALS(boost::none, catalog->lookupNSSByUUID(opCtx, fromUuid));
WriteUnitOfWork wuow(opCtx);
CollectionOptions fromCollectionOptions;
fromCollectionOptions.uuid = fromUuid;
ASSERT_TRUE(db->createCollection(opCtx, fromNss, fromCollectionOptions));
- ASSERT_EQUALS(fromNss, *catalog.lookupNSSByUUID(opCtx, fromUuid));
+ ASSERT_EQUALS(fromNss, *catalog->lookupNSSByUUID(opCtx, fromUuid));
wuow.commit();
});
@@ -372,8 +372,9 @@ TEST_F(DatabaseTest, RenameCollectionPreservesUuidOfSourceCollectionAndUpdatesUu
auto stayTemp = false;
ASSERT_OK(db->renameCollection(opCtx, fromNss, toNss, stayTemp));
- ASSERT_FALSE(CollectionCatalog::get(opCtx).lookupCollectionByNamespace(opCtx, fromNss));
- auto toCollection = CollectionCatalog::get(opCtx).lookupCollectionByNamespace(opCtx, toNss);
+ auto catalog = CollectionCatalog::get(opCtx);
+ ASSERT_FALSE(catalog->lookupCollectionByNamespace(opCtx, fromNss));
+ auto toCollection = catalog->lookupCollectionByNamespace(opCtx, toNss);
ASSERT_TRUE(toCollection);
auto toCollectionOptions =
@@ -383,7 +384,7 @@ TEST_F(DatabaseTest, RenameCollectionPreservesUuidOfSourceCollectionAndUpdatesUu
ASSERT_TRUE(toUuid);
ASSERT_EQUALS(fromUuid, *toUuid);
- ASSERT_EQUALS(toNss, *catalog.lookupNSSByUUID(opCtx, *toUuid));
+ ASSERT_EQUALS(toNss, *catalog->lookupNSSByUUID(opCtx, *toUuid));
wuow.commit();
});
diff --git a/src/mongo/db/catalog/drop_collection.cpp b/src/mongo/db/catalog/drop_collection.cpp
index d7f8d943d19..2642b81bc50 100644
--- a/src/mongo/db/catalog/drop_collection.cpp
+++ b/src/mongo/db/catalog/drop_collection.cpp
@@ -99,7 +99,7 @@ Status _dropView(OperationContext* opCtx,
collectionName,
Top::LockType::NotLocked,
AutoStatsTracker::LogMode::kUpdateCurOp,
- CollectionCatalog::get(opCtx).getDatabaseProfileLevel(collectionName.db()));
+ CollectionCatalog::get(opCtx)->getDatabaseProfileLevel(collectionName.db()));
if (opCtx->writesAreReplicated() &&
!repl::ReplicationCoordinator::get(opCtx)->canAcceptWritesFor(opCtx, collectionName)) {
@@ -135,7 +135,7 @@ Status _abortIndexBuildsAndDrop(OperationContext* opCtx,
opCtx->recoveryUnit()->abandonSnapshot();
CollectionPtr coll =
- CollectionCatalog::get(opCtx).lookupCollectionByNamespace(opCtx, startingNss);
+ CollectionCatalog::get(opCtx)->lookupCollectionByNamespace(opCtx, startingNss);
Status status = _checkNssAndReplState(opCtx, coll);
if (!status.isOK()) {
return status;
@@ -153,7 +153,7 @@ Status _abortIndexBuildsAndDrop(OperationContext* opCtx,
startingNss,
Top::LockType::NotLocked,
AutoStatsTracker::LogMode::kUpdateCurOp,
- CollectionCatalog::get(opCtx).getDatabaseProfileLevel(startingNss.db()));
+ CollectionCatalog::get(opCtx)->getDatabaseProfileLevel(startingNss.db()));
IndexBuildsCoordinator* indexBuildsCoord = IndexBuildsCoordinator::get(opCtx);
const UUID collectionUUID = coll->uuid();
@@ -185,7 +185,7 @@ Status _abortIndexBuildsAndDrop(OperationContext* opCtx,
// disk state, which may have changed when we released the collection lock temporarily.
opCtx->recoveryUnit()->abandonSnapshot();
- coll = CollectionCatalog::get(opCtx).lookupCollectionByUUID(opCtx, collectionUUID);
+ coll = CollectionCatalog::get(opCtx)->lookupCollectionByUUID(opCtx, collectionUUID);
status = _checkNssAndReplState(opCtx, coll);
if (!status.isOK()) {
return status;
@@ -233,7 +233,7 @@ Status _dropCollection(OperationContext* opCtx,
BSONObjBuilder* result) {
Lock::CollectionLock collLock(opCtx, collectionName, MODE_X);
const CollectionPtr& coll =
- CollectionCatalog::get(opCtx).lookupCollectionByNamespace(opCtx, collectionName);
+ CollectionCatalog::get(opCtx)->lookupCollectionByNamespace(opCtx, collectionName);
Status status = _checkNssAndReplState(opCtx, coll);
if (!status.isOK()) {
return status;
@@ -251,7 +251,7 @@ Status _dropCollection(OperationContext* opCtx,
collectionName,
Top::LockType::NotLocked,
AutoStatsTracker::LogMode::kUpdateCurOp,
- CollectionCatalog::get(opCtx).getDatabaseProfileLevel(collectionName.db()));
+ CollectionCatalog::get(opCtx)->getDatabaseProfileLevel(collectionName.db()));
WriteUnitOfWork wunit(opCtx);
@@ -294,7 +294,7 @@ Status dropCollection(OperationContext* opCtx,
return Status(ErrorCodes::NamespaceNotFound, "ns not found");
}
- if (CollectionCatalog::get(opCtx).lookupCollectionByNamespace(opCtx, collectionName)) {
+ if (CollectionCatalog::get(opCtx)->lookupCollectionByNamespace(opCtx, collectionName)) {
return _abortIndexBuildsAndDrop(
opCtx,
std::move(autoDb),
@@ -378,7 +378,7 @@ Status dropCollectionForApplyOps(OperationContext* opCtx,
}
const CollectionPtr& coll =
- CollectionCatalog::get(opCtx).lookupCollectionByNamespace(opCtx, collectionName);
+ CollectionCatalog::get(opCtx)->lookupCollectionByNamespace(opCtx, collectionName);
BSONObjBuilder unusedBuilder;
if (!coll) {
diff --git a/src/mongo/db/catalog/drop_database.cpp b/src/mongo/db/catalog/drop_database.cpp
index 61c5dd953bd..3a177f3f907 100644
--- a/src/mongo/db/catalog/drop_database.cpp
+++ b/src/mongo/db/catalog/drop_database.cpp
@@ -221,7 +221,9 @@ Status _dropDatabase(OperationContext* opCtx, const std::string& dbName, bool ab
}
std::vector<NamespaceString> collectionsToDrop;
- for (auto collIt = db->begin(opCtx); collIt != db->end(opCtx); ++collIt) {
+ auto catalog = CollectionCatalog::get(opCtx);
+ for (auto collIt = catalog->begin(opCtx, db->name()); collIt != catalog->end(opCtx);
+ ++collIt) {
auto collection = *collIt;
if (!collection) {
break;
@@ -267,7 +269,7 @@ Status _dropDatabase(OperationContext* opCtx, const std::string& dbName, bool ab
if (!abortIndexBuilds) {
IndexBuildsCoordinator::get(opCtx)->assertNoIndexBuildInProgForCollection(
- CollectionCatalog::get(opCtx).lookupCollectionByNamespace(opCtx, nss)->uuid());
+ catalog->lookupCollectionByNamespace(opCtx, nss)->uuid());
}
writeConflictRetry(opCtx, "dropDatabase_collection", nss.ns(), [&] {
diff --git a/src/mongo/db/catalog/index_builds_manager.cpp b/src/mongo/db/catalog/index_builds_manager.cpp
index 10c4edf6e92..78c49b3dd14 100644
--- a/src/mongo/db/catalog/index_builds_manager.cpp
+++ b/src/mongo/db/catalog/index_builds_manager.cpp
@@ -391,9 +391,9 @@ StatusWith<int> IndexBuildsManager::_moveRecordToLostAndFound(
RecordId dupRecord) {
invariant(opCtx->lockState()->isCollectionLockedForMode(nss, MODE_IX));
- auto originalCollection = CollectionCatalog::get(opCtx).lookupCollectionByNamespace(opCtx, nss);
- CollectionPtr localCollection =
- CollectionCatalog::get(opCtx).lookupCollectionByNamespace(opCtx, lostAndFoundNss);
+ auto catalog = CollectionCatalog::get(opCtx);
+ auto originalCollection = catalog->lookupCollectionByNamespace(opCtx, nss);
+ CollectionPtr localCollection = catalog->lookupCollectionByNamespace(opCtx, lostAndFoundNss);
// Create the collection if it doesn't exist.
if (!localCollection) {
diff --git a/src/mongo/db/catalog/multi_index_block.cpp b/src/mongo/db/catalog/multi_index_block.cpp
index d02bc489eaa..df41b1929ba 100644
--- a/src/mongo/db/catalog/multi_index_block.cpp
+++ b/src/mongo/db/catalog/multi_index_block.cpp
@@ -688,7 +688,7 @@ Status MultiIndexBlock::drainBackgroundWrites(
ReadSourceScope readSourceScope(opCtx, readSource);
const CollectionPtr& coll =
- CollectionCatalog::get(opCtx).lookupCollectionByUUID(opCtx, _collectionUUID.get());
+ CollectionCatalog::get(opCtx)->lookupCollectionByUUID(opCtx, _collectionUUID.get());
// Drain side-writes table for each index. This only drains what is visible. Assuming intent
// locks are held on the user collection, more writes can come in after this drain completes.
diff --git a/src/mongo/db/catalog/rename_collection.cpp b/src/mongo/db/catalog/rename_collection.cpp
index cf03e0b86de..0019f1b18ad 100644
--- a/src/mongo/db/catalog/rename_collection.cpp
+++ b/src/mongo/db/catalog/rename_collection.cpp
@@ -67,7 +67,7 @@ namespace {
MONGO_FAIL_POINT_DEFINE(writeConflictInRenameCollCopyToTmp);
boost::optional<NamespaceString> getNamespaceFromUUID(OperationContext* opCtx, const UUID& uuid) {
- return CollectionCatalog::get(opCtx).lookupNSSByUUID(opCtx, uuid);
+ return CollectionCatalog::get(opCtx)->lookupNSSByUUID(opCtx, uuid);
}
bool isCollectionSharded(OperationContext* opCtx, const NamespaceString& nss) {
@@ -110,8 +110,8 @@ Status checkSourceAndTargetNamespaces(OperationContext* opCtx,
str::stream()
<< "Database " << source.db() << " does not exist or is drop pending");
- const auto sourceColl =
- CollectionCatalog::get(opCtx).lookupCollectionByNamespace(opCtx, source);
+ auto catalog = CollectionCatalog::get(opCtx);
+ const auto sourceColl = catalog->lookupCollectionByNamespace(opCtx, source);
if (!sourceColl) {
if (ViewCatalog::get(db)->lookup(opCtx, source.ns()))
return Status(ErrorCodes::CommandNotSupportedOnView,
@@ -122,8 +122,7 @@ Status checkSourceAndTargetNamespaces(OperationContext* opCtx,
IndexBuildsCoordinator::get(opCtx)->assertNoIndexBuildInProgForCollection(sourceColl->uuid());
- const auto targetColl =
- CollectionCatalog::get(opCtx).lookupCollectionByNamespace(opCtx, target);
+ const auto targetColl = catalog->lookupCollectionByNamespace(opCtx, target);
if (!targetColl) {
if (ViewCatalog::get(db)->lookup(opCtx, target.ns()))
@@ -316,17 +315,16 @@ Status renameCollectionWithinDB(OperationContext* opCtx,
return status;
auto db = DatabaseHolder::get(opCtx)->getDb(opCtx, source.db());
- const auto sourceColl =
- CollectionCatalog::get(opCtx).lookupCollectionByNamespace(opCtx, source);
- const auto targetColl =
- CollectionCatalog::get(opCtx).lookupCollectionByNamespace(opCtx, target);
+ auto catalog = CollectionCatalog::get(opCtx);
+ const auto sourceColl = catalog->lookupCollectionByNamespace(opCtx, source);
+ const auto targetColl = catalog->lookupCollectionByNamespace(opCtx, target);
AutoStatsTracker statsTracker(
opCtx,
source,
Top::LockType::NotLocked,
AutoStatsTracker::LogMode::kUpdateCurOp,
- CollectionCatalog::get(opCtx).getDatabaseProfileLevel(source.db()));
+ CollectionCatalog::get(opCtx)->getDatabaseProfileLevel(source.db()));
if (!targetColl) {
return renameCollectionDirectly(opCtx, db, sourceColl->uuid(), source, target, options);
@@ -360,17 +358,17 @@ Status renameCollectionWithinDBForApplyOps(OperationContext* opCtx,
auto db = DatabaseHolder::get(opCtx)->getDb(opCtx, source.db());
const auto sourceColl =
- CollectionCatalog::get(opCtx).lookupCollectionByNamespace(opCtx, source);
+ CollectionCatalog::get(opCtx)->lookupCollectionByNamespace(opCtx, source);
AutoStatsTracker statsTracker(
opCtx,
source,
Top::LockType::NotLocked,
AutoStatsTracker::LogMode::kUpdateCurOp,
- CollectionCatalog::get(opCtx).getDatabaseProfileLevel(source.db()));
+ CollectionCatalog::get(opCtx)->getDatabaseProfileLevel(source.db()));
return writeConflictRetry(opCtx, "renameCollection", target.ns(), [&] {
- auto targetColl = CollectionCatalog::get(opCtx).lookupCollectionByNamespace(opCtx, target);
+ auto targetColl = CollectionCatalog::get(opCtx)->lookupCollectionByNamespace(opCtx, target);
WriteUnitOfWork wuow(opCtx);
if (targetColl) {
if (sourceColl->uuid() == targetColl->uuid()) {
@@ -414,7 +412,7 @@ Status renameCollectionWithinDBForApplyOps(OperationContext* opCtx,
auto collToDropBasedOnUUID = getNamespaceFromUUID(opCtx, uuidToDrop.get());
if (collToDropBasedOnUUID && !collToDropBasedOnUUID->isDropPendingNamespace()) {
invariant(collToDropBasedOnUUID->db() == target.db());
- targetColl = CollectionCatalog::get(opCtx).lookupCollectionByNamespace(
+ targetColl = CollectionCatalog::get(opCtx)->lookupCollectionByNamespace(
opCtx, *collToDropBasedOnUUID);
}
}
@@ -485,10 +483,10 @@ Status renameBetweenDBs(OperationContext* opCtx,
source,
Top::LockType::NotLocked,
AutoStatsTracker::LogMode::kUpdateCurOp,
- CollectionCatalog::get(opCtx).getDatabaseProfileLevel(source.db()));
+ CollectionCatalog::get(opCtx)->getDatabaseProfileLevel(source.db()));
- const auto sourceColl =
- CollectionCatalog::get(opCtx).lookupCollectionByNamespace(opCtx, source);
+ auto catalog = CollectionCatalog::get(opCtx);
+ const auto sourceColl = catalog->lookupCollectionByNamespace(opCtx, source);
if (!sourceColl) {
if (sourceDB && ViewCatalog::get(sourceDB)->lookup(opCtx, source.ns()))
return Status(ErrorCodes::CommandNotSupportedOnView,
@@ -511,9 +509,8 @@ Status renameBetweenDBs(OperationContext* opCtx,
// Check if the target namespace exists and if dropTarget is true.
// Return a non-OK status if target exists and dropTarget is not true or if the collection
// is sharded.
- const auto targetColl = targetDB
- ? CollectionCatalog::get(opCtx).lookupCollectionByNamespace(opCtx, target)
- : nullptr;
+ const auto targetColl =
+ targetDB ? catalog->lookupCollectionByNamespace(opCtx, target) : nullptr;
if (targetColl) {
if (sourceColl->uuid() == targetColl->uuid()) {
invariant(source == target);
@@ -730,7 +727,7 @@ void doLocalRenameIfOptionsAndIndexesHaveNotChanged(OperationContext* opCtx,
BSONObj originalCollectionOptions) {
AutoGetDb dbLock(opCtx, targetNs.db(), MODE_X);
auto collection = dbLock.getDb()
- ? CollectionCatalog::get(opCtx).lookupCollectionByNamespace(opCtx, targetNs)
+ ? CollectionCatalog::get(opCtx)->lookupCollectionByNamespace(opCtx, targetNs)
: nullptr;
BSONObj collectionOptions = {};
if (collection) {
@@ -865,7 +862,7 @@ Status renameCollectionForApplyOps(OperationContext* opCtx,
NamespaceString sourceNss(sourceNsElt.valueStringData());
NamespaceString targetNss(targetNsElt.valueStringData());
if (uuidToRename) {
- auto nss = CollectionCatalog::get(opCtx).lookupNSSByUUID(opCtx, uuidToRename.get());
+ auto nss = CollectionCatalog::get(opCtx)->lookupNSSByUUID(opCtx, uuidToRename.get());
if (nss)
sourceNss = *nss;
}
diff --git a/src/mongo/db/catalog/rename_collection_test.cpp b/src/mongo/db/catalog/rename_collection_test.cpp
index 79e0e09910b..b34d7eedf48 100644
--- a/src/mongo/db/catalog/rename_collection_test.cpp
+++ b/src/mongo/db/catalog/rename_collection_test.cpp
@@ -437,7 +437,8 @@ CollectionUUID _getCollectionUuid(OperationContext* opCtx, const NamespaceString
* Get collection namespace by UUID.
*/
NamespaceString _getCollectionNssFromUUID(OperationContext* opCtx, const UUID& uuid) {
- const CollectionPtr& source = CollectionCatalog::get(opCtx).lookupCollectionByUUID(opCtx, uuid);
+ const CollectionPtr& source =
+ CollectionCatalog::get(opCtx)->lookupCollectionByUUID(opCtx, uuid);
return source ? source->ns() : NamespaceString();
}
@@ -502,7 +503,7 @@ CollectionPtr _getCollection_inlock(OperationContext* opCtx, const NamespaceStri
if (!db) {
return nullptr;
}
- return CollectionCatalog::get(opCtx).lookupCollectionByNamespace(opCtx, nss);
+ return CollectionCatalog::get(opCtx)->lookupCollectionByNamespace(opCtx, nss);
}
TEST_F(RenameCollectionTest, RenameCollectionReturnsNamespaceNotFoundIfDatabaseDoesNotExist) {
@@ -680,7 +681,7 @@ TEST_F(RenameCollectionTest, RenameCollectionForApplyOpsDropTargetByUUIDTargetEx
ASSERT_TRUE(_collectionExists(_opCtx.get(), collB));
// The original B should exist too, but with a temporary name
const auto& tmpB =
- CollectionCatalog::get(_opCtx.get()).lookupNSSByUUID(_opCtx.get(), collBUUID);
+ CollectionCatalog::get(_opCtx.get())->lookupNSSByUUID(_opCtx.get(), collBUUID);
ASSERT(tmpB);
ASSERT_TRUE(tmpB->coll().startsWith("tmp"));
ASSERT_TRUE(*tmpB != collB);
@@ -713,7 +714,7 @@ TEST_F(RenameCollectionTest,
ASSERT_TRUE(_collectionExists(_opCtx.get(), collB));
// The original B should exist too, but with a temporary name
const auto& tmpB =
- CollectionCatalog::get(_opCtx.get()).lookupNSSByUUID(_opCtx.get(), collBUUID);
+ CollectionCatalog::get(_opCtx.get())->lookupNSSByUUID(_opCtx.get(), collBUUID);
ASSERT(tmpB);
ASSERT_TRUE(*tmpB != collB);
ASSERT_TRUE(tmpB->coll().startsWith("tmp"));
@@ -739,7 +740,7 @@ TEST_F(RenameCollectionTest,
ASSERT_TRUE(_collectionExists(_opCtx.get(), collB));
// The original B should exist too, but with a temporary name
const auto& tmpB =
- CollectionCatalog::get(_opCtx.get()).lookupNSSByUUID(_opCtx.get(), collBUUID);
+ CollectionCatalog::get(_opCtx.get())->lookupNSSByUUID(_opCtx.get(), collBUUID);
ASSERT(tmpB);
ASSERT_TRUE(*tmpB != collB);
ASSERT_TRUE(tmpB->coll().startsWith("tmp"));
diff --git a/src/mongo/db/catalog/uncommitted_collections.cpp b/src/mongo/db/catalog/uncommitted_collections.cpp
index cf38046f224..205351fb8ca 100644
--- a/src/mongo/db/catalog/uncommitted_collections.cpp
+++ b/src/mongo/db/catalog/uncommitted_collections.cpp
@@ -125,7 +125,10 @@ void UncommittedCollections::erase(UUID uuid, NamespaceString nss, UncommittedCo
void UncommittedCollections::rollback(OperationContext* opCtx,
CollectionUUID uuid,
UncommittedCollectionsMap* map) {
- auto collPtr = CollectionCatalog::get(opCtx).deregisterCollection(opCtx, uuid);
+ std::shared_ptr<Collection> collPtr;
+ CollectionCatalog::write(opCtx, [&](CollectionCatalog& catalog) {
+ collPtr = catalog.deregisterCollection(opCtx, uuid);
+ });
auto nss = collPtr.get()->ns();
map->_collections[uuid] = std::move(collPtr);
map->_nssIndex.insert({nss, uuid});
@@ -144,7 +147,9 @@ void UncommittedCollections::commit(OperationContext* opCtx,
auto collPtr = it->second.get();
auto nss = it->second->ns();
- CollectionCatalog::get(opCtx).registerCollection(uuid, it->second);
+ CollectionCatalog::write(
+ opCtx, [&](CollectionCatalog& catalog) { catalog.registerCollection(uuid, it->second); });
+
map->_collections.erase(it);
map->_nssIndex.erase(nss);
auto collListUnowned = getUncommittedCollections(opCtx).getResources();
@@ -153,7 +158,7 @@ void UncommittedCollections::commit(OperationContext* opCtx,
UncommittedCollections::rollback(opCtx, uuid, collListUnowned.lock().get());
});
opCtx->recoveryUnit()->onCommit([opCtx, uuid, collPtr](boost::optional<Timestamp> commitTs) {
- CollectionCatalog::get(opCtx).makeCollectionVisible(uuid);
+ collPtr->setCommitted(true);
// If a commitTs exists, by this point a collection should have a minimum visible snapshot
// equal to `commitTs`.
invariant(!commitTs ||
diff --git a/src/mongo/db/catalog/validate_state.cpp b/src/mongo/db/catalog/validate_state.cpp
index 77ecc28ca20..61a159b8c62 100644
--- a/src/mongo/db/catalog/validate_state.cpp
+++ b/src/mongo/db/catalog/validate_state.cpp
@@ -78,7 +78,7 @@ ValidateState::ValidateState(OperationContext* opCtx,
_database = _databaseLock->getDb() ? _databaseLock->getDb() : nullptr;
if (_database)
- _collection = CollectionCatalog::get(opCtx).lookupCollectionByNamespace(opCtx, _nss);
+ _collection = CollectionCatalog::get(opCtx)->lookupCollectionByNamespace(opCtx, _nss);
if (!_collection) {
if (_database && ViewCatalog::get(_database)->lookup(opCtx, _nss.ns())) {
@@ -297,7 +297,7 @@ void ValidateState::_relockDatabaseAndCollection(OperationContext* opCtx) {
uasserted(ErrorCodes::Interrupted, collErrMsg);
}
- _collection = CollectionCatalog::get(opCtx).lookupCollectionByUUID(opCtx, *_uuid);
+ _collection = CollectionCatalog::get(opCtx)->lookupCollectionByUUID(opCtx, *_uuid);
uassert(ErrorCodes::Interrupted, collErrMsg, _collection);
// The namespace of the collection can be changed during a same database collection rename.
diff --git a/src/mongo/db/catalog_raii.cpp b/src/mongo/db/catalog_raii.cpp
index 5fae19c8114..ba7dbe54cc2 100644
--- a/src/mongo/db/catalog_raii.cpp
+++ b/src/mongo/db/catalog_raii.cpp
@@ -97,7 +97,8 @@ AutoGetCollection::AutoGetCollection(OperationContext* opCtx,
}
_collLock.emplace(opCtx, nsOrUUID, modeColl, deadline);
- _resolvedNss = CollectionCatalog::get(opCtx).resolveNamespaceStringOrUUID(opCtx, nsOrUUID);
+ auto catalog = CollectionCatalog::get(opCtx);
+ _resolvedNss = catalog->resolveNamespaceStringOrUUID(opCtx, nsOrUUID);
// Wait for a configured amount of time after acquiring locks if the failpoint is enabled
setAutoGetCollectionWait.execute(
@@ -121,7 +122,7 @@ AutoGetCollection::AutoGetCollection(OperationContext* opCtx,
if (!db)
return;
- _coll = CollectionCatalog::get(opCtx).lookupCollectionByNamespace(opCtx, _resolvedNss);
+ _coll = catalog->lookupCollectionByNamespace(opCtx, _resolvedNss);
invariant(!nsOrUUID.uuid() || _coll,
str::stream() << "Collection for " << _resolvedNss.ns()
<< " disappeared after successufully resolving "
@@ -190,9 +191,9 @@ Collection* AutoGetCollection::getWritableCollection(CollectionCatalog::Lifetime
const Collection* _originalCollection;
};
- auto& catalog = CollectionCatalog::get(_opCtx);
+ auto catalog = CollectionCatalog::get(_opCtx);
_writableColl =
- catalog.lookupCollectionByNamespaceForMetadataWrite(_opCtx, mode, _resolvedNss);
+ catalog->lookupCollectionByNamespaceForMetadataWrite(_opCtx, mode, _resolvedNss);
if (mode == CollectionCatalog::LifetimeMode::kManagedInWriteUnitOfWork) {
_opCtx->recoveryUnit()->registerChange(
std::make_unique<WritableCollectionReset>(*this, _coll.get()));
@@ -216,15 +217,15 @@ AutoGetCollectionLockFree::AutoGetCollectionLockFree(OperationContext* opCtx,
setAutoGetCollectionWait.execute(
[&](const BSONObj& data) { sleepFor(Milliseconds(data["waitForMillis"].numberInt())); });
- _resolvedNss = CollectionCatalog::get(opCtx).resolveNamespaceStringOrUUID(opCtx, nsOrUUID);
- _collection =
- CollectionCatalog::get(opCtx).lookupCollectionByNamespaceForRead(opCtx, _resolvedNss);
+ auto catalog = CollectionCatalog::get(opCtx);
+ _resolvedNss = catalog->resolveNamespaceStringOrUUID(opCtx, nsOrUUID);
+ _collection = catalog->lookupCollectionByNamespaceForRead(opCtx, _resolvedNss);
// When we restore from yield on this CollectionPtr we will update _collection above and use its
// new pointer in the CollectionPtr
_collectionPtr = CollectionPtr(
opCtx, _collection.get(), [this](OperationContext* opCtx, CollectionUUID uuid) {
- _collection = CollectionCatalog::get(opCtx).lookupCollectionByUUIDForRead(opCtx, uuid);
+ _collection = CollectionCatalog::get(opCtx)->lookupCollectionByUUIDForRead(opCtx, uuid);
return _collection.get();
});
@@ -262,10 +263,10 @@ CollectionWriter::CollectionWriter(OperationContext* opCtx,
_mode(mode),
_sharedImpl(std::make_shared<SharedImpl>(this)) {
- _storedCollection = CollectionCatalog::get(opCtx).lookupCollectionByUUID(opCtx, uuid);
+ _storedCollection = CollectionCatalog::get(opCtx)->lookupCollectionByUUID(opCtx, uuid);
_sharedImpl->_writableCollectionInitializer = [opCtx,
uuid](CollectionCatalog::LifetimeMode mode) {
- return CollectionCatalog::get(opCtx).lookupCollectionByUUIDForMetadataWrite(
+ return CollectionCatalog::get(opCtx)->lookupCollectionByUUIDForMetadataWrite(
opCtx, mode, uuid);
};
}
@@ -277,10 +278,10 @@ CollectionWriter::CollectionWriter(OperationContext* opCtx,
_opCtx(opCtx),
_mode(mode),
_sharedImpl(std::make_shared<SharedImpl>(this)) {
- _storedCollection = CollectionCatalog::get(opCtx).lookupCollectionByNamespace(opCtx, nss);
+ _storedCollection = CollectionCatalog::get(opCtx)->lookupCollectionByNamespace(opCtx, nss);
_sharedImpl->_writableCollectionInitializer = [opCtx,
nss](CollectionCatalog::LifetimeMode mode) {
- return CollectionCatalog::get(opCtx).lookupCollectionByNamespaceForMetadataWrite(
+ return CollectionCatalog::get(opCtx)->lookupCollectionByNamespaceForMetadataWrite(
opCtx, mode, nss);
};
}
@@ -310,7 +311,7 @@ CollectionWriter::~CollectionWriter() {
}
if (_mode == CollectionCatalog::LifetimeMode::kUnmanagedClone && _writableCollection) {
- CollectionCatalog::get(_opCtx).discardUnmanagedClone(_opCtx, _writableCollection);
+ CollectionCatalog::discardUnmanagedClone(_opCtx, _writableCollection);
}
}
@@ -362,7 +363,7 @@ Collection* CollectionWriter::getWritableCollection() {
void CollectionWriter::commitToCatalog() {
dassert(_mode == CollectionCatalog::LifetimeMode::kUnmanagedClone);
dassert(_writableCollection);
- CollectionCatalog::get(_opCtx).commitUnmanagedClone(_opCtx, _writableCollection);
+ CollectionCatalog::commitUnmanagedClone(_opCtx, _writableCollection);
_writableCollection = nullptr;
}
@@ -378,16 +379,6 @@ AutoGetOrCreateDb::AutoGetOrCreateDb(OperationContext* opCtx,
invariant(mode == MODE_IX || mode == MODE_X);
}
-ConcealCollectionCatalogChangesBlock::ConcealCollectionCatalogChangesBlock(OperationContext* opCtx)
- : _opCtx(opCtx) {
- CollectionCatalog::get(_opCtx).onCloseCatalog(_opCtx);
-}
-
-ConcealCollectionCatalogChangesBlock::~ConcealCollectionCatalogChangesBlock() {
- invariant(_opCtx);
- CollectionCatalog::get(_opCtx).onOpenCatalog(_opCtx);
-}
-
ReadSourceScope::ReadSourceScope(OperationContext* opCtx,
RecoveryUnit::ReadSource readSource,
boost::optional<Timestamp> provided)
diff --git a/src/mongo/db/catalog_raii.h b/src/mongo/db/catalog_raii.h
index 196b4c52a38..f5e7a8e5f26 100644
--- a/src/mongo/db/catalog_raii.h
+++ b/src/mongo/db/catalog_raii.h
@@ -406,35 +406,6 @@ private:
};
/**
- * RAII-style class. Hides changes to the CollectionCatalog for the life of the object, so that
- * calls to CollectionCatalog::lookupNSSByUUID will return results as before the RAII object was
- * instantiated.
- *
- * The caller must hold the global exclusive lock for the life of the instance.
- */
-class ConcealCollectionCatalogChangesBlock {
- ConcealCollectionCatalogChangesBlock(const ConcealCollectionCatalogChangesBlock&) = delete;
- ConcealCollectionCatalogChangesBlock& operator=(const ConcealCollectionCatalogChangesBlock&) =
- delete;
-
-public:
- /**
- * Conceals future CollectionCatalog changes and stashes a pointer to the opCtx for the
- * destructor to use.
- */
- ConcealCollectionCatalogChangesBlock(OperationContext* opCtx);
-
- /**
- * Reveals CollectionCatalog changes.
- */
- ~ConcealCollectionCatalogChangesBlock();
-
-private:
- // Needed for the destructor to access the CollectionCatalog in order to call onOpenCatalog.
- OperationContext* _opCtx;
-};
-
-/**
* RAII type to set and restore the timestamp read source on the recovery unit.
*
* Snapshot is abandoned in constructor and destructor, so it can only be used before
diff --git a/src/mongo/db/cloner.cpp b/src/mongo/db/cloner.cpp
index 2e2a6f417d6..20fd51253cf 100644
--- a/src/mongo/db/cloner.cpp
+++ b/src/mongo/db/cloner.cpp
@@ -103,7 +103,8 @@ struct Cloner::Fun {
// Make sure database still exists after we resume from the temp release
auto databaseHolder = DatabaseHolder::get(opCtx);
auto db = databaseHolder->openDb(opCtx, _dbName);
- auto collection = CollectionCatalog::get(opCtx).lookupCollectionByNamespace(opCtx, nss);
+ auto catalog = CollectionCatalog::get(opCtx);
+ auto collection = catalog->lookupCollectionByNamespace(opCtx, nss);
if (!collection) {
writeConflictRetry(opCtx, "createCollection", nss.ns(), [&] {
opCtx->checkForInterrupt();
@@ -117,7 +118,7 @@ struct Cloner::Fun {
str::stream()
<< "collection creation failed during clone [" << nss << "]");
wunit.commit();
- collection = CollectionCatalog::get(opCtx).lookupCollectionByNamespace(opCtx, nss);
+ collection = catalog->lookupCollectionByNamespace(opCtx, nss);
invariant(collection,
str::stream() << "Missing collection during clone [" << nss << "]");
});
@@ -153,7 +154,7 @@ struct Cloner::Fun {
str::stream() << "Database " << _dbName << " dropped while cloning",
db != nullptr);
- collection = CollectionCatalog::get(opCtx).lookupCollectionByNamespace(opCtx, nss);
+ collection = catalog->lookupCollectionByNamespace(opCtx, nss);
uassert(28594,
str::stream() << "Collection " << nss << " dropped while cloning",
collection);
@@ -349,6 +350,7 @@ Status Cloner::_createCollectionsForDb(
auto db = databaseHolder->openDb(opCtx, dbName);
invariant(opCtx->lockState()->isDbLockedForMode(dbName, MODE_X));
+ auto catalog = CollectionCatalog::get(opCtx);
auto collCount = 0;
for (auto&& params : createCollectionParams) {
if (MONGO_unlikely(movePrimaryFailPoint.shouldFail()) && collCount > 0) {
@@ -366,8 +368,7 @@ Status Cloner::_createCollectionsForDb(
opCtx->checkForInterrupt();
WriteUnitOfWork wunit(opCtx);
- CollectionPtr collection =
- CollectionCatalog::get(opCtx).lookupCollectionByNamespace(opCtx, nss);
+ CollectionPtr collection = catalog->lookupCollectionByNamespace(opCtx, nss);
if (collection) {
if (!params.shardedColl) {
// If the collection is unsharded then we want to fail when a collection
diff --git a/src/mongo/db/commands/count_cmd.cpp b/src/mongo/db/commands/count_cmd.cpp
index b2e23e4887c..77c8fb9bb22 100644
--- a/src/mongo/db/commands/count_cmd.cpp
+++ b/src/mongo/db/commands/count_cmd.cpp
@@ -126,7 +126,7 @@ public:
const auto hasTerm = false;
return authSession->checkAuthForFind(
- CollectionCatalog::get(opCtx).resolveNamespaceStringOrUUID(
+ CollectionCatalog::get(opCtx)->resolveNamespaceStringOrUUID(
opCtx, CommandHelpers::parseNsOrUUID(dbname, cmdObj)),
hasTerm);
}
diff --git a/src/mongo/db/commands/create_indexes.cpp b/src/mongo/db/commands/create_indexes.cpp
index 37ee281f48a..3ecaafbc401 100644
--- a/src/mongo/db/commands/create_indexes.cpp
+++ b/src/mongo/db/commands/create_indexes.cpp
@@ -570,7 +570,7 @@ bool runCreateIndexesWithCoordinator(OperationContext* opCtx,
ns,
Top::LockType::WriteLocked,
AutoStatsTracker::LogMode::kUpdateTopAndCurOp,
- CollectionCatalog::get(opCtx).getDatabaseProfileLevel(ns.db()));
+ CollectionCatalog::get(opCtx)->getDatabaseProfileLevel(ns.db()));
auto buildUUID = UUID::gen();
ReplIndexBuildState::IndexCatalogStats stats;
diff --git a/src/mongo/db/commands/dbcheck.cpp b/src/mongo/db/commands/dbcheck.cpp
index 41a52193ba3..a8ff511fe6d 100644
--- a/src/mongo/db/commands/dbcheck.cpp
+++ b/src/mongo/db/commands/dbcheck.cpp
@@ -138,7 +138,8 @@ std::unique_ptr<DbCheckRun> fullDatabaseRun(OperationContext* opCtx,
int64_t max = std::numeric_limits<int64_t>::max();
auto rate = invocation.getMaxCountPerSecond();
- for (auto collIt = db->begin(opCtx); collIt != db->end(opCtx); ++collIt) {
+ auto catalog = CollectionCatalog::get(opCtx);
+ for (auto collIt = catalog->begin(opCtx, db->name()); collIt != catalog->end(opCtx); ++collIt) {
auto coll = *collIt;
if (!coll) {
break;
@@ -334,7 +335,7 @@ private:
}
auto collection =
- CollectionCatalog::get(opCtx).lookupCollectionByNamespace(opCtx, info.nss);
+ CollectionCatalog::get(opCtx)->lookupCollectionByNamespace(opCtx, info.nss);
if (!collection) {
return false;
}
diff --git a/src/mongo/db/commands/dbcommands.cpp b/src/mongo/db/commands/dbcommands.cpp
index 1306678f0e7..bd5f6ed23e1 100644
--- a/src/mongo/db/commands/dbcommands.cpp
+++ b/src/mongo/db/commands/dbcommands.cpp
@@ -835,7 +835,7 @@ public:
stdx::lock_guard<Client> lk(*opCtx->getClient());
// TODO: OldClientContext legacy, needs to be removed
CurOp::get(opCtx)->enter_inlock(
- dbname.c_str(), CollectionCatalog::get(opCtx).getDatabaseProfileLevel(dbname));
+ dbname.c_str(), CollectionCatalog::get(opCtx)->getDatabaseProfileLevel(dbname));
}
db->getStats(opCtx, &result, scale);
diff --git a/src/mongo/db/commands/dbcommands_d.cpp b/src/mongo/db/commands/dbcommands_d.cpp
index bfe0fc17cb1..66b175fd6bb 100644
--- a/src/mongo/db/commands/dbcommands_d.cpp
+++ b/src/mongo/db/commands/dbcommands_d.cpp
@@ -112,7 +112,7 @@ Status _setProfileSettings(OperationContext* opCtx,
mongo::CollectionCatalog::ProfileSettings newSettings) {
invariant(db);
- auto currSettings = CollectionCatalog::get(opCtx).getDatabaseProfileSettings(dbName);
+ auto currSettings = CollectionCatalog::get(opCtx)->getDatabaseProfileSettings(dbName);
if (currSettings == newSettings) {
return Status::OK();
@@ -120,7 +120,9 @@ Status _setProfileSettings(OperationContext* opCtx,
if (newSettings.level == 0) {
// No need to create the profile collection.
- CollectionCatalog::get(opCtx).setDatabaseProfileSettings(dbName, newSettings);
+ CollectionCatalog::write(opCtx, [&](CollectionCatalog& catalog) {
+ catalog.setDatabaseProfileSettings(dbName, newSettings);
+ });
return Status::OK();
}
@@ -135,7 +137,9 @@ Status _setProfileSettings(OperationContext* opCtx,
return status;
}
- CollectionCatalog::get(opCtx).setDatabaseProfileSettings(dbName, newSettings);
+ CollectionCatalog::write(opCtx, [&](CollectionCatalog& catalog) {
+ catalog.setDatabaseProfileSettings(dbName, newSettings);
+ });
return Status::OK();
}
@@ -175,7 +179,7 @@ protected:
// Fetches the database profiling level + filter or the server default if the db does not
// exist.
- auto oldSettings = CollectionCatalog::get(opCtx).getDatabaseProfileSettings(dbName);
+ auto oldSettings = CollectionCatalog::get(opCtx)->getDatabaseProfileSettings(dbName);
if (!readOnly) {
if (!db) {
diff --git a/src/mongo/db/commands/dbhash.cpp b/src/mongo/db/commands/dbhash.cpp
index 35512276a51..192243c49ee 100644
--- a/src/mongo/db/commands/dbhash.cpp
+++ b/src/mongo/db/commands/dbhash.cpp
@@ -318,7 +318,7 @@ private:
std::string _hashCollection(OperationContext* opCtx, Database* db, const NamespaceString& nss) {
CollectionPtr collection =
- CollectionCatalog::get(opCtx).lookupCollectionByNamespace(opCtx, nss);
+ CollectionCatalog::get(opCtx)->lookupCollectionByNamespace(opCtx, nss);
invariant(collection);
boost::optional<Lock::CollectionLock> collLock;
diff --git a/src/mongo/db/commands/distinct.cpp b/src/mongo/db/commands/distinct.cpp
index 24ff3604b35..f261336321f 100644
--- a/src/mongo/db/commands/distinct.cpp
+++ b/src/mongo/db/commands/distinct.cpp
@@ -124,7 +124,7 @@ public:
const auto hasTerm = false;
return authSession->checkAuthForFind(
- CollectionCatalog::get(opCtx).resolveNamespaceStringOrUUID(
+ CollectionCatalog::get(opCtx)->resolveNamespaceStringOrUUID(
opCtx, CommandHelpers::parseNsOrUUID(dbname, cmdObj)),
hasTerm);
}
diff --git a/src/mongo/db/commands/find_and_modify.cpp b/src/mongo/db/commands/find_and_modify.cpp
index 4586958eeb7..dc19a0c2d4c 100644
--- a/src/mongo/db/commands/find_and_modify.cpp
+++ b/src/mongo/db/commands/find_and_modify.cpp
@@ -473,7 +473,7 @@ public:
stdx::lock_guard<Client> lk(*opCtx->getClient());
CurOp::get(opCtx)->enter_inlock(
nsString.ns().c_str(),
- CollectionCatalog::get(opCtx).getDatabaseProfileLevel(nsString.db()));
+ CollectionCatalog::get(opCtx)->getDatabaseProfileLevel(nsString.db()));
}
assertCanWrite(opCtx, nsString);
@@ -532,7 +532,7 @@ public:
stdx::lock_guard<Client> lk(*opCtx->getClient());
CurOp::get(opCtx)->enter_inlock(
nsString.ns().c_str(),
- CollectionCatalog::get(opCtx).getDatabaseProfileLevel(nsString.db()));
+ CollectionCatalog::get(opCtx)->getDatabaseProfileLevel(nsString.db()));
}
assertCanWrite(opCtx, nsString);
@@ -547,7 +547,7 @@ public:
assertCanWrite(opCtx, nsString);
createdCollection =
- CollectionCatalog::get(opCtx).lookupCollectionByNamespace(opCtx, nsString);
+ CollectionCatalog::get(opCtx)->lookupCollectionByNamespace(opCtx, nsString);
// If someone else beat us to creating the collection, do nothing
if (!createdCollection) {
@@ -558,7 +558,7 @@ public:
wuow.commit();
createdCollection =
- CollectionCatalog::get(opCtx).lookupCollectionByNamespace(opCtx, nsString);
+ CollectionCatalog::get(opCtx)->lookupCollectionByNamespace(opCtx, nsString);
}
invariant(createdCollection);
diff --git a/src/mongo/db/commands/find_cmd.cpp b/src/mongo/db/commands/find_cmd.cpp
index 142d6bb9464..96334588aa5 100644
--- a/src/mongo/db/commands/find_cmd.cpp
+++ b/src/mongo/db/commands/find_cmd.cpp
@@ -227,7 +227,7 @@ public:
const auto hasTerm = _request.body.hasField(kTermField);
uassertStatusOK(authSession->checkAuthForFind(
- CollectionCatalog::get(opCtx).resolveNamespaceStringOrUUID(
+ CollectionCatalog::get(opCtx)->resolveNamespaceStringOrUUID(
opCtx, CommandHelpers::parseNsOrUUID(_dbName, _request.body)),
hasTerm));
}
diff --git a/src/mongo/db/commands/getmore_cmd.cpp b/src/mongo/db/commands/getmore_cmd.cpp
index eeda03b5df0..053082b86a9 100644
--- a/src/mongo/db/commands/getmore_cmd.cpp
+++ b/src/mongo/db/commands/getmore_cmd.cpp
@@ -395,7 +395,7 @@ public:
_request.nss,
Top::LockType::NotLocked,
AutoStatsTracker::LogMode::kUpdateTopAndCurOp,
- CollectionCatalog::get(opCtx).getDatabaseProfileLevel(_request.nss.db()));
+ CollectionCatalog::get(opCtx)->getDatabaseProfileLevel(_request.nss.db()));
}
} else {
invariant(cursorPin->getExecutor()->lockPolicy() ==
@@ -425,7 +425,7 @@ public:
_request.nss,
Top::LockType::ReadLocked,
AutoStatsTracker::LogMode::kUpdateTopAndCurOp,
- CollectionCatalog::get(opCtx).getDatabaseProfileLevel(_request.nss.db()));
+ CollectionCatalog::get(opCtx)->getDatabaseProfileLevel(_request.nss.db()));
// Check whether we are allowed to read from this node after acquiring our locks.
uassertStatusOK(repl::ReplicationCoordinator::get(opCtx)->checkCanServeReadsFor(
diff --git a/src/mongo/db/commands/killcursors_cmd.cpp b/src/mongo/db/commands/killcursors_cmd.cpp
index 9acf3b85b31..8e9aa80b0c9 100644
--- a/src/mongo/db/commands/killcursors_cmd.cpp
+++ b/src/mongo/db/commands/killcursors_cmd.cpp
@@ -74,7 +74,7 @@ private:
nss,
Top::LockType::NotLocked,
AutoStatsTracker::LogMode::kUpdateTopAndCurOp,
- CollectionCatalog::get(opCtx).getDatabaseProfileLevel(nss.db()));
+ CollectionCatalog::get(opCtx)->getDatabaseProfileLevel(nss.db()));
}
auto cursorManager = CursorManager::get(opCtx);
diff --git a/src/mongo/db/commands/list_collections.cpp b/src/mongo/db/commands/list_collections.cpp
index bd93f374fac..92e42f5f689 100644
--- a/src/mongo/db/commands/list_collections.cpp
+++ b/src/mongo/db/commands/list_collections.cpp
@@ -322,7 +322,7 @@ public:
Lock::CollectionLock clk(opCtx, nss, MODE_IS);
CollectionPtr collection =
- CollectionCatalog::get(opCtx).lookupCollectionByNamespace(opCtx, nss);
+ CollectionCatalog::get(opCtx)->lookupCollectionByNamespace(opCtx, nss);
BSONObj collBson =
buildCollectionBson(opCtx, collection, includePendingDrops, nameOnly);
if (!collBson.isEmpty()) {
diff --git a/src/mongo/db/commands/list_databases.cpp b/src/mongo/db/commands/list_databases.cpp
index 9da849e8cae..354801cdd7d 100644
--- a/src/mongo/db/commands/list_databases.cpp
+++ b/src/mongo/db/commands/list_databases.cpp
@@ -179,7 +179,7 @@ public:
b.appendBool(
"empty",
- CollectionCatalog::get(opCtx).getAllCollectionUUIDsFromDb(dbname).empty());
+ CollectionCatalog::get(opCtx)->getAllCollectionUUIDsFromDb(dbname).empty());
}
BSONObj curDbObj = b.obj();
diff --git a/src/mongo/db/commands/list_indexes.cpp b/src/mongo/db/commands/list_indexes.cpp
index 1b1386e1902..1573dfccd36 100644
--- a/src/mongo/db/commands/list_indexes.cpp
+++ b/src/mongo/db/commands/list_indexes.cpp
@@ -129,7 +129,7 @@ public:
}
// Check for the listIndexes ActionType on the database.
- const auto nss = CollectionCatalog::get(opCtx).resolveNamespaceStringOrUUID(
+ const auto nss = CollectionCatalog::get(opCtx)->resolveNamespaceStringOrUUID(
opCtx, CommandHelpers::parseNsOrUUID(dbname, cmdObj));
if (authzSession->isAuthorizedForActionsOnResource(ResourcePattern::forExactNamespace(nss),
ActionType::listIndexes)) {
diff --git a/src/mongo/db/commands/mr_test.cpp b/src/mongo/db/commands/mr_test.cpp
index 8ac437c6eed..da4dc97f507 100644
--- a/src/mongo/db/commands/mr_test.cpp
+++ b/src/mongo/db/commands/mr_test.cpp
@@ -556,7 +556,7 @@ TEST_F(MapReduceCommandTest, ReplacingExistingOutputCollectionPreservesIndexes)
ASSERT_NOT_EQUALS(
*options.uuid,
- *CollectionCatalog::get(_opCtx.get()).lookupUUIDByNSS(_opCtx.get(), outputNss))
+ *CollectionCatalog::get(_opCtx.get())->lookupUUIDByNSS(_opCtx.get(), outputNss))
<< "Output collection " << outputNss << " was not replaced";
_assertTemporaryCollectionsAreDropped();
diff --git a/src/mongo/db/commands/oplog_application_checks.cpp b/src/mongo/db/commands/oplog_application_checks.cpp
index 52aa18a2a76..b6651773d80 100644
--- a/src/mongo/db/commands/oplog_application_checks.cpp
+++ b/src/mongo/db/commands/oplog_application_checks.cpp
@@ -65,9 +65,9 @@ Status OplogApplicationChecks::checkOperationAuthorization(OperationContext* opC
if (oplogEntry.hasField("ui"_sd)) {
// ns by UUID overrides the ns specified if they are different.
- auto& catalog = CollectionCatalog::get(opCtx);
+ auto catalog = CollectionCatalog::get(opCtx);
boost::optional<NamespaceString> uuidCollNS =
- catalog.lookupNSSByUUID(opCtx, getUUIDFromOplogEntry(oplogEntry));
+ catalog->lookupNSSByUUID(opCtx, getUUIDFromOplogEntry(oplogEntry));
if (uuidCollNS && *uuidCollNS != ns)
ns = *uuidCollNS;
}
diff --git a/src/mongo/db/commands/profile_common.cpp b/src/mongo/db/commands/profile_common.cpp
index 8124366a638..45727007ab2 100644
--- a/src/mongo/db/commands/profile_common.cpp
+++ b/src/mongo/db/commands/profile_common.cpp
@@ -123,7 +123,7 @@ bool ProfileCmdBase::run(OperationContext* opCtx,
// newSettings.level may differ from profilingLevel: profilingLevel is part of the request,
// and if the request specifies {profile: -1, ...} then we want to show the unchanged value
// (0, 1, or 2).
- auto newSettings = CollectionCatalog::get(opCtx).getDatabaseProfileSettings(dbName);
+ auto newSettings = CollectionCatalog::get(opCtx)->getDatabaseProfileSettings(dbName);
newState.append("level"_sd, newSettings.level);
newState.append("slowms"_sd, serverGlobalParams.slowMS);
newState.append("sampleRate"_sd, serverGlobalParams.sampleRate);
diff --git a/src/mongo/db/commands/run_aggregate.cpp b/src/mongo/db/commands/run_aggregate.cpp
index 7c8f6565026..0dcc034f4ad 100644
--- a/src/mongo/db/commands/run_aggregate.cpp
+++ b/src/mongo/db/commands/run_aggregate.cpp
@@ -292,7 +292,7 @@ StatusWith<StringMap<ExpressionContext::ResolvedNamespace>> resolveInvolvedNames
// views, simply assume that the namespace is a collection.
resolvedNamespaces[involvedNs.coll()] = {involvedNs, std::vector<BSONObj>{}};
} else if (!db ||
- CollectionCatalog::get(opCtx).lookupCollectionByNamespace(opCtx, involvedNs)) {
+ CollectionCatalog::get(opCtx)->lookupCollectionByNamespace(opCtx, involvedNs)) {
// If the aggregation database exists and 'involvedNs' refers to a collection namespace,
// then we resolve it as an empty pipeline in order to read directly from the underlying
// collection. If the database doesn't exist, then we still resolve it as an empty
@@ -342,8 +342,9 @@ Status collatorCompatibleWithPipeline(OperationContext* opCtx,
if (!viewCatalog) {
return Status::OK();
}
+ auto catalog = CollectionCatalog::get(opCtx);
for (auto&& potentialViewNs : liteParsedPipeline.getInvolvedNamespaces()) {
- if (CollectionCatalog::get(opCtx).lookupCollectionByNamespace(opCtx, potentialViewNs)) {
+ if (catalog->lookupCollectionByNamespace(opCtx, potentialViewNs)) {
continue;
}
diff --git a/src/mongo/db/commands/set_feature_compatibility_version_command.cpp b/src/mongo/db/commands/set_feature_compatibility_version_command.cpp
index 0a2d2e598f0..46f18e13da0 100644
--- a/src/mongo/db/commands/set_feature_compatibility_version_command.cpp
+++ b/src/mongo/db/commands/set_feature_compatibility_version_command.cpp
@@ -363,12 +363,12 @@ private:
* TODO SERVER-51871: This method can be removed once 5.0 becomes last-lts.
*/
void _deleteHaystackIndexesOnUpgrade(OperationContext* opCtx) {
- auto& collCatalog = CollectionCatalog::get(opCtx);
- for (const auto& db : collCatalog.getAllDbNames()) {
- for (auto collIt = collCatalog.begin(opCtx, db); collIt != collCatalog.end(opCtx);
+ auto collCatalog = CollectionCatalog::get(opCtx);
+ for (const auto& db : collCatalog->getAllDbNames()) {
+ for (auto collIt = collCatalog->begin(opCtx, db); collIt != collCatalog->end(opCtx);
++collIt) {
NamespaceStringOrUUID collName(
- collCatalog.lookupNSSByUUID(opCtx, collIt.uuid().get()).get());
+ collCatalog->lookupNSSByUUID(opCtx, collIt.uuid().get()).get());
AutoGetCollectionForRead coll(opCtx, collName);
auto idxCatalog = coll->getIndexCatalog();
std::vector<const IndexDescriptor*> haystackIndexes;
diff --git a/src/mongo/db/commands/test_commands.cpp b/src/mongo/db/commands/test_commands.cpp
index c3813857782..382b8978400 100644
--- a/src/mongo/db/commands/test_commands.cpp
+++ b/src/mongo/db/commands/test_commands.cpp
@@ -93,7 +93,7 @@ public:
WriteUnitOfWork wunit(opCtx);
UnreplicatedWritesBlock unreplicatedWritesBlock(opCtx);
CollectionPtr collection =
- CollectionCatalog::get(opCtx).lookupCollectionByNamespace(opCtx, nss);
+ CollectionCatalog::get(opCtx)->lookupCollectionByNamespace(opCtx, nss);
if (!collection) {
collection = db->createCollection(opCtx, nss);
if (!collection) {
diff --git a/src/mongo/db/concurrency/d_concurrency.cpp b/src/mongo/db/concurrency/d_concurrency.cpp
index ea4738e55aa..abbd0c875c2 100644
--- a/src/mongo/db/concurrency/d_concurrency.cpp
+++ b/src/mongo/db/concurrency/d_concurrency.cpp
@@ -273,8 +273,7 @@ Lock::CollectionLock::CollectionLock(OperationContext* opCtx,
// 'nsOrUUID' must be a UUID and dbName.
- auto& collectionCatalog = CollectionCatalog::get(opCtx);
- auto nss = collectionCatalog.resolveNamespaceStringOrUUID(opCtx, nssOrUUID);
+ auto nss = CollectionCatalog::get(opCtx)->resolveNamespaceStringOrUUID(opCtx, nssOrUUID);
// The UUID cannot move between databases so this one dassert is sufficient.
dassert(_opCtx->lockState()->isDbLockedForMode(nss.db(),
@@ -297,7 +296,7 @@ Lock::CollectionLock::CollectionLock(OperationContext* opCtx,
// We looked up UUID without a collection lock so it's possible that the
// collection name changed now. Look it up again.
prevResolvedNss = nss;
- nss = collectionCatalog.resolveNamespaceStringOrUUID(opCtx, nssOrUUID);
+ nss = CollectionCatalog::get(opCtx)->resolveNamespaceStringOrUUID(opCtx, nssOrUUID);
} while (nss != prevResolvedNss);
}
diff --git a/src/mongo/db/concurrency/lock_manager.cpp b/src/mongo/db/concurrency/lock_manager.cpp
index e81bc2f46ff..25987286caf 100644
--- a/src/mongo/db/concurrency/lock_manager.cpp
+++ b/src/mongo/db/concurrency/lock_manager.cpp
@@ -954,8 +954,8 @@ std::string ResourceId::toString() const {
}
if (getType() == RESOURCE_DATABASE || getType() == RESOURCE_COLLECTION) {
- CollectionCatalog& catalog = CollectionCatalog::get(getGlobalServiceContext());
- boost::optional<std::string> resourceName = catalog.lookupResourceName(*this);
+ auto catalog = CollectionCatalog::get(getGlobalServiceContext());
+ boost::optional<std::string> resourceName = catalog->lookupResourceName(*this);
if (resourceName) {
ss << ", " << *resourceName;
}
diff --git a/src/mongo/db/curop.cpp b/src/mongo/db/curop.cpp
index 9c6a0685565..b4c5e64d14a 100644
--- a/src/mongo/db/curop.cpp
+++ b/src/mongo/db/curop.cpp
@@ -493,7 +493,7 @@ bool CurOp::completeAndLogOperation(OperationContext* opCtx,
bool shouldLogSlowOp, shouldProfileAtLevel1;
if (auto filter =
- CollectionCatalog::get(opCtx).getDatabaseProfileSettings(getNSS().db()).filter) {
+ CollectionCatalog::get(opCtx)->getDatabaseProfileSettings(getNSS().db()).filter) {
bool passesFilter = filter->matches(opCtx, _debug, *this);
shouldLogSlowOp = passesFilter;
diff --git a/src/mongo/db/curop.h b/src/mongo/db/curop.h
index 739ba91acc5..72e146c5d15 100644
--- a/src/mongo/db/curop.h
+++ b/src/mongo/db/curop.h
@@ -441,7 +441,7 @@ public:
if (_dbprofile <= 0)
return false;
- if (CollectionCatalog::get(opCtx).getDatabaseProfileSettings(getNSS().db()).filter)
+ if (CollectionCatalog::get(opCtx)->getDatabaseProfileSettings(getNSS().db()).filter)
return true;
return elapsedTimeExcludingPauses() >= Milliseconds{serverGlobalParams.slowMS};
diff --git a/src/mongo/db/db_raii.cpp b/src/mongo/db/db_raii.cpp
index 5336b8c6368..cb1d875cb1f 100644
--- a/src/mongo/db/db_raii.cpp
+++ b/src/mongo/db/db_raii.cpp
@@ -261,7 +261,8 @@ AutoGetCollectionForReadLockFree::AutoGetCollectionForReadLockFree(
OperationContext* opCtx,
const NamespaceStringOrUUID& nsOrUUID,
AutoGetCollectionViewMode viewMode,
- Date_t deadline) {
+ Date_t deadline)
+ : _catalogStash(opCtx) {
// Supported lock-free reads should never have an open storage snapshot prior to calling
// this helper. The storage snapshot and in-memory state fetched here must be consistent.
invariant(supportsLockFreeRead(opCtx) && !opCtx->recoveryUnit()->isActive());
@@ -271,6 +272,7 @@ AutoGetCollectionForReadLockFree::AutoGetCollectionForReadLockFree(
// state. Therefore we must fetch the repl state beforehand, to compare with afterwards.
long long replTerm = repl::ReplicationCoordinator::get(opCtx)->getTerm();
+ auto catalog = CollectionCatalog::get(opCtx);
_autoGetCollectionForReadBase.emplace(opCtx, nsOrUUID, viewMode, deadline);
// A lock request does not always find a collection to lock.
@@ -297,15 +299,17 @@ AutoGetCollectionForReadLockFree::AutoGetCollectionForReadLockFree(
opCtx->recoveryUnit()->preallocateSnapshot();
}
- auto newCollection = CollectionCatalog::get(opCtx).lookupCollectionByUUIDForRead(
+ auto newCatalog = CollectionCatalog::get(opCtx);
+ auto newCollection = newCatalog->lookupCollectionByUUIDForRead(
opCtx, _autoGetCollectionForReadBase.get()->uuid());
// The collection may have been dropped since the previous lookup, run the loop one more
// time to cleanup if newCollection is nullptr
- if (newCollection &&
+ if (newCollection && catalog == newCatalog &&
_autoGetCollectionForReadBase.get()->getMinimumVisibleSnapshot() ==
newCollection->getMinimumVisibleSnapshot() &&
replTerm == repl::ReplicationCoordinator::get(opCtx)->getTerm()) {
+ _catalogStash.stash(std::move(catalog));
break;
}
@@ -367,7 +371,7 @@ AutoGetCollectionForReadCommandBase<AutoGetCollectionForReadType>::
_autoCollForRead.getNss(),
Top::LockType::ReadLocked,
logMode,
- CollectionCatalog::get(opCtx).getDatabaseProfileLevel(_autoCollForRead.getNss().db()),
+ CollectionCatalog::get(opCtx)->getDatabaseProfileLevel(_autoCollForRead.getNss().db()),
deadline) {
if (!_autoCollForRead.getView()) {
@@ -401,7 +405,7 @@ OldClientContext::OldClientContext(OperationContext* opCtx, const std::string& n
stdx::lock_guard<Client> lk(*_opCtx->getClient());
currentOp->enter_inlock(ns.c_str(),
- CollectionCatalog::get(opCtx).getDatabaseProfileLevel(_db->name()));
+ CollectionCatalog::get(opCtx)->getDatabaseProfileLevel(_db->name()));
}
AutoGetCollectionForReadCommandMaybeLockFree::AutoGetCollectionForReadCommandMaybeLockFree(
diff --git a/src/mongo/db/db_raii.h b/src/mongo/db/db_raii.h
index b62a010832f..d8d7a0a3390 100644
--- a/src/mongo/db/db_raii.h
+++ b/src/mongo/db/db_raii.h
@@ -198,6 +198,7 @@ public:
private:
boost::optional<AutoGetCollectionForReadBase<AutoGetCollectionLockFree>>
_autoGetCollectionForReadBase;
+ CollectionCatalogStasher _catalogStash;
};
/**
diff --git a/src/mongo/db/dbhelpers.cpp b/src/mongo/db/dbhelpers.cpp
index 4730e429f1c..ee9a0956c8b 100644
--- a/src/mongo/db/dbhelpers.cpp
+++ b/src/mongo/db/dbhelpers.cpp
@@ -130,7 +130,7 @@ bool Helpers::findById(OperationContext* opCtx,
// TODO ForRead?
CollectionPtr collection =
- CollectionCatalog::get(opCtx).lookupCollectionByNamespace(opCtx, NamespaceString(ns));
+ CollectionCatalog::get(opCtx)->lookupCollectionByNamespace(opCtx, NamespaceString(ns));
if (!collection) {
return false;
}
@@ -307,7 +307,7 @@ void Helpers::emptyCollection(OperationContext* opCtx, const NamespaceString& ns
OldClientContext context(opCtx, nss.ns());
repl::UnreplicatedWritesBlock uwb(opCtx);
CollectionPtr collection = context.db()
- ? CollectionCatalog::get(opCtx).lookupCollectionByNamespace(opCtx, nss)
+ ? CollectionCatalog::get(opCtx)->lookupCollectionByNamespace(opCtx, nss)
: nullptr;
deleteObjects(opCtx, collection, nss, BSONObj(), false);
}
diff --git a/src/mongo/db/exec/requires_collection_stage.cpp b/src/mongo/db/exec/requires_collection_stage.cpp
index 7df44e2b8ed..d77b8f3eff7 100644
--- a/src/mongo/db/exec/requires_collection_stage.cpp
+++ b/src/mongo/db/exec/requires_collection_stage.cpp
@@ -69,8 +69,8 @@ void RequiresCollectionStage::doRestoreState(const RestoreContext& context) {
// If we didn't get a valid collection but can still find the UUID in the catalog then we
// treat this as a rename.
if (!coll) {
- const CollectionCatalog& catalog = CollectionCatalog::get(opCtx());
- auto newNss = catalog.lookupNSSByUUID(opCtx(), _collectionUUID);
+ auto catalog = CollectionCatalog::get(opCtx());
+ auto newNss = catalog->lookupNSSByUUID(opCtx(), _collectionUUID);
if (newNss && *newNss != _nss) {
collectionRenamed(*newNss);
}
diff --git a/src/mongo/db/exec/requires_collection_stage.h b/src/mongo/db/exec/requires_collection_stage.h
index 4e3484abf53..a820fcf4aa6 100644
--- a/src/mongo/db/exec/requires_collection_stage.h
+++ b/src/mongo/db/exec/requires_collection_stage.h
@@ -86,7 +86,7 @@ protected:
private:
// This can only be called when the plan stage is attached to an operation context.
uint64_t getCatalogEpoch() const {
- return CollectionCatalog::get(opCtx()).getEpoch();
+ return CollectionCatalog::get(opCtx())->getEpoch();
}
// Pointer to a CollectionPtr that is stored at a high level in a AutoGetCollection or other
diff --git a/src/mongo/db/free_mon/free_mon_mongod.cpp b/src/mongo/db/free_mon/free_mon_mongod.cpp
index 2a76481037d..327898b392a 100644
--- a/src/mongo/db/free_mon/free_mon_mongod.cpp
+++ b/src/mongo/db/free_mon/free_mon_mongod.cpp
@@ -232,9 +232,9 @@ public:
}
void collect(OperationContext* opCtx, BSONObjBuilder& builder) {
- auto& catalog = CollectionCatalog::get(opCtx);
+ auto catalog = CollectionCatalog::get(opCtx);
for (auto& nss : _namespaces) {
- auto optUUID = catalog.lookupUUIDByNSS(opCtx, nss);
+ auto optUUID = catalog->lookupUUIDByNSS(opCtx, nss);
if (optUUID) {
builder << nss.toString() << optUUID.get();
}
diff --git a/src/mongo/db/index_build_entry_helpers.cpp b/src/mongo/db/index_build_entry_helpers.cpp
index 6b4265e8552..2ca9fb4f7aa 100644
--- a/src/mongo/db/index_build_entry_helpers.cpp
+++ b/src/mongo/db/index_build_entry_helpers.cpp
@@ -177,7 +177,7 @@ void ensureIndexBuildEntriesNamespaceExists(OperationContext* opCtx) {
invariant(db);
// Create the collection if it doesn't exist.
- if (!CollectionCatalog::get(opCtx).lookupCollectionByNamespace(
+ if (!CollectionCatalog::get(opCtx)->lookupCollectionByNamespace(
opCtx, NamespaceString::kIndexBuildEntryNamespace)) {
WriteUnitOfWork wuow(opCtx);
CollectionOptions defaultCollectionOptions;
diff --git a/src/mongo/db/index_builds_coordinator.cpp b/src/mongo/db/index_builds_coordinator.cpp
index b4d692200cd..72e37ca7def 100644
--- a/src/mongo/db/index_builds_coordinator.cpp
+++ b/src/mongo/db/index_builds_coordinator.cpp
@@ -803,8 +803,8 @@ void IndexBuildsCoordinator::abortAllIndexBuildsForInitialSync(OperationContext*
namespace {
NamespaceString getNsFromUUID(OperationContext* opCtx, const UUID& uuid) {
- auto& catalog = CollectionCatalog::get(opCtx);
- auto nss = catalog.lookupNSSByUUID(opCtx, uuid);
+ auto catalog = CollectionCatalog::get(opCtx);
+ auto nss = catalog->lookupNSSByUUID(opCtx, uuid);
uassert(ErrorCodes::NamespaceNotFound, "No namespace with UUID " + uuid.toString(), nss);
return *nss;
}
@@ -1347,6 +1347,7 @@ void IndexBuildsCoordinator::restartIndexBuildsForRecovery(
OperationContext* opCtx,
const IndexBuilds& buildsToRestart,
const std::vector<ResumeIndexInfo>& buildsToResume) {
+ auto catalog = CollectionCatalog::get(opCtx);
stdx::unordered_set<UUID, UUID::Hash> successfullyResumed;
@@ -1355,7 +1356,7 @@ void IndexBuildsCoordinator::restartIndexBuildsForRecovery(
auto collUUID = resumeInfo.getCollectionUUID();
boost::optional<NamespaceString> nss =
- CollectionCatalog::get(opCtx).lookupNSSByUUID(opCtx, resumeInfo.getCollectionUUID());
+ catalog->lookupNSSByUUID(opCtx, resumeInfo.getCollectionUUID());
invariant(nss);
std::vector<BSONObj> indexSpecs;
@@ -1418,8 +1419,7 @@ void IndexBuildsCoordinator::restartIndexBuildsForRecovery(
continue;
}
- boost::optional<NamespaceString> nss =
- CollectionCatalog::get(opCtx).lookupNSSByUUID(opCtx, build.collUUID);
+ boost::optional<NamespaceString> nss = catalog->lookupNSSByUUID(opCtx, build.collUUID);
invariant(nss);
LOGV2(20660,
@@ -1657,7 +1657,7 @@ Status IndexBuildsCoordinator::_setUpIndexBuildForTwoPhaseRecovery(
// case when an index builds is restarted during recovery.
Lock::DBLock dbLock(opCtx, dbName, MODE_IX);
Lock::CollectionLock collLock(opCtx, nssOrUuid, MODE_X);
- auto collection = CollectionCatalog::get(opCtx).lookupCollectionByUUID(opCtx, collectionUUID);
+ auto collection = CollectionCatalog::get(opCtx)->lookupCollectionByUUID(opCtx, collectionUUID);
invariant(collection);
const auto& nss = collection->ns();
const auto protocol = IndexBuildProtocol::kTwoPhase;
@@ -2092,7 +2092,7 @@ void IndexBuildsCoordinator::_runIndexBuildInner(
// dropped while the index build is still registered for the collection -- until abortIndexBuild
// is called. The collection can be renamed, but it is OK for the name to be stale just for
// logging purposes.
- auto collectionSharedPtr = CollectionCatalog::get(opCtx).lookupCollectionByUUIDForRead(
+ auto collectionSharedPtr = CollectionCatalog::get(opCtx)->lookupCollectionByUUIDForRead(
opCtx, replState->collectionUUID);
CollectionPtr collection(collectionSharedPtr.get(), CollectionPtr::NoYieldTag{});
invariant(collection,
@@ -2321,7 +2321,7 @@ CollectionPtr IndexBuildsCoordinator::_setUpForScanCollectionAndInsertSortedKeys
invariant(_indexBuildsManager.isBackgroundBuilding(replState->buildUUID));
auto collection =
- CollectionCatalog::get(opCtx).lookupCollectionByUUID(opCtx, replState->collectionUUID);
+ CollectionCatalog::get(opCtx)->lookupCollectionByUUID(opCtx, replState->collectionUUID);
invariant(collection);
// Set up the thread's currentOp information to display createIndexes cmd information.
diff --git a/src/mongo/db/index_builds_coordinator_mongod.cpp b/src/mongo/db/index_builds_coordinator_mongod.cpp
index a44ccca9686..e3e4a47991b 100644
--- a/src/mongo/db/index_builds_coordinator_mongod.cpp
+++ b/src/mongo/db/index_builds_coordinator_mongod.cpp
@@ -243,7 +243,7 @@ IndexBuildsCoordinatorMongod::_startIndexBuild(OperationContext* opCtx,
invariant(!opCtx->lockState()->isRSTLExclusive(), buildUUID.toString());
- const auto nss = CollectionCatalog::get(opCtx).resolveNamespaceStringOrUUID(opCtx, nssOrUuid);
+ const auto nss = CollectionCatalog::get(opCtx)->resolveNamespaceStringOrUUID(opCtx, nssOrUuid);
auto& oss = OperationShardingState::get(opCtx);
const auto shardVersion = oss.getShardVersion(nss);
diff --git a/src/mongo/db/introspect.cpp b/src/mongo/db/introspect.cpp
index 12dc4c200f1..29b67e4c2eb 100644
--- a/src/mongo/db/introspect.cpp
+++ b/src/mongo/db/introspect.cpp
@@ -136,7 +136,8 @@ void profile(OperationContext* opCtx, NetworkOp op) {
EnforcePrepareConflictsBlock enforcePrepare(opCtx);
uassertStatusOK(createProfileCollection(opCtx, db));
- auto coll = CollectionCatalog::get(opCtx).lookupCollectionByNamespace(opCtx, dbProfilingNS);
+ auto coll =
+ CollectionCatalog::get(opCtx)->lookupCollectionByNamespace(opCtx, dbProfilingNS);
invariant(!opCtx->shouldParticipateInFlowControl());
WriteUnitOfWork wuow(opCtx);
@@ -166,7 +167,7 @@ Status createProfileCollection(OperationContext* opCtx, Database* db) {
// and see the collection exists in order to break free.
return writeConflictRetry(opCtx, "createProfileCollection", dbProfilingNS.ns(), [&] {
const CollectionPtr collection =
- CollectionCatalog::get(opCtx).lookupCollectionByNamespace(opCtx, dbProfilingNS);
+ CollectionCatalog::get(opCtx)->lookupCollectionByNamespace(opCtx, dbProfilingNS);
if (collection) {
if (!collection->isCapped()) {
return Status(ErrorCodes::NamespaceExists,
diff --git a/src/mongo/db/matcher/expression_text.cpp b/src/mongo/db/matcher/expression_text.cpp
index 3f72bafb102..5dcebcd40e5 100644
--- a/src/mongo/db/matcher/expression_text.cpp
+++ b/src/mongo/db/matcher/expression_text.cpp
@@ -67,7 +67,7 @@ TextMatchExpression::TextMatchExpression(OperationContext* opCtx,
db);
CollectionPtr collection =
- CollectionCatalog::get(opCtx).lookupCollectionByNamespace(opCtx, nss);
+ CollectionCatalog::get(opCtx)->lookupCollectionByNamespace(opCtx, nss);
uassert(ErrorCodes::IndexNotFound,
str::stream() << "text index required for $text query (no such collection '"
diff --git a/src/mongo/db/mongod_main.cpp b/src/mongo/db/mongod_main.cpp
index 09863d97363..05c9ed67511 100644
--- a/src/mongo/db/mongod_main.cpp
+++ b/src/mongo/db/mongod_main.cpp
@@ -261,7 +261,7 @@ void logStartup(OperationContext* opCtx) {
AutoGetOrCreateDb autoDb(opCtx, startupLogCollectionName.db(), mongo::MODE_X);
Database* db = autoDb.getDb();
CollectionPtr collection =
- CollectionCatalog::get(opCtx).lookupCollectionByNamespace(opCtx, startupLogCollectionName);
+ CollectionCatalog::get(opCtx)->lookupCollectionByNamespace(opCtx, startupLogCollectionName);
WriteUnitOfWork wunit(opCtx);
if (!collection) {
BSONObj options = BSON("capped" << true << "size" << 10 * 1024 * 1024);
@@ -269,7 +269,7 @@ void logStartup(OperationContext* opCtx) {
CollectionOptions collectionOptions = uassertStatusOK(
CollectionOptions::parse(options, CollectionOptions::ParseKind::parseForCommand));
uassertStatusOK(db->userCreateNS(opCtx, startupLogCollectionName, collectionOptions));
- collection = CollectionCatalog::get(opCtx).lookupCollectionByNamespace(
+ collection = CollectionCatalog::get(opCtx)->lookupCollectionByNamespace(
opCtx, startupLogCollectionName);
}
invariant(collection);
diff --git a/src/mongo/db/op_observer_impl.cpp b/src/mongo/db/op_observer_impl.cpp
index 992606858bd..4d2feccb796 100644
--- a/src/mongo/db/op_observer_impl.cpp
+++ b/src/mongo/db/op_observer_impl.cpp
@@ -757,7 +757,7 @@ void OpObserverImpl::onCollMod(OperationContext* opCtx,
return;
}
const CollectionPtr& coll =
- CollectionCatalog::get(opCtx).lookupCollectionByNamespace(opCtx, nss);
+ CollectionCatalog::get(opCtx)->lookupCollectionByNamespace(opCtx, nss);
invariant(coll->uuid() == uuid);
invariant(DurableCatalog::get(opCtx)->isEqualToMetadataUUID(opCtx, coll->getCatalogId(), uuid));
diff --git a/src/mongo/db/ops/update.cpp b/src/mongo/db/ops/update.cpp
index 455afeadbf9..afc5a59a88e 100644
--- a/src/mongo/db/ops/update.cpp
+++ b/src/mongo/db/ops/update.cpp
@@ -67,7 +67,7 @@ UpdateResult update(OperationContext* opCtx, Database* db, const UpdateRequest&
// The update stage does not create its own collection. As such, if the update is
// an upsert, create the collection that the update stage inserts into beforehand.
writeConflictRetry(opCtx, "createCollection", nsString.ns(), [&] {
- collection = CollectionCatalog::get(opCtx).lookupCollectionByNamespace(opCtx, nsString);
+ collection = CollectionCatalog::get(opCtx)->lookupCollectionByNamespace(opCtx, nsString);
if (collection || !request.isUpsert()) {
return;
}
diff --git a/src/mongo/db/ops/write_ops_exec.cpp b/src/mongo/db/ops/write_ops_exec.cpp
index 7083b5caed9..21960215359 100644
--- a/src/mongo/db/ops/write_ops_exec.cpp
+++ b/src/mongo/db/ops/write_ops_exec.cpp
@@ -218,7 +218,7 @@ void makeCollection(OperationContext* opCtx, const NamespaceString& ns) {
Lock::CollectionLock collLock(opCtx, ns, MODE_IX);
assertCanWrite_inlock(opCtx, ns);
- if (!CollectionCatalog::get(opCtx).lookupCollectionByNamespace(
+ if (!CollectionCatalog::get(opCtx)->lookupCollectionByNamespace(
opCtx,
ns)) { // someone else may have beat us to it.
uassertStatusOK(userAllowedCreateNS(ns));
@@ -401,7 +401,7 @@ bool insertBatchAndHandleErrors(OperationContext* opCtx,
}
curOp.raiseDbProfileLevel(
- CollectionCatalog::get(opCtx).getDatabaseProfileLevel(wholeOp.getNamespace().db()));
+ CollectionCatalog::get(opCtx)->getDatabaseProfileLevel(wholeOp.getNamespace().db()));
assertCanWrite_inlock(opCtx, wholeOp.getNamespace());
CurOpFailpointHelpers::waitWhileFailPointEnabled(
@@ -667,7 +667,7 @@ static SingleWriteResult performSingleUpdateOp(OperationContext* opCtx,
auto& curOp = *CurOp::get(opCtx);
if (collection->getDb()) {
- curOp.raiseDbProfileLevel(CollectionCatalog::get(opCtx).getDatabaseProfileLevel(ns.db()));
+ curOp.raiseDbProfileLevel(CollectionCatalog::get(opCtx)->getDatabaseProfileLevel(ns.db()));
}
assertCanWrite_inlock(opCtx, ns);
@@ -906,7 +906,7 @@ static SingleWriteResult performSingleDeleteOp(OperationContext* opCtx,
AutoGetCollection collection(opCtx, ns, fixLockModeForSystemDotViewsChanges(ns, MODE_IX));
if (collection.getDb()) {
- curOp.raiseDbProfileLevel(CollectionCatalog::get(opCtx).getDatabaseProfileLevel(ns.db()));
+ curOp.raiseDbProfileLevel(CollectionCatalog::get(opCtx)->getDatabaseProfileLevel(ns.db()));
}
assertCanWrite_inlock(opCtx, ns);
diff --git a/src/mongo/db/pipeline/document_source_change_stream_test.cpp b/src/mongo/db/pipeline/document_source_change_stream_test.cpp
index f8a88034efe..89c207471a0 100644
--- a/src/mongo/db/pipeline/document_source_change_stream_test.cpp
+++ b/src/mongo/db/pipeline/document_source_change_stream_test.cpp
@@ -479,7 +479,9 @@ TEST_F(ChangeStreamStageTest, ShouldRejectBothStartAtOperationTimeAndResumeAfter
// Need to put the collection in the collection catalog so the resume token is valid.
std::shared_ptr<Collection> collection = std::make_shared<CollectionMock>(nss);
- CollectionCatalog::get(expCtx->opCtx).registerCollection(testUuid(), std::move(collection));
+ CollectionCatalog::write(expCtx->opCtx, [&](CollectionCatalog& catalog) {
+ catalog.registerCollection(testUuid(), std::move(collection));
+ });
ASSERT_THROWS_CODE(
DSChangeStream::createFromBson(
@@ -499,8 +501,9 @@ TEST_F(ChangeStreamStageTest, ShouldRejectBothStartAfterAndResumeAfterOptions) {
// Need to put the collection in the collection catalog so the resume token is validcollection
std::shared_ptr<Collection> collection = std::make_shared<CollectionMock>(nss);
- auto& catalog = CollectionCatalog::get(opCtx);
- catalog.registerCollection(testUuid(), std::move(collection));
+ CollectionCatalog::write(opCtx, [&](CollectionCatalog& catalog) {
+ catalog.registerCollection(testUuid(), std::move(collection));
+ });
ASSERT_THROWS_CODE(
DSChangeStream::createFromBson(
@@ -521,8 +524,9 @@ TEST_F(ChangeStreamStageTest, ShouldRejectBothStartAtOperationTimeAndStartAfterO
// Need to put the collection in the collection catalog so the resume token is valid.
std::shared_ptr<Collection> collection = std::make_shared<CollectionMock>(nss);
- auto& catalog = CollectionCatalog::get(opCtx);
- catalog.registerCollection(testUuid(), std::move(collection));
+ CollectionCatalog::write(opCtx, [&](CollectionCatalog& catalog) {
+ catalog.registerCollection(testUuid(), std::move(collection));
+ });
ASSERT_THROWS_CODE(
DSChangeStream::createFromBson(
@@ -542,8 +546,9 @@ TEST_F(ChangeStreamStageTest, ShouldRejectResumeAfterWithResumeTokenMissingUUID)
// Need to put the collection in the collection catalog so the resume token is valid.
std::shared_ptr<Collection> collection = std::make_shared<CollectionMock>(nss);
- auto& catalog = CollectionCatalog::get(opCtx);
- catalog.registerCollection(testUuid(), std::move(collection));
+ CollectionCatalog::write(opCtx, [&](CollectionCatalog& catalog) {
+ catalog.registerCollection(testUuid(), std::move(collection));
+ });
ASSERT_THROWS_CODE(
DSChangeStream::createFromBson(
@@ -1704,7 +1709,10 @@ TEST_F(ChangeStreamStageTest, DocumentKeyShouldIncludeShardKeyFromResumeToken) {
const auto uuid = testUuid();
std::shared_ptr<Collection> collection = std::make_shared<CollectionMock>(nss);
- CollectionCatalog::get(getExpCtx()->opCtx).registerCollection(uuid, std::move(collection));
+ CollectionCatalog::write(getExpCtx()->opCtx, [&](CollectionCatalog& catalog) {
+ catalog.registerCollection(uuid, std::move(collection));
+ });
+
BSONObj o2 = BSON("_id" << 1 << "shardKey" << 2);
auto resumeToken = makeResumeToken(ts, uuid, o2);
@@ -1749,7 +1757,9 @@ TEST_F(ChangeStreamStageTest, DocumentKeyShouldNotIncludeShardKeyFieldsIfNotPres
const auto uuid = testUuid();
std::shared_ptr<Collection> collection = std::make_shared<CollectionMock>(nss);
- CollectionCatalog::get(getExpCtx()->opCtx).registerCollection(uuid, std::move(collection));
+ CollectionCatalog::write(getExpCtx()->opCtx, [&](CollectionCatalog& catalog) {
+ catalog.registerCollection(uuid, std::move(collection));
+ });
BSONObj o2 = BSON("_id" << 1 << "shardKey" << 2);
auto resumeToken = makeResumeToken(ts, uuid, o2);
@@ -1791,7 +1801,9 @@ TEST_F(ChangeStreamStageTest, ResumeAfterFailsIfResumeTokenDoesNotContainUUID) {
const auto uuid = testUuid();
std::shared_ptr<Collection> collection = std::make_shared<CollectionMock>(nss);
- CollectionCatalog::get(getExpCtx()->opCtx).registerCollection(uuid, std::move(collection));
+ CollectionCatalog::write(getExpCtx()->opCtx, [&](CollectionCatalog& catalog) {
+ catalog.registerCollection(uuid, std::move(collection));
+ });
// Create a resume token from only the timestamp.
auto resumeToken = makeResumeToken(ts);
@@ -1844,7 +1856,9 @@ TEST_F(ChangeStreamStageTest, ResumeAfterWithTokenFromInvalidateShouldFail) {
// Need to put the collection in the collection catalog so the resume token is valid.
std::shared_ptr<Collection> collection = std::make_shared<CollectionMock>(nss);
- CollectionCatalog::get(expCtx->opCtx).registerCollection(testUuid(), std::move(collection));
+ CollectionCatalog::write(expCtx->opCtx, [&](CollectionCatalog& catalog) {
+ catalog.registerCollection(testUuid(), std::move(collection));
+ });
const auto resumeTokenInvalidate =
makeResumeToken(kDefaultTs,
@@ -2504,7 +2518,9 @@ TEST_F(ChangeStreamStageDBTest, DocumentKeyShouldIncludeShardKeyFromResumeToken)
const auto uuid = testUuid();
std::shared_ptr<Collection> collection = std::make_shared<CollectionMock>(nss);
- CollectionCatalog::get(getExpCtx()->opCtx).registerCollection(uuid, std::move(collection));
+ CollectionCatalog::write(getExpCtx()->opCtx, [&](CollectionCatalog& catalog) {
+ catalog.registerCollection(uuid, std::move(collection));
+ });
BSONObj o2 = BSON("_id" << 1 << "shardKey" << 2);
auto resumeToken = makeResumeToken(ts, uuid, o2);
@@ -2540,7 +2556,9 @@ TEST_F(ChangeStreamStageDBTest, DocumentKeyShouldNotIncludeShardKeyFieldsIfNotPr
const auto uuid = testUuid();
std::shared_ptr<Collection> collection = std::make_shared<CollectionMock>(nss);
- CollectionCatalog::get(getExpCtx()->opCtx).registerCollection(uuid, std::move(collection));
+ CollectionCatalog::write(getExpCtx()->opCtx, [&](CollectionCatalog& catalog) {
+ catalog.registerCollection(uuid, std::move(collection));
+ });
BSONObj o2 = BSON("_id" << 1 << "shardKey" << 2);
auto resumeToken = makeResumeToken(ts, uuid, o2);
@@ -2577,7 +2595,9 @@ TEST_F(ChangeStreamStageDBTest, DocumentKeyShouldNotIncludeShardKeyIfResumeToken
const auto uuid = testUuid();
std::shared_ptr<Collection> collection = std::make_shared<CollectionMock>(nss);
- CollectionCatalog::get(getExpCtx()->opCtx).registerCollection(uuid, std::move(collection));
+ CollectionCatalog::write(getExpCtx()->opCtx, [&](CollectionCatalog& catalog) {
+ catalog.registerCollection(uuid, std::move(collection));
+ });
// Create a resume token from only the timestamp.
auto resumeToken = makeResumeToken(ts);
@@ -2613,7 +2633,9 @@ TEST_F(ChangeStreamStageDBTest, ResumeAfterWithTokenFromInvalidateShouldFail) {
// Need to put the collection in the collection catalog so the resume token is valid.
std::shared_ptr<Collection> collection = std::make_shared<CollectionMock>(nss);
- CollectionCatalog::get(expCtx->opCtx).registerCollection(testUuid(), std::move(collection));
+ CollectionCatalog::write(expCtx->opCtx, [&](CollectionCatalog& catalog) {
+ catalog.registerCollection(testUuid(), std::move(collection));
+ });
const auto resumeTokenInvalidate =
makeResumeToken(kDefaultTs,
@@ -2634,7 +2656,9 @@ TEST_F(ChangeStreamStageDBTest, ResumeAfterWithTokenFromDropDatabase) {
const auto uuid = testUuid();
std::shared_ptr<Collection> collection = std::make_shared<CollectionMock>(nss);
- CollectionCatalog::get(getExpCtx()->opCtx).registerCollection(uuid, std::move(collection));
+ CollectionCatalog::write(getExpCtx()->opCtx, [&](CollectionCatalog& catalog) {
+ catalog.registerCollection(uuid, std::move(collection));
+ });
// Create a resume token from only the timestamp, similar to a 'dropDatabase' entry.
auto resumeToken = makeResumeToken(
@@ -2663,7 +2687,9 @@ TEST_F(ChangeStreamStageDBTest, StartAfterSucceedsEvenIfResumeTokenDoesNotContai
const auto uuid = testUuid();
std::shared_ptr<Collection> collection = std::make_shared<CollectionMock>(nss);
- CollectionCatalog::get(getExpCtx()->opCtx).registerCollection(uuid, std::move(collection));
+ CollectionCatalog::write(getExpCtx()->opCtx, [&](CollectionCatalog& catalog) {
+ catalog.registerCollection(uuid, std::move(collection));
+ });
// Create a resume token from only the timestamp, similar to a 'dropDatabase' entry.
auto resumeToken = makeResumeToken(kDefaultTs);
diff --git a/src/mongo/db/pipeline/document_source_cursor.cpp b/src/mongo/db/pipeline/document_source_cursor.cpp
index 2bfc196c45c..341320c9a64 100644
--- a/src/mongo/db/pipeline/document_source_cursor.cpp
+++ b/src/mongo/db/pipeline/document_source_cursor.cpp
@@ -219,7 +219,7 @@ Value DocumentSourceCursor::serialize(boost::optional<ExplainOptions::Verbosity>
AutoGetDb dbLock(opCtx, _exec->nss().db(), lockMode);
Lock::CollectionLock collLock(opCtx, _exec->nss(), lockMode);
auto collection = dbLock.getDb()
- ? CollectionCatalog::get(opCtx).lookupCollectionByNamespace(opCtx, _exec->nss())
+ ? CollectionCatalog::get(opCtx)->lookupCollectionByNamespace(opCtx, _exec->nss())
: nullptr;
Explain::explainStages(_exec.get(),
diff --git a/src/mongo/db/pipeline/process_interface/common_mongod_process_interface.cpp b/src/mongo/db/pipeline/process_interface/common_mongod_process_interface.cpp
index c3db998e1d7..be933e699f8 100644
--- a/src/mongo/db/pipeline/process_interface/common_mongod_process_interface.cpp
+++ b/src/mongo/db/pipeline/process_interface/common_mongod_process_interface.cpp
@@ -440,7 +440,7 @@ bool CommonMongodProcessInterface::fieldsHaveSupportingUniqueIndex(
auto databaseHolder = DatabaseHolder::get(opCtx);
auto db = databaseHolder->getDb(opCtx, nss.db());
auto collection =
- db ? CollectionCatalog::get(opCtx).lookupCollectionByNamespace(opCtx, nss) : nullptr;
+ db ? CollectionCatalog::get(opCtx)->lookupCollectionByNamespace(opCtx, nss) : nullptr;
if (!collection) {
return fieldPaths == std::set<FieldPath>{"_id"};
}
diff --git a/src/mongo/db/query/find.cpp b/src/mongo/db/query/find.cpp
index 36bf3c974ed..bb6115b03df 100644
--- a/src/mongo/db/query/find.cpp
+++ b/src/mongo/db/query/find.cpp
@@ -274,7 +274,7 @@ Message getMore(OperationContext* opCtx,
nss,
Top::LockType::NotLocked,
AutoStatsTracker::LogMode::kUpdateTopAndCurOp,
- CollectionCatalog::get(opCtx).getDatabaseProfileLevel(nss.db()));
+ CollectionCatalog::get(opCtx)->getDatabaseProfileLevel(nss.db()));
auto view = autoDb.getDb() ? ViewCatalog::get(autoDb.getDb())->lookup(opCtx, nss.ns())
: nullptr;
uassert(
@@ -291,7 +291,7 @@ Message getMore(OperationContext* opCtx,
nss,
Top::LockType::ReadLocked,
AutoStatsTracker::LogMode::kUpdateTopAndCurOp,
- CollectionCatalog::get(opCtx).getDatabaseProfileLevel(nss.db()));
+ CollectionCatalog::get(opCtx)->getDatabaseProfileLevel(nss.db()));
// This checks to make sure the operation is allowed on a replicated node. Since we are not
// passing in a query object (necessary to check SlaveOK query option), we allow reads
diff --git a/src/mongo/db/query/plan_insert_listener.cpp b/src/mongo/db/query/plan_insert_listener.cpp
index bda996c605a..b1140df2641 100644
--- a/src/mongo/db/query/plan_insert_listener.cpp
+++ b/src/mongo/db/query/plan_insert_listener.cpp
@@ -81,7 +81,7 @@ std::shared_ptr<CappedInsertNotifier> getCappedInsertNotifier(OperationContext*
// We can only wait if we have a collection; otherwise we should retry immediately when
// we hit EOF.
- auto collection = CollectionCatalog::get(opCtx).lookupCollectionByNamespaceForRead(opCtx, nss);
+ auto collection = CollectionCatalog::get(opCtx)->lookupCollectionByNamespaceForRead(opCtx, nss);
invariant(collection);
return collection->getCappedInsertNotifier();
diff --git a/src/mongo/db/repair.cpp b/src/mongo/db/repair.cpp
index 44bba37e1a0..efc23ea847b 100644
--- a/src/mongo/db/repair.cpp
+++ b/src/mongo/db/repair.cpp
@@ -72,7 +72,7 @@ Status rebuildIndexesForNamespace(OperationContext* opCtx,
const NamespaceString& nss,
StorageEngine* engine) {
opCtx->checkForInterrupt();
- auto collection = CollectionCatalog::get(opCtx).lookupCollectionByNamespace(opCtx, nss);
+ auto collection = CollectionCatalog::get(opCtx)->lookupCollectionByNamespace(opCtx, nss);
auto swIndexNameObjs = getIndexNameObjs(opCtx, collection->getCatalogId());
if (!swIndexNameObjs.isOK())
return swIndexNameObjs.getStatus();
@@ -122,7 +122,7 @@ Status dropUnfinishedIndexes(OperationContext* opCtx, const CollectionPtr& colle
Status repairCollections(OperationContext* opCtx,
StorageEngine* engine,
const std::string& dbName) {
- auto colls = CollectionCatalog::get(opCtx).getAllCollectionNamesFromDb(opCtx, dbName);
+ auto colls = CollectionCatalog::get(opCtx)->getAllCollectionNamesFromDb(opCtx, dbName);
for (const auto& nss : colls) {
auto status = repair::repairCollection(opCtx, engine, nss);
@@ -191,14 +191,14 @@ Status repairCollection(OperationContext* opCtx,
Status status = Status::OK();
{
- auto collection = CollectionCatalog::get(opCtx).lookupCollectionByNamespace(opCtx, nss);
+ auto collection = CollectionCatalog::get(opCtx)->lookupCollectionByNamespace(opCtx, nss);
status = engine->repairRecordStore(opCtx, collection->getCatalogId(), nss);
}
// Need to lookup from catalog again because the old collection object was invalidated by
// repairRecordStore.
- auto collection = CollectionCatalog::get(opCtx).lookupCollectionByNamespaceForMetadataWrite(
+ auto collection = CollectionCatalog::get(opCtx)->lookupCollectionByNamespaceForMetadataWrite(
opCtx, CollectionCatalog::LifetimeMode::kInplace, nss);
// If data was modified during repairRecordStore, we know to rebuild indexes without needing
diff --git a/src/mongo/db/repl/apply_ops.cpp b/src/mongo/db/repl/apply_ops.cpp
index 8643f55ffff..fbf15b2d70d 100644
--- a/src/mongo/db/repl/apply_ops.cpp
+++ b/src/mongo/db/repl/apply_ops.cpp
@@ -150,7 +150,8 @@ Status _applyOps(OperationContext* opCtx,
// NamespaceNotFound.
// Additionally for inserts, we fail early on non-existent collections.
Lock::CollectionLock collectionLock(opCtx, nss, MODE_IX);
- auto collection = CollectionCatalog::get(opCtx).lookupCollectionByNamespace(opCtx, nss);
+ auto collection =
+ CollectionCatalog::get(opCtx)->lookupCollectionByNamespace(opCtx, nss);
if (!collection && (*opType == 'i' || *opType == 'u')) {
uasserted(
ErrorCodes::AtomicityFailure,
@@ -316,7 +317,7 @@ Status _checkPrecondition(OperationContext* opCtx,
return {ErrorCodes::NamespaceNotFound, "database in ns does not exist: " + nss.ns()};
}
CollectionPtr collection =
- CollectionCatalog::get(opCtx).lookupCollectionByNamespace(opCtx, nss);
+ CollectionCatalog::get(opCtx)->lookupCollectionByNamespace(opCtx, nss);
if (!collection) {
return {ErrorCodes::NamespaceNotFound, "collection in ns does not exist: " + nss.ns()};
}
diff --git a/src/mongo/db/repl/dbcheck.cpp b/src/mongo/db/repl/dbcheck.cpp
index 6dfae59d3ba..cd46bf05173 100644
--- a/src/mongo/db/repl/dbcheck.cpp
+++ b/src/mongo/db/repl/dbcheck.cpp
@@ -234,11 +234,11 @@ std::string hashCollectionInfo(const DbCheckCollectionInformation& info) {
std::pair<boost::optional<UUID>, boost::optional<UUID>> getPrevAndNextUUIDs(
OperationContext* opCtx, const CollectionPtr& collection) {
- const CollectionCatalog& catalog = CollectionCatalog::get(opCtx);
+ auto catalog = CollectionCatalog::get(opCtx);
const UUID uuid = collection->uuid();
std::vector<CollectionUUID> collectionUUIDs =
- catalog.getAllCollectionUUIDsFromDb(collection->ns().db());
+ catalog->getAllCollectionUUIDsFromDb(collection->ns().db());
auto uuidIt = std::find(collectionUUIDs.begin(), collectionUUIDs.end(), uuid);
invariant(uuidIt != collectionUUIDs.end());
@@ -386,7 +386,7 @@ AutoGetCollectionForDbCheck::AutoGetCollectionForDbCheck(OperationContext* opCtx
std::string msg;
_collection = _agd.getDb()
- ? CollectionCatalog::get(opCtx).lookupCollectionByNamespace(opCtx, nss)
+ ? CollectionCatalog::get(opCtx)->lookupCollectionByNamespace(opCtx, nss)
: nullptr;
// If the collection gets deleted after the check is launched, record that in the health log.
@@ -461,7 +461,7 @@ Status dbCheckDatabaseOnSecondary(OperationContext* opCtx,
const DbCheckOplogCollection& entry) {
// dbCheckCollectionResult-specific stuff.
auto uuid = uassertStatusOK(UUID::parse(entry.getUuid().toString()));
- auto collection = CollectionCatalog::get(opCtx).lookupCollectionByUUID(opCtx, uuid);
+ auto collection = CollectionCatalog::get(opCtx)->lookupCollectionByUUID(opCtx, uuid);
if (!collection) {
Status status(ErrorCodes::NamespaceNotFound, "Could not find collection for dbCheck");
diff --git a/src/mongo/db/repl/idempotency_test_fixture.cpp b/src/mongo/db/repl/idempotency_test_fixture.cpp
index 7207fc8aaaf..6590ccbc031 100644
--- a/src/mongo/db/repl/idempotency_test_fixture.cpp
+++ b/src/mongo/db/repl/idempotency_test_fixture.cpp
@@ -354,15 +354,15 @@ std::string IdempotencyTest::computeDataHash(const CollectionPtr& collection) {
std::vector<CollectionState> IdempotencyTest::validateAllCollections() {
std::vector<CollectionState> collStates;
- auto& catalog = CollectionCatalog::get(_opCtx.get());
- auto dbs = catalog.getAllDbNames();
+ auto catalog = CollectionCatalog::get(_opCtx.get());
+ auto dbs = catalog->getAllDbNames();
for (auto& db : dbs) {
// Skip local database.
if (db != "local") {
std::vector<NamespaceString> collectionNames;
{
Lock::DBLock lk(_opCtx.get(), db, MODE_S);
- collectionNames = catalog.getAllCollectionNamesFromDb(_opCtx.get(), db);
+ collectionNames = catalog->getAllCollectionNamesFromDb(_opCtx.get(), db);
}
for (const auto& nss : collectionNames) {
collStates.push_back(validate(nss));
diff --git a/src/mongo/db/repl/oplog.cpp b/src/mongo/db/repl/oplog.cpp
index 21e05660a16..32ce306829b 100644
--- a/src/mongo/db/repl/oplog.cpp
+++ b/src/mongo/db/repl/oplog.cpp
@@ -136,7 +136,7 @@ void abortIndexBuilds(OperationContext* opCtx,
commandType == OplogEntry::CommandType::kDropIndexes ||
commandType == OplogEntry::CommandType::kRenameCollection) {
const boost::optional<UUID> collUUID =
- CollectionCatalog::get(opCtx).lookupUUIDByNSS(opCtx, nss);
+ CollectionCatalog::get(opCtx)->lookupUUIDByNSS(opCtx, nss);
invariant(collUUID);
indexBuildsCoordinator->abortCollectionIndexBuilds(opCtx, nss, *collUUID, reason);
@@ -182,7 +182,7 @@ void createIndexForApplyOps(OperationContext* opCtx,
auto databaseHolder = DatabaseHolder::get(opCtx);
auto db = databaseHolder->getDb(opCtx, indexNss.ns());
auto indexCollection =
- db ? CollectionCatalog::get(opCtx).lookupCollectionByNamespace(opCtx, indexNss) : nullptr;
+ db ? CollectionCatalog::get(opCtx)->lookupCollectionByNamespace(opCtx, indexNss) : nullptr;
uassert(ErrorCodes::NamespaceNotFound,
str::stream() << "Failed to create index due to missing collection: " << indexNss.ns(),
indexCollection);
@@ -584,7 +584,7 @@ void createOplog(OperationContext* opCtx,
OldClientContext ctx(opCtx, oplogCollectionName.ns());
CollectionPtr collection =
- CollectionCatalog::get(opCtx).lookupCollectionByNamespace(opCtx, oplogCollectionName);
+ CollectionCatalog::get(opCtx)->lookupCollectionByNamespace(opCtx, oplogCollectionName);
if (collection) {
if (replSettings.getOplogSizeBytes() != 0) {
@@ -671,8 +671,8 @@ std::pair<OptionalCollectionUUID, NamespaceString> extractCollModUUIDAndNss(
return std::pair<OptionalCollectionUUID, NamespaceString>(boost::none, extractNs(ns, cmd));
}
CollectionUUID uuid = ui.get();
- auto& catalog = CollectionCatalog::get(opCtx);
- const auto nsByUUID = catalog.lookupNSSByUUID(opCtx, uuid);
+ auto catalog = CollectionCatalog::get(opCtx);
+ const auto nsByUUID = catalog->lookupNSSByUUID(opCtx, uuid);
uassert(ErrorCodes::NamespaceNotFound,
str::stream() << "Failed to apply operation due to missing collection (" << uuid
<< "): " << redact(cmd.toString()),
@@ -681,8 +681,8 @@ std::pair<OptionalCollectionUUID, NamespaceString> extractCollModUUIDAndNss(
}
NamespaceString extractNsFromUUID(OperationContext* opCtx, const UUID& uuid) {
- auto& catalog = CollectionCatalog::get(opCtx);
- auto nss = catalog.lookupNSSByUUID(opCtx, uuid);
+ auto catalog = CollectionCatalog::get(opCtx);
+ auto nss = catalog->lookupNSSByUUID(opCtx, uuid);
uassert(ErrorCodes::NamespaceNotFound, "No namespace with UUID " + uuid.toString(), nss);
return *nss;
}
@@ -1046,8 +1046,8 @@ Status applyOperation_inlock(OperationContext* opCtx,
NamespaceString requestNss;
CollectionPtr collection = nullptr;
if (auto uuid = op.getUuid()) {
- CollectionCatalog& catalog = CollectionCatalog::get(opCtx);
- collection = catalog.lookupCollectionByUUID(opCtx, uuid.get());
+ auto catalog = CollectionCatalog::get(opCtx);
+ collection = catalog->lookupCollectionByUUID(opCtx, uuid.get());
uassert(ErrorCodes::NamespaceNotFound,
str::stream() << "Failed to apply operation due to missing collection ("
<< uuid.get() << "): " << redact(opOrGroupedInserts.toBSON()),
@@ -1059,7 +1059,7 @@ Status applyOperation_inlock(OperationContext* opCtx,
invariant(requestNss.coll().size());
dassert(opCtx->lockState()->isCollectionLockedForMode(requestNss, MODE_IX),
requestNss.ns());
- collection = CollectionCatalog::get(opCtx).lookupCollectionByNamespace(opCtx, requestNss);
+ collection = CollectionCatalog::get(opCtx)->lookupCollectionByNamespace(opCtx, requestNss);
}
BSONObj o = op.getObject();
@@ -1536,7 +1536,7 @@ Status applyCommand_inlock(OperationContext* opCtx,
Lock::DBLock lock(opCtx, nss.db(), MODE_IS);
auto databaseHolder = DatabaseHolder::get(opCtx);
auto db = databaseHolder->getDb(opCtx, nss.ns());
- if (db && !CollectionCatalog::get(opCtx).lookupCollectionByNamespace(opCtx, nss) &&
+ if (db && !CollectionCatalog::get(opCtx)->lookupCollectionByNamespace(opCtx, nss) &&
ViewCatalog::get(db)->lookup(opCtx, nss.ns())) {
return {ErrorCodes::CommandNotSupportedOnView,
str::stream() << "applyOps not supported on view:" << nss.ns()};
diff --git a/src/mongo/db/repl/oplog_applier_utils.cpp b/src/mongo/db/repl/oplog_applier_utils.cpp
index 0eebd195222..757eafa89be 100644
--- a/src/mongo/db/repl/oplog_applier_utils.cpp
+++ b/src/mongo/db/repl/oplog_applier_utils.cpp
@@ -63,7 +63,7 @@ CachedCollectionProperties::getCollectionPropertiesImpl(OperationContext* opCtx,
const NamespaceString& nss) {
CollectionProperties collProperties;
- auto collection = CollectionCatalog::get(opCtx).lookupCollectionByNamespace(opCtx, nss);
+ auto collection = CollectionCatalog::get(opCtx)->lookupCollectionByNamespace(opCtx, nss);
if (!collection) {
return collProperties;
@@ -169,8 +169,8 @@ NamespaceString OplogApplierUtils::parseUUIDOrNs(OperationContext* opCtx,
}
const auto& uuid = optionalUuid.get();
- auto& catalog = CollectionCatalog::get(opCtx);
- auto nss = catalog.lookupNSSByUUID(opCtx, uuid);
+ auto catalog = CollectionCatalog::get(opCtx);
+ auto nss = catalog->lookupNSSByUUID(opCtx, uuid);
uassert(ErrorCodes::NamespaceNotFound,
str::stream() << "No namespace with UUID " << uuid.toString(),
nss);
diff --git a/src/mongo/db/repl/replication_coordinator_impl.cpp b/src/mongo/db/repl/replication_coordinator_impl.cpp
index e6e14253d46..3780faa5d9c 100644
--- a/src/mongo/db/repl/replication_coordinator_impl.cpp
+++ b/src/mongo/db/repl/replication_coordinator_impl.cpp
@@ -2842,7 +2842,7 @@ bool ReplicationCoordinatorImpl::canAcceptWritesFor_UNSAFE(OperationContext* opC
} else {
auto uuid = nsOrUUID.uuid();
invariant(uuid, nsOrUUID.toString());
- if (auto ns = CollectionCatalog::get(opCtx).lookupNSSByUUID(opCtx, *uuid)) {
+ if (auto ns = CollectionCatalog::get(opCtx)->lookupNSSByUUID(opCtx, *uuid)) {
if (!ns->isSystemDotProfile()) {
return false;
}
diff --git a/src/mongo/db/repl/replication_recovery.cpp b/src/mongo/db/repl/replication_recovery.cpp
index 5673e10251a..b029e4fa53a 100644
--- a/src/mongo/db/repl/replication_recovery.cpp
+++ b/src/mongo/db/repl/replication_recovery.cpp
@@ -654,7 +654,7 @@ void ReplicationRecoveryImpl::_truncateOplogTo(OperationContext* opCtx,
AutoGetDb autoDb(opCtx, oplogNss.db(), MODE_IX);
Lock::CollectionLock oplogCollectionLoc(opCtx, oplogNss, MODE_X);
auto oplogCollection =
- CollectionCatalog::get(opCtx).lookupCollectionByNamespace(opCtx, oplogNss);
+ CollectionCatalog::get(opCtx)->lookupCollectionByNamespace(opCtx, oplogNss);
if (!oplogCollection) {
fassertFailedWithStatusNoTrace(
34418,
diff --git a/src/mongo/db/repl/rollback_impl.cpp b/src/mongo/db/repl/rollback_impl.cpp
index f72cf0a33e6..9bb33c3168b 100644
--- a/src/mongo/db/repl/rollback_impl.cpp
+++ b/src/mongo/db/repl/rollback_impl.cpp
@@ -564,10 +564,10 @@ void RollbackImpl::_runPhaseFromAbortToReconstructPreparedTxns(
void RollbackImpl::_correctRecordStoreCounts(OperationContext* opCtx) {
// This function explicitly does not check for shutdown since a clean shutdown post oplog
// truncation is not allowed to occur until the record store counts are corrected.
- const auto& catalog = CollectionCatalog::get(opCtx);
+ auto catalog = CollectionCatalog::get(opCtx);
for (const auto& uiCount : _newCounts) {
const auto uuid = uiCount.first;
- const auto coll = catalog.lookupCollectionByUUID(opCtx, uuid);
+ const auto coll = catalog->lookupCollectionByUUID(opCtx, uuid);
invariant(coll,
str::stream() << "The collection with UUID " << uuid
<< " is unexpectedly missing in the CollectionCatalog");
@@ -665,7 +665,7 @@ void RollbackImpl::_correctRecordStoreCounts(OperationContext* opCtx) {
}
Status RollbackImpl::_findRecordStoreCounts(OperationContext* opCtx) {
- const auto& catalog = CollectionCatalog::get(opCtx);
+ auto catalog = CollectionCatalog::get(opCtx);
auto storageEngine = opCtx->getServiceContext()->getStorageEngine();
LOGV2(21604, "Finding record store counts");
@@ -676,7 +676,7 @@ Status RollbackImpl::_findRecordStoreCounts(OperationContext* opCtx) {
continue;
}
- auto nss = catalog.lookupNSSByUUID(opCtx, uuid);
+ auto nss = catalog->lookupNSSByUUID(opCtx, uuid);
StorageInterface::CollectionCount oldCount = 0;
// Drop-pending collections are not visible to rollback via the catalog when they are
@@ -1131,11 +1131,11 @@ boost::optional<BSONObj> RollbackImpl::_findDocumentById(OperationContext* opCtx
}
Status RollbackImpl::_writeRollbackFiles(OperationContext* opCtx) {
- const auto& catalog = CollectionCatalog::get(opCtx);
+ auto catalog = CollectionCatalog::get(opCtx);
auto storageEngine = opCtx->getServiceContext()->getStorageEngine();
for (auto&& entry : _observerInfo.rollbackDeletedIdsMap) {
const auto& uuid = entry.first;
- const auto nss = catalog.lookupNSSByUUID(opCtx, uuid);
+ const auto nss = catalog->lookupNSSByUUID(opCtx, uuid);
// Drop-pending collections are not visible to rollback via the catalog when they are
// managed by the storage engine. See StorageEngine::supportsPendingDrops().
diff --git a/src/mongo/db/repl/rs_rollback.cpp b/src/mongo/db/repl/rs_rollback.cpp
index ba48a3f4e2c..59f515c2076 100644
--- a/src/mongo/db/repl/rs_rollback.cpp
+++ b/src/mongo/db/repl/rs_rollback.cpp
@@ -903,7 +903,7 @@ void dropIndex(OperationContext* opCtx,
void rollbackCreateIndexes(OperationContext* opCtx, UUID uuid, std::set<std::string> indexNames) {
boost::optional<NamespaceString> nss =
- CollectionCatalog::get(opCtx).lookupNSSByUUID(opCtx, uuid);
+ CollectionCatalog::get(opCtx)->lookupNSSByUUID(opCtx, uuid);
invariant(nss);
Lock::DBLock dbLock(opCtx, nss->db(), MODE_X);
CollectionWriter collection(opCtx, uuid);
@@ -966,13 +966,12 @@ void rollbackCreateIndexes(OperationContext* opCtx, UUID uuid, std::set<std::str
void rollbackDropIndexes(OperationContext* opCtx,
UUID uuid,
std::map<std::string, BSONObj> indexNames) {
- boost::optional<NamespaceString> nss =
- CollectionCatalog::get(opCtx).lookupNSSByUUID(opCtx, uuid);
+ auto catalog = CollectionCatalog::get(opCtx);
+ boost::optional<NamespaceString> nss = catalog->lookupNSSByUUID(opCtx, uuid);
invariant(nss);
Lock::DBLock dbLock(opCtx, nss->db(), MODE_IX);
Lock::CollectionLock collLock(opCtx, *nss, MODE_X);
- CollectionPtr collection =
- CollectionCatalog::get(opCtx).lookupCollectionByNamespace(opCtx, *nss);
+ CollectionPtr collection = catalog->lookupCollectionByNamespace(opCtx, *nss);
// If we cannot find the collection, we skip over dropping the index.
if (!collection) {
@@ -1082,7 +1081,7 @@ void renameOutOfTheWay(OperationContext* opCtx, RenameCollectionInfo info, Datab
// Finds the UUID of the collection that we are renaming out of the way.
auto collection =
- CollectionCatalog::get(opCtx).lookupCollectionByNamespace(opCtx, info.renameTo);
+ CollectionCatalog::get(opCtx)->lookupCollectionByNamespace(opCtx, info.renameTo);
invariant(collection);
// The generated unique collection name is only guaranteed to exist if the database is
@@ -1319,7 +1318,7 @@ void rollback_internal::syncFixUp(OperationContext* opCtx,
// UUID -> doc id -> doc
stdx::unordered_map<UUID, std::map<DocID, BSONObj>, UUID::Hash> goodVersions;
- auto& catalog = CollectionCatalog::get(opCtx);
+ auto catalog = CollectionCatalog::get(opCtx);
// Fetches all the goodVersions of each document from the current sync source.
unsigned long long numFetched = 0;
@@ -1330,7 +1329,7 @@ void rollback_internal::syncFixUp(OperationContext* opCtx,
invariant(!doc._id.eoo()); // This is checked when we insert to the set.
UUID uuid = doc.uuid;
- boost::optional<NamespaceString> nss = catalog.lookupNSSByUUID(opCtx, uuid);
+ boost::optional<NamespaceString> nss = catalog->lookupNSSByUUID(opCtx, uuid);
try {
if (nss) {
@@ -1480,8 +1479,7 @@ void rollback_internal::syncFixUp(OperationContext* opCtx,
fixUpInfo.indexBuildsToRestart.end(),
[&](auto build) { return build.second.collUUID == uuid; }));
- boost::optional<NamespaceString> nss =
- CollectionCatalog::get(opCtx).lookupNSSByUUID(opCtx, uuid);
+ boost::optional<NamespaceString> nss = catalog->lookupNSSByUUID(opCtx, uuid);
// Do not attempt to acquire the database lock with an empty namespace. We should survive
// an attempt to drop a non-existent collection.
if (!nss) {
@@ -1500,7 +1498,7 @@ void rollback_internal::syncFixUp(OperationContext* opCtx,
Database* db = dbLock.getDb();
if (db) {
CollectionPtr collection =
- CollectionCatalog::get(opCtx).lookupCollectionByUUID(opCtx, uuid);
+ CollectionCatalog::get(opCtx)->lookupCollectionByUUID(opCtx, uuid);
dropCollection(opCtx, *nss, collection, db);
LOGV2_DEBUG(21698,
1,
@@ -1558,8 +1556,7 @@ void rollback_internal::syncFixUp(OperationContext* opCtx,
// occurs and then the collection is dropped. If we do not first re-create the
// collection, we will not be able to retrieve the collection's catalog entries.
for (auto uuid : fixUpInfo.collectionsToResyncMetadata) {
- boost::optional<NamespaceString> nss =
- CollectionCatalog::get(opCtx).lookupNSSByUUID(opCtx, uuid);
+ boost::optional<NamespaceString> nss = catalog->lookupNSSByUUID(opCtx, uuid);
invariant(nss);
LOGV2(21702,
@@ -1689,7 +1686,7 @@ void rollback_internal::syncFixUp(OperationContext* opCtx,
unique_ptr<RemoveSaver> removeSaver;
invariant(!fixUpInfo.collectionsToDrop.count(uuid));
- boost::optional<NamespaceString> nss = catalog.lookupNSSByUUID(opCtx, uuid);
+ boost::optional<NamespaceString> nss = catalog->lookupNSSByUUID(opCtx, uuid);
if (!nss) {
nss = NamespaceString();
}
@@ -1949,8 +1946,7 @@ void rollback_internal::syncFixUp(OperationContext* opCtx,
Lock::DBLock oplogDbLock(opCtx, oplogNss.db(), MODE_IX);
Lock::CollectionLock oplogCollectionLoc(opCtx, oplogNss, MODE_X);
OldClientContext ctx(opCtx, oplogNss.ns());
- auto oplogCollection =
- CollectionCatalog::get(opCtx).lookupCollectionByNamespace(opCtx, oplogNss);
+ auto oplogCollection = catalog->lookupCollectionByNamespace(opCtx, oplogNss);
if (!oplogCollection) {
fassertFailedWithStatusNoTrace(
40495,
diff --git a/src/mongo/db/repl/rs_rollback_test.cpp b/src/mongo/db/repl/rs_rollback_test.cpp
index 39b1b9201f4..9c7314b8b78 100644
--- a/src/mongo/db/repl/rs_rollback_test.cpp
+++ b/src/mongo/db/repl/rs_rollback_test.cpp
@@ -388,8 +388,8 @@ int _testRollbackDelete(OperationContext* opCtx,
auto databaseHolder = DatabaseHolder::get(opCtx);
auto db = databaseHolder->getDb(opCtx, "test");
ASSERT_TRUE(db);
- auto collection =
- CollectionCatalog::get(opCtx).lookupCollectionByNamespace(opCtx, NamespaceString("test.t"));
+ auto collection = CollectionCatalog::get(opCtx)->lookupCollectionByNamespace(
+ opCtx, NamespaceString("test.t"));
if (!collection) {
return -1;
}
@@ -540,7 +540,7 @@ TEST_F(RSRollbackTest, RollbackCreateIndexCommand) {
{
Lock::DBLock dbLock(_opCtx.get(), nss.db(), MODE_S);
auto indexCatalog = CollectionCatalog::get(_opCtx.get())
- .lookupCollectionByNamespace(_opCtx.get(), nss)
+ ->lookupCollectionByNamespace(_opCtx.get(), nss)
->getIndexCatalog();
ASSERT(indexCatalog);
ASSERT_EQUALS(1, indexCatalog->numIndexesReady(_opCtx.get()));
@@ -585,7 +585,7 @@ TEST_F(RSRollbackTest, RollbackCreateIndexCommandIndexNotInCatalog) {
{
Lock::DBLock dbLock(_opCtx.get(), nss.db(), MODE_S);
auto indexCatalog = CollectionCatalog::get(_opCtx.get())
- .lookupCollectionByNamespace(_opCtx.get(), nss)
+ ->lookupCollectionByNamespace(_opCtx.get(), nss)
->getIndexCatalog();
ASSERT(indexCatalog);
ASSERT_EQUALS(1, indexCatalog->numIndexesReady(_opCtx.get()));
@@ -624,7 +624,7 @@ TEST_F(RSRollbackTest, RollbackDropIndexCommandWithOneIndex) {
{
Lock::DBLock dbLock(_opCtx.get(), nss.db(), MODE_S);
auto indexCatalog = CollectionCatalog::get(_opCtx.get())
- .lookupCollectionByNamespace(_opCtx.get(), nss)
+ ->lookupCollectionByNamespace(_opCtx.get(), nss)
->getIndexCatalog();
ASSERT(indexCatalog);
ASSERT_EQUALS(2, indexCatalog->numIndexesReady(_opCtx.get()));
@@ -666,7 +666,7 @@ TEST_F(RSRollbackTest, RollbackDropIndexCommandWithMultipleIndexes) {
{
Lock::DBLock dbLock(_opCtx.get(), nss.db(), MODE_S);
auto indexCatalog = CollectionCatalog::get(_opCtx.get())
- .lookupCollectionByNamespace(_opCtx.get(), nss)
+ ->lookupCollectionByNamespace(_opCtx.get(), nss)
->getIndexCatalog();
ASSERT(indexCatalog);
ASSERT_EQUALS(3, indexCatalog->numIndexesReady(_opCtx.get()));
@@ -711,7 +711,7 @@ TEST_F(RSRollbackTest, RollingBackCreateAndDropOfSameIndexIgnoresBothCommands) {
{
Lock::DBLock dbLock(_opCtx.get(), nss.db(), MODE_S);
auto indexCatalog = CollectionCatalog::get(_opCtx.get())
- .lookupCollectionByNamespace(_opCtx.get(), nss)
+ ->lookupCollectionByNamespace(_opCtx.get(), nss)
->getIndexCatalog();
ASSERT(indexCatalog);
ASSERT_EQUALS(1, indexCatalog->numIndexesReady(_opCtx.get()));
@@ -807,7 +807,7 @@ TEST_F(RSRollbackTest, RollingBackDropAndCreateOfSameIndexNameWithDifferentSpecs
{
Lock::DBLock dbLock(_opCtx.get(), nss.db(), MODE_S);
auto indexCatalog = CollectionCatalog::get(_opCtx.get())
- .lookupCollectionByNamespace(_opCtx.get(), nss)
+ ->lookupCollectionByNamespace(_opCtx.get(), nss)
->getIndexCatalog();
ASSERT(indexCatalog);
ASSERT_EQUALS(2, indexCatalog->numIndexesReady(_opCtx.get()));
@@ -930,7 +930,7 @@ TEST_F(RSRollbackTest, RollbackDropIndexOnCollectionWithTwoExistingIndexes) {
numIndexesOnColl(
_opCtx.get(),
nss,
- CollectionCatalog::get(_opCtx.get()).lookupCollectionByNamespace(_opCtx.get(), nss)));
+ CollectionCatalog::get(_opCtx.get())->lookupCollectionByNamespace(_opCtx.get(), nss)));
}
TEST_F(RSRollbackTest, RollbackTwoIndexDropsPrecededByTwoIndexCreationsOnSameCollection) {
@@ -967,7 +967,7 @@ TEST_F(RSRollbackTest, RollbackTwoIndexDropsPrecededByTwoIndexCreationsOnSameCol
numIndexesOnColl(
_opCtx.get(),
nss,
- CollectionCatalog::get(_opCtx.get()).lookupCollectionByNamespace(_opCtx.get(), nss)));
+ CollectionCatalog::get(_opCtx.get())->lookupCollectionByNamespace(_opCtx.get(), nss)));
}
TEST_F(RSRollbackTest, RollbackMultipleCreateIndexesOnSameCollection) {
@@ -1009,7 +1009,7 @@ TEST_F(RSRollbackTest, RollbackMultipleCreateIndexesOnSameCollection) {
numIndexesOnColl(
_opCtx.get(),
nss,
- CollectionCatalog::get(_opCtx.get()).lookupCollectionByNamespace(_opCtx.get(), nss)));
+ CollectionCatalog::get(_opCtx.get())->lookupCollectionByNamespace(_opCtx.get(), nss)));
}
TEST_F(RSRollbackTest, RollbackCreateDropRecreateIndexOnCollection) {
@@ -1054,7 +1054,7 @@ TEST_F(RSRollbackTest, RollbackCreateDropRecreateIndexOnCollection) {
numIndexesOnColl(
_opCtx.get(),
nss,
- CollectionCatalog::get(_opCtx.get()).lookupCollectionByNamespace(_opCtx.get(), nss)));
+ CollectionCatalog::get(_opCtx.get())->lookupCollectionByNamespace(_opCtx.get(), nss)));
}
TEST_F(RSRollbackTest, RollbackCommitIndexBuild) {
@@ -1102,7 +1102,7 @@ TEST_F(RSRollbackTest, RollbackCommitIndexBuild) {
_replicationProcess.get()));
auto collAfterRollback =
- CollectionCatalog::get(_opCtx.get()).lookupCollectionByNamespace(_opCtx.get(), nss);
+ CollectionCatalog::get(_opCtx.get())->lookupCollectionByNamespace(_opCtx.get(), nss);
// Make sure the collection indexes are in the proper state post-rollback.
ASSERT_EQUALS(1, numIndexesOnColl(_opCtx.get(), nss, collAfterRollback));
@@ -1160,7 +1160,7 @@ TEST_F(RSRollbackTest, RollbackAbortIndexBuild) {
_replicationProcess.get()));
auto collAfterRollback =
- CollectionCatalog::get(_opCtx.get()).lookupCollectionByNamespace(_opCtx.get(), nss);
+ CollectionCatalog::get(_opCtx.get())->lookupCollectionByNamespace(_opCtx.get(), nss);
// Make sure the collection indexes are in the proper state post-rollback.
ASSERT_EQUALS(1, numIndexesOnColl(_opCtx.get(), nss, collAfterRollback));
@@ -1223,7 +1223,7 @@ TEST_F(RSRollbackTest, AbortedIndexBuildsAreRestarted) {
_replicationProcess.get()));
auto collAfterRollback =
- CollectionCatalog::get(_opCtx.get()).lookupCollectionByNamespace(_opCtx.get(), nss);
+ CollectionCatalog::get(_opCtx.get())->lookupCollectionByNamespace(_opCtx.get(), nss);
// Make sure the collection indexes are in the proper state post-rollback.
ASSERT_EQUALS(1, numIndexesOnColl(_opCtx.get(), nss, collAfterRollback));
@@ -1281,7 +1281,7 @@ TEST_F(RSRollbackTest, AbortedIndexBuildsAreNotRestartedWhenStartIsRolledBack) {
_replicationProcess.get()));
auto collAfterRollback =
- CollectionCatalog::get(_opCtx.get()).lookupCollectionByNamespace(_opCtx.get(), nss);
+ CollectionCatalog::get(_opCtx.get())->lookupCollectionByNamespace(_opCtx.get(), nss);
// The aborted index build should have been dropped.
ASSERT_EQUALS(1, numIndexesOnColl(_opCtx.get(), nss, collAfterRollback));
@@ -1820,7 +1820,7 @@ TEST_F(RSRollbackTest, RollbackApplyOpsCommand) {
AutoGetOrCreateDb autoDb(_opCtx.get(), "test", MODE_X);
mongo::WriteUnitOfWork wuow(_opCtx.get());
coll = CollectionCatalog::get(_opCtx.get())
- .lookupCollectionByNamespace(_opCtx.get(), NamespaceString("test.t"));
+ ->lookupCollectionByNamespace(_opCtx.get(), NamespaceString("test.t"));
if (!coll) {
coll =
autoDb.getDb()->createCollection(_opCtx.get(), NamespaceString("test.t"), options);
@@ -1968,7 +1968,7 @@ TEST_F(RSRollbackTest, RollbackCreateCollectionCommand) {
auto db = databaseHolder->getDb(_opCtx.get(), "test");
ASSERT_TRUE(db);
ASSERT_FALSE(CollectionCatalog::get(_opCtx.get())
- .lookupCollectionByNamespace(_opCtx.get(), NamespaceString("test.t")));
+ ->lookupCollectionByNamespace(_opCtx.get(), NamespaceString("test.t")));
}
}
diff --git a/src/mongo/db/repl/storage_interface_impl.cpp b/src/mongo/db/repl/storage_interface_impl.cpp
index a140825e8c8..352a0bf9f8e 100644
--- a/src/mongo/db/repl/storage_interface_impl.cpp
+++ b/src/mongo/db/repl/storage_interface_impl.cpp
@@ -480,7 +480,7 @@ Status StorageInterfaceImpl::createCollection(OperationContext* opCtx,
AutoGetOrCreateDb databaseWriteGuard(opCtx, nss.db(), MODE_IX);
auto db = databaseWriteGuard.getDb();
invariant(db);
- if (CollectionCatalog::get(opCtx).lookupCollectionByNamespace(opCtx, nss)) {
+ if (CollectionCatalog::get(opCtx)->lookupCollectionByNamespace(opCtx, nss)) {
return Status(ErrorCodes::NamespaceExists,
str::stream() << "Collection " << nss.ns() << " already exists.");
}
@@ -1352,13 +1352,13 @@ Status StorageInterfaceImpl::isAdminDbValid(OperationContext* opCtx) {
return Status::OK();
}
- CollectionPtr usersCollection = CollectionCatalog::get(opCtx).lookupCollectionByNamespace(
- opCtx, AuthorizationManager::usersCollectionNamespace);
+ auto catalog = CollectionCatalog::get(opCtx);
+ CollectionPtr usersCollection =
+ catalog->lookupCollectionByNamespace(opCtx, AuthorizationManager::usersCollectionNamespace);
const bool hasUsers =
usersCollection && !Helpers::findOne(opCtx, usersCollection, BSONObj(), false).isNull();
- CollectionPtr adminVersionCollection =
- CollectionCatalog::get(opCtx).lookupCollectionByNamespace(
- opCtx, AuthorizationManager::versionCollectionNamespace);
+ CollectionPtr adminVersionCollection = catalog->lookupCollectionByNamespace(
+ opCtx, AuthorizationManager::versionCollectionNamespace);
BSONObj authSchemaVersionDocument;
if (!adminVersionCollection ||
!Helpers::findOne(opCtx,
diff --git a/src/mongo/db/run_op_kill_cursors.cpp b/src/mongo/db/run_op_kill_cursors.cpp
index db251bee5c0..d9a42c7f7d8 100644
--- a/src/mongo/db/run_op_kill_cursors.cpp
+++ b/src/mongo/db/run_op_kill_cursors.cpp
@@ -63,7 +63,7 @@ bool killCursorIfAuthorized(OperationContext* opCtx, CursorId id) {
nss,
Top::LockType::NotLocked,
AutoStatsTracker::LogMode::kUpdateTopAndCurOp,
- CollectionCatalog::get(opCtx).getDatabaseProfileLevel(nss.db()));
+ CollectionCatalog::get(opCtx)->getDatabaseProfileLevel(nss.db()));
}
AuthorizationSession* as = AuthorizationSession::get(opCtx->getClient());
diff --git a/src/mongo/db/s/migration_destination_manager.cpp b/src/mongo/db/s/migration_destination_manager.cpp
index 2955886c127..8d42b161ddc 100644
--- a/src/mongo/db/s/migration_destination_manager.cpp
+++ b/src/mongo/db/s/migration_destination_manager.cpp
@@ -817,7 +817,7 @@ void MigrationDestinationManager::cloneCollectionIndexesAndOptions(
AutoGetOrCreateDb autoCreateDb(opCtx, nss.db(), MODE_X);
auto db = autoCreateDb.getDb();
- auto collection = CollectionCatalog::get(opCtx).lookupCollectionByNamespace(opCtx, nss);
+ auto collection = CollectionCatalog::get(opCtx)->lookupCollectionByNamespace(opCtx, nss);
if (collection) {
checkUUIDsMatch(collection);
} else {
@@ -834,7 +834,7 @@ void MigrationDestinationManager::cloneCollectionIndexesAndOptions(
createDefaultIndexes,
collectionOptionsAndIndexes.idIndexSpec));
wuow.commit();
- collection = CollectionCatalog::get(opCtx).lookupCollectionByNamespace(opCtx, nss);
+ collection = CollectionCatalog::get(opCtx)->lookupCollectionByNamespace(opCtx, nss);
}
auto indexSpecs = checkEmptyOrGetMissingIndexesFromDonor(collection);
diff --git a/src/mongo/db/s/migration_util.cpp b/src/mongo/db/s/migration_util.cpp
index 1e5011a6472..01651ece011 100644
--- a/src/mongo/db/s/migration_util.cpp
+++ b/src/mongo/db/s/migration_util.cpp
@@ -530,16 +530,16 @@ void submitOrphanRanges(OperationContext* opCtx, const NamespaceString& nss, con
}
void submitOrphanRangesForCleanup(OperationContext* opCtx) {
- auto& catalog = CollectionCatalog::get(opCtx);
- const auto& dbs = catalog.getAllDbNames();
+ auto catalog = CollectionCatalog::get(opCtx);
+ const auto& dbs = catalog->getAllDbNames();
for (const auto& dbName : dbs) {
if (dbName == NamespaceString::kLocalDb)
continue;
- for (auto collIt = catalog.begin(opCtx, dbName); collIt != catalog.end(opCtx); ++collIt) {
+ for (auto collIt = catalog->begin(opCtx, dbName); collIt != catalog->end(opCtx); ++collIt) {
auto uuid = collIt.uuid().get();
- auto nss = catalog.lookupNSSByUUID(opCtx, uuid).get();
+ auto nss = catalog->lookupNSSByUUID(opCtx, uuid).get();
LOGV2_DEBUG(22034,
2,
"Upgrade: Processing collection for orphaned range cleanup",
diff --git a/src/mongo/db/s/resharding/resharding_oplog_fetcher.cpp b/src/mongo/db/s/resharding/resharding_oplog_fetcher.cpp
index b04e86afe80..2dce0fee77b 100644
--- a/src/mongo/db/s/resharding/resharding_oplog_fetcher.cpp
+++ b/src/mongo/db/s/resharding/resharding_oplog_fetcher.cpp
@@ -191,7 +191,7 @@ boost::optional<ReshardingDonorOplogId> ReshardingOplogFetcher::iterate(
// Create the destination collection if necessary.
writeConflictRetry(opCtx, "createReshardingLocalOplogBuffer", toWriteToNss.toString(), [&] {
const CollectionPtr toWriteTo =
- CollectionCatalog::get(opCtx).lookupCollectionByNamespace(opCtx, toWriteToNss);
+ CollectionCatalog::get(opCtx)->lookupCollectionByNamespace(opCtx, toWriteToNss);
if (toWriteTo) {
return;
}
diff --git a/src/mongo/db/s/resharding_destined_recipient_test.cpp b/src/mongo/db/s/resharding_destined_recipient_test.cpp
index 164a51fda5e..2247fb81c62 100644
--- a/src/mongo/db/s/resharding_destined_recipient_test.cpp
+++ b/src/mongo/db/s/resharding_destined_recipient_test.cpp
@@ -181,7 +181,7 @@ protected:
client.createCollection(kNss.ns());
client.createCollection(NamespaceString::kSessionTransactionsTableNamespace.ns());
- ReshardingEnv env(CollectionCatalog::get(opCtx).lookupUUIDByNSS(opCtx, kNss).value());
+ ReshardingEnv env(CollectionCatalog::get(opCtx)->lookupUUIDByNSS(opCtx, kNss).value());
env.destShard = kShardList[1].getName();
env.version = ChunkVersion(1, 0, OID::gen());
env.dbVersion = databaseVersion::makeNew();
diff --git a/src/mongo/db/s/resharding_util.cpp b/src/mongo/db/s/resharding_util.cpp
index 971b8726ba5..8a0256ae23c 100644
--- a/src/mongo/db/s/resharding_util.cpp
+++ b/src/mongo/db/s/resharding_util.cpp
@@ -67,7 +67,7 @@ namespace {
UUID getCollectionUuid(OperationContext* opCtx, const NamespaceString& nss) {
dassert(opCtx->lockState()->isCollectionLockedForMode(nss, MODE_IS));
- auto uuid = CollectionCatalog::get(opCtx).lookupUUIDByNSS(opCtx, nss);
+ auto uuid = CollectionCatalog::get(opCtx)->lookupUUIDByNSS(opCtx, nss);
invariant(uuid);
return *uuid;
@@ -317,7 +317,7 @@ void createSlimOplogView(OperationContext* opCtx, Database* db) {
{
// Create 'system.views' in a separate WUOW if it does not exist.
WriteUnitOfWork wuow(opCtx);
- CollectionPtr coll = CollectionCatalog::get(opCtx).lookupCollectionByNamespace(
+ CollectionPtr coll = CollectionCatalog::get(opCtx)->lookupCollectionByNamespace(
opCtx, NamespaceString(db->getSystemViewsName()));
if (!coll) {
coll = db->createCollection(opCtx, NamespaceString(db->getSystemViewsName()));
diff --git a/src/mongo/db/s/set_shard_version_command.cpp b/src/mongo/db/s/set_shard_version_command.cpp
index 68a8ebbb1c8..7b97a17b8e3 100644
--- a/src/mongo/db/s/set_shard_version_command.cpp
+++ b/src/mongo/db/s/set_shard_version_command.cpp
@@ -150,7 +150,7 @@ public:
// Views do not require a shard version check. We do not care about invalid system views
// for this check, only to validate if a view already exists for this namespace.
if (autoDb->getDb() &&
- !CollectionCatalog::get(opCtx).lookupCollectionByNamespace(opCtx, nss) &&
+ !CollectionCatalog::get(opCtx)->lookupCollectionByNamespace(opCtx, nss) &&
ViewCatalog::get(autoDb->getDb())
->lookupWithoutValidatingDurableViews(opCtx, nss.ns())) {
return true;
diff --git a/src/mongo/db/startup_recovery.cpp b/src/mongo/db/startup_recovery.cpp
index 45101c407cb..1e636fc920b 100644
--- a/src/mongo/db/startup_recovery.cpp
+++ b/src/mongo/db/startup_recovery.cpp
@@ -94,8 +94,9 @@ Status restoreMissingFeatureCompatibilityVersionDocument(OperationContext* opCtx
// If the server configuration collection, which contains the FCV document, does not exist, then
// create it.
- if (!CollectionCatalog::get(opCtx).lookupCollectionByNamespace(
- opCtx, NamespaceString::kServerConfigurationNamespace)) {
+ auto catalog = CollectionCatalog::get(opCtx);
+ if (!catalog->lookupCollectionByNamespace(opCtx,
+ NamespaceString::kServerConfigurationNamespace)) {
// (Generic FCV reference): This FCV reference should exist across LTS binary versions.
LOGV2(4926905,
"Re-creating featureCompatibilityVersion document that was deleted. Creating new "
@@ -105,8 +106,8 @@ Status restoreMissingFeatureCompatibilityVersionDocument(OperationContext* opCtx
createCollection(opCtx, fcvNss.db().toString(), BSON("create" << fcvNss.coll())));
}
- const CollectionPtr& fcvColl = CollectionCatalog::get(opCtx).lookupCollectionByNamespace(
- opCtx, NamespaceString::kServerConfigurationNamespace);
+ const CollectionPtr& fcvColl =
+ catalog->lookupCollectionByNamespace(opCtx, NamespaceString::kServerConfigurationNamespace);
invariant(fcvColl);
// Restore the featureCompatibilityVersion document if it is missing.
@@ -216,7 +217,8 @@ enum class EnsureIndexPolicy { kBuildMissing, kError };
Status ensureCollectionProperties(OperationContext* opCtx,
Database* db,
EnsureIndexPolicy ensureIndexPolicy) {
- for (auto collIt = db->begin(opCtx); collIt != db->end(opCtx); ++collIt) {
+ auto catalog = CollectionCatalog::get(opCtx);
+ for (auto collIt = catalog->begin(opCtx, db->name()); collIt != catalog->end(opCtx); ++collIt) {
auto coll = collIt.getWritableCollection(opCtx, CollectionCatalog::LifetimeMode::kInplace);
if (!coll) {
break;
@@ -325,7 +327,7 @@ void assertCappedOplog(OperationContext* opCtx, Database* db) {
const NamespaceString oplogNss(NamespaceString::kRsOplogNamespace);
invariant(opCtx->lockState()->isDbLockedForMode(oplogNss.db(), MODE_IS));
const CollectionPtr& oplogCollection =
- CollectionCatalog::get(opCtx).lookupCollectionByNamespace(opCtx, oplogNss);
+ CollectionCatalog::get(opCtx)->lookupCollectionByNamespace(opCtx, oplogNss);
if (oplogCollection && !oplogCollection->isCapped()) {
LOGV2_FATAL_NOTRACE(
40115,
@@ -391,10 +393,11 @@ void reconcileCatalogAndRebuildUnfinishedIndexes(
ino.second.emplace_back(std::move(indexesToRebuild.second.back()));
}
+ auto catalog = CollectionCatalog::get(opCtx);
for (const auto& entry : nsToIndexNameObjMap) {
NamespaceString collNss(entry.first);
- auto collection = CollectionCatalog::get(opCtx).lookupCollectionByNamespace(opCtx, collNss);
+ auto collection = catalog->lookupCollectionByNamespace(opCtx, collNss);
for (const auto& indexName : entry.second.first) {
LOGV2(21004,
"Rebuilding index. Collection: {collNss} Index: {indexName}",
@@ -437,7 +440,7 @@ void setReplSetMemberInStandaloneMode(OperationContext* opCtx) {
}
invariant(opCtx->lockState()->isW());
- CollectionPtr collection = CollectionCatalog::get(opCtx).lookupCollectionByNamespace(
+ CollectionPtr collection = CollectionCatalog::get(opCtx)->lookupCollectionByNamespace(
opCtx, NamespaceString::kSystemReplSetNamespace);
if (collection && !collection->isEmpty(opCtx)) {
setReplSetMemberInStandaloneMode(opCtx->getServiceContext(), true);
@@ -465,7 +468,7 @@ void startupRepair(OperationContext* opCtx, StorageEngine* storageEngine) {
// order to allow downgrading to older binary versions.
auto abortRepairOnFCVErrors = makeGuard(
[&] { StorageRepairObserver::get(opCtx->getServiceContext())->onRepairDone(opCtx); });
- if (auto fcvColl = CollectionCatalog::get(opCtx).lookupCollectionByNamespace(
+ if (auto fcvColl = CollectionCatalog::get(opCtx)->lookupCollectionByNamespace(
opCtx, NamespaceString::kServerConfigurationNamespace)) {
auto databaseHolder = DatabaseHolder::get(opCtx);
databaseHolder->openDb(opCtx, fcvColl->ns().db());
diff --git a/src/mongo/db/storage/durable_catalog_impl.cpp b/src/mongo/db/storage/durable_catalog_impl.cpp
index bc5bbc5f2d8..e32f43796d4 100644
--- a/src/mongo/db/storage/durable_catalog_impl.cpp
+++ b/src/mongo/db/storage/durable_catalog_impl.cpp
@@ -856,7 +856,7 @@ StatusWith<std::pair<RecordId, std::unique_ptr<RecordStore>>> DurableCatalogImpl
invariant(opCtx->lockState()->isCollectionLockedForMode(nss, MODE_IX));
invariant(nss.coll().size() > 0);
- if (CollectionCatalog::get(opCtx).lookupCollectionByNamespace(opCtx, nss)) {
+ if (CollectionCatalog::get(opCtx)->lookupCollectionByNamespace(opCtx, nss)) {
throw WriteConflictException();
}
@@ -905,7 +905,7 @@ StatusWith<DurableCatalog::ImportResult> DurableCatalogImpl::importCollection(
uassert(ErrorCodes::NamespaceExists,
str::stream() << "Collection already exists. NS: " << nss,
- !CollectionCatalog::get(opCtx).lookupCollectionByNamespace(opCtx, nss));
+ !CollectionCatalog::get(opCtx)->lookupCollectionByNamespace(opCtx, nss));
BSONCollectionCatalogEntry::MetaData md;
const BSONElement mdElement = metadata["md"];
diff --git a/src/mongo/db/storage/kv/durable_catalog_test.cpp b/src/mongo/db/storage/kv/durable_catalog_test.cpp
index 3bd2294958e..c9e3fdb72f7 100644
--- a/src/mongo/db/storage/kv/durable_catalog_test.cpp
+++ b/src/mongo/db/storage/kv/durable_catalog_test.cpp
@@ -96,8 +96,9 @@ public:
RecordId catalogId = coll.first;
std::shared_ptr<Collection> collection = std::make_shared<CollectionMock>(nss, catalogId);
- CollectionCatalog::get(operationContext())
- .registerCollection(options.uuid.get(), std::move(collection));
+ CollectionCatalog::write(operationContext(), [&](CollectionCatalog& catalog) {
+ catalog.registerCollection(options.uuid.get(), std::move(collection));
+ });
wuow.commit();
diff --git a/src/mongo/db/storage/storage_engine_impl.cpp b/src/mongo/db/storage/storage_engine_impl.cpp
index 388f845b58f..742fb034569 100644
--- a/src/mongo/db/storage/storage_engine_impl.cpp
+++ b/src/mongo/db/storage/storage_engine_impl.cpp
@@ -287,8 +287,9 @@ void StorageEngineImpl::_initCollection(OperationContext* opCtx,
auto collectionFactory = Collection::Factory::get(getGlobalServiceContext());
auto collection = collectionFactory->make(opCtx, nss, catalogId, uuid, std::move(rs));
- auto& collectionCatalog = CollectionCatalog::get(getGlobalServiceContext());
- collectionCatalog.registerCollection(uuid, std::move(collection));
+ CollectionCatalog::write(opCtx, [&](CollectionCatalog& catalog) {
+ catalog.registerCollection(uuid, std::move(collection));
+ });
}
void StorageEngineImpl::closeCatalog(OperationContext* opCtx) {
@@ -297,7 +298,9 @@ void StorageEngineImpl::closeCatalog(OperationContext* opCtx) {
LOGV2_FOR_RECOVERY(4615632, kCatalogLogLevel.toInt(), "loadCatalog:");
_dumpCatalog(opCtx);
}
- CollectionCatalog::get(opCtx).deregisterAllCollections();
+
+ CollectionCatalog::write(
+ opCtx, [&](CollectionCatalog& catalog) { catalog.deregisterAllCollections(); });
_catalog.reset();
_catalogRecordStore.reset();
@@ -663,7 +666,9 @@ void StorageEngineImpl::cleanShutdown() {
_timestampMonitor->removeListener(&_minOfCheckpointAndOldestTimestampListener);
}
- CollectionCatalog::get(getGlobalServiceContext()).deregisterAllCollections();
+ CollectionCatalog::write(getGlobalServiceContext(), [](CollectionCatalog& catalog) {
+ catalog.deregisterAllCollections();
+ });
_catalog.reset();
_catalogRecordStore.reset();
@@ -698,7 +703,7 @@ RecoveryUnit* StorageEngineImpl::newRecoveryUnit() {
}
std::vector<std::string> StorageEngineImpl::listDatabases() const {
- return CollectionCatalog::get(getGlobalServiceContext()).getAllDbNames();
+ return CollectionCatalog::get(getGlobalServiceContext())->getAllDbNames();
}
Status StorageEngineImpl::closeDatabase(OperationContext* opCtx, StringData db) {
@@ -707,15 +712,15 @@ Status StorageEngineImpl::closeDatabase(OperationContext* opCtx, StringData db)
}
Status StorageEngineImpl::dropDatabase(OperationContext* opCtx, StringData db) {
+ auto catalog = CollectionCatalog::get(opCtx);
{
- auto dbs = CollectionCatalog::get(opCtx).getAllDbNames();
+ auto dbs = catalog->getAllDbNames();
if (std::count(dbs.begin(), dbs.end(), db.toString()) == 0) {
return Status(ErrorCodes::NamespaceNotFound, "db not found to drop");
}
}
- std::vector<NamespaceString> toDrop =
- CollectionCatalog::get(opCtx).getAllCollectionNamesFromDb(opCtx, db);
+ std::vector<NamespaceString> toDrop = catalog->getAllCollectionNamesFromDb(opCtx, db);
// Do not timestamp any of the following writes. This will remove entries from the catalog as
// well as drop any underlying tables. It's not expected for dropping tables to be reversible
@@ -750,8 +755,9 @@ Status StorageEngineImpl::_dropCollectionsNoTimestamp(OperationContext* opCtx,
Status firstError = Status::OK();
WriteUnitOfWork untimestampedDropWuow(opCtx);
+ auto collectionCatalog = CollectionCatalog::get(opCtx);
for (auto& nss : toDrop) {
- auto coll = CollectionCatalog::get(opCtx).lookupCollectionByNamespace(opCtx, nss);
+ auto coll = collectionCatalog->lookupCollectionByNamespace(opCtx, nss);
// No need to remove the indexes from the IndexCatalog because eliminating the Collection
// will have the same effect.
@@ -773,10 +779,12 @@ Status StorageEngineImpl::_dropCollectionsNoTimestamp(OperationContext* opCtx,
firstError = result;
}
- auto removedColl = CollectionCatalog::get(opCtx).deregisterCollection(opCtx, coll->uuid());
- opCtx->recoveryUnit()->registerChange(
- CollectionCatalog::get(opCtx).makeFinishDropCollectionChange(std::move(removedColl),
- coll->uuid()));
+ std::shared_ptr<Collection> removedColl;
+ CollectionCatalog::write(opCtx, [&](CollectionCatalog& catalog) {
+ removedColl = catalog.deregisterCollection(opCtx, coll->uuid());
+ });
+ opCtx->recoveryUnit()->registerChange(CollectionCatalog::makeFinishDropCollectionChange(
+ opCtx, std::move(removedColl), coll->uuid()));
}
untimestampedDropWuow.commit();
@@ -857,9 +865,10 @@ Status StorageEngineImpl::repairRecordStore(OperationContext* opCtx,
}
// After repairing, re-initialize the collection with a valid RecordStore.
- auto& collectionCatalog = CollectionCatalog::get(getGlobalServiceContext());
- auto uuid = collectionCatalog.lookupUUIDByNSS(opCtx, nss).get();
- collectionCatalog.deregisterCollection(opCtx, uuid);
+ CollectionCatalog::write(opCtx, [&](CollectionCatalog& catalog) {
+ auto uuid = catalog.lookupUUIDByNSS(opCtx, nss).get();
+ catalog.deregisterCollection(opCtx, uuid);
+ });
_initCollection(opCtx, catalogId, nss, false);
return status;
diff --git a/src/mongo/db/storage/storage_engine_test_fixture.h b/src/mongo/db/storage/storage_engine_test_fixture.h
index e9c9946cd5c..05ccd5edb44 100644
--- a/src/mongo/db/storage/storage_engine_test_fixture.h
+++ b/src/mongo/db/storage/storage_engine_test_fixture.h
@@ -70,7 +70,9 @@ public:
wuow.commit();
}
std::shared_ptr<Collection> coll = std::make_shared<CollectionMock>(ns, catalogId);
- CollectionCatalog::get(opCtx).registerCollection(options.uuid.get(), std::move(coll));
+ CollectionCatalog::write(opCtx, [&](CollectionCatalog& catalog) {
+ catalog.registerCollection(options.uuid.get(), std::move(coll));
+ });
return {{_storageEngine->getCatalog()->getEntry(catalogId)}};
}
@@ -90,7 +92,7 @@ public:
Status dropIndexTable(OperationContext* opCtx, NamespaceString nss, std::string indexName) {
RecordId catalogId =
- CollectionCatalog::get(opCtx).lookupCollectionByNamespace(opCtx, nss)->getCatalogId();
+ CollectionCatalog::get(opCtx)->lookupCollectionByNamespace(opCtx, nss)->getCatalogId();
std::string indexIdent =
_storageEngine->getCatalog()->getIndexIdent(opCtx, catalogId, indexName);
return dropIdent(opCtx->recoveryUnit(), indexIdent);
@@ -158,7 +160,7 @@ public:
BSONObj spec = builder.append("name", key).append("v", 2).done();
CollectionPtr collection =
- CollectionCatalog::get(opCtx).lookupCollectionByNamespace(opCtx, collNs);
+ CollectionCatalog::get(opCtx)->lookupCollectionByNamespace(opCtx, collNs);
auto descriptor = std::make_unique<IndexDescriptor>(IndexNames::findPluginName(spec), spec);
auto ret = DurableCatalog::get(opCtx)->prepareForIndexBuild(opCtx,
@@ -171,12 +173,12 @@ public:
void indexBuildSuccess(OperationContext* opCtx, NamespaceString collNs, std::string key) {
CollectionPtr collection =
- CollectionCatalog::get(opCtx).lookupCollectionByNamespace(opCtx, collNs);
+ CollectionCatalog::get(opCtx)->lookupCollectionByNamespace(opCtx, collNs);
DurableCatalog::get(opCtx)->indexBuildSuccess(opCtx, collection->getCatalogId(), key);
}
Status removeEntry(OperationContext* opCtx, StringData collNs, DurableCatalog* catalog) {
- CollectionPtr collection = CollectionCatalog::get(opCtx).lookupCollectionByNamespace(
+ CollectionPtr collection = CollectionCatalog::get(opCtx)->lookupCollectionByNamespace(
opCtx, NamespaceString(collNs));
return dynamic_cast<DurableCatalogImpl*>(catalog)->_removeEntry(opCtx,
collection->getCatalogId());
diff --git a/src/mongo/db/storage/storage_util.cpp b/src/mongo/db/storage/storage_util.cpp
index 1556b3469c4..1e333584fd5 100644
--- a/src/mongo/db/storage/storage_util.cpp
+++ b/src/mongo/db/storage/storage_util.cpp
@@ -128,58 +128,60 @@ Status dropCollection(OperationContext* opCtx,
// RecoveryUnit throughout but not the same OperationContext.
auto recoveryUnit = opCtx->recoveryUnit();
auto storageEngine = opCtx->getServiceContext()->getStorageEngine();
- const auto& collectionCatalog = CollectionCatalog::get(opCtx);
+
// Schedule the second phase of drop to delete the data when it is no longer in use, if the
// first phase is successuflly committed.
- opCtx->recoveryUnit()->onCommit([recoveryUnit, storageEngine, &collectionCatalog, nss, ident](
- boost::optional<Timestamp> commitTimestamp) {
- StorageEngine::DropIdentCallback onDrop = [storageEngine, &collectionCatalog, ns = nss] {
- // Nothing to do if not using directoryperdb or there are still collections in the
- // database.
- if (!storageEngine->isUsingDirectoryPerDb() ||
- collectionCatalog.begin(nullptr, ns.db()) != collectionCatalog.end(nullptr)) {
- return;
- }
-
- boost::system::error_code ec;
- boost::filesystem::remove(storageEngine->getFilesystemPathForDb(ns.db().toString()),
- ec);
-
- if (!ec) {
- LOGV2(4888200, "Removed empty database directory", "db"_attr = ns.db());
- } else if (collectionCatalog.begin(nullptr, ns.db()) ==
- collectionCatalog.end(nullptr)) {
- // It is possible for a new collection to be created in the database between
- // when we check whether the database is empty and actually attempting to
- // remove the directory. In this case, don't log that the removal failed
- // because it is expected.
- LOGV2(4888201,
- "Failed to remove database directory",
- "db"_attr = ns.db(),
- "error"_attr = ec.message());
- }
- };
-
- if (storageEngine->supportsPendingDrops()) {
- if (!commitTimestamp) {
- // Standalone mode will not provide a timestamp.
- commitTimestamp = Timestamp::min();
+ opCtx->recoveryUnit()->onCommit(
+ [svcCtx = opCtx->getServiceContext(), recoveryUnit, storageEngine, nss, ident](
+ boost::optional<Timestamp> commitTimestamp) {
+ StorageEngine::DropIdentCallback onDrop = [svcCtx, storageEngine, ns = nss] {
+ // Nothing to do if not using directoryperdb or there are still collections in the
+ // database.
+ auto collectionCatalog = CollectionCatalog::get(svcCtx);
+ if (!storageEngine->isUsingDirectoryPerDb() ||
+ collectionCatalog->begin(nullptr, ns.db()) != collectionCatalog->end(nullptr)) {
+ return;
+ }
+
+ boost::system::error_code ec;
+ boost::filesystem::remove(storageEngine->getFilesystemPathForDb(ns.db().toString()),
+ ec);
+
+ if (!ec) {
+ LOGV2(4888200, "Removed empty database directory", "db"_attr = ns.db());
+ } else if (collectionCatalog->begin(nullptr, ns.db()) ==
+ collectionCatalog->end(nullptr)) {
+ // It is possible for a new collection to be created in the database between
+ // when we check whether the database is empty and actually attempting to
+ // remove the directory. In this case, don't log that the removal failed
+ // because it is expected.
+ LOGV2(4888201,
+ "Failed to remove database directory",
+ "db"_attr = ns.db(),
+ "error"_attr = ec.message());
+ }
+ };
+
+ if (storageEngine->supportsPendingDrops()) {
+ if (!commitTimestamp) {
+ // Standalone mode will not provide a timestamp.
+ commitTimestamp = Timestamp::min();
+ }
+ LOGV2(22214,
+ "Deferring table drop for collection",
+ logAttrs(nss),
+ "ident"_attr = ident->getIdent(),
+ "commitTimestamp"_attr = commitTimestamp);
+ storageEngine->addDropPendingIdent(*commitTimestamp, nss, ident, std::move(onDrop));
+ } else {
+ // Intentionally ignoring failure here. Since we've removed the metadata pointing to
+ // the collection, we should never see it again anyway.
+ storageEngine->getEngine()
+ ->dropIdent(recoveryUnit, ident->getIdent(), std::move(onDrop))
+ .ignore();
}
- LOGV2(22214,
- "Deferring table drop for collection",
- logAttrs(nss),
- "ident"_attr = ident->getIdent(),
- "commitTimestamp"_attr = commitTimestamp);
- storageEngine->addDropPendingIdent(*commitTimestamp, nss, ident, std::move(onDrop));
- } else {
- // Intentionally ignoring failure here. Since we've removed the metadata pointing to
- // the collection, we should never see it again anyway.
- storageEngine->getEngine()
- ->dropIdent(recoveryUnit, ident->getIdent(), std::move(onDrop))
- .ignore();
- }
- });
+ });
return Status::OK();
}
diff --git a/src/mongo/db/storage/wiredtiger/oplog_stones_server_status_section.cpp b/src/mongo/db/storage/wiredtiger/oplog_stones_server_status_section.cpp
index b496e931afb..58788e9439a 100644
--- a/src/mongo/db/storage/wiredtiger/oplog_stones_server_status_section.cpp
+++ b/src/mongo/db/storage/wiredtiger/oplog_stones_server_status_section.cpp
@@ -76,7 +76,7 @@ public:
NamespaceString::kRsOplogNamespace,
Top::LockType::ReadLocked,
AutoStatsTracker::LogMode::kUpdateTop,
- CollectionCatalog::get(opCtx).getDatabaseProfileLevel(NamespaceString::kLocalDb));
+ CollectionCatalog::get(opCtx)->getDatabaseProfileLevel(NamespaceString::kLocalDb));
oplog->getRecordStore()->getOplogTruncateStats(builder);
}
return builder.obj();
diff --git a/src/mongo/db/storage/wiredtiger/wiredtiger_kv_engine.cpp b/src/mongo/db/storage/wiredtiger/wiredtiger_kv_engine.cpp
index 0d12d95fd3b..519b2bc8b69 100644
--- a/src/mongo/db/storage/wiredtiger/wiredtiger_kv_engine.cpp
+++ b/src/mongo/db/storage/wiredtiger/wiredtiger_kv_engine.cpp
@@ -1477,7 +1477,7 @@ Status WiredTigerKVEngine::createGroupedSortedDataInterface(OperationContext* op
}
// Some unittests use a OperationContextNoop that can't support such lookups.
auto ns = collOptions.uuid
- ? *CollectionCatalog::get(opCtx).lookupNSSByUUID(opCtx, *collOptions.uuid)
+ ? *CollectionCatalog::get(opCtx)->lookupNSSByUUID(opCtx, *collOptions.uuid)
: NamespaceString();
StatusWith<std::string> result = WiredTigerIndex::generateCreateString(
diff --git a/src/mongo/db/transaction_history_iterator.cpp b/src/mongo/db/transaction_history_iterator.cpp
index c4bd0c0b2f4..e0584763df7 100644
--- a/src/mongo/db/transaction_history_iterator.cpp
+++ b/src/mongo/db/transaction_history_iterator.cpp
@@ -84,7 +84,7 @@ BSONObj findOneOplogEntry(OperationContext* opCtx,
NamespaceString::kRsOplogNamespace,
Top::LockType::ReadLocked,
AutoStatsTracker::LogMode::kUpdateTop,
- CollectionCatalog::get(opCtx).getDatabaseProfileLevel(NamespaceString::kLocalDb),
+ CollectionCatalog::get(opCtx)->getDatabaseProfileLevel(NamespaceString::kLocalDb),
Date_t::max());
auto exec = uassertStatusOK(
diff --git a/src/mongo/db/transaction_participant.cpp b/src/mongo/db/transaction_participant.cpp
index 4c39e30292e..e0d63e8ebac 100644
--- a/src/mongo/db/transaction_participant.cpp
+++ b/src/mongo/db/transaction_participant.cpp
@@ -374,7 +374,7 @@ TransactionParticipant::getOldestActiveTimestamp(Timestamp stableTimestamp) {
}
auto collection =
- CollectionCatalog::get(opCtx.get()).lookupCollectionByNamespace(opCtx.get(), nss);
+ CollectionCatalog::get(opCtx.get())->lookupCollectionByNamespace(opCtx.get(), nss);
if (!collection) {
return boost::none;
}
@@ -1143,8 +1143,9 @@ Timestamp TransactionParticipant::Participant::prepareTransaction(
for (const auto& transactionOp : completedTransactionOperations) {
transactionOperationUuids.insert(transactionOp.getUuid().get());
}
+ auto catalog = CollectionCatalog::get(opCtx);
for (const auto& uuid : transactionOperationUuids) {
- auto collection = CollectionCatalog::get(opCtx).lookupCollectionByUUID(opCtx, uuid);
+ auto collection = catalog->lookupCollectionByUUID(opCtx, uuid);
uassert(ErrorCodes::OperationNotSupportedInTransaction,
str::stream() << "prepareTransaction failed because one of the transaction "
"operations was done against a temporary collection '"
diff --git a/src/mongo/db/transaction_participant_test.cpp b/src/mongo/db/transaction_participant_test.cpp
index c9f9b547586..28b57e27dac 100644
--- a/src/mongo/db/transaction_participant_test.cpp
+++ b/src/mongo/db/transaction_participant_test.cpp
@@ -4229,7 +4229,7 @@ TEST_F(TxnParticipantTest, OldestActiveTransactionTimestamp) {
AutoGetOrCreateDb autoDb(opCtx(), nss.db(), MODE_X);
WriteUnitOfWork wuow(opCtx());
- auto coll = CollectionCatalog::get(opCtx()).lookupCollectionByNamespace(opCtx(), nss);
+ auto coll = CollectionCatalog::get(opCtx())->lookupCollectionByNamespace(opCtx(), nss);
ASSERT(coll);
OpDebug* const nullOpDebug = nullptr;
ASSERT_OK(
@@ -4241,7 +4241,7 @@ TEST_F(TxnParticipantTest, OldestActiveTransactionTimestamp) {
Timestamp ts(1, i);
AutoGetOrCreateDb autoDb(opCtx(), nss.db(), MODE_X);
WriteUnitOfWork wuow(opCtx());
- auto coll = CollectionCatalog::get(opCtx()).lookupCollectionByNamespace(opCtx(), nss);
+ auto coll = CollectionCatalog::get(opCtx())->lookupCollectionByNamespace(opCtx(), nss);
ASSERT(coll);
auto cursor = coll->getCursor(opCtx());
while (auto record = cursor->next()) {
diff --git a/src/mongo/db/ttl.cpp b/src/mongo/db/ttl.cpp
index 5cdbcd9b80f..e3b28438d13 100644
--- a/src/mongo/db/ttl.cpp
+++ b/src/mongo/db/ttl.cpp
@@ -190,6 +190,7 @@ private:
ON_BLOCK_EXIT([&] { ttlPasses.increment(); });
// Get all TTL indexes from every collection.
+ auto collectionCatalog = CollectionCatalog::get(opCtxPtr.get());
for (const std::pair<UUID, std::string>& ttlInfo : ttlInfos) {
auto uuid = ttlInfo.first;
auto indexName = ttlInfo.second;
@@ -197,12 +198,11 @@ private:
// Skip collections that have not been made visible yet. The TTLCollectionCache already
// has the index information available, so we want to avoid removing it until the
// collection is visible.
- const CollectionCatalog& collectionCatalog = CollectionCatalog::get(opCtxPtr.get());
- if (collectionCatalog.isCollectionAwaitingVisibility(uuid)) {
+ if (collectionCatalog->isCollectionAwaitingVisibility(uuid)) {
continue;
}
- auto nss = collectionCatalog.lookupNSSByUUID(&opCtx, uuid);
+ auto nss = collectionCatalog->lookupNSSByUUID(&opCtx, uuid);
if (!nss) {
ttlCollectionCache.deregisterTTLInfo(ttlInfo);
continue;
diff --git a/src/mongo/db/views/durable_view_catalog.cpp b/src/mongo/db/views/durable_view_catalog.cpp
index 838d883fc0d..1031f87682f 100644
--- a/src/mongo/db/views/durable_view_catalog.cpp
+++ b/src/mongo/db/views/durable_view_catalog.cpp
@@ -110,8 +110,8 @@ void DurableViewCatalogImpl::_iterate(OperationContext* opCtx,
ViewCatalogLookupBehavior lookupBehavior) {
invariant(opCtx->lockState()->isCollectionLockedForMode(_db->getSystemViewsName(), MODE_IS));
- const CollectionPtr& systemViews =
- CollectionCatalog::get(opCtx).lookupCollectionByNamespace(opCtx, _db->getSystemViewsName());
+ CollectionPtr systemViews = CollectionCatalog::get(opCtx)->lookupCollectionByNamespace(
+ opCtx, _db->getSystemViewsName());
if (!systemViews) {
return;
}
@@ -186,7 +186,7 @@ void DurableViewCatalogImpl::upsert(OperationContext* opCtx,
dassert(opCtx->lockState()->isCollectionLockedForMode(systemViewsNs, MODE_X));
const CollectionPtr& systemViews =
- CollectionCatalog::get(opCtx).lookupCollectionByNamespace(opCtx, systemViewsNs);
+ CollectionCatalog::get(opCtx)->lookupCollectionByNamespace(opCtx, systemViewsNs);
invariant(systemViews);
const bool requireIndex = false;
@@ -217,8 +217,8 @@ void DurableViewCatalogImpl::remove(OperationContext* opCtx, const NamespaceStri
dassert(opCtx->lockState()->isDbLockedForMode(_db->name(), MODE_IX));
dassert(opCtx->lockState()->isCollectionLockedForMode(name, MODE_IX));
- const CollectionPtr& systemViews =
- CollectionCatalog::get(opCtx).lookupCollectionByNamespace(opCtx, _db->getSystemViewsName());
+ CollectionPtr systemViews = CollectionCatalog::get(opCtx)->lookupCollectionByNamespace(
+ opCtx, _db->getSystemViewsName());
dassert(opCtx->lockState()->isCollectionLockedForMode(systemViews->ns(), MODE_X));
if (!systemViews)
diff --git a/src/mongo/db/views/view_catalog.cpp b/src/mongo/db/views/view_catalog.cpp
index 98931116973..c606eede46a 100644
--- a/src/mongo/db/views/view_catalog.cpp
+++ b/src/mongo/db/views/view_catalog.cpp
@@ -221,9 +221,9 @@ Status ViewCatalog::_createOrUpdateView(WithLock lk,
_viewMap[viewName.ns()] = view;
// Register the view in the CollectionCatalog mapping from ResourceID->namespace
- CollectionCatalog& catalog = CollectionCatalog::get(opCtx);
auto viewRid = ResourceId(RESOURCE_COLLECTION, viewName.ns());
- catalog.addResource(viewRid, viewName.ns());
+ CollectionCatalog::write(
+ opCtx, [&](CollectionCatalog& catalog) { catalog.addResource(viewRid, viewName.ns()); });
opCtx->recoveryUnit()->onRollback([this, viewName, opCtx, viewRid]() {
{
@@ -232,10 +232,12 @@ Status ViewCatalog::_createOrUpdateView(WithLock lk,
this->_viewGraphNeedsRefresh = true;
}
- CollectionCatalog& catalog = CollectionCatalog::get(opCtx);
- catalog.removeResource(viewRid, viewName.ns());
+ CollectionCatalog::write(opCtx, [&](CollectionCatalog& catalog) {
+ catalog.removeResource(viewRid, viewName.ns());
+ });
});
+
// Reload the view catalog with the changes applied.
return _reload(lk, opCtx, ViewCatalogLookupBehavior::kValidateDurableViews);
}
@@ -462,8 +464,10 @@ Status ViewCatalog::modifyView(OperationContext* opCtx,
this->_viewMap[viewName.ns()] = std::move(definition);
}
auto viewRid = ResourceId(RESOURCE_COLLECTION, viewName.ns());
- CollectionCatalog& catalog = CollectionCatalog::get(opCtx);
- catalog.addResource(viewRid, viewName.ns());
+
+ CollectionCatalog::write(opCtx, [&](CollectionCatalog& catalog) {
+ catalog.addResource(viewRid, viewName.ns());
+ });
});
return _createOrUpdateView(lk,
@@ -502,15 +506,17 @@ Status ViewCatalog::dropView(OperationContext* opCtx, const NamespaceString& vie
_viewGraph.remove(savedDefinition.name());
_viewMap.erase(viewName.ns());
- CollectionCatalog& catalog = CollectionCatalog::get(opCtx);
auto viewRid = ResourceId(RESOURCE_COLLECTION, viewName.ns());
- catalog.removeResource(viewRid, viewName.ns());
+ CollectionCatalog::write(
+ opCtx, [&](CollectionCatalog& catalog) { catalog.removeResource(viewRid, viewName.ns()); });
opCtx->recoveryUnit()->onRollback([this, viewName, savedDefinition, opCtx, viewRid]() {
this->_viewGraphNeedsRefresh = true;
this->_viewMap[viewName.ns()] = std::make_shared<ViewDefinition>(savedDefinition);
- CollectionCatalog& catalog = CollectionCatalog::get(opCtx);
- catalog.addResource(viewRid, viewName.ns());
+
+ CollectionCatalog::write(opCtx, [&](CollectionCatalog& catalog) {
+ catalog.addResource(viewRid, viewName.ns());
+ });
});
// Reload the view catalog with the changes applied.
diff --git a/src/mongo/db/views/view_catalog_test.cpp b/src/mongo/db/views/view_catalog_test.cpp
index d84ed428c4b..c2d7367db3e 100644
--- a/src/mongo/db/views/view_catalog_test.cpp
+++ b/src/mongo/db/views/view_catalog_test.cpp
@@ -513,8 +513,8 @@ TEST_F(ViewCatalogFixture, LookupRIDExistingView) {
ASSERT_OK(createView(operationContext(), viewName, viewOn, emptyPipeline, emptyCollation));
auto resourceID = ResourceId(RESOURCE_COLLECTION, "db.view"_sd);
- auto& collectionCatalog = CollectionCatalog::get(operationContext());
- ASSERT(collectionCatalog.lookupResourceName(resourceID).get() == "db.view");
+ auto collectionCatalog = CollectionCatalog::get(operationContext());
+ ASSERT(collectionCatalog->lookupResourceName(resourceID).get() == "db.view");
}
TEST_F(ViewCatalogFixture, LookupRIDExistingViewRollback) {
@@ -533,8 +533,8 @@ TEST_F(ViewCatalogFixture, LookupRIDExistingViewRollback) {
operationContext(), viewName, viewOn, emptyPipeline, emptyCollation));
}
auto resourceID = ResourceId(RESOURCE_COLLECTION, "db.view"_sd);
- auto& collectionCatalog = CollectionCatalog::get(operationContext());
- ASSERT(!collectionCatalog.lookupResourceName(resourceID));
+ auto collectionCatalog = CollectionCatalog::get(operationContext());
+ ASSERT(!collectionCatalog->lookupResourceName(resourceID));
}
TEST_F(ViewCatalogFixture, LookupRIDAfterDrop) {
@@ -545,8 +545,8 @@ TEST_F(ViewCatalogFixture, LookupRIDAfterDrop) {
ASSERT_OK(dropView(operationContext(), viewName));
auto resourceID = ResourceId(RESOURCE_COLLECTION, "db.view"_sd);
- auto& collectionCatalog = CollectionCatalog::get(operationContext());
- ASSERT(!collectionCatalog.lookupResourceName(resourceID));
+ auto collectionCatalog = CollectionCatalog::get(operationContext());
+ ASSERT(!collectionCatalog->lookupResourceName(resourceID));
}
TEST_F(ViewCatalogFixture, LookupRIDAfterDropRollback) {
@@ -554,11 +554,11 @@ TEST_F(ViewCatalogFixture, LookupRIDAfterDropRollback) {
const NamespaceString viewOn("db.coll");
auto resourceID = ResourceId(RESOURCE_COLLECTION, "db.view"_sd);
- auto& collectionCatalog = CollectionCatalog::get(operationContext());
{
WriteUnitOfWork wunit(operationContext());
ASSERT_OK(createView(operationContext(), viewName, viewOn, emptyPipeline, emptyCollation));
- ASSERT(collectionCatalog.lookupResourceName(resourceID).get() == viewName.ns());
+ ASSERT(CollectionCatalog::get(operationContext())->lookupResourceName(resourceID).get() ==
+ viewName.ns());
wunit.commit();
}
@@ -574,7 +574,8 @@ TEST_F(ViewCatalogFixture, LookupRIDAfterDropRollback) {
ASSERT_OK(getViewCatalog()->dropView(operationContext(), viewName));
}
- ASSERT(collectionCatalog.lookupResourceName(resourceID).get() == viewName.ns());
+ ASSERT(CollectionCatalog::get(operationContext())->lookupResourceName(resourceID).get() ==
+ viewName.ns());
}
TEST_F(ViewCatalogFixture, LookupRIDAfterModify) {
@@ -582,10 +583,10 @@ TEST_F(ViewCatalogFixture, LookupRIDAfterModify) {
const NamespaceString viewOn("db.coll");
auto resourceID = ResourceId(RESOURCE_COLLECTION, "db.view"_sd);
- auto& collectionCatalog = CollectionCatalog::get(operationContext());
ASSERT_OK(createView(operationContext(), viewName, viewOn, emptyPipeline, emptyCollation));
ASSERT_OK(modifyView(operationContext(), viewName, viewOn, emptyPipeline));
- ASSERT(collectionCatalog.lookupResourceName(resourceID).get() == viewName.ns());
+ ASSERT(CollectionCatalog::get(operationContext())->lookupResourceName(resourceID).get() ==
+ viewName.ns());
}
TEST_F(ViewCatalogFixture, LookupRIDAfterModifyRollback) {
@@ -593,11 +594,11 @@ TEST_F(ViewCatalogFixture, LookupRIDAfterModifyRollback) {
const NamespaceString viewOn("db.coll");
auto resourceID = ResourceId(RESOURCE_COLLECTION, "db.view"_sd);
- auto& collectionCatalog = CollectionCatalog::get(operationContext());
{
WriteUnitOfWork wunit(operationContext());
ASSERT_OK(createView(operationContext(), viewName, viewOn, emptyPipeline, emptyCollation));
- ASSERT(collectionCatalog.lookupResourceName(resourceID).get() == viewName.ns());
+ ASSERT(CollectionCatalog::get(operationContext())->lookupResourceName(resourceID).get() ==
+ viewName.ns());
wunit.commit();
}
{
@@ -611,9 +612,11 @@ TEST_F(ViewCatalogFixture, LookupRIDAfterModifyRollback) {
WriteUnitOfWork wunit(operationContext());
ASSERT_OK(
getViewCatalog()->modifyView(operationContext(), viewName, viewOn, emptyPipeline));
- ASSERT(collectionCatalog.lookupResourceName(resourceID).get() == viewName.ns());
+ ASSERT(CollectionCatalog::get(operationContext())->lookupResourceName(resourceID).get() ==
+ viewName.ns());
}
- ASSERT(collectionCatalog.lookupResourceName(resourceID).get() == viewName.ns());
+ ASSERT(CollectionCatalog::get(operationContext())->lookupResourceName(resourceID).get() ==
+ viewName.ns());
}
TEST_F(ViewCatalogFixture, CreateViewThenDropAndLookup) {
diff --git a/src/mongo/dbtests/catalogtests.cpp b/src/mongo/dbtests/catalogtests.cpp
index 460e0904e82..e980a010e3f 100644
--- a/src/mongo/dbtests/catalogtests.cpp
+++ b/src/mongo/dbtests/catalogtests.cpp
@@ -46,7 +46,7 @@ namespace mongo {
namespace {
bool collectionExists(OperationContext* opCtx, NamespaceString nss) {
- return CollectionCatalog::get(opCtx).lookupCollectionByNamespace(opCtx, nss) != nullptr;
+ return CollectionCatalog::get(opCtx)->lookupCollectionByNamespace(opCtx, nss) != nullptr;
}
class ConcurrentCreateCollectionTest {
diff --git a/src/mongo/dbtests/counttests.cpp b/src/mongo/dbtests/counttests.cpp
index 836ca016f26..4f89a1cce3b 100644
--- a/src/mongo/dbtests/counttests.cpp
+++ b/src/mongo/dbtests/counttests.cpp
@@ -52,7 +52,7 @@ public:
WriteUnitOfWork wunit(&_opCtx);
auto collection =
- CollectionCatalog::get(&_opCtx).lookupCollectionByNamespaceForMetadataWrite(
+ CollectionCatalog::get(&_opCtx)->lookupCollectionByNamespaceForMetadataWrite(
&_opCtx, CollectionCatalog::LifetimeMode::kManagedInWriteUnitOfWork, nss());
if (collection) {
_database->dropCollection(&_opCtx, nss()).transitional_ignore();
diff --git a/src/mongo/dbtests/dbtests.cpp b/src/mongo/dbtests/dbtests.cpp
index 92e3cccd9c5..c083e433558 100644
--- a/src/mongo/dbtests/dbtests.cpp
+++ b/src/mongo/dbtests/dbtests.cpp
@@ -104,7 +104,7 @@ Status createIndexFromSpec(OperationContext* opCtx, StringData ns, const BSONObj
Collection* coll;
{
WriteUnitOfWork wunit(opCtx);
- coll = CollectionCatalog::get(opCtx).lookupCollectionByNamespaceForMetadataWrite(
+ coll = CollectionCatalog::get(opCtx)->lookupCollectionByNamespaceForMetadataWrite(
opCtx, CollectionCatalog::LifetimeMode::kInplace, NamespaceString(ns));
if (!coll) {
coll = autoDb.getDb()->createCollection(opCtx, NamespaceString(ns));
diff --git a/src/mongo/dbtests/dbtests.h b/src/mongo/dbtests/dbtests.h
index 310b8235796..37179dc2ff8 100644
--- a/src/mongo/dbtests/dbtests.h
+++ b/src/mongo/dbtests/dbtests.h
@@ -80,7 +80,7 @@ public:
}
CollectionPtr getCollection() const {
- return CollectionCatalog::get(_opCtx).lookupCollectionByNamespace(_opCtx, _nss);
+ return CollectionCatalog::get(_opCtx)->lookupCollectionByNamespace(_opCtx, _nss);
}
private:
diff --git a/src/mongo/dbtests/pdfiletests.cpp b/src/mongo/dbtests/pdfiletests.cpp
index 5fd4540a6c3..8535e81942c 100644
--- a/src/mongo/dbtests/pdfiletests.cpp
+++ b/src/mongo/dbtests/pdfiletests.cpp
@@ -60,7 +60,7 @@ protected:
return NamespaceString("unittests.pdfiletests.Insert");
}
CollectionPtr collection() {
- return CollectionCatalog::get(&_opCtx).lookupCollectionByNamespace(&_opCtx, nss());
+ return CollectionCatalog::get(&_opCtx)->lookupCollectionByNamespace(&_opCtx, nss());
}
const ServiceContext::UniqueOperationContext _opCtxPtr = cc().makeOperationContext();
@@ -76,7 +76,7 @@ public:
BSONObj x = BSON("x" << 1);
ASSERT(x["_id"].type() == 0);
CollectionPtr coll =
- CollectionCatalog::get(&_opCtx).lookupCollectionByNamespace(&_opCtx, nss());
+ CollectionCatalog::get(&_opCtx)->lookupCollectionByNamespace(&_opCtx, nss());
if (!coll) {
coll = _context.db()->createCollection(&_opCtx, nss());
}
diff --git a/src/mongo/dbtests/plan_executor_invalidation_test.cpp b/src/mongo/dbtests/plan_executor_invalidation_test.cpp
index 7d52881cc51..bc84c166615 100644
--- a/src/mongo/dbtests/plan_executor_invalidation_test.cpp
+++ b/src/mongo/dbtests/plan_executor_invalidation_test.cpp
@@ -118,14 +118,14 @@ public:
}
const CollectionPtr& collection() const {
- _coll = CollectionCatalog::get(&_opCtx).lookupCollectionByNamespace(&_opCtx, nss);
+ _coll = CollectionCatalog::get(&_opCtx)->lookupCollectionByNamespace(&_opCtx, nss);
return _coll;
}
void truncateCollection() const {
WriteUnitOfWork wunit(&_opCtx);
auto collection =
- CollectionCatalog::get(&_opCtx).lookupCollectionByNamespaceForMetadataWrite(
+ CollectionCatalog::get(&_opCtx)->lookupCollectionByNamespaceForMetadataWrite(
&_opCtx, CollectionCatalog::LifetimeMode::kManagedInWriteUnitOfWork, nss);
ASSERT_OK(collection->truncate(&_opCtx));
wunit.commit();
diff --git a/src/mongo/dbtests/query_plan_executor.cpp b/src/mongo/dbtests/query_plan_executor.cpp
index 83fbe89282b..1f8751a7d9e 100644
--- a/src/mongo/dbtests/query_plan_executor.cpp
+++ b/src/mongo/dbtests/query_plan_executor.cpp
@@ -178,7 +178,7 @@ protected:
private:
const IndexDescriptor* getIndex(Database* db, const BSONObj& obj) {
CollectionPtr collection =
- CollectionCatalog::get(&_opCtx).lookupCollectionByNamespace(&_opCtx, nss);
+ CollectionCatalog::get(&_opCtx)->lookupCollectionByNamespace(&_opCtx, nss);
std::vector<const IndexDescriptor*> indexes;
collection->getIndexCatalog()->findIndexesByKeyPattern(&_opCtx, obj, false, &indexes);
ASSERT_LTE(indexes.size(), 1U);
@@ -312,7 +312,7 @@ protected:
insert(BSON("_id" << 3 << "a" << 3));
insert(BSON("_id" << 4 << "a" << 4));
- return CollectionCatalog::get(&_opCtx).lookupCollectionByNamespace(&_opCtx, nss);
+ return CollectionCatalog::get(&_opCtx)->lookupCollectionByNamespace(&_opCtx, nss);
}
/**
diff --git a/src/mongo/dbtests/query_stage_count_scan.cpp b/src/mongo/dbtests/query_stage_count_scan.cpp
index 91f0d847ad4..dc9a544bede 100644
--- a/src/mongo/dbtests/query_stage_count_scan.cpp
+++ b/src/mongo/dbtests/query_stage_count_scan.cpp
@@ -90,7 +90,7 @@ public:
}
CollectionPtr getCollection() {
- return CollectionCatalog::get(&_opCtx).lookupCollectionByNamespace(&_opCtx, ns());
+ return CollectionCatalog::get(&_opCtx)->lookupCollectionByNamespace(&_opCtx, ns());
}
const IndexDescriptor* getIndex(Database* db, const BSONObj& obj) {
diff --git a/src/mongo/dbtests/query_stage_fetch.cpp b/src/mongo/dbtests/query_stage_fetch.cpp
index 87ff271b8c5..7d09edc25a1 100644
--- a/src/mongo/dbtests/query_stage_fetch.cpp
+++ b/src/mongo/dbtests/query_stage_fetch.cpp
@@ -103,7 +103,7 @@ public:
dbtests::WriteContextForTests ctx(&_opCtx, ns());
Database* db = ctx.db();
CollectionPtr coll =
- CollectionCatalog::get(&_opCtx).lookupCollectionByNamespace(&_opCtx, nss());
+ CollectionCatalog::get(&_opCtx)->lookupCollectionByNamespace(&_opCtx, nss());
if (!coll) {
WriteUnitOfWork wuow(&_opCtx);
coll = db->createCollection(&_opCtx, nss());
@@ -170,7 +170,7 @@ public:
OldClientContext ctx(&_opCtx, ns());
Database* db = ctx.db();
CollectionPtr coll =
- CollectionCatalog::get(&_opCtx).lookupCollectionByNamespace(&_opCtx, nss());
+ CollectionCatalog::get(&_opCtx)->lookupCollectionByNamespace(&_opCtx, nss());
if (!coll) {
WriteUnitOfWork wuow(&_opCtx);
coll = db->createCollection(&_opCtx, nss());
diff --git a/src/mongo/dbtests/query_stage_merge_sort.cpp b/src/mongo/dbtests/query_stage_merge_sort.cpp
index 1bb4df5d87e..8d64f664e85 100644
--- a/src/mongo/dbtests/query_stage_merge_sort.cpp
+++ b/src/mongo/dbtests/query_stage_merge_sort.cpp
@@ -149,7 +149,7 @@ public:
dbtests::WriteContextForTests ctx(&_opCtx, ns());
Database* db = ctx.db();
CollectionPtr coll =
- CollectionCatalog::get(&_opCtx).lookupCollectionByNamespace(&_opCtx, nss());
+ CollectionCatalog::get(&_opCtx)->lookupCollectionByNamespace(&_opCtx, nss());
if (!coll) {
WriteUnitOfWork wuow(&_opCtx);
coll = db->createCollection(&_opCtx, nss());
@@ -219,7 +219,7 @@ public:
dbtests::WriteContextForTests ctx(&_opCtx, ns());
Database* db = ctx.db();
CollectionPtr coll =
- CollectionCatalog::get(&_opCtx).lookupCollectionByNamespace(&_opCtx, nss());
+ CollectionCatalog::get(&_opCtx)->lookupCollectionByNamespace(&_opCtx, nss());
if (!coll) {
WriteUnitOfWork wuow(&_opCtx);
coll = db->createCollection(&_opCtx, nss());
@@ -288,7 +288,7 @@ public:
dbtests::WriteContextForTests ctx(&_opCtx, ns());
Database* db = ctx.db();
CollectionPtr coll =
- CollectionCatalog::get(&_opCtx).lookupCollectionByNamespace(&_opCtx, nss());
+ CollectionCatalog::get(&_opCtx)->lookupCollectionByNamespace(&_opCtx, nss());
if (!coll) {
WriteUnitOfWork wuow(&_opCtx);
coll = db->createCollection(&_opCtx, nss());
@@ -358,7 +358,7 @@ public:
dbtests::WriteContextForTests ctx(&_opCtx, ns());
Database* db = ctx.db();
CollectionPtr coll =
- CollectionCatalog::get(&_opCtx).lookupCollectionByNamespace(&_opCtx, nss());
+ CollectionCatalog::get(&_opCtx)->lookupCollectionByNamespace(&_opCtx, nss());
if (!coll) {
WriteUnitOfWork wuow(&_opCtx);
coll = db->createCollection(&_opCtx, nss());
@@ -432,7 +432,7 @@ public:
dbtests::WriteContextForTests ctx(&_opCtx, ns());
Database* db = ctx.db();
CollectionPtr coll =
- CollectionCatalog::get(&_opCtx).lookupCollectionByNamespace(&_opCtx, nss());
+ CollectionCatalog::get(&_opCtx)->lookupCollectionByNamespace(&_opCtx, nss());
if (!coll) {
WriteUnitOfWork wuow(&_opCtx);
coll = db->createCollection(&_opCtx, nss());
@@ -500,7 +500,7 @@ public:
dbtests::WriteContextForTests ctx(&_opCtx, ns());
Database* db = ctx.db();
CollectionPtr coll =
- CollectionCatalog::get(&_opCtx).lookupCollectionByNamespace(&_opCtx, nss());
+ CollectionCatalog::get(&_opCtx)->lookupCollectionByNamespace(&_opCtx, nss());
if (!coll) {
WriteUnitOfWork wuow(&_opCtx);
coll = db->createCollection(&_opCtx, nss());
@@ -558,7 +558,7 @@ public:
dbtests::WriteContextForTests ctx(&_opCtx, ns());
Database* db = ctx.db();
CollectionPtr coll =
- CollectionCatalog::get(&_opCtx).lookupCollectionByNamespace(&_opCtx, nss());
+ CollectionCatalog::get(&_opCtx)->lookupCollectionByNamespace(&_opCtx, nss());
if (!coll) {
WriteUnitOfWork wuow(&_opCtx);
coll = db->createCollection(&_opCtx, nss());
@@ -678,7 +678,7 @@ public:
dbtests::WriteContextForTests ctx(&_opCtx, ns());
Database* db = ctx.db();
CollectionPtr coll =
- CollectionCatalog::get(&_opCtx).lookupCollectionByNamespace(&_opCtx, nss());
+ CollectionCatalog::get(&_opCtx)->lookupCollectionByNamespace(&_opCtx, nss());
if (!coll) {
WriteUnitOfWork wuow(&_opCtx);
coll = db->createCollection(&_opCtx, nss());
@@ -781,7 +781,7 @@ public:
dbtests::WriteContextForTests ctx(&_opCtx, ns());
Database* db = ctx.db();
CollectionPtr coll =
- CollectionCatalog::get(&_opCtx).lookupCollectionByNamespace(&_opCtx, nss());
+ CollectionCatalog::get(&_opCtx)->lookupCollectionByNamespace(&_opCtx, nss());
if (!coll) {
WriteUnitOfWork wuow(&_opCtx);
coll = db->createCollection(&_opCtx, nss());
@@ -853,7 +853,7 @@ public:
dbtests::WriteContextForTests ctx(&_opCtx, ns());
Database* db = ctx.db();
CollectionPtr coll =
- CollectionCatalog::get(&_opCtx).lookupCollectionByNamespace(&_opCtx, nss());
+ CollectionCatalog::get(&_opCtx)->lookupCollectionByNamespace(&_opCtx, nss());
if (!coll) {
WriteUnitOfWork wuow(&_opCtx);
coll = db->createCollection(&_opCtx, nss());
diff --git a/src/mongo/dbtests/query_stage_sort.cpp b/src/mongo/dbtests/query_stage_sort.cpp
index c35511ab247..918c6b127ed 100644
--- a/src/mongo/dbtests/query_stage_sort.cpp
+++ b/src/mongo/dbtests/query_stage_sort.cpp
@@ -260,7 +260,7 @@ public:
dbtests::WriteContextForTests ctx(&_opCtx, ns());
Database* db = ctx.db();
CollectionPtr coll =
- CollectionCatalog::get(&_opCtx).lookupCollectionByNamespace(&_opCtx, nss());
+ CollectionCatalog::get(&_opCtx)->lookupCollectionByNamespace(&_opCtx, nss());
if (!coll) {
WriteUnitOfWork wuow(&_opCtx);
coll = db->createCollection(&_opCtx, nss());
@@ -283,7 +283,7 @@ public:
dbtests::WriteContextForTests ctx(&_opCtx, ns());
Database* db = ctx.db();
CollectionPtr coll =
- CollectionCatalog::get(&_opCtx).lookupCollectionByNamespace(&_opCtx, nss());
+ CollectionCatalog::get(&_opCtx)->lookupCollectionByNamespace(&_opCtx, nss());
if (!coll) {
WriteUnitOfWork wuow(&_opCtx);
coll = db->createCollection(&_opCtx, nss());
@@ -315,7 +315,7 @@ public:
dbtests::WriteContextForTests ctx(&_opCtx, ns());
Database* db = ctx.db();
CollectionPtr coll =
- CollectionCatalog::get(&_opCtx).lookupCollectionByNamespace(&_opCtx, nss());
+ CollectionCatalog::get(&_opCtx)->lookupCollectionByNamespace(&_opCtx, nss());
if (!coll) {
WriteUnitOfWork wuow(&_opCtx);
coll = db->createCollection(&_opCtx, nss());
@@ -341,7 +341,7 @@ public:
dbtests::WriteContextForTests ctx(&_opCtx, ns());
Database* db = ctx.db();
CollectionPtr coll =
- CollectionCatalog::get(&_opCtx).lookupCollectionByNamespace(&_opCtx, nss());
+ CollectionCatalog::get(&_opCtx)->lookupCollectionByNamespace(&_opCtx, nss());
if (!coll) {
WriteUnitOfWork wuow(&_opCtx);
coll = db->createCollection(&_opCtx, nss());
@@ -456,7 +456,7 @@ public:
dbtests::WriteContextForTests ctx(&_opCtx, ns());
Database* db = ctx.db();
CollectionPtr coll =
- CollectionCatalog::get(&_opCtx).lookupCollectionByNamespace(&_opCtx, nss());
+ CollectionCatalog::get(&_opCtx)->lookupCollectionByNamespace(&_opCtx, nss());
if (!coll) {
WriteUnitOfWork wuow(&_opCtx);
coll = db->createCollection(&_opCtx, nss());
@@ -560,7 +560,7 @@ public:
dbtests::WriteContextForTests ctx(&_opCtx, ns());
Database* db = ctx.db();
CollectionPtr coll =
- CollectionCatalog::get(&_opCtx).lookupCollectionByNamespace(&_opCtx, nss());
+ CollectionCatalog::get(&_opCtx)->lookupCollectionByNamespace(&_opCtx, nss());
if (!coll) {
WriteUnitOfWork wuow(&_opCtx);
coll = db->createCollection(&_opCtx, nss());
diff --git a/src/mongo/dbtests/query_stage_update.cpp b/src/mongo/dbtests/query_stage_update.cpp
index 93fb705f8ea..c55d7bd17f1 100644
--- a/src/mongo/dbtests/query_stage_update.cpp
+++ b/src/mongo/dbtests/query_stage_update.cpp
@@ -276,7 +276,7 @@ public:
OpDebug* opDebug = &curOp.debug();
UpdateDriver driver(_expCtx);
CollectionPtr coll =
- CollectionCatalog::get(&_opCtx).lookupCollectionByNamespace(&_opCtx, nss);
+ CollectionCatalog::get(&_opCtx)->lookupCollectionByNamespace(&_opCtx, nss);
ASSERT(coll);
// Get the RecordIds that would be returned by an in-order scan.
diff --git a/src/mongo/dbtests/querytests.cpp b/src/mongo/dbtests/querytests.cpp
index 377aee1f160..18a41bfdb10 100644
--- a/src/mongo/dbtests/querytests.cpp
+++ b/src/mongo/dbtests/querytests.cpp
@@ -68,7 +68,7 @@ public:
WriteUnitOfWork wunit(&_opCtx);
_database = _context.db();
auto collection =
- CollectionCatalog::get(&_opCtx).lookupCollectionByNamespace(&_opCtx, nss());
+ CollectionCatalog::get(&_opCtx)->lookupCollectionByNamespace(&_opCtx, nss());
if (collection) {
_database->dropCollection(&_opCtx, nss()).transitional_ignore();
}
@@ -228,7 +228,7 @@ public:
{
WriteUnitOfWork wunit(&_opCtx);
Database* db = ctx.db();
- if (CollectionCatalog::get(&_opCtx).lookupCollectionByNamespace(&_opCtx, nss())) {
+ if (CollectionCatalog::get(&_opCtx)->lookupCollectionByNamespace(&_opCtx, nss())) {
_collection = nullptr;
db->dropCollection(&_opCtx, nss()).transitional_ignore();
}
diff --git a/src/mongo/dbtests/repltests.cpp b/src/mongo/dbtests/repltests.cpp
index c6808079dd2..6266eeb7336 100644
--- a/src/mongo/dbtests/repltests.cpp
+++ b/src/mongo/dbtests/repltests.cpp
@@ -129,7 +129,7 @@ public:
WriteUnitOfWork wuow(&_opCtx);
CollectionPtr c =
- CollectionCatalog::get(&_opCtx).lookupCollectionByNamespace(&_opCtx, nss());
+ CollectionCatalog::get(&_opCtx)->lookupCollectionByNamespace(&_opCtx, nss());
if (!c) {
c = ctx.db()->createCollection(&_opCtx, nss());
}
@@ -202,7 +202,7 @@ protected:
OldClientContext ctx(&_opCtx, ns());
Database* db = ctx.db();
CollectionPtr coll =
- CollectionCatalog::get(&_opCtx).lookupCollectionByNamespace(&_opCtx, nss());
+ CollectionCatalog::get(&_opCtx)->lookupCollectionByNamespace(&_opCtx, nss());
if (!coll) {
WriteUnitOfWork wunit(&_opCtx);
coll = db->createCollection(&_opCtx, nss());
@@ -260,7 +260,7 @@ protected:
WriteUnitOfWork wunit(&_opCtx);
Database* db = ctx.db();
Collection* coll =
- CollectionCatalog::get(&_opCtx).lookupCollectionByNamespaceForMetadataWrite(
+ CollectionCatalog::get(&_opCtx)->lookupCollectionByNamespaceForMetadataWrite(
&_opCtx, CollectionCatalog::LifetimeMode::kInplace, nss);
if (!coll) {
coll = db->createCollection(&_opCtx, nss);
@@ -276,7 +276,7 @@ protected:
WriteUnitOfWork wunit(&_opCtx);
Database* db = ctx.db();
CollectionPtr coll =
- CollectionCatalog::get(&_opCtx).lookupCollectionByNamespace(&_opCtx, nss());
+ CollectionCatalog::get(&_opCtx)->lookupCollectionByNamespace(&_opCtx, nss());
if (!coll) {
coll = db->createCollection(&_opCtx, nss());
}
diff --git a/src/mongo/dbtests/rollbacktests.cpp b/src/mongo/dbtests/rollbacktests.cpp
index 80c40965418..3144d699bca 100644
--- a/src/mongo/dbtests/rollbacktests.cpp
+++ b/src/mongo/dbtests/rollbacktests.cpp
@@ -62,8 +62,8 @@ void dropDatabase(OperationContext* opCtx, const NamespaceString& nss) {
}
}
bool collectionExists(OperationContext* opCtx, OldClientContext* ctx, const string& ns) {
- return (bool)CollectionCatalog::get(opCtx).lookupCollectionByNamespace(opCtx,
- NamespaceString(ns));
+ return (bool)CollectionCatalog::get(opCtx)->lookupCollectionByNamespace(opCtx,
+ NamespaceString(ns));
}
void createCollection(OperationContext* opCtx, const NamespaceString& nss) {
@@ -90,12 +90,12 @@ Status truncateCollection(OperationContext* opCtx, const NamespaceString& nss) {
}
void insertRecord(OperationContext* opCtx, const NamespaceString& nss, const BSONObj& data) {
- auto coll = CollectionCatalog::get(opCtx).lookupCollectionByNamespace(opCtx, nss);
+ auto coll = CollectionCatalog::get(opCtx)->lookupCollectionByNamespace(opCtx, nss);
OpDebug* const nullOpDebug = nullptr;
ASSERT_OK(coll->insertDocument(opCtx, InsertStatement(data), nullOpDebug, false));
}
void assertOnlyRecord(OperationContext* opCtx, const NamespaceString& nss, const BSONObj& data) {
- auto coll = CollectionCatalog::get(opCtx).lookupCollectionByNamespace(opCtx, nss);
+ auto coll = CollectionCatalog::get(opCtx)->lookupCollectionByNamespace(opCtx, nss);
auto cursor = coll->getCursor(opCtx);
auto record = cursor->next();
@@ -105,15 +105,15 @@ void assertOnlyRecord(OperationContext* opCtx, const NamespaceString& nss, const
ASSERT(!cursor->next());
}
void assertEmpty(OperationContext* opCtx, const NamespaceString& nss) {
- auto coll = CollectionCatalog::get(opCtx).lookupCollectionByNamespace(opCtx, nss);
+ auto coll = CollectionCatalog::get(opCtx)->lookupCollectionByNamespace(opCtx, nss);
ASSERT(!coll->getCursor(opCtx)->next());
}
bool indexExists(OperationContext* opCtx, const NamespaceString& nss, const string& idxName) {
- auto coll = CollectionCatalog::get(opCtx).lookupCollectionByNamespace(opCtx, nss);
+ auto coll = CollectionCatalog::get(opCtx)->lookupCollectionByNamespace(opCtx, nss);
return coll->getIndexCatalog()->findIndexByName(opCtx, idxName, true) != nullptr;
}
bool indexReady(OperationContext* opCtx, const NamespaceString& nss, const string& idxName) {
- auto coll = CollectionCatalog::get(opCtx).lookupCollectionByNamespace(opCtx, nss);
+ auto coll = CollectionCatalog::get(opCtx)->lookupCollectionByNamespace(opCtx, nss);
return coll->getIndexCatalog()->findIndexByName(opCtx, idxName, false) != nullptr;
}
size_t getNumIndexEntries(OperationContext* opCtx,
@@ -121,7 +121,7 @@ size_t getNumIndexEntries(OperationContext* opCtx,
const string& idxName) {
size_t numEntries = 0;
- auto coll = CollectionCatalog::get(opCtx).lookupCollectionByNamespace(opCtx, nss);
+ auto coll = CollectionCatalog::get(opCtx)->lookupCollectionByNamespace(opCtx, nss);
const IndexCatalog* catalog = coll->getIndexCatalog();
auto desc = catalog->findIndexByName(opCtx, idxName, false);
diff --git a/src/mongo/dbtests/storage_timestamp_tests.cpp b/src/mongo/dbtests/storage_timestamp_tests.cpp
index 83c078f340f..8df1072c25e 100644
--- a/src/mongo/dbtests/storage_timestamp_tests.cpp
+++ b/src/mongo/dbtests/storage_timestamp_tests.cpp
@@ -2559,7 +2559,7 @@ public:
// Assert the 'a_1' and `b_1` indexes becomes ready at the last oplog entry time.
RecordId renamedCatalogId = CollectionCatalog::get(_opCtx)
- .lookupCollectionByNamespace(_opCtx, renamedNss)
+ ->lookupCollectionByNamespace(_opCtx, renamedNss)
->getCatalogId();
ASSERT_TRUE(getIndexMetaData(
getMetaDataAtTime(durableCatalog, renamedCatalogId, indexCommitTs), "a_1")
diff --git a/src/mongo/s/commands/cluster_profile_cmd.cpp b/src/mongo/s/commands/cluster_profile_cmd.cpp
index 299e4af3817..85a20f1ab24 100644
--- a/src/mongo/s/commands/cluster_profile_cmd.cpp
+++ b/src/mongo/s/commands/cluster_profile_cmd.cpp
@@ -64,7 +64,7 @@ protected:
"values",
profilingLevel == -1 || profilingLevel == 0);
- const auto oldSettings = CollectionCatalog::get(opCtx).getDatabaseProfileSettings(dbName);
+ const auto oldSettings = CollectionCatalog::get(opCtx)->getDatabaseProfileSettings(dbName);
if (auto filterOrUnset = request.getFilter()) {
auto newSettings = oldSettings;
@@ -73,7 +73,9 @@ protected:
} else {
newSettings.filter = nullptr;
}
- CollectionCatalog::get(opCtx).setDatabaseProfileSettings(dbName, newSettings);
+ CollectionCatalog::write(opCtx, [&](CollectionCatalog& catalog) {
+ catalog.setDatabaseProfileSettings(dbName, newSettings);
+ });
}
return oldSettings;