|
@@ -1235,10 +1235,11 @@ public class BlockManager {
|
|
|
}
|
|
|
|
|
|
/**
|
|
|
- *
|
|
|
- * @param b
|
|
|
+ * Mark a replica (of a contiguous block) or an internal block (of a striped
|
|
|
+ * block group) as corrupt.
|
|
|
+ * @param b Indicating the reported bad block and the corresponding BlockInfo
|
|
|
+ * stored in blocksMap.
|
|
|
* @param storageInfo storage that contains the block, if known. null otherwise.
|
|
|
- * @throws IOException
|
|
|
*/
|
|
|
private void markBlockAsCorrupt(BlockToMarkCorrupt b,
|
|
|
DatanodeStorageInfo storageInfo,
|
|
@@ -1258,8 +1259,13 @@ public class BlockManager {
|
|
|
storageInfo.addBlock(b.stored, b.corrupted);
|
|
|
}
|
|
|
|
|
|
- // Add this replica to corruptReplicas Map
|
|
|
- corruptReplicas.addToCorruptReplicasMap(b.corrupted, node, b.reason,
|
|
|
+ // Add this replica to corruptReplicas Map. For striped blocks, we always
|
|
|
+ // use the id of whole striped block group when adding to corruptReplicas
|
|
|
+ Block corrupted = new Block(b.corrupted);
|
|
|
+ if (b.stored.isStriped()) {
|
|
|
+ corrupted.setBlockId(b.stored.getBlockId());
|
|
|
+ }
|
|
|
+ corruptReplicas.addToCorruptReplicasMap(corrupted, node, b.reason,
|
|
|
b.reasonCode);
|
|
|
|
|
|
NumberReplicas numberOfReplicas = countNodes(b.stored);
|
|
@@ -1283,7 +1289,7 @@ public class BlockManager {
|
|
|
if (hasEnoughLiveReplicas || hasMoreCorruptReplicas
|
|
|
|| corruptedDuringWrite) {
|
|
|
// the block is over-replicated so invalidate the replicas immediately
|
|
|
- invalidateBlock(b, node);
|
|
|
+ invalidateBlock(b, node, numberOfReplicas);
|
|
|
} else if (namesystem.isPopulatingReplQueues()) {
|
|
|
// add the block to neededReplication
|
|
|
updateNeededReplications(b.stored, -1, 0);
|
|
@@ -1295,8 +1301,8 @@ public class BlockManager {
|
|
|
* @return true if the block was successfully invalidated and no longer
|
|
|
* present in the BlocksMap
|
|
|
*/
|
|
|
- private boolean invalidateBlock(BlockToMarkCorrupt b, DatanodeInfo dn
|
|
|
- ) throws IOException {
|
|
|
+ private boolean invalidateBlock(BlockToMarkCorrupt b, DatanodeInfo dn,
|
|
|
+ NumberReplicas nr) throws IOException {
|
|
|
blockLog.info("BLOCK* invalidateBlock: {} on {}", b, dn);
|
|
|
DatanodeDescriptor node = getDatanodeManager().getDatanode(dn);
|
|
|
if (node == null) {
|
|
@@ -1305,7 +1311,6 @@ public class BlockManager {
|
|
|
}
|
|
|
|
|
|
// Check how many copies we have of the block
|
|
|
- NumberReplicas nr = countNodes(b.stored);
|
|
|
if (nr.replicasOnStaleNodes() > 0) {
|
|
|
blockLog.info("BLOCK* invalidateBlocks: postponing " +
|
|
|
"invalidation of {} on {} because {} replica(s) are located on " +
|
|
@@ -1313,17 +1318,14 @@ public class BlockManager {
|
|
|
nr.replicasOnStaleNodes());
|
|
|
postponeBlock(b.corrupted);
|
|
|
return false;
|
|
|
- } else if (nr.liveReplicas() >= 1) {
|
|
|
- // If we have at least one copy on a live node, then we can delete it.
|
|
|
+ } else {
|
|
|
+ // we already checked the number of replicas in the caller of this
|
|
|
+ // function and know there are enough live replicas, so we can delete it.
|
|
|
addToInvalidates(b.corrupted, dn);
|
|
|
removeStoredBlock(b.stored, node);
|
|
|
blockLog.debug("BLOCK* invalidateBlocks: {} on {} listed for deletion.",
|
|
|
b, dn);
|
|
|
return true;
|
|
|
- } else {
|
|
|
- blockLog.info("BLOCK* invalidateBlocks: {} on {} is the only copy and" +
|
|
|
- " was not deleted", b, dn);
|
|
|
- return false;
|
|
|
}
|
|
|
}
|
|
|
|
|
@@ -2782,7 +2784,7 @@ public class BlockManager {
|
|
|
" but corrupt replicas map has " + corruptReplicasCount);
|
|
|
}
|
|
|
if ((corruptReplicasCount > 0) && (numLiveReplicas >= fileReplication)) {
|
|
|
- invalidateCorruptReplicas(storedBlock, reportedBlock);
|
|
|
+ invalidateCorruptReplicas(storedBlock, reportedBlock, num);
|
|
|
}
|
|
|
return storedBlock;
|
|
|
}
|
|
@@ -2814,18 +2816,20 @@ public class BlockManager {
|
|
|
*
|
|
|
* @param blk Block whose corrupt replicas need to be invalidated
|
|
|
*/
|
|
|
- private void invalidateCorruptReplicas(BlockInfo blk, Block reported) {
|
|
|
+ private void invalidateCorruptReplicas(BlockInfo blk, Block reported,
|
|
|
+ NumberReplicas numberReplicas) {
|
|
|
Collection<DatanodeDescriptor> nodes = corruptReplicas.getNodes(blk);
|
|
|
boolean removedFromBlocksMap = true;
|
|
|
if (nodes == null)
|
|
|
return;
|
|
|
// make a copy of the array of nodes in order to avoid
|
|
|
// ConcurrentModificationException, when the block is removed from the node
|
|
|
- DatanodeDescriptor[] nodesCopy = nodes.toArray(new DatanodeDescriptor[0]);
|
|
|
+ DatanodeDescriptor[] nodesCopy =
|
|
|
+ nodes.toArray(new DatanodeDescriptor[nodes.size()]);
|
|
|
for (DatanodeDescriptor node : nodesCopy) {
|
|
|
try {
|
|
|
if (!invalidateBlock(new BlockToMarkCorrupt(reported, blk, null,
|
|
|
- Reason.ANY), node)) {
|
|
|
+ Reason.ANY), node, numberReplicas)) {
|
|
|
removedFromBlocksMap = false;
|
|
|
}
|
|
|
} catch (IOException e) {
|