|
@@ -122,6 +122,9 @@ class FSNamesystem implements FSConstants {
|
|
// Stats on overall usage
|
|
// Stats on overall usage
|
|
//
|
|
//
|
|
long totalCapacity = 0, totalRemaining = 0;
|
|
long totalCapacity = 0, totalRemaining = 0;
|
|
|
|
+
|
|
|
|
+ // total number of connections per live datanode
|
|
|
|
+ int totalLoad = 0;
|
|
|
|
|
|
|
|
|
|
//
|
|
//
|
|
@@ -136,26 +139,13 @@ class FSNamesystem implements FSConstants {
|
|
Random r = new Random();
|
|
Random r = new Random();
|
|
|
|
|
|
/**
|
|
/**
|
|
- * Stores a set of DatanodeDescriptor objects, sorted by heartbeat.
|
|
|
|
|
|
+ * Stores a set of DatanodeDescriptor objects.
|
|
* This is a subset of {@link #datanodeMap}, containing nodes that are
|
|
* This is a subset of {@link #datanodeMap}, containing nodes that are
|
|
* considered alive.
|
|
* considered alive.
|
|
* The {@link HeartbeatMonitor} periodically checks for outdated entries,
|
|
* The {@link HeartbeatMonitor} periodically checks for outdated entries,
|
|
- * and removes them from the set.
|
|
|
|
- */
|
|
|
|
- TreeSet<DatanodeDescriptor> heartbeats =
|
|
|
|
- new TreeSet<DatanodeDescriptor>(
|
|
|
|
- new Comparator<DatanodeDescriptor>() {
|
|
|
|
- public int compare(DatanodeDescriptor d1, DatanodeDescriptor d2) {
|
|
|
|
- long lu1 = d1.getLastUpdate();
|
|
|
|
- long lu2 = d2.getLastUpdate();
|
|
|
|
- if (lu1 < lu2)
|
|
|
|
- return -1;
|
|
|
|
- if (lu1 > lu2)
|
|
|
|
- return 1;
|
|
|
|
- return d1.getStorageID().compareTo(d2.getStorageID());
|
|
|
|
- }
|
|
|
|
- }
|
|
|
|
- );
|
|
|
|
|
|
+ * and removes them from the list.
|
|
|
|
+ */
|
|
|
|
+ ArrayList<DatanodeDescriptor> heartbeats = new ArrayList<DatanodeDescriptor>();
|
|
|
|
|
|
//
|
|
//
|
|
// Store set of Blocks that need to be replicated 1 or more times.
|
|
// Store set of Blocks that need to be replicated 1 or more times.
|
|
@@ -189,8 +179,11 @@ class FSNamesystem implements FSConstants {
|
|
private int maxReplicationStreams;
|
|
private int maxReplicationStreams;
|
|
// MIN_REPLICATION is how many copies we need in place or else we disallow the write
|
|
// MIN_REPLICATION is how many copies we need in place or else we disallow the write
|
|
private int minReplication;
|
|
private int minReplication;
|
|
- // HEARTBEAT_RECHECK is how often a datanode sends its hearbeat
|
|
|
|
- private int heartBeatRecheck;
|
|
|
|
|
|
+ // heartbeatRecheckInterval is how often namenode checks for expired datanodes
|
|
|
|
+ private long heartbeatRecheckInterval;
|
|
|
|
+ // heartbeatExpireInterval is how long namenode waits for datanode to report
|
|
|
|
+ // heartbeat
|
|
|
|
+ private long heartbeatExpireInterval;
|
|
|
|
|
|
public static FSNamesystem fsNamesystemObject;
|
|
public static FSNamesystem fsNamesystemObject;
|
|
private String localMachine;
|
|
private String localMachine;
|
|
@@ -222,7 +215,10 @@ class FSNamesystem implements FSConstants {
|
|
+ " must be less than dfs.replication.max = "
|
|
+ " must be less than dfs.replication.max = "
|
|
+ maxReplication );
|
|
+ maxReplication );
|
|
this.maxReplicationStreams = conf.getInt("dfs.max-repl-streams", 2);
|
|
this.maxReplicationStreams = conf.getInt("dfs.max-repl-streams", 2);
|
|
- this.heartBeatRecheck= 1000;
|
|
|
|
|
|
+ long heartbeatInterval = conf.getLong("dfs.heartbeat.interval", 3) * 1000;
|
|
|
|
+ this.heartbeatRecheckInterval = 5 * 60 * 1000; // 5 minutes
|
|
|
|
+ this.heartbeatExpireInterval = 2 * heartbeatRecheckInterval +
|
|
|
|
+ 10 * heartbeatInterval;
|
|
|
|
|
|
this.localMachine = addr.getHostName();
|
|
this.localMachine = addr.getHostName();
|
|
this.port = addr.getPort();
|
|
this.port = addr.getPort();
|
|
@@ -1206,7 +1202,7 @@ class FSNamesystem implements FSConstants {
|
|
// The same datanode has been just restarted to serve the same data
|
|
// The same datanode has been just restarted to serve the same data
|
|
// storage. We do not need to remove old data blocks, the delta will
|
|
// storage. We do not need to remove old data blocks, the delta will
|
|
// be calculated on the next block report from the datanode
|
|
// be calculated on the next block report from the datanode
|
|
- NameNode.stateChangeLog.debug(
|
|
|
|
|
|
+ NameNode.stateChangeLog.info(
|
|
"BLOCK* NameSystem.registerDatanode: "
|
|
"BLOCK* NameSystem.registerDatanode: "
|
|
+ "node restarted." );
|
|
+ "node restarted." );
|
|
return;
|
|
return;
|
|
@@ -1249,6 +1245,8 @@ class FSNamesystem implements FSConstants {
|
|
}
|
|
}
|
|
// register new datanode
|
|
// register new datanode
|
|
DatanodeDescriptor nodeDescr = new DatanodeDescriptor( nodeReg );
|
|
DatanodeDescriptor nodeDescr = new DatanodeDescriptor( nodeReg );
|
|
|
|
+ // unless we get a heartbeat from this datanode, we will not mark it Alive
|
|
|
|
+ nodeDescr.isAlive = false;
|
|
unprotectedAddDatanode( nodeDescr );
|
|
unprotectedAddDatanode( nodeDescr );
|
|
getEditLog().logAddDatanode( nodeDescr );
|
|
getEditLog().logAddDatanode( nodeDescr );
|
|
return;
|
|
return;
|
|
@@ -1283,6 +1281,11 @@ class FSNamesystem implements FSConstants {
|
|
return newID;
|
|
return newID;
|
|
}
|
|
}
|
|
|
|
|
|
|
|
+ private boolean isDatanodeDead(DatanodeDescriptor node) {
|
|
|
|
+ return (node.getLastUpdate() <
|
|
|
|
+ (System.currentTimeMillis() - heartbeatExpireInterval));
|
|
|
|
+ }
|
|
|
|
+
|
|
/**
|
|
/**
|
|
* The given node has reported in. This method should:
|
|
* The given node has reported in. This method should:
|
|
* 1) Record the heartbeat, so the datanode isn't timed out
|
|
* 1) Record the heartbeat, so the datanode isn't timed out
|
|
@@ -1303,19 +1306,36 @@ class FSNamesystem implements FSConstants {
|
|
synchronized (heartbeats) {
|
|
synchronized (heartbeats) {
|
|
synchronized (datanodeMap) {
|
|
synchronized (datanodeMap) {
|
|
DatanodeDescriptor nodeinfo = getDatanode( nodeID );
|
|
DatanodeDescriptor nodeinfo = getDatanode( nodeID );
|
|
- needBlockReport = nodeinfo.isDead();
|
|
|
|
|
|
+ needBlockReport = isDatanodeDead(nodeinfo);
|
|
|
|
|
|
if (nodeinfo == null)
|
|
if (nodeinfo == null)
|
|
// We do not accept unregistered guests
|
|
// We do not accept unregistered guests
|
|
throw new UnregisteredDatanodeException( nodeID );
|
|
throw new UnregisteredDatanodeException( nodeID );
|
|
- removeHeartbeat(nodeinfo);
|
|
|
|
|
|
+ if (nodeinfo.isAlive) {
|
|
|
|
+ updateStats(nodeinfo, false);
|
|
|
|
+ }
|
|
nodeinfo.updateHeartbeat(capacity, remaining, xceiverCount);
|
|
nodeinfo.updateHeartbeat(capacity, remaining, xceiverCount);
|
|
- addHeartbeat(nodeinfo);
|
|
|
|
|
|
+ updateStats(nodeinfo, true);
|
|
|
|
+ if (!nodeinfo.isAlive) {
|
|
|
|
+ heartbeats.add(nodeinfo);
|
|
|
|
+ nodeinfo.isAlive = true;
|
|
|
|
+ }
|
|
}
|
|
}
|
|
}
|
|
}
|
|
return needBlockReport;
|
|
return needBlockReport;
|
|
}
|
|
}
|
|
|
|
|
|
|
|
+ private void updateStats(DatanodeDescriptor node, boolean isAdded) {
|
|
|
|
+ if (isAdded) {
|
|
|
|
+ totalCapacity += node.getCapacity();
|
|
|
|
+ totalRemaining += node.getRemaining();
|
|
|
|
+ totalLoad += node.getXceiverCount();
|
|
|
|
+ } else {
|
|
|
|
+ totalCapacity -= node.getCapacity();
|
|
|
|
+ totalRemaining -= node.getRemaining();
|
|
|
|
+ totalLoad -= node.getXceiverCount();
|
|
|
|
+ }
|
|
|
|
+ }
|
|
/**
|
|
/**
|
|
* Periodically calls heartbeatCheck().
|
|
* Periodically calls heartbeatCheck().
|
|
*/
|
|
*/
|
|
@@ -1326,7 +1346,7 @@ class FSNamesystem implements FSConstants {
|
|
while (fsRunning) {
|
|
while (fsRunning) {
|
|
heartbeatCheck();
|
|
heartbeatCheck();
|
|
try {
|
|
try {
|
|
- Thread.sleep(heartBeatRecheck);
|
|
|
|
|
|
+ Thread.sleep(heartbeatRecheckInterval);
|
|
} catch (InterruptedException ie) {
|
|
} catch (InterruptedException ie) {
|
|
}
|
|
}
|
|
}
|
|
}
|
|
@@ -1355,7 +1375,11 @@ class FSNamesystem implements FSConstants {
|
|
* @author hairong
|
|
* @author hairong
|
|
*/
|
|
*/
|
|
private void removeDatanode( DatanodeDescriptor nodeInfo ) {
|
|
private void removeDatanode( DatanodeDescriptor nodeInfo ) {
|
|
- removeHeartbeat(nodeInfo);
|
|
|
|
|
|
+ if (nodeInfo.isAlive) {
|
|
|
|
+ updateStats(nodeInfo, false);
|
|
|
|
+ heartbeats.remove(nodeInfo);
|
|
|
|
+ nodeInfo.isAlive = false;
|
|
|
|
+ }
|
|
|
|
|
|
Block deadblocks[] = nodeInfo.getBlocks();
|
|
Block deadblocks[] = nodeInfo.getBlocks();
|
|
if( deadblocks != null )
|
|
if( deadblocks != null )
|
|
@@ -1380,17 +1404,6 @@ class FSNamesystem implements FSConstants {
|
|
+ "node " + nodeDescr.getName() + " is added to datanodeMap." );
|
|
+ "node " + nodeDescr.getName() + " is added to datanodeMap." );
|
|
}
|
|
}
|
|
|
|
|
|
- private void addHeartbeat( DatanodeDescriptor nodeDescr ) {
|
|
|
|
- heartbeats.add(nodeDescr);
|
|
|
|
- totalCapacity += nodeDescr.capacity;
|
|
|
|
- totalRemaining += nodeDescr.remaining;
|
|
|
|
- }
|
|
|
|
-
|
|
|
|
- private void removeHeartbeat( DatanodeDescriptor nodeDescr ) {
|
|
|
|
- totalCapacity -= nodeDescr.getCapacity();
|
|
|
|
- totalRemaining -= nodeDescr.getRemaining();
|
|
|
|
- heartbeats.remove(nodeDescr);
|
|
|
|
- }
|
|
|
|
|
|
|
|
/**
|
|
/**
|
|
* Physically remove node from datanodeMap.
|
|
* Physically remove node from datanodeMap.
|
|
@@ -1412,18 +1425,30 @@ class FSNamesystem implements FSConstants {
|
|
/**
|
|
/**
|
|
* Check if there are any expired heartbeats, and if so,
|
|
* Check if there are any expired heartbeats, and if so,
|
|
* whether any blocks have to be re-replicated.
|
|
* whether any blocks have to be re-replicated.
|
|
- */
|
|
|
|
- synchronized void heartbeatCheck() {
|
|
|
|
- synchronized (heartbeats) {
|
|
|
|
- DatanodeDescriptor nodeInfo = null;
|
|
|
|
-
|
|
|
|
- while ((heartbeats.size() > 0) &&
|
|
|
|
- ((nodeInfo = heartbeats.first()) != null) &&
|
|
|
|
- (nodeInfo.isDead())) {
|
|
|
|
- NameNode.stateChangeLog.info("BLOCK* NameSystem.heartbeatCheck: "
|
|
|
|
- + "lost heartbeat from " + nodeInfo.getName());
|
|
|
|
- removeDatanode( nodeInfo );
|
|
|
|
|
|
+ * While removing dead datanodes, make sure that only one datanode is marked
|
|
|
|
+ * dead at a time within the synchronized section. Otherwise, a cascading
|
|
|
|
+ * effect causes more datanodes to be declared dead.
|
|
|
|
+ */
|
|
|
|
+ void heartbeatCheck() {
|
|
|
|
+ boolean allAlive = false;
|
|
|
|
+ while (!allAlive) {
|
|
|
|
+ boolean foundDead = false;
|
|
|
|
+ synchronized(this) {
|
|
|
|
+ synchronized (heartbeats) {
|
|
|
|
+ for (Iterator<DatanodeDescriptor> it = heartbeats.iterator();
|
|
|
|
+ it.hasNext();) {
|
|
|
|
+ DatanodeDescriptor nodeInfo = it.next();
|
|
|
|
+ if (isDatanodeDead(nodeInfo)) {
|
|
|
|
+ NameNode.stateChangeLog.info("BLOCK* NameSystem.heartbeatCheck: "
|
|
|
|
+ + "lost heartbeat from " + nodeInfo.getName());
|
|
|
|
+ removeDatanode( nodeInfo );
|
|
|
|
+ foundDead = true;
|
|
|
|
+ break;
|
|
|
|
+ }
|
|
|
|
+ }
|
|
|
|
+ }
|
|
}
|
|
}
|
|
|
|
+ allAlive = ! foundDead;
|
|
}
|
|
}
|
|
}
|
|
}
|
|
|
|
|
|
@@ -1744,7 +1769,7 @@ class FSNamesystem implements FSConstants {
|
|
synchronized (datanodeMap) {
|
|
synchronized (datanodeMap) {
|
|
for(Iterator<DatanodeDescriptor> it = datanodeMap.values().iterator(); it.hasNext(); ) {
|
|
for(Iterator<DatanodeDescriptor> it = datanodeMap.values().iterator(); it.hasNext(); ) {
|
|
DatanodeDescriptor node = it.next();
|
|
DatanodeDescriptor node = it.next();
|
|
- if( node.isDead() )
|
|
|
|
|
|
+ if( isDatanodeDead(node))
|
|
dead.add( node );
|
|
dead.add( node );
|
|
else
|
|
else
|
|
live.add( node );
|
|
live.add( node );
|
|
@@ -1946,151 +1971,88 @@ class FSNamesystem implements FSConstants {
|
|
Collection<DatanodeDescriptor> forbiddenNodes,
|
|
Collection<DatanodeDescriptor> forbiddenNodes,
|
|
UTF8 clientMachine,
|
|
UTF8 clientMachine,
|
|
long blockSize) {
|
|
long blockSize) {
|
|
|
|
+ Collection<DatanodeDescriptor> targets = new ArrayList<DatanodeDescriptor>();
|
|
|
|
+
|
|
if (desiredReplicates > heartbeats.size()) {
|
|
if (desiredReplicates > heartbeats.size()) {
|
|
LOG.warn("Replication requested of "+desiredReplicates
|
|
LOG.warn("Replication requested of "+desiredReplicates
|
|
+" is larger than cluster size ("+heartbeats.size()
|
|
+" is larger than cluster size ("+heartbeats.size()
|
|
+"). Using cluster size.");
|
|
+"). Using cluster size.");
|
|
desiredReplicates = heartbeats.size();
|
|
desiredReplicates = heartbeats.size();
|
|
|
|
+ if (desiredReplicates == 0) {
|
|
|
|
+ LOG.warn("While choosing target, totalMachines is " + desiredReplicates);
|
|
|
|
+ }
|
|
}
|
|
}
|
|
-
|
|
|
|
- Collection<DatanodeDescriptor> alreadyChosen;
|
|
|
|
- alreadyChosen = new TreeSet<DatanodeDescriptor>();
|
|
|
|
- Collection<DatanodeDescriptor> targets = new ArrayList<DatanodeDescriptor>();
|
|
|
|
-
|
|
|
|
- for (int i = 0; i < desiredReplicates; i++) {
|
|
|
|
- DatanodeDescriptor target = chooseTarget(forbiddenNodes, alreadyChosen,
|
|
|
|
- clientMachine, blockSize);
|
|
|
|
- if (target == null)
|
|
|
|
- break; // calling chooseTarget again won't help
|
|
|
|
- targets.add(target);
|
|
|
|
- alreadyChosen.add(target);
|
|
|
|
- }
|
|
|
|
- return (DatanodeDescriptor[]) targets.toArray(new DatanodeDescriptor[targets.size()]);
|
|
|
|
- }
|
|
|
|
-
|
|
|
|
- /**
|
|
|
|
- * Choose a target from available machines, excepting the
|
|
|
|
- * given ones.
|
|
|
|
- *
|
|
|
|
- * Right now it chooses randomly from available boxes. In future could
|
|
|
|
- * choose according to capacity and load-balancing needs (or even
|
|
|
|
- * network-topology, to avoid inter-switch traffic).
|
|
|
|
- * @param forbidden1 DatanodeDescriptor targets not allowed, null allowed.
|
|
|
|
- * @param forbidden2 DatanodeDescriptor targets not allowed, null allowed.
|
|
|
|
- * @return DatanodeDescriptor instance to use or null if something went wrong
|
|
|
|
- * (a log message is emitted if null is returned).
|
|
|
|
- */
|
|
|
|
- DatanodeDescriptor chooseTarget(Collection<DatanodeDescriptor> forbidden1,
|
|
|
|
- Collection<DatanodeDescriptor> forbidden2,
|
|
|
|
- UTF8 clientMachine,
|
|
|
|
- long blockSize) {
|
|
|
|
- //
|
|
|
|
- // Check if there are any available targets at all
|
|
|
|
- //
|
|
|
|
- int totalMachines = heartbeats.size();
|
|
|
|
- if (totalMachines == 0) {
|
|
|
|
- LOG.warn("While choosing target, totalMachines is " + totalMachines);
|
|
|
|
- return null;
|
|
|
|
- }
|
|
|
|
-
|
|
|
|
- //
|
|
|
|
- // Build a map of forbidden hostnames from the two forbidden sets.
|
|
|
|
- //
|
|
|
|
- Collection<DatanodeDescriptor> forbiddenMachines = new TreeSet();
|
|
|
|
- if (forbidden1 != null) {
|
|
|
|
- for (Iterator<DatanodeDescriptor> it = forbidden1.iterator(); it.hasNext(); ) {
|
|
|
|
- DatanodeDescriptor cur = it.next();
|
|
|
|
- forbiddenMachines.add(cur);
|
|
|
|
- }
|
|
|
|
- }
|
|
|
|
- if (forbidden2 != null) {
|
|
|
|
- for (Iterator<DatanodeDescriptor> it = forbidden2.iterator(); it.hasNext(); ) {
|
|
|
|
- DatanodeDescriptor cur = it.next();
|
|
|
|
- forbiddenMachines.add(cur);
|
|
|
|
- }
|
|
|
|
- }
|
|
|
|
-
|
|
|
|
- double avgLoad = 0.0;
|
|
|
|
- //
|
|
|
|
- // Build list of machines we can actually choose from
|
|
|
|
- //
|
|
|
|
- List<DatanodeDescriptor> targetList = new ArrayList<DatanodeDescriptor>();
|
|
|
|
- for (Iterator<DatanodeDescriptor> it = heartbeats.iterator(); it.hasNext(); ) {
|
|
|
|
- DatanodeDescriptor node = it.next();
|
|
|
|
- if (! forbiddenMachines.contains(node)) {
|
|
|
|
- targetList.add(node);
|
|
|
|
- avgLoad += node.getXceiverCount();
|
|
|
|
- }
|
|
|
|
- }
|
|
|
|
- if (targetList.size() > 0) { avgLoad = avgLoad/targetList.size(); }
|
|
|
|
-
|
|
|
|
- Collections.shuffle(targetList);
|
|
|
|
|
|
|
|
- //
|
|
|
|
- // Now pick one
|
|
|
|
- //
|
|
|
|
- if (targetList.size() > 0) {
|
|
|
|
- //
|
|
|
|
- // If the requester's machine is in the targetList,
|
|
|
|
- // and it's got the capacity, pick it.
|
|
|
|
- //
|
|
|
|
- if (clientMachine != null && clientMachine.getLength() > 0) {
|
|
|
|
- for (Iterator<DatanodeDescriptor> it = targetList.iterator(); it.hasNext(); ) {
|
|
|
|
- DatanodeDescriptor node = it.next();
|
|
|
|
- if (clientMachine.toString().equals(node.getHost())) {
|
|
|
|
- if ((node.getRemaining() >= blockSize * MIN_BLOCKS_FOR_WRITE) &&
|
|
|
|
- (node.getXceiverCount() <= (2.0 * avgLoad))) {
|
|
|
|
- return node;
|
|
|
|
- }
|
|
|
|
- }
|
|
|
|
- }
|
|
|
|
- }
|
|
|
|
-
|
|
|
|
- //
|
|
|
|
- // Otherwise, choose node according to target capacity
|
|
|
|
- //
|
|
|
|
- for (Iterator<DatanodeDescriptor> it = targetList.iterator(); it.hasNext(); ) {
|
|
|
|
- DatanodeDescriptor node = it.next();
|
|
|
|
|
|
+ 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())) {
|
|
if ((node.getRemaining() >= blockSize * MIN_BLOCKS_FOR_WRITE) &&
|
|
if ((node.getRemaining() >= blockSize * MIN_BLOCKS_FOR_WRITE) &&
|
|
(node.getXceiverCount() <= (2.0 * avgLoad))) {
|
|
(node.getXceiverCount() <= (2.0 * avgLoad))) {
|
|
- return node;
|
|
|
|
|
|
+ targets.add(node);
|
|
|
|
+ desiredReplicates--;
|
|
|
|
+ break;
|
|
}
|
|
}
|
|
|
|
+ }
|
|
}
|
|
}
|
|
|
|
+ }
|
|
|
|
+ }
|
|
|
|
|
|
- //
|
|
|
|
- // If we are still not able to find a good node, then see if
|
|
|
|
- // we can pick the clientmachine itself.
|
|
|
|
- //
|
|
|
|
- if (clientMachine != null && clientMachine.getLength() > 0) {
|
|
|
|
- for (Iterator<DatanodeDescriptor> it = targetList.iterator(); it.hasNext(); ) {
|
|
|
|
- DatanodeDescriptor node = it.next();
|
|
|
|
- if (clientMachine.toString().equals(node.getHost()) &&
|
|
|
|
- node.getRemaining() >= blockSize) {
|
|
|
|
- return node;
|
|
|
|
- }
|
|
|
|
- }
|
|
|
|
|
|
+ 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.getRemaining() >= blockSize * MIN_BLOCKS_FOR_WRITE) &&
|
|
|
|
+ (node.getXceiverCount() <= (2.0 * avgLoad))) {
|
|
|
|
+ target = node;
|
|
|
|
+ break;
|
|
|
|
+ } else {
|
|
|
|
+ idx = (idx+1) % nNodes;
|
|
|
|
+ rejected++;
|
|
}
|
|
}
|
|
-
|
|
|
|
- //
|
|
|
|
- // That should do the trick. But we might not be able
|
|
|
|
- // to pick any node if the target was out of bytes. As
|
|
|
|
- // a last resort, pick the first valid one we can find.
|
|
|
|
- //
|
|
|
|
- for (Iterator<DatanodeDescriptor> it = targetList.iterator(); it.hasNext(); ) {
|
|
|
|
- DatanodeDescriptor node = it.next();
|
|
|
|
- if (node.getRemaining() >= blockSize) {
|
|
|
|
- return node;
|
|
|
|
- }
|
|
|
|
|
|
+ }
|
|
|
|
+ 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.getRemaining() >= blockSize) {
|
|
|
|
+ target = node;
|
|
|
|
+ break;
|
|
|
|
+ } else {
|
|
|
|
+ idx = (idx + 1) % nNodes;
|
|
|
|
+ rejected++;
|
|
|
|
+ }
|
|
}
|
|
}
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ if (target == null) {
|
|
LOG.warn("Could not find any nodes with sufficient capacity");
|
|
LOG.warn("Could not find any nodes with sufficient capacity");
|
|
- return null;
|
|
|
|
- } else {
|
|
|
|
- LOG.warn("Zero targets found, forbidden1.size=" +
|
|
|
|
- ( forbidden1 != null ? forbidden1.size() : 0 ) +
|
|
|
|
- " forbidden2.size()=" +
|
|
|
|
- ( forbidden2 != null ? forbidden2.size() : 0 ));
|
|
|
|
- return null;
|
|
|
|
|
|
+ break; // making one more pass over heartbeats would not help
|
|
|
|
+ }
|
|
|
|
+ targets.add(target);
|
|
}
|
|
}
|
|
|
|
+
|
|
|
|
+ return (DatanodeDescriptor[]) targets.toArray(new DatanodeDescriptor[targets.size()]);
|
|
}
|
|
}
|
|
|
|
|
|
/**
|
|
/**
|