summaryrefslogtreecommitdiff
path: root/src/mongo/db/s
diff options
context:
space:
mode:
authormathisbessamdb <mathis.bessa@mongodb.com>2023-03-30 22:46:03 +0000
committerEvergreen Agent <no-reply@evergreen.mongodb.com>2023-03-30 23:59:43 +0000
commit4b802111fba9c2c02cc6faecd83724a902dbd223 (patch)
treebdce416af6327784a97ac466a0a1324b4738951a /src/mongo/db/s
parented52ca8e4e41f3bcc35d24e177925c1dd30f7796 (diff)
downloadmongo-4b802111fba9c2c02cc6faecd83724a902dbd223.tar.gz
SERVER-73115 Always include tenant in collection name attributes in log lines
Diffstat (limited to 'src/mongo/db/s')
-rw-r--r--src/mongo/db/s/analyze_shard_key_cmd.cpp5
-rw-r--r--src/mongo/db/s/analyze_shard_key_cmd_util.cpp12
-rw-r--r--src/mongo/db/s/auto_split_vector.cpp10
-rw-r--r--src/mongo/db/s/balancer/balancer_chunk_selection_policy_impl.cpp6
-rw-r--r--src/mongo/db/s/balancer/balancer_defragmentation_policy_impl.cpp18
-rw-r--r--src/mongo/db/s/balancer/balancer_policy.cpp6
-rw-r--r--src/mongo/db/s/cleanup_orphaned_cmd.cpp6
-rw-r--r--src/mongo/db/s/collection_metadata.cpp2
-rw-r--r--src/mongo/db/s/collection_sharding_runtime.cpp8
-rw-r--r--src/mongo/db/s/collmod_coordinator.cpp2
-rw-r--r--src/mongo/db/s/compact_structured_encryption_data_coordinator.cpp2
-rw-r--r--src/mongo/db/s/config/configsvr_abort_reshard_collection_command.cpp2
-rw-r--r--src/mongo/db/s/config/configsvr_run_restore_command.cpp2
-rw-r--r--src/mongo/db/s/config/sharding_catalog_manager_chunk_operations.cpp4
-rw-r--r--src/mongo/db/s/config/sharding_catalog_manager_collection_operations.cpp8
-rw-r--r--src/mongo/db/s/configure_query_analyzer_cmd.cpp2
-rw-r--r--src/mongo/db/s/create_collection_coordinator.cpp35
-rw-r--r--src/mongo/db/s/drop_collection_coordinator.cpp20
-rw-r--r--src/mongo/db/s/drop_database_coordinator.cpp4
-rw-r--r--src/mongo/db/s/flush_routing_table_cache_updates_command.cpp2
-rw-r--r--src/mongo/db/s/metadata_manager.cpp8
-rw-r--r--src/mongo/db/s/migration_coordinator.cpp6
-rw-r--r--src/mongo/db/s/migration_destination_manager.cpp8
-rw-r--r--src/mongo/db/s/migration_source_manager.cpp2
-rw-r--r--src/mongo/db/s/migration_util.cpp4
-rw-r--r--src/mongo/db/s/move_primary_coordinator.cpp2
-rw-r--r--src/mongo/db/s/move_primary_source_manager.cpp2
-rw-r--r--src/mongo/db/s/op_observer_sharding_impl.cpp2
-rw-r--r--src/mongo/db/s/query_analysis_writer.cpp71
-rw-r--r--src/mongo/db/s/range_deleter_service_op_observer.cpp2
-rw-r--r--src/mongo/db/s/range_deletion_util.cpp28
-rw-r--r--src/mongo/db/s/refine_collection_shard_key_coordinator.cpp2
-rw-r--r--src/mongo/db/s/rename_collection_coordinator.cpp4
-rw-r--r--src/mongo/db/s/rename_collection_participant_service.cpp4
-rw-r--r--src/mongo/db/s/reshard_collection_coordinator.cpp2
-rw-r--r--src/mongo/db/s/resharding/resharding_coordinator_commit_monitor.cpp4
-rw-r--r--src/mongo/db/s/resharding/resharding_coordinator_service.cpp8
-rw-r--r--src/mongo/db/s/resharding/resharding_donor_service.cpp8
-rw-r--r--src/mongo/db/s/resharding/resharding_manual_cleanup.cpp2
-rw-r--r--src/mongo/db/s/resharding/resharding_op_observer.cpp2
-rw-r--r--src/mongo/db/s/resharding/resharding_recipient_service.cpp4
-rw-r--r--src/mongo/db/s/session_catalog_migration_destination.cpp6
-rw-r--r--src/mongo/db/s/set_allow_migrations_coordinator.cpp2
-rw-r--r--src/mongo/db/s/shard_filtering_metadata_refresh.cpp10
-rw-r--r--src/mongo/db/s/shard_metadata_util.cpp6
-rw-r--r--src/mongo/db/s/shard_server_catalog_cache_loader.cpp18
-rw-r--r--src/mongo/db/s/sharding_ddl_coordinator.cpp2
-rw-r--r--src/mongo/db/s/sharding_index_catalog_ddl_util.cpp12
-rw-r--r--src/mongo/db/s/sharding_initialization_mongod.cpp6
-rw-r--r--src/mongo/db/s/sharding_recovery_service.cpp22
-rw-r--r--src/mongo/db/s/split_vector.cpp12
-rw-r--r--src/mongo/db/s/user_writes_recoverable_critical_section_service.cpp28
52 files changed, 213 insertions, 242 deletions
diff --git a/src/mongo/db/s/analyze_shard_key_cmd.cpp b/src/mongo/db/s/analyze_shard_key_cmd.cpp
index 5a0a3557245..a8a10345abd 100644
--- a/src/mongo/db/s/analyze_shard_key_cmd.cpp
+++ b/src/mongo/db/s/analyze_shard_key_cmd.cpp
@@ -79,10 +79,7 @@ public:
uassertStatusOK(validateNamespace(nss));
const auto collUuid = uassertStatusOK(validateCollectionOptionsLocally(opCtx, nss));
- LOGV2(6875001,
- "Start analyzing shard key",
- "namespace"_attr = nss,
- "shardKey"_attr = key);
+ LOGV2(6875001, "Start analyzing shard key", logAttrs(nss), "shardKey"_attr = key);
Response response;
diff --git a/src/mongo/db/s/analyze_shard_key_cmd_util.cpp b/src/mongo/db/s/analyze_shard_key_cmd_util.cpp
index 6dac0d170f2..6bcec29207b 100644
--- a/src/mongo/db/s/analyze_shard_key_cmd_util.cpp
+++ b/src/mongo/db/s/analyze_shard_key_cmd_util.cpp
@@ -347,7 +347,7 @@ CardinalityFrequencyMetrics calculateCardinalityAndFrequencyUnique(OperationCont
int64_t numDocs) {
LOGV2(6915302,
"Calculating cardinality and frequency for a unique shard key",
- "namespace"_attr = nss,
+ logAttrs(nss),
"shardKey"_attr = shardKey);
CardinalityFrequencyMetrics metrics;
@@ -401,7 +401,7 @@ CardinalityFrequencyMetrics calculateCardinalityAndFrequencyGeneric(OperationCon
const BSONObj& hintIndexKey) {
LOGV2(6915303,
"Calculating cardinality and frequency for a non-unique shard key",
- "namespace"_attr = nss,
+ logAttrs(nss),
"shardKey"_attr = shardKey,
"indexKey"_attr = hintIndexKey);
@@ -458,7 +458,7 @@ MonotonicityMetrics calculateMonotonicity(OperationContext* opCtx,
const BSONObj& shardKey) {
LOGV2(6915304,
"Calculating monotonicity",
- "namespace"_attr = collection->ns(),
+ logAttrs(collection->ns()),
"shardKey"_attr = shardKey);
MonotonicityMetrics metrics;
@@ -538,7 +538,7 @@ MonotonicityMetrics calculateMonotonicity(OperationContext* opCtx,
auto coefficientThreshold = gMonotonicityCorrelationCoefficientThreshold.load();
LOGV2(6875302,
"Calculated monotonicity",
- "namespace"_attr = collection->ns(),
+ logAttrs(collection->ns()),
"shardKey"_attr = shardKey,
"indexKey"_attr = indexKeyPattern,
"numRecords"_attr = recordIds.size(),
@@ -778,7 +778,7 @@ KeyCharacteristicsMetrics calculateKeyCharacteristicsMetrics(OperationContext* o
LOGV2(6915305,
"Calculating metrics about the characteristics of the shard key",
- "namespace"_attr = nss,
+ logAttrs(nss),
"shardKey"_attr = shardKeyBson,
"indexKey"_attr = indexKeyBson);
analyzeShardKeyPauseBeforeCalculatingKeyCharacteristicsMetrics.pauseWhileSet(opCtx);
@@ -816,7 +816,7 @@ std::pair<ReadDistributionMetrics, WriteDistributionMetrics> calculateReadWriteD
const KeyPattern& shardKey) {
LOGV2(6915306,
"Calculating metrics about the read and write distribution",
- "namespace"_attr = nss,
+ logAttrs(nss),
"shardKey"_attr = shardKey);
analyzeShardKeyPauseBeforeCalculatingReadWriteDistributionMetrics.pauseWhileSet(opCtx);
diff --git a/src/mongo/db/s/auto_split_vector.cpp b/src/mongo/db/s/auto_split_vector.cpp
index d33ca4485a9..c4acaca4376 100644
--- a/src/mongo/db/s/auto_split_vector.cpp
+++ b/src/mongo/db/s/auto_split_vector.cpp
@@ -201,7 +201,7 @@ std::pair<std::vector<BSONObj>, bool> autoSplitVector(OperationContext* opCtx,
LOGV2_WARNING(
5865001,
"Possible low cardinality key detected in range. Range contains only a single key.",
- "namespace"_attr = collection.getNss(),
+ logAttrs(collection.getNss()),
"minKey"_attr = redact(prettyKey(keyPattern, minKey)),
"maxKey"_attr = redact(prettyKey(keyPattern, maxKey)),
"key"_attr = redact(prettyKey(shardKeyIdx->keyPattern(), firstKeyInOriginalChunk)));
@@ -210,7 +210,7 @@ std::pair<std::vector<BSONObj>, bool> autoSplitVector(OperationContext* opCtx,
LOGV2(6492600,
"Requested split points lookup for range",
- "namespace"_attr = nss,
+ logAttrs(nss),
"minKey"_attr = redact(prettyKey(keyPattern, minKey)),
"maxKey"_attr = redact(prettyKey(keyPattern, maxKey)),
"direction"_attr = forward ? "forwards" : "backwards");
@@ -374,7 +374,7 @@ std::pair<std::vector<BSONObj>, bool> autoSplitVector(OperationContext* opCtx,
if (reachedMaxBSONSize) {
LOGV2(5865002,
"Max BSON response size reached for split vector before the end of chunk",
- "namespace"_attr = nss,
+ logAttrs(nss),
"minKey"_attr = redact(prettyKey(shardKeyIdx->keyPattern(), minKey)),
"maxKey"_attr = redact(prettyKey(shardKeyIdx->keyPattern(), maxKey)));
}
@@ -384,14 +384,14 @@ std::pair<std::vector<BSONObj>, bool> autoSplitVector(OperationContext* opCtx,
for (const auto& frequentKey : tooFrequentKeys) {
LOGV2_WARNING(5865004,
"Possible low cardinality key detected",
- "namespace"_attr = nss,
+ logAttrs(nss),
"key"_attr = redact(prettyKey(keyPattern, frequentKey)));
}
if (elapsedMillisToFindSplitPoints > serverGlobalParams.slowMS.load()) {
LOGV2_WARNING(5865005,
"Finding the auto split vector completed",
- "namespace"_attr = nss,
+ logAttrs(nss),
"keyPattern"_attr = redact(keyPattern),
"numSplits"_attr = splitKeys.size(),
"duration"_attr = Milliseconds(elapsedMillisToFindSplitPoints));
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 c7dac82d521..6ffbe40a4da 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
@@ -378,7 +378,7 @@ StatusWith<SplitInfoVector> BalancerChunkSelectionPolicyImpl::selectChunksToSpli
21852,
"Unable to enforce zone range policy for collection {namespace}: {error}",
"Unable to enforce zone range policy for collection",
- "namespace"_attr = nss.ns(),
+ logAttrs(nss),
"error"_attr = candidatesStatus.getStatus());
}
@@ -457,7 +457,7 @@ StatusWith<MigrateInfoVector> BalancerChunkSelectionPolicyImpl::selectChunksToMo
LOGV2_DEBUG(5966401,
1,
"Not balancing explicitly disabled collection",
- "namespace"_attr = coll.getNss(),
+ logAttrs(coll.getNss()),
"allowBalance"_attr = coll.getAllowBalance(),
"allowMigrations"_attr = coll.getAllowMigrations(),
"permitMigrations"_attr = coll.getPermitMigrations(),
@@ -489,7 +489,7 @@ StatusWith<MigrateInfoVector> BalancerChunkSelectionPolicyImpl::selectChunksToMo
} else if (!swMigrateCandidates.isOK()) {
LOGV2_WARNING(21853,
"Unable to balance collection",
- "namespace"_attr = nss.ns(),
+ logAttrs(nss),
"error"_attr = swMigrateCandidates.getStatus());
continue;
}
diff --git a/src/mongo/db/s/balancer/balancer_defragmentation_policy_impl.cpp b/src/mongo/db/s/balancer/balancer_defragmentation_policy_impl.cpp
index e7130f99e9b..93f3fea7d23 100644
--- a/src/mongo/db/s/balancer/balancer_defragmentation_policy_impl.cpp
+++ b/src/mongo/db/s/balancer/balancer_defragmentation_policy_impl.cpp
@@ -139,7 +139,7 @@ void handleActionResult(OperationContext* opCtx,
LOGV2_DEBUG(6261701,
1,
"Hit retriable error while defragmenting collection",
- "namespace"_attr = nss,
+ logAttrs(nss),
"uuid"_attr = uuid,
"currentPhase"_attr = currentPhase,
"error"_attr = redact(status));
@@ -147,7 +147,7 @@ void handleActionResult(OperationContext* opCtx,
} else {
LOGV2_ERROR(6258601,
"Defragmentation for collection hit non-retriable error",
- "namespace"_attr = nss,
+ logAttrs(nss),
"uuid"_attr = uuid,
"currentPhase"_attr = currentPhase,
"error"_attr = redact(status));
@@ -536,7 +536,7 @@ public:
LOGV2_DEBUG(6290000,
1,
"Migration failed during collection defragmentation",
- "namespace"_attr = _nss,
+ logAttrs(_nss),
"uuid"_attr = _uuid,
"currentPhase"_attr = getType(),
"error"_attr = redact(migrationResponse));
@@ -580,7 +580,7 @@ public:
LOGV2_ERROR(6290001,
"Encountered non-retriable error on migration during "
"collection defragmentation",
- "namespace"_attr = _nss,
+ logAttrs(_nss),
"uuid"_attr = _uuid,
"currentPhase"_attr = getType(),
"error"_attr = redact(migrationResponse));
@@ -841,7 +841,7 @@ private:
LOGV2_WARNING(
6172701,
"Chunk with no estimated size detected while building MoveAndMergeChunksPhase",
- "namespace"_attr = _nss,
+ logAttrs(_nss),
"uuid"_attr = _uuid,
"range"_attr = chunk.getRange());
_abort(DefragmentationPhaseEnum::kMergeAndMeasureChunks);
@@ -970,7 +970,7 @@ private:
1,
"Postponing small chunk processing due to pending range deletion "
"on recipient shard(s)",
- "namespace"_attr = _nss,
+ logAttrs(_nss),
"uuid"_attr = _uuid,
"range"_attr = (*candidateIt)->range,
"estimatedSizeBytes"_attr = (*candidateIt)->estimatedSizeBytes,
@@ -981,7 +981,7 @@ private:
} else {
LOGV2(6290003,
"Discarding small chunk due to pending range deletion on recipient shard",
- "namespace"_attr = _nss,
+ logAttrs(_nss),
"uuid"_attr = _uuid,
"range"_attr = (*candidateIt)->range,
"estimatedSizeBytes"_attr = (*candidateIt)->estimatedSizeBytes,
@@ -1477,7 +1477,7 @@ std::unique_ptr<DefragmentationPhase> BalancerDefragmentationPolicyImpl::_transi
afterBuildingNextDefragmentationPhase.pauseWhileSet();
LOGV2(6172702,
"Collection defragmentation transitioned to new phase",
- "namespace"_attr = coll.getNss(),
+ logAttrs(coll.getNss()),
"phase"_attr = nextPhaseObject
? DefragmentationPhase_serializer(nextPhaseObject->getType())
: kNoPhase,
@@ -1485,7 +1485,7 @@ std::unique_ptr<DefragmentationPhase> BalancerDefragmentationPolicyImpl::_transi
} catch (const DBException& e) {
LOGV2_ERROR(6153101,
"Error while building defragmentation phase on collection",
- "namespace"_attr = coll.getNss(),
+ logAttrs(coll.getNss()),
"uuid"_attr = coll.getUuid(),
"phase"_attr = nextPhase,
"error"_attr = e);
diff --git a/src/mongo/db/s/balancer/balancer_policy.cpp b/src/mongo/db/s/balancer/balancer_policy.cpp
index d267ffc55bf..d09fa6eae65 100644
--- a/src/mongo/db/s/balancer/balancer_policy.cpp
+++ b/src/mongo/db/s/balancer/balancer_policy.cpp
@@ -508,7 +508,7 @@ MigrateInfosWithReason BalancerPolicy::balance(
"cannot be balanced. This should be corrected by either assigning shards "
"to the zone or by deleting it.",
"zone"_attr = redact(zone),
- "namespace"_attr = distribution.nss());
+ logAttrs(distribution.nss()));
}
continue;
}
@@ -575,7 +575,7 @@ bool BalancerPolicy::_singleZoneBalanceBasedOnDataSize(
LOGV2_DEBUG(6581601,
1,
"Balancing single zone",
- "namespace"_attr = distribution.nss().ns(),
+ logAttrs(distribution.nss()),
"zone"_attr = zone,
"fromShardId"_attr = from,
"fromShardDataSize"_attr = fromSize,
@@ -618,7 +618,7 @@ bool BalancerPolicy::_singleZoneBalanceBasedOnDataSize(
if (numJumboChunks) {
LOGV2_WARNING(6581602,
"Shard has only jumbo chunks for this collection and cannot be balanced",
- "namespace"_attr = distribution.nss().ns(),
+ logAttrs(distribution.nss()),
"shardId"_attr = from,
"zone"_attr = zone,
"numJumboChunks"_attr = numJumboChunks);
diff --git a/src/mongo/db/s/cleanup_orphaned_cmd.cpp b/src/mongo/db/s/cleanup_orphaned_cmd.cpp
index 395a15c79fc..690f6b57a90 100644
--- a/src/mongo/db/s/cleanup_orphaned_cmd.cpp
+++ b/src/mongo/db/s/cleanup_orphaned_cmd.cpp
@@ -71,7 +71,7 @@ CleanupResult cleanupOrphanedData(OperationContext* opCtx,
"{namespace} does not exist",
"cleanupOrphaned skipping waiting for orphaned data cleanup because "
"collection does not exist",
- "namespace"_attr = ns.ns());
+ logAttrs(ns));
return CleanupResult::kDone;
}
collectionUuid.emplace(autoColl.getCollection()->uuid());
@@ -85,7 +85,7 @@ CleanupResult cleanupOrphanedData(OperationContext* opCtx,
"{namespace} is not sharded",
"cleanupOrphaned skipping waiting for orphaned data cleanup because "
"collection is not sharded",
- "namespace"_attr = ns.ns());
+ logAttrs(ns));
return CleanupResult::kDone;
}
range.emplace(optCollDescr->getMinKey(), optCollDescr->getMaxKey());
@@ -121,7 +121,7 @@ CleanupResult cleanupOrphanedData(OperationContext* opCtx,
LOGV2(4416003,
"cleanupOrphaned going to wait for range deletion tasks to complete",
- "namespace"_attr = ns.ns(),
+ logAttrs(ns),
"collectionUUID"_attr = *collectionUuid,
"numRemainingDeletionTasks"_attr = numRemainingDeletionTasks);
diff --git a/src/mongo/db/s/collection_metadata.cpp b/src/mongo/db/s/collection_metadata.cpp
index 17e07fa9a01..973bf101580 100644
--- a/src/mongo/db/s/collection_metadata.cpp
+++ b/src/mongo/db/s/collection_metadata.cpp
@@ -101,7 +101,7 @@ void CollectionMetadata::throwIfReshardingInProgress(NamespaceString const& nss)
const auto& reshardingFields = getReshardingFields();
// Throw if the coordinator is not in states "aborting", "committing", or "done".
if (reshardingFields && reshardingFields->getState() < CoordinatorStateEnum::kAborting) {
- LOGV2(5277122, "reshardCollection in progress", "namespace"_attr = nss.toString());
+ LOGV2(5277122, "reshardCollection in progress", logAttrs(nss));
uasserted(ErrorCodes::ReshardCollectionInProgress,
"reshardCollection is in progress for namespace " + nss.toString());
diff --git a/src/mongo/db/s/collection_sharding_runtime.cpp b/src/mongo/db/s/collection_sharding_runtime.cpp
index 11493489e73..9354a6a3775 100644
--- a/src/mongo/db/s/collection_sharding_runtime.cpp
+++ b/src/mongo/db/s/collection_sharding_runtime.cpp
@@ -273,7 +273,7 @@ void CollectionShardingRuntime::setFilteringMetadata(OperationContext* opCtx,
LOGV2(21917,
"Marking collection {namespace} as unsharded",
"Marking collection as unsharded",
- "namespace"_attr = _nss.ns());
+ logAttrs(_nss));
_metadataType = MetadataType::kUnsharded;
_metadataManager.reset();
++_numMetadataManagerChanges;
@@ -304,7 +304,7 @@ void CollectionShardingRuntime::_clearFilteringMetadata(OperationContext* opCtx,
1,
"Clearing metadata for collection {namespace}",
"Clearing collection metadata",
- "namespace"_attr = _nss,
+ logAttrs(_nss),
"collIsDropped"_attr = collIsDropped);
// If the collection is sharded and it's being dropped we might need to clean up some state.
@@ -378,7 +378,7 @@ Status CollectionShardingRuntime::waitForClean(OperationContext* opCtx,
{logv2::LogComponent::kShardingMigration},
"Finished waiting for deletion of {namespace} range {orphanRange}",
"Finished waiting for deletion of orphans",
- "namespace"_attr = nss.ns(),
+ logAttrs(nss),
"orphanRange"_attr = redact(orphanRange.toString()));
return Status::OK();
}
@@ -387,7 +387,7 @@ Status CollectionShardingRuntime::waitForClean(OperationContext* opCtx,
{logv2::LogComponent::kShardingMigration},
"Waiting for deletion of {namespace} range {orphanRange}",
"Waiting for deletion of orphans",
- "namespace"_attr = nss.ns(),
+ logAttrs(nss),
"orphanRange"_attr = orphanRange);
try {
opCtx->runWithDeadline(
diff --git a/src/mongo/db/s/collmod_coordinator.cpp b/src/mongo/db/s/collmod_coordinator.cpp
index 1159b4e10a7..f6164cad0a2 100644
--- a/src/mongo/db/s/collmod_coordinator.cpp
+++ b/src/mongo/db/s/collmod_coordinator.cpp
@@ -409,7 +409,7 @@ ExecutorFuture<void> CollModCoordinator::_runImpl(
!status.isA<ErrorCategory::ShutdownError>()) {
LOGV2_ERROR(5757002,
"Error running collMod",
- "namespace"_attr = nss(),
+ logAttrs(nss()),
"error"_attr = redact(status));
}
return status;
diff --git a/src/mongo/db/s/compact_structured_encryption_data_coordinator.cpp b/src/mongo/db/s/compact_structured_encryption_data_coordinator.cpp
index 2817b009055..399df32325f 100644
--- a/src/mongo/db/s/compact_structured_encryption_data_coordinator.cpp
+++ b/src/mongo/db/s/compact_structured_encryption_data_coordinator.cpp
@@ -173,7 +173,7 @@ void doRenameOperation(const CompactStructuredEncryptionDataState& state,
LOGV2_DEBUG(7299603,
1,
"Create collection failed because namespace already exists",
- "namespace"_attr = ecocNss);
+ logAttrs(ecocNss));
}
if (MONGO_unlikely(fleCompactHangAfterECOCCreate.shouldFail())) {
diff --git a/src/mongo/db/s/config/configsvr_abort_reshard_collection_command.cpp b/src/mongo/db/s/config/configsvr_abort_reshard_collection_command.cpp
index 57f11dd1472..5471e0b7126 100644
--- a/src/mongo/db/s/config/configsvr_abort_reshard_collection_command.cpp
+++ b/src/mongo/db/s/config/configsvr_abort_reshard_collection_command.cpp
@@ -116,7 +116,7 @@ public:
LOGV2(5403501,
"Aborting resharding operation",
- "namespace"_attr = ns(),
+ logAttrs(ns()),
"reshardingUUID"_attr = reshardingUUID);
assertExistsReshardingDocument(opCtx, reshardingUUID);
diff --git a/src/mongo/db/s/config/configsvr_run_restore_command.cpp b/src/mongo/db/s/config/configsvr_run_restore_command.cpp
index 26e334e5200..774b110f777 100644
--- a/src/mongo/db/s/config/configsvr_run_restore_command.cpp
+++ b/src/mongo/db/s/config/configsvr_run_restore_command.cpp
@@ -305,7 +305,7 @@ public:
"doc"_attr = doc,
"shouldRestore"_attr = shouldRestore,
logAttrs(coll->ns().dbName()),
- "dbNss"_attr = dbNss.toString());
+ "dbNss"_attr = dbNss);
if (shouldRestore) {
// This database had at least one collection restored.
diff --git a/src/mongo/db/s/config/sharding_catalog_manager_chunk_operations.cpp b/src/mongo/db/s/config/sharding_catalog_manager_chunk_operations.cpp
index c16e53a9c34..b39022158f9 100644
--- a/src/mongo/db/s/config/sharding_catalog_manager_chunk_operations.cpp
+++ b/src/mongo/db/s/config/sharding_catalog_manager_chunk_operations.cpp
@@ -1543,7 +1543,7 @@ void ShardingCatalogManager::upgradeChunksHistory(OperationContext* opCtx,
LOGV2(620650,
"Resetting the 'historyIsAt40' field for all chunks in collection {namespace} in "
"order to force all chunks' history to get recreated",
- "namespace"_attr = nss.ns());
+ logAttrs(nss));
BatchedCommandRequest request([collUuid = coll.getUuid()] {
write_ops::UpdateCommandRequest updateOp(ChunkType::ConfigNS);
@@ -2122,7 +2122,7 @@ void ShardingCatalogManager::splitOrMarkJumbo(OperationContext* opCtx,
"Couldn't mark chunk with namespace {namespace} and min key {minKey} as "
"jumbo due to {error}",
"Couldn't mark chunk as jumbo",
- "namespace"_attr = redact(nss.ns()),
+ "namespace"_attr = redact(toStringForLogging(nss)),
"minKey"_attr = redact(chunk.getMin()),
"error"_attr = redact(status.getStatus()));
}
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 44176d8a36b..f92b75aab48 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
@@ -274,7 +274,7 @@ void ShardingCatalogManager::refineCollectionShardKey(OperationContext* opCtx,
"refineCollectionShardKey updated collection entry for {namespace}: took "
"{durationMillis} ms. Total time taken: {totalTimeMillis} ms.",
"refineCollectionShardKey updated collection entry",
- "namespace"_attr = nss.ns(),
+ logAttrs(nss),
"durationMillis"_attr = timers->executionTimer.millis(),
"totalTimeMillis"_attr = timers->totalTimer.millis());
timers->executionTimer.reset();
@@ -308,7 +308,7 @@ void ShardingCatalogManager::refineCollectionShardKey(OperationContext* opCtx,
"refineCollectionShardKey: updated chunk entries for {namespace}: took "
"{durationMillis} ms. Total time taken: {totalTimeMillis} ms.",
"refineCollectionShardKey: updated chunk entries",
- "namespace"_attr = nss.ns(),
+ logAttrs(nss),
"durationMillis"_attr = timers->executionTimer.millis(),
"totalTimeMillis"_attr = timers->totalTimer.millis());
timers->executionTimer.reset();
@@ -332,7 +332,7 @@ void ShardingCatalogManager::refineCollectionShardKey(OperationContext* opCtx,
"refineCollectionShardKey: updated zone entries for {namespace}: took "
"{durationMillis} ms. Total time taken: {totalTimeMillis} ms.",
"refineCollectionShardKey: updated zone entries",
- "namespace"_attr = nss.ns(),
+ logAttrs(nss),
"durationMillis"_attr = timers->executionTimer.millis(),
"totalTimeMillis"_attr = timers->totalTimer.millis());
@@ -376,7 +376,7 @@ void ShardingCatalogManager::refineCollectionShardKey(OperationContext* opCtx,
"in {namespace}",
"refineCollectionShardKey: failed to best-effort refresh all shards containing chunks",
"error"_attr = ex.toStatus(),
- "namespace"_attr = nss.ns());
+ logAttrs(nss));
}
}
diff --git a/src/mongo/db/s/configure_query_analyzer_cmd.cpp b/src/mongo/db/s/configure_query_analyzer_cmd.cpp
index 461182b12c7..ab918556719 100644
--- a/src/mongo/db/s/configure_query_analyzer_cmd.cpp
+++ b/src/mongo/db/s/configure_query_analyzer_cmd.cpp
@@ -181,7 +181,7 @@ public:
// document.
LOGV2(6915001,
"Persisting query analyzer configuration",
- "namespace"_attr = nss,
+ logAttrs(nss),
"collectionUUID"_attr = collUuid,
"mode"_attr = mode,
"sampleRate"_attr = sampleRate);
diff --git a/src/mongo/db/s/create_collection_coordinator.cpp b/src/mongo/db/s/create_collection_coordinator.cpp
index 7cf3ac16467..e2c1ce1e85c 100644
--- a/src/mongo/db/s/create_collection_coordinator.cpp
+++ b/src/mongo/db/s/create_collection_coordinator.cpp
@@ -563,7 +563,7 @@ ExecutorFuture<void> CreateCollectionCoordinator::_runImpl(
LOGV2_DEBUG(5458704,
1,
"Removing partial changes from previous run",
- "namespace"_attr = nss());
+ logAttrs(nss()));
_updateSession(opCtx);
cleanupPartialChunksFromPreviousAttempt(
@@ -659,7 +659,7 @@ ExecutorFuture<void> CreateCollectionCoordinator::_runImpl(
!status.isA<ErrorCategory::ShutdownError>()) {
LOGV2_ERROR(5458702,
"Error running create collection",
- "namespace"_attr = originalNss(),
+ logAttrs(originalNss()),
"error"_attr = redact(status));
auto opCtxHolder = cc().makeOperationContext();
@@ -801,8 +801,7 @@ CreateCollectionCoordinator::_checkIfCollectionAlreadyShardedWithSameOptions(
}
void CreateCollectionCoordinator::_checkCommandArguments(OperationContext* opCtx) {
- LOGV2_DEBUG(
- 5277902, 2, "Create collection _checkCommandArguments", "namespace"_attr = originalNss());
+ LOGV2_DEBUG(5277902, 2, "Create collection _checkCommandArguments", logAttrs(originalNss()));
if (originalNss().dbName() == DatabaseName::kConfig) {
// Only allowlisted collections in config may be sharded (unless we are in test mode)
@@ -1046,8 +1045,7 @@ void CreateCollectionCoordinator::_releaseCriticalSections(OperationContext* opC
void CreateCollectionCoordinator::_createCollectionAndIndexes(
OperationContext* opCtx, const ShardKeyPattern& shardKeyPattern) {
- LOGV2_DEBUG(
- 5277903, 2, "Create collection _createCollectionAndIndexes", "namespace"_attr = nss());
+ LOGV2_DEBUG(5277903, 2, "Create collection _createCollectionAndIndexes", logAttrs(nss()));
const auto& collationBSON = _doc.getTranslatedRequestParams()->getCollation();
boost::optional<Collation> collation;
@@ -1101,10 +1099,7 @@ void CreateCollectionCoordinator::_createCollectionAndIndexes(
auto createStatus = getStatusFromCommandResult(createRes);
if (!createStatus.isOK() && createStatus.code() == ErrorCodes::NamespaceExists) {
- LOGV2_DEBUG(5909400,
- 3,
- "Timeseries namespace already exists",
- "namespace"_attr = viewName.toString());
+ LOGV2_DEBUG(5909400, 3, "Timeseries namespace already exists", logAttrs(viewName));
} else {
uassertStatusOK(createStatus);
}
@@ -1152,7 +1147,7 @@ void CreateCollectionCoordinator::_createCollectionAndIndexes(
void CreateCollectionCoordinator::_createPolicy(OperationContext* opCtx,
const ShardKeyPattern& shardKeyPattern) {
- LOGV2_DEBUG(6042001, 2, "Create collection _createPolicy", "namespace"_attr = nss());
+ LOGV2_DEBUG(6042001, 2, "Create collection _createPolicy", logAttrs(nss()));
_collectionEmpty = checkIfCollectionIsEmpty(opCtx, nss());
_splitPolicy = InitialSplitPolicy::calculateOptimizationStrategy(
@@ -1167,7 +1162,7 @@ void CreateCollectionCoordinator::_createPolicy(OperationContext* opCtx,
void CreateCollectionCoordinator::_createChunks(OperationContext* opCtx,
const ShardKeyPattern& shardKeyPattern) {
- LOGV2_DEBUG(5277904, 2, "Create collection _createChunks", "namespace"_attr = nss());
+ LOGV2_DEBUG(5277904, 2, "Create collection _createChunks", logAttrs(nss()));
_initialChunks = _splitPolicy->createFirstChunks(
opCtx, shardKeyPattern, {*_collectionUUID, ShardingState::get(opCtx)->shardId()});
@@ -1178,10 +1173,8 @@ void CreateCollectionCoordinator::_createChunks(OperationContext* opCtx,
void CreateCollectionCoordinator::_createCollectionOnNonPrimaryShards(
OperationContext* opCtx, const OperationSessionInfo& osi) {
- LOGV2_DEBUG(5277905,
- 2,
- "Create collection _createCollectionOnNonPrimaryShards",
- "namespace"_attr = nss());
+ LOGV2_DEBUG(
+ 5277905, 2, "Create collection _createCollectionOnNonPrimaryShards", logAttrs(nss()));
std::vector<AsyncRequestsSender::Request> requests;
std::set<ShardId> initializedShards;
@@ -1240,7 +1233,7 @@ void CreateCollectionCoordinator::_createCollectionOnNonPrimaryShards(
void CreateCollectionCoordinator::_commit(OperationContext* opCtx,
const std::shared_ptr<executor::TaskExecutor>& executor) {
- LOGV2_DEBUG(5277906, 2, "Create collection _commit", "namespace"_attr = nss());
+ LOGV2_DEBUG(5277906, 2, "Create collection _commit", logAttrs(nss()));
if (MONGO_unlikely(failAtCommitCreateCollectionCoordinator.shouldFail())) {
LOGV2_DEBUG(6960301, 2, "About to hit failAtCommitCreateCollectionCoordinator fail point");
@@ -1309,13 +1302,13 @@ void CreateCollectionCoordinator::_commit(OperationContext* opCtx,
notifyChangeStreamsOnShardCollection(
opCtx, nss(), *_collectionUUID, _request.toBSON(), CommitPhase::kSuccessful);
- LOGV2_DEBUG(5277907, 2, "Collection successfully committed", "namespace"_attr = nss());
+ LOGV2_DEBUG(5277907, 2, "Collection successfully committed", logAttrs(nss()));
forceShardFilteringMetadataRefresh(opCtx, nss());
} catch (const DBException& ex) {
LOGV2(5277908,
"Failed to obtain collection's placement version, so it will be recovered",
- "namespace"_attr = nss(),
+ logAttrs(nss()),
"error"_attr = redact(ex));
// If the refresh fails, then set the placement version to UNKNOWN and let a future
@@ -1354,7 +1347,7 @@ void CreateCollectionCoordinator::_commit(OperationContext* opCtx,
LOGV2(5277901,
"Created initial chunk(s)",
- "namespace"_attr = nss(),
+ logAttrs(nss()),
"numInitialChunks"_attr = _initialChunks->chunks.size(),
"initialCollectionPlacementVersion"_attr = _initialChunks->collPlacementVersion());
@@ -1365,7 +1358,7 @@ void CreateCollectionCoordinator::_commit(OperationContext* opCtx,
LOGV2(5458701,
"Collection created",
- "namespace"_attr = nss(),
+ logAttrs(nss()),
"UUID"_attr = _result->getCollectionUUID(),
"placementVersion"_attr = _result->getCollectionVersion());
}
diff --git a/src/mongo/db/s/drop_collection_coordinator.cpp b/src/mongo/db/s/drop_collection_coordinator.cpp
index 72a2612734a..f132c385522 100644
--- a/src/mongo/db/s/drop_collection_coordinator.cpp
+++ b/src/mongo/db/s/drop_collection_coordinator.cpp
@@ -122,7 +122,7 @@ void DropCollectionCoordinator::dropCollectionLocally(OperationContext* opCtx,
LOGV2_DEBUG(5280920,
1,
"Namespace not found while trying to delete local collection",
- "namespace"_attr = nss);
+ logAttrs(nss));
}
// Force the refresh of the catalog cache to purge outdated information. Note also that this
@@ -179,7 +179,7 @@ ExecutorFuture<void> DropCollectionCoordinator::_runImpl(
!status.isA<ErrorCategory::ShutdownError>()) {
LOGV2_ERROR(5280901,
"Error running drop collection",
- "namespace"_attr = nss(),
+ logAttrs(nss()),
"error"_attr = redact(status));
}
return status;
@@ -257,7 +257,7 @@ void DropCollectionCoordinator::_freezeMigrations(
void DropCollectionCoordinator::_enterCriticalSection(
std::shared_ptr<executor::ScopedTaskExecutor> executor) {
- LOGV2_DEBUG(7038100, 2, "Acquiring critical section", "namespace"_attr = nss());
+ LOGV2_DEBUG(7038100, 2, "Acquiring critical section", logAttrs(nss()));
auto opCtxHolder = cc().makeOperationContext();
auto* opCtx = opCtxHolder.get();
@@ -278,7 +278,7 @@ void DropCollectionCoordinator::_enterCriticalSection(
Grid::get(opCtx)->shardRegistry()->getAllShardIds(opCtx),
**executor);
- LOGV2_DEBUG(7038101, 2, "Acquired critical section", "namespace"_attr = nss());
+ LOGV2_DEBUG(7038101, 2, "Acquired critical section", logAttrs(nss()));
}
void DropCollectionCoordinator::_commitDropCollection(
@@ -289,11 +289,7 @@ void DropCollectionCoordinator::_commitDropCollection(
const auto collIsSharded = bool(_doc.getCollInfo());
- LOGV2_DEBUG(5390504,
- 2,
- "Dropping collection",
- "namespace"_attr = nss(),
- "sharded"_attr = collIsSharded);
+ LOGV2_DEBUG(5390504, 2, "Dropping collection", logAttrs(nss()), "sharded"_attr = collIsSharded);
if (collIsSharded) {
invariant(_doc.getCollInfo());
@@ -333,12 +329,12 @@ void DropCollectionCoordinator::_commitDropCollection(
sharding_ddl_util::removeQueryAnalyzerMetadataFromConfig(opCtx, nss(), boost::none);
ShardingLogging::get(opCtx)->logChange(opCtx, "dropCollection", nss().ns());
- LOGV2(5390503, "Collection dropped", "namespace"_attr = nss());
+ LOGV2(5390503, "Collection dropped", logAttrs(nss()));
}
void DropCollectionCoordinator::_exitCriticalSection(
std::shared_ptr<executor::ScopedTaskExecutor> executor) {
- LOGV2_DEBUG(7038102, 2, "Releasing critical section", "namespace"_attr = nss());
+ LOGV2_DEBUG(7038102, 2, "Releasing critical section", logAttrs(nss()));
auto opCtxHolder = cc().makeOperationContext();
auto* opCtx = opCtxHolder.get();
@@ -359,7 +355,7 @@ void DropCollectionCoordinator::_exitCriticalSection(
Grid::get(opCtx)->shardRegistry()->getAllShardIds(opCtx),
**executor);
- LOGV2_DEBUG(7038103, 2, "Released critical section", "namespace"_attr = nss());
+ LOGV2_DEBUG(7038103, 2, "Released critical section", logAttrs(nss()));
}
} // namespace mongo
diff --git a/src/mongo/db/s/drop_database_coordinator.cpp b/src/mongo/db/s/drop_database_coordinator.cpp
index c6bfc15a2db..869f317e83c 100644
--- a/src/mongo/db/s/drop_database_coordinator.cpp
+++ b/src/mongo/db/s/drop_database_coordinator.cpp
@@ -365,13 +365,13 @@ ExecutorFuture<void> DropDatabaseCoordinator::_runImpl(
LOGV2_DEBUG(5494504,
2,
"Completing collection drop from previous primary",
- "namespace"_attr = coll.getNss());
+ logAttrs(coll.getNss()));
_dropShardedCollection(opCtx, coll, executor);
}
for (const auto& coll : allCollectionsForDb) {
const auto& nss = coll.getNss();
- LOGV2_DEBUG(5494505, 2, "Dropping collection", "namespace"_attr = nss);
+ LOGV2_DEBUG(5494505, 2, "Dropping collection", logAttrs(nss));
sharding_ddl_util::stopMigrations(opCtx, nss, coll.getUuid());
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 e0e75cdef57..6293ca44321 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
@@ -136,7 +136,7 @@ public:
1,
"Forcing remote routing table refresh for {namespace}",
"Forcing remote routing table refresh",
- "namespace"_attr = ns());
+ logAttrs(ns()));
onCollectionPlacementVersionMismatch(opCtx, ns(), boost::none);
}
diff --git a/src/mongo/db/s/metadata_manager.cpp b/src/mongo/db/s/metadata_manager.cpp
index 58fe60d946e..944caf1e768 100644
--- a/src/mongo/db/s/metadata_manager.cpp
+++ b/src/mongo/db/s/metadata_manager.cpp
@@ -183,7 +183,7 @@ void MetadataManager::setFilteringMetadata(CollectionMetadata remoteMetadata) {
"collection placement version",
"Ignoring incoming metadata update for this namespace because the active "
"(current) metadata has the same or a newer collection placement version",
- "namespace"_attr = _nss.ns(),
+ logAttrs(_nss),
"activeMetadata"_attr = activeMetadata.toStringBasic(),
"remoteMetadata"_attr = remoteMetadata.toStringBasic());
return;
@@ -194,7 +194,7 @@ void MetadataManager::setFilteringMetadata(CollectionMetadata remoteMetadata) {
"{remoteMetadata} has a newer collection placement version",
"Updating metadata for this namespace because the remote metadata has a newer "
"collection placement version",
- "namespace"_attr = _nss.ns(),
+ logAttrs(_nss),
"activeMetadata"_attr = activeMetadata.toStringBasic(),
"remoteMetadata"_attr = remoteMetadata.toStringBasic());
@@ -275,7 +275,7 @@ SharedSemiFuture<void> MetadataManager::cleanUpRange(ChunkRange const& range,
"dependent queries finish",
"Deletion of the collection's specified range will be scheduled after all "
"possibly dependent queries finish",
- "namespace"_attr = _nss.ns(),
+ logAttrs(_nss),
"range"_attr = redact(range.toString()));
++overlapMetadata->numContingentRangeDeletionTasks;
// Schedule the range for deletion once the overlapping metadata object is destroyed
@@ -291,7 +291,7 @@ SharedSemiFuture<void> MetadataManager::cleanUpRange(ChunkRange const& range,
{logv2::LogComponent::kShardingMigration},
"Scheduling deletion of {namespace} range {range}",
"Scheduling deletion of the collection's specified range",
- "namespace"_attr = _nss.ns(),
+ logAttrs(_nss),
"range"_attr = redact(range.toString()));
return _submitRangeForDeletion(
diff --git a/src/mongo/db/s/migration_coordinator.cpp b/src/mongo/db/s/migration_coordinator.cpp
index 03c14f24c4c..0c08a71a1f4 100644
--- a/src/mongo/db/s/migration_coordinator.cpp
+++ b/src/mongo/db/s/migration_coordinator.cpp
@@ -212,7 +212,7 @@ SharedSemiFuture<void> MigrationCoordinator::_commitMigrationOnDonorAndRecipient
"Bumping transaction number with lsid {lsid} and current txnNumber {currentTxnNumber} on "
"recipient shard {recipientShardId} for commit of collection {nss}",
"Bumping transaction number on recipient shard for commit",
- "namespace"_attr = _migrationInfo.getNss(),
+ logAttrs(_migrationInfo.getNss()),
"recipientShardId"_attr = _migrationInfo.getRecipientShardId(),
"lsid"_attr = _migrationInfo.getLsid(),
"currentTxnNumber"_attr = _migrationInfo.getTxnNumber(),
@@ -337,7 +337,7 @@ void MigrationCoordinator::_abortMigrationOnDonorAndRecipient(OperationContext*
"{currentTxnNumber} on "
"recipient shard {recipientShardId} for abort of collection {nss}",
"Bumping transaction number on recipient shard for abort",
- "namespace"_attr = _migrationInfo.getNss(),
+ logAttrs(_migrationInfo.getNss()),
"recipientShardId"_attr = _migrationInfo.getRecipientShardId(),
"lsid"_attr = _migrationInfo.getLsid(),
"currentTxnNumber"_attr = _migrationInfo.getTxnNumber(),
@@ -351,7 +351,7 @@ void MigrationCoordinator::_abortMigrationOnDonorAndRecipient(OperationContext*
1,
"Failed to advance transaction number on recipient shard for abort and/or "
"marking range deletion task on recipient as ready for processing",
- "namespace"_attr = _migrationInfo.getNss(),
+ logAttrs(_migrationInfo.getNss()),
"migrationId"_attr = _migrationInfo.getId(),
"recipientShardId"_attr = _migrationInfo.getRecipientShardId(),
"currentTxnNumber"_attr = _migrationInfo.getTxnNumber(),
diff --git a/src/mongo/db/s/migration_destination_manager.cpp b/src/mongo/db/s/migration_destination_manager.cpp
index e5608787dd6..4d5dbece3a7 100644
--- a/src/mongo/db/s/migration_destination_manager.cpp
+++ b/src/mongo/db/s/migration_destination_manager.cpp
@@ -1200,7 +1200,7 @@ void MigrationDestinationManager::_migrateDriver(OperationContext* outerOpCtx,
"Starting receiving end of chunk migration",
"chunkMin"_attr = redact(_min),
"chunkMax"_attr = redact(_max),
- "namespace"_attr = _nss.ns(),
+ logAttrs(_nss),
"fromShard"_attr = _fromShard,
"epoch"_attr = _epoch,
"sessionId"_attr = *_sessionId,
@@ -1251,7 +1251,7 @@ void MigrationDestinationManager::_migrateDriver(OperationContext* outerOpCtx,
"overlaps with a range already scheduled for deletion",
"Migration paused because the requested range overlaps with a range already "
"scheduled for deletion",
- "namespace"_attr = _nss.ns(),
+ logAttrs(_nss),
"range"_attr = redact(range.toString()),
"migrationId"_attr = _migrationId->toBSON());
@@ -1807,7 +1807,7 @@ bool MigrationDestinationManager::_flushPendingWrites(OperationContext* opCtx,
"{chunkMin} -> {chunkMax}; waiting to reach this operation: {lastOpApplied}",
"Migration commit waiting for majority replication; waiting until the last "
"operation applied has been replicated",
- "namespace"_attr = _nss.ns(),
+ logAttrs(_nss),
"chunkMin"_attr = redact(_min),
"chunkMax"_attr = redact(_max),
"lastOpApplied"_attr = op,
@@ -1819,7 +1819,7 @@ bool MigrationDestinationManager::_flushPendingWrites(OperationContext* opCtx,
LOGV2(22008,
"Migration commit succeeded flushing to secondaries for {namespace}, {min} -> {max}",
"Migration commit succeeded flushing to secondaries",
- "namespace"_attr = _nss.ns(),
+ logAttrs(_nss),
"chunkMin"_attr = redact(_min),
"chunkMax"_attr = redact(_max),
"migrationId"_attr = _migrationId->toBSON());
diff --git a/src/mongo/db/s/migration_source_manager.cpp b/src/mongo/db/s/migration_source_manager.cpp
index 21bbaac190f..e56f7effee8 100644
--- a/src/mongo/db/s/migration_source_manager.cpp
+++ b/src/mongo/db/s/migration_source_manager.cpp
@@ -625,7 +625,7 @@ void MigrationSourceManager::commitChunkMetadataOnConfig() {
"Waiting for migration cleanup after chunk commit for the namespace {namespace} "
"and range {range}",
"Waiting for migration cleanup after chunk commit",
- "namespace"_attr = nss(),
+ logAttrs(nss()),
"range"_attr = redact(range.toString()),
"migrationId"_attr = _coordinator->getMigrationId());
diff --git a/src/mongo/db/s/migration_util.cpp b/src/mongo/db/s/migration_util.cpp
index 520650cc39c..6106752b5c5 100644
--- a/src/mongo/db/s/migration_util.cpp
+++ b/src/mongo/db/s/migration_util.cpp
@@ -1034,7 +1034,7 @@ void resumeMigrationCoordinationsOnStepUp(OperationContext* opCtx) {
void recoverMigrationCoordinations(OperationContext* opCtx,
NamespaceString nss,
CancellationToken cancellationToken) {
- LOGV2_DEBUG(4798501, 2, "Starting migration recovery", "namespace"_attr = nss);
+ LOGV2_DEBUG(4798501, 2, "Starting migration recovery", logAttrs(nss));
unsigned migrationRecoveryCount = 0;
@@ -1329,7 +1329,7 @@ void asyncRecoverMigrationUntilSuccessOrStepDown(OperationContext* opCtx,
// This is expected in the event of a stepdown.
LOGV2(6316100,
"Interrupted deferred migration recovery",
- "namespace"_attr = nss,
+ logAttrs(nss),
"error"_attr = redact(ex));
}
})
diff --git a/src/mongo/db/s/move_primary_coordinator.cpp b/src/mongo/db/s/move_primary_coordinator.cpp
index 1b695aafc45..1acdefda3b3 100644
--- a/src/mongo/db/s/move_primary_coordinator.cpp
+++ b/src/mongo/db/s/move_primary_coordinator.cpp
@@ -590,7 +590,7 @@ void MovePrimaryCoordinator::dropStaleDataOnDonor(OperationContext* opCtx) const
if (!dropStatus.isOK()) {
LOGV2_WARNING(7120210,
"Failed to drop stale collection on donor",
- "namespace"_attr = nss,
+ logAttrs(nss),
"error"_attr = redact(dropStatus));
}
}
diff --git a/src/mongo/db/s/move_primary_source_manager.cpp b/src/mongo/db/s/move_primary_source_manager.cpp
index 2b41a4195ef..fcf62395449 100644
--- a/src/mongo/db/s/move_primary_source_manager.cpp
+++ b/src/mongo/db/s/move_primary_source_manager.cpp
@@ -471,7 +471,7 @@ Status MovePrimarySourceManager::cleanStaleData(OperationContext* opCtx) {
LOGV2(22045,
"Failed to drop cloned collection {namespace} in movePrimary: {error}",
"Failed to drop cloned collection in movePrimary",
- "namespace"_attr = coll,
+ logAttrs(coll),
"error"_attr = redact(dropStatus));
}
}
diff --git a/src/mongo/db/s/op_observer_sharding_impl.cpp b/src/mongo/db/s/op_observer_sharding_impl.cpp
index c0a419f1dc5..df2e29b84e0 100644
--- a/src/mongo/db/s/op_observer_sharding_impl.cpp
+++ b/src/mongo/db/s/op_observer_sharding_impl.cpp
@@ -79,7 +79,7 @@ void assertNoMovePrimaryInProgress(OperationContext* opCtx, const NamespaceStrin
const auto scopedDss =
DatabaseShardingState::assertDbLockedAndAcquireShared(opCtx, nss.dbName());
if (scopedDss->isMovePrimaryInProgress()) {
- LOGV2(4908600, "assertNoMovePrimaryInProgress", "namespace"_attr = nss.toString());
+ LOGV2(4908600, "assertNoMovePrimaryInProgress", logAttrs(nss));
uasserted(ErrorCodes::MovePrimaryInProgress,
"movePrimary is in progress for namespace " + nss.toString());
diff --git a/src/mongo/db/s/query_analysis_writer.cpp b/src/mongo/db/s/query_analysis_writer.cpp
index ef563b88e32..ce7e1cc1a44 100644
--- a/src/mongo/db/s/query_analysis_writer.cpp
+++ b/src/mongo/db/s/query_analysis_writer.cpp
@@ -87,7 +87,7 @@ BSONObj createSampledQueriesTTLIndex(OperationContext* opCtx) {
LOGV2_DEBUG(7078401,
1,
"Creation of the TTL index for the collection storing sampled queries",
- "namespace"_attr = NamespaceString::kConfigSampledQueriesNamespace,
+ logAttrs(NamespaceString::kConfigSampledQueriesNamespace),
"response"_attr = redact(resObj));
return resObj;
@@ -110,7 +110,7 @@ BSONObj createSampledQueriesDiffTTLIndex(OperationContext* opCtx) {
LOGV2_DEBUG(7078402,
1,
"Creation of the TTL index for the collection storing sampled diffs",
- "namespace"_attr = NamespaceString::kConfigSampledQueriesDiffNamespace,
+ logAttrs(NamespaceString::kConfigSampledQueriesDiffNamespace),
"response"_attr = redact(resObj));
return resObj;
@@ -341,14 +341,14 @@ ExecutorFuture<void> QueryAnalysisWriter::createTTLIndexes(OperationContext* opC
auto status = getStatusFromCommandResult(createSampledQueriesTTLIndex(opCtx));
if (!status.isOK() && status != ErrorCodes::IndexAlreadyExists) {
if (tryCount % 100 == 0) {
- LOGV2_WARNING(
- 7078404,
- "Still retrying to create sampled queries TTL index; "
- "please create an index on {namespace} with specification "
- "{specification}.",
- "namespace"_attr = NamespaceString::kConfigSampledQueriesNamespace,
- "specification"_attr = QueryAnalysisWriter::kSampledQueriesTTLIndexSpec,
- "tries"_attr = tryCount);
+ LOGV2_WARNING(7078404,
+ "Still retrying to create sampled queries TTL index; "
+ "please create an index on {namespace} with specification "
+ "{specification}.",
+ logAttrs(NamespaceString::kConfigSampledQueriesNamespace),
+ "specification"_attr =
+ QueryAnalysisWriter::kSampledQueriesTTLIndexSpec,
+ "tries"_attr = tryCount);
}
return status;
}
@@ -356,14 +356,14 @@ ExecutorFuture<void> QueryAnalysisWriter::createTTLIndexes(OperationContext* opC
status = getStatusFromCommandResult(createSampledQueriesDiffTTLIndex(opCtx));
if (!status.isOK() && status != ErrorCodes::IndexAlreadyExists) {
if (tryCount % 100 == 0) {
- LOGV2_WARNING(
- 7078405,
- "Still retrying to create sampled queries diff TTL index; "
- "please create an index on {namespace} with specification "
- "{specification}.",
- "namespace"_attr = NamespaceString::kConfigSampledQueriesDiffNamespace,
- "specification"_attr = QueryAnalysisWriter::kSampledQueriesDiffTTLIndexSpec,
- "tries"_attr = tryCount);
+ LOGV2_WARNING(7078405,
+ "Still retrying to create sampled queries diff TTL index; "
+ "please create an index on {namespace} with specification "
+ "{specification}.",
+ logAttrs(NamespaceString::kConfigSampledQueriesDiffNamespace),
+ "specification"_attr =
+ QueryAnalysisWriter::kSampledQueriesDiffTTLIndexSpec,
+ "tries"_attr = tryCount);
}
return status;
}
@@ -417,7 +417,7 @@ void QueryAnalysisWriter::_flush(OperationContext* opCtx, Buffer* buffer) {
LOGV2_DEBUG(7372300,
1,
"About to flush the sample buffer",
- "namespace"_attr = nss,
+ logAttrs(nss),
"numDocs"_attr = buffer->getCount());
std::swap(tmpBuffer, *buffer);
@@ -454,11 +454,8 @@ void QueryAnalysisWriter::_flush(OperationContext* opCtx, Buffer* buffer) {
// We don't add a document that is above the size limit to the buffer so we should have
// added at least one document to 'docsToInsert'.
invariant(!docsToInsert.empty());
- LOGV2_DEBUG(6876102,
- 2,
- "Persisting samples",
- "namespace"_attr = nss,
- "numDocs"_attr = docsToInsert.size());
+ LOGV2_DEBUG(
+ 6876102, 2, "Persisting samples", logAttrs(nss), "numDocs"_attr = docsToInsert.size());
insertDocuments(opCtx, nss, docsToInsert, [&](const BSONObj& resObj) {
BatchedCommandResponse res;
@@ -506,17 +503,14 @@ bool QueryAnalysisWriter::Buffer::add(BSONObj doc) {
LOGV2_DEBUG(7372301,
4,
"Ignoring a sample due to its size",
- "namespace"_attr = _nss,
+ logAttrs(_nss),
"size"_attr = doc.objsize(),
"doc"_attr = redact(doc));
return false;
}
- LOGV2_DEBUG(7372302,
- 4,
- "Adding a sample to the buffer",
- "namespace"_attr = _nss,
- "doc"_attr = redact(doc));
+ LOGV2_DEBUG(
+ 7372302, 4, "Adding a sample to the buffer", logAttrs(_nss), "doc"_attr = redact(doc));
_docs.push_back(std::move(doc));
_numBytes += _docs.back().objsize();
return true;
@@ -617,10 +611,8 @@ ExecutorFuture<void> QueryAnalysisWriter::_addReadQuery(
}
})
.onError([this, nss](Status status) {
- LOGV2(7047302,
- "Failed to add read query",
- "namespace"_attr = nss,
- "error"_attr = redact(status));
+ LOGV2(
+ 7047302, "Failed to add read query", logAttrs(nss), "error"_attr = redact(status));
});
}
@@ -668,7 +660,7 @@ ExecutorFuture<void> QueryAnalysisWriter::addUpdateQuery(
.onError([this, nss = updateCmd.getNamespace()](Status status) {
LOGV2(7075301,
"Failed to add update query",
- "namespace"_attr = nss,
+ logAttrs(nss),
"error"_attr = redact(status));
});
}
@@ -724,7 +716,7 @@ ExecutorFuture<void> QueryAnalysisWriter::addDeleteQuery(
.onError([this, nss = deleteCmd.getNamespace()](Status status) {
LOGV2(7075303,
"Failed to add delete query",
- "namespace"_attr = nss,
+ logAttrs(nss),
"error"_attr = redact(status));
});
}
@@ -781,7 +773,7 @@ ExecutorFuture<void> QueryAnalysisWriter::addFindAndModifyQuery(
.onError([this, nss = findAndModifyCmd.getNamespace()](Status status) {
LOGV2(7075305,
"Failed to add findAndModify query",
- "namespace"_attr = nss,
+ logAttrs(nss),
"error"_attr = redact(status));
});
}
@@ -830,10 +822,7 @@ ExecutorFuture<void> QueryAnalysisWriter::addDiff(const UUID& sampleId,
}
})
.onError([this, nss](Status status) {
- LOGV2(7075401,
- "Failed to add diff",
- "namespace"_attr = nss,
- "error"_attr = redact(status));
+ LOGV2(7075401, "Failed to add diff", logAttrs(nss), "error"_attr = redact(status));
});
}
diff --git a/src/mongo/db/s/range_deleter_service_op_observer.cpp b/src/mongo/db/s/range_deleter_service_op_observer.cpp
index 5b19700c518..3123aa306a4 100644
--- a/src/mongo/db/s/range_deleter_service_op_observer.cpp
+++ b/src/mongo/db/s/range_deleter_service_op_observer.cpp
@@ -61,7 +61,7 @@ void registerTaskWithOngoingQueriesOnOpLogEntryCommit(OperationContext* opCtx,
LOGV2_INFO(
7179200,
"Range deletion will be scheduled after all possibly dependent queries finish",
- "namespace"_attr = rdt.getNss(),
+ logAttrs(rdt.getNss()),
"range"_attr = rdt.getRange(),
"cursorsDirectlyReferringTheNamespace"_attr = openCursorsIds);
}
diff --git a/src/mongo/db/s/range_deletion_util.cpp b/src/mongo/db/s/range_deletion_util.cpp
index f6f73a32db8..48d606806e8 100644
--- a/src/mongo/db/s/range_deletion_util.cpp
+++ b/src/mongo/db/s/range_deletion_util.cpp
@@ -94,10 +94,8 @@ StatusWith<int> deleteNextBatch(OperationContext* opCtx,
const auto shardKeyIdx =
findShardKeyPrefixedIndex(opCtx, collection, keyPattern, /*requireSingleKey=*/false);
if (!shardKeyIdx) {
- LOGV2_ERROR(23765,
- "Unable to find shard key index",
- "keyPattern"_attr = keyPattern,
- "namespace"_attr = nss.ns());
+ LOGV2_ERROR(
+ 23765, "Unable to find shard key index", "keyPattern"_attr = keyPattern, logAttrs(nss));
// When a shard key index is not found, the range deleter gets stuck and indefinitely logs
// an error message. This sleep is aimed at avoiding logging too aggressively in order to
@@ -125,7 +123,7 @@ StatusWith<int> deleteNextBatch(OperationContext* opCtx,
"Begin removal of range",
"min"_attr = min,
"max"_attr = max,
- "namespace"_attr = nss.ns());
+ logAttrs(nss));
auto deleteStageParams = std::make_unique<DeleteStageParams>();
deleteStageParams->fromMigrate = true;
@@ -180,7 +178,7 @@ StatusWith<int> deleteNextBatch(OperationContext* opCtx,
"Cursor error while trying to delete range",
"min"_attr = redact(min),
"max"_attr = redact(max),
- "namespace"_attr = nss,
+ logAttrs(nss),
"stats"_attr = redact(stats),
"error"_attr = redact(ex.toStatus()));
throw;
@@ -283,7 +281,7 @@ ExecutorFuture<void> waitForDeletionsToMajorityReplicate(
LOGV2_DEBUG(5346202,
1,
"Waiting for majority replication of local deletions",
- "namespace"_attr = nss.ns(),
+ logAttrs(nss),
"collectionUUID"_attr = collectionUuid,
"range"_attr = redact(range.toString()),
"clientOpTime"_attr = clientOpTime);
@@ -354,7 +352,7 @@ Status deleteRangeInBatches(OperationContext* opCtx,
LOGV2_DEBUG(6777800,
1,
"Starting batch deletion",
- "namespace"_attr = collection.getNss(),
+ logAttrs(collection.getNss()),
"collectionUUID"_attr = collectionUuid,
"range"_attr = redact(range.toString()),
"numDocsToRemovePerBatch"_attr = numDocsToRemovePerBatch,
@@ -386,7 +384,7 @@ Status deleteRangeInBatches(OperationContext* opCtx,
1,
"Deleted documents in pass",
"numDeleted"_attr = numDeleted,
- "namespace"_attr = nss.ns(),
+ logAttrs(nss),
"collectionUUID"_attr = collectionUuid,
"range"_attr = range.toString());
@@ -486,7 +484,7 @@ SharedSemiFuture<void> removeDocumentsInRange(
LOGV2_DEBUG(23772,
1,
"Beginning deletion of documents",
- "namespace"_attr = nss.ns(),
+ logAttrs(nss),
"range"_attr = redact(range.toString()));
return deleteRangeInBatchesWithExecutor(
@@ -507,7 +505,7 @@ SharedSemiFuture<void> removeDocumentsInRange(
LOGV2_DEBUG(5346201,
1,
"Finished waiting for majority for deleted batch",
- "namespace"_attr = nss,
+ logAttrs(nss),
"range"_attr = redact(range.toString()));
// Propagate any errors to the onCompletion() handler below.
return s;
@@ -520,13 +518,13 @@ SharedSemiFuture<void> removeDocumentsInRange(
1,
"Completed deletion of documents in {namespace} range {range}",
"Completed deletion of documents",
- "namespace"_attr = nss.ns(),
+ logAttrs(nss),
"range"_attr = redact(range.toString()));
} else {
LOGV2(23774,
"Failed to delete documents in {namespace} range {range} due to {error}",
"Failed to delete documents",
- "namespace"_attr = nss.ns(),
+ logAttrs(nss),
"range"_attr = redact(range.toString()),
"error"_attr = redact(s));
}
@@ -555,7 +553,7 @@ SharedSemiFuture<void> removeDocumentsInRange(
"{namespace} due to {error}",
"Failed to delete range deletion task",
"range"_attr = range,
- "namespace"_attr = nss,
+ logAttrs(nss),
"error"_attr = e.what());
return e.toStatus();
@@ -566,7 +564,7 @@ SharedSemiFuture<void> removeDocumentsInRange(
"Completed removal of persistent range deletion task for {namespace} "
"range {range}",
"Completed removal of persistent range deletion task",
- "namespace"_attr = nss.ns(),
+ logAttrs(nss),
"range"_attr = redact(range.toString()));
// Propagate any errors to callers waiting on the result.
diff --git a/src/mongo/db/s/refine_collection_shard_key_coordinator.cpp b/src/mongo/db/s/refine_collection_shard_key_coordinator.cpp
index 378ff7427c6..7c8f41ed1b1 100644
--- a/src/mongo/db/s/refine_collection_shard_key_coordinator.cpp
+++ b/src/mongo/db/s/refine_collection_shard_key_coordinator.cpp
@@ -160,7 +160,7 @@ ExecutorFuture<void> RefineCollectionShardKeyCoordinator::_runImpl(
.onError([this, anchor = shared_from_this()](const Status& status) {
LOGV2_ERROR(5277700,
"Error running refine collection shard key",
- "namespace"_attr = nss(),
+ logAttrs(nss()),
"error"_attr = redact(status));
return status;
diff --git a/src/mongo/db/s/rename_collection_coordinator.cpp b/src/mongo/db/s/rename_collection_coordinator.cpp
index bdd03c41169..8610f11aa14 100644
--- a/src/mongo/db/s/rename_collection_coordinator.cpp
+++ b/src/mongo/db/s/rename_collection_coordinator.cpp
@@ -484,14 +484,14 @@ ExecutorFuture<void> RenameCollectionCoordinator::_runImpl(
BSON("source" << nss().toString() << "destination"
<< _request.getTo().toString()),
ShardingCatalogClient::kMajorityWriteConcern);
- LOGV2(5460504, "Collection renamed", "namespace"_attr = nss());
+ LOGV2(5460504, "Collection renamed", logAttrs(nss()));
}))
.onError([this, anchor = shared_from_this()](const Status& status) {
if (!status.isA<ErrorCategory::NotPrimaryError>() &&
!status.isA<ErrorCategory::ShutdownError>()) {
LOGV2_ERROR(5460505,
"Error running rename collection",
- "namespace"_attr = nss(),
+ logAttrs(nss()),
"error"_attr = redact(status));
}
diff --git a/src/mongo/db/s/rename_collection_participant_service.cpp b/src/mongo/db/s/rename_collection_participant_service.cpp
index 7dbb4553a6d..4a5648833ba 100644
--- a/src/mongo/db/s/rename_collection_participant_service.cpp
+++ b/src/mongo/db/s/rename_collection_participant_service.cpp
@@ -63,7 +63,7 @@ void dropCollectionLocally(OperationContext* opCtx,
LOGV2_DEBUG(5515100,
1,
"Dropped target collection locally on renameCollection participant.",
- "namespace"_attr = nss);
+ logAttrs(nss));
}
/*
@@ -111,7 +111,7 @@ void renameOrDropTarget(OperationContext* opCtx,
LOGV2_DEBUG(5515101,
1,
"Source namespace not found while trying to rename collection on participant",
- "namespace"_attr = fromNss);
+ logAttrs(fromNss));
dropCollectionLocally(opCtx, toNss, options.markFromMigrate);
deleteRangeDeletionTasksForRename(opCtx, fromNss, toNss);
}
diff --git a/src/mongo/db/s/reshard_collection_coordinator.cpp b/src/mongo/db/s/reshard_collection_coordinator.cpp
index b496231f3ea..8f59b884955 100644
--- a/src/mongo/db/s/reshard_collection_coordinator.cpp
+++ b/src/mongo/db/s/reshard_collection_coordinator.cpp
@@ -195,7 +195,7 @@ ExecutorFuture<void> ReshardCollectionCoordinator::_runImpl(
.onError([this, anchor = shared_from_this()](const Status& status) {
LOGV2_ERROR(6206401,
"Error running reshard collection",
- "namespace"_attr = nss(),
+ logAttrs(nss()),
"error"_attr = redact(status));
return status;
});
diff --git a/src/mongo/db/s/resharding/resharding_coordinator_commit_monitor.cpp b/src/mongo/db/s/resharding/resharding_coordinator_commit_monitor.cpp
index 963ec6a7c54..6cc7f93abad 100644
--- a/src/mongo/db/s/resharding/resharding_coordinator_commit_monitor.cpp
+++ b/src/mongo/db/s/resharding/resharding_coordinator_commit_monitor.cpp
@@ -136,7 +136,7 @@ CoordinatorCommitMonitor::queryRemainingOperationTimeForRecipients() const {
LOGV2_DEBUG(5392001,
kDiagnosticLogLevel,
"Querying recipient shards for the remaining operation time",
- "namespace"_attr = _ns);
+ logAttrs(_ns));
auto opCtx = CancelableOperationContext(cc().makeOperationContext(), _cancelToken, _executor);
auto executor = _networkExecutor ? _networkExecutor : _executor;
@@ -192,7 +192,7 @@ CoordinatorCommitMonitor::queryRemainingOperationTimeForRecipients() const {
LOGV2_DEBUG(5392002,
kDiagnosticLogLevel,
"Finished querying recipient shards for the remaining operation time",
- "namespace"_attr = _ns,
+ logAttrs(_ns),
"remainingTime"_attr = maxRemainingTime);
return {minRemainingTime, maxRemainingTime};
diff --git a/src/mongo/db/s/resharding/resharding_coordinator_service.cpp b/src/mongo/db/s/resharding/resharding_coordinator_service.cpp
index 8268672f9da..1960a1236bd 100644
--- a/src/mongo/db/s/resharding/resharding_coordinator_service.cpp
+++ b/src/mongo/db/s/resharding/resharding_coordinator_service.cpp
@@ -1258,7 +1258,7 @@ void ReshardingCoordinator::installCoordinatorDoc(
"Transitioned resharding coordinator state",
"newState"_attr = CoordinatorState_serializer(doc.getState()),
"oldState"_attr = CoordinatorState_serializer(_coordinatorDoc.getState()),
- "namespace"_attr = doc.getSourceNss(),
+ logAttrs(doc.getSourceNss()),
"collectionUUID"_attr = doc.getSourceUUID(),
"reshardingUUID"_attr = doc.getReshardingUUID());
@@ -1393,7 +1393,7 @@ ExecutorFuture<void> ReshardingCoordinator::_initializeCoordinator(
auto nss = _coordinatorDoc.getSourceNss();
LOGV2(4956903,
"Resharding failed",
- "namespace"_attr = nss.ns(),
+ logAttrs(nss),
"newShardKeyPattern"_attr = _coordinatorDoc.getReshardingKey(),
"error"_attr = status);
@@ -1466,7 +1466,7 @@ ExecutorFuture<ReshardingCoordinatorDocument> ReshardingCoordinator::_runUntilRe
auto nss = _coordinatorDoc.getSourceNss();
LOGV2(4956902,
"Resharding failed",
- "namespace"_attr = nss.ns(),
+ logAttrs(nss),
"newShardKeyPattern"_attr = _coordinatorDoc.getReshardingKey(),
"error"_attr = status);
@@ -2329,7 +2329,7 @@ void ReshardingCoordinator::_updateChunkImbalanceMetrics(const NamespaceString&
} catch (const DBException& ex) {
LOGV2_WARNING(5543000,
"Encountered error while trying to update resharding chunk imbalance metrics",
- "namespace"_attr = nss,
+ logAttrs(nss),
"error"_attr = redact(ex.toStatus()));
}
}
diff --git a/src/mongo/db/s/resharding/resharding_donor_service.cpp b/src/mongo/db/s/resharding/resharding_donor_service.cpp
index a4ea5af221f..aafe846b7cb 100644
--- a/src/mongo/db/s/resharding/resharding_donor_service.cpp
+++ b/src/mongo/db/s/resharding/resharding_donor_service.cpp
@@ -277,7 +277,7 @@ ExecutorFuture<void> ReshardingDonorService::DonorStateMachine::_runUntilBlockin
LOGV2(4956400,
"Resharding operation donor state machine failed",
- "namespace"_attr = _metadata.getSourceNss(),
+ logAttrs(_metadata.getSourceNss()),
"reshardingUUID"_attr = _metadata.getReshardingUUID(),
"error"_attr = status);
@@ -642,7 +642,7 @@ void ReshardingDonorService::DonorStateMachine::
LOGV2_DEBUG(5390702,
2,
"Collection being resharded now ready for recipients to begin cloning",
- "namespace"_attr = _metadata.getSourceNss(),
+ logAttrs(_metadata.getSourceNss()),
"minFetchTimestamp"_attr = minFetchTimestamp,
"bytesToClone"_attr = bytesToClone,
"documentsToClone"_attr = documentsToClone,
@@ -763,7 +763,7 @@ void ReshardingDonorService::DonorStateMachine::
LOGV2_DEBUG(5279504,
0,
"Committed oplog entries to temporarily block writes for resharding",
- "namespace"_attr = _metadata.getSourceNss(),
+ logAttrs(_metadata.getSourceNss()),
"reshardingUUID"_attr = _metadata.getReshardingUUID(),
"numRecipients"_attr = _recipientShardIds.size(),
"duration"_attr = duration_cast<Milliseconds>(latency.elapsed()));
@@ -857,7 +857,7 @@ void ReshardingDonorService::DonorStateMachine::_transitionState(DonorShardConte
"Transitioned resharding donor state",
"newState"_attr = DonorState_serializer(newState),
"oldState"_attr = DonorState_serializer(oldState),
- "namespace"_attr = _metadata.getSourceNss(),
+ logAttrs(_metadata.getSourceNss()),
"collectionUUID"_attr = _metadata.getSourceUUID(),
"reshardingUUID"_attr = _metadata.getReshardingUUID());
}
diff --git a/src/mongo/db/s/resharding/resharding_manual_cleanup.cpp b/src/mongo/db/s/resharding/resharding_manual_cleanup.cpp
index 9bd3bcf65a6..1fe78a5b943 100644
--- a/src/mongo/db/s/resharding/resharding_manual_cleanup.cpp
+++ b/src/mongo/db/s/resharding/resharding_manual_cleanup.cpp
@@ -103,7 +103,7 @@ void ReshardingCleaner<Service, StateMachine, ReshardingDocument>::clean(Operati
LOGV2(5403503,
"Cleaning up resharding operation",
- "namespace"_attr = _originalCollectionNss,
+ logAttrs(_originalCollectionNss),
"reshardingUUID"_attr = _reshardingUUID,
"serviceType"_attr = Service::kServiceName);
diff --git a/src/mongo/db/s/resharding/resharding_op_observer.cpp b/src/mongo/db/s/resharding/resharding_op_observer.cpp
index e052f14c473..63ef981a2b2 100644
--- a/src/mongo/db/s/resharding/resharding_op_observer.cpp
+++ b/src/mongo/db/s/resharding/resharding_op_observer.cpp
@@ -237,7 +237,7 @@ void ReshardingOpObserver::onUpdate(OperationContext* opCtx, const OplogUpdateEn
LOGV2_INFO(6148200,
"Interrupted while waiting for resharding coordinator to be rebuilt;"
" will retry on new primary",
- "namespace"_attr = newCoordinatorDoc.getSourceNss(),
+ logAttrs(newCoordinatorDoc.getSourceNss()),
"reshardingUUID"_attr = newCoordinatorDoc.getReshardingUUID(),
"error"_attr = redact(ex.toStatus()));
}
diff --git a/src/mongo/db/s/resharding/resharding_recipient_service.cpp b/src/mongo/db/s/resharding/resharding_recipient_service.cpp
index b15d8fd0e88..1a75eb2831f 100644
--- a/src/mongo/db/s/resharding/resharding_recipient_service.cpp
+++ b/src/mongo/db/s/resharding/resharding_recipient_service.cpp
@@ -280,7 +280,7 @@ ReshardingRecipientService::RecipientStateMachine::_runUntilStrictConsistencyOrE
LOGV2(4956500,
"Resharding operation recipient state machine failed",
- "namespace"_attr = _metadata.getSourceNss(),
+ logAttrs(_metadata.getSourceNss()),
"reshardingUUID"_attr = _metadata.getReshardingUUID(),
"error"_attr = redact(status));
@@ -891,7 +891,7 @@ void ReshardingRecipientService::RecipientStateMachine::_transitionState(
"Transitioned resharding recipient state",
"newState"_attr = RecipientState_serializer(newState),
"oldState"_attr = RecipientState_serializer(oldState),
- "namespace"_attr = _metadata.getSourceNss(),
+ logAttrs(_metadata.getSourceNss()),
"collectionUUID"_attr = _metadata.getSourceUUID(),
"reshardingUUID"_attr = _metadata.getReshardingUUID());
}
diff --git a/src/mongo/db/s/session_catalog_migration_destination.cpp b/src/mongo/db/s/session_catalog_migration_destination.cpp
index dec574174e2..5ea8c1f5413 100644
--- a/src/mongo/db/s/session_catalog_migration_destination.cpp
+++ b/src/mongo/db/s/session_catalog_migration_destination.cpp
@@ -305,7 +305,7 @@ void SessionCatalogMigrationDestination::_retrieveSessionStateFromSource(Service
"Recipient finished draining oplog entries for retryable writes "
"and transactions from donor again after receiving "
"_recvChunkCommit",
- "namespace"_attr = _nss,
+ logAttrs(_nss),
"migrationSessionId"_attr = _migrationSessionId,
"fromShard"_attr = _fromShard);
break;
@@ -328,7 +328,7 @@ void SessionCatalogMigrationDestination::_retrieveSessionStateFromSource(Service
"Recipient finished draining oplog entries for retryable writes and "
"transactions from donor for the first time, before receiving "
"_recvChunkCommit",
- "namespace"_attr = _nss,
+ logAttrs(_nss),
"migrationSessionId"_attr = _migrationSessionId,
"fromShard"_attr = _fromShard);
_state = State::ReadyToCommit;
@@ -550,7 +550,7 @@ std::string SessionCatalogMigrationDestination::getErrMsg() {
void SessionCatalogMigrationDestination::_errorOccurred(StringData errMsg) {
LOGV2(5087102,
"Recipient failed to copy oplog entries for retryable writes and transactions from donor",
- "namespace"_attr = _nss,
+ logAttrs(_nss),
"migrationSessionId"_attr = _migrationSessionId,
"fromShard"_attr = _fromShard,
"error"_attr = errMsg);
diff --git a/src/mongo/db/s/set_allow_migrations_coordinator.cpp b/src/mongo/db/s/set_allow_migrations_coordinator.cpp
index 256cbcd6adc..5aa755edf10 100644
--- a/src/mongo/db/s/set_allow_migrations_coordinator.cpp
+++ b/src/mongo/db/s/set_allow_migrations_coordinator.cpp
@@ -120,7 +120,7 @@ ExecutorFuture<void> SetAllowMigrationsCoordinator::_runImpl(
.onError([this, anchor = shared_from_this()](const Status& status) {
LOGV2_ERROR(5622700,
"Error running set allow migrations",
- "namespace"_attr = nss(),
+ logAttrs(nss()),
"error"_attr = redact(status));
return status;
});
diff --git a/src/mongo/db/s/shard_filtering_metadata_refresh.cpp b/src/mongo/db/s/shard_filtering_metadata_refresh.cpp
index c9a7b682678..8b8455b4483 100644
--- a/src/mongo/db/s/shard_filtering_metadata_refresh.cpp
+++ b/src/mongo/db/s/shard_filtering_metadata_refresh.cpp
@@ -486,7 +486,7 @@ void onCollectionPlacementVersionMismatch(OperationContext* opCtx,
"Metadata refresh requested for {namespace} at chunk version "
"{chunkVersionReceived}",
"Metadata refresh requested for collection",
- "namespace"_attr = nss,
+ logAttrs(nss),
"chunkVersionReceived"_attr = chunkVersionReceived);
while (true) {
@@ -563,7 +563,7 @@ Status onCollectionPlacementVersionMismatchNoExcept(
LOGV2(22062,
"Failed to refresh metadata for {namespace} due to {error}",
"Failed to refresh metadata for collection",
- "namespace"_attr = nss,
+ logAttrs(nss),
"error"_attr = redact(ex));
return ex.toStatus();
}
@@ -594,7 +594,7 @@ CollectionMetadata forceGetCurrentMetadata(OperationContext* opCtx, const Namesp
LOGV2(505070,
"Namespace {namespace} not found, collection may have been dropped",
"Namespace not found, collection may have been dropped",
- "namespace"_attr = nss,
+ logAttrs(nss),
"error"_attr = redact(ex));
return CollectionMetadata();
}
@@ -645,7 +645,7 @@ ChunkVersion forceShardFilteringMetadataRefresh(OperationContext* opCtx,
LOGV2_DEBUG(22063,
1,
"Skipping metadata refresh because collection already is up-to-date",
- "namespace"_attr = nss,
+ logAttrs(nss),
"latestCollectionPlacementVersion"_attr =
metadata.getCollPlacementVersion(),
"refreshedCollectionPlacementVersion"_attr = cm.getVersion());
@@ -670,7 +670,7 @@ ChunkVersion forceShardFilteringMetadataRefresh(OperationContext* opCtx,
LOGV2_DEBUG(22064,
1,
"Skipping metadata refresh because collection already is up-to-date",
- "namespace"_attr = nss,
+ logAttrs(nss),
"latestCollectionPlacementVersion"_attr =
metadata.getCollPlacementVersion(),
"refreshedCollectionPlacementVersion"_attr = cm.getVersion());
diff --git a/src/mongo/db/s/shard_metadata_util.cpp b/src/mongo/db/s/shard_metadata_util.cpp
index 59dbbca4d91..3a6f9f14e78 100644
--- a/src/mongo/db/s/shard_metadata_util.cpp
+++ b/src/mongo/db/s/shard_metadata_util.cpp
@@ -416,13 +416,13 @@ Status dropChunksAndDeleteCollectionsEntry(OperationContext* opCtx, const Namesp
} catch (const DBException& ex) {
LOGV2_ERROR(5966301,
"Failed to drop persisted chunk metadata and collection entry",
- "namespace"_attr = nss,
+ logAttrs(nss),
"error"_attr = redact(ex.toStatus()));
return ex.toStatus();
}
- LOGV2(5966302, "Dropped persisted chunk metadata and collection entry", "namespace"_attr = nss);
+ LOGV2(5966302, "Dropped persisted chunk metadata and collection entry", logAttrs(nss));
return Status::OK();
}
@@ -442,7 +442,7 @@ void dropChunks(OperationContext* opCtx, const NamespaceString& nss) {
}
}
- LOGV2_DEBUG(22091, 1, "Dropped persisted chunk metadata", "namespace"_attr = nss);
+ LOGV2_DEBUG(22091, 1, "Dropped persisted chunk metadata", logAttrs(nss));
}
Status deleteDatabasesEntry(OperationContext* opCtx, StringData dbName) {
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 ea7e634703b..2f3f83161e6 100644
--- a/src/mongo/db/s/shard_server_catalog_cache_loader.cpp
+++ b/src/mongo/db/s/shard_server_catalog_cache_loader.cpp
@@ -80,7 +80,7 @@ void dropChunksIfEpochChanged(OperationContext* opCtx,
LOGV2(5990400,
"Dropped persisted chunk metadata due to epoch change",
- "namespace"_attr = nss,
+ logAttrs(nss),
"currentEpoch"_attr = currentEpoch,
"previousEpoch"_attr = maxLoaderVersion.epoch());
}
@@ -135,7 +135,7 @@ Status persistCollectionAndChangedChunks(OperationContext* opCtx,
return status;
}
- LOGV2(3463204, "Persisted collection entry and chunk metadata", "namespace"_attr = nss);
+ LOGV2(3463204, "Persisted collection entry and chunk metadata", logAttrs(nss));
return Status::OK();
}
@@ -659,7 +659,7 @@ StatusWith<CollectionAndChangedChunks> ShardServerCatalogCacheLoader::_runSecond
2,
"Cache loader on secondary successfully waited for primary refresh "
"and replication of collection",
- "namespace"_attr = nss,
+ logAttrs(nss),
"duration"_attr = Milliseconds(t.millis()));
// Read the local metadata.
@@ -722,7 +722,7 @@ ShardServerCatalogCacheLoader::_schedulePrimaryGetChunksSince(
"Cache loader remotely refreshed for collection {namespace} from version "
"{oldCollectionPlacementVersion} and no metadata was found",
"Cache loader remotely refreshed for collection and no metadata was found",
- "namespace"_attr = nss,
+ logAttrs(nss),
"oldCollectionPlacementVersion"_attr = maxLoaderVersion);
return swCollectionAndChangedChunks;
}
@@ -760,7 +760,7 @@ ShardServerCatalogCacheLoader::_schedulePrimaryGetChunksSince(
"version {oldCollectionPlacementVersion} and found collection placement version "
"{refreshedCollectionPlacementVersion}",
"Cache loader remotely refreshed for collection",
- "namespace"_attr = nss,
+ logAttrs(nss),
"oldCollectionPlacementVersion"_attr = maxLoaderVersion,
"refreshedCollectionPlacementVersion"_attr =
collAndChunks.changedChunks.back().getVersion());
@@ -1064,13 +1064,13 @@ void ShardServerCatalogCacheLoader::_runCollAndChunksTasks(const NamespaceString
LOGV2(22094,
"Failed to persist chunk metadata update for collection {namespace} due to shutdown",
"Failed to persist chunk metadata update for collection due to shutdown",
- "namespace"_attr = nss);
+ logAttrs(nss));
inShutdown = true;
} catch (const DBException& ex) {
LOGV2(22095,
"Failed to persist chunk metadata update for collection {namespace} {error}",
"Failed to persist chunk metadata update for collection",
- "namespace"_attr = nss,
+ logAttrs(nss),
"error"_attr = redact(ex));
}
@@ -1112,7 +1112,7 @@ void ShardServerCatalogCacheLoader::_runCollAndChunksTasks(const NamespaceString
"Cache loader failed to schedule a persisted metadata update task. Clearing task "
"list so that scheduling will be attempted by the next caller to refresh this "
"namespace",
- "namespace"_attr = nss,
+ logAttrs(nss),
"error"_attr = redact(status));
{
@@ -1244,7 +1244,7 @@ void ShardServerCatalogCacheLoader::_updatePersistedCollAndChunksMetadata(
"{oldCollectionPlacementVersion} to collection placement version "
"{newCollectionPlacementVersion}",
"Successfully updated persisted chunk metadata for collection",
- "namespace"_attr = nss,
+ logAttrs(nss),
"oldCollectionPlacementVersion"_attr = task.minQueryVersion,
"newCollectionPlacementVersion"_attr = task.maxQueryVersion);
}
diff --git a/src/mongo/db/s/sharding_ddl_coordinator.cpp b/src/mongo/db/s/sharding_ddl_coordinator.cpp
index 52d42e37568..0606cba0993 100644
--- a/src/mongo/db/s/sharding_ddl_coordinator.cpp
+++ b/src/mongo/db/s/sharding_ddl_coordinator.cpp
@@ -170,7 +170,7 @@ ExecutorFuture<void> ShardingDDLCoordinator::_translateTimeseriesNss(
if (!status.isOK()) {
LOGV2_WARNING(6675600,
"Failed to fetch information for the bucket namespace",
- "namespace"_attr = originalNss().makeTimeseriesBucketsNamespace(),
+ logAttrs(originalNss().makeTimeseriesBucketsNamespace()),
"coordinatorId"_attr = _coordId,
"error"_attr = redact(status));
}
diff --git a/src/mongo/db/s/sharding_index_catalog_ddl_util.cpp b/src/mongo/db/s/sharding_index_catalog_ddl_util.cpp
index 99102399ecb..3c137b52488 100644
--- a/src/mongo/db/s/sharding_index_catalog_ddl_util.cpp
+++ b/src/mongo/db/s/sharding_index_catalog_ddl_util.cpp
@@ -97,8 +97,8 @@ void renameCollectionShardingIndexCatalog(OperationContext* opCtx,
"than current collection index version",
"collectionIndexVersion"_attr = toIndexVersion,
"expectedIndexVersion"_attr = indexVersion,
- "fromNss"_attr = redact(fromNss.toString()),
- "toNss"_attr = redact(toNss.toString()));
+ "fromNss"_attr = redact(toStringForLogging(fromNss)),
+ "toNss"_attr = redact(toStringForLogging(toNss)));
return;
}
toUuid.emplace(collectionTo.getUuid());
@@ -197,7 +197,7 @@ void addShardingIndexCatalogEntryToCollection(OperationContext* opCtx,
"older than current collection index version",
"collectionIndexVersion"_attr = *collection.getIndexVersion(),
"expectedIndexVersion"_attr = lastmod,
- "nss"_attr = redact(userCollectionNss.toString()));
+ "nss"_attr = redact(toStringForLogging(userCollectionNss)));
return;
}
}
@@ -274,7 +274,7 @@ void removeShardingIndexCatalogEntryFromCollection(OperationContext* opCtx,
"version older than current collection index version",
"collectionIndexVersion"_attr = *collection.getIndexVersion(),
"expectedIndexVersion"_attr = lastmod,
- "nss"_attr = redact(nss.toString()));
+ "nss"_attr = redact(toStringForLogging(nss)));
return;
}
}
@@ -348,7 +348,7 @@ void replaceCollectionShardingIndexCatalog(OperationContext* opCtx,
"current collection index version",
"collectionIndexVersion"_attr = *collection.getIndexVersion(),
"expectedIndexVersion"_attr = indexVersion,
- "nss"_attr = redact(nss.toString()));
+ "nss"_attr = redact(toStringForLogging(nss)));
return;
}
}
@@ -425,7 +425,7 @@ void dropCollectionShardingIndexCatalog(OperationContext* opCtx, const Namespace
1,
"dropCollectionGlobalIndexesMetadata did not found collection, "
"skipping dropping index metadata",
- "nss"_attr = redact(nss.toString()));
+ "nss"_attr = redact(toStringForLogging(nss)));
return;
}
auto collection = ShardAuthoritativeCollectionType::parse(
diff --git a/src/mongo/db/s/sharding_initialization_mongod.cpp b/src/mongo/db/s/sharding_initialization_mongod.cpp
index f9d2235c119..7d01c558fef 100644
--- a/src/mongo/db/s/sharding_initialization_mongod.cpp
+++ b/src/mongo/db/s/sharding_initialization_mongod.cpp
@@ -375,13 +375,13 @@ bool ShardingInitializationMongoD::initializeShardingAwarenessIfNeeded(Operation
LOGV2_WARNING(7445900,
"Started with ShardServer role, but no shardIdentity document was "
"found on disk.",
- "namespace"_attr = NamespaceString::kServerConfigurationNamespace);
+ logAttrs(NamespaceString::kServerConfigurationNamespace));
} else {
LOGV2_WARNING(22074,
"Started with ShardServer role, but no shardIdentity document was "
"found on disk. This most likely means this server has not yet been "
"added to a sharded cluster.",
- "namespace"_attr = NamespaceString::kServerConfigurationNamespace);
+ logAttrs(NamespaceString::kServerConfigurationNamespace));
}
return false;
}
@@ -406,7 +406,7 @@ bool ShardingInitializationMongoD::initializeShardingAwarenessIfNeeded(Operation
"Not started with --shardsvr, but a shardIdentity document was found "
"on disk in {namespace}: {shardIdentityDocument}",
"Not started with --shardsvr, but a shardIdentity document was found on disk",
- "namespace"_attr = NamespaceString::kServerConfigurationNamespace,
+ logAttrs(NamespaceString::kServerConfigurationNamespace),
"shardIdentityDocument"_attr = shardIdentityBSON);
}
return false;
diff --git a/src/mongo/db/s/sharding_recovery_service.cpp b/src/mongo/db/s/sharding_recovery_service.cpp
index 828a092a048..0187558302f 100644
--- a/src/mongo/db/s/sharding_recovery_service.cpp
+++ b/src/mongo/db/s/sharding_recovery_service.cpp
@@ -149,7 +149,7 @@ void ShardingRecoveryService::acquireRecoverableCriticalSectionBlockWrites(
LOGV2_DEBUG(5656600,
3,
"Acquiring recoverable critical section blocking writes",
- "namespace"_attr = nss,
+ logAttrs(nss),
"reason"_attr = reason,
"writeConcern"_attr = writeConcern);
@@ -203,7 +203,7 @@ void ShardingRecoveryService::acquireRecoverableCriticalSectionBlockWrites(
3,
"The recoverable critical section was already acquired to block "
"writes, do nothing",
- "namespace"_attr = nss,
+ logAttrs(nss),
"reason"_attr = reason,
"writeConcern"_attr = writeConcern);
@@ -248,7 +248,7 @@ void ShardingRecoveryService::acquireRecoverableCriticalSectionBlockWrites(
LOGV2_DEBUG(5656602,
2,
"Acquired recoverable critical section blocking writes",
- "namespace"_attr = nss,
+ logAttrs(nss),
"reason"_attr = reason,
"writeConcern"_attr = writeConcern);
}
@@ -262,7 +262,7 @@ void ShardingRecoveryService::promoteRecoverableCriticalSectionToBlockAlsoReads(
LOGV2_DEBUG(5656603,
3,
"Promoting recoverable critical section to also block reads",
- "namespace"_attr = nss,
+ logAttrs(nss),
"reason"_attr = reason,
"writeConcern"_attr = writeConcern);
@@ -323,7 +323,7 @@ void ShardingRecoveryService::promoteRecoverableCriticalSectionToBlockAlsoReads(
3,
"The recoverable critical section was already promoted to also block "
"reads, do nothing",
- "namespace"_attr = nss,
+ logAttrs(nss),
"reason"_attr = reason,
"writeConcern"_attr = writeConcern);
return;
@@ -375,7 +375,7 @@ void ShardingRecoveryService::promoteRecoverableCriticalSectionToBlockAlsoReads(
LOGV2_DEBUG(5656605,
2,
"Promoted recoverable critical section to also block reads",
- "namespace"_attr = nss,
+ logAttrs(nss),
"reason"_attr = reason,
"writeConcern"_attr = writeConcern);
}
@@ -390,7 +390,7 @@ void ShardingRecoveryService::releaseRecoverableCriticalSection(
LOGV2_DEBUG(5656606,
3,
"Releasing recoverable critical section",
- "namespace"_attr = nss,
+ logAttrs(nss),
"reason"_attr = reason,
"writeConcern"_attr = writeConcern);
@@ -430,7 +430,7 @@ void ShardingRecoveryService::releaseRecoverableCriticalSection(
LOGV2_DEBUG(5656607,
3,
"The recoverable critical section was already released, do nothing",
- "namespace"_attr = nss,
+ logAttrs(nss),
"reason"_attr = reason,
"writeConcern"_attr = writeConcern);
return;
@@ -446,7 +446,7 @@ void ShardingRecoveryService::releaseRecoverableCriticalSection(
2,
"Impossible to release recoverable critical section since it was taken by "
"another operation with different reason",
- "namespace"_attr = nss,
+ logAttrs(nss),
"callerReason"_attr = reason,
"storedReason"_attr = collCSDoc.getReason(),
"writeConcern"_attr = writeConcern);
@@ -505,7 +505,7 @@ void ShardingRecoveryService::releaseRecoverableCriticalSection(
LOGV2_DEBUG(5656608,
2,
"Released recoverable critical section",
- "namespace"_attr = nss,
+ logAttrs(nss),
"reason"_attr = reason,
"writeConcern"_attr = writeConcern);
}
@@ -602,7 +602,7 @@ void ShardingRecoveryService::recoverIndexesCatalog(OperationContext* opCtx) {
2,
"Skipping attempting to clear indexes for a view in "
"recoverIndexCatalogs",
- "namespace"_attr = collName);
+ logAttrs(collName));
}
}
DBDirectClient client(opCtx);
diff --git a/src/mongo/db/s/split_vector.cpp b/src/mongo/db/s/split_vector.cpp
index a8e55d2ac11..2f5c22e16a9 100644
--- a/src/mongo/db/s/split_vector.cpp
+++ b/src/mongo/db/s/split_vector.cpp
@@ -133,7 +133,7 @@ std::vector<BSONObj> splitVector(OperationContext* opCtx,
LOGV2(22107,
"Requested split points lookup for chunk {namespace} {minKey} -->> {maxKey}",
"Requested split points lookup for chunk",
- "namespace"_attr = nss.toString(),
+ logAttrs(nss),
"minKey"_attr = redact(prettyKey(keyPattern, minKey)),
"maxKey"_attr = redact(prettyKey(keyPattern, maxKey)));
@@ -208,7 +208,7 @@ std::vector<BSONObj> splitVector(OperationContext* opCtx,
"Possible low cardinality key detected in {namespace} - range {minKey} -->> "
"{maxKey} contains only the key {key}",
"Possible low cardinality key detected in range. Range contains only a single key.",
- "namespace"_attr = nss.toString(),
+ logAttrs(nss),
"minKey"_attr = redact(prettyKey(shardKeyIdx->keyPattern(), minKey)),
"maxKey"_attr = redact(prettyKey(shardKeyIdx->keyPattern(), maxKey)),
"key"_attr = redact(prettyKey(shardKeyIdx->keyPattern(), currKey)));
@@ -252,7 +252,7 @@ std::vector<BSONObj> splitVector(OperationContext* opCtx,
"of chunk {namespace} {minKey} -->> {maxKey}",
"Max BSON response size reached for split vector before the end "
"of chunk",
- "namespace"_attr = nss.toString(),
+ logAttrs(nss),
"minKey"_attr =
redact(prettyKey(shardKeyIdx->keyPattern(), minKey)),
"maxKey"_attr =
@@ -280,7 +280,7 @@ std::vector<BSONObj> splitVector(OperationContext* opCtx,
"the end of chunk {namespace} {minKey} -->> {maxKey}",
"Max number of requested split points reached before the end of chunk",
"numSplitPoints"_attr = numChunks,
- "namespace"_attr = nss.toString(),
+ logAttrs(nss),
"minKey"_attr = redact(prettyKey(shardKeyIdx->keyPattern(), minKey)),
"maxKey"_attr = redact(prettyKey(shardKeyIdx->keyPattern(), maxKey)));
break;
@@ -328,7 +328,7 @@ std::vector<BSONObj> splitVector(OperationContext* opCtx,
"Possible low cardinality key detected in {namespace} - key is "
"{key}",
"Possible low cardinality key detected",
- "namespace"_attr = nss.toString(),
+ logAttrs(nss),
"key"_attr = redact(prettyKey(shardKeyIdx->keyPattern(), *it)));
}
@@ -341,7 +341,7 @@ std::vector<BSONObj> splitVector(OperationContext* opCtx,
"Finding the split vector for {namespace} over {keyPattern} keyCount: {keyCount} "
"numSplits: {numSplits} lookedAt: {currCount} took {duration}",
"Finding the split vector completed",
- "namespace"_attr = nss.toString(),
+ logAttrs(nss),
"keyPattern"_attr = redact(keyPattern),
"keyCount"_attr = keyCount,
"numSplits"_attr = splitKeys.size(),
diff --git a/src/mongo/db/s/user_writes_recoverable_critical_section_service.cpp b/src/mongo/db/s/user_writes_recoverable_critical_section_service.cpp
index 798671a3f3d..66b31f3dbfb 100644
--- a/src/mongo/db/s/user_writes_recoverable_critical_section_service.cpp
+++ b/src/mongo/db/s/user_writes_recoverable_critical_section_service.cpp
@@ -96,7 +96,7 @@ void acquireRecoverableCriticalSection(OperationContext* opCtx,
LOGV2_DEBUG(6351900,
3,
"Acquiring user writes recoverable critical section",
- "namespace"_attr = nss,
+ logAttrs(nss),
"blockShardedDDL"_attr = blockShardedDDL,
"blockUserWrites"_attr = blockUserWrites);
@@ -130,7 +130,7 @@ void acquireRecoverableCriticalSection(OperationContext* opCtx,
LOGV2_DEBUG(6351914,
3,
"The user writes recoverable critical section was already acquired",
- "namespace"_attr = nss);
+ logAttrs(nss));
return;
}
@@ -148,7 +148,7 @@ void acquireRecoverableCriticalSection(OperationContext* opCtx,
LOGV2_DEBUG(6351901,
2,
"Acquired user writes recoverable critical section",
- "namespace"_attr = nss,
+ logAttrs(nss),
"blockShardedDDL"_attr = blockShardedDDL,
"blockUserWrites"_attr = blockUserWrites);
}
@@ -209,7 +209,7 @@ void UserWritesRecoverableCriticalSectionService::
LOGV2_DEBUG(6351902,
3,
"Promoting user writes recoverable critical section to also block reads",
- "namespace"_attr = nss);
+ logAttrs(nss));
invariant(nss == UserWritesRecoverableCriticalSectionService::kGlobalUserWritesNamespace);
invariant(!opCtx->lockState()->isLocked());
@@ -240,7 +240,7 @@ void UserWritesRecoverableCriticalSectionService::
"The user writes recoverable critical section was already promoted to also "
"block user "
"writes, do nothing",
- "namespace"_attr = nss);
+ logAttrs(nss));
return;
}
@@ -252,7 +252,7 @@ void UserWritesRecoverableCriticalSectionService::
LOGV2_DEBUG(6351904,
2,
"Promoted user writes recoverable critical section to also block user writes",
- "namespace"_attr = nss);
+ logAttrs(nss));
}
void UserWritesRecoverableCriticalSectionService::
@@ -265,7 +265,7 @@ void UserWritesRecoverableCriticalSectionService::
LOGV2_DEBUG(6351905,
3,
"Demoting user writes recoverable critical section to no longer block user writes",
- "namespace"_attr = nss);
+ logAttrs(nss));
invariant(nss == UserWritesRecoverableCriticalSectionService::kGlobalUserWritesNamespace);
invariant(!opCtx->lockState()->isLocked());
@@ -280,7 +280,7 @@ void UserWritesRecoverableCriticalSectionService::
6351906,
3,
"The user writes recoverable critical section was not currently taken, do nothing",
- "namespace"_attr = nss);
+ logAttrs(nss));
return;
}
@@ -293,7 +293,7 @@ void UserWritesRecoverableCriticalSectionService::
3,
"The user writes recoverable critical section was already not blocking "
"user writes, do nothing",
- "namespace"_attr = nss);
+ logAttrs(nss));
return;
}
@@ -305,14 +305,13 @@ void UserWritesRecoverableCriticalSectionService::
LOGV2_DEBUG(6351908,
2,
"Demoted user writes recoverable critical section to no longer block user writes",
- "namespace"_attr = nss);
+ logAttrs(nss));
}
void UserWritesRecoverableCriticalSectionService::releaseRecoverableCriticalSection(
OperationContext* opCtx, const NamespaceString& nss) {
- LOGV2_DEBUG(
- 6351909, 3, "Releasing user writes recoverable critical section", "namespace"_attr = nss);
+ LOGV2_DEBUG(6351909, 3, "Releasing user writes recoverable critical section", logAttrs(nss));
invariant(nss == UserWritesRecoverableCriticalSectionService::kGlobalUserWritesNamespace);
invariant(!opCtx->lockState()->isLocked());
@@ -328,7 +327,7 @@ void UserWritesRecoverableCriticalSectionService::releaseRecoverableCriticalSect
6351910,
3,
"The user writes recoverable critical section was already released, do nothing",
- "namespace"_attr = nss);
+ logAttrs(nss));
return;
}
@@ -358,8 +357,7 @@ void UserWritesRecoverableCriticalSectionService::releaseRecoverableCriticalSect
uassertStatusOK(getStatusFromWriteCommandReply(commandReply));
}
- LOGV2_DEBUG(
- 6351911, 2, "Released user writes recoverable critical section", "namespace"_attr = nss);
+ LOGV2_DEBUG(6351911, 2, "Released user writes recoverable critical section", logAttrs(nss));
}
void UserWritesRecoverableCriticalSectionService::recoverRecoverableCriticalSections(