|
@@ -17,6 +17,7 @@
|
|
|
*/
|
|
|
package org.apache.hadoop.hdfs.server.blockmanagement;
|
|
|
|
|
|
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_HA_NAMENODES_KEY_PREFIX;
|
|
|
import static org.apache.hadoop.util.ExitUtil.terminate;
|
|
|
|
|
|
import java.io.IOException;
|
|
@@ -195,7 +196,7 @@ public class BlockManager implements BlockStatsMXBean {
|
|
|
* notified of all block deletions that might have been pending
|
|
|
* when the failover happened.
|
|
|
*/
|
|
|
- private final Set<BlockInfo> postponedMisreplicatedBlocks = Sets.newHashSet();
|
|
|
+ private final Set<Block> postponedMisreplicatedBlocks = Sets.newHashSet();
|
|
|
|
|
|
/**
|
|
|
* Maps a StorageID to the set of blocks that are "extra" for this
|
|
@@ -336,7 +337,8 @@ public class BlockManager implements BlockStatsMXBean {
|
|
|
DFSConfigKeys.DFS_NAMENODE_REPLICATION_STREAMS_HARD_LIMIT_KEY,
|
|
|
DFSConfigKeys.DFS_NAMENODE_REPLICATION_STREAMS_HARD_LIMIT_DEFAULT);
|
|
|
this.shouldCheckForEnoughRacks =
|
|
|
- conf.get(DFSConfigKeys.NET_TOPOLOGY_SCRIPT_FILE_NAME_KEY) != null;
|
|
|
+ conf.get(DFSConfigKeys.NET_TOPOLOGY_SCRIPT_FILE_NAME_KEY) == null
|
|
|
+ ? false : true;
|
|
|
|
|
|
this.blocksInvalidateWorkPct = DFSUtil.getInvalidateWorkPctPerIteration(conf);
|
|
|
this.blocksReplWorkMultiplier = DFSUtil.getReplWorkMultiplier(conf);
|
|
@@ -455,7 +457,8 @@ public class BlockManager implements BlockStatsMXBean {
|
|
|
|
|
|
/** Should the access keys be updated? */
|
|
|
boolean shouldUpdateBlockKey(final long updateTime) throws IOException {
|
|
|
- return isBlockTokenEnabled() && blockTokenSecretManager.updateKeys(updateTime);
|
|
|
+ return isBlockTokenEnabled()? blockTokenSecretManager.updateKeys(updateTime)
|
|
|
+ : false;
|
|
|
}
|
|
|
|
|
|
public void activate(Configuration conf) {
|
|
@@ -508,14 +511,14 @@ public class BlockManager implements BlockStatsMXBean {
|
|
|
synchronized (neededReplications) {
|
|
|
out.println("Metasave: Blocks waiting for replication: " +
|
|
|
neededReplications.size());
|
|
|
- for (BlockInfo block : neededReplications) {
|
|
|
+ for (Block block : neededReplications) {
|
|
|
dumpBlockMeta(block, out);
|
|
|
}
|
|
|
}
|
|
|
|
|
|
// Dump any postponed over-replicated blocks
|
|
|
out.println("Mis-replicated blocks that have been postponed:");
|
|
|
- for (BlockInfo block : postponedMisreplicatedBlocks) {
|
|
|
+ for (Block block : postponedMisreplicatedBlocks) {
|
|
|
dumpBlockMeta(block, out);
|
|
|
}
|
|
|
|
|
@@ -533,9 +536,11 @@ public class BlockManager implements BlockStatsMXBean {
|
|
|
* Dump the metadata for the given block in a human-readable
|
|
|
* form.
|
|
|
*/
|
|
|
- private void dumpBlockMeta(BlockInfo block, PrintWriter out) {
|
|
|
- List<DatanodeDescriptor> containingNodes = new ArrayList<>();
|
|
|
- List<DatanodeStorageInfo> containingLiveReplicasNodes = new ArrayList<>();
|
|
|
+ private void dumpBlockMeta(Block block, PrintWriter out) {
|
|
|
+ List<DatanodeDescriptor> containingNodes =
|
|
|
+ new ArrayList<DatanodeDescriptor>();
|
|
|
+ List<DatanodeStorageInfo> containingLiveReplicasNodes =
|
|
|
+ new ArrayList<>();
|
|
|
|
|
|
NumberReplicas numReplicas = new NumberReplicas();
|
|
|
// source node returned is not used
|
|
@@ -543,16 +548,17 @@ public class BlockManager implements BlockStatsMXBean {
|
|
|
containingLiveReplicasNodes, numReplicas,
|
|
|
UnderReplicatedBlocks.LEVEL);
|
|
|
|
|
|
- // containingLiveReplicasNodes can include READ_ONLY_SHARED replicas which
|
|
|
- // are not included in the numReplicas.liveReplicas() count
|
|
|
+ // containingLiveReplicasNodes can include READ_ONLY_SHARED replicas which are
|
|
|
+ // not included in the numReplicas.liveReplicas() count
|
|
|
assert containingLiveReplicasNodes.size() >= numReplicas.liveReplicas();
|
|
|
int usableReplicas = numReplicas.liveReplicas() +
|
|
|
numReplicas.decommissionedAndDecommissioning();
|
|
|
-
|
|
|
- BlockCollection bc = block.getBlockCollection();
|
|
|
- String fileName = (bc == null) ? "[orphaned]" : bc.getName();
|
|
|
- out.print(fileName + ": ");
|
|
|
-
|
|
|
+
|
|
|
+ if (block instanceof BlockInfo) {
|
|
|
+ BlockCollection bc = ((BlockInfo) block).getBlockCollection();
|
|
|
+ String fileName = (bc == null) ? "[orphaned]" : bc.getName();
|
|
|
+ out.print(fileName + ": ");
|
|
|
+ }
|
|
|
// l: == live:, d: == decommissioned c: == corrupt e: == excess
|
|
|
out.print(block + ((usableReplicas > 0)? "" : " MISSING") +
|
|
|
" (replicas:" +
|
|
@@ -561,8 +567,8 @@ public class BlockManager implements BlockStatsMXBean {
|
|
|
" c: " + numReplicas.corruptReplicas() +
|
|
|
" e: " + numReplicas.excessReplicas() + ") ");
|
|
|
|
|
|
- Collection<DatanodeDescriptor> corruptNodes =
|
|
|
- corruptReplicas.getNodes(block);
|
|
|
+ Collection<DatanodeDescriptor> corruptNodes =
|
|
|
+ corruptReplicas.getNodes(block);
|
|
|
|
|
|
for (DatanodeStorageInfo storage : getStorages(block)) {
|
|
|
final DatanodeDescriptor node = storage.getDatanodeDescriptor();
|
|
@@ -799,8 +805,7 @@ public class BlockManager implements BlockStatsMXBean {
|
|
|
final long offset, final long length, final int nrBlocksToReturn,
|
|
|
final AccessMode mode) throws IOException {
|
|
|
int curBlk;
|
|
|
- long curPos = 0;
|
|
|
- long blkSize;
|
|
|
+ long curPos = 0, blkSize = 0;
|
|
|
int nrBlocks = (blocks[0].getNumBytes() == 0) ? 0 : blocks.length;
|
|
|
for (curBlk = 0; curBlk < nrBlocks; curBlk++) {
|
|
|
blkSize = blocks[curBlk].getNumBytes();
|
|
@@ -1190,11 +1195,10 @@ public class BlockManager implements BlockStatsMXBean {
|
|
|
}
|
|
|
|
|
|
/**
|
|
|
- * 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 b
|
|
|
* @param storageInfo storage that contains the block, if known. null otherwise.
|
|
|
+ * @throws IOException
|
|
|
*/
|
|
|
private void markBlockAsCorrupt(BlockToMarkCorrupt b,
|
|
|
DatanodeStorageInfo storageInfo,
|
|
@@ -1215,7 +1219,7 @@ public class BlockManager implements BlockStatsMXBean {
|
|
|
}
|
|
|
|
|
|
// Add this replica to corruptReplicas Map
|
|
|
- corruptReplicas.addToCorruptReplicasMap(b.stored, node, b.reason,
|
|
|
+ corruptReplicas.addToCorruptReplicasMap(b.corrupted, node, b.reason,
|
|
|
b.reasonCode);
|
|
|
|
|
|
NumberReplicas numberOfReplicas = countNodes(b.stored);
|
|
@@ -1237,7 +1241,7 @@ public class BlockManager implements BlockStatsMXBean {
|
|
|
if (hasEnoughLiveReplicas || hasMoreCorruptReplicas
|
|
|
|| corruptedDuringWrite) {
|
|
|
// the block is over-replicated so invalidate the replicas immediately
|
|
|
- invalidateBlock(b, node, numberOfReplicas);
|
|
|
+ invalidateBlock(b, node);
|
|
|
} else if (namesystem.isPopulatingReplQueues()) {
|
|
|
// add the block to neededReplication
|
|
|
updateNeededReplications(b.stored, -1, 0);
|
|
@@ -1245,15 +1249,12 @@ public class BlockManager implements BlockStatsMXBean {
|
|
|
}
|
|
|
|
|
|
/**
|
|
|
- * Invalidates the given block on the given datanode. Note that before this
|
|
|
- * call we have already checked the current live replicas of the block and
|
|
|
- * make sure it's safe to invalidate the replica.
|
|
|
- *
|
|
|
- * @return true if the replica was successfully invalidated and no longer
|
|
|
- * associated with the DataNode.
|
|
|
+ * Invalidates the given block on the given datanode.
|
|
|
+ * @return true if the block was successfully invalidated and no longer
|
|
|
+ * present in the BlocksMap
|
|
|
*/
|
|
|
- private boolean invalidateBlock(BlockToMarkCorrupt b, DatanodeInfo dn,
|
|
|
- NumberReplicas nr) throws IOException {
|
|
|
+ private boolean invalidateBlock(BlockToMarkCorrupt b, DatanodeInfo dn
|
|
|
+ ) throws IOException {
|
|
|
blockLog.info("BLOCK* invalidateBlock: {} on {}", b, dn);
|
|
|
DatanodeDescriptor node = getDatanodeManager().getDatanode(dn);
|
|
|
if (node == null) {
|
|
@@ -1262,30 +1263,35 @@ public class BlockManager implements BlockStatsMXBean {
|
|
|
}
|
|
|
|
|
|
// 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 " +
|
|
|
"nodes with potentially out-of-date block reports", b, dn,
|
|
|
nr.replicasOnStaleNodes());
|
|
|
- postponeBlock(b.stored);
|
|
|
+ postponeBlock(b.corrupted);
|
|
|
return false;
|
|
|
- } else {
|
|
|
- // we already checked the number of replicas in the caller of this
|
|
|
- // function and we know there is at least one copy on a live node, so we
|
|
|
- // can delete it.
|
|
|
+ } else if (nr.liveReplicas() >= 1) {
|
|
|
+ // If we have at least one copy on a live node, then 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;
|
|
|
}
|
|
|
}
|
|
|
|
|
|
+
|
|
|
public void setPostponeBlocksFromFuture(boolean postpone) {
|
|
|
this.shouldPostponeBlocksFromFuture = postpone;
|
|
|
}
|
|
|
|
|
|
- private void postponeBlock(BlockInfo blk) {
|
|
|
+
|
|
|
+ private void postponeBlock(Block blk) {
|
|
|
if (postponedMisreplicatedBlocks.add(blk)) {
|
|
|
postponedMisreplicatedBlocksCount.incrementAndGet();
|
|
|
}
|
|
@@ -1359,7 +1365,7 @@ public class BlockManager implements BlockStatsMXBean {
|
|
|
int requiredReplication, numEffectiveReplicas;
|
|
|
List<DatanodeDescriptor> containingNodes;
|
|
|
DatanodeDescriptor srcNode;
|
|
|
- BlockCollection bc;
|
|
|
+ BlockCollection bc = null;
|
|
|
int additionalReplRequired;
|
|
|
|
|
|
int scheduledWork = 0;
|
|
@@ -1518,9 +1524,9 @@ public class BlockManager implements BlockStatsMXBean {
|
|
|
DatanodeStorageInfo[] targets = rw.targets;
|
|
|
if (targets != null && targets.length != 0) {
|
|
|
StringBuilder targetList = new StringBuilder("datanode(s)");
|
|
|
- for (DatanodeStorageInfo target : targets) {
|
|
|
+ for (int k = 0; k < targets.length; k++) {
|
|
|
targetList.append(' ');
|
|
|
- targetList.append(target.getDatanodeDescriptor());
|
|
|
+ targetList.append(targets[k].getDatanodeDescriptor());
|
|
|
}
|
|
|
blockLog.info("BLOCK* ask {} to replicate {} to {}", rw.srcNode,
|
|
|
rw.block, targetList);
|
|
@@ -1597,8 +1603,8 @@ public class BlockManager implements BlockStatsMXBean {
|
|
|
List<DatanodeDescriptor> datanodeDescriptors = null;
|
|
|
if (nodes != null) {
|
|
|
datanodeDescriptors = new ArrayList<>(nodes.size());
|
|
|
- for (String nodeStr : nodes) {
|
|
|
- DatanodeDescriptor node = datanodeManager.getDatanodeDescriptor(nodeStr);
|
|
|
+ for (int i = 0; i < nodes.size(); i++) {
|
|
|
+ DatanodeDescriptor node = datanodeManager.getDatanodeDescriptor(nodes.get(i));
|
|
|
if (node != null) {
|
|
|
datanodeDescriptors.add(node);
|
|
|
}
|
|
@@ -1637,7 +1643,7 @@ public class BlockManager implements BlockStatsMXBean {
|
|
|
* the given block
|
|
|
*/
|
|
|
@VisibleForTesting
|
|
|
- DatanodeDescriptor chooseSourceDatanode(BlockInfo block,
|
|
|
+ DatanodeDescriptor chooseSourceDatanode(Block block,
|
|
|
List<DatanodeDescriptor> containingNodes,
|
|
|
List<DatanodeStorageInfo> nodesContainingLiveReplicas,
|
|
|
NumberReplicas numReplicas,
|
|
@@ -1717,16 +1723,16 @@ public class BlockManager implements BlockStatsMXBean {
|
|
|
if (timedOutItems != null) {
|
|
|
namesystem.writeLock();
|
|
|
try {
|
|
|
- for (BlockInfo timedOutItem : timedOutItems) {
|
|
|
+ for (int i = 0; i < timedOutItems.length; i++) {
|
|
|
/*
|
|
|
* Use the blockinfo from the blocksmap to be certain we're working
|
|
|
* with the most up-to-date block information (e.g. genstamp).
|
|
|
*/
|
|
|
- BlockInfo bi = getStoredBlock(timedOutItem);
|
|
|
+ BlockInfo bi = getStoredBlock(timedOutItems[i]);
|
|
|
if (bi == null) {
|
|
|
continue;
|
|
|
}
|
|
|
- NumberReplicas num = countNodes(timedOutItem);
|
|
|
+ NumberReplicas num = countNodes(timedOutItems[i]);
|
|
|
if (isNeededReplication(bi, getReplication(bi), num.liveReplicas())) {
|
|
|
neededReplications.add(bi, num.liveReplicas(),
|
|
|
num.decommissionedAndDecommissioning(), getReplication(bi));
|
|
@@ -1743,7 +1749,7 @@ public class BlockManager implements BlockStatsMXBean {
|
|
|
|
|
|
public long requestBlockReportLeaseId(DatanodeRegistration nodeReg) {
|
|
|
assert namesystem.hasReadLock();
|
|
|
- DatanodeDescriptor node;
|
|
|
+ DatanodeDescriptor node = null;
|
|
|
try {
|
|
|
node = datanodeManager.getDatanode(nodeReg);
|
|
|
} catch (UnregisteredNodeException e) {
|
|
@@ -2005,7 +2011,7 @@ public class BlockManager implements BlockStatsMXBean {
|
|
|
startIndex += (base+1);
|
|
|
}
|
|
|
}
|
|
|
- Iterator<BlockInfo> it = postponedMisreplicatedBlocks.iterator();
|
|
|
+ Iterator<Block> it = postponedMisreplicatedBlocks.iterator();
|
|
|
for (int tmp = 0; tmp < startIndex; tmp++) {
|
|
|
it.next();
|
|
|
}
|
|
@@ -2100,7 +2106,7 @@ public class BlockManager implements BlockStatsMXBean {
|
|
|
long oldGenerationStamp, long oldNumBytes,
|
|
|
DatanodeStorageInfo[] newStorages) throws IOException {
|
|
|
assert namesystem.hasWriteLock();
|
|
|
- BlockToMarkCorrupt b;
|
|
|
+ BlockToMarkCorrupt b = null;
|
|
|
if (block.getGenerationStamp() != oldGenerationStamp) {
|
|
|
b = new BlockToMarkCorrupt(oldBlock, block, oldGenerationStamp,
|
|
|
"genstamp does not match " + oldGenerationStamp
|
|
@@ -2702,7 +2708,7 @@ public class BlockManager implements BlockStatsMXBean {
|
|
|
" but corrupt replicas map has " + corruptReplicasCount);
|
|
|
}
|
|
|
if ((corruptReplicasCount > 0) && (numLiveReplicas >= fileReplication)) {
|
|
|
- invalidateCorruptReplicas(storedBlock, reportedBlock, num);
|
|
|
+ invalidateCorruptReplicas(storedBlock, reportedBlock);
|
|
|
}
|
|
|
return storedBlock;
|
|
|
}
|
|
@@ -2735,20 +2741,18 @@ public class BlockManager implements BlockStatsMXBean {
|
|
|
*
|
|
|
* @param blk Block whose corrupt replicas need to be invalidated
|
|
|
*/
|
|
|
- private void invalidateCorruptReplicas(BlockInfo blk, Block reported,
|
|
|
- NumberReplicas numberReplicas) {
|
|
|
+ private void invalidateCorruptReplicas(BlockInfo blk, Block reported) {
|
|
|
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[nodes.size()]);
|
|
|
+ DatanodeDescriptor[] nodesCopy = nodes.toArray(new DatanodeDescriptor[0]);
|
|
|
for (DatanodeDescriptor node : nodesCopy) {
|
|
|
try {
|
|
|
if (!invalidateBlock(new BlockToMarkCorrupt(reported, blk, null,
|
|
|
- Reason.ANY), node, numberReplicas)) {
|
|
|
+ Reason.ANY), node)) {
|
|
|
removedFromBlocksMap = false;
|
|
|
}
|
|
|
} catch (IOException e) {
|
|
@@ -2798,6 +2802,7 @@ public class BlockManager implements BlockStatsMXBean {
|
|
|
replicationQueuesInitializer.join();
|
|
|
} catch (final InterruptedException e) {
|
|
|
LOG.warn("Interrupted while waiting for replicationQueueInitializer. Returning..");
|
|
|
+ return;
|
|
|
} finally {
|
|
|
replicationQueuesInitializer = null;
|
|
|
}
|
|
@@ -3159,7 +3164,8 @@ public class BlockManager implements BlockStatsMXBean {
|
|
|
CachedBlock cblock = namesystem.getCacheManager().getCachedBlocks()
|
|
|
.get(new CachedBlock(storedBlock.getBlockId(), (short) 0, false));
|
|
|
if (cblock != null) {
|
|
|
- boolean removed = node.getPendingCached().remove(cblock);
|
|
|
+ boolean removed = false;
|
|
|
+ removed |= node.getPendingCached().remove(cblock);
|
|
|
removed |= node.getCached().remove(cblock);
|
|
|
removed |= node.getPendingUncached().remove(cblock);
|
|
|
if (removed) {
|
|
@@ -3375,7 +3381,7 @@ public class BlockManager implements BlockStatsMXBean {
|
|
|
int excess = 0;
|
|
|
int stale = 0;
|
|
|
Collection<DatanodeDescriptor> nodesCorrupt = corruptReplicas.getNodes(b);
|
|
|
- for (DatanodeStorageInfo storage : blocksMap.getStorages(b, State.NORMAL)) {
|
|
|
+ for(DatanodeStorageInfo storage : blocksMap.getStorages(b, State.NORMAL)) {
|
|
|
final DatanodeDescriptor node = storage.getDatanodeDescriptor();
|
|
|
if ((nodesCorrupt != null) && (nodesCorrupt.contains(node))) {
|
|
|
corrupt++;
|
|
@@ -3396,8 +3402,7 @@ public class BlockManager implements BlockStatsMXBean {
|
|
|
stale++;
|
|
|
}
|
|
|
}
|
|
|
- return new NumberReplicas(live, decommissioned, decommissioning, corrupt,
|
|
|
- excess, stale);
|
|
|
+ return new NumberReplicas(live, decommissioned, decommissioning, corrupt, excess, stale);
|
|
|
}
|
|
|
|
|
|
/**
|
|
@@ -3580,6 +3585,8 @@ public class BlockManager implements BlockStatsMXBean {
|
|
|
String src, BlockInfo[] blocks) {
|
|
|
for (BlockInfo b: blocks) {
|
|
|
if (!b.isComplete()) {
|
|
|
+ final BlockInfoUnderConstruction uc =
|
|
|
+ (BlockInfoUnderConstruction)b;
|
|
|
final int numNodes = b.numNodes();
|
|
|
final int min = getMinStorageNum(b);
|
|
|
final BlockUCState state = b.getBlockUCState();
|
|
@@ -3705,7 +3712,11 @@ public class BlockManager implements BlockStatsMXBean {
|
|
|
return blocksMap.getBlockCollection(b);
|
|
|
}
|
|
|
|
|
|
- public void removeBlockFromMap(BlockInfo block) {
|
|
|
+ public int numCorruptReplicas(Block block) {
|
|
|
+ return corruptReplicas.numCorruptReplicas(block);
|
|
|
+ }
|
|
|
+
|
|
|
+ public void removeBlockFromMap(Block block) {
|
|
|
removeFromExcessReplicateMap(block);
|
|
|
blocksMap.removeBlock(block);
|
|
|
// If block is removed from blocksMap remove it from corruptReplicasMap
|
|
@@ -3715,7 +3726,7 @@ public class BlockManager implements BlockStatsMXBean {
|
|
|
/**
|
|
|
* If a block is removed from blocksMap, remove it from excessReplicateMap.
|
|
|
*/
|
|
|
- private void removeFromExcessReplicateMap(BlockInfo block) {
|
|
|
+ private void removeFromExcessReplicateMap(Block block) {
|
|
|
for (DatanodeStorageInfo info : getStorages(block)) {
|
|
|
String uuid = info.getDatanodeDescriptor().getDatanodeUuid();
|
|
|
LightWeightLinkedSet<BlockInfo> excessReplicas =
|
|
@@ -3746,14 +3757,14 @@ public class BlockManager implements BlockStatsMXBean {
|
|
|
/**
|
|
|
* Get the replicas which are corrupt for a given block.
|
|
|
*/
|
|
|
- public Collection<DatanodeDescriptor> getCorruptReplicas(BlockInfo block) {
|
|
|
+ public Collection<DatanodeDescriptor> getCorruptReplicas(Block block) {
|
|
|
return corruptReplicas.getNodes(block);
|
|
|
}
|
|
|
|
|
|
/**
|
|
|
* Get reason for certain corrupted replicas for a given block and a given dn.
|
|
|
*/
|
|
|
- public String getCorruptReason(BlockInfo block, DatanodeDescriptor node) {
|
|
|
+ public String getCorruptReason(Block block, DatanodeDescriptor node) {
|
|
|
return corruptReplicas.getCorruptReason(block, node);
|
|
|
}
|
|
|
|
|
@@ -3847,7 +3858,7 @@ public class BlockManager implements BlockStatsMXBean {
|
|
|
datanodeManager.clearPendingQueues();
|
|
|
postponedMisreplicatedBlocks.clear();
|
|
|
postponedMisreplicatedBlocksCount.set(0);
|
|
|
- }
|
|
|
+ };
|
|
|
|
|
|
public static LocatedBlock newLocatedBlock(
|
|
|
ExtendedBlock b, DatanodeStorageInfo[] storages,
|