瀏覽代碼

HADOOP-825. Fix HDFS daemons when configured with new URI syntax.

git-svn-id: https://svn.apache.org/repos/asf/lucene/hadoop/trunk@488401 13f79535-47bb-0310-9956-ffa450edef68
Doug Cutting 18 年之前
父節點
當前提交
608bfc4a2d

+ 3 - 0
CHANGES.txt

@@ -105,6 +105,9 @@ Trunk (unreleased changes)
     sorts which take longer than the task timeout do not fail.
     (Devaraj Das via cutting)
 
+30. HADOOP-825.  Fix HDFS daemons when configured with new URI syntax.
+    (omalley via cutting)
+
 
 Release 0.9.2 - 2006-12-15
 

+ 4 - 6
src/java/org/apache/hadoop/dfs/DFSck.java

@@ -20,6 +20,7 @@ package org.apache.hadoop.dfs;
 import java.io.IOException;
 import java.io.InputStream;
 import java.io.InputStreamReader;
+import java.net.InetSocketAddress;
 import java.net.URL;
 import java.net.URLConnection;
 import java.net.URLEncoder;
@@ -68,13 +69,10 @@ public class DFSck extends ToolBase {
   }
   
   private String getInfoServer() throws IOException {
-    String fsName = conf.get("fs.default.name", "local");
-    if (fsName.equals("local")) {
-      throw new IOException("This tool only checks DFS, but your config uses 'local' FS.");
-    }
-    String[] splits = fsName.split(":", 2);
+    InetSocketAddress addr = 
+      DataNode.createSocketAddr(conf.get("fs.default.name"));
     int infoPort = conf.getInt("dfs.info.port", 50070);
-    return splits[0]+":"+infoPort;
+    return addr.getHostName() + ":" + infoPort;
   }
   
   /**

+ 20 - 7
src/java/org/apache/hadoop/dfs/DataNode.java

@@ -19,6 +19,7 @@ package org.apache.hadoop.dfs;
 
 import org.apache.commons.logging.*;
 
+import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.ipc.*;
 import org.apache.hadoop.conf.*;
 import org.apache.hadoop.metrics.Metrics;
@@ -75,18 +76,30 @@ public class DataNode implements FSConstants, Runnable {
     //
 
     /**
-     * Util method to build socket addr from string
+     * Util method to build socket addr from either:
+     *   <host>:<post>
+     *   <fs>://<host>:<port>/<path>
      */
-    public static InetSocketAddress createSocketAddr(String s) throws IOException {
-        String target = s;
+    public static InetSocketAddress createSocketAddr(String target
+                                                     ) throws IOException {
         int colonIndex = target.indexOf(':');
         if (colonIndex < 0) {
-            throw new RuntimeException("Not a host:port pair: " + s);
+            throw new RuntimeException("Not a host:port pair: " + target);
+        }
+        String hostname;
+        int port;
+        if (!target.contains("/")) {
+          // must be the old style <host>:<port>
+          hostname = target.substring(0, colonIndex);
+          port = Integer.parseInt(target.substring(colonIndex + 1));
+        } else {
+          // a new uri
+          URI addr = new Path(target).toUri();
+          hostname = addr.getHost();
+          port = addr.getPort();
         }
-        String host = target.substring(0, colonIndex);
-        int port = Integer.parseInt(target.substring(colonIndex + 1));
 
-        return new InetSocketAddress(host, port);
+        return new InetSocketAddress(hostname, port);
     }
 
     DatanodeProtocol namenode;

+ 6 - 4
src/java/org/apache/hadoop/dfs/FSNamesystem.java

@@ -194,9 +194,11 @@ class FSNamesystem implements FSConstants {
      * dirs is a list oif directories where the filesystem directory state 
      * is stored
      */
-    public FSNamesystem(File[] dirs, NameNode nn, Configuration conf) throws IOException {
+    public FSNamesystem(File[] dirs, 
+                        String hostname,
+                        int port,
+                        NameNode nn, Configuration conf) throws IOException {
         fsNamesystemObject = this;
-        InetSocketAddress addr = DataNode.createSocketAddr(conf.get("fs.default.name", "local"));
         this.maxReplication = conf.getInt("dfs.replication.max", 512);
         this.minReplication = conf.getInt("dfs.replication.min", 1);
         if( minReplication <= 0 )
@@ -220,8 +222,8 @@ class FSNamesystem implements FSConstants {
         this.heartbeatExpireInterval = 2 * heartbeatRecheckInterval +
             10 * heartbeatInterval;
 
-        this.localMachine = addr.getHostName();
-        this.port = addr.getPort();
+        this.localMachine = hostname;
+        this.port = port;
         this.dir = new FSDirectory(dirs);
         this.dir.loadFSImage( conf );
         this.safeMode = new SafeModeInfo( conf );

+ 22 - 8
src/java/org/apache/hadoop/dfs/NameNode.java

@@ -26,6 +26,7 @@ import org.apache.hadoop.conf.*;
 import org.apache.hadoop.util.StringUtils;
 
 import java.io.*;
+import java.net.*;
 
 import org.apache.hadoop.metrics.MetricsRecord;
 import org.apache.hadoop.metrics.Metrics;
@@ -129,25 +130,38 @@ public class NameNode implements ClientProtocol, DatanodeProtocol, FSConstants {
       }
     }
     
-    private NameNodeMetrics myMetrics = null;
+    private NameNodeMetrics myMetrics = new NameNodeMetrics();
+    
+    /**
+     * Initialize the server
+     * @param dirs the list of working directories
+     * @param hostname which hostname to bind to
+     * @param port the port number to bind to
+     * @param conf the configuration
+     */
+    private void init(File[] dirs, String hostname, int port, 
+                      Configuration conf) throws IOException {
+      this.namesystem = new FSNamesystem(dirs, hostname, port, this, conf);
+      this.handlerCount = conf.getInt("dfs.namenode.handler.count", 10);
+      this.server = RPC.getServer(this, hostname, port, handlerCount, 
+                                  false, conf);
+      this.server.start();      
+    }
     
     /**
      * Create a NameNode at the default location
      */
     public NameNode(Configuration conf) throws IOException {
-       this(getDirs(conf),DataNode.createSocketAddr(conf.get("fs.default.name", "local")).getHostName(),
-                       DataNode.createSocketAddr(conf.get("fs.default.name", "local")).getPort(), conf);
+      InetSocketAddress addr = 
+        DataNode.createSocketAddr(conf.get("fs.default.name"));
+      init(getDirs(conf), addr.getHostName(), addr.getPort(), conf);
     }
 
     /**
      * Create a NameNode at the specified location and start it.
      */
     public NameNode(File[] dirs, String bindAddress, int port, Configuration conf) throws IOException {
-        this.namesystem = new FSNamesystem(dirs, this, conf);
-        this.handlerCount = conf.getInt("dfs.namenode.handler.count", 10);
-        this.server = RPC.getServer(this, bindAddress, port, handlerCount, false, conf);
-        this.server.start();
-        myMetrics = new NameNodeMetrics();
+       init(dirs, bindAddress, port, conf);
     }
 
     /** Return the configured directories where name data is stored. */