소스 검색

HADOOP-5097. Remove static variable JspHelper.fsn. (szetszwo)

git-svn-id: https://svn.apache.org/repos/asf/hadoop/core/trunk@740064 13f79535-47bb-0310-9956-ffa450edef68
Tsz-wo Sze 16 년 전
부모
커밋
c2ca26f6d8

+ 3 - 0
CHANGES.txt

@@ -56,6 +56,9 @@ Trunk (unreleased changes)
     or the main Java task in Hadoop's case, kills the entire subtree of
     or the main Java task in Hadoop's case, kills the entire subtree of
     processes. (Ravi Gummadi via ddas)
     processes. (Ravi Gummadi via ddas)
 
 
+    HADOOP-5097. Remove static variable JspHelper.fsn, a static reference to
+    a non-singleton FSNamesystem object.  (szetszwo)
+
   OPTIMIZATIONS
   OPTIMIZATIONS
 
 
   BUG FIXES
   BUG FIXES

+ 0 - 14
src/hdfs/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java

@@ -61,7 +61,6 @@ import java.io.IOException;
 import java.io.PrintWriter;
 import java.io.PrintWriter;
 import java.io.DataOutputStream;
 import java.io.DataOutputStream;
 import java.net.InetAddress;
 import java.net.InetAddress;
-import java.net.InetSocketAddress;
 import java.util.*;
 import java.util.*;
 import java.util.Map.Entry;
 import java.util.Map.Entry;
 
 
@@ -244,8 +243,6 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean {
   private int replIndex = 0;
   private int replIndex = 0;
 
 
   private static FSNamesystem fsNamesystemObject;
   private static FSNamesystem fsNamesystemObject;
-  /** 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 SafeModeInfo safeMode;  // safe mode information
   private Host2NodesMap host2DataNodeMap = new Host2NodesMap();
   private Host2NodesMap host2DataNodeMap = new Host2NodesMap();
     
     
@@ -292,7 +289,6 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean {
     this.systemStart = now();
     this.systemStart = now();
     setConfigurationParameters(conf);
     setConfigurationParameters(conf);
 
 
-    this.nameNodeAddress = nn.getNameNodeAddress();
     this.registerMBean(conf); // register the MBean for the FSNamesystemStutus
     this.registerMBean(conf); // register the MBean for the FSNamesystemStutus
     this.dir = new FSDirectory(this, conf);
     this.dir = new FSDirectory(this, conf);
     StartupOption startOpt = NameNode.getStartupOption(conf);
     StartupOption startOpt = NameNode.getStartupOption(conf);
@@ -3457,16 +3453,6 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean {
     return datanodeMap.get(name);
     return datanodeMap.get(name);
   }
   }
 
 
-  /**
-   * @deprecated use {@link NameNode#getNameNodeAddress()} instead.
-   */
-  @Deprecated
-  public InetSocketAddress getDFSNameNodeAddress() {
-    return nameNodeAddress;
-  }
-
-  /**
-   */
   public Date getStartTime() {
   public Date getStartTime() {
     return new Date(systemStart); 
     return new Date(systemStart); 
   }
   }

+ 6 - 10
src/hdfs/org/apache/hadoop/hdfs/server/namenode/FileDataServlet.java

@@ -34,6 +34,8 @@ import org.apache.hadoop.security.UnixUserGroupInformation;
  * @see org.apache.hadoop.hdfs.HftpFileSystem
  * @see org.apache.hadoop.hdfs.HftpFileSystem
  */
  */
 public class FileDataServlet extends DfsServlet {
 public class FileDataServlet extends DfsServlet {
+  /** For java.io.Serializable */
+  private static final long serialVersionUID = 1L;
 
 
   /** Create a redirection URI */
   /** Create a redirection URI */
   protected URI createUri(FileStatus i, UnixUserGroupInformation ugi,
   protected URI createUri(FileStatus i, UnixUserGroupInformation ugi,
@@ -54,26 +56,20 @@ public class FileDataServlet extends DfsServlet {
         "/streamFile", "filename=" + i.getPath() + "&ugi=" + ugi, null);
         "/streamFile", "filename=" + i.getPath() + "&ugi=" + ugi, null);
   }
   }
 
 
-  private static JspHelper jspHelper = null;
-
   /** Select a datanode to service this request.
   /** Select a datanode to service this request.
    * Currently, this looks at no more than the first five blocks of a file,
    * Currently, this looks at no more than the first five blocks of a file,
    * selecting a datanode randomly from the most represented.
    * selecting a datanode randomly from the most represented.
    */
    */
-  private static DatanodeID pickSrcDatanode(FileStatus i,
+  private DatanodeID pickSrcDatanode(FileStatus i,
       ClientProtocol nnproxy) throws IOException {
       ClientProtocol nnproxy) throws IOException {
-    // a race condition can happen by initializing a static member this way.
-    // A proper fix should make JspHelper a singleton. Since it doesn't affect 
-    // correctness, we leave it as is for now.
-    if (jspHelper == null)
-      jspHelper = new JspHelper();
     final LocatedBlocks blks = nnproxy.getBlockLocations(
     final LocatedBlocks blks = nnproxy.getBlockLocations(
         i.getPath().toUri().getPath(), 0, 1);
         i.getPath().toUri().getPath(), 0, 1);
     if (i.getLen() == 0 || blks.getLocatedBlocks().size() <= 0) {
     if (i.getLen() == 0 || blks.getLocatedBlocks().size() <= 0) {
       // pick a random datanode
       // pick a random datanode
-      return jspHelper.randomNode();
+      NameNode nn = (NameNode)getServletContext().getAttribute("name.node");
+      return nn.getNamesystem().getRandomDatanode();
     }
     }
-    return jspHelper.bestNode(blks.get(0));
+    return JspHelper.bestNode(blks.get(0));
   }
   }
 
 
   /**
   /**

+ 37 - 36
src/hdfs/org/apache/hadoop/hdfs/server/namenode/JspHelper.java

@@ -32,51 +32,39 @@ import javax.servlet.http.HttpServletRequest;
 import javax.servlet.jsp.JspWriter;
 import javax.servlet.jsp.JspWriter;
 
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hdfs.DFSClient;
 import org.apache.hadoop.hdfs.DFSClient;
-import org.apache.hadoop.hdfs.protocol.DatanodeID;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.hdfs.protocol.FSConstants.UpgradeAction;
 import org.apache.hadoop.hdfs.protocol.FSConstants.UpgradeAction;
 import org.apache.hadoop.hdfs.server.common.HdfsConstants;
 import org.apache.hadoop.hdfs.server.common.HdfsConstants;
 import org.apache.hadoop.hdfs.server.common.UpgradeStatusReport;
 import org.apache.hadoop.hdfs.server.common.UpgradeStatusReport;
-import org.apache.hadoop.hdfs.server.datanode.DataNode;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.util.StringUtils;
 import org.apache.hadoop.net.NetUtils;
 import org.apache.hadoop.net.NetUtils;
-import org.apache.hadoop.security.*;
+import org.apache.hadoop.security.UnixUserGroupInformation;
+import org.apache.hadoop.util.StringUtils;
+import org.apache.hadoop.util.VersionInfo;
 
 
 public class JspHelper {
 public class JspHelper {
   final static public String WEB_UGI_PROPERTY_NAME = "dfs.web.ugi";
   final static public String WEB_UGI_PROPERTY_NAME = "dfs.web.ugi";
 
 
-  static FSNamesystem fsn = null;
-  public static InetSocketAddress nameNodeAddr;
   public static final Configuration conf = new Configuration();
   public static final Configuration conf = new Configuration();
   public static final UnixUserGroupInformation webUGI
   public static final UnixUserGroupInformation webUGI
   = UnixUserGroupInformation.createImmutable(
   = UnixUserGroupInformation.createImmutable(
       conf.getStrings(WEB_UGI_PROPERTY_NAME));
       conf.getStrings(WEB_UGI_PROPERTY_NAME));
 
 
-  public static final int defaultChunkSizeToView = 
+  private static final int defaultChunkSizeToView = 
     conf.getInt("dfs.default.chunk.view.size", 32 * 1024);
     conf.getInt("dfs.default.chunk.view.size", 32 * 1024);
-  static Random rand = new Random();
-
-  public JspHelper() {
-    if (DataNode.getDataNode() != null) {
-      nameNodeAddr = DataNode.getDataNode().getNameNodeAddr();
-    }
-    else {
-      fsn = FSNamesystem.getFSNamesystem();
-      nameNodeAddr = fsn.getDFSNameNodeAddress(); 
-    }      
+  static final Random rand = new Random();
 
 
+  static {
     UnixUserGroupInformation.saveToConf(conf,
     UnixUserGroupInformation.saveToConf(conf,
         UnixUserGroupInformation.UGI_PROPERTY_NAME, webUGI);
         UnixUserGroupInformation.UGI_PROPERTY_NAME, webUGI);
   }
   }
 
 
-  public DatanodeID randomNode() throws IOException {
-    return fsn.getRandomDatanode();
-  }
+  /** Private constructor for preventing creating JspHelper object. */
+  private JspHelper() {} 
 
 
-  public DatanodeInfo bestNode(LocatedBlock blk) throws IOException {
+  public static DatanodeInfo bestNode(LocatedBlock blk) throws IOException {
     TreeSet<DatanodeInfo> deadNodes = new TreeSet<DatanodeInfo>();
     TreeSet<DatanodeInfo> deadNodes = new TreeSet<DatanodeInfo>();
     DatanodeInfo chosenNode = null;
     DatanodeInfo chosenNode = null;
     int failures = 0;
     int failures = 0;
@@ -115,7 +103,8 @@ public class JspHelper {
     s.close();
     s.close();
     return chosenNode;
     return chosenNode;
   }
   }
-  public void streamBlockInAscii(InetSocketAddress addr, long blockId, 
+
+  public static void streamBlockInAscii(InetSocketAddress addr, long blockId, 
                                  long genStamp, long blockSize, 
                                  long genStamp, long blockSize, 
                                  long offsetIntoBlock, long chunkSizeToView, JspWriter out) 
                                  long offsetIntoBlock, long chunkSizeToView, JspWriter out) 
     throws IOException {
     throws IOException {
@@ -155,24 +144,20 @@ public class JspHelper {
     s.close();
     s.close();
     out.print(new String(buf));
     out.print(new String(buf));
   }
   }
-  public void DFSNodesStatus(ArrayList<DatanodeDescriptor> live,
-                             ArrayList<DatanodeDescriptor> dead) {
-    if (fsn != null)
-      fsn.DFSNodesStatus(live, dead);
-  }
-  public void addTableHeader(JspWriter out) throws IOException {
+
+  public static void addTableHeader(JspWriter out) throws IOException {
     out.print("<table border=\"1\""+
     out.print("<table border=\"1\""+
               " cellpadding=\"2\" cellspacing=\"2\">");
               " cellpadding=\"2\" cellspacing=\"2\">");
     out.print("<tbody>");
     out.print("<tbody>");
   }
   }
-  public void addTableRow(JspWriter out, String[] columns) throws IOException {
+  public static void addTableRow(JspWriter out, String[] columns) throws IOException {
     out.print("<tr>");
     out.print("<tr>");
     for (int i = 0; i < columns.length; i++) {
     for (int i = 0; i < columns.length; i++) {
       out.print("<td style=\"vertical-align: top;\"><B>"+columns[i]+"</B><br></td>");
       out.print("<td style=\"vertical-align: top;\"><B>"+columns[i]+"</B><br></td>");
     }
     }
     out.print("</tr>");
     out.print("</tr>");
   }
   }
-  public void addTableRow(JspWriter out, String[] columns, int row) throws IOException {
+  public static void addTableRow(JspWriter out, String[] columns, int row) throws IOException {
     out.print("<tr>");
     out.print("<tr>");
       
       
     for (int i = 0; i < columns.length; i++) {
     for (int i = 0; i < columns.length; i++) {
@@ -185,17 +170,17 @@ public class JspHelper {
     }
     }
     out.print("</tr>");
     out.print("</tr>");
   }
   }
-  public void addTableFooter(JspWriter out) throws IOException {
+  public static void addTableFooter(JspWriter out) throws IOException {
     out.print("</tbody></table>");
     out.print("</tbody></table>");
   }
   }
 
 
-  public String getSafeModeText() {
+  public static String getSafeModeText(FSNamesystem fsn) {
     if (!fsn.isInSafeMode())
     if (!fsn.isInSafeMode())
       return "";
       return "";
     return "Safe mode is ON. <em>" + fsn.getSafeModeTip() + "</em><br>";
     return "Safe mode is ON. <em>" + fsn.getSafeModeTip() + "</em><br>";
   }
   }
 
 
-  public String getInodeLimitText() {
+  public static String getInodeLimitText(FSNamesystem fsn) {
     long inodes = fsn.dir.totalInodes();
     long inodes = fsn.dir.totalInodes();
     long blocks = fsn.getBlocksTotal();
     long blocks = fsn.getBlocksTotal();
     long maxobjects = fsn.getMaxObjects();
     long maxobjects = fsn.getMaxObjects();
@@ -217,7 +202,7 @@ public class JspHelper {
     return str;
     return str;
   }
   }
 
 
-  public String getUpgradeStatusText() {
+  public static String getUpgradeStatusText(FSNamesystem fsn) {
     String statusText = "";
     String statusText = "";
     try {
     try {
       UpgradeStatusReport status = 
       UpgradeStatusReport status = 
@@ -231,7 +216,7 @@ public class JspHelper {
     return statusText;
     return statusText;
   }
   }
 
 
-  public void sortNodeList(ArrayList<DatanodeDescriptor> nodes,
+  public static void sortNodeList(ArrayList<DatanodeDescriptor> nodes,
                            String field, String order) {
                            String field, String order) {
         
         
     class NodeComapare implements Comparator<DatanodeDescriptor> {
     class NodeComapare implements Comparator<DatanodeDescriptor> {
@@ -370,4 +355,20 @@ public class JspHelper {
       file = "..." + file.substring(start, file.length());
       file = "..." + file.substring(start, file.length());
     out.print("<title>HDFS:" + file + "</title>");
     out.print("<title>HDFS:" + file + "</title>");
   }
   }
+
+  /** Convert a String to chunk-size-to-view. */
+  public static int string2ChunkSizeToView(String s) {
+    int n = s == null? 0: Integer.parseInt(s);
+    return n > 0? n: defaultChunkSizeToView;
+  }
+
+  /** Return a table containing version information. */
+  public static String getVersionTable(FSNamesystem fsn) {
+    return "<div id='dfstable'><table>"       
+        + "\n  <tr><td id='col1'>Started:</td><td>" + fsn.getStartTime() + "</td></tr>\n"
+        + "\n  <tr><td id='col1'>Version:</td><td>" + VersionInfo.getVersion() + ", " + VersionInfo.getRevision()
+        + "\n  <tr><td id='col1'>Compiled:</td><td>" + VersionInfo.getDate() + " by " + VersionInfo.getUser() + " from " + VersionInfo.getBranch()
+        + "\n  <tr><td id='col1'>Upgrades:</td><td>" + getUpgradeStatusText(fsn)
+        + "\n</table></div>";
+  }
 }
 }

+ 10 - 18
src/webapps/datanode/browseBlock.jsp

@@ -17,12 +17,11 @@
 %>
 %>
 
 
 <%!
 <%!
-  static JspHelper jspHelper = new JspHelper();
+  static final DataNode datanode = DataNode.getDataNode();
 
 
   public void generateFileDetails(JspWriter out, HttpServletRequest req) 
   public void generateFileDetails(JspWriter out, HttpServletRequest req) 
     throws IOException {
     throws IOException {
 
 
-    int chunkSizeToView = 0;
     long startOffset = 0;
     long startOffset = 0;
     int datanodePort;
     int datanodePort;
 
 
@@ -47,10 +46,7 @@
     if (namenodeInfoPortStr != null)
     if (namenodeInfoPortStr != null)
       namenodeInfoPort = Integer.parseInt(namenodeInfoPortStr);
       namenodeInfoPort = Integer.parseInt(namenodeInfoPortStr);
 
 
-    String chunkSizeToViewStr = req.getParameter("chunkSizeToView");
-    if (chunkSizeToViewStr != null && Integer.parseInt(chunkSizeToViewStr) > 0)
-      chunkSizeToView = Integer.parseInt(chunkSizeToViewStr);
-    else chunkSizeToView = jspHelper.defaultChunkSizeToView;
+    final int chunkSizeToView = JspHelper.string2ChunkSizeToView(req.getParameter("chunkSizeToView"));
 
 
     String startOffsetStr = req.getParameter("startOffset");
     String startOffsetStr = req.getParameter("startOffset");
     if (startOffsetStr == null || Long.parseLong(startOffsetStr) < 0)
     if (startOffsetStr == null || Long.parseLong(startOffsetStr) < 0)
@@ -71,7 +67,7 @@
     } 
     } 
     blockSize = Long.parseLong(blockSizeStr);
     blockSize = Long.parseLong(blockSizeStr);
 
 
-    DFSClient dfs = new DFSClient(jspHelper.nameNodeAddr, jspHelper.conf);
+    final DFSClient dfs = new DFSClient(datanode.getNameNodeAddr(), JspHelper.conf);
     List<LocatedBlock> blocks = 
     List<LocatedBlock> blocks = 
       dfs.namenode.getBlockLocations(filename, 0, Long.MAX_VALUE).getLocatedBlocks();
       dfs.namenode.getBlockLocations(filename, 0, Long.MAX_VALUE).getLocatedBlocks();
     //Add the various links for looking at the file contents
     //Add the various links for looking at the file contents
@@ -87,7 +83,7 @@
     LocatedBlock lastBlk = blocks.get(blocks.size() - 1);
     LocatedBlock lastBlk = blocks.get(blocks.size() - 1);
     long blockId = lastBlk.getBlock().getBlockId();
     long blockId = lastBlk.getBlock().getBlockId();
     try {
     try {
-      chosenNode = jspHelper.bestNode(lastBlk);
+      chosenNode = JspHelper.bestNode(lastBlk);
     } catch (IOException e) {
     } catch (IOException e) {
       out.print(e.toString());
       out.print(e.toString());
       dfs.close();
       dfs.close();
@@ -157,7 +153,7 @@
     }
     }
     out.println("</table>");
     out.println("</table>");
     out.print("<hr>");
     out.print("<hr>");
-    String namenodeHost = jspHelper.nameNodeAddr.getHostName();
+    String namenodeHost = datanode.getNameNodeAddr().getHostName();
     out.print("<br><a href=\"http://" + 
     out.print("<br><a href=\"http://" + 
               InetAddress.getByName(namenodeHost).getCanonicalHostName() + ":" +
               InetAddress.getByName(namenodeHost).getCanonicalHostName() + ":" +
               namenodeInfoPort + "/dfshealth.jsp\">Go back to DFS home</a>");
               namenodeInfoPort + "/dfshealth.jsp\">Go back to DFS home</a>");
@@ -168,7 +164,6 @@
     throws IOException {
     throws IOException {
     long startOffset = 0;
     long startOffset = 0;
     int datanodePort = 0; 
     int datanodePort = 0; 
-    int chunkSizeToView = 0;
 
 
     String namenodeInfoPortStr = req.getParameter("namenodeInfoPort");
     String namenodeInfoPortStr = req.getParameter("namenodeInfoPort");
     int namenodeInfoPort = -1;
     int namenodeInfoPort = -1;
@@ -208,10 +203,7 @@
     }
     }
     blockSize = Long.parseLong(blockSizeStr);
     blockSize = Long.parseLong(blockSizeStr);
     
     
-    String chunkSizeToViewStr = req.getParameter("chunkSizeToView");
-    if (chunkSizeToViewStr != null && Integer.parseInt(chunkSizeToViewStr) > 0)
-      chunkSizeToView = Integer.parseInt(chunkSizeToViewStr);
-    else chunkSizeToView = jspHelper.defaultChunkSizeToView;
+    final int chunkSizeToView = JspHelper.string2ChunkSizeToView(req.getParameter("chunkSizeToView"));
 
 
     String startOffsetStr = req.getParameter("startOffset");
     String startOffsetStr = req.getParameter("startOffset");
     if (startOffsetStr == null || Long.parseLong(startOffsetStr) < 0)
     if (startOffsetStr == null || Long.parseLong(startOffsetStr) < 0)
@@ -240,7 +232,7 @@
     out.print("<hr>");
     out.print("<hr>");
 
 
     //Determine the prev & next blocks
     //Determine the prev & next blocks
-    DFSClient dfs = new DFSClient(jspHelper.nameNodeAddr, jspHelper.conf);
+    final DFSClient dfs = new DFSClient(datanode.getNameNodeAddr(), JspHelper.conf);
     long nextStartOffset = 0;
     long nextStartOffset = 0;
     long nextBlockSize = 0;
     long nextBlockSize = 0;
     String nextBlockIdStr = null;
     String nextBlockIdStr = null;
@@ -261,7 +253,7 @@
             nextGenStamp = Long.toString(nextBlock.getBlock().getGenerationStamp());
             nextGenStamp = Long.toString(nextBlock.getBlock().getGenerationStamp());
             nextStartOffset = 0;
             nextStartOffset = 0;
             nextBlockSize = nextBlock.getBlock().getNumBytes();
             nextBlockSize = nextBlock.getBlock().getNumBytes();
-            DatanodeInfo d = jspHelper.bestNode(nextBlock);
+            DatanodeInfo d = JspHelper.bestNode(nextBlock);
             String datanodeAddr = d.getName();
             String datanodeAddr = d.getName();
             nextDatanodePort = Integer.parseInt(
             nextDatanodePort = Integer.parseInt(
                                       datanodeAddr.substring(
                                       datanodeAddr.substring(
@@ -315,7 +307,7 @@
             if (prevStartOffset < 0)
             if (prevStartOffset < 0)
               prevStartOffset = 0;
               prevStartOffset = 0;
             prevBlockSize = prevBlock.getBlock().getNumBytes();
             prevBlockSize = prevBlock.getBlock().getNumBytes();
-            DatanodeInfo d = jspHelper.bestNode(prevBlock);
+            DatanodeInfo d = JspHelper.bestNode(prevBlock);
             String datanodeAddr = d.getName();
             String datanodeAddr = d.getName();
             prevDatanodePort = Integer.parseInt(
             prevDatanodePort = Integer.parseInt(
                                       datanodeAddr.substring(
                                       datanodeAddr.substring(
@@ -353,7 +345,7 @@
     out.print("<hr>");
     out.print("<hr>");
     out.print("<textarea cols=\"100\" rows=\"25\" wrap=\"virtual\" style=\"width:100%\" READONLY>");
     out.print("<textarea cols=\"100\" rows=\"25\" wrap=\"virtual\" style=\"width:100%\" READONLY>");
     try {
     try {
-    jspHelper.streamBlockInAscii(
+    JspHelper.streamBlockInAscii(
             new InetSocketAddress(req.getServerName(), datanodePort), blockId, 
             new InetSocketAddress(req.getServerName(), datanodePort), blockId, 
             genStamp, blockSize, startOffset, chunkSizeToView, out);
             genStamp, blockSize, startOffset, chunkSizeToView, out);
     } catch (Exception e){
     } catch (Exception e){

+ 8 - 8
src/webapps/datanode/browseDirectory.jsp

@@ -17,7 +17,7 @@
   import="java.text.DateFormat"
   import="java.text.DateFormat"
 %>
 %>
 <%!
 <%!
-  static JspHelper jspHelper = new JspHelper();
+  static final DataNode datanode = DataNode.getDataNode();
   
   
   public void generateDirectoryStructure( JspWriter out, 
   public void generateDirectoryStructure( JspWriter out, 
                                           HttpServletRequest req,
                                           HttpServletRequest req,
@@ -34,7 +34,7 @@
     if (namenodeInfoPortStr != null)
     if (namenodeInfoPortStr != null)
       namenodeInfoPort = Integer.parseInt(namenodeInfoPortStr);
       namenodeInfoPort = Integer.parseInt(namenodeInfoPortStr);
     
     
-    DFSClient dfs = new DFSClient(jspHelper.nameNodeAddr, jspHelper.conf);
+    final DFSClient dfs = new DFSClient(datanode.getNameNodeAddr(), JspHelper.conf);
     String target = dir;
     String target = dir;
     final FileStatus targetStatus = dfs.getFileInfo(target);
     final FileStatus targetStatus = dfs.getFileInfo(target);
     if (targetStatus == null) { // not exists
     if (targetStatus == null) { // not exists
@@ -55,7 +55,7 @@
         if (locations == null || locations.length == 0) {
         if (locations == null || locations.length == 0) {
           out.print("Empty file");
           out.print("Empty file");
         } else {
         } else {
-          DatanodeInfo chosenNode = jspHelper.bestNode(firstBlock);
+          DatanodeInfo chosenNode = JspHelper.bestNode(firstBlock);
           String fqdn = InetAddress.getByName(chosenNode.getHost()).
           String fqdn = InetAddress.getByName(chosenNode.getHost()).
             getCanonicalHostName();
             getCanonicalHostName();
           String datanodeAddr = chosenNode.getName();
           String datanodeAddr = chosenNode.getName();
@@ -99,9 +99,9 @@
         out.print("Empty directory");
         out.print("Empty directory");
       }
       }
       else {
       else {
-        jspHelper.addTableHeader(out);
+        JspHelper.addTableHeader(out);
         int row=0;
         int row=0;
-        jspHelper.addTableRow(out, headings, row++);
+        JspHelper.addTableRow(out, headings, row++);
         String cols [] = new String[headings.length];
         String cols [] = new String[headings.length];
         for (int i = 0; i < files.length; i++) {
         for (int i = 0; i < files.length; i++) {
           //Get the location of the first block of the file
           //Get the location of the first block of the file
@@ -126,12 +126,12 @@
           cols[6] = files[i].getPermission().toString();
           cols[6] = files[i].getPermission().toString();
           cols[7] = files[i].getOwner();
           cols[7] = files[i].getOwner();
           cols[8] = files[i].getGroup();
           cols[8] = files[i].getGroup();
-          jspHelper.addTableRow(out, cols, row++);
+          JspHelper.addTableRow(out, cols, row++);
         }
         }
-        jspHelper.addTableFooter(out);
+        JspHelper.addTableFooter(out);
       }
       }
     } 
     } 
-    String namenodeHost = jspHelper.nameNodeAddr.getHostName();
+    String namenodeHost = datanode.getNameNodeAddr().getHostName();
     out.print("<br><a href=\"http://" + 
     out.print("<br><a href=\"http://" + 
               InetAddress.getByName(namenodeHost).getCanonicalHostName() + ":" +
               InetAddress.getByName(namenodeHost).getCanonicalHostName() + ":" +
               namenodeInfoPort + "/dfshealth.jsp\">Go back to DFS home</a>");
               namenodeInfoPort + "/dfshealth.jsp\">Go back to DFS home</a>");

+ 6 - 16
src/webapps/datanode/tail.jsp

@@ -18,14 +18,10 @@
 %>
 %>
 
 
 <%!
 <%!
-  static JspHelper jspHelper = new JspHelper();
+  static final DataNode datanode = DataNode.getDataNode();
 
 
   public void generateFileChunks(JspWriter out, HttpServletRequest req) 
   public void generateFileChunks(JspWriter out, HttpServletRequest req) 
     throws IOException {
     throws IOException {
-    long startOffset = 0;
-    
-    int chunkSizeToView = 0;
-
     String referrer = req.getParameter("referrer");
     String referrer = req.getParameter("referrer");
     boolean noLink = false;
     boolean noLink = false;
     if (referrer == null) {
     if (referrer == null) {
@@ -43,10 +39,7 @@
     if (namenodeInfoPortStr != null)
     if (namenodeInfoPortStr != null)
       namenodeInfoPort = Integer.parseInt(namenodeInfoPortStr);
       namenodeInfoPort = Integer.parseInt(namenodeInfoPortStr);
     
     
-    String chunkSizeToViewStr = req.getParameter("chunkSizeToView");
-    if (chunkSizeToViewStr != null && Integer.parseInt(chunkSizeToViewStr) > 0)
-      chunkSizeToView = Integer.parseInt(chunkSizeToViewStr);
-    else chunkSizeToView = jspHelper.defaultChunkSizeToView;
+    final int chunkSizeToView = JspHelper.string2ChunkSizeToView(req.getParameter("chunkSizeToView"));
 
 
     if (!noLink) {
     if (!noLink) {
       out.print("<h3>Tail of File: ");
       out.print("<h3>Tail of File: ");
@@ -70,8 +63,7 @@
                 referrer+ "\">");
                 referrer+ "\">");
 
 
     //fetch the block from the datanode that has the last block for this file
     //fetch the block from the datanode that has the last block for this file
-    DFSClient dfs = new DFSClient(jspHelper.nameNodeAddr, 
-                                         jspHelper.conf);
+    final DFSClient dfs = new DFSClient(datanode.getNameNodeAddr(), JspHelper.conf);
     List<LocatedBlock> blocks = 
     List<LocatedBlock> blocks = 
       dfs.namenode.getBlockLocations(filename, 0, Long.MAX_VALUE).getLocatedBlocks();
       dfs.namenode.getBlockLocations(filename, 0, Long.MAX_VALUE).getLocatedBlocks();
     if (blocks == null || blocks.size() == 0) {
     if (blocks == null || blocks.size() == 0) {
@@ -85,7 +77,7 @@
     long genStamp = lastBlk.getBlock().getGenerationStamp();
     long genStamp = lastBlk.getBlock().getGenerationStamp();
     DatanodeInfo chosenNode;
     DatanodeInfo chosenNode;
     try {
     try {
-      chosenNode = jspHelper.bestNode(lastBlk);
+      chosenNode = JspHelper.bestNode(lastBlk);
     } catch (IOException e) {
     } catch (IOException e) {
       out.print(e.toString());
       out.print(e.toString());
       dfs.close();
       dfs.close();
@@ -93,12 +85,10 @@
     }      
     }      
     InetSocketAddress addr = NetUtils.createSocketAddr(chosenNode.getName());
     InetSocketAddress addr = NetUtils.createSocketAddr(chosenNode.getName());
     //view the last chunkSizeToView bytes while Tailing
     //view the last chunkSizeToView bytes while Tailing
-    if (blockSize >= chunkSizeToView)
-      startOffset = blockSize - chunkSizeToView;
-    else startOffset = 0;
+    final long startOffset = blockSize >= chunkSizeToView? blockSize - chunkSizeToView: 0;
 
 
     out.print("<textarea cols=\"100\" rows=\"25\" wrap=\"virtual\" style=\"width:100%\" READONLY>");
     out.print("<textarea cols=\"100\" rows=\"25\" wrap=\"virtual\" style=\"width:100%\" READONLY>");
-    jspHelper.streamBlockInAscii(addr, blockId, genStamp, blockSize, startOffset, chunkSizeToView, out);
+    JspHelper.streamBlockInAscii(addr, blockId, genStamp, blockSize, startOffset, chunkSizeToView, out);
     out.print("</textarea>");
     out.print("</textarea>");
     dfs.close();
     dfs.close();
   }
   }

+ 5 - 14
src/webapps/hdfs/dfshealth.jsp

@@ -17,8 +17,6 @@
   import="java.net.URLEncoder"
   import="java.net.URLEncoder"
 %>
 %>
 <%!
 <%!
-  JspHelper jspHelper = new JspHelper();
-
   int rowNum = 0;
   int rowNum = 0;
   int colNum = 0;
   int colNum = 0;
 
 
@@ -161,7 +159,7 @@
     FSNamesystem fsn = nn.getNamesystem();
     FSNamesystem fsn = nn.getNamesystem();
     ArrayList<DatanodeDescriptor> live = new ArrayList<DatanodeDescriptor>();
     ArrayList<DatanodeDescriptor> live = new ArrayList<DatanodeDescriptor>();
     ArrayList<DatanodeDescriptor> dead = new ArrayList<DatanodeDescriptor>();
     ArrayList<DatanodeDescriptor> dead = new ArrayList<DatanodeDescriptor>();
-    jspHelper.DFSNodesStatus(live, dead);
+    fsn.DFSNodesStatus(live, dead);
 
 
     sorterField = request.getParameter("sorter/field");
     sorterField = request.getParameter("sorter/field");
     sorterOrder = request.getParameter("sorter/order");
     sorterOrder = request.getParameter("sorter/order");
@@ -235,22 +233,15 @@
     
     
 <body>
 <body>
 <h1>NameNode '<%=namenodeLabel%>'</h1>
 <h1>NameNode '<%=namenodeLabel%>'</h1>
-
-
-<div id="dfstable"> <table>	  
-<tr> <td id="col1"> Started: <td> <%= fsn.getStartTime()%>
-<tr> <td id="col1"> Version: <td> <%= VersionInfo.getVersion()%>, <%= VersionInfo.getRevision()%>
-<tr> <td id="col1"> Compiled: <td> <%= VersionInfo.getDate()%> by <%= VersionInfo.getUser()%> from <%= VersionInfo.getBranch()%>
-<tr> <td id="col1"> Upgrades: <td> <%= jspHelper.getUpgradeStatusText()%>
-</table></div><br>				      
-
+<%= JspHelper.getVersionTable(fsn) %>
+<br />
 <b><a href="/nn_browsedfscontent.jsp">Browse the filesystem</a></b><br>
 <b><a href="/nn_browsedfscontent.jsp">Browse the filesystem</a></b><br>
 <b><a href="/logs/">Namenode Logs</a></b>
 <b><a href="/logs/">Namenode Logs</a></b>
 
 
 <hr>
 <hr>
 <h3>Cluster Summary</h3>
 <h3>Cluster Summary</h3>
-<b> <%= jspHelper.getSafeModeText()%> </b>
-<b> <%= jspHelper.getInodeLimitText()%> </b>
+<b> <%= JspHelper.getSafeModeText(fsn)%> </b>
+<b> <%= JspHelper.getInodeLimitText(fsn)%> </b>
 <%
 <%
     generateDFSHealthReport(out, nn, request); 
     generateDFSHealthReport(out, nn, request); 
 %>
 %>

+ 7 - 16
src/webapps/hdfs/dfsnodelist.jsp

@@ -16,8 +16,6 @@ contentType="text/html; charset=UTF-8"
 	import="java.net.URLEncoder"
 	import="java.net.URLEncoder"
 %>
 %>
 <%!
 <%!
-	JspHelper jspHelper = new JspHelper();
-
 	int rowNum = 0;
 	int rowNum = 0;
 	int colNum = 0;
 	int colNum = 0;
 
 
@@ -127,7 +125,7 @@ public void generateDFSNodesList(JspWriter out,
 throws IOException {
 throws IOException {
 	ArrayList<DatanodeDescriptor> live = new ArrayList<DatanodeDescriptor>();    
 	ArrayList<DatanodeDescriptor> live = new ArrayList<DatanodeDescriptor>();    
 	ArrayList<DatanodeDescriptor> dead = new ArrayList<DatanodeDescriptor>();
 	ArrayList<DatanodeDescriptor> dead = new ArrayList<DatanodeDescriptor>();
-	jspHelper.DFSNodesStatus(live, dead);
+	nn.getNamesystem().DFSNodesStatus(live, dead);
 
 
 	whatNodes = request.getParameter("whatNodes"); // show only live or only dead nodes
 	whatNodes = request.getParameter("whatNodes"); // show only live or only dead nodes
 	sorterField = request.getParameter("sorter/field");
 	sorterField = request.getParameter("sorter/field");
@@ -137,8 +135,8 @@ throws IOException {
 	if ( sorterOrder == null )
 	if ( sorterOrder == null )
 		sorterOrder = "ASC";
 		sorterOrder = "ASC";
 
 
-	jspHelper.sortNodeList(live, sorterField, sorterOrder);
-	jspHelper.sortNodeList(dead, "name", "ASC");
+	JspHelper.sortNodeList(live, sorterField, sorterOrder);
+	JspHelper.sortNodeList(dead, "name", "ASC");
 
 
 	// Find out common suffix. Should this be before or after the sort?
 	// Find out common suffix. Should this be before or after the sort?
 	String port_suffix = null;
 	String port_suffix = null;
@@ -203,7 +201,7 @@ throws IOException {
 						NodeHeaderStr("pcremaining") + "> Remaining <br>(%) <th " +
 						NodeHeaderStr("pcremaining") + "> Remaining <br>(%) <th " +
 						NodeHeaderStr("blocks") + "> Blocks\n" );
 						NodeHeaderStr("blocks") + "> Blocks\n" );
 
 
-				jspHelper.sortNodeList(live, sorterField, sorterOrder);
+				JspHelper.sortNodeList(live, sorterField, sorterOrder);
 				for ( int i=0; i < live.size(); i++ ) {
 				for ( int i=0; i < live.size(); i++ ) {
 					generateNodeData(out, live.get(i), port_suffix, true, nnHttpPort);
 					generateNodeData(out, live.get(i), port_suffix, true, nnHttpPort);
 				}
 				}
@@ -218,7 +216,7 @@ throws IOException {
 				out.print( "<table border=1 cellspacing=0> <tr id=\"row1\"> " +
 				out.print( "<table border=1 cellspacing=0> <tr id=\"row1\"> " +
 				"<td> Node \n" );
 				"<td> Node \n" );
 
 
-				jspHelper.sortNodeList(dead, "name", "ASC");
+				JspHelper.sortNodeList(dead, "name", "ASC");
 				for ( int i=0; i < dead.size() ; i++ ) {
 				for ( int i=0; i < dead.size() ; i++ ) {
 					generateNodeData(out, dead.get(i), port_suffix, false, nnHttpPort);
 					generateNodeData(out, dead.get(i), port_suffix, false, nnHttpPort);
 				}
 				}
@@ -243,15 +241,8 @@ String namenodeLabel = nn.getNameNodeAddress().getHostName() + ":" + nn.getNameN
   
   
 <body>
 <body>
 <h1>NameNode '<%=namenodeLabel%>'</h1>
 <h1>NameNode '<%=namenodeLabel%>'</h1>
-
-
-<div id="dfstable"> <table>	  
-<tr> <td id="col1"> Started: <td> <%= fsn.getStartTime()%>
-<tr> <td id="col1"> Version: <td> <%= VersionInfo.getVersion()%>, r<%= VersionInfo.getRevision()%>
-<tr> <td id="col1"> Compiled: <td> <%= VersionInfo.getDate()%> by <%= VersionInfo.getUser()%>
-<tr> <td id="col1"> Upgrades: <td> <%= jspHelper.getUpgradeStatusText()%>
-</table></div><br>				      
-
+<%= JspHelper.getVersionTable(fsn) %>
+<br />
 <b><a href="/nn_browsedfscontent.jsp">Browse the filesystem</a></b><br>
 <b><a href="/nn_browsedfscontent.jsp">Browse the filesystem</a></b><br>
 <b><a href="/logs/">Namenode Logs</a></b><br>
 <b><a href="/logs/">Namenode Logs</a></b><br>
 <b><a href=/dfshealth.jsp> Go back to DFS home</a></b>
 <b><a href=/dfshealth.jsp> Go back to DFS home</a></b>