summaryrefslogtreecommitdiff
diff options
context:
space:
mode:
authorTommaso Tocci <tommaso.tocci@mongodb.com>2020-07-21 17:56:40 +0200
committerEvergreen Agent <no-reply@evergreen.mongodb.com>2020-07-29 09:23:42 +0000
commitc7ac54720b88f97ad016fadd80b73d21dc45f5c2 (patch)
treebf460362da4b6b991ad25a24f2abec9b6498104f
parent9950c7db67a7034e700a33c983e991914ecafd3d (diff)
downloadmongo-c7ac54720b88f97ad016fadd80b73d21dc45f5c2.tar.gz
SERVER-48992 Implement database cache on top of ReadThroughCache
-rw-r--r--src/mongo/db/s/README.md1
-rw-r--r--src/mongo/db/s/sharding_initialization_mongod.cpp4
-rw-r--r--src/mongo/db/s/sharding_mongod_test_fixture.cpp6
-rw-r--r--src/mongo/s/SConscript1
-rw-r--r--src/mongo/s/catalog_cache.cpp247
-rw-r--r--src/mongo/s/catalog_cache.h48
-rw-r--r--src/mongo/s/mongos_main.cpp4
-rw-r--r--src/mongo/s/sharding_router_test_fixture.cpp17
-rw-r--r--src/mongo/s/write_ops/chunk_manager_targeter.cpp3
9 files changed, 128 insertions, 203 deletions
diff --git a/src/mongo/db/s/README.md b/src/mongo/db/s/README.md
index 1b3cd84a71e..b4737b57764 100644
--- a/src/mongo/db/s/README.md
+++ b/src/mongo/db/s/README.md
@@ -112,7 +112,6 @@ Methods that will mark routing table cache information as stale (sharded collect
Methods that will mark routing table cache information as stale (database).
* [onStaleDatabaseVersion](https://github.com/mongodb/mongo/blob/62d9485657717bf61fbb870cb3d09b52b1a614dd/src/mongo/s/catalog_cache.h#L197-L205)
-* [invalidateDatabaseEntry](https://github.com/mongodb/mongo/blob/62d9485657717bf61fbb870cb3d09b52b1a614dd/src/mongo/s/catalog_cache.h#L256-L260)
* [purgeDatabase](https://github.com/mongodb/mongo/blob/62d9485657717bf61fbb870cb3d09b52b1a614dd/src/mongo/s/catalog_cache.h#L282-L286)
## Shard versioning and database versioning
diff --git a/src/mongo/db/s/sharding_initialization_mongod.cpp b/src/mongo/db/s/sharding_initialization_mongod.cpp
index f8a6c671e5c..af391226e71 100644
--- a/src/mongo/db/s/sharding_initialization_mongod.cpp
+++ b/src/mongo/db/s/sharding_initialization_mongod.cpp
@@ -559,8 +559,8 @@ void initializeGlobalShardingStateForMongoD(OperationContext* opCtx,
globalConnPool.addHook(new ShardingConnectionHook(makeEgressHooksList(service)));
- auto catalogCache =
- std::make_unique<CatalogCache>(CatalogCacheLoader::get(opCtx), catalogCacheExecutor);
+ auto catalogCache = std::make_unique<CatalogCache>(
+ service, CatalogCacheLoader::get(opCtx), catalogCacheExecutor);
// List of hooks which will be called by the ShardRegistry when it discovers a shard has been
// removed.
diff --git a/src/mongo/db/s/sharding_mongod_test_fixture.cpp b/src/mongo/db/s/sharding_mongod_test_fixture.cpp
index 911c4c1da07..68c4d291924 100644
--- a/src/mongo/db/s/sharding_mongod_test_fixture.cpp
+++ b/src/mongo/db/s/sharding_mongod_test_fixture.cpp
@@ -254,10 +254,12 @@ Status ShardingMongodTestFixture::initializeGlobalShardingStateForMongodForTest(
auto distLockManagerPtr = makeDistLockManager(std::move(distLockCatalogPtr));
_distLockManager = distLockManagerPtr.get();
+ auto catalogCache = std::make_unique<CatalogCache>(
+ getServiceContext(), CatalogCacheLoader::get(getServiceContext()), catalogCacheExecutor());
+
auto const grid = Grid::get(operationContext());
grid->init(makeShardingCatalogClient(std::move(distLockManagerPtr)),
- std::make_unique<CatalogCache>(CatalogCacheLoader::get(getServiceContext()),
- catalogCacheExecutor()),
+ std::move(catalogCache),
makeShardRegistry(configConnStr),
makeClusterCursorManager(),
makeBalancerConfiguration(),
diff --git a/src/mongo/s/SConscript b/src/mongo/s/SConscript
index 15c63de30b8..3a364a924c2 100644
--- a/src/mongo/s/SConscript
+++ b/src/mongo/s/SConscript
@@ -257,6 +257,7 @@ env.Library(
'$BUILD_DIR/mongo/db/logical_time_metadata_hook',
'$BUILD_DIR/mongo/util/concurrency/thread_pool',
'$BUILD_DIR/mongo/executor/task_executor_pool',
+ '$BUILD_DIR/mongo/util/caching',
'client/shard_interface',
'query/cluster_cursor_manager',
'sharding_routing_table',
diff --git a/src/mongo/s/catalog_cache.cpp b/src/mongo/s/catalog_cache.cpp
index e9de231f82e..de74b95e528 100644
--- a/src/mongo/s/catalog_cache.cpp
+++ b/src/mongo/s/catalog_cache.cpp
@@ -67,6 +67,7 @@ namespace {
// server is found to be inconsistent.
const int kMaxInconsistentRoutingInfoRefreshAttempts = 3;
+const int kDatabaseCacheSize = 10000;
/**
* Returns whether two shard versions have a matching epoch.
*/
@@ -139,15 +140,19 @@ std::shared_ptr<ThreadPool> CatalogCache::makeDefaultThreadPool() {
ThreadPool::Options options;
options.poolName = "CatalogCache";
options.minThreads = 0;
- options.maxThreads = 6;
+ options.maxThreads = ThreadPool::Options::kUnlimited;
auto executor = std::make_shared<ThreadPool>(std::move(options));
executor->startup();
return executor;
}
-CatalogCache::CatalogCache(CatalogCacheLoader& cacheLoader, std::shared_ptr<ThreadPool> executor)
- : _cacheLoader(cacheLoader), _executor(executor){};
+CatalogCache::CatalogCache(ServiceContext* const service,
+ CatalogCacheLoader& cacheLoader,
+ std::shared_ptr<ThreadPool> executor)
+ : _cacheLoader(cacheLoader),
+ _executor(executor),
+ _databaseCache(service, *_executor, _cacheLoader) {}
CatalogCache::~CatalogCache() = default;
@@ -158,35 +163,33 @@ StatusWith<CachedDatabaseInfo> CatalogCache::getDatabase(OperationContext* opCtx
"hold the lock during a network call, and can lead to a deadlock as described in "
"SERVER-37398.");
try {
- while (true) {
- stdx::unique_lock<Latch> ul(_mutex);
+ // TODO SERVER-49724: Make ReadThroughCache support StringData keys
+ auto dbEntry = _databaseCache.acquire(opCtx, dbName.toString());
+ if (!dbEntry) {
+ return {ErrorCodes::NamespaceNotFound,
+ str::stream() << "database " << dbName << " not found"};
+ }
+ const auto primaryShardExists =
+ Grid::get(opCtx)->shardRegistry()->getShard(opCtx, dbEntry->getPrimary()).isOK();
- auto& dbEntry = _databases[dbName];
+ if (!primaryShardExists) {
+ LOGV2_FOR_CATALOG_REFRESH(
+ 4947103,
+ 2,
+ "Invalidating cached database entry because its primary shard hasn't been found",
+ "db"_attr = dbName);
+ _databaseCache.invalidate(dbName.toString());
+ dbEntry = _databaseCache.acquire(opCtx, dbName.toString());
if (!dbEntry) {
- dbEntry = std::make_shared<DatabaseInfoEntry>();
- }
-
- if (dbEntry->needsRefresh) {
- auto refreshNotification = dbEntry->refreshCompletionNotification;
- if (!refreshNotification) {
- refreshNotification = (dbEntry->refreshCompletionNotification =
- std::make_shared<Notification<Status>>());
- _scheduleDatabaseRefresh(ul, dbName, dbEntry);
- }
-
- // Wait on the notification outside of the mutex.
- ul.unlock();
- uassertStatusOK(refreshNotification->get(opCtx));
-
- // Once the refresh is complete, loop around to get the refreshed cache entry.
- continue;
+ return {ErrorCodes::NamespaceNotFound,
+ str::stream() << "database " << dbName << " not found"};
}
-
- auto primaryShard = uassertStatusOKWithContext(
- Grid::get(opCtx)->shardRegistry()->getShard(opCtx, dbEntry->dbt->getPrimary()),
- str::stream() << "could not find the primary shard for database " << dbName);
- return {CachedDatabaseInfo(*dbEntry->dbt, std::move(primaryShard))};
}
+
+ const auto primaryShard = uassertStatusOKWithContext(
+ Grid::get(opCtx)->shardRegistry()->getShard(opCtx, dbEntry->getPrimary()),
+ str::stream() << "could not find the primary shard for database " << dbName);
+ return {CachedDatabaseInfo(*dbEntry, std::move(primaryShard))};
} catch (const DBException& ex) {
return ex.toStatus();
}
@@ -229,6 +232,14 @@ CatalogCache::RefreshResult CatalogCache::_getCollectionRoutingInfoAt(
while (true) {
const auto swDbInfo = getDatabase(opCtx, nss.db());
if (!swDbInfo.isOK()) {
+ if (swDbInfo == ErrorCodes::NamespaceNotFound) {
+ LOGV2_FOR_CATALOG_REFRESH(
+ 4947102,
+ 2,
+ "Invalidating cached collection entry because its database has been dropped",
+ "namespace"_attr = nss);
+ purgeCollection(nss);
+ }
return {swDbInfo.getStatus(), refreshActionTaken};
}
@@ -292,7 +303,8 @@ CatalogCache::RefreshResult CatalogCache::_getCollectionRoutingInfoAt(
StatusWith<CachedDatabaseInfo> CatalogCache::getDatabaseWithRefresh(OperationContext* opCtx,
StringData dbName) {
- invalidateDatabaseEntry(dbName);
+ // TODO SERVER-49724: Make ReadThroughCache support StringData keys
+ _databaseCache.invalidate(dbName.toString());
return getDatabase(opCtx, dbName);
}
@@ -324,24 +336,9 @@ StatusWith<CachedCollectionRoutingInfo> CatalogCache::getShardedCollectionRoutin
void CatalogCache::onStaleDatabaseVersion(const StringData dbName,
const DatabaseVersion& databaseVersion) {
- stdx::lock_guard<Latch> lg(_mutex);
-
- const auto itDbEntry = _databases.find(dbName);
- if (itDbEntry == _databases.end()) {
- // The database was dropped.
- return;
- } else if (itDbEntry->second->needsRefresh) {
- // Refresh has been scheduled for the database already
- return;
- } else if (!itDbEntry->second->dbt ||
- databaseVersion::equal(itDbEntry->second->dbt->getVersion(), databaseVersion)) {
- // If the versions match, the cached database info is stale, so mark it as needs refresh.
- LOGV2(22642,
- "Marking cached database entry for {db} as stale",
- "Marking cached database entry as stale",
- "db"_attr = dbName);
- itDbEntry->second->needsRefresh = true;
- }
+ // TODO SERVER-49856: Use a comparable DatabaseVersion to advance the time of the cached entry
+ // istead of invalidating it.
+ _databaseCache.invalidate(dbName.toString());
}
void CatalogCache::onStaleShardVersion(CachedCollectionRoutingInfo&& ccriToInvalidate,
@@ -440,16 +437,6 @@ void CatalogCache::checkEpochOrThrow(const NamespaceString& nss,
foundVersion.epoch() == targetCollectionVersion.epoch());
}
-void CatalogCache::invalidateDatabaseEntry(const StringData dbName) {
- stdx::lock_guard<Latch> lg(_mutex);
- auto itDbEntry = _databases.find(dbName);
- if (itDbEntry == _databases.end()) {
- // The database was dropped.
- return;
- }
- itDbEntry->second->needsRefresh = true;
-}
-
void CatalogCache::invalidateShardForShardedCollection(const NamespaceString& nss,
const ShardId& staleShardId) {
_createOrGetCollectionEntryAndMarkShardStale(nss, staleShardId);
@@ -459,30 +446,10 @@ void CatalogCache::invalidateEntriesThatReferenceShard(const ShardId& shardId) {
stdx::lock_guard<Latch> lg(_mutex);
LOGV2(22643,
- "Starting to invalidate databases and collections with data on shard {shardId}",
- "Starting to invalidate databases and collections referencing a specific shard",
+ "Starting to invalidate collections with data on shard {shardId}",
+ "Starting to invalidate collections referencing a specific shard",
"shardId"_attr = shardId);
- // Invalidate databases with this shard as their primary.
- for (const auto& [dbNs, dbInfoEntry] : _databases) {
- LOGV2_DEBUG(22644,
- 3,
- "Checking if database {db} has primary shard: {primaryShardId}",
- "Checking if database matches primary shard",
- "db"_attr = dbNs,
- "primaryShardId"_attr = shardId);
- if (!dbInfoEntry->needsRefresh && dbInfoEntry->dbt->getPrimary() == shardId) {
- LOGV2_DEBUG(22645,
- 3,
- "Invalidating cache entry for database {db} that has primary shard "
- "{primaryShardId}",
- "Invalidating database cache entry",
- "db"_attr = dbNs,
- "primaryShardId"_attr = shardId);
- dbInfoEntry->needsRefresh = true;
- }
- }
-
// Invalidate collections which contain data on this shard.
for (const auto& [db, collInfoMap] : _collectionsByDb) {
for (const auto& [collNs, collRoutingInfoEntry] : collInfoMap) {
@@ -534,14 +501,14 @@ void CatalogCache::purgeCollection(const NamespaceString& nss) {
}
void CatalogCache::purgeDatabase(StringData dbName) {
+ _databaseCache.invalidate(dbName.toString());
stdx::lock_guard<Latch> lg(_mutex);
- _databases.erase(dbName);
_collectionsByDb.erase(dbName);
}
void CatalogCache::purgeAllDatabases() {
+ _databaseCache.invalidateAll();
stdx::lock_guard<Latch> lg(_mutex);
- _databases.clear();
_collectionsByDb.clear();
}
@@ -551,8 +518,8 @@ void CatalogCache::report(BSONObjBuilder* builder) const {
size_t numDatabaseEntries;
size_t numCollectionEntries{0};
{
+ numDatabaseEntries = _databaseCache.getCacheInfo().size();
stdx::lock_guard<Latch> ul(_mutex);
- numDatabaseEntries = _databases.size();
for (const auto& entry : _collectionsByDb) {
numCollectionEntries += entry.second.size();
}
@@ -595,78 +562,6 @@ void CatalogCache::checkAndRecordOperationBlockedByRefresh(OperationContext* opC
}
}
-void CatalogCache::_scheduleDatabaseRefresh(WithLock lk,
- StringData dbName,
- std::shared_ptr<DatabaseInfoEntry> dbEntry) {
-
- // TODO (SERVER-34164): Track and increment stats for database refreshes
-
- LOGV2_FOR_CATALOG_REFRESH(24102,
- 1,
- "Refreshing cached database entry for {db}; current cached database "
- "info is {currentDbInfo}",
- "Refreshing cached database entry",
- "db"_attr = dbName,
- "currentDbInfo"_attr =
- (dbEntry->dbt ? dbEntry->dbt->toBSON() : BSONObj()));
-
- Timer t{};
-
- _cacheLoader.getDatabase(dbName)
- .thenRunOn(_executor)
- .then([ =, dbName = dbName.toString() ](const DatabaseType& dbt) noexcept {
- const auto dbVersionAfterRefresh = dbt.getVersion();
- const auto dbVersionHasChanged =
- (!dbEntry->dbt ||
- (dbEntry->dbt &&
- !databaseVersion::equal(dbVersionAfterRefresh, dbEntry->dbt->getVersion())));
-
- stdx::lock_guard<Latch> lg(_mutex);
-
- LOGV2_FOR_CATALOG_REFRESH(
- 24101,
- dbVersionHasChanged ? 0 : 1,
- "Refreshed cached database entry for {db} to version {newDbVersion}"
- "from version {oldDbVersion}. Took {duration}",
- "Refreshed cached database entry",
- "db"_attr = dbName,
- "newDbVersion"_attr = dbVersionAfterRefresh,
- "oldDbVersion"_attr =
- (dbEntry->dbt ? dbEntry->dbt->getVersion().toBSON() : BSONObj()),
- "duration"_attr = Milliseconds(t.millis()));
-
- dbEntry->needsRefresh = false;
- dbEntry->refreshCompletionNotification->set(Status::OK());
- dbEntry->refreshCompletionNotification = nullptr;
-
- dbEntry->dbt = std::move(dbt);
- })
- .onError([ =, dbName = dbName.toString() ](Status errStatus) noexcept {
- stdx::lock_guard<Latch> lg(_mutex);
-
- LOGV2_OPTIONS(24100,
- {logv2::LogComponent::kShardingCatalogRefresh},
- "Error refreshing cached database entry for {db}. Took {duration} and "
- "failed due to {error}",
- "Error refreshing cached database entry",
- "db"_attr = dbName,
- "duration"_attr = Milliseconds(t.millis()),
- "error"_attr = redact(errStatus));
-
- // Clear the notification so the next 'getDatabase' kicks off a new refresh attempt.
- dbEntry->refreshCompletionNotification->set(errStatus);
- dbEntry->refreshCompletionNotification = nullptr;
-
- if (errStatus == ErrorCodes::NamespaceNotFound) {
- // The refresh found that the database was dropped, so remove its entry
- // from the cache.
- _databases.erase(dbName);
- _collectionsByDb.erase(dbName);
- }
- })
- .getAsync([](auto) {});
-}
-
void CatalogCache::_scheduleCollectionRefresh(WithLock lk,
ServiceContext* service,
std::shared_ptr<CollectionRoutingInfoEntry> collEntry,
@@ -883,6 +778,50 @@ void CatalogCache::Stats::report(BSONObjBuilder* builder) const {
}
}
+CatalogCache::DatabaseCache::DatabaseCache(ServiceContext* service,
+ ThreadPoolInterface& threadPool,
+ CatalogCacheLoader& catalogCacheLoader)
+ : ReadThroughCache(_mutex,
+ service,
+ threadPool,
+ [this](OperationContext* opCtx,
+ const std::string& dbName,
+ const ValueHandle& db) { return _lookupDatabase(opCtx, dbName); },
+ kDatabaseCacheSize),
+ _catalogCacheLoader(catalogCacheLoader){};
+
+CatalogCache::DatabaseCache::LookupResult CatalogCache::DatabaseCache::_lookupDatabase(
+ OperationContext* opCtx, const std::string& dbName) {
+
+ // TODO (SERVER-34164): Track and increment stats for database refreshes
+
+ LOGV2_FOR_CATALOG_REFRESH(24102, 1, "Refreshing cached database entry", "db"_attr = dbName);
+
+ Timer t{};
+ try {
+ auto newDb = _catalogCacheLoader.getDatabase(dbName).get();
+ auto const newDbVersion = newDb.getVersion();
+ LOGV2_FOR_CATALOG_REFRESH(24101,
+ 2,
+ "Refreshed cached database entry",
+ "db"_attr = dbName,
+ "newDbVersion"_attr = newDbVersion,
+ "duration"_attr = Milliseconds(t.millis()));
+ return CatalogCache::DatabaseCache::LookupResult(boost::make_optional(std::move(newDb)));
+ } catch (const DBException& ex) {
+ LOGV2_FOR_CATALOG_REFRESH(24100,
+ 1,
+ "Error refreshing cached database entry",
+ "db"_attr = dbName,
+ "duration"_attr = Milliseconds(t.millis()),
+ "error"_attr = redact(ex));
+ if (ex.code() == ErrorCodes::NamespaceNotFound) {
+ return CatalogCache::DatabaseCache::LookupResult(boost::none);
+ }
+ throw;
+ }
+}
+
CachedDatabaseInfo::CachedDatabaseInfo(DatabaseType dbt, std::shared_ptr<Shard> primaryShard)
: _dbt(std::move(dbt)), _primaryShard(std::move(primaryShard)) {}
diff --git a/src/mongo/s/catalog_cache.h b/src/mongo/s/catalog_cache.h
index 0d8b639c68d..66305dd11c2 100644
--- a/src/mongo/s/catalog_cache.h
+++ b/src/mongo/s/catalog_cache.h
@@ -42,6 +42,7 @@
#include "mongo/util/concurrency/notification.h"
#include "mongo/util/concurrency/thread_pool.h"
#include "mongo/util/concurrency/with_lock.h"
+#include "mongo/util/read_through_cache.h"
#include "mongo/util/string_map.h"
namespace mongo {
@@ -129,7 +130,9 @@ class CatalogCache {
CatalogCache& operator=(const CatalogCache&) = delete;
public:
- CatalogCache(CatalogCacheLoader& cacheLoader, std::shared_ptr<ThreadPool> executor);
+ CatalogCache(ServiceContext* service,
+ CatalogCacheLoader& cacheLoader,
+ std::shared_ptr<ThreadPool> executor);
~CatalogCache();
/**
@@ -255,12 +258,6 @@ public:
const ShardId& shardId) const;
/**
- * Non-blocking method, which indiscriminately causes the database entry for the specified
- * database to be refreshed the next time getDatabase is called.
- */
- void invalidateDatabaseEntry(const StringData dbName);
-
- /**
* Non-blocking method, which invalidates the shard for the routing table for the specified
* namespace. If that shard is targetted in the future, getCollectionRoutingInfo will wait on a
* refresh.
@@ -340,31 +337,20 @@ private:
std::shared_ptr<RoutingTableHistory> routingInfo;
};
- /**
- * Cache entry describing a database.
- */
- struct DatabaseInfoEntry {
- // Specifies whether this cache entry needs a refresh (in which case 'dbt' will either be
- // unset if the cache entry has never been loaded, or should not be relied on).
- bool needsRefresh{true};
+ class DatabaseCache : public ReadThroughCache<std::string, DatabaseType> {
+ public:
+ DatabaseCache(ServiceContext* service,
+ ThreadPoolInterface& threadPool,
+ CatalogCacheLoader& catalogCacheLoader);
- // Contains a notification to be waited on for the refresh to complete (only available if
- // needsRefresh is true)
- std::shared_ptr<Notification<Status>> refreshCompletionNotification;
+ private:
+ LookupResult _lookupDatabase(OperationContext* opCtx, const std::string& dbName);
- // Contains the cached info about the database (only available if needsRefresh is false)
- boost::optional<DatabaseType> dbt;
+ CatalogCacheLoader& _catalogCacheLoader;
+ Mutex _mutex = MONGO_MAKE_LATCH("DatabaseCache::_mutex");
};
/**
- * Non-blocking call which schedules an asynchronous refresh for the specified database. The
- * database entry must be in the 'needsRefresh' state.
- */
- void _scheduleDatabaseRefresh(WithLock,
- StringData dbName,
- std::shared_ptr<DatabaseInfoEntry> dbEntry);
-
- /**
* Non-blocking call which schedules an asynchronous refresh for the specified namespace. The
* namespace must be in the 'needRefresh' state.
*/
@@ -498,15 +484,13 @@ private:
std::shared_ptr<ThreadPool> _executor;
- using DatabaseInfoMap = StringMap<std::shared_ptr<DatabaseInfoEntry>>;
using CollectionInfoMap = StringMap<std::shared_ptr<CollectionRoutingInfoEntry>>;
using CollectionsByDbMap = StringMap<CollectionInfoMap>;
- // Mutex to serialize access to the structures below
- mutable Mutex _mutex = MONGO_MAKE_LATCH("CatalogCache::_mutex");
+ DatabaseCache _databaseCache;
- // Map from DB name to the info for that database
- DatabaseInfoMap _databases;
+ // Mutex to serialize access to the collection cache
+ mutable Mutex _mutex = MONGO_MAKE_LATCH("CatalogCache::_mutex");
// Map from full collection name to the routing info for that collection, grouped by database
CollectionsByDbMap _collectionsByDb;
};
diff --git a/src/mongo/s/mongos_main.cpp b/src/mongo/s/mongos_main.cpp
index fcb9b6c0b2e..daf57daa1ba 100644
--- a/src/mongo/s/mongos_main.cpp
+++ b/src/mongo/s/mongos_main.cpp
@@ -423,8 +423,8 @@ Status initializeSharding(OperationContext* opCtx) {
CatalogCacheLoader::set(opCtx->getServiceContext(),
std::make_unique<ConfigServerCatalogCacheLoader>(catalogCacheExecutor));
- auto catalogCache =
- std::make_unique<CatalogCache>(CatalogCacheLoader::get(opCtx), catalogCacheExecutor);
+ auto catalogCache = std::make_unique<CatalogCache>(
+ opCtx->getServiceContext(), CatalogCacheLoader::get(opCtx), catalogCacheExecutor);
// List of hooks which will be called by the ShardRegistry when it discovers a shard has been
// removed.
diff --git a/src/mongo/s/sharding_router_test_fixture.cpp b/src/mongo/s/sharding_router_test_fixture.cpp
index 7142c90ae11..591be7e8760 100644
--- a/src/mongo/s/sharding_router_test_fixture.cpp
+++ b/src/mongo/s/sharding_router_test_fixture.cpp
@@ -171,17 +171,18 @@ ShardingTestFixture::ShardingTestFixture()
CatalogCacheLoader::set(
service, std::make_unique<ConfigServerCatalogCacheLoader>(catalogCacheExecutor()));
+ auto catalogCache = std::make_unique<CatalogCache>(
+ service, CatalogCacheLoader::get(service), catalogCacheExecutor());
// For now initialize the global grid object. All sharding objects will be accessible from there
// until we get rid of it.
auto const grid = Grid::get(operationContext());
- grid->init(
- makeShardingCatalogClient(std::move(uniqueDistLockManager)),
- std::make_unique<CatalogCache>(CatalogCacheLoader::get(service), catalogCacheExecutor()),
- std::move(shardRegistry),
- std::make_unique<ClusterCursorManager>(service->getPreciseClockSource()),
- std::make_unique<BalancerConfiguration>(),
- std::move(executorPool),
- _mockNetwork);
+ grid->init(makeShardingCatalogClient(std::move(uniqueDistLockManager)),
+ std::move(catalogCache),
+ std::move(shardRegistry),
+ std::make_unique<ClusterCursorManager>(service->getPreciseClockSource()),
+ std::make_unique<BalancerConfiguration>(),
+ std::move(executorPool),
+ _mockNetwork);
if (grid->catalogClient()) {
grid->catalogClient()->startup();
diff --git a/src/mongo/s/write_ops/chunk_manager_targeter.cpp b/src/mongo/s/write_ops/chunk_manager_targeter.cpp
index ae40d648c28..20bbb4e28cd 100644
--- a/src/mongo/s/write_ops/chunk_manager_targeter.cpp
+++ b/src/mongo/s/write_ops/chunk_manager_targeter.cpp
@@ -824,8 +824,7 @@ void ChunkManagerTargeter::_refreshShardVersionNow(OperationContext* opCtx) {
}
void ChunkManagerTargeter::_refreshDbVersionNow(OperationContext* opCtx) {
- Grid::get(opCtx)->catalogCache()->onStaleDatabaseVersion(
- _nss.db(), std::move(_routingInfo->db().databaseVersion()));
+ uassertStatusOK(Grid::get(opCtx)->catalogCache()->getDatabaseWithRefresh(opCtx, _nss.db()));
_init(opCtx);
}