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/19 11:41:23 UTC

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

Author: cutting
Date: Wed Jul 19 02:41:22 2006
New Revision: 423416

URL: http://svn.apache.org/viewvc?rev=423416&view=rev
Log:
HADOOP-347.  Add web-based browsing of DFS content.  Contributed by Devaraj Das.

Added:
    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/
    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
    lucene/hadoop/trunk/build.xml
    lucene/hadoop/trunk/conf/hadoop-default.xml
    lucene/hadoop/trunk/src/java/org/apache/hadoop/dfs/DataNode.java
    lucene/hadoop/trunk/src/java/org/apache/hadoop/dfs/FSNamesystem.java
    lucene/hadoop/trunk/src/java/org/apache/hadoop/mapred/StatusHttpServer.java
    lucene/hadoop/trunk/src/webapps/dfs/index.html

Modified: lucene/hadoop/trunk/CHANGES.txt
URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/CHANGES.txt?rev=423416&r1=423415&r2=423416&view=diff
==============================================================================
--- lucene/hadoop/trunk/CHANGES.txt (original)
+++ lucene/hadoop/trunk/CHANGES.txt Wed Jul 19 02:41:22 2006
@@ -50,6 +50,11 @@
 14. HADOOP-342.  Add a tool for log analysis: Logalyzer.
     (Arun C Murthy via cutting)
 
+15. HADOOP-347.  Add web-based browsing of DFS content.  The namenode
+    redirects browsing requests to datanodes.  Content requests are
+    redirected to datanodes where the data is local when possible.
+    (Devaraj Das via cutting)
+
 
 Release 0.4.0 - 2006-06-28
 

Modified: lucene/hadoop/trunk/build.xml
URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/build.xml?rev=423416&r1=423415&r2=423416&view=diff
==============================================================================
--- lucene/hadoop/trunk/build.xml (original)
+++ lucene/hadoop/trunk/build.xml Wed Jul 19 02:41:22 2006
@@ -91,6 +91,7 @@
     <mkdir dir="${build.webapps}/task/WEB-INF"/>
     <mkdir dir="${build.webapps}/job/WEB-INF"/>
     <mkdir dir="${build.webapps}/dfs/WEB-INF"/>
+    <mkdir dir="${build.webapps}/datanode/WEB-INF"/>
     <mkdir dir="${build.examples}"/>
     <mkdir dir="${build.minimr}"/>
  
@@ -131,6 +132,7 @@
   
   <target name="compile-core" depends="init, record-parser">
 
+    <copy file="${src.webapps}/datanode/browseDirectory.jsp" todir="${src.webapps}/dfs/"/>
     <jsp-compile
      uriroot="${src.webapps}/task"
      outputdir="${build.src}"
@@ -150,6 +152,13 @@
      outputdir="${build.src}"
      package="org.apache.hadoop.dfs"
      webxml="${build.webapps}/dfs/WEB-INF/web.xml">
+    </jsp-compile>
+
+    <jsp-compile
+     uriroot="${src.webapps}/datanode"
+     outputdir="${build.src}"
+     package="org.apache.hadoop.dfs"
+     webxml="${build.webapps}/datanode/WEB-INF/web.xml">
     </jsp-compile>
 
     <javac 

Modified: lucene/hadoop/trunk/conf/hadoop-default.xml
URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/conf/hadoop-default.xml?rev=423416&r1=423415&r2=423416&view=diff
==============================================================================
--- lucene/hadoop/trunk/conf/hadoop-default.xml (original)
+++ lucene/hadoop/trunk/conf/hadoop-default.xml Wed Jul 19 02:41:22 2006
@@ -103,6 +103,13 @@
 </property>
 
 <property>
+  <name>dfs.default.chunk.view.size</name>
+  <value>2048</value>
+  <description>The number of bytes to view for a file on the browser.
+  </description>
+</property>
+
+<property>
   <name>dfs.datanode.du.reserved</name>
   <value>0</value>
   <description>Reserved space in bytes. Always leave this much space free for non dfs use

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=423416&r1=423415&r2=423416&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 19 02:41:22 2006
@@ -21,6 +21,7 @@
 import org.apache.hadoop.conf.*;
 import org.apache.hadoop.util.*;
 import org.apache.hadoop.util.DiskChecker.DiskErrorException;
+import org.apache.hadoop.mapred.StatusHttpServer;
 
 import java.io.*;
 import java.net.*;
@@ -92,7 +93,13 @@
     Daemon dataXceiveServer = null;
     long blockReportInterval;
     private DataStorage storage = null;
-
+    private StatusHttpServer infoServer;
+    private static int infoPort;
+    private static int port;
+    private static String localMachine;
+    private static InetSocketAddress nameNodeAddr;
+    private static DataNode datanodeObject = null;
+    static Date startTime = new Date(System.currentTimeMillis());
     /**
      * Create the DataNode given a configuration and a dataDir.
      * 'dataDir' is where the blocks are stored.
@@ -103,8 +110,17 @@
              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, false);
+        //create a servlet to serve full-file content
+        try {
+          this.infoServer.addServlet(null, "/streamFile/*",
+                "org.apache.hadoop.dfs.StreamFile", null);
+        } catch (Exception e) {LOG.warn("addServlet threw exception", e);}
+        this.infoServer.start();
+        datanodeObject = this;
     }
-
+    
     /**
      * A DataNode can also be created with configuration information
      * explicitly given.
@@ -149,8 +165,42 @@
         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
+     * 
+     */
+    public static DataNode getDataNode() {
+        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
      */
@@ -183,6 +233,10 @@
      * Returns only after shutdown is complete.
      */
     public void shutdown() {
+        try {
+          infoServer.stop();
+        } catch (Exception e) {
+        }
         this.shouldRun = false;
         ((DataXceiveServer) this.dataXceiveServer.getRunnable()).kill();
         try {
@@ -953,4 +1007,5 @@
         Configuration conf = new Configuration();
         runAndWait(conf);
     }
+
 }

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=423416&r1=423415&r2=423416&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 19 02:41:22 2006
@@ -103,7 +103,8 @@
     StatusHttpServer infoServer;
     int infoPort;
     Date startTime;
-        
+    int dataNodeInfoPort;
+    
     //
     Random r = new Random();
 
@@ -169,6 +170,7 @@
     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"));
@@ -1929,5 +1931,36 @@
            return node;
       }
       return null;
+    }
+    /** Stop at and return the detanode at index (used for content browsing)*/
+    private DatanodeInfo getDatanodeByIndex( int index ) {
+      int i = 0;
+      for (Iterator it = datanodeMap.values().iterator(); it.hasNext(); ) {
+        DatanodeInfo node = (DatanodeInfo) it.next();
+        if( i == index )
+           return node;
+        i++;
+      }
+      return null;
+    }
+    
+    public String randomDataNode() {
+      int size = datanodeMap.size();
+      int index = 0;
+      if (size != 0)
+        index = r.nextInt() % size;
+      DatanodeInfo d = getDatanodeByIndex(index);
+      if (d != null) {
+        return d.getHost();
+      }
+      return null;
+    }
+    
+    public int getNameNodeInfoPort() {
+      return infoPort;
+    }
+
+    public int getDataNodeInfoPort() {
+      return dataNodeInfoPort;
     }
 }

Added: 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=423416&view=auto
==============================================================================
--- lucene/hadoop/trunk/src/java/org/apache/hadoop/dfs/JspHelper.java (added)
+++ lucene/hadoop/trunk/src/java/org/apache/hadoop/dfs/JspHelper.java Wed Jul 19 02:41:22 2006
@@ -0,0 +1,157 @@
+/**
+ * Copyright 2005 The Apache Software Foundation
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.dfs;
+
+import javax.servlet.*;
+import javax.servlet.jsp.*;
+import javax.servlet.http.*;
+import java.io.*;
+import java.util.*;
+import java.net.*;
+import org.apache.hadoop.dfs.*;
+import org.apache.hadoop.io.*;
+import org.apache.hadoop.conf.*;
+
+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 = 
+                        conf.getInt("dfs.default.chunk.view.size",2 * 1024);
+    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();
+      }
+      else {
+        FSNamesystem fsn = FSNamesystem.getFSNamesystem();
+        dataNodeInfoPort = fsn.getDataNodeInfoPort();
+        nameNodeAddr = new InetSocketAddress(fsn.getDFSNameNodeMachine(),
+                  fsn.getDFSNameNodePort()); 
+      }      
+    }
+    public DatanodeInfo bestNode(LocatedBlock blk) throws IOException {
+      TreeSet deadNodes = new TreeSet();
+      DatanodeInfo chosenNode = null;
+      int failures = 0;
+      Socket s = null;
+      DatanodeInfo [] nodes = blk.getLocations();
+      if (nodes == null || nodes.length == 0) {
+        throw new IOException("No nodes contain this block");
+      }
+      while (s == null) {
+        if (chosenNode == null) {
+          do {
+            chosenNode = nodes[rand.nextInt(nodes.length)];
+          } while (deadNodes.contains(chosenNode));
+        }
+        int index = rand.nextInt(nodes.length);
+        chosenNode = nodes[index];
+
+        //just ping to check whether the node is alive
+        InetSocketAddress targetAddr = DataNode.createSocketAddr(chosenNode.getHost() + ":" + dataNodeInfoPort);
+        
+        try {
+          s = new Socket();
+          s.connect(targetAddr, FSConstants.READ_TIMEOUT);
+          s.setSoTimeout(FSConstants.READ_TIMEOUT);
+        } catch (IOException e) {
+          deadNodes.add(chosenNode);
+          s.close();
+          s = null;
+          failures++;
+        }
+        if (failures == nodes.length)
+          throw new IOException("Could not reach the block containing the data. Please try again");
+        
+      }
+      s.close();
+      return chosenNode;
+    }
+    public void streamBlockInAscii(InetSocketAddress addr, long blockId, long blockSize, 
+            long offsetIntoBlock, long chunkSizeToView, JspWriter out) 
+      throws IOException {
+      if (chunkSizeToView == 0) return;
+      Socket s = new Socket();
+      s.connect(addr, FSConstants.READ_TIMEOUT);
+      s.setSoTimeout(FSConstants.READ_TIMEOUT);
+      //
+      // Xmit header info to datanode
+      //
+      DataOutputStream os = new DataOutputStream(new BufferedOutputStream(s.getOutputStream()));
+      os.write(FSConstants.OP_READSKIP_BLOCK);
+      new Block(blockId, blockSize).write(os);
+      os.writeLong(offsetIntoBlock);
+      os.flush();
+
+      //
+      // Get bytes in block, set streams
+      //
+      DataInputStream in = new DataInputStream(new BufferedInputStream(s.getInputStream()));
+      long curBlockSize = in.readLong();
+      long amtSkipped = in.readLong();
+      if (curBlockSize != blockSize) {
+        throw new IOException("Recorded block size is " + blockSize + ", but datanode reports size of " + curBlockSize);
+      }
+      if (amtSkipped != offsetIntoBlock) {
+        throw new IOException("Asked for offset of " + offsetIntoBlock + ", but only received offset of " + amtSkipped);
+      }
+      
+      long amtToRead = chunkSizeToView;
+      if (amtToRead + offsetIntoBlock > blockSize)
+        amtToRead = blockSize - offsetIntoBlock;
+      byte[] buf = new byte[(int)amtToRead];
+      int readOffset = 0;
+      while (true) {
+        int numRead = in.read(buf, readOffset, (int)amtToRead);
+        if (numRead == -1)
+          throw new IOException("Could not read data from datanode");
+        amtToRead -= numRead;
+        readOffset += numRead;
+        if (amtToRead == 0)
+          break;
+      }
+      s.close();
+      in.close();
+      out.print(new String(buf));
+    }
+    public void addTableHeader(JspWriter out) throws IOException {
+      out.print("<table border=\"1\""+
+                " cellpadding=\"2\" cellspacing=\"2\">");
+      out.print("<tbody>");
+    }
+    public void addTableRow(JspWriter out, String[] columns) throws IOException {
+      out.print("<tr>");
+      for (int i = 0; i < columns.length; i++) {
+        out.print("<td style=\"vertical-align: top;\"><B>"+columns[i]+"</B><br></td>");
+      }
+      out.print("</tr>");
+    }
+    public void addTableFooter(JspWriter out) throws IOException {
+      out.print("</tbody></table>");
+    }
+
+}

Added: 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=423416&view=auto
==============================================================================
--- lucene/hadoop/trunk/src/java/org/apache/hadoop/dfs/StreamFile.java (added)
+++ lucene/hadoop/trunk/src/java/org/apache/hadoop/dfs/StreamFile.java Wed Jul 19 02:41:22 2006
@@ -0,0 +1,69 @@
+/**
+ * Copyright 2005 The Apache Software Foundation
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.dfs;
+
+
+import javax.servlet.*;
+import javax.servlet.http.*;
+import java.io.*;
+import java.util.*;
+import java.net.*;
+import org.apache.hadoop.dfs.*;
+import org.apache.hadoop.fs.*;
+import org.apache.hadoop.io.*;
+import org.apache.hadoop.conf.*;
+import java.text.DateFormat;
+
+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();
+    }
+  }
+  public void doGet(HttpServletRequest request, HttpServletResponse response)
+                                     throws ServletException, IOException {
+    String filename = request.getParameter("filename");
+    if (filename == null || filename.length() == 0) {
+      response.setContentType("text/plain");
+      PrintWriter out = response.getWriter();
+      out.print("Invalid input");
+      return;
+    }
+    DFSClient dfs = new DFSClient(nameNodeAddr, conf);
+    FSInputStream in = dfs.open(new UTF8(filename));
+    OutputStream os = response.getOutputStream();
+    response.setHeader("Content-Disposition", "attachment; filename=\"" + 
+                        filename + "\"");
+    response.setContentType("application/octet-stream");
+    byte buf[] = new byte[4096];
+    try {
+      int bytesRead;
+      while ((bytesRead = in.read(buf)) != -1) {
+        os.write(buf, 0, bytesRead);
+      }
+    } finally {
+      in.close();
+      os.close();
+    }
+  }
+}

Modified: lucene/hadoop/trunk/src/java/org/apache/hadoop/mapred/StatusHttpServer.java
URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/java/org/apache/hadoop/mapred/StatusHttpServer.java?rev=423416&r1=423415&r2=423416&view=diff
==============================================================================
--- lucene/hadoop/trunk/src/java/org/apache/hadoop/mapred/StatusHttpServer.java (original)
+++ lucene/hadoop/trunk/src/java/org/apache/hadoop/mapred/StatusHttpServer.java Wed Jul 19 02:41:22 2006
@@ -25,6 +25,7 @@
 import org.mortbay.http.handler.ResourceHandler;
 import org.mortbay.http.SocketListener;
 import org.mortbay.jetty.servlet.WebApplicationContext;
+import org.mortbay.jetty.servlet.ServletHttpContext;
 
 /**
  * Create a Jetty embedded server to answer http requests. The primary goal
@@ -88,6 +89,26 @@
    */
   public void setAttribute(String name, Object value) {
     webAppContext.setAttribute(name,value);
+  }
+
+  /**
+   * Add a servlet in the server
+   * @param name The name of the servlet (can be passed as null)
+   * @param pathSpec The path spec for the servlet
+   * @param classname The class name for the servlet
+   * @param contextPath The context path (can be null, defaults to "/")
+   */
+  public void addServlet(String name, String pathSpec, String classname,
+     String contextPath) 
+ throws ClassNotFoundException, InstantiationException, IllegalAccessException {
+    String tmpContextPath = contextPath;
+    if (tmpContextPath == null) tmpContextPath = "/";
+    ServletHttpContext context = 
+                    (ServletHttpContext)webServer.getContext(tmpContextPath);
+    if (name == null)
+      context.addServlet(pathSpec, classname);
+    else
+      context.addServlet(name, pathSpec, classname);
   }
   
   /**

Added: lucene/hadoop/trunk/src/webapps/datanode/browseBlock.jsp
URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/webapps/datanode/browseBlock.jsp?rev=423416&view=auto
==============================================================================
--- lucene/hadoop/trunk/src/webapps/datanode/browseBlock.jsp (added)
+++ lucene/hadoop/trunk/src/webapps/datanode/browseBlock.jsp Wed Jul 19 02:41:22 2006
@@ -0,0 +1,186 @@
+<%@ page
+  contentType="text/html; charset=UTF-8"
+  import="javax.servlet.*"
+  import="javax.servlet.http.*"
+  import="java.io.*"
+  import="java.util.*"
+  import="java.net.*"
+  import="org.apache.hadoop.dfs.*"
+  import="org.apache.hadoop.io.*"
+  import="org.apache.hadoop.conf.*"
+  import="java.text.DateFormat"
+%>
+
+<%!
+  static JspHelper jspHelper = new JspHelper();
+
+  public void generateFileChunks(JspWriter out, HttpServletRequest req) 
+    throws IOException {
+    long startOffset = 0;
+    
+    int chunkSizeToView = 0;
+
+    String filename = req.getParameter("filename");
+    if (filename == null) {
+      out.print("Invalid input (filename absent)");
+      return;
+    }
+    
+    String blockIdStr = null;
+    long blockId = 0;
+    blockIdStr = req.getParameter("blockId");
+    if (blockIdStr == null) {
+      out.print("Invalid input (blockId absent)");
+      return;
+    }
+    blockId = Long.parseLong(blockIdStr);
+
+    String blockSizeStr;
+    long blockSize = 0;
+    blockSizeStr = req.getParameter("blockSize"); 
+    if (blockSizeStr == null) {
+      out.print("Invalid input (blockSize absent)");
+      return;
+    }
+    blockSize = Long.parseLong(blockSizeStr);
+    
+    String chunkSizeToViewStr = req.getParameter("chunkSizeToView");
+    if (chunkSizeToViewStr != null && Integer.parseInt(chunkSizeToViewStr) > 0)
+      chunkSizeToView = Integer.parseInt(chunkSizeToViewStr);
+    else chunkSizeToView = jspHelper.defaultChunkSizeToView;
+
+    String startOffsetStr;
+    startOffsetStr = req.getParameter("startOffset");
+    if (startOffsetStr == null || Long.parseLong(startOffsetStr) < 0)
+      startOffset = 0;
+    else startOffset = Long.parseLong(startOffsetStr);
+   
+    out.print("<h2>File: " + filename + "</h2>");
+    out.print("<a href=\"http://" + req.getServerName() + ":" + 
+              req.getServerPort() + "/browseData.jsp?filename=" + filename + 
+              "\">Go back to File details</a><br>");
+    out.print("<b>Chunk Size to view (in bytes, upto file's DFS blocksize): </b>");
+    out.print("<input type=\"hidden\" name=\"blockId\" value=\"" + blockId + 
+              "\">");
+    out.print("<input type=\"hidden\" name=\"blockSize\" value=\"" + 
+              blockSize + "\">");
+    out.print("<input type=\"hidden\" name=\"startOffset\" value=\"" + 
+              startOffset + "\">");
+    out.print("<input type=\"hidden\" name=\"filename\" value=\"" + filename +
+              "\">");
+    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>");
+
+    //Determine the prev & next blocks
+    DFSClient dfs = null;
+    long nextStartOffset = 0;
+    long nextBlockSize = 0;
+    String nextBlockIdStr = null;
+    String nextHost = req.getServerName();
+    //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) {
+          if (i != blocks.length - 1) {
+            LocatedBlock nextBlock = blocks[i+1];
+            nextBlockIdStr = Long.toString(nextBlock.getBlock().getBlockId());
+            nextStartOffset = 0;
+            nextBlockSize = nextBlock.getBlock().getNumBytes();
+            String host = jspHelper.bestNode(nextBlock).getHost();
+            nextHost = InetAddress.getByName(host).getCanonicalHostName();
+          }
+        }
+      }
+    } 
+    else {
+      //we are in the same block
+      nextBlockIdStr = blockIdStr;
+      nextStartOffset = startOffset + chunkSizeToView;
+      nextBlockSize = blockSize;
+    }
+    String nextUrl = null;
+    if (nextBlockIdStr != null) {
+      nextUrl = "http://" + nextHost + ":" + 
+                jspHelper.datanode.getDataNodeInfoPort() + 
+                "/browseBlock.jsp?blockId=" + nextBlockIdStr +
+                "&blockSize=" + nextBlockSize + "&startOffset=" + 
+                nextStartOffset + "&filename=" + filename +
+                "&chunkSizeToView=" + chunkSizeToView;
+      out.print("<a href=\"" + nextUrl + "\">Next</a>&nbsp;&nbsp;");        
+    }
+    //determine data for the prev link
+    String prevBlockIdStr = null;
+    long prevStartOffset = 0;
+    long prevBlockSize = 0;
+    String prevHost = req.getServerName();
+    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) {
+          if (i != 0) {
+            LocatedBlock prevBlock = blocks[i-1];
+            prevBlockIdStr = Long.toString(prevBlock.getBlock().getBlockId());
+            prevStartOffset = prevBlock.getBlock().getNumBytes() - chunkSizeToView;
+            if (prevStartOffset < 0)
+              prevStartOffset = 0;
+            prevBlockSize = prevBlock.getBlock().getNumBytes();
+            String host = jspHelper.bestNode(prevBlock).getHost();
+            prevHost = InetAddress.getByName(host).getCanonicalHostName();
+          }
+        }
+      }
+    }
+    else {
+      //we are in the same block
+      prevBlockIdStr = blockIdStr;
+      prevStartOffset = startOffset - chunkSizeToView;
+      if (prevStartOffset < 0) prevStartOffset = 0;
+      prevBlockSize = blockSize;
+    }
+
+    String prevUrl = null;
+    if (prevBlockIdStr != null) {
+      prevUrl = "http://" + prevHost + ":" + 
+                jspHelper.datanode.getDataNodeInfoPort() + 
+                "/browseBlock.jsp?blockId=" + prevBlockIdStr + 
+                "&blockSize=" + prevBlockSize + "&startOffset=" + 
+                prevStartOffset + "&filename=" + filename + 
+                "&chunkSizeToView=" + chunkSizeToView;
+      out.print("<a href=\"" + prevUrl + "\">Prev</a>&nbsp;&nbsp;");
+    }
+    out.print("<hr>");
+
+    jspHelper.streamBlockInAscii(jspHelper.dataNodeAddr, blockId, 
+                                 blockSize, startOffset, chunkSizeToView, out);
+    //dfs.close();
+  }
+
+%>
+
+
+
+<html>
+
+<title>Hadoop DFS File Viewer</title>
+
+<body>
+<form action="/browseBlock.jsp" method=GET>
+<% 
+   generateFileChunks(out,request);
+%>
+</form>
+<hr>
+
+<h2>Local logs</h2>
+<a href="/logs/">Log</a> directory
+
+<hr>
+<a href="http://lucene.apache.org/hadoop">Hadoop</a>, 2006.<br>
+</body>
+</html>

Added: lucene/hadoop/trunk/src/webapps/datanode/browseData.jsp
URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/webapps/datanode/browseData.jsp?rev=423416&view=auto
==============================================================================
--- lucene/hadoop/trunk/src/webapps/datanode/browseData.jsp (added)
+++ lucene/hadoop/trunk/src/webapps/datanode/browseData.jsp Wed Jul 19 02:41:22 2006
@@ -0,0 +1,133 @@
+<%@ page
+  contentType="text/html; charset=UTF-8"
+  import="javax.servlet.*"
+  import="javax.servlet.http.*"
+  import="java.io.*"
+  import="java.util.*"
+  import="java.net.*"
+  import="org.apache.hadoop.dfs.*"
+  import="org.apache.hadoop.io.*"
+  import="org.apache.hadoop.conf.*"
+  import="java.text.DateFormat"
+%>
+<%!
+  static JspHelper jspHelper = new JspHelper();
+
+  public void generateFileDetails(JspWriter out, HttpServletRequest req) 
+    throws IOException {
+
+    String filename = req.getParameter("filename");
+    if (filename == null || filename.length() == 0) {
+      out.print("Invalid input");
+      return;
+    }
+
+    String blockSizeStr = req.getParameter("blockSize"); 
+    long blockSize;
+    if (blockSizeStr != null && blockSizeStr.length() != 0)
+      blockSize = Long.parseLong(blockSizeStr);
+
+    DFSClient dfs = new DFSClient(jspHelper.nameNodeAddr, jspHelper.conf);
+    LocatedBlock[] blocks = dfs.namenode.open(filename);
+    out.print("<h2>Filename: "+filename+"</h2>");
+    out.print("<a href=\"http://" + req.getServerName() + ":" + 
+              req.getServerPort() + 
+              "/browseDirectory.jsp?dir=" + new File(filename).getParent() + 
+              "\"><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;
+    out.print("<a href=\"" + downloadUrl + "\">Download this file</a><br>");
+    
+    DatanodeInfo chosenNode;
+    //URL for TAIL 
+    LocatedBlock lastBlk = blocks[blocks.length - 1];
+    long blockId = lastBlk.getBlock().getBlockId();
+    blockSize = lastBlk.getBlock().getNumBytes();
+    try {
+      chosenNode = jspHelper.bestNode(lastBlk);
+    } catch (IOException e) {
+      out.print(e.toString());
+      //dfs.close();
+      return;
+    }
+    fqdn = InetAddress.getByName(chosenNode.getHost()).getCanonicalHostName();
+    String tailUrl = "http://" + fqdn + ":" +
+                     jspHelper.datanode.getDataNodeInfoPort() + 
+                     "/tail.jsp?filename=" + filename;
+    out.print("<a href=\"" + tailUrl + "\">TAIL this file</a><br>");
+
+    //URL for chunk viewing of a file
+    LocatedBlock firstBlk = blocks[0];
+    blockId = firstBlk.getBlock().getBlockId();
+    blockSize = firstBlk.getBlock().getNumBytes();
+    try {
+      chosenNode = jspHelper.bestNode(firstBlk);
+    } catch (IOException e) {
+      out.print(e.toString());
+      //dfs.close();
+      return;
+    }
+    fqdn = InetAddress.getByName(chosenNode.getHost()).getCanonicalHostName();
+    String chunkViewUrl = "http://" + fqdn + ":" +
+                     jspHelper.datanode.getDataNodeInfoPort() + 
+                     "/browseBlock.jsp?blockId=" + Long.toString(blockId) +
+                     "&tail=false&blockSize=" + blockSize +
+                     "&filename=" + filename;
+    out.print("<a href=\"" + chunkViewUrl + 
+              "\">View this file (in a chunked fashion)</a><br>");
+    out.print("<hr>"); 
+    out.print("<B>Total number of blocks: "+blocks.length+"</B><HR>");
+    //generate a table and dump the info
+    String [] headings = new String[2];
+    headings[0] = "Block ID"; headings[1] = "Datanodes containing this block"; 
+    jspHelper.addTableHeader(out);
+    jspHelper.addTableRow(out, headings);
+    String cols [] = new String[2];
+    for (int i = 0; i < blocks.length; i++) {
+      blockId = blocks[i].getBlock().getBlockId();
+      blockSize = blocks[i].getBlock().getNumBytes();
+      cols[0] = "blk_" + Long.toString(blockId);
+      DatanodeInfo[] locs = blocks[i].getLocations();
+      String locations = new String();
+      for (int j = 0; j < locs.length; j++) {
+        fqdn = InetAddress.getByName(locs[j].getHost()).getCanonicalHostName();
+        String blockUrl = "http://"+ fqdn + ":" +
+                          jspHelper.dataNodeInfoPort +
+                          "/browseBlock.jsp?blockId=" + Long.toString(blockId) +
+                          "&blockSize=" + blockSize +
+                          "&filename=" + filename;
+        locations += "<a href=\"" + blockUrl + "\">" + fqdn + "</a>";
+        if (j < locs.length - 1)
+          locations += ", ";
+      }
+      cols[1] = locations;
+      jspHelper.addTableRow(out, cols);
+    }
+    jspHelper.addTableFooter(out);
+    //dfs.close();
+  }
+
+%>
+
+<html>
+
+<title>Hadoop DFS File Browsing</title>
+
+<body>
+
+<% 
+   generateFileDetails(out,request);
+%>
+<hr>
+
+<h2>Local logs</h2>
+<a href="/logs/">Log</a> directory
+
+<hr>
+<a href="http://lucene.apache.org/hadoop">Hadoop</a>, 2006.<br>
+</body>
+</html>

Added: lucene/hadoop/trunk/src/webapps/datanode/browseDirectory.jsp
URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/webapps/datanode/browseDirectory.jsp?rev=423416&view=auto
==============================================================================
--- lucene/hadoop/trunk/src/webapps/datanode/browseDirectory.jsp (added)
+++ lucene/hadoop/trunk/src/webapps/datanode/browseDirectory.jsp Wed Jul 19 02:41:22 2006
@@ -0,0 +1,115 @@
+<%@ page
+  contentType="text/html; charset=UTF-8"
+  import="javax.servlet.*"
+  import="javax.servlet.http.*"
+  import="java.io.*"
+  import="java.util.*"
+  import="java.net.*"
+  import="org.apache.hadoop.dfs.*"
+  import="org.apache.hadoop.io.*"
+  import="org.apache.hadoop.conf.*"
+  import="java.text.DateFormat"
+%>
+<%!
+  static JspHelper jspHelper = new JspHelper();
+  
+  public void generateDirectoryStructure(JspWriter out, HttpServletRequest req) 
+    throws IOException {
+    String dir = req.getParameter("dir");
+    if (dir == null || dir.length() == 0) {
+      out.print("Invalid input");
+      return;
+    }
+    
+    DFSClient dfs = new DFSClient(jspHelper.nameNodeAddr, jspHelper.conf);
+    DFSFileInfo[] files = dfs.listPaths(new UTF8(dir));
+    //generate a table and dump the info
+    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;
+    if ((parent = f.getParent()) != null)
+      out.print("<a href=\"" + req.getRequestURL() + "?dir=" + parent + 
+                "\">Go to parent directory</a><br>");
+
+    jspHelper.addTableHeader(out);
+    jspHelper.addTableRow(out, headings);
+    String cols [] = new String[5];
+    for (int i = 0; i < files.length; i++) {
+      //Get the location of the first block of the file
+      if (files[i].getPath().endsWith(".crc")) continue;
+      if (!files[i].isDir()) {
+        LocatedBlock[] blocks = dfs.namenode.open(files[i].getPath());
+        DatanodeInfo [] locations = blocks[0].getLocations();
+        if (locations.length == 0) {
+          cols[0] = files[i].getPath();
+          cols[1] = "file";
+          cols[2] = Long.toString(files[i].getLen());
+          cols[3] = Short.toString(files[i].getReplication());
+          cols[4] = Long.toString(files[i].getBlockSize());
+          jspHelper.addTableRow(out, cols);
+          continue;
+        }
+        DatanodeInfo chosenNode = jspHelper.bestNode(blocks[0]);
+        String fqdn = InetAddress.getByName(chosenNode.getHost()).getCanonicalHostName();
+        String datanodeUrl = "http://"+fqdn+":" +
+                             jspHelper.dataNodeInfoPort + 
+                             "/browseData.jsp?filename=" +
+                             files[i].getPath() + "&blockSize=" + 
+                             files[i].getBlockSize();
+        cols[0] = "<a href=\""+datanodeUrl+"\">"+files[i].getPath()+"</a>";
+        cols[1] = "file";
+        cols[2] = Long.toString(files[i].getLen());
+        cols[3] = Short.toString(files[i].getReplication());
+        cols[4] = Long.toString(files[i].getBlockSize());
+        jspHelper.addTableRow(out, cols);
+      }
+      else {
+        String datanodeUrl = req.getRequestURL()+"?dir="+files[i].getPath();
+        cols[0] = "<a href=\""+datanodeUrl+"\">"+files[i].getPath()+"</a>";
+        cols[1] = "dir";
+        cols[2] = "0";
+        cols[3] = Short.toString(files[i].getReplication());
+        cols[4] = Long.toString(files[i].getBlockSize());
+        jspHelper.addTableRow(out, cols);
+      }
+    }
+    jspHelper.addTableFooter(out);
+    //dfs.close();
+  }
+
+%>
+
+<html>
+
+<title>Hadoop DFS Directory Browsing</title>
+
+<body>
+
+<% 
+   generateDirectoryStructure(out,request);
+%>
+<hr>
+
+<h2>Local logs</h2>
+<a href="/logs/">Log</a> directory
+
+<hr>
+<a href="http://lucene.apache.org/hadoop">Hadoop</a>, 2006.<br>
+</body>
+</html>

Added: lucene/hadoop/trunk/src/webapps/datanode/tail.jsp
URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/webapps/datanode/tail.jsp?rev=423416&view=auto
==============================================================================
--- lucene/hadoop/trunk/src/webapps/datanode/tail.jsp (added)
+++ lucene/hadoop/trunk/src/webapps/datanode/tail.jsp Wed Jul 19 02:41:22 2006
@@ -0,0 +1,99 @@
+<%@ page
+  contentType="text/html; charset=UTF-8"
+  import="javax.servlet.*"
+  import="javax.servlet.http.*"
+  import="java.io.*"
+  import="java.util.*"
+  import="java.net.*"
+  import="org.apache.hadoop.dfs.*"
+  import="org.apache.hadoop.io.*"
+  import="org.apache.hadoop.conf.*"
+  import="java.text.DateFormat"
+%>
+
+<%!
+  static JspHelper jspHelper = new JspHelper();
+
+  public void generateFileChunks(JspWriter out, HttpServletRequest req) 
+    throws IOException {
+    long startOffset = 0;
+    
+    int chunkSizeToView = 0;
+
+    String filename = req.getParameter("filename");
+    if (filename == null) {
+      out.print("Invalid input (filename absent)");
+      return;
+    }
+
+    String chunkSizeToViewStr = req.getParameter("chunkSizeToView");
+    if (chunkSizeToViewStr != null && Integer.parseInt(chunkSizeToViewStr) > 0)
+      chunkSizeToView = Integer.parseInt(chunkSizeToViewStr);
+    else chunkSizeToView = jspHelper.defaultChunkSizeToView;
+    
+    out.print("<h2>File: " + filename + "</h2>");
+    out.print("<a href=\"http://" + req.getServerName() + ":" + 
+              req.getServerPort() + "/browseData.jsp?filename=" + filename + 
+              "\">Go back to File details</a><br>");
+    out.print("<b>Chunk Size to view (in bytes, upto file's DFS blocksize): </b>");
+    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("<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, 
+                                         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();
+      return;
+    }
+    LocatedBlock lastBlk = blocks[blocks.length - 1];
+    long blockSize = lastBlk.getBlock().getNumBytes();
+    long blockId = lastBlk.getBlock().getBlockId();
+    DatanodeInfo chosenNode;
+    try {
+      chosenNode = jspHelper.bestNode(lastBlk);
+    } catch (IOException e) {
+      out.print(e.toString());
+      //dfs.close();
+      return;
+    }      
+    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();
+  }
+
+%>
+
+
+
+<html>
+<meta http-equiv="refresh" content=60>
+<title>Hadoop DFS File Viewer</title>
+
+<body>
+<form action="/tail.jsp" method=GET>
+<% 
+   generateFileChunks(out,request);
+%>
+</form>
+<hr>
+
+<h2>Local logs</h2>
+<a href="/logs/">Log</a> directory
+
+<hr>
+<a href="http://lucene.apache.org/hadoop">Hadoop</a>, 2006.<br>
+</body>
+</html>

Added: lucene/hadoop/trunk/src/webapps/dfs/browseDirectory.jsp
URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/webapps/dfs/browseDirectory.jsp?rev=423416&view=auto
==============================================================================
--- lucene/hadoop/trunk/src/webapps/dfs/browseDirectory.jsp (added)
+++ lucene/hadoop/trunk/src/webapps/dfs/browseDirectory.jsp Wed Jul 19 02:41:22 2006
@@ -0,0 +1,115 @@
+<%@ page
+  contentType="text/html; charset=UTF-8"
+  import="javax.servlet.*"
+  import="javax.servlet.http.*"
+  import="java.io.*"
+  import="java.util.*"
+  import="java.net.*"
+  import="org.apache.hadoop.dfs.*"
+  import="org.apache.hadoop.io.*"
+  import="org.apache.hadoop.conf.*"
+  import="java.text.DateFormat"
+%>
+<%!
+  static JspHelper jspHelper = new JspHelper();
+  
+  public void generateDirectoryStructure(JspWriter out, HttpServletRequest req) 
+    throws IOException {
+    String dir = req.getParameter("dir");
+    if (dir == null || dir.length() == 0) {
+      out.print("Invalid input");
+      return;
+    }
+    
+    DFSClient dfs = new DFSClient(jspHelper.nameNodeAddr, jspHelper.conf);
+    DFSFileInfo[] files = dfs.listPaths(new UTF8(dir));
+    //generate a table and dump the info
+    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;
+    if ((parent = f.getParent()) != null)
+      out.print("<a href=\"" + req.getRequestURL() + "?dir=" + parent + 
+                "\">Go to parent directory</a><br>");
+
+    jspHelper.addTableHeader(out);
+    jspHelper.addTableRow(out, headings);
+    String cols [] = new String[5];
+    for (int i = 0; i < files.length; i++) {
+      //Get the location of the first block of the file
+      if (files[i].getPath().endsWith(".crc")) continue;
+      if (!files[i].isDir()) {
+        LocatedBlock[] blocks = dfs.namenode.open(files[i].getPath());
+        DatanodeInfo [] locations = blocks[0].getLocations();
+        if (locations.length == 0) {
+          cols[0] = files[i].getPath();
+          cols[1] = "file";
+          cols[2] = Long.toString(files[i].getLen());
+          cols[3] = Short.toString(files[i].getReplication());
+          cols[4] = Long.toString(files[i].getBlockSize());
+          jspHelper.addTableRow(out, cols);
+          continue;
+        }
+        DatanodeInfo chosenNode = jspHelper.bestNode(blocks[0]);
+        String fqdn = InetAddress.getByName(chosenNode.getHost()).getCanonicalHostName();
+        String datanodeUrl = "http://"+fqdn+":" +
+                             jspHelper.dataNodeInfoPort + 
+                             "/browseData.jsp?filename=" +
+                             files[i].getPath() + "&blockSize=" + 
+                             files[i].getBlockSize();
+        cols[0] = "<a href=\""+datanodeUrl+"\">"+files[i].getPath()+"</a>";
+        cols[1] = "file";
+        cols[2] = Long.toString(files[i].getLen());
+        cols[3] = Short.toString(files[i].getReplication());
+        cols[4] = Long.toString(files[i].getBlockSize());
+        jspHelper.addTableRow(out, cols);
+      }
+      else {
+        String datanodeUrl = req.getRequestURL()+"?dir="+files[i].getPath();
+        cols[0] = "<a href=\""+datanodeUrl+"\">"+files[i].getPath()+"</a>";
+        cols[1] = "dir";
+        cols[2] = "0";
+        cols[3] = Short.toString(files[i].getReplication());
+        cols[4] = Long.toString(files[i].getBlockSize());
+        jspHelper.addTableRow(out, cols);
+      }
+    }
+    jspHelper.addTableFooter(out);
+    //dfs.close();
+  }
+
+%>
+
+<html>
+
+<title>Hadoop DFS Directory Browsing</title>
+
+<body>
+
+<% 
+   generateDirectoryStructure(out,request);
+%>
+<hr>
+
+<h2>Local logs</h2>
+<a href="/logs/">Log</a> directory
+
+<hr>
+<a href="http://lucene.apache.org/hadoop">Hadoop</a>, 2006.<br>
+</body>
+</html>

Modified: lucene/hadoop/trunk/src/webapps/dfs/index.html
URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/webapps/dfs/index.html?rev=423416&r1=423415&r2=423416&view=diff
==============================================================================
--- lucene/hadoop/trunk/src/webapps/dfs/index.html (original)
+++ lucene/hadoop/trunk/src/webapps/dfs/index.html Wed Jul 19 02:41:22 2006
@@ -1,4 +1,3 @@
-<meta HTTP-EQUIV="REFRESH" content="0;url=dfshealth.jsp"/>
 <html>
 
 <head>
@@ -12,6 +11,7 @@
 <ul>
 
 <li><a href="dfshealth.jsp">DFS Health/Status</a></li>
+<li><a href="nn_browsedfscontent.jsp">Browse DFS Content</a></li>
 
 </ul>
 

Added: 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=423416&view=auto
==============================================================================
--- lucene/hadoop/trunk/src/webapps/dfs/nn_browsedfscontent.jsp (added)
+++ lucene/hadoop/trunk/src/webapps/dfs/nn_browsedfscontent.jsp Wed Jul 19 02:41:22 2006
@@ -0,0 +1,49 @@
+<%@ page
+  contentType="text/html; charset=UTF-8"
+  import="javax.servlet.*"
+  import="javax.servlet.http.*"
+  import="java.io.*"
+  import="java.util.*"
+  import="org.apache.hadoop.dfs.*"
+  import="java.text.DateFormat"
+  import="java.net.InetAddress"
+%>
+<%!
+  FSNamesystem fsn = FSNamesystem.getFSNamesystem();
+  public void redirectToRandomDataNode(HttpServletResponse resp) throws IOException {
+    String datanode = fsn.randomDataNode();
+    String redirectLocation;
+    String nodeToRedirect;
+    int redirectPort;
+    if (datanode != null) {
+      nodeToRedirect = datanode;
+      redirectPort = fsn.getDataNodeInfoPort();
+    }
+    else {
+      nodeToRedirect = fsn.getDFSNameNodeMachine();
+      redirectPort = fsn.getNameNodeInfoPort();
+    }
+    String fqdn = InetAddress.getByName(nodeToRedirect).getCanonicalHostName();
+    redirectLocation = "http://" + fqdn + ":" + redirectPort + 
+                       "/browseDirectory.jsp?dir=/";
+    resp.sendRedirect(redirectLocation);
+  }
+%>
+
+<html>
+
+<title></title>
+
+<body>
+<% 
+   redirectToRandomDataNode(response); 
+%>
+<hr>
+
+<h2>Local logs</h2>
+<a href="/logs/">Log</a> directory
+
+<hr>
+<a href="http://lucene.apache.org/hadoop">Hadoop</a>, 2006.<br>
+</body>
+</html>