|
@@ -57,6 +57,7 @@ import org.apache.hadoop.hdfs.security.token.block.BlockTokenSecretManager;
|
|
|
import org.apache.hadoop.hdfs.security.token.block.BlockTokenSecretManager.AccessMode;
|
|
|
import org.apache.hadoop.hdfs.security.token.block.DataEncryptionKey;
|
|
|
import org.apache.hadoop.hdfs.security.token.block.ExportedBlockKeys;
|
|
|
+import org.apache.hadoop.hdfs.server.blockmanagement.CorruptReplicasMap.Reason;
|
|
|
import org.apache.hadoop.hdfs.server.blockmanagement.PendingDataNodeMessages.ReportedBlockInfo;
|
|
|
import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.BlockUCState;
|
|
|
import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.ReplicaState;
|
|
@@ -1046,7 +1047,8 @@ public class BlockManager {
|
|
|
+ blk + " not found");
|
|
|
return;
|
|
|
}
|
|
|
- markBlockAsCorrupt(new BlockToMarkCorrupt(storedBlock, reason), dn);
|
|
|
+ markBlockAsCorrupt(new BlockToMarkCorrupt(storedBlock, reason,
|
|
|
+ Reason.CORRUPTION_REPORTED), dn);
|
|
|
}
|
|
|
|
|
|
private void markBlockAsCorrupt(BlockToMarkCorrupt b,
|
|
@@ -1069,7 +1071,8 @@ public class BlockManager {
|
|
|
node.addBlock(b.stored);
|
|
|
|
|
|
// Add this replica to corruptReplicas Map
|
|
|
- corruptReplicas.addToCorruptReplicasMap(b.corrupted, node, b.reason);
|
|
|
+ corruptReplicas.addToCorruptReplicasMap(b.corrupted, node, b.reason,
|
|
|
+ b.reasonCode);
|
|
|
if (countNodes(b.stored).liveReplicas() >= bc.getBlockReplication()) {
|
|
|
// the block is over-replicated so invalidate the replicas immediately
|
|
|
invalidateBlock(b, node);
|
|
@@ -1570,22 +1573,27 @@ public class BlockManager {
|
|
|
final BlockInfo stored;
|
|
|
/** The reason to mark corrupt. */
|
|
|
final String reason;
|
|
|
-
|
|
|
- BlockToMarkCorrupt(BlockInfo corrupted, BlockInfo stored, String reason) {
|
|
|
+ /** The reason code to be stored */
|
|
|
+ final Reason reasonCode;
|
|
|
+
|
|
|
+ BlockToMarkCorrupt(BlockInfo corrupted, BlockInfo stored, String reason,
|
|
|
+ Reason reasonCode) {
|
|
|
Preconditions.checkNotNull(corrupted, "corrupted is null");
|
|
|
Preconditions.checkNotNull(stored, "stored is null");
|
|
|
|
|
|
this.corrupted = corrupted;
|
|
|
this.stored = stored;
|
|
|
this.reason = reason;
|
|
|
+ this.reasonCode = reasonCode;
|
|
|
}
|
|
|
|
|
|
- BlockToMarkCorrupt(BlockInfo stored, String reason) {
|
|
|
- this(stored, stored, reason);
|
|
|
+ BlockToMarkCorrupt(BlockInfo stored, String reason, Reason reasonCode) {
|
|
|
+ this(stored, stored, reason, reasonCode);
|
|
|
}
|
|
|
|
|
|
- BlockToMarkCorrupt(BlockInfo stored, long gs, String reason) {
|
|
|
- this(new BlockInfo(stored), stored, reason);
|
|
|
+ BlockToMarkCorrupt(BlockInfo stored, long gs, String reason,
|
|
|
+ Reason reasonCode) {
|
|
|
+ this(new BlockInfo(stored), stored, reason, reasonCode);
|
|
|
//the corrupted block in datanode has a different generation stamp
|
|
|
corrupted.setGenerationStamp(gs);
|
|
|
}
|
|
@@ -1930,9 +1938,11 @@ assert storedBlock.findDatanode(dn) < 0 : "Block " + block
|
|
|
return storedBlock;
|
|
|
}
|
|
|
|
|
|
- //add replica if appropriate
|
|
|
+ // Add replica if appropriate. If the replica was previously corrupt
|
|
|
+ // but now okay, it might need to be updated.
|
|
|
if (reportedState == ReplicaState.FINALIZED
|
|
|
- && storedBlock.findDatanode(dn) < 0) {
|
|
|
+ && (storedBlock.findDatanode(dn) < 0
|
|
|
+ || corruptReplicas.isReplicaCorrupt(storedBlock, dn))) {
|
|
|
toAdd.add(storedBlock);
|
|
|
}
|
|
|
return storedBlock;
|
|
@@ -2023,12 +2033,13 @@ assert storedBlock.findDatanode(dn) < 0 : "Block " + block
|
|
|
return new BlockToMarkCorrupt(storedBlock, reportedGS,
|
|
|
"block is " + ucState + " and reported genstamp " + reportedGS
|
|
|
+ " does not match genstamp in block map "
|
|
|
- + storedBlock.getGenerationStamp());
|
|
|
+ + storedBlock.getGenerationStamp(), Reason.GENSTAMP_MISMATCH);
|
|
|
} else if (storedBlock.getNumBytes() != reported.getNumBytes()) {
|
|
|
return new BlockToMarkCorrupt(storedBlock,
|
|
|
"block is " + ucState + " and reported length " +
|
|
|
reported.getNumBytes() + " does not match " +
|
|
|
- "length in block map " + storedBlock.getNumBytes());
|
|
|
+ "length in block map " + storedBlock.getNumBytes(),
|
|
|
+ Reason.SIZE_MISMATCH);
|
|
|
} else {
|
|
|
return null; // not corrupt
|
|
|
}
|
|
@@ -2044,7 +2055,7 @@ assert storedBlock.findDatanode(dn) < 0 : "Block " + block
|
|
|
return new BlockToMarkCorrupt(storedBlock, reportedGS,
|
|
|
"reported " + reportedState + " replica with genstamp " + reportedGS
|
|
|
+ " does not match COMPLETE block's genstamp in block map "
|
|
|
- + storedBlock.getGenerationStamp());
|
|
|
+ + storedBlock.getGenerationStamp(), Reason.GENSTAMP_MISMATCH);
|
|
|
} else { // COMPLETE block, same genstamp
|
|
|
if (reportedState == ReplicaState.RBW) {
|
|
|
// If it's a RBW report for a COMPLETE block, it may just be that
|
|
@@ -2057,7 +2068,8 @@ assert storedBlock.findDatanode(dn) < 0 : "Block " + block
|
|
|
return null;
|
|
|
} else {
|
|
|
return new BlockToMarkCorrupt(storedBlock,
|
|
|
- "reported replica has invalid state " + reportedState);
|
|
|
+ "reported replica has invalid state " + reportedState,
|
|
|
+ Reason.INVALID_STATE);
|
|
|
}
|
|
|
}
|
|
|
case RUR: // should not be reported
|
|
@@ -2068,7 +2080,7 @@ assert storedBlock.findDatanode(dn) < 0 : "Block " + block
|
|
|
" on " + dn + " size " + storedBlock.getNumBytes();
|
|
|
// log here at WARN level since this is really a broken HDFS invariant
|
|
|
LOG.warn(msg);
|
|
|
- return new BlockToMarkCorrupt(storedBlock, msg);
|
|
|
+ return new BlockToMarkCorrupt(storedBlock, msg, Reason.INVALID_STATE);
|
|
|
}
|
|
|
}
|
|
|
|
|
@@ -2184,6 +2196,11 @@ assert storedBlock.findDatanode(dn) < 0 : "Block " + block
|
|
|
logAddStoredBlock(storedBlock, node);
|
|
|
}
|
|
|
} else {
|
|
|
+ // if the same block is added again and the replica was corrupt
|
|
|
+ // previously because of a wrong gen stamp, remove it from the
|
|
|
+ // corrupt block list.
|
|
|
+ corruptReplicas.removeFromCorruptReplicasMap(block, node,
|
|
|
+ Reason.GENSTAMP_MISMATCH);
|
|
|
curReplicaDelta = 0;
|
|
|
blockLog.warn("BLOCK* addStoredBlock: "
|
|
|
+ "Redundant addStoredBlock request received for " + storedBlock
|
|
@@ -2280,7 +2297,8 @@ assert storedBlock.findDatanode(dn) < 0 : "Block " + block
|
|
|
DatanodeDescriptor[] nodesCopy = nodes.toArray(new DatanodeDescriptor[0]);
|
|
|
for (DatanodeDescriptor node : nodesCopy) {
|
|
|
try {
|
|
|
- if (!invalidateBlock(new BlockToMarkCorrupt(blk, null), node)) {
|
|
|
+ if (!invalidateBlock(new BlockToMarkCorrupt(blk, null,
|
|
|
+ Reason.ANY), node)) {
|
|
|
removedFromBlocksMap = false;
|
|
|
}
|
|
|
} catch (IOException e) {
|