diff options
Diffstat (limited to 'src/mongo/db/s')
51 files changed, 1122 insertions, 501 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 diff --git a/src/mongo/db/s/chunk_splitter.cpp b/src/mongo/db/s/chunk_splitter.cpp index f49d08743a7..3c420e838fb 100644 --- a/src/mongo/db/s/chunk_splitter.cpp +++ b/src/mongo/db/s/chunk_splitter.cpp @@ -44,6 +44,7 @@ #include "mongo/db/s/split_chunk.h" #include "mongo/db/s/split_vector.h" #include "mongo/db/service_context.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" @@ -204,8 +205,11 @@ bool isAutoBalanceEnabled(OperationContext* opCtx, auto collStatus = Grid::get(opCtx)->catalogClient()->getCollection(opCtx, nss); if (!collStatus.isOK()) { - log() << "Auto-split for " << nss << " failed to load collection metadata" - << causedBy(redact(collStatus.getStatus())); + LOGV2(21903, + "Auto-split for {nss} failed to load collection " + "metadata{causedBy_collStatus_getStatus}", + "nss"_attr = nss, + "causedBy_collStatus_getStatus"_attr = causedBy(redact(collStatus.getStatus()))); return false; } @@ -245,7 +249,7 @@ void ChunkSplitter::onStepUp() { } _isPrimary = true; - log() << "The ChunkSplitter has started and will accept autosplit tasks."; + LOGV2(21904, "The ChunkSplitter has started and will accept autosplit tasks."); } void ChunkSplitter::onStepDown() { @@ -255,8 +259,9 @@ void ChunkSplitter::onStepDown() { } _isPrimary = false; - log() << "The ChunkSplitter has stopped and will no longer run new autosplit tasks. Any " - << "autosplit tasks that have already started will be allowed to finish."; + LOGV2(21905, + "The ChunkSplitter has stopped and will no longer run new autosplit tasks. Any autosplit " + "tasks that have already started will be allowed to finish."); } void ChunkSplitter::waitForIdle() { @@ -312,9 +317,13 @@ void ChunkSplitter::_runAutosplit(std::shared_ptr<ChunkSplitStateDriver> chunkSp const uint64_t maxChunkSizeBytes = balancerConfig->getMaxChunkSizeBytes(); - LOG(1) << "about to initiate autosplit: " << redact(chunk.toString()) - << " dataWritten since last check: " << dataWritten - << " maxChunkSizeBytes: " << maxChunkSizeBytes; + LOGV2_DEBUG(21906, + 1, + "about to initiate autosplit: {chunk} dataWritten since last check: " + "{dataWritten} maxChunkSizeBytes: {maxChunkSizeBytes}", + "chunk"_attr = redact(chunk.toString()), + "dataWritten"_attr = dataWritten, + "maxChunkSizeBytes"_attr = maxChunkSizeBytes); chunkSplitStateDriver->prepareSplit(); auto splitPoints = uassertStatusOK(splitVector(opCtx.get(), @@ -328,9 +337,11 @@ void ChunkSplitter::_runAutosplit(std::shared_ptr<ChunkSplitStateDriver> chunkSp maxChunkSizeBytes)); if (splitPoints.empty()) { - LOG(1) - << "ChunkSplitter attempted split but not enough split points were found for chunk " - << redact(chunk.toString()); + LOGV2_DEBUG(21907, + 1, + "ChunkSplitter attempted split but not enough split points were found for " + "chunk {chunk}", + "chunk"_attr = redact(chunk.toString())); // Reset our size estimate that we had prior to splitVector to 0, while still counting // the bytes that have been written in parallel to this split task chunkSplitStateDriver->abandonPrepare(); @@ -377,12 +388,19 @@ void ChunkSplitter::_runAutosplit(std::shared_ptr<ChunkSplitStateDriver> chunkSp const bool shouldBalance = isAutoBalanceEnabled(opCtx.get(), nss, balancerConfig); - log() << "autosplitted " << nss << " chunk: " << redact(chunk.toString()) << " into " - << (splitPoints.size() + 1) << " parts (maxChunkSizeBytes " << maxChunkSizeBytes - << ")" - << (topChunkMinKey.isEmpty() ? "" - : " (top chunk migration suggested" + - (std::string)(shouldBalance ? ")" : ", but no migrations allowed)")); + LOGV2( + 21908, + "autosplitted {nss} chunk: {chunk} into {splitPoints_size_1} parts (maxChunkSizeBytes " + "{maxChunkSizeBytes}){topChunkMinKey_isEmpty_top_chunk_migration_suggested_std_string_" + "shouldBalance_but_no_migrations_allowed}", + "nss"_attr = nss, + "chunk"_attr = redact(chunk.toString()), + "splitPoints_size_1"_attr = (splitPoints.size() + 1), + "maxChunkSizeBytes"_attr = maxChunkSizeBytes, + "topChunkMinKey_isEmpty_top_chunk_migration_suggested_std_string_shouldBalance_but_no_migrations_allowed"_attr = + (topChunkMinKey.isEmpty() ? "" + : " (top chunk migration suggested" + + (std::string)(shouldBalance ? ")" : ", but no migrations allowed)"))); // Because the ShardServerOpObserver uses the metadata from the CSS for tracking incoming // writes, if we split a chunk but do not force a CSS refresh, subsequent inserts will see @@ -403,13 +421,20 @@ void ChunkSplitter::_runAutosplit(std::shared_ptr<ChunkSplitStateDriver> chunkSp // assumption that succeeding inserts will fall on the top chunk. moveChunk(opCtx.get(), nss, topChunkMinKey); } catch (const DBException& ex) { - log() << "Top-chunk optimization failed to move chunk " - << redact(ChunkRange(min, max).toString()) << " in collection " << nss - << " after a successful split" << causedBy(redact(ex.toStatus())); + LOGV2(21909, + "Top-chunk optimization failed to move chunk {ChunkRange_min_max} in collection " + "{nss} after a successful split{causedBy_ex_toStatus}", + "ChunkRange_min_max"_attr = redact(ChunkRange(min, max).toString()), + "nss"_attr = nss, + "causedBy_ex_toStatus"_attr = causedBy(redact(ex.toStatus()))); } } catch (const DBException& ex) { - log() << "Unable to auto-split chunk " << redact(ChunkRange(min, max).toString()) - << " in namespace " << nss << causedBy(redact(ex.toStatus())); + LOGV2(21910, + "Unable to auto-split chunk {ChunkRange_min_max} in namespace " + "{nss}{causedBy_ex_toStatus}", + "ChunkRange_min_max"_attr = redact(ChunkRange(min, max).toString()), + "nss"_attr = nss, + "causedBy_ex_toStatus"_attr = causedBy(redact(ex.toStatus()))); } } diff --git a/src/mongo/db/s/cleanup_orphaned_cmd.cpp b/src/mongo/db/s/cleanup_orphaned_cmd.cpp index 444fd536a50..b7b565f6726 100644 --- a/src/mongo/db/s/cleanup_orphaned_cmd.cpp +++ b/src/mongo/db/s/cleanup_orphaned_cmd.cpp @@ -48,6 +48,7 @@ #include "mongo/db/s/shard_filtering_metadata_refresh.h" #include "mongo/db/s/sharding_state.h" #include "mongo/db/service_context.h" +#include "mongo/logv2/log.h" #include "mongo/s/request_types/migration_secondary_throttle_options.h" #include "mongo/util/log.h" @@ -80,8 +81,9 @@ CleanupResult cleanupOrphanedData(OperationContext* opCtx, auto* const css = CollectionShardingRuntime::get(opCtx, ns); const auto metadata = css->getCurrentMetadata(); if (!metadata->isSharded()) { - LOG(0) << "skipping orphaned data cleanup for " << ns.ns() - << ", collection is not sharded"; + LOGV2(21911, + "skipping orphaned data cleanup for {ns_ns}, collection is not sharded", + "ns_ns"_attr = ns.ns()); return CleanupResult::kDone; } @@ -92,7 +94,7 @@ CleanupResult cleanupOrphanedData(OperationContext* opCtx, << "could not cleanup orphaned data, start key " << startingFromKey << " does not match shard key pattern " << keyPattern; - log() << *errMsg; + LOGV2(21912, "{errMsg}", "errMsg"_attr = *errMsg); return CleanupResult::kError; } } else { @@ -101,8 +103,12 @@ CleanupResult cleanupOrphanedData(OperationContext* opCtx, targetRange = css->getNextOrphanRange(startingFromKey); if (!targetRange) { - LOG(1) << "cleanupOrphaned requested for " << ns.toString() << " starting from " - << redact(startingFromKey) << ", no orphan ranges remain"; + LOGV2_DEBUG(21913, + 1, + "cleanupOrphaned requested for {ns} starting from {startingFromKey}, no " + "orphan ranges remain", + "ns"_attr = ns.toString(), + "startingFromKey"_attr = redact(startingFromKey)); return CleanupResult::kDone; } @@ -115,16 +121,21 @@ CleanupResult cleanupOrphanedData(OperationContext* opCtx, // Sleep waiting for our own deletion. We don't actually care about any others, so there is no // need to call css::waitForClean() here. - LOG(1) << "cleanupOrphaned requested for " << ns.toString() << " starting from " - << redact(startingFromKey) << ", removing next orphan range " - << redact(targetRange->toString()) << "; waiting..."; + LOGV2_DEBUG(21914, + 1, + "cleanupOrphaned requested for {ns} starting from {startingFromKey}, removing next " + "orphan range {targetRange}; waiting...", + "ns"_attr = ns.toString(), + "startingFromKey"_attr = redact(startingFromKey), + "targetRange"_attr = redact(targetRange->toString())); Status result = cleanupCompleteFuture.getNoThrow(opCtx); - LOG(1) << "Finished waiting for last " << ns.toString() << " orphan range cleanup"; + LOGV2_DEBUG( + 21915, 1, "Finished waiting for last {ns} orphan range cleanup", "ns"_attr = ns.toString()); if (!result.isOK()) { - log() << redact(result.reason()); + LOGV2(21916, "{result_reason}", "result_reason"_attr = redact(result.reason())); *errMsg = result.reason(); return CleanupResult::kError; } diff --git a/src/mongo/db/s/collection_sharding_runtime.cpp b/src/mongo/db/s/collection_sharding_runtime.cpp index f5d679b83b4..d2ce632ad95 100644 --- a/src/mongo/db/s/collection_sharding_runtime.cpp +++ b/src/mongo/db/s/collection_sharding_runtime.cpp @@ -39,6 +39,7 @@ #include "mongo/db/s/sharded_connection_info.h" #include "mongo/db/s/sharding_runtime_d_params_gen.h" #include "mongo/db/s/sharding_state.h" +#include "mongo/logv2/log.h" #include "mongo/util/duration.h" #include "mongo/util/log.h" @@ -205,7 +206,10 @@ void CollectionShardingRuntime::setFilteringMetadata(OperationContext* opCtx, stdx::lock_guard lk(_metadataManagerLock); if (!newMetadata.isSharded()) { - LOG(0) << "Marking collection " << _nss.ns() << " as " << newMetadata.toStringBasic(); + LOGV2(21917, + "Marking collection {nss_ns} as {newMetadata_Basic}", + "nss_ns"_attr = _nss.ns(), + "newMetadata_Basic"_attr = newMetadata.toStringBasic()); _metadataType = MetadataType::kUnsharded; _metadataManager.reset(); ++_numMetadataManagerChanges; @@ -269,13 +273,18 @@ Status CollectionShardingRuntime::waitForClean(OperationContext* opCtx, stillScheduled = self->_metadataManager->trackOrphanedDataCleanup(orphanRange); if (!stillScheduled) { - log() << "Finished deleting " << nss.ns() << " range " - << redact(orphanRange.toString()); + LOGV2(21918, + "Finished deleting {nss_ns} range {orphanRange}", + "nss_ns"_attr = nss.ns(), + "orphanRange"_attr = redact(orphanRange.toString())); return Status::OK(); } } - log() << "Waiting for deletion of " << nss.ns() << " range " << orphanRange; + LOGV2(21919, + "Waiting for deletion of {nss_ns} range {orphanRange}", + "nss_ns"_attr = nss.ns(), + "orphanRange"_attr = orphanRange); Status result = stillScheduled->getNoThrow(opCtx); diff --git a/src/mongo/db/s/config/configsvr_add_shard_command.cpp b/src/mongo/db/s/config/configsvr_add_shard_command.cpp index bf61721b30f..6f6b449b0cf 100644 --- a/src/mongo/db/s/config/configsvr_add_shard_command.cpp +++ b/src/mongo/db/s/config/configsvr_add_shard_command.cpp @@ -42,6 +42,7 @@ #include "mongo/db/repl/repl_set_config.h" #include "mongo/db/repl/replication_coordinator.h" #include "mongo/db/s/config/sharding_catalog_manager.h" +#include "mongo/logv2/log.h" #include "mongo/s/catalog/type_shard.h" #include "mongo/s/grid.h" #include "mongo/s/request_types/add_shard_request_type.h" @@ -129,8 +130,10 @@ public: parsedRequest.hasMaxSize() ? parsedRequest.getMaxSize() : kMaxSizeMBDefault); if (!addShardResult.isOK()) { - log() << "addShard request '" << parsedRequest << "'" - << "failed" << causedBy(addShardResult.getStatus()); + LOGV2(21920, + "addShard request '{parsedRequest}'failed{causedBy_addShardResult_getStatus}", + "parsedRequest"_attr = parsedRequest, + "causedBy_addShardResult_getStatus"_attr = causedBy(addShardResult.getStatus())); uassertStatusOK(addShardResult.getStatus()); } diff --git a/src/mongo/db/s/config/configsvr_move_primary_command.cpp b/src/mongo/db/s/config/configsvr_move_primary_command.cpp index fe5c843303e..1595d7fdcdd 100644 --- a/src/mongo/db/s/config/configsvr_move_primary_command.cpp +++ b/src/mongo/db/s/config/configsvr_move_primary_command.cpp @@ -44,6 +44,7 @@ #include "mongo/db/repl/repl_client_info.h" #include "mongo/db/s/config/sharding_catalog_manager.h" #include "mongo/db/server_options.h" +#include "mongo/logv2/log.h" #include "mongo/s/catalog/type_database.h" #include "mongo/s/catalog_cache.h" #include "mongo/s/client/shard_registry.h" @@ -158,8 +159,13 @@ public: const auto toShard = [&]() { auto toShardStatus = shardRegistry->getShard(opCtx, to); if (!toShardStatus.isOK()) { - log() << "Could not move database '" << dbname << "' to shard '" << to - << causedBy(toShardStatus.getStatus()); + LOGV2(21921, + "Could not move database '{dbname}' to shard " + "'{to}{causedBy_toShardStatus_getStatus}", + "dbname"_attr = dbname, + "to"_attr = to, + "causedBy_toShardStatus_getStatus"_attr = + causedBy(toShardStatus.getStatus())); uassertStatusOKWithContext(toShardStatus.getStatus(), str::stream() << "Could not move database '" << dbname << "' to shard '" << to << "'"); diff --git a/src/mongo/db/s/config/configsvr_refine_collection_shard_key_command.cpp b/src/mongo/db/s/config/configsvr_refine_collection_shard_key_command.cpp index 3b6a9337014..3d862d632d1 100644 --- a/src/mongo/db/s/config/configsvr_refine_collection_shard_key_command.cpp +++ b/src/mongo/db/s/config/configsvr_refine_collection_shard_key_command.cpp @@ -38,6 +38,7 @@ #include "mongo/db/s/config/sharding_catalog_manager.h" #include "mongo/db/s/shard_key_util.h" #include "mongo/db/server_options.h" +#include "mongo/logv2/log.h" #include "mongo/s/catalog/dist_lock_manager.h" #include "mongo/s/grid.h" #include "mongo/s/request_types/refine_collection_shard_key_gen.h" @@ -148,7 +149,9 @@ public: collType.getUnique(), false); // createIndexIfPossible - LOG(0) << "CMD: refineCollectionShardKey: " << request().toBSON({}); + LOGV2(21922, + "CMD: refineCollectionShardKey: {request}", + "request"_attr = request().toBSON({})); audit::logRefineCollectionShardKey(opCtx->getClient(), nss.ns(), proposedKey); diff --git a/src/mongo/db/s/config/configsvr_remove_shard_command.cpp b/src/mongo/db/s/config/configsvr_remove_shard_command.cpp index 9fd9e7298b4..2b3cfc9a1ae 100644 --- a/src/mongo/db/s/config/configsvr_remove_shard_command.cpp +++ b/src/mongo/db/s/config/configsvr_remove_shard_command.cpp @@ -42,6 +42,7 @@ #include "mongo/db/operation_context.h" #include "mongo/db/repl/read_concern_args.h" #include "mongo/db/s/config/sharding_catalog_manager.h" +#include "mongo/logv2/log.h" #include "mongo/s/catalog/type_database.h" #include "mongo/s/catalog_cache.h" #include "mongo/s/client/shard_registry.h" @@ -116,7 +117,7 @@ public: try { return shardingCatalogManager->removeShard(opCtx, shardId); } catch (const DBException& ex) { - LOG(0) << "Failed to remove shard due to " << redact(ex); + LOGV2(21923, "Failed to remove shard due to {ex}", "ex"_attr = redact(ex)); throw; } }(); diff --git a/src/mongo/db/s/config/sharding_catalog_manager_collection_operations.cpp b/src/mongo/db/s/config/sharding_catalog_manager_collection_operations.cpp index 38ef0df3f11..b1d23be5b0c 100644 --- a/src/mongo/db/s/config/sharding_catalog_manager_collection_operations.cpp +++ b/src/mongo/db/s/config/sharding_catalog_manager_collection_operations.cpp @@ -56,6 +56,7 @@ #include "mongo/db/s/sharding_logging.h" #include "mongo/executor/network_interface.h" #include "mongo/executor/task_executor.h" +#include "mongo/logv2/log.h" #include "mongo/rpc/get_status_from_command_result.h" #include "mongo/s/balancer_configuration.h" #include "mongo/s/catalog/sharding_catalog_client_impl.h" @@ -421,15 +422,16 @@ void ShardingCatalogManager::dropCollection(OperationContext* opCtx, const Names BSONObj(), ShardingCatalogClient::kMajorityWriteConcern)); - LOG(1) << "dropCollection " << nss.ns() << " started"; + LOGV2_DEBUG(21924, 1, "dropCollection {nss_ns} started", "nss_ns"_attr = nss.ns()); sendDropCollectionToAllShards(opCtx, nss); - LOG(1) << "dropCollection " << nss.ns() << " shard data deleted"; + LOGV2_DEBUG(21925, 1, "dropCollection {nss_ns} shard data deleted", "nss_ns"_attr = nss.ns()); removeChunksAndTagsForDroppedCollection(opCtx, nss); - LOG(1) << "dropCollection " << nss.ns() << " chunk and tag data deleted"; + LOGV2_DEBUG( + 21926, 1, "dropCollection {nss_ns} chunk and tag data deleted", "nss_ns"_attr = nss.ns()); // Mark the collection as dropped CollectionType coll; @@ -442,11 +444,12 @@ void ShardingCatalogManager::dropCollection(OperationContext* opCtx, const Names uassertStatusOK(ShardingCatalogClientImpl::updateShardingCatalogEntryForCollection( opCtx, nss, coll, upsert)); - LOG(1) << "dropCollection " << nss.ns() << " collection marked as dropped"; + LOGV2_DEBUG( + 21927, 1, "dropCollection {nss_ns} collection marked as dropped", "nss_ns"_attr = nss.ns()); sendSSVToAllShards(opCtx, nss); - LOG(1) << "dropCollection " << nss.ns() << " completed"; + LOGV2_DEBUG(21928, 1, "dropCollection {nss_ns} completed", "nss_ns"_attr = nss.ns()); ShardingLogging::get(opCtx)->logChange( opCtx, "dropCollection", nss.ns(), BSONObj(), ShardingCatalogClient::kMajorityWriteConcern); @@ -455,8 +458,10 @@ void ShardingCatalogManager::dropCollection(OperationContext* opCtx, const Names void ShardingCatalogManager::ensureDropCollectionCompleted(OperationContext* opCtx, const NamespaceString& nss) { - LOG(1) << "Ensuring config entries for " << nss.ns() - << " from previous dropCollection are cleared"; + LOGV2_DEBUG(21929, + 1, + "Ensuring config entries for {nss_ns} from previous dropCollection are cleared", + "nss_ns"_attr = nss.ns()); sendDropCollectionToAllShards(opCtx, nss); removeChunksAndTagsForDroppedCollection(opCtx, nss); sendSSVToAllShards(opCtx, nss); @@ -479,7 +484,7 @@ void ShardingCatalogManager::generateUUIDsForExistingShardedCollections(Operatio .docs; if (shardedColls.empty()) { - LOG(0) << "all sharded collections already have UUIDs"; + LOGV2(21930, "all sharded collections already have UUIDs"); // We did a local read of the collections collection above and found that all sharded // collections already have UUIDs. However, the data may not be majority committed (a @@ -491,8 +496,10 @@ void ShardingCatalogManager::generateUUIDsForExistingShardedCollections(Operatio } // Generate and persist a new UUID for each collection that did not have a UUID. - LOG(0) << "generating UUIDs for " << shardedColls.size() - << " sharded collections that do not yet have a UUID"; + LOGV2( + 21931, + "generating UUIDs for {shardedColls_size} sharded collections that do not yet have a UUID", + "shardedColls_size"_attr = shardedColls.size()); for (auto& coll : shardedColls) { auto collType = uassertStatusOK(CollectionType::fromBSON(coll)); invariant(!collType.getUUID()); @@ -502,8 +509,12 @@ void ShardingCatalogManager::generateUUIDsForExistingShardedCollections(Operatio uassertStatusOK(ShardingCatalogClientImpl::updateShardingCatalogEntryForCollection( opCtx, collType.getNs(), collType, false /* upsert */)); - LOG(2) << "updated entry in config.collections for sharded collection " << collType.getNs() - << " with generated UUID " << uuid; + LOGV2_DEBUG(21932, + 2, + "updated entry in config.collections for sharded collection {collType_getNs} " + "with generated UUID {uuid}", + "collType_getNs"_attr = collType.getNs(), + "uuid"_attr = uuid); } } @@ -617,9 +628,12 @@ void ShardingCatalogManager::refineCollectionShardKey(OperationContext* opCtx, true /* startTransaction */, txnNumber)); - log() << "refineCollectionShardKey: updated collection entry for '" << nss.ns() - << "': took " << executionTimer.millis() - << " ms. Total time taken: " << totalTimer.millis() << " ms."; + LOGV2(21933, + "refineCollectionShardKey: updated collection entry for '{nss_ns}': took " + "{executionTimer_millis} ms. Total time taken: {totalTimer_millis} ms.", + "nss_ns"_attr = nss.ns(), + "executionTimer_millis"_attr = executionTimer.millis(), + "totalTimer_millis"_attr = totalTimer.millis()); executionTimer.reset(); // Update all config.chunks entries for the given namespace by setting (i) their epoch to @@ -627,7 +641,7 @@ void ShardingCatalogManager::refineCollectionShardKey(OperationContext* opCtx, // MinKey (except for the global max chunk where the max bounds are set to MaxKey), and // unsetting (iii) their jumbo field. if (MONGO_unlikely(hangRefineCollectionShardKeyBeforeUpdatingChunks.shouldFail())) { - log() << "Hit hangRefineCollectionShardKeyBeforeUpdatingChunks failpoint"; + LOGV2(21934, "Hit hangRefineCollectionShardKeyBeforeUpdatingChunks failpoint"); hangRefineCollectionShardKeyBeforeUpdatingChunks.pauseWhileSet(opCtx); } @@ -653,9 +667,12 @@ void ShardingCatalogManager::refineCollectionShardKey(OperationContext* opCtx, false, // startTransaction txnNumber)); - log() << "refineCollectionShardKey: updated chunk entries for '" << nss.ns() << "': took " - << executionTimer.millis() << " ms. Total time taken: " << totalTimer.millis() - << " ms."; + LOGV2(21935, + "refineCollectionShardKey: updated chunk entries for '{nss_ns}': took " + "{executionTimer_millis} ms. Total time taken: {totalTimer_millis} ms.", + "nss_ns"_attr = nss.ns(), + "executionTimer_millis"_attr = executionTimer.millis(), + "totalTimer_millis"_attr = totalTimer.millis()); executionTimer.reset(); // Update all config.tags entries for the given namespace by setting their bounds for each @@ -680,12 +697,15 @@ void ShardingCatalogManager::refineCollectionShardKey(OperationContext* opCtx, false, // startTransaction txnNumber)); - log() << "refineCollectionShardKey: updated zone entries for '" << nss.ns() << "': took " - << executionTimer.millis() << " ms. Total time taken: " << totalTimer.millis() - << " ms."; + LOGV2(21936, + "refineCollectionShardKey: updated zone entries for '{nss_ns}': took " + "{executionTimer_millis} ms. Total time taken: {totalTimer_millis} ms.", + "nss_ns"_attr = nss.ns(), + "executionTimer_millis"_attr = executionTimer.millis(), + "totalTimer_millis"_attr = totalTimer.millis()); if (MONGO_unlikely(hangRefineCollectionShardKeyBeforeCommit.shouldFail())) { - log() << "Hit hangRefineCollectionShardKeyBeforeCommit failpoint"; + LOGV2(21937, "Hit hangRefineCollectionShardKeyBeforeCommit failpoint"); hangRefineCollectionShardKeyBeforeCommit.pauseWhileSet(opCtx); } diff --git a/src/mongo/db/s/config/sharding_catalog_manager_database_operations.cpp b/src/mongo/db/s/config/sharding_catalog_manager_database_operations.cpp index 4028d41a34d..c46a65966a7 100644 --- a/src/mongo/db/s/config/sharding_catalog_manager_database_operations.cpp +++ b/src/mongo/db/s/config/sharding_catalog_manager_database_operations.cpp @@ -39,6 +39,7 @@ #include "mongo/db/repl/repl_client_info.h" #include "mongo/db/server_options.h" #include "mongo/db/write_concern.h" +#include "mongo/logv2/log.h" #include "mongo/s/catalog/type_database.h" #include "mongo/s/catalog_cache.h" #include "mongo/s/client/shard.h" @@ -150,7 +151,7 @@ DatabaseType ShardingCatalogManager::createDatabase(OperationContext* opCtx, // Pick a primary shard for the new database. DatabaseType db(dbName.toString(), shardPtr->getId(), false, databaseVersion::makeNew()); - log() << "Registering new database " << db << " in sharding catalog"; + LOGV2(21938, "Registering new database {db} in sharding catalog", "db"_attr = db); // Do this write with majority writeConcern to guarantee that the shard sees the write when it // receives the _flushDatabaseCacheUpdates. @@ -214,7 +215,7 @@ void ShardingCatalogManager::enableSharding(OperationContext* opCtx, Milliseconds{30000}), &unusedResult)); - log() << "Enabling sharding for database [" << dbName << "] in config db"; + LOGV2(21939, "Enabling sharding for database [{dbName}] in config db", "dbName"_attr = dbName); uassertStatusOK(Grid::get(opCtx)->catalogClient()->updateConfigDocument( opCtx, @@ -303,8 +304,10 @@ Status ShardingCatalogManager::commitMovePrimary(OperationContext* opCtx, ShardingCatalogClient::kLocalWriteConcern); if (!updateStatus.isOK()) { - log() << "error committing movePrimary: " << dbname - << causedBy(redact(updateStatus.getStatus())); + LOGV2(21940, + "error committing movePrimary: {dbname}{causedBy_updateStatus_getStatus}", + "dbname"_attr = dbname, + "causedBy_updateStatus_getStatus"_attr = causedBy(redact(updateStatus.getStatus()))); return updateStatus.getStatus(); } diff --git a/src/mongo/db/s/config/sharding_catalog_manager_shard_operations.cpp b/src/mongo/db/s/config/sharding_catalog_manager_shard_operations.cpp index c388e826010..b2148656e52 100644 --- a/src/mongo/db/s/config/sharding_catalog_manager_shard_operations.cpp +++ b/src/mongo/db/s/config/sharding_catalog_manager_shard_operations.cpp @@ -59,6 +59,7 @@ #include "mongo/db/s/type_shard_identity.h" #include "mongo/db/wire_version.h" #include "mongo/executor/task_executor.h" +#include "mongo/logv2/log.h" #include "mongo/rpc/get_status_from_command_result.h" #include "mongo/s/catalog/config_server_version.h" #include "mongo/s/catalog/sharding_catalog_client.h" @@ -161,7 +162,9 @@ StatusWith<Shard::CommandResponse> ShardingCatalogManager::_runCommandForAddShar _executorForAddShard->wait(swCallbackHandle.getValue()); if (response.status == ErrorCodes::ExceededTimeLimit) { - LOG(0) << "Operation timed out with status " << redact(response.status); + LOGV2(21941, + "Operation timed out with status {response_status}", + "response_status"_attr = redact(response.status)); } if (!response.isOK()) { @@ -672,7 +675,9 @@ StatusWith<std::string> ShardingCatalogManager::addShard( return versionResponse.getValue().commandStatus; } - log() << "going to insert new entry for shard into config.shards: " << shardType.toString(); + LOGV2(21942, + "going to insert new entry for shard into config.shards: {shardType}", + "shardType"_attr = shardType.toString()); Status result = Grid::get(opCtx)->catalogClient()->insertConfigDocument( opCtx, @@ -680,7 +685,10 @@ StatusWith<std::string> ShardingCatalogManager::addShard( shardType.toBSON(), ShardingCatalogClient::kLocalWriteConcern); if (!result.isOK()) { - log() << "error adding shard: " << shardType.toBSON() << " err: " << result.reason(); + LOGV2(21943, + "error adding shard: {shardType} err: {result_reason}", + "shardType"_attr = shardType.toBSON(), + "result_reason"_attr = result.reason()); return result; } } @@ -698,8 +706,11 @@ StatusWith<std::string> ShardingCatalogManager::addShard( true, ShardingCatalogClient::kLocalWriteConcern); if (!status.isOK()) { - log() << "adding shard " << shardConnectionString.toString() - << " even though could not add database " << dbName; + LOGV2(21944, + "adding shard {shardConnectionString} even though could not add database " + "{dbName}", + "shardConnectionString"_attr = shardConnectionString.toString(), + "dbName"_attr = dbName); } } } @@ -773,7 +784,7 @@ RemoveShardProgress ShardingCatalogManager::removeShard(OperationContext* opCtx, auto* const catalogClient = Grid::get(opCtx)->catalogClient(); if (!isShardCurrentlyDraining) { - log() << "going to start draining shard: " << name; + LOGV2(21945, "going to start draining shard: {name}", "name"_attr = name); // Record start in changelog uassertStatusOK(ShardingLogging::get(opCtx)->logChangeChecked( @@ -811,9 +822,9 @@ RemoveShardProgress ShardingCatalogManager::removeShard(OperationContext* opCtx, if (chunkCount > 0 || databaseCount > 0) { // Still more draining to do - LOG(0) << "chunkCount: " << chunkCount; - LOG(0) << "databaseCount: " << databaseCount; - LOG(0) << "jumboCount: " << jumboCount; + LOGV2(21946, "chunkCount: {chunkCount}", "chunkCount"_attr = chunkCount); + LOGV2(21947, "databaseCount: {databaseCount}", "databaseCount"_attr = databaseCount); + LOGV2(21948, "jumboCount: {jumboCount}", "jumboCount"_attr = jumboCount); return {RemoveShardProgress::ONGOING, boost::optional<RemoveShardProgress::DrainingShardUsage>( @@ -821,7 +832,7 @@ RemoveShardProgress ShardingCatalogManager::removeShard(OperationContext* opCtx, } // Draining is done, now finish removing the shard. - log() << "going to remove shard: " << name; + LOGV2(21949, "going to remove shard: {name}", "name"_attr = name); audit::logRemoveShard(opCtx->getClient(), name); uassertStatusOKWithContext( diff --git a/src/mongo/db/s/database_sharding_state.cpp b/src/mongo/db/s/database_sharding_state.cpp index 9580cf6026a..ed018daec96 100644 --- a/src/mongo/db/s/database_sharding_state.cpp +++ b/src/mongo/db/s/database_sharding_state.cpp @@ -35,6 +35,7 @@ #include "mongo/db/operation_context.h" #include "mongo/db/s/operation_sharding_state.h" +#include "mongo/logv2/log.h" #include "mongo/s/database_version_helpers.h" #include "mongo/s/stale_exception.h" #include "mongo/util/fail_point.h" @@ -120,8 +121,12 @@ void DatabaseShardingState::setDbVersion(OperationContext* opCtx, boost::optional<DatabaseVersion> newDbVersion, DSSLock&) { invariant(opCtx->lockState()->isDbLockedForMode(_dbName, MODE_X)); - log() << "setting this node's cached database version for " << _dbName << " to " - << (newDbVersion ? newDbVersion->toBSON() : BSONObj()); + LOGV2(21950, + "setting this node's cached database version for {dbName} to " + "{newDbVersion_newDbVersion_BSONObj}", + "dbName"_attr = _dbName, + "newDbVersion_newDbVersion_BSONObj"_attr = + (newDbVersion ? newDbVersion->toBSON() : BSONObj())); _dbVersion = newDbVersion; } diff --git a/src/mongo/db/s/flush_database_cache_updates_command.cpp b/src/mongo/db/s/flush_database_cache_updates_command.cpp index 5e4fadfaae0..e96159b3cbd 100644 --- a/src/mongo/db/s/flush_database_cache_updates_command.cpp +++ b/src/mongo/db/s/flush_database_cache_updates_command.cpp @@ -50,6 +50,7 @@ #include "mongo/s/request_types/flush_database_cache_updates_gen.h" +#include "mongo/logv2/log.h" #include "mongo/util/log.h" namespace mongo { @@ -133,7 +134,10 @@ public: oss.waitForMigrationCriticalSectionSignal(opCtx); if (request().getSyncFromConfig()) { - LOG(1) << "Forcing remote routing table refresh for " << _dbName(); + LOGV2_DEBUG(21981, + 1, + "Forcing remote routing table refresh for {dbName}", + "dbName"_attr = _dbName()); forceDatabaseRefresh(opCtx, _dbName()); } diff --git a/src/mongo/db/s/flush_routing_table_cache_updates_command.cpp b/src/mongo/db/s/flush_routing_table_cache_updates_command.cpp index f2791bf1fbd..2a2fd18a89f 100644 --- a/src/mongo/db/s/flush_routing_table_cache_updates_command.cpp +++ b/src/mongo/db/s/flush_routing_table_cache_updates_command.cpp @@ -44,6 +44,7 @@ #include "mongo/db/s/operation_sharding_state.h" #include "mongo/db/s/shard_filtering_metadata_refresh.h" #include "mongo/db/s/sharding_state.h" +#include "mongo/logv2/log.h" #include "mongo/s/catalog_cache_loader.h" #include "mongo/s/grid.h" #include "mongo/s/request_types/flush_routing_table_cache_updates_gen.h" @@ -132,7 +133,8 @@ public: oss.waitForMigrationCriticalSectionSignal(opCtx); if (request().getSyncFromConfig()) { - LOG(1) << "Forcing remote routing table refresh for " << ns(); + LOGV2_DEBUG( + 21982, 1, "Forcing remote routing table refresh for {ns}", "ns"_attr = ns()); forceShardFilteringMetadataRefresh(opCtx, ns()); } diff --git a/src/mongo/db/s/merge_chunks_command.cpp b/src/mongo/db/s/merge_chunks_command.cpp index d6db92888e2..5fd4574590b 100644 --- a/src/mongo/db/s/merge_chunks_command.cpp +++ b/src/mongo/db/s/merge_chunks_command.cpp @@ -42,6 +42,7 @@ #include "mongo/db/s/collection_sharding_state.h" #include "mongo/db/s/shard_filtering_metadata_refresh.h" #include "mongo/db/s/sharding_state.h" +#include "mongo/logv2/log.h" #include "mongo/s/catalog/type_chunk.h" #include "mongo/s/client/shard_registry.h" #include "mongo/s/grid.h" @@ -227,8 +228,11 @@ void mergeChunks(OperationContext* opCtx, if ((!commandStatus.isOK() || !writeConcernStatus.isOK()) && checkMetadataForSuccess(opCtx, nss, epoch, ChunkRange(minKey, maxKey))) { - LOG(1) << "mergeChunk [" << redact(minKey) << "," << redact(maxKey) - << ") has already been committed."; + LOGV2_DEBUG(21983, + 1, + "mergeChunk [{minKey},{maxKey}) has already been committed.", + "minKey"_attr = redact(minKey), + "maxKey"_attr = redact(maxKey)); return; } diff --git a/src/mongo/db/s/metadata_manager.cpp b/src/mongo/db/s/metadata_manager.cpp index f038c328052..ca3c324e7f8 100644 --- a/src/mongo/db/s/metadata_manager.cpp +++ b/src/mongo/db/s/metadata_manager.cpp @@ -42,6 +42,7 @@ #include "mongo/db/range_arithmetic.h" #include "mongo/db/s/range_deletion_util.h" #include "mongo/db/s/sharding_runtime_d_params_gen.h" +#include "mongo/logv2/log.h" #include "mongo/s/grid.h" #include "mongo/util/assert_util.h" #include "mongo/util/fail_point.h" @@ -167,14 +168,21 @@ void MetadataManager::setFilteringMetadata(CollectionMetadata remoteMetadata) { // We already have the same or newer version if (activeMetadata.getCollVersion().epoch() == remoteMetadata.getCollVersion().epoch() && activeMetadata.getCollVersion() >= remoteMetadata.getCollVersion()) { - LOG(1) << "Ignoring update of active metadata " << activeMetadata.toStringBasic() - << " with an older " << remoteMetadata.toStringBasic(); + LOGV2_DEBUG(21984, + 1, + "Ignoring update of active metadata {activeMetadata_Basic} with an older " + "{remoteMetadata_Basic}", + "activeMetadata_Basic"_attr = activeMetadata.toStringBasic(), + "remoteMetadata_Basic"_attr = remoteMetadata.toStringBasic()); return; } - LOG(0) << "Updating metadata for collection " << _nss.ns() << " from " - << activeMetadata.toStringBasic() << " to " << remoteMetadata.toStringBasic() - << " due to version change"; + LOGV2(21985, + "Updating metadata for collection {nss_ns} from {activeMetadata_Basic} to " + "{remoteMetadata_Basic} due to version change", + "nss_ns"_attr = _nss.ns(), + "activeMetadata_Basic"_attr = activeMetadata.toStringBasic(), + "remoteMetadata_Basic"_attr = remoteMetadata.toStringBasic()); // Resolve any receiving chunks, which might have completed by now for (auto it = _receivingChunks.begin(); it != _receivingChunks.end();) { @@ -187,8 +195,12 @@ void MetadataManager::setFilteringMetadata(CollectionMetadata remoteMetadata) { // The remote metadata contains a chunk we were earlier in the process of receiving, so we // deem it successfully received - LOG(2) << "Verified chunk " << redact(receivingRange.toString()) << " for collection " - << _nss.ns() << " has been migrated to this shard earlier"; + LOGV2_DEBUG(21986, + 2, + "Verified chunk {receivingRange} for collection {nss_ns} has been migrated to " + "this shard earlier", + "receivingRange"_attr = redact(receivingRange.toString()), + "nss_ns"_attr = _nss.ns()); _receivingChunks.erase(it); it = _receivingChunks.begin(); @@ -279,8 +291,11 @@ SharedSemiFuture<void> MetadataManager::beginReceive(ChunkRange const& range) { _receivingChunks.emplace(range.getMin().getOwned(), range.getMax().getOwned()); - log() << "Scheduling deletion of any documents in " << _nss.ns() << " range " - << redact(range.toString()) << " before migrating in a chunk covering the range"; + LOGV2(21987, + "Scheduling deletion of any documents in {nss_ns} range {range} before migrating in a " + "chunk covering the range", + "nss_ns"_attr = _nss.ns(), + "range"_attr = redact(range.toString())); return _submitRangeForDeletion( lg, SemiFuture<void>::makeReady(), range, Seconds(orphanCleanupDelaySecs.load())); @@ -292,8 +307,11 @@ void MetadataManager::forgetReceive(ChunkRange const& range) { // This is potentially a partially received chunk, which needs to be cleaned up. We know none // of these documents are in use, so they can go straight to the deletion queue. - log() << "Abandoning in-migration of " << _nss.ns() << " range " << range - << "; scheduling deletion of any documents already copied"; + LOGV2(21988, + "Abandoning in-migration of {nss_ns} range {range}; scheduling deletion of any documents " + "already copied", + "nss_ns"_attr = _nss.ns(), + "range"_attr = range); invariant(!_overlapsInUseChunk(lg, range)); @@ -327,8 +345,11 @@ SharedSemiFuture<void> MetadataManager::cleanUpRange(ChunkRange const& range, shouldDelayBeforeDeletion ? Seconds(orphanCleanupDelaySecs.load()) : Seconds(0); if (overlapMetadata) { - log() << "Deletion of " << _nss.ns() << " range " << redact(range.toString()) - << " will be scheduled after all possibly dependent queries finish"; + LOGV2(21989, + "Deletion of {nss_ns} range {range} will be scheduled after all possibly dependent " + "queries finish", + "nss_ns"_attr = _nss.ns(), + "range"_attr = redact(range.toString())); ++overlapMetadata->numContingentRangeDeletionTasks; // Schedule the range for deletion once the overlapping metadata object is destroyed // (meaning no more queries can be using the range) and obtain a future which will be @@ -339,7 +360,10 @@ SharedSemiFuture<void> MetadataManager::cleanUpRange(ChunkRange const& range, delayForActiveQueriesOnSecondariesToComplete); } else { // No running queries can depend on this range, so queue it for deletion immediately. - log() << "Scheduling deletion of " << _nss.ns() << " range " << redact(range.toString()); + LOGV2(21990, + "Scheduling deletion of {nss_ns} range {range}", + "nss_ns"_attr = _nss.ns(), + "range"_attr = redact(range.toString())); return _submitRangeForDeletion( lg, SemiFuture<void>::makeReady(), range, delayForActiveQueriesOnSecondariesToComplete); diff --git a/src/mongo/db/s/migration_chunk_cloner_source_legacy.cpp b/src/mongo/db/s/migration_chunk_cloner_source_legacy.cpp index 454fe2ac76b..76dd0bcde8e 100644 --- a/src/mongo/db/s/migration_chunk_cloner_source_legacy.cpp +++ b/src/mongo/db/s/migration_chunk_cloner_source_legacy.cpp @@ -52,6 +52,7 @@ #include "mongo/executor/remote_command_response.h" #include "mongo/executor/task_executor.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/client/shard_registry.h" #include "mongo/s/grid.h" @@ -182,7 +183,9 @@ void LogTransactionOperationsForShardingHandler::commit(boost::optional<Timestam auto idElement = documentKey["_id"]; if (idElement.eoo()) { - warning() << "Received a document with no id, ignoring: " << redact(documentKey); + LOGV2_WARNING(21994, + "Received a document with no id, ignoring: {documentKey}", + "documentKey"_attr = redact(documentKey)); continue; } @@ -396,7 +399,9 @@ void MigrationChunkClonerSourceLegacy::cancelClone(OperationContext* opCtx) { kRecvChunkAbort, _args.getNss(), _sessionId)) .getStatus(); if (!status.isOK()) { - LOG(0) << "Failed to cancel migration " << causedBy(redact(status)); + LOGV2(21991, + "Failed to cancel migration {causedBy_status}", + "causedBy_status"_attr = causedBy(redact(status))); } } // Intentional fall through @@ -419,8 +424,10 @@ void MigrationChunkClonerSourceLegacy::onInsertOp(OperationContext* opCtx, BSONElement idElement = insertedDoc["_id"]; if (idElement.eoo()) { - warning() << "logInsertOp got a document with no _id field, ignoring inserted document: " - << redact(insertedDoc); + LOGV2_WARNING(21995, + "logInsertOp got a document with no _id field, ignoring inserted document: " + "{insertedDoc}", + "insertedDoc"_attr = redact(insertedDoc)); return; } @@ -450,8 +457,10 @@ void MigrationChunkClonerSourceLegacy::onUpdateOp(OperationContext* opCtx, BSONElement idElement = postImageDoc["_id"]; if (idElement.eoo()) { - warning() << "logUpdateOp got a document with no _id field, ignoring updatedDoc: " - << redact(postImageDoc); + LOGV2_WARNING( + 21996, + "logUpdateOp got a document with no _id field, ignoring updatedDoc: {postImageDoc}", + "postImageDoc"_attr = redact(postImageDoc)); return; } @@ -488,8 +497,10 @@ void MigrationChunkClonerSourceLegacy::onDeleteOp(OperationContext* opCtx, BSONElement idElement = deletedDocId["_id"]; if (idElement.eoo()) { - warning() << "logDeleteOp got a document with no _id field, ignoring deleted doc: " - << redact(deletedDocId); + LOGV2_WARNING( + 21997, + "logDeleteOp got a document with no _id field, ignoring deleted doc: {deletedDocId}", + "deletedDocId"_attr = redact(deletedDocId)); return; } @@ -994,13 +1005,19 @@ Status MigrationChunkClonerSourceLegacy::_checkRecipientCloningStatus(OperationC const std::size_t cloneLocsRemaining = _cloneLocs.size(); if (_forceJumbo && _jumboChunkCloneState) { - log() << "moveChunk data transfer progress: " << redact(res) - << " mem used: " << _memoryUsed - << " documents cloned so far: " << _jumboChunkCloneState->docsCloned; + LOGV2(21992, + "moveChunk data transfer progress: {res} mem used: {memoryUsed} documents cloned " + "so far: {jumboChunkCloneState_docsCloned}", + "res"_attr = redact(res), + "memoryUsed"_attr = _memoryUsed, + "jumboChunkCloneState_docsCloned"_attr = _jumboChunkCloneState->docsCloned); } else { - log() << "moveChunk data transfer progress: " << redact(res) - << " mem used: " << _memoryUsed - << " documents remaining to clone: " << cloneLocsRemaining; + LOGV2(21993, + "moveChunk data transfer progress: {res} mem used: {memoryUsed} documents " + "remaining to clone: {cloneLocsRemaining}", + "res"_attr = redact(res), + "memoryUsed"_attr = _memoryUsed, + "cloneLocsRemaining"_attr = cloneLocsRemaining); } if (res["state"].String() == "steady") { diff --git a/src/mongo/db/s/migration_destination_manager.cpp b/src/mongo/db/s/migration_destination_manager.cpp index a11bb49b8a4..54f4a01e87b 100644 --- a/src/mongo/db/s/migration_destination_manager.cpp +++ b/src/mongo/db/s/migration_destination_manager.cpp @@ -63,6 +63,7 @@ #include "mongo/db/session_catalog_mongod.h" #include "mongo/db/storage/remove_saver.h" #include "mongo/db/transaction_participant.h" +#include "mongo/logv2/log.h" #include "mongo/s/catalog/type_chunk.h" #include "mongo/s/client/shard_registry.h" #include "mongo/s/grid.h" @@ -240,7 +241,7 @@ void MigrationDestinationManager::setState(State newState) { } void MigrationDestinationManager::_setStateFail(StringData msg) { - log() << msg; + LOGV2(21998, "{msg}", "msg"_attr = msg); { stdx::lock_guard<Latch> sl(_mutex); _errmsg = msg.toString(); @@ -252,7 +253,7 @@ void MigrationDestinationManager::_setStateFail(StringData msg) { } void MigrationDestinationManager::_setStateFailWarn(StringData msg) { - warning() << msg; + LOGV2_WARNING(22010, "{msg}", "msg"_attr = msg); { stdx::lock_guard<Latch> sl(_mutex); _errmsg = msg.toString(); @@ -436,7 +437,9 @@ repl::OpTime MigrationDestinationManager::cloneDocumentsFromDonor( } catch (...) { stdx::lock_guard<Client> lk(*opCtx->getClient()); opCtx->getServiceContext()->killOperation(lk, opCtx, ErrorCodes::Error(51008)); - log() << "Batch insertion failed " << causedBy(redact(exceptionToStatus())); + LOGV2(21999, + "Batch insertion failed {causedBy_exceptionToStatus}", + "causedBy_exceptionToStatus"_attr = causedBy(redact(exceptionToStatus()))); } }}; auto inserterThreadJoinGuard = makeGuard([&] { @@ -813,9 +816,15 @@ void MigrationDestinationManager::_migrateDriver(OperationContext* outerOpCtx) { invariant(!_min.isEmpty()); invariant(!_max.isEmpty()); - log() << "Starting receiving end of migration of chunk " << redact(_min) << " -> " - << redact(_max) << " for collection " << _nss.ns() << " from " << _fromShard - << " at epoch " << _epoch.toString() << " with session id " << *_sessionId; + LOGV2(22000, + "Starting receiving end of migration of chunk {min} -> {max} for collection {nss_ns} " + "from {fromShard} at epoch {epoch} with session id {sessionId}", + "min"_attr = redact(_min), + "max"_attr = redact(_max), + "nss_ns"_attr = _nss.ns(), + "fromShard"_attr = _fromShard, + "epoch"_attr = _epoch.toString(), + "sessionId"_attr = *_sessionId); MoveTimingHelper timing( outerOpCtx, "to", _nss.ns(), _min, _max, 6 /* steps */, &_errmsg, ShardId(), ShardId()); @@ -823,7 +832,7 @@ void MigrationDestinationManager::_migrateDriver(OperationContext* outerOpCtx) { const auto initialState = getState(); if (initialState == ABORT) { - error() << "Migration abort requested before it started"; + LOGV2_ERROR(22013, "Migration abort requested before it started"); return; } @@ -843,9 +852,11 @@ void MigrationDestinationManager::_migrateDriver(OperationContext* outerOpCtx) { if (_enableResumableRangeDeleter) { while (migrationutil::checkForConflictingDeletions( outerOpCtx, range, donorCollectionOptionsAndIndexes.uuid)) { - LOG(0) << "Migration paused because range overlaps with a " - "range that is scheduled for deletion: collection: " - << _nss.ns() << " range: " << redact(range.toString()); + LOGV2(22001, + "Migration paused because range overlaps with a " + "range that is scheduled for deletion: collection: {nss_ns} range: {range}", + "nss_ns"_attr = _nss.ns(), + "range"_attr = redact(range.toString())); auto status = CollectionShardingRuntime::waitForClean( outerOpCtx, _nss, donorCollectionOptionsAndIndexes.uuid, range); @@ -984,8 +995,9 @@ void MigrationDestinationManager::_migrateDriver(OperationContext* outerOpCtx) { repl::ReplClientInfo::forClient(opCtx->getClient()).getLastOp(), _writeConcern); if (replStatus.status.code() == ErrorCodes::WriteConcernFailed) { - warning() << "secondaryThrottle on, but doc insert timed out; " - "continuing"; + LOGV2_WARNING(22011, + "secondaryThrottle on, but doc insert timed out; " + "continuing"); } else { uassertStatusOK(replStatus.status); } @@ -1059,7 +1071,8 @@ void MigrationDestinationManager::_migrateDriver(OperationContext* outerOpCtx) { opCtx->checkForInterrupt(); if (getState() == ABORT) { - log() << "Migration aborted while waiting for replication at catch up stage"; + LOGV2(22002, + "Migration aborted while waiting for replication at catch up stage"); return; } @@ -1067,7 +1080,7 @@ void MigrationDestinationManager::_migrateDriver(OperationContext* outerOpCtx) { break; if (i > 100) { - log() << "secondaries having hard time keeping up with migrate"; + LOGV2(22003, "secondaries having hard time keeping up with migrate"); } sleepmillis(20); @@ -1087,14 +1100,14 @@ void MigrationDestinationManager::_migrateDriver(OperationContext* outerOpCtx) { // Pause to wait for replication. This will prevent us from going into critical section // until we're ready. - log() << "Waiting for replication to catch up before entering critical section"; + LOGV2(22004, "Waiting for replication to catch up before entering critical section"); auto awaitReplicationResult = repl::ReplicationCoordinator::get(opCtx)->awaitReplication( opCtx, lastOpApplied, _writeConcern); uassertStatusOKWithContext(awaitReplicationResult.status, awaitReplicationResult.status.codeString()); - log() << "Chunk data replicated successfully."; + LOGV2(22005, "Chunk data replicated successfully."); } { @@ -1131,7 +1144,7 @@ void MigrationDestinationManager::_migrateDriver(OperationContext* outerOpCtx) { } if (getState() == ABORT) { - log() << "Migration aborted while transferring mods"; + LOGV2(22006, "Migration aborted while transferring mods"); return; } @@ -1254,7 +1267,7 @@ bool MigrationDestinationManager::_applyMigrateOp(OperationContext* opCtx, const std::string errMsg = str::stream() << "cannot migrate chunk, local document " << redact(localDoc) << " has same _id as reloaded remote document " << redact(updatedDoc); - warning() << errMsg; + LOGV2_WARNING(22012, "{errMsg}", "errMsg"_attr = errMsg); // Exception will abort migration cleanly uasserted(16977, errMsg); @@ -1277,14 +1290,22 @@ bool MigrationDestinationManager::_flushPendingWrites(OperationContext* opCtx, repl::OpTime op(lastOpApplied); static Occasionally sampler; if (sampler.tick()) { - log() << "migrate commit waiting for a majority of slaves for '" << _nss.ns() << "' " - << redact(_min) << " -> " << redact(_max) << " waiting for: " << op; + LOGV2(22007, + "migrate commit waiting for a majority of slaves for '{nss_ns}' {min} -> {max} " + "waiting for: {op}", + "nss_ns"_attr = _nss.ns(), + "min"_attr = redact(_min), + "max"_attr = redact(_max), + "op"_attr = op); } return false; } - log() << "migrate commit succeeded flushing to secondaries for '" << _nss.ns() << "' " - << redact(_min) << " -> " << redact(_max); + LOGV2(22008, + "migrate commit succeeded flushing to secondaries for '{nss_ns}' {min} -> {max}", + "nss_ns"_attr = _nss.ns(), + "min"_attr = redact(_min), + "max"_attr = redact(_max)); return true; } @@ -1330,8 +1351,10 @@ void MigrationDestinationManager::_forgetPending(OperationContext* opCtx, ChunkR // checking this here is that in the future we shouldn't have this problem. if (!optMetadata || !(*optMetadata)->isSharded() || (*optMetadata)->getCollVersion().epoch() != _epoch) { - LOG(0) << "No need to forget pending chunk " << redact(range.toString()) - << " because the epoch for " << _nss.ns() << " changed"; + LOGV2(22009, + "No need to forget pending chunk {range} because the epoch for {nss_ns} changed", + "range"_attr = redact(range.toString()), + "nss_ns"_attr = _nss.ns()); return; } diff --git a/src/mongo/db/s/migration_destination_manager_legacy_commands.cpp b/src/mongo/db/s/migration_destination_manager_legacy_commands.cpp index acfdbb306f1..50d63eec920 100644 --- a/src/mongo/db/s/migration_destination_manager_legacy_commands.cpp +++ b/src/mongo/db/s/migration_destination_manager_legacy_commands.cpp @@ -43,6 +43,7 @@ #include "mongo/db/s/shard_filtering_metadata_refresh.h" #include "mongo/db/s/sharding_state.h" #include "mongo/db/s/start_chunk_clone_request.h" +#include "mongo/logv2/log.h" #include "mongo/s/chunk_version.h" #include "mongo/s/request_types/migration_secondary_throttle_options.h" #include "mongo/util/assert_util.h" @@ -202,7 +203,7 @@ public: Status const status = mdm->startCommit(sessionId); mdm->report(result, opCtx, false); if (!status.isOK()) { - log() << status.reason(); + LOGV2(22014, "{status_reason}", "status_reason"_attr = status.reason()); uassertStatusOK(status); } return true; @@ -250,7 +251,7 @@ public: Status const status = mdm->abort(migrationSessionIdStatus.getValue()); mdm->report(result, opCtx, false); if (!status.isOK()) { - log() << status.reason(); + LOGV2(22015, "{status_reason}", "status_reason"_attr = status.reason()); uassertStatusOK(status); } } else if (migrationSessionIdStatus == ErrorCodes::NoSuchKey) { diff --git a/src/mongo/db/s/migration_source_manager.cpp b/src/mongo/db/s/migration_source_manager.cpp index 34012e759d4..c94b0777ad2 100644 --- a/src/mongo/db/s/migration_source_manager.cpp +++ b/src/mongo/db/s/migration_source_manager.cpp @@ -57,6 +57,7 @@ #include "mongo/db/s/sharding_statistics.h" #include "mongo/executor/task_executor.h" #include "mongo/executor/task_executor_pool.h" +#include "mongo/logv2/log.h" #include "mongo/s/catalog/type_chunk.h" #include "mongo/s/catalog/type_shard_collection.h" #include "mongo/s/catalog_cache_loader.h" @@ -164,8 +165,11 @@ MigrationSourceManager::MigrationSourceManager(OperationContext* opCtx, "Destination shard cannot be the same as source", _args.getFromShardId() != _args.getToShardId()); - log() << "Starting chunk migration " << redact(_args.toString()) - << " with expected collection version epoch " << _args.getVersionEpoch(); + LOGV2(22016, + "Starting chunk migration {args} with expected collection version epoch " + "{args_getVersionEpoch}", + "args"_attr = redact(_args.toString()), + "args_getVersionEpoch"_attr = _args.getVersionEpoch()); // Force refresh of the metadata to ensure we have the latest forceShardFilteringMetadataRefresh(_opCtx, getNss()); @@ -391,7 +395,7 @@ Status MigrationSourceManager::enterCriticalSection() { << signalStatus.toString()}; } - log() << "Migration successfully entered critical section"; + LOGV2(22017, "Migration successfully entered critical section"); scopedGuard.dismiss(); return Status::OK(); @@ -502,8 +506,10 @@ Status MigrationSourceManager::commitChunkMetadataOnConfig() { } // Migration succeeded - LOG(0) << "Migration succeeded and updated collection version to " - << refreshedMetadata->getCollVersion(); + LOGV2( + 22018, + "Migration succeeded and updated collection version to {refreshedMetadata_getCollVersion}", + "refreshedMetadata_getCollVersion"_attr = refreshedMetadata->getCollVersion()); if (_enableResumableRangeDeleter) { _coordinator->setMigrationDecision( @@ -547,8 +553,10 @@ Status MigrationSourceManager::commitChunkMetadataOnConfig() { if (_enableResumableRangeDeleter) { if (_args.getWaitForDelete()) { - log() << "Waiting for cleanup of " << getNss().ns() << " range " - << redact(range.toString()); + LOGV2(22019, + "Waiting for cleanup of {getNss_ns} range {range}", + "getNss_ns"_attr = getNss().ns(), + "range"_attr = redact(range.toString())); invariant(_scheduledRangeDeletionOnSuccess); auto scheduleSW = _scheduledRangeDeletionOnSuccess->getNoThrow(_opCtx); @@ -575,8 +583,10 @@ Status MigrationSourceManager::commitChunkMetadataOnConfig() { }(); if (_args.getWaitForDelete()) { - log() << "Waiting for cleanup of " << getNss().ns() << " range " - << redact(range.toString()); + LOGV2(22020, + "Waiting for cleanup of {getNss_ns} range {range}", + "getNss_ns"_attr = getNss().ns(), + "range"_attr = redact(range.toString())); auto deleteStatus = cleanupCompleteFuture.getNoThrow(_opCtx); @@ -592,8 +602,10 @@ Status MigrationSourceManager::commitChunkMetadataOnConfig() { return {ErrorCodes::OrphanedRangeCleanUpFailed, orphanedRangeCleanUpErrMsg + redact(cleanupCompleteFuture.getNoThrow(_opCtx))}; } else { - log() << "Leaving cleanup of " << getNss().ns() << " range " << redact(range.toString()) - << " to complete in background"; + LOGV2(22021, + "Leaving cleanup of {getNss_ns} range {range} to complete in background", + "getNss_ns"_attr = getNss().ns(), + "range"_attr = redact(range.toString())); } } @@ -616,8 +628,10 @@ void MigrationSourceManager::cleanupOnError() { try { _cleanup(); } catch (const DBException& ex) { - warning() << "Failed to clean up migration: " << redact(_args.toString()) - << "due to: " << redact(ex); + LOGV2_WARNING(22022, + "Failed to clean up migration: {args}due to: {ex}", + "args"_attr = redact(_args.toString()), + "ex"_attr = redact(ex)); } } diff --git a/src/mongo/db/s/migration_util.cpp b/src/mongo/db/s/migration_util.cpp index 58f9198057c..a5938c94b79 100644 --- a/src/mongo/db/s/migration_util.cpp +++ b/src/mongo/db/s/migration_util.cpp @@ -52,6 +52,7 @@ #include "mongo/db/write_concern.h" #include "mongo/executor/task_executor_pool.h" #include "mongo/executor/thread_pool_task_executor.h" +#include "mongo/logv2/log.h" #include "mongo/rpc/get_status_from_command_result.h" #include "mongo/s/catalog/type_chunk.h" #include "mongo/s/client/shard.h" @@ -83,7 +84,7 @@ void sendToRecipient(OperationContext* opCtx, const ShardId& recipientId, const auto recipientShard = uassertStatusOK(Grid::get(opCtx)->shardRegistry()->getShard(opCtx, recipientId)); - LOG(1) << "Sending request " << cmd.toBSON({}) << " to recipient."; + LOGV2_DEBUG(22023, 1, "Sending request {cmd} to recipient.", "cmd"_attr = cmd.toBSON({})); auto response = recipientShard->runCommandWithFixedRetryAttempts( opCtx, @@ -150,14 +151,20 @@ ExecutorFuture<bool> submitRangeDeletionTask(OperationContext* opCtx, // does not match the UUID of the deletion task, force a filtering metadata refresh // once, because this node may have just stepped up and therefore may have a stale // cache. - LOG(0) << "Filtering metadata for namespace in deletion task " - << deletionTask.toBSON() - << (css->getCurrentMetadataIfKnown() - ? (css->getCurrentMetadata()->isSharded() - ? " has UUID that does not match UUID of the deletion task" - : " is unsharded") - : " is not known") - << ", forcing a refresh of " << deletionTask.getNss(); + LOGV2( + 22024, + "Filtering metadata for namespace in deletion task " + "{deletionTask}{css_getCurrentMetadataIfKnown_css_getCurrentMetadata_isSharded_" + "has_UUID_that_does_not_match_UUID_of_the_deletion_task_is_unsharded_is_not_" + "known}, forcing a refresh of {deletionTask_getNss}", + "deletionTask"_attr = deletionTask.toBSON(), + "css_getCurrentMetadataIfKnown_css_getCurrentMetadata_isSharded_has_UUID_that_does_not_match_UUID_of_the_deletion_task_is_unsharded_is_not_known"_attr = + (css->getCurrentMetadataIfKnown() + ? (css->getCurrentMetadata()->isSharded() + ? " has UUID that does not match UUID of the deletion task" + : " is unsharded") + : " is not known"), + "deletionTask_getNss"_attr = deletionTask.getNss()); // TODO (SERVER-46075): Add an asynchronous version of // forceShardFilteringMetadataRefresh to avoid blocking on the network in the @@ -178,15 +185,20 @@ ExecutorFuture<bool> submitRangeDeletionTask(OperationContext* opCtx, autoColl.emplace(opCtx, deletionTask.getNss(), MODE_IS); if (!css->getCurrentMetadataIfKnown() || !css->getCurrentMetadata()->isSharded() || !css->getCurrentMetadata()->uuidMatches(deletionTask.getCollectionUuid())) { - LOG(0) << "Even after forced refresh, filtering metadata for namespace in deletion " - "task " - << deletionTask.toBSON() - << (css->getCurrentMetadataIfKnown() - ? (css->getCurrentMetadata()->isSharded() - ? " has UUID that does not match UUID of the deletion task" - : " is unsharded") - : " is not known") - << ", deleting the task."; + LOGV2( + 22025, + "Even after forced refresh, filtering metadata for namespace in deletion " + "task " + "{deletionTask}{css_getCurrentMetadataIfKnown_css_getCurrentMetadata_isSharded_" + "has_UUID_that_does_not_match_UUID_of_the_deletion_task_is_unsharded_is_not_" + "known}, deleting the task.", + "deletionTask"_attr = deletionTask.toBSON(), + "css_getCurrentMetadataIfKnown_css_getCurrentMetadata_isSharded_has_UUID_that_does_not_match_UUID_of_the_deletion_task_is_unsharded_is_not_known"_attr = + (css->getCurrentMetadataIfKnown() + ? (css->getCurrentMetadata()->isSharded() + ? " has UUID that does not match UUID of the deletion task" + : " is unsharded") + : " is not known")); autoColl.reset(); deleteRangeDeletionTaskLocally( @@ -194,7 +206,9 @@ ExecutorFuture<bool> submitRangeDeletionTask(OperationContext* opCtx, return false; } - LOG(0) << "Submitting range deletion task " << deletionTask.toBSON(); + LOGV2(22026, + "Submitting range deletion task {deletionTask}", + "deletionTask"_attr = deletionTask.toBSON()); const auto whenToClean = deletionTask.getWhenToClean() == CleanWhenEnum::kNow ? CollectionShardingRuntime::kNow @@ -204,8 +218,12 @@ ExecutorFuture<bool> submitRangeDeletionTask(OperationContext* opCtx, if (cleanupCompleteFuture.isReady() && !cleanupCompleteFuture.getNoThrow(opCtx).isOK()) { - LOG(0) << "Failed to submit range deletion task " << deletionTask.toBSON() - << causedBy(cleanupCompleteFuture.getNoThrow(opCtx)); + LOGV2(22027, + "Failed to submit range deletion task " + "{deletionTask}{causedBy_cleanupCompleteFuture_getNoThrow_opCtx}", + "deletionTask"_attr = deletionTask.toBSON(), + "causedBy_cleanupCompleteFuture_getNoThrow_opCtx"_attr = + causedBy(cleanupCompleteFuture.getNoThrow(opCtx))); return false; } return true; @@ -225,7 +243,7 @@ void submitPendingDeletions(OperationContext* opCtx) { } void resubmitRangeDeletionsOnStepUp(ServiceContext* serviceContext) { - LOG(0) << "Starting pending deletion submission thread."; + LOGV2(22028, "Starting pending deletion submission thread."); auto executor = Grid::get(serviceContext)->getExecutorPool()->getFixedExecutor(); @@ -258,8 +276,9 @@ void forEachOrphanRange(OperationContext* opCtx, const NamespaceString& nss, Cal RangeMap{SimpleBSONObjComparator::kInstance.makeBSONObjIndexedMap<BSONObj>()}; if (!metadata->isSharded()) { - LOG(0) << "Upgrade: skipping orphaned range enumeration for " << nss - << ", collection is not sharded"; + LOGV2(22029, + "Upgrade: skipping orphaned range enumeration for {nss}, collection is not sharded", + "nss"_attr = nss); return; } @@ -268,8 +287,12 @@ void forEachOrphanRange(OperationContext* opCtx, const NamespaceString& nss, Cal while (true) { auto range = metadata->getNextOrphanRange(emptyChunkMap, startingKey); if (!range) { - LOG(2) << "Upgrade: Completed orphaned range enumeration for " << nss.toString() - << " starting from " << redact(startingKey) << ", no orphan ranges remain"; + LOGV2_DEBUG(22030, + 2, + "Upgrade: Completed orphaned range enumeration for {nss} starting from " + "{startingKey}, no orphan ranges remain", + "nss"_attr = nss.toString(), + "startingKey"_attr = redact(startingKey)); return; } @@ -287,7 +310,11 @@ void submitOrphanRanges(OperationContext* opCtx, const NamespaceString& nss, con if (version == ChunkVersion::UNSHARDED()) return; - LOG(2) << "Upgrade: Cleaning up existing orphans for " << nss << " : " << uuid; + LOGV2_DEBUG(22031, + 2, + "Upgrade: Cleaning up existing orphans for {nss} : {uuid}", + "nss"_attr = nss, + "uuid"_attr = uuid); std::vector<RangeDeletionTask> deletions; forEachOrphanRange(opCtx, nss, [&deletions, &opCtx, &nss, &uuid](const auto& range) { @@ -305,14 +332,19 @@ void submitOrphanRanges(OperationContext* opCtx, const NamespaceString& nss, con NamespaceString::kRangeDeletionNamespace); for (const auto& task : deletions) { - LOG(2) << "Upgrade: Submitting range for cleanup: " << task.getRange() << " from " - << nss; + LOGV2_DEBUG(22032, + 2, + "Upgrade: Submitting range for cleanup: {task_getRange} from {nss}", + "task_getRange"_attr = task.getRange(), + "nss"_attr = nss); store.add(opCtx, task); } } catch (ExceptionFor<ErrorCodes::NamespaceNotFound>& e) { - LOG(0) << "Upgrade: Failed to cleanup orphans for " << nss - << " because the namespace was not found: " << e.what() - << ", the collection must have been dropped"; + LOGV2(22033, + "Upgrade: Failed to cleanup orphans for {nss} because the namespace was not found: " + "{e_what}, the collection must have been dropped", + "nss"_attr = nss, + "e_what"_attr = e.what()); } } @@ -327,7 +359,7 @@ void submitOrphanRangesForCleanup(OperationContext* opCtx) { for (auto collIt = catalog.begin(dbName); collIt != catalog.end(); ++collIt) { auto uuid = collIt.uuid().get(); auto nss = catalog.lookupNSSByUUID(opCtx, uuid).get(); - LOG(2) << "Upgrade: processing collection: " << nss; + LOGV2_DEBUG(22034, 2, "Upgrade: processing collection: {nss}", "nss"_attr = nss); submitOrphanRanges(opCtx, nss, uuid); } @@ -504,8 +536,11 @@ void ensureChunkVersionIsGreaterThan(OperationContext* opCtx, repl::MemberState::RS_PRIMARY && term == repl::ReplicationCoordinator::get(opCtx)->getTerm()); - LOG(0) << "_configsvrEnsureChunkVersionIsGreaterThan failed after " << attempts - << " attempts " << causedBy(redact(ex.toStatus())) << " . Will try again."; + LOGV2(22035, + "_configsvrEnsureChunkVersionIsGreaterThan failed after {attempts} attempts " + "{causedBy_ex_toStatus} . Will try again.", + "attempts"_attr = attempts, + "causedBy_ex_toStatus"_attr = causedBy(redact(ex.toStatus()))); } } } @@ -551,15 +586,18 @@ void refreshFilteringMetadataUntilSuccess(OperationContext* opCtx, const Namespa repl::MemberState::RS_PRIMARY && term == repl::ReplicationCoordinator::get(opCtx)->getTerm()); - LOG(0) << "Failed to refresh metadata for " << nss.ns() << " after " << attempts - << " attempts " << causedBy(redact(ex.toStatus())) - << ". Will try to refresh again."; + LOGV2(22036, + "Failed to refresh metadata for {nss_ns} after {attempts} attempts " + "{causedBy_ex_toStatus}. Will try to refresh again.", + "nss_ns"_attr = nss.ns(), + "attempts"_attr = attempts, + "causedBy_ex_toStatus"_attr = causedBy(redact(ex.toStatus()))); } } } void resumeMigrationCoordinationsOnStepUp(ServiceContext* serviceContext) { - LOG(0) << "Starting migration coordinator stepup recovery thread."; + LOGV2(22037, "Starting migration coordinator stepup recovery thread."); auto executor = Grid::get(serviceContext)->getExecutorPool()->getFixedExecutor(); ExecutorFuture<void>(executor).getAsync([serviceContext](const Status& status) { @@ -590,7 +628,9 @@ void resumeMigrationCoordinationsOnStepUp(ServiceContext* serviceContext) { auto& replClientInfo = repl::ReplClientInfo::forClient(opCtx->getClient()); replClientInfo.setLastOpToSystemLastOpTime(opCtx); const auto lastOpTime = replClientInfo.getLastOp(); - LOG(0) << "Waiting for OpTime " << lastOpTime << " to become majority committed"; + LOGV2(22038, + "Waiting for OpTime {lastOpTime} to become majority committed", + "lastOpTime"_attr = lastOpTime); WriteConcernResult unusedWCResult; uassertStatusOK( waitForWriteConcern(opCtx, @@ -604,7 +644,7 @@ void resumeMigrationCoordinationsOnStepUp(ServiceContext* serviceContext) { opCtx, NamespaceString::kMigrationCoordinatorsNamespace); Query query; store.forEach(opCtx, query, [&opCtx](const MigrationCoordinatorDocument& doc) { - LOG(0) << "Recovering migration " << doc.toBSON(); + LOGV2(22039, "Recovering migration {doc}", "doc"_attr = doc.toBSON()); // Create a MigrationCoordinator to complete the coordination. MigrationCoordinator coordinator(doc.getId(), @@ -643,15 +683,20 @@ void resumeMigrationCoordinationsOnStepUp(ServiceContext* serviceContext) { if (!refreshedMetadata || !(*refreshedMetadata)->isSharded() || !(*refreshedMetadata)->uuidMatches(doc.getCollectionUuid())) { - LOG(0) << "Even after forced refresh, filtering metadata for namespace in " - "migration coordinator doc " - << doc.toBSON() - << (!refreshedMetadata || !(*refreshedMetadata)->isSharded() - ? "is not known" - : "has UUID that does not match the collection UUID in the " - "coordinator doc") - << ". Deleting the range deletion tasks on the donor and recipient as " - "well as the migration coordinator document on this node."; + LOGV2( + 22040, + "Even after forced refresh, filtering metadata for namespace in " + "migration coordinator doc " + "{doc}{refreshedMetadata_refreshedMetadata_isSharded_is_not_known_has_UUID_" + "that_does_not_match_the_collection_UUID_in_the_coordinator_doc}. Deleting " + "the range deletion tasks on the donor and recipient as " + "well as the migration coordinator document on this node.", + "doc"_attr = doc.toBSON(), + "refreshedMetadata_refreshedMetadata_isSharded_is_not_known_has_UUID_that_does_not_match_the_collection_UUID_in_the_coordinator_doc"_attr = + (!refreshedMetadata || !(*refreshedMetadata)->isSharded() + ? "is not known" + : "has UUID that does not match the collection UUID in the " + "coordinator doc")); // TODO (SERVER-45707): Test that range deletion tasks are eventually // deleted even if the collection is dropped before migration coordination @@ -672,8 +717,9 @@ void resumeMigrationCoordinationsOnStepUp(ServiceContext* serviceContext) { return true; }); } catch (const DBException& ex) { - LOG(0) << "Failed to resume coordinating migrations on stepup " - << causedBy(ex.toStatus()); + LOGV2(22041, + "Failed to resume coordinating migrations on stepup {causedBy_ex_toStatus}", + "causedBy_ex_toStatus"_attr = causedBy(ex.toStatus())); } }); } diff --git a/src/mongo/db/s/move_primary_source_manager.cpp b/src/mongo/db/s/move_primary_source_manager.cpp index ec87bce1bfa..c6b97b5b1be 100644 --- a/src/mongo/db/s/move_primary_source_manager.cpp +++ b/src/mongo/db/s/move_primary_source_manager.cpp @@ -41,6 +41,7 @@ #include "mongo/db/s/sharding_logging.h" #include "mongo/db/s/sharding_state_recovery.h" #include "mongo/db/s/sharding_statistics.h" +#include "mongo/logv2/log.h" #include "mongo/rpc/get_status_from_command_result.h" #include "mongo/s/catalog/type_shard_database.h" #include "mongo/s/catalog_cache.h" @@ -74,7 +75,11 @@ Status MovePrimarySourceManager::clone(OperationContext* opCtx) { invariant(_state == kCreated); auto scopedGuard = makeGuard([&] { cleanupOnError(opCtx); }); - log() << "Moving " << _dbname << " primary from: " << _fromShard << " to: " << _toShard; + LOGV2(22042, + "Moving {dbname} primary from: {fromShard} to: {toShard}", + "dbname"_attr = _dbname, + "fromShard"_attr = _fromShard, + "toShard"_attr = _toShard); // Record start in changelog uassertStatusOK(ShardingLogging::get(opCtx)->logChangeChecked( @@ -195,7 +200,7 @@ Status MovePrimarySourceManager::enterCriticalSection(OperationContext* opCtx) { << signalStatus.toString()}; } - log() << "movePrimary successfully entered critical section"; + LOGV2(22043, "movePrimary successfully entered critical section"); scopedGuard.dismiss(); return Status::OK(); @@ -244,9 +249,10 @@ Status MovePrimarySourceManager::commitOnConfig(OperationContext* opCtx) { // Need to get the latest optime in case the refresh request goes to a secondary -- // otherwise the read won't wait for the write that _configsvrCommitMovePrimary may have // done - log() << "Error occurred while committing the movePrimary. Performing a majority write " - "against the config server to obtain its latest optime" - << causedBy(redact(commitStatus)); + LOGV2(22044, + "Error occurred while committing the movePrimary. Performing a majority write " + "against the config server to obtain its latest optime{causedBy_commitStatus}", + "causedBy_commitStatus"_attr = causedBy(redact(commitStatus))); Status validateStatus = ShardingLogging::get(opCtx)->logChangeChecked( opCtx, @@ -337,7 +343,10 @@ Status MovePrimarySourceManager::cleanStaleData(OperationContext* opCtx) { client.runCommand(_dbname.toString(), BSON("drop" << coll.coll()), dropCollResult); Status dropStatus = getStatusFromCommandResult(dropCollResult); if (!dropStatus.isOK()) { - log() << "failed to drop cloned collection " << coll << causedBy(redact(dropStatus)); + LOGV2(22045, + "failed to drop cloned collection {coll}{causedBy_dropStatus}", + "coll"_attr = coll, + "causedBy_dropStatus"_attr = causedBy(redact(dropStatus))); } } @@ -363,8 +372,10 @@ void MovePrimarySourceManager::cleanupOnError(OperationContext* opCtx) { } catch (const ExceptionForCat<ErrorCategory::NotMasterError>& ex) { BSONObjBuilder requestArgsBSON; _requestArgs.serialize(&requestArgsBSON); - warning() << "Failed to clean up movePrimary: " << redact(requestArgsBSON.obj()) - << "due to: " << redact(ex); + LOGV2_WARNING(22046, + "Failed to clean up movePrimary: {requestArgsBSON_obj}due to: {ex}", + "requestArgsBSON_obj"_attr = redact(requestArgsBSON.obj()), + "ex"_attr = redact(ex)); } } diff --git a/src/mongo/db/s/periodic_balancer_config_refresher.cpp b/src/mongo/db/s/periodic_balancer_config_refresher.cpp index f9cab569d89..96c33e0ceb3 100644 --- a/src/mongo/db/s/periodic_balancer_config_refresher.cpp +++ b/src/mongo/db/s/periodic_balancer_config_refresher.cpp @@ -35,6 +35,7 @@ #include "mongo/db/operation_context.h" #include "mongo/db/service_context.h" +#include "mongo/logv2/log.h" #include "mongo/s/balancer_configuration.h" #include "mongo/s/grid.h" #include "mongo/util/log.h" @@ -60,7 +61,9 @@ PeriodicJobAnchor launchBalancerConfigRefresher(ServiceContext* serviceContext) Status status = balancerConfig->refreshAndCheck(opCtx.get()); if (!status.isOK()) { - log() << "Failed to refresh balancer configuration" << causedBy(status); + LOGV2(22048, + "Failed to refresh balancer configuration{causedBy_status}", + "causedBy_status"_attr = causedBy(status)); } }, Seconds(30)); diff --git a/src/mongo/db/s/periodic_sharded_index_consistency_checker.cpp b/src/mongo/db/s/periodic_sharded_index_consistency_checker.cpp index 387d2c72652..b30403d42ff 100644 --- a/src/mongo/db/s/periodic_sharded_index_consistency_checker.cpp +++ b/src/mongo/db/s/periodic_sharded_index_consistency_checker.cpp @@ -37,6 +37,7 @@ #include "mongo/db/operation_context.h" #include "mongo/db/s/sharding_runtime_d_params_gen.h" #include "mongo/db/service_context.h" +#include "mongo/logv2/log.h" #include "mongo/s/grid.h" #include "mongo/s/query/cluster_aggregate.h" #include "mongo/util/log.h" @@ -77,7 +78,7 @@ void PeriodicShardedIndexConsistencyChecker::_launchShardedIndexConsistencyCheck return; } - log() << "Checking consistency of sharded collection indexes across the cluster"; + LOGV2(22049, "Checking consistency of sharded collection indexes across the cluster"); const auto aggRequestBSON = fromjson( "{pipeline: [{$indexStats: {}}," @@ -141,8 +142,12 @@ void PeriodicShardedIndexConsistencyChecker::_launchShardedIndexConsistencyCheck } break; } catch (const ExceptionForCat<ErrorCategory::StaleShardVersionError>& ex) { - log() << "Attempt " << tries << " to check index consistency for " - << nss << " received StaleShardVersion error" << causedBy(ex); + LOGV2(22050, + "Attempt {tries} to check index consistency for {nss} received " + "StaleShardVersion error{causedBy_ex}", + "tries"_attr = tries, + "nss"_attr = nss, + "causedBy_ex"_attr = causedBy(ex)); if (canRetry) { continue; } @@ -151,14 +156,19 @@ void PeriodicShardedIndexConsistencyChecker::_launchShardedIndexConsistencyCheck } } - log() << "Found " << numShardedCollsWithInconsistentIndexes - << " collections with inconsistent indexes"; + LOGV2(22051, + "Found {numShardedCollsWithInconsistentIndexes} collections with " + "inconsistent indexes", + "numShardedCollsWithInconsistentIndexes"_attr = + numShardedCollsWithInconsistentIndexes); // Update the count. _numShardedCollsWithInconsistentIndexes.store( numShardedCollsWithInconsistentIndexes); } catch (DBException& ex) { - log() << "Failed to check index consistency " << causedBy(ex.toStatus()); + LOGV2(22052, + "Failed to check index consistency {causedBy_ex_toStatus}", + "causedBy_ex_toStatus"_attr = causedBy(ex.toStatus())); } }, Milliseconds(shardedIndexConsistencyCheckIntervalMS)); diff --git a/src/mongo/db/s/scoped_operation_completion_sharding_actions.cpp b/src/mongo/db/s/scoped_operation_completion_sharding_actions.cpp index 5a8d900a810..2328a973aef 100644 --- a/src/mongo/db/s/scoped_operation_completion_sharding_actions.cpp +++ b/src/mongo/db/s/scoped_operation_completion_sharding_actions.cpp @@ -38,6 +38,7 @@ #include "mongo/db/s/operation_sharding_state.h" #include "mongo/db/s/shard_filtering_metadata_refresh.h" #include "mongo/db/s/sharding_state.h" +#include "mongo/logv2/log.h" #include "mongo/s/cannot_implicitly_create_collection_info.h" #include "mongo/s/stale_exception.h" #include "mongo/util/log.h" @@ -89,16 +90,18 @@ ScopedOperationCompletionShardingActions::~ScopedOperationCompletionShardingActi auto handleMismatchStatus = onShardVersionMismatchNoExcept( _opCtx, staleInfo->getNss(), staleInfo->getVersionReceived()); if (!handleMismatchStatus.isOK()) - log() << "Failed to handle stale version exception" - << causedBy(redact(handleMismatchStatus)); + LOGV2(22053, + "Failed to handle stale version exception{causedBy_handleMismatchStatus}", + "causedBy_handleMismatchStatus"_attr = causedBy(redact(handleMismatchStatus))); } else if (auto staleInfo = status->extraInfo<StaleDbRoutingVersion>()) { auto handleMismatchStatus = onDbVersionMismatchNoExcept(_opCtx, staleInfo->getDb(), staleInfo->getVersionReceived(), staleInfo->getVersionWanted()); if (!handleMismatchStatus.isOK()) - log() << "Failed to handle database version exception" - << causedBy(redact(handleMismatchStatus)); + LOGV2(22054, + "Failed to handle database version exception{causedBy_handleMismatchStatus}", + "causedBy_handleMismatchStatus"_attr = causedBy(redact(handleMismatchStatus))); } else if (auto cannotImplicitCreateCollInfo = status->extraInfo<CannotImplicitlyCreateCollectionInfo>()) { if (ShardingState::get(_opCtx)->enabled() && @@ -107,8 +110,11 @@ ScopedOperationCompletionShardingActions::~ScopedOperationCompletionShardingActi auto handleCannotImplicitCreateStatus = onCannotImplicitlyCreateCollection(_opCtx, cannotImplicitCreateCollInfo->getNss()); if (!handleCannotImplicitCreateStatus.isOK()) - log() << "Failed to handle CannotImplicitlyCreateCollection exception" - << causedBy(redact(handleCannotImplicitCreateStatus)); + LOGV2(22055, + "Failed to handle CannotImplicitlyCreateCollection " + "exception{causedBy_handleCannotImplicitCreateStatus}", + "causedBy_handleCannotImplicitCreateStatus"_attr = + causedBy(redact(handleCannotImplicitCreateStatus))); } } } diff --git a/src/mongo/db/s/set_shard_version_command.cpp b/src/mongo/db/s/set_shard_version_command.cpp index 839c87ea602..a28ba943bb4 100644 --- a/src/mongo/db/s/set_shard_version_command.cpp +++ b/src/mongo/db/s/set_shard_version_command.cpp @@ -47,6 +47,7 @@ #include "mongo/db/s/sharded_connection_info.h" #include "mongo/db/s/sharding_state.h" #include "mongo/db/views/view_catalog.h" +#include "mongo/logv2/log.h" #include "mongo/s/client/shard_registry.h" #include "mongo/s/grid.h" #include "mongo/s/request_types/set_shard_version_request.h" @@ -305,7 +306,7 @@ public: if (critSecSignal) { collLock.reset(); autoDb.reset(); - log() << "waiting till out of critical section"; + LOGV2(22056, "waiting till out of critical section"); critSecSignal->waitFor(opCtx, Seconds(10)); } @@ -326,7 +327,7 @@ public: if (critSecSignal) { collLock.reset(); autoDb.reset(); - log() << "waiting till out of critical section"; + LOGV2(22057, "waiting till out of critical section"); critSecSignal->waitFor(opCtx, Seconds(10)); } @@ -371,7 +372,7 @@ public: << ", stored shard version is " << currVersion.toString() << causedBy(redact(status)); - warning() << errmsg; + LOGV2_WARNING(22058, "{errmsg}", "errmsg"_attr = errmsg); result.append("ns", nss.ns()); result.append("code", status.code()); @@ -390,7 +391,7 @@ public: static Occasionally sampler; if (sampler.tick()) { - warning() << errmsg; + LOGV2_WARNING(22059, "{errmsg}", "errmsg"_attr = errmsg); } // WARNING: the exact fields below are important for compatibility with mongos diff --git a/src/mongo/db/s/shard_filtering_metadata_refresh.cpp b/src/mongo/db/s/shard_filtering_metadata_refresh.cpp index 5926ddcb456..ed66137c3d4 100644 --- a/src/mongo/db/s/shard_filtering_metadata_refresh.cpp +++ b/src/mongo/db/s/shard_filtering_metadata_refresh.cpp @@ -42,6 +42,7 @@ #include "mongo/db/s/operation_sharding_state.h" #include "mongo/db/s/sharding_state.h" #include "mongo/db/s/sharding_statistics.h" +#include "mongo/logv2/log.h" #include "mongo/s/catalog_cache.h" #include "mongo/s/grid.h" #include "mongo/util/fail_point.h" @@ -63,8 +64,11 @@ void onShardVersionMismatch(OperationContext* opCtx, invariant(ShardingState::get(opCtx)->canAcceptShardedCommands()); - LOG(2) << "Metadata refresh requested for " << nss.ns() << " at shard version " - << shardVersionReceived; + LOGV2_DEBUG(22061, + 2, + "Metadata refresh requested for {nss_ns} at shard version {shardVersionReceived}", + "nss_ns"_attr = nss.ns(), + "shardVersionReceived"_attr = shardVersionReceived); ShardingStatistics::get(opCtx).countStaleConfigErrors.addAndFetch(1); @@ -134,7 +138,10 @@ Status onShardVersionMismatchNoExcept(OperationContext* opCtx, onShardVersionMismatch(opCtx, nss, shardVersionReceived, forceRefreshFromThisThread); return Status::OK(); } catch (const DBException& ex) { - log() << "Failed to refresh metadata for collection " << nss << causedBy(redact(ex)); + LOGV2(22062, + "Failed to refresh metadata for collection {nss}{causedBy_ex}", + "nss"_attr = nss, + "causedBy_ex"_attr = causedBy(redact(ex))); return ex.toStatus(); } } @@ -179,8 +186,13 @@ ChunkVersion forceShardFilteringMetadataRefresh(OperationContext* opCtx, if (metadata->isSharded() && metadata->getCollVersion().epoch() == cm->getVersion().epoch() && metadata->getCollVersion() >= cm->getVersion()) { - LOG(1) << "Skipping refresh of metadata for " << nss << " " - << metadata->getCollVersion() << " with an older " << cm->getVersion(); + LOGV2_DEBUG(22063, + 1, + "Skipping refresh of metadata for {nss} {metadata_getCollVersion} with " + "an older {cm_getVersion}", + "nss"_attr = nss, + "metadata_getCollVersion"_attr = metadata->getCollVersion(), + "cm_getVersion"_attr = cm->getVersion()); return metadata->getShardVersion(); } } @@ -202,8 +214,13 @@ ChunkVersion forceShardFilteringMetadataRefresh(OperationContext* opCtx, if (metadata->isSharded() && metadata->getCollVersion().epoch() == cm->getVersion().epoch() && metadata->getCollVersion() >= cm->getVersion()) { - LOG(1) << "Skipping refresh of metadata for " << nss << " " - << metadata->getCollVersion() << " with an older " << cm->getVersion(); + LOGV2_DEBUG(22064, + 1, + "Skipping refresh of metadata for {nss} {metadata_getCollVersion} with " + "an older {cm_getVersion}", + "nss"_attr = nss, + "metadata_getCollVersion"_attr = metadata->getCollVersion(), + "cm_getVersion"_attr = cm->getVersion()); return metadata->getShardVersion(); } } @@ -225,8 +242,10 @@ Status onDbVersionMismatchNoExcept( onDbVersionMismatch(opCtx, dbName, clientDbVersion, serverDbVersion); return Status::OK(); } catch (const DBException& ex) { - log() << "Failed to refresh databaseVersion for database " << dbName - << causedBy(redact(ex)); + LOGV2(22065, + "Failed to refresh databaseVersion for database {dbName}{causedBy_ex}", + "dbName"_attr = dbName, + "causedBy_ex"_attr = causedBy(redact(ex))); return ex.toStatus(); } } @@ -266,10 +285,14 @@ void forceDatabaseRefresh(OperationContext* opCtx, const StringData dbName) { const auto cachedDbVersion = dss->getDbVersion(opCtx, dssLock); if (cachedDbVersion && cachedDbVersion->getUuid() == refreshedDbVersion.getUuid() && cachedDbVersion->getLastMod() >= refreshedDbVersion.getLastMod()) { - LOG(2) << "Skipping setting cached databaseVersion for " << dbName - << " to refreshed version " << refreshedDbVersion.toBSON() - << " because current cached databaseVersion is already " - << cachedDbVersion->toBSON(); + LOGV2_DEBUG(22066, + 2, + "Skipping setting cached databaseVersion for {dbName} to refreshed version " + "{refreshedDbVersion} because current cached databaseVersion is already " + "{cachedDbVersion}", + "dbName"_attr = dbName, + "refreshedDbVersion"_attr = refreshedDbVersion.toBSON(), + "cachedDbVersion"_attr = cachedDbVersion->toBSON()); return; } } diff --git a/src/mongo/db/s/shard_metadata_util.cpp b/src/mongo/db/s/shard_metadata_util.cpp index d08c5f4cd79..37500a2ee15 100644 --- a/src/mongo/db/s/shard_metadata_util.cpp +++ b/src/mongo/db/s/shard_metadata_util.cpp @@ -38,6 +38,7 @@ #include "mongo/db/dbdirectclient.h" #include "mongo/db/ops/write_ops.h" #include "mongo/db/write_concern_options.h" +#include "mongo/logv2/log.h" #include "mongo/rpc/get_status_from_command_result.h" #include "mongo/rpc/unique_message.h" #include "mongo/s/catalog/type_chunk.h" @@ -425,7 +426,10 @@ Status dropChunksAndDeleteCollectionsEntry(OperationContext* opCtx, const Namesp } } - LOG(1) << "Successfully cleared persisted chunk metadata for collection '" << nss << "'."; + LOGV2_DEBUG(22090, + 1, + "Successfully cleared persisted chunk metadata for collection '{nss}'.", + "nss"_attr = nss); return Status::OK(); } catch (const DBException& ex) { return ex.toStatus(); @@ -444,7 +448,10 @@ void dropChunks(OperationContext* opCtx, const NamespaceString& nss) { } } - LOG(1) << "Successfully cleared persisted chunk metadata for collection '" << nss << "'."; + LOGV2_DEBUG(22091, + 1, + "Successfully cleared persisted chunk metadata for collection '{nss}'.", + "nss"_attr = nss); } Status deleteDatabasesEntry(OperationContext* opCtx, StringData dbName) { @@ -464,7 +471,10 @@ Status deleteDatabasesEntry(OperationContext* opCtx, StringData dbName) { uassertStatusOK( getStatusFromWriteCommandResponse(deleteCommandResponse->getCommandReply())); - LOG(1) << "Successfully cleared persisted metadata for db '" << dbName.toString() << "'."; + LOGV2_DEBUG(22092, + 1, + "Successfully cleared persisted metadata for db '{dbName}'.", + "dbName"_attr = dbName.toString()); return Status::OK(); } catch (const DBException& ex) { return ex.toStatus(); diff --git a/src/mongo/db/s/shard_server_catalog_cache_loader.cpp b/src/mongo/db/s/shard_server_catalog_cache_loader.cpp index 682954e7b43..449e812bc2c 100644 --- a/src/mongo/db/s/shard_server_catalog_cache_loader.cpp +++ b/src/mongo/db/s/shard_server_catalog_cache_loader.cpp @@ -43,6 +43,7 @@ #include "mongo/db/repl/replication_coordinator.h" #include "mongo/db/s/shard_metadata_util.h" #include "mongo/db/s/sharding_state.h" +#include "mongo/logv2/log.h" #include "mongo/s/catalog/type_shard_collection.h" #include "mongo/s/catalog/type_shard_database.h" #include "mongo/s/client/shard_registry.h" @@ -90,7 +91,7 @@ void dropChunksIfEpochChanged(OperationContext* opCtx, dropChunks(opCtx, nss); if (MONGO_unlikely(hangPersistCollectionAndChangedChunksAfterDropChunks.shouldFail())) { - log() << "Hit hangPersistCollectionAndChangedChunksAfterDropChunks failpoint"; + LOGV2(22093, "Hit hangPersistCollectionAndChangedChunksAfterDropChunks failpoint"); hangPersistCollectionAndChangedChunksAfterDropChunks.pauseWhileSet(opCtx); } } @@ -954,12 +955,15 @@ void ShardServerCatalogCacheLoader::_runCollAndChunksTasks(const NamespaceString _updatePersistedCollAndChunksMetadata(context.opCtx(), nss); taskFinished = true; } catch (const ExceptionForCat<ErrorCategory::ShutdownError>&) { - LOG(0) << "Failed to persist chunk metadata update for collection '" << nss - << "' due to shutdown."; + LOGV2(22094, + "Failed to persist chunk metadata update for collection '{nss}' due to shutdown.", + "nss"_attr = nss); inShutdown = true; } catch (const DBException& ex) { - LOG(0) << "Failed to persist chunk metadata update for collection '" << nss - << causedBy(redact(ex)); + LOGV2(22095, + "Failed to persist chunk metadata update for collection '{nss}{causedBy_ex}", + "nss"_attr = nss, + "causedBy_ex"_attr = causedBy(redact(ex))); } { @@ -988,10 +992,12 @@ void ShardServerCatalogCacheLoader::_runCollAndChunksTasks(const NamespaceString _threadPool.schedule([this, nss](auto status) { if (ErrorCodes::isCancelationError(status.code())) { - LOG(0) << "Cache loader failed to schedule a persisted metadata update" - << " task for namespace '" << nss << "' due to '" << redact(status) - << "'. Clearing task list so that scheduling will be attempted by the next" - << " caller to refresh this namespace."; + LOGV2(22096, + "Cache loader failed to schedule a persisted metadata update task for namespace " + "'{nss}' due to '{status}'. Clearing task list so that scheduling will be " + "attempted by the next caller to refresh this namespace.", + "nss"_attr = nss, + "status"_attr = redact(status)); { stdx::lock_guard<Latch> lock(_mutex); @@ -1014,11 +1020,15 @@ void ShardServerCatalogCacheLoader::_runDbTasks(StringData dbName) { _updatePersistedDbMetadata(context.opCtx(), dbName); taskFinished = true; } catch (const ExceptionForCat<ErrorCategory::ShutdownError>&) { - LOG(0) << "Failed to persist metadata update for db '" << dbName << "' due to shutdown."; + LOGV2(22097, + "Failed to persist metadata update for db '{dbName}' due to shutdown.", + "dbName"_attr = dbName); inShutdown = true; } catch (const DBException& ex) { - LOG(0) << "Failed to persist chunk metadata update for database " << dbName - << causedBy(redact(ex)); + LOGV2(22098, + "Failed to persist chunk metadata update for database {dbName}{causedBy_ex}", + "dbName"_attr = dbName, + "causedBy_ex"_attr = causedBy(redact(ex))); } { @@ -1047,10 +1057,12 @@ void ShardServerCatalogCacheLoader::_runDbTasks(StringData dbName) { _threadPool.schedule([this, name = dbName.toString()](auto status) { if (ErrorCodes::isCancelationError(status.code())) { - LOG(0) << "Cache loader failed to schedule a persisted metadata update" - << " task for namespace '" << name << "' due to '" << redact(status) - << "'. Clearing task list so that scheduling will be attempted by the next" - << " caller to refresh this namespace."; + LOGV2(22099, + "Cache loader failed to schedule a persisted metadata update task for namespace " + "'{name}' due to '{status}'. Clearing task list so that scheduling will be " + "attempted by the next caller to refresh this namespace.", + "name"_attr = name, + "status"_attr = redact(status)); { stdx::lock_guard<Latch> lock(_mutex); diff --git a/src/mongo/db/s/sharded_connection_info.cpp b/src/mongo/db/s/sharded_connection_info.cpp index 228572e1354..7348c90c486 100644 --- a/src/mongo/db/s/sharded_connection_info.cpp +++ b/src/mongo/db/s/sharded_connection_info.cpp @@ -34,6 +34,7 @@ #include "mongo/db/s/sharded_connection_info.h" #include "mongo/db/client.h" +#include "mongo/logv2/log.h" #include "mongo/util/log.h" namespace mongo { @@ -51,7 +52,7 @@ ShardedConnectionInfo* ShardedConnectionInfo::get(Client* client, bool create) { auto& current = clientSCI(client); if (!current && create) { - LOG(1) << "entering shard mode for connection"; + LOGV2_DEBUG(22060, 1, "entering shard mode for connection"); current.emplace(); } diff --git a/src/mongo/db/s/sharding_initialization_mongod.cpp b/src/mongo/db/s/sharding_initialization_mongod.cpp index 7c81c06766c..ec320b06c07 100644 --- a/src/mongo/db/s/sharding_initialization_mongod.cpp +++ b/src/mongo/db/s/sharding_initialization_mongod.cpp @@ -54,6 +54,7 @@ #include "mongo/db/s/transaction_coordinator_service.h" #include "mongo/db/server_options.h" #include "mongo/executor/task_executor_pool.h" +#include "mongo/logv2/log.h" #include "mongo/rpc/metadata/egress_metadata_hook_list.h" #include "mongo/s/catalog_cache.h" #include "mongo/s/client/shard_connection.h" @@ -102,13 +103,18 @@ public: ->getFixedExecutor() ->schedule([serviceContext = _serviceContext, connStr = state.connStr](Status status) { if (ErrorCodes::isCancelationError(status.code())) { - LOG(2) << "Unable to schedule confirmed set update due to " << status; + LOGV2_DEBUG(22067, + 2, + "Unable to schedule confirmed set update due to {status}", + "status"_attr = status); return; } invariant(status); try { - LOG(0) << "Updating config server with confirmed set " << connStr; + LOGV2(22068, + "Updating config server with confirmed set {connStr}", + "connStr"_attr = connStr); Grid::get(serviceContext)->shardRegistry()->updateReplSetHosts(connStr); if (MONGO_unlikely(failUpdateShardIdentityConfigString.shouldFail())) { @@ -130,7 +136,7 @@ public: ShardingInitializationMongoD::updateShardIdentityConfigString(opCtx.get(), connStr); } catch (const ExceptionForCat<ErrorCategory::ShutdownError>& e) { - LOG(0) << "Unable to update config server due to " << e; + LOGV2(22069, "Unable to update config server due to {e}", "e"_attr = e); } }); } @@ -138,7 +144,10 @@ public: try { Grid::get(_serviceContext)->shardRegistry()->updateReplSetHosts(state.connStr); } catch (const DBException& ex) { - LOG(2) << "Unable to update config server with possible set due to " << ex; + LOGV2_DEBUG(22070, + 2, + "Unable to update config server with possible set due to {ex}", + "ex"_attr = ex); } } void onDroppedSet(const Key&) noexcept final {} @@ -176,8 +185,11 @@ void ShardingInitializationMongoD::initializeShardingEnvironmentOnShardServer( Grid::get(opCtx)->setShardingInitialized(); - LOG(0) << "Finished initializing sharding components for " - << (isStandaloneOrPrimary ? "primary" : "secondary") << " node."; + LOGV2(22071, + "Finished initializing sharding components for {isStandaloneOrPrimary_primary_secondary} " + "node.", + "isStandaloneOrPrimary_primary_secondary"_attr = + (isStandaloneOrPrimary ? "primary" : "secondary")); } ShardingInitializationMongoD::ShardingInitializationMongoD() @@ -269,11 +281,13 @@ bool ShardingInitializationMongoD::initializeShardingAwarenessIfNeeded(Operation if (serverGlobalParams.clusterRole == ClusterRole::ShardServer) { if (!foundShardIdentity) { - warning() << "Started with --shardsvr, but no shardIdentity document was found on " - "disk in " - << NamespaceString::kServerConfigurationNamespace - << ". This most likely means this server has not yet been added to a " - "sharded cluster."; + LOGV2_WARNING(22074, + "Started with --shardsvr, but no shardIdentity document was found on " + "disk in {NamespaceString_kServerConfigurationNamespace}. This most " + "likely means this server has not yet been added to a " + "sharded cluster.", + "NamespaceString_kServerConfigurationNamespace"_attr = + NamespaceString::kServerConfigurationNamespace); return false; } @@ -292,10 +306,13 @@ bool ShardingInitializationMongoD::initializeShardingAwarenessIfNeeded(Operation } else { // Warn if a shardIdentity document is found on disk but *not* started with --shardsvr. if (!shardIdentityBSON.isEmpty()) { - warning() << "Not started with --shardsvr, but a shardIdentity document was found " - "on disk in " - << NamespaceString::kServerConfigurationNamespace << ": " - << shardIdentityBSON; + LOGV2_WARNING( + 22075, + "Not started with --shardsvr, but a shardIdentity document was found " + "on disk in {NamespaceString_kServerConfigurationNamespace}: {shardIdentityBSON}", + "NamespaceString_kServerConfigurationNamespace"_attr = + NamespaceString::kServerConfigurationNamespace, + "shardIdentityBSON"_attr = shardIdentityBSON); } return false; } @@ -310,7 +327,9 @@ void ShardingInitializationMongoD::initializeFromShardIdentity( shardIdentity.validate(), "Invalid shard identity document found when initializing sharding state"); - log() << "initializing sharding state with: " << shardIdentity; + LOGV2(22072, + "initializing sharding state with: {shardIdentity}", + "shardIdentity"_attr = shardIdentity); const auto& configSvrConnStr = shardIdentity.getConfigsvrConnectionString(); @@ -361,17 +380,24 @@ void ShardingInitializationMongoD::updateShardIdentityConfigString( auto result = update(opCtx, autoDb.getDb(), updateReq); if (result.numMatched == 0) { - warning() << "failed to update config string of shard identity document because " - << "it does not exist. This shard could have been removed from the cluster"; + LOGV2_WARNING(22076, + "failed to update config string of shard identity document because it " + "does not exist. This shard could have been removed from the cluster"); } else { - LOG(2) << "Updated config server connection string in shardIdentity document to" - << newConnectionString; + LOGV2_DEBUG(22073, + 2, + "Updated config server connection string in shardIdentity document " + "to{newConnectionString}", + "newConnectionString"_attr = newConnectionString); } } catch (const DBException& exception) { auto status = exception.toStatus(); if (!ErrorCodes::isNotMasterError(status.code())) { - warning() << "Error encountered while trying to update config connection string to " - << newConnectionString.toString() << causedBy(redact(status)); + LOGV2_WARNING(22077, + "Error encountered while trying to update config connection string to " + "{newConnectionString}{causedBy_status}", + "newConnectionString"_attr = newConnectionString.toString(), + "causedBy_status"_attr = causedBy(redact(status))); } } } diff --git a/src/mongo/db/s/sharding_logging.cpp b/src/mongo/db/s/sharding_logging.cpp index c3d07903ceb..e714e59e73d 100644 --- a/src/mongo/db/s/sharding_logging.cpp +++ b/src/mongo/db/s/sharding_logging.cpp @@ -37,6 +37,7 @@ #include "mongo/db/s/sharding_state.h" #include "mongo/db/server_options.h" #include "mongo/executor/network_interface.h" +#include "mongo/logv2/log.h" #include "mongo/s/catalog/type_changelog.h" #include "mongo/s/grid.h" #include "mongo/util/log.h" @@ -80,7 +81,9 @@ Status ShardingLogging::logAction(OperationContext* opCtx, if (result.isOK()) { _actionLogCollectionCreated.store(1); } else { - log() << "couldn't create config.actionlog collection:" << causedBy(result); + LOGV2(22078, + "couldn't create config.actionlog collection:{causedBy_result}", + "causedBy_result"_attr = causedBy(result)); return result; } } @@ -106,7 +109,9 @@ Status ShardingLogging::logChangeChecked(OperationContext* opCtx, if (result.isOK()) { _changeLogCollectionCreated.store(1); } else { - log() << "couldn't create config.changelog collection:" << causedBy(result); + LOGV2(22079, + "couldn't create config.changelog collection:{causedBy_result}", + "causedBy_result"_attr = causedBy(result)); return result; } } @@ -144,15 +149,22 @@ Status ShardingLogging::_log(OperationContext* opCtx, changeLog.setDetails(detail); BSONObj changeLogBSON = changeLog.toBSON(); - log() << "about to log metadata event into " << logCollName << ": " << redact(changeLogBSON); + LOGV2(22080, + "about to log metadata event into {logCollName}: {changeLogBSON}", + "logCollName"_attr = logCollName, + "changeLogBSON"_attr = redact(changeLogBSON)); const NamespaceString nss("config", logCollName); Status result = Grid::get(opCtx)->catalogClient()->insertConfigDocument( opCtx, nss, changeLogBSON, writeConcern); if (!result.isOK()) { - warning() << "Error encountered while logging config change with ID [" << changeId - << "] into collection " << logCollName << ": " << redact(result); + LOGV2_WARNING(22081, + "Error encountered while logging config change with ID [{changeId}] into " + "collection {logCollName}: {result}", + "changeId"_attr = changeId, + "logCollName"_attr = logCollName, + "result"_attr = redact(result)); } return result; diff --git a/src/mongo/db/s/sharding_state.cpp b/src/mongo/db/s/sharding_state.cpp index 441d303a038..50c127f4de0 100644 --- a/src/mongo/db/s/sharding_state.cpp +++ b/src/mongo/db/s/sharding_state.cpp @@ -35,6 +35,7 @@ #include "mongo/db/operation_context.h" #include "mongo/db/server_options.h" +#include "mongo/logv2/log.h" #include "mongo/util/log.h" namespace mongo { @@ -69,7 +70,9 @@ void ShardingState::setInitialized(ShardId shardId, OID clusterId) { void ShardingState::setInitialized(Status failedStatus) { invariant(!failedStatus.isOK()); - log() << "Failed to initialize sharding components" << causedBy(failedStatus); + LOGV2(22082, + "Failed to initialize sharding components{causedBy_failedStatus}", + "causedBy_failedStatus"_attr = causedBy(failedStatus)); stdx::unique_lock<Latch> ul(_mutex); invariant(_getInitializationState() == InitializationState::kNew); diff --git a/src/mongo/db/s/sharding_state_recovery.cpp b/src/mongo/db/s/sharding_state_recovery.cpp index f8a869564ca..98d7a915e49 100644 --- a/src/mongo/db/s/sharding_state_recovery.cpp +++ b/src/mongo/db/s/sharding_state_recovery.cpp @@ -50,6 +50,7 @@ #include "mongo/db/s/sharding_state.h" #include "mongo/db/write_concern.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" @@ -151,7 +152,10 @@ Status modifyRecoveryDocument(OperationContext* opCtx, auto const grid = Grid::get(opCtx); BSONObj updateObj = RecoveryDocument::createChangeObj(grid->configOpTime(), change); - LOG(1) << "Changing sharding recovery document " << redact(updateObj); + LOGV2_DEBUG(22083, + 1, + "Changing sharding recovery document {updateObj}", + "updateObj"_attr = redact(updateObj)); UpdateRequest updateReq(NamespaceString::kServerConfigurationNamespace); updateReq.setQuery(RecoveryDocument::getQuery()); @@ -196,7 +200,9 @@ void ShardingStateRecovery::endMetadataOp(OperationContext* opCtx) { Status status = modifyRecoveryDocument(opCtx, RecoveryDocument::Decrement, WriteConcernOptions()); if (!status.isOK()) { - warning() << "Failed to decrement minOpTimeUpdaters due to " << redact(status); + LOGV2_WARNING(22088, + "Failed to decrement minOpTimeUpdaters due to {status}", + "status"_attr = redact(status)); } } @@ -223,24 +229,30 @@ Status ShardingStateRecovery::recover(OperationContext* opCtx) { const auto recoveryDoc = std::move(recoveryDocStatus.getValue()); - log() << "Sharding state recovery process found document " << redact(recoveryDoc.toBSON()); + LOGV2(22084, + "Sharding state recovery process found document {recoveryDoc}", + "recoveryDoc"_attr = redact(recoveryDoc.toBSON())); if (!recoveryDoc.getMinOpTimeUpdaters()) { // Treat the minOpTime as up-to-date const auto prevOpTime = grid->advanceConfigOpTime( opCtx, recoveryDoc.getMinOpTime(), "sharding state recovery document"); if (prevOpTime) { - log() - << "No in flight metadata change operations, so config server optime updated from " - << *prevOpTime << " to " << recoveryDoc.getMinOpTime(); + LOGV2(22085, + "No in flight metadata change operations, so config server optime updated from " + "{prevOpTime} to {recoveryDoc_getMinOpTime}", + "prevOpTime"_attr = *prevOpTime, + "recoveryDoc_getMinOpTime"_attr = recoveryDoc.getMinOpTime()); } return Status::OK(); } - log() << "Sharding state recovery document indicates there were " - << recoveryDoc.getMinOpTimeUpdaters() - << " metadata change operations in flight. Contacting the config server primary in order " - "to retrieve the most recent opTime."; + LOGV2( + 22086, + "Sharding state recovery document indicates there were {recoveryDoc_getMinOpTimeUpdaters} " + "metadata change operations in flight. Contacting the config server primary in order " + "to retrieve the most recent opTime.", + "recoveryDoc_getMinOpTimeUpdaters"_attr = recoveryDoc.getMinOpTimeUpdaters()); // Need to fetch the latest uptime from the config server, so do a logging write Status status = ShardingLogging::get(opCtx)->logChangeChecked( @@ -252,12 +264,16 @@ Status ShardingStateRecovery::recover(OperationContext* opCtx) { if (!status.isOK()) return status; - log() << "Sharding state recovered. New config server opTime is " << grid->configOpTime(); + LOGV2(22087, + "Sharding state recovered. New config server opTime is {grid_configOpTime}", + "grid_configOpTime"_attr = grid->configOpTime()); // Finally, clear the recovery document so next time we don't need to recover status = modifyRecoveryDocument(opCtx, RecoveryDocument::Clear, kLocalWriteConcern); if (!status.isOK()) { - warning() << "Failed to reset sharding state recovery document due to " << redact(status); + LOGV2_WARNING(22089, + "Failed to reset sharding state recovery document due to {status}", + "status"_attr = redact(status)); } return Status::OK(); diff --git a/src/mongo/db/s/shardsvr_shard_collection.cpp b/src/mongo/db/s/shardsvr_shard_collection.cpp index c56eaeb0470..61d844b6fe8 100644 --- a/src/mongo/db/s/shardsvr_shard_collection.cpp +++ b/src/mongo/db/s/shardsvr_shard_collection.cpp @@ -51,6 +51,7 @@ #include "mongo/db/s/shard_filtering_metadata_refresh.h" #include "mongo/db/s/sharding_logging.h" #include "mongo/db/s/sharding_state.h" +#include "mongo/logv2/log.h" #include "mongo/rpc/get_status_from_command_result.h" #include "mongo/s/balancer_configuration.h" #include "mongo/s/catalog/sharding_catalog_client_impl.h" @@ -92,7 +93,9 @@ const ReadPreferenceSetting kConfigReadSelector(ReadPreference::Nearest, TagSet{ */ void uassertStatusOKWithWarning(const Status& status) { if (!status.isOK()) { - warning() << "shardsvrShardCollection failed" << causedBy(redact(status)); + LOGV2_WARNING(22103, + "shardsvrShardCollection failed{causedBy_status}", + "causedBy_status"_attr = causedBy(redact(status))); uassertStatusOK(status); } } @@ -467,7 +470,7 @@ void logStartShardCollection(OperationContext* opCtx, const ShardsvrShardCollection& request, const ShardCollectionTargetState& prerequisites, const ShardId& dbPrimaryShardId) { - LOG(0) << "CMD: shardcollection: " << cmdObj; + LOGV2(22100, "CMD: shardcollection: {cmdObj}", "cmdObj"_attr = cmdObj); audit::logShardCollection( opCtx->getClient(), nss.ns(), prerequisites.shardKeyPattern.toBSON(), request.getUnique()); @@ -722,8 +725,12 @@ UUID shardCollection(OperationContext* opCtx, writeChunkDocumentsAndRefreshShards(*targetState, initialChunks); } - LOG(0) << "Created " << initialChunks.chunks.size() << " chunk(s) for: " << nss - << ", producing collection version " << initialChunks.collVersion(); + LOGV2(22101, + "Created {initialChunks_chunks_size} chunk(s) for: {nss}, producing collection version " + "{initialChunks_collVersion}", + "initialChunks_chunks_size"_attr = initialChunks.chunks.size(), + "nss"_attr = nss, + "initialChunks_collVersion"_attr = initialChunks.collVersion()); ShardingLogging::get(opCtx)->logChange( @@ -813,7 +820,7 @@ public: uuid); if (MONGO_unlikely(pauseShardCollectionBeforeReturning.shouldFail())) { - log() << "Hit pauseShardCollectionBeforeReturning"; + LOGV2(22102, "Hit pauseShardCollectionBeforeReturning"); pauseShardCollectionBeforeReturning.pauseWhileSet(opCtx); } diff --git a/src/mongo/db/s/split_chunk_command.cpp b/src/mongo/db/s/split_chunk_command.cpp index 4ddc478c405..2bc27bb5fb8 100644 --- a/src/mongo/db/s/split_chunk_command.cpp +++ b/src/mongo/db/s/split_chunk_command.cpp @@ -42,6 +42,7 @@ #include "mongo/db/s/operation_sharding_state.h" #include "mongo/db/s/sharding_state.h" #include "mongo/db/s/split_chunk.h" +#include "mongo/logv2/log.h" #include "mongo/s/catalog/type_chunk.h" #include "mongo/util/log.h" #include "mongo/util/str.h" @@ -120,7 +121,7 @@ public: auto parseShardNameStatus = bsonExtractStringField(cmdObj, "from", &shardName); uassertStatusOK(parseShardNameStatus); - log() << "received splitChunk request: " << redact(cmdObj); + LOGV2(22104, "received splitChunk request: {cmdObj}", "cmdObj"_attr = redact(cmdObj)); vector<BSONObj> splitKeys; { diff --git a/src/mongo/db/s/split_chunk_test.cpp b/src/mongo/db/s/split_chunk_test.cpp index 5adb6c0b359..563692cac50 100644 --- a/src/mongo/db/s/split_chunk_test.cpp +++ b/src/mongo/db/s/split_chunk_test.cpp @@ -42,6 +42,7 @@ #include "mongo/executor/remote_command_request.h" #include "mongo/executor/remote_command_response.h" #include "mongo/executor/task_executor.h" +#include "mongo/logv2/log.h" #include "mongo/s/catalog/dist_lock_manager_mock.h" #include "mongo/s/catalog/type_chunk.h" #include "mongo/s/catalog/type_collection.h" @@ -169,8 +170,8 @@ void SplitChunkTest::expectLock() { dynamic_cast<DistLockManagerMock*>(distLock()) ->expectLock( [this](StringData name, StringData whyMessage, Milliseconds) { - LOG(0) << name; - LOG(0) << whyMessage; + LOGV2(22105, "{name}", "name"_attr = name); + LOGV2(22106, "{whyMessage}", "whyMessage"_attr = whyMessage); }, Status::OK()); } diff --git a/src/mongo/db/s/split_vector.cpp b/src/mongo/db/s/split_vector.cpp index 6b257208da1..725a60f95b1 100644 --- a/src/mongo/db/s/split_vector.cpp +++ b/src/mongo/db/s/split_vector.cpp @@ -44,6 +44,7 @@ #include "mongo/db/namespace_string.h" #include "mongo/db/query/internal_plans.h" #include "mongo/db/query/plan_executor.h" +#include "mongo/logv2/log.h" #include "mongo/util/log.h" namespace mongo { @@ -130,8 +131,11 @@ StatusWith<std::vector<BSONObj>> splitVector(OperationContext* opCtx, return emptyVector; } - log() << "request split points lookup for chunk " << nss.toString() << " " << redact(minKey) - << " -->> " << redact(maxKey); + LOGV2(22107, + "request split points lookup for chunk {nss} {minKey} -->> {maxKey}", + "nss"_attr = nss.toString(), + "minKey"_attr = redact(minKey), + "maxKey"_attr = redact(maxKey)); // We'll use the average object size and number of object to find approximately how many // keys each chunk should have. We'll split at half the maxChunkSizeBytes or @@ -141,8 +145,10 @@ StatusWith<std::vector<BSONObj>> splitVector(OperationContext* opCtx, long long keyCount = maxChunkSizeBytes.get() / (2 * avgRecSize); if (maxChunkObjects.get() && (maxChunkObjects.get() < keyCount)) { - log() << "limiting split vector to " << maxChunkObjects.get() << " (from " << keyCount - << ") objects "; + LOGV2(22108, + "limiting split vector to {maxChunkObjects_get} (from {keyCount}) objects ", + "maxChunkObjects_get"_attr = maxChunkObjects.get(), + "keyCount"_attr = keyCount); keyCount = maxChunkObjects.get(); } @@ -195,9 +201,14 @@ StatusWith<std::vector<BSONObj>> splitVector(OperationContext* opCtx, if (currKey.woCompare(maxKeyInChunk) == 0) { // Range contains only documents with a single key value. So we cannot possibly find a // split point, and there is no need to scan any further. - warning() << "possible low cardinality key detected in " << nss.toString() - << " - range " << redact(minKey) << " -->> " << redact(maxKey) - << " contains only the key " << redact(prettyKey(idx->keyPattern(), currKey)); + LOGV2_WARNING(22113, + "possible low cardinality key detected in {nss} - range {minKey} -->> " + "{maxKey} contains only the key {prettyKey_idx_keyPattern_currKey}", + "nss"_attr = nss.toString(), + "minKey"_attr = redact(minKey), + "maxKey"_attr = redact(maxKey), + "prettyKey_idx_keyPattern_currKey"_attr = + redact(prettyKey(idx->keyPattern(), currKey))); std::vector<BSONObj> emptyVector; return emptyVector; } @@ -232,9 +243,12 @@ StatusWith<std::vector<BSONObj>> splitVector(OperationContext* opCtx, continue; } - log() << "Max BSON response size reached for split vector before the" - << " end of chunk " << nss.toString() << " " << redact(minKey) - << " -->> " << redact(maxKey); + LOGV2(22109, + "Max BSON response size reached for split vector before the end " + "of chunk {nss} {minKey} -->> {maxKey}", + "nss"_attr = nss.toString(), + "minKey"_attr = redact(minKey), + "maxKey"_attr = redact(maxKey)); break; } @@ -242,15 +256,22 @@ StatusWith<std::vector<BSONObj>> splitVector(OperationContext* opCtx, splitKeys.push_back(currKey.getOwned()); currCount = 0; numChunks++; - LOG(4) << "picked a split key: " << redact(currKey); + LOGV2_DEBUG(22110, + 4, + "picked a split key: {currKey}", + "currKey"_attr = redact(currKey)); } } // Stop if we have enough split points. if (maxSplitPoints && maxSplitPoints.get() && (numChunks >= maxSplitPoints.get())) { - log() << "max number of requested split points reached (" << numChunks - << ") before the end of chunk " << nss.toString() << " " << redact(minKey) - << " -->> " << redact(maxKey); + LOGV2(22111, + "max number of requested split points reached ({numChunks}) before the " + "end of chunk {nss} {minKey} -->> {maxKey}", + "numChunks"_attr = numChunks, + "nss"_attr = nss.toString(), + "minKey"_attr = redact(minKey), + "maxKey"_attr = redact(maxKey)); break; } @@ -273,7 +294,9 @@ StatusWith<std::vector<BSONObj>> splitVector(OperationContext* opCtx, force = false; keyCount = currCount / 2; currCount = 0; - log() << "splitVector doing another cycle because of force, keyCount now: " << keyCount; + LOGV2(22112, + "splitVector doing another cycle because of force, keyCount now: {keyCount}", + "keyCount"_attr = keyCount); exec = InternalPlanner::indexScan(opCtx, collection, @@ -294,18 +317,28 @@ StatusWith<std::vector<BSONObj>> splitVector(OperationContext* opCtx, // Warn for keys that are more numerous than maxChunkSizeBytes allows. for (auto it = tooFrequentKeys.cbegin(); it != tooFrequentKeys.cend(); ++it) { - warning() << "possible low cardinality key detected in " << nss.toString() - << " - key is " << redact(prettyKey(idx->keyPattern(), *it)); + LOGV2_WARNING(22114, + "possible low cardinality key detected in {nss} - key is " + "{prettyKey_idx_keyPattern_it}", + "nss"_attr = nss.toString(), + "prettyKey_idx_keyPattern_it"_attr = + redact(prettyKey(idx->keyPattern(), *it))); } // Remove the sentinel at the beginning before returning splitKeys.erase(splitKeys.begin()); if (timer.millis() > serverGlobalParams.slowMS) { - warning() << "Finding the split vector for " << nss.toString() << " over " - << redact(keyPattern) << " keyCount: " << keyCount - << " numSplits: " << splitKeys.size() << " lookedAt: " << currCount - << " took " << timer.millis() << "ms"; + LOGV2_WARNING( + 22115, + "Finding the split vector for {nss} over {keyPattern} keyCount: {keyCount} " + "numSplits: {splitKeys_size} lookedAt: {currCount} took {timer_millis}ms", + "nss"_attr = nss.toString(), + "keyPattern"_attr = redact(keyPattern), + "keyCount"_attr = keyCount, + "splitKeys_size"_attr = splitKeys.size(), + "currCount"_attr = currCount, + "timer_millis"_attr = timer.millis()); } } diff --git a/src/mongo/db/s/transaction_coordinator.cpp b/src/mongo/db/s/transaction_coordinator.cpp index 9393168a86b..f890ee3ef5a 100644 --- a/src/mongo/db/s/transaction_coordinator.cpp +++ b/src/mongo/db/s/transaction_coordinator.cpp @@ -37,6 +37,7 @@ #include "mongo/db/s/transaction_coordinator_metrics_observer.h" #include "mongo/db/s/wait_for_majority_service.h" #include "mongo/db/server_options.h" +#include "mongo/logv2/log.h" #include "mongo/s/grid.h" #include "mongo/util/fail_point.h" #include "mongo/util/log.h" @@ -63,7 +64,7 @@ ExecutorFuture<void> waitForMajorityWithHangFailpoint(ServiceContext* service, if (auto sfp = failpoint.scoped(); MONGO_unlikely(sfp.isActive())) { const BSONObj& data = sfp.getData(); - LOG(0) << "Hit " << failPointName << " failpoint"; + LOGV2(22445, "Hit {failPointName} failpoint", "failPointName"_attr = failPointName); // Run the hang failpoint asynchronously on a different thread to avoid self deadlocks. return ExecutorFuture<void>(executor).then( @@ -200,9 +201,14 @@ TransactionCoordinator::TransactionCoordinator(OperationContext* operationContex } if (_decision->getDecision() == CommitDecision::kCommit) { - LOG(3) << txn::txnIdToString(_lsid, _txnNumber) - << " Advancing cluster time to the commit timestamp " - << *_decision->getCommitTimestamp(); + LOGV2_DEBUG(22446, + 3, + "{txn_txnIdToString_lsid_txnNumber} Advancing cluster time to " + "the commit timestamp {decision_getCommitTimestamp}", + "txn_txnIdToString_lsid_txnNumber"_attr = + txn::txnIdToString(_lsid, _txnNumber), + "decision_getCommitTimestamp"_attr = + *_decision->getCommitTimestamp()); uassertStatusOK(LogicalClock::get(_serviceContext) ->advanceClusterTime( @@ -382,8 +388,11 @@ void TransactionCoordinator::_done(Status status) { str::stream() << "Coordinator " << _lsid.getId() << ':' << _txnNumber << " stopped due to: " << status.reason()); - LOG(3) << txn::txnIdToString(_lsid, _txnNumber) << " Two-phase commit completed with " - << redact(status); + LOGV2_DEBUG(22447, + 3, + "{txn_txnIdToString_lsid_txnNumber} Two-phase commit completed with {status}", + "txn_txnIdToString_lsid_txnNumber"_attr = txn::txnIdToString(_lsid, _txnNumber), + "status"_attr = redact(status)); stdx::unique_lock<Latch> ul(_mutex); @@ -413,7 +422,9 @@ void TransactionCoordinator::_done(Status status) { void TransactionCoordinator::_logSlowTwoPhaseCommit( const txn::CoordinatorCommitDecision& decision) { - log() << _twoPhaseCommitInfoForLog(decision); + LOGV2(22448, + "{twoPhaseCommitInfoForLog_decision}", + "twoPhaseCommitInfoForLog_decision"_attr = _twoPhaseCommitInfoForLog(decision)); } std::string TransactionCoordinator::_twoPhaseCommitInfoForLog( diff --git a/src/mongo/db/s/transaction_coordinator_catalog.cpp b/src/mongo/db/s/transaction_coordinator_catalog.cpp index fc0612515b2..0b90a8694f3 100644 --- a/src/mongo/db/s/transaction_coordinator_catalog.cpp +++ b/src/mongo/db/s/transaction_coordinator_catalog.cpp @@ -33,6 +33,7 @@ #include "mongo/db/s/transaction_coordinator_catalog.h" +#include "mongo/logv2/log.h" #include "mongo/s/grid.h" #include "mongo/util/log.h" @@ -46,10 +47,12 @@ TransactionCoordinatorCatalog::~TransactionCoordinatorCatalog() { void TransactionCoordinatorCatalog::exitStepUp(Status status) { if (status.isOK()) { - LOG(0) << "Incoming coordinateCommit requests are now enabled"; + LOGV2(22438, "Incoming coordinateCommit requests are now enabled"); } else { - warning() << "Coordinator recovery failed and coordinateCommit requests will not be allowed" - << causedBy(status); + LOGV2_WARNING(22444, + "Coordinator recovery failed and coordinateCommit requests will not be " + "allowed{causedBy_status}", + "causedBy_status"_attr = causedBy(status)); } stdx::lock_guard<Latch> lk(_mutex); @@ -80,8 +83,11 @@ void TransactionCoordinatorCatalog::insert(OperationContext* opCtx, TxnNumber txnNumber, std::shared_ptr<TransactionCoordinator> coordinator, bool forStepUp) { - LOG(3) << "Inserting coordinator " << lsid.getId() << ':' << txnNumber - << " into in-memory catalog"; + LOGV2_DEBUG(22439, + 3, + "Inserting coordinator {lsid_getId}:{txnNumber} into in-memory catalog", + "lsid_getId"_attr = lsid.getId(), + "txnNumber"_attr = txnNumber); stdx::unique_lock<Latch> ul(_mutex); if (!forStepUp) { @@ -153,8 +159,11 @@ TransactionCoordinatorCatalog::getLatestOnSession(OperationContext* opCtx, } void TransactionCoordinatorCatalog::_remove(const LogicalSessionId& lsid, TxnNumber txnNumber) { - LOG(3) << "Removing coordinator " << lsid.getId() << ':' << txnNumber - << " from in-memory catalog"; + LOGV2_DEBUG(22440, + 3, + "Removing coordinator {lsid_getId}:{txnNumber} from in-memory catalog", + "lsid_getId"_attr = lsid.getId(), + "txnNumber"_attr = txnNumber); stdx::lock_guard<Latch> lk(_mutex); @@ -175,7 +184,7 @@ void TransactionCoordinatorCatalog::_remove(const LogicalSessionId& lsid, TxnNum } if (_coordinatorsBySession.empty()) { - LOG(3) << "Signaling last active coordinator removed"; + LOGV2_DEBUG(22441, 3, "Signaling last active coordinator removed"); _noActiveCoordinatorsCV.notify_all(); } } @@ -185,9 +194,11 @@ void TransactionCoordinatorCatalog::join() { while (!_noActiveCoordinatorsCV.wait_for( ul, stdx::chrono::seconds{5}, [this] { return _coordinatorsBySession.empty(); })) { - LOG(0) << "After 5 seconds of wait there are still " << _coordinatorsBySession.size() - << " sessions left with active coordinators which have not yet completed"; - LOG(0) << _toString(ul); + LOGV2(22442, + "After 5 seconds of wait there are still {coordinatorsBySession_size} sessions left " + "with active coordinators which have not yet completed", + "coordinatorsBySession_size"_attr = _coordinatorsBySession.size()); + LOGV2(22443, "{ul}", "ul"_attr = _toString(ul)); } } diff --git a/src/mongo/db/s/transaction_coordinator_futures_util.cpp b/src/mongo/db/s/transaction_coordinator_futures_util.cpp index 5f19b0eeb46..f243396228e 100644 --- a/src/mongo/db/s/transaction_coordinator_futures_util.cpp +++ b/src/mongo/db/s/transaction_coordinator_futures_util.cpp @@ -36,6 +36,7 @@ #include "mongo/client/remote_command_targeter.h" #include "mongo/db/auth/authorization_session.h" #include "mongo/db/s/sharding_state.h" +#include "mongo/logv2/log.h" #include "mongo/rpc/get_status_from_command_result.h" #include "mongo/s/grid.h" #include "mongo/transport/service_entry_point.h" @@ -96,7 +97,7 @@ Future<executor::TaskExecutor::ResponseStatus> AsyncWorkScheduler::scheduleRemot ->grantInternalAuthorization(opCtx->getClient()); if (MONGO_unlikely(hangWhileTargetingLocalHost.shouldFail())) { - LOG(0) << "Hit hangWhileTargetingLocalHost failpoint"; + LOGV2(22449, "Hit hangWhileTargetingLocalHost failpoint"); hangWhileTargetingLocalHost.pauseWhileSet(opCtx); } @@ -232,7 +233,9 @@ Future<AsyncWorkScheduler::HostAndShard> AsyncWorkScheduler::_targetHostAsync( const auto shard = uassertStatusOK(shardRegistry->getShard(opCtx, shardId)); if (MONGO_unlikely(hangWhileTargetingRemoteHost.shouldFail())) { - LOG(0) << "Hit hangWhileTargetingRemoteHost failpoint for shard " << shardId; + LOGV2(22450, + "Hit hangWhileTargetingRemoteHost failpoint for shard {shardId}", + "shardId"_attr = shardId); hangWhileTargetingRemoteHost.pauseWhileSet(opCtx); } diff --git a/src/mongo/db/s/transaction_coordinator_service.cpp b/src/mongo/db/s/transaction_coordinator_service.cpp index 3ac1212a468..131a79eef8a 100644 --- a/src/mongo/db/s/transaction_coordinator_service.cpp +++ b/src/mongo/db/s/transaction_coordinator_service.cpp @@ -37,6 +37,7 @@ #include "mongo/db/s/transaction_coordinator_document_gen.h" #include "mongo/db/transaction_participant_gen.h" #include "mongo/db/write_concern.h" +#include "mongo/logv2/log.h" #include "mongo/s/grid.h" #include "mongo/util/log.h" @@ -184,8 +185,10 @@ void TransactionCoordinatorService::onStepUp(OperationContext* opCtx, replClientInfo.setLastOpToSystemLastOpTime(opCtx); const auto lastOpTime = replClientInfo.getLastOp(); - LOG(3) << "Waiting for OpTime " << lastOpTime - << " to become majority committed"; + LOGV2_DEBUG(22451, + 3, + "Waiting for OpTime {lastOpTime} to become majority committed", + "lastOpTime"_attr = lastOpTime); WriteConcernResult unusedWCResult; uassertStatusOK(waitForWriteConcern( @@ -198,8 +201,10 @@ void TransactionCoordinatorService::onStepUp(OperationContext* opCtx, auto coordinatorDocs = txn::readAllCoordinatorDocs(opCtx); - LOG(0) << "Need to resume coordinating commit for " << coordinatorDocs.size() - << " transactions"; + LOGV2(22452, + "Need to resume coordinating commit for {coordinatorDocs_size} " + "transactions", + "coordinatorDocs_size"_attr = coordinatorDocs.size()); const auto service = opCtx->getServiceContext(); const auto clockSource = service->getFastClockSource(); @@ -208,7 +213,10 @@ void TransactionCoordinatorService::onStepUp(OperationContext* opCtx, auto& scheduler = catalogAndScheduler->scheduler; for (const auto& doc : coordinatorDocs) { - LOG(3) << "Going to resume coordinating commit for " << doc.toBSON(); + LOGV2_DEBUG(22453, + 3, + "Going to resume coordinating commit for {doc}", + "doc"_attr = doc.toBSON()); const auto lsid = *doc.getId().getSessionId(); const auto txnNumber = *doc.getId().getTxnNumber(); @@ -274,7 +282,7 @@ void TransactionCoordinatorService::joinPreviousRound() { if (!_catalogAndSchedulerToCleanup) return; - LOG(0) << "Waiting for coordinator tasks from previous term to complete"; + LOGV2(22454, "Waiting for coordinator tasks from previous term to complete"); // Block until all coordinators scheduled the previous time the service was primary to have // drained. Because the scheduler was interrupted, it should be extremely rare for there to be diff --git a/src/mongo/db/s/transaction_coordinator_test.cpp b/src/mongo/db/s/transaction_coordinator_test.cpp index 93083cb9739..3d6be9ff152 100644 --- a/src/mongo/db/s/transaction_coordinator_test.cpp +++ b/src/mongo/db/s/transaction_coordinator_test.cpp @@ -39,6 +39,7 @@ #include "mongo/db/s/transaction_coordinator_document_gen.h" #include "mongo/db/s/transaction_coordinator_metrics_observer.h" #include "mongo/db/s/transaction_coordinator_test_fixture.h" +#include "mongo/logv2/log.h" #include "mongo/unittest/unittest.h" #include "mongo/util/clock_source_mock.h" #include "mongo/util/log.h" @@ -87,7 +88,9 @@ void killClientOpCtx(ServiceContext* service, const std::string& clientName) { sleepmillis(50); } - error() << "Timed out trying to find and kill client opCtx with name: " << clientName; + LOGV2_ERROR(22462, + "Timed out trying to find and kill client opCtx with name: {clientName}", + "clientName"_attr = clientName); ASSERT_FALSE(true); } @@ -1513,7 +1516,7 @@ TEST_F(TransactionCoordinatorMetricsTest, SimpleTwoPhaseCommitRealCoordinator) { checkMetrics(expectedMetrics); - log() << "Create the coordinator."; + LOGV2(22455, "Create the coordinator."); expectedStats.createTime = advanceClockSourceAndReturnNewNow(); expectedStats.totalDuration = Microseconds(0); @@ -1531,8 +1534,9 @@ TEST_F(TransactionCoordinatorMetricsTest, SimpleTwoPhaseCommitRealCoordinator) { checkStats(stats, expectedStats); checkMetrics(expectedMetrics); - log() << "Start two-phase commit (allow the coordinator to progress to writing the participant " - "list)."; + LOGV2(22456, + "Start two-phase commit (allow the coordinator to progress to writing the participant " + "list)."); expectedStats.writingParticipantListStartTime = advanceClockSourceAndReturnNewNow(); tickSource()->advance(Microseconds(100)); @@ -1552,7 +1556,7 @@ TEST_F(TransactionCoordinatorMetricsTest, SimpleTwoPhaseCommitRealCoordinator) { checkStats(stats, expectedStats); checkMetrics(expectedMetrics); - log() << "Allow the coordinator to progress to waiting for votes."; + LOGV2(22457, "Allow the coordinator to progress to waiting for votes."); expectedStats.waitingForVotesStartTime = advanceClockSourceAndReturnNewNow(); tickSource()->advance(Microseconds(100)); @@ -1573,7 +1577,7 @@ TEST_F(TransactionCoordinatorMetricsTest, SimpleTwoPhaseCommitRealCoordinator) { checkStats(stats, expectedStats); checkMetrics(expectedMetrics); - log() << "Allow the coordinator to progress to writing the decision."; + LOGV2(22458, "Allow the coordinator to progress to writing the decision."); expectedStats.writingDecisionStartTime = advanceClockSourceAndReturnNewNow(); tickSource()->advance(Microseconds(100)); @@ -1599,7 +1603,7 @@ TEST_F(TransactionCoordinatorMetricsTest, SimpleTwoPhaseCommitRealCoordinator) { checkStats(stats, expectedStats); checkMetrics(expectedMetrics); - log() << "Allow the coordinator to progress to waiting for acks."; + LOGV2(22459, "Allow the coordinator to progress to waiting for acks."); expectedStats.waitingForDecisionAcksStartTime = advanceClockSourceAndReturnNewNow(); tickSource()->advance(Microseconds(100)); @@ -1623,7 +1627,7 @@ TEST_F(TransactionCoordinatorMetricsTest, SimpleTwoPhaseCommitRealCoordinator) { checkStats(stats, expectedStats); checkMetrics(expectedMetrics); - log() << "Allow the coordinator to progress to deleting the coordinator doc."; + LOGV2(22460, "Allow the coordinator to progress to deleting the coordinator doc."); expectedStats.deletingCoordinatorDocStartTime = advanceClockSourceAndReturnNewNow(); tickSource()->advance(Microseconds(100)); @@ -1649,7 +1653,7 @@ TEST_F(TransactionCoordinatorMetricsTest, SimpleTwoPhaseCommitRealCoordinator) { checkStats(stats, expectedStats); checkMetrics(expectedMetrics); - log() << "Allow the coordinator to complete."; + LOGV2(22461, "Allow the coordinator to complete."); expectedStats.endTime = advanceClockSourceAndReturnNewNow(); tickSource()->advance(Microseconds(100)); diff --git a/src/mongo/db/s/transaction_coordinator_util.cpp b/src/mongo/db/s/transaction_coordinator_util.cpp index 7d98befd9d2..f0968bc0644 100644 --- a/src/mongo/db/s/transaction_coordinator_util.cpp +++ b/src/mongo/db/s/transaction_coordinator_util.cpp @@ -43,6 +43,7 @@ #include "mongo/db/s/transaction_coordinator_futures_util.h" #include "mongo/db/s/transaction_coordinator_worker_curop_repository.h" #include "mongo/db/write_concern.h" +#include "mongo/logv2/log.h" #include "mongo/rpc/get_status_from_command_result.h" #include "mongo/util/fail_point.h" #include "mongo/util/log.h" @@ -105,10 +106,13 @@ repl::OpTime persistParticipantListBlocking(OperationContext* opCtx, const LogicalSessionId& lsid, TxnNumber txnNumber, const std::vector<ShardId>& participantList) { - LOG(3) << txnIdToString(lsid, txnNumber) << " Going to write participant list"; + LOGV2_DEBUG(22463, + 3, + "{txnIdToString_lsid_txnNumber} Going to write participant list", + "txnIdToString_lsid_txnNumber"_attr = txnIdToString(lsid, txnNumber)); if (MONGO_unlikely(hangBeforeWritingParticipantList.shouldFail())) { - LOG(0) << "Hit hangBeforeWritingParticipantList failpoint"; + LOGV2(22464, "Hit hangBeforeWritingParticipantList failpoint"); hangBeforeWritingParticipantList.pauseWhileSet(opCtx); } @@ -167,7 +171,10 @@ repl::OpTime persistParticipantListBlocking(OperationContext* opCtx, // Throw any other error. uassertStatusOK(upsertStatus); - LOG(3) << txnIdToString(lsid, txnNumber) << " Wrote participant list"; + LOGV2_DEBUG(22465, + 3, + "{txnIdToString_lsid_txnNumber} Wrote participant list", + "txnIdToString_lsid_txnNumber"_attr = txnIdToString(lsid, txnNumber)); return repl::ReplClientInfo::forClient(opCtx->getClient()).getLastOp(); } @@ -240,7 +247,7 @@ Future<PrepareVoteConsensus> sendPrepare(ServiceContext* service, opCtx, lsid, txnNumber, CoordinatorAction::kSendingPrepare); if (MONGO_unlikely(hangBeforeSendingPrepare.shouldFail())) { - LOG(0) << "Hit hangBeforeSendingPrepare failpoint"; + LOGV2(22466, "Hit hangBeforeSendingPrepare failpoint"); hangBeforeSendingPrepare.pauseWhileSet(opCtx); } }; @@ -289,11 +296,14 @@ repl::OpTime persistDecisionBlocking(OperationContext* opCtx, const std::vector<ShardId>& participantList, const txn::CoordinatorCommitDecision& decision) { const bool isCommit = decision.getDecision() == txn::CommitDecision::kCommit; - LOG(3) << txnIdToString(lsid, txnNumber) << " Going to write decision " - << (isCommit ? "commit" : "abort"); + LOGV2_DEBUG(22467, + 3, + "{txnIdToString_lsid_txnNumber} Going to write decision {isCommit_commit_abort}", + "txnIdToString_lsid_txnNumber"_attr = txnIdToString(lsid, txnNumber), + "isCommit_commit_abort"_attr = (isCommit ? "commit" : "abort")); if (MONGO_unlikely(hangBeforeWritingDecision.shouldFail())) { - LOG(0) << "Hit hangBeforeWritingDecision failpoint"; + LOGV2(22468, "Hit hangBeforeWritingDecision failpoint"); hangBeforeWritingDecision.pauseWhileSet(opCtx); } @@ -357,8 +367,11 @@ repl::OpTime persistDecisionBlocking(OperationContext* opCtx, << doc); } - LOG(3) << txnIdToString(lsid, txnNumber) << " Wrote decision " - << (isCommit ? "commit" : "abort"); + LOGV2_DEBUG(22469, + 3, + "{txnIdToString_lsid_txnNumber} Wrote decision {isCommit_commit_abort}", + "txnIdToString_lsid_txnNumber"_attr = txnIdToString(lsid, txnNumber), + "isCommit_commit_abort"_attr = (isCommit ? "commit" : "abort")); return repl::ReplClientInfo::forClient(opCtx->getClient()).getLastOp(); } @@ -402,7 +415,7 @@ Future<void> sendCommit(ServiceContext* service, opCtx, lsid, txnNumber, CoordinatorAction::kSendingCommit); if (MONGO_unlikely(hangBeforeSendingCommit.shouldFail())) { - LOG(0) << "Hit hangBeforeSendingCommit failpoint"; + LOGV2(22470, "Hit hangBeforeSendingCommit failpoint"); hangBeforeSendingCommit.pauseWhileSet(opCtx); } }; @@ -432,7 +445,7 @@ Future<void> sendAbort(ServiceContext* service, opCtx, lsid, txnNumber, CoordinatorAction::kSendingAbort); if (MONGO_unlikely(hangBeforeSendingAbort.shouldFail())) { - LOG(0) << "Hit hangBeforeSendingAbort failpoint"; + LOGV2(22471, "Hit hangBeforeSendingAbort failpoint"); hangBeforeSendingAbort.pauseWhileSet(opCtx); } }; @@ -449,10 +462,13 @@ namespace { void deleteCoordinatorDocBlocking(OperationContext* opCtx, const LogicalSessionId& lsid, TxnNumber txnNumber) { - LOG(3) << txnIdToString(lsid, txnNumber) << " Going to delete coordinator doc"; + LOGV2_DEBUG(22472, + 3, + "{txnIdToString_lsid_txnNumber} Going to delete coordinator doc", + "txnIdToString_lsid_txnNumber"_attr = txnIdToString(lsid, txnNumber)); if (MONGO_unlikely(hangBeforeDeletingCoordinatorDoc.shouldFail())) { - LOG(0) << "Hit hangBeforeDeletingCoordinatorDoc failpoint"; + LOGV2(22473, "Hit hangBeforeDeletingCoordinatorDoc failpoint"); hangBeforeDeletingCoordinatorDoc.pauseWhileSet(opCtx); } @@ -504,10 +520,13 @@ void deleteCoordinatorDocBlocking(OperationContext* opCtx, << doc); } - LOG(3) << txnIdToString(lsid, txnNumber) << " Deleted coordinator doc"; + LOGV2_DEBUG(22474, + 3, + "{txnIdToString_lsid_txnNumber} Deleted coordinator doc", + "txnIdToString_lsid_txnNumber"_attr = txnIdToString(lsid, txnNumber)); hangAfterDeletingCoordinatorDoc.execute([&](const BSONObj& data) { - LOG(0) << "Hit hangAfterDeletingCoordinatorDoc failpoint"; + LOGV2(22475, "Hit hangAfterDeletingCoordinatorDoc failpoint"); if (!data["useUninterruptibleSleep"].eoo()) { hangAfterDeletingCoordinatorDoc.pauseWhileSet(); } else { @@ -576,8 +595,14 @@ Future<PrepareResponse> sendPrepareToShard(ServiceContext* service, isLocalShard, commandObj = commandObj.getOwned(), operationContextFn] { - LOG(3) << txnIdToString(lsid, txnNumber) << " Coordinator going to send command " - << commandObj << " to " << (isLocalShard ? "local " : "") << "shard " << shardId; + LOGV2_DEBUG(22476, + 3, + "{txnIdToString_lsid_txnNumber} Coordinator going to send command " + "{commandObj} to {isLocalShard_local}shard {shardId}", + "txnIdToString_lsid_txnNumber"_attr = txnIdToString(lsid, txnNumber), + "commandObj"_attr = commandObj, + "isLocalShard_local"_attr = (isLocalShard ? "local " : ""), + "shardId"_attr = shardId); return scheduler .scheduleRemoteCommand( @@ -605,16 +630,26 @@ Future<PrepareResponse> sendPrepareToShard(ServiceContext* service, << shardId << ", which is not an expected behavior. " "Interpreting the response as vote to abort"); - LOG(0) << txnIdToString(lsid, txnNumber) << " " << redact(abortStatus); + LOGV2(22477, + "{txnIdToString_lsid_txnNumber} {abortStatus}", + "txnIdToString_lsid_txnNumber"_attr = + txnIdToString(lsid, txnNumber), + "abortStatus"_attr = redact(abortStatus)); return PrepareResponse{ shardId, PrepareVote::kAbort, boost::none, abortStatus}; } - LOG(3) << txnIdToString(lsid, txnNumber) - << " Coordinator shard received a vote to commit from shard " - << shardId - << " with prepareTimestamp: " << prepareTimestampField.timestamp(); + LOGV2_DEBUG(22478, + 3, + "{txnIdToString_lsid_txnNumber} Coordinator shard received a " + "vote to commit from shard {shardId} with prepareTimestamp: " + "{prepareTimestampField_timestamp}", + "txnIdToString_lsid_txnNumber"_attr = + txnIdToString(lsid, txnNumber), + "shardId"_attr = shardId, + "prepareTimestampField_timestamp"_attr = + prepareTimestampField.timestamp()); return PrepareResponse{shardId, PrepareVote::kCommit, @@ -622,8 +657,15 @@ Future<PrepareResponse> sendPrepareToShard(ServiceContext* service, boost::none}; } - LOG(3) << txnIdToString(lsid, txnNumber) << " Coordinator shard received " - << status << " from shard " << shardId << " for " << commandObj; + LOGV2_DEBUG(22479, + 3, + "{txnIdToString_lsid_txnNumber} Coordinator shard received " + "{status} from shard {shardId} for {commandObj}", + "txnIdToString_lsid_txnNumber"_attr = + txnIdToString(lsid, txnNumber), + "status"_attr = status, + "shardId"_attr = shardId, + "commandObj"_attr = commandObj); if (ErrorCodes::isVoteAbortError(status.code())) { return PrepareResponse{ @@ -652,8 +694,11 @@ Future<PrepareResponse> sendPrepareToShard(ServiceContext* service, return std::move(f).onError<ErrorCodes::TransactionCoordinatorReachedAbortDecision>( [lsid, txnNumber, shardId](const Status& status) { - LOG(3) << txnIdToString(lsid, txnNumber) - << " Prepare stopped retrying due to retrying being cancelled"; + LOGV2_DEBUG(22480, + 3, + "{txnIdToString_lsid_txnNumber} Prepare stopped retrying due to retrying " + "being cancelled", + "txnIdToString_lsid_txnNumber"_attr = txnIdToString(lsid, txnNumber)); return PrepareResponse{shardId, boost::none, boost::none, status}; }); } @@ -682,8 +727,14 @@ Future<void> sendDecisionToShard(ServiceContext* service, isLocalShard, operationContextFn, commandObj = commandObj.getOwned()] { - LOG(3) << txnIdToString(lsid, txnNumber) << " Coordinator going to send command " - << commandObj << " to " << (isLocalShard ? "local " : "") << "shard " << shardId; + LOGV2_DEBUG(22481, + 3, + "{txnIdToString_lsid_txnNumber} Coordinator going to send command " + "{commandObj} to {isLocalShard_local}shard {shardId}", + "txnIdToString_lsid_txnNumber"_attr = txnIdToString(lsid, txnNumber), + "commandObj"_attr = commandObj, + "isLocalShard_local"_attr = (isLocalShard ? "local " : ""), + "shardId"_attr = shardId); return scheduler .scheduleRemoteCommand( @@ -699,9 +750,15 @@ Future<void> sendDecisionToShard(ServiceContext* service, status = wcStatus; } - LOG(3) << txnIdToString(lsid, txnNumber) << " Coordinator shard received " - << status << " in response to " << commandObj << " from shard " - << shardId; + LOGV2_DEBUG(22482, + 3, + "{txnIdToString_lsid_txnNumber} Coordinator shard received " + "{status} in response to {commandObj} from shard {shardId}", + "txnIdToString_lsid_txnNumber"_attr = + txnIdToString(lsid, txnNumber), + "status"_attr = status, + "commandObj"_attr = commandObj, + "shardId"_attr = shardId); if (ErrorCodes::isVoteAbortError(status.code())) { // Interpret voteAbort errors as an ack. diff --git a/src/mongo/db/s/txn_two_phase_commit_cmds.cpp b/src/mongo/db/s/txn_two_phase_commit_cmds.cpp index 01bd1fed90a..6246e02d1ae 100644 --- a/src/mongo/db/s/txn_two_phase_commit_cmds.cpp +++ b/src/mongo/db/s/txn_two_phase_commit_cmds.cpp @@ -39,6 +39,7 @@ #include "mongo/db/s/transaction_coordinator_service.h" #include "mongo/db/session_catalog_mongod.h" #include "mongo/db/transaction_participant.h" +#include "mongo/logv2/log.h" #include "mongo/rpc/get_status_from_command_result.h" #include "mongo/util/log.h" @@ -99,10 +100,12 @@ public: "prepareTransaction must be run within a transaction", txnParticipant); - LOG(3) - << "Participant shard received prepareTransaction for transaction with txnNumber " - << opCtx->getTxnNumber() << " on session " - << opCtx->getLogicalSessionId()->toBSON(); + LOGV2_DEBUG(22483, + 3, + "Participant shard received prepareTransaction for transaction with " + "txnNumber {opCtx_getTxnNumber} on session {opCtx_getLogicalSessionId}", + "opCtx_getTxnNumber"_attr = opCtx->getTxnNumber(), + "opCtx_getLogicalSessionId"_attr = opCtx->getLogicalSessionId()->toBSON()); uassert(ErrorCodes::NoSuchTransaction, "Transaction isn't in progress", @@ -199,10 +202,14 @@ std::set<ShardId> validateParticipants(OperationContext* opCtx, } ss << ']'; - LOG(3) << "Coordinator shard received request to coordinate commit with " - "participant list " - << ss.str() << " for " << opCtx->getLogicalSessionId()->getId() << ':' - << opCtx->getTxnNumber(); + LOGV2_DEBUG( + 22484, + 3, + "Coordinator shard received request to coordinate commit with " + "participant list {ss_str} for {opCtx_getLogicalSessionId_getId}:{opCtx_getTxnNumber}", + "ss_str"_attr = ss.str(), + "opCtx_getLogicalSessionId_getId"_attr = opCtx->getLogicalSessionId()->getId(), + "opCtx_getTxnNumber"_attr = opCtx->getTxnNumber()); return participantsSet; } @@ -234,7 +241,7 @@ public: validateParticipants(opCtx, cmd.getParticipants())); if (MONGO_unlikely(hangAfterStartingCoordinateCommit.shouldFail())) { - LOG(0) << "Hit hangAfterStartingCoordinateCommit failpoint"; + LOGV2(22485, "Hit hangAfterStartingCoordinateCommit failpoint"); hangAfterStartingCoordinateCommit.pauseWhileSet(opCtx); } @@ -274,8 +281,13 @@ public: // No coordinator was found in memory. Recover the decision from the local participant. - LOG(3) << "Going to recover decision from local participant for " - << opCtx->getLogicalSessionId()->getId() << ':' << opCtx->getTxnNumber(); + LOGV2_DEBUG(22486, + 3, + "Going to recover decision from local participant for " + "{opCtx_getLogicalSessionId_getId}:{opCtx_getTxnNumber}", + "opCtx_getLogicalSessionId_getId"_attr = + opCtx->getLogicalSessionId()->getId(), + "opCtx_getTxnNumber"_attr = opCtx->getTxnNumber()); boost::optional<SharedSemiFuture<void>> participantExitPrepareFuture; { diff --git a/src/mongo/db/s/wait_for_majority_service.cpp b/src/mongo/db/s/wait_for_majority_service.cpp index 1864335150a..d2895908c49 100644 --- a/src/mongo/db/s/wait_for_majority_service.cpp +++ b/src/mongo/db/s/wait_for_majority_service.cpp @@ -39,6 +39,7 @@ #include "mongo/db/write_concern.h" #include "mongo/executor/network_interface_factory.h" #include "mongo/executor/thread_pool_task_executor.h" +#include "mongo/logv2/log.h" #include "mongo/util/concurrency/thread_pool.h" #include "mongo/util/log.h" @@ -184,7 +185,7 @@ void WaitForMajorityService::_periodicallyWaitForMajority(ServiceContext* servic _opCtx->waitForConditionOrInterrupt( _hasNewOpTimeCV, lk, [&] { return !_queuedOpTimes.empty() || _inShutDown; }); } catch (const DBException& e) { - LOG(1) << "Unable to wait for new op time due to: " << e; + LOGV2_DEBUG(22487, 1, "Unable to wait for new op time due to: {e}", "e"_attr = e); } _opCtx = nullptr; |