Browse Source

HDFS-6252. Phase out the old web UI in HDFS. Contributed by Haohui Mai.

Haohui Mai 10 năm trước cách đây
mục cha
commit
f11c8eb096
25 tập tin đã thay đổi với 31 bổ sung4130 xóa
  1. 0 21
      hadoop-common-project/hadoop-common/src/test/resources/webapps/test/testjsp.jsp
  2. 2 0
      hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
  3. 4 379
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/JspHelper.java
  4. 3 667
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DatanodeJspHelper.java
  5. 3 1152
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NamenodeJspHelper.java
  6. 0 49
      hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/datanode/browseBlock.jsp
  7. 0 69
      hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/datanode/browseDirectory.jsp
  8. 0 58
      hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/datanode/dataNodeHome.jsp
  9. 0 50
      hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/datanode/tail.jsp
  10. 0 87
      hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/hdfs/block_info_xml.jsp
  11. 0 85
      hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/hdfs/corrupt_files.jsp
  12. 0 91
      hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/hdfs/corrupt_replicas_xml.jsp
  13. 0 45
      hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/hdfs/decommission.jsp
  14. 0 139
      hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/hdfs/decommission.xsl
  15. 0 45
      hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/hdfs/dfsclusterhealth.jsp
  16. 0 170
      hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/hdfs/dfsclusterhealth.xsl
  17. 0 88
      hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/hdfs/dfsclusterhealth_utils.xsl
  18. 0 75
      hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/hdfs/dfshealth.jsp
  19. 0 59
      hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/hdfs/dfsnodelist.jsp
  20. 0 46
      hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/hdfs/nn_browsedfscontent.jsp
  21. 0 42
      hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/journal/journalstatus.jsp
  22. 0 45
      hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/secondary/status.jsp
  23. 12 188
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/common/TestJspHelper.java
  24. 0 188
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDatanodeJsp.java
  25. 7 292
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNameNodeJspHelper.java

+ 0 - 21
hadoop-common-project/hadoop-common/src/test/resources/webapps/test/testjsp.jsp

@@ -1,21 +0,0 @@
-<?xml version="1.0" encoding="UTF-8"?><%!
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file 
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you 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.
- */
-%>
-<%@ page contentType="text/html; charset=UTF-8" %> 
-Hello world!

+ 2 - 0
hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt

@@ -21,6 +21,8 @@ Release 2.7.0 - UNRELEASED
 
     HDFS-7209. Populate EDEK cache when creating encryption zone. (Yi Liu via wang)
 
+    HDFS-6252. Phase out the old web UI in HDFS. (wheat9)
+
   OPTIMIZATIONS
 
   BUG FIXES

+ 4 - 379
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/JspHelper.java

@@ -24,52 +24,30 @@ import static org.apache.hadoop.fs.CommonConfigurationKeys.HADOOP_HTTP_STATIC_US
 import java.io.ByteArrayInputStream;
 import java.io.DataInputStream;
 import java.io.IOException;
-import java.io.UnsupportedEncodingException;
-import java.net.InetAddress;
 import java.net.InetSocketAddress;
-import java.net.Socket;
-import java.net.URL;
-import java.net.URLEncoder;
 import java.util.Arrays;
-import java.util.Collections;
 import java.util.Comparator;
 import java.util.HashMap;
-import java.util.List;
 
 import javax.servlet.ServletContext;
 import javax.servlet.http.HttpServletRequest;
-import javax.servlet.jsp.JspWriter;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.hdfs.BlockReader;
-import org.apache.hadoop.hdfs.BlockReaderFactory;
-import org.apache.hadoop.hdfs.ClientContext;
-import org.apache.hadoop.hdfs.DFSClient;
 import org.apache.hadoop.hdfs.DFSUtil;
-import org.apache.hadoop.hdfs.RemotePeerFactory;
-import org.apache.hadoop.hdfs.net.Peer;
-import org.apache.hadoop.hdfs.net.TcpPeerServer;
 import org.apache.hadoop.hdfs.protocol.DatanodeID;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
-import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
-import org.apache.hadoop.hdfs.protocol.datatransfer.sasl.SaslDataTransferClient;
-import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier;
 import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier;
-import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor;
-import org.apache.hadoop.hdfs.server.datanode.CachingStrategy;
 import org.apache.hadoop.hdfs.server.namenode.NameNode;
 import org.apache.hadoop.hdfs.server.namenode.NameNodeHttpServer;
 import org.apache.hadoop.hdfs.web.resources.DelegationParam;
 import org.apache.hadoop.hdfs.web.resources.DoAsParam;
 import org.apache.hadoop.hdfs.web.resources.UserParam;
-import org.apache.hadoop.http.HtmlQuoting;
-import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.net.NetUtils;
 import org.apache.hadoop.security.AccessControlException;
 import org.apache.hadoop.security.SecurityUtil;
@@ -79,9 +57,7 @@ import org.apache.hadoop.security.authentication.util.KerberosName;
 import org.apache.hadoop.security.authorize.ProxyServers;
 import org.apache.hadoop.security.authorize.ProxyUsers;
 import org.apache.hadoop.security.token.Token;
-import org.apache.hadoop.util.VersionInfo;
 
-import com.google.common.base.Charsets;
 
 @InterfaceAudience.Private
 public class JspHelper {
@@ -93,8 +69,8 @@ public class JspHelper {
   private static final Log LOG = LogFactory.getLog(JspHelper.class);
 
   /** Private constructor for preventing creating JspHelper object. */
-  private JspHelper() {} 
-  
+  private JspHelper() {}
+
   // data structure to count number of blocks on datanodes.
   private static class NodeRecord extends DatanodeInfo {
     int frequency;
@@ -103,7 +79,7 @@ public class JspHelper {
       super(info);
       this.frequency = count;
     }
-    
+
     @Override
     public boolean equals(Object obj) {
       // Sufficient to use super equality as datanodes are uniquely identified
@@ -126,43 +102,8 @@ public class JspHelper {
         return -1;
       } else if (o1.frequency > o2.frequency) {
         return 1;
-      } 
-      return 0;
-    }
-  }
-  
-  /**
-   * convenience method for canonicalizing host name.
-   * @param addr name:port or name 
-   * @return canonicalized host name
-   */
-   public static String canonicalize(String addr) {
-    // default port 1 is supplied to allow addr without port.
-    // the port will be ignored.
-    return NetUtils.createSocketAddr(addr, 1).getAddress()
-           .getCanonicalHostName();
-  }
-
-  /**
-   * A helper class that generates the correct URL for different schema.
-   *
-   */
-  public static final class Url {
-    public static String authority(String scheme, DatanodeID d) {
-      String fqdn = (d.getIpAddr() != null && !d.getIpAddr().isEmpty())?
-          canonicalize(d.getIpAddr()): 
-          d.getHostName();
-      if (scheme.equals("http")) {
-        return fqdn + ":" + d.getInfoPort();
-      } else if (scheme.equals("https")) {
-        return fqdn + ":" + d.getInfoSecurePort();
-      } else {
-        throw new IllegalArgumentException("Unknown scheme:" + scheme);
       }
-    }
-
-    public static String url(String scheme, DatanodeID d) {
-      return scheme + "://" + authority(scheme, d);
+      return 0;
     }
   }
 
@@ -186,12 +127,6 @@ public class JspHelper {
     return bestNode(nodes, false);
   }
 
-  public static DatanodeInfo bestNode(LocatedBlock blk, Configuration conf)
-      throws IOException {
-    DatanodeInfo[] nodes = blk.getLocations();
-    return bestNode(nodes, true);
-  }
-
   private static DatanodeInfo bestNode(DatanodeInfo[] nodes, boolean doRandom)
       throws IOException {
     if (nodes == null || nodes.length == 0) {
@@ -210,293 +145,6 @@ public class JspHelper {
     return nodes[index];
   }
 
-  public static void streamBlockInAscii(InetSocketAddress addr, String poolId,
-      long blockId, final Token<BlockTokenIdentifier> blockToken, long genStamp,
-      long blockSize, long offsetIntoBlock, long chunkSizeToView,
-      JspWriter out, final Configuration conf, DFSClient.Conf dfsConf,
-      final DFSClient dfs, final SaslDataTransferClient saslClient)
-          throws IOException {
-    if (chunkSizeToView == 0) return;
-    int amtToRead = (int)Math.min(chunkSizeToView, blockSize - offsetIntoBlock);
-      
-    DatanodeID datanodeId = new DatanodeID(addr.getAddress().getHostAddress(),
-      addr.getHostName(), poolId, addr.getPort(), 0, 0, 0);
-    BlockReader blockReader = new BlockReaderFactory(dfsConf).
-      setInetSocketAddress(addr).
-      setBlock(new ExtendedBlock(poolId, blockId, 0, genStamp)).
-      setFileName(BlockReaderFactory.getFileName(addr, poolId, blockId)).
-      setBlockToken(blockToken).
-      setStartOffset(offsetIntoBlock).
-      setLength(amtToRead).
-      setVerifyChecksum(true).
-      setClientName("JspHelper").
-      setClientCacheContext(ClientContext.getFromConf(conf)).
-      setDatanodeInfo(new DatanodeInfo(datanodeId)).
-      setCachingStrategy(CachingStrategy.newDefaultStrategy()).
-      setConfiguration(conf).
-      setRemotePeerFactory(new RemotePeerFactory() {
-        @Override
-        public Peer newConnectedPeer(InetSocketAddress addr,
-            Token<BlockTokenIdentifier> blockToken, DatanodeID datanodeId)
-            throws IOException {
-          Peer peer = null;
-          Socket sock = NetUtils.getDefaultSocketFactory(conf).createSocket();
-          try {
-            sock.connect(addr, HdfsServerConstants.READ_TIMEOUT);
-            sock.setSoTimeout(HdfsServerConstants.READ_TIMEOUT);
-            peer = TcpPeerServer.peerFromSocketAndKey(saslClient, sock, dfs,
-                blockToken, datanodeId);
-          } finally {
-            if (peer == null) {
-              IOUtils.closeSocket(sock);
-            }
-          }
-          return peer;
-        }
-      }).
-      build();
-
-    final byte[] buf = new byte[amtToRead];
-    try {
-      int readOffset = 0;
-      int retries = 2;
-      while (amtToRead > 0) {
-        int numRead = amtToRead;
-        try {
-          blockReader.readFully(buf, readOffset, amtToRead);
-        } catch (IOException e) {
-          retries--;
-          if (retries == 0)
-            throw new IOException("Could not read data from datanode");
-          continue;
-        }
-        amtToRead -= numRead;
-        readOffset += numRead;
-      }
-    } finally {
-      blockReader.close();
-    }
-    out.print(HtmlQuoting.quoteHtmlChars(new String(buf, Charsets.UTF_8)));
-  }
-
-  public static void addTableHeader(JspWriter out) throws IOException {
-    out.print("<table border=\"1\""+
-              " cellpadding=\"2\" cellspacing=\"2\">");
-    out.print("<tbody>");
-  }
-  public static 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 static void addTableRow(JspWriter out, String[] columns, int row) throws IOException {
-    out.print("<tr>");
-      
-    for (int i = 0; i < columns.length; i++) {
-      if (row/2*2 == row) {//even
-        out.print("<td style=\"vertical-align: top;background-color:LightGrey;\"><B>"+columns[i]+"</B><br></td>");
-      } else {
-        out.print("<td style=\"vertical-align: top;background-color:LightBlue;\"><B>"+columns[i]+"</B><br></td>");
-          
-      }
-    }
-    out.print("</tr>");
-  }
-  public static void addTableFooter(JspWriter out) throws IOException {
-    out.print("</tbody></table>");
-  }
-
-  public static void sortNodeList(final List<DatanodeDescriptor> nodes,
-                           String field, String order) {
-        
-    class NodeComapare implements Comparator<DatanodeDescriptor> {
-      static final int 
-        FIELD_NAME              = 1,
-        FIELD_LAST_CONTACT      = 2,
-        FIELD_BLOCKS            = 3,
-        FIELD_CAPACITY          = 4,
-        FIELD_USED              = 5,
-        FIELD_PERCENT_USED      = 6,
-        FIELD_NONDFS_USED       = 7,
-        FIELD_REMAINING         = 8,
-        FIELD_PERCENT_REMAINING = 9,
-        FIELD_ADMIN_STATE       = 10,
-        FIELD_DECOMMISSIONED    = 11,
-        SORT_ORDER_ASC          = 1,
-        SORT_ORDER_DSC          = 2;
-
-      int sortField = FIELD_NAME;
-      int sortOrder = SORT_ORDER_ASC;
-            
-      public NodeComapare(String field, String order) {
-        if (field.equals("lastcontact")) {
-          sortField = FIELD_LAST_CONTACT;
-        } else if (field.equals("capacity")) {
-          sortField = FIELD_CAPACITY;
-        } else if (field.equals("used")) {
-          sortField = FIELD_USED;
-        } else if (field.equals("nondfsused")) {
-          sortField = FIELD_NONDFS_USED;
-        } else if (field.equals("remaining")) {
-          sortField = FIELD_REMAINING;
-        } else if (field.equals("pcused")) {
-          sortField = FIELD_PERCENT_USED;
-        } else if (field.equals("pcremaining")) {
-          sortField = FIELD_PERCENT_REMAINING;
-        } else if (field.equals("blocks")) {
-          sortField = FIELD_BLOCKS;
-        } else if (field.equals("adminstate")) {
-          sortField = FIELD_ADMIN_STATE;
-        } else if (field.equals("decommissioned")) {
-          sortField = FIELD_DECOMMISSIONED;
-        } else {
-          sortField = FIELD_NAME;
-        }
-                
-        if (order.equals("DSC")) {
-          sortOrder = SORT_ORDER_DSC;
-        } else {
-          sortOrder = SORT_ORDER_ASC;
-        }
-      }
-
-      @Override
-      public int compare(DatanodeDescriptor d1,
-                         DatanodeDescriptor d2) {
-        int ret = 0;
-        switch (sortField) {
-        case FIELD_LAST_CONTACT:
-          ret = (int) (d2.getLastUpdate() - d1.getLastUpdate());
-          break;
-        case FIELD_CAPACITY:
-          long  dlong = d1.getCapacity() - d2.getCapacity();
-          ret = (dlong < 0) ? -1 : ((dlong > 0) ? 1 : 0);
-          break;
-        case FIELD_USED:
-          dlong = d1.getDfsUsed() - d2.getDfsUsed();
-          ret = (dlong < 0) ? -1 : ((dlong > 0) ? 1 : 0);
-          break;
-        case FIELD_NONDFS_USED:
-          dlong = d1.getNonDfsUsed() - d2.getNonDfsUsed();
-          ret = (dlong < 0) ? -1 : ((dlong > 0) ? 1 : 0);
-          break;
-        case FIELD_REMAINING:
-          dlong = d1.getRemaining() - d2.getRemaining();
-          ret = (dlong < 0) ? -1 : ((dlong > 0) ? 1 : 0);
-          break;
-        case FIELD_PERCENT_USED:
-          double ddbl =((d1.getDfsUsedPercent())-
-                        (d2.getDfsUsedPercent()));
-          ret = (ddbl < 0) ? -1 : ((ddbl > 0) ? 1 : 0);
-          break;
-        case FIELD_PERCENT_REMAINING:
-          ddbl =((d1.getRemainingPercent())-
-                 (d2.getRemainingPercent()));
-          ret = (ddbl < 0) ? -1 : ((ddbl > 0) ? 1 : 0);
-          break;
-        case FIELD_BLOCKS:
-          ret = d1.numBlocks() - d2.numBlocks();
-          break;
-        case FIELD_ADMIN_STATE:
-          ret = d1.getAdminState().toString().compareTo(
-              d2.getAdminState().toString());
-          break;
-        case FIELD_DECOMMISSIONED:
-          ret = DFSUtil.DECOM_COMPARATOR.compare(d1, d2);
-          break;
-        case FIELD_NAME: 
-          ret = d1.getHostName().compareTo(d2.getHostName());
-          break;
-        default:
-          throw new IllegalArgumentException("Invalid sortField");
-        }
-        return (sortOrder == SORT_ORDER_DSC) ? -ret : ret;
-      }
-    }
-        
-    Collections.sort(nodes, new NodeComapare(field, order));
-  }
-
-  public static void printPathWithLinks(String dir, JspWriter out, 
-                                        int namenodeInfoPort,
-                                        String tokenString,
-                                        String nnAddress
-                                        ) throws IOException {
-    try {
-      String[] parts = dir.split(Path.SEPARATOR);
-      StringBuilder tempPath = new StringBuilder(dir.length());
-      out.print("<a href=\"browseDirectory.jsp" + "?dir="+ Path.SEPARATOR
-          + "&namenodeInfoPort=" + namenodeInfoPort
-          + getDelegationTokenUrlParam(tokenString) 
-          + getUrlParam(NAMENODE_ADDRESS, nnAddress) + "\">" + Path.SEPARATOR
-          + "</a>");
-      tempPath.append(Path.SEPARATOR);
-      for (int i = 0; i < parts.length-1; i++) {
-        if (!parts[i].equals("")) {
-          tempPath.append(parts[i]);
-          out.print("<a href=\"browseDirectory.jsp" + "?dir="
-              + HtmlQuoting.quoteHtmlChars(tempPath.toString()) + "&namenodeInfoPort=" + namenodeInfoPort
-              + getDelegationTokenUrlParam(tokenString)
-              + getUrlParam(NAMENODE_ADDRESS, nnAddress));
-          out.print("\">" + HtmlQuoting.quoteHtmlChars(parts[i]) + "</a>" + Path.SEPARATOR);
-          tempPath.append(Path.SEPARATOR);
-        }
-      }
-      if(parts.length > 0) {
-        out.print(HtmlQuoting.quoteHtmlChars(parts[parts.length-1]));
-      }
-    }
-    catch (UnsupportedEncodingException ex) {
-      ex.printStackTrace();
-    }
-  }
-
-  public static void printGotoForm(JspWriter out,
-                                   int namenodeInfoPort,
-                                   String tokenString,
-                                   String file,
-                                   String nnAddress) throws IOException {
-    out.print("<form action=\"browseDirectory.jsp\" method=\"get\" name=\"goto\">");
-    out.print("Goto : ");
-    out.print("<input name=\"dir\" type=\"text\" width=\"50\" id=\"dir\" value=\""+ HtmlQuoting.quoteHtmlChars(file)+"\"/>");
-    out.print("<input name=\"go\" type=\"submit\" value=\"go\"/>");
-    out.print("<input name=\"namenodeInfoPort\" type=\"hidden\" "
-        + "value=\"" + namenodeInfoPort  + "\"/>");
-    if (UserGroupInformation.isSecurityEnabled()) {
-      out.print("<input name=\"" + DELEGATION_PARAMETER_NAME
-          + "\" type=\"hidden\" value=\"" + tokenString + "\"/>");
-    }
-    out.print("<input name=\""+ NAMENODE_ADDRESS +"\" type=\"hidden\" "
-        + "value=\"" + nnAddress  + "\"/>");
-    out.print("</form>");
-  }
-  
-  public static void createTitle(JspWriter out, 
-                                 HttpServletRequest req, 
-                                 String  file) throws IOException{
-    if(file == null) file = "";
-    int start = Math.max(0,file.length() - 100);
-    if(start != 0)
-      file = "..." + file.substring(start, file.length());
-    out.print("<title>HDFS:" + file + "</title>");
-  }
-
-  /** Convert a String to chunk-size-to-view. */
-  public static int string2ChunkSizeToView(String s, int defaultValue) {
-    int n = s == null? 0: Integer.parseInt(s);
-    return n > 0? n: defaultValue;
-  }
-
-  /** Return a table containing version information. */
-  public static String getVersionTable() {
-    return "<div class='dfstable'><table>"       
-        + "\n  <tr><td class='col1'>Version:</td><td>" + VersionInfo.getVersion() + ", " + VersionInfo.getRevision() + "</td></tr>"
-        + "\n  <tr><td class='col1'>Compiled:</td><td>" + VersionInfo.getDate() + " by " + VersionInfo.getUser() + " from " + VersionInfo.getBranch() + "</td></tr>"
-        + "\n</table></div>";
-  }
-
   /**
    * Validate filename. 
    * @return null if the filename is invalid.
@@ -516,29 +164,6 @@ public class JspHelper {
     return value == null? null: Long.parseLong(value);
   }
 
-  /**
-   * Validate a URL.
-   * @return null if the value is invalid.
-   *         Otherwise, return the validated URL String.
-   */
-  public static String validateURL(String value) {
-    try {
-      return URLEncoder.encode(new URL(value).toString(), "UTF-8");
-    } catch (IOException e) {
-      return null;
-    }
-  }
-  
-  /**
-   * If security is turned off, what is the default web user?
-   * @param conf the configuration to look in
-   * @return the remote user that was configuration
-   */
-  public static UserGroupInformation getDefaultWebUser(Configuration conf
-                                                       ) throws IOException {
-    return UserGroupInformation.createRemoteUser(getDefaultWebUserName(conf));
-  }
-
   private static String getDefaultWebUserName(Configuration conf
       ) throws IOException {
     String user = conf.get(

+ 3 - 667
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DatanodeJspHelper.java

@@ -17,53 +17,19 @@
  */
 package org.apache.hadoop.hdfs.server.datanode;
 
-import java.io.File;
-import java.io.FileNotFoundException;
-import java.io.IOException;
-import java.net.InetSocketAddress;
-import java.net.URL;
-import java.net.URLEncoder;
-import java.security.PrivilegedExceptionAction;
-import java.text.SimpleDateFormat;
-import java.util.Date;
-import java.util.List;
-
-import javax.servlet.ServletContext;
-import javax.servlet.http.HttpServletRequest;
-import javax.servlet.http.HttpServletResponse;
-import javax.servlet.jsp.JspWriter;
-
-import org.apache.commons.lang.StringEscapeUtils;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.UnresolvedLinkException;
 import org.apache.hadoop.hdfs.DFSClient;
-import org.apache.hadoop.hdfs.DFSConfigKeys;
-import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
-import org.apache.hadoop.hdfs.protocol.DirectoryListing;
-import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
-import org.apache.hadoop.hdfs.protocol.LocatedBlock;
-import org.apache.hadoop.hdfs.protocol.datatransfer.sasl.SaslDataTransferClient;
-import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier;
-import org.apache.hadoop.hdfs.security.token.block.BlockTokenSecretManager;
 import org.apache.hadoop.hdfs.server.common.JspHelper;
-import org.apache.hadoop.http.HtmlQuoting;
 import org.apache.hadoop.net.NetUtils;
-import org.apache.hadoop.security.AccessControlException;
 import org.apache.hadoop.security.UserGroupInformation;
-import org.apache.hadoop.security.token.Token;
-import org.apache.hadoop.util.ServletUtil;
-import org.apache.hadoop.util.StringUtils;
-import org.apache.hadoop.util.VersionInfo;
 
-import com.google.common.base.Predicate;
-import com.google.common.collect.Iterables;
+import javax.servlet.http.HttpServletRequest;
+import java.io.IOException;
+import java.security.PrivilegedExceptionAction;
 
 @InterfaceAudience.Private
 public class DatanodeJspHelper {
-  private static final int PREV_BLOCK = -1;
-  private static final int NEXT_BLOCK = 1;
-
   private static DFSClient getDFSClient(final UserGroupInformation user,
                                         final String addr,
                                         final Configuration conf
@@ -77,604 +43,6 @@ public class DatanodeJspHelper {
         }
       });
   }
-
-  /**
-   * Get the default chunk size.
-   * @param conf the configuration
-   * @return the number of bytes to chunk in
-   */
-  private static int getDefaultChunkSize(Configuration conf) {
-    return conf.getInt(DFSConfigKeys.DFS_DEFAULT_CHUNK_VIEW_SIZE_KEY,
-                       DFSConfigKeys.DFS_DEFAULT_CHUNK_VIEW_SIZE_DEFAULT);
-  }
-
-  static void generateDirectoryStructure(JspWriter out, 
-                                         HttpServletRequest req,
-                                         HttpServletResponse resp,
-                                         Configuration conf
-                                         ) throws IOException,
-                                                  InterruptedException {
-    final String dir = JspHelper.validatePath(
-        StringEscapeUtils.unescapeHtml(req.getParameter("dir")));
-    if (dir == null) {
-      out.print("Invalid input");
-      return;
-    }
-    String tokenString = req.getParameter(JspHelper.DELEGATION_PARAMETER_NAME);
-    UserGroupInformation ugi = JspHelper.getUGI(req, conf);
-    String namenodeInfoPortStr = req.getParameter("namenodeInfoPort");
-    int namenodeInfoPort = -1;
-    if (namenodeInfoPortStr != null)
-      namenodeInfoPort = Integer.parseInt(namenodeInfoPortStr);
-    final String nnAddr = req.getParameter(JspHelper.NAMENODE_ADDRESS);
-    if (nnAddr == null){
-      out.print(JspHelper.NAMENODE_ADDRESS + " url param is null");
-      return;
-    }
-    
-    DFSClient dfs = getDFSClient(ugi, nnAddr, conf);
-    String target = dir;
-    final HdfsFileStatus targetStatus = dfs.getFileInfo(target);
-    if (targetStatus == null) { // not exists
-      out.print("<h3>File or directory : " + StringEscapeUtils.escapeHtml(target) + " does not exist</h3>");
-      JspHelper.printGotoForm(out, namenodeInfoPort, tokenString, target,
-          nnAddr);
-    } else {
-      if (!targetStatus.isDir()) { // a file
-        List<LocatedBlock> blocks = dfs.getNamenode().getBlockLocations(dir, 0, 1)
-            .getLocatedBlocks();
-
-        LocatedBlock firstBlock = null;
-        DatanodeInfo[] locations = null;
-        if (blocks.size() > 0) {
-          firstBlock = blocks.get(0);
-          locations = firstBlock.getLocations();
-        }
-        if (locations == null || locations.length == 0) {
-          out.print("Empty file");
-        } else {
-          DatanodeInfo chosenNode = JspHelper.bestNode(firstBlock, conf);
-          int datanodePort = chosenNode.getXferPort();
-          String redirectLocation = JspHelper.Url.url(req.getScheme(),
-              chosenNode)
-              + "/browseBlock.jsp?blockId="
-              + firstBlock.getBlock().getBlockId() + "&blockSize="
-              + firstBlock.getBlock().getNumBytes() + "&genstamp="
-              + firstBlock.getBlock().getGenerationStamp() + "&filename="
-              + URLEncoder.encode(dir, "UTF-8") + "&datanodePort="
-              + datanodePort + "&namenodeInfoPort=" + namenodeInfoPort
-              + JspHelper.getDelegationTokenUrlParam(tokenString)
-              + JspHelper.getUrlParam(JspHelper.NAMENODE_ADDRESS, nnAddr);
-          resp.sendRedirect(redirectLocation);
-        }
-        return;
-      }
-      // directory
-      // generate a table and dump the info
-      String[] headings = { "Name", "Type", "Size", "Replication",
-          "Block Size", "Modification Time", "Permission", "Owner", "Group" };
-      out.print("<h3>Contents of directory ");
-      JspHelper.printPathWithLinks(dir, out, namenodeInfoPort, tokenString,
-          nnAddr);
-      out.print("</h3><hr>");
-      JspHelper.printGotoForm(out, namenodeInfoPort, tokenString, dir, nnAddr);
-      out.print("<hr>");
-
-      File f = new File(dir);
-      String parent;
-      if ((parent = f.getParent()) != null)
-        out.print("<a href=\"" + req.getRequestURL() + "?dir=" + parent
-            + "&namenodeInfoPort=" + namenodeInfoPort
-            + JspHelper.getDelegationTokenUrlParam(tokenString)
-            + JspHelper.getUrlParam(JspHelper.NAMENODE_ADDRESS, nnAddr)
-            + "\">Go to parent directory</a><br>");
-
-      DirectoryListing thisListing = 
-        dfs.listPaths(target, HdfsFileStatus.EMPTY_NAME);
-      if (thisListing == null || thisListing.getPartialListing().length == 0) {
-        out.print("Empty directory");
-      } else {
-        JspHelper.addTableHeader(out);
-        int row = 0;
-        JspHelper.addTableRow(out, headings, row++);
-        String cols[] = new String[headings.length];
-        do {
-          HdfsFileStatus[] files = thisListing.getPartialListing();
-          for (int i = 0; i < files.length; i++) {
-            String localFileName = files[i].getLocalName();
-            // Get the location of the first block of the file
-            if (!files[i].isDir()) {
-              cols[1] = "file";
-              cols[2] = StringUtils.byteDesc(files[i].getLen());
-              cols[3] = Short.toString(files[i].getReplication());
-              cols[4] = StringUtils.byteDesc(files[i].getBlockSize());
-            } else {
-              cols[1] = "dir";
-              cols[2] = "";
-              cols[3] = "";
-              cols[4] = "";
-            }
-            String datanodeUrl = req.getRequestURL() + "?dir="
-              + URLEncoder.encode(files[i].getFullName(target), "UTF-8")
-              + "&namenodeInfoPort=" + namenodeInfoPort
-              + JspHelper.getDelegationTokenUrlParam(tokenString)
-              + JspHelper.getUrlParam(JspHelper.NAMENODE_ADDRESS, nnAddr);
-            cols[0] = "<a href=\"" + datanodeUrl + "\">"
-              + HtmlQuoting.quoteHtmlChars(localFileName) + "</a>";
-            cols[5] = new SimpleDateFormat("yyyy-MM-dd HH:mm").format(
-                new Date((files[i].getModificationTime())));
-            cols[6] = files[i].getPermission().toString();
-            cols[7] = files[i].getOwner();
-            cols[8] = files[i].getGroup();
-            JspHelper.addTableRow(out, cols, row++);
-          }
-          if (!thisListing.hasMore()) {
-            break;
-          }
-          thisListing = dfs.listPaths(target, thisListing.getLastName());
-        } while (thisListing != null);
-        JspHelper.addTableFooter(out);
-      }
-    }
-    out.print("<br><a href=\"///"
-        + JspHelper.canonicalize(nnAddr) + ":"
-        + namenodeInfoPort + "/dfshealth.jsp\">Go back to DFS home</a>");
-    dfs.close();
-  }
-
-  static void generateFileDetails(JspWriter out, 
-                                  HttpServletRequest req,
-                                  Configuration conf
-                                  ) throws IOException,
-                                           InterruptedException {
-
-    long startOffset = 0;
-    int datanodePort;
-
-    final Long blockId = JspHelper.validateLong(req.getParameter("blockId"));
-    if (blockId == null) {
-      out.print("Invalid input (blockId absent)");
-      return;
-    }
-    String tokenString = req.getParameter(JspHelper.DELEGATION_PARAMETER_NAME);
-    UserGroupInformation ugi = JspHelper.getUGI(req, conf);
-
-    String datanodePortStr = req.getParameter("datanodePort");
-    if (datanodePortStr == null) {
-      out.print("Invalid input (datanodePort absent)");
-      return;
-    }
-    datanodePort = Integer.parseInt(datanodePortStr);
-
-    final Long genStamp = JspHelper.validateLong(req.getParameter("genstamp"));
-    if (genStamp == null) {
-      out.print("Invalid input (genstamp absent)");
-      return;
-    }
-    String namenodeInfoPortStr = req.getParameter("namenodeInfoPort");
-    int namenodeInfoPort = -1;
-    if (namenodeInfoPortStr != null)
-      namenodeInfoPort = Integer.parseInt(namenodeInfoPortStr);
-    final String nnAddr = StringEscapeUtils.escapeHtml(
-        req.getParameter(JspHelper.NAMENODE_ADDRESS));
-    if (nnAddr == null){
-      out.print(JspHelper.NAMENODE_ADDRESS + " url param is null");
-      return;
-    }
-
-    final int chunkSizeToView = JspHelper.string2ChunkSizeToView(
-        req.getParameter("chunkSizeToView"), getDefaultChunkSize(conf));
-
-    String startOffsetStr = req.getParameter("startOffset");
-    if (startOffsetStr == null || Long.parseLong(startOffsetStr) < 0)
-      startOffset = 0;
-    else
-      startOffset = Long.parseLong(startOffsetStr);
-
-    String path = StringEscapeUtils.unescapeHtml(req.getParameter("filename"));
-    if (path == null) {
-      path = req.getPathInfo() == null ? "/" : req.getPathInfo();
-    }
-    final String filename = JspHelper.validatePath(path);
-    if (filename == null) {
-      out.print("Invalid input");
-      return;
-    }
-
-    final String blockSizeStr = req.getParameter("blockSize");
-    if (blockSizeStr == null || blockSizeStr.length() == 0) {
-      out.print("Invalid input");
-      return;
-    }
-    long blockSize = Long.parseLong(blockSizeStr);
-
-    final DFSClient dfs = getDFSClient(ugi, nnAddr, conf);
-    List<LocatedBlock> blocks = dfs.getNamenode().getBlockLocations(filename, 0,
-        Long.MAX_VALUE).getLocatedBlocks();
-    // Add the various links for looking at the file contents
-    // URL for downloading the full file
-    String downloadUrl = "/streamFile" + ServletUtil.encodePath(filename)
-        + JspHelper.getUrlParam(JspHelper.NAMENODE_ADDRESS, nnAddr, true)
-        + JspHelper.getDelegationTokenUrlParam(tokenString);
-    out.print("<a name=\"viewOptions\"></a>");
-    out.print("<a href=\"" + downloadUrl + "\">Download this file</a><br>");
-
-    DatanodeInfo chosenNode;
-    // URL for TAIL
-    LocatedBlock lastBlk = blocks.get(blocks.size() - 1);
-    try {
-      chosenNode = JspHelper.bestNode(lastBlk, conf);
-    } catch (IOException e) {
-      out.print(e.toString());
-      dfs.close();
-      return;
-    }
-
-    String tailUrl = "///" + JspHelper.Url.authority(req.getScheme(), chosenNode)
-        + "/tail.jsp?filename=" + URLEncoder.encode(filename, "UTF-8")
-        + "&namenodeInfoPort=" + namenodeInfoPort
-        + "&chunkSizeToView=" + chunkSizeToView
-        + JspHelper.getDelegationTokenUrlParam(tokenString)
-        + JspHelper.getUrlParam(JspHelper.NAMENODE_ADDRESS, nnAddr)
-        + "&referrer=" + URLEncoder.encode(
-            req.getRequestURL() + "?" + req.getQueryString(), "UTF-8");
-    out.print("<a href=\"" + tailUrl + "\">Tail this file</a><br>");
-
-    out.print("<form action=\"/browseBlock.jsp\" method=GET>");
-    out.print("<b>Chunk size to view (in bytes, up to file's DFS block size): </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=\"hidden\" name=\"genstamp\" value=\"" + genStamp
-        + "\">");
-    out.print("<input type=\"hidden\" name=\"datanodePort\" value=\""
-        + datanodePort + "\">");
-    out.print("<input type=\"hidden\" name=\"namenodeInfoPort\" value=\""
-        + namenodeInfoPort + "\">");
-    out.print("<input type=\"hidden\" name=\"" + JspHelper.NAMENODE_ADDRESS
-        + "\" value=\"" + nnAddr + "\">");
-    out.print("<input type=\"text\" name=\"chunkSizeToView\" value="
-        + chunkSizeToView + " size=10 maxlength=10>");
-    out.print("&nbsp;&nbsp;<input type=\"submit\" name=\"submit\" value=\"Refresh\">");
-    out.print("</form>");
-    out.print("<hr>");
-    out.print("<a name=\"blockDetails\"></a>");
-    out.print("<B>Total number of blocks: " + blocks.size() + "</B><br>");
-    // generate a table and dump the info
-    out.println("\n<table>");
-    
-    String nnCanonicalName = JspHelper.canonicalize(nnAddr);
-    for (LocatedBlock cur : blocks) {
-      out.print("<tr>");
-      final String blockidstring = Long.toString(cur.getBlock().getBlockId());
-      blockSize = cur.getBlock().getNumBytes();
-      out.print("<td>" + blockidstring + ":</td>");
-      DatanodeInfo[] locs = cur.getLocations();
-      for (int j = 0; j < locs.length; j++) {
-        String datanodeAddr = locs[j].getXferAddr();
-        datanodePort = locs[j].getXferPort();
-        String blockUrl = "///" + JspHelper.Url.authority(req.getScheme(), locs[j])
-            + "/browseBlock.jsp?blockId=" + blockidstring
-            + "&blockSize=" + blockSize
-            + "&filename=" + URLEncoder.encode(filename, "UTF-8")
-            + "&datanodePort=" + datanodePort
-            + "&genstamp=" + cur.getBlock().getGenerationStamp()
-            + "&namenodeInfoPort=" + namenodeInfoPort
-            + "&chunkSizeToView=" + chunkSizeToView
-            + JspHelper.getDelegationTokenUrlParam(tokenString)
-            + JspHelper.getUrlParam(JspHelper.NAMENODE_ADDRESS, nnAddr);
-
-        String blockInfoUrl = "///" + nnCanonicalName + ":"
-            + namenodeInfoPort
-            + "/block_info_xml.jsp?blockId=" + blockidstring;
-        out.print("<td>&nbsp</td><td><a href=\"" + blockUrl + "\">"
-            + datanodeAddr + "</a></td><td>"
-            + "<a href=\"" + blockInfoUrl + "\">View Block Info</a></td>");
-      }
-      out.println("</tr>");
-    }
-    out.println("</table>");
-    out.print("<hr>");
-    out.print("<br><a href=\"///"
-        + nnCanonicalName + ":"
-        + namenodeInfoPort + "/dfshealth.jsp\">Go back to DFS home</a>");
-    dfs.close();
-  }
-
-  static void generateFileChunks(JspWriter out, HttpServletRequest req,
-                                 Configuration conf
-                                 ) throws IOException,
-                                          InterruptedException {
-    long startOffset = 0;
-    int datanodePort = 0;
-
-    final String namenodeInfoPortStr = req.getParameter("namenodeInfoPort");
-    final String nnAddr = req.getParameter(JspHelper.NAMENODE_ADDRESS);
-    if (nnAddr == null) {
-      out.print(JspHelper.NAMENODE_ADDRESS + " url param is null");
-      return;
-    }
-    final String tokenString = req.getParameter(JspHelper.DELEGATION_PARAMETER_NAME);
-    UserGroupInformation ugi = JspHelper.getUGI(req, conf);
-    int namenodeInfoPort = -1;
-    if (namenodeInfoPortStr != null)
-      namenodeInfoPort = Integer.parseInt(namenodeInfoPortStr);
-
-    final String filename = JspHelper
-        .validatePath(StringEscapeUtils.unescapeHtml(req.getParameter("filename")));
-    if (filename == null) {
-      out.print("Invalid input (filename absent)");
-      return;
-    }
-    
-    final Long blockId = JspHelper.validateLong(req.getParameter("blockId"));
-    if (blockId == null) {
-      out.print("Invalid input (blockId absent)");
-      return;
-    }
-    
-    final DFSClient dfs = getDFSClient(ugi, nnAddr, conf);
-
-    String bpid = null;
-    Token<BlockTokenIdentifier> blockToken = BlockTokenSecretManager.DUMMY_TOKEN;
-    List<LocatedBlock> blks = dfs.getNamenode().getBlockLocations(filename, 0,
-        Long.MAX_VALUE).getLocatedBlocks();
-    if (blks == null || blks.size() == 0) {
-      out.print("Can't locate file blocks");
-      dfs.close();
-      return;
-    }
-
-    boolean needBlockToken = conf.getBoolean(
-            DFSConfigKeys.DFS_BLOCK_ACCESS_TOKEN_ENABLE_KEY, 
-            DFSConfigKeys.DFS_BLOCK_ACCESS_TOKEN_ENABLE_DEFAULT);
-
-    for (int i = 0; i < blks.size(); i++) {
-      if (blks.get(i).getBlock().getBlockId() == blockId) {
-        bpid = blks.get(i).getBlock().getBlockPoolId();
-        if (needBlockToken) {
-          blockToken = blks.get(i).getBlockToken();
-        }
-        break;
-      }
-    }
-
-    final Long genStamp = JspHelper.validateLong(req.getParameter("genstamp"));
-    if (genStamp == null) {
-      out.print("Invalid input (genstamp absent)");
-      return;
-    }
-
-    long blockSize = 0;
-    final String blockSizeStr = req.getParameter("blockSize");
-    if (blockSizeStr == null) {
-      out.print("Invalid input (blockSize absent)");
-      return;
-    }
-    blockSize = Long.parseLong(blockSizeStr);
-
-    final int chunkSizeToView = JspHelper.string2ChunkSizeToView(req
-        .getParameter("chunkSizeToView"), getDefaultChunkSize(conf));
-
-    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("<h3>File: ");
-    JspHelper.printPathWithLinks(filename, out, namenodeInfoPort,
-                                 tokenString, nnAddr);
-    out.print("</h3><hr>");
-    String parent = new File(filename).getParent();
-    JspHelper.printGotoForm(out, namenodeInfoPort, tokenString, parent, nnAddr);
-    out.print("<hr>");
-    out.print("<a href=\"/browseDirectory.jsp?dir=" + URLEncoder.encode(parent, "UTF-8")
-        + "&namenodeInfoPort=" + namenodeInfoPort
-        + JspHelper.getDelegationTokenUrlParam(tokenString)
-        + JspHelper.getUrlParam(JspHelper.NAMENODE_ADDRESS, nnAddr)
-        + "\"><i>Go back to dir listing</i></a><br>");
-    out.print("<a href=\"#viewOptions\">Advanced view/download options</a><br>");
-    out.print("<hr>");
-
-    String authority = req.getServerName() + ":" + req.getServerPort();
-    String nextUrl = generateLinksForAdjacentBlock(NEXT_BLOCK, authority,
-        datanodePort, startOffset, chunkSizeToView, blockSize, blockId,
-        genStamp, dfs, filename, conf, req.getScheme(), tokenString,
-        namenodeInfoPort, nnAddr);
-    if (nextUrl != null) {
-      out.print("<a href=\"" + nextUrl + "\">View Next chunk</a>&nbsp;&nbsp;");
-    }
-
-    String prevUrl = generateLinksForAdjacentBlock(PREV_BLOCK, authority,
-        datanodePort, startOffset, chunkSizeToView, blockSize, blockId,
-        genStamp, dfs, filename, conf, req.getScheme(), tokenString,
-        namenodeInfoPort, nnAddr);
-    if (prevUrl != null) {
-      out.print("<a href=\"" + prevUrl + "\">View Prev chunk</a>&nbsp;&nbsp;");
-    }
-
-    out.print("<hr>");
-    out.print("<textarea cols=\"100\" rows=\"25\" wrap=\"virtual\" style=\"width:100%\" READONLY>");
-    try {
-      JspHelper.streamBlockInAscii(new InetSocketAddress(req.getServerName(),
-          datanodePort), bpid, blockId, blockToken, genStamp, blockSize,
-          startOffset, chunkSizeToView, out, conf, dfs.getConf(),
-          dfs, getSaslDataTransferClient(req));
-    } catch (Exception e) {
-      out.print(e);
-    }
-    out.print("</textarea>");
-    dfs.close();
-  }
-
-  private static String generateLinksForAdjacentBlock(final int direction,
-      String authority, int datanodePort, long startOffset,
-      int chunkSizeToView, long blockSize, long blockId, Long genStamp,
-      final DFSClient dfs, final String filename, final Configuration conf,
-      final String scheme, final String tokenString,
-      final int namenodeInfoPort, final String nnAddr)
-      throws AccessControlException, FileNotFoundException,
-      UnresolvedLinkException, IOException {
-
-    boolean found = false;
-    if ((direction == NEXT_BLOCK && startOffset + chunkSizeToView < blockSize)
-        || (direction == PREV_BLOCK && startOffset != 0)) {
-      // we are in the same block
-      found = true;
-
-      if (direction == NEXT_BLOCK) {
-        startOffset = startOffset + chunkSizeToView;
-      } else {
-        startOffset = Math.max(0, startOffset - chunkSizeToView);
-      }
-    } else {
-      List<LocatedBlock> blocks = dfs.getNamenode().getBlockLocations(filename, 0,
-          Long.MAX_VALUE).getLocatedBlocks();
-
-      final long curBlockId = blockId;
-      int curBlockIdx = Iterables.indexOf(blocks, new Predicate<LocatedBlock>() {
-        @Override
-        public boolean apply(LocatedBlock b) {
-          return b.getBlock().getBlockId() == curBlockId;
-        }
-      });
-      found = curBlockIdx != -1 &&
-          ((direction == NEXT_BLOCK && curBlockIdx < blocks.size() - 1)
-              || (direction == PREV_BLOCK && curBlockIdx > 0));
-
-      if (found) {
-        LocatedBlock nextBlock = blocks.get(curBlockIdx + direction);
-
-        blockId = nextBlock.getBlock().getBlockId();
-        genStamp = nextBlock.getBlock().getGenerationStamp();
-        startOffset = 0;
-        blockSize = nextBlock.getBlock().getNumBytes();
-        DatanodeInfo d = JspHelper.bestNode(nextBlock, conf);
-        datanodePort = d.getXferPort();
-        authority = JspHelper.Url.authority(scheme, d);
-      }
-    }
-
-    if (found) {
-      return "///" + authority
-          + "/browseBlock.jsp?blockId=" + blockId
-          + "&blockSize=" + blockSize
-          + "&startOffset=" + startOffset
-          + "&genstamp=" + genStamp
-          + "&filename=" + URLEncoder.encode(filename, "UTF-8")
-          + "&chunkSizeToView=" + chunkSizeToView
-          + "&datanodePort=" + datanodePort
-          + "&namenodeInfoPort=" + namenodeInfoPort
-          + JspHelper.getDelegationTokenUrlParam(tokenString)
-          + JspHelper.getUrlParam(JspHelper.NAMENODE_ADDRESS, nnAddr);
-    } else {
-      return null;
-    }
-  }
-
-  static void generateFileChunksForTail(JspWriter out, HttpServletRequest req,
-                                        Configuration conf
-                                        ) throws IOException,
-                                                 InterruptedException {
-    String referrer = null;
-    boolean noLink = false;
-    try {
-      referrer = new URL(req.getParameter("referrer")).toString();
-    } catch (IOException e) {
-      referrer = null;
-      noLink = true;
-    }
-
-    final String filename = JspHelper
-        .validatePath(StringEscapeUtils.unescapeHtml(req.getParameter("filename")));
-    if (filename == null) {
-      out.print("Invalid input (file name absent)");
-      return;
-    }
-    String tokenString = req.getParameter(JspHelper.DELEGATION_PARAMETER_NAME);
-    UserGroupInformation ugi = JspHelper.getUGI(req, conf);
-
-    String namenodeInfoPortStr = req.getParameter("namenodeInfoPort");
-    String nnAddr = StringEscapeUtils.escapeHtml(req.getParameter(JspHelper.NAMENODE_ADDRESS));
-    int namenodeInfoPort = -1;
-    if (namenodeInfoPortStr != null)
-      namenodeInfoPort = Integer.parseInt(namenodeInfoPortStr);
-
-    final int chunkSizeToView = JspHelper.string2ChunkSizeToView(req
-        .getParameter("chunkSizeToView"), getDefaultChunkSize(conf));
-
-    if (!noLink) {
-      out.print("<h3>Tail of File: ");
-      JspHelper.printPathWithLinks(filename, out, namenodeInfoPort, 
-                                   tokenString, nnAddr);
-      out.print("</h3><hr>");
-      out.print("<a href=\"" + referrer + "\">Go Back to File View</a><hr>");
-    } else {
-      out.print("<h3>" + filename + "</h3>");
-    }
-    out.print("<b>Chunk size to view (in bytes, up to file's DFS block size): </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
-        + "\">");
-    out.print("<input type=\"hidden\" name=\"namenodeInfoPort\" value=\""
-        + namenodeInfoPort + "\">");
-    out.print("<input type=\"hidden\" name=\"" + JspHelper.NAMENODE_ADDRESS
-        + "\" value=\"" + nnAddr + "\">");
-    if (!noLink)
-      out.print("<input type=\"hidden\" name=\"referrer\" value=\"" + referrer
-          + "\">");
-
-    // fetch the block from the datanode that has the last block for this file
-    final DFSClient dfs = getDFSClient(ugi, nnAddr, conf);
-    List<LocatedBlock> blocks = dfs.getNamenode().getBlockLocations(filename, 0,
-        Long.MAX_VALUE).getLocatedBlocks();
-    if (blocks == null || blocks.size() == 0) {
-      out.print("No datanodes contain blocks of file " + filename);
-      dfs.close();
-      return;
-    }
-    LocatedBlock lastBlk = blocks.get(blocks.size() - 1);
-    String poolId = lastBlk.getBlock().getBlockPoolId();
-    long blockSize = lastBlk.getBlock().getNumBytes();
-    long blockId = lastBlk.getBlock().getBlockId();
-    Token<BlockTokenIdentifier> accessToken = lastBlk.getBlockToken();
-    long genStamp = lastBlk.getBlock().getGenerationStamp();
-    DatanodeInfo chosenNode;
-    try {
-      chosenNode = JspHelper.bestNode(lastBlk, conf);
-    } catch (IOException e) {
-      out.print(e.toString());
-      dfs.close();
-      return;
-    }
-    InetSocketAddress addr = 
-      NetUtils.createSocketAddr(chosenNode.getXferAddr());
-    // view the last chunkSizeToView bytes while Tailing
-    final long startOffset = blockSize >= chunkSizeToView ? blockSize
-        - chunkSizeToView : 0;
-
-    out.print("<textarea cols=\"100\" rows=\"25\" wrap=\"virtual\" style=\"width:100%\" READONLY>");
-    JspHelper.streamBlockInAscii(addr, poolId, blockId, accessToken, genStamp,
-        blockSize, startOffset, chunkSizeToView, out, conf, dfs.getConf(),
-        dfs, getSaslDataTransferClient(req));
-    out.print("</textarea>");
-    dfs.close();
-  }
-  
   
   /** Get DFSClient for a namenode corresponding to the BPID from a datanode */
   public static DFSClient getDFSClient(final HttpServletRequest request,
@@ -683,36 +51,4 @@ public class DatanodeJspHelper {
     final String nnAddr = request.getParameter(JspHelper.NAMENODE_ADDRESS);
     return getDFSClient(ugi, nnAddr, conf);
   }
-
-  /** Return a table containing version information. */
-  public static String getVersionTable(ServletContext context) {
-    StringBuilder sb = new StringBuilder();
-    final DataNode dataNode = (DataNode) context.getAttribute("datanode");
-    sb.append("<div class='dfstable'><table>");
-    sb.append("<tr><td class='col1'>Version:</td><td>");
-    sb.append(VersionInfo.getVersion() + ", " + VersionInfo.getRevision());
-    sb.append("</td></tr>\n" + "\n  <tr><td class='col1'>Compiled:</td><td>"
-        + VersionInfo.getDate());
-    sb.append(" by " + VersionInfo.getUser() + " from "
-        + VersionInfo.getBranch());
-    if (dataNode != null) {
-      sb.append("</td></tr>\n  <tr><td class='col1'>Cluster ID:</td><td>"
-          + dataNode.getClusterId());
-    }
-    sb.append("</td></tr>\n</table></div>");
-    return sb.toString();
-  }
-
-  /**
-   * Gets the {@link SaslDataTransferClient} from the {@link DataNode} attached
-   * to the servlet context.
-   *
-   * @return SaslDataTransferClient from DataNode
-   */
-  private static SaslDataTransferClient getSaslDataTransferClient(
-      HttpServletRequest req) {
-    DataNode dataNode = (DataNode)req.getSession().getServletContext()
-      .getAttribute("datanode");
-    return dataNode.saslClient;
-  }
 }

+ 3 - 1152
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NamenodeJspHelper.java

@@ -17,615 +17,20 @@
  */
 package org.apache.hadoop.hdfs.server.namenode;
 
-import static org.apache.hadoop.hdfs.DFSUtil.percent2String;
-
-import java.io.IOException;
-import java.lang.management.ManagementFactory;
-import java.lang.management.MemoryMXBean;
-import java.lang.management.MemoryUsage;
-import java.net.InetSocketAddress;
-import java.net.URI;
-import java.net.URLEncoder;
-import java.security.PrivilegedExceptionAction;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.Iterator;
-import java.util.List;
-import java.util.Map;
-
-import javax.servlet.ServletContext;
-import javax.servlet.http.HttpServletRequest;
-import javax.servlet.http.HttpServletResponse;
-import javax.servlet.jsp.JspWriter;
-
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.ha.HAServiceProtocol.HAServiceState;
-import org.apache.hadoop.hdfs.DFSConfigKeys;
-import org.apache.hadoop.hdfs.DFSUtil;
-import org.apache.hadoop.hdfs.protocol.Block;
-import org.apache.hadoop.hdfs.protocol.DatanodeID;
 import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier;
-import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
 import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor;
-import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeManager;
-import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeStorageInfo;
-import org.apache.hadoop.hdfs.server.common.JspHelper;
-import org.apache.hadoop.hdfs.server.common.Storage;
-import org.apache.hadoop.hdfs.server.common.Storage.StorageDirectory;
-import org.apache.hadoop.hdfs.server.namenode.JournalSet.JournalAndStream;
-import org.apache.hadoop.hdfs.server.namenode.startupprogress.Phase;
-import org.apache.hadoop.hdfs.server.namenode.startupprogress.StartupProgress;
-import org.apache.hadoop.hdfs.server.namenode.startupprogress.StartupProgressView;
-import org.apache.hadoop.hdfs.server.namenode.startupprogress.Status;
-import org.apache.hadoop.hdfs.server.namenode.startupprogress.Step;
-import org.apache.hadoop.hdfs.server.namenode.startupprogress.StepType;
 import org.apache.hadoop.hdfs.server.protocol.NamenodeProtocols;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.net.NodeBase;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.token.Token;
-import org.apache.hadoop.util.ServletUtil;
-import org.apache.hadoop.util.StringUtils;
-import org.apache.hadoop.util.Time;
-import org.apache.hadoop.util.VersionInfo;
-import org.znerd.xmlenc.XMLOutputter;
 
-import com.google.common.base.Preconditions;
+import javax.servlet.http.HttpServletRequest;
+import java.io.IOException;
+import java.security.PrivilegedExceptionAction;
 
 class NamenodeJspHelper {
-  static String fraction2String(double value) {
-    return StringUtils.format("%.2f", value);
-  }
-
-  static String fraction2String(long numerator, long denominator) {
-    return fraction2String(numerator/(double)denominator);
-  }
-
-  static String getSafeModeText(FSNamesystem fsn) {
-    if (fsn == null || !fsn.isInSafeMode())
-      return "";
-    return "Safe mode is ON. <em>" + fsn.getSafeModeTip() + "</em><br>";
-  }
-
-  /**
-   * returns security mode of the cluster (namenode)
-   * @return "on" if security is on, and "off" otherwise
-   */
-  static String getSecurityModeText() {
-    if(UserGroupInformation.isSecurityEnabled()) {
-      return "<div class=\"security\">Security is <em>ON</em></div>";
-    } else {
-      return "<div class=\"security\">Security is <em>OFF</em></div>";
-    }
-  }
-
-  static String getRollingUpgradeText(FSNamesystem fsn) {
-    if (fsn == null) {
-      return "";
-    }
-
-    DatanodeManager dm = fsn.getBlockManager().getDatanodeManager();
-    Map<String, Integer> list = dm.getDatanodesSoftwareVersions();
-    if(list.size() > 1) {
-      StringBuffer status = new StringBuffer("Rolling upgrades in progress. " +
-      "There are " + list.size() + " versions of datanodes currently live: ");
-      for(Map.Entry<String, Integer> ver: list.entrySet()) {
-        status.append(ver.getKey() + "(" + ver.getValue() + "), ");
-      }
-      return status.substring(0, status.length()-2);
-    }
-    return "";
-  }
-
-  static String getInodeLimitText(FSNamesystem fsn) {
-    if (fsn == null) {
-      return "";
-    }
-
-    long inodes = fsn.dir.totalInodes();
-    long blocks = fsn.getBlocksTotal();
-    long maxobjects = fsn.getMaxObjects();
-
-    MemoryMXBean mem = ManagementFactory.getMemoryMXBean();
-    MemoryUsage heap = mem.getHeapMemoryUsage();
-    long totalMemory = heap.getUsed();
-    long maxMemory = heap.getMax();
-    long commitedMemory = heap.getCommitted();
-    
-    MemoryUsage nonHeap = mem.getNonHeapMemoryUsage();
-    long totalNonHeap = nonHeap.getUsed();
-    long maxNonHeap = nonHeap.getMax();
-    long commitedNonHeap = nonHeap.getCommitted();
-
-    long used = (totalMemory * 100) / commitedMemory;
-    long usedNonHeap = (totalNonHeap * 100) / commitedNonHeap;
-
-    String str = "<div>" + inodes + " files and directories, " + blocks + " blocks = "
-        + (inodes + blocks) + " total";
-    if (maxobjects != 0) {
-      long pct = ((inodes + blocks) * 100) / maxobjects;
-      str += " / " + maxobjects + " (" + pct + "%)";
-    }
-    str += ".</div>";
-    str += "<div>Heap Memory used " + StringUtils.byteDesc(totalMemory) + " is "
-        + " " + used + "% of Commited Heap Memory "
-        + StringUtils.byteDesc(commitedMemory)
-        + ". Max Heap Memory is " + StringUtils.byteDesc(maxMemory) +
-        ". </div>";
-    str += "<div>Non Heap Memory used " + StringUtils.byteDesc(totalNonHeap) + " is"
-        + " " + usedNonHeap + "% of " + " Commited Non Heap Memory "
-        + StringUtils.byteDesc(commitedNonHeap) + ". Max Non Heap Memory is "
-        + StringUtils.byteDesc(maxNonHeap) + ".</div>";
-    return str;
-  }
-
-  /** Return a table containing version information. */
-  static String getVersionTable(FSNamesystem fsn) {
-    StringBuilder sb = new StringBuilder();
-    sb.append("<div class='dfstable'><table>");
-    if (fsn != null) {
-      sb.append("\n  <tr><td class='col1'>Started:</td><td>" + fsn.getStartTime());
-    }
-    sb.append("</td></tr>\n" + "\n  <tr><td class='col1'>Version:</td><td>");
-    sb.append(VersionInfo.getVersion() + ", " + VersionInfo.getRevision());
-    sb.append("</td></tr>\n" + "\n  <tr><td class='col1'>Compiled:</td><td>" + VersionInfo.getDate());
-    sb.append(" by " + VersionInfo.getUser() + " from " + VersionInfo.getBranch());
-    if (fsn != null) {
-      sb.append("</td></tr>\n  <tr><td class='col1'>Cluster ID:</td><td>" + fsn.getClusterId());
-      sb.append("</td></tr>\n  <tr><td class='col1'>Block Pool ID:</td><td>" + fsn.getBlockPoolId());
-    }
-    sb.append("</td></tr>\n</table></div>");
-    return sb.toString();
-  }
-
-  /**
-   * Generate warning text if there are corrupt files.
-   * @return a warning if files are corrupt, otherwise return an empty string.
-   */
-  static String getCorruptFilesWarning(FSNamesystem fsn) {
-    if (fsn == null) {
-      return "";
-    }
-
-    long missingBlocks = fsn.getMissingBlocksCount();
-    if (missingBlocks > 0) {
-      StringBuilder result = new StringBuilder();
-
-      // Warning class is typically displayed in RED.
-      result.append("<div>"); // opening tag of outer <div>.
-      result.append("<a class=\"warning\" href=\"/corrupt_files.jsp\" title=\"List corrupt files\">\n");
-      result.append("<b>WARNING : There are " + missingBlocks
-          + " missing blocks. Please check the logs or run fsck in order to identify the missing blocks.</b>");
-      result.append("</a>");
-
-      result.append("<div class=\"small\">See the Hadoop FAQ for common causes and potential solutions.</div>");
-      result.append("</div>\n"); // closing tag of outer <div>.
-
-      return result.toString();
-    }
-    return "";
-  }
-
-  static void generateSnapshotReport(JspWriter out, FSNamesystem fsn)
-      throws IOException {
-    if (fsn == null) {
-      return;
-    }
-    out.println("<div id=\"snapshotstats\"><div class=\"dfstable\">"
-        + "<table class=\"storage\" title=\"Snapshot Summary\">\n"
-        + "<thead><tr><td><b>Snapshottable directories</b></td>"
-        + "<td><b>Snapshotted directories</b></td></tr></thead>");
-
-    out.println(String.format("<td>%d</td><td>%d</td>", fsn.getNumSnapshottableDirs(), fsn.getNumSnapshots()));
-    out.println("</table></div></div>");
-  }
-
-  static class HealthJsp {
-    private int rowNum = 0;
-    private int colNum = 0;
-    private String sorterField = null;
-    private String sorterOrder = null;
-
-    private String rowTxt() {
-      colNum = 0;
-      return "<tr class=\"" + (((rowNum++) % 2 == 0) ? "rowNormal" : "rowAlt")
-          + "\"> ";
-    }
-
-    private String colTxt() {
-      return "<td id=\"col" + ++colNum + "\"> ";
-    }
-
-    private String colTxt(String title) {
-      return "<td id=\"col" + ++colNum + "\" title=\"" + title + "\"> ";
-    }
-
-    private void counterReset() {
-      colNum = 0;
-      rowNum = 0;
-    }
-
-    void generateConfReport(JspWriter out, NameNode nn,
-        HttpServletRequest request) throws IOException {
-      FSNamesystem fsn = nn.getNamesystem();
-      if (fsn == null) {
-        return;
-      }
-      FSImage fsImage = fsn.getFSImage();
-      List<Storage.StorageDirectory> removedStorageDirs 
-        = fsImage.getStorage().getRemovedStorageDirs();
-
-      // FS Image storage configuration
-      out.print("<h3> " + nn.getRole() + " Storage: </h3>");
-      out.print("<div class=\"dfstable\"> <table class=\"storage\" title=\"NameNode Storage\">\n"
-              + "<thead><tr><td><b>Storage Directory</b></td><td><b>Type</b></td><td><b>State</b></td></tr></thead>");
-
-      StorageDirectory st = null;
-      for (Iterator<StorageDirectory> it
-             = fsImage.getStorage().dirIterator(); it.hasNext();) {
-        st = it.next();
-        String dir = "" + st.getRoot();
-        String type = "" + st.getStorageDirType();
-        out.print("<tr><td>" + dir + "</td><td>" + type
-            + "</td><td>Active</td></tr>");
-      }
-
-      long storageDirsSize = removedStorageDirs.size();
-      for (int i = 0; i < storageDirsSize; i++) {
-        st = removedStorageDirs.get(i);
-        String dir = "" + st.getRoot();
-        String type = "" + st.getStorageDirType();
-        out.print("<tr><td>" + dir + "</td><td>" + type
-            + "</td><td><span class=\"failed\">Failed</span></td></tr>");
-      }
-
-      out.print("</table></div>\n");
-    }
-    
-    /**
-     * Generate an HTML report containing the current status of the HDFS
-     * journals.
-     */
-    void generateJournalReport(JspWriter out, NameNode nn,
-        HttpServletRequest request) throws IOException {
-      if (nn.getNamesystem() == null) {
-        return;
-      }
-      FSEditLog log = nn.getFSImage().getEditLog();
-      Preconditions.checkArgument(log != null, "no edit log set in %s", nn);
-      
-      out.println("<h3> " + nn.getRole() + " Journal Status: </h3>");
-
-      out.println("<b>Current transaction ID:</b> " +
-          nn.getFSImage().getLastAppliedOrWrittenTxId() + "<br/>");
-      
-      
-      boolean openForWrite = log.isOpenForWrite();
-      
-      out.println("<div class=\"dfstable\">");
-      out.println("<table class=\"storage\" title=\"NameNode Journals\">\n"
-              + "<thead><tr><td><b>Journal Manager</b></td><td><b>State</b></td></tr></thead>");
-      for (JournalAndStream jas : log.getJournals()) {
-        out.print("<tr>");
-        out.print("<td>");
-        
-        /**
-         * Insert a line break every 3 journal nodes to avoid a very wide line.
-         */  
-        JournalManager manager = jas.getManager();
-        if (null != manager) {
-          String[] managers = manager.toString().split(",");
-
-          for (int i = 0; i < managers.length; ++i) {
-            out.print(managers[i]);
-
-            if (i < managers.length - 1) {
-              out.print(",");
-            }
-
-            if ((i+1) % 3 == 0) {
-              out.print("<br/>");
-            }
-          }
-
-          if (jas.isRequired()) {
-            out.print(" [required]");
-          }
-        }
-        out.print("</td><td>");
-        
-        if (jas.isDisabled()) {
-          out.print("<span class=\"failed\">Failed</span>");
-        } else if (openForWrite) {
-          EditLogOutputStream elos = jas.getCurrentStream();
-          if (elos != null) {
-            out.println(elos.generateReport());
-          } else {
-            out.println("not currently writing");
-          }
-        } else {
-          out.println("open for read");
-        }
-        out.println("</td></tr>");
-      }
-      
-      out.println("</table></div>");
-    }
-
-    void generateHealthReport(JspWriter out, NameNode nn,
-        HttpServletRequest request) throws IOException {
-      FSNamesystem fsn = nn.getNamesystem();
-      if (fsn == null) {
-        return;
-      }
-      final DatanodeManager dm = fsn.getBlockManager().getDatanodeManager();
-      final List<DatanodeDescriptor> live = new ArrayList<DatanodeDescriptor>();
-      final List<DatanodeDescriptor> dead = new ArrayList<DatanodeDescriptor>();
-      dm.fetchDatanodes(live, dead, true);
-
-      int liveDecommissioned = 0;
-      for (DatanodeDescriptor d : live) {
-        liveDecommissioned += d.isDecommissioned() ? 1 : 0;
-      }
-
-      int deadDecommissioned = 0;
-      for (DatanodeDescriptor d : dead) {
-        deadDecommissioned += d.isDecommissioned() ? 1 : 0;
-      }
-      
-      final List<DatanodeDescriptor> decommissioning = dm.getDecommissioningNodes();
-
-      sorterField = request.getParameter("sorter/field");
-      sorterOrder = request.getParameter("sorter/order");
-      if (sorterField == null)
-        sorterField = "name";
-      if (sorterOrder == null)
-        sorterOrder = "ASC";
-
-      // Find out common suffix. Should this be before or after the sort?
-      String port_suffix = null;
-      if (live.size() > 0) {
-        String name = live.get(0).getXferAddr();
-        int idx = name.indexOf(':');
-        if (idx > 0) {
-          port_suffix = name.substring(idx);
-        }
-
-        for (int i = 1; port_suffix != null && i < live.size(); i++) {
-          if (live.get(i).getXferAddr().endsWith(port_suffix) == false) {
-            port_suffix = null;
-            break;
-          }
-        }
-      }
-
-      counterReset();
-      long[] fsnStats = fsn.getStats();
-      long total = fsnStats[0];
-      long remaining = fsnStats[2];
-      long used = fsnStats[1];
-      long nonDFS = total - remaining - used;
-      nonDFS = nonDFS < 0 ? 0 : nonDFS;
-      float percentUsed = DFSUtil.getPercentUsed(used, total);
-      float percentRemaining = DFSUtil.getPercentRemaining(remaining, total);
-      float median = 0;
-      float max = 0;
-      float min = 0;
-      float dev = 0;
-      
-      if (live.size() > 0) {
-        float totalDfsUsed = 0;
-        float[] usages = new float[live.size()];
-        int i = 0;
-        for (DatanodeDescriptor dn : live) {
-          usages[i++] = dn.getDfsUsedPercent();
-          totalDfsUsed += dn.getDfsUsedPercent();
-        }
-        totalDfsUsed /= live.size();
-        Arrays.sort(usages);
-        median = usages[usages.length/2];
-        max = usages[usages.length - 1];
-        min = usages[0];
-        
-        for (i = 0; i < usages.length; i++) {
-          dev += (usages[i] - totalDfsUsed) * (usages[i] - totalDfsUsed);
-        }
-        dev = (float) Math.sqrt(dev/usages.length);
-      }
-
-      long bpUsed = fsnStats[6];
-      float percentBpUsed = DFSUtil.getPercentUsed(bpUsed, total);
-
-      // don't show under-replicated/missing blocks or corrupt files for SBN
-      // since the standby namenode doesn't compute replication queues 
-      String underReplicatedBlocks = "";
-      if (nn.getServiceState() == HAServiceState.ACTIVE) {
-    	  underReplicatedBlocks = rowTxt() 
-              + colTxt("Excludes missing blocks.")
-              + "Number of Under-Replicated Blocks" + colTxt() + ":" + colTxt()
-              + fsn.getBlockManager().getUnderReplicatedNotMissingBlocks(); 
-      }
-      out.print("<div class=\"dfstable\"> <table>\n" + rowTxt() + colTxt()
-          + "Configured Capacity" + colTxt() + ":" + colTxt()
-          + StringUtils.byteDesc(total) + rowTxt() + colTxt() + "DFS Used"
-          + colTxt() + ":" + colTxt() + StringUtils.byteDesc(used) + rowTxt()
-          + colTxt() + "Non DFS Used" + colTxt() + ":" + colTxt()
-          + StringUtils.byteDesc(nonDFS) + rowTxt() + colTxt()
-          + "DFS Remaining" + colTxt() + ":" + colTxt()
-          + StringUtils.byteDesc(remaining) + rowTxt() + colTxt() + "DFS Used%"
-          + colTxt() + ":" + colTxt()
-          + percent2String(percentUsed) + rowTxt()
-          + colTxt() + "DFS Remaining%" + colTxt() + ":" + colTxt()
-          + percent2String(percentRemaining)
-          + rowTxt() + colTxt() + "Block Pool Used" + colTxt() + ":" + colTxt()
-          + StringUtils.byteDesc(bpUsed) + rowTxt()
-          + colTxt() + "Block Pool Used%"+ colTxt() + ":" + colTxt()
-          + percent2String(percentBpUsed) 
-          + rowTxt() + colTxt() + "DataNodes usages" + colTxt() + ":" + colTxt()
-          + "Min %" + colTxt() + "Median %" + colTxt() + "Max %" + colTxt()
-          + "stdev %" + rowTxt() + colTxt() + colTxt() + colTxt()
-          + percent2String(min)
-          + colTxt() + percent2String(median)
-          + colTxt() + percent2String(max)
-          + colTxt() + percent2String(dev)
-          + rowTxt() + colTxt()
-          + "<a href=\"dfsnodelist.jsp?whatNodes=LIVE\">Live Nodes</a> "
-          + colTxt() + ":" + colTxt() + live.size()
-          + " (Decommissioned: " + liveDecommissioned + ")"
-          + rowTxt() + colTxt()
-          + "<a href=\"dfsnodelist.jsp?whatNodes=DEAD\">Dead Nodes</a> "
-          + colTxt() + ":" + colTxt() + dead.size() 
-          + " (Decommissioned: " + deadDecommissioned + ")"
-          + rowTxt() + colTxt()
-          + "<a href=\"dfsnodelist.jsp?whatNodes=DECOMMISSIONING\">"
-          + "Decommissioning Nodes</a> "
-          + colTxt() + ":" + colTxt() + decommissioning.size()
-          + underReplicatedBlocks
-          + "</table></div><br>\n");
-
-      if (live.isEmpty() && dead.isEmpty()) {
-        out.print("There are no datanodes in the cluster.");
-      }
-    }
-
-    /**
-     * Generates the Startup Progress report.
-     * 
-     * @param out JspWriter to receive output
-     * @param prog StartupProgress tracking NameNode startup progress
-     * @throws IOException thrown if there is an I/O error
-     */
-    void generateStartupProgress(JspWriter out, StartupProgress prog)
-        throws IOException {
-      StartupProgressView view = prog.createView();
-      FormattedWriter fout = new FormattedWriter(out);
-      fout.println("<div id=\"startupprogress\">");
-      fout.println("<div><span>Elapsed Time:</span> %s</div>",
-        StringUtils.formatTime(view.getElapsedTime()));
-      fout.println("<div><span>Percent Complete:</span> %s</div>",
-        StringUtils.formatPercent(view.getPercentComplete(), 2));
-      fout.println("<table>");
-      fout.println("<tr>");
-      fout.println("<th>Phase</th>");
-      fout.println("<th>Completion</th>");
-      fout.println("<th>Elapsed Time</th>");
-      fout.println("</tr>");
-      for (Phase phase: view.getPhases()) {
-        final String timeClass;
-        Status status = view.getStatus(phase);
-        if (status == Status.PENDING) {
-          timeClass = "later";
-        } else if (status == Status.RUNNING) {
-          timeClass = "current";
-        } else {
-          timeClass = "prior";
-        }
-
-        fout.println("<tr class=\"phase %s\">", timeClass);
-        printPhase(fout, view, phase);
-        fout.println("</tr>");
-
-        for (Step step: view.getSteps(phase)) {
-          fout.println("<tr class=\"step %s\">", timeClass);
-          printStep(fout, view, phase, step);
-          fout.println("</tr>");
-        }
-      }
-      fout.println("</table>");
-      fout.println("</div>");
-    }
-
-    /**
-     * Prints one line of content for a phase in the Startup Progress report.
-     * 
-     * @param fout FormattedWriter to receive output
-     * @param view StartupProgressView containing information to print
-     * @param phase Phase to print
-     * @throws IOException thrown if there is an I/O error
-     */
-    private void printPhase(FormattedWriter fout, StartupProgressView view,
-        Phase phase) throws IOException {
-      StringBuilder phaseLine = new StringBuilder();
-      phaseLine.append(phase.getDescription());
-      String file = view.getFile(phase);
-      if (file != null) {
-        phaseLine.append(" ").append(file);
-      }
-      long size = view.getSize(phase);
-      if (size != Long.MIN_VALUE) {
-        phaseLine.append(" (").append(StringUtils.byteDesc(size)).append(")");
-      }
-      fout.println("<td class=\"startupdesc\">%s</td>", phaseLine.toString());
-      fout.println("<td>%s</td>", StringUtils.formatPercent(
-        view.getPercentComplete(phase), 2));
-      fout.println("<td>%s</td>", view.getStatus(phase) == Status.PENDING ? "" :
-        StringUtils.formatTime(view.getElapsedTime(phase)));
-    }
-
-    /**
-     * Prints one line of content for a step in the Startup Progress report.
-     * 
-     * @param fout FormattedWriter to receive output
-     * @param view StartupProgressView containing information to print
-     * @param phase Phase to print
-     * @param step Step to print
-     * @throws IOException thrown if there is an I/O error
-     */
-    private void printStep(FormattedWriter fout, StartupProgressView view,
-        Phase phase, Step step) throws IOException {
-      StringBuilder stepLine = new StringBuilder();
-      String file = step.getFile();
-      if (file != null) {
-        stepLine.append(file);
-      }
-      long size = step.getSize();
-      if (size != Long.MIN_VALUE) {
-        stepLine.append(" (").append(StringUtils.byteDesc(size)).append(")");
-      }
-      StepType type = step.getType();
-      if (type != null) {
-        stepLine.append(" ").append(type.getDescription());
-      }
-
-      fout.println("<td class=\"startupdesc\">%s (%d/%d)</td>",
-        stepLine.toString(), view.getCount(phase, step),
-        view.getTotal(phase, step));
-      fout.println("<td>%s</td>", StringUtils.formatPercent(
-        view.getPercentComplete(phase), 2));
-      fout.println("<td>%s</td>", view.getStatus(phase) == Status.PENDING ? "" :
-        StringUtils.formatTime(view.getElapsedTime(phase)));
-    }
-
-    /**
-     * JspWriter wrapper that helps simplify printing formatted lines.
-     */
-    private static class FormattedWriter {
-      private final JspWriter out;
-
-      /**
-       * Creates a new FormattedWriter that delegates to the given JspWriter.
-       * 
-       * @param out JspWriter to wrap
-       */
-      FormattedWriter(JspWriter out) {
-        this.out = out;
-      }
-
-      /**
-       * Prints one formatted line, followed by line terminator, using the
-       * English locale.
-       * 
-       * @param format String format
-       * @param args Object... any number of arguments to match format
-       * @throws IOException thrown if there is an I/O error
-       */
-      void println(String format, Object... args) throws IOException {
-        out.println(StringUtils.format(format, args));
-      }
-    }
-  }
 
   static String getDelegationToken(final NamenodeProtocols nn,
       HttpServletRequest request, Configuration conf,
@@ -646,558 +51,4 @@ class NamenodeJspHelper {
         ).getDatanodeManager().getNetworkTopology().chooseRandom(
         NodeBase.ROOT);
   }
-  
-  static void redirectToRandomDataNode(ServletContext context,
-      HttpServletRequest request, HttpServletResponse resp) throws IOException,
-      InterruptedException {
-    final NameNode nn = NameNodeHttpServer.getNameNodeFromContext(context);
-    final Configuration conf = (Configuration) context
-        .getAttribute(JspHelper.CURRENT_CONF);
-    // We can't redirect if there isn't a DN to redirect to.
-    // Lets instead show a proper error message.
-    FSNamesystem fsn = nn.getNamesystem();
-
-    DatanodeID datanode = null;
-    if (fsn != null && fsn.getNumLiveDataNodes() >= 1) {
-      datanode = getRandomDatanode(nn);
-    }
-
-    if (datanode == null) {
-      throw new IOException("Can't browse the DFS since there are no " +
-          "live nodes available to redirect to.");
-    }
-
-    UserGroupInformation ugi = JspHelper.getUGI(context, request, conf);
-    // if the user is defined, get a delegation token and stringify it
-    String tokenString = getDelegationToken(
-        nn.getRpcServer(), request, conf, ugi);
-
-    InetSocketAddress rpcAddr = nn.getNameNodeAddress();
-    String rpcHost = rpcAddr.getAddress().isAnyLocalAddress()
-      ? URI.create(request.getRequestURL().toString()).getHost()
-      : rpcAddr.getAddress().getHostAddress();
-    String addr = rpcHost + ":" + rpcAddr.getPort();
-
-    final String redirectLocation =
-        JspHelper.Url.url(request.getScheme(), datanode)
-        + "/browseDirectory.jsp?namenodeInfoPort="
-        + request.getServerPort() + "&dir=/"
-        + (tokenString == null ? "" :
-           JspHelper.getDelegationTokenUrlParam(tokenString))
-        + JspHelper.getUrlParam(JspHelper.NAMENODE_ADDRESS, addr);
-
-    resp.sendRedirect(redirectLocation);
-  }
-
-  /**
-   * Returns a descriptive label for the running NameNode.  If the NameNode has
-   * initialized to the point of running its RPC server, then this label consists
-   * of the host and port of the RPC server.  Otherwise, the label is a message
-   * stating that the NameNode is still initializing.
-   * 
-   * @param nn NameNode to describe
-   * @return String NameNode label
-   */
-  static String getNameNodeLabel(NameNode nn) {
-    return nn.getRpcServer() != null ? nn.getNameNodeAddressHostPortString() :
-      "initializing";
-  }
-
-  static class NodeListJsp {
-    private int rowNum = 0;
-
-    private long diskBytes = 1024 * 1024 * 1024;
-    private String diskByteStr = "GB";
-
-    private String sorterField = null;
-    private String sorterOrder = null;
-
-    private String whatNodes = "LIVE";
-
-    private String rowTxt() {
-      return "<tr class=\"" + (((rowNum++) % 2 == 0) ? "rowNormal" : "rowAlt")
-          + "\"> ";
-    }
-
-    private void counterReset() {
-      rowNum = 0;
-    }
-
-    private String nodeHeaderStr(String name) {
-      String ret = "class=header";
-      String order = "ASC";
-      if (name.equals(sorterField)) {
-        ret += sorterOrder;
-        if (sorterOrder.equals("ASC"))
-          order = "DSC";
-      }
-      ret += " onClick=\"window.document.location="
-          + "'/dfsnodelist.jsp?whatNodes=" + whatNodes + "&sorter/field="
-          + name + "&sorter/order=" + order
-          + "'\" title=\"sort on this column\"";
-
-      return ret;
-    }
-
-    private void generateNodeDataHeader(JspWriter out, DatanodeDescriptor d,
-        String suffix, boolean alive, int nnInfoPort, String nnaddr, String scheme)
-        throws IOException {
-      // from nn_browsedfscontent.jsp:
-      String url = "///" + JspHelper.Url.authority(scheme, d)
-          + "/browseDirectory.jsp?namenodeInfoPort=" + nnInfoPort + "&dir="
-          + URLEncoder.encode("/", "UTF-8")
-          + JspHelper.getUrlParam(JspHelper.NAMENODE_ADDRESS, nnaddr);
-
-      String name = d.getXferAddrWithHostname();
-      if (!name.matches("\\d+\\.\\d+.\\d+\\.\\d+.*"))
-        name = name.replaceAll("\\.[^.:]*", "");
-      int idx = (suffix != null && name.endsWith(suffix)) ? name
-          .indexOf(suffix) : -1;
-
-      out.print(rowTxt() + "<td class=\"name\"> <a title=\"" + url
-          + "\" href=\"" + url + "\">"
-          + ((idx > 0) ? name.substring(0, idx) : name) + "</a>"
-          + ((alive) ? "" : "\n") + "<td class=\"address\">" + d.getXferAddr());
-    }
-
-    void generateDecommissioningNodeData(JspWriter out, DatanodeDescriptor d,
-        String suffix, boolean alive, int nnInfoPort, String nnaddr, String scheme)
-        throws IOException {
-      generateNodeDataHeader(out, d, suffix, alive, nnInfoPort, nnaddr, scheme);
-      if (!alive) {
-        return;
-      }
-
-      long decommRequestTime = d.decommissioningStatus.getStartTime();
-      long timestamp = d.getLastUpdate();
-      long currentTime = Time.now();
-      long hoursSinceDecommStarted = (currentTime - decommRequestTime)/3600000;
-      long remainderMinutes = ((currentTime - decommRequestTime)/60000) % 60;
-      out.print("<td class=\"lastcontact\"> "
-          + ((currentTime - timestamp) / 1000)
-          + "<td class=\"underreplicatedblocks\">"
-          + d.decommissioningStatus.getUnderReplicatedBlocks()
-          + "<td class=\"blockswithonlydecommissioningreplicas\">"
-          + d.decommissioningStatus.getDecommissionOnlyReplicas() 
-          + "<td class=\"underrepblocksinfilesunderconstruction\">"
-          + d.decommissioningStatus.getUnderReplicatedInOpenFiles()
-          + "<td class=\"timesincedecommissionrequest\">"
-          + hoursSinceDecommStarted + " hrs " + remainderMinutes + " mins"
-          + "\n");
-    }
-    
-    void generateNodeData(JspWriter out, DatanodeDescriptor d, String suffix,
-        boolean alive, int nnInfoPort, String nnaddr, String scheme) throws IOException {
-      /*
-       * Say the datanode is dn1.hadoop.apache.org with ip 192.168.0.5 we use:
-       * 1) d.getHostName():d.getPort() to display. Domain and port are stripped
-       *    if they are common across the nodes. i.e. "dn1" 
-       * 2) d.getHostName():d.getInfoPort() for url and title.
-       *    i.e. "http://dn1.hadoop.apache.org:50075/..."
-       * 3) d.getXferAddr() for "Transferring Address". i.e. "192.168.0.5:50010"
-       * Note that "d.getHost():d.getPort()" is what DFS clients use to
-       * interact with datanodes.
-       */
-
-      generateNodeDataHeader(out, d, suffix, alive, nnInfoPort, nnaddr, scheme);
-      if (!alive) {
-        out.print("<td class=\"decommissioned\"> " + 
-            d.isDecommissioned() + "\n");
-        return;
-      }
-
-      long c = d.getCapacity();
-      long u = d.getDfsUsed();
-      long nu = d.getNonDfsUsed();
-      long r = d.getRemaining();
-      final double percentUsedValue = d.getDfsUsedPercent();
-      String percentUsed = fraction2String(percentUsedValue);
-      String percentRemaining = fraction2String(d.getRemainingPercent());
-
-      String adminState = d.getAdminState().toString();
-
-      long timestamp = d.getLastUpdate();
-      long currentTime = Time.now();
-      
-      long bpUsed = d.getBlockPoolUsed();
-      String percentBpUsed = fraction2String(d.getBlockPoolUsedPercent());
-
-      out.print("<td class=\"lastcontact\"> "
-          + ((currentTime - timestamp) / 1000)
-          + "<td class=\"adminstate\">"
-          + adminState
-          + "<td align=\"right\" class=\"capacity\">"
-          + fraction2String(c, diskBytes)
-          + "<td align=\"right\" class=\"used\">"
-          + fraction2String(u, diskBytes)
-          + "<td align=\"right\" class=\"nondfsused\">"
-          + fraction2String(nu, diskBytes)
-          + "<td align=\"right\" class=\"remaining\">"
-          + fraction2String(r, diskBytes)
-          + "<td align=\"right\" class=\"pcused\">"
-          + percentUsed
-          + "<td class=\"pcused\">"
-          + ServletUtil.percentageGraph((int)percentUsedValue, 100) 
-          + "<td align=\"right\" class=\"pcremaining\">"
-          + percentRemaining 
-          + "<td title=" + "\"blocks scheduled : "
-          + d.getBlocksScheduled() + "\" class=\"blocks\">" + d.numBlocks()+"\n"
-          + "<td align=\"right\" class=\"bpused\">"
-          + fraction2String(bpUsed, diskBytes)
-          + "<td align=\"right\" class=\"pcbpused\">"
-          + percentBpUsed
-          + "<td align=\"right\" class=\"volfails\">"
-          + d.getVolumeFailures()
-          + "<td align=\"right\" class=\"version\">"
-          + d.getSoftwareVersion() + "\n");
-    }
-
-    void generateNodesList(ServletContext context, JspWriter out,
-        HttpServletRequest request) throws IOException {
-      final NameNode nn = NameNodeHttpServer.getNameNodeFromContext(context);
-      final FSNamesystem ns = nn.getNamesystem();
-      if (ns == null) {
-        return;
-      }
-      final DatanodeManager dm = ns.getBlockManager().getDatanodeManager();
-
-      final List<DatanodeDescriptor> live = new ArrayList<DatanodeDescriptor>();
-      final List<DatanodeDescriptor> dead = new ArrayList<DatanodeDescriptor>();
-      dm.fetchDatanodes(live, dead, true);
-
-      String nnaddr = nn.getServiceRpcAddress().getAddress().getHostName() + ":"
-          + nn.getServiceRpcAddress().getPort();
-
-      whatNodes = request.getParameter("whatNodes"); // show only live or only
-                                                     // dead nodes
-      if (null == whatNodes || whatNodes.isEmpty()) {
-        out.print("Invalid input");
-        return;
-      }
-      sorterField = request.getParameter("sorter/field");
-      sorterOrder = request.getParameter("sorter/order");
-      if (sorterField == null)
-        sorterField = "name";
-      if (sorterOrder == null)
-        sorterOrder = "ASC";
-
-      JspHelper.sortNodeList(live, sorterField, sorterOrder);
-
-      // Find out common suffix. Should this be before or after the sort?
-      String port_suffix = null;
-      if (live.size() > 0) {
-        String name = live.get(0).getXferAddr();
-        int idx = name.indexOf(':');
-        if (idx > 0) {
-          port_suffix = name.substring(idx);
-        }
-
-        for (int i = 1; port_suffix != null && i < live.size(); i++) {
-          if (live.get(i).getXferAddr().endsWith(port_suffix) == false) {
-            port_suffix = null;
-            break;
-          }
-        }
-      }
-
-      counterReset();
-
-      if (live.isEmpty() && dead.isEmpty()) {
-        out.print("There are no datanodes in the cluster");
-      } else {
-
-        int nnInfoPort = request.getServerPort();
-        out.print("<div id=\"dfsnodetable\"> ");
-        if (whatNodes.equals("LIVE")) {
-          out.print("<a name=\"LiveNodes\" id=\"title\">" + "Live Datanodes : "
-              + live.size() + "</a>"
-              + "<br><br>\n<table class=\"nodes\">\n");
-
-          counterReset();
-
-          if (live.size() > 0) {
-            if (live.get(0).getCapacity() > 1024 * diskBytes) {
-              diskBytes *= 1024;
-              diskByteStr = "TB";
-            }
-
-            out.print("<tr class=\"headerRow\"> <th " + nodeHeaderStr("name")
-                + "> Node <th " + nodeHeaderStr("address")
-                + "> Transferring<br>Address <th "
-                + nodeHeaderStr("lastcontact")
-                + "> Last <br>Contact <th " + nodeHeaderStr("adminstate")
-                + "> Admin State <th " + nodeHeaderStr("capacity")
-                + "> Configured <br>Capacity (" + diskByteStr + ") <th "
-                + nodeHeaderStr("used") + "> Used <br>(" + diskByteStr
-                + ") <th " + nodeHeaderStr("nondfsused")
-                + "> Non DFS <br>Used (" + diskByteStr + ") <th "
-                + nodeHeaderStr("remaining") + "> Remaining <br>("
-                + diskByteStr + ") <th " + nodeHeaderStr("pcused")
-                + "> Used <br>(%) <th " + nodeHeaderStr("pcused")
-                + "> Used <br>(%) <th " + nodeHeaderStr("pcremaining")
-                + "> Remaining <br>(%) <th " + nodeHeaderStr("blocks")
-                + "> Blocks <th "
-                + nodeHeaderStr("bpused") + "> Block Pool<br>Used (" 
-                + diskByteStr + ") <th "
-                + nodeHeaderStr("pcbpused")
-                + "> Block Pool<br>Used (%)" + " <th "
-                + nodeHeaderStr("volfails")
-                +"> Failed Volumes <th "
-                + nodeHeaderStr("versionString")
-                +"> Version\n");
-
-            JspHelper.sortNodeList(live, sorterField, sorterOrder);
-            for (int i = 0; i < live.size(); i++) {
-              generateNodeData(out, live.get(i), port_suffix, true, nnInfoPort,
-                  nnaddr, request.getScheme());
-            }
-          }
-          out.print("</table>\n");
-        } else if (whatNodes.equals("DEAD")) {
-
-          out.print("<br> <a name=\"DeadNodes\" id=\"title\"> "
-              + " Dead Datanodes : " + dead.size() + "</a><br><br>\n");
-
-          if (dead.size() > 0) {
-            out.print("<table border=1 cellspacing=0> <tr id=\"row1\"> "
-                + "<th " + nodeHeaderStr("node")
-                + "> Node <th " + nodeHeaderStr("address")
-                + "> Transferring<br>Address <th "
-                + nodeHeaderStr("decommissioned")
-                + "> Decommissioned\n");
-
-            JspHelper.sortNodeList(dead, sorterField, sorterOrder);
-            for (int i = 0; i < dead.size(); i++) {
-              generateNodeData(out, dead.get(i), port_suffix, false,
-                  nnInfoPort, nnaddr, request.getScheme());
-            }
-
-            out.print("</table>\n");
-          }
-        } else if (whatNodes.equals("DECOMMISSIONING")) {
-          // Decommissioning Nodes
-          final List<DatanodeDescriptor> decommissioning = dm.getDecommissioningNodes();
-          out.print("<br> <a name=\"DecommissioningNodes\" id=\"title\"> "
-              + " Decommissioning Datanodes : " + decommissioning.size()
-              + "</a><br><br>\n");
-          if (decommissioning.size() > 0) {
-            out.print("<table border=1 cellspacing=0> <tr class=\"headRow\"> "
-                + "<th " + nodeHeaderStr("name") 
-                + "> Node <th " + nodeHeaderStr("address")
-                + "> Transferring<br>Address <th "
-                + nodeHeaderStr("lastcontact")
-                + "> Last <br>Contact <th "
-                + nodeHeaderStr("underreplicatedblocks")
-                + "> Under Replicated Blocks <th "
-                + nodeHeaderStr("blockswithonlydecommissioningreplicas")
-                + "> Blocks With No <br> Live Replicas <th "
-                + nodeHeaderStr("underrepblocksinfilesunderconstruction")
-                + "> Under Replicated Blocks <br> In Files Under Construction" 
-                + " <th " + nodeHeaderStr("timesincedecommissionrequest")
-                + "> Time Since Decommissioning Started"
-                );
-
-            JspHelper.sortNodeList(decommissioning, "name", "ASC");
-            for (int i = 0; i < decommissioning.size(); i++) {
-              generateDecommissioningNodeData(out, decommissioning.get(i),
-                  port_suffix, true, nnInfoPort, nnaddr, request.getScheme());
-            }
-            out.print("</table>\n");
-          }
-        } else {
-          out.print("Invalid input");
-        }
-        out.print("</div>");
-      }
-    }
-  }
-
-  private static String getLocalParentDir(INode inode) {
-    final INode parent = inode.isRoot() ? inode : inode.getParent();
-    String parentDir = "";
-    if (parent != null) {
-      parentDir = parent.getFullPathName();
-    }
-    return (parentDir != null) ? parentDir : "";
-  }
-
-  // utility class used in block_info_xml.jsp
-  static class XMLBlockInfo {
-    final Block block;
-    final INodeFile inode;
-    final BlockManager blockManager;
-    
-    XMLBlockInfo(FSNamesystem fsn, Long blockId) {
-      this.blockManager = fsn != null ? fsn.getBlockManager() : null;
-
-      if (blockId == null) {
-        this.block = null;
-        this.inode = null;
-      } else {
-        this.block = new Block(blockId);
-        this.inode = blockManager != null ?
-          ((INode)blockManager.getBlockCollection(block)).asFile() :
-          null;
-      }
-    }
-
-    public void toXML(XMLOutputter doc) throws IOException {
-      doc.startTag("block_info");
-      if (block == null) {
-        doc.startTag("error");
-        doc.pcdata("blockId must be a Long");
-        doc.endTag();
-      }else{
-        doc.startTag("block_id");
-        doc.pcdata(""+block.getBlockId());
-        doc.endTag();
-
-        doc.startTag("block_name");
-        doc.pcdata(block.getBlockName());
-        doc.endTag();
-
-        if (inode != null) {
-          doc.startTag("file");
-
-          doc.startTag("local_name");
-          doc.pcdata(inode.getLocalName());
-          doc.endTag();
-
-          doc.startTag("local_directory");
-          doc.pcdata(getLocalParentDir(inode));
-          doc.endTag();
-
-          doc.startTag("user_name");
-          doc.pcdata(inode.getUserName());
-          doc.endTag();
-
-          doc.startTag("group_name");
-          doc.pcdata(inode.getGroupName());
-          doc.endTag();
-
-          doc.startTag("is_directory");
-          doc.pcdata(""+inode.isDirectory());
-          doc.endTag();
-
-          doc.startTag("access_time");
-          doc.pcdata(""+inode.getAccessTime());
-          doc.endTag();
-
-          doc.startTag("is_under_construction");
-          doc.pcdata(""+inode.isUnderConstruction());
-          doc.endTag();
-
-          doc.startTag("ds_quota");
-          doc.pcdata(""+inode.getQuotaCounts().get(Quota.DISKSPACE));
-          doc.endTag();
-
-          doc.startTag("permission_status");
-          doc.pcdata(inode.getPermissionStatus().toString());
-          doc.endTag();
-
-          doc.startTag("replication");
-          doc.pcdata(""+inode.getFileReplication());
-          doc.endTag();
-
-          doc.startTag("disk_space_consumed");
-          doc.pcdata(""+inode.diskspaceConsumed());
-          doc.endTag();
-
-          doc.startTag("preferred_block_size");
-          doc.pcdata(""+inode.getPreferredBlockSize());
-          doc.endTag();
-
-          doc.endTag(); // </file>
-        } 
-
-        doc.startTag("replicas");
-        for(DatanodeStorageInfo storage : (blockManager != null ?
-                blockManager.getStorages(block) :
-                Collections.<DatanodeStorageInfo>emptyList())) {
-          doc.startTag("replica");
-
-          DatanodeDescriptor dd = storage.getDatanodeDescriptor();
-
-          doc.startTag("host_name");
-          doc.pcdata(dd.getHostName());
-          doc.endTag();
-
-          boolean isCorrupt = blockManager.getCorruptReplicaBlockIds(0,
-                                block.getBlockId()) != null;
-          
-          doc.startTag("is_corrupt");
-          doc.pcdata(""+isCorrupt);
-          doc.endTag();
-          
-          doc.endTag(); // </replica>
-        }
-        doc.endTag(); // </replicas>
-                
-      }
-      
-      doc.endTag(); // </block_info>
-      
-    }
-  }
-  
-  // utility class used in corrupt_replicas_xml.jsp
-  static class XMLCorruptBlockInfo {
-    final Configuration conf;
-    final Long startingBlockId;
-    final int numCorruptBlocks;
-    final BlockManager blockManager;
-    
-    XMLCorruptBlockInfo(FSNamesystem fsn, Configuration conf,
-                               int numCorruptBlocks, Long startingBlockId) {
-      this.blockManager = fsn != null ? fsn.getBlockManager() : null;
-      this.conf = conf;
-      this.numCorruptBlocks = numCorruptBlocks;
-      this.startingBlockId = startingBlockId;
-    }
-
-
-    public void toXML(XMLOutputter doc) throws IOException {
-      
-      doc.startTag("corrupt_block_info");
-      
-      if (numCorruptBlocks < 0 || numCorruptBlocks > 100) {
-        doc.startTag("error");
-        doc.pcdata("numCorruptBlocks must be >= 0 and <= 100");
-        doc.endTag();
-      }
-      
-      doc.startTag(DFSConfigKeys.DFS_REPLICATION_KEY);
-      doc.pcdata(""+conf.getInt(DFSConfigKeys.DFS_REPLICATION_KEY, 
-                                DFSConfigKeys.DFS_REPLICATION_DEFAULT));
-      doc.endTag();
-      
-      doc.startTag("num_missing_blocks");
-      doc.pcdata("" + (blockManager != null ?
-        blockManager.getMissingBlocksCount() : 0));
-      doc.endTag();
-      
-      doc.startTag("num_corrupt_replica_blocks");
-      doc.pcdata("" + (blockManager != null ?
-        blockManager.getCorruptReplicaBlocksCount() : 0));
-      doc.endTag();
-     
-      doc.startTag("corrupt_replica_block_ids");
-      final long[] corruptBlockIds = blockManager != null ?
-        blockManager.getCorruptReplicaBlockIds(numCorruptBlocks,
-        startingBlockId) : null;
-      if (corruptBlockIds != null) {
-        for (Long blockId: corruptBlockIds) {
-          doc.startTag("block_id");
-          doc.pcdata(""+blockId);
-          doc.endTag();
-        }
-      }
-      
-      doc.endTag(); // </corrupt_replica_block_ids>
-
-      doc.endTag(); // </corrupt_block_info>
-      
-      doc.getWriter().flush();
-    }
-  }    
 }

+ 0 - 49
hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/datanode/browseBlock.jsp

@@ -1,49 +0,0 @@
-<%
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file 
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you 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.
- */
-%>
-<%@ page
-  contentType="text/html; charset=UTF-8"
-  import="org.apache.hadoop.hdfs.server.common.JspHelper"
-  import="org.apache.hadoop.util.ServletUtil"
-  import="org.apache.hadoop.conf.Configuration"
-%>
-<%!
-  //for java.io.Serializable
-  private static final long serialVersionUID = 1L;
-%>
-<!DOCTYPE html>
-<html>
-<head>
-<%JspHelper.createTitle(out, request, request.getParameter("filename")); %>
-</head>
-<body onload="document.goto.dir.focus()">
-<% 
-  Configuration conf = 
-     (Configuration) application.getAttribute(JspHelper.CURRENT_CONF);
-  DatanodeJspHelper.generateFileChunks(out, request, conf); 
-%>
-<hr>
-<% DatanodeJspHelper.generateFileDetails(out, request, conf); %>
-
-<h2>Local logs</h2>
-<a href="/logs/">Log</a> directory
-
-<%
-out.println(ServletUtil.htmlFooter());
-%>

+ 0 - 69
hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/datanode/browseDirectory.jsp

@@ -1,69 +0,0 @@
-<%
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file 
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you 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.
- */
-%>
-<%@ page
-  contentType="text/html; charset=UTF-8"
-  import="java.io.IOException"
-
-  import="org.apache.hadoop.hdfs.server.common.JspHelper"
-  import="org.apache.hadoop.util.ServletUtil"
-  import="org.apache.hadoop.conf.Configuration"
-%>
-<%!
-  //for java.io.Serializable
-  private static final long serialVersionUID = 1L;
-%>
-<!DOCTYPE html>
-<html>
-<head>
-<style type=text/css>
-<!--
-body 
-  {
-  font-face:sanserif;
-  }
--->
-</style>
-<%JspHelper.createTitle(out, request, request.getParameter("dir")); %>
-</head>
-
-<body onload="document.goto.dir.focus()">
-<% 
-  try {
-    Configuration conf = 
-      (Configuration) application.getAttribute(JspHelper.CURRENT_CONF);
-    DatanodeJspHelper.generateDirectoryStructure(out,request,response, conf);
-  }
-  catch(IOException ioe) {
-    String msg = ioe.getLocalizedMessage();
-    int i = msg.indexOf("\n");
-    if (i >= 0) {
-      msg = msg.substring(0, i);
-    }
-    out.print("<h3>" + msg + "</h3>");
-  }
-%>
-<hr>
-
-<h2>Local logs</h2>
-<a href="/logs/">Log</a> directory
-
-<%
-out.println(ServletUtil.htmlFooter());
-%>

+ 0 - 58
hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/datanode/dataNodeHome.jsp

@@ -1,58 +0,0 @@
-<%
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file 
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you 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.
- */
-%>
-<%@page import="org.apache.hadoop.hdfs.tools.GetConf"%>
-<%@page import="org.apache.hadoop.hdfs.server.datanode.DatanodeJspHelper"%>
-<%@page import="org.apache.hadoop.hdfs.server.datanode.DataNode"%>
-<%@ page
-  contentType="text/html; charset=UTF-8"
-  import="org.apache.hadoop.util.ServletUtil"
-%>
-<%!
-  //for java.io.Serializable
-  private static final long serialVersionUID = 1L;
-%>
-<%  
-  DataNode dataNode = (DataNode)getServletContext().getAttribute("datanode"); 
-  String state = dataNode.isDatanodeUp()?"active":"inactive";
-  String dataNodeLabel = dataNode.getDisplayName();
-%>
-
-<!DOCTYPE html>
-<html>
-<head>
-<link rel="stylesheet" type="text/css" href="/static/hadoop.css">
-<title>Hadoop DataNode&nbsp;<%=dataNodeLabel%></title>
-</head>    
-<body>
-<h1>DataNode '<%=dataNodeLabel%>' (<%=state%>)</h1>
-<%= DatanodeJspHelper.getVersionTable(getServletContext()) %>
-<br />
-<b><a href="/logs/">DataNode Logs</a></b>
-<br />
-<b><a href="/logLevel">View/Set Log Level</a></b>
-<br />
-<b><a href="/metrics">Metrics</a></b>
-<br />
-<b><a href="/conf">Configuration</a></b>
-<br />
-<b><a href="/blockScannerReport">Block Scanner Report</a></b>
-<%
-out.println(ServletUtil.htmlFooter());
-%>

+ 0 - 50
hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/datanode/tail.jsp

@@ -1,50 +0,0 @@
-<%
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file 
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you 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.
- */
-%>
-<%@ page
-  contentType="text/html; charset=UTF-8"
-  import="org.apache.hadoop.hdfs.server.common.JspHelper"
-  import="org.apache.hadoop.util.ServletUtil"
-  import="org.apache.hadoop.conf.Configuration"
-%>
-<%!
-  //for java.io.Serializable
-  private static final long serialVersionUID = 1L;
-%>
-<!DOCTYPE html>
-<html>
-<head>
-<%JspHelper.createTitle(out, request, request.getParameter("filename")); %>
-</head>
-<body>
-<form action="/tail.jsp" method="GET">
-<% 
-   Configuration conf = 
-     (Configuration) application.getAttribute(JspHelper.CURRENT_CONF);
-   DatanodeJspHelper.generateFileChunksForTail(out,request, conf); 
-%>
-</form>
-<hr>
-
-<h2>Local logs</h2>
-<a href="/logs/">Log</a> directory
-
-<%
-out.println(ServletUtil.htmlFooter());
-%>

+ 0 - 87
hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/hdfs/block_info_xml.jsp

@@ -1,87 +0,0 @@
-<?xml version="1.0" encoding="UTF-8"?><%!
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file 
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you 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.
- */
- 
- /*
- 
-  This script outputs information about a block (as XML). The script accepts a 
-  GET parameter named blockId which should be block id (as a long).
-
-  Example output is below (the blockId was 8888705098093096373):
-    <block_info>
-      <block_id>8888705098093096373</block_id>
-      <block_name>blk_8888705098093096373</block_name>
-      <file>
-        <local_name>some_file_name</local_name>
-        <local_directory>/input/</local_directory>
-        <user_name>user_name</user_name>
-        <group_name>supergroup</group_name>
-        <is_directory>false</is_directory>
-        <access_time>1251166313680</access_time>
-        <is_under_construction>false</is_under_construction>
-        <ds_quota>-1</ds_quota>
-        <permission_status>user_name:supergroup:rw-r--r--</permission_status>
-        <replication>1</replication>
-        <disk_space_consumed>2815</disk_space_consumed>
-        <preferred_block_size>67108864</preferred_block_size>
-      </file>
-      <replicas>
-        <replica>
-          <host_name>hostname</host_name>
-          <is_corrupt>false</is_corrupt>
-        </replica>
-      </replicas>
-    </block_info> 
-
-  Notes:
-    - block_info/file will only exist if the file can be found
-    - block_info/replicas can contain 0 or more children 
-    - If an error exists, block_info/error will exist and contain a human
-      readable error message
- 
-*/
- 
-%>
-<%@ page
-  contentType="application/xml"
-  import="org.apache.hadoop.hdfs.server.namenode.NamenodeJspHelper.XMLBlockInfo"
-  import="org.apache.hadoop.hdfs.server.common.JspHelper"
-  import="org.znerd.xmlenc.*"
-%>
-<%!
-  //for java.io.Serializable
-  private static final long serialVersionUID = 1L;  
-%>
-<%
-NameNode nn = NameNodeHttpServer.getNameNodeFromContext(application);
-String namenodeRole = nn.getRole().toString();
-FSNamesystem fsn = nn.getNamesystem();
-
-Long blockId = null;
-try {
-  blockId = JspHelper.validateLong(request.getParameter("blockId"));
-} catch(NumberFormatException e) {
-  blockId = null;
-}
-
-
-XMLBlockInfo bi = new XMLBlockInfo(fsn, blockId);
-XMLOutputter doc = new XMLOutputter(out, "UTF-8");
-bi.toXML(doc);
-
-%>

+ 0 - 85
hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/hdfs/corrupt_files.jsp

@@ -1,85 +0,0 @@
-
-<%
-  /*
-   * Licensed to the Apache Software Foundation (ASF) under one
-   * or more contributor license agreements.  See the NOTICE file
-   * distributed with this work for additional information
-   * regarding copyright ownership.  The ASF licenses this file
-   * to you 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.
-   */
-%>
-<%@ page contentType="text/html; charset=UTF-8"
-	import="org.apache.hadoop.util.ServletUtil"
-	import="org.apache.hadoop.fs.FileStatus"
-	import="org.apache.hadoop.fs.FileUtil"
-	import="org.apache.hadoop.fs.Path"
-	import="org.apache.hadoop.ha.HAServiceProtocol.HAServiceState"
-	import="java.util.Collection"
-	import="java.util.Collections"
-	import="java.util.Arrays" %>
-<%!//for java.io.Serializable
-  private static final long serialVersionUID = 1L;%>
-<%
-  NameNode nn = NameNodeHttpServer.getNameNodeFromContext(application);
-  FSNamesystem fsn = nn.getNamesystem();
-  HAServiceState nnHAState = nn.getServiceState();
-  boolean isActive = (nnHAState == HAServiceState.ACTIVE);
-  String namenodeRole = nn.getRole().toString();
-  String namenodeLabel = NamenodeJspHelper.getNameNodeLabel(nn);
-  Collection<FSNamesystem.CorruptFileBlockInfo> corruptFileBlocks = fsn != null ?
-    fsn.listCorruptFileBlocks("/", null) :
-    Collections.<FSNamesystem.CorruptFileBlockInfo>emptyList();
-  int corruptFileCount = corruptFileBlocks.size();
-%>
-
-<!DOCTYPE html>
-<html>
-<link rel="stylesheet" type="text/css" href="/static/hadoop.css">
-<title>Hadoop <%=namenodeRole%>&nbsp;<%=namenodeLabel%></title>
-<body>
-<h1><%=namenodeRole%> '<%=namenodeLabel%>'</h1>
-<%=NamenodeJspHelper.getVersionTable(fsn)%>
-<br>
-<% if (isActive && fsn != null) { %> 
-  <b><a href="/nn_browsedfscontent.jsp">Browse the filesystem</a></b>
-  <br>
-<% } %> 
-<b><a href="/logs/"><%=namenodeRole%> Logs</a></b>
-<br>
-<b><a href=/dfshealth.jsp> Go back to DFS home</a></b>
-<hr>
-<h3>Reported Corrupt Files</h3>
-<%
-  if (corruptFileCount == 0) {
-%>
-    <i>No missing blocks found at the moment.</i> <br>
-    Please run fsck for a thorough health analysis.
-<%
-  } else {
-    for (FSNamesystem.CorruptFileBlockInfo c : corruptFileBlocks) {
-      String currentFileBlock = c.toString();
-%>
-      <%=currentFileBlock%><br>
-<%
-    }
-%>
-    <p>
-      <b>Total:</b> At least <%=corruptFileCount%> corrupt file(s)
-    </p>
-<%
-  }
-%>
-
-<%
-  out.println(ServletUtil.htmlFooter());
-%>

+ 0 - 91
hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/hdfs/corrupt_replicas_xml.jsp

@@ -1,91 +0,0 @@
-<?xml version="1.0" encoding="UTF-8"?><%!
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file 
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you 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.
- */
- 
- /*
- 
-  This script outputs information about corrupt replicas on the system (as XML). 
-  
-  The script takes two GET parameters:
-    - numCorruptBlocks The number of corrupt blocks to return. Must be >= 0 &&
-      <= 100. Defaults to 10.
-    - startingBlockId The block id (as a long) from which to begin iterating. 
-      Output does not include the starting block id (it begins at the following
-      block id). If not given, iteration starts from beginning. 
-
-  Example output is below:
-      <corrupt_block_info>
-        <dfs_replication>1</dfs_replication>
-        <num_missing_blocks>1</num_missing_blocks>
-        <num_corrupt_replica_blocks>1</num_corrupt_replica_blocks>
-        <corrupt_replica_block_ids>
-          <block_id>-2207002825050436217</block_id>
-        </corrupt_replica_block_ids>
-      </corrupt_block_info>
-
-  Notes:
-    - corrupt_block_info/corrupt_replica_block_ids will 0 to numCorruptBlocks
-      children
-    - If an error exists, corrupt_block_info/error will exist and
-      contain a human readable error message
- 
-*/
- 
-%>
-<%@ page
-  contentType="application/xml"
-  import="java.io.IOException"
-  import="java.util.List"
-  import="org.apache.hadoop.conf.Configuration"
-  import="org.apache.hadoop.hdfs.server.common.JspHelper"
-  import="org.apache.hadoop.hdfs.server.namenode.NamenodeJspHelper.XMLCorruptBlockInfo"
-  import="org.apache.hadoop.util.ServletUtil"
-  import="org.znerd.xmlenc.*"
-%>
-<%!
-  private static final long serialVersionUID = 1L;
-%>
-<%
-
-  NameNode nn = NameNodeHttpServer.getNameNodeFromContext(application);
-  FSNamesystem fsn = nn.getNamesystem();
-
-  Integer numCorruptBlocks = 10;
-  try {
-    Long l = JspHelper.validateLong(request.getParameter("numCorruptBlocks"));
-    if (l != null) {
-      numCorruptBlocks = l.intValue();
-    }
-  } catch(NumberFormatException e) {
-    
-  }
-
-  Long startingBlockId = null;
-  try {
-    startingBlockId =
-      JspHelper.validateLong(request.getParameter("startingBlockId"));
-  } catch(NumberFormatException e) { 
-  }  
-
-  XMLCorruptBlockInfo cbi = new XMLCorruptBlockInfo(fsn,
-                                                    new Configuration(),
-                                                    numCorruptBlocks,
-                                                    startingBlockId);
-  XMLOutputter doc = new XMLOutputter(out, "UTF-8");
-  cbi.toXML(doc);
-%>

+ 0 - 45
hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/hdfs/decommission.jsp

@@ -1,45 +0,0 @@
-<?xml version="1.0" encoding="UTF-8"?>
-<?xml-stylesheet type="text/xsl" href="decommission.xsl"?>
-<%!
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file 
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you 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.
- */
-%>
-<%@ page 
-  contentType="application/xml"
-
-  import="org.apache.hadoop.util.ServletUtil"
-  import="org.apache.hadoop.hdfs.server.namenode.ClusterJspHelper.DecommissionStatus"
-  import="java.util.List"
-  import="org.znerd.xmlenc.*"
-%>
-<%!
-  //for java.io.Serializable
-  private static final long serialVersionUID = 1L;
-%>
-<%
-  /**
-   * This JSP page provides decomission nodes information cross cluster. 
-   * It lists the date nodes with their decommission states/progress 
-   * reported by each name node. 
-   * It eleminates the data nodes who are not in decommission states.
-   */
-  final ClusterJspHelper clusterhealthjsp  = new ClusterJspHelper();
-   DecommissionStatus dInfo = clusterhealthjsp.generateDecommissioningReport();
-   XMLOutputter doc = new XMLOutputter(out, "UTF-8");
-   dInfo.toXML(doc);
-%>

+ 0 - 139
hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/hdfs/decommission.xsl

@@ -1,139 +0,0 @@
-<!--
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you 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.
- */
--->
-
-<xsl:stylesheet version="1.0"
-  xmlns:xsl="http://www.w3.org/1999/XSL/Transform">
-
-  <xsl:include href="dfsclusterhealth_utils.xsl" />
-
-  <xsl:output method="html" encoding="UTF-8" />
-
-  <xsl:template match="/">
-    <html>
-      <head>
-        <link rel="stylesheet" type="text/css" href="static/hadoop.css" />
-        <title>
-          Hadoop cluster
-          <xsl:value-of select="cluster/@clusterId" />
-        </title>
-      </head>
-      <body>
-
-        <h1>
-          Cluster '
-          <xsl:value-of select="cluster/@clusterId" />
-          '
-        </h1>
-
-        <h2>Decommissioning Status</h2>
-
-        <xsl:if test="count(cluster/decommissioningReport/item)">
-          <div id="dfstable">
-            <table>
-              <tbody>
-                <xsl:for-each select="cluster/decommissioningReport/item">
-                  <tr class="rowNormal">
-                    <td id="col1">
-                      <xsl:value-of select="@label" />
-                    </td>
-                    <td id="col2">:</td>
-                    <td id="col3">
-                      <xsl:value-of select="@value" />
-                    </td>
-                  </tr>
-                </xsl:for-each>
-              </tbody>
-            </table>
-          </div>
-
-          <br />
-        </xsl:if>
-
-        <xsl:if test="count(cluster/datanodes/node)">
-
-          <div id="dfstable">
-            <table border="1" cellpadding="10" cellspacing="0">
-              <thead>
-                <xsl:for-each select="cluster/datanodes/node[1]/item">
-                  <th>
-                    <xsl:value-of select="@label" />
-                  </th>
-                </xsl:for-each>
-              </thead>
-              <tbody>
-                <xsl:for-each select="cluster/datanodes/node">
-                  <tr>
-                    <xsl:for-each select="item">
-                      <td>
-
-                        <xsl:call-template name="displayValue">
-                          <xsl:with-param name="value">
-                            <xsl:value-of select="@value" />
-                          </xsl:with-param>
-                          <xsl:with-param name="unit">
-                            <xsl:value-of select="@unit" />
-                          </xsl:with-param>
-                          <xsl:with-param name="link">
-                            <xsl:value-of select="@link" />
-
-                          </xsl:with-param>
-                        </xsl:call-template>
-                      </td>
-                    </xsl:for-each>
-                  </tr>
-                </xsl:for-each>
-              </tbody>
-            </table>
-          </div>
-
-        </xsl:if>
-
-        <xsl:if test="count(cluster/unreportedNamenodes/node)">
-          <h2>Unreported Namenodes</h2>
-          <div id="dfstable">
-            <table border="1" cellpadding="10" cellspacing="0">
-              <tbody>
-                <xsl:for-each select="cluster/unreportedNamenodes/node">
-                  <tr class="rowNormal">
-                    <td id="col1">
-                      <xsl:value-of select="@name" />
-                    </td>
-                    <td id="col2">
-                      <xsl:value-of select="@exception" />
-                    </td>
-                  </tr>
-                </xsl:for-each>
-              </tbody>
-            </table>
-          </div>
-        </xsl:if>
-
-        <xsl:if test="count(cluster/message/item)">
-          <h4>Exception</h4>
-          <xsl:for-each select="cluster/message/item">
-            <xsl:value-of select="@msg" />
-          </xsl:for-each>
-        </xsl:if>
-
-      </body>
-    </html>
-  </xsl:template>
-</xsl:stylesheet> 
-

+ 0 - 45
hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/hdfs/dfsclusterhealth.jsp

@@ -1,45 +0,0 @@
-<?xml version="1.0" encoding="UTF-8"?>
-<?xml-stylesheet type="text/xsl" href="dfsclusterhealth.xsl"?>
-<%!
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file 
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you 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.
- */
-%>
-<%@ page 
-  contentType="application/xml"
-
-  import="org.apache.hadoop.util.ServletUtil"
-  import="org.apache.hadoop.hdfs.server.namenode.ClusterJspHelper.ClusterStatus"
-  import="java.util.List"
-  import="org.znerd.xmlenc.*"
-%>
-<%!
-  //for java.io.Serializable
-  private static final long serialVersionUID = 1L;
-%>
-<%
-   /** 
-    * This JSP page provides cluster summary in XML format. It lists information
-    * such as total files and blocks, total capacity, total used/freed spaces,etc. 
-    * accorss cluster reported by all name nodes.  
-    * It also lists information such as used space per name node. 
-    */
-   final ClusterJspHelper clusterhealthjsp  = new ClusterJspHelper();
-   ClusterStatus cInfo = clusterhealthjsp.generateClusterHealthReport();
-   XMLOutputter doc = new XMLOutputter(out, "UTF-8");
-   cInfo.toXML(doc);
-%>

+ 0 - 170
hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/hdfs/dfsclusterhealth.xsl

@@ -1,170 +0,0 @@
-<!--
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you 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.
- */
--->
-
-<xsl:stylesheet version="1.0"
-  xmlns:xsl="http://www.w3.org/1999/XSL/Transform">
-
-  <xsl:include href="dfsclusterhealth_utils.xsl" />
-
-  <xsl:output method="html" encoding="UTF-8" />
-
-  <xsl:template match="/">
-    <html>
-      <head>
-        <link rel="stylesheet" type="text/css" href="static/hadoop.css" />
-        <style type="text/css">th,span {width:8em;}</style>
-        <title>
-          Hadoop cluster
-          <xsl:value-of select="cluster/@clusterId" />
-        </title>
-      </head>
-      <body>
-
-        <h1>
-          Cluster '
-          <xsl:value-of select="cluster/@clusterId" />
-          '
-        </h1>
-
-        <h2>Cluster Summary</h2>
-        <xsl:if test="count(cluster/storage/item)">
-          <div id="dfstable">
-            <table>
-              <tbody>
-                <xsl:for-each select="cluster/storage/item">
-                  <tr class="rowNormal">
-                    <td id="col1">
-                      <xsl:value-of select="@label" />
-                    </td>
-                    <td id="col2">:</td>
-                    <td id="col3">
-
-                      <xsl:call-template name="displayValue">
-                        <xsl:with-param name="value">
-                          <xsl:value-of select="@value" />
-                        </xsl:with-param>
-                        <xsl:with-param name="unit">
-                          <xsl:value-of select="@unit" />
-                        </xsl:with-param>
-                        <xsl:with-param name="link">
-                          <xsl:value-of select="@link" />
-
-                        </xsl:with-param>
-                      </xsl:call-template>
-                    </td>
-                  </tr>
-                </xsl:for-each>
-              </tbody>
-            </table>
-          </div>
-
-          <br />
-          <hr />
-        </xsl:if>
-        <xsl:if test="count(cluster/namenodes/node)">
-          <h2>Namenodes</h2>
-
-          <div id="dfstable">
-            <table>
-              <tbody>
-                <tr class="rowNormal">
-                  <td id="col1">Number of namenodes</td>
-                  <td id="col2">:</td>
-                  <td id="col3">
-                    <xsl:value-of select="count(cluster/namenodes/node)" />
-                  </td>
-                </tr>
-              </tbody>
-            </table>
-          </div>
-
-          <br />
-
-          <div id="dfstable">
-            <table border="1" cellpadding="10" cellspacing="0">
-
-              <thead>
-                <xsl:for-each select="cluster/namenodes/node[1]/item">
-                  <th>
-                    <SPAN><xsl:value-of select="@label" /></SPAN>
-                  </th>
-                </xsl:for-each>
-              </thead>
-
-              <tbody>
-                <xsl:for-each select="cluster/namenodes/node">
-                  <tr>
-                    <xsl:for-each select="item">
-                      <td>
-
-                        <xsl:call-template name="displayValue">
-                          <xsl:with-param name="value">
-                            <xsl:value-of select="@value" />
-                          </xsl:with-param>
-                          <xsl:with-param name="unit">
-                            <xsl:value-of select="@unit" />
-                          </xsl:with-param>
-                          <xsl:with-param name="link">
-                            <xsl:value-of select="@link" />
-
-                          </xsl:with-param>
-                        </xsl:call-template>
-                      </td>
-                    </xsl:for-each>
-                  </tr>
-                </xsl:for-each>
-              </tbody>
-
-            </table>
-          </div>
-        </xsl:if>
-
-        <xsl:if test="count(cluster/unreportedNamenodes/node)">
-          <h2>Unreported Namenodes</h2>
-          <div id="dfstable">
-            <table border="1" cellpadding="10" cellspacing="0">
-              <tbody>
-                <xsl:for-each select="cluster/unreportedNamenodes/node">
-                  <tr class="rowNormal">
-                    <td id="col1">
-                      <xsl:value-of select="@name" />
-                    </td>
-                    <td id="col2">
-                      <xsl:value-of select="@exception" />
-                    </td>
-                  </tr>
-                </xsl:for-each>
-              </tbody>
-            </table>
-          </div>
-        </xsl:if>
-
-        <xsl:if test="count(cluster/message/item)">
-          <h4>Exception</h4>
-          <xsl:for-each select="cluster/message/item">
-            <xsl:value-of select="@msg" />
-          </xsl:for-each>
-        </xsl:if>
-
-      </body>
-    </html>
-  </xsl:template>
-</xsl:stylesheet> 
-

+ 0 - 88
hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/hdfs/dfsclusterhealth_utils.xsl

@@ -1,88 +0,0 @@
-<!--
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you 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.
- */
--->
-
-<xsl:stylesheet version="1.0" xmlns:xsl="http://www.w3.org/1999/XSL/Transform">
-
-  <xsl:template name="humanReadableBytes">
-
-    <xsl:param name="number"/>
-
-    <xsl:variable name="kb" select="1024"/>
-    <xsl:variable name="Mb" select="$kb * 1024"/>
-    <xsl:variable name="Gb" select="$Mb * 1024"/>
-    <xsl:variable name="Tb" select="$Gb * 1024"/>
-    <xsl:variable name="Pb" select="$Tb * 1024"/>
-
-     
-    <xsl:choose>
-      <xsl:when test="$number &lt; $kb"><xsl:value-of select="format-number($number, '#,###.##')"/> b</xsl:when>
-      <xsl:when test="$number &lt; $Mb"><xsl:value-of select="format-number($number div $kb, '#,###.00')"/> kb</xsl:when>
-      <xsl:when test="$number &lt; $Gb"><xsl:value-of select="format-number($number div $Mb, '#,###.00')"/> Mb</xsl:when>
-      <xsl:when test="$number &lt; $Tb"><xsl:value-of select="format-number($number div $Gb, '#,###.00')"/> Gb</xsl:when>
-
-      <xsl:when test="$number &lt; $Pb"><xsl:value-of select="format-number($number div $Tb, '#,###.00')"/> Tb</xsl:when>
-      <xsl:when test="$number &lt; ($Pb * 1024)"><xsl:value-of select="format-number($number div $Pb, '#,###.00')"/> Pb</xsl:when>
-      <xsl:otherwise><xsl:value-of select="format-number($number, '#,###.00')"/> b</xsl:otherwise>
-    </xsl:choose>
-
-  </xsl:template>
-
-  <xsl:template name="percentage">
-    <xsl:param name="number"/>
-    <xsl:value-of select="format-number($number, '0.000%')"/>
-  </xsl:template>
-
-  <!--
-    Displays value:
-      - if it has parameter unit="b" then call humanReadableBytes
-      - if it has parameter link then call displayLink
-  -->
-  <xsl:template name="displayValue">
-    <xsl:param name="value"/>
-    <xsl:param name="unit"/>
-
-    <xsl:param name="link"/>
-    <xsl:choose>
-      <xsl:when test="$unit = 'b'">
-        <xsl:call-template name="humanReadableBytes">
-          <xsl:with-param name="number">
-            <xsl:value-of select="@value"/>
-          </xsl:with-param>
-        </xsl:call-template>
-      </xsl:when>
-
-      <xsl:when test="$unit = '%'">
-        <xsl:call-template name="percentage">
-          <xsl:with-param name="number">
-            <xsl:value-of select="@value"/>
-          </xsl:with-param>
-        </xsl:call-template>
-      </xsl:when>
-      <xsl:when test="string-length($link) &gt; 0">
-        <a href="{$link}"><xsl:value-of select="$value"/></a>
-
-      </xsl:when>
-      <xsl:otherwise><xsl:value-of select="$value"/></xsl:otherwise>
-    </xsl:choose>
-
-  </xsl:template>
-
-</xsl:stylesheet> 
-

+ 0 - 75
hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/hdfs/dfshealth.jsp

@@ -1,75 +0,0 @@
-<%
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file 
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you 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.
- */
-%>
-<%@ page
-  contentType="text/html; charset=UTF-8"
-  import="org.apache.hadoop.util.ServletUtil"
-  import="org.apache.hadoop.ha.HAServiceProtocol.HAServiceState"
-%>
-<%!
-  //for java.io.Serializable
-  private static final long serialVersionUID = 1L;
-%>
-<%
-  final NamenodeJspHelper.HealthJsp healthjsp  = new NamenodeJspHelper.HealthJsp();
-  NameNode nn = NameNodeHttpServer.getNameNodeFromContext(application);
-  FSNamesystem fsn = nn.getNamesystem();
-  HAServiceState nnHAState = nn.getServiceState();
-  boolean isActive = (nnHAState == HAServiceState.ACTIVE);
-  String namenodeRole = nn.getRole().toString();
-  String namenodeState = nnHAState.toString();
-  String namenodeLabel = NamenodeJspHelper.getNameNodeLabel(nn);
-%>
-
-<!DOCTYPE html>
-<html>
-<head>
-<link rel="stylesheet" type="text/css" href="/static/hadoop.css">
-<title>Hadoop <%=namenodeRole%>&nbsp;<%=namenodeLabel%></title>
-</head>    
-<body>
-<h1><%=namenodeRole%> '<%=namenodeLabel%>' (<%=namenodeState%>)</h1>
-<%= NamenodeJspHelper.getVersionTable(fsn) %>
-<br />
-<% if (isActive && fsn != null) { %> 
-  <b><a href="/nn_browsedfscontent.jsp">Browse the filesystem</a></b><br>
-<% } %> 
-<b><a href="/logs/"><%=namenodeRole%> Logs</a></b>
-
-<hr>
-<h3>Cluster Summary</h3>
-<b> <%= NamenodeJspHelper.getSecurityModeText()%> </b>
-<b> <%= NamenodeJspHelper.getSafeModeText(fsn)%> </b>
-<b> <%= NamenodeJspHelper.getRollingUpgradeText(fsn)%> </b>
-<b> <%= NamenodeJspHelper.getInodeLimitText(fsn)%> </b>
-<%= NamenodeJspHelper.getCorruptFilesWarning(fsn)%>
-
-<% healthjsp.generateHealthReport(out, nn, request); %>
-<% healthjsp.generateJournalReport(out, nn, request); %>
-<hr/>
-<% healthjsp.generateConfReport(out, nn, request); %>
-<hr/>
-<h3>Snapshot Summary</h3>
-<% NamenodeJspHelper.generateSnapshotReport(out, fsn); %>
-<hr/>
-<h3>Startup Progress</h3>
-<% healthjsp.generateStartupProgress(out, nn.getStartupProgress()); %>
-<hr/><p><a href="http://hadoop.apache.org/core">Hadoop</a>, 2013.&nbsp;<a href="dfshealth.html">New UI</a></p>
-</body>
-</html>

+ 0 - 59
hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/hdfs/dfsnodelist.jsp

@@ -1,59 +0,0 @@
-<%
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file 
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you 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.
- */
-%>
-<%@ page
-  contentType="text/html; charset=UTF-8"
-  import="org.apache.hadoop.util.ServletUtil"
-  import="org.apache.hadoop.ha.HAServiceProtocol.HAServiceState"
-%>
-<%!
-  //for java.io.Serializable
-  private static final long serialVersionUID = 1L;
-%>
-<%
-final NamenodeJspHelper.NodeListJsp nodelistjsp = new NamenodeJspHelper.NodeListJsp();
-NameNode nn = NameNodeHttpServer.getNameNodeFromContext(application);
-String namenodeRole = nn.getRole().toString();
-FSNamesystem fsn = nn.getNamesystem();
-HAServiceState nnHAState = nn.getServiceState();
-boolean isActive = (nnHAState == HAServiceState.ACTIVE);
-String namenodeLabel = NamenodeJspHelper.getNameNodeLabel(nn);
-%>
-
-<!DOCTYPE html>
-<html>
-
-<link rel="stylesheet" type="text/css" href="/static/hadoop.css">
-<title>Hadoop <%=namenodeRole%>&nbsp;<%=namenodeLabel%></title>
-  
-<body>
-<h1><%=namenodeRole%> '<%=namenodeLabel%>'</h1>
-<%= NamenodeJspHelper.getVersionTable(fsn) %>
-<br />
-<% if (isActive && fsn != null) { %> 
-  <b><a href="/nn_browsedfscontent.jsp">Browse the filesystem</a></b><br>
-<% } %> 
-<b><a href="/logs/"><%=namenodeRole%> Logs</a></b><br>
-<b><a href=/dfshealth.jsp> Go back to DFS home</a></b>
-<hr>
-<% nodelistjsp.generateNodesList(application, out, request); %>
-
-<%
-out.println(ServletUtil.htmlFooter());
-%>

+ 0 - 46
hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/hdfs/nn_browsedfscontent.jsp

@@ -1,46 +0,0 @@
-<%
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file 
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you 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.
- */
-%>
-<%@ page
-  contentType="text/html; charset=UTF-8"
-  import="org.apache.hadoop.conf.Configuration"
-  import="org.apache.hadoop.util.ServletUtil"
-  import="org.apache.hadoop.hdfs.server.common.JspHelper"
-%>
-<%!
-  //for java.io.Serializable
-  private static final long serialVersionUID = 1L;
-%>
-<!DOCTYPE html>
-<html>
-
-<title></title>
-
-<body>
-<% 
-  NamenodeJspHelper.redirectToRandomDataNode(application, request, response); 
-%>
-<hr>
-
-<h2>Local logs</h2>
-<a href="/logs/">Log</a> directory
-
-<%
-out.println(ServletUtil.htmlFooter());
-%>

+ 0 - 42
hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/journal/journalstatus.jsp

@@ -1,42 +0,0 @@
-<%
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file 
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you 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.
- */
-%>
-<%@ page
-  contentType="text/html; charset=UTF-8"
-  import="org.apache.hadoop.hdfs.server.common.JspHelper"
-  import="org.apache.hadoop.util.ServletUtil"
-%>
-<%!
-  //for java.io.Serializable
-  private static final long serialVersionUID = 1L;
-%>
-
-<!DOCTYPE html>
-<html>
-<link rel="stylesheet" type="text/css" href="/static/hadoop.css">
-<title>Hadoop JournalNode</title>
-    
-<body>
-<h1>JournalNode</h1>
-<%= JspHelper.getVersionTable() %>
-<hr />
-
-<br />
-<b><a href="/logs/">Logs</a></b>
-<%= ServletUtil.htmlFooter() %>

+ 0 - 45
hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/secondary/status.jsp

@@ -1,45 +0,0 @@
-<%
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file 
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you 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.
- */
-%>
-<%@ page
-  contentType="text/html; charset=UTF-8"
-  import="org.apache.hadoop.hdfs.server.common.JspHelper"
-  import="org.apache.hadoop.util.ServletUtil"
-%>
-<%!
-  //for java.io.Serializable
-  private static final long serialVersionUID = 1L;
-%>
-
-<!DOCTYPE html>
-<html>
-<link rel="stylesheet" type="text/css" href="/static/hadoop.css">
-<title>Hadoop SecondaryNameNode</title>
-    
-<body>
-<h1>SecondaryNameNode</h1>
-<%= JspHelper.getVersionTable() %>
-<hr />
-<pre>
-<%= application.getAttribute("secondary.name.node").toString() %>
-</pre>
-
-<br />
-<b><a href="/logs/">Logs</a></b>
-<%= ServletUtil.htmlFooter() %>

+ 12 - 188
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/common/TestJspHelper.java

@@ -17,39 +17,12 @@
  */
 package org.apache.hadoop.hdfs.server.common;
 
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.assertNotNull;
-import static org.junit.Assert.fail;
-import static org.mockito.Mockito.*;
-
-import java.io.IOException;
-import java.io.StringReader;
-import java.net.InetSocketAddress;
-import java.util.ArrayList;
-
-import javax.servlet.ServletContext;
-import javax.servlet.http.HttpServletRequest;
-import javax.servlet.jsp.JspWriter;
-import javax.xml.parsers.DocumentBuilder;
-import javax.xml.parsers.DocumentBuilderFactory;
-import javax.xml.parsers.ParserConfigurationException;
-
-
-import static com.google.common.base.Strings.*;
-
-import junit.framework.Assert;
+import org.junit.Assert;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
-import org.apache.hadoop.hdfs.protocol.DatanodeID;
 import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier;
-import org.apache.hadoop.hdfs.server.blockmanagement.BlockManagerTestUtil;
-import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor;
 import org.apache.hadoop.hdfs.server.namenode.NameNodeHttpServer;
-import org.apache.hadoop.hdfs.server.protocol.DatanodeStorage;
-import org.apache.hadoop.hdfs.server.protocol.StorageReport;
 import org.apache.hadoop.hdfs.web.resources.DoAsParam;
 import org.apache.hadoop.hdfs.web.resources.UserParam;
 import org.apache.hadoop.io.DataInputBuffer;
@@ -66,17 +39,21 @@ import org.apache.hadoop.security.token.TokenIdentifier;
 import org.apache.hadoop.security.token.delegation.AbstractDelegationTokenSecretManager;
 import org.junit.BeforeClass;
 import org.junit.Test;
-import org.mockito.ArgumentCaptor;
-import org.mockito.Mockito;
-import org.mockito.invocation.InvocationOnMock;
-import org.mockito.stubbing.Answer;
-import org.xml.sax.InputSource;
-import org.xml.sax.SAXException;
+
+import javax.servlet.ServletContext;
+import javax.servlet.http.HttpServletRequest;
+import java.io.IOException;
+import java.net.InetSocketAddress;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
 
 public class TestJspHelper {
 
   private final Configuration conf = new HdfsConfiguration();
-  private String jspWriterOutput = "";
 
   // allow user with TGT to run tests
   @BeforeClass
@@ -405,32 +382,6 @@ public class TestJspHelper {
     }
   }
 
-  @Test
-  public void testPrintGotoFormWritesValidXML() throws IOException,
-         ParserConfigurationException, SAXException {
-    JspWriter mockJspWriter = mock(JspWriter.class);
-    ArgumentCaptor<String> arg = ArgumentCaptor.forClass(String.class);
-    doAnswer(new Answer<Object>() {
-      @Override
-      public Object answer(InvocationOnMock invok) {
-        Object[] args = invok.getArguments();
-        jspWriterOutput += (String) args[0];
-        return null;
-      }
-    }).when(mockJspWriter).print(arg.capture());
-
-    jspWriterOutput = "";
-
-    JspHelper.printGotoForm(mockJspWriter, 424242, "a token string",
-            "foobar/file", "0.0.0.0");
-
-    DocumentBuilder parser =
-        DocumentBuilderFactory.newInstance().newDocumentBuilder();
-    InputSource is = new InputSource();
-    is.setCharacterStream(new StringReader(jspWriterOutput));
-    parser.parse(is);
-  }
-
   private HttpServletRequest getMockRequest(String remoteUser, String user, String doAs) {
     HttpServletRequest request = mock(HttpServletRequest.class);
     when(request.getParameter(UserParam.NAME)).thenReturn(user);
@@ -464,119 +415,7 @@ public class TestJspHelper {
                           ugi.getAuthenticationMethod());
     }
   }
-  
-  @Test
-  public void testSortNodeByFields() throws Exception {
-    DatanodeID dnId1 = new DatanodeID("127.0.0.1", "localhost1", "datanode1",
-        1234, 2345, 3456, 4567);
-    DatanodeID dnId2 = new DatanodeID("127.0.0.2", "localhost2", "datanode2",
-        1235, 2346, 3457, 4568);
-
-    // Setup DatanodeDescriptors with one storage each.
-    DatanodeDescriptor dnDesc1 = new DatanodeDescriptor(dnId1, "rack1");
-    DatanodeDescriptor dnDesc2 = new DatanodeDescriptor(dnId2, "rack2");
-
-    // Update the DatanodeDescriptors with their attached storages.
-    BlockManagerTestUtil.updateStorage(dnDesc1, new DatanodeStorage("dnStorage1"));
-    BlockManagerTestUtil.updateStorage(dnDesc2, new DatanodeStorage("dnStorage2"));
-
-    DatanodeStorage dns1 = new DatanodeStorage("dnStorage1");
-    DatanodeStorage dns2 = new DatanodeStorage("dnStorage2");
-
-    StorageReport[] report1 = new StorageReport[] {
-        new StorageReport(dns1, false, 1024, 100, 924, 100)
-    };
-    StorageReport[] report2 = new StorageReport[] {
-        new StorageReport(dns2, false, 2500, 200, 1848, 200)
-    };
-    dnDesc1.updateHeartbeat(report1, 5L, 3L, 10, 2);
-    dnDesc2.updateHeartbeat(report2, 10L, 2L, 20, 1);
-
-    ArrayList<DatanodeDescriptor> live = new ArrayList<DatanodeDescriptor>();
-    live.add(dnDesc1);
-    live.add(dnDesc2);
-      
-    JspHelper.sortNodeList(live, "unexists", "ASC");
-    Assert.assertEquals(dnDesc1, live.get(0));
-    Assert.assertEquals(dnDesc2, live.get(1));    
-    JspHelper.sortNodeList(live, "unexists", "DSC");
-    Assert.assertEquals(dnDesc2, live.get(0));
-    Assert.assertEquals(dnDesc1, live.get(1));  
-    
-    // test sorting by capacity
-    JspHelper.sortNodeList(live, "capacity", "ASC");
-    Assert.assertEquals(dnDesc1, live.get(0));
-    Assert.assertEquals(dnDesc2, live.get(1));    
-    JspHelper.sortNodeList(live, "capacity", "DSC");
-    Assert.assertEquals(dnDesc2, live.get(0));
-    Assert.assertEquals(dnDesc1, live.get(1));
 
-    // test sorting by used
-    JspHelper.sortNodeList(live, "used", "ASC");
-    Assert.assertEquals(dnDesc1, live.get(0));
-    Assert.assertEquals(dnDesc2, live.get(1));    
-    JspHelper.sortNodeList(live, "used", "DSC");
-    Assert.assertEquals(dnDesc2, live.get(0));
-    Assert.assertEquals(dnDesc1, live.get(1)); 
-    
-    // test sorting by nondfsused
-    JspHelper.sortNodeList(live, "nondfsused", "ASC");
-    Assert.assertEquals(dnDesc1, live.get(0));
-    Assert.assertEquals(dnDesc2, live.get(1));
-    
-    JspHelper.sortNodeList(live, "nondfsused", "DSC");
-    Assert.assertEquals(dnDesc2, live.get(0));
-    Assert.assertEquals(dnDesc1, live.get(1));
-   
-    // test sorting by remaining
-    JspHelper.sortNodeList(live, "remaining", "ASC");
-    Assert.assertEquals(dnDesc1, live.get(0));
-    Assert.assertEquals(dnDesc2, live.get(1));
-    
-    JspHelper.sortNodeList(live, "remaining", "DSC");
-    Assert.assertEquals(dnDesc2, live.get(0));
-    Assert.assertEquals(dnDesc1, live.get(1));
-  }
-  
-  @Test
-  public void testPrintMethods() throws IOException {
-    JspWriter out = mock(JspWriter.class);      
-    HttpServletRequest req = mock(HttpServletRequest.class);
-    
-    final StringBuffer buffer = new StringBuffer();
-    
-    ArgumentCaptor<String> arg = ArgumentCaptor.forClass(String.class);
-    doAnswer(new Answer<String>() {      
-      @Override
-      public String answer(InvocationOnMock invok) {
-        Object[] args = invok.getArguments();
-        buffer.append(args[0]);
-        return null;
-      }
-    }).when(out).print(arg.capture());
-    
-    
-    JspHelper.createTitle(out, req, "testfile.txt");
-    verify(out, times(1)).print(Mockito.anyString());
-    
-    JspHelper.addTableHeader(out);
-    verify(out, times(1 + 2)).print(anyString());                  
-     
-    JspHelper.addTableRow(out, new String[] {" row11", "row12 "});
-    verify(out, times(1 + 2 + 4)).print(anyString());      
-    
-    JspHelper.addTableRow(out, new String[] {" row11", "row12 "}, 3);
-    verify(out, times(1 + 2 + 4 + 4)).print(Mockito.anyString());
-      
-    JspHelper.addTableRow(out, new String[] {" row21", "row22"});
-    verify(out, times(1 + 2 + 4 + 4 + 4)).print(anyString());      
-      
-    JspHelper.addTableFooter(out);
-    verify(out, times(1 + 2 + 4 + 4 + 4 + 1)).print(anyString());
-    
-    assertFalse(isNullOrEmpty(buffer.toString()));               
-  }
-  
   @Test
   public void testReadWriteReplicaState() {
     try {
@@ -596,21 +435,6 @@ public class TestJspHelper {
       fail("testReadWrite ex error ReplicaState");
     }
   }
-
-  @Test 
-  public void testAuthority(){
-    DatanodeID dnWithIp = new DatanodeID("127.0.0.1", "hostName", null,
-        50020, 50075, 50076, 50010);
-    assertNotNull(JspHelper.Url.authority("http", dnWithIp));
-
-    DatanodeID dnWithNullIp = new DatanodeID(null, "hostName", null,
-        50020, 50075, 50076, 50010);
-    assertNotNull(JspHelper.Url.authority("http", dnWithNullIp));
-
-    DatanodeID dnWithEmptyIp = new DatanodeID("", "hostName", null,
-        50020, 50075, 50076, 50010);
-    assertNotNull(JspHelper.Url.authority("http", dnWithEmptyIp));
-  }
  
   private static String clientAddr = "1.1.1.1";
   private static String chainedClientAddr = clientAddr+", 2.2.2.2";

+ 0 - 188
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDatanodeJsp.java

@@ -1,188 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you 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.hdfs.server.datanode;
-
-import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.fail;
-
-import java.io.DataOutputStream;
-import java.io.IOException;
-import java.net.InetSocketAddress;
-import java.net.URL;
-import java.net.URLEncoder;
-import java.util.regex.Matcher;
-import java.util.regex.Pattern;
-
-import javax.servlet.http.HttpServletRequest;
-import javax.servlet.jsp.JspWriter;
-
-import org.apache.commons.lang.StringEscapeUtils;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.hdfs.DFSTestUtil;
-import org.apache.hadoop.hdfs.HdfsConfiguration;
-import org.apache.hadoop.hdfs.MiniDFSCluster;
-import org.apache.hadoop.hdfs.server.common.JspHelper;
-import org.apache.hadoop.hdfs.server.namenode.NameNode;
-import org.apache.hadoop.net.NetUtils;
-import org.apache.hadoop.util.ServletUtil;
-import org.junit.Test;
-import org.mockito.Mockito;
-
-public class TestDatanodeJsp {
-  
-  private static final String FILE_DATA = "foo bar baz biz buz";
-  private static final HdfsConfiguration CONF = new HdfsConfiguration();
-  private static String viewFilePage;
-  
-  private static void testViewingFile(MiniDFSCluster cluster, String filePath)
-      throws IOException {
-    FileSystem fs = cluster.getFileSystem();
-    
-    Path testPath = new Path(filePath);
-    if (!fs.exists(testPath)) {
-      DFSTestUtil.writeFile(fs, testPath, FILE_DATA);
-    }
-    
-    InetSocketAddress nnIpcAddress = cluster.getNameNode().getNameNodeAddress();
-    InetSocketAddress nnHttpAddress = cluster.getNameNode().getHttpAddress();
-    String base = JspHelper.Url.url("http", cluster.getDataNodes().get(0)
-        .getDatanodeId());
-
-    URL url = new URL(base + "/"
-        + "browseDirectory.jsp" + JspHelper.getUrlParam("dir", 
-            URLEncoder.encode(testPath.toString(), "UTF-8"), true)
-        + JspHelper.getUrlParam("namenodeInfoPort", Integer
-            .toString(nnHttpAddress.getPort())) + JspHelper
-            .getUrlParam("nnaddr", "localhost:" + nnIpcAddress.getPort()));
-    
-    viewFilePage = StringEscapeUtils.unescapeHtml(DFSTestUtil.urlGet(url));
-    
-    assertTrue("page should show preview of file contents, got: " + viewFilePage,
-        viewFilePage.contains(FILE_DATA));
-    
-    assertTrue("page should show link to download file", viewFilePage
-        .contains("/streamFile" + ServletUtil.encodePath(filePath)
-            + "?nnaddr=localhost:" + nnIpcAddress.getPort()));
-    
-    // check whether able to tail the file
-    String regex = "<a.+href=\"(.+?)\">Tail\\s*this\\s*file\\<\\/a\\>";
-    assertFileContents(regex, "Tail this File");
-    
-    // check whether able to 'Go Back to File View' after tailing the file
-    regex = "<a.+href=\"(.+?)\">Go\\s*Back\\s*to\\s*File\\s*View\\<\\/a\\>";
-    assertFileContents(regex, "Go Back to File View");
-
-    regex = "<a href=\"///" + nnHttpAddress.getHostName() + ":" +
-      nnHttpAddress.getPort() + "/dfshealth.jsp\">Go back to DFS home</a>";
-    assertTrue("page should generate DFS home scheme without explicit scheme", viewFilePage.contains(regex));
-  }
-  
-  private static void assertFileContents(String regex, String text)
-      throws IOException {
-    Pattern compile = Pattern.compile(regex);
-    Matcher matcher = compile.matcher(viewFilePage);
-    if (matcher.find()) {
-      // got hyperlink for Tail this file
-      String u = matcher.group(1);
-      String urlString = u.startsWith("///") ? ("http://" + u.substring(3)) : u;
-      viewFilePage = StringEscapeUtils.unescapeHtml(DFSTestUtil
-          .urlGet(new URL(urlString)));
-      assertTrue("page should show preview of file contents", viewFilePage
-          .contains(FILE_DATA));
-    } else {
-      fail(text + " hyperlink should be there in the page content : "
-          + viewFilePage);
-    }
-  }
-  
-  @Test
-  public void testViewFileJsp() throws IOException {
-    MiniDFSCluster cluster = null;
-    try {
-      cluster = new MiniDFSCluster.Builder(CONF).build();
-      cluster.waitActive();
-      String paths[] = {
-        "/test-file",
-        "/tmp/test-file",
-        "/tmp/test-file%with goofy&characters",
-        "/foo bar/foo bar",
-        "/foo+bar/foo+bar",
-        "/foo;bar/foo;bar",
-        "/foo=bar/foo=bar",
-        "/foo,bar/foo,bar",
-        "/foo?bar/foo?bar",
-        "/foo\">bar/foo\">bar"
-      };
-      for (String p : paths) {
-        testViewingFile(cluster, p);
-        testViewingFile(cluster, p);
-      }
-    } finally {
-      if (cluster != null) {
-        cluster.shutdown();
-      }
-    }
-  }
-  
-  @Test
-  public void testGenStamp() throws Exception {
-    MiniDFSCluster cluster = new MiniDFSCluster.Builder(CONF).numDataNodes(1)
-        .build();
-    try {
-      FileSystem fs = cluster.getFileSystem();
-      Path testFile = new Path("/test/mkdirs/TestchunkSizeToView");
-      writeFile(fs, testFile);
-      JspWriter writerMock = Mockito.mock(JspWriter.class);
-      HttpServletRequest reqMock = Mockito.mock(HttpServletRequest.class);
-      setTheMockExpectationsFromReq(testFile, reqMock);
-      DatanodeJspHelper.generateFileDetails(writerMock, reqMock, CONF);
-      Mockito.verify(writerMock, Mockito.atLeastOnce()).print(
-          "<input type=\"hidden\" name=\"genstamp\" value=\"987654321\">");
-    } finally {
-      cluster.shutdown();
-    }
-  }
-
-  private void setTheMockExpectationsFromReq(Path testFile,
-      HttpServletRequest reqMock) {
-    Mockito.doReturn("987654321").when(reqMock).getParameter("genstamp");
-    Mockito.doReturn("1234").when(reqMock).getParameter("blockId");
-    Mockito.doReturn("8081").when(reqMock).getParameter("datanodePort");
-    Mockito.doReturn("8080").when(reqMock).getParameter("namenodeInfoPort");
-    Mockito.doReturn("100").when(reqMock).getParameter("chunkSizeToView");
-    Mockito.doReturn("1").when(reqMock).getParameter("startOffset");
-    Mockito.doReturn("1024").when(reqMock).getParameter("blockSize");
-    Mockito.doReturn(NetUtils.getHostPortString(NameNode.getAddress(CONF)))
-        .when(reqMock).getParameter("nnaddr");
-    Mockito.doReturn(testFile.toString()).when(reqMock).getPathInfo();
-    Mockito.doReturn("http").when(reqMock).getScheme();
-  }
-
-  static Path writeFile(FileSystem fs, Path f) throws IOException {
-    DataOutputStream out = fs.create(f);
-    try {
-      out.writeBytes("umamahesh: " + f);
-    } finally {
-      out.close();
-    }
-    assertTrue(fs.exists(f));
-    return f;
-  }
-
-}

+ 7 - 292
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNameNodeJspHelper.java

@@ -17,52 +17,24 @@
  */
 package org.apache.hadoop.hdfs.server.namenode;
 
-import static org.apache.hadoop.hdfs.server.namenode.startupprogress.Phase.LOADING_EDITS;
-import static org.apache.hadoop.hdfs.server.namenode.startupprogress.Phase.LOADING_FSIMAGE;
-import static org.apache.hadoop.hdfs.server.namenode.startupprogress.Phase.SAFEMODE;
-import static org.apache.hadoop.hdfs.server.namenode.startupprogress.Phase.SAVING_CHECKPOINT;
-import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.fail;
-import static org.mockito.Mockito.atLeastOnce;
-import static org.mockito.Mockito.doAnswer;
-import static org.mockito.Mockito.mock;
-import static org.mockito.Mockito.verify;
-import static org.mockito.Mockito.when;
-
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.List;
-import java.util.regex.Matcher;
-import java.util.regex.Pattern;
-
-import javax.servlet.ServletContext;
-import javax.servlet.http.HttpServletRequest;
-import javax.servlet.http.HttpServletResponse;
-import javax.servlet.jsp.JspWriter;
-
+import com.google.common.collect.ImmutableSet;
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor;
-import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeManager;
-import org.apache.hadoop.hdfs.server.common.JspHelper;
 import org.apache.hadoop.hdfs.server.datanode.DataNode;
-import org.apache.hadoop.hdfs.server.namenode.startupprogress.StartupProgress;
 import org.apache.hadoop.hdfs.server.protocol.NamenodeProtocols;
-import org.apache.hadoop.hdfs.web.resources.UserParam;
 import org.apache.hadoop.security.UserGroupInformation;
-import org.apache.hadoop.util.VersionInfo;
 import org.junit.AfterClass;
 import org.junit.Assert;
 import org.junit.BeforeClass;
 import org.junit.Test;
-import org.mockito.ArgumentCaptor;
-import org.mockito.invocation.InvocationOnMock;
-import org.mockito.stubbing.Answer;
-import org.znerd.xmlenc.XMLOutputter;
 
-import com.google.common.collect.ImmutableSet;
+import javax.servlet.http.HttpServletRequest;
+import java.io.IOException;
+
+import static org.junit.Assert.assertTrue;
+import static org.mockito.Mockito.mock;
 
 public class TestNameNodeJspHelper {
 
@@ -70,7 +42,6 @@ public class TestNameNodeJspHelper {
   
   private static MiniDFSCluster cluster;
   private static Configuration conf;
-  private static final String NAMENODE_ATTRIBUTE_KEY = "name.node";  
   
   @BeforeClass
   public static void setUp() throws Exception {
@@ -97,143 +68,14 @@ public class TestNameNodeJspHelper {
     Assert.assertEquals(null, tokenString);
   }
 
-  @Test
-  public void testSecurityModeText() {
-    conf.set(DFSConfigKeys.HADOOP_SECURITY_AUTHENTICATION, "kerberos");
-    UserGroupInformation.setConfiguration(conf);
-    String securityOnOff = NamenodeJspHelper.getSecurityModeText();
-    Assert.assertTrue("security mode doesn't match. Should be ON",
-        securityOnOff.contains("ON"));
-    // Security is enabled
-    conf.set(DFSConfigKeys.HADOOP_SECURITY_AUTHENTICATION, "simple");
-    UserGroupInformation.setConfiguration(conf);
-
-    securityOnOff = NamenodeJspHelper.getSecurityModeText();
-    Assert.assertTrue("security mode doesn't match. Should be OFF",
-        securityOnOff.contains("OFF"));
-  }
-
-  @Test
-  public void testGenerateStartupProgress() throws Exception {
-    cluster.waitClusterUp();
-    NamenodeJspHelper.HealthJsp jsp = new NamenodeJspHelper.HealthJsp();
-    StartupProgress prog = NameNode.getStartupProgress();
-    JspWriter out = mock(JspWriter.class);
-    jsp.generateStartupProgress(out, prog);
-    ArgumentCaptor<String> captor = ArgumentCaptor.forClass(String.class);
-    verify(out, atLeastOnce()).println(captor.capture());
-    List<String> contents = captor.getAllValues();
-
-    // Verify 100% overall completion and all phases mentioned in output.
-    Assert.assertTrue(containsMatch(contents, "Elapsed Time\\:"));
-    Assert.assertTrue(containsMatch(contents, "Percent Complete\\:.*?100\\.00%"));
-    Assert.assertTrue(containsMatch(contents, LOADING_FSIMAGE.getDescription()));
-    Assert.assertTrue(containsMatch(contents, LOADING_EDITS.getDescription()));
-    Assert.assertTrue(containsMatch(contents,
-      SAVING_CHECKPOINT.getDescription()));
-    Assert.assertTrue(containsMatch(contents, SAFEMODE.getDescription()));
-  }
-
-  @Test
-  public void testGetRollingUpgradeText() {
-    Assert.assertEquals("", NamenodeJspHelper.getRollingUpgradeText(null));
-  }
-
-  /**
-   * Tests for non-null, non-empty NameNode label.
-   */
-  @Test
-  public void testGetNameNodeLabel() {
-    String nameNodeLabel = NamenodeJspHelper.getNameNodeLabel(
-      cluster.getNameNode());
-    Assert.assertNotNull(nameNodeLabel);
-    Assert.assertFalse(nameNodeLabel.isEmpty());
-  }
-
-  /**
-   * Tests for non-null, non-empty NameNode label when called before
-   * initialization of the NameNode RPC server.
-   */
-  @Test
-  public void testGetNameNodeLabelNullRpcServer() {
-    NameNode nn = mock(NameNode.class);
-    when(nn.getRpcServer()).thenReturn(null);
-    String nameNodeLabel = NamenodeJspHelper.getNameNodeLabel(
-      cluster.getNameNode());
-    Assert.assertNotNull(nameNodeLabel);
-    Assert.assertFalse(nameNodeLabel.isEmpty());
-  }
-
-  /**
-   * Tests that passing a null FSNamesystem to generateSnapshotReport does not
-   * throw NullPointerException.
-   */
-  @Test
-  public void testGenerateSnapshotReportNullNamesystem() throws Exception {
-    NamenodeJspHelper.generateSnapshotReport(mock(JspWriter.class), null);
-  }
-
-  /**
-   * Tests that redirectToRandomDataNode does not throw NullPointerException if
-   * it finds a null FSNamesystem.
-   */
-  @Test(expected=IOException.class)
-  public void testRedirectToRandomDataNodeNullNamesystem() throws Exception {
-    NameNode nn = mock(NameNode.class);
-    when(nn.getNamesystem()).thenReturn(null);
-    ServletContext context = mock(ServletContext.class);
-    when(context.getAttribute("name.node")).thenReturn(nn);
-    NamenodeJspHelper.redirectToRandomDataNode(context,
-      mock(HttpServletRequest.class), mock(HttpServletResponse.class));
-  }
-
-  /**
-   * Tests that XMLBlockInfo does not throw NullPointerException if it finds a
-   * null FSNamesystem.
-   */
-  @Test
-  public void testXMLBlockInfoNullNamesystem() throws IOException {
-    XMLOutputter doc = new XMLOutputter(mock(JspWriter.class), "UTF-8");
-    new NamenodeJspHelper.XMLBlockInfo(null, 1L).toXML(doc);
-  }
-
-  /**
-   * Tests that XMLCorruptBlockInfo does not throw NullPointerException if it
-   * finds a null FSNamesystem.
-   */
-  @Test
-  public void testXMLCorruptBlockInfoNullNamesystem() throws IOException {
-    XMLOutputter doc = new XMLOutputter(mock(JspWriter.class), "UTF-8");
-    new NamenodeJspHelper.XMLCorruptBlockInfo(null, mock(Configuration.class),
-      10, 1L).toXML(doc);
-  }
-
-  /**
-   * Checks if the list contains any string that partially matches the regex.
-   *
-   * @param list List<String> containing strings to check
-   * @param regex String regex to check
-   * @return boolean true if some string in list partially matches regex
-   */
-  private static boolean containsMatch(List<String> list, String regex) {
-    Pattern pattern = Pattern.compile(regex);
-    for (String str: list) {
-      if (pattern.matcher(str).find()) {
-        return true;
-      }
-    }
-    return false;
-  }
-
   @Test(timeout = 15000)
   public void testGetRandomDatanode() {
-    ImmutableSet<String> set = ImmutableSet.of();
     NameNode nameNode = cluster.getNameNode();
     ImmutableSet.Builder<String> builder = ImmutableSet.builder();
     for (DataNode dataNode : cluster.getDataNodes()) {
       builder.add(dataNode.getDisplayName());
     }
-    set = builder.build();
+    ImmutableSet<String> set = builder.build();
 
     for (int i = 0; i < 10; i++) {
       DatanodeDescriptor dnDescriptor = NamenodeJspHelper
@@ -242,131 +84,4 @@ public class TestNameNodeJspHelper {
           set.contains(dnDescriptor.toString()));
     }
   }
-      
-  @Test(timeout = 15000)
-  public void testNamenodeJspHelperRedirectToRandomDataNode() throws IOException, InterruptedException {
-    final String urlPart = "browseDirectory.jsp?namenodeInfoPort=";                     
-    
-    ServletContext context = mock(ServletContext.class);
-    HttpServletRequest request = mock(HttpServletRequest.class);
-    HttpServletResponse resp = mock(HttpServletResponse.class);          
-    
-    when(request.getScheme()).thenReturn("http");
-    when(request.getParameter(UserParam.NAME)).thenReturn("localuser");
-    when(context.getAttribute(NAMENODE_ATTRIBUTE_KEY)).thenReturn(
-        cluster.getNameNode());
-    when(context.getAttribute(JspHelper.CURRENT_CONF)).thenReturn(conf);    
-    ArgumentCaptor<String> captor = ArgumentCaptor.forClass(String.class);
-    doAnswer(new Answer<String>() {
-      @Override
-     public String answer(InvocationOnMock invocation) throws Throwable {
-        return null;
-        }
-    }).when(resp).sendRedirect(captor.capture());
-
-    NamenodeJspHelper.redirectToRandomDataNode(context, request, resp);    
-    assertTrue(captor.getValue().contains(urlPart));    
-  }
-  
-  private enum DataNodeStatus {
-    LIVE("[Live Datanodes(| +):(| +)]\\d"), 
-    DEAD("[Dead Datanodes(| +):(| +)]\\d");
-
-    private final Pattern pattern;
-
-    public Pattern getPattern() {
-      return pattern;
-    }
-
-    DataNodeStatus(String line) {
-      this.pattern = Pattern.compile(line);
-    }
-  }
-
-  private void checkDeadLiveNodes(NameNode nameNode, int deadCount,
-      int lifeCount) {
-    FSNamesystem ns = nameNode.getNamesystem();
-    DatanodeManager dm = ns.getBlockManager().getDatanodeManager();
-    List<DatanodeDescriptor> live = new ArrayList<DatanodeDescriptor>();
-    List<DatanodeDescriptor> dead = new ArrayList<DatanodeDescriptor>();
-    dm.fetchDatanodes(live, dead, true);
-    assertTrue("checkDeadLiveNodes error !!!", (live.size() == lifeCount)
-        && dead.size() == deadCount);
-  }
-
-  @Test(timeout = 15000)
-  public void testNodeListJspGenerateNodesList() throws IOException {
-    String output;
-    NameNode nameNode = cluster.getNameNode();
-    ServletContext context = mock(ServletContext.class);
-    when(context.getAttribute("name.node")).thenReturn(nameNode);
-    when(context.getAttribute(NameNodeHttpServer.NAMENODE_ADDRESS_ATTRIBUTE_KEY))
-        .thenReturn(cluster.getNameNode().getHttpAddress());    
-    checkDeadLiveNodes(nameNode, 0, DATA_NODES_AMOUNT);
-    output = getOutputFromGeneratedNodesList(context, DataNodeStatus.LIVE);
-    assertCounts(DataNodeStatus.LIVE, output, DATA_NODES_AMOUNT);
-    output = getOutputFromGeneratedNodesList(context, DataNodeStatus.DEAD);
-    assertCounts(DataNodeStatus.DEAD, output, 0);    
-  }
-
-  private void assertCounts(DataNodeStatus dataNodeStatus, String output,
-      int expectedCount) {
-    Matcher matcher = DataNodeStatus.LIVE.getPattern().matcher(output);
-    if (matcher.find()) {
-      String digitLine = output.substring(matcher.start(), matcher.end())
-          .trim();
-      assertTrue("assertCounts error. actual != expected",
-          Integer.parseInt(digitLine) == expectedCount);
-    } else {
-      fail("assertCount matcher error");
-    }
-  }
-
-  private String getOutputFromGeneratedNodesList(ServletContext context,
-      DataNodeStatus dnStatus) throws IOException {
-    JspWriter out = mock(JspWriter.class);
-    ArgumentCaptor<String> captor = ArgumentCaptor.forClass(String.class);
-    NamenodeJspHelper.NodeListJsp nodelistjsp = new NamenodeJspHelper.NodeListJsp();
-    final StringBuffer buffer = new StringBuffer();
-    doAnswer(new Answer<String>() {
-      @Override
-      public String answer(InvocationOnMock invok) {
-        Object[] args = invok.getArguments();
-        buffer.append((String) args[0]);
-        return null;
-      }
-    }).when(out).print(captor.capture());
-    HttpServletRequest request = mock(HttpServletRequest.class);
-    when(request.getScheme()).thenReturn("http");
-    when(request.getParameter("whatNodes")).thenReturn(dnStatus.name());
-    nodelistjsp.generateNodesList(context, out, request);
-    return buffer.toString();
-  }
-
-  @Test(timeout = 15000)
-  public void testGetInodeLimitText() {
-    NameNode nameNode = cluster.getNameNode();
-    FSNamesystem fsn = nameNode.getNamesystem();
-    ImmutableSet<String> patterns = 
-        ImmutableSet.of("files and directories", "Heap Memory used", "Non Heap Memory used");        
-    String line = NamenodeJspHelper.getInodeLimitText(fsn);    
-    for(String pattern: patterns) {
-      assertTrue("testInodeLimitText error " + pattern,
-          line.contains(pattern));
-    }    
-  }
-  
-  @Test(timeout = 15000)
-  public void testGetVersionTable() {
-    NameNode nameNode = cluster.getNameNode();
-    FSNamesystem fsn = nameNode.getNamesystem();
-    ImmutableSet<String> patterns = ImmutableSet.of(VersionInfo.getVersion(), 
-        VersionInfo.getRevision(), VersionInfo.getUser(), VersionInfo.getBranch(),
-        fsn.getClusterId(), fsn.getBlockPoolId());
-    String line = NamenodeJspHelper.getVersionTable(fsn);
-    for(String pattern: patterns) {
-       assertTrue("testGetVersionTable error " + pattern,
-           line.contains(pattern));
-    }
-  }  
 }