|
@@ -807,9 +807,11 @@ public class BlockManager {
|
|
|
* Mark the block belonging to datanode as corrupt
|
|
|
* @param blk Block to be marked as corrupt
|
|
|
* @param dn Datanode which holds the corrupt replica
|
|
|
+ * @param reason a textual reason why the block should be marked corrupt,
|
|
|
+ * for logging purposes
|
|
|
*/
|
|
|
public void findAndMarkBlockAsCorrupt(final ExtendedBlock blk,
|
|
|
- final DatanodeInfo dn) throws IOException {
|
|
|
+ final DatanodeInfo dn, String reason) throws IOException {
|
|
|
namesystem.writeLock();
|
|
|
try {
|
|
|
final BlockInfo storedBlock = getStoredBlock(blk.getLocalBlock());
|
|
@@ -822,14 +824,15 @@ public class BlockManager {
|
|
|
+ blk + " not found.");
|
|
|
return;
|
|
|
}
|
|
|
- markBlockAsCorrupt(storedBlock, dn);
|
|
|
+ markBlockAsCorrupt(storedBlock, dn, reason);
|
|
|
} finally {
|
|
|
namesystem.writeUnlock();
|
|
|
}
|
|
|
}
|
|
|
|
|
|
private void markBlockAsCorrupt(BlockInfo storedBlock,
|
|
|
- DatanodeInfo dn) throws IOException {
|
|
|
+ DatanodeInfo dn,
|
|
|
+ String reason) throws IOException {
|
|
|
assert storedBlock != null : "storedBlock should not be null";
|
|
|
DatanodeDescriptor node = getDatanodeManager().getDatanode(dn);
|
|
|
if (node == null) {
|
|
@@ -853,7 +856,7 @@ public class BlockManager {
|
|
|
node.addBlock(storedBlock);
|
|
|
|
|
|
// Add this replica to corruptReplicas Map
|
|
|
- corruptReplicas.addToCorruptReplicasMap(storedBlock, node);
|
|
|
+ corruptReplicas.addToCorruptReplicasMap(storedBlock, node, reason);
|
|
|
if (countNodes(storedBlock).liveReplicas() >= inode.getReplication()) {
|
|
|
// the block is over-replicated so invalidate the replicas immediately
|
|
|
invalidateBlock(storedBlock, node);
|
|
@@ -1315,6 +1318,21 @@ public class BlockManager {
|
|
|
this.reportedState = reportedState;
|
|
|
}
|
|
|
}
|
|
|
+
|
|
|
+ /**
|
|
|
+ * BlockToMarkCorrupt is used to build the "toCorrupt" list, which is a
|
|
|
+ * list of blocks that should be considered corrupt due to a block report.
|
|
|
+ */
|
|
|
+ private static class BlockToMarkCorrupt {
|
|
|
+ final BlockInfo blockInfo;
|
|
|
+ final String reason;
|
|
|
+
|
|
|
+ BlockToMarkCorrupt(BlockInfo blockInfo, String reason) {
|
|
|
+ super();
|
|
|
+ this.blockInfo = blockInfo;
|
|
|
+ this.reason = reason;
|
|
|
+ }
|
|
|
+ }
|
|
|
|
|
|
/**
|
|
|
* The given datanode is reporting all its blocks.
|
|
@@ -1369,7 +1387,7 @@ public class BlockManager {
|
|
|
Collection<BlockInfo> toAdd = new LinkedList<BlockInfo>();
|
|
|
Collection<Block> toRemove = new LinkedList<Block>();
|
|
|
Collection<Block> toInvalidate = new LinkedList<Block>();
|
|
|
- Collection<BlockInfo> toCorrupt = new LinkedList<BlockInfo>();
|
|
|
+ Collection<BlockToMarkCorrupt> toCorrupt = new LinkedList<BlockToMarkCorrupt>();
|
|
|
Collection<StatefulBlockInfo> toUC = new LinkedList<StatefulBlockInfo>();
|
|
|
reportDiff(node, report, toAdd, toRemove, toInvalidate, toCorrupt, toUC);
|
|
|
|
|
@@ -1389,8 +1407,8 @@ public class BlockManager {
|
|
|
+ " does not belong to any file.");
|
|
|
addToInvalidates(b, node);
|
|
|
}
|
|
|
- for (BlockInfo b : toCorrupt) {
|
|
|
- markBlockAsCorrupt(b, node);
|
|
|
+ for (BlockToMarkCorrupt b : toCorrupt) {
|
|
|
+ markBlockAsCorrupt(b.blockInfo, node, b.reason);
|
|
|
}
|
|
|
}
|
|
|
|
|
@@ -1421,8 +1439,10 @@ public class BlockManager {
|
|
|
|
|
|
// If block is corrupt, mark it and continue to next block.
|
|
|
BlockUCState ucState = storedBlock.getBlockUCState();
|
|
|
- if (isReplicaCorrupt(iblk, reportedState, storedBlock, ucState, node)) {
|
|
|
- markBlockAsCorrupt(storedBlock, node);
|
|
|
+ BlockToMarkCorrupt c = checkReplicaCorrupt(
|
|
|
+ iblk, reportedState, storedBlock, ucState, node);
|
|
|
+ if (c != null) {
|
|
|
+ markBlockAsCorrupt(c.blockInfo, node, c.reason);
|
|
|
continue;
|
|
|
}
|
|
|
|
|
@@ -1444,7 +1464,7 @@ public class BlockManager {
|
|
|
Collection<BlockInfo> toAdd, // add to DatanodeDescriptor
|
|
|
Collection<Block> toRemove, // remove from DatanodeDescriptor
|
|
|
Collection<Block> toInvalidate, // should be removed from DN
|
|
|
- Collection<BlockInfo> toCorrupt, // add to corrupt replicas list
|
|
|
+ Collection<BlockToMarkCorrupt> toCorrupt, // add to corrupt replicas list
|
|
|
Collection<StatefulBlockInfo> toUC) { // add to under-construction list
|
|
|
// place a delimiter in the list which separates blocks
|
|
|
// that have been reported from those that have not
|
|
@@ -1507,7 +1527,7 @@ public class BlockManager {
|
|
|
final Block block, final ReplicaState reportedState,
|
|
|
final Collection<BlockInfo> toAdd,
|
|
|
final Collection<Block> toInvalidate,
|
|
|
- final Collection<BlockInfo> toCorrupt,
|
|
|
+ final Collection<BlockToMarkCorrupt> toCorrupt,
|
|
|
final Collection<StatefulBlockInfo> toUC) {
|
|
|
|
|
|
if(LOG.isDebugEnabled()) {
|
|
@@ -1538,8 +1558,10 @@ public class BlockManager {
|
|
|
return storedBlock;
|
|
|
}
|
|
|
|
|
|
- if (isReplicaCorrupt(block, reportedState, storedBlock, ucState, dn)) {
|
|
|
- toCorrupt.add(storedBlock);
|
|
|
+ BlockToMarkCorrupt c = checkReplicaCorrupt(
|
|
|
+ block, reportedState, storedBlock, ucState, dn);
|
|
|
+ if (c != null) {
|
|
|
+ toCorrupt.add(c);
|
|
|
return storedBlock;
|
|
|
}
|
|
|
|
|
@@ -1563,8 +1585,11 @@ public class BlockManager {
|
|
|
* as switch statements, on the theory that it is easier to understand
|
|
|
* the combinatorics of reportedState and ucState that way. It should be
|
|
|
* at least as efficient as boolean expressions.
|
|
|
+ *
|
|
|
+ * @return a BlockToMarkCorrupt object, or null if the replica is not corrupt
|
|
|
*/
|
|
|
- private boolean isReplicaCorrupt(Block iblk, ReplicaState reportedState,
|
|
|
+ private BlockToMarkCorrupt checkReplicaCorrupt(
|
|
|
+ Block iblk, ReplicaState reportedState,
|
|
|
BlockInfo storedBlock, BlockUCState ucState,
|
|
|
DatanodeDescriptor dn) {
|
|
|
switch(reportedState) {
|
|
@@ -1572,17 +1597,31 @@ public class BlockManager {
|
|
|
switch(ucState) {
|
|
|
case COMPLETE:
|
|
|
case COMMITTED:
|
|
|
- return (storedBlock.getGenerationStamp() != iblk.getGenerationStamp()
|
|
|
- || storedBlock.getNumBytes() != iblk.getNumBytes());
|
|
|
+ if (storedBlock.getGenerationStamp() != iblk.getGenerationStamp()) {
|
|
|
+ return new BlockToMarkCorrupt(storedBlock,
|
|
|
+ "block is " + ucState + " and reported genstamp " +
|
|
|
+ iblk.getGenerationStamp() + " does not match " +
|
|
|
+ "genstamp in block map " + storedBlock.getGenerationStamp());
|
|
|
+ } else if (storedBlock.getNumBytes() != iblk.getNumBytes()) {
|
|
|
+ return new BlockToMarkCorrupt(storedBlock,
|
|
|
+ "block is " + ucState + " and reported length " +
|
|
|
+ iblk.getNumBytes() + " does not match " +
|
|
|
+ "length in block map " + storedBlock.getNumBytes());
|
|
|
+ } else {
|
|
|
+ return null; // not corrupt
|
|
|
+ }
|
|
|
default:
|
|
|
- return false;
|
|
|
+ return null;
|
|
|
}
|
|
|
case RBW:
|
|
|
case RWR:
|
|
|
if (!storedBlock.isComplete()) {
|
|
|
- return false;
|
|
|
+ return null; // not corrupt
|
|
|
} else if (storedBlock.getGenerationStamp() != iblk.getGenerationStamp()) {
|
|
|
- return true;
|
|
|
+ return new BlockToMarkCorrupt(storedBlock,
|
|
|
+ "reported " + reportedState + " replica with genstamp " +
|
|
|
+ iblk.getGenerationStamp() + " does not match COMPLETE block's " +
|
|
|
+ "genstamp in block map " + storedBlock.getGenerationStamp());
|
|
|
} else { // COMPLETE block, same genstamp
|
|
|
if (reportedState == ReplicaState.RBW) {
|
|
|
// If it's a RBW report for a COMPLETE block, it may just be that
|
|
@@ -1592,18 +1631,22 @@ public class BlockManager {
|
|
|
LOG.info("Received an RBW replica for block " + storedBlock +
|
|
|
" on " + dn.getName() + ": ignoring it, since the block is " +
|
|
|
"complete with the same generation stamp.");
|
|
|
- return false;
|
|
|
+ return null;
|
|
|
} else {
|
|
|
- return true;
|
|
|
+ return new BlockToMarkCorrupt(storedBlock,
|
|
|
+ "reported replica has invalid state " + reportedState);
|
|
|
}
|
|
|
}
|
|
|
case RUR: // should not be reported
|
|
|
case TEMPORARY: // should not be reported
|
|
|
default:
|
|
|
- LOG.warn("Unexpected replica state " + reportedState
|
|
|
- + " for block: " + storedBlock +
|
|
|
- " on " + dn.getName() + " size " + storedBlock.getNumBytes());
|
|
|
- return true;
|
|
|
+ String msg = "Unexpected replica state " + reportedState
|
|
|
+ + " for block: " + storedBlock +
|
|
|
+ " on " + dn.getName() + " size " + storedBlock.getNumBytes();
|
|
|
+ // log here at WARN level since this is really a broken HDFS
|
|
|
+ // invariant
|
|
|
+ LOG.warn(msg);
|
|
|
+ return new BlockToMarkCorrupt(storedBlock, msg);
|
|
|
}
|
|
|
}
|
|
|
|
|
@@ -2134,7 +2177,7 @@ public class BlockManager {
|
|
|
// blockReceived reports a finalized block
|
|
|
Collection<BlockInfo> toAdd = new LinkedList<BlockInfo>();
|
|
|
Collection<Block> toInvalidate = new LinkedList<Block>();
|
|
|
- Collection<BlockInfo> toCorrupt = new LinkedList<BlockInfo>();
|
|
|
+ Collection<BlockToMarkCorrupt> toCorrupt = new LinkedList<BlockToMarkCorrupt>();
|
|
|
Collection<StatefulBlockInfo> toUC = new LinkedList<StatefulBlockInfo>();
|
|
|
processReportedBlock(node, block, ReplicaState.FINALIZED,
|
|
|
toAdd, toInvalidate, toCorrupt, toUC);
|
|
@@ -2155,8 +2198,8 @@ public class BlockManager {
|
|
|
+ " does not belong to any file.");
|
|
|
addToInvalidates(b, node);
|
|
|
}
|
|
|
- for (BlockInfo b : toCorrupt) {
|
|
|
- markBlockAsCorrupt(b, node);
|
|
|
+ for (BlockToMarkCorrupt b : toCorrupt) {
|
|
|
+ markBlockAsCorrupt(b.blockInfo, node, b.reason);
|
|
|
}
|
|
|
}
|
|
|
|