HDFS-5009. Include storage information in the LocatedBlock.

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/branches/HDFS-2832@1519691 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
Tsz-wo Sze 2013-09-03 14:03:35 +00:00
parent f7e3bc553d
commit 3f070e83b1
31 changed files with 269 additions and 186 deletions

View File

@ -13,5 +13,7 @@ IMPROVEMENTS:
HDFS-4987. Namenode changes to track multiple storages per datanode. HDFS-4987. Namenode changes to track multiple storages per datanode.
(szetszwo) (szetszwo)
HDFS-5154. Fix TestBlockManager and TestDatanodeDescriptor after HDFS-4987. HDFS-5154. Fix TestBlockManager and TestDatanodeDescriptor after HDFS-4987.
(Junping Du via szetszwo) (Junping Du via szetszwo)
HDFS-5009. Include storage information in the LocatedBlock. (szetszwo)

View File

@ -85,7 +85,6 @@ import org.apache.hadoop.util.Daemon;
import org.apache.hadoop.util.DataChecksum; import org.apache.hadoop.util.DataChecksum;
import org.apache.hadoop.util.Progressable; import org.apache.hadoop.util.Progressable;
import org.apache.hadoop.util.Time; import org.apache.hadoop.util.Time;
import org.mortbay.log.Log;
import com.google.common.annotations.VisibleForTesting; import com.google.common.annotations.VisibleForTesting;
import com.google.common.cache.CacheBuilder; import com.google.common.cache.CacheBuilder;
@ -312,6 +311,8 @@ public class DFSOutputStream extends FSOutputSummer
private DataInputStream blockReplyStream; private DataInputStream blockReplyStream;
private ResponseProcessor response = null; private ResponseProcessor response = null;
private volatile DatanodeInfo[] nodes = null; // list of targets for current block private volatile DatanodeInfo[] nodes = null; // list of targets for current block
//TODO: update storage IDs
private volatile String[] storageIDs = null;
private LoadingCache<DatanodeInfo, DatanodeInfo> excludedNodes = private LoadingCache<DatanodeInfo, DatanodeInfo> excludedNodes =
CacheBuilder.newBuilder() CacheBuilder.newBuilder()
.expireAfterWrite( .expireAfterWrite(
@ -1039,7 +1040,8 @@ public class DFSOutputStream extends FSOutputSummer
// update pipeline at the namenode // update pipeline at the namenode
ExtendedBlock newBlock = new ExtendedBlock( ExtendedBlock newBlock = new ExtendedBlock(
block.getBlockPoolId(), block.getBlockId(), block.getNumBytes(), newGS); block.getBlockPoolId(), block.getBlockId(), block.getNumBytes(), newGS);
dfsClient.namenode.updatePipeline(dfsClient.clientName, block, newBlock, nodes); dfsClient.namenode.updatePipeline(dfsClient.clientName, block, newBlock,
nodes, storageIDs);
// update client side generation stamp // update client side generation stamp
block = newBlock; block = newBlock;
} }

View File

@ -982,7 +982,7 @@ public interface ClientProtocol {
*/ */
@AtMostOnce @AtMostOnce
public void updatePipeline(String clientName, ExtendedBlock oldBlock, public void updatePipeline(String clientName, ExtendedBlock oldBlock,
ExtendedBlock newBlock, DatanodeID[] newNodes) ExtendedBlock newBlock, DatanodeID[] newNodes, String[] newStorageIDs)
throws IOException; throws IOException;
/** /**

View File

@ -21,6 +21,7 @@ import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability; import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.hdfs.StorageType; import org.apache.hadoop.hdfs.StorageType;
import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier; import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier;
import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeStorageInfo;
import org.apache.hadoop.security.token.Token; import org.apache.hadoop.security.token.Token;
/** /**
@ -35,6 +36,8 @@ public class LocatedBlock {
private ExtendedBlock b; private ExtendedBlock b;
private long offset; // offset of the first byte of the block in the file private long offset; // offset of the first byte of the block in the file
private DatanodeInfo[] locs; private DatanodeInfo[] locs;
/** Storage ID for each replica */
private String[] storageIDs;
// Storage type for each replica, if reported. // Storage type for each replica, if reported.
private StorageType[] storageTypes; private StorageType[] storageTypes;
// corrupt flag is true if all of the replicas of a block are corrupt. // corrupt flag is true if all of the replicas of a block are corrupt.
@ -53,10 +56,22 @@ public class LocatedBlock {
public LocatedBlock(ExtendedBlock b, DatanodeInfo[] locs, long startOffset, public LocatedBlock(ExtendedBlock b, DatanodeInfo[] locs, long startOffset,
boolean corrupt) { boolean corrupt) {
this(b, locs, null, startOffset, corrupt); this(b, locs, null, null, startOffset, corrupt);
} }
public LocatedBlock(ExtendedBlock b, DatanodeInfo[] locs, public static LocatedBlock createLocatedBlock(ExtendedBlock b,
DatanodeStorageInfo[] storages, long startOffset, boolean corrupt) {
final DatanodeInfo[] locs = new DatanodeInfo[storages.length];
final String[] storageIDs = new String[storages.length];
final StorageType[] storageType = new StorageType[storages.length];
for(int i = 0; i < storages.length; i++) {
locs[i] = storages[i].getDatanodeDescriptor();
storageIDs[i] = storages[i].getStorageID();
storageType[i] = storages[i].getStorageType();
}
return new LocatedBlock(b, locs, storageIDs, storageType, startOffset, corrupt);
}
public LocatedBlock(ExtendedBlock b, DatanodeInfo[] locs, String[] storageIDs,
StorageType[] storageTypes, long startOffset, StorageType[] storageTypes, long startOffset,
boolean corrupt) { boolean corrupt) {
this.b = b; this.b = b;
@ -67,6 +82,7 @@ public class LocatedBlock {
} else { } else {
this.locs = locs; this.locs = locs;
} }
this.storageIDs = storageIDs;
this.storageTypes = storageTypes; this.storageTypes = storageTypes;
} }
@ -94,6 +110,10 @@ public class LocatedBlock {
return storageTypes; return storageTypes;
} }
public String[] getStorageIDs() {
return storageIDs;
}
public long getStartOffset() { public long getStartOffset() {
return offset; return offset;
} }

View File

@ -813,10 +813,12 @@ public class ClientNamenodeProtocolServerSideTranslatorPB implements
UpdatePipelineRequestProto req) throws ServiceException { UpdatePipelineRequestProto req) throws ServiceException {
try { try {
List<DatanodeIDProto> newNodes = req.getNewNodesList(); List<DatanodeIDProto> newNodes = req.getNewNodesList();
server List<String> newStorageIDs = req.getStorageIDsList();
.updatePipeline(req.getClientName(), PBHelper.convert(req server.updatePipeline(req.getClientName(),
.getOldBlock()), PBHelper.convert(req.getNewBlock()), PBHelper PBHelper.convert(req.getOldBlock()),
.convert(newNodes.toArray(new DatanodeIDProto[newNodes.size()]))); PBHelper.convert(req.getNewBlock()),
PBHelper.convert(newNodes.toArray(new DatanodeIDProto[newNodes.size()])),
newStorageIDs.toArray(new String[newStorageIDs.size()]));
return VOID_UPDATEPIPELINE_RESPONSE; return VOID_UPDATEPIPELINE_RESPONSE;
} catch (IOException e) { } catch (IOException e) {
throw new ServiceException(e); throw new ServiceException(e);

View File

@ -780,12 +780,13 @@ public class ClientNamenodeProtocolTranslatorPB implements
@Override @Override
public void updatePipeline(String clientName, ExtendedBlock oldBlock, public void updatePipeline(String clientName, ExtendedBlock oldBlock,
ExtendedBlock newBlock, DatanodeID[] newNodes) throws IOException { ExtendedBlock newBlock, DatanodeID[] newNodes, String[] storageIDs) throws IOException {
UpdatePipelineRequestProto req = UpdatePipelineRequestProto.newBuilder() UpdatePipelineRequestProto req = UpdatePipelineRequestProto.newBuilder()
.setClientName(clientName) .setClientName(clientName)
.setOldBlock(PBHelper.convert(oldBlock)) .setOldBlock(PBHelper.convert(oldBlock))
.setNewBlock(PBHelper.convert(newBlock)) .setNewBlock(PBHelper.convert(newBlock))
.addAllNewNodes(Arrays.asList(PBHelper.convert(newNodes))) .addAllNewNodes(Arrays.asList(PBHelper.convert(newNodes)))
.addAllStorageIDs(Arrays.asList(storageIDs))
.build(); .build();
try { try {
rpcProxy.updatePipeline(null, req); rpcProxy.updatePipeline(null, req);

View File

@ -575,6 +575,8 @@ public class PBHelper {
builder.addStorageTypes(PBHelper.convertStorageType(storageTypes[i])); builder.addStorageTypes(PBHelper.convertStorageType(storageTypes[i]));
} }
} }
builder.addAllStorageIDs(Arrays.asList(b.getStorageIDs()));
return builder.setB(PBHelper.convert(b.getBlock())) return builder.setB(PBHelper.convert(b.getBlock()))
.setBlockToken(PBHelper.convert(b.getBlockToken())) .setBlockToken(PBHelper.convert(b.getBlockToken()))
.setCorrupt(b.isCorrupt()).setOffset(b.getStartOffset()).build(); .setCorrupt(b.isCorrupt()).setOffset(b.getStartOffset()).build();
@ -602,6 +604,7 @@ public class PBHelper {
} }
LocatedBlock lb = new LocatedBlock(PBHelper.convert(proto.getB()), targets, LocatedBlock lb = new LocatedBlock(PBHelper.convert(proto.getB()), targets,
proto.getStorageIDsList().toArray(new String[proto.getStorageIDsCount()]),
storageTypes, proto.getOffset(), proto.getCorrupt()); storageTypes, proto.getOffset(), proto.getCorrupt());
lb.setBlockToken(PBHelper.convert(proto.getBlockToken())); lb.setBlockToken(PBHelper.convert(proto.getBlockToken()));
return lb; return lb;

View File

@ -356,10 +356,10 @@ public class BlockInfo extends Block implements LightWeightGSet.LinkedElement {
* @return BlockInfoUnderConstruction - an under construction block. * @return BlockInfoUnderConstruction - an under construction block.
*/ */
public BlockInfoUnderConstruction convertToBlockUnderConstruction( public BlockInfoUnderConstruction convertToBlockUnderConstruction(
BlockUCState s, DatanodeDescriptor[] targets) { BlockUCState s, DatanodeStorageInfo[] targets) {
if(isComplete()) { if(isComplete()) {
return new BlockInfoUnderConstruction( return new BlockInfoUnderConstruction(this,
this, getBlockCollection().getBlockReplication(), s, targets); getBlockCollection().getBlockReplication(), s, targets);
} }
// the block is already under construction // the block is already under construction
BlockInfoUnderConstruction ucBlock = (BlockInfoUnderConstruction)this; BlockInfoUnderConstruction ucBlock = (BlockInfoUnderConstruction)this;

View File

@ -19,7 +19,6 @@ package org.apache.hadoop.hdfs.server.blockmanagement;
import java.io.IOException; import java.io.IOException;
import java.util.ArrayList; import java.util.ArrayList;
import java.util.Collections;
import java.util.Iterator; import java.util.Iterator;
import java.util.List; import java.util.List;
@ -64,12 +63,12 @@ public class BlockInfoUnderConstruction extends BlockInfo {
* corresponding replicas. * corresponding replicas.
*/ */
static class ReplicaUnderConstruction extends Block { static class ReplicaUnderConstruction extends Block {
private DatanodeDescriptor expectedLocation; private final DatanodeStorageInfo expectedLocation;
private ReplicaState state; private ReplicaState state;
private boolean chosenAsPrimary; private boolean chosenAsPrimary;
ReplicaUnderConstruction(Block block, ReplicaUnderConstruction(Block block,
DatanodeDescriptor target, DatanodeStorageInfo target,
ReplicaState state) { ReplicaState state) {
super(block); super(block);
this.expectedLocation = target; this.expectedLocation = target;
@ -83,7 +82,7 @@ public class BlockInfoUnderConstruction extends BlockInfo {
* It is not guaranteed, but expected, that the data-node actually has * It is not guaranteed, but expected, that the data-node actually has
* the replica. * the replica.
*/ */
DatanodeDescriptor getExpectedLocation() { private DatanodeStorageInfo getExpectedStorageLocation() {
return expectedLocation; return expectedLocation;
} }
@ -119,7 +118,7 @@ public class BlockInfoUnderConstruction extends BlockInfo {
* Is data-node the replica belongs to alive. * Is data-node the replica belongs to alive.
*/ */
boolean isAlive() { boolean isAlive() {
return expectedLocation.isAlive; return expectedLocation.getDatanodeDescriptor().isAlive;
} }
@Override // Block @Override // Block
@ -163,7 +162,7 @@ public class BlockInfoUnderConstruction extends BlockInfo {
*/ */
public BlockInfoUnderConstruction(Block blk, int replication, public BlockInfoUnderConstruction(Block blk, int replication,
BlockUCState state, BlockUCState state,
DatanodeDescriptor[] targets) { DatanodeStorageInfo[] targets) {
super(blk, replication); super(blk, replication);
assert getBlockUCState() != BlockUCState.COMPLETE : assert getBlockUCState() != BlockUCState.COMPLETE :
"BlockInfoUnderConstruction cannot be in COMPLETE state"; "BlockInfoUnderConstruction cannot be in COMPLETE state";
@ -187,7 +186,7 @@ public class BlockInfoUnderConstruction extends BlockInfo {
} }
/** Set expected locations */ /** Set expected locations */
public void setExpectedLocations(DatanodeDescriptor[] targets) { public void setExpectedLocations(DatanodeStorageInfo[] targets) {
int numLocations = targets == null ? 0 : targets.length; int numLocations = targets == null ? 0 : targets.length;
this.replicas = new ArrayList<ReplicaUnderConstruction>(numLocations); this.replicas = new ArrayList<ReplicaUnderConstruction>(numLocations);
for(int i = 0; i < numLocations; i++) for(int i = 0; i < numLocations; i++)
@ -199,12 +198,12 @@ public class BlockInfoUnderConstruction extends BlockInfo {
* Create array of expected replica locations * Create array of expected replica locations
* (as has been assigned by chooseTargets()). * (as has been assigned by chooseTargets()).
*/ */
public DatanodeDescriptor[] getExpectedLocations() { public DatanodeStorageInfo[] getExpectedStorageLocations() {
int numLocations = replicas == null ? 0 : replicas.size(); int numLocations = replicas == null ? 0 : replicas.size();
DatanodeDescriptor[] locations = new DatanodeDescriptor[numLocations]; DatanodeStorageInfo[] storages = new DatanodeStorageInfo[numLocations];
for(int i = 0; i < numLocations; i++) for(int i = 0; i < numLocations; i++)
locations[i] = replicas.get(i).getExpectedLocation(); storages[i] = replicas.get(i).getExpectedStorageLocation();
return locations; return storages;
} }
/** Get the number of expected locations */ /** Get the number of expected locations */
@ -279,27 +278,29 @@ public class BlockInfoUnderConstruction extends BlockInfo {
if (!(replicas.get(i).isAlive() && !replicas.get(i).getChosenAsPrimary())) { if (!(replicas.get(i).isAlive() && !replicas.get(i).getChosenAsPrimary())) {
continue; continue;
} }
if (replicas.get(i).getExpectedLocation().getLastUpdate() > mostRecentLastUpdate) { final ReplicaUnderConstruction ruc = replicas.get(i);
primary = replicas.get(i); final long lastUpdate = ruc.getExpectedStorageLocation().getDatanodeDescriptor().getLastUpdate();
if (lastUpdate > mostRecentLastUpdate) {
primaryNodeIndex = i; primaryNodeIndex = i;
mostRecentLastUpdate = primary.getExpectedLocation().getLastUpdate(); primary = ruc;
mostRecentLastUpdate = lastUpdate;
} }
} }
if (primary != null) { if (primary != null) {
primary.getExpectedLocation().addBlockToBeRecovered(this); primary.getExpectedStorageLocation().getDatanodeDescriptor().addBlockToBeRecovered(this);
primary.setChosenAsPrimary(true); primary.setChosenAsPrimary(true);
NameNode.blockStateChangeLog.info("BLOCK* " + this NameNode.blockStateChangeLog.info("BLOCK* " + this
+ " recovery started, primary=" + primary); + " recovery started, primary=" + primary);
} }
} }
void addReplicaIfNotPresent(DatanodeDescriptor dn, void addReplicaIfNotPresent(DatanodeStorageInfo storage,
Block block, Block block,
ReplicaState rState) { ReplicaState rState) {
for(ReplicaUnderConstruction r : replicas) for(ReplicaUnderConstruction r : replicas)
if(r.getExpectedLocation() == dn) if(r.getExpectedStorageLocation() == storage)
return; return;
replicas.add(new ReplicaUnderConstruction(block, dn, rState)); replicas.add(new ReplicaUnderConstruction(block, storage, rState));
} }
@Override // BlockInfo @Override // BlockInfo

View File

@ -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); for (DatanodeStorageInfo storage : blocksMap.getStorages(block)) {
jt.hasNext();) { final DatanodeDescriptor node = storage.getDatanodeDescriptor();
DatanodeDescriptor node = jt.next();
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 = BlockInfoUnderConstruction ucBlock = bc.setLastBlock(oldBlock, targets);
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) { for (DatanodeStorageInfo storage : targets) {
String datanodeId = dd.getStorageID(); invalidateBlocks.remove(storage.getStorageID(), oldBlock);
invalidateBlocks.remove(datanodeId, 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 = for(DatanodeStorageInfo storage : blocksMap.getStorages(block)) {
blocksMap.nodeIterator(block); it.hasNext();) { final String storageID = storage.getStorageID();
String storageID = it.next().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); for(DatanodeStorageInfo storage : blocksMap.getStorages(blk)) {
it.hasNext();) { final DatanodeDescriptor d = storage.getDatanodeDescriptor();
final DatanodeDescriptor d = it.next();
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 for(DatanodeStorageInfo storage : blocksMap.getStorages(b)) {
.hasNext();) { final DatanodeDescriptor node = storage.getDatanodeDescriptor();
DatanodeDescriptor node = it.next();
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()) { for(DatanodeStorageInfo storage : blocksMap.getStorages(block)) {
DatanodeDescriptor node = it.next(); 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); for(DatanodeStorageInfo storage : blocksMap.getStorages(block)) {
it.hasNext();) { final DatanodeDescriptor cur = storage.getDatanodeDescriptor();
DatanodeDescriptor cur = it.next();
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()) { for(DatanodeStorageInfo storage : blocksMap.getStorages(b)) {
DatanodeDescriptor node = nodeIter.next(); 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()) { for(DatanodeStorageInfo storage : blocksMap.getStorages(b)) {
DatanodeDescriptor node = nodeIter.next(); 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()) { for(DatanodeStorageInfo storage : blocksMap.getStorages(block)) {
DatanodeDescriptor node = nodeIter.next(); 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) { public DatanodeStorageInfo[] getStorages(BlockInfo block) {
DatanodeDescriptor[] nodes = final DatanodeStorageInfo[] storages = new DatanodeStorageInfo[block.numNodes()];
new DatanodeDescriptor[block.numNodes()]; int i = 0;
Iterator<DatanodeDescriptor> it = blocksMap.nodeIterator(block); for(DatanodeStorageInfo s : blocksMap.getStorages(block)) {
for (int i = 0; it != null && it.hasNext(); i++) { storages[i++] = s;
nodes[i] = it.next();
} }
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); for(DatanodeStorageInfo storage : blocksMap.getStorages(b)) {
it.hasNext();) { final DatanodeDescriptor cur = storage.getDatanodeDescriptor();
DatanodeDescriptor cur = it.next();
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) { public Iterable<DatanodeStorageInfo> getStorages(final Block block) {
return blocksMap.nodeIterator(block); return blocksMap.getStorages(block);
} }
public int numCorruptReplicas(Block block) { public int numCorruptReplicas(Block block) {

View File

@ -29,11 +29,11 @@ import org.apache.hadoop.util.LightWeightGSet;
* the datanodes that store the block. * the datanodes that store the block.
*/ */
class BlocksMap { class BlocksMap {
private static class NodeIterator implements Iterator<DatanodeDescriptor> { private static class StorageIterator implements Iterator<DatanodeStorageInfo> {
private BlockInfo blockInfo; private BlockInfo blockInfo;
private int nextIdx = 0; private int nextIdx = 0;
NodeIterator(BlockInfo blkInfo) { StorageIterator(BlockInfo blkInfo) {
this.blockInfo = blkInfo; this.blockInfo = blkInfo;
} }
@ -44,8 +44,8 @@ class BlocksMap {
} }
@Override @Override
public DatanodeDescriptor next() { public DatanodeStorageInfo next() {
return blockInfo.getDatanode(nextIdx++); return blockInfo.getStorageInfo(nextIdx++);
} }
@Override @Override
@ -115,18 +115,23 @@ class BlocksMap {
/** /**
* Searches for the block in the BlocksMap and * Searches for the block in the BlocksMap and
* returns Iterator that iterates through the nodes the block belongs to. * returns {@link Iterable} that iterates through the nodes the block belongs to.
*/ */
Iterator<DatanodeDescriptor> nodeIterator(Block b) { Iterable<DatanodeStorageInfo> getStorages(Block b) {
return nodeIterator(blocks.get(b)); return getStorages(blocks.get(b));
} }
/** /**
* For a block that has already been retrieved from the BlocksMap * For a block that has already been retrieved from the BlocksMap
* returns Iterator that iterates through the nodes the block belongs to. * returns {@link Iterable} that iterates through the nodes the block belongs to.
*/ */
Iterator<DatanodeDescriptor> nodeIterator(BlockInfo storedBlock) { Iterable<DatanodeStorageInfo> getStorages(final BlockInfo storedBlock) {
return new NodeIterator(storedBlock); return new Iterable<DatanodeStorageInfo>() {
@Override
public Iterator<DatanodeStorageInfo> iterator() {
return new StorageIterator(storedBlock);
}
};
} }
/** counts number of containing nodes. Better than using iterator. */ /** counts number of containing nodes. Better than using iterator. */

View File

@ -23,7 +23,13 @@ import java.io.IOException;
import java.io.PrintWriter; import java.io.PrintWriter;
import java.net.InetAddress; import java.net.InetAddress;
import java.net.UnknownHostException; import java.net.UnknownHostException;
import java.util.*; import java.util.ArrayList;
import java.util.Arrays;
import java.util.Comparator;
import java.util.Iterator;
import java.util.List;
import java.util.NavigableMap;
import java.util.TreeMap;
import org.apache.commons.logging.Log; import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory; import org.apache.commons.logging.LogFactory;
@ -48,10 +54,17 @@ import org.apache.hadoop.hdfs.server.namenode.HostFileManager.EntrySet;
import org.apache.hadoop.hdfs.server.namenode.HostFileManager.MutableEntrySet; import org.apache.hadoop.hdfs.server.namenode.HostFileManager.MutableEntrySet;
import org.apache.hadoop.hdfs.server.namenode.NameNode; import org.apache.hadoop.hdfs.server.namenode.NameNode;
import org.apache.hadoop.hdfs.server.namenode.Namesystem; import org.apache.hadoop.hdfs.server.namenode.Namesystem;
import org.apache.hadoop.hdfs.server.protocol.*; import org.apache.hadoop.hdfs.server.protocol.BalancerBandwidthCommand;
import org.apache.hadoop.hdfs.server.protocol.BlockCommand;
import org.apache.hadoop.hdfs.server.protocol.BlockRecoveryCommand;
import org.apache.hadoop.hdfs.server.protocol.BlockRecoveryCommand.RecoveringBlock; import org.apache.hadoop.hdfs.server.protocol.BlockRecoveryCommand.RecoveringBlock;
import org.apache.hadoop.hdfs.server.protocol.DatanodeCommand;
import org.apache.hadoop.hdfs.server.protocol.DatanodeProtocol;
import org.apache.hadoop.hdfs.server.protocol.DatanodeRegistration;
import org.apache.hadoop.hdfs.server.protocol.DatanodeStorage;
import org.apache.hadoop.hdfs.server.protocol.DisallowedDatanodeException;
import org.apache.hadoop.hdfs.server.protocol.RegisterCommand;
import org.apache.hadoop.hdfs.util.CyclicIteration; import org.apache.hadoop.hdfs.util.CyclicIteration;
import org.apache.hadoop.io.IOUtils;
import org.apache.hadoop.ipc.Server; import org.apache.hadoop.ipc.Server;
import org.apache.hadoop.net.CachedDNSToSwitchMapping; import org.apache.hadoop.net.CachedDNSToSwitchMapping;
import org.apache.hadoop.net.DNSToSwitchMapping; import org.apache.hadoop.net.DNSToSwitchMapping;
@ -423,6 +436,20 @@ public class DatanodeManager {
return node; return node;
} }
public DatanodeStorageInfo[] getDatanodeStorageInfos(
DatanodeID[] datanodeID, String[] storageIDs)
throws UnregisteredNodeException {
if (datanodeID.length == 0) {
return null;
}
final DatanodeStorageInfo[] storages = new DatanodeStorageInfo[datanodeID.length];
for(int i = 0; i < datanodeID.length; i++) {
final DatanodeDescriptor dd = getDatanode(datanodeID[i]);
storages[i] = dd.getStorageInfo(storageIDs[i]);
}
return storages;
}
/** Prints information about all datanodes. */ /** Prints information about all datanodes. */
void datanodeDump(final PrintWriter out) { void datanodeDump(final PrintWriter out) {
synchronized (datanodeMap) { synchronized (datanodeMap) {
@ -1151,32 +1178,32 @@ public class DatanodeManager {
BlockRecoveryCommand brCommand = new BlockRecoveryCommand( BlockRecoveryCommand brCommand = new BlockRecoveryCommand(
blocks.length); blocks.length);
for (BlockInfoUnderConstruction b : blocks) { for (BlockInfoUnderConstruction b : blocks) {
DatanodeDescriptor[] expectedLocations = b.getExpectedLocations(); final DatanodeStorageInfo[] storages = b.getExpectedStorageLocations();
// Skip stale nodes during recovery - not heart beated for some time (30s by default). // Skip stale nodes during recovery - not heart beated for some time (30s by default).
List<DatanodeDescriptor> recoveryLocations = final List<DatanodeStorageInfo> recoveryLocations =
new ArrayList<DatanodeDescriptor>(expectedLocations.length); new ArrayList<DatanodeStorageInfo>(storages.length);
for (int i = 0; i < expectedLocations.length; i++) { for (int i = 0; i < storages.length; i++) {
if (!expectedLocations[i].isStale(this.staleInterval)) { if (!storages[i].getDatanodeDescriptor().isStale(staleInterval)) {
recoveryLocations.add(expectedLocations[i]); recoveryLocations.add(storages[i]);
} }
} }
// If we only get 1 replica after eliminating stale nodes, then choose all // If we only get 1 replica after eliminating stale nodes, then choose all
// replicas for recovery and let the primary data node handle failures. // replicas for recovery and let the primary data node handle failures.
if (recoveryLocations.size() > 1) { if (recoveryLocations.size() > 1) {
if (recoveryLocations.size() != expectedLocations.length) { if (recoveryLocations.size() != storages.length) {
LOG.info("Skipped stale nodes for recovery : " + LOG.info("Skipped stale nodes for recovery : " +
(expectedLocations.length - recoveryLocations.size())); (storages.length - recoveryLocations.size()));
} }
brCommand.add(new RecoveringBlock( brCommand.add(new RecoveringBlock(
new ExtendedBlock(blockPoolId, b), new ExtendedBlock(blockPoolId, b),
recoveryLocations.toArray(new DatanodeDescriptor[recoveryLocations.size()]), DatanodeStorageInfo.toDatanodeInfos(recoveryLocations),
b.getBlockRecoveryId())); b.getBlockRecoveryId()));
} else { } else {
// If too many replicas are stale, then choose all replicas to participate // If too many replicas are stale, then choose all replicas to participate
// in block recovery. // in block recovery.
brCommand.add(new RecoveringBlock( brCommand.add(new RecoveringBlock(
new ExtendedBlock(blockPoolId, b), new ExtendedBlock(blockPoolId, b),
expectedLocations, DatanodeStorageInfo.toDatanodeInfos(storages),
b.getBlockRecoveryId())); b.getBlockRecoveryId()));
} }
} }

View File

@ -17,9 +17,12 @@
*/ */
package org.apache.hadoop.hdfs.server.blockmanagement; package org.apache.hadoop.hdfs.server.blockmanagement;
import java.util.Arrays;
import java.util.Iterator; import java.util.Iterator;
import java.util.List;
import org.apache.hadoop.hdfs.StorageType; import org.apache.hadoop.hdfs.StorageType;
import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
import org.apache.hadoop.hdfs.server.protocol.DatanodeStorage; import org.apache.hadoop.hdfs.server.protocol.DatanodeStorage;
import org.apache.hadoop.hdfs.server.protocol.DatanodeStorage.State; import org.apache.hadoop.hdfs.server.protocol.DatanodeStorage.State;
import org.apache.hadoop.hdfs.server.protocol.StorageReport; import org.apache.hadoop.hdfs.server.protocol.StorageReport;
@ -29,6 +32,17 @@ import org.apache.hadoop.hdfs.server.protocol.StorageReport;
* by this class. * by this class.
*/ */
public class DatanodeStorageInfo { public class DatanodeStorageInfo {
static DatanodeInfo[] toDatanodeInfos(DatanodeStorageInfo[] storages) {
return toDatanodeInfos(Arrays.asList(storages));
}
static DatanodeInfo[] toDatanodeInfos(List<DatanodeStorageInfo> storages) {
final DatanodeInfo[] datanodes = new DatanodeInfo[storages.size()];
for(int i = 0; i < storages.size(); i++) {
datanodes[i] = storages.get(i).getDatanodeDescriptor();
}
return datanodes;
}
/** /**
* Iterates over the list of blocks belonging to the data-node. * Iterates over the list of blocks belonging to the data-node.
*/ */
@ -65,7 +79,7 @@ public class DatanodeStorageInfo {
private long remaining; private long remaining;
private volatile BlockInfo blockList = null; private volatile BlockInfo blockList = null;
DatanodeStorageInfo(DatanodeDescriptor dn, DatanodeStorage s) { public DatanodeStorageInfo(DatanodeDescriptor dn, DatanodeStorage s) {
this.dn = dn; this.dn = dn;
this.storageID = s.getStorageID(); this.storageID = s.getStorageID();
this.storageType = s.getStorageType(); this.storageType = s.getStorageType();
@ -92,6 +106,10 @@ public class DatanodeStorageInfo {
return storageID; return storageID;
} }
public StorageType getStorageType() {
return storageType;
}
public long getCapacity() { public long getCapacity() {
return capacity; return capacity;
} }

View File

@ -34,5 +34,5 @@ public interface MutableBlockCollection extends BlockCollection {
* and set the locations. * and set the locations.
*/ */
public BlockInfoUnderConstruction setLastBlock(BlockInfo lastBlock, public BlockInfoUnderConstruction setLastBlock(BlockInfo lastBlock,
DatanodeDescriptor[] locations) throws IOException; DatanodeStorageInfo[] storages) throws IOException;
} }

View File

@ -61,6 +61,7 @@ import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoUnderConstruction; import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoUnderConstruction;
import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager; import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor; import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor;
import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeStorageInfo;
import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.BlockUCState; import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.BlockUCState;
import org.apache.hadoop.hdfs.server.namenode.INode.BlocksMapUpdateInfo; import org.apache.hadoop.hdfs.server.namenode.INode.BlocksMapUpdateInfo;
import org.apache.hadoop.hdfs.server.namenode.INodeReference.WithCount; import org.apache.hadoop.hdfs.server.namenode.INodeReference.WithCount;
@ -345,7 +346,7 @@ public class FSDirectory implements Closeable {
* Add a block to the file. Returns a reference to the added block. * Add a block to the file. Returns a reference to the added block.
*/ */
BlockInfo addBlock(String path, INodesInPath inodesInPath, Block block, BlockInfo addBlock(String path, INodesInPath inodesInPath, Block block,
DatanodeDescriptor targets[]) throws IOException { DatanodeStorageInfo[] targets) throws IOException {
waitForReady(); waitForReady();
writeLock(); writeLock();

View File

@ -168,6 +168,7 @@ import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor; import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor;
import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeManager; import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeManager;
import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeStatistics; import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeStatistics;
import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeStorageInfo;
import org.apache.hadoop.hdfs.server.blockmanagement.OutOfV1GenerationStampsException; import org.apache.hadoop.hdfs.server.blockmanagement.OutOfV1GenerationStampsException;
import org.apache.hadoop.hdfs.server.common.GenerationStamp; import org.apache.hadoop.hdfs.server.common.GenerationStamp;
import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.BlockUCState; import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.BlockUCState;
@ -2484,8 +2485,14 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
} }
// choose targets for the new block to be allocated. // choose targets for the new block to be allocated.
final DatanodeDescriptor targets[] = getBlockManager().chooseTarget( // TODO: chooseTarget(..) should be changed to return DatanodeStorageInfo's
final DatanodeDescriptor chosenDatanodes[] = getBlockManager().chooseTarget(
src, replication, clientNode, excludedNodes, blockSize, favoredNodes); src, replication, clientNode, excludedNodes, blockSize, favoredNodes);
final DatanodeStorageInfo[] targets = new DatanodeStorageInfo[chosenDatanodes.length];
for(int i = 0; i < targets.length; i++) {
final DatanodeDescriptor dd = chosenDatanodes[i];
targets[i] = dd.getStorageInfos().iterator().next();
}
// Part II. // Part II.
// Allocate a new block, add it to the INode and the BlocksMap. // Allocate a new block, add it to the INode and the BlocksMap.
@ -2607,7 +2614,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
src + ". Returning previously allocated block " + lastBlockInFile); src + ". Returning previously allocated block " + lastBlockInFile);
long offset = pendingFile.computeFileSize(); long offset = pendingFile.computeFileSize();
onRetryBlock[0] = makeLocatedBlock(lastBlockInFile, onRetryBlock[0] = makeLocatedBlock(lastBlockInFile,
((BlockInfoUnderConstruction)lastBlockInFile).getExpectedLocations(), ((BlockInfoUnderConstruction)lastBlockInFile).getExpectedStorageLocations(),
offset); offset);
return iip; return iip;
} else { } else {
@ -2625,11 +2632,10 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
return iip; return iip;
} }
LocatedBlock makeLocatedBlock(Block blk, LocatedBlock makeLocatedBlock(Block blk, DatanodeStorageInfo[] locs,
DatanodeInfo[] locs,
long offset) throws IOException { long offset) throws IOException {
LocatedBlock lBlk = new LocatedBlock( LocatedBlock lBlk = LocatedBlock.createLocatedBlock(
getExtendedBlock(blk), locs, offset); getExtendedBlock(blk), locs, offset, false);
getBlockManager().setBlockToken( getBlockManager().setBlockToken(
lBlk, BlockTokenSecretManager.AccessMode.WRITE); lBlk, BlockTokenSecretManager.AccessMode.WRITE);
return lBlk; return lBlk;
@ -2852,13 +2858,14 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
* @throws QuotaExceededException If addition of block exceeds space quota * @throws QuotaExceededException If addition of block exceeds space quota
*/ */
BlockInfo saveAllocatedBlock(String src, INodesInPath inodesInPath, BlockInfo saveAllocatedBlock(String src, INodesInPath inodesInPath,
Block newBlock, DatanodeDescriptor targets[]) throws IOException { Block newBlock, DatanodeStorageInfo[] targets)
throws IOException {
assert hasWriteLock(); assert hasWriteLock();
BlockInfo b = dir.addBlock(src, inodesInPath, newBlock, targets); BlockInfo b = dir.addBlock(src, inodesInPath, newBlock, targets);
NameNode.stateChangeLog.info("BLOCK* allocateBlock: " + src + ". " NameNode.stateChangeLog.info("BLOCK* allocateBlock: " + src + ". "
+ getBlockPoolId() + " " + b); + getBlockPoolId() + " " + b);
for (DatanodeDescriptor dn : targets) { for (DatanodeStorageInfo storage : targets) {
dn.incBlocksScheduled(); storage.getDatanodeDescriptor().incBlocksScheduled();
} }
return b; return b;
} }
@ -3622,7 +3629,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
final BlockInfoUnderConstruction uc = (BlockInfoUnderConstruction)lastBlock; final BlockInfoUnderConstruction uc = (BlockInfoUnderConstruction)lastBlock;
// setup the last block locations from the blockManager if not known // setup the last block locations from the blockManager if not known
if (uc.getNumExpectedLocations() == 0) { if (uc.getNumExpectedLocations() == 0) {
uc.setExpectedLocations(blockManager.getNodes(lastBlock)); uc.setExpectedLocations(blockManager.getStorages(lastBlock));
} }
// start recovery of the last block for this file // start recovery of the last block for this file
long blockRecoveryId = nextGenerationStamp(isLegacyBlock(uc)); long blockRecoveryId = nextGenerationStamp(isLegacyBlock(uc));
@ -3777,24 +3784,18 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
// find the DatanodeDescriptor objects // find the DatanodeDescriptor objects
// There should be no locations in the blockManager till now because the // There should be no locations in the blockManager till now because the
// file is underConstruction // file is underConstruction
DatanodeDescriptor[] descriptors = null; final DatanodeStorageInfo[] storages = blockManager.getDatanodeManager()
if (newtargets.length > 0) { .getDatanodeStorageInfos(newtargets, newtargetstorages);
descriptors = new DatanodeDescriptor[newtargets.length]; if (closeFile && storages != null) {
for(int i = 0; i < newtargets.length; i++) {
descriptors[i] = blockManager.getDatanodeManager().getDatanode(
newtargets[i]);
}
}
if ((closeFile) && (descriptors != null)) {
// the file is getting closed. Insert block locations into blockManager. // the file is getting closed. Insert block locations into blockManager.
// Otherwise fsck will report these blocks as MISSING, especially if the // Otherwise fsck will report these blocks as MISSING, especially if the
// blocksReceived from Datanodes take a long time to arrive. // blocksReceived from Datanodes take a long time to arrive.
for (int i = 0; i < descriptors.length; i++) { for (int i = 0; i < storages.length; i++) {
descriptors[i].addBlock(newtargetstorages[i], storedBlock); storages[i].addBlock(storedBlock);
} }
} }
// add pipeline locations into the INodeUnderConstruction // add pipeline locations into the INodeUnderConstruction
pendingFile.setLastBlock(storedBlock, descriptors); pendingFile.setLastBlock(storedBlock, storages);
} }
if (closeFile) { if (closeFile) {
@ -5639,7 +5640,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
* @throws IOException if any error occurs * @throws IOException if any error occurs
*/ */
void updatePipeline(String clientName, ExtendedBlock oldBlock, void updatePipeline(String clientName, ExtendedBlock oldBlock,
ExtendedBlock newBlock, DatanodeID[] newNodes) ExtendedBlock newBlock, DatanodeID[] newNodes, String[] newStorageIDs)
throws IOException { throws IOException {
CacheEntry cacheEntry = RetryCache.waitForCompletion(retryCache); CacheEntry cacheEntry = RetryCache.waitForCompletion(retryCache);
if (cacheEntry != null && cacheEntry.isSuccess()) { if (cacheEntry != null && cacheEntry.isSuccess()) {
@ -5662,7 +5663,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
assert newBlock.getBlockId()==oldBlock.getBlockId() : newBlock + " and " assert newBlock.getBlockId()==oldBlock.getBlockId() : newBlock + " and "
+ oldBlock + " has different block identifier"; + oldBlock + " has different block identifier";
updatePipelineInternal(clientName, oldBlock, newBlock, newNodes, updatePipelineInternal(clientName, oldBlock, newBlock, newNodes,
cacheEntry != null); newStorageIDs, cacheEntry != null);
success = true; success = true;
} finally { } finally {
writeUnlock(); writeUnlock();
@ -5674,7 +5675,8 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
/** @see #updatePipeline(String, ExtendedBlock, ExtendedBlock, DatanodeID[]) */ /** @see #updatePipeline(String, ExtendedBlock, ExtendedBlock, DatanodeID[]) */
private void updatePipelineInternal(String clientName, ExtendedBlock oldBlock, private void updatePipelineInternal(String clientName, ExtendedBlock oldBlock,
ExtendedBlock newBlock, DatanodeID[] newNodes, boolean logRetryCache) ExtendedBlock newBlock, DatanodeID[] newNodes, String[] newStorageIDs,
boolean logRetryCache)
throws IOException { throws IOException {
assert hasWriteLock(); assert hasWriteLock();
// check the vadility of the block and lease holder name // check the vadility of the block and lease holder name
@ -5698,15 +5700,9 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
blockinfo.setNumBytes(newBlock.getNumBytes()); blockinfo.setNumBytes(newBlock.getNumBytes());
// find the DatanodeDescriptor objects // find the DatanodeDescriptor objects
final DatanodeManager dm = getBlockManager().getDatanodeManager(); final DatanodeStorageInfo[] storages = blockManager.getDatanodeManager()
DatanodeDescriptor[] descriptors = null; .getDatanodeStorageInfos(newNodes, newStorageIDs);
if (newNodes.length > 0) { blockinfo.setExpectedLocations(storages);
descriptors = new DatanodeDescriptor[newNodes.length];
for(int i = 0; i < newNodes.length; i++) {
descriptors[i] = dm.getDatanode(newNodes[i]);
}
}
blockinfo.setExpectedLocations(descriptors);
String src = leaseManager.findPath(pendingFile); String src = leaseManager.findPath(pendingFile);
dir.persistBlocks(src, pendingFile, logRetryCache); dir.persistBlocks(src, pendingFile, logRetryCache);

View File

@ -28,6 +28,7 @@ import org.apache.hadoop.hdfs.protocol.QuotaExceededException;
import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo; import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoUnderConstruction; import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoUnderConstruction;
import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor; import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor;
import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeStorageInfo;
import org.apache.hadoop.hdfs.server.blockmanagement.MutableBlockCollection; import org.apache.hadoop.hdfs.server.blockmanagement.MutableBlockCollection;
import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.BlockUCState; import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.BlockUCState;
import org.apache.hadoop.hdfs.server.namenode.snapshot.INodeFileUnderConstructionWithSnapshot; import org.apache.hadoop.hdfs.server.namenode.snapshot.INodeFileUnderConstructionWithSnapshot;
@ -180,7 +181,7 @@ public class INodeFileUnderConstruction extends INodeFile implements MutableBloc
*/ */
@Override @Override
public BlockInfoUnderConstruction setLastBlock(BlockInfo lastBlock, public BlockInfoUnderConstruction setLastBlock(BlockInfo lastBlock,
DatanodeDescriptor[] targets) throws IOException { DatanodeStorageInfo[] targets) throws IOException {
if (numBlocks() == 0) { if (numBlocks() == 0) {
throw new IOException("Failed to set last block: File is empty."); throw new IOException("Failed to set last block: File is empty.");
} }

View File

@ -611,9 +611,9 @@ class NameNodeRpcServer implements NamenodeProtocols {
@Override // ClientProtocol @Override // ClientProtocol
public void updatePipeline(String clientName, ExtendedBlock oldBlock, public void updatePipeline(String clientName, ExtendedBlock oldBlock,
ExtendedBlock newBlock, DatanodeID[] newNodes) ExtendedBlock newBlock, DatanodeID[] newNodes, String[] newStorageIDs)
throws IOException { throws IOException {
namesystem.updatePipeline(clientName, oldBlock, newBlock, newNodes); namesystem.updatePipeline(clientName, oldBlock, newBlock, newNodes, newStorageIDs);
} }
@Override // DatanodeProtocol @Override // DatanodeProtocol

View File

@ -48,6 +48,7 @@ import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifie
import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager; import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor; import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor;
import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeManager; import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeManager;
import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeStorageInfo;
import org.apache.hadoop.hdfs.server.common.JspHelper; import org.apache.hadoop.hdfs.server.common.JspHelper;
import org.apache.hadoop.hdfs.server.common.Storage; import org.apache.hadoop.hdfs.server.common.Storage;
import org.apache.hadoop.hdfs.server.common.Storage.StorageDirectory; import org.apache.hadoop.hdfs.server.common.Storage.StorageDirectory;
@ -1065,11 +1066,10 @@ class NamenodeJspHelper {
} }
doc.startTag("replicas"); doc.startTag("replicas");
for(final Iterator<DatanodeDescriptor> it = blockManager.datanodeIterator(block); for(DatanodeStorageInfo storage : blockManager.getStorages(block)) {
it.hasNext(); ) {
doc.startTag("replica"); doc.startTag("replica");
DatanodeDescriptor dd = it.next(); DatanodeDescriptor dd = storage.getDatanodeDescriptor();
doc.startTag("host_name"); doc.startTag("host_name");
doc.pcdata(dd.getHostName()); doc.pcdata(dd.getHostName());

View File

@ -437,6 +437,7 @@ message UpdatePipelineRequestProto {
required ExtendedBlockProto oldBlock = 2; required ExtendedBlockProto oldBlock = 2;
required ExtendedBlockProto newBlock = 3; required ExtendedBlockProto newBlock = 3;
repeated DatanodeIDProto newNodes = 4; repeated DatanodeIDProto newNodes = 4;
repeated string storageIDs = 5;
} }
message UpdatePipelineResponseProto { // void response message UpdatePipelineResponseProto { // void response

View File

@ -134,6 +134,7 @@ message LocatedBlockProto {
required hadoop.common.TokenProto blockToken = 5; required hadoop.common.TokenProto blockToken = 5;
repeated StorageTypeProto storageTypes = 6; repeated StorageTypeProto storageTypes = 6;
repeated string storageIDs = 7;
} }
message DataEncryptionKeyProto { message DataEncryptionKeyProto {

View File

@ -61,8 +61,8 @@ import org.apache.hadoop.fs.FileContext;
import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.FileSystem.Statistics; import org.apache.hadoop.fs.FileSystem.Statistics;
import org.apache.hadoop.fs.Options.Rename; import org.apache.hadoop.fs.Options.Rename;
import org.apache.hadoop.fs.permission.FsPermission;
import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.Path;
import org.apache.hadoop.fs.permission.FsPermission;
import org.apache.hadoop.hdfs.MiniDFSCluster.NameNodeInfo; import org.apache.hadoop.hdfs.MiniDFSCluster.NameNodeInfo;
import org.apache.hadoop.hdfs.client.HdfsDataInputStream; import org.apache.hadoop.hdfs.client.HdfsDataInputStream;
import org.apache.hadoop.hdfs.protocol.DatanodeID; import org.apache.hadoop.hdfs.protocol.DatanodeID;
@ -79,6 +79,7 @@ import org.apache.hadoop.hdfs.security.token.block.ExportedBlockKeys;
import org.apache.hadoop.hdfs.server.blockmanagement.BlockManagerTestUtil; import org.apache.hadoop.hdfs.server.blockmanagement.BlockManagerTestUtil;
import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor; import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor;
import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeManager; import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeManager;
import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeStorageInfo;
import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.StartupOption; import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.StartupOption;
import org.apache.hadoop.hdfs.server.common.StorageInfo; import org.apache.hadoop.hdfs.server.common.StorageInfo;
import org.apache.hadoop.hdfs.server.datanode.DataNode; import org.apache.hadoop.hdfs.server.datanode.DataNode;
@ -86,6 +87,7 @@ import org.apache.hadoop.hdfs.server.datanode.TestTransferRbw;
import org.apache.hadoop.hdfs.server.namenode.FSNamesystem; import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
import org.apache.hadoop.hdfs.server.namenode.NameNode; import org.apache.hadoop.hdfs.server.namenode.NameNode;
import org.apache.hadoop.hdfs.server.protocol.DatanodeRegistration; import org.apache.hadoop.hdfs.server.protocol.DatanodeRegistration;
import org.apache.hadoop.hdfs.server.protocol.DatanodeStorage;
import org.apache.hadoop.io.IOUtils; import org.apache.hadoop.io.IOUtils;
import org.apache.hadoop.net.NetUtils; import org.apache.hadoop.net.NetUtils;
import org.apache.hadoop.security.ShellBasedUnixGroupsMapping; import org.apache.hadoop.security.ShellBasedUnixGroupsMapping;
@ -856,6 +858,13 @@ public class DFSTestUtil {
rackLocation); rackLocation);
} }
public static DatanodeStorageInfo createDatanodeStorageInfo(
String storageID, String ip) {
return new DatanodeStorageInfo(
getDatanodeDescriptor(ip, "defaultRack"),
new DatanodeStorage(storageID));
}
public static DatanodeDescriptor getDatanodeDescriptor(String ipAddr, public static DatanodeDescriptor getDatanodeDescriptor(String ipAddr,
int port, String rackLocation) { int port, String rackLocation) {
DatanodeID dnId = new DatanodeID(ipAddr, "host", "", port, DatanodeID dnId = new DatanodeID(ipAddr, "host", "", port,

View File

@ -85,9 +85,8 @@ public class BlockManagerTestUtil {
final Set<String> rackSet = new HashSet<String>(0); final Set<String> rackSet = new HashSet<String>(0);
final Collection<DatanodeDescriptor> corruptNodes = final Collection<DatanodeDescriptor> corruptNodes =
getCorruptReplicas(blockManager).getNodes(b); getCorruptReplicas(blockManager).getNodes(b);
for (Iterator<DatanodeDescriptor> it = blockManager.blocksMap.nodeIterator(b); for(DatanodeStorageInfo storage : blockManager.blocksMap.getStorages(b)) {
it.hasNext();) { final DatanodeDescriptor cur = storage.getDatanodeDescriptor();
DatanodeDescriptor cur = it.next();
if (!cur.isDecommissionInProgress() && !cur.isDecommissioned()) { if (!cur.isDecommissionInProgress() && !cur.isDecommissioned()) {
if ((corruptNodes == null ) || !corruptNodes.contains(cur)) { if ((corruptNodes == null ) || !corruptNodes.contains(cur)) {
String rackName = cur.getNetworkLocation(); String rackName = cur.getNetworkLocation();

View File

@ -31,18 +31,19 @@ import org.junit.Test;
public class TestBlockInfoUnderConstruction { public class TestBlockInfoUnderConstruction {
@Test @Test
public void testInitializeBlockRecovery() throws Exception { public void testInitializeBlockRecovery() throws Exception {
DatanodeDescriptor dd1 = DFSTestUtil.getDatanodeDescriptor("10.10.1.1", DatanodeStorageInfo s1 = DFSTestUtil.createDatanodeStorageInfo("10.10.1.1", "s1");
"default"); DatanodeDescriptor dd1 = s1.getDatanodeDescriptor();
DatanodeDescriptor dd2 = DFSTestUtil.getDatanodeDescriptor("10.10.1.2", DatanodeStorageInfo s2 = DFSTestUtil.createDatanodeStorageInfo("10.10.1.2", "s2");
"default"); DatanodeDescriptor dd2 = s2.getDatanodeDescriptor();
DatanodeDescriptor dd3 = DFSTestUtil.getDatanodeDescriptor("10.10.1.3", DatanodeStorageInfo s3 = DFSTestUtil.createDatanodeStorageInfo("10.10.1.3", "s3");
"default"); DatanodeDescriptor dd3 = s3.getDatanodeDescriptor();
dd1.isAlive = dd2.isAlive = dd3.isAlive = true; dd1.isAlive = dd2.isAlive = dd3.isAlive = true;
BlockInfoUnderConstruction blockInfo = new BlockInfoUnderConstruction( BlockInfoUnderConstruction blockInfo = new BlockInfoUnderConstruction(
new Block(0, 0, GenerationStamp.LAST_RESERVED_STAMP), new Block(0, 0, GenerationStamp.LAST_RESERVED_STAMP),
3, 3,
BlockUCState.UNDER_CONSTRUCTION, BlockUCState.UNDER_CONSTRUCTION,
new DatanodeDescriptor[] {dd1, dd2, dd3}); new DatanodeStorageInfo[] {s1, s2, s3});
// Recovery attempt #1. // Recovery attempt #1.
long currentTime = System.currentTimeMillis(); long currentTime = System.currentTimeMillis();

View File

@ -20,7 +20,6 @@ package org.apache.hadoop.hdfs.server.blockmanagement;
import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertEquals;
import java.util.ArrayList; import java.util.ArrayList;
import java.util.List;
import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hdfs.DFSConfigKeys; import org.apache.hadoop.hdfs.DFSConfigKeys;
@ -162,10 +161,13 @@ public class TestHeartbeatHandling {
dd1.setLastUpdate(System.currentTimeMillis()); dd1.setLastUpdate(System.currentTimeMillis());
dd2.setLastUpdate(System.currentTimeMillis()); dd2.setLastUpdate(System.currentTimeMillis());
dd3.setLastUpdate(System.currentTimeMillis()); dd3.setLastUpdate(System.currentTimeMillis());
final DatanodeStorageInfo[] storages = {
dd1.getStorageInfos().iterator().next(),
dd2.getStorageInfos().iterator().next(),
dd3.getStorageInfos().iterator().next()};
BlockInfoUnderConstruction blockInfo = new BlockInfoUnderConstruction( BlockInfoUnderConstruction blockInfo = new BlockInfoUnderConstruction(
new Block(0, 0, GenerationStamp.LAST_RESERVED_STAMP), 3, new Block(0, 0, GenerationStamp.LAST_RESERVED_STAMP), 3,
BlockUCState.UNDER_RECOVERY, BlockUCState.UNDER_RECOVERY, storages);
new DatanodeDescriptor[] {dd1, dd2, dd3});
dd1.addBlockToBeRecovered(blockInfo); dd1.addBlockToBeRecovered(blockInfo);
DatanodeCommand[] cmds = DatanodeCommand[] cmds =
NameNodeAdapter.sendHeartBeat(nodeReg1, dd1, namesystem).getCommands(); NameNodeAdapter.sendHeartBeat(nodeReg1, dd1, namesystem).getCommands();
@ -187,8 +189,7 @@ public class TestHeartbeatHandling {
dd3.setLastUpdate(System.currentTimeMillis()); dd3.setLastUpdate(System.currentTimeMillis());
blockInfo = new BlockInfoUnderConstruction( blockInfo = new BlockInfoUnderConstruction(
new Block(0, 0, GenerationStamp.LAST_RESERVED_STAMP), 3, new Block(0, 0, GenerationStamp.LAST_RESERVED_STAMP), 3,
BlockUCState.UNDER_RECOVERY, BlockUCState.UNDER_RECOVERY, storages);
new DatanodeDescriptor[] {dd1, dd2, dd3});
dd1.addBlockToBeRecovered(blockInfo); dd1.addBlockToBeRecovered(blockInfo);
cmds = NameNodeAdapter.sendHeartBeat(nodeReg1, dd1, namesystem).getCommands(); cmds = NameNodeAdapter.sendHeartBeat(nodeReg1, dd1, namesystem).getCommands();
assertEquals(1, cmds.length); assertEquals(1, cmds.length);
@ -209,8 +210,7 @@ public class TestHeartbeatHandling {
dd3.setLastUpdate(System.currentTimeMillis() - 80 * 1000); dd3.setLastUpdate(System.currentTimeMillis() - 80 * 1000);
blockInfo = new BlockInfoUnderConstruction( blockInfo = new BlockInfoUnderConstruction(
new Block(0, 0, GenerationStamp.LAST_RESERVED_STAMP), 3, new Block(0, 0, GenerationStamp.LAST_RESERVED_STAMP), 3,
BlockUCState.UNDER_RECOVERY, BlockUCState.UNDER_RECOVERY, storages);
new DatanodeDescriptor[] {dd1, dd2, dd3});
dd1.addBlockToBeRecovered(blockInfo); dd1.addBlockToBeRecovered(blockInfo);
cmds = NameNodeAdapter.sendHeartBeat(nodeReg1, dd1, namesystem).getCommands(); cmds = NameNodeAdapter.sendHeartBeat(nodeReg1, dd1, namesystem).getCommands();
assertEquals(1, cmds.length); assertEquals(1, cmds.length);

View File

@ -98,11 +98,9 @@ public class TestNodeCount {
} }
// find out a non-excess node // find out a non-excess node
final Iterator<DatanodeDescriptor> iter = bm.blocksMap
.nodeIterator(block.getLocalBlock());
DatanodeDescriptor nonExcessDN = null; DatanodeDescriptor nonExcessDN = null;
while (iter.hasNext()) { for(DatanodeStorageInfo storage : bm.blocksMap.getStorages(block.getLocalBlock())) {
DatanodeDescriptor dn = iter.next(); final DatanodeDescriptor dn = storage.getDatanodeDescriptor();
Collection<Block> blocks = bm.excessReplicateMap.get(dn.getStorageID()); Collection<Block> blocks = bm.excessReplicateMap.get(dn.getStorageID());
if (blocks == null || !blocks.contains(block.getLocalBlock()) ) { if (blocks == null || !blocks.contains(block.getLocalBlock()) ) {
nonExcessDN = dn; nonExcessDN = dn;

View File

@ -51,6 +51,7 @@ import org.apache.hadoop.hdfs.server.datanode.DataNodeTestUtils;
import org.apache.hadoop.hdfs.server.namenode.FSClusterStats; import org.apache.hadoop.hdfs.server.namenode.FSClusterStats;
import org.apache.hadoop.hdfs.server.namenode.NameNode; import org.apache.hadoop.hdfs.server.namenode.NameNode;
import org.apache.hadoop.hdfs.server.namenode.Namesystem; import org.apache.hadoop.hdfs.server.namenode.Namesystem;
import org.apache.hadoop.hdfs.server.protocol.DatanodeStorage;
import org.apache.hadoop.net.NetworkTopology; import org.apache.hadoop.net.NetworkTopology;
import org.apache.hadoop.net.Node; import org.apache.hadoop.net.Node;
import org.apache.hadoop.util.Time; import org.apache.hadoop.util.Time;
@ -71,6 +72,7 @@ public class TestReplicationPolicy {
private static BlockPlacementPolicy replicator; private static BlockPlacementPolicy replicator;
private static final String filename = "/dummyfile.txt"; private static final String filename = "/dummyfile.txt";
private static DatanodeDescriptor dataNodes[]; private static DatanodeDescriptor dataNodes[];
private static String[] storageIDs;
// The interval for marking a datanode as stale, // The interval for marking a datanode as stale,
private static long staleInterval = private static long staleInterval =
DFSConfigKeys.DFS_NAMENODE_STALE_DATANODE_INTERVAL_DEFAULT; DFSConfigKeys.DFS_NAMENODE_STALE_DATANODE_INTERVAL_DEFAULT;
@ -1143,11 +1145,12 @@ public class TestReplicationPolicy {
info.setBlockCollection(mbc); info.setBlockCollection(mbc);
bm.addBlockCollection(info, mbc); bm.addBlockCollection(info, mbc);
DatanodeDescriptor[] dnAry = {dataNodes[0]}; DatanodeStorageInfo[] storageAry = {new DatanodeStorageInfo(
dataNodes[0], new DatanodeStorage("s1"))};
final BlockInfoUnderConstruction ucBlock = final BlockInfoUnderConstruction ucBlock =
info.convertToBlockUnderConstruction(BlockUCState.UNDER_CONSTRUCTION, info.convertToBlockUnderConstruction(BlockUCState.UNDER_CONSTRUCTION,
dnAry); storageAry);
when(mbc.setLastBlock((BlockInfo) any(), (DatanodeDescriptor[]) any())) when(mbc.setLastBlock((BlockInfo) any(), (DatanodeStorageInfo[]) any()))
.thenReturn(ucBlock); .thenReturn(ucBlock);
bm.convertLastBlockToUnderConstruction(mbc); bm.convertLastBlockToUnderConstruction(mbc);

View File

@ -48,7 +48,8 @@ public class TestUnderReplicatedBlocks {
// but the block does not get put into the under-replicated blocks queue // but the block does not get put into the under-replicated blocks queue
final BlockManager bm = cluster.getNamesystem().getBlockManager(); final BlockManager bm = cluster.getNamesystem().getBlockManager();
ExtendedBlock b = DFSTestUtil.getFirstBlock(fs, FILE_PATH); ExtendedBlock b = DFSTestUtil.getFirstBlock(fs, FILE_PATH);
DatanodeDescriptor dn = bm.blocksMap.nodeIterator(b.getLocalBlock()).next(); DatanodeDescriptor dn = bm.blocksMap.getStorages(b.getLocalBlock())
.iterator().next().getDatanodeDescriptor();
bm.addToInvalidates(b.getLocalBlock(), dn); bm.addToInvalidates(b.getLocalBlock(), dn);
// Compute the invalidate work in NN, and trigger the heartbeat from DN // Compute the invalidate work in NN, and trigger the heartbeat from DN
BlockManagerTestUtil.computeAllPendingWork(bm); BlockManagerTestUtil.computeAllPendingWork(bm);

View File

@ -33,7 +33,7 @@ import org.apache.hadoop.hdfs.protocol.DatanodeID;
import org.apache.hadoop.hdfs.protocol.ExtendedBlock; import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo; import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoUnderConstruction; import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoUnderConstruction;
import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor; import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeStorageInfo;
import org.apache.hadoop.hdfs.server.common.HdfsServerConstants; import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
import org.junit.Test; import org.junit.Test;
@ -50,7 +50,7 @@ public class TestCommitBlockSynchronization {
throws IOException { throws IOException {
Configuration conf = new Configuration(); Configuration conf = new Configuration();
FSImage image = new FSImage(conf); FSImage image = new FSImage(conf);
DatanodeDescriptor[] targets = new DatanodeDescriptor[0]; final DatanodeStorageInfo[] targets = {};
FSNamesystem namesystem = new FSNamesystem(conf, image); FSNamesystem namesystem = new FSNamesystem(conf, image);
FSNamesystem namesystemSpy = spy(namesystem); FSNamesystem namesystemSpy = spy(namesystem);

View File

@ -47,6 +47,7 @@ import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoUnderConstruction;
import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager; import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
import org.apache.hadoop.hdfs.server.blockmanagement.BlockManagerTestUtil; import org.apache.hadoop.hdfs.server.blockmanagement.BlockManagerTestUtil;
import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor; import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor;
import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeStorageInfo;
import org.apache.hadoop.hdfs.server.datanode.DataNode; import org.apache.hadoop.hdfs.server.datanode.DataNode;
import org.apache.hadoop.hdfs.server.datanode.DataNodeTestUtils; import org.apache.hadoop.hdfs.server.datanode.DataNodeTestUtils;
import org.apache.hadoop.hdfs.server.namenode.FSNamesystem; import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
@ -523,16 +524,17 @@ public class TestPipelinesFailover {
(BlockInfoUnderConstruction)storedBlock; (BlockInfoUnderConstruction)storedBlock;
// We expect that the replica with the most recent heart beat will be // We expect that the replica with the most recent heart beat will be
// the one to be in charge of the synchronization / recovery protocol. // the one to be in charge of the synchronization / recovery protocol.
DatanodeDescriptor[] datanodes = ucBlock.getExpectedLocations(); final DatanodeStorageInfo[] storages = ucBlock.getExpectedStorageLocations();
DatanodeDescriptor expectedPrimary = datanodes[0]; DatanodeStorageInfo expectedPrimary = storages[0];
long mostRecentLastUpdate = expectedPrimary.getLastUpdate(); long mostRecentLastUpdate = expectedPrimary.getDatanodeDescriptor().getLastUpdate();
for (int i = 1; i < datanodes.length; i++) { for (int i = 1; i < storages.length; i++) {
if (datanodes[i].getLastUpdate() > mostRecentLastUpdate) { final long lastUpdate = storages[i].getDatanodeDescriptor().getLastUpdate();
expectedPrimary = datanodes[i]; if (lastUpdate > mostRecentLastUpdate) {
mostRecentLastUpdate = expectedPrimary.getLastUpdate(); expectedPrimary = storages[i];
mostRecentLastUpdate = lastUpdate;
} }
} }
return expectedPrimary; return expectedPrimary.getDatanodeDescriptor();
} }
private DistributedFileSystem createFsAsOtherUser( private DistributedFileSystem createFsAsOtherUser(

View File

@ -702,9 +702,10 @@ public class TestRetryCacheWithHA {
DatanodeInfo[] newNodes = new DatanodeInfo[2]; DatanodeInfo[] newNodes = new DatanodeInfo[2];
newNodes[0] = nodes[0]; newNodes[0] = nodes[0];
newNodes[1] = nodes[1]; newNodes[1] = nodes[1];
String[] storageIDs = {"s0", "s1"};
client.getNamenode().updatePipeline(client.getClientName(), oldBlock, client.getNamenode().updatePipeline(client.getClientName(), oldBlock,
newBlock, newNodes); newBlock, newNodes, storageIDs);
out.close(); out.close();
} }
@ -714,10 +715,10 @@ public class TestRetryCacheWithHA {
.getNamesystem(0).getFSDirectory().getINode4Write(file).asFile(); .getNamesystem(0).getFSDirectory().getINode4Write(file).asFile();
BlockInfoUnderConstruction blkUC = BlockInfoUnderConstruction blkUC =
(BlockInfoUnderConstruction) (fileNode.getBlocks())[1]; (BlockInfoUnderConstruction) (fileNode.getBlocks())[1];
int datanodeNum = blkUC.getExpectedLocations().length; int datanodeNum = blkUC.getExpectedStorageLocations().length;
for (int i = 0; i < CHECKTIMES && datanodeNum != 2; i++) { for (int i = 0; i < CHECKTIMES && datanodeNum != 2; i++) {
Thread.sleep(1000); Thread.sleep(1000);
datanodeNum = blkUC.getExpectedLocations().length; datanodeNum = blkUC.getExpectedStorageLocations().length;
} }
return datanodeNum == 2; return datanodeNum == 2;
} }