|
@@ -1893,8 +1893,10 @@ public class BlockManager implements BlockStatsMXBean {
|
|
|
DatanodeStorageInfo storageInfo,
|
|
|
DatanodeDescriptor node) throws IOException {
|
|
|
if (b.getStored().isDeleted()) {
|
|
|
- blockLog.debug("BLOCK markBlockAsCorrupt: {} cannot be marked as" +
|
|
|
- " corrupt as it does not belong to any file", b);
|
|
|
+ if(blockLog.isDebugEnabled()) {
|
|
|
+ blockLog.debug("BLOCK markBlockAsCorrupt: {} cannot be marked as" +
|
|
|
+ " corrupt as it does not belong to any file", b);
|
|
|
+ }
|
|
|
addToInvalidates(b.getCorrupted(), node);
|
|
|
return;
|
|
|
}
|
|
@@ -1975,10 +1977,12 @@ public class BlockManager implements BlockStatsMXBean {
|
|
|
|
|
|
// Check how many copies we have of the block
|
|
|
if (nr.replicasOnStaleNodes() > 0 && !deleteCorruptReplicaImmediately) {
|
|
|
- blockLog.debug("BLOCK* invalidateBlocks: postponing " +
|
|
|
- "invalidation of {} on {} because {} replica(s) are located on " +
|
|
|
- "nodes with potentially out-of-date block reports", b, dn,
|
|
|
- nr.replicasOnStaleNodes());
|
|
|
+ if(blockLog.isDebugEnabled()) {
|
|
|
+ blockLog.debug("BLOCK* invalidateBlocks: postponing " +
|
|
|
+ "invalidation of {} on {} because {} replica(s) are located on " +
|
|
|
+ "nodes with potentially out-of-date block reports", b, dn,
|
|
|
+ nr.replicasOnStaleNodes());
|
|
|
+ }
|
|
|
postponeBlock(b.getCorrupted());
|
|
|
return false;
|
|
|
} else {
|
|
@@ -2220,8 +2224,10 @@ public class BlockManager implements BlockStatsMXBean {
|
|
|
int pendingNum = pendingReconstruction.getNumReplicas(block);
|
|
|
if (hasEnoughEffectiveReplicas(block, numReplicas, pendingNum)) {
|
|
|
neededReconstruction.remove(block, priority);
|
|
|
- blockLog.debug("BLOCK* Removing {} from neededReconstruction as" +
|
|
|
- " it has enough replicas", block);
|
|
|
+ if(blockLog.isDebugEnabled()) {
|
|
|
+ blockLog.debug("BLOCK* Removing {} from neededReconstruction as" +
|
|
|
+ " it has enough replicas", block);
|
|
|
+ }
|
|
|
NameNode.getNameNodeMetrics().incNumTimesReReplicationNotScheduled();
|
|
|
return null;
|
|
|
}
|
|
@@ -2317,8 +2323,10 @@ public class BlockManager implements BlockStatsMXBean {
|
|
|
if (hasEnoughEffectiveReplicas(block, numReplicas, pendingNum)) {
|
|
|
neededReconstruction.remove(block, priority);
|
|
|
rw.resetTargets();
|
|
|
- blockLog.debug("BLOCK* Removing {} from neededReconstruction as" +
|
|
|
- " it has enough replicas", block);
|
|
|
+ if(blockLog.isDebugEnabled()) {
|
|
|
+ blockLog.debug("BLOCK* Removing {} from neededReconstruction as" +
|
|
|
+ " it has enough replicas", block);
|
|
|
+ }
|
|
|
return false;
|
|
|
}
|
|
|
|
|
@@ -2349,8 +2357,10 @@ public class BlockManager implements BlockStatsMXBean {
|
|
|
// The reason we use 'pending' is so we can retry
|
|
|
// reconstructions that fail after an appropriate amount of time.
|
|
|
pendingReconstruction.increment(block, targets);
|
|
|
- blockLog.debug("BLOCK* block {} is moved from neededReconstruction to "
|
|
|
- + "pendingReconstruction", block);
|
|
|
+ if(blockLog.isDebugEnabled()) {
|
|
|
+ blockLog.debug("BLOCK* block {} is moved from neededReconstruction to "
|
|
|
+ + "pendingReconstruction", block);
|
|
|
+ }
|
|
|
|
|
|
int numEffectiveReplicas = numReplicas.liveReplicas() + pendingNum;
|
|
|
// remove from neededReconstruction
|
|
@@ -2896,9 +2906,11 @@ public class BlockManager implements BlockStatsMXBean {
|
|
|
|
|
|
if(blockLog.isDebugEnabled()) {
|
|
|
for (Block b : invalidatedBlocks) {
|
|
|
- blockLog.debug("BLOCK* processReport 0x{} with lease ID 0x{}: {} on node {} size {} " +
|
|
|
- "does not belong to any file.", strBlockReportId, fullBrLeaseId, b,
|
|
|
- node, b.getNumBytes());
|
|
|
+ if(blockLog.isDebugEnabled()) {
|
|
|
+ blockLog.debug("BLOCK* processReport 0x{} with lease ID 0x{}: {} on node {} size {} " +
|
|
|
+ "does not belong to any file.", strBlockReportId, fullBrLeaseId, b,
|
|
|
+ node, b.getNumBytes());
|
|
|
+ }
|
|
|
}
|
|
|
}
|
|
|
|
|
@@ -3056,9 +3068,11 @@ public class BlockManager implements BlockStatsMXBean {
|
|
|
}
|
|
|
}
|
|
|
if (isCorrupt) {
|
|
|
- blockLog.debug("BLOCK* markBlockReplicasAsCorrupt: mark block replica" +
|
|
|
- " {} on {} as corrupt because the dn is not in the new committed " +
|
|
|
- "storage list.", b, storage.getDatanodeDescriptor());
|
|
|
+ if(blockLog.isDebugEnabled()) {
|
|
|
+ blockLog.debug("BLOCK* markBlockReplicasAsCorrupt: mark block replica" +
|
|
|
+ " {} on {} as corrupt because the dn is not in the new committed " +
|
|
|
+ "storage list.", b, storage.getDatanodeDescriptor());
|
|
|
+ }
|
|
|
markBlockAsCorrupt(b, storage, storage.getDatanodeDescriptor());
|
|
|
}
|
|
|
}
|
|
@@ -3575,8 +3589,10 @@ public class BlockManager implements BlockStatsMXBean {
|
|
|
}
|
|
|
if (storedBlock == null || storedBlock.isDeleted()) {
|
|
|
// If this block does not belong to anyfile, then we are done.
|
|
|
- blockLog.debug("BLOCK* addStoredBlock: {} on {} size {} but it does not" +
|
|
|
- " belong to any file", block, node, block.getNumBytes());
|
|
|
+ if(blockLog.isDebugEnabled()) {
|
|
|
+ blockLog.debug("BLOCK* addStoredBlock: {} on {} size {} but it does not" +
|
|
|
+ " belong to any file", block, node, block.getNumBytes());
|
|
|
+ }
|
|
|
|
|
|
// we could add this block to invalidate set of this datanode.
|
|
|
// it will happen in next block report otherwise.
|
|
@@ -3605,9 +3621,11 @@ public class BlockManager implements BlockStatsMXBean {
|
|
|
corruptReplicas.removeFromCorruptReplicasMap(block, node,
|
|
|
Reason.GENSTAMP_MISMATCH);
|
|
|
curReplicaDelta = 0;
|
|
|
- blockLog.debug("BLOCK* addStoredBlock: Redundant addStoredBlock request"
|
|
|
- + " received for {} on node {} size {}", storedBlock, node,
|
|
|
- storedBlock.getNumBytes());
|
|
|
+ if(blockLog.isDebugEnabled()) {
|
|
|
+ blockLog.debug("BLOCK* addStoredBlock: Redundant addStoredBlock request"
|
|
|
+ + " received for {} on node {} size {}", storedBlock, node,
|
|
|
+ storedBlock.getNumBytes());
|
|
|
+ }
|
|
|
}
|
|
|
|
|
|
// Now check for completion of blocks and safe block count
|
|
@@ -3708,8 +3726,10 @@ public class BlockManager implements BlockStatsMXBean {
|
|
|
removedFromBlocksMap = false;
|
|
|
}
|
|
|
} catch (IOException e) {
|
|
|
- blockLog.debug("invalidateCorruptReplicas error in deleting bad block"
|
|
|
- + " {} on {}", blk, node, e);
|
|
|
+ if(blockLog.isDebugEnabled()) {
|
|
|
+ blockLog.debug("invalidateCorruptReplicas error in deleting bad block"
|
|
|
+ + " {} on {}", blk, node, e);
|
|
|
+ }
|
|
|
removedFromBlocksMap = false;
|
|
|
}
|
|
|
}
|
|
@@ -4158,8 +4178,10 @@ public class BlockManager implements BlockStatsMXBean {
|
|
|
//
|
|
|
final Block blockToInvalidate = getBlockOnStorage(storedBlock, chosen);
|
|
|
addToInvalidates(blockToInvalidate, chosen.getDatanodeDescriptor());
|
|
|
- blockLog.debug("BLOCK* chooseExcessRedundancies: "
|
|
|
- + "({}, {}) is added to invalidated blocks set", chosen, storedBlock);
|
|
|
+ if(blockLog.isDebugEnabled()) {
|
|
|
+ blockLog.debug("BLOCK* chooseExcessRedundancies: "
|
|
|
+ + "({}, {}) is added to invalidated blocks set", chosen, storedBlock);
|
|
|
+ }
|
|
|
}
|
|
|
|
|
|
private void removeStoredBlock(DatanodeStorageInfo storageInfo, Block block,
|
|
@@ -4220,8 +4242,7 @@ public class BlockManager implements BlockStatsMXBean {
|
|
|
for (ReplicaUnderConstruction r : staleReplicas) {
|
|
|
removeStoredBlock(block,
|
|
|
r.getExpectedStorageLocation().getDatanodeDescriptor());
|
|
|
- NameNode.blockStateChangeLog
|
|
|
- .debug("BLOCK* Removing stale replica {}" + " of {}", r,
|
|
|
+ blockLog.debug("BLOCK* Removing stale replica {} of {}", r,
|
|
|
Block.toString(r));
|
|
|
}
|
|
|
}
|
|
@@ -4350,8 +4371,10 @@ public class BlockManager implements BlockStatsMXBean {
|
|
|
maxNumBlocksToLog, numBlocksLogged);
|
|
|
}
|
|
|
for (Block b : toInvalidate) {
|
|
|
- blockLog.debug("BLOCK* addBlock: block {} on node {} size {} does not " +
|
|
|
- "belong to any file", b, node, b.getNumBytes());
|
|
|
+ if(blockLog.isDebugEnabled()) {
|
|
|
+ blockLog.debug("BLOCK* addBlock: block {} on node {} size {} does not " +
|
|
|
+ "belong to any file", b, node, b.getNumBytes());
|
|
|
+ }
|
|
|
addToInvalidates(b, node);
|
|
|
}
|
|
|
for (BlockToMarkCorrupt b : toCorrupt) {
|
|
@@ -4432,9 +4455,11 @@ public class BlockManager implements BlockStatsMXBean {
|
|
|
blockLog.debug("BLOCK* block {}: {} is received from {}",
|
|
|
rdbi.getStatus(), rdbi.getBlock(), node);
|
|
|
}
|
|
|
- blockLog.debug("*BLOCK* NameNode.processIncrementalBlockReport: from "
|
|
|
- + "{} receiving: {}, received: {}, deleted: {}", node, receiving,
|
|
|
- received, deleted);
|
|
|
+ if(blockLog.isDebugEnabled()) {
|
|
|
+ blockLog.debug("*BLOCK* NameNode.processIncrementalBlockReport: from "
|
|
|
+ + "{} receiving: {}, received: {}, deleted: {}", node, receiving,
|
|
|
+ received, deleted);
|
|
|
+ }
|
|
|
}
|
|
|
|
|
|
/**
|