summaryrefslogtreecommitdiff
diff options
context:
space:
mode:
authorRandolph Tan <randolph@10gen.com>2020-03-20 13:34:08 -0400
committerEvergreen Agent <no-reply@evergreen.mongodb.com>2020-04-01 20:46:42 +0000
commitc5794ea2233a08cae94665759f1ecbb7af10d099 (patch)
treededa923cf0a6d7262b030eb78e5f98c389202409
parentf088b63424736855b103bb0e4f6618837df0e4f9 (diff)
downloadmongo-c5794ea2233a08cae94665759f1ecbb7af10d099.tar.gz
SERVER-46799 Update sharding log lines to adhere to LOGV2 style guide
(cherry picked from commit 33f4d4229df474c3537d113048dfad511b34e32d)
-rw-r--r--src/mongo/db/s/balancer/balancer_chunk_selection_policy_impl.cpp27
-rw-r--r--src/mongo/db/s/balancer/balancer_policy.cpp94
-rw-r--r--src/mongo/db/s/balancer/cluster_statistics_impl.cpp9
-rw-r--r--src/mongo/db/s/balancer/migration_manager.cpp37
-rw-r--r--src/mongo/db/s/balancer/scoped_migration_request.cpp32
-rw-r--r--src/mongo/db/s/chunk_splitter.cpp65
-rw-r--r--src/mongo/db/s/cleanup_orphaned_cmd.cpp72
-rw-r--r--src/mongo/db/s/collection_sharding_runtime.cpp16
-rw-r--r--src/mongo/db/s/config/configsvr_add_shard_command.cpp7
-rw-r--r--src/mongo/s/write_ops/chunk_manager_targeter.cpp24
10 files changed, 210 insertions, 173 deletions
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 5b18930c30b..9666d231950 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
@@ -219,11 +219,10 @@ StatusWith<SplitInfoVector> BalancerChunkSelectionPolicyImpl::selectChunksToSpli
continue;
} else if (!candidatesStatus.isOK()) {
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()));
+ "Unable to enforce tag range policy for collection {namespace}: {error}",
+ "Unable to enforce tag range policy for collection",
+ "namespace"_attr = nss.ns(),
+ "error"_attr = candidatesStatus.getStatus());
continue;
}
@@ -286,8 +285,11 @@ StatusWith<MigrateInfoVector> BalancerChunkSelectionPolicyImpl::selectChunksToMo
const NamespaceString nss(coll.getNs());
if (!coll.getAllowBalance()) {
- LOGV2_DEBUG(
- 21851, 1, "Not balancing collection {nss}; explicitly disabled.", "nss"_attr = nss);
+ LOGV2_DEBUG(21851,
+ 1,
+ "Not balancing collection {namespace}; explicitly disabled.",
+ "Not balancing explicitly disabled collection",
+ "namespace"_attr = nss);
continue;
}
@@ -297,12 +299,11 @@ StatusWith<MigrateInfoVector> BalancerChunkSelectionPolicyImpl::selectChunksToMo
// Namespace got dropped before we managed to get to it, so just skip it
continue;
} else if (!candidatesStatus.isOK()) {
- LOGV2_WARNING(
- 21853,
- "Unable to balance collection {nss_ns}{causedBy_candidatesStatus_getStatus}",
- "nss_ns"_attr = nss.ns(),
- "causedBy_candidatesStatus_getStatus"_attr =
- causedBy(candidatesStatus.getStatus()));
+ LOGV2_WARNING(21853,
+ "Unable to balance collection {namespace}: {error}",
+ "Unable to balance collection",
+ "namespace"_attr = nss.ns(),
+ "error"_attr = candidatesStatus.getStatus());
continue;
}
diff --git a/src/mongo/db/s/balancer/balancer_policy.cpp b/src/mongo/db/s/balancer/balancer_policy.cpp
index 3fc275c2d33..ccec530f9d2 100644
--- a/src/mongo/db/s/balancer/balancer_policy.cpp
+++ b/src/mongo/db/s/balancer/balancer_policy.cpp
@@ -344,9 +344,10 @@ MigrateInfo chooseRandomMigration(const ShardStatisticsVector& shardStats,
LOGV2_DEBUG(21880,
1,
- "balancerShouldReturnRandomMigrations: source: {sourceShardId} dest: {destShardId}",
- "sourceShardId"_attr = sourceShardId,
- "destShardId"_attr = destShardId);
+ "balancerShouldReturnRandomMigrations: source: {fromShardId} dest: {toShardId}",
+ "balancerShouldReturnRandomMigrations",
+ "fromShardId"_attr = sourceShardId,
+ "toShardId"_attr = destShardId);
const auto& chunks = distribution.getChunks(sourceShardId);
@@ -408,6 +409,8 @@ vector<MigrateInfo> BalancerPolicy::balance(const ShardStatisticsVector& shardSt
LOGV2_WARNING(21889,
"Chunk {chunk} is on a draining shard, but no appropriate "
"recipient found",
+ "Chunk is on a draining shard, but no appropriate "
+ "recipient found",
"chunk"_attr = redact(chunk.toString()));
}
continue;
@@ -424,8 +427,9 @@ vector<MigrateInfo> BalancerPolicy::balance(const ShardStatisticsVector& shardSt
if (migrations.empty()) {
LOGV2_WARNING(21890,
"Unable to find any chunk to move from draining shard "
- "{stat_shardId}. numJumboChunks: {numJumboChunks}",
- "stat_shardId"_attr = stat.shardId,
+ "{shardId}. numJumboChunks: {numJumboChunks}",
+ "Unable to find any chunk to move from draining shard",
+ "shardId"_attr = stat.shardId,
"numJumboChunks"_attr = numJumboChunks);
}
}
@@ -451,9 +455,10 @@ vector<MigrateInfo> BalancerPolicy::balance(const ShardStatisticsVector& shardSt
if (chunk.getJumbo()) {
LOGV2_WARNING(
21891,
- "Chunk {chunk} violates zone {tag}, but it is jumbo and cannot be moved",
+ "Chunk {chunk} violates zone {zone}, but it is jumbo and cannot be moved",
+ "Chunk violates zone, but it is jumbo and cannot be moved",
"chunk"_attr = redact(chunk.toString()),
- "tag"_attr = redact(tag));
+ "zone"_attr = redact(tag));
continue;
}
@@ -461,11 +466,12 @@ vector<MigrateInfo> BalancerPolicy::balance(const ShardStatisticsVector& shardSt
_getLeastLoadedReceiverShard(shardStats, distribution, tag, *usedShards);
if (!to.isValid()) {
if (migrations.empty()) {
- LOGV2_WARNING(
- 21892,
- "Chunk {chunk} violates zone {tag}, but no appropriate recipient found",
- "chunk"_attr = redact(chunk.toString()),
- "tag"_attr = redact(tag));
+ LOGV2_WARNING(21892,
+ "Chunk {chunk} violates zone {zone}, but no appropriate "
+ "recipient found",
+ "Chunk violates zone, but no appropriate recipient found",
+ "chunk"_attr = redact(chunk.toString()),
+ "zone"_attr = redact(tag));
}
continue;
}
@@ -506,12 +512,14 @@ vector<MigrateInfo> BalancerPolicy::balance(const ShardStatisticsVector& shardSt
if (!tag.empty()) {
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());
+ "Zone {zone} in collection {namespace} 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.",
+ "Zone in collection 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.",
+ "zone"_attr = redact(tag),
+ "namespace"_attr = distribution.nss());
}
continue;
}
@@ -570,7 +578,10 @@ bool BalancerPolicy::_singleZoneBalance(const ShardStatisticsVector& shardStats,
const ShardId to = _getLeastLoadedReceiverShard(shardStats, distribution, tag, *usedShards);
if (!to.isValid()) {
if (migrations->empty()) {
- LOGV2(21882, "No available shards to take chunks for zone [{tag}]", "tag"_attr = tag);
+ LOGV2(21882,
+ "No available shards to take chunks for zone {zone}",
+ "No available shards to take chunks for zone",
+ "zone"_attr = tag);
}
return false;
}
@@ -583,22 +594,21 @@ bool BalancerPolicy::_singleZoneBalance(const ShardStatisticsVector& shardStats,
const size_t imbalance = max - idealNumberOfChunksPerShardForTag;
- 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);
+ 21883,
+ 1,
+ "collection: {namespace}, zone: {zone}, donor: {fromShardId} chunks on "
+ " {fromShardChunkCount}, receiver: {toShardId} chunks on {toShardChunkCount}, "
+ "ideal: {idealNumberOfChunksPerShardForTag}, threshold: {chunkCountImbalanceThreshold}",
+ "Balancing single zone",
+ "namespace"_attr = distribution.nss().ns(),
+ "zone"_attr = tag,
+ "fromShardId"_attr = from,
+ "fromShardChunkCount"_attr = max,
+ "toShardId"_attr = to,
+ "toShardChunkCount"_attr = min,
+ "idealNumberOfChunksPerShardForTag"_attr = idealNumberOfChunksPerShardForTag,
+ "chunkCountImbalanceThreshold"_attr = kDefaultImbalanceThreshold);
// Check whether it is necessary to balance within this zone
if (imbalance < kDefaultImbalanceThreshold)
@@ -624,13 +634,15 @@ bool BalancerPolicy::_singleZoneBalance(const ShardStatisticsVector& shardStats,
}
if (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);
+ LOGV2_WARNING(
+ 21894,
+ "Shard: {shardId}, collection: {namespace} has only jumbo chunks for "
+ "zone \'{zone}\' and cannot be balanced. Jumbo chunks count: {numJumboChunks}",
+ "Shard has only jumbo chunks for and cannot be balanced",
+ "shardId"_attr = from,
+ "namespace"_attr = distribution.nss().ns(),
+ "zone"_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 ec0f946299e..da0cfe916b8 100644
--- a/src/mongo/db/s/balancer/cluster_statistics_impl.cpp
+++ b/src/mongo/db/s/balancer/cluster_statistics_impl.cpp
@@ -142,11 +142,10 @@ StatusWith<std::vector<ShardStatistics>> ClusterStatisticsImpl::getStats(Operati
// 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
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()));
+ "Unable to obtain shard version for {shardId}: {error}",
+ "Unable to obtain shard version",
+ "shardId"_attr = shard.getName(),
+ "error"_attr = 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 100855905b7..cb97201876c 100644
--- a/src/mongo/db/s/balancer/migration_manager.cpp
+++ b/src/mongo/db/s/balancer/migration_manager.cpp
@@ -236,10 +236,9 @@ void MigrationManager::startRecoveryAndAcquireDistLocks(OperationContext* opCtx)
if (!statusWithMigrationsQueryResponse.isOK()) {
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())));
+ "recovery. Abandoning balancer recovery: {error}",
+ "Unable to read config.migrations documents for balancer migration recovery",
+ "error"_attr = redact(statusWithMigrationsQueryResponse.getStatus()));
return;
}
@@ -250,12 +249,11 @@ void MigrationManager::startRecoveryAndAcquireDistLocks(OperationContext* opCtx)
// this migration, but without parsing the migration document we cannot identify which
// distlock must be released. So we must release all distlocks.
LOGV2(21897,
- "Unable to parse config.migrations document '{migration}' for balancer migration "
- "recovery. Abandoning balancer "
- "recovery.{causedBy_statusWithMigrationType_getStatus}",
+ "Unable to parse config.migrations document '{migration}' for balancer"
+ "migration recovery. Abandoning balancer recovery: {error}",
+ "Unable to parse config.migrations document for balancer migration recovery",
"migration"_attr = redact(migration.toString()),
- "causedBy_statusWithMigrationType_getStatus"_attr =
- causedBy(redact(statusWithMigrationType.getStatus())));
+ "error"_attr = redact(statusWithMigrationType.getStatus()));
return;
}
MigrationType migrateType = std::move(statusWithMigrationType.getValue());
@@ -273,12 +271,13 @@ void MigrationManager::startRecoveryAndAcquireDistLocks(OperationContext* opCtx)
opCtx, migrateType.getNss().ns(), whyMessage, _lockSessionID);
if (!statusWithDistLockHandle.isOK()) {
LOGV2(21898,
- "Failed to acquire distributed lock for collection '{migrateType_getNss_ns}' "
+ "Failed to acquire distributed lock for collection {namespace} "
"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())));
+ "recovery: {error}",
+ "Failed to acquire distributed lock for collection "
+ "during balancer recovery of an active migration",
+ "namespace"_attr = migrateType.getNss().ns(),
+ "error"_attr = redact(statusWithDistLockHandle.getStatus()));
return;
}
}
@@ -330,11 +329,11 @@ void MigrationManager::finishRecovery(OperationContext* opCtx,
// config primary was active and the dist locks have been held by the balancer
// throughout. Abort migration recovery.
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())));
+ "Unable to reload chunk metadata for collection {namespace} during balancer "
+ "recovery. Abandoning recovery: {error}",
+ "Unable to reload chunk metadata for collection during balancer recovery",
+ "namespace"_attr = nss,
+ "error"_attr = 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 f44956aff83..fed4280904f 100644
--- a/src/mongo/db/s/balancer/scoped_migration_request.cpp
+++ b/src/mongo/db/s/balancer/scoped_migration_request.cpp
@@ -69,10 +69,10 @@ ScopedMigrationRequest::~ScopedMigrationRequest() {
if (!result.isOK()) {
LOGV2(21900,
- "Failed to remove config.migrations document for migration "
- "'{migrationDocumentIdentifier}'{causedBy_result}",
- "migrationDocumentIdentifier"_attr = migrationDocumentIdentifier.toString(),
- "causedBy_result"_attr = causedBy(redact(result)));
+ "Failed to remove config.migrations document for migration '{migration}': {error}",
+ "Failed to remove config.migrations document for migration",
+ "migration"_attr = migrationDocumentIdentifier.toString(),
+ "error"_attr = redact(result));
}
}
@@ -144,13 +144,16 @@ StatusWith<ScopedMigrationRequest> ScopedMigrationRequest::writeMigration(
MigrateInfo activeMigrateInfo = statusWithActiveMigration.getValue().toMigrateInfo();
if (activeMigrateInfo.to != migrateInfo.to ||
activeMigrateInfo.from != migrateInfo.from) {
- 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)));
+ LOGV2(
+ 21901,
+ "Failed to write document '{newMigration}' to config.migrations because there "
+ "is already an active migration for that chunk: "
+ "'{activeMigration}': {error}",
+ "Failed to write document to config.migrations because there "
+ "is already an active migration for that chunk",
+ "newMigration"_attr = redact(migrateInfo.toString()),
+ "activeMigration"_attr = redact(activeMigrateInfo.toString()),
+ "error"_attr = redact(result));
return result;
}
@@ -203,9 +206,10 @@ void ScopedMigrationRequest::keepDocumentOnDestruct() {
_opCtx = nullptr;
LOGV2_DEBUG(21902,
1,
- "Keeping config.migrations document with namespace '{nss}' and minKey '{minKey}' "
- "for balancer recovery",
- "nss"_attr = _nss,
+ "Keeping config.migrations document with namespace {namespace} and minKey "
+ "{minKey} for balancer recovery",
+ "Keeping config.migrations document for balancer recovery",
+ "namespace"_attr = _nss,
"minKey"_attr = _minKey);
}
diff --git a/src/mongo/db/s/chunk_splitter.cpp b/src/mongo/db/s/chunk_splitter.cpp
index 9a918df0e93..f7c59338c54 100644
--- a/src/mongo/db/s/chunk_splitter.cpp
+++ b/src/mongo/db/s/chunk_splitter.cpp
@@ -205,10 +205,10 @@ bool isAutoBalanceEnabled(OperationContext* opCtx,
auto collStatus = Grid::get(opCtx)->catalogClient()->getCollection(opCtx, nss);
if (!collStatus.isOK()) {
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())));
+ "Auto-split for {namespace} failed to load collection metadata: {error}",
+ "Auto-split failed to load collection metadata",
+ "namespace"_attr = nss,
+ "error"_attr = redact(collStatus.getStatus()));
return false;
}
@@ -248,7 +248,7 @@ void ChunkSplitter::onStepUp() {
}
_isPrimary = true;
- LOGV2(21904, "The ChunkSplitter has started and will accept autosplit tasks.");
+ LOGV2(21904, "The ChunkSplitter has started and will accept autosplit tasks");
}
void ChunkSplitter::onStepDown() {
@@ -260,7 +260,7 @@ void ChunkSplitter::onStepDown() {
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.");
+ "tasks that have already started will be allowed to finish");
}
void ChunkSplitter::waitForIdle() {
@@ -319,9 +319,10 @@ void ChunkSplitter::_runAutosplit(std::shared_ptr<ChunkSplitStateDriver> chunkSp
LOGV2_DEBUG(21906,
1,
"about to initiate autosplit: {chunk} dataWritten since last check: "
- "{dataWritten} maxChunkSizeBytes: {maxChunkSizeBytes}",
+ "{dataWrittenBytes} maxChunkSizeBytes: {maxChunkSizeBytes}",
+ "about to initiate autosplit",
"chunk"_attr = redact(chunk.toString()),
- "dataWritten"_attr = dataWritten,
+ "dataWrittenBytes"_attr = dataWritten,
"maxChunkSizeBytes"_attr = maxChunkSizeBytes);
chunkSplitStateDriver->prepareSplit();
@@ -340,6 +341,8 @@ void ChunkSplitter::_runAutosplit(std::shared_ptr<ChunkSplitStateDriver> chunkSp
1,
"ChunkSplitter attempted split but not enough split points were found for "
"chunk {chunk}",
+ "ChunkSplitter attempted split but not enough split points were found for "
+ "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
@@ -387,19 +390,18 @@ void ChunkSplitter::_runAutosplit(std::shared_ptr<ChunkSplitStateDriver> chunkSp
const bool shouldBalance = isAutoBalanceEnabled(opCtx.get(), nss, balancerConfig);
- 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)")));
+ LOGV2(21908,
+ "autosplitted {namespace} chunk: {chunk} with {splitPoints} split points "
+ "(maxChunkSizeBytes: {maxChunkSizeBytes}). {extraInfo}",
+ "autosplitted chunk",
+ "namespace"_attr = nss,
+ "chunk"_attr = redact(chunk.toString()),
+ "splitPoints"_attr = splitPoints.size(),
+ "maxChunkSizeBytes"_attr = maxChunkSizeBytes,
+ "extraInfo"_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
@@ -421,19 +423,20 @@ void ChunkSplitter::_runAutosplit(std::shared_ptr<ChunkSplitStateDriver> chunkSp
moveChunk(opCtx.get(), nss, topChunkMinKey);
} catch (const DBException& ex) {
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())));
+ "Top-chunk optimization failed to move chunk {chunk} in collection "
+ "{namespace} after a successful split: {error}",
+ "Top-chunk optimization failed to move chunk after a successful split",
+ "chunk"_attr = redact(ChunkRange(min, max).toString()),
+ "namespace"_attr = nss,
+ "error"_attr = redact(ex.toStatus()));
}
} catch (const DBException& ex) {
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())));
+ "Unable to auto-split chunk {chunk} in namespace {namespace}: {error}",
+ "Unable to auto-split chunk",
+ "chunk"_attr = redact(ChunkRange(min, max).toString()),
+ "namespace"_attr = nss,
+ "error"_attr = redact(ex.toStatus()));
}
}
diff --git a/src/mongo/db/s/cleanup_orphaned_cmd.cpp b/src/mongo/db/s/cleanup_orphaned_cmd.cpp
index 1085db00c3b..b9791237e92 100644
--- a/src/mongo/db/s/cleanup_orphaned_cmd.cpp
+++ b/src/mongo/db/s/cleanup_orphaned_cmd.cpp
@@ -98,8 +98,10 @@ CleanupResult cleanupOrphanedData(OperationContext* opCtx,
if (!autoColl.getCollection()) {
LOGV2(4416000,
"cleanupOrphaned skipping waiting for orphaned data cleanup because "
+ "{namespace} does not exist",
+ "cleanupOrphaned skipping waiting for orphaned data cleanup because "
"collection does not exist",
- "ns_ns"_attr = ns.ns());
+ "namespace"_attr = ns.ns());
return CleanupResult::kDone;
}
collectionUuid.emplace(autoColl.getCollection()->uuid());
@@ -109,8 +111,10 @@ CleanupResult cleanupOrphanedData(OperationContext* opCtx,
if (!collDesc.isSharded()) {
LOGV2(4416001,
"cleanupOrphaned skipping waiting for orphaned data cleanup because "
+ "{namespace} is not sharded",
+ "cleanupOrphaned skipping waiting for orphaned data cleanup because "
"collection is not sharded",
- "ns_ns"_attr = ns.ns());
+ "namespace"_attr = ns.ns());
return CleanupResult::kDone;
}
range.emplace(collDesc.getMinKey(), collDesc.getMaxKey());
@@ -120,12 +124,13 @@ CleanupResult cleanupOrphanedData(OperationContext* opCtx,
// cleanupOrphaned logic did if 'startingFromKey' is present.
BSONObj keyPattern = collDesc.getKeyPattern();
if (!startingFromKeyConst.isEmpty() && !collDesc.isValidKey(startingFromKeyConst)) {
- *errMsg = str::stream()
- << "could not cleanup orphaned data, start key " << startingFromKeyConst
- << " does not match shard key pattern " << keyPattern;
-
- LOGV2(4416002, "{errMsg}", "errMsg"_attr = *errMsg);
- return CleanupResult::kError;
+ LOGV2_ERROR_OPTIONS(
+ 4416002,
+ {logv2::UserAssertAfterLog(ErrorCodes::OrphanedRangeCleanUpFailed)},
+ "Could not cleanup orphaned data because start key does not match shard key "
+ "pattern",
+ "startKey"_attr = startingFromKeyConst,
+ "shardKeyPattern"_attr = keyPattern);
}
}
@@ -140,7 +145,7 @@ CleanupResult cleanupOrphanedData(OperationContext* opCtx,
migrationutil::checkForConflictingDeletions(opCtx, *range, *collectionUuid)) {
LOGV2(4416003,
"cleanupOrphaned going to wait for range deletion tasks to complete",
- "nss"_attr = ns.ns(),
+ "namespace"_attr = ns.ns(),
"collectionUUID"_attr = *collectionUuid,
"numRemainingDeletionTasks"_attr = numRemainingDeletionTasks);
@@ -170,19 +175,22 @@ CleanupResult cleanupOrphanedData(OperationContext* opCtx,
LOGV2(21911,
"cleanupOrphaned skipping orphaned data cleanup because collection is not "
"sharded",
- "ns_ns"_attr = ns.ns());
+ "namespace"_attr = ns.ns());
return CleanupResult::kDone;
}
BSONObj keyPattern = collDesc.getKeyPattern();
if (!startingFromKey.isEmpty()) {
if (!collDesc.isValidKey(startingFromKey)) {
- *errMsg = str::stream()
- << "could not cleanup orphaned data, start key " << startingFromKey
- << " does not match shard key pattern " << keyPattern;
-
- LOGV2(21912, "{errMsg}", "errMsg"_attr = *errMsg);
- return CleanupResult::kError;
+ LOGV2_ERROR_OPTIONS(
+ 21912,
+ {logv2::UserAssertAfterLog(ErrorCodes::OrphanedRangeCleanUpFailed)},
+ "Could not cleanup orphaned data, start key {startKey} does not match "
+ "shard key pattern {shardKeyPattern}",
+ "Could not cleanup orphaned data because start key does not match shard "
+ "key pattern",
+ "startKey"_attr = startingFromKey,
+ "shardKeyPattern"_attr = keyPattern);
}
} else {
startingFromKey = collDesc.getMinKey();
@@ -193,7 +201,7 @@ CleanupResult cleanupOrphanedData(OperationContext* opCtx,
LOGV2_DEBUG(21913,
1,
"cleanupOrphaned returning because no orphan ranges remain",
- "ns"_attr = ns.toString(),
+ "namespace"_attr = ns.toString(),
"startingFromKey"_attr = redact(startingFromKey));
return CleanupResult::kDone;
@@ -208,26 +216,30 @@ 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.
- 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()));
+ LOGV2_DEBUG(21914,
+ 1,
+ "cleanupOrphaned requested for {namespace} starting from {startingFromKey}, "
+ "removing next orphan range {targetRange}; waiting...",
+ "cleanupOrphaned requested",
+ "namespace"_attr = ns.toString(),
+ "startingFromKey"_attr = redact(startingFromKey),
+ "targetRange"_attr = redact(targetRange->toString()));
Status result = cleanupCompleteFuture.getNoThrow(opCtx);
LOGV2_DEBUG(21915,
1,
- "Finished waiting for last {ns} orphan range cleanup",
- "ns"_attr = ns.toString());
+ "Finished waiting for last {namespace} orphan range cleanup",
+ "Finished waiting for last orphan range cleanup in collection",
+ "namespace"_attr = ns.toString());
if (!result.isOK()) {
- LOGV2(21916, "{result_reason}", "result_reason"_attr = redact(result.reason()));
- *errMsg = result.reason();
- return CleanupResult::kError;
+ LOGV2_ERROR_OPTIONS(21916,
+ {logv2::UserAssertAfterLog(result.code())},
+ "Error waiting for last {namespace} orphan range cleanup: {error}",
+ "Error waiting for last orphan range cleanup in collection",
+ "namespace"_attr = ns.ns(),
+ "error"_attr = redact(result.reason()));
}
return CleanupResult::kContinue;
diff --git a/src/mongo/db/s/collection_sharding_runtime.cpp b/src/mongo/db/s/collection_sharding_runtime.cpp
index f4a53662f62..5cf6cf99568 100644
--- a/src/mongo/db/s/collection_sharding_runtime.cpp
+++ b/src/mongo/db/s/collection_sharding_runtime.cpp
@@ -204,9 +204,9 @@ void CollectionShardingRuntime::setFilteringMetadata(OperationContext* opCtx,
if (!newMetadata.isSharded()) {
LOGV2(21917,
- "Marking collection {nss_ns} as {newMetadata_Basic}",
- "nss_ns"_attr = _nss.ns(),
- "newMetadata_Basic"_attr = newMetadata.toStringBasic());
+ "Marking collection {namespace} as unsharded",
+ "Marking collection as unsharded",
+ "namespace"_attr = _nss.ns());
_metadataType = MetadataType::kUnsharded;
_metadataManager.reset();
++_numMetadataManagerChanges;
@@ -273,8 +273,9 @@ Status CollectionShardingRuntime::waitForClean(OperationContext* opCtx,
if (!stillScheduled) {
LOGV2_OPTIONS(21918,
{logv2::LogComponent::kShardingMigration},
- "Finished waiting for deletion of {nss_ns} range {orphanRange}",
- "nss_ns"_attr = nss.ns(),
+ "Finished waiting for deletion of {namespace} range {orphanRange}",
+ "Finished waiting for deletion of orphans",
+ "namespace"_attr = nss.ns(),
"orphanRange"_attr = redact(orphanRange.toString()));
return Status::OK();
}
@@ -282,8 +283,9 @@ Status CollectionShardingRuntime::waitForClean(OperationContext* opCtx,
LOGV2_OPTIONS(21919,
{logv2::LogComponent::kShardingMigration},
- "Waiting for deletion of {nss_ns} range {orphanRange}",
- "nss_ns"_attr = nss.ns(),
+ "Waiting for deletion of {namespace} range {orphanRange}",
+ "Waiting for deletion of orphans",
+ "namespace"_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 414370cfc23..282c809038d 100644
--- a/src/mongo/db/s/config/configsvr_add_shard_command.cpp
+++ b/src/mongo/db/s/config/configsvr_add_shard_command.cpp
@@ -130,9 +130,10 @@ public:
if (!addShardResult.isOK()) {
LOGV2(21920,
- "addShard request '{parsedRequest}'failed{causedBy_addShardResult_getStatus}",
- "parsedRequest"_attr = parsedRequest,
- "causedBy_addShardResult_getStatus"_attr = causedBy(addShardResult.getStatus()));
+ "addShard request '{request}' failed: {error}",
+ "addShard request failed",
+ "request"_attr = parsedRequest,
+ "error"_attr = addShardResult.getStatus());
uassertStatusOK(addShardResult.getStatus());
}
diff --git a/src/mongo/s/write_ops/chunk_manager_targeter.cpp b/src/mongo/s/write_ops/chunk_manager_targeter.cpp
index 80bb7a6ff16..ae05b9e3fff 100644
--- a/src/mongo/s/write_ops/chunk_manager_targeter.cpp
+++ b/src/mongo/s/write_ops/chunk_manager_targeter.cpp
@@ -273,9 +273,10 @@ CompareResult compareAllShardVersions(const CachedCollectionRoutingInfo& routing
} catch (const DBException& ex) {
LOGV2_WARNING(22915,
"could not lookup shard {shardId} in local cache, shard metadata may "
- "have changed or be unavailable{causedBy_ex}",
+ "have changed or be unavailable: {error}",
+ "Could not lookup shard in local cache",
"shardId"_attr = shardId,
- "causedBy_ex"_attr = causedBy(ex));
+ "error"_attr = ex);
return CompareResult_Unknown;
}
@@ -734,11 +735,12 @@ Status ChunkManagerTargeter::refreshIfNeeded(OperationContext* opCtx, bool* wasC
LOGV2_DEBUG(22912,
4,
"ChunkManagerTargeter checking if refresh is needed, "
- "needsTargetingRefresh({needsTargetingRefresh}) remoteShardVersions empty "
- "({remoteShardVersions_empty})) remoteDbVersion empty ({remoteDbVersion})",
+ "needsTargetingRefresh({needsTargetingRefresh}) has remoteShardVersion "
+ "({hasRemoteShardVersions})) has remoteDbVersion ({hasRemoteDbVersion})",
+ "ChunkManagerTargeter checking if refresh is needed",
"needsTargetingRefresh"_attr = _needsTargetingRefresh,
- "remoteShardVersions_empty"_attr = _remoteShardVersions.empty(),
- "remoteDbVersion"_attr = !_remoteDbVersion);
+ "hasRemoteShardVersions"_attr = !_remoteShardVersions.empty(),
+ "hasRemoteDbVersion"_attr = static_cast<bool>(_remoteDbVersion));
//
// Did we have any stale config or targeting errors at all?
@@ -794,8 +796,9 @@ Status ChunkManagerTargeter::refreshIfNeeded(OperationContext* opCtx, bool* wasC
LOGV2_DEBUG(22913,
4,
- "ChunkManagerTargeter shard versions comparison result: {int_result}",
- "int_result"_attr = (int)result);
+ "ChunkManagerTargeter shard versions comparison result: {result}",
+ "ChunkManagerTargeter shard versions comparison",
+ "result"_attr = static_cast<int>(result));
// Reset the versions
_remoteShardVersions.clear();
@@ -816,8 +819,9 @@ Status ChunkManagerTargeter::refreshIfNeeded(OperationContext* opCtx, bool* wasC
LOGV2_DEBUG(22914,
4,
- "ChunkManagerTargeter database versions comparison result: {int_result}",
- "int_result"_attr = (int)result);
+ "ChunkManagerTargeter database versions comparison result: {result}",
+ "ChunkManagerTargeter database versions comparison",
+ "result"_attr = static_cast<int>(result));
// Reset the version
_remoteDbVersion = boost::none;