summaryrefslogtreecommitdiff
diff options
context:
space:
mode:
authorKaloian Manassiev <kaloian.manassiev@mongodb.com>2017-02-24 13:15:07 -0500
committerKaloian Manassiev <kaloian.manassiev@mongodb.com>2017-03-01 16:04:31 -0500
commit3b0cfc71832f8bed99f27efba2e07a4258c24f29 (patch)
tree919dff67385217d5c35fcd3e63932637e795939d
parentd50fe6a7b43302a9f8afd6b40702ce287caf0d86 (diff)
downloadmongo-3b0cfc71832f8bed99f27efba2e07a4258c24f29.tar.gz
SERVER-28106 Cleanup the contract of ChunkManager::findIntersectingChunk
(cherry picked from commit 21c2b106d16d69590d46b02cf252bae8ed055b3d)
-rw-r--r--src/mongo/db/s/balancer/balancer.cpp2
-rw-r--r--src/mongo/db/s/balancer/balancer_chunk_selection_policy_impl.cpp8
-rw-r--r--src/mongo/db/s/balancer/migration_manager.cpp6
-rw-r--r--src/mongo/s/chunk_manager.cpp84
-rw-r--r--src/mongo/s/chunk_manager.h25
-rw-r--r--src/mongo/s/chunk_manager_targeter.cpp53
-rw-r--r--src/mongo/s/chunk_manager_targeter.h8
-rw-r--r--src/mongo/s/commands/cluster_aggregate.cpp31
-rw-r--r--src/mongo/s/commands/cluster_find_and_modify_cmd.cpp20
-rw-r--r--src/mongo/s/commands/cluster_map_reduce_cmd.cpp2
-rw-r--r--src/mongo/s/commands/cluster_merge_chunks_cmd.cpp2
-rw-r--r--src/mongo/s/commands/cluster_move_chunk_cmd.cpp4
-rw-r--r--src/mongo/s/commands/cluster_shard_collection_cmd.cpp6
-rw-r--r--src/mongo/s/commands/cluster_split_cmd.cpp6
-rw-r--r--src/mongo/s/commands/cluster_write.cpp4
-rw-r--r--src/mongo/s/config.cpp3
16 files changed, 101 insertions, 163 deletions
diff --git a/src/mongo/db/s/balancer/balancer.cpp b/src/mongo/db/s/balancer/balancer.cpp
index 75c9d5243cd..96a7f24fce9 100644
--- a/src/mongo/db/s/balancer/balancer.cpp
+++ b/src/mongo/db/s/balancer/balancer.cpp
@@ -631,7 +631,7 @@ void Balancer::_splitOrMarkJumbo(OperationContext* txn,
auto scopedCM = uassertStatusOK(ScopedChunkManager::refreshAndGet(txn, nss));
const auto cm = scopedCM.cm().get();
- auto chunk = cm->findIntersectingChunkWithSimpleCollation(txn, minKey);
+ auto chunk = cm->findIntersectingChunkWithSimpleCollation(minKey);
try {
const auto splitPoints = uassertStatusOK(shardutil::selectChunkSplitPoints(
diff --git a/src/mongo/db/s/balancer/balancer_chunk_selection_policy_impl.cpp b/src/mongo/db/s/balancer/balancer_chunk_selection_policy_impl.cpp
index 7e6b3f32686..4f3905b61bd 100644
--- a/src/mongo/db/s/balancer/balancer_chunk_selection_policy_impl.cpp
+++ b/src/mongo/db/s/balancer/balancer_chunk_selection_policy_impl.cpp
@@ -390,7 +390,7 @@ StatusWith<SplitInfoVector> BalancerChunkSelectionPolicyImpl::_getSplitCandidate
const auto& tagRange = tagRangeEntry.second;
shared_ptr<Chunk> chunkAtZoneMin =
- cm->findIntersectingChunkWithSimpleCollation(txn, tagRange.min);
+ cm->findIntersectingChunkWithSimpleCollation(tagRange.min);
invariant(chunkAtZoneMin->getMax().woCompare(tagRange.min) > 0);
if (chunkAtZoneMin->getMin().woCompare(tagRange.min)) {
@@ -402,7 +402,7 @@ StatusWith<SplitInfoVector> BalancerChunkSelectionPolicyImpl::_getSplitCandidate
continue;
shared_ptr<Chunk> chunkAtZoneMax =
- cm->findIntersectingChunkWithSimpleCollation(txn, tagRange.max);
+ cm->findIntersectingChunkWithSimpleCollation(tagRange.max);
// We need to check that both the chunk's minKey does not match the zone's max and also that
// the max is not equal, which would only happen in the case of the zone ending in MaxKey.
@@ -441,7 +441,7 @@ StatusWith<MigrateInfoVector> BalancerChunkSelectionPolicyImpl::_getMigrateCandi
const auto& tagRange = tagRangeEntry.second;
shared_ptr<Chunk> chunkAtZoneMin =
- cm->findIntersectingChunkWithSimpleCollation(txn, tagRange.min);
+ cm->findIntersectingChunkWithSimpleCollation(tagRange.min);
if (chunkAtZoneMin->getMin().woCompare(tagRange.min)) {
return {ErrorCodes::IllegalOperation,
@@ -460,7 +460,7 @@ StatusWith<MigrateInfoVector> BalancerChunkSelectionPolicyImpl::_getMigrateCandi
continue;
shared_ptr<Chunk> chunkAtZoneMax =
- cm->findIntersectingChunkWithSimpleCollation(txn, tagRange.max);
+ cm->findIntersectingChunkWithSimpleCollation(tagRange.max);
// We need to check that both the chunk's minKey does not match the zone's max and also that
// the max is not equal, which would only happen in the case of the zone ending in MaxKey.
diff --git a/src/mongo/db/s/balancer/migration_manager.cpp b/src/mongo/db/s/balancer/migration_manager.cpp
index 3aa2a463289..cbf269de1a8 100644
--- a/src/mongo/db/s/balancer/migration_manager.cpp
+++ b/src/mongo/db/s/balancer/migration_manager.cpp
@@ -259,7 +259,7 @@ Status MigrationManager::executeManualMigration(
const auto& scopedCM = scopedCMStatus.getValue();
- auto chunk = scopedCM.cm()->findIntersectingChunkWithSimpleCollation(txn, migrateInfo.minKey);
+ auto chunk = scopedCM.cm()->findIntersectingChunkWithSimpleCollation(migrateInfo.minKey);
invariant(chunk);
Status commandStatus = _processRemoteCommandResponse(
@@ -416,7 +416,7 @@ void MigrationManager::finishRecovery(OperationContext* txn,
migrateInfos.pop_front();
auto chunk =
- scopedCM.cm()->findIntersectingChunkWithSimpleCollation(txn, migrationInfo.minKey);
+ scopedCM.cm()->findIntersectingChunkWithSimpleCollation(migrationInfo.minKey);
invariant(chunk);
if (chunk->getShardId() != migrationInfo.from) {
@@ -536,7 +536,7 @@ shared_ptr<Notification<RemoteCommandResponse>> MigrationManager::_schedule(
auto const chunkManager = statusWithScopedChunkManager.getValue().cm();
- auto chunk = chunkManager->findIntersectingChunkWithSimpleCollation(txn, migrateInfo.minKey);
+ auto chunk = chunkManager->findIntersectingChunkWithSimpleCollation(migrateInfo.minKey);
invariant(chunk);
// If the chunk is not found exactly as requested, the caller must have stale data
diff --git a/src/mongo/s/chunk_manager.cpp b/src/mongo/s/chunk_manager.cpp
index fa14a893e66..be9ee5a25eb 100644
--- a/src/mongo/s/chunk_manager.cpp
+++ b/src/mongo/s/chunk_manager.cpp
@@ -43,10 +43,8 @@
#include "mongo/db/query/query_planner.h"
#include "mongo/db/query/query_planner_common.h"
#include "mongo/s/catalog/sharding_catalog_client.h"
-#include "mongo/s/catalog_cache.h"
#include "mongo/s/chunk_diff.h"
#include "mongo/s/client/shard_registry.h"
-#include "mongo/s/config.h"
#include "mongo/s/grid.h"
#include "mongo/util/log.h"
#include "mongo/util/timer.h"
@@ -193,7 +191,6 @@ void ChunkManager::loadExistingRanges(OperationContext* txn, const ChunkManager*
// TODO: Merge into diff code above, so we validate in one place
if (isChunkMapValid(chunkMap)) {
_chunkMap = std::move(chunkMap);
- _shardIds = std::move(shardIds);
_shardVersions = std::move(shardVersions);
_chunkRangeMap = _constructRanges(_chunkMap);
@@ -324,66 +321,30 @@ bool ChunkManager::_load(OperationContext* txn,
}
}
-StatusWith<shared_ptr<Chunk>> ChunkManager::findIntersectingChunk(OperationContext* txn,
- const BSONObj& shardKey,
- const BSONObj& collation) const {
+std::shared_ptr<Chunk> ChunkManager::findIntersectingChunk(const BSONObj& shardKey,
+ const BSONObj& collation) const {
const bool hasSimpleCollation = (collation.isEmpty() && !_defaultCollator) ||
SimpleBSONObjComparator::kInstance.evaluate(collation == CollationSpec::kSimpleSpec);
if (!hasSimpleCollation) {
for (BSONElement elt : shardKey) {
- if (CollationIndexKey::isCollatableType(elt.type())) {
- return Status(ErrorCodes::ShardKeyNotFound,
- "cannot target single shard due to collation");
- }
- }
- }
-
- BSONObj chunkMin;
- shared_ptr<Chunk> chunk;
- {
- ChunkMap::const_iterator it = _chunkMap.upper_bound(shardKey);
- if (it != _chunkMap.end()) {
- chunkMin = it->first;
- chunk = it->second;
+ uassert(ErrorCodes::ShardKeyNotFound,
+ str::stream() << "Cannot target single shard due to collation of key "
+ << elt.fieldNameStringData(),
+ !CollationIndexKey::isCollatableType(elt.type()));
}
}
- if (!chunk) {
- // TODO: This should be an invariant
- msgasserted(8070,
- str::stream() << "couldn't find a chunk intersecting: " << shardKey
- << " for ns: "
- << _nss.ns()
- << " at version: "
- << _version.toString()
- << ", number of chunks: "
- << _chunkMap.size());
- }
+ const auto it = _chunkMap.upper_bound(shardKey);
+ uassert(ErrorCodes::ShardKeyNotFound,
+ str::stream() << "Cannot target single shard using key " << shardKey,
+ it != _chunkMap.end() && it->second->containsKey(shardKey));
- if (chunk->containsKey(shardKey)) {
- return chunk;
- }
-
- // TODO: This should be an invariant
- log() << redact(chunkMin.toString());
- log() << redact((*chunk).toString());
- log() << redact(shardKey);
-
- // Proactively force a reload on the chunk manager in case it somehow got inconsistent
- auto config = uassertStatusOK(Grid::get(txn)->catalogCache()->getDatabase(txn, _nss.db()));
- config->getChunkManagerIfExists(txn, _nss.ns(), true);
-
- msgasserted(13141, "Chunk map pointed to incorrect chunk");
+ return it->second;
}
-shared_ptr<Chunk> ChunkManager::findIntersectingChunkWithSimpleCollation(
- OperationContext* txn, const BSONObj& shardKey) const {
- auto chunk = findIntersectingChunk(txn, shardKey, CollationSpec::kSimpleSpec);
-
- // findIntersectingChunk() should succeed in targeting a single shard, since we have the simple
- // collation.
- massertStatusOK(chunk.getStatus());
- return chunk.getValue();
+std::shared_ptr<Chunk> ChunkManager::findIntersectingChunkWithSimpleCollation(
+ const BSONObj& shardKey) const {
+ return findIntersectingChunk(shardKey, CollationSpec::kSimpleSpec);
}
void ChunkManager::getShardIdsForQuery(OperationContext* txn,
@@ -410,10 +371,12 @@ void ChunkManager::getShardIdsForQuery(OperationContext* txn,
// Fast path for targeting equalities on the shard key.
auto shardKeyToFind = _keyPattern.extractShardKeyFromQuery(*cq);
if (!shardKeyToFind.isEmpty()) {
- auto chunk = findIntersectingChunk(txn, shardKeyToFind, collation);
- if (chunk.isOK()) {
- shardIds->insert(chunk.getValue()->getShardId());
+ try {
+ auto chunk = findIntersectingChunk(shardKeyToFind, collation);
+ shardIds->insert(chunk->getShardId());
return;
+ } catch (const DBException&) {
+ // The query uses multiple shards
}
}
@@ -436,7 +399,7 @@ void ChunkManager::getShardIdsForQuery(OperationContext* txn,
getShardIdsForRange(it->first /*min*/, it->second /*max*/, shardIds);
// once we know we need to visit all shards no need to keep looping
- if (shardIds->size() == _shardIds.size()) {
+ if (shardIds->size() == _shardVersions.size()) {
break;
}
}
@@ -468,14 +431,17 @@ void ChunkManager::getShardIdsForRange(const BSONObj& min,
// No need to iterate through the rest of the ranges, because we already know we need to use
// all shards.
- if (shardIds->size() == _shardIds.size()) {
+ if (shardIds->size() == _shardVersions.size()) {
break;
}
}
}
void ChunkManager::getAllShardIds(set<ShardId>* all) const {
- all->insert(_shardIds.begin(), _shardIds.end());
+ std::transform(_shardVersions.begin(),
+ _shardVersions.end(),
+ std::inserter(*all, all->begin()),
+ [](const ShardVersionMap::value_type& pair) { return pair.first; });
}
IndexBounds ChunkManager::getIndexBoundsForQuery(const BSONObj& key,
diff --git a/src/mongo/s/chunk_manager.h b/src/mongo/s/chunk_manager.h
index f184c60ee5a..08d1d9fe229 100644
--- a/src/mongo/s/chunk_manager.h
+++ b/src/mongo/s/chunk_manager.h
@@ -110,27 +110,24 @@ public:
//
/**
- * Given a key that has been extracted from a document, returns the
- * chunk that contains that key.
+ * Given a shard key (or a prefix) that has been extracted from a document, returns the chunk
+ * that contains that key.
*
- * For instance, to locate the chunk for document {a : "foo" , b : "bar"}
- * when the shard key is {a : "hashed"}, you can call
- * findIntersectingChunk() on {a : hash("foo") }
+ * Example: findIntersectingChunk({a : hash('foo')}) locates the chunk for document
+ * {a: 'foo', b: 'bar'} if the shard key is {a : 'hashed'}.
*
* If 'collation' is empty, we use the collection default collation for targeting.
*
- * Returns the error status ShardKeyNotFound if unable to target a single shard due to the
- * collation.
+ * Throws a DBException with the ShardKeyNotFound code if unable to target a single shard due to
+ * collation or due to the key not matching the shard key pattern.
*/
- StatusWith<std::shared_ptr<Chunk>> findIntersectingChunk(OperationContext* txn,
- const BSONObj& shardKey,
- const BSONObj& collation) const;
+ std::shared_ptr<Chunk> findIntersectingChunk(const BSONObj& shardKey,
+ const BSONObj& collation) const;
/**
- * Finds the intersecting chunk, assuming the simple collation.
+ * Same as findIntersectingChunk, but assumes the simple collation.
*/
- std::shared_ptr<Chunk> findIntersectingChunkWithSimpleCollation(OperationContext* txn,
- const BSONObj& shardKey) const;
+ std::shared_ptr<Chunk> findIntersectingChunkWithSimpleCollation(const BSONObj& shardKey) const;
/**
* Finds the shard IDs for a given filter and collation. If collation is empty, we use the
@@ -257,8 +254,6 @@ private:
// constructed map must cover the complete space from [MinKey, MaxKey).
ChunkRangeMap _chunkRangeMap;
- std::set<ShardId> _shardIds;
-
// Max known version per shard
ShardVersionMap _shardVersions;
diff --git a/src/mongo/s/chunk_manager_targeter.cpp b/src/mongo/s/chunk_manager_targeter.cpp
index 43e10250456..2ffc7efeb55 100644
--- a/src/mongo/s/chunk_manager_targeter.cpp
+++ b/src/mongo/s/chunk_manager_targeter.cpp
@@ -335,7 +335,7 @@ Status ChunkManagerTargeter::targetInsert(OperationContext* txn,
// Target the shard key or database primary
if (!shardKey.isEmpty()) {
- return targetShardKey(txn, shardKey, CollationSpec::kSimpleSpec, doc.objsize(), endpoint);
+ *endpoint = targetShardKey(shardKey, CollationSpec::kSimpleSpec, doc.objsize()).release();
} else {
if (!_primary) {
return Status(ErrorCodes::NamespaceNotFound,
@@ -344,8 +344,9 @@ Status ChunkManagerTargeter::targetInsert(OperationContext* txn,
}
*endpoint = new ShardEndpoint(_primary->getId(), ChunkVersion::UNSHARDED());
- return Status::OK();
}
+
+ return Status::OK();
}
Status ChunkManagerTargeter::targetUpdate(OperationContext* txn,
@@ -416,13 +417,13 @@ Status ChunkManagerTargeter::targetUpdate(OperationContext* txn,
// Target the shard key, query, or replacement doc
if (!shardKey.isEmpty()) {
- // We can't rely on our query targeting to be exact
- ShardEndpoint* endpoint = NULL;
- Status result = targetShardKey(
- txn, shardKey, collation, (query.objsize() + updateExpr.objsize()), &endpoint);
- if (result.isOK()) {
- endpoints->push_back(endpoint);
- return result;
+ try {
+ endpoints->push_back(
+ targetShardKey(shardKey, collation, (query.objsize() + updateExpr.objsize()))
+ .release());
+ return Status::OK();
+ } catch (const DBException&) {
+ // This update is potentially not constrained to a single shard
}
}
@@ -499,12 +500,11 @@ Status ChunkManagerTargeter::targetDelete(OperationContext* txn,
// Target the shard key or delete query
if (!shardKey.isEmpty()) {
- // We can't rely on our query targeting to be exact
- ShardEndpoint* endpoint = NULL;
- Status result = targetShardKey(txn, shardKey, collation, 0, &endpoint);
- if (result.isOK()) {
- endpoints->push_back(endpoint);
- return result;
+ try {
+ endpoints->push_back(targetShardKey(shardKey, collation, 0).release());
+ return Status::OK();
+ } catch (const DBException&) {
+ // This delete is potentially not constrained to a single shard
}
}
@@ -578,28 +578,19 @@ Status ChunkManagerTargeter::targetQuery(OperationContext* txn,
return Status::OK();
}
-Status ChunkManagerTargeter::targetShardKey(OperationContext* txn,
- const BSONObj& shardKey,
- const BSONObj& collation,
- long long estDataSize,
- ShardEndpoint** endpoint) const {
- invariant(NULL != _manager);
-
- auto chunk = _manager->findIntersectingChunk(txn, shardKey, collation);
- if (!chunk.isOK()) {
- return chunk.getStatus();
- }
+std::unique_ptr<ShardEndpoint> ChunkManagerTargeter::targetShardKey(const BSONObj& shardKey,
+ const BSONObj& collation,
+ long long estDataSize) const {
+ auto chunk = _manager->findIntersectingChunk(shardKey, collation);
// Track autosplit stats for sharded collections
// Note: this is only best effort accounting and is not accurate.
if (estDataSize > 0) {
- _stats->chunkSizeDelta[chunk.getValue()->getMin()] += estDataSize;
+ _stats->chunkSizeDelta[chunk->getMin()] += estDataSize;
}
- *endpoint = new ShardEndpoint(chunk.getValue()->getShardId(),
- _manager->getVersion(chunk.getValue()->getShardId()));
-
- return Status::OK();
+ return stdx::make_unique<ShardEndpoint>(chunk->getShardId(),
+ _manager->getVersion(chunk->getShardId()));
}
Status ChunkManagerTargeter::targetCollection(vector<ShardEndpoint*>* endpoints) const {
diff --git a/src/mongo/s/chunk_manager_targeter.h b/src/mongo/s/chunk_manager_targeter.h
index cea71318458..9c8f136dad8 100644
--- a/src/mongo/s/chunk_manager_targeter.h
+++ b/src/mongo/s/chunk_manager_targeter.h
@@ -154,11 +154,9 @@ private:
*
* If 'collation' is empty, we use the collection default collation for targeting.
*/
- Status targetShardKey(OperationContext* txn,
- const BSONObj& doc,
- const BSONObj& collation,
- long long estDataSize,
- ShardEndpoint** endpoint) const;
+ std::unique_ptr<ShardEndpoint> targetShardKey(const BSONObj& doc,
+ const BSONObj& collation,
+ long long estDataSize) const;
// Full namespace of the collection for this targeter
const NamespaceString _nss;
diff --git a/src/mongo/s/commands/cluster_aggregate.cpp b/src/mongo/s/commands/cluster_aggregate.cpp
index 9f62f50613f..7c7bf6686dd 100644
--- a/src/mongo/s/commands/cluster_aggregate.cpp
+++ b/src/mongo/s/commands/cluster_aggregate.cpp
@@ -122,21 +122,24 @@ Status ClusterAggregate::runAggregate(OperationContext* txn,
pipeline.getValue()->optimizePipeline();
- // If the first $match stage is an exact match on the shard key (with a simple collation or
- // no string matching), we only have to send it to one shard, so send the command to that
- // shard.
- BSONObj firstMatchQuery = pipeline.getValue()->getInitialQuery();
- BSONObj shardKeyMatches;
- shardKeyMatches = uassertStatusOK(
- chunkMgr->getShardKeyPattern().extractShardKeyFromQuery(txn, firstMatchQuery));
- bool singleShard = false;
- if (!shardKeyMatches.isEmpty()) {
- auto chunk = chunkMgr->findIntersectingChunk(
- txn, shardKeyMatches, request.getValue().getCollation());
- if (chunk.isOK()) {
- singleShard = true;
+ // If the first $match stage is an exact match on the shard key (with a simple collation or no
+ // string matching), we only have to send it to one shard, so send the command to that shard.
+ const bool singleShard = [&]() {
+ BSONObj firstMatchQuery = pipeline.getValue()->getInitialQuery();
+ BSONObj shardKeyMatches = uassertStatusOK(
+ chunkMgr->getShardKeyPattern().extractShardKeyFromQuery(txn, firstMatchQuery));
+
+ if (shardKeyMatches.isEmpty()) {
+ return false;
}
- }
+
+ try {
+ chunkMgr->findIntersectingChunk(shardKeyMatches, request.getValue().getCollation());
+ return true;
+ } catch (const DBException&) {
+ return false;
+ }
+ }();
// Don't need to split pipeline if the first $match is an exact match on shard key, unless
// there is a stage that needs to be run on the primary shard.
diff --git a/src/mongo/s/commands/cluster_find_and_modify_cmd.cpp b/src/mongo/s/commands/cluster_find_and_modify_cmd.cpp
index c49a4517962..eabb8661051 100644
--- a/src/mongo/s/commands/cluster_find_and_modify_cmd.cpp
+++ b/src/mongo/s/commands/cluster_find_and_modify_cmd.cpp
@@ -124,16 +124,9 @@ public:
}
BSONObj shardKey = status.getValue();
- auto chunk = chunkMgr->findIntersectingChunk(txn, shardKey, collation);
+ auto chunk = chunkMgr->findIntersectingChunk(shardKey, collation);
- if (!chunk.isOK()) {
- uasserted(ErrorCodes::ShardKeyNotFound,
- "findAndModify must target a single shard, but was not able to due "
- "to non-simple collation");
- }
-
- auto shardStatus =
- Grid::get(txn)->shardRegistry()->getShard(txn, chunk.getValue()->getShardId());
+ auto shardStatus = Grid::get(txn)->shardRegistry()->getShard(txn, chunk->getShardId());
if (!shardStatus.isOK()) {
return shardStatus.getStatus();
}
@@ -208,14 +201,7 @@ public:
}
BSONObj shardKey = status.getValue();
- auto chunkStatus = chunkMgr->findIntersectingChunk(txn, shardKey, collation);
- if (!chunkStatus.isOK()) {
- uasserted(ErrorCodes::ShardKeyNotFound,
- "findAndModify must target a single shard, but was not able to due to "
- "non-simple collation");
- }
-
- const auto& chunk = chunkStatus.getValue();
+ auto chunk = chunkMgr->findIntersectingChunk(shardKey, collation);
const bool ok = _runCommand(txn, conf, chunkMgr, chunk->getShardId(), nss, cmdObj, result);
if (ok) {
diff --git a/src/mongo/s/commands/cluster_map_reduce_cmd.cpp b/src/mongo/s/commands/cluster_map_reduce_cmd.cpp
index 5ba01d49f4f..6d4a4155365 100644
--- a/src/mongo/s/commands/cluster_map_reduce_cmd.cpp
+++ b/src/mongo/s/commands/cluster_map_reduce_cmd.cpp
@@ -621,7 +621,7 @@ public:
invariant(size < std::numeric_limits<int>::max());
// key reported should be the chunk's minimum
- shared_ptr<Chunk> c = cm->findIntersectingChunkWithSimpleCollation(txn, key);
+ shared_ptr<Chunk> c = cm->findIntersectingChunkWithSimpleCollation(key);
if (!c) {
warning() << "Mongod reported " << size << " bytes inserted for key " << key
<< " but can't find chunk";
diff --git a/src/mongo/s/commands/cluster_merge_chunks_cmd.cpp b/src/mongo/s/commands/cluster_merge_chunks_cmd.cpp
index 70894d96f4d..2aaeeaeabb0 100644
--- a/src/mongo/s/commands/cluster_merge_chunks_cmd.cpp
+++ b/src/mongo/s/commands/cluster_merge_chunks_cmd.cpp
@@ -151,7 +151,7 @@ public:
minKey = cm->getShardKeyPattern().normalizeShardKey(minKey);
maxKey = cm->getShardKeyPattern().normalizeShardKey(maxKey);
- shared_ptr<Chunk> firstChunk = cm->findIntersectingChunkWithSimpleCollation(txn, minKey);
+ shared_ptr<Chunk> firstChunk = cm->findIntersectingChunkWithSimpleCollation(minKey);
BSONObjBuilder remoteCmdObjB;
remoteCmdObjB.append(cmdObj[ClusterMergeChunksCommand::nsField()]);
diff --git a/src/mongo/s/commands/cluster_move_chunk_cmd.cpp b/src/mongo/s/commands/cluster_move_chunk_cmd.cpp
index df3a177692b..f0e4e7f6973 100644
--- a/src/mongo/s/commands/cluster_move_chunk_cmd.cpp
+++ b/src/mongo/s/commands/cluster_move_chunk_cmd.cpp
@@ -153,7 +153,7 @@ public:
return false;
}
- chunk = cm->findIntersectingChunkWithSimpleCollation(txn, shardKey);
+ chunk = cm->findIntersectingChunkWithSimpleCollation(shardKey);
} else {
// bounds
if (!cm->getShardKeyPattern().isShardKey(bounds[0].Obj()) ||
@@ -168,7 +168,7 @@ public:
BSONObj minKey = cm->getShardKeyPattern().normalizeShardKey(bounds[0].Obj());
BSONObj maxKey = cm->getShardKeyPattern().normalizeShardKey(bounds[1].Obj());
- chunk = cm->findIntersectingChunkWithSimpleCollation(txn, minKey);
+ chunk = cm->findIntersectingChunkWithSimpleCollation(minKey);
if (chunk->getMin().woCompare(minKey) != 0 || chunk->getMax().woCompare(maxKey) != 0) {
errmsg = str::stream() << "no chunk found with the shard key bounds "
diff --git a/src/mongo/s/commands/cluster_shard_collection_cmd.cpp b/src/mongo/s/commands/cluster_shard_collection_cmd.cpp
index 4146682243f..79f792e0c7b 100644
--- a/src/mongo/s/commands/cluster_shard_collection_cmd.cpp
+++ b/src/mongo/s/commands/cluster_shard_collection_cmd.cpp
@@ -651,7 +651,7 @@ public:
// 3. Subdivide the big chunks by splitting at each of the points in "allSplits"
// that we haven't already split by.
auto currentChunk =
- chunkManager->findIntersectingChunkWithSimpleCollation(txn, allSplits[0]);
+ chunkManager->findIntersectingChunkWithSimpleCollation(allSplits[0]);
std::vector<BSONObj> subSplits;
for (unsigned i = 0; i <= allSplits.size(); i++) {
@@ -675,8 +675,8 @@ public:
}
if (i < allSplits.size()) {
- currentChunk = chunkManager->findIntersectingChunkWithSimpleCollation(
- txn, allSplits[i]);
+ currentChunk =
+ chunkManager->findIntersectingChunkWithSimpleCollation(allSplits[i]);
}
} else {
BSONObj splitPoint(allSplits[i]);
diff --git a/src/mongo/s/commands/cluster_split_cmd.cpp b/src/mongo/s/commands/cluster_split_cmd.cpp
index d6cdcb4ba83..57e5a54881b 100644
--- a/src/mongo/s/commands/cluster_split_cmd.cpp
+++ b/src/mongo/s/commands/cluster_split_cmd.cpp
@@ -203,7 +203,7 @@ public:
return false;
}
- chunk = cm->findIntersectingChunkWithSimpleCollation(txn, shardKey);
+ chunk = cm->findIntersectingChunkWithSimpleCollation(shardKey);
} else if (!bounds.isEmpty()) {
// bounds
if (!cm->getShardKeyPattern().isShardKey(bounds[0].Obj()) ||
@@ -218,7 +218,7 @@ public:
BSONObj minKey = cm->getShardKeyPattern().normalizeShardKey(bounds[0].Obj());
BSONObj maxKey = cm->getShardKeyPattern().normalizeShardKey(bounds[1].Obj());
- chunk = cm->findIntersectingChunkWithSimpleCollation(txn, minKey);
+ chunk = cm->findIntersectingChunkWithSimpleCollation(minKey);
if (chunk->getMin().woCompare(minKey) != 0 || chunk->getMax().woCompare(maxKey) != 0) {
errmsg = str::stream() << "no chunk found with the shard key bounds "
@@ -239,7 +239,7 @@ public:
// Check shard key size when manually provided
uassertStatusOK(ShardKeyPattern::checkShardKeySize(middle));
- chunk = cm->findIntersectingChunkWithSimpleCollation(txn, middle);
+ chunk = cm->findIntersectingChunkWithSimpleCollation(middle);
if (chunk->getMin().woCompare(middle) == 0 || chunk->getMax().woCompare(middle) == 0) {
errmsg = str::stream() << "new split key " << middle
diff --git a/src/mongo/s/commands/cluster_write.cpp b/src/mongo/s/commands/cluster_write.cpp
index b3b68e3af04..c36c47750dd 100644
--- a/src/mongo/s/commands/cluster_write.cpp
+++ b/src/mongo/s/commands/cluster_write.cpp
@@ -189,7 +189,7 @@ void splitIfNeeded(OperationContext* txn, const NamespaceString& nss, const Targ
for (auto it = stats.chunkSizeDelta.cbegin(); it != stats.chunkSizeDelta.cend(); ++it) {
std::shared_ptr<Chunk> chunk;
try {
- chunk = scopedCM.cm()->findIntersectingChunkWithSimpleCollation(txn, it->first);
+ chunk = scopedCM.cm()->findIntersectingChunkWithSimpleCollation(it->first);
} catch (const AssertionException& ex) {
warning() << "could not find chunk while checking for auto-split: "
<< causedBy(redact(ex));
@@ -484,7 +484,7 @@ void updateChunkWriteStatsAndSplitIfNeeded(OperationContext* txn,
// up-to-date view of the chunk we are about to move
auto scopedCM = uassertStatusOK(ScopedChunkManager::refreshAndGet(txn, nss));
auto suggestedChunk = scopedCM.cm()->findIntersectingChunkWithSimpleCollation(
- txn, suggestedMigrateChunk->getMin());
+ suggestedMigrateChunk->getMin());
ChunkType chunkToMove;
chunkToMove.setNS(nss.ns());
diff --git a/src/mongo/s/config.cpp b/src/mongo/s/config.cpp
index 153ddf71152..c16f671ba78 100644
--- a/src/mongo/s/config.cpp
+++ b/src/mongo/s/config.cpp
@@ -90,8 +90,7 @@ std::shared_ptr<ChunkManager> DBConfig::getChunkManagerIfExists(OperationContext
try {
return getChunkManager(txn, ns, shouldReload, forceReload);
- } catch (AssertionException& e) {
- warning() << "chunk manager not found for " << ns << causedBy(e);
+ } catch (const DBException&) {
return nullptr;
}
}