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 cu...@apache.org on 2006/07/26 14:15:07 UTC

svn commit: r425718 - in /lucene/hadoop/trunk: ./ src/java/org/apache/hadoop/dfs/ src/webapps/datanode/ src/webapps/dfs/

Author: cutting
Date: Wed Jul 26 05:15:06 2006
New Revision: 425718

URL: http://svn.apache.org/viewvc?rev=425718&view=rev
Log:
HADOOP-375.  Permit multiple datanodes to run on a single host.  Contributed by Devaraj.

Modified:
    lucene/hadoop/trunk/CHANGES.txt
    lucene/hadoop/trunk/src/java/org/apache/hadoop/dfs/DataNode.java
    lucene/hadoop/trunk/src/java/org/apache/hadoop/dfs/DatanodeInfo.java
    lucene/hadoop/trunk/src/java/org/apache/hadoop/dfs/DatanodeRegistration.java
    lucene/hadoop/trunk/src/java/org/apache/hadoop/dfs/FSNamesystem.java
    lucene/hadoop/trunk/src/java/org/apache/hadoop/dfs/JspHelper.java
    lucene/hadoop/trunk/src/java/org/apache/hadoop/dfs/StreamFile.java
    lucene/hadoop/trunk/src/webapps/datanode/browseBlock.jsp
    lucene/hadoop/trunk/src/webapps/datanode/browseData.jsp
    lucene/hadoop/trunk/src/webapps/datanode/browseDirectory.jsp
    lucene/hadoop/trunk/src/webapps/datanode/tail.jsp
    lucene/hadoop/trunk/src/webapps/dfs/browseDirectory.jsp
    lucene/hadoop/trunk/src/webapps/dfs/nn_browsedfscontent.jsp

Modified: lucene/hadoop/trunk/CHANGES.txt
URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/CHANGES.txt?rev=425718&r1=425717&r2=425718&view=diff
==============================================================================
--- lucene/hadoop/trunk/CHANGES.txt (original)
+++ lucene/hadoop/trunk/CHANGES.txt Wed Jul 26 05:15:06 2006
@@ -93,6 +93,10 @@
 26. HADOOP-335.  Refactor DFS namespace/transaction logging in
     namenode.   (Konstantin Shvachko via cutting)
 
+27. HADOOP-375.  Fix handling of the datanode HTTP daemon's port so
+    that multiple datanode's can be run on a single host.
+    (Devaraj Das via cutting)
+
 
 Release 0.4.0 - 2006-06-28
 

Modified: lucene/hadoop/trunk/src/java/org/apache/hadoop/dfs/DataNode.java
URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/java/org/apache/hadoop/dfs/DataNode.java?rev=425718&r1=425717&r2=425718&view=diff
==============================================================================
--- lucene/hadoop/trunk/src/java/org/apache/hadoop/dfs/DataNode.java (original)
+++ lucene/hadoop/trunk/src/java/org/apache/hadoop/dfs/DataNode.java Wed Jul 26 05:15:06 2006
@@ -96,12 +96,9 @@
     long blockReportInterval;
     private DataStorage storage = null;
     private StatusHttpServer infoServer;
-    private static int infoPort;
-    private static int port;
-    private static String localMachine;
+    private int infoPort;
     private static InetSocketAddress nameNodeAddr;
     private static DataNode datanodeObject = null;
-    static Date startTime = new Date(System.currentTimeMillis());
     private class DataNodeMetrics {
       private MetricsRecord metricsRecord = null;
       
@@ -158,8 +155,6 @@
         this(InetAddress.getLocalHost().getHostName(), 
              new File(datadir),
              createSocketAddr(conf.get("fs.default.name", "local")), conf);
-        // register datanode
-        register();
         infoPort = conf.getInt("dfs.datanode.info.port", 50075);
         this.infoServer = new StatusHttpServer("datanode", infoPort, true);
         //create a servlet to serve full-file content
@@ -168,6 +163,9 @@
                 "org.apache.hadoop.dfs.StreamFile", null);
         } catch (Exception e) {LOG.warn("addServlet threw exception", e);}
         this.infoServer.start();
+        infoPort = this.infoServer.getPort();
+        // register datanode
+        register();
         datanodeObject = this;
     }
     
@@ -215,9 +213,7 @@
         conf.getLong("dfs.blockreport.intervalMsec", BLOCKREPORT_INTERVAL);
       this.blockReportInterval =
         blockReportIntervalBasis - new Random().nextInt((int)(blockReportIntervalBasis/10));
-      localMachine = machineName;
       this.nameNodeAddr = nameNodeAddr;
-      port = tmpPort;
     }
 
     /** Return the DataNode object
@@ -227,30 +223,10 @@
         return datanodeObject;
     } 
 
-    public String getDataNodeMachine() {
-      return localMachine;
-    }
-
-    public int getDataNodePort() {
-      return port;
-    }
-
-    public int getDataNodeInfoPort() {
-        return infoPort;
-    }
-
     public InetSocketAddress getNameNodeAddr() {
       return nameNodeAddr;
     }
     
-    public InetSocketAddress getDataNodeAddr() {
-        return new InetSocketAddress(localMachine, port);
-    }
-    
-    public Date getStartTime() {
-      return startTime;
-    }
-    
     /**
      * Return the namenode's identifier
      */
@@ -271,6 +247,7 @@
      * @throws IOException
      */
     private void register() throws IOException {
+      dnRegistration.infoPort = infoPort;
       dnRegistration = namenode.register( dnRegistration );
       if( storage.getStorageID().equals("") ) {
         storage.setStorageID( dnRegistration.getStorageID());

Modified: lucene/hadoop/trunk/src/java/org/apache/hadoop/dfs/DatanodeInfo.java
URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/java/org/apache/hadoop/dfs/DatanodeInfo.java?rev=425718&r1=425717&r2=425718&view=diff
==============================================================================
--- lucene/hadoop/trunk/src/java/org/apache/hadoop/dfs/DatanodeInfo.java (original)
+++ lucene/hadoop/trunk/src/java/org/apache/hadoop/dfs/DatanodeInfo.java Wed Jul 26 05:15:06 2006
@@ -36,6 +36,7 @@
   protected long remaining;
   protected long lastUpdate;
   protected int xceiverCount;
+  protected int infoPort; //the port where the infoserver is running
 
   DatanodeInfo() {
     this( new String(), new String() );
@@ -72,6 +73,9 @@
   /** @deprecated Use {@link #getLastUpdate()} instead. */
   public long lastUpdate() { return getLastUpdate(); }
 
+  /** The port at which the http server is running*/
+  public int infoPort() { return infoPort; }
+
   /** A formatted string for reporting the status of the DataNode. */
   public String getDatanodeReport() {
     StringBuffer buffer = new StringBuffer();
@@ -106,6 +110,7 @@
     out.writeLong(remaining);
     out.writeLong(lastUpdate);
     out.writeInt(xceiverCount);
+    out.writeInt(infoPort);
   }
 
   /**
@@ -120,5 +125,6 @@
     this.remaining = in.readLong();
     this.lastUpdate = in.readLong();
     this.xceiverCount = in.readInt();
+    this.infoPort = in.readInt();
   }
 }

Modified: lucene/hadoop/trunk/src/java/org/apache/hadoop/dfs/DatanodeRegistration.java
URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/java/org/apache/hadoop/dfs/DatanodeRegistration.java?rev=425718&r1=425717&r2=425718&view=diff
==============================================================================
--- lucene/hadoop/trunk/src/java/org/apache/hadoop/dfs/DatanodeRegistration.java (original)
+++ lucene/hadoop/trunk/src/java/org/apache/hadoop/dfs/DatanodeRegistration.java Wed Jul 26 05:15:06 2006
@@ -28,6 +28,7 @@
   int version;            /// current Datanode version
   String registrationID;  /// a unique per namenode id; indicates   
                           /// the namenode the datanode is registered with
+  int infoPort;
 
   /**
    * Default constructor.
@@ -70,6 +71,7 @@
     new UTF8( this.name ).write(out);
     new UTF8( this.storageID ).write(out);
     new UTF8( this.registrationID ).write(out);   
+    out.writeInt(this.infoPort);
   }
 
   /**
@@ -83,5 +85,6 @@
     this.storageID = uStr.toString();
     uStr.readFields(in);
     this.registrationID = uStr.toString();   
+    this.infoPort = in.readInt();
   }
 }

Modified: lucene/hadoop/trunk/src/java/org/apache/hadoop/dfs/FSNamesystem.java
URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/java/org/apache/hadoop/dfs/FSNamesystem.java?rev=425718&r1=425717&r2=425718&view=diff
==============================================================================
--- lucene/hadoop/trunk/src/java/org/apache/hadoop/dfs/FSNamesystem.java (original)
+++ lucene/hadoop/trunk/src/java/org/apache/hadoop/dfs/FSNamesystem.java Wed Jul 26 05:15:06 2006
@@ -106,7 +106,6 @@
     StatusHttpServer infoServer;
     int infoPort;
     Date startTime;
-    int dataNodeInfoPort;
     
     //
     Random r = new Random();
@@ -176,7 +175,6 @@
     public FSNamesystem(File dir, Configuration conf) throws IOException {
         fsNamesystemObject = this;
         this.infoPort = conf.getInt("dfs.info.port", 50070);
-        this.dataNodeInfoPort = conf.getInt("dfs.datanode.info.port", 50075);
         this.infoServer = new StatusHttpServer("dfs", infoPort, false);
         this.infoServer.start();
         InetSocketAddress addr = DataNode.createSocketAddr(conf.get("fs.default.name", "local"));
@@ -1085,8 +1083,10 @@
               + "new storageID " + nodeReg.getStorageID() + " assigned." );
         }
         // register new datanode
+        DatanodeDescriptor dinfo;
         datanodeMap.put(nodeReg.getStorageID(), 
-                        new DatanodeDescriptor( nodeReg ) ) ;
+                        (dinfo = new DatanodeDescriptor( nodeReg ) ) ) ;
+        dinfo.infoPort = nodeReg.infoPort;
         NameNode.stateChangeLog.debug(
             "BLOCK* NameSystem.registerDatanode: "
             + "node registered." );
@@ -1153,6 +1153,7 @@
             NameNode.stateChangeLog.debug("BLOCK* NameSystem.gotHeartbeat: "
                     +"brand-new heartbeat from "+nodeID.getName() );
             nodeinfo = new DatanodeDescriptor(nodeID, capacity, remaining, xceiverCount);
+            nodeinfo.infoPort = ((DatanodeRegistration)nodeID).infoPort;
             datanodeMap.put(nodeinfo.getStorageID(), nodeinfo);
             capacityDiff = capacity;
             remainingDiff = remaining;
@@ -1950,7 +1951,7 @@
         index = r.nextInt(size);
         DatanodeInfo d = getDatanodeByIndex(index);
         if (d != null) {
-          return d.getHost();
+          return d.getHost() + ":" + d.infoPort();
         }
       }
       return null;
@@ -1960,7 +1961,4 @@
       return infoPort;
     }
 
-    public int getDataNodeInfoPort() {
-      return dataNodeInfoPort;
-    }
 }

Modified: lucene/hadoop/trunk/src/java/org/apache/hadoop/dfs/JspHelper.java
URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/java/org/apache/hadoop/dfs/JspHelper.java?rev=425718&r1=425717&r2=425718&view=diff
==============================================================================
--- lucene/hadoop/trunk/src/java/org/apache/hadoop/dfs/JspHelper.java (original)
+++ lucene/hadoop/trunk/src/java/org/apache/hadoop/dfs/JspHelper.java Wed Jul 26 05:15:06 2006
@@ -28,10 +28,6 @@
 
 public class JspHelper {
     static InetSocketAddress nameNodeAddr;
-    static InetSocketAddress dataNodeAddr;
-    static String dataNodeLabel;
-    static int dataNodeInfoPort;
-    static DataNode datanode = null;
     static Configuration conf = new Configuration();
 
     static int defaultChunkSizeToView = 
@@ -39,16 +35,11 @@
     static Random rand = new Random();
 
     public JspHelper() {
-      if ((datanode = DataNode.getDataNode()) != null) {
-        dataNodeInfoPort = datanode.getDataNodeInfoPort();
-        nameNodeAddr = datanode.getNameNodeAddr();
-        dataNodeAddr = datanode.getDataNodeAddr();
-        dataNodeLabel = datanode.getDataNodeMachine() + ":" +
-                        datanode.getDataNodePort();
+      if (DataNode.getDataNode() != null) {
+        nameNodeAddr = DataNode.getDataNode().getNameNodeAddr();
       }
       else {
         FSNamesystem fsn = FSNamesystem.getFSNamesystem();
-        dataNodeInfoPort = fsn.getDataNodeInfoPort();
         nameNodeAddr = new InetSocketAddress(fsn.getDFSNameNodeMachine(),
                   fsn.getDFSNameNodePort()); 
       }      
@@ -72,7 +63,7 @@
         chosenNode = nodes[index];
 
         //just ping to check whether the node is alive
-        InetSocketAddress targetAddr = DataNode.createSocketAddr(chosenNode.getHost() + ":" + dataNodeInfoPort);
+        InetSocketAddress targetAddr = DataNode.createSocketAddr(chosenNode.getHost() + ":" + chosenNode.infoPort());
         
         try {
           s = new Socket();
@@ -125,10 +116,18 @@
         amtToRead = blockSize - offsetIntoBlock;
       byte[] buf = new byte[(int)amtToRead];
       int readOffset = 0;
+      int retries = 2;
       while (true) {
-        int numRead = in.read(buf, readOffset, (int)amtToRead);
-        if (numRead == -1)
-          throw new IOException("Could not read data from datanode");
+        int numRead;
+        try {
+          numRead = in.read(buf, readOffset, (int)amtToRead);
+        }
+        catch (IOException e) {
+          retries--;
+          if (retries == 0)
+            throw new IOException("Could not read data from datanode");
+          continue;
+        }
         amtToRead -= numRead;
         readOffset += numRead;
         if (amtToRead == 0)

Modified: lucene/hadoop/trunk/src/java/org/apache/hadoop/dfs/StreamFile.java
URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/java/org/apache/hadoop/dfs/StreamFile.java?rev=425718&r1=425717&r2=425718&view=diff
==============================================================================
--- lucene/hadoop/trunk/src/java/org/apache/hadoop/dfs/StreamFile.java (original)
+++ lucene/hadoop/trunk/src/java/org/apache/hadoop/dfs/StreamFile.java Wed Jul 26 05:15:06 2006
@@ -30,13 +30,11 @@
 public class StreamFile extends HttpServlet {
 
   static InetSocketAddress nameNodeAddr;
-  static int dataNodeInfoPort;
   static DataNode datanode = null;
   static Configuration conf = new Configuration();
   Random rand = new Random();
   static {
     if ((datanode = DataNode.getDataNode()) != null) {
-      dataNodeInfoPort = datanode.getDataNodeInfoPort();
       nameNodeAddr = datanode.getNameNodeAddr();
     }
   }
@@ -64,6 +62,7 @@
     } finally {
       in.close();
       os.close();
+      dfs.close();
     }
   }
 }

Modified: lucene/hadoop/trunk/src/webapps/datanode/browseBlock.jsp
URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/webapps/datanode/browseBlock.jsp?rev=425718&r1=425717&r2=425718&view=diff
==============================================================================
--- lucene/hadoop/trunk/src/webapps/datanode/browseBlock.jsp (original)
+++ lucene/hadoop/trunk/src/webapps/datanode/browseBlock.jsp Wed Jul 26 05:15:06 2006
@@ -17,7 +17,7 @@
   public void generateFileChunks(JspWriter out, HttpServletRequest req) 
     throws IOException {
     long startOffset = 0;
-    
+    int datanodePort = 0; 
     int chunkSizeToView = 0;
 
     String filename = req.getParameter("filename");
@@ -49,12 +49,18 @@
       chunkSizeToView = Integer.parseInt(chunkSizeToViewStr);
     else chunkSizeToView = jspHelper.defaultChunkSizeToView;
 
-    String startOffsetStr;
-    startOffsetStr = req.getParameter("startOffset");
+    String startOffsetStr = req.getParameter("startOffset");
     if (startOffsetStr == null || Long.parseLong(startOffsetStr) < 0)
       startOffset = 0;
     else startOffset = Long.parseLong(startOffsetStr);
-   
+
+    String datanodePortStr = req.getParameter("datanodePort");
+    if (datanodePortStr == null) {
+      out.print("Invalid input (datanodePort absent)");
+      return;
+    }
+    datanodePort = Integer.parseInt(datanodePortStr);
+
     out.print("<h2>File: " + filename + "</h2>");
     out.print("<a href=\"http://" + req.getServerName() + ":" + 
               req.getServerPort() + "/browseData.jsp?filename=" + filename + 
@@ -68,20 +74,24 @@
               startOffset + "\">");
     out.print("<input type=\"hidden\" name=\"filename\" value=\"" + filename +
               "\">");
+    out.print("<input type=\"hidden\" name=\"datanodePort\" value=\"" + 
+              datanodePort+ "\">");
     out.print("<input type=\"text\" name=\"chunkSizeToView\" value=" +
               chunkSizeToView + " size=10 maxlength=10>");
     out.print("&nbsp;&nbsp;<input type=\"submit\" name=\"submit\" value=\"Refresh\"><hr>");
+   out.print("</form>");
 
     //Determine the prev & next blocks
-    DFSClient dfs = null;
+    DFSClient dfs = new DFSClient(jspHelper.nameNodeAddr, jspHelper.conf);
     long nextStartOffset = 0;
     long nextBlockSize = 0;
     String nextBlockIdStr = null;
     String nextHost = req.getServerName();
+    int nextPort = req.getServerPort();
+    int nextDatanodePort = datanodePort;
     //determine data for the next link
     if (startOffset + chunkSizeToView >= blockSize) {
       //we have to go to the next block from this point onwards
-      dfs = new DFSClient(jspHelper.nameNodeAddr, jspHelper.conf);
       LocatedBlock[] blocks = dfs.namenode.open(filename);
       for (int i = 0; i < blocks.length; i++) {
         if (blocks[i].getBlock().getBlockId() == blockId) {
@@ -90,8 +100,14 @@
             nextBlockIdStr = Long.toString(nextBlock.getBlock().getBlockId());
             nextStartOffset = 0;
             nextBlockSize = nextBlock.getBlock().getNumBytes();
-            String host = jspHelper.bestNode(nextBlock).getHost();
-            nextHost = InetAddress.getByName(host).getCanonicalHostName();
+            DatanodeInfo d = jspHelper.bestNode(nextBlock);
+            String datanodeAddr = d.getName();
+            nextDatanodePort = Integer.parseInt(
+                                      datanodeAddr.substring(
+                                           datanodeAddr.indexOf(':') + 1, 
+                                      datanodeAddr.length())); 
+            nextHost = InetAddress.getByName(d.getHost()).getCanonicalHostName();
+            nextPort = d.infoPort(); 
           }
         }
       }
@@ -105,11 +121,12 @@
     String nextUrl = null;
     if (nextBlockIdStr != null) {
       nextUrl = "http://" + nextHost + ":" + 
-                jspHelper.datanode.getDataNodeInfoPort() + 
+                nextPort + 
                 "/browseBlock.jsp?blockId=" + nextBlockIdStr +
                 "&blockSize=" + nextBlockSize + "&startOffset=" + 
                 nextStartOffset + "&filename=" + filename +
-                "&chunkSizeToView=" + chunkSizeToView;
+                "&chunkSizeToView=" + chunkSizeToView + 
+                "&datanodePort=" + nextDatanodePort;
       out.print("<a href=\"" + nextUrl + "\">Next</a>&nbsp;&nbsp;");        
     }
     //determine data for the prev link
@@ -117,9 +134,9 @@
     long prevStartOffset = 0;
     long prevBlockSize = 0;
     String prevHost = req.getServerName();
+    int prevPort = req.getServerPort();
+    int prevDatanodePort = datanodePort;
     if (startOffset == 0) {
-      if (dfs == null) 
-        dfs = new DFSClient(jspHelper.nameNodeAddr, jspHelper.conf);
       LocatedBlock [] blocks = dfs.namenode.open(filename);
       for (int i = 0; i < blocks.length; i++) {
         if (blocks[i].getBlock().getBlockId() == blockId) {
@@ -130,8 +147,14 @@
             if (prevStartOffset < 0)
               prevStartOffset = 0;
             prevBlockSize = prevBlock.getBlock().getNumBytes();
-            String host = jspHelper.bestNode(prevBlock).getHost();
-            prevHost = InetAddress.getByName(host).getCanonicalHostName();
+            DatanodeInfo d = jspHelper.bestNode(prevBlock);
+            String datanodeAddr = d.getName();
+            prevDatanodePort = Integer.parseInt(
+                                      datanodeAddr.substring(
+                                          datanodeAddr.indexOf(':') + 1, 
+                                      datanodeAddr.length())); 
+            prevHost = InetAddress.getByName(d.getHost()).getCanonicalHostName();
+            prevPort = d.infoPort();
           }
         }
       }
@@ -147,24 +170,26 @@
     String prevUrl = null;
     if (prevBlockIdStr != null) {
       prevUrl = "http://" + prevHost + ":" + 
-                jspHelper.datanode.getDataNodeInfoPort() + 
+                prevPort + 
                 "/browseBlock.jsp?blockId=" + prevBlockIdStr + 
                 "&blockSize=" + prevBlockSize + "&startOffset=" + 
                 prevStartOffset + "&filename=" + filename + 
-                "&chunkSizeToView=" + chunkSizeToView;
+                "&chunkSizeToView=" + chunkSizeToView +
+                "&datanodePort=" + prevDatanodePort;
       out.print("<a href=\"" + prevUrl + "\">Prev</a>&nbsp;&nbsp;");
     }
     out.print("<hr>");
-
-    jspHelper.streamBlockInAscii(jspHelper.dataNodeAddr, blockId, 
-                                 blockSize, startOffset, chunkSizeToView, out);
-    //dfs.close();
+    try {
+    jspHelper.streamBlockInAscii(
+            new InetSocketAddress(req.getServerName(), datanodePort), blockId, 
+            blockSize, startOffset, chunkSizeToView, out);
+    } catch (Exception e){
+        out.print(e);
+    }
+    dfs.close();
   }
 
 %>
-
-
-
 <html>
 
 <title>Hadoop DFS File Viewer</title>
@@ -174,7 +199,6 @@
 <% 
    generateFileChunks(out,request);
 %>
-</form>
 <hr>
 
 <h2>Local logs</h2>

Modified: lucene/hadoop/trunk/src/webapps/datanode/browseData.jsp
URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/webapps/datanode/browseData.jsp?rev=425718&r1=425717&r2=425718&view=diff
==============================================================================
--- lucene/hadoop/trunk/src/webapps/datanode/browseData.jsp (original)
+++ lucene/hadoop/trunk/src/webapps/datanode/browseData.jsp Wed Jul 26 05:15:06 2006
@@ -36,10 +36,9 @@
               "\"><i>Go back to dir listing</i></a><br><hr>");
     //Add the various links for looking at the file contents
     //URL for downloading the full file
-    String fqdn = InetAddress.getByName(jspHelper.datanode.getDataNodeMachine()).getCanonicalHostName();
-    String downloadUrl = "http://" + fqdn + 
-                         ":" + jspHelper.datanode.getDataNodeInfoPort() + 
-                         "/streamFile?" + "filename=" + filename;
+    String downloadUrl = "http://" + req.getServerName() + ":" +
+                         + req.getServerPort() + "/streamFile?" + "filename=" +
+                         filename;
     out.print("<a href=\"" + downloadUrl + "\">Download this file</a><br>");
     
     DatanodeInfo chosenNode;
@@ -51,12 +50,13 @@
       chosenNode = jspHelper.bestNode(lastBlk);
     } catch (IOException e) {
       out.print(e.toString());
-      //dfs.close();
+      dfs.close();
       return;
     }
-    fqdn = InetAddress.getByName(chosenNode.getHost()).getCanonicalHostName();
+    String fqdn = 
+           InetAddress.getByName(chosenNode.getHost()).getCanonicalHostName();
     String tailUrl = "http://" + fqdn + ":" +
-                     jspHelper.datanode.getDataNodeInfoPort() + 
+                     chosenNode.infoPort() + 
                      "/tail.jsp?filename=" + filename;
     out.print("<a href=\"" + tailUrl + "\">TAIL this file</a><br>");
 
@@ -68,15 +68,20 @@
       chosenNode = jspHelper.bestNode(firstBlk);
     } catch (IOException e) {
       out.print(e.toString());
-      //dfs.close();
+      dfs.close();
       return;
     }
+    String datanodeAddr = chosenNode.getName();
+    int datanodePort = Integer.parseInt(datanodeAddr.substring(
+                                          datanodeAddr.indexOf(':') + 1, 
+                                      datanodeAddr.length())); 
     fqdn = InetAddress.getByName(chosenNode.getHost()).getCanonicalHostName();
     String chunkViewUrl = "http://" + fqdn + ":" +
-                     jspHelper.datanode.getDataNodeInfoPort() + 
+                     chosenNode.infoPort() + 
                      "/browseBlock.jsp?blockId=" + Long.toString(blockId) +
                      "&tail=false&blockSize=" + blockSize +
-                     "&filename=" + filename;
+                     "&filename=" + filename +
+                     "&datanodePort=" + datanodePort;
     out.print("<a href=\"" + chunkViewUrl + 
               "\">View this file (in a chunked fashion)</a><br>");
     out.print("<hr>"); 
@@ -94,12 +99,17 @@
       DatanodeInfo[] locs = blocks[i].getLocations();
       String locations = new String();
       for (int j = 0; j < locs.length; j++) {
+        datanodeAddr = locs[j].getName();
+        datanodePort = Integer.parseInt(datanodeAddr.substring(
+                                          datanodeAddr.indexOf(':') + 1, 
+                                      datanodeAddr.length())); 
         fqdn = InetAddress.getByName(locs[j].getHost()).getCanonicalHostName();
         String blockUrl = "http://"+ fqdn + ":" +
-                          jspHelper.dataNodeInfoPort +
+                          locs[j].infoPort() +
                           "/browseBlock.jsp?blockId=" + Long.toString(blockId) +
                           "&blockSize=" + blockSize +
-                          "&filename=" + filename;
+                          "&filename=" + filename + 
+                          "&datanodePort=" + datanodePort;
         locations += "<a href=\"" + blockUrl + "\">" + fqdn + "</a>";
         if (j < locs.length - 1)
           locations += ", ";
@@ -108,7 +118,7 @@
       jspHelper.addTableRow(out, cols);
     }
     jspHelper.addTableFooter(out);
-    //dfs.close();
+    dfs.close();
   }
 
 %>

Modified: lucene/hadoop/trunk/src/webapps/datanode/browseDirectory.jsp
URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/webapps/datanode/browseDirectory.jsp?rev=425718&r1=425717&r2=425718&view=diff
==============================================================================
--- lucene/hadoop/trunk/src/webapps/datanode/browseDirectory.jsp (original)
+++ lucene/hadoop/trunk/src/webapps/datanode/browseDirectory.jsp Wed Jul 26 05:15:06 2006
@@ -27,19 +27,7 @@
     String [] headings = new String[5];
     headings[0] = "Name"; headings[1] = "Type"; headings[2] = "Size";
     headings[3] = "Replication"; headings[4] = "BlockSize";
-    if (jspHelper.datanode != null) {
-      out.print("<h2>You are at DataNode " + jspHelper.dataNodeLabel + "</h2><hr>");
-      out.print("This DataNode has been up since "+jspHelper.datanode.getStartTime()+".<br>");
-    }
-    else {
-      out.print("<h2>There are no datanodes in the cluster</h2><hr>");
-    }
     out.print("<h3>Contents of directory " + dir + "</h3><hr>");
-    if (files == null || files.length == 0) {
-      out.print("Empty directory");
-      //dfs.close();
-      return;
-    }
 
     File f = new File(dir);
     String parent;
@@ -47,6 +35,12 @@
       out.print("<a href=\"" + req.getRequestURL() + "?dir=" + parent + 
                 "\">Go to parent directory</a><br>");
 
+    if (files == null || files.length == 0) {
+      out.print("Empty directory");
+      dfs.close();
+      return;
+    }
+
     jspHelper.addTableHeader(out);
     jspHelper.addTableRow(out, headings);
     String cols [] = new String[5];
@@ -68,7 +62,7 @@
         DatanodeInfo chosenNode = jspHelper.bestNode(blocks[0]);
         String fqdn = InetAddress.getByName(chosenNode.getHost()).getCanonicalHostName();
         String datanodeUrl = "http://"+fqdn+":" +
-                             jspHelper.dataNodeInfoPort + 
+                             chosenNode.infoPort() + 
                              "/browseData.jsp?filename=" +
                              files[i].getPath() + "&blockSize=" + 
                              files[i].getBlockSize();
@@ -90,7 +84,7 @@
       }
     }
     jspHelper.addTableFooter(out);
-    //dfs.close();
+    dfs.close();
   }
 
 %>

Modified: lucene/hadoop/trunk/src/webapps/datanode/tail.jsp
URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/webapps/datanode/tail.jsp?rev=425718&r1=425717&r2=425718&view=diff
==============================================================================
--- lucene/hadoop/trunk/src/webapps/datanode/tail.jsp (original)
+++ lucene/hadoop/trunk/src/webapps/datanode/tail.jsp Wed Jul 26 05:15:06 2006
@@ -42,15 +42,13 @@
     out.print("<input type=\"hidden\" name=\"filename\" value=\"" + filename +
               "\">");
 
-    DFSClient dfs = null;
-    InetSocketAddress addr = jspHelper.dataNodeAddr;
     //fetch the block from the datanode that has the last block for this file
-    if (dfs == null) dfs = new DFSClient(jspHelper.nameNodeAddr, 
+    DFSClient dfs = new DFSClient(jspHelper.nameNodeAddr, 
                                          jspHelper.conf);
     LocatedBlock blocks[] = dfs.namenode.open(filename);
     if (blocks == null || blocks.length == 0) {
       out.print("No datanodes contain blocks of file "+filename);
-      //dfs.close();
+      dfs.close();
       return;
     }
     LocatedBlock lastBlk = blocks[blocks.length - 1];
@@ -61,17 +59,17 @@
       chosenNode = jspHelper.bestNode(lastBlk);
     } catch (IOException e) {
       out.print(e.toString());
-      //dfs.close();
+      dfs.close();
       return;
     }      
-    addr = DataNode.createSocketAddr(chosenNode.getName());
+    InetSocketAddress addr = DataNode.createSocketAddr(chosenNode.getName());
     //view the last chunkSizeToView bytes while Tailing
     if (blockSize >= chunkSizeToView)
       startOffset = blockSize - chunkSizeToView;
     else startOffset = 0;
 
     jspHelper.streamBlockInAscii(addr, blockId, blockSize, startOffset, chunkSizeToView, out);
-    //dfs.close();
+    dfs.close();
   }
 
 %>

Modified: lucene/hadoop/trunk/src/webapps/dfs/browseDirectory.jsp
URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/webapps/dfs/browseDirectory.jsp?rev=425718&r1=425717&r2=425718&view=diff
==============================================================================
--- lucene/hadoop/trunk/src/webapps/dfs/browseDirectory.jsp (original)
+++ lucene/hadoop/trunk/src/webapps/dfs/browseDirectory.jsp Wed Jul 26 05:15:06 2006
@@ -27,19 +27,7 @@
     String [] headings = new String[5];
     headings[0] = "Name"; headings[1] = "Type"; headings[2] = "Size";
     headings[3] = "Replication"; headings[4] = "BlockSize";
-    if (jspHelper.datanode != null) {
-      out.print("<h2>You are at DataNode " + jspHelper.dataNodeLabel + "</h2><hr>");
-      out.print("This DataNode has been up since "+jspHelper.datanode.getStartTime()+".<br>");
-    }
-    else {
-      out.print("<h2>There are no datanodes in the cluster</h2><hr>");
-    }
     out.print("<h3>Contents of directory " + dir + "</h3><hr>");
-    if (files == null || files.length == 0) {
-      out.print("Empty directory");
-      //dfs.close();
-      return;
-    }
 
     File f = new File(dir);
     String parent;
@@ -47,6 +35,12 @@
       out.print("<a href=\"" + req.getRequestURL() + "?dir=" + parent + 
                 "\">Go to parent directory</a><br>");
 
+    if (files == null || files.length == 0) {
+      out.print("Empty directory");
+      dfs.close();
+      return;
+    }
+
     jspHelper.addTableHeader(out);
     jspHelper.addTableRow(out, headings);
     String cols [] = new String[5];
@@ -68,7 +62,7 @@
         DatanodeInfo chosenNode = jspHelper.bestNode(blocks[0]);
         String fqdn = InetAddress.getByName(chosenNode.getHost()).getCanonicalHostName();
         String datanodeUrl = "http://"+fqdn+":" +
-                             jspHelper.dataNodeInfoPort + 
+                             chosenNode.infoPort() + 
                              "/browseData.jsp?filename=" +
                              files[i].getPath() + "&blockSize=" + 
                              files[i].getBlockSize();
@@ -90,7 +84,7 @@
       }
     }
     jspHelper.addTableFooter(out);
-    //dfs.close();
+    dfs.close();
   }
 
 %>

Modified: lucene/hadoop/trunk/src/webapps/dfs/nn_browsedfscontent.jsp
URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/webapps/dfs/nn_browsedfscontent.jsp?rev=425718&r1=425717&r2=425718&view=diff
==============================================================================
--- lucene/hadoop/trunk/src/webapps/dfs/nn_browsedfscontent.jsp (original)
+++ lucene/hadoop/trunk/src/webapps/dfs/nn_browsedfscontent.jsp Wed Jul 26 05:15:06 2006
@@ -16,8 +16,8 @@
     String nodeToRedirect;
     int redirectPort;
     if (datanode != null) {
-      nodeToRedirect = datanode;
-      redirectPort = fsn.getDataNodeInfoPort();
+      redirectPort = Integer.parseInt(datanode.substring(datanode.indexOf(':') + 1));
+      nodeToRedirect = datanode.substring(0, datanode.indexOf(':'));
     }
     else {
       nodeToRedirect = fsn.getDFSNameNodeMachine();