summaryrefslogtreecommitdiff
path: root/src/mongo/db/s/balancer
diff options
context:
space:
mode:
Diffstat (limited to 'src/mongo/db/s/balancer')
-rw-r--r--src/mongo/db/s/balancer/balancer.cpp110
-rw-r--r--src/mongo/db/s/balancer/balancer_chunk_selection_policy_impl.cpp20
-rw-r--r--src/mongo/db/s/balancer/balancer_policy.cpp83
-rw-r--r--src/mongo/db/s/balancer/cluster_statistics_impl.cpp9
-rw-r--r--src/mongo/db/s/balancer/migration_manager.cpp41
-rw-r--r--src/mongo/db/s/balancer/scoped_migration_request.cpp27
6 files changed, 198 insertions, 92 deletions
diff --git a/src/mongo/db/s/balancer/balancer.cpp b/src/mongo/db/s/balancer/balancer.cpp
index 25109941176..212b6d79713 100644
--- a/src/mongo/db/s/balancer/balancer.cpp
+++ b/src/mongo/db/s/balancer/balancer.cpp
@@ -46,6 +46,7 @@
#include "mongo/db/s/balancer/balancer_chunk_selection_policy_impl.h"
#include "mongo/db/s/balancer/cluster_statistics_impl.h"
#include "mongo/db/s/sharding_logging.h"
+#include "mongo/logv2/log.h"
#include "mongo/s/balancer_configuration.h"
#include "mongo/s/catalog/type_chunk.h"
#include "mongo/s/catalog_cache.h"
@@ -157,7 +158,7 @@ void warnOnMultiVersion(const vector<ClusterStatistics::ShardStatistics>& cluste
sb << stat.shardId << " is at " << stat.mongoVersion << "; ";
}
- warning() << sb.str();
+ LOGV2_WARNING(21875, "{sb_str}", "sb_str"_attr = sb.str());
}
} // namespace
@@ -247,7 +248,10 @@ Status Balancer::rebalanceSingleChunk(OperationContext* opCtx, const ChunkType&
auto migrateInfo = std::move(migrateStatus.getValue());
if (!migrateInfo) {
- LOG(1) << "Unable to find more appropriate location for chunk " << redact(chunk.toString());
+ LOGV2_DEBUG(21854,
+ 1,
+ "Unable to find more appropriate location for chunk {chunk}",
+ "chunk"_attr = redact(chunk.toString()));
return Status::OK();
}
@@ -307,14 +311,14 @@ void Balancer::_mainThread() {
_state = kStopped;
_joinCond.notify_all();
- LOG(1) << "Balancer thread terminated";
+ LOGV2_DEBUG(21855, 1, "Balancer thread terminated");
});
Client::initThread("Balancer");
auto opCtx = cc().makeOperationContext();
auto shardingContext = Grid::get(opCtx.get());
- log() << "CSRS balancer is starting";
+ LOGV2(21856, "CSRS balancer is starting");
{
stdx::lock_guard<Latch> scopedLock(_mutex);
@@ -327,9 +331,13 @@ void Balancer::_mainThread() {
while (!_stopRequested()) {
Status refreshStatus = balancerConfig->refreshAndCheck(opCtx.get());
if (!refreshStatus.isOK()) {
- warning() << "Balancer settings could not be loaded and will be retried in "
- << durationCount<Seconds>(kInitBackoffInterval) << " seconds"
- << causedBy(refreshStatus);
+ LOGV2_WARNING(
+ 21876,
+ "Balancer settings could not be loaded and will be retried in "
+ "{durationCount_Seconds_kInitBackoffInterval} seconds{causedBy_refreshStatus}",
+ "durationCount_Seconds_kInitBackoffInterval"_attr =
+ durationCount<Seconds>(kInitBackoffInterval),
+ "causedBy_refreshStatus"_attr = causedBy(refreshStatus));
_sleepFor(opCtx.get(), kInitBackoffInterval);
continue;
@@ -338,13 +346,13 @@ void Balancer::_mainThread() {
break;
}
- log() << "CSRS balancer thread is recovering";
+ LOGV2(21857, "CSRS balancer thread is recovering");
_migrationManager.finishRecovery(opCtx.get(),
balancerConfig->getMaxChunkSizeBytes(),
balancerConfig->getSecondaryThrottle());
- log() << "CSRS balancer thread is recovered";
+ LOGV2(21858, "CSRS balancer thread is recovered");
// Main balancer loop
while (!_stopRequested()) {
@@ -359,22 +367,28 @@ void Balancer::_mainThread() {
Status refreshStatus = balancerConfig->refreshAndCheck(opCtx.get());
if (!refreshStatus.isOK()) {
- warning() << "Skipping balancing round" << causedBy(refreshStatus);
+ LOGV2_WARNING(21877,
+ "Skipping balancing round{causedBy_refreshStatus}",
+ "causedBy_refreshStatus"_attr = causedBy(refreshStatus));
_endRound(opCtx.get(), kBalanceRoundDefaultInterval);
continue;
}
if (!balancerConfig->shouldBalance()) {
- LOG(1) << "Skipping balancing round because balancing is disabled";
+ LOGV2_DEBUG(21859, 1, "Skipping balancing round because balancing is disabled");
_endRound(opCtx.get(), kBalanceRoundDefaultInterval);
continue;
}
{
- LOG(1) << "*** start balancing round. "
- << "waitForDelete: " << balancerConfig->waitForDelete()
- << ", secondaryThrottle: "
- << balancerConfig->getSecondaryThrottle().toBSON();
+ LOGV2_DEBUG(
+ 21860,
+ 1,
+ "*** start balancing round. waitForDelete: {balancerConfig_waitForDelete}, "
+ "secondaryThrottle: {balancerConfig_getSecondaryThrottle}",
+ "balancerConfig_waitForDelete"_attr = balancerConfig->waitForDelete(),
+ "balancerConfig_getSecondaryThrottle"_attr =
+ balancerConfig->getSecondaryThrottle().toBSON());
static Occasionally sampler;
if (sampler.tick()) {
@@ -383,16 +397,18 @@ void Balancer::_mainThread() {
Status status = _enforceTagRanges(opCtx.get());
if (!status.isOK()) {
- warning() << "Failed to enforce tag ranges" << causedBy(status);
+ LOGV2_WARNING(21878,
+ "Failed to enforce tag ranges{causedBy_status}",
+ "causedBy_status"_attr = causedBy(status));
} else {
- LOG(1) << "Done enforcing tag range boundaries.";
+ LOGV2_DEBUG(21861, 1, "Done enforcing tag range boundaries.");
}
const auto candidateChunks =
uassertStatusOK(_chunkSelectionPolicy->selectChunksToMove(opCtx.get()));
if (candidateChunks.empty()) {
- LOG(1) << "no need to move any chunk";
+ LOGV2_DEBUG(21862, 1, "no need to move any chunk");
_balancedLastTime = 0;
} else {
_balancedLastTime = _moveChunks(opCtx.get(), candidateChunks);
@@ -405,7 +421,7 @@ void Balancer::_mainThread() {
.ignore();
}
- LOG(1) << "*** End of balancing round";
+ LOGV2_DEBUG(21863, 1, "*** End of balancing round");
}
Milliseconds balancerInterval =
@@ -413,16 +429,19 @@ void Balancer::_mainThread() {
overrideBalanceRoundInterval.execute([&](const BSONObj& data) {
balancerInterval = Milliseconds(data["intervalMs"].numberInt());
- log() << "overrideBalanceRoundInterval: using shorter balancing interval: "
- << balancerInterval;
+ LOGV2(21864,
+ "overrideBalanceRoundInterval: using shorter balancing interval: "
+ "{balancerInterval}",
+ "balancerInterval"_attr = balancerInterval);
});
_endRound(opCtx.get(), balancerInterval);
} catch (const DBException& e) {
- log() << "caught exception while doing balance: " << e.what();
+ LOGV2(
+ 21865, "caught exception while doing balance: {e_what}", "e_what"_attr = e.what());
// Just to match the opening statement if in log level 1
- LOG(1) << "*** End of balancing round";
+ LOGV2_DEBUG(21866, 1, "*** End of balancing round");
// This round failed, tell the world!
roundDetails.setFailed(e.what());
@@ -451,7 +470,7 @@ void Balancer::_mainThread() {
_threadOperationContext = nullptr;
}
- log() << "CSRS balancer is now stopped";
+ LOGV2(21867, "CSRS balancer is now stopped");
}
bool Balancer::_stopRequested() {
@@ -516,8 +535,11 @@ bool Balancer::_checkOIDs(OperationContext* opCtx) {
if (oids.count(x) == 0) {
oids[x] = shardId;
} else {
- log() << "error: 2 machines have " << x << " as oid machine piece: " << shardId
- << " and " << oids[x];
+ LOGV2(21868,
+ "error: 2 machines have {x} as oid machine piece: {shardId} and {oids_x}",
+ "x"_attr = x,
+ "shardId"_attr = shardId,
+ "oids_x"_attr = oids[x]);
result = uassertStatusOK(s->runCommandWithFixedRetryAttempts(
opCtx,
@@ -542,7 +564,7 @@ bool Balancer::_checkOIDs(OperationContext* opCtx) {
return false;
}
} else {
- log() << "warning: oidMachine not set on: " << s->toString();
+ LOGV2(21869, "warning: oidMachine not set on: {s}", "s"_attr = s->toString());
}
}
@@ -574,8 +596,11 @@ Status Balancer::_enforceTagRanges(OperationContext* opCtx) {
ChunkRange(splitInfo.minKey, splitInfo.maxKey),
splitInfo.splitKeys);
if (!splitStatus.isOK()) {
- warning() << "Failed to enforce tag range for chunk " << redact(splitInfo.toString())
- << causedBy(redact(splitStatus.getStatus()));
+ LOGV2_WARNING(
+ 21879,
+ "Failed to enforce tag range for chunk {splitInfo}{causedBy_splitStatus_getStatus}",
+ "splitInfo"_attr = redact(splitInfo.toString()),
+ "causedBy_splitStatus_getStatus"_attr = causedBy(redact(splitStatus.getStatus())));
}
}
@@ -588,7 +613,7 @@ int Balancer::_moveChunks(OperationContext* opCtx,
// If the balancer was disabled since we started this round, don't start new chunk moves
if (_stopRequested() || !balancerConfig->shouldBalance()) {
- LOG(1) << "Skipping balancing round because balancer was stopped";
+ LOGV2_DEBUG(21870, 1, "Skipping balancing round because balancer was stopped");
return 0;
}
@@ -626,15 +651,20 @@ int Balancer::_moveChunks(OperationContext* opCtx,
if (status == ErrorCodes::ChunkTooBig || status == ErrorCodes::ExceededMemoryLimit) {
numChunksProcessed++;
- log() << "Performing a split because migration " << redact(requestIt->toString())
- << " failed for size reasons" << causedBy(redact(status));
+ LOGV2(21871,
+ "Performing a split because migration {requestIt} failed for size "
+ "reasons{causedBy_status}",
+ "requestIt"_attr = redact(requestIt->toString()),
+ "causedBy_status"_attr = causedBy(redact(status)));
_splitOrMarkJumbo(opCtx, requestIt->nss, requestIt->minKey);
continue;
}
- log() << "Balancer move " << redact(requestIt->toString()) << " failed"
- << causedBy(redact(status));
+ LOGV2(21872,
+ "Balancer move {requestIt} failed{causedBy_status}",
+ "requestIt"_attr = redact(requestIt->toString()),
+ "causedBy_status"_attr = causedBy(redact(status)));
}
return numChunksProcessed;
@@ -660,7 +690,8 @@ void Balancer::_splitOrMarkJumbo(OperationContext* opCtx,
boost::none));
if (splitPoints.empty()) {
- log() << "Marking chunk " << redact(chunk.toString()) << " as jumbo.";
+ LOGV2(
+ 21873, "Marking chunk {chunk} as jumbo.", "chunk"_attr = redact(chunk.toString()));
chunk.markAsJumbo();
auto status = Grid::get(opCtx)->catalogClient()->updateConfigDocument(
@@ -671,9 +702,12 @@ void Balancer::_splitOrMarkJumbo(OperationContext* opCtx,
false,
ShardingCatalogClient::kMajorityWriteConcern);
if (!status.isOK()) {
- log() << "Couldn't set jumbo for chunk with namespace " << redact(nss.ns())
- << " and min key " << redact(chunk.getMin())
- << causedBy(redact(status.getStatus()));
+ LOGV2(21874,
+ "Couldn't set jumbo for chunk with namespace {nss_ns} and min key "
+ "{chunk_getMin}{causedBy_status_getStatus}",
+ "nss_ns"_attr = redact(nss.ns()),
+ "chunk_getMin"_attr = redact(chunk.getMin()),
+ "causedBy_status_getStatus"_attr = causedBy(redact(status.getStatus())));
}
return;
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 f387556b02b..9f50b4f5169 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
@@ -39,6 +39,7 @@
#include "mongo/base/status_with.h"
#include "mongo/bson/bsonobj_comparator_interface.h"
+#include "mongo/logv2/log.h"
#include "mongo/s/balancer_configuration.h"
#include "mongo/s/catalog/type_chunk.h"
#include "mongo/s/catalog/type_collection.h"
@@ -218,8 +219,12 @@ StatusWith<SplitInfoVector> BalancerChunkSelectionPolicyImpl::selectChunksToSpli
// Namespace got dropped before we managed to get to it, so just skip it
continue;
} else if (!candidatesStatus.isOK()) {
- warning() << "Unable to enforce tag range policy for collection " << nss.ns()
- << causedBy(candidatesStatus.getStatus());
+ LOGV2_WARNING(21852,
+ "Unable to enforce tag range policy for collection "
+ "{nss_ns}{causedBy_candidatesStatus_getStatus}",
+ "nss_ns"_attr = nss.ns(),
+ "causedBy_candidatesStatus_getStatus"_attr =
+ causedBy(candidatesStatus.getStatus()));
continue;
}
@@ -282,7 +287,8 @@ StatusWith<MigrateInfoVector> BalancerChunkSelectionPolicyImpl::selectChunksToMo
const NamespaceString nss(coll.getNs());
if (!coll.getAllowBalance()) {
- LOG(1) << "Not balancing collection " << nss << "; explicitly disabled.";
+ LOGV2_DEBUG(
+ 21851, 1, "Not balancing collection {nss}; explicitly disabled.", "nss"_attr = nss);
continue;
}
@@ -292,8 +298,12 @@ StatusWith<MigrateInfoVector> BalancerChunkSelectionPolicyImpl::selectChunksToMo
// Namespace got dropped before we managed to get to it, so just skip it
continue;
} else if (!candidatesStatus.isOK()) {
- warning() << "Unable to balance collection " << nss.ns()
- << causedBy(candidatesStatus.getStatus());
+ LOGV2_WARNING(
+ 21853,
+ "Unable to balance collection {nss_ns}{causedBy_candidatesStatus_getStatus}",
+ "nss_ns"_attr = nss.ns(),
+ "causedBy_candidatesStatus_getStatus"_attr =
+ causedBy(candidatesStatus.getStatus()));
continue;
}
diff --git a/src/mongo/db/s/balancer/balancer_policy.cpp b/src/mongo/db/s/balancer/balancer_policy.cpp
index 022379992f0..44e3f1302b5 100644
--- a/src/mongo/db/s/balancer/balancer_policy.cpp
+++ b/src/mongo/db/s/balancer/balancer_policy.cpp
@@ -36,6 +36,7 @@
#include <random>
#include "mongo/db/s/balancer/type_migration.h"
+#include "mongo/logv2/log.h"
#include "mongo/s/catalog/type_shard.h"
#include "mongo/s/catalog/type_tags.h"
#include "mongo/util/fail_point.h"
@@ -342,8 +343,11 @@ MigrateInfo chooseRandomMigration(const ShardStatisticsVector& shardStats,
const int destIndex = indices[choice];
const auto& destShardId = shardStats[destIndex].shardId;
- LOG(1) << "balancerShouldReturnRandomMigrations: source: " << sourceShardId
- << " dest: " << destShardId;
+ LOGV2_DEBUG(21880,
+ 1,
+ "balancerShouldReturnRandomMigrations: source: {sourceShardId} dest: {destShardId}",
+ "sourceShardId"_attr = sourceShardId,
+ "destShardId"_attr = destShardId);
const auto& chunks = distribution.getChunks(sourceShardId);
@@ -361,7 +365,7 @@ vector<MigrateInfo> BalancerPolicy::balance(const ShardStatisticsVector& shardSt
if (MONGO_unlikely(balancerShouldReturnRandomMigrations.shouldFail()) &&
!distribution.nss().isConfigDB()) {
- LOG(1) << "balancerShouldReturnRandomMigrations failpoint is set";
+ LOGV2_DEBUG(21881, 1, "balancerShouldReturnRandomMigrations failpoint is set");
if (shardStats.size() < 2)
return migrations;
@@ -402,8 +406,10 @@ vector<MigrateInfo> BalancerPolicy::balance(const ShardStatisticsVector& shardSt
_getLeastLoadedReceiverShard(shardStats, distribution, tag, *usedShards);
if (!to.isValid()) {
if (migrations.empty()) {
- warning() << "Chunk " << redact(chunk.toString())
- << " is on a draining shard, but no appropriate recipient found";
+ LOGV2_WARNING(21889,
+ "Chunk {chunk} is on a draining shard, but no appropriate "
+ "recipient found",
+ "chunk"_attr = redact(chunk.toString()));
}
continue;
}
@@ -417,8 +423,11 @@ vector<MigrateInfo> BalancerPolicy::balance(const ShardStatisticsVector& shardSt
}
if (migrations.empty()) {
- warning() << "Unable to find any chunk to move from draining shard " << stat.shardId
- << ". numJumboChunks: " << numJumboChunks;
+ LOGV2_WARNING(21890,
+ "Unable to find any chunk to move from draining shard "
+ "{stat_shardId}. numJumboChunks: {numJumboChunks}",
+ "stat_shardId"_attr = stat.shardId,
+ "numJumboChunks"_attr = numJumboChunks);
}
}
}
@@ -441,8 +450,11 @@ vector<MigrateInfo> BalancerPolicy::balance(const ShardStatisticsVector& shardSt
continue;
if (chunk.getJumbo()) {
- warning() << "Chunk " << redact(chunk.toString()) << " violates zone "
- << redact(tag) << ", but it is jumbo and cannot be moved";
+ LOGV2_WARNING(
+ 21891,
+ "Chunk {chunk} violates zone {tag}, but it is jumbo and cannot be moved",
+ "chunk"_attr = redact(chunk.toString()),
+ "tag"_attr = redact(tag));
continue;
}
@@ -450,8 +462,11 @@ vector<MigrateInfo> BalancerPolicy::balance(const ShardStatisticsVector& shardSt
_getLeastLoadedReceiverShard(shardStats, distribution, tag, *usedShards);
if (!to.isValid()) {
if (migrations.empty()) {
- warning() << "Chunk " << redact(chunk.toString()) << " violates zone "
- << redact(tag) << ", but no appropriate recipient found";
+ LOGV2_WARNING(
+ 21892,
+ "Chunk {chunk} violates zone {tag}, but no appropriate recipient found",
+ "chunk"_attr = redact(chunk.toString()),
+ "tag"_attr = redact(tag));
}
continue;
}
@@ -490,10 +505,14 @@ vector<MigrateInfo> BalancerPolicy::balance(const ShardStatisticsVector& shardSt
// should not be possible so warn the operator to correct it.
if (totalNumberOfShardsWithTag == 0) {
if (!tag.empty()) {
- warning() << "Zone " << redact(tag) << " in collection " << distribution.nss()
- << " has no assigned shards and chunks which fall into it cannot be "
- "balanced. This should be corrected by either assigning shards to the "
- "zone or by deleting it.";
+ LOGV2_WARNING(
+ 21893,
+ "Zone {tag} in collection {distribution_nss} has no assigned shards and chunks "
+ "which fall into it cannot be "
+ "balanced. This should be corrected by either assigning shards to the "
+ "zone or by deleting it.",
+ "tag"_attr = redact(tag),
+ "distribution_nss"_attr = distribution.nss());
}
continue;
}
@@ -552,7 +571,7 @@ bool BalancerPolicy::_singleZoneBalance(const ShardStatisticsVector& shardStats,
const ShardId to = _getLeastLoadedReceiverShard(shardStats, distribution, tag, *usedShards);
if (!to.isValid()) {
if (migrations->empty()) {
- log() << "No available shards to take chunks for zone [" << tag << "]";
+ LOGV2(21882, "No available shards to take chunks for zone [{tag}]", "tag"_attr = tag);
}
return false;
}
@@ -565,12 +584,22 @@ bool BalancerPolicy::_singleZoneBalance(const ShardStatisticsVector& shardStats,
const size_t imbalance = max - idealNumberOfChunksPerShardForTag;
- LOG(1) << "collection : " << distribution.nss().ns();
- LOG(1) << "zone : " << tag;
- LOG(1) << "donor : " << from << " chunks on " << max;
- LOG(1) << "receiver : " << to << " chunks on " << min;
- LOG(1) << "ideal : " << idealNumberOfChunksPerShardForTag;
- LOG(1) << "threshold : " << kDefaultImbalanceThreshold;
+ LOGV2_DEBUG(21883,
+ 1,
+ "collection : {distribution_nss_ns}",
+ "distribution_nss_ns"_attr = distribution.nss().ns());
+ LOGV2_DEBUG(21884, 1, "zone : {tag}", "tag"_attr = tag);
+ LOGV2_DEBUG(
+ 21885, 1, "donor : {from} chunks on {max}", "from"_attr = from, "max"_attr = max);
+ LOGV2_DEBUG(21886, 1, "receiver : {to} chunks on {min}", "to"_attr = to, "min"_attr = min);
+ LOGV2_DEBUG(21887,
+ 1,
+ "ideal : {idealNumberOfChunksPerShardForTag}",
+ "idealNumberOfChunksPerShardForTag"_attr = idealNumberOfChunksPerShardForTag);
+ LOGV2_DEBUG(21888,
+ 1,
+ "threshold : {kDefaultImbalanceThreshold}",
+ "kDefaultImbalanceThreshold"_attr = kDefaultImbalanceThreshold);
// Check whether it is necessary to balance within this zone
if (imbalance < kDefaultImbalanceThreshold)
@@ -596,9 +625,13 @@ bool BalancerPolicy::_singleZoneBalance(const ShardStatisticsVector& shardStats,
}
if (numJumboChunks) {
- warning() << "Shard: " << from << ", collection: " << distribution.nss().ns()
- << " has only jumbo chunks for zone \'" << tag
- << "\' and cannot be balanced. Jumbo chunks count: " << numJumboChunks;
+ LOGV2_WARNING(21894,
+ "Shard: {from}, collection: {distribution_nss_ns} has only jumbo chunks for "
+ "zone \'{tag}\' and cannot be balanced. Jumbo chunks count: {numJumboChunks}",
+ "from"_attr = from,
+ "distribution_nss_ns"_attr = distribution.nss().ns(),
+ "tag"_attr = tag,
+ "numJumboChunks"_attr = numJumboChunks);
}
return false;
diff --git a/src/mongo/db/s/balancer/cluster_statistics_impl.cpp b/src/mongo/db/s/balancer/cluster_statistics_impl.cpp
index 9fd3ebf675d..73e47dd5c9d 100644
--- a/src/mongo/db/s/balancer/cluster_statistics_impl.cpp
+++ b/src/mongo/db/s/balancer/cluster_statistics_impl.cpp
@@ -38,6 +38,7 @@
#include "mongo/base/status_with.h"
#include "mongo/bson/util/bson_extract.h"
#include "mongo/client/read_preference.h"
+#include "mongo/logv2/log.h"
#include "mongo/s/catalog/type_shard.h"
#include "mongo/s/client/shard_registry.h"
#include "mongo/s/grid.h"
@@ -141,8 +142,12 @@ StatusWith<std::vector<ShardStatistics>> ClusterStatisticsImpl::getStats(Operati
} else {
// Since the mongod version is only used for reporting, there is no need to fail the
// entire round if it cannot be retrieved, so just leave it empty
- log() << "Unable to obtain shard version for " << shard.getName()
- << causedBy(mongoDVersionStatus.getStatus());
+ LOGV2(21895,
+ "Unable to obtain shard version for "
+ "{shard_getName}{causedBy_mongoDVersionStatus_getStatus}",
+ "shard_getName"_attr = shard.getName(),
+ "causedBy_mongoDVersionStatus_getStatus"_attr =
+ causedBy(mongoDVersionStatus.getStatus()));
}
std::set<std::string> shardTags;
diff --git a/src/mongo/db/s/balancer/migration_manager.cpp b/src/mongo/db/s/balancer/migration_manager.cpp
index ae0997fa9de..a14202c643d 100644
--- a/src/mongo/db/s/balancer/migration_manager.cpp
+++ b/src/mongo/db/s/balancer/migration_manager.cpp
@@ -44,6 +44,7 @@
#include "mongo/db/s/balancer/scoped_migration_request.h"
#include "mongo/db/s/balancer/type_migration.h"
#include "mongo/executor/task_executor_pool.h"
+#include "mongo/logv2/log.h"
#include "mongo/rpc/get_status_from_command_result.h"
#include "mongo/s/catalog_cache.h"
#include "mongo/s/client/shard_registry.h"
@@ -234,9 +235,12 @@ void MigrationManager::startRecoveryAndAcquireDistLocks(OperationContext* opCtx)
boost::none);
if (!statusWithMigrationsQueryResponse.isOK()) {
- log() << "Unable to read config.migrations collection documents for balancer migration"
- << " recovery. Abandoning balancer recovery."
- << causedBy(redact(statusWithMigrationsQueryResponse.getStatus()));
+ LOGV2(21896,
+ "Unable to read config.migrations collection documents for balancer migration "
+ "recovery. Abandoning balancer "
+ "recovery.{causedBy_statusWithMigrationsQueryResponse_getStatus}",
+ "causedBy_statusWithMigrationsQueryResponse_getStatus"_attr =
+ causedBy(redact(statusWithMigrationsQueryResponse.getStatus())));
return;
}
@@ -246,9 +250,13 @@ void MigrationManager::startRecoveryAndAcquireDistLocks(OperationContext* opCtx)
// The format of this migration document is incorrect. The balancer holds a distlock for
// this migration, but without parsing the migration document we cannot identify which
// distlock must be released. So we must release all distlocks.
- log() << "Unable to parse config.migrations document '" << redact(migration.toString())
- << "' for balancer migration recovery. Abandoning balancer recovery."
- << causedBy(redact(statusWithMigrationType.getStatus()));
+ LOGV2(21897,
+ "Unable to parse config.migrations document '{migration}' for balancer migration "
+ "recovery. Abandoning balancer "
+ "recovery.{causedBy_statusWithMigrationType_getStatus}",
+ "migration"_attr = redact(migration.toString()),
+ "causedBy_statusWithMigrationType_getStatus"_attr =
+ causedBy(redact(statusWithMigrationType.getStatus())));
return;
}
MigrationType migrateType = std::move(statusWithMigrationType.getValue());
@@ -265,11 +273,13 @@ void MigrationManager::startRecoveryAndAcquireDistLocks(OperationContext* opCtx)
auto statusWithDistLockHandle = distLockManager->tryLockWithLocalWriteConcern(
opCtx, migrateType.getNss().ns(), whyMessage, _lockSessionID);
if (!statusWithDistLockHandle.isOK()) {
- log() << "Failed to acquire distributed lock for collection '"
- << migrateType.getNss().ns()
- << "' during balancer recovery of an active migration. Abandoning"
- << " balancer recovery."
- << causedBy(redact(statusWithDistLockHandle.getStatus()));
+ LOGV2(21898,
+ "Failed to acquire distributed lock for collection '{migrateType_getNss_ns}' "
+ "during balancer recovery of an active migration. Abandoning balancer "
+ "recovery.{causedBy_statusWithDistLockHandle_getStatus}",
+ "migrateType_getNss_ns"_attr = migrateType.getNss().ns(),
+ "causedBy_statusWithDistLockHandle_getStatus"_attr =
+ causedBy(redact(statusWithDistLockHandle.getStatus())));
return;
}
}
@@ -320,9 +330,12 @@ void MigrationManager::finishRecovery(OperationContext* opCtx,
// This shouldn't happen because the collection was intact and sharded when the previous
// config primary was active and the dist locks have been held by the balancer
// throughout. Abort migration recovery.
- log() << "Unable to reload chunk metadata for collection '" << nss
- << "' during balancer recovery. Abandoning recovery."
- << causedBy(redact(routingInfoStatus.getStatus()));
+ LOGV2(21899,
+ "Unable to reload chunk metadata for collection '{nss}' during balancer "
+ "recovery. Abandoning recovery.{causedBy_routingInfoStatus_getStatus}",
+ "nss"_attr = nss,
+ "causedBy_routingInfoStatus_getStatus"_attr =
+ causedBy(redact(routingInfoStatus.getStatus())));
return;
}
diff --git a/src/mongo/db/s/balancer/scoped_migration_request.cpp b/src/mongo/db/s/balancer/scoped_migration_request.cpp
index 3333b5f3ab8..b06cb1fa9f0 100644
--- a/src/mongo/db/s/balancer/scoped_migration_request.cpp
+++ b/src/mongo/db/s/balancer/scoped_migration_request.cpp
@@ -35,6 +35,7 @@
#include "mongo/db/s/balancer/type_migration.h"
#include "mongo/db/write_concern_options.h"
+#include "mongo/logv2/log.h"
#include "mongo/s/client/shard_registry.h"
#include "mongo/s/grid.h"
#include "mongo/util/log.h"
@@ -68,8 +69,11 @@ ScopedMigrationRequest::~ScopedMigrationRequest() {
_opCtx, MigrationType::ConfigNS, migrationDocumentIdentifier, kMajorityWriteConcern);
if (!result.isOK()) {
- LOG(0) << "Failed to remove config.migrations document for migration '"
- << migrationDocumentIdentifier.toString() << "'" << causedBy(redact(result));
+ LOGV2(21900,
+ "Failed to remove config.migrations document for migration "
+ "'{migrationDocumentIdentifier}'{causedBy_result}",
+ "migrationDocumentIdentifier"_attr = migrationDocumentIdentifier.toString(),
+ "causedBy_result"_attr = causedBy(redact(result)));
}
}
@@ -141,10 +145,13 @@ StatusWith<ScopedMigrationRequest> ScopedMigrationRequest::writeMigration(
MigrateInfo activeMigrateInfo = statusWithActiveMigration.getValue().toMigrateInfo();
if (activeMigrateInfo.to != migrateInfo.to ||
activeMigrateInfo.from != migrateInfo.from) {
- log() << "Failed to write document '" << redact(migrateInfo.toString())
- << "' to config.migrations because there is already an active migration for"
- << " that chunk: '" << redact(activeMigrateInfo.toString()) << "'."
- << causedBy(redact(result));
+ LOGV2(21901,
+ "Failed to write document '{migrateInfo}' to config.migrations because there "
+ "is already an active migration for that chunk: "
+ "'{activeMigrateInfo}'.{causedBy_result}",
+ "migrateInfo"_attr = redact(migrateInfo.toString()),
+ "activeMigrateInfo"_attr = redact(activeMigrateInfo.toString()),
+ "causedBy_result"_attr = causedBy(redact(result)));
return result;
}
@@ -195,8 +202,12 @@ Status ScopedMigrationRequest::tryToRemoveMigration() {
void ScopedMigrationRequest::keepDocumentOnDestruct() {
invariant(_opCtx);
_opCtx = nullptr;
- LOG(1) << "Keeping config.migrations document with namespace '" << _nss << "' and minKey '"
- << _minKey << "' for balancer recovery";
+ LOGV2_DEBUG(21902,
+ 1,
+ "Keeping config.migrations document with namespace '{nss}' and minKey '{minKey}' "
+ "for balancer recovery",
+ "nss"_attr = _nss,
+ "minKey"_attr = _minKey);
}
} // namespace mongo