HDFS-11832. Switch leftover logs to slf4j format in BlockManager.java. Contributed by Hui Xu and Chen Liang.
This commit is contained in:
parent
6c6a7a5962
commit
a7f085d6bf
|
@ -498,14 +498,13 @@ public class BlockManager implements BlockStatsMXBean {
|
|||
|
||||
bmSafeMode = new BlockManagerSafeMode(this, namesystem, haEnabled, conf);
|
||||
|
||||
LOG.info("defaultReplication = " + defaultReplication);
|
||||
LOG.info("maxReplication = " + maxReplication);
|
||||
LOG.info("minReplication = " + minReplication);
|
||||
LOG.info("maxReplicationStreams = " + maxReplicationStreams);
|
||||
LOG.info("redundancyRecheckInterval = " + redundancyRecheckIntervalMs +
|
||||
"ms");
|
||||
LOG.info("encryptDataTransfer = " + encryptDataTransfer);
|
||||
LOG.info("maxNumBlocksToLog = " + maxNumBlocksToLog);
|
||||
LOG.info("defaultReplication = {}", defaultReplication);
|
||||
LOG.info("maxReplication = {}", maxReplication);
|
||||
LOG.info("minReplication = {}", minReplication);
|
||||
LOG.info("maxReplicationStreams = {}", maxReplicationStreams);
|
||||
LOG.info("redundancyRecheckInterval = {}ms", redundancyRecheckIntervalMs);
|
||||
LOG.info("encryptDataTransfer = {}", encryptDataTransfer);
|
||||
LOG.info("maxNumBlocksToLog = {}", maxNumBlocksToLog);
|
||||
}
|
||||
|
||||
private static BlockTokenSecretManager createBlockTokenSecretManager(
|
||||
|
@ -513,7 +512,8 @@ public class BlockManager implements BlockStatsMXBean {
|
|||
final boolean isEnabled = conf.getBoolean(
|
||||
DFSConfigKeys.DFS_BLOCK_ACCESS_TOKEN_ENABLE_KEY,
|
||||
DFSConfigKeys.DFS_BLOCK_ACCESS_TOKEN_ENABLE_DEFAULT);
|
||||
LOG.info(DFSConfigKeys.DFS_BLOCK_ACCESS_TOKEN_ENABLE_KEY + "=" + isEnabled);
|
||||
LOG.info("{} = {}", DFSConfigKeys.DFS_BLOCK_ACCESS_TOKEN_ENABLE_KEY,
|
||||
isEnabled);
|
||||
|
||||
if (!isEnabled) {
|
||||
if (UserGroupInformation.isSecurityEnabled()) {
|
||||
|
@ -534,12 +534,10 @@ public class BlockManager implements BlockStatsMXBean {
|
|||
DFSConfigKeys.DFS_BLOCK_ACCESS_TOKEN_LIFETIME_DEFAULT);
|
||||
final String encryptionAlgorithm = conf.get(
|
||||
DFSConfigKeys.DFS_DATA_ENCRYPTION_ALGORITHM_KEY);
|
||||
LOG.info(DFSConfigKeys.DFS_BLOCK_ACCESS_KEY_UPDATE_INTERVAL_KEY
|
||||
+ "=" + updateMin + " min(s), "
|
||||
+ DFSConfigKeys.DFS_BLOCK_ACCESS_TOKEN_LIFETIME_KEY
|
||||
+ "=" + lifetimeMin + " min(s), "
|
||||
+ DFSConfigKeys.DFS_DATA_ENCRYPTION_ALGORITHM_KEY
|
||||
+ "=" + encryptionAlgorithm);
|
||||
LOG.info("{}={} min(s), {}={} min(s), {}={}",
|
||||
DFSConfigKeys.DFS_BLOCK_ACCESS_KEY_UPDATE_INTERVAL_KEY, updateMin,
|
||||
DFSConfigKeys.DFS_BLOCK_ACCESS_TOKEN_LIFETIME_KEY, lifetimeMin,
|
||||
DFSConfigKeys.DFS_DATA_ENCRYPTION_ALGORITHM_KEY, encryptionAlgorithm);
|
||||
|
||||
String nsId = DFSUtil.getNamenodeNameServiceId(conf);
|
||||
boolean isHaEnabled = HAUtil.isHAEnabled(conf, nsId);
|
||||
|
@ -692,8 +690,8 @@ public class BlockManager implements BlockStatsMXBean {
|
|||
Collection<DatanodeDescriptor> corruptNodes =
|
||||
corruptReplicas.getNodes(block);
|
||||
if (corruptNodes == null) {
|
||||
LOG.warn(block.getBlockId() +
|
||||
" is corrupt but has no associated node.");
|
||||
LOG.warn("{} is corrupt but has no associated node.",
|
||||
block.getBlockId());
|
||||
continue;
|
||||
}
|
||||
int numNodesToFind = corruptNodes.size();
|
||||
|
@ -1156,9 +1154,9 @@ public class BlockManager implements BlockStatsMXBean {
|
|||
final int numCorruptNodes = numReplicas.corruptReplicas();
|
||||
final int numCorruptReplicas = corruptReplicas.numCorruptReplicas(blk);
|
||||
if (numCorruptNodes != numCorruptReplicas) {
|
||||
LOG.warn("Inconsistent number of corrupt replicas for "
|
||||
+ blk + " blockMap has " + numCorruptNodes
|
||||
+ " but corrupt replicas map has " + numCorruptReplicas);
|
||||
LOG.warn("Inconsistent number of corrupt replicas for {}"
|
||||
+ " blockMap has {} but corrupt replicas map has {}",
|
||||
blk, numCorruptNodes, numCorruptReplicas);
|
||||
}
|
||||
|
||||
final int numNodes = blocksMap.numNodes(blk);
|
||||
|
@ -1232,7 +1230,7 @@ public class BlockManager implements BlockStatsMXBean {
|
|||
Collections.<LocatedBlock> emptyList(), null, false, feInfo, ecPolicy);
|
||||
} else {
|
||||
if (LOG.isDebugEnabled()) {
|
||||
LOG.debug("blocks = " + java.util.Arrays.asList(blocks));
|
||||
LOG.debug("blocks = {}", java.util.Arrays.asList(blocks));
|
||||
}
|
||||
final AccessMode mode = needBlockToken? BlockTokenIdentifier.AccessMode.READ: null;
|
||||
final List<LocatedBlock> locatedblocks = createLocatedBlockList(
|
||||
|
@ -1829,8 +1827,7 @@ public class BlockManager implements BlockStatsMXBean {
|
|||
numReplicas);
|
||||
if(srcNodes == null || srcNodes.length == 0) {
|
||||
// block can not be reconstructed from any node
|
||||
LOG.debug("Block " + block + " cannot be reconstructed " +
|
||||
"from any node");
|
||||
LOG.debug("Block {} cannot be reconstructed from any node", block);
|
||||
return null;
|
||||
}
|
||||
|
||||
|
@ -2272,11 +2269,9 @@ public class BlockManager implements BlockStatsMXBean {
|
|||
removeBlock(b);
|
||||
}
|
||||
if (trackBlockCounts) {
|
||||
if (LOG.isDebugEnabled()) {
|
||||
LOG.debug("Adjusting safe-mode totals for deletion."
|
||||
+ "decreasing safeBlocks by " + numRemovedSafe
|
||||
+ ", totalBlocks by " + numRemovedComplete);
|
||||
}
|
||||
LOG.debug("Adjusting safe-mode totals for deletion."
|
||||
+ "decreasing safeBlocks by {}, totalBlocks by {}",
|
||||
numRemovedSafe, numRemovedComplete);
|
||||
bmSafeMode.adjustBlockTotals(-numRemovedSafe, -numRemovedComplete);
|
||||
}
|
||||
}
|
||||
|
@ -2440,18 +2435,14 @@ public class BlockManager implements BlockStatsMXBean {
|
|||
|
||||
BlockInfo bi = getStoredBlock(b);
|
||||
if (bi == null) {
|
||||
if (LOG.isDebugEnabled()) {
|
||||
LOG.debug("BLOCK* rescanPostponedMisreplicatedBlocks: " +
|
||||
"Postponed mis-replicated block " + b + " no longer found " +
|
||||
"in block map.");
|
||||
}
|
||||
LOG.debug("BLOCK* rescanPostponedMisreplicatedBlocks: " +
|
||||
"Postponed mis-replicated block {} no longer found " +
|
||||
"in block map.", b);
|
||||
continue;
|
||||
}
|
||||
MisReplicationResult res = processMisReplicatedBlock(bi);
|
||||
if (LOG.isDebugEnabled()) {
|
||||
LOG.debug("BLOCK* rescanPostponedMisreplicatedBlocks: " +
|
||||
"Re-scanned block " + b + ", result is " + res);
|
||||
}
|
||||
LOG.debug("BLOCK* rescanPostponedMisreplicatedBlocks: " +
|
||||
"Re-scanned block {}, result is {}", b, res);
|
||||
if (res == MisReplicationResult.POSTPONE) {
|
||||
rescannedMisreplicatedBlocks.add(b);
|
||||
}
|
||||
|
@ -2461,10 +2452,9 @@ public class BlockManager implements BlockStatsMXBean {
|
|||
rescannedMisreplicatedBlocks.clear();
|
||||
long endSize = postponedMisreplicatedBlocks.size();
|
||||
namesystem.writeUnlock();
|
||||
LOG.info("Rescan of postponedMisreplicatedBlocks completed in " +
|
||||
(Time.monotonicNow() - startTime) + " msecs. " +
|
||||
endSize + " blocks are left. " +
|
||||
(startSize - endSize) + " blocks were removed.");
|
||||
LOG.info("Rescan of postponedMisreplicatedBlocks completed in {}" +
|
||||
" msecs. {} blocks are left. {} blocks were removed.",
|
||||
(Time.monotonicNow() - startTime), endSize, (startSize - endSize));
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -2603,9 +2593,9 @@ public class BlockManager implements BlockStatsMXBean {
|
|||
ReplicaState reportedState = iblk.getState();
|
||||
|
||||
if (LOG.isDebugEnabled()) {
|
||||
LOG.debug("Initial report of block " + iblk.getBlockName()
|
||||
+ " on " + storageInfo.getDatanodeDescriptor() + " size " +
|
||||
iblk.getNumBytes() + " replicaState = " + reportedState);
|
||||
LOG.debug("Initial report of block {} on {} size {} replicaState = {}",
|
||||
iblk.getBlockName(), storageInfo.getDatanodeDescriptor(),
|
||||
iblk.getNumBytes(), reportedState);
|
||||
}
|
||||
if (shouldPostponeBlocksFromFuture && isGenStampInFuture(iblk)) {
|
||||
queueReportedBlock(storageInfo, iblk, reportedState,
|
||||
|
@ -2684,11 +2674,8 @@ public class BlockManager implements BlockStatsMXBean {
|
|||
|
||||
ReplicaState reportedState = replica.getState();
|
||||
|
||||
if (LOG.isDebugEnabled()) {
|
||||
LOG.debug("Reported block " + replica
|
||||
+ " on " + dn + " size " + replica.getNumBytes()
|
||||
+ " replicaState = " + reportedState);
|
||||
}
|
||||
LOG.debug("Reported block {} on {} size {} replicaState = {}",
|
||||
replica, dn, replica.getNumBytes(), reportedState);
|
||||
|
||||
if (shouldPostponeBlocksFromFuture
|
||||
&& isGenStampInFuture(replica)) {
|
||||
|
@ -2754,9 +2741,7 @@ public class BlockManager implements BlockStatsMXBean {
|
|||
BlockUCState ucState = storedBlock.getBlockUCState();
|
||||
|
||||
// Block is on the NN
|
||||
if (LOG.isDebugEnabled()) {
|
||||
LOG.debug("In memory blockUCState = " + ucState);
|
||||
}
|
||||
LOG.debug("In memory blockUCState = {}", ucState);
|
||||
|
||||
// Ignore replicas already scheduled to be removed from the DN
|
||||
if (invalidateBlocks.contains(dn, replica)) {
|
||||
|
@ -2798,13 +2783,10 @@ public class BlockManager implements BlockStatsMXBean {
|
|||
private void queueReportedBlock(DatanodeStorageInfo storageInfo, Block block,
|
||||
ReplicaState reportedState, String reason) {
|
||||
assert shouldPostponeBlocksFromFuture;
|
||||
|
||||
if (LOG.isDebugEnabled()) {
|
||||
LOG.debug("Queueing reported block " + block +
|
||||
" in state " + reportedState +
|
||||
" from datanode " + storageInfo.getDatanodeDescriptor() +
|
||||
" for later processing because " + reason + ".");
|
||||
}
|
||||
|
||||
LOG.debug("Queueing reported block {} in state {}" +
|
||||
" from datanode {} for later processing because {}.",
|
||||
block, reportedState, storageInfo.getDatanodeDescriptor(), reason);
|
||||
pendingDNMessages.enqueueReportedBlock(storageInfo, block, reportedState);
|
||||
}
|
||||
|
||||
|
@ -2825,9 +2807,7 @@ public class BlockManager implements BlockStatsMXBean {
|
|||
private void processQueuedMessages(Iterable<ReportedBlockInfo> rbis)
|
||||
throws IOException {
|
||||
for (ReportedBlockInfo rbi : rbis) {
|
||||
if (LOG.isDebugEnabled()) {
|
||||
LOG.debug("Processing previouly queued message " + rbi);
|
||||
}
|
||||
LOG.debug("Processing previouly queued message {}", rbi);
|
||||
if (rbi.getReportedState() == null) {
|
||||
// This is a DELETE_BLOCK request
|
||||
DatanodeStorageInfo storageInfo = rbi.getStorageInfo();
|
||||
|
@ -2852,8 +2832,8 @@ public class BlockManager implements BlockStatsMXBean {
|
|||
"block postponement.";
|
||||
int count = pendingDNMessages.count();
|
||||
if (count > 0) {
|
||||
LOG.info("Processing " + count + " messages from DataNodes " +
|
||||
"that were previously queued during standby state");
|
||||
LOG.info("Processing {} messages from DataNodes " +
|
||||
"that were previously queued during standby state", count);
|
||||
}
|
||||
processQueuedMessages(pendingDNMessages.takeAll());
|
||||
assert pendingDNMessages.count() == 0;
|
||||
|
@ -2935,9 +2915,8 @@ public class BlockManager implements BlockStatsMXBean {
|
|||
// the block report got a little bit delayed after the pipeline
|
||||
// closed. So, ignore this report, assuming we will get a
|
||||
// FINALIZED replica later. See HDFS-2791
|
||||
LOG.info("Received an RBW replica for " + storedBlock +
|
||||
" on " + dn + ": ignoring it, since it is " +
|
||||
"complete with the same genstamp");
|
||||
LOG.info("Received an RBW replica for {} on {}: ignoring it, since "
|
||||
+ "it is complete with the same genstamp", storedBlock, dn);
|
||||
return null;
|
||||
} else {
|
||||
return new BlockToMarkCorrupt(new Block(reported), storedBlock,
|
||||
|
@ -2952,7 +2931,7 @@ public class BlockManager implements BlockStatsMXBean {
|
|||
+ " for block: " + storedBlock +
|
||||
" on " + dn + " size " + storedBlock.getNumBytes();
|
||||
// log here at WARN level since this is really a broken HDFS invariant
|
||||
LOG.warn(msg);
|
||||
LOG.warn("{}", msg);
|
||||
return new BlockToMarkCorrupt(new Block(reported), storedBlock, msg,
|
||||
Reason.INVALID_STATE);
|
||||
}
|
||||
|
@ -3131,9 +3110,9 @@ public class BlockManager implements BlockStatsMXBean {
|
|||
int corruptReplicasCount = corruptReplicas.numCorruptReplicas(storedBlock);
|
||||
int numCorruptNodes = num.corruptReplicas();
|
||||
if (numCorruptNodes != corruptReplicasCount) {
|
||||
LOG.warn("Inconsistent number of corrupt replicas for " +
|
||||
storedBlock + ". blockMap has " + numCorruptNodes +
|
||||
" but corrupt replicas map has " + corruptReplicasCount);
|
||||
LOG.warn("Inconsistent number of corrupt replicas for {}" +
|
||||
". blockMap has {} but corrupt replicas map has {}",
|
||||
storedBlock, numCorruptNodes, corruptReplicasCount);
|
||||
}
|
||||
if ((corruptReplicasCount > 0) && (numLiveReplicas >= fileRedundancy)) {
|
||||
invalidateCorruptReplicas(storedBlock, reportedBlock, num);
|
||||
|
@ -3298,12 +3277,13 @@ public class BlockManager implements BlockStatsMXBean {
|
|||
/ totalBlocks, 1.0);
|
||||
|
||||
if (!blocksItr.hasNext()) {
|
||||
LOG.info("Total number of blocks = " + blocksMap.size());
|
||||
LOG.info("Number of invalid blocks = " + nrInvalid);
|
||||
LOG.info("Number of under-replicated blocks = " + nrUnderReplicated);
|
||||
LOG.info("Number of over-replicated blocks = " + nrOverReplicated
|
||||
+ ((nrPostponed > 0) ? (" (" + nrPostponed + " postponed)") : ""));
|
||||
LOG.info("Number of blocks being written = " + nrUnderConstruction);
|
||||
LOG.info("Total number of blocks = {}", blocksMap.size());
|
||||
LOG.info("Number of invalid blocks = {}", nrInvalid);
|
||||
LOG.info("Number of under-replicated blocks = {}", nrUnderReplicated);
|
||||
LOG.info("Number of over-replicated blocks = {}{}", nrOverReplicated,
|
||||
((nrPostponed > 0) ? (" (" + nrPostponed + " postponed)") : ""));
|
||||
LOG.info("Number of blocks being written = {}",
|
||||
nrUnderConstruction);
|
||||
NameNode.stateChangeLog
|
||||
.info("STATE* Replication Queue initialization "
|
||||
+ "scan for invalid, over- and under-replicated blocks "
|
||||
|
@ -3730,11 +3710,8 @@ public class BlockManager implements BlockStatsMXBean {
|
|||
|
||||
final DatanodeDescriptor node = storageInfo.getDatanodeDescriptor();
|
||||
|
||||
if(LOG.isDebugEnabled()) {
|
||||
LOG.debug("Reported block " + block
|
||||
+ " on " + node + " size " + block.getNumBytes()
|
||||
+ " replicaState = " + reportedState);
|
||||
}
|
||||
LOG.debug("Reported block {} on {} size {} replicaState = {}",
|
||||
block, node, block.getNumBytes(), reportedState);
|
||||
|
||||
if (shouldPostponeBlocksFromFuture &&
|
||||
isGenStampInFuture(block)) {
|
||||
|
@ -3756,9 +3733,7 @@ public class BlockManager implements BlockStatsMXBean {
|
|||
|
||||
BlockUCState ucState = storedBlock.getBlockUCState();
|
||||
// Block is on the NN
|
||||
if(LOG.isDebugEnabled()) {
|
||||
LOG.debug("In memory blockUCState = " + ucState);
|
||||
}
|
||||
LOG.debug("In memory blockUCState = {}", ucState);
|
||||
|
||||
// Ignore replicas already scheduled to be removed from the DN
|
||||
if(invalidateBlocks.contains(node, block)) {
|
||||
|
@ -4013,8 +3988,8 @@ public class BlockManager implements BlockStatsMXBean {
|
|||
numExtraRedundancy++;
|
||||
}
|
||||
}
|
||||
LOG.info("Invalidated " + numExtraRedundancy
|
||||
+ " extra redundancy blocks on " + srcNode + " after it is in service");
|
||||
LOG.info("Invalidated {} extra redundancy blocks on {} after "
|
||||
+ "it is in service", numExtraRedundancy, srcNode);
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -4175,8 +4150,8 @@ public class BlockManager implements BlockStatsMXBean {
|
|||
try {
|
||||
DatanodeDescriptor dnDescriptor = datanodeManager.getDatanode(dn);
|
||||
if (dnDescriptor == null) {
|
||||
LOG.warn("DataNode " + dn + " cannot be found with UUID " +
|
||||
dn.getDatanodeUuid() + ", removing block invalidation work.");
|
||||
LOG.warn("DataNode {} cannot be found with UUID {}" +
|
||||
", removing block invalidation work.", dn, dn.getDatanodeUuid());
|
||||
invalidateBlocks.remove(dn);
|
||||
return 0;
|
||||
}
|
||||
|
|
|
@ -70,13 +70,14 @@ class InvalidateBlocks {
|
|||
}
|
||||
|
||||
private void printBlockDeletionTime(final Logger log) {
|
||||
log.info(DFSConfigKeys.DFS_NAMENODE_STARTUP_DELAY_BLOCK_DELETION_SEC_KEY
|
||||
+ " is set to " + DFSUtil.durationToString(pendingPeriodInMs));
|
||||
log.info("{} is set to {}",
|
||||
DFSConfigKeys.DFS_NAMENODE_STARTUP_DELAY_BLOCK_DELETION_SEC_KEY,
|
||||
DFSUtil.durationToString(pendingPeriodInMs));
|
||||
SimpleDateFormat sdf = new SimpleDateFormat("yyyy MMM dd HH:mm:ss");
|
||||
Calendar calendar = new GregorianCalendar();
|
||||
calendar.add(Calendar.SECOND, (int) (this.pendingPeriodInMs / 1000));
|
||||
log.info("The block deletion will start around "
|
||||
+ sdf.format(calendar.getTime()));
|
||||
log.info("The block deletion will start around {}",
|
||||
sdf.format(calendar.getTime()));
|
||||
}
|
||||
|
||||
/** @return the number of blocks to be invalidated . */
|
||||
|
@ -173,11 +174,9 @@ class InvalidateBlocks {
|
|||
synchronized List<Block> invalidateWork(final DatanodeDescriptor dn) {
|
||||
final long delay = getInvalidationDelay();
|
||||
if (delay > 0) {
|
||||
if (BlockManager.LOG.isDebugEnabled()) {
|
||||
BlockManager.LOG
|
||||
.debug("Block deletion is delayed during NameNode startup. "
|
||||
+ "The deletion will start after " + delay + " ms.");
|
||||
}
|
||||
BlockManager.LOG
|
||||
.debug("Block deletion is delayed during NameNode startup. "
|
||||
+ "The deletion will start after {} ms.", delay);
|
||||
return null;
|
||||
}
|
||||
final LightWeightHashSet<Block> set = node2blocks.get(dn);
|
||||
|
|
Loading…
Reference in New Issue