You are viewing a plain text version of this content. The canonical link for it is here.
Posted to common-commits@hadoop.apache.org by sh...@apache.org on 2008/11/11 03:45:58 UTC

svn commit: r712923 - in /hadoop/core/trunk: ./ src/hdfs/org/apache/hadoop/hdfs/server/namenode/ src/webapps/hdfs/

Author: shv
Date: Mon Nov 10 18:45:58 2008
New Revision: 712923

URL: http://svn.apache.org/viewvc?rev=712923&view=rev
Log:
HADOOP-4618. Move http server from FSNamesystem into NameNode. Contributed by Konstantin Shvachko.

Modified:
    hadoop/core/trunk/CHANGES.txt
    hadoop/core/trunk/src/hdfs/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
    hadoop/core/trunk/src/hdfs/org/apache/hadoop/hdfs/server/namenode/JspHelper.java
    hadoop/core/trunk/src/hdfs/org/apache/hadoop/hdfs/server/namenode/NameNode.java
    hadoop/core/trunk/src/webapps/hdfs/dfshealth.jsp
    hadoop/core/trunk/src/webapps/hdfs/nn_browsedfscontent.jsp

Modified: hadoop/core/trunk/CHANGES.txt
URL: http://svn.apache.org/viewvc/hadoop/core/trunk/CHANGES.txt?rev=712923&r1=712922&r2=712923&view=diff
==============================================================================
--- hadoop/core/trunk/CHANGES.txt (original)
+++ hadoop/core/trunk/CHANGES.txt Mon Nov 10 18:45:58 2008
@@ -7,7 +7,7 @@
     HADOOP-4210. Fix findbugs warnings for equals implementations of mapred ID
     classes. Removed public, static ID::read and ID::forName; made ID an
     abstract class. (Suresh Srinivas via cdouglas)
-    
+
     HADOOP-4253. Fix various warnings generated by findbugs. 
     Following deprecated methods in RawLocalFileSystem are removed:
   	  public String getName()
@@ -15,6 +15,13 @@
   	  public void release(Path p) 
     (Suresh Srinivas via johan)
 
+    HADOOP-4618. Move http server from FSNamesystem into NameNode.
+      FSNamesystem.getNameNodeInfoPort() is removed.
+      FSNamesystem.getDFSNameNodeMachine() and FSNamesystem.getDFSNameNodePort()
+        replaced by FSNamesystem.getDFSNameNodeAddress().
+      NameNode(bindAddress, conf) is removed.
+    (shv)
+
   NEW FEATURES
 
     HADOOP-4575. Add a proxy service for relaying HsftpFileSystem requests.

Modified: hadoop/core/trunk/src/hdfs/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
URL: http://svn.apache.org/viewvc/hadoop/core/trunk/src/hdfs/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java?rev=712923&r1=712922&r2=712923&view=diff
==============================================================================
--- hadoop/core/trunk/src/hdfs/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java (original)
+++ hadoop/core/trunk/src/hdfs/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java Mon Nov 10 18:45:58 2008
@@ -33,11 +33,9 @@
 import org.apache.hadoop.security.UnixUserGroupInformation;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.util.*;
-import org.apache.hadoop.http.HttpServer;
 import org.apache.hadoop.metrics.util.MBeanUtil;
 import org.apache.hadoop.net.CachedDNSToSwitchMapping;
 import org.apache.hadoop.net.DNSToSwitchMapping;
-import org.apache.hadoop.net.NetUtils;
 import org.apache.hadoop.net.NetworkTopology;
 import org.apache.hadoop.net.ScriptBasedMapping;
 import org.apache.hadoop.hdfs.server.namenode.LeaseManager.Lease;
@@ -187,14 +185,6 @@
   private Map<String, Collection<Block>> excessReplicateMap = 
     new TreeMap<String, Collection<Block>>();
 
-  //
-  // For the HTTP browsing interface
-  //
-  HttpServer infoServer;
-  int infoPort;
-  Date startTime;
-    
-  //
   Random r = new Random();
 
   /**
@@ -247,6 +237,8 @@
   private long decommissionRecheckInterval;
   // default block size of a file
   private long defaultBlockSize = 0;
+  // allow appending to hdfs files
+  private boolean supportAppends = true;
 
   /**
    * Last block index used for replication work.
@@ -254,8 +246,8 @@
   private int replIndex = 0;
 
   public static FSNamesystem fsNamesystemObject;
-  private String localMachine;
-  private int port;
+  /** NameNode RPC address */
+  private InetSocketAddress nameNodeAddress = null; // TODO: name-node has this field, it should be removed here
   private SafeModeInfo safeMode;  // safe mode information
   private Host2NodesMap host2DataNodeMap = new Host2NodesMap();
     
@@ -300,11 +292,9 @@
    */
   private void initialize(NameNode nn, Configuration conf) throws IOException {
     this.systemStart = now();
-    this.startTime = new Date(systemStart); 
     setConfigurationParameters(conf);
 
-    this.localMachine = nn.getNameNodeAddress().getHostName();
-    this.port = nn.getNameNodeAddress().getPort();
+    this.nameNodeAddress = nn.getNameNodeAddress();
     this.registerMBean(conf); // register the MBean for the FSNamesystemStutus
     this.dir = new FSDirectory(this, conf);
     StartupOption startOpt = NameNode.getStartupOption(conf);
@@ -343,45 +333,6 @@
     if (dnsToSwitchMapping instanceof CachedDNSToSwitchMapping) {
       dnsToSwitchMapping.resolve(new ArrayList<String>(hostsReader.getHosts()));
     }
-
-    String infoAddr = 
-      NetUtils.getServerAddress(conf, "dfs.info.bindAddress", 
-                                "dfs.info.port", "dfs.http.address");
-    InetSocketAddress infoSocAddr = NetUtils.createSocketAddr(infoAddr);
-    String infoHost = infoSocAddr.getHostName();
-    int tmpInfoPort = infoSocAddr.getPort();
-    this.infoServer = new HttpServer("hdfs", infoHost, tmpInfoPort, 
-        tmpInfoPort == 0, conf);
-    if (conf.getBoolean("dfs.https.enable", false)) {
-      boolean needClientAuth = conf.getBoolean("dfs.https.need.client.auth", false);
-      InetSocketAddress secInfoSocAddr = NetUtils.createSocketAddr(conf.get(
-          "dfs.https.address", infoHost + ":" + 0));
-      Configuration sslConf = new Configuration(false);
-      sslConf.addResource(conf.get("dfs.https.server.keystore.resource",
-          "ssl-server.xml"));
-      this.infoServer.addSslListener(secInfoSocAddr, sslConf, needClientAuth);
-      // assume same ssl port for all datanodes
-      InetSocketAddress datanodeSslPort = NetUtils.createSocketAddr(conf.get(
-          "dfs.datanode.https.address", infoHost + ":" + 50475));
-      this.infoServer.setAttribute("datanode.https.port", datanodeSslPort
-          .getPort());
-    }
-    this.infoServer.setAttribute("name.node", nn);
-    this.infoServer.setAttribute("name.node.address", nn.getNameNodeAddress());
-    this.infoServer.setAttribute("name.system.image", getFSImage());
-    this.infoServer.setAttribute("name.conf", conf);
-    this.infoServer.addInternalServlet("fsck", "/fsck", FsckServlet.class);
-    this.infoServer.addInternalServlet("getimage", "/getimage", GetImageServlet.class);
-    this.infoServer.addInternalServlet("listPaths", "/listPaths/*", ListPathsServlet.class);
-    this.infoServer.addInternalServlet("data", "/data/*", FileDataServlet.class);
-    this.infoServer.addInternalServlet("checksum", "/fileChecksum/*",
-        FileChecksumServlets.RedirectServlet.class);
-    this.infoServer.start();
-
-    // The web-server port can be ephemeral... ensure we have the correct info
-    this.infoPort = this.infoServer.getPort();
-    conf.set("dfs.http.address", infoHost + ":" + infoPort);
-    LOG.info("Web-server up at: " + infoHost + ":" + infoPort);
   }
 
   public static Collection<File> getNamespaceDirs(Configuration conf) {
@@ -475,6 +426,7 @@
     this.blockInvalidateLimit = Math.max(this.blockInvalidateLimit, 
                                          20*(int)(heartbeatInterval/1000));
     this.accessTimePrecision = conf.getLong("dfs.access.time.precision", 0);
+    this.supportAppends = conf.getBoolean("dfs.support.append", true);
   }
 
   /**
@@ -507,12 +459,10 @@
     fsRunning = false;
     try {
       if (pendingReplications != null) pendingReplications.stop();
-      if (infoServer != null) infoServer.stop();
       if (hbthread != null) hbthread.interrupt();
       if (replthread != null) replthread.interrupt();
       if (dnthread != null) dnthread.interrupt();
       if (smmthread != null) smmthread.interrupt();
-    } catch (InterruptedException ie) {
     } finally {
       // using finally to ensure we also wait for lease daemon
       try {
@@ -1156,10 +1106,9 @@
    */
   LocatedBlock appendFile(String src, String holder, String clientMachine
       ) throws IOException {
-
-    if (NameNode.stateChangeLog.isDebugEnabled()) {
-      NameNode.stateChangeLog.debug("DIR* NameSystem.appendFile: file "
-          +src+" for "+holder+" at "+clientMachine);
+    if (supportAppends == false) {
+      throw new IOException("Append to hdfs not supported." +
+                            " Please refer to dfs.support.append configuration parameter.");
     }
     startFileInternal(src, null, holder, clientMachine, false, true, 
                       (short)maxReplication, (long)0);
@@ -3487,20 +3436,19 @@
   public DatanodeInfo getDataNodeInfo(String name) {
     return datanodeMap.get(name);
   }
-  /** 
-   */
-  public String getDFSNameNodeMachine() {
-    return localMachine;
-  }
+
   /**
-   */ 
-  public int getDFSNameNodePort() {
-    return port;
+   * @deprecated use {@link NameNode#getNameNodeAddress()} instead.
+   */
+  @Deprecated
+  public InetSocketAddress getDFSNameNodeAddress() {
+    return nameNodeAddress;
   }
+
   /**
    */
   public Date getStartTime() {
-    return startTime;
+    return new Date(systemStart); 
   }
     
   short getMaxReplication()     { return (short)maxReplication; }
@@ -3823,10 +3771,6 @@
   public DatanodeDescriptor getRandomDatanode() {
     return replicator.chooseTarget(1, null, null, 0)[0];
   }
-    
-  public int getNameNodeInfoPort() {
-    return infoPort;
-  }
 
   /**
    * SafeModeInfo contains information related to the safe mode.

Modified: hadoop/core/trunk/src/hdfs/org/apache/hadoop/hdfs/server/namenode/JspHelper.java
URL: http://svn.apache.org/viewvc/hadoop/core/trunk/src/hdfs/org/apache/hadoop/hdfs/server/namenode/JspHelper.java?rev=712923&r1=712922&r2=712923&view=diff
==============================================================================
--- hadoop/core/trunk/src/hdfs/org/apache/hadoop/hdfs/server/namenode/JspHelper.java (original)
+++ hadoop/core/trunk/src/hdfs/org/apache/hadoop/hdfs/server/namenode/JspHelper.java Mon Nov 10 18:45:58 2008
@@ -65,8 +65,7 @@
     }
     else {
       fsn = FSNamesystem.getFSNamesystem();
-      nameNodeAddr = new InetSocketAddress(fsn.getDFSNameNodeMachine(),
-                                           fsn.getDFSNameNodePort()); 
+      nameNodeAddr = fsn.getDFSNameNodeAddress(); 
     }      
 
     UnixUserGroupInformation.saveToConf(conf,

Modified: hadoop/core/trunk/src/hdfs/org/apache/hadoop/hdfs/server/namenode/NameNode.java
URL: http://svn.apache.org/viewvc/hadoop/core/trunk/src/hdfs/org/apache/hadoop/hdfs/server/namenode/NameNode.java?rev=712923&r1=712922&r2=712923&view=diff
==============================================================================
--- hadoop/core/trunk/src/hdfs/org/apache/hadoop/hdfs/server/namenode/NameNode.java (original)
+++ hadoop/core/trunk/src/hdfs/org/apache/hadoop/hdfs/server/namenode/NameNode.java Mon Nov 10 18:45:58 2008
@@ -38,6 +38,7 @@
 import org.apache.hadoop.hdfs.server.protocol.NamenodeProtocol;
 import org.apache.hadoop.hdfs.server.protocol.NamespaceInfo;
 import org.apache.hadoop.hdfs.server.protocol.UpgradeCommand;
+import org.apache.hadoop.http.HttpServer;
 import org.apache.hadoop.ipc.*;
 import org.apache.hadoop.conf.*;
 import org.apache.hadoop.util.StringUtils;
@@ -67,8 +68,8 @@
  * 'NameNode' refers to both this class as well as the 'NameNode server'.
  * The 'FSNamesystem' class actually performs most of the filesystem
  * management.  The majority of the 'NameNode' class itself is concerned
- * with exposing the IPC interface to the outside world, plus some
- * configuration management.
+ * with exposing the IPC interface and the http server to the outside world,
+ * plus some configuration management.
  *
  * NameNode implements the ClientProtocol interface, which allows
  * clients to ask for DFS services.  ClientProtocol is not
@@ -103,14 +104,16 @@
 
   public static final Log LOG = LogFactory.getLog(NameNode.class.getName());
   public static final Log stateChangeLog = LogFactory.getLog("org.apache.hadoop.hdfs.StateChange");
-  public FSNamesystem namesystem;
+  public FSNamesystem namesystem; // TODO: This should private. Use getNamesystem() instead. 
+  /** RPC server */
   private Server server;
+  /** RPC server address */
+  private InetSocketAddress serverAddress = null;
+  /** httpServer */
+  private HttpServer httpServer;
+  /** HTTP server address */
+  private InetSocketAddress httpAddress = null;
   private Thread emptier;
-  private int handlerCount = 2;
-  private boolean supportAppends = true; // allow appending to hdfs files
-    
-  private InetSocketAddress nameNodeAddress = null;
-    
   /** only used for testing purposes  */
   private boolean stopRequested = false;
 
@@ -122,6 +125,10 @@
 
   static NameNodeMetrics myMetrics;
 
+  public FSNamesystem getNamesystem() {
+    return namesystem;
+  }
+
   public static NameNodeMetrics getNameNodeMetrics() {
     return myMetrics;
   }
@@ -139,45 +146,90 @@
     String portString = port == DEFAULT_PORT ? "" : (":"+port);
     return URI.create("hdfs://"+ namenode.getHostName()+portString);
   }
-  
+
   /**
-   * Initialize the server
+   * Initialize name-node.
    * 
-   * @param address hostname:port to bind to
    * @param conf the configuration
    */
-  private void initialize(String address, Configuration conf) throws IOException {
-    InetSocketAddress socAddr = NameNode.getAddress(address);
-    this.supportAppends = conf.getBoolean("dfs.support.append", true);
-    this.handlerCount = conf.getInt("dfs.namenode.handler.count", 10);
+  private void initialize(Configuration conf) throws IOException {
+    InetSocketAddress socAddr = NameNode.getAddress(conf);
+    int handlerCount = conf.getInt("dfs.namenode.handler.count", 10);
+    // create rpc server 
     this.server = RPC.getServer(this, socAddr.getHostName(), socAddr.getPort(),
                                 handlerCount, false, conf);
 
     // The rpc-server port can be ephemeral... ensure we have the correct info
-    this.nameNodeAddress = this.server.getListenerAddress(); 
-    FileSystem.setDefaultUri(conf, getUri(nameNodeAddress));
-    LOG.info("Namenode up at: " + this.nameNodeAddress);
+    this.serverAddress = this.server.getListenerAddress(); 
+    FileSystem.setDefaultUri(conf, getUri(serverAddress));
+    LOG.info("Namenode up at: " + this.serverAddress);
 
     myMetrics = new NameNodeMetrics(conf, this);
 
     this.namesystem = new FSNamesystem(this, conf);
+    startHttpServer(conf);
     this.server.start();  //start RPC server   
+    startTrashEmptier(conf);
+  }
 
+  private void startTrashEmptier(Configuration conf) throws IOException {
     this.emptier = new Thread(new Trash(conf).getEmptier(), "Trash Emptier");
     this.emptier.setDaemon(true);
     this.emptier.start();
   }
-    
+
+  private void startHttpServer(Configuration conf) throws IOException {
+    String infoAddr = 
+      NetUtils.getServerAddress(conf, "dfs.info.bindAddress", 
+                                "dfs.info.port", "dfs.http.address");
+    InetSocketAddress infoSocAddr = NetUtils.createSocketAddr(infoAddr);
+    String infoHost = infoSocAddr.getHostName();
+    int infoPort = infoSocAddr.getPort();
+    this.httpServer = new HttpServer("hdfs", infoHost, infoPort, 
+        infoPort == 0, conf);
+    if (conf.getBoolean("dfs.https.enable", false)) {
+      boolean needClientAuth = conf.getBoolean("dfs.https.need.client.auth", false);
+      InetSocketAddress secInfoSocAddr = NetUtils.createSocketAddr(conf.get(
+          "dfs.https.address", infoHost + ":" + 0));
+      Configuration sslConf = new Configuration(false);
+      sslConf.addResource(conf.get("dfs.https.server.keystore.resource",
+          "ssl-server.xml"));
+      this.httpServer.addSslListener(secInfoSocAddr, sslConf, needClientAuth);
+      // assume same ssl port for all datanodes
+      InetSocketAddress datanodeSslPort = NetUtils.createSocketAddr(conf.get(
+          "dfs.datanode.https.address", infoHost + ":" + 50475));
+      this.httpServer.setAttribute("datanode.https.port", datanodeSslPort
+          .getPort());
+    }
+    this.httpServer.setAttribute("name.node", this);
+    this.httpServer.setAttribute("name.node.address", getNameNodeAddress());
+    this.httpServer.setAttribute("name.system.image", getFSImage());
+    this.httpServer.setAttribute("name.conf", conf);
+    this.httpServer.addInternalServlet("fsck", "/fsck", FsckServlet.class);
+    this.httpServer.addInternalServlet("getimage", "/getimage", GetImageServlet.class);
+    this.httpServer.addInternalServlet("listPaths", "/listPaths/*", ListPathsServlet.class);
+    this.httpServer.addInternalServlet("data", "/data/*", FileDataServlet.class);
+    this.httpServer.addInternalServlet("checksum", "/fileChecksum/*",
+        FileChecksumServlets.RedirectServlet.class);
+    this.httpServer.start();
+
+    // The web-server port can be ephemeral... ensure we have the correct info
+    infoPort = this.httpServer.getPort();
+    this.httpAddress = new InetSocketAddress(infoHost, infoPort);
+    conf.set("dfs.http.address", infoHost + ":" + infoPort);
+    LOG.info("Web-server up at: " + infoHost + ":" + infoPort);
+  }
+
   /**
    * Start NameNode.
    * <p>
    * The name-node can be started with one of the following startup options:
    * <ul> 
-   * <li>{@link org.apache.hadoop.hdfs.server.common.HdfsConstants.StartupOption#REGULAR REGULAR} - normal startup</li>
-   * <li>{@link org.apache.hadoop.hdfs.server.common.HdfsConstants.StartupOption#FORMAT FORMAT} - format name node</li>
-   * <li>{@link org.apache.hadoop.hdfs.server.common.HdfsConstants.StartupOption#UPGRADE UPGRADE} - start the cluster  
+   * <li>{@link StartupOption#REGULAR REGULAR} - normal name node startup</li>
+   * <li>{@link StartupOption#FORMAT FORMAT} - format name node</li>
+   * <li>{@link StartupOption#UPGRADE UPGRADE} - start the cluster  
    * upgrade and create a snapshot of the current file system state</li> 
-   * <li>{@link org.apache.hadoop.hdfs.server.common.HdfsConstants.StartupOption#ROLLBACK ROLLBACK} - roll the  
+   * <li>{@link StartupOption#ROLLBACK ROLLBACK} - roll the  
    *            cluster back to the previous state</li>
    * </ul>
    * The option is passed via configuration field: 
@@ -191,21 +243,8 @@
    * @throws IOException
    */
   public NameNode(Configuration conf) throws IOException {
-    this(FileSystem.getDefaultUri(conf).getAuthority(), conf);
-  }
-
-  /**
-   * Create a NameNode at the specified location and start it.
-   * 
-   * The conf will be modified to reflect the actual ports on which 
-   * the NameNode is up and running if the user passes the port as
-   * <code>zero</code>.  
-   */
-  public NameNode(String bindAddress,
-                  Configuration conf
-                  ) throws IOException {
     try {
-      initialize(bindAddress, conf);
+      initialize(conf);
     } catch (IOException e) {
       this.stop();
       throw e;
@@ -230,6 +269,11 @@
     if (stopRequested)
       return;
     stopRequested = true;
+    try {
+      if (httpServer != null) httpServer.stop();
+    } catch (InterruptedException ie) {
+      LOG.error(StringUtils.stringifyException(ie));
+    }
     if(namesystem != null) namesystem.close();
     if(emptier != null) emptier.interrupt();
     if(server != null) server.stop();
@@ -313,11 +357,6 @@
       stateChangeLog.debug("*DIR* NameNode.append: file "
           +src+" for "+clientName+" at "+clientMachine);
     }
-    if (supportAppends == false) {
-      throw new IOException("Append to hdfs not supported." +
-                            " Please refer to dfs.support.append configuration parameter.");
-    }
-
     LocatedBlock info = namesystem.appendFile(src, clientName, clientMachine);
     myMetrics.numFilesAppended.inc();
     return info;
@@ -725,7 +764,17 @@
    * @return the address on which the NameNodes is listening to.
    */
   public InetSocketAddress getNameNodeAddress() {
-    return nameNodeAddress;
+    return serverAddress;
+  }
+
+  /**
+   * Returns the address of the NameNodes http server, 
+   * which is used to access the name-node web UI.
+   * 
+   * @return the http address.
+   */
+  public InetSocketAddress getHttpAddress() {
+    return httpAddress;
   }
 
   NetworkTopology getNetworkTopology() {
@@ -802,8 +851,7 @@
       StartupOption.IMPORT.getName() + "]");
   }
 
-  private static StartupOption parseArguments(String args[], 
-                                              Configuration conf) {
+  private static StartupOption parseArguments(String args[]) {
     int argsLen = (args == null) ? 0 : args.length;
     StartupOption startOpt = StartupOption.REGULAR;
     for(int i=0; i < argsLen; i++) {
@@ -823,7 +871,6 @@
       } else
         return null;
     }
-    setStartupOption(conf, startOpt);
     return startOpt;
   }
 
@@ -840,11 +887,12 @@
                                  Configuration conf) throws IOException {
     if (conf == null)
       conf = new Configuration();
-    StartupOption startOpt = parseArguments(argv, conf);
+    StartupOption startOpt = parseArguments(argv);
     if (startOpt == null) {
       printUsage();
       return null;
     }
+    setStartupOption(conf, startOpt);
 
     switch (startOpt) {
       case FORMAT:

Modified: hadoop/core/trunk/src/webapps/hdfs/dfshealth.jsp
URL: http://svn.apache.org/viewvc/hadoop/core/trunk/src/webapps/hdfs/dfshealth.jsp?rev=712923&r1=712922&r2=712923&view=diff
==============================================================================
--- hadoop/core/trunk/src/webapps/hdfs/dfshealth.jsp (original)
+++ hadoop/core/trunk/src/webapps/hdfs/dfshealth.jsp Mon Nov 10 18:45:58 2008
@@ -14,8 +14,7 @@
   import="java.lang.Math"
   import="java.net.URLEncoder"
 %>
-<%!FSNamesystem fsn = FSNamesystem.getFSNamesystem();
-  String namenodeLabel = fsn.getDFSNameNodeMachine() + ":" + fsn.getDFSNameNodePort();
+<%!
   JspHelper jspHelper = new JspHelper();
 
   int rowNum = 0;
@@ -49,7 +48,8 @@
   }
       
   public void generateNodeData( JspWriter out, DatanodeDescriptor d,
-                                    String suffix, boolean alive )
+                                    String suffix, boolean alive,
+                                    int nnHttpPort )
     throws IOException {
       
     /* Say the datanode is dn1.hadoop.apache.org with ip 192.168.0.5
@@ -67,7 +67,7 @@
     // from nn_browsedfscontent.jsp:
     String url = "http://" + d.getHostName() + ":" + d.getInfoPort() +
                  "/browseDirectory.jsp?namenodeInfoPort=" +
-                 fsn.getNameNodeInfoPort() + "&dir=" +
+                 nnHttpPort + "&dir=" +
                  URLEncoder.encode("/", "UTF-8");
      
     String name = d.getHostName() + ":" + d.getPort();
@@ -118,8 +118,10 @@
   }
 
   public void generateDFSHealthReport(JspWriter out,
+                                      NameNode nn,
                                       HttpServletRequest request)
                                       throws IOException {
+    FSNamesystem fsn = nn.getNamesystem();
     ArrayList<DatanodeDescriptor> live = new ArrayList<DatanodeDescriptor>();
     ArrayList<DatanodeDescriptor> dead = new ArrayList<DatanodeDescriptor>();
     jspHelper.DFSNodesStatus(live, dead);
@@ -190,6 +192,7 @@
 
         counterReset();
         
+  int nnHttpPort = nn.getHttpAddress().getPort();
 	if ( live.size() > 0 ) {
             
             if ( live.get(0).getCapacity() > 1024 * diskBytes ) {
@@ -216,7 +219,7 @@
             
       jspHelper.sortNodeList(live, sorterField, sorterOrder);
       for ( int i=0; i < live.size(); i++ ) {
-        generateNodeData( out, live.get(i), port_suffix, true );
+        generateNodeData(out, live.get(i), port_suffix, true, nnHttpPort);
       }
     }
     out.print("</table>\n");
@@ -232,7 +235,7 @@
 	    
       jspHelper.sortNodeList(dead, "name", "ASC");
 	    for ( int i=0; i < dead.size() ; i++ ) {
-                generateNodeData( out, dead.get(i), port_suffix, false );
+        generateNodeData(out, dead.get(i), port_suffix, false, nnHttpPort);
 	    }
 	    
 	    out.print("</table>\n");
@@ -241,6 +244,12 @@
     }
   }%>
 
+<%
+  NameNode nn = (NameNode)application.getAttribute("name.node");
+  FSNamesystem fsn = nn.getNamesystem();
+  String namenodeLabel = nn.getNameNodeAddress().getHostName() + ":" + nn.getNameNodeAddress().getPort();
+%>
+
 <html>
 
 <link rel="stylesheet" type="text/css" href="/static/hadoop.css">
@@ -264,8 +273,8 @@
 <h3>Cluster Summary</h3>
 <b> <%= jspHelper.getSafeModeText()%> </b>
 <b> <%= jspHelper.getInodeLimitText()%> </b>
-<% 
-    generateDFSHealthReport(out, request); 
+<%
+    generateDFSHealthReport(out, nn, request); 
 %>
 
 <%

Modified: hadoop/core/trunk/src/webapps/hdfs/nn_browsedfscontent.jsp
URL: http://svn.apache.org/viewvc/hadoop/core/trunk/src/webapps/hdfs/nn_browsedfscontent.jsp?rev=712923&r1=712922&r2=712923&view=diff
==============================================================================
--- hadoop/core/trunk/src/webapps/hdfs/nn_browsedfscontent.jsp (original)
+++ hadoop/core/trunk/src/webapps/hdfs/nn_browsedfscontent.jsp Mon Nov 10 18:45:58 2008
@@ -14,8 +14,10 @@
   import="java.net.URLEncoder"
 %>
 <%!
-  FSNamesystem fsn = FSNamesystem.getFSNamesystem();
-  public void redirectToRandomDataNode(HttpServletResponse resp) throws IOException {
+  public void redirectToRandomDataNode(
+                            NameNode nn, 
+                            HttpServletResponse resp) throws IOException {
+    FSNamesystem fsn = nn.getNamesystem();
     String datanode = fsn.randomDataNode();
     String redirectLocation;
     String nodeToRedirect;
@@ -25,13 +27,13 @@
       nodeToRedirect = datanode.substring(0, datanode.indexOf(':'));
     }
     else {
-      nodeToRedirect = fsn.getDFSNameNodeMachine();
-      redirectPort = fsn.getNameNodeInfoPort();
+      nodeToRedirect = nn.getHttpAddress().getHostName();
+      redirectPort = nn.getHttpAddress().getPort();
     }
     String fqdn = InetAddress.getByName(nodeToRedirect).getCanonicalHostName();
     redirectLocation = "http://" + fqdn + ":" + redirectPort + 
                        "/browseDirectory.jsp?namenodeInfoPort=" + 
-                       fsn.getNameNodeInfoPort() +
+                       nn.getHttpAddress().getPort() +
                        "&dir=" + URLEncoder.encode("/", "UTF-8");
     resp.sendRedirect(redirectLocation);
   }
@@ -43,7 +45,8 @@
 
 <body>
 <% 
-   redirectToRandomDataNode(response); 
+  NameNode nn = (NameNode)application.getAttribute("name.node");
+  redirectToRandomDataNode(nn, response); 
 %>
 <hr>