|
@@ -1541,6 +1541,7 @@ public class BlockManager implements BlockStatsMXBean {
|
|
if (LOG.isDebugEnabled()) {
|
|
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 AccessMode mode = needBlockToken? BlockTokenIdentifier.AccessMode.READ: null;
|
|
|
|
|
|
LocatedBlockBuilder locatedBlocks = providedStorageMap
|
|
LocatedBlockBuilder locatedBlocks = providedStorageMap
|
|
@@ -1873,8 +1874,7 @@ public class BlockManager implements BlockStatsMXBean {
|
|
}
|
|
}
|
|
|
|
|
|
if (storage == null) {
|
|
if (storage == null) {
|
|
- blockLog.debug("BLOCK* findAndMarkBlockAsCorrupt: {} not found on {}",
|
|
|
|
- blk, dn);
|
|
|
|
|
|
+ blockLog.debug("BLOCK* findAndMarkBlockAsCorrupt: {} not found on {}", blk, dn);
|
|
return;
|
|
return;
|
|
}
|
|
}
|
|
markBlockAsCorrupt(new BlockToMarkCorrupt(reportedBlock, storedBlock,
|
|
markBlockAsCorrupt(new BlockToMarkCorrupt(reportedBlock, storedBlock,
|
|
@@ -1893,7 +1893,7 @@ public class BlockManager implements BlockStatsMXBean {
|
|
DatanodeStorageInfo storageInfo,
|
|
DatanodeStorageInfo storageInfo,
|
|
DatanodeDescriptor node) throws IOException {
|
|
DatanodeDescriptor node) throws IOException {
|
|
if (b.getStored().isDeleted()) {
|
|
if (b.getStored().isDeleted()) {
|
|
- if(blockLog.isDebugEnabled()) {
|
|
|
|
|
|
+ if (blockLog.isDebugEnabled()) {
|
|
blockLog.debug("BLOCK markBlockAsCorrupt: {} cannot be marked as" +
|
|
blockLog.debug("BLOCK markBlockAsCorrupt: {} cannot be marked as" +
|
|
" corrupt as it does not belong to any file", b);
|
|
" corrupt as it does not belong to any file", b);
|
|
}
|
|
}
|
|
@@ -1977,7 +1977,7 @@ public class BlockManager implements BlockStatsMXBean {
|
|
|
|
|
|
// Check how many copies we have of the block
|
|
// Check how many copies we have of the block
|
|
if (nr.replicasOnStaleNodes() > 0 && !deleteCorruptReplicaImmediately) {
|
|
if (nr.replicasOnStaleNodes() > 0 && !deleteCorruptReplicaImmediately) {
|
|
- if(blockLog.isDebugEnabled()) {
|
|
|
|
|
|
+ if (blockLog.isDebugEnabled()) {
|
|
blockLog.debug("BLOCK* invalidateBlocks: postponing " +
|
|
blockLog.debug("BLOCK* invalidateBlocks: postponing " +
|
|
"invalidation of {} on {} because {} replica(s) are located on " +
|
|
"invalidation of {} on {} because {} replica(s) are located on " +
|
|
"nodes with potentially out-of-date block reports", b, dn,
|
|
"nodes with potentially out-of-date block reports", b, dn,
|
|
@@ -1990,8 +1990,7 @@ public class BlockManager implements BlockStatsMXBean {
|
|
// function and know there are enough live replicas, so we can delete it.
|
|
// function and know there are enough live replicas, so we can delete it.
|
|
addToInvalidates(b.getCorrupted(), dn);
|
|
addToInvalidates(b.getCorrupted(), dn);
|
|
removeStoredBlock(b.getStored(), node);
|
|
removeStoredBlock(b.getStored(), node);
|
|
- blockLog.debug("BLOCK* invalidateBlocks: {} on {} listed for deletion.",
|
|
|
|
- b, dn);
|
|
|
|
|
|
+ blockLog.debug("BLOCK* invalidateBlocks: {} on {} listed for deletion.", b, dn);
|
|
return true;
|
|
return true;
|
|
}
|
|
}
|
|
}
|
|
}
|
|
@@ -2156,13 +2155,11 @@ public class BlockManager implements BlockStatsMXBean {
|
|
for (DatanodeStorageInfo target : targets) {
|
|
for (DatanodeStorageInfo target : targets) {
|
|
targetList.append(' ').append(target.getDatanodeDescriptor());
|
|
targetList.append(' ').append(target.getDatanodeDescriptor());
|
|
}
|
|
}
|
|
- blockLog.debug("BLOCK* ask {} to replicate {} to {}", rw.getSrcNodes(),
|
|
|
|
- rw.getBlock(), targetList);
|
|
|
|
|
|
+ blockLog.debug("BLOCK* ask {} to replicate {} to {}",
|
|
|
|
+ rw.getSrcNodes(), rw.getBlock(), targetList);
|
|
}
|
|
}
|
|
}
|
|
}
|
|
-
|
|
|
|
- blockLog.debug(
|
|
|
|
- "BLOCK* neededReconstruction = {} pendingReconstruction = {}",
|
|
|
|
|
|
+ blockLog.debug("BLOCK* neededReconstruction = {} pendingReconstruction = {}",
|
|
neededReconstruction.size(), pendingReconstruction.size());
|
|
neededReconstruction.size(), pendingReconstruction.size());
|
|
}
|
|
}
|
|
|
|
|
|
@@ -2201,7 +2198,7 @@ public class BlockManager implements BlockStatsMXBean {
|
|
liveBlockIndices, liveBusyBlockIndices, excludeReconstructed, priority);
|
|
liveBlockIndices, liveBusyBlockIndices, excludeReconstructed, priority);
|
|
short requiredRedundancy = getExpectedLiveRedundancyNum(block,
|
|
short requiredRedundancy = getExpectedLiveRedundancyNum(block,
|
|
numReplicas);
|
|
numReplicas);
|
|
- if(srcNodes == null || srcNodes.length == 0) {
|
|
|
|
|
|
+ if (srcNodes == null || srcNodes.length == 0) {
|
|
// block can not be reconstructed from any node
|
|
// block can not be reconstructed from any node
|
|
LOG.debug("Block {} cannot be reconstructed from any node", block);
|
|
LOG.debug("Block {} cannot be reconstructed from any node", block);
|
|
NameNode.getNameNodeMetrics().incNumTimesReReplicationNotScheduled();
|
|
NameNode.getNameNodeMetrics().incNumTimesReReplicationNotScheduled();
|
|
@@ -2225,10 +2222,8 @@ public class BlockManager implements BlockStatsMXBean {
|
|
int pendingNum = pendingReconstruction.getNumReplicas(block);
|
|
int pendingNum = pendingReconstruction.getNumReplicas(block);
|
|
if (hasEnoughEffectiveReplicas(block, numReplicas, pendingNum)) {
|
|
if (hasEnoughEffectiveReplicas(block, numReplicas, pendingNum)) {
|
|
neededReconstruction.remove(block, priority);
|
|
neededReconstruction.remove(block, priority);
|
|
- if(blockLog.isDebugEnabled()) {
|
|
|
|
- blockLog.debug("BLOCK* Removing {} from neededReconstruction as" +
|
|
|
|
- " it has enough replicas", block);
|
|
|
|
- }
|
|
|
|
|
|
+ blockLog.debug("BLOCK* Removing {} from neededReconstruction as it has enough replicas",
|
|
|
|
+ block);
|
|
NameNode.getNameNodeMetrics().incNumTimesReReplicationNotScheduled();
|
|
NameNode.getNameNodeMetrics().incNumTimesReReplicationNotScheduled();
|
|
return null;
|
|
return null;
|
|
}
|
|
}
|
|
@@ -2328,10 +2323,8 @@ public class BlockManager implements BlockStatsMXBean {
|
|
if (hasEnoughEffectiveReplicas(block, numReplicas, pendingNum)) {
|
|
if (hasEnoughEffectiveReplicas(block, numReplicas, pendingNum)) {
|
|
neededReconstruction.remove(block, priority);
|
|
neededReconstruction.remove(block, priority);
|
|
rw.resetTargets();
|
|
rw.resetTargets();
|
|
- if(blockLog.isDebugEnabled()) {
|
|
|
|
- blockLog.debug("BLOCK* Removing {} from neededReconstruction as" +
|
|
|
|
- " it has enough replicas", block);
|
|
|
|
- }
|
|
|
|
|
|
+ blockLog.debug("BLOCK* Removing {} from neededReconstruction as it has enough replicas",
|
|
|
|
+ block);
|
|
return false;
|
|
return false;
|
|
}
|
|
}
|
|
|
|
|
|
@@ -2362,10 +2355,8 @@ public class BlockManager implements BlockStatsMXBean {
|
|
// The reason we use 'pending' is so we can retry
|
|
// The reason we use 'pending' is so we can retry
|
|
// reconstructions that fail after an appropriate amount of time.
|
|
// reconstructions that fail after an appropriate amount of time.
|
|
pendingReconstruction.increment(block, targets);
|
|
pendingReconstruction.increment(block, targets);
|
|
- if(blockLog.isDebugEnabled()) {
|
|
|
|
- blockLog.debug("BLOCK* block {} is moved from neededReconstruction to "
|
|
|
|
- + "pendingReconstruction", block);
|
|
|
|
- }
|
|
|
|
|
|
+ blockLog.debug("BLOCK* block {} is moved from neededReconstruction to pendingReconstruction",
|
|
|
|
+ block);
|
|
|
|
|
|
int numEffectiveReplicas = numReplicas.liveReplicas() + pendingNum;
|
|
int numEffectiveReplicas = numReplicas.liveReplicas() + pendingNum;
|
|
// remove from neededReconstruction
|
|
// remove from neededReconstruction
|
|
@@ -2758,9 +2749,11 @@ public class BlockManager implements BlockStatsMXBean {
|
|
removeBlock(b);
|
|
removeBlock(b);
|
|
}
|
|
}
|
|
if (trackBlockCounts) {
|
|
if (trackBlockCounts) {
|
|
- LOG.debug("Adjusting safe-mode totals for deletion."
|
|
|
|
- + "decreasing safeBlocks by {}, totalBlocks by {}",
|
|
|
|
- numRemovedSafe, numRemovedComplete);
|
|
|
|
|
|
+ if (LOG.isDebugEnabled()) {
|
|
|
|
+ LOG.debug("Adjusting safe-mode totals for deletion."
|
|
|
|
+ + "decreasing safeBlocks by {}, totalBlocks by {}",
|
|
|
|
+ numRemovedSafe, numRemovedComplete);
|
|
|
|
+ }
|
|
bmSafeMode.adjustBlockTotals(-numRemovedSafe, -numRemovedComplete);
|
|
bmSafeMode.adjustBlockTotals(-numRemovedSafe, -numRemovedComplete);
|
|
}
|
|
}
|
|
}
|
|
}
|
|
@@ -2913,13 +2906,11 @@ public class BlockManager implements BlockStatsMXBean {
|
|
namesystem.writeUnlock("processReport");
|
|
namesystem.writeUnlock("processReport");
|
|
}
|
|
}
|
|
|
|
|
|
- if(blockLog.isDebugEnabled()) {
|
|
|
|
|
|
+ if (blockLog.isDebugEnabled()) {
|
|
for (Block b : invalidatedBlocks) {
|
|
for (Block b : invalidatedBlocks) {
|
|
- 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());
|
|
|
|
- }
|
|
|
|
|
|
+ blockLog.debug("BLOCK* processReport 0x{} with lease ID 0x{}: {} on node {} size {} " +
|
|
|
|
+ "does not belong to any file.", strBlockReportId, fullBrLeaseId, b,
|
|
|
|
+ node, b.getNumBytes());
|
|
}
|
|
}
|
|
}
|
|
}
|
|
|
|
|
|
@@ -2951,9 +2942,10 @@ public class BlockManager implements BlockStatsMXBean {
|
|
node.setLastBlockReportTime(now());
|
|
node.setLastBlockReportTime(now());
|
|
node.setLastBlockReportMonotonic(Time.monotonicNow());
|
|
node.setLastBlockReportMonotonic(Time.monotonicNow());
|
|
}
|
|
}
|
|
- LOG.debug("Processing RPC with index {} out of total {} RPCs in "
|
|
|
|
- + "processReport 0x{}", context.getCurRpc(),
|
|
|
|
- context.getTotalRpcs(), Long.toHexString(context.getReportId()));
|
|
|
|
|
|
+ if (LOG.isDebugEnabled()) {
|
|
|
|
+ LOG.debug("Processing RPC with index {} out of total {} RPCs in processReport 0x{}",
|
|
|
|
+ context.getCurRpc(), context.getTotalRpcs(), Long.toHexString(context.getReportId()));
|
|
|
|
+ }
|
|
}
|
|
}
|
|
} finally {
|
|
} finally {
|
|
namesystem.writeUnlock("removeBRLeaseIfNeeded");
|
|
namesystem.writeUnlock("removeBRLeaseIfNeeded");
|
|
@@ -2978,14 +2970,16 @@ public class BlockManager implements BlockStatsMXBean {
|
|
|
|
|
|
BlockInfo bi = getStoredBlock(b);
|
|
BlockInfo bi = getStoredBlock(b);
|
|
if (bi == null) {
|
|
if (bi == null) {
|
|
- LOG.debug("BLOCK* rescanPostponedMisreplicatedBlocks: " +
|
|
|
|
- "Postponed mis-replicated block {} no longer found " +
|
|
|
|
- "in block map.", b);
|
|
|
|
|
|
+ if (LOG.isDebugEnabled()) {
|
|
|
|
+ LOG.debug("BLOCK* rescanPostponedMisreplicatedBlocks: " +
|
|
|
|
+ "Postponed mis-replicated block {} no longer found " +
|
|
|
|
+ "in block map.", b);
|
|
|
|
+ }
|
|
continue;
|
|
continue;
|
|
}
|
|
}
|
|
MisReplicationResult res = processMisReplicatedBlock(bi);
|
|
MisReplicationResult res = processMisReplicatedBlock(bi);
|
|
- LOG.debug("BLOCK* rescanPostponedMisreplicatedBlocks: " +
|
|
|
|
- "Re-scanned block {}, result is {}", b, res);
|
|
|
|
|
|
+ LOG.debug("BLOCK* rescanPostponedMisreplicatedBlocks: Re-scanned block {}, result is {}",
|
|
|
|
+ b, res);
|
|
if (res == MisReplicationResult.POSTPONE) {
|
|
if (res == MisReplicationResult.POSTPONE) {
|
|
rescannedMisreplicatedBlocks.add(b);
|
|
rescannedMisreplicatedBlocks.add(b);
|
|
}
|
|
}
|
|
@@ -3077,7 +3071,7 @@ public class BlockManager implements BlockStatsMXBean {
|
|
}
|
|
}
|
|
}
|
|
}
|
|
if (isCorrupt) {
|
|
if (isCorrupt) {
|
|
- if(blockLog.isDebugEnabled()) {
|
|
|
|
|
|
+ if (blockLog.isDebugEnabled()) {
|
|
blockLog.debug("BLOCK* markBlockReplicasAsCorrupt: mark block replica" +
|
|
blockLog.debug("BLOCK* markBlockReplicasAsCorrupt: mark block replica" +
|
|
" {} on {} as corrupt because the dn is not in the new committed " +
|
|
" {} on {} as corrupt because the dn is not in the new committed " +
|
|
"storage list.", b, storage.getDatanodeDescriptor());
|
|
"storage list.", b, storage.getDatanodeDescriptor());
|
|
@@ -3113,6 +3107,7 @@ public class BlockManager implements BlockStatsMXBean {
|
|
iblk.getBlockName(), storageInfo.getDatanodeDescriptor(),
|
|
iblk.getBlockName(), storageInfo.getDatanodeDescriptor(),
|
|
iblk.getNumBytes(), reportedState);
|
|
iblk.getNumBytes(), reportedState);
|
|
}
|
|
}
|
|
|
|
+
|
|
if (shouldPostponeBlocksFromFuture && isGenStampInFuture(iblk)) {
|
|
if (shouldPostponeBlocksFromFuture && isGenStampInFuture(iblk)) {
|
|
queueReportedBlock(storageInfo, iblk, reportedState,
|
|
queueReportedBlock(storageInfo, iblk, reportedState,
|
|
QUEUE_REASON_FUTURE_GENSTAMP);
|
|
QUEUE_REASON_FUTURE_GENSTAMP);
|
|
@@ -3329,9 +3324,11 @@ public class BlockManager implements BlockStatsMXBean {
|
|
ReplicaState reportedState, String reason) {
|
|
ReplicaState reportedState, String reason) {
|
|
assert shouldPostponeBlocksFromFuture;
|
|
assert shouldPostponeBlocksFromFuture;
|
|
|
|
|
|
- LOG.debug("Queueing reported block {} in state {}" +
|
|
|
|
- " from datanode {} for later processing because {}.",
|
|
|
|
- block, reportedState, storageInfo.getDatanodeDescriptor(), reason);
|
|
|
|
|
|
+ if (LOG.isDebugEnabled()) {
|
|
|
|
+ LOG.debug("Queueing reported block {} in state {}" +
|
|
|
|
+ " from datanode {} for later processing because {}.",
|
|
|
|
+ block, reportedState, storageInfo.getDatanodeDescriptor(), reason);
|
|
|
|
+ }
|
|
pendingDNMessages.enqueueReportedBlock(storageInfo, block, reportedState);
|
|
pendingDNMessages.enqueueReportedBlock(storageInfo, block, reportedState);
|
|
}
|
|
}
|
|
|
|
|
|
@@ -3598,11 +3595,8 @@ public class BlockManager implements BlockStatsMXBean {
|
|
}
|
|
}
|
|
if (storedBlock == null || storedBlock.isDeleted()) {
|
|
if (storedBlock == null || storedBlock.isDeleted()) {
|
|
// If this block does not belong to anyfile, then we are done.
|
|
// If this block does not belong to anyfile, then we are done.
|
|
- if(blockLog.isDebugEnabled()) {
|
|
|
|
- blockLog.debug("BLOCK* addStoredBlock: {} on {} size {} but it does not" +
|
|
|
|
- " belong to any file", block, node, block.getNumBytes());
|
|
|
|
- }
|
|
|
|
-
|
|
|
|
|
|
+ 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.
|
|
// we could add this block to invalidate set of this datanode.
|
|
// it will happen in next block report otherwise.
|
|
// it will happen in next block report otherwise.
|
|
return block;
|
|
return block;
|
|
@@ -3630,7 +3624,7 @@ public class BlockManager implements BlockStatsMXBean {
|
|
corruptReplicas.removeFromCorruptReplicasMap(block, node,
|
|
corruptReplicas.removeFromCorruptReplicasMap(block, node,
|
|
Reason.GENSTAMP_MISMATCH);
|
|
Reason.GENSTAMP_MISMATCH);
|
|
curReplicaDelta = 0;
|
|
curReplicaDelta = 0;
|
|
- if(blockLog.isDebugEnabled()) {
|
|
|
|
|
|
+ if (blockLog.isDebugEnabled()) {
|
|
blockLog.debug("BLOCK* addStoredBlock: Redundant addStoredBlock request"
|
|
blockLog.debug("BLOCK* addStoredBlock: Redundant addStoredBlock request"
|
|
+ " received for {} on node {} size {}", storedBlock, node,
|
|
+ " received for {} on node {} size {}", storedBlock, node,
|
|
storedBlock.getNumBytes());
|
|
storedBlock.getNumBytes());
|
|
@@ -3735,10 +3729,8 @@ public class BlockManager implements BlockStatsMXBean {
|
|
removedFromBlocksMap = false;
|
|
removedFromBlocksMap = false;
|
|
}
|
|
}
|
|
} catch (IOException e) {
|
|
} catch (IOException e) {
|
|
- if(blockLog.isDebugEnabled()) {
|
|
|
|
- blockLog.debug("invalidateCorruptReplicas error in deleting bad block"
|
|
|
|
- + " {} on {}", blk, node, e);
|
|
|
|
- }
|
|
|
|
|
|
+ blockLog.debug("invalidateCorruptReplicas error in deleting bad block {} on {}",
|
|
|
|
+ blk, node, e);
|
|
removedFromBlocksMap = false;
|
|
removedFromBlocksMap = false;
|
|
}
|
|
}
|
|
}
|
|
}
|
|
@@ -3920,8 +3912,8 @@ public class BlockManager implements BlockStatsMXBean {
|
|
BlockInfo blk = iter.next();
|
|
BlockInfo blk = iter.next();
|
|
MisReplicationResult r = processMisReplicatedBlock(blk);
|
|
MisReplicationResult r = processMisReplicatedBlock(blk);
|
|
processed++;
|
|
processed++;
|
|
- LOG.debug("BLOCK* processMisReplicatedBlocks: " +
|
|
|
|
- "Re-scanned block {}, result is {}", blk, r);
|
|
|
|
|
|
+ LOG.debug("BLOCK* processMisReplicatedBlocks: Re-scanned block {}, result is {}",
|
|
|
|
+ blk, r);
|
|
}
|
|
}
|
|
} finally {
|
|
} finally {
|
|
namesystem.writeUnlock("processMisReplicatedBlocks");
|
|
namesystem.writeUnlock("processMisReplicatedBlocks");
|
|
@@ -4187,10 +4179,8 @@ public class BlockManager implements BlockStatsMXBean {
|
|
//
|
|
//
|
|
final Block blockToInvalidate = getBlockOnStorage(storedBlock, chosen);
|
|
final Block blockToInvalidate = getBlockOnStorage(storedBlock, chosen);
|
|
addToInvalidates(blockToInvalidate, chosen.getDatanodeDescriptor());
|
|
addToInvalidates(blockToInvalidate, chosen.getDatanodeDescriptor());
|
|
- if(blockLog.isDebugEnabled()) {
|
|
|
|
- blockLog.debug("BLOCK* chooseExcessRedundancies: "
|
|
|
|
- + "({}, {}) is added to invalidated blocks set", chosen, storedBlock);
|
|
|
|
- }
|
|
|
|
|
|
+ blockLog.debug("BLOCK* chooseExcessRedundancies: ({}, {}) is added to invalidated blocks set",
|
|
|
|
+ chosen, storedBlock);
|
|
}
|
|
}
|
|
|
|
|
|
private void removeStoredBlock(DatanodeStorageInfo storageInfo, Block block,
|
|
private void removeStoredBlock(DatanodeStorageInfo storageInfo, Block block,
|
|
@@ -4212,8 +4202,8 @@ public class BlockManager implements BlockStatsMXBean {
|
|
assert (namesystem.hasWriteLock());
|
|
assert (namesystem.hasWriteLock());
|
|
{
|
|
{
|
|
if (storedBlock == null || !blocksMap.removeNode(storedBlock, node)) {
|
|
if (storedBlock == null || !blocksMap.removeNode(storedBlock, node)) {
|
|
- blockLog.debug("BLOCK* removeStoredBlock: {} has already been" +
|
|
|
|
- " removed from node {}", storedBlock, node);
|
|
|
|
|
|
+ blockLog.debug("BLOCK* removeStoredBlock: {} has already been removed from node {}",
|
|
|
|
+ storedBlock, node);
|
|
return;
|
|
return;
|
|
}
|
|
}
|
|
|
|
|
|
@@ -4225,8 +4215,10 @@ public class BlockManager implements BlockStatsMXBean {
|
|
removed |= node.getCached().remove(cblock);
|
|
removed |= node.getCached().remove(cblock);
|
|
removed |= node.getPendingUncached().remove(cblock);
|
|
removed |= node.getPendingUncached().remove(cblock);
|
|
if (removed) {
|
|
if (removed) {
|
|
- blockLog.debug("BLOCK* removeStoredBlock: {} removed from caching "
|
|
|
|
- + "related lists on node {}", storedBlock, node);
|
|
|
|
|
|
+ if (blockLog.isDebugEnabled()) {
|
|
|
|
+ blockLog.debug("BLOCK* removeStoredBlock: {} removed from caching "
|
|
|
|
+ + "related lists on node {}", storedBlock, node);
|
|
|
|
+ }
|
|
}
|
|
}
|
|
}
|
|
}
|
|
|
|
|
|
@@ -4251,8 +4243,9 @@ public class BlockManager implements BlockStatsMXBean {
|
|
for (ReplicaUnderConstruction r : staleReplicas) {
|
|
for (ReplicaUnderConstruction r : staleReplicas) {
|
|
removeStoredBlock(block,
|
|
removeStoredBlock(block,
|
|
r.getExpectedStorageLocation().getDatanodeDescriptor());
|
|
r.getExpectedStorageLocation().getDatanodeDescriptor());
|
|
- blockLog.debug("BLOCK* Removing stale replica {} of {}", r,
|
|
|
|
- Block.toString(r));
|
|
|
|
|
|
+ if (blockLog.isDebugEnabled()) {
|
|
|
|
+ blockLog.debug("BLOCK* Removing stale replica {} of {}", r, Block.toString(r));
|
|
|
|
+ }
|
|
}
|
|
}
|
|
}
|
|
}
|
|
/**
|
|
/**
|
|
@@ -4380,10 +4373,8 @@ public class BlockManager implements BlockStatsMXBean {
|
|
maxNumBlocksToLog, numBlocksLogged);
|
|
maxNumBlocksToLog, numBlocksLogged);
|
|
}
|
|
}
|
|
for (Block b : toInvalidate) {
|
|
for (Block b : toInvalidate) {
|
|
- if(blockLog.isDebugEnabled()) {
|
|
|
|
- blockLog.debug("BLOCK* addBlock: block {} on node {} size {} does not " +
|
|
|
|
- "belong to any file", b, node, b.getNumBytes());
|
|
|
|
- }
|
|
|
|
|
|
+ blockLog.debug("BLOCK* addBlock: block {} on node {} size {} does not belong to any file",
|
|
|
|
+ b, node, b.getNumBytes());
|
|
addToInvalidates(b, node);
|
|
addToInvalidates(b, node);
|
|
}
|
|
}
|
|
for (BlockToMarkCorrupt b : toCorrupt) {
|
|
for (BlockToMarkCorrupt b : toCorrupt) {
|
|
@@ -4464,7 +4455,7 @@ public class BlockManager implements BlockStatsMXBean {
|
|
blockLog.debug("BLOCK* block {}: {} is received from {}",
|
|
blockLog.debug("BLOCK* block {}: {} is received from {}",
|
|
rdbi.getStatus(), rdbi.getBlock(), node);
|
|
rdbi.getStatus(), rdbi.getBlock(), node);
|
|
}
|
|
}
|
|
- if(blockLog.isDebugEnabled()) {
|
|
|
|
|
|
+ if (blockLog.isDebugEnabled()) {
|
|
blockLog.debug("*BLOCK* NameNode.processIncrementalBlockReport: from "
|
|
blockLog.debug("*BLOCK* NameNode.processIncrementalBlockReport: from "
|
|
+ "{} receiving: {}, received: {}, deleted: {}", node, receiving,
|
|
+ "{} receiving: {}, received: {}, deleted: {}", node, receiving,
|
|
received, deleted);
|
|
received, deleted);
|
|
@@ -4843,8 +4834,10 @@ public class BlockManager implements BlockStatsMXBean {
|
|
} finally {
|
|
} finally {
|
|
namesystem.writeUnlock("invalidateWorkForOneNode");
|
|
namesystem.writeUnlock("invalidateWorkForOneNode");
|
|
}
|
|
}
|
|
- blockLog.debug("BLOCK* {}: ask {} to delete {}", getClass().getSimpleName(),
|
|
|
|
- dn, toInvalidate);
|
|
|
|
|
|
+ if (blockLog.isDebugEnabled()) {
|
|
|
|
+ blockLog.debug("BLOCK* {}: ask {} to delete {}",
|
|
|
|
+ getClass().getSimpleName(), dn, toInvalidate);
|
|
|
|
+ }
|
|
return toInvalidate.size();
|
|
return toInvalidate.size();
|
|
}
|
|
}
|
|
|
|
|
|
@@ -5110,8 +5103,8 @@ public class BlockManager implements BlockStatsMXBean {
|
|
}
|
|
}
|
|
}
|
|
}
|
|
if (isSleep) {
|
|
if (isSleep) {
|
|
- LOG.debug("Clear markedDeleteQueue over {}" +
|
|
|
|
- " millisecond to release the write lock", deleteBlockLockTimeMs);
|
|
|
|
|
|
+ LOG.debug("Clear markedDeleteQueue over {} millisecond to release the write lock",
|
|
|
|
+ deleteBlockLockTimeMs);
|
|
}
|
|
}
|
|
try {
|
|
try {
|
|
Thread.sleep(deleteBlockUnlockIntervalTimeMs);
|
|
Thread.sleep(deleteBlockUnlockIntervalTimeMs);
|