|
@@ -500,9 +500,8 @@ public class BlockManager {
|
|
Collection<DatanodeDescriptor> corruptNodes =
|
|
Collection<DatanodeDescriptor> corruptNodes =
|
|
corruptReplicas.getNodes(block);
|
|
corruptReplicas.getNodes(block);
|
|
|
|
|
|
- for (Iterator<DatanodeDescriptor> jt = blocksMap.nodeIterator(block);
|
|
|
|
- jt.hasNext();) {
|
|
|
|
- DatanodeDescriptor node = jt.next();
|
|
|
|
|
|
+ for (DatanodeStorageInfo storage : blocksMap.getStorages(block)) {
|
|
|
|
+ final DatanodeDescriptor node = storage.getDatanodeDescriptor();
|
|
String state = "";
|
|
String state = "";
|
|
if (corruptNodes != null && corruptNodes.contains(node)) {
|
|
if (corruptNodes != null && corruptNodes.contains(node)) {
|
|
state = "(corrupt)";
|
|
state = "(corrupt)";
|
|
@@ -662,10 +661,9 @@ public class BlockManager {
|
|
assert oldBlock == getStoredBlock(oldBlock) :
|
|
assert oldBlock == getStoredBlock(oldBlock) :
|
|
"last block of the file is not in blocksMap";
|
|
"last block of the file is not in blocksMap";
|
|
|
|
|
|
- DatanodeDescriptor[] targets = getNodes(oldBlock);
|
|
|
|
|
|
+ DatanodeStorageInfo[] targets = getStorages(oldBlock);
|
|
|
|
|
|
- BlockInfoUnderConstruction ucBlock =
|
|
|
|
- bc.setLastBlock(oldBlock, targets);
|
|
|
|
|
|
+ BlockInfoUnderConstruction ucBlock = bc.setLastBlock(oldBlock, targets);
|
|
blocksMap.replaceBlock(ucBlock);
|
|
blocksMap.replaceBlock(ucBlock);
|
|
|
|
|
|
// Remove block from replication queue.
|
|
// Remove block from replication queue.
|
|
@@ -675,9 +673,8 @@ public class BlockManager {
|
|
pendingReplications.remove(ucBlock);
|
|
pendingReplications.remove(ucBlock);
|
|
|
|
|
|
// remove this block from the list of pending blocks to be deleted.
|
|
// remove this block from the list of pending blocks to be deleted.
|
|
- for (DatanodeDescriptor dd : targets) {
|
|
|
|
- String datanodeId = dd.getStorageID();
|
|
|
|
- invalidateBlocks.remove(datanodeId, oldBlock);
|
|
|
|
|
|
+ for (DatanodeStorageInfo storage : targets) {
|
|
|
|
+ invalidateBlocks.remove(storage.getStorageID(), oldBlock);
|
|
}
|
|
}
|
|
|
|
|
|
// Adjust safe-mode totals, since under-construction blocks don't
|
|
// Adjust safe-mode totals, since under-construction blocks don't
|
|
@@ -699,9 +696,8 @@ public class BlockManager {
|
|
private List<String> getValidLocations(Block block) {
|
|
private List<String> getValidLocations(Block block) {
|
|
ArrayList<String> machineSet =
|
|
ArrayList<String> machineSet =
|
|
new ArrayList<String>(blocksMap.numNodes(block));
|
|
new ArrayList<String>(blocksMap.numNodes(block));
|
|
- for(Iterator<DatanodeDescriptor> it =
|
|
|
|
- blocksMap.nodeIterator(block); it.hasNext();) {
|
|
|
|
- String storageID = it.next().getStorageID();
|
|
|
|
|
|
+ for(DatanodeStorageInfo storage : blocksMap.getStorages(block)) {
|
|
|
|
+ final String storageID = storage.getStorageID();
|
|
// filter invalidate replicas
|
|
// filter invalidate replicas
|
|
if(!invalidateBlocks.contains(storageID, block)) {
|
|
if(!invalidateBlocks.contains(storageID, block)) {
|
|
machineSet.add(storageID);
|
|
machineSet.add(storageID);
|
|
@@ -775,9 +771,9 @@ public class BlockManager {
|
|
+ ", blk=" + blk);
|
|
+ ", blk=" + blk);
|
|
}
|
|
}
|
|
final BlockInfoUnderConstruction uc = (BlockInfoUnderConstruction)blk;
|
|
final BlockInfoUnderConstruction uc = (BlockInfoUnderConstruction)blk;
|
|
- final DatanodeDescriptor[] locations = uc.getExpectedLocations();
|
|
|
|
|
|
+ final DatanodeStorageInfo[] storages = uc.getExpectedStorageLocations();
|
|
final ExtendedBlock eb = new ExtendedBlock(namesystem.getBlockPoolId(), blk);
|
|
final ExtendedBlock eb = new ExtendedBlock(namesystem.getBlockPoolId(), blk);
|
|
- return new LocatedBlock(eb, locations, pos, false);
|
|
|
|
|
|
+ return LocatedBlock.createLocatedBlock(eb, storages, pos, false);
|
|
}
|
|
}
|
|
|
|
|
|
// get block locations
|
|
// get block locations
|
|
@@ -795,9 +791,8 @@ public class BlockManager {
|
|
final DatanodeDescriptor[] machines = new DatanodeDescriptor[numMachines];
|
|
final DatanodeDescriptor[] machines = new DatanodeDescriptor[numMachines];
|
|
int j = 0;
|
|
int j = 0;
|
|
if (numMachines > 0) {
|
|
if (numMachines > 0) {
|
|
- for(Iterator<DatanodeDescriptor> it = blocksMap.nodeIterator(blk);
|
|
|
|
- it.hasNext();) {
|
|
|
|
- final DatanodeDescriptor d = it.next();
|
|
|
|
|
|
+ for(DatanodeStorageInfo storage : blocksMap.getStorages(blk)) {
|
|
|
|
+ final DatanodeDescriptor d = storage.getDatanodeDescriptor();
|
|
final boolean replicaCorrupt = corruptReplicas.isReplicaCorrupt(blk, d);
|
|
final boolean replicaCorrupt = corruptReplicas.isReplicaCorrupt(blk, d);
|
|
if (isCorrupt || (!isCorrupt && !replicaCorrupt))
|
|
if (isCorrupt || (!isCorrupt && !replicaCorrupt))
|
|
machines[j++] = d;
|
|
machines[j++] = d;
|
|
@@ -1017,9 +1012,8 @@ public class BlockManager {
|
|
*/
|
|
*/
|
|
private void addToInvalidates(Block b) {
|
|
private void addToInvalidates(Block b) {
|
|
StringBuilder datanodes = new StringBuilder();
|
|
StringBuilder datanodes = new StringBuilder();
|
|
- for (Iterator<DatanodeDescriptor> it = blocksMap.nodeIterator(b); it
|
|
|
|
- .hasNext();) {
|
|
|
|
- DatanodeDescriptor node = it.next();
|
|
|
|
|
|
+ for(DatanodeStorageInfo storage : blocksMap.getStorages(b)) {
|
|
|
|
+ final DatanodeDescriptor node = storage.getDatanodeDescriptor();
|
|
invalidateBlocks.add(b, node, false);
|
|
invalidateBlocks.add(b, node, false);
|
|
datanodes.append(node).append(" ");
|
|
datanodes.append(node).append(" ");
|
|
}
|
|
}
|
|
@@ -1466,10 +1460,10 @@ public class BlockManager {
|
|
int decommissioned = 0;
|
|
int decommissioned = 0;
|
|
int corrupt = 0;
|
|
int corrupt = 0;
|
|
int excess = 0;
|
|
int excess = 0;
|
|
- Iterator<DatanodeDescriptor> it = blocksMap.nodeIterator(block);
|
|
|
|
|
|
+
|
|
Collection<DatanodeDescriptor> nodesCorrupt = corruptReplicas.getNodes(block);
|
|
Collection<DatanodeDescriptor> nodesCorrupt = corruptReplicas.getNodes(block);
|
|
- while(it.hasNext()) {
|
|
|
|
- DatanodeDescriptor node = it.next();
|
|
|
|
|
|
+ for(DatanodeStorageInfo storage : blocksMap.getStorages(block)) {
|
|
|
|
+ final DatanodeDescriptor node = storage.getDatanodeDescriptor();
|
|
LightWeightLinkedSet<Block> excessBlocks =
|
|
LightWeightLinkedSet<Block> excessBlocks =
|
|
excessReplicateMap.get(node.getStorageID());
|
|
excessReplicateMap.get(node.getStorageID());
|
|
if ((nodesCorrupt != null) && (nodesCorrupt.contains(node)))
|
|
if ((nodesCorrupt != null) && (nodesCorrupt.contains(node)))
|
|
@@ -1790,7 +1784,7 @@ public class BlockManager {
|
|
// If block is under construction, add this replica to its list
|
|
// If block is under construction, add this replica to its list
|
|
if (isBlockUnderConstruction(storedBlock, ucState, reportedState)) {
|
|
if (isBlockUnderConstruction(storedBlock, ucState, reportedState)) {
|
|
((BlockInfoUnderConstruction)storedBlock).addReplicaIfNotPresent(
|
|
((BlockInfoUnderConstruction)storedBlock).addReplicaIfNotPresent(
|
|
- node, iblk, reportedState);
|
|
|
|
|
|
+ node.getStorageInfo(storageID), iblk, reportedState);
|
|
//and fall through to next clause
|
|
//and fall through to next clause
|
|
}
|
|
}
|
|
//add replica if appropriate
|
|
//add replica if appropriate
|
|
@@ -2093,7 +2087,7 @@ assert storedBlock.findDatanode(dn) < 0 : "Block " + block
|
|
DatanodeDescriptor node, String storageID,
|
|
DatanodeDescriptor node, String storageID,
|
|
ReplicaState reportedState)
|
|
ReplicaState reportedState)
|
|
throws IOException {
|
|
throws IOException {
|
|
- block.addReplicaIfNotPresent(node, block, reportedState);
|
|
|
|
|
|
+ block.addReplicaIfNotPresent(node.getStorageInfo(storageID), block, reportedState);
|
|
if (reportedState == ReplicaState.FINALIZED && block.findDatanode(node) < 0) {
|
|
if (reportedState == ReplicaState.FINALIZED && block.findDatanode(node) < 0) {
|
|
addStoredBlock(block, node, storageID, null, true);
|
|
addStoredBlock(block, node, storageID, null, true);
|
|
}
|
|
}
|
|
@@ -2425,9 +2419,8 @@ assert storedBlock.findDatanode(dn) < 0 : "Block " + block
|
|
Collection<DatanodeDescriptor> nonExcess = new ArrayList<DatanodeDescriptor>();
|
|
Collection<DatanodeDescriptor> nonExcess = new ArrayList<DatanodeDescriptor>();
|
|
Collection<DatanodeDescriptor> corruptNodes = corruptReplicas
|
|
Collection<DatanodeDescriptor> corruptNodes = corruptReplicas
|
|
.getNodes(block);
|
|
.getNodes(block);
|
|
- for (Iterator<DatanodeDescriptor> it = blocksMap.nodeIterator(block);
|
|
|
|
- it.hasNext();) {
|
|
|
|
- DatanodeDescriptor cur = it.next();
|
|
|
|
|
|
+ for(DatanodeStorageInfo storage : blocksMap.getStorages(block)) {
|
|
|
|
+ final DatanodeDescriptor cur = storage.getDatanodeDescriptor();
|
|
if (cur.areBlockContentsStale()) {
|
|
if (cur.areBlockContentsStale()) {
|
|
LOG.info("BLOCK* processOverReplicatedBlock: " +
|
|
LOG.info("BLOCK* processOverReplicatedBlock: " +
|
|
"Postponing processing of over-replicated " +
|
|
"Postponing processing of over-replicated " +
|
|
@@ -2747,10 +2740,9 @@ assert storedBlock.findDatanode(dn) < 0 : "Block " + block
|
|
int corrupt = 0;
|
|
int corrupt = 0;
|
|
int excess = 0;
|
|
int excess = 0;
|
|
int stale = 0;
|
|
int stale = 0;
|
|
- Iterator<DatanodeDescriptor> nodeIter = blocksMap.nodeIterator(b);
|
|
|
|
Collection<DatanodeDescriptor> nodesCorrupt = corruptReplicas.getNodes(b);
|
|
Collection<DatanodeDescriptor> nodesCorrupt = corruptReplicas.getNodes(b);
|
|
- while (nodeIter.hasNext()) {
|
|
|
|
- DatanodeDescriptor node = nodeIter.next();
|
|
|
|
|
|
+ for(DatanodeStorageInfo storage : blocksMap.getStorages(b)) {
|
|
|
|
+ final DatanodeDescriptor node = storage.getDatanodeDescriptor();
|
|
if ((nodesCorrupt != null) && (nodesCorrupt.contains(node))) {
|
|
if ((nodesCorrupt != null) && (nodesCorrupt.contains(node))) {
|
|
corrupt++;
|
|
corrupt++;
|
|
} else if (node.isDecommissionInProgress() || node.isDecommissioned()) {
|
|
} else if (node.isDecommissionInProgress() || node.isDecommissioned()) {
|
|
@@ -2787,10 +2779,9 @@ assert storedBlock.findDatanode(dn) < 0 : "Block " + block
|
|
}
|
|
}
|
|
// else proceed with fast case
|
|
// else proceed with fast case
|
|
int live = 0;
|
|
int live = 0;
|
|
- Iterator<DatanodeDescriptor> nodeIter = blocksMap.nodeIterator(b);
|
|
|
|
Collection<DatanodeDescriptor> nodesCorrupt = corruptReplicas.getNodes(b);
|
|
Collection<DatanodeDescriptor> nodesCorrupt = corruptReplicas.getNodes(b);
|
|
- while (nodeIter.hasNext()) {
|
|
|
|
- DatanodeDescriptor node = nodeIter.next();
|
|
|
|
|
|
+ for(DatanodeStorageInfo storage : blocksMap.getStorages(b)) {
|
|
|
|
+ final DatanodeDescriptor node = storage.getDatanodeDescriptor();
|
|
if ((nodesCorrupt == null) || (!nodesCorrupt.contains(node)))
|
|
if ((nodesCorrupt == null) || (!nodesCorrupt.contains(node)))
|
|
live++;
|
|
live++;
|
|
}
|
|
}
|
|
@@ -2802,10 +2793,9 @@ assert storedBlock.findDatanode(dn) < 0 : "Block " + block
|
|
int curReplicas = num.liveReplicas();
|
|
int curReplicas = num.liveReplicas();
|
|
int curExpectedReplicas = getReplication(block);
|
|
int curExpectedReplicas = getReplication(block);
|
|
BlockCollection bc = blocksMap.getBlockCollection(block);
|
|
BlockCollection bc = blocksMap.getBlockCollection(block);
|
|
- Iterator<DatanodeDescriptor> nodeIter = blocksMap.nodeIterator(block);
|
|
|
|
StringBuilder nodeList = new StringBuilder();
|
|
StringBuilder nodeList = new StringBuilder();
|
|
- while (nodeIter.hasNext()) {
|
|
|
|
- DatanodeDescriptor node = nodeIter.next();
|
|
|
|
|
|
+ for(DatanodeStorageInfo storage : blocksMap.getStorages(block)) {
|
|
|
|
+ final DatanodeDescriptor node = storage.getDatanodeDescriptor();
|
|
nodeList.append(node);
|
|
nodeList.append(node);
|
|
nodeList.append(" ");
|
|
nodeList.append(" ");
|
|
}
|
|
}
|
|
@@ -2902,14 +2892,13 @@ assert storedBlock.findDatanode(dn) < 0 : "Block " + block
|
|
return blocksMap.size();
|
|
return blocksMap.size();
|
|
}
|
|
}
|
|
|
|
|
|
- public DatanodeDescriptor[] getNodes(BlockInfo block) {
|
|
|
|
- DatanodeDescriptor[] nodes =
|
|
|
|
- new DatanodeDescriptor[block.numNodes()];
|
|
|
|
- Iterator<DatanodeDescriptor> it = blocksMap.nodeIterator(block);
|
|
|
|
- for (int i = 0; it != null && it.hasNext(); i++) {
|
|
|
|
- nodes[i] = it.next();
|
|
|
|
|
|
+ public DatanodeStorageInfo[] getStorages(BlockInfo block) {
|
|
|
|
+ final DatanodeStorageInfo[] storages = new DatanodeStorageInfo[block.numNodes()];
|
|
|
|
+ int i = 0;
|
|
|
|
+ for(DatanodeStorageInfo s : blocksMap.getStorages(block)) {
|
|
|
|
+ storages[i++] = s;
|
|
}
|
|
}
|
|
- return nodes;
|
|
|
|
|
|
+ return storages;
|
|
}
|
|
}
|
|
|
|
|
|
public int getTotalBlocks() {
|
|
public int getTotalBlocks() {
|
|
@@ -3038,9 +3027,8 @@ assert storedBlock.findDatanode(dn) < 0 : "Block " + block
|
|
corruptReplicas.getNodes(b);
|
|
corruptReplicas.getNodes(b);
|
|
int numExpectedReplicas = getReplication(b);
|
|
int numExpectedReplicas = getReplication(b);
|
|
String rackName = null;
|
|
String rackName = null;
|
|
- for (Iterator<DatanodeDescriptor> it = blocksMap.nodeIterator(b);
|
|
|
|
- it.hasNext();) {
|
|
|
|
- DatanodeDescriptor cur = it.next();
|
|
|
|
|
|
+ for(DatanodeStorageInfo storage : blocksMap.getStorages(b)) {
|
|
|
|
+ final DatanodeDescriptor cur = storage.getDatanodeDescriptor();
|
|
if (!cur.isDecommissionInProgress() && !cur.isDecommissioned()) {
|
|
if (!cur.isDecommissionInProgress() && !cur.isDecommissioned()) {
|
|
if ((corruptNodes == null ) || !corruptNodes.contains(cur)) {
|
|
if ((corruptNodes == null ) || !corruptNodes.contains(cur)) {
|
|
if (numExpectedReplicas == 1 ||
|
|
if (numExpectedReplicas == 1 ||
|
|
@@ -3084,8 +3072,8 @@ assert storedBlock.findDatanode(dn) < 0 : "Block " + block
|
|
}
|
|
}
|
|
|
|
|
|
/** @return an iterator of the datanodes. */
|
|
/** @return an iterator of the datanodes. */
|
|
- public Iterator<DatanodeDescriptor> datanodeIterator(final Block block) {
|
|
|
|
- return blocksMap.nodeIterator(block);
|
|
|
|
|
|
+ public Iterable<DatanodeStorageInfo> getStorages(final Block block) {
|
|
|
|
+ return blocksMap.getStorages(block);
|
|
}
|
|
}
|
|
|
|
|
|
public int numCorruptReplicas(Block block) {
|
|
public int numCorruptReplicas(Block block) {
|