|
@@ -23,11 +23,13 @@ import org.apache.hadoop.io.*;
|
|
|
import org.apache.hadoop.conf.*;
|
|
|
import org.apache.hadoop.util.*;
|
|
|
import org.apache.hadoop.mapred.StatusHttpServer;
|
|
|
+import org.apache.hadoop.net.NetworkTopology;
|
|
|
+import org.apache.hadoop.net.NodeBase;
|
|
|
import org.apache.hadoop.fs.Path;
|
|
|
|
|
|
import java.io.*;
|
|
|
-import java.net.InetSocketAddress;
|
|
|
import java.util.*;
|
|
|
+
|
|
|
import javax.servlet.ServletContext;
|
|
|
import javax.servlet.ServletException;
|
|
|
import javax.servlet.http.HttpServlet;
|
|
@@ -189,6 +191,11 @@ class FSNamesystem implements FSConstants {
|
|
|
private String localMachine;
|
|
|
private int port;
|
|
|
private SafeModeInfo safeMode; // safe mode information
|
|
|
+
|
|
|
+ // datanode networktoplogy
|
|
|
+ NetworkTopology clusterMap = new NetworkTopology();
|
|
|
+ // for block replicas placement
|
|
|
+ Replicator replicator = new Replicator();
|
|
|
|
|
|
/**
|
|
|
* dirs is a list oif directories where the filesystem directory state
|
|
@@ -497,10 +504,12 @@ class FSNamesystem implements FSConstants {
|
|
|
machineSets[i] = new DatanodeDescriptor[0];
|
|
|
} else {
|
|
|
machineSets[i] = new DatanodeDescriptor[containingNodes.size()];
|
|
|
- int j = 0;
|
|
|
- for (Iterator<DatanodeDescriptor> it = containingNodes.iterator(); it.hasNext(); j++) {
|
|
|
- machineSets[i][j] = it.next();
|
|
|
- }
|
|
|
+ ArrayList<DatanodeDescriptor> containingNodesList =
|
|
|
+ new ArrayList<DatanodeDescriptor>(containingNodes.size());
|
|
|
+ containingNodesList.addAll(containingNodes);
|
|
|
+
|
|
|
+ machineSets[i] = replicator.sortByDistance(
|
|
|
+ getDatanodeByHost(clientMachine), containingNodesList);
|
|
|
}
|
|
|
}
|
|
|
|
|
@@ -671,9 +680,9 @@ class FSNamesystem implements FSConstants {
|
|
|
}
|
|
|
}
|
|
|
|
|
|
- // Get the array of replication targets
|
|
|
- DatanodeDescriptor targets[] = chooseTargets(replication, null,
|
|
|
- clientMachine, blockSize);
|
|
|
+ // Get the array of replication targets
|
|
|
+ DatanodeDescriptor targets[] = replicator.chooseTarget(replication,
|
|
|
+ getDatanodeByHost(clientMachine.toString()), null, blockSize);
|
|
|
if (targets.length < this.minReplication) {
|
|
|
throw new IOException("failed to create file "+src
|
|
|
+" on client " + clientMachine
|
|
@@ -754,9 +763,13 @@ class FSNamesystem implements FSConstants {
|
|
|
throw new NotReplicatedYetException("Not replicated yet");
|
|
|
}
|
|
|
|
|
|
- // Get the array of replication targets
|
|
|
- DatanodeDescriptor targets[] = chooseTargets(pendingFile.getReplication(),
|
|
|
- null, pendingFile.getClientMachine(), pendingFile.getBlockSize());
|
|
|
+ // Get the array of replication targets
|
|
|
+ String clientHost = pendingFile.getClientMachine().toString();
|
|
|
+ DatanodeDescriptor targets[] = replicator.chooseTarget(
|
|
|
+ (int)(pendingFile.getReplication()),
|
|
|
+ getDatanodeByHost(clientHost),
|
|
|
+ null,
|
|
|
+ pendingFile.getBlockSize());
|
|
|
if (targets.length < this.minReplication) {
|
|
|
throw new IOException("File " + src + " could only be replicated to " +
|
|
|
targets.length + " nodes, instead of " +
|
|
@@ -1422,7 +1435,8 @@ class FSNamesystem implements FSConstants {
|
|
|
* @see DataNode#register()
|
|
|
* @author Konstantin Shvachko
|
|
|
*/
|
|
|
- public synchronized void registerDatanode( DatanodeRegistration nodeReg
|
|
|
+ public synchronized void registerDatanode( DatanodeRegistration nodeReg,
|
|
|
+ String networkLocation
|
|
|
) throws IOException {
|
|
|
NameNode.stateChangeLog.info(
|
|
|
"BLOCK* NameSystem.registerDatanode: "
|
|
@@ -1434,6 +1448,8 @@ class FSNamesystem implements FSConstants {
|
|
|
DatanodeDescriptor nodeN = getDatanodeByName( nodeReg.getName() );
|
|
|
|
|
|
if( nodeN != null && nodeN != nodeS ) {
|
|
|
+ NameNode.LOG.info( "BLOCK* NameSystem.registerDatanode: "
|
|
|
+ + "node from name: " + nodeN.getName() );
|
|
|
// nodeN previously served a different data storage,
|
|
|
// which is not served by anybody anymore.
|
|
|
removeDatanode( nodeN );
|
|
@@ -1457,11 +1473,15 @@ class FSNamesystem implements FSConstants {
|
|
|
// data storage, which from now on will be served by a new node.
|
|
|
NameNode.stateChangeLog.debug(
|
|
|
"BLOCK* NameSystem.registerDatanode: "
|
|
|
- + "node " + nodeS.name
|
|
|
+ + "node " + nodeS.getName()
|
|
|
+ " is replaced by " + nodeReg.getName() + "." );
|
|
|
}
|
|
|
getEditLog().logRemoveDatanode( nodeS );
|
|
|
+ // update cluster map
|
|
|
+ clusterMap.remove( nodeS );
|
|
|
nodeS.updateRegInfo( nodeReg );
|
|
|
+ nodeS.setNetworkLocation( networkLocation );
|
|
|
+ clusterMap.add( nodeS );
|
|
|
getEditLog().logAddDatanode( nodeS );
|
|
|
|
|
|
// also treat the registration message as a heartbeat
|
|
@@ -1484,7 +1504,8 @@ class FSNamesystem implements FSConstants {
|
|
|
+ "new storageID " + nodeReg.getStorageID() + " assigned." );
|
|
|
}
|
|
|
// register new datanode
|
|
|
- DatanodeDescriptor nodeDescr = new DatanodeDescriptor( nodeReg );
|
|
|
+ DatanodeDescriptor nodeDescr
|
|
|
+ = new DatanodeDescriptor( nodeReg, networkLocation );
|
|
|
unprotectedAddDatanode( nodeDescr );
|
|
|
getEditLog().logAddDatanode( nodeDescr );
|
|
|
|
|
@@ -1636,6 +1657,7 @@ class FSNamesystem implements FSConstants {
|
|
|
removeStoredBlock(it.next(), nodeInfo);
|
|
|
}
|
|
|
unprotectedRemoveDatanode(nodeInfo);
|
|
|
+ clusterMap.remove(nodeInfo);
|
|
|
}
|
|
|
|
|
|
void unprotectedRemoveDatanode( DatanodeDescriptor nodeDescr ) {
|
|
@@ -1649,6 +1671,7 @@ class FSNamesystem implements FSConstants {
|
|
|
|
|
|
void unprotectedAddDatanode( DatanodeDescriptor nodeDescr ) {
|
|
|
datanodeMap.put( nodeDescr.getStorageID(), nodeDescr );
|
|
|
+ clusterMap.add(nodeDescr);
|
|
|
NameNode.stateChangeLog.debug(
|
|
|
"BLOCK* NameSystem.unprotectedAddDatanode: "
|
|
|
+ "node " + nodeDescr.getName() + " is added to datanodeMap." );
|
|
@@ -1661,7 +1684,8 @@ class FSNamesystem implements FSConstants {
|
|
|
* @param nodeID node
|
|
|
*/
|
|
|
void wipeDatanode( DatanodeID nodeID ) {
|
|
|
- datanodeMap.remove(nodeID.getStorageID());
|
|
|
+ String key = nodeID.getStorageID();
|
|
|
+ datanodeMap.remove(key);
|
|
|
NameNode.stateChangeLog.debug(
|
|
|
"BLOCK* NameSystem.wipeDatanode: "
|
|
|
+ nodeID.getName() + " storage " + nodeID.getStorageID()
|
|
@@ -2008,6 +2032,7 @@ class FSNamesystem implements FSConstants {
|
|
|
* Total raw bytes.
|
|
|
*/
|
|
|
public long totalCapacity() {
|
|
|
+
|
|
|
synchronized (heartbeats) {
|
|
|
return totalCapacity;
|
|
|
}
|
|
@@ -2294,6 +2319,23 @@ class FSNamesystem implements FSConstants {
|
|
|
return count;
|
|
|
}
|
|
|
|
|
|
+ /*
|
|
|
+ * Filter nodes that are marked for decommison in the given list.
|
|
|
+ * Return a list of non-decommissioned nodes
|
|
|
+ */
|
|
|
+ private List<DatanodeDescriptor> filterDecommissionedNodes(
|
|
|
+ Collection<DatanodeDescriptor> nodelist) {
|
|
|
+ List<DatanodeDescriptor> nonCommissionedNodeList =
|
|
|
+ new ArrayList<DatanodeDescriptor>();
|
|
|
+ for (Iterator<DatanodeDescriptor> it = nodelist.iterator();
|
|
|
+ it.hasNext(); ) {
|
|
|
+ DatanodeDescriptor node = it.next();
|
|
|
+ if (!node.isDecommissionInProgress() && !node.isDecommissioned()) {
|
|
|
+ nonCommissionedNodeList.add(node);
|
|
|
+ }
|
|
|
+ }
|
|
|
+ return nonCommissionedNodeList;
|
|
|
+ }
|
|
|
/*
|
|
|
* Return true if there are any blocks in neededReplication that
|
|
|
* reside on the specified node. Otherwise returns false.
|
|
@@ -2395,14 +2437,15 @@ class FSNamesystem implements FSConstants {
|
|
|
// not be scheduled for removal on that node
|
|
|
if (containingNodes != null && containingNodes.contains(srcNode)
|
|
|
&& (excessBlocks == null || ! excessBlocks.contains(block))) {
|
|
|
-
|
|
|
// filter out containingNodes that are marked for decommission.
|
|
|
- int numCurrentReplica = countContainingNodes(containingNodes);
|
|
|
-
|
|
|
- DatanodeDescriptor targets[] = chooseTargets(
|
|
|
+ List<DatanodeDescriptor> nodes =
|
|
|
+ filterDecommissionedNodes(containingNodes);
|
|
|
+ int numCurrentReplica = nodes.size();
|
|
|
+ DatanodeDescriptor targets[] = replicator.chooseTarget(
|
|
|
Math.min( fileINode.getReplication() - numCurrentReplica,
|
|
|
- this.maxReplicationStreams - xmitsInProgress),
|
|
|
- containingNodes, null, blockSize);
|
|
|
+ this.maxReplicationStreams - xmitsInProgress),
|
|
|
+ datanodeMap.get(srcNode.getStorageID()),
|
|
|
+ nodes, null, blockSize);
|
|
|
if (targets.length > 0) {
|
|
|
// Build items to return
|
|
|
replicateBlocks.add(block);
|
|
@@ -2471,110 +2514,470 @@ class FSNamesystem implements FSConstants {
|
|
|
return results;
|
|
|
}
|
|
|
}
|
|
|
-
|
|
|
- /**
|
|
|
- * Get a certain number of targets, if possible.
|
|
|
- * If not, return as many as we can.
|
|
|
- * Only live nodes contained in {@link #heartbeats} are
|
|
|
- * targeted for replication.
|
|
|
- *
|
|
|
- * @param desiredReplicates
|
|
|
- * number of duplicates wanted.
|
|
|
- * @param forbiddenNodes
|
|
|
- * of DatanodeDescriptor instances that should not be considered targets.
|
|
|
- * @return array of DatanodeDescriptor instances uses as targets.
|
|
|
- */
|
|
|
- DatanodeDescriptor[] chooseTargets(
|
|
|
- int desiredReplicates,
|
|
|
- Collection<DatanodeDescriptor> forbiddenNodes,
|
|
|
- UTF8 clientMachine,
|
|
|
- long blockSize) {
|
|
|
- Collection<DatanodeDescriptor> targets = new ArrayList<DatanodeDescriptor>();
|
|
|
+
|
|
|
+ /** The class is responsible for choosing the desired number of targets
|
|
|
+ * for placing block replicas.
|
|
|
+ * The replica placement strategy is that if the writer is on a datanode,
|
|
|
+ * the 1st replica is placed on the local machine,
|
|
|
+ * otherwise a random datanode. The 2nd replica is placed on a datanode
|
|
|
+ * that is on a different rack. The 3rd replica is placed on a datanode
|
|
|
+ * which is on the same rack as the first replca.
|
|
|
+ * @author hairong
|
|
|
+ *
|
|
|
+ */
|
|
|
+ class Replicator {
|
|
|
+ private class NotEnoughReplicasException extends Exception {
|
|
|
+ NotEnoughReplicasException( String msg ) {
|
|
|
+ super( msg );
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
+ /**
|
|
|
+ * choose <i>numOfReplicas</i> data nodes for <i>writer</i> to replicate
|
|
|
+ * a block with size <i>blocksize</i>
|
|
|
+ * If not, return as many as we can.
|
|
|
+ *
|
|
|
+ * @param numOfReplicas: number of replicas wanted.
|
|
|
+ * @param writer: the writer's machine, null if not in the cluster.
|
|
|
+ * @param excludedNodes: datanodesthat should not be considered targets.
|
|
|
+ * @param blocksize: size of the data to be written.
|
|
|
+ * @return array of DatanodeDescriptor instances chosen as targets
|
|
|
+ * and sorted as a pipeline.
|
|
|
+ */
|
|
|
+ DatanodeDescriptor[] chooseTarget(int numOfReplicas,
|
|
|
+ DatanodeDescriptor writer,
|
|
|
+ List<DatanodeDescriptor> excludedNodes,
|
|
|
+ long blocksize ) {
|
|
|
+ if( excludedNodes == null) {
|
|
|
+ excludedNodes = new ArrayList<DatanodeDescriptor>();
|
|
|
+ }
|
|
|
|
|
|
- if (desiredReplicates > heartbeats.size()) {
|
|
|
- LOG.warn("Replication requested of "+desiredReplicates
|
|
|
- +" is larger than cluster size ("+heartbeats.size()
|
|
|
- +"). Using cluster size.");
|
|
|
- desiredReplicates = heartbeats.size();
|
|
|
- if (desiredReplicates == 0) {
|
|
|
- LOG.warn("While choosing target, totalMachines is " + desiredReplicates);
|
|
|
- }
|
|
|
+ return chooseTarget(numOfReplicas, writer,
|
|
|
+ new ArrayList<DatanodeDescriptor>(), excludedNodes, blocksize);
|
|
|
+ }
|
|
|
+
|
|
|
+ /*
|
|
|
+ * re-replicate <i>numOfReplicas</i>
|
|
|
+ /**
|
|
|
+ * choose <i>numOfReplicas</i> data nodes for <i>writer</i>
|
|
|
+ * to re-replicate a block with size <i>blocksize</i>
|
|
|
+ * If not, return as many as we can.
|
|
|
+ *
|
|
|
+ * @param numOfReplicas: additional number of replicas wanted.
|
|
|
+ * @param writer: the writer's machine, null if not in the cluster.
|
|
|
+ * @param choosenNodes: datanodes that have been choosen as targets.
|
|
|
+ * @param excludedNodes: datanodesthat should not be considered targets.
|
|
|
+ * @param blocksize: size of the data to be written.
|
|
|
+ * @return array of DatanodeDescriptor instances chosen as target
|
|
|
+ * and sorted as a pipeline.
|
|
|
+ */
|
|
|
+ DatanodeDescriptor[] chooseTarget(int numOfReplicas,
|
|
|
+ DatanodeDescriptor writer,
|
|
|
+ List<DatanodeDescriptor> choosenNodes,
|
|
|
+ List<DatanodeDescriptor> excludedNodes,
|
|
|
+ long blocksize ) {
|
|
|
+ if( numOfReplicas == 0 )
|
|
|
+ return new DatanodeDescriptor[0];
|
|
|
+
|
|
|
+ if( excludedNodes == null) {
|
|
|
+ excludedNodes = new ArrayList<DatanodeDescriptor>();
|
|
|
}
|
|
|
|
|
|
- double avgLoad = 0.0;
|
|
|
- if (heartbeats.size() != 0) {
|
|
|
- avgLoad = (double) totalLoad() / heartbeats.size();
|
|
|
- }
|
|
|
- // choose local replica first
|
|
|
- if (desiredReplicates != 0) {
|
|
|
- // make sure that the client machine is not forbidden
|
|
|
- if (clientMachine != null && clientMachine.getLength() > 0) {
|
|
|
- for (Iterator<DatanodeDescriptor> it = heartbeats.iterator();
|
|
|
- it.hasNext();) {
|
|
|
- DatanodeDescriptor node = it.next();
|
|
|
- if ((forbiddenNodes == null || !forbiddenNodes.contains(node)) &&
|
|
|
- clientMachine.toString().equals(node.getHost()) &&
|
|
|
- !node.isDecommissionInProgress() && !node.isDecommissioned()) {
|
|
|
- if ((node.getRemaining() >= blockSize * MIN_BLOCKS_FOR_WRITE) &&
|
|
|
- (node.getXceiverCount() <= (2.0 * avgLoad))) {
|
|
|
- targets.add(node);
|
|
|
- desiredReplicates--;
|
|
|
- break;
|
|
|
- }
|
|
|
- }
|
|
|
+ int clusterSize = clusterMap.getNumOfLeaves();
|
|
|
+ int totalNumOfReplicas = choosenNodes.size()+numOfReplicas;
|
|
|
+ if( totalNumOfReplicas > clusterSize) {
|
|
|
+ numOfReplicas -= (totalNumOfReplicas-clusterSize);
|
|
|
+ totalNumOfReplicas = clusterSize;
|
|
|
+ }
|
|
|
+
|
|
|
+ int maxNodesPerRack =
|
|
|
+ (totalNumOfReplicas-1)/clusterMap.getNumOfRacks()+2;
|
|
|
+
|
|
|
+ List<DatanodeDescriptor> results =
|
|
|
+ new ArrayList<DatanodeDescriptor>(choosenNodes);
|
|
|
+ excludedNodes.addAll(choosenNodes);
|
|
|
+
|
|
|
+ if(!clusterMap.contains(writer))
|
|
|
+ writer=null;
|
|
|
+
|
|
|
+ DatanodeDescriptor localNode = chooseTarget(numOfReplicas, writer,
|
|
|
+ clusterMap.getLeaves(NodeBase.ROOT),
|
|
|
+ excludedNodes, blocksize, maxNodesPerRack, results );
|
|
|
+
|
|
|
+ results.removeAll(choosenNodes);
|
|
|
+
|
|
|
+ // sorting nodes to form a pipeline
|
|
|
+ return getPipeline((writer==null)?localNode:writer, results);
|
|
|
+ }
|
|
|
+
|
|
|
+ /* choose <i>numOfReplicas</i> from <i>clusterNodes</i> */
|
|
|
+ private DatanodeDescriptor chooseTarget(int numOfReplicas,
|
|
|
+ DatanodeDescriptor writer,
|
|
|
+ DatanodeDescriptor[] clusterNodes,
|
|
|
+ List<DatanodeDescriptor> excludedNodes,
|
|
|
+ long blocksize,
|
|
|
+ int maxNodesPerRack,
|
|
|
+ List<DatanodeDescriptor> results) {
|
|
|
+
|
|
|
+ if( numOfReplicas == 0 ) return writer;
|
|
|
+
|
|
|
+ int numOfResults = results.size();
|
|
|
+ if(writer == null && (numOfResults==1 || numOfResults==2) ) {
|
|
|
+ writer = results.get(0);
|
|
|
+ }
|
|
|
+
|
|
|
+ try {
|
|
|
+ switch( numOfResults ) {
|
|
|
+ case 0:
|
|
|
+ writer = chooseLocalNode(writer, clusterNodes, excludedNodes,
|
|
|
+ blocksize, maxNodesPerRack, results);
|
|
|
+ if(--numOfReplicas == 0) break;
|
|
|
+ case 1:
|
|
|
+ chooseRemoteRack(1, writer, clusterNodes, excludedNodes,
|
|
|
+ blocksize, maxNodesPerRack, results);
|
|
|
+ if(--numOfReplicas == 0) break;
|
|
|
+ case 2:
|
|
|
+ if(clusterMap.isOnSameRack(results.get(0), results.get(1))) {
|
|
|
+ chooseRemoteRack(1, writer, clusterNodes, excludedNodes,
|
|
|
+ blocksize, maxNodesPerRack, results);
|
|
|
+ } else {
|
|
|
+ chooseLocalRack(writer, clusterNodes, excludedNodes,
|
|
|
+ blocksize, maxNodesPerRack, results);
|
|
|
}
|
|
|
+ if(--numOfReplicas == 0) break;
|
|
|
+ default:
|
|
|
+ chooseRandom(numOfReplicas, clusterNodes, excludedNodes,
|
|
|
+ blocksize, maxNodesPerRack, results);
|
|
|
}
|
|
|
+ } catch (NotEnoughReplicasException e) {
|
|
|
+ LOG.warn("Not be able to place enough replicas, still in need of "
|
|
|
+ + numOfReplicas );
|
|
|
}
|
|
|
-
|
|
|
- for (int i = 0; i < desiredReplicates; i++) {
|
|
|
- DatanodeDescriptor target = null;
|
|
|
- //
|
|
|
- // Otherwise, choose node according to target capacity
|
|
|
- //
|
|
|
- int nNodes = heartbeats.size();
|
|
|
- int idx = r.nextInt(nNodes);
|
|
|
- int rejected = 0;
|
|
|
- while (target == null && rejected < nNodes ) {
|
|
|
- DatanodeDescriptor node = heartbeats.get(idx);
|
|
|
- if ((forbiddenNodes == null || !forbiddenNodes.contains(node)) &&
|
|
|
- !targets.contains(node) &&
|
|
|
- !node.isDecommissionInProgress() && !node.isDecommissioned() &&
|
|
|
- (node.getRemaining() >= blockSize * MIN_BLOCKS_FOR_WRITE) &&
|
|
|
- (node.getXceiverCount() <= (2.0 * avgLoad))) {
|
|
|
- target = node;
|
|
|
+ return writer;
|
|
|
+ }
|
|
|
+
|
|
|
+ /* choose <i>localMachine</i> as the target.
|
|
|
+ * if <i>localMachine</i> is not availabe,
|
|
|
+ * choose a node on the same rack
|
|
|
+ * @return the choosen node
|
|
|
+ */
|
|
|
+ private DatanodeDescriptor chooseLocalNode(
|
|
|
+ DatanodeDescriptor localMachine,
|
|
|
+ DatanodeDescriptor[] nodes,
|
|
|
+ List<DatanodeDescriptor> excludedNodes,
|
|
|
+ long blocksize,
|
|
|
+ int maxNodesPerRack,
|
|
|
+ List<DatanodeDescriptor> results)
|
|
|
+ throws NotEnoughReplicasException {
|
|
|
+ // if no local machine, randomly choose one node
|
|
|
+ if(localMachine == null)
|
|
|
+ return chooseRandom(nodes, excludedNodes,
|
|
|
+ blocksize, maxNodesPerRack, results);
|
|
|
+
|
|
|
+ // otherwise try local machine first
|
|
|
+ if(!excludedNodes.contains(localMachine)) {
|
|
|
+ excludedNodes.add(localMachine);
|
|
|
+ if( isGoodTarget(localMachine, blocksize, maxNodesPerRack, results)) {
|
|
|
+ results.add(localMachine);
|
|
|
+ return localMachine;
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
+ // try a node on local rack
|
|
|
+ return chooseLocalRack(localMachine, nodes, excludedNodes,
|
|
|
+ blocksize, maxNodesPerRack, results);
|
|
|
+ }
|
|
|
+
|
|
|
+ /* choose one node from the rack that <i>localMachine</i> is on.
|
|
|
+ * if no such node is availabe, choose one node from the rack where
|
|
|
+ * a second replica is on.
|
|
|
+ * if still no such node is available, choose a random node
|
|
|
+ * in the cluster <i>nodes</i>.
|
|
|
+ * @return the choosen node
|
|
|
+ */
|
|
|
+ private DatanodeDescriptor chooseLocalRack(
|
|
|
+ DatanodeDescriptor localMachine,
|
|
|
+ DatanodeDescriptor[] nodes,
|
|
|
+ List<DatanodeDescriptor> excludedNodes,
|
|
|
+ long blocksize,
|
|
|
+ int maxNodesPerRack,
|
|
|
+ List<DatanodeDescriptor> results)
|
|
|
+ throws NotEnoughReplicasException {
|
|
|
+ // no local machine, so choose a random machine
|
|
|
+ if( localMachine == null ) {
|
|
|
+ return chooseRandom(nodes, excludedNodes,
|
|
|
+ blocksize, maxNodesPerRack, results );
|
|
|
+ }
|
|
|
+
|
|
|
+ // choose one from the local rack
|
|
|
+ try {
|
|
|
+ return chooseRandom(
|
|
|
+ clusterMap.getLeaves( localMachine.getNetworkLocation() ),
|
|
|
+ excludedNodes, blocksize, maxNodesPerRack, results);
|
|
|
+ } catch (NotEnoughReplicasException e1) {
|
|
|
+ // find the second replica
|
|
|
+ DatanodeDescriptor newLocal=null;
|
|
|
+ for(Iterator<DatanodeDescriptor> iter=results.iterator();
|
|
|
+ iter.hasNext();) {
|
|
|
+ DatanodeDescriptor nextNode = iter.next();
|
|
|
+ if(nextNode != localMachine) {
|
|
|
+ newLocal = nextNode;
|
|
|
break;
|
|
|
- } else {
|
|
|
- idx = (idx+1) % nNodes;
|
|
|
- rejected++;
|
|
|
}
|
|
|
}
|
|
|
- if (target == null) {
|
|
|
- idx = r.nextInt(nNodes);
|
|
|
- rejected = 0;
|
|
|
- while (target == null && rejected < nNodes ) {
|
|
|
- DatanodeDescriptor node = heartbeats.get(idx);
|
|
|
- if ((forbiddenNodes == null || !forbiddenNodes.contains(node)) &&
|
|
|
- !targets.contains(node) &&
|
|
|
- !node.isDecommissionInProgress() && !node.isDecommissioned() &&
|
|
|
- node.getRemaining() >= blockSize) {
|
|
|
- target = node;
|
|
|
- break;
|
|
|
- } else {
|
|
|
- idx = (idx + 1) % nNodes;
|
|
|
- rejected++;
|
|
|
- }
|
|
|
+ if( newLocal != null ) {
|
|
|
+ try {
|
|
|
+ return chooseRandom(
|
|
|
+ clusterMap.getLeaves( newLocal.getNetworkLocation() ),
|
|
|
+ excludedNodes, blocksize, maxNodesPerRack, results);
|
|
|
+ } catch( NotEnoughReplicasException e2 ) {
|
|
|
+ //otherwise randomly choose one from the network
|
|
|
+ return chooseRandom(nodes, excludedNodes,
|
|
|
+ blocksize, maxNodesPerRack, results);
|
|
|
}
|
|
|
+ } else {
|
|
|
+ //otherwise randomly choose one from the network
|
|
|
+ return chooseRandom(nodes, excludedNodes,
|
|
|
+ blocksize, maxNodesPerRack, results);
|
|
|
}
|
|
|
-
|
|
|
- if (target == null) {
|
|
|
- LOG.warn("Could not find any nodes with sufficient capacity");
|
|
|
- break; // making one more pass over heartbeats would not help
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
+ /* choose <i>numOfReplicas</i> nodes from the racks
|
|
|
+ * that <i>localMachine</i> is NOT on.
|
|
|
+ * if not enough nodes are availabe, choose the remaining ones
|
|
|
+ * from the local rack
|
|
|
+ */
|
|
|
+
|
|
|
+ private void chooseRemoteRack( int numOfReplicas,
|
|
|
+ DatanodeDescriptor localMachine,
|
|
|
+ DatanodeDescriptor[] nodes,
|
|
|
+ List<DatanodeDescriptor> excludedNodes,
|
|
|
+ long blocksize,
|
|
|
+ int maxReplicasPerRack,
|
|
|
+ List<DatanodeDescriptor> results)
|
|
|
+ throws NotEnoughReplicasException {
|
|
|
+ // get all the nodes on the local rack
|
|
|
+ DatanodeDescriptor[] nodesOnRack = clusterMap.getLeaves(
|
|
|
+ localMachine.getNetworkLocation() );
|
|
|
+
|
|
|
+ // can we speed up this??? using hashing sets?
|
|
|
+ DatanodeDescriptor[] nodesOnRemoteRack
|
|
|
+ = new DatanodeDescriptor[nodes.length-nodesOnRack.length];
|
|
|
+ HashSet<DatanodeDescriptor> set1 = new HashSet<DatanodeDescriptor>(nodes.length);
|
|
|
+ HashSet<DatanodeDescriptor> set2 = new HashSet<DatanodeDescriptor>(nodesOnRack.length);
|
|
|
+ for(int i=0; i<nodes.length; i++) {
|
|
|
+ set1.add(nodes[i]);
|
|
|
+ }
|
|
|
+ for(int i=0; i<nodesOnRack.length; i++) {
|
|
|
+ set2.add(nodesOnRack[i]);
|
|
|
+ }
|
|
|
+ set1.removeAll(set2);
|
|
|
+ nodesOnRemoteRack = set1.toArray(nodesOnRemoteRack);
|
|
|
+
|
|
|
+ int oldNumOfReplicas = results.size();
|
|
|
+ // randomly choose one node from remote racks
|
|
|
+ try {
|
|
|
+ chooseRandom( numOfReplicas, nodesOnRemoteRack, excludedNodes,
|
|
|
+ blocksize, maxReplicasPerRack, results );
|
|
|
+ } catch (NotEnoughReplicasException e) {
|
|
|
+ chooseRandom( numOfReplicas-(results.size()-oldNumOfReplicas),
|
|
|
+ nodesOnRack, excludedNodes, blocksize,
|
|
|
+ maxReplicasPerRack, results);
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
+ /* Randomly choose one target from <i>nodes</i>.
|
|
|
+ * @return the choosen node
|
|
|
+ */
|
|
|
+ private DatanodeDescriptor chooseRandom(
|
|
|
+ DatanodeDescriptor[] nodes,
|
|
|
+ List<DatanodeDescriptor> excludedNodes,
|
|
|
+ long blocksize,
|
|
|
+ int maxNodesPerRack,
|
|
|
+ List<DatanodeDescriptor> results)
|
|
|
+ throws NotEnoughReplicasException {
|
|
|
+ DatanodeDescriptor result;
|
|
|
+ do {
|
|
|
+ DatanodeDescriptor[] selectedNodes =
|
|
|
+ chooseRandom(1, nodes, excludedNodes);
|
|
|
+ if(selectedNodes.length == 0 ) {
|
|
|
+ throw new NotEnoughReplicasException(
|
|
|
+ "Not able to place enough replicas" );
|
|
|
+ }
|
|
|
+ result = (DatanodeDescriptor)(selectedNodes[0]);
|
|
|
+ } while( !isGoodTarget( result, blocksize, maxNodesPerRack, results));
|
|
|
+ results.add(result);
|
|
|
+ return result;
|
|
|
+ }
|
|
|
+
|
|
|
+ /* Randomly choose <i>numOfReplicas</i> targets from <i>nodes</i>.
|
|
|
+ */
|
|
|
+ private void chooseRandom(int numOfReplicas,
|
|
|
+ DatanodeDescriptor[] nodes,
|
|
|
+ List<DatanodeDescriptor> excludedNodes,
|
|
|
+ long blocksize,
|
|
|
+ int maxNodesPerRack,
|
|
|
+ List<DatanodeDescriptor> results)
|
|
|
+ throws NotEnoughReplicasException {
|
|
|
+ boolean toContinue = true;
|
|
|
+ do {
|
|
|
+ DatanodeDescriptor[] selectedNodes =
|
|
|
+ chooseRandom(numOfReplicas, nodes, excludedNodes);
|
|
|
+ if(selectedNodes.length < numOfReplicas) {
|
|
|
+ toContinue = false;
|
|
|
+ }
|
|
|
+ for(int i=0; i<selectedNodes.length; i++) {
|
|
|
+ DatanodeDescriptor result = (DatanodeDescriptor)(selectedNodes[i]);
|
|
|
+ if( isGoodTarget( result, blocksize, maxNodesPerRack, results)) {
|
|
|
+ numOfReplicas--;
|
|
|
+ results.add(result);
|
|
|
+ }
|
|
|
+ } // end of for
|
|
|
+ } while (numOfReplicas>0 && toContinue );
|
|
|
+
|
|
|
+ if(numOfReplicas>0) {
|
|
|
+ throw new NotEnoughReplicasException(
|
|
|
+ "Not able to place enough replicas");
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
+ /* Randomly choose one node from <i>nodes</i>.
|
|
|
+ * @return the choosen node
|
|
|
+ */
|
|
|
+ private DatanodeDescriptor[] chooseRandom(int numOfReplicas,
|
|
|
+ DatanodeDescriptor[] nodes,
|
|
|
+ List<DatanodeDescriptor> excludedNodes) {
|
|
|
+ List<DatanodeDescriptor> results =
|
|
|
+ new ArrayList<DatanodeDescriptor>();
|
|
|
+ int numOfAvailableNodes = 0;
|
|
|
+ for(int i=0; i<nodes.length; i++) {
|
|
|
+ if( !excludedNodes.contains(nodes[i]) ) {
|
|
|
+ numOfAvailableNodes++;
|
|
|
+ }
|
|
|
+ }
|
|
|
+ numOfReplicas = (numOfAvailableNodes<numOfReplicas)?
|
|
|
+ numOfAvailableNodes:numOfReplicas;
|
|
|
+ while( numOfReplicas > 0 ) {
|
|
|
+ DatanodeDescriptor choosenNode = nodes[r.nextInt(nodes.length)];
|
|
|
+ if(!excludedNodes.contains(choosenNode) &&
|
|
|
+ !choosenNode.isDecommissionInProgress() &&
|
|
|
+ !choosenNode.isDecommissioned()) {
|
|
|
+ results.add( choosenNode );
|
|
|
+ excludedNodes.add(choosenNode);
|
|
|
+ numOfReplicas--;
|
|
|
}
|
|
|
- targets.add(target);
|
|
|
}
|
|
|
+ return (DatanodeDescriptor[])results.toArray(
|
|
|
+ new DatanodeDescriptor[results.size()]);
|
|
|
+ }
|
|
|
+
|
|
|
+ /* judge if a node is a good target.
|
|
|
+ * return true if <i>node</i> has enough space,
|
|
|
+ * does not have too much load, and the rack does not have too many nodes
|
|
|
+ */
|
|
|
+ private boolean isGoodTarget( DatanodeDescriptor node,
|
|
|
+ long blockSize, int maxTargetPerLoc,
|
|
|
+ List<DatanodeDescriptor> results) {
|
|
|
|
|
|
- return (DatanodeDescriptor[]) targets.toArray(new DatanodeDescriptor[targets.size()]);
|
|
|
- }
|
|
|
+ // check if the node is (being) decommissed
|
|
|
+ if(node.isDecommissionInProgress() || node.isDecommissioned()) {
|
|
|
+ return false;
|
|
|
+ }
|
|
|
+
|
|
|
+ // check the remaining capacity of the target machine
|
|
|
+ if(blockSize* FSConstants.MIN_BLOCKS_FOR_WRITE>node.getRemaining() ) {
|
|
|
+ return false;
|
|
|
+ }
|
|
|
+
|
|
|
+ // check the communication traffic of the target machine
|
|
|
+ double avgLoad = 0;
|
|
|
+ int size = clusterMap.getNumOfLeaves();
|
|
|
+ if( size != 0 ) {
|
|
|
+ avgLoad = (double)totalLoad()/size;
|
|
|
+ }
|
|
|
+ if(node.getXceiverCount() > (2.0 * avgLoad)) {
|
|
|
+ return false;
|
|
|
+ }
|
|
|
+
|
|
|
+ // check if the target rack has chosen too many nodes
|
|
|
+ String rackname = node.getNetworkLocation();
|
|
|
+ int counter=1;
|
|
|
+ for( Iterator<DatanodeDescriptor> iter = results.iterator();
|
|
|
+ iter.hasNext(); ) {
|
|
|
+ DatanodeDescriptor result = iter.next();
|
|
|
+ if(rackname.equals(result.getNetworkLocation())) {
|
|
|
+ counter++;
|
|
|
+ }
|
|
|
+ }
|
|
|
+ if(counter>maxTargetPerLoc) {
|
|
|
+ return false;
|
|
|
+ }
|
|
|
+ return true;
|
|
|
+ }
|
|
|
+
|
|
|
+ /* Return a pipeline of nodes.
|
|
|
+ * The pipeline is formed finding a shortest path that
|
|
|
+ * starts from the writer and tranverses all <i>nodes</i>
|
|
|
+ * This is basically a traveling salesman problem.
|
|
|
+ */
|
|
|
+ private DatanodeDescriptor[] getPipeline(
|
|
|
+ DatanodeDescriptor writer,
|
|
|
+ List<DatanodeDescriptor> nodes ) {
|
|
|
+ int numOfNodes = nodes.size();
|
|
|
+ DatanodeDescriptor[] results = new DatanodeDescriptor[numOfNodes];
|
|
|
+ if( numOfNodes==0 ) return results;
|
|
|
+
|
|
|
+ synchronized( clusterMap ) {
|
|
|
+ int index=0;
|
|
|
+ if(writer == null || !clusterMap.contains(writer)) {
|
|
|
+ writer = nodes.get(0);
|
|
|
+ }
|
|
|
+ for( ;index<numOfNodes; index++ ) {
|
|
|
+ DatanodeDescriptor shortestNode = null;
|
|
|
+ int shortestDistance = Integer.MAX_VALUE;
|
|
|
+ int shortestIndex = index;
|
|
|
+ for( int i=index; i<numOfNodes; i++ ) {
|
|
|
+ DatanodeDescriptor currentNode = nodes.get(i);
|
|
|
+ int currentDistance = clusterMap.getDistance( writer, currentNode );
|
|
|
+ if(shortestDistance>currentDistance ) {
|
|
|
+ shortestDistance = currentDistance;
|
|
|
+ shortestNode = currentNode;
|
|
|
+ shortestIndex = i;
|
|
|
+ }
|
|
|
+ }
|
|
|
+ //switch position index & shortestIndex
|
|
|
+ if( index != shortestIndex ) {
|
|
|
+ nodes.set(shortestIndex, nodes.get(index));
|
|
|
+ nodes.set(index, shortestNode);
|
|
|
+ }
|
|
|
+ writer = shortestNode;
|
|
|
+ }
|
|
|
+ }
|
|
|
+ return nodes.toArray( results );
|
|
|
+ }
|
|
|
+
|
|
|
+ /** Return datanodes that sorted by their distances to <i>reader</i>
|
|
|
+ */
|
|
|
+ DatanodeDescriptor[] sortByDistance(
|
|
|
+ final DatanodeDescriptor reader,
|
|
|
+ List<DatanodeDescriptor> nodes ) {
|
|
|
+ synchronized(clusterMap) {
|
|
|
+ if(reader != null && clusterMap.contains(reader)) {
|
|
|
+ java.util.Collections.sort(nodes, new Comparator<DatanodeDescriptor>() {
|
|
|
+ public int compare(DatanodeDescriptor n1, DatanodeDescriptor n2) {
|
|
|
+ return clusterMap.getDistance(reader, n1)
|
|
|
+ -clusterMap.getDistance(reader, n2);
|
|
|
+ }
|
|
|
+ });
|
|
|
+ }
|
|
|
+ }
|
|
|
+ return (DatanodeDescriptor[])nodes.toArray(
|
|
|
+ new DatanodeDescriptor[nodes.size()]);
|
|
|
+ }
|
|
|
+
|
|
|
+ } //end of Replicator
|
|
|
+
|
|
|
|
|
|
/**
|
|
|
* Information about the file while it is being written to.
|
|
@@ -2664,6 +3067,18 @@ class FSNamesystem implements FSConstants {
|
|
|
}
|
|
|
return null;
|
|
|
}
|
|
|
+
|
|
|
+ /* Find data node by its host name. */
|
|
|
+ private DatanodeDescriptor getDatanodeByHost( String name ) {
|
|
|
+ for (Iterator<DatanodeDescriptor> it = datanodeMap.values().iterator();
|
|
|
+ it.hasNext(); ) {
|
|
|
+ DatanodeDescriptor node = it.next();
|
|
|
+ if( node.getHost().equals(name) )
|
|
|
+ return node;
|
|
|
+ }
|
|
|
+ return null;
|
|
|
+ }
|
|
|
+
|
|
|
/** Stop at and return the datanode at index (used for content browsing)*/
|
|
|
private DatanodeInfo getDatanodeByIndex( int index ) {
|
|
|
int i = 0;
|
|
@@ -2802,6 +3217,9 @@ class FSNamesystem implements FSConstants {
|
|
|
"STATE* SafeModeInfo.leave: " + "Safe mode is OFF." );
|
|
|
reached = -1;
|
|
|
safeMode = null;
|
|
|
+ NameNode.stateChangeLog.info("STATE* Network topology has "
|
|
|
+ +clusterMap.getNumOfRacks()+" racks and "
|
|
|
+ +clusterMap.getNumOfLeaves()+ " datanodes");
|
|
|
}
|
|
|
|
|
|
/**
|