Browse Source

HADOOP-3620. Namenode should synchronously resolve a datanode's network location when the datanode registers. Contributed by Hairong Kuang.

git-svn-id: https://svn.apache.org/repos/asf/hadoop/core/trunk@682536 13f79535-47bb-0310-9956-ffa450edef68
Hairong Kuang 17 years ago
parent
commit
392d4fb8db

+ 3 - 0
CHANGES.txt

@@ -134,6 +134,9 @@ Trunk (unreleased changes)
     HADOOP-3694. Improve unit test performance by changing
     HADOOP-3694. Improve unit test performance by changing
     MiniDFSCluster to listen only on 127.0.0.1.  (cutting)
     MiniDFSCluster to listen only on 127.0.0.1.  (cutting)
 
 
+    HADOOP-3620. Namenode should synchronously resolve a datanode's network
+    location when the datanode registers. (hairong)
+
   OPTIMIZATIONS
   OPTIMIZATIONS
 
 
     HADOOP-3556. Removed lock contention in MD5Hash by changing the 
     HADOOP-3556. Removed lock contention in MD5Hash by changing the 

+ 1 - 1
conf/hadoop-default.xml

@@ -1257,7 +1257,7 @@ creations/deletions), or "all".</description>
 
 
 <property>
 <property>
   <name>topology.script.number.args</name>
   <name>topology.script.number.args</name>
-  <value>20</value>
+  <value>100</value>
   <description> The max number of args that the script configured with 
   <description> The max number of args that the script configured with 
     topology.script.file.name should be run with. Each arg is an
     topology.script.file.name should be run with. Each arg is an
     IP address.
     IP address.

+ 39 - 0
src/core/org/apache/hadoop/net/NetUtils.java

@@ -20,9 +20,11 @@ package org.apache.hadoop.net;
 import java.io.IOException;
 import java.io.IOException;
 import java.io.InputStream;
 import java.io.InputStream;
 import java.io.OutputStream;
 import java.io.OutputStream;
+import java.net.InetAddress;
 import java.net.InetSocketAddress;
 import java.net.InetSocketAddress;
 import java.net.Socket;
 import java.net.Socket;
 import java.net.URI;
 import java.net.URI;
+import java.net.UnknownHostException;
 import java.util.Map.Entry;
 import java.util.Map.Entry;
 import java.util.*;
 import java.util.*;
 
 
@@ -366,4 +368,41 @@ public class NetUtils {
     return (socket.getChannel() == null) ? 
     return (socket.getChannel() == null) ? 
             socket.getOutputStream() : new SocketOutputStream(socket, timeout);            
             socket.getOutputStream() : new SocketOutputStream(socket, timeout);            
   }
   }
+  
+  /** 
+   * Given a string representation of a host, return its ip address
+   * in textual presentation.
+   * 
+   * @param name a string representation of a host:
+   *             either a textual representation its IP address or its host name
+   * @return its IP address in the string format
+   */
+  public static String normalizeHostName(String name) {
+    if (Character.digit(name.charAt(0), 16) != -1) { // it is an IP
+      return name;
+    } else {
+      try {
+        InetAddress ipAddress = InetAddress.getByName(name);
+        return ipAddress.getHostAddress();
+      } catch (UnknownHostException e) {
+        return name;
+      }
+    }
+  }
+  
+  /** 
+   * Given a collection of string representation of hosts, return a list of
+   * corresponding IP addresses in the textual representation.
+   * 
+   * @param names a collection of string representations of hosts
+   * @return a list of corresponding IP addresses in the string format
+   * @see #normalizeHostName(String)
+   */
+  public static List<String> normalizeHostNames(Collection<String> names) {
+    List<String> hostNames = new ArrayList<String>(names.size());
+    for (String name : names) {
+      hostNames.add(normalizeHostName(name));
+    }
+    return hostNames;
+  }
 }
 }

+ 0 - 6
src/core/org/apache/hadoop/net/NetworkTopology.java

@@ -38,7 +38,6 @@ import org.apache.commons.logging.LogFactory;
  */
  */
 public class NetworkTopology {
 public class NetworkTopology {
   public final static String DEFAULT_RACK = "/default-rack";
   public final static String DEFAULT_RACK = "/default-rack";
-  public final static String UNRESOLVED = "";
   public final static int DEFAULT_HOST_LEVEL = 2;
   public final static int DEFAULT_HOST_LEVEL = 2;
   public static final Log LOG = 
   public static final Log LOG = 
     LogFactory.getLog("org.apache.hadoop.net.NetworkTopology");
     LogFactory.getLog("org.apache.hadoop.net.NetworkTopology");
@@ -344,11 +343,6 @@ public class NetworkTopology {
    */ 
    */ 
   public void remove(Node node) {
   public void remove(Node node) {
     if (node==null) return;
     if (node==null) return;
-    if (NetworkTopology.UNRESOLVED.equals(node.getNetworkLocation())) {
-      // the node's network location has not resolved yet; 
-      // so it is not in the network topology
-      return;  
-    }
     if( node instanceof InnerNode ) {
     if( node instanceof InnerNode ) {
       throw new IllegalArgumentException(
       throw new IllegalArgumentException(
         "Not allow to remove an inner node: "+NodeBase.getPath(node));
         "Not allow to remove an inner node: "+NodeBase.getPath(node));

+ 31 - 39
src/core/org/apache/hadoop/net/ScriptBasedMapping.java

@@ -20,7 +20,6 @@ package org.apache.hadoop.net;
 
 
 import java.util.*;
 import java.util.*;
 import java.io.*;
 import java.io.*;
-import java.net.*;
 
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.commons.logging.LogFactory;
@@ -32,66 +31,70 @@ import org.apache.hadoop.conf.*;
  * This class implements the {@link DNSToSwitchMapping} interface using a 
  * This class implements the {@link DNSToSwitchMapping} interface using a 
  * script configured via topology.script.file.name .
  * script configured via topology.script.file.name .
  */
  */
-public final class ScriptBasedMapping implements Configurable, 
-DNSToSwitchMapping
+public final class ScriptBasedMapping extends CachedDNSToSwitchMapping 
+implements Configurable
 {
 {
+  public ScriptBasedMapping() {
+    super(new RawScriptBasedMapping());
+  }
+  
+  public ScriptBasedMapping(Configuration conf) {
+    this();
+    setConf(conf);
+  }
+  
+  public Configuration getConf() {
+    return ((RawScriptBasedMapping)rawMapping).getConf();
+  }
+  
+  public void setConf(Configuration conf) {
+    ((RawScriptBasedMapping)rawMapping).setConf(conf);
+  }
+  
+  private static final class RawScriptBasedMapping
+  implements DNSToSwitchMapping {
   private String scriptName;
   private String scriptName;
   private Configuration conf;
   private Configuration conf;
   private int maxArgs; //max hostnames per call of the script
   private int maxArgs; //max hostnames per call of the script
-  private Map<String, String> cache = new TreeMap<String, String>();
   private static Log LOG = 
   private static Log LOG = 
     LogFactory.getLog("org.apache.hadoop.net.ScriptBasedMapping");
     LogFactory.getLog("org.apache.hadoop.net.ScriptBasedMapping");
   public void setConf (Configuration conf) {
   public void setConf (Configuration conf) {
     this.scriptName = conf.get("topology.script.file.name");
     this.scriptName = conf.get("topology.script.file.name");
-    this.maxArgs = conf.getInt("topology.script.number.args", 20);
+    this.maxArgs = conf.getInt("topology.script.number.args", 100);
     this.conf = conf;
     this.conf = conf;
   }
   }
   public Configuration getConf () {
   public Configuration getConf () {
     return conf;
     return conf;
   }
   }
-
-  public ScriptBasedMapping() {}
+  
+  public RawScriptBasedMapping() {}
   
   
   public List<String> resolve(List<String> names) {
   public List<String> resolve(List<String> names) {
     List <String> m = new ArrayList<String>(names.size());
     List <String> m = new ArrayList<String>(names.size());
     
     
+    if (names.isEmpty()) {
+      return m;
+    }
+
     if (scriptName == null) {
     if (scriptName == null) {
       for (int i = 0; i < names.size(); i++) {
       for (int i = 0; i < names.size(); i++) {
         m.add(NetworkTopology.DEFAULT_RACK);
         m.add(NetworkTopology.DEFAULT_RACK);
       }
       }
       return m;
       return m;
     }
     }
-    List<String> hosts = new ArrayList<String>(names.size());
-    for (String name : names) {
-      name = getHostName(name);
-      if (cache.get(name) == null) {
-        hosts.add(name);
-      } 
-    }
     
     
-    int i = 0;
-    String output = runResolveCommand(hosts);
+    String output = runResolveCommand(names);
     if (output != null) {
     if (output != null) {
       StringTokenizer allSwitchInfo = new StringTokenizer(output);
       StringTokenizer allSwitchInfo = new StringTokenizer(output);
       while (allSwitchInfo.hasMoreTokens()) {
       while (allSwitchInfo.hasMoreTokens()) {
         String switchInfo = allSwitchInfo.nextToken();
         String switchInfo = allSwitchInfo.nextToken();
-        cache.put(hosts.get(i++), switchInfo);
-      }
-    }
-    for (String name : names) {
-      //now everything is in the cache
-      name = getHostName(name);
-      if (cache.get(name) != null) {
-        m.add(cache.get(name));
-      } else { //resolve all or nothing
-        return null;
+        m.add(switchInfo);
       }
       }
     }
     }
     return m;
     return m;
   }
   }
   
   
   private String runResolveCommand(List<String> args) {
   private String runResolveCommand(List<String> args) {
-    InetAddress ipaddr = null;
     int loopCount = 0;
     int loopCount = 0;
     if (args.size() == 0) {
     if (args.size() == 0) {
       return null;
       return null;
@@ -104,12 +107,7 @@ DNSToSwitchMapping
       cmdList.add(scriptName);
       cmdList.add(scriptName);
       for (numProcessed = start; numProcessed < (start + maxArgs) && 
       for (numProcessed = start; numProcessed < (start + maxArgs) && 
            numProcessed < args.size(); numProcessed++) {
            numProcessed < args.size(); numProcessed++) {
-        try {
-          ipaddr = InetAddress.getByName(args.get(numProcessed));
-        } catch (UnknownHostException uh) {
-          return null;
-        }
-        cmdList.add(ipaddr.getHostAddress()); 
+        cmdList.add(args.get(numProcessed)); 
       }
       }
       File dir = null;
       File dir = null;
       String userDir;
       String userDir;
@@ -129,11 +127,5 @@ DNSToSwitchMapping
     }
     }
     return allOutput.toString();
     return allOutput.toString();
   }
   }
-  private String getHostName(String hostWithPort) {
-    int j;
-    if ((j = hostWithPort.indexOf(':')) != -1) {
-      hostWithPort = hostWithPort.substring(0, j);
-    }
-    return hostWithPort;
   }
   }
 }
 }

+ 3 - 5
src/hdfs/org/apache/hadoop/hdfs/protocol/DatanodeInfo.java

@@ -43,7 +43,7 @@ public class DatanodeInfo extends DatanodeID implements Node {
   protected long remaining;
   protected long remaining;
   protected long lastUpdate;
   protected long lastUpdate;
   protected int xceiverCount;
   protected int xceiverCount;
-  protected String location = NetworkTopology.UNRESOLVED;
+  protected String location = NetworkTopology.DEFAULT_RACK;
 
 
   /** HostName as suplied by the datanode during registration as its 
   /** HostName as suplied by the datanode during registration as its 
    * name. Namenode uses datanode IP address as the name.
    * name. Namenode uses datanode IP address as the name.
@@ -146,8 +146,7 @@ public class DatanodeInfo extends DatanodeID implements Node {
     long r = getRemaining();
     long r = getRemaining();
     long u = getDfsUsed();
     long u = getDfsUsed();
     buffer.append("Name: "+name+"\n");
     buffer.append("Name: "+name+"\n");
-    if (!NetworkTopology.UNRESOLVED.equals(location) && 
-        !NetworkTopology.DEFAULT_RACK.equals(location)) {
+    if (!NetworkTopology.DEFAULT_RACK.equals(location)) {
       buffer.append("Rack: "+location+"\n");
       buffer.append("Rack: "+location+"\n");
     }
     }
     buffer.append("Decommission Status : ");
     buffer.append("Decommission Status : ");
@@ -173,8 +172,7 @@ public class DatanodeInfo extends DatanodeID implements Node {
     long r = getRemaining();
     long r = getRemaining();
     long u = getDfsUsed();
     long u = getDfsUsed();
     buffer.append(name);
     buffer.append(name);
-    if (!NetworkTopology.UNRESOLVED.equals(location) &&
-        !NetworkTopology.DEFAULT_RACK.equals(location)) {
+    if (!NetworkTopology.DEFAULT_RACK.equals(location)) {
       buffer.append(" "+location);
       buffer.append(" "+location);
     }
     }
     if (isDecommissioned()) {
     if (isDecommissioned()) {

+ 4 - 16
src/hdfs/org/apache/hadoop/hdfs/server/datanode/DataNode.java

@@ -136,10 +136,9 @@ public class DataNode extends Configured
   ThreadGroup threadGroup = null;
   ThreadGroup threadGroup = null;
   long blockReportInterval;
   long blockReportInterval;
   //disallow the sending of BR before instructed to do so
   //disallow the sending of BR before instructed to do so
-  long lastBlockReport = Long.MAX_VALUE;
+  long lastBlockReport = 0;
   boolean resetBlockReportTime = true;
   boolean resetBlockReportTime = true;
   long initialBlockReportDelay = BLOCKREPORT_INITIAL_DELAY * 1000L;
   long initialBlockReportDelay = BLOCKREPORT_INITIAL_DELAY * 1000L;
-  private boolean waitForFirstBlockReportRequest = false;
   long lastHeartbeat = 0;
   long lastHeartbeat = 0;
   long heartBeatInterval;
   long heartBeatInterval;
   private DataStorage storage = null;
   private DataStorage storage = null;
@@ -533,7 +532,9 @@ public class DataNode extends Configured
           + dnRegistration.getStorageID() 
           + dnRegistration.getStorageID() 
           + ". Expecting " + storage.getStorageID());
           + ". Expecting " + storage.getStorageID());
     }
     }
-    waitForFirstBlockReportRequest = true;
+    
+    // random short delay - helps scatter the BR from all DNs
+    scheduleBlockReport(initialBlockReportDelay);
   }
   }
 
 
   /**
   /**
@@ -834,19 +835,6 @@ public class DataNode extends Configured
       // start distributed upgrade here
       // start distributed upgrade here
       processDistributedUpgradeCommand((UpgradeCommand)cmd);
       processDistributedUpgradeCommand((UpgradeCommand)cmd);
       break;
       break;
-    case DatanodeProtocol.DNA_BLOCKREPORT:
-      // only send BR when receive request the 1st time
-      if (waitForFirstBlockReportRequest) {
-        LOG.info("DatanodeCommand action: DNA_BLOCKREPORT - scheduled");
-        // dropping all following BR requests
-        waitForFirstBlockReportRequest = false;
-        // random short delay - helps scatter the BR from all DNs
-        scheduleBlockReport(initialBlockReportDelay);
-      } else {
-        LOG.info("DatanodeCommand action: DNA_BLOCKREPORT" +
-            "- ignored becaused one is already scheduled");
-      }
-      break;
     case DatanodeProtocol.DNA_RECOVERBLOCK:
     case DatanodeProtocol.DNA_RECOVERBLOCK:
       recoverBlocks(bcmd.getBlocks(), bcmd.getTargets());
       recoverBlocks(bcmd.getBlocks(), bcmd.getTargets());
       break;
       break;

+ 0 - 17
src/hdfs/org/apache/hadoop/hdfs/server/namenode/DatanodeDescriptor.java

@@ -97,8 +97,6 @@ public class DatanodeDescriptor extends DatanodeInfo {
   /** A set of blocks to be invalidated by this datanode */
   /** A set of blocks to be invalidated by this datanode */
   private Set<Block> invalidateBlocks = new TreeSet<Block>();
   private Set<Block> invalidateBlocks = new TreeSet<Block>();
 
 
-  boolean processedBlockReport = false;
-  
   /* Variables for maintaning number of blocks scheduled to be written to
   /* Variables for maintaning number of blocks scheduled to be written to
    * this datanode. This count is approximate and might be slightly higger
    * this datanode. This count is approximate and might be slightly higger
    * in case of errors (e.g. datanode does not report if an error occurs 
    * in case of errors (e.g. datanode does not report if an error occurs 
@@ -307,21 +305,6 @@ public class DatanodeDescriptor extends DatanodeInfo {
     }
     }
   }
   }
   
   
-  /**
-   * Set the bit signifying that the first block report from this datanode has been 
-   * processed
-   */
-  void setBlockReportProcessed(boolean val) {
-    processedBlockReport = val;
-  }
-  
-  /**
-   * Have we processed any block report from this datanode yet
-   */
-  boolean getBlockReportProcessed() {
-    return processedBlockReport;
-  }
-
   BlockCommand getReplicationCommand(int maxTransfers) {
   BlockCommand getReplicationCommand(int maxTransfers) {
     List<BlockTargetPair> blocktargetlist = replicateBlocks.poll(maxTransfers);
     List<BlockTargetPair> blocktargetlist = replicateBlocks.poll(maxTransfers);
     return blocktargetlist == null? null:
     return blocktargetlist == null? null:

+ 62 - 119
src/hdfs/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java

@@ -34,6 +34,7 @@ import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.util.*;
 import org.apache.hadoop.util.*;
 import org.apache.hadoop.http.HttpServer;
 import org.apache.hadoop.http.HttpServer;
 import org.apache.hadoop.metrics.util.MBeanUtil;
 import org.apache.hadoop.metrics.util.MBeanUtil;
+import org.apache.hadoop.net.CachedDNSToSwitchMapping;
 import org.apache.hadoop.net.DNSToSwitchMapping;
 import org.apache.hadoop.net.DNSToSwitchMapping;
 import org.apache.hadoop.net.NetUtils;
 import org.apache.hadoop.net.NetUtils;
 import org.apache.hadoop.net.NetworkTopology;
 import org.apache.hadoop.net.NetworkTopology;
@@ -62,7 +63,6 @@ import java.net.InetAddress;
 import java.net.InetSocketAddress;
 import java.net.InetSocketAddress;
 import java.util.*;
 import java.util.*;
 import java.util.Map.Entry;
 import java.util.Map.Entry;
-import java.util.concurrent.LinkedBlockingQueue;
 
 
 import javax.management.NotCompliantMBeanException;
 import javax.management.NotCompliantMBeanException;
 import javax.management.ObjectName;
 import javax.management.ObjectName;
@@ -221,7 +221,6 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean {
   public Daemon lmthread = null;   // LeaseMonitor thread
   public Daemon lmthread = null;   // LeaseMonitor thread
   Daemon smmthread = null;  // SafeModeMonitor thread
   Daemon smmthread = null;  // SafeModeMonitor thread
   public Daemon replthread = null;  // Replication thread
   public Daemon replthread = null;  // Replication thread
-  Daemon resthread = null; //ResolutionMonitor thread
   
   
   volatile boolean fsRunning = true;
   volatile boolean fsRunning = true;
   long systemStart = 0;
   long systemStart = 0;
@@ -260,8 +259,6 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean {
   // datanode networktoplogy
   // datanode networktoplogy
   NetworkTopology clusterMap = new NetworkTopology();
   NetworkTopology clusterMap = new NetworkTopology();
   private DNSToSwitchMapping dnsToSwitchMapping;
   private DNSToSwitchMapping dnsToSwitchMapping;
-  private LinkedBlockingQueue<DatanodeDescriptor> resolutionQueue = 
-    new LinkedBlockingQueue <DatanodeDescriptor>();
   
   
   // for block replicas placement
   // for block replicas placement
   ReplicationTargetChooser replicator;
   ReplicationTargetChooser replicator;
@@ -318,11 +315,9 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean {
     this.hbthread = new Daemon(new HeartbeatMonitor());
     this.hbthread = new Daemon(new HeartbeatMonitor());
     this.lmthread = new Daemon(leaseManager.createMonitor());
     this.lmthread = new Daemon(leaseManager.createMonitor());
     this.replthread = new Daemon(new ReplicationMonitor());
     this.replthread = new Daemon(new ReplicationMonitor());
-    this.resthread = new Daemon(new ResolutionMonitor());
     hbthread.start();
     hbthread.start();
     lmthread.start();
     lmthread.start();
     replthread.start();
     replthread.start();
-    resthread.start();
 
 
     this.hostsReader = new HostsFileReader(conf.get("dfs.hosts",""),
     this.hostsReader = new HostsFileReader(conf.get("dfs.hosts",""),
                                            conf.get("dfs.hosts.exclude",""));
                                            conf.get("dfs.hosts.exclude",""));
@@ -332,6 +327,15 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean {
     this.dnsToSwitchMapping = ReflectionUtils.newInstance(
     this.dnsToSwitchMapping = ReflectionUtils.newInstance(
         conf.getClass("topology.node.switch.mapping.impl", ScriptBasedMapping.class,
         conf.getClass("topology.node.switch.mapping.impl", ScriptBasedMapping.class,
             DNSToSwitchMapping.class), conf);
             DNSToSwitchMapping.class), conf);
+    
+    /* If the dns to swith mapping supports cache, resolve network 
+     * locations of those hosts in the include list, 
+     * and store the mapping in the cache; so future calls to resolve
+     * will be fast.
+     */
+    if (dnsToSwitchMapping instanceof CachedDNSToSwitchMapping) {
+      dnsToSwitchMapping.resolve(new ArrayList<String>(hostsReader.getHosts()));
+    }
 
 
     String infoAddr = 
     String infoAddr = 
       NetUtils.getServerAddress(conf, "dfs.info.bindAddress", 
       NetUtils.getServerAddress(conf, "dfs.info.bindAddress", 
@@ -484,7 +488,6 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean {
       if (infoServer != null) infoServer.stop();
       if (infoServer != null) infoServer.stop();
       if (hbthread != null) hbthread.interrupt();
       if (hbthread != null) hbthread.interrupt();
       if (replthread != null) replthread.interrupt();
       if (replthread != null) replthread.interrupt();
-      if (resthread != null) resthread.interrupt();
       if (dnthread != null) dnthread.interrupt();
       if (dnthread != null) dnthread.interrupt();
       if (smmthread != null) smmthread.interrupt();
       if (smmthread != null) smmthread.interrupt();
     } catch (InterruptedException ie) {
     } catch (InterruptedException ie) {
@@ -1868,79 +1871,6 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean {
     return dir.getListing(src);
     return dir.getListing(src);
   }
   }
 
 
-  public void addToResolutionQueue(DatanodeDescriptor d) {
-    while (!resolutionQueue.add(d)) {
-      LOG.warn("Couldn't add to the Resolution queue now. Will " +
-               "try again");
-      try {
-        Thread.sleep(2000);
-      } catch (InterruptedException ie) {}
-    }
-  }
-  
-  private class ResolutionMonitor implements Runnable {
-    public void run() {
-      try {
-        while (fsRunning) {
-          try {
-            List <DatanodeDescriptor> datanodes = 
-              new ArrayList<DatanodeDescriptor>(resolutionQueue.size());
-            // Block if the queue is empty
-            datanodes.add(resolutionQueue.take());
-            resolutionQueue.drainTo(datanodes);
-            List<String> dnHosts = new ArrayList<String>(datanodes.size());
-            for (DatanodeDescriptor d : datanodes) {
-              dnHosts.add(d.getName());
-            }
-            List<String> rName = dnsToSwitchMapping.resolve(dnHosts);
-            if (rName == null) {
-              LOG.error("The resolve call returned null! Using " + 
-                  NetworkTopology.DEFAULT_RACK + " for some hosts");
-              rName = new ArrayList<String>(dnHosts.size());
-              for (int i = 0; i < dnHosts.size(); i++) {
-                rName.add(NetworkTopology.DEFAULT_RACK);
-              }
-            }
-            int i = 0;
-            for (String m : rName) {
-              DatanodeDescriptor d = datanodes.get(i++); 
-              d.setNetworkLocation(m);
-              clusterMap.add(d);
-            }
-          } catch (InterruptedException e) {
-              FSNamesystem.LOG.debug("ResolutionMonitor thread received " +
-                                     "InterruptException. " + e);
-          }
-        }
-      } catch (Exception e) {
-        FSNamesystem.LOG.error(StringUtils.stringifyException(e));
-      }
-    }
-  }
-  
-  /**
-   * Has the block report of the datanode represented by nodeReg been processed
-   * yet.
-   * @param nodeReg
-   * @return true or false
-   */
-  synchronized boolean blockReportProcessed(DatanodeRegistration nodeReg)
-  throws IOException {
-    return getDatanode(nodeReg).getBlockReportProcessed();
-  }
-  
-  /**
-   * Has the datanode been resolved to a switch/rack
-   */
-  synchronized boolean isResolved(DatanodeRegistration dnReg) {
-    try {
-      return !getDatanode(dnReg).getNetworkLocation()
-            .equals(NetworkTopology.UNRESOLVED);
-    } catch (IOException ie) {
-      return false;
-    }
-  }
-    
   /////////////////////////////////////////////////////////
   /////////////////////////////////////////////////////////
   //
   //
   // These methods are called by datanodes
   // These methods are called by datanodes
@@ -1971,11 +1901,6 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean {
    */
    */
   public synchronized void registerDatanode(DatanodeRegistration nodeReg
   public synchronized void registerDatanode(DatanodeRegistration nodeReg
                                             ) throws IOException {
                                             ) throws IOException {
-
-    if (!verifyNodeRegistration(nodeReg)) {
-      throw new DisallowedDatanodeException(nodeReg);
-    }
-
     String dnAddress = Server.getRemoteAddress();
     String dnAddress = Server.getRemoteAddress();
     if (dnAddress == null) {
     if (dnAddress == null) {
       // Mostly called inside an RPC.
       // Mostly called inside an RPC.
@@ -1983,6 +1908,11 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean {
       dnAddress = nodeReg.getHost();
       dnAddress = nodeReg.getHost();
     }      
     }      
 
 
+    // check if the datanode is allowed to be connect to the namenode
+    if (!verifyNodeRegistration(nodeReg, dnAddress)) {
+      throw new DisallowedDatanodeException(nodeReg);
+    }
+
     String hostName = nodeReg.getHost();
     String hostName = nodeReg.getHost();
       
       
     // update the datanode's name with ip:port
     // update the datanode's name with ip:port
@@ -2038,9 +1968,10 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean {
       clusterMap.remove(nodeS);
       clusterMap.remove(nodeS);
       nodeS.updateRegInfo(nodeReg);
       nodeS.updateRegInfo(nodeReg);
       nodeS.setHostName(hostName);
       nodeS.setHostName(hostName);
-      nodeS.setNetworkLocation(NetworkTopology.UNRESOLVED);
-      nodeS.setBlockReportProcessed(false);
-      addToResolutionQueue(nodeS);
+      
+      // resolve network location
+      resolveNetworkLocation(nodeS);
+      clusterMap.add(nodeS);
         
         
       // also treat the registration message as a heartbeat
       // also treat the registration message as a heartbeat
       synchronized(heartbeats) {
       synchronized(heartbeats) {
@@ -2065,9 +1996,10 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean {
     }
     }
     // register new datanode
     // register new datanode
     DatanodeDescriptor nodeDescr 
     DatanodeDescriptor nodeDescr 
-      = new DatanodeDescriptor(nodeReg, NetworkTopology.UNRESOLVED, hostName);
+      = new DatanodeDescriptor(nodeReg, NetworkTopology.DEFAULT_RACK, hostName);
+    resolveNetworkLocation(nodeDescr);
     unprotectedAddDatanode(nodeDescr);
     unprotectedAddDatanode(nodeDescr);
-    addToResolutionQueue(nodeDescr);
+    clusterMap.add(nodeDescr);
       
       
     // also treat the registration message as a heartbeat
     // also treat the registration message as a heartbeat
     synchronized(heartbeats) {
     synchronized(heartbeats) {
@@ -2079,6 +2011,33 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean {
     return;
     return;
   }
   }
     
     
+  /* Resolve a node's network location */
+  private void resolveNetworkLocation (DatanodeDescriptor node) {
+    List<String> names = new ArrayList<String>(1);
+    if (dnsToSwitchMapping instanceof CachedDNSToSwitchMapping) {
+      // get the node's IP address
+      names.add(node.getHost());
+    } else {
+      // get the node's host name
+      String hostName = node.getHostName();
+      int colon = hostName.indexOf(":");
+      hostName = (colon==-1)?hostName:hostName.substring(0,colon);
+      names.add(hostName);
+    }
+    
+    // resolve its network location
+    List<String> rName = dnsToSwitchMapping.resolve(names);
+    String networkLocation;
+    if (rName == null) {
+      LOG.error("The resolve call returned null! Using " + 
+          NetworkTopology.DEFAULT_RACK + " for host " + names);
+      networkLocation = NetworkTopology.DEFAULT_RACK;
+    } else {
+      networkLocation = rName.get(0);
+    }
+    node.setNetworkLocation(networkLocation);
+  }
+  
   /**
   /**
    * Get registrationID for datanodes based on the namespaceID.
    * Get registrationID for datanodes based on the namespaceID.
    * 
    * 
@@ -2171,16 +2130,6 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean {
       }
       }
     }
     }
 
 
-    // If the datanode has (just) been resolved and we haven't ever processed 
-    // a block report from it yet, ask for one now.
-    if (!blockReportProcessed(nodeReg)) {
-      // If we never processed a block report from this datanode, we shouldn't
-      // have any work for that as well
-      assert(cmd == null);
-      if (isResolved(nodeReg)) {
-        return DatanodeCommand.BLOCKREPORT;
-      }
-    }
     //check distributed upgrade
     //check distributed upgrade
     if (cmd == null) {
     if (cmd == null) {
       cmd = getDistributedUpgradeCommand();
       cmd = getDistributedUpgradeCommand();
@@ -2717,13 +2666,6 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean {
       throw new DisallowedDatanodeException(node);
       throw new DisallowedDatanodeException(node);
     }
     }
     
     
-    if (node.getNetworkLocation().equals(NetworkTopology.UNRESOLVED)) {
-      LOG.info("Ignoring block report from " + nodeID.getName() + 
-          " because rack location for this datanode is still to be resolved."); 
-      return; //drop the block report if the dn hasn't been resolved
-    }
-
-    node.setBlockReportProcessed(true);
     //
     //
     // Modify the (block-->datanode) map, according to the difference
     // Modify the (block-->datanode) map, according to the difference
     // between the old and new block report.
     // between the old and new block report.
@@ -3526,22 +3468,23 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean {
   }
   }
 
 
   /** 
   /** 
-   * Keeps track of which datanodes are allowed to connect to the namenode.
+   * Keeps track of which datanodes/ipaddress are allowed to connect to the namenode.
    */
    */
-  private boolean inHostsList(DatanodeID node) {
+  private boolean inHostsList(DatanodeID node, String ipAddr) {
     Set<String> hostsList = hostsReader.getHosts();
     Set<String> hostsList = hostsReader.getHosts();
     return (hostsList.isEmpty() || 
     return (hostsList.isEmpty() || 
-            hostsList.contains(node.getName()) || 
+            (ipAddr != null && hostsList.contains(ipAddr)) ||
             hostsList.contains(node.getHost()) ||
             hostsList.contains(node.getHost()) ||
+            hostsList.contains(node.getName()) || 
             ((node instanceof DatanodeInfo) && 
             ((node instanceof DatanodeInfo) && 
              hostsList.contains(((DatanodeInfo)node).getHostName())));
              hostsList.contains(((DatanodeInfo)node).getHostName())));
   }
   }
-
-
-  private boolean inExcludedHostsList(DatanodeID node) {
+  
+  private boolean inExcludedHostsList(DatanodeID node, String ipAddr) {
     Set<String> excludeList = hostsReader.getExcludedHosts();
     Set<String> excludeList = hostsReader.getExcludedHosts();
-    return (excludeList.contains(node.getName()) ||
+    return  ((ipAddr != null && excludeList.contains(ipAddr)) ||
             excludeList.contains(node.getHost()) ||
             excludeList.contains(node.getHost()) ||
+            excludeList.contains(node.getName()) ||
             ((node instanceof DatanodeInfo) && 
             ((node instanceof DatanodeInfo) && 
              excludeList.contains(((DatanodeInfo)node).getHostName())));
              excludeList.contains(((DatanodeInfo)node).getHostName())));
   }
   }
@@ -3569,10 +3512,10 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean {
            it.hasNext();) {
            it.hasNext();) {
         DatanodeDescriptor node = it.next();
         DatanodeDescriptor node = it.next();
         // Check if not include.
         // Check if not include.
-        if (!inHostsList(node)) {
+        if (!inHostsList(node, null)) {
           node.setDecommissioned();  // case 2.
           node.setDecommissioned();  // case 2.
         } else {
         } else {
-          if (inExcludedHostsList(node)) {
+          if (inExcludedHostsList(node, null)) {
             if (!node.isDecommissionInProgress() && 
             if (!node.isDecommissionInProgress() && 
                 !node.isDecommissioned()) {
                 !node.isDecommissioned()) {
               startDecommission(node);   // case 3.
               startDecommission(node);   // case 3.
@@ -3602,12 +3545,12 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean {
    * Returns TRUE if node is registered (including when it is on the 
    * Returns TRUE if node is registered (including when it is on the 
    * exclude list and is being decommissioned). 
    * exclude list and is being decommissioned). 
    */
    */
-  public synchronized boolean verifyNodeRegistration(DatanodeRegistration nodeReg) 
+  private synchronized boolean verifyNodeRegistration(DatanodeRegistration nodeReg, String ipAddr) 
     throws IOException {
     throws IOException {
-    if (!inHostsList(nodeReg)) {
+    if (!inHostsList(nodeReg, ipAddr)) {
       return false;    
       return false;    
     }
     }
-    if (inExcludedHostsList(nodeReg)) {
+    if (inExcludedHostsList(nodeReg, ipAddr)) {
       DatanodeDescriptor node = getDatanode(nodeReg);
       DatanodeDescriptor node = getDatanode(nodeReg);
       if (node == null) {
       if (node == null) {
         throw new IOException("verifyNodeRegistration: unknown datanode " +
         throw new IOException("verifyNodeRegistration: unknown datanode " +

+ 1 - 1
src/hdfs/org/apache/hadoop/hdfs/server/namenode/NameNode.java

@@ -136,7 +136,7 @@ public class NameNode implements ClientProtocol, DatanodeProtocol,
     String portString = port == DEFAULT_PORT ? "" : (":"+port);
     String portString = port == DEFAULT_PORT ? "" : (":"+port);
     return URI.create("hdfs://"+ namenode.getHostName()+portString);
     return URI.create("hdfs://"+ namenode.getHostName()+portString);
   }
   }
-
+  
   /**
   /**
    * Initialize the server
    * Initialize the server
    * 
    * 

+ 0 - 11
src/hdfs/org/apache/hadoop/hdfs/server/protocol/DatanodeCommand.java

@@ -28,12 +28,6 @@ public abstract class DatanodeCommand implements Writable {
     public void write(DataOutput out) {}
     public void write(DataOutput out) {}
   }
   }
 
 
-  static class BlockReport extends DatanodeCommand {
-    private BlockReport() {super(DatanodeProtocol.DNA_BLOCKREPORT);}
-    public void readFields(DataInput in) {}
-    public void write(DataOutput out) {}
-  }
-
   static class Finalize extends DatanodeCommand {
   static class Finalize extends DatanodeCommand {
     private Finalize() {super(DatanodeProtocol.DNA_FINALIZE);}
     private Finalize() {super(DatanodeProtocol.DNA_FINALIZE);}
     public void readFields(DataInput in) {}
     public void readFields(DataInput in) {}
@@ -45,10 +39,6 @@ public abstract class DatanodeCommand implements Writable {
         new WritableFactory() {
         new WritableFactory() {
           public Writable newInstance() {return new Register();}
           public Writable newInstance() {return new Register();}
         });
         });
-    WritableFactories.setFactory(BlockReport.class,
-        new WritableFactory() {
-          public Writable newInstance() {return new BlockReport();}
-        });
     WritableFactories.setFactory(Finalize.class,
     WritableFactories.setFactory(Finalize.class,
         new WritableFactory() {
         new WritableFactory() {
           public Writable newInstance() {return new Finalize();}
           public Writable newInstance() {return new Finalize();}
@@ -56,7 +46,6 @@ public abstract class DatanodeCommand implements Writable {
   }
   }
 
 
   public static final DatanodeCommand REGISTER = new Register();
   public static final DatanodeCommand REGISTER = new Register();
-  public static final DatanodeCommand BLOCKREPORT = new BlockReport();
   public static final DatanodeCommand FINALIZE = new Finalize();
   public static final DatanodeCommand FINALIZE = new Finalize();
 
 
   private int action;
   private int action;

+ 3 - 4
src/hdfs/org/apache/hadoop/hdfs/server/protocol/DatanodeProtocol.java

@@ -35,9 +35,9 @@ import org.apache.hadoop.ipc.VersionedProtocol;
  **********************************************************************/
  **********************************************************************/
 public interface DatanodeProtocol extends VersionedProtocol {
 public interface DatanodeProtocol extends VersionedProtocol {
   /**
   /**
-   * 16: Block parameter added to nextGenerationStamp().
+   * 17: Remove the request for block report.
    */
    */
-  public static final long versionID = 16L;
+  public static final long versionID = 17L;
   
   
   // error code
   // error code
   final static int NOTIFY = 0;
   final static int NOTIFY = 0;
@@ -54,8 +54,7 @@ public interface DatanodeProtocol extends VersionedProtocol {
   final static int DNA_SHUTDOWN = 3;   // shutdown node
   final static int DNA_SHUTDOWN = 3;   // shutdown node
   final static int DNA_REGISTER = 4;   // re-register
   final static int DNA_REGISTER = 4;   // re-register
   final static int DNA_FINALIZE = 5;   // finalize previous upgrade
   final static int DNA_FINALIZE = 5;   // finalize previous upgrade
-  final static int DNA_BLOCKREPORT = 6;   // request a block report
-  final static int DNA_RECOVERBLOCK = 7;  // request a block recovery
+  final static int DNA_RECOVERBLOCK = 6;  // request a block recovery
 
 
   /** 
   /** 
    * Register Datanode.
    * Register Datanode.

+ 2 - 19
src/test/org/apache/hadoop/hdfs/MiniDFSCluster.java

@@ -23,7 +23,6 @@ import java.net.InetSocketAddress;
 import java.util.ArrayList;
 import java.util.ArrayList;
 import java.util.Collection;
 import java.util.Collection;
 import java.nio.channels.FileChannel;
 import java.nio.channels.FileChannel;
-import java.nio.ByteBuffer;
 import java.util.Random;
 import java.util.Random;
 import java.io.RandomAccessFile;
 import java.io.RandomAccessFile;
 
 
@@ -72,7 +71,6 @@ public class MiniDFSCluster {
                          new ArrayList<DataNodeProperties>();
                          new ArrayList<DataNodeProperties>();
   private File base_dir;
   private File base_dir;
   private File data_dir;
   private File data_dir;
-  private DNSToSwitchMapping dnsToSwitchMapping;
   
   
   
   
   /**
   /**
@@ -400,7 +398,8 @@ public class MiniDFSCluster {
         String name = hosts[i - curDatanodesNum];
         String name = hosts[i - curDatanodesNum];
         System.out.println("Adding node with hostname : " + name + " to rack "+
         System.out.println("Adding node with hostname : " + name + " to rack "+
                             racks[i-curDatanodesNum]);
                             racks[i-curDatanodesNum]);
-        StaticMapping.addNodeToRack(name, racks[i-curDatanodesNum]);
+        StaticMapping.addNodeToRack(name,
+                                    racks[i-curDatanodesNum]);
       }
       }
       Configuration newconf = new Configuration(dnConf); // save config
       Configuration newconf = new Configuration(dnConf); // save config
       if (hosts != null) {
       if (hosts != null) {
@@ -713,22 +712,6 @@ public class MiniDFSCluster {
       } catch (Exception e) {
       } catch (Exception e) {
       }
       }
     }
     }
-    int numResolved = 0;
-    do {
-      numResolved = 0;
-      for (DatanodeInfo info : dnInfos) {
-        if (!info.getNetworkLocation().equals(NetworkTopology.UNRESOLVED)) {
-          numResolved++;
-        } else {
-          try {
-            Thread.sleep(500);
-          } catch (Exception e) {
-          }
-          dnInfos = client.datanodeReport(DatanodeReportType.LIVE);
-          break;
-        }
-      }
-    } while (numResolved != numDataNodes);
 
 
     client.close();
     client.close();
   }
   }

+ 0 - 17
src/test/org/apache/hadoop/hdfs/NNThroughputBenchmark.java

@@ -699,23 +699,6 @@ public class NNThroughputBenchmark implements FSConstants {
         datanodes[idx].sendHeartbeat();
         datanodes[idx].sendHeartbeat();
         prevDNName = datanodes[idx].getName();
         prevDNName = datanodes[idx].getName();
       }
       }
-      int numResolved = 0;
-      DatanodeInfo[] dnInfos = nameNode.getDatanodeReport(DatanodeReportType.ALL);
-      do {
-        numResolved = 0;
-        for (DatanodeInfo info : dnInfos) {
-          if (!info.getNetworkLocation().equals(NetworkTopology.UNRESOLVED)) {
-            numResolved++;
-          } else {
-            try {
-              Thread.sleep(2);
-            } catch (Exception e) {
-            }
-            dnInfos = nameNode.getDatanodeReport(DatanodeReportType.LIVE);
-            break;
-          }
-        }
-      } while (numResolved != nrDatanodes);
 
 
       // create files 
       // create files 
       LOG.info("Creating " + nrFiles + " with " + blocksPerFile + " blocks each.");
       LOG.info("Creating " + nrFiles + " with " + blocksPerFile + " blocks each.");