|
@@ -122,11 +122,11 @@ class FSNamesystem implements FSConstants {
|
|
|
// Stats on overall usage
|
|
|
//
|
|
|
long totalCapacity = 0, totalRemaining = 0;
|
|
|
-
|
|
|
+
|
|
|
// total number of connections per live datanode
|
|
|
int totalLoad = 0;
|
|
|
|
|
|
-
|
|
|
+
|
|
|
//
|
|
|
// For the HTTP browsing interface
|
|
|
//
|
|
@@ -1296,20 +1296,21 @@ class FSNamesystem implements FSConstants {
|
|
|
* @return true if block report is required or false otherwise.
|
|
|
* @throws IOException
|
|
|
*/
|
|
|
- public synchronized boolean gotHeartbeat( DatanodeID nodeID,
|
|
|
- long capacity,
|
|
|
- long remaining,
|
|
|
- int xceiverCount
|
|
|
- ) throws IOException {
|
|
|
+ public boolean gotHeartbeat( DatanodeID nodeID,
|
|
|
+ long capacity,
|
|
|
+ long remaining,
|
|
|
+ int xceiverCount
|
|
|
+ ) throws IOException {
|
|
|
boolean needBlockReport;
|
|
|
synchronized (heartbeats) {
|
|
|
synchronized (datanodeMap) {
|
|
|
DatanodeDescriptor nodeinfo = getDatanode( nodeID );
|
|
|
needBlockReport = isDatanodeDead(nodeinfo);
|
|
|
|
|
|
- if (nodeinfo == null)
|
|
|
+ if (nodeinfo == null) {
|
|
|
// We do not accept unregistered guests
|
|
|
throw new UnregisteredDatanodeException( nodeID );
|
|
|
+ }
|
|
|
if (nodeinfo.isAlive) {
|
|
|
updateStats(nodeinfo, false);
|
|
|
}
|
|
@@ -1325,6 +1326,10 @@ class FSNamesystem implements FSConstants {
|
|
|
}
|
|
|
|
|
|
private void updateStats(DatanodeDescriptor node, boolean isAdded) {
|
|
|
+ //
|
|
|
+ // The statistics are protected by the heartbeat lock
|
|
|
+ //
|
|
|
+ assert(Thread.holdsLock(heartbeats));
|
|
|
if (isAdded) {
|
|
|
totalCapacity += node.getCapacity();
|
|
|
totalRemaining += node.getRemaining();
|
|
@@ -1431,19 +1436,39 @@ class FSNamesystem implements FSConstants {
|
|
|
boolean allAlive = false;
|
|
|
while (!allAlive) {
|
|
|
boolean foundDead = false;
|
|
|
- synchronized(this) {
|
|
|
- synchronized (heartbeats) {
|
|
|
+ DatanodeID nodeID = null;
|
|
|
+
|
|
|
+ // locate the first dead node.
|
|
|
+ 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;
|
|
|
+ nodeID = nodeInfo;
|
|
|
break;
|
|
|
}
|
|
|
}
|
|
|
+ }
|
|
|
+
|
|
|
+ // acquire the fsnamesystem lock, and then remove the dead node.
|
|
|
+ if (foundDead) {
|
|
|
+ synchronized (this) {
|
|
|
+ synchronized(heartbeats) {
|
|
|
+ synchronized (datanodeMap) {
|
|
|
+ DatanodeDescriptor nodeInfo = null;
|
|
|
+ try {
|
|
|
+ nodeInfo = getDatanode(nodeID);
|
|
|
+ } catch (IOException e) {
|
|
|
+ nodeInfo = null;
|
|
|
+ }
|
|
|
+ if (nodeInfo != null && isDatanodeDead(nodeInfo)) {
|
|
|
+ NameNode.stateChangeLog.info("BLOCK* NameSystem.heartbeatCheck: "
|
|
|
+ + "lost heartbeat from " + nodeInfo.getName());
|
|
|
+ removeDatanode(nodeInfo);
|
|
|
+ }
|
|
|
+ }
|
|
|
+ }
|
|
|
}
|
|
|
}
|
|
|
allAlive = ! foundDead;
|
|
@@ -1735,30 +1760,39 @@ class FSNamesystem implements FSConstants {
|
|
|
}
|
|
|
|
|
|
/**
|
|
|
- * Total raw bytes
|
|
|
+ * Total raw bytes.
|
|
|
*/
|
|
|
public long totalCapacity() {
|
|
|
+ synchronized (heartbeats) {
|
|
|
return totalCapacity;
|
|
|
+ }
|
|
|
}
|
|
|
|
|
|
/**
|
|
|
- * Total non-used raw bytes
|
|
|
+ * Total non-used raw bytes.
|
|
|
*/
|
|
|
public long totalRemaining() {
|
|
|
+ synchronized (heartbeats) {
|
|
|
return totalRemaining;
|
|
|
+ }
|
|
|
}
|
|
|
|
|
|
/**
|
|
|
+ * Total number of connections.
|
|
|
*/
|
|
|
+ public int totalLoad() {
|
|
|
+ synchronized (heartbeats) {
|
|
|
+ return totalLoad;
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
public synchronized DatanodeInfo[] datanodeReport() {
|
|
|
DatanodeInfo results[] = null;
|
|
|
- synchronized (heartbeats) {
|
|
|
- synchronized (datanodeMap) {
|
|
|
+ synchronized (datanodeMap) {
|
|
|
results = new DatanodeInfo[datanodeMap.size()];
|
|
|
int i = 0;
|
|
|
for(Iterator<DatanodeDescriptor> it = datanodeMap.values().iterator(); it.hasNext(); )
|
|
|
results[i++] = new DatanodeInfo( it.next() );
|
|
|
- }
|
|
|
}
|
|
|
return results;
|
|
|
}
|
|
@@ -1767,8 +1801,7 @@ class FSNamesystem implements FSConstants {
|
|
|
*/
|
|
|
public synchronized void DFSNodesStatus( ArrayList<DatanodeDescriptor> live,
|
|
|
ArrayList<DatanodeDescriptor> dead ) {
|
|
|
- synchronized (heartbeats) {
|
|
|
- synchronized (datanodeMap) {
|
|
|
+ synchronized (datanodeMap) {
|
|
|
for(Iterator<DatanodeDescriptor> it = datanodeMap.values().iterator(); it.hasNext(); ) {
|
|
|
DatanodeDescriptor node = it.next();
|
|
|
if( isDatanodeDead(node))
|
|
@@ -1776,7 +1809,6 @@ class FSNamesystem implements FSConstants {
|
|
|
else
|
|
|
live.add( node );
|
|
|
}
|
|
|
- }
|
|
|
}
|
|
|
}
|
|
|
/**
|
|
@@ -2018,7 +2050,7 @@ class FSNamesystem implements FSConstants {
|
|
|
|
|
|
double avgLoad = 0.0;
|
|
|
if (heartbeats.size() != 0) {
|
|
|
- avgLoad = (double) totalLoad / heartbeats.size();
|
|
|
+ avgLoad = (double) totalLoad() / heartbeats.size();
|
|
|
}
|
|
|
// choose local replica first
|
|
|
if (desiredReplicates != 0) {
|