Browse Source

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

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/trunk@1591732 13f79535-47bb-0310-9956-ffa450edef68
Haohui Mai 11 years ago
parent
commit
6420249d47
34 changed files with 38 additions and 5724 deletions
  1. 2 0
      hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
  2. 1 518
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/JspHelper.java
  3. 0 705
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DatanodeJspHelper.java
  4. 0 922
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ClusterJspHelper.java
  5. 0 1200
      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. 3 2
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDecommission.java
  24. 20 22
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestMissingBlocksAlert.java
  25. 1 55
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/qjournal/TestNNWithQJM.java
  26. 0 5
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/qjournal/server/TestJournalNode.java
  27. 0 200
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/common/TestJspHelper.java
  28. 0 188
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDatanodeJsp.java
  29. 0 59
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestClusterJspHelper.java
  30. 0 131
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestCorruptFilesJsp.java
  31. 11 14
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestHostsFiles.java
  32. 0 372
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNameNodeJspHelper.java
  33. 0 14
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestSecondaryWebUi.java
  34. 0 74
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestHAWebUI.java

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

@@ -127,6 +127,8 @@ Trunk (Unreleased)
 
     HDFS-6246. Remove 'dfs.support.append' flag from trunk code. (umamahesh)
 
+    HDFS-6252. Phase out the old web UI in HDFS. (wheat9)
+
   OPTIMIZATIONS
 
   BUG FIXES

+ 1 - 518
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/JspHelper.java

@@ -87,475 +87,10 @@ public class JspHelper {
   public static final String CURRENT_CONF = "current.conf";
   public static final String DELEGATION_PARAMETER_NAME = DelegationParam.NAME;
   public static final String NAMENODE_ADDRESS = "nnaddr";
-  static final String SET_DELEGATION = "&" + DELEGATION_PARAMETER_NAME +
-                                              "=";
   private static final Log LOG = LogFactory.getLog(JspHelper.class);
 
   /** Private constructor for preventing creating JspHelper object. */
-  private JspHelper() {} 
-  
-  // data structure to count number of blocks on datanodes.
-  private static class NodeRecord extends DatanodeInfo {
-    int frequency;
-
-    public NodeRecord(DatanodeInfo info, int count) {
-      super(info);
-      this.frequency = count;
-    }
-    
-    @Override
-    public boolean equals(Object obj) {
-      // Sufficient to use super equality as datanodes are uniquely identified
-      // by DatanodeID
-      return (this == obj) || super.equals(obj);
-    }
-    @Override
-    public int hashCode() {
-      // Super implementation is sufficient
-      return super.hashCode();
-    }
-  }
-
-  // compare two records based on their frequency
-  private static class NodeRecordComparator implements Comparator<NodeRecord> {
-
-    @Override
-    public int compare(NodeRecord o1, NodeRecord o2) {
-      if (o1.frequency < o2.frequency) {
-        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);
-    }
-  }
-
-  public static DatanodeInfo bestNode(LocatedBlocks blks, Configuration conf)
-      throws IOException {
-    HashMap<DatanodeInfo, NodeRecord> map =
-      new HashMap<DatanodeInfo, NodeRecord>();
-    for (LocatedBlock block : blks.getLocatedBlocks()) {
-      DatanodeInfo[] nodes = block.getLocations();
-      for (DatanodeInfo node : nodes) {
-        NodeRecord record = map.get(node);
-        if (record == null) {
-          map.put(node, new NodeRecord(node, 1));
-        } else {
-          record.frequency++;
-        }
-      }
-    }
-    NodeRecord[] nodes = map.values().toArray(new NodeRecord[map.size()]);
-    Arrays.sort(nodes, new NodeRecordComparator());
-    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) {
-      throw new IOException("No nodes contain this block");
-    }
-    int l = 0;
-    while (l < nodes.length && !nodes[l].isDecommissioned()) {
-      ++l;
-    }
-
-    if (l == 0) {
-      throw new IOException("No active nodes contain this block");
-    }
-
-    int index = doRandom ? DFSUtil.getRandom().nextInt(l) : 0;
-    return nodes[index];
-  }
-
-  public static void streamBlockInAscii(InetSocketAddress addr, String poolId,
-      long blockId, Token<BlockTokenIdentifier> blockToken, long genStamp,
-      long blockSize, long offsetIntoBlock, long chunkSizeToView,
-      JspWriter out, final Configuration conf, DFSClient.Conf dfsConf,
-      final DataEncryptionKey encryptionKey)
-          throws IOException {
-    if (chunkSizeToView == 0) return;
-    int amtToRead = (int)Math.min(chunkSizeToView, blockSize - offsetIntoBlock);
-      
-    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(
-          new DatanodeID(addr.getAddress().getHostAddress(),
-              addr.getHostName(), poolId, addr.getPort(), 0, 0, 0))).
-      setCachingStrategy(CachingStrategy.newDefaultStrategy()).
-      setConfiguration(conf).
-      setRemotePeerFactory(new RemotePeerFactory() {
-        @Override
-        public Peer newConnectedPeer(InetSocketAddress addr)
-            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(sock, encryptionKey);
-          } 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,
-        FIELD_BLOCKPOOL_USED    = 12,
-        FIELD_PERBLOCKPOOL_USED = 13,
-        FIELD_FAILED_VOLUMES    = 14,
-        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 if (field.equals("bpused")) {
-          sortField = FIELD_BLOCKPOOL_USED;
-        } else if (field.equals("pcbpused")) {
-          sortField = FIELD_PERBLOCKPOOL_USED;
-        } else if (field.equals("volfails")) {
-          sortField = FIELD_FAILED_VOLUMES;
-        } 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;
-        case FIELD_BLOCKPOOL_USED:
-          dlong = d1.getBlockPoolUsed() - d2.getBlockPoolUsed();
-          ret = (dlong < 0) ? -1 : ((dlong > 0) ? 1 : 0);
-          break;
-        case FIELD_PERBLOCKPOOL_USED:
-          ddbl = d1.getBlockPoolUsedPercent() - d2.getBlockPoolUsedPercent();
-          ret = (ddbl < 0) ? -1 : ((ddbl > 0) ? 1 : 0);
-          break;
-        case FIELD_FAILED_VOLUMES:
-          int dint = d1.getVolumeFailures() - d2.getVolumeFailures();
-          ret = (dint < 0) ? -1 : ((dint > 0) ? 1 : 0);
-          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.
-   *         Otherwise, return the validated filename.
-   */
-  public static String validatePath(String p) {
-    return p == null || p.length() == 0?
-        null: new Path(p).toUri().getPath();
-  }
-
-  /**
-   * Validate a long value. 
-   * @return null if the value is invalid.
-   *         Otherwise, return the validated Long object.
-   */
-  public static Long validateLong(String value) {
-    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 JspHelper() {}
 
   private static String getDefaultWebUserName(Configuration conf
       ) throws IOException {
@@ -736,56 +271,4 @@ public class JspHelper {
     return username;
   }
 
-  /**
-   * Returns the url parameter for the given token string.
-   * @param tokenString
-   * @return url parameter
-   */
-  public static String getDelegationTokenUrlParam(String tokenString) {
-    if (tokenString == null ) {
-      return "";
-    }
-    if (UserGroupInformation.isSecurityEnabled()) {
-      return SET_DELEGATION + tokenString;
-    } else {
-      return "";
-    }
-  }
-
-  /**
-   * Returns the url parameter for the given string, prefixed with
-   * paramSeparator.
-   * 
-   * @param name parameter name
-   * @param val parameter value
-   * @param paramSeparator URL parameter prefix, i.e. either '?' or '&'
-   * @return url parameter
-   */
-  public static String getUrlParam(String name, String val, String paramSeparator) {
-    return val == null ? "" : paramSeparator + name + "=" + val;
-  }
-  
-  /**
-   * Returns the url parameter for the given string, prefixed with '?' if
-   * firstParam is true, prefixed with '&' if firstParam is false.
-   * 
-   * @param name parameter name
-   * @param val parameter value
-   * @param firstParam true if this is the first parameter in the list, false otherwise
-   * @return url parameter
-   */
-  public static String getUrlParam(String name, String val, boolean firstParam) {
-    return getUrlParam(name, val, firstParam ? "?" : "&");
-  }
-  
-  /**
-   * Returns the url parameter for the given string, prefixed with '&'.
-   * 
-   * @param name parameter name
-   * @param val parameter value
-   * @return url parameter
-   */
-  public static String getUrlParam(String name, String val) {
-    return getUrlParam(name, val, false);
-  }
 }

+ 0 - 705
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DatanodeJspHelper.java

@@ -1,705 +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 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.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;
-
-@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
-                                        ) throws IOException,
-                                                 InterruptedException {
-    return
-      user.doAs(new PrivilegedExceptionAction<DFSClient>() {
-        @Override
-        public DFSClient run() throws IOException {
-          return new DFSClient(NetUtils.createSocketAddr(addr), conf);
-        }
-      });
-  }
-
-  /**
-   * 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.getDataEncryptionKey());
-    } 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.getDataEncryptionKey());
-    out.print("</textarea>");
-    dfs.close();
-  }
-  
-  
-  /** Get DFSClient for a namenode corresponding to the BPID from a datanode */
-  public static DFSClient getDFSClient(final HttpServletRequest request,
-      final DataNode datanode, final Configuration conf,
-      final UserGroupInformation ugi) throws IOException, InterruptedException {
-    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();
-  }
-
-}

+ 0 - 922
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ClusterJspHelper.java

@@ -1,922 +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.namenode;
-
-import java.io.BufferedReader;
-import java.io.IOException;
-import java.io.InputStreamReader;
-import java.net.InetSocketAddress;
-import java.net.URI;
-import java.net.URL;
-import java.net.URLConnection;
-import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.Iterator;
-import java.util.List;
-import java.util.Map;
-import java.util.Map.Entry;
-import java.util.Set;
-
-import javax.management.MalformedObjectNameException;
-
-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.hdfs.DFSConfigKeys;
-import org.apache.hadoop.hdfs.DFSUtil;
-import org.apache.hadoop.hdfs.DFSUtil.ConfiguredNNAddress;
-import org.apache.hadoop.hdfs.protocol.DatanodeInfo.AdminStates;
-import org.apache.hadoop.util.StringUtils;
-import org.codehaus.jackson.JsonNode;
-import org.codehaus.jackson.map.ObjectMapper;
-import org.codehaus.jackson.type.TypeReference;
-import org.znerd.xmlenc.XMLOutputter;
-
-import com.google.common.base.Charsets;
-
-/**
- * This class generates the data that is needed to be displayed on cluster web 
- * console.
- */
-@InterfaceAudience.Private
-class ClusterJspHelper {
-  private static final Log LOG = LogFactory.getLog(ClusterJspHelper.class);
-  public static final String OVERALL_STATUS = "overall-status";
-  public static final String DEAD = "Dead";
-  private static final String JMX_QRY = 
-    "/jmx?qry=Hadoop:service=NameNode,name=NameNodeInfo";
-  
-  /**
-   * JSP helper function that generates cluster health report.  When 
-   * encountering exception while getting Namenode status, the exception will 
-   * be listed on the page with corresponding stack trace.
-   */
-  ClusterStatus generateClusterHealthReport() {
-    ClusterStatus cs = new ClusterStatus();
-    Configuration conf = new Configuration();
-    List<ConfiguredNNAddress> nns = null;
-    try {
-      nns = DFSUtil.flattenAddressMap(
-          DFSUtil.getNNServiceRpcAddresses(conf));
-    } catch (Exception e) {
-      // Could not build cluster status
-      cs.setError(e);
-      return cs;
-    }
-    
-    // Process each namenode and add it to ClusterStatus
-    for (ConfiguredNNAddress cnn : nns) {
-      InetSocketAddress isa = cnn.getAddress();
-      NamenodeMXBeanHelper nnHelper = null;
-      try {
-        nnHelper = new NamenodeMXBeanHelper(isa, conf);
-        String mbeanProps= queryMbean(nnHelper.httpAddress, conf);
-        NamenodeStatus nn = nnHelper.getNamenodeStatus(mbeanProps);
-        if (cs.clusterid.isEmpty() || cs.clusterid.equals("")) { // Set clusterid only once
-          cs.clusterid = nnHelper.getClusterId(mbeanProps);
-        }
-        cs.addNamenodeStatus(nn);
-      } catch ( Exception e ) {
-        // track exceptions encountered when connecting to namenodes
-        cs.addException(isa.getHostName(), e);
-        continue;
-      } 
-    }
-    return cs;
-  }
-
-  /**
-   * Helper function that generates the decommissioning report.  Connect to each
-   * Namenode over http via JmxJsonServlet to collect the data nodes status.
-   */
-  DecommissionStatus generateDecommissioningReport() {
-    String clusterid = "";
-    Configuration conf = new Configuration();
-    List<ConfiguredNNAddress> cnns = null;
-    try {
-      cnns = DFSUtil.flattenAddressMap(
-          DFSUtil.getNNServiceRpcAddresses(conf));
-    } catch (Exception e) {
-      // catch any exception encountered other than connecting to namenodes
-      DecommissionStatus dInfo = new DecommissionStatus(clusterid, e);
-      return dInfo;
-    }
-    
-    // Outer map key is datanode. Inner map key is namenode and the value is 
-    // decom status of the datanode for the corresponding namenode
-    Map<String, Map<String, String>> statusMap = 
-      new HashMap<String, Map<String, String>>();
-    
-    // Map of exceptions encountered when connecting to namenode
-    // key is namenode and value is exception
-    Map<String, Exception> decommissionExceptions = 
-      new HashMap<String, Exception>();
-    
-    List<String> unreportedNamenode = new ArrayList<String>();
-    for (ConfiguredNNAddress cnn : cnns) {
-      InetSocketAddress isa = cnn.getAddress();
-      NamenodeMXBeanHelper nnHelper = null;
-      try {
-        nnHelper = new NamenodeMXBeanHelper(isa, conf);
-        String mbeanProps= queryMbean(nnHelper.httpAddress, conf);
-        if (clusterid.equals("")) {
-          clusterid = nnHelper.getClusterId(mbeanProps);
-        }
-        nnHelper.getDecomNodeInfoForReport(statusMap, mbeanProps);
-      } catch (Exception e) {
-        // catch exceptions encountered while connecting to namenodes
-        String nnHost = isa.getHostName();
-        decommissionExceptions.put(nnHost, e);
-        unreportedNamenode.add(nnHost);
-        continue;
-      } 
-    }
-    updateUnknownStatus(statusMap, unreportedNamenode);
-    getDecommissionNodeClusterState(statusMap);
-    return new DecommissionStatus(statusMap, clusterid,
-        getDatanodeHttpPort(conf), decommissionExceptions);
-  }
-  
-  /**
-   * Based on the state of the datanode at each namenode, marks the overall
-   * state of the datanode across all the namenodes, to one of the following:
-   * <ol>
-   * <li>{@link DecommissionStates#DECOMMISSIONED}</li>
-   * <li>{@link DecommissionStates#DECOMMISSION_INPROGRESS}</li>
-   * <li>{@link DecommissionStates#PARTIALLY_DECOMMISSIONED}</li>
-   * <li>{@link DecommissionStates#UNKNOWN}</li>
-   * </ol>
-   * 
-   * @param statusMap
-   *          map whose key is datanode, value is an inner map with key being
-   *          namenode, value being decommission state.
-   */
-  private void getDecommissionNodeClusterState(
-      Map<String, Map<String, String>> statusMap) {
-    if (statusMap == null || statusMap.isEmpty()) {
-      return;
-    }
-    
-    // For each datanodes
-    Iterator<Entry<String, Map<String, String>>> it = 
-      statusMap.entrySet().iterator();
-    while (it.hasNext()) {
-      // Map entry for a datanode:
-      // key is namenode, value is datanode status at the namenode
-      Entry<String, Map<String, String>> entry = it.next();
-      Map<String, String> nnStatus = entry.getValue();
-      if (nnStatus == null || nnStatus.isEmpty()) {
-        continue;
-      }
-      
-      boolean isUnknown = false;
-      int unknown = 0;
-      int decommissioned = 0;
-      int decomInProg = 0;
-      int inservice = 0;
-      int dead = 0;
-      DecommissionStates overallState = DecommissionStates.UNKNOWN;
-      // Process a datanode state from each namenode
-      for (Map.Entry<String, String> m : nnStatus.entrySet()) {
-        String status = m.getValue();
-        if (status.equals(DecommissionStates.UNKNOWN.toString())) {
-          isUnknown = true;
-          unknown++;
-        } else 
-          if (status.equals(AdminStates.DECOMMISSION_INPROGRESS.toString())) {
-          decomInProg++;
-        } else if (status.equals(AdminStates.DECOMMISSIONED.toString())) {
-          decommissioned++;
-        } else if (status.equals(AdminStates.NORMAL.toString())) {
-          inservice++;
-        } else if (status.equals(DEAD)) {
-          // dead
-          dead++;
-        }
-      }
-      
-      // Consolidate all the states from namenode in to overall state
-      int nns = nnStatus.keySet().size();
-      if ((inservice + dead + unknown) == nns) {
-        // Do not display this data node. Remove this entry from status map.  
-        it.remove();
-      } else if (isUnknown) {
-        overallState = DecommissionStates.UNKNOWN;
-      } else if (decommissioned == nns) {
-        overallState = DecommissionStates.DECOMMISSIONED;
-      } else if ((decommissioned + decomInProg) == nns) {
-        overallState = DecommissionStates.DECOMMISSION_INPROGRESS;
-      } else if ((decommissioned + decomInProg < nns) 
-        && (decommissioned + decomInProg > 0)){
-        overallState = DecommissionStates.PARTIALLY_DECOMMISSIONED;
-      } else {
-        LOG.warn("Cluster console encounters a not handled situtation.");
-      }
-        
-      // insert overall state
-      nnStatus.put(OVERALL_STATUS, overallState.toString());
-    }
-  }
-
-  /**
-   * update unknown status in datanode status map for every unreported namenode
-   */
-  private void updateUnknownStatus(Map<String, Map<String, String>> statusMap,
-      List<String> unreportedNn) {
-    if (unreportedNn == null || unreportedNn.isEmpty()) {
-      // no unreported namenodes
-      return;
-    }
-    
-    for (Map.Entry<String, Map<String,String>> entry : statusMap.entrySet()) {
-      String dn = entry.getKey();
-      Map<String, String> nnStatus = entry.getValue();
-      for (String nn : unreportedNn) {
-        nnStatus.put(nn, DecommissionStates.UNKNOWN.toString());
-      }
-      statusMap.put(dn, nnStatus);
-    }
-  }
-
-  /**
-   * Get datanode http port from configration
-   */
-  private int getDatanodeHttpPort(Configuration conf) {
-    String address = conf.get(DFSConfigKeys.DFS_DATANODE_HTTP_ADDRESS_KEY, "");
-    if (address.equals("")) {
-      return -1;
-    }
-    return Integer.parseInt(address.split(":")[1]);
-  }
-  
-  /**
-   * Class for connecting to Namenode over http via JmxJsonServlet 
-   * to get JMX attributes exposed by the MXBean.  
-   */
-  static class NamenodeMXBeanHelper {
-    private static final ObjectMapper mapper = new ObjectMapper();
-    private final String host;
-    private final URI httpAddress;
-    
-    NamenodeMXBeanHelper(InetSocketAddress addr, Configuration conf)
-        throws IOException, MalformedObjectNameException {
-      this.host = addr.getHostName();
-      this.httpAddress = DFSUtil.getInfoServer(addr, conf,
-          DFSUtil.getHttpClientScheme(conf));
-    }
-    
-    /** Get the map corresponding to the JSON string */
-    private static Map<String, Map<String, Object>> getNodeMap(String json)
-        throws IOException {
-      TypeReference<Map<String, Map<String, Object>>> type = 
-        new TypeReference<Map<String, Map<String, Object>>>() { };
-      return mapper.readValue(json, type);
-    }
-    
-    /**
-     * Get the number of live datanodes.
-     * 
-     * @param json JSON string that contains live node status.
-     * @param nn namenode status to return information in
-     */
-    private static void getLiveNodeCount(String json, NamenodeStatus nn)
-        throws IOException {
-      // Map of datanode host to (map of attribute name to value)
-      Map<String, Map<String, Object>> nodeMap = getNodeMap(json);
-      if (nodeMap == null || nodeMap.isEmpty()) {
-        return;
-      }
-      
-      nn.liveDatanodeCount = nodeMap.size();
-      for (Entry<String, Map<String, Object>> entry : nodeMap.entrySet()) {
-        // Inner map of attribute name to value
-        Map<String, Object> innerMap = entry.getValue();
-        if (innerMap != null) {
-          if (innerMap.get("adminState")
-              .equals(AdminStates.DECOMMISSIONED.toString())) {
-            nn.liveDecomCount++;
-          }
-        }
-      }
-    }
-  
-    /**
-     * Count the number of dead datanode.
-     * 
-     * @param nn namenode
-     * @param json JSON string
-     */
-    private static void getDeadNodeCount(String json, NamenodeStatus nn)
-        throws IOException {
-      Map<String, Map<String, Object>> nodeMap = getNodeMap(json);
-      if (nodeMap == null || nodeMap.isEmpty()) {
-        return;
-      }
-      
-      nn.deadDatanodeCount = nodeMap.size();
-      for (Entry<String, Map<String, Object>> entry : nodeMap.entrySet()) {
-        Map<String, Object> innerMap = entry.getValue();
-        if (innerMap != null && !innerMap.isEmpty()) {
-          if (((Boolean) innerMap.get("decommissioned"))
-              .booleanValue() == true) {
-            nn.deadDecomCount++;
-          }
-        }
-      }
-    }
-  
-    public String getClusterId(String props) throws IOException {
-      return getProperty(props, "ClusterId").getTextValue();
-    }
-    
-    public NamenodeStatus getNamenodeStatus(String props) throws IOException,
-        MalformedObjectNameException, NumberFormatException {
-      NamenodeStatus nn = new NamenodeStatus();
-      nn.host = host;
-      nn.filesAndDirectories = getProperty(props, "TotalFiles").getLongValue();
-      nn.capacity = getProperty(props, "Total").getLongValue();
-      nn.free = getProperty(props, "Free").getLongValue();
-      nn.bpUsed = getProperty(props, "BlockPoolUsedSpace").getLongValue();
-      nn.nonDfsUsed = getProperty(props, "NonDfsUsedSpace").getLongValue();
-      nn.blocksCount = getProperty(props, "TotalBlocks").getLongValue();
-      nn.missingBlocksCount = getProperty(props, "NumberOfMissingBlocks")
-          .getLongValue();
-      nn.httpAddress = httpAddress.toURL();
-      getLiveNodeCount(getProperty(props, "LiveNodes").asText(), nn);
-      getDeadNodeCount(getProperty(props, "DeadNodes").asText(), nn);
-      nn.softwareVersion = getProperty(props, "SoftwareVersion").getTextValue();
-      return nn;
-    }
-    
-    /**
-     * Get the decommission node information.
-     * @param statusMap data node status map
-     * @param props string
-     */
-    private void getDecomNodeInfoForReport(
-        Map<String, Map<String, String>> statusMap, String props)
-        throws IOException, MalformedObjectNameException {
-      getLiveNodeStatus(statusMap, host, getProperty(props, "LiveNodes")
-          .asText());
-      getDeadNodeStatus(statusMap, host, getProperty(props, "DeadNodes")
-          .asText());
-      getDecommissionNodeStatus(statusMap, host,
-          getProperty(props, "DecomNodes").asText());
-    }
-  
-    /**
-     * Store the live datanode status information into datanode status map and
-     * DecommissionNode.
-     * 
-     * @param statusMap Map of datanode status. Key is datanode, value
-     *          is an inner map whose key is namenode, value is datanode status.
-     *          reported by each namenode.
-     * @param namenodeHost host name of the namenode
-     * @param json JSON string contains datanode status
-     * @throws IOException
-     */
-    private static void getLiveNodeStatus(
-        Map<String, Map<String, String>> statusMap, String namenodeHost,
-        String json) throws IOException {
-      Map<String, Map<String, Object>> nodeMap = getNodeMap(json);
-      if (nodeMap != null && !nodeMap.isEmpty()) {
-        List<String> liveDecommed = new ArrayList<String>();
-        for (Map.Entry<String, Map<String, Object>> entry: nodeMap.entrySet()) {
-          Map<String, Object> innerMap = entry.getValue();
-          String dn = entry.getKey();
-          if (innerMap != null) {
-            if (innerMap.get("adminState").equals(
-                AdminStates.DECOMMISSIONED.toString())) {
-              liveDecommed.add(dn);
-            }
-            // the inner map key is namenode, value is datanode status.
-            Map<String, String> nnStatus = statusMap.get(dn);
-            if (nnStatus == null) {
-              nnStatus = new HashMap<String, String>();
-            }
-            nnStatus.put(namenodeHost, (String) innerMap.get("adminState"));
-            // map whose key is datanode, value is the inner map.
-            statusMap.put(dn, nnStatus);
-          }
-        }
-      }
-    }
-  
-    /**
-     * Store the dead datanode information into datanode status map and
-     * DecommissionNode.
-     * 
-     * @param statusMap map with key being datanode, value being an
-     *          inner map (key:namenode, value:decommisionning state).
-     * @param host datanode hostname
-     * @param json String
-     * @throws IOException
-     */
-    private static void getDeadNodeStatus(
-        Map<String, Map<String, String>> statusMap, String host,
-        String json) throws IOException {
-      Map<String, Map<String, Object>> nodeMap = getNodeMap(json);
-      if (nodeMap == null || nodeMap.isEmpty()) {
-        return;
-      }
-      List<String> deadDn = new ArrayList<String>();
-      List<String> deadDecommed = new ArrayList<String>();
-      for (Entry<String, Map<String, Object>> entry : nodeMap.entrySet()) {
-        deadDn.add(entry.getKey());
-        Map<String, Object> deadNodeDetailMap = entry.getValue();
-        String dn = entry.getKey();
-        if (deadNodeDetailMap != null && !deadNodeDetailMap.isEmpty()) {
-          // NN - status
-          Map<String, String> nnStatus = statusMap.get(dn);
-          if (nnStatus == null) {
-            nnStatus = new HashMap<String, String>();
-          }
-          if (((Boolean) deadNodeDetailMap.get("decommissioned"))
-              .booleanValue() == true) {
-            deadDecommed.add(dn);
-            nnStatus.put(host, AdminStates.DECOMMISSIONED.toString());
-          } else {
-            nnStatus.put(host, DEAD);
-          }
-          // dn-nn-status
-          statusMap.put(dn, nnStatus);
-        }
-      }
-    }
-  
-    /**
-     * Get the decommisioning datanode information.
-     * 
-     * @param dataNodeStatusMap map with key being datanode, value being an
-     *          inner map (key:namenode, value:decommisionning state).
-     * @param host datanode
-     * @param json String
-     */
-    private static void getDecommissionNodeStatus(
-        Map<String, Map<String, String>> dataNodeStatusMap, String host,
-        String json) throws IOException {
-      Map<String, Map<String, Object>> nodeMap = getNodeMap(json);
-      if (nodeMap == null || nodeMap.isEmpty()) {
-        return;
-      }
-      List<String> decomming = new ArrayList<String>();
-      for (Entry<String, Map<String, Object>> entry : nodeMap.entrySet()) {
-        String dn = entry.getKey();
-        decomming.add(dn);
-        // nn-status
-        Map<String, String> nnStatus = new HashMap<String, String>();
-        if (dataNodeStatusMap.containsKey(dn)) {
-          nnStatus = dataNodeStatusMap.get(dn);
-        }
-        nnStatus.put(host, AdminStates.DECOMMISSION_INPROGRESS.toString());
-        // dn-nn-status
-        dataNodeStatusMap.put(dn, nnStatus);
-      }
-    }
-  }
-
-  /**
-   * This class contains cluster statistics.
-   */
-  static class ClusterStatus {
-    /** Exception indicates failure to get cluster status */
-    Exception error = null;
-    
-    /** Cluster status information */
-    String clusterid = "";
-    long total_sum = 0;
-    long free_sum = 0;
-    long clusterDfsUsed = 0;
-    long nonDfsUsed_sum = 0;
-    long totalFilesAndDirectories = 0;
-    
-    /** List of namenodes in the cluster */
-    final List<NamenodeStatus> nnList = new ArrayList<NamenodeStatus>();
-    
-    /** Map of namenode host and exception encountered when getting status */
-    final Map<String, Exception> nnExceptions = new HashMap<String, Exception>();
-    
-    public void setError(Exception e) {
-      error = e;
-    }
-    
-    public void addNamenodeStatus(NamenodeStatus nn) {
-      nnList.add(nn);
-      
-      // Add namenode status to cluster status
-      totalFilesAndDirectories += nn.filesAndDirectories;
-      total_sum += nn.capacity;
-      free_sum += nn.free;
-      clusterDfsUsed += nn.bpUsed;
-      nonDfsUsed_sum += nn.nonDfsUsed;
-    }
-
-    public void addException(String host, Exception e) {
-      nnExceptions.put(host, e);
-    }
-
-    public void toXML(XMLOutputter doc) throws IOException {
-      if (error != null) {
-        // general exception, only print exception message onto web page.
-        createGeneralException(doc, clusterid,
-            StringUtils.stringifyException(error));
-        doc.getWriter().flush();
-        return;
-      }
-      
-      int size = nnList.size();
-      long total = 0L, free = 0L, nonDfsUsed = 0l;
-      float dfsUsedPercent = 0.0f, dfsRemainingPercent = 0.0f;
-      if (size > 0) {
-        total = total_sum / size;
-        free = free_sum / size;
-        nonDfsUsed = nonDfsUsed_sum / size;
-        dfsUsedPercent = DFSUtil.getPercentUsed(clusterDfsUsed, total);
-        dfsRemainingPercent = DFSUtil.getPercentRemaining(free, total);
-      }
-    
-      doc.startTag("cluster");
-      doc.attribute("clusterId", clusterid);
-    
-      doc.startTag("storage");
-    
-      toXmlItemBlock(doc, "Total Files And Directories",
-          Long.toString(totalFilesAndDirectories));
-    
-      toXmlItemBlock(doc, "Configured Capacity", StringUtils.byteDesc(total));
-    
-      toXmlItemBlock(doc, "DFS Used", StringUtils.byteDesc(clusterDfsUsed));
-    
-      toXmlItemBlock(doc, "Non DFS Used", StringUtils.byteDesc(nonDfsUsed));
-    
-      toXmlItemBlock(doc, "DFS Remaining", StringUtils.byteDesc(free));
-    
-      // dfsUsedPercent
-      toXmlItemBlock(doc, "DFS Used%", DFSUtil.percent2String(dfsUsedPercent));
-    
-      // dfsRemainingPercent
-      toXmlItemBlock(doc, "DFS Remaining%", DFSUtil.percent2String(dfsRemainingPercent));
-    
-      doc.endTag(); // storage
-    
-      doc.startTag("namenodes");
-      // number of namenodes
-      toXmlItemBlock(doc, "NamenodesCount", Integer.toString(size));
-    
-      for (NamenodeStatus nn : nnList) {
-        doc.startTag("node");
-        toXmlItemBlockWithLink(doc, nn.host, nn.httpAddress, "NameNode");
-        toXmlItemBlock(doc, "Blockpool Used",
-            StringUtils.byteDesc(nn.bpUsed));
-        toXmlItemBlock(doc, "Blockpool Used%",
-            DFSUtil.percent2String(DFSUtil.getPercentUsed(nn.bpUsed, total)));
-        toXmlItemBlock(doc, "Files And Directories",
-            Long.toString(nn.filesAndDirectories));
-        toXmlItemBlock(doc, "Blocks", Long.toString(nn.blocksCount));
-        toXmlItemBlock(doc, "Missing Blocks",
-            Long.toString(nn.missingBlocksCount));
-        toXmlItemBlockWithLink(doc, nn.liveDatanodeCount + " ("
-            + nn.liveDecomCount + ")", new URL(nn.httpAddress,
-            "/dfsnodelist.jsp?whatNodes=LIVE"),
-            "Live Datanode (Decommissioned)");
-        toXmlItemBlockWithLink(doc, nn.deadDatanodeCount + " ("
-            + nn.deadDecomCount + ")", new URL(nn.httpAddress,
-            "/dfsnodelist.jsp?whatNodes=DEAD"),
-            "Dead Datanode (Decommissioned)");
-        toXmlItemBlock(doc, "Software Version", nn.softwareVersion);
-        doc.endTag(); // node
-      }
-      doc.endTag(); // namenodes
-    
-      createNamenodeExceptionMsg(doc, nnExceptions);
-      doc.endTag(); // cluster
-      doc.getWriter().flush();
-    }
-  }
-  
-  /**
-   * This class stores namenode statistics to be used to generate cluster
-   * web console report.
-   */
-  static class NamenodeStatus {
-    String host = "";
-    long capacity = 0L;
-    long free = 0L;
-    long bpUsed = 0L;
-    long nonDfsUsed = 0L;
-    long filesAndDirectories = 0L;
-    long blocksCount = 0L;
-    long missingBlocksCount = 0L;
-    int liveDatanodeCount = 0;
-    int liveDecomCount = 0;
-    int deadDatanodeCount = 0;
-    int deadDecomCount = 0;
-    URL httpAddress = null;
-    String softwareVersion = "";
-  }
-
-  /**
-   * cluster-wide decommission state of a datanode
-   */
-  public enum DecommissionStates {
-    /*
-     * If datanode state is decommissioning at one or more namenodes and 
-     * decommissioned at the rest of the namenodes.
-     */
-    DECOMMISSION_INPROGRESS("Decommission In Progress"),
-    
-    /* If datanode state at all the namenodes is decommissioned */
-    DECOMMISSIONED("Decommissioned"),
-    
-    /*
-     * If datanode state is not decommissioning at one or more namenodes and 
-     * decommissioned/decommissioning at the rest of the namenodes.
-     */
-    PARTIALLY_DECOMMISSIONED("Partially Decommissioning"),
-    
-    /*
-     * If datanode state is not known at a namenode, due to problems in getting
-     * the datanode state from the namenode.
-     */
-    UNKNOWN("Unknown");
-
-    final String value;
-    
-    DecommissionStates(final String v) {
-      this.value = v;
-    }
-
-    @Override
-    public String toString() {
-      return value;
-    }
-  }
-
-  /**
-   * This class consolidates the decommissioning datanodes information in the
-   * cluster and generates decommissioning reports in XML.
-   */
-  static class DecommissionStatus {
-    /** Error when set indicates failure to get decomission status*/
-    final Exception error;
-    
-    /** Map of dn host <-> (Map of NN host <-> decommissioning state) */
-    final Map<String, Map<String, String>> statusMap;
-    final String clusterid;
-    final int httpPort;
-    int decommissioned = 0;   // total number of decommissioned nodes
-    int decommissioning = 0;  // total number of decommissioning datanodes
-    int partial = 0;          // total number of partially decommissioned nodes
-    
-    /** Map of namenode and exception encountered when getting decom status */
-    Map<String, Exception> exceptions = new HashMap<String, Exception>();
-
-    private DecommissionStatus(Map<String, Map<String, String>> statusMap,
-        String cid, int httpPort, Map<String, Exception> exceptions) {
-      this(statusMap, cid, httpPort, exceptions, null);
-    }
-
-    public DecommissionStatus(String cid, Exception e) {
-      this(null, cid, -1, null, e);
-    }
-    
-    private DecommissionStatus(Map<String, Map<String, String>> statusMap,
-        String cid, int httpPort, Map<String, Exception> exceptions,
-        Exception error) {
-      this.statusMap = statusMap;
-      this.clusterid = cid;
-      this.httpPort = httpPort;
-      this.exceptions = exceptions;
-      this.error = error;
-    }
-
-    /**
-     * Generate decommissioning datanode report in XML format
-     * 
-     * @param doc
-     *          , xmloutputter
-     * @throws IOException
-     */
-    public void toXML(XMLOutputter doc) throws IOException {
-      if (error != null) {
-        createGeneralException(doc, clusterid,
-            StringUtils.stringifyException(error));
-        doc.getWriter().flush();
-        return;
-      } 
-      if (statusMap == null || statusMap.isEmpty()) {
-        // none of the namenodes has reported, print exceptions from each nn.
-        doc.startTag("cluster");
-        createNamenodeExceptionMsg(doc, exceptions);
-        doc.endTag();
-        doc.getWriter().flush();
-        return;
-      }
-      doc.startTag("cluster");
-      doc.attribute("clusterId", clusterid);
-
-      doc.startTag("decommissioningReport");
-      countDecommissionDatanodes();
-      toXmlItemBlock(doc, DecommissionStates.DECOMMISSIONED.toString(),
-          Integer.toString(decommissioned));
-
-      toXmlItemBlock(doc,
-          DecommissionStates.DECOMMISSION_INPROGRESS.toString(),
-          Integer.toString(decommissioning));
-
-      toXmlItemBlock(doc,
-          DecommissionStates.PARTIALLY_DECOMMISSIONED.toString(),
-          Integer.toString(partial));
-
-      doc.endTag(); // decommissioningReport
-
-      doc.startTag("datanodes");
-      Set<String> dnSet = statusMap.keySet();
-      for (String dnhost : dnSet) {
-        Map<String, String> nnStatus = statusMap.get(dnhost);
-        if (nnStatus == null || nnStatus.isEmpty()) {
-          continue;
-        }
-        String overallStatus = nnStatus.get(OVERALL_STATUS);
-        // check if datanode is in decommission states
-        if (overallStatus != null
-            && (overallStatus.equals(AdminStates.DECOMMISSION_INPROGRESS
-                .toString())
-                || overallStatus.equals(AdminStates.DECOMMISSIONED.toString())
-                || overallStatus
-                    .equals(DecommissionStates.PARTIALLY_DECOMMISSIONED
-                        .toString()) || overallStatus
-                .equals(DecommissionStates.UNKNOWN.toString()))) {
-          doc.startTag("node");
-          // dn
-          toXmlItemBlockWithLink(doc, dnhost, new URL("http", dnhost, httpPort,
-              ""), "DataNode");
-
-          // overall status first
-          toXmlItemBlock(doc, OVERALL_STATUS, overallStatus);
-
-          for (Map.Entry<String, String> m : nnStatus.entrySet()) {
-            String nn = m.getKey();
-            if (nn.equals(OVERALL_STATUS)) {
-              continue;
-            }
-            // xml
-            toXmlItemBlock(doc, nn, nnStatus.get(nn));
-          }
-          doc.endTag(); // node
-        }
-      }
-      doc.endTag(); // datanodes
-
-      createNamenodeExceptionMsg(doc, exceptions);
-
-      doc.endTag();// cluster
-    } // toXML
-
-    /**
-     * Count the total number of decommissioned/decommission_inprogress/
-     * partially decommissioned datanodes.
-     */
-    private void countDecommissionDatanodes() {
-      for (String dn : statusMap.keySet()) {
-        Map<String, String> nnStatus = statusMap.get(dn);
-        String status = nnStatus.get(OVERALL_STATUS);
-        if (status.equals(DecommissionStates.DECOMMISSIONED.toString())) {
-          decommissioned++;
-        } else if (status.equals(DecommissionStates.DECOMMISSION_INPROGRESS
-            .toString())) {
-          decommissioning++;
-        } else if (status.equals(DecommissionStates.PARTIALLY_DECOMMISSIONED
-            .toString())) {
-          partial++;
-        }
-      }
-    }
-  }
-
-  /**
-   * Generate a XML block as such, <item label=key value=value/>
-   */
-  private static void toXmlItemBlock(XMLOutputter doc, String key, String value)
-      throws IOException {
-    doc.startTag("item");
-    doc.attribute("label", key);
-    doc.attribute("value", value);
-    doc.endTag();
-  }
-
-  /**
-   * Generate a XML block as such, <item label="Node" value="hostname"
-   * link="http://hostname:50070" />
-   */
-  private static void toXmlItemBlockWithLink(XMLOutputter doc, String value,
-      URL url, String label) throws IOException {
-    doc.startTag("item");
-    doc.attribute("label", label);
-    doc.attribute("value", value);
-    doc.attribute("link", url.toString());
-    doc.endTag(); // item
-  }
-
-  /**
-   * create the XML for exceptions that we encountered when connecting to
-   * namenode.
-   */
-  private static void createNamenodeExceptionMsg(XMLOutputter doc,
-      Map<String, Exception> exceptionMsg) throws IOException {
-    if (exceptionMsg.size() > 0) {
-      doc.startTag("unreportedNamenodes");
-      for (Map.Entry<String, Exception> m : exceptionMsg.entrySet()) {
-        doc.startTag("node");
-        doc.attribute("name", m.getKey());
-        doc.attribute("exception",
-            StringUtils.stringifyException(m.getValue()));
-        doc.endTag();// node
-      }
-      doc.endTag(); // unreportedNamnodes
-    }
-  }
-
-  /**
-   * create XML block from general exception.
-   */
-  private static void createGeneralException(XMLOutputter doc,
-      String clusterid, String eMsg) throws IOException {
-    doc.startTag("cluster");
-    doc.attribute("clusterId", clusterid);
-    doc.startTag("message");
-    doc.startTag("item");
-    doc.attribute("msg", eMsg);
-    doc.endTag(); // item
-    doc.endTag(); // message
-    doc.endTag(); // cluster
-  }
-  
-  /**
-   * Read in the content from a URL
-   * @param url URL To read
-   * @return the text from the output
-   * @throws IOException if something went wrong
-   */
-  private static String readOutput(URL url) throws IOException {
-    StringBuilder out = new StringBuilder();
-    URLConnection connection = url.openConnection();
-    BufferedReader in = new BufferedReader(
-                            new InputStreamReader(
-                            connection.getInputStream(), Charsets.UTF_8));
-    String inputLine;
-    while ((inputLine = in.readLine()) != null) {
-      out.append(inputLine);
-    }
-    in.close();
-    return out.toString();
-  }
-
-  private static String queryMbean(URI httpAddress, Configuration conf)
-    throws IOException {
-    /**
-     * Although the other namenode might support HTTPS, it is fundamentally
-     * broken to get the JMX via an HTTPS connection inside the namenode,
-     * because in HTTPS set up the principal of the client and the one of
-     * the namenode differs. Therefore, there is no guarantees that the
-     * HTTPS connection can be set up.
-     *
-     * As a result, we just hard code the connection as an HTTP connection.
-     */
-    URL url = new URL(httpAddress.toURL(), JMX_QRY);
-    return readOutput(url);
-  }
-  /**
-   * In order to query a namenode mxbean, a http connection in the form of
-   * "http://hostname/jmx?qry=Hadoop:service=NameNode,name=NameNodeInfo"
-   * is sent to namenode.  JMX attributes are exposed via JmxJsonServelet on 
-   * the namenode side.
-   */
-  private static JsonNode getProperty(String props, String propertyname)
-  throws IOException {
-    if (props == null || props.equals("") || propertyname == null 
-        || propertyname.equals("")) {
-      return null;
-    }
-    ObjectMapper m = new ObjectMapper();
-    JsonNode rootNode = m.readValue(props, JsonNode.class);
-    JsonNode jn = rootNode.get("beans").get(0).get(propertyname);
-    return jn;
-  }
-} 

+ 0 - 1200
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NamenodeJspHelper.java

@@ -1,1200 +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.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.*;
-
-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;
-
-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 filesystem objects";
-    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,
-      final UserGroupInformation ugi) throws IOException, InterruptedException {
-    Token<DelegationTokenIdentifier> token = ugi
-        .doAs(new PrivilegedExceptionAction<Token<DelegationTokenIdentifier>>() {
-          @Override
-          public Token<DelegationTokenIdentifier> run() throws IOException {
-            return nn.getDelegationToken(new Text(ugi.getUserName()));
-          }
-        });
-    return token == null ? null : token.encodeToUrlString();
-  }
-
-  /** @return a randomly chosen datanode. */
-  static DatanodeDescriptor getRandomDatanode(final NameNode namenode) {
-    return (DatanodeDescriptor)namenode.getNamesystem().getBlockManager(
-        ).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);
-      long currentTime = Time.now();
-      long timestamp = d.getLastUpdate();
-      if (!alive) {
-        out.print("<td class=\"lastcontact\"> "
-            + new Date(timestamp) 
-            + "<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 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("lastcontact")
-                + "> Last <br>Contact <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() %>

+ 3 - 2
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDecommission.java

@@ -66,6 +66,7 @@ public class TestDecommission {
   static final int NAMENODE_REPLICATION_INTERVAL = 1; //replication interval
 
   final Random myrand = new Random();
+  Path dir;
   Path hostsFile;
   Path excludeFile;
   FileSystem localFileSys;
@@ -78,7 +79,7 @@ public class TestDecommission {
     // Set up the hosts/exclude files.
     localFileSys = FileSystem.getLocal(conf);
     Path workingDir = localFileSys.getWorkingDirectory();
-    Path dir = new Path(workingDir, PathUtils.getTestDirName(getClass()) + "/work-dir/decommission");
+    dir = new Path(workingDir, PathUtils.getTestDirName(getClass()) + "/work-dir/decommission");
     hostsFile = new Path(dir, "hosts");
     excludeFile = new Path(dir, "exclude");
     
@@ -98,7 +99,7 @@ public class TestDecommission {
   
   @After
   public void teardown() throws IOException {
-    cleanupFile(localFileSys, excludeFile.getParent());
+    cleanupFile(localFileSys, dir);
     if (cluster != null) {
       cluster.shutdown();
     }

+ 20 - 22
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestMissingBlocksAlert.java

@@ -17,13 +17,6 @@
  */
 package org.apache.hadoop.hdfs;
 
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertTrue;
-
-import java.io.IOException;
-import java.net.URL;
-
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
@@ -32,8 +25,16 @@ import org.apache.hadoop.fs.FSDataInputStream;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
+import org.junit.Assert;
 import org.junit.Test;
 
+import javax.management.*;
+import java.io.IOException;
+import java.lang.management.ManagementFactory;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
 /**
  * The test makes sure that NameNode detects presense blocks that do not have
  * any valid replicas. In addition, it verifies that HDFS front page displays
@@ -45,8 +46,11 @@ public class TestMissingBlocksAlert {
                            LogFactory.getLog(TestMissingBlocksAlert.class);
   
   @Test
-  public void testMissingBlocksAlert() throws IOException, 
-                                       InterruptedException {
+  public void testMissingBlocksAlert()
+          throws IOException, InterruptedException,
+                 MalformedObjectNameException, AttributeNotFoundException,
+                 MBeanException, ReflectionException,
+                 InstanceNotFoundException {
     
     MiniDFSCluster cluster = null;
     
@@ -94,14 +98,11 @@ public class TestMissingBlocksAlert {
       assertEquals(4, dfs.getUnderReplicatedBlocksCount());
       assertEquals(3, bm.getUnderReplicatedNotMissingBlocks());
 
-
-      // Now verify that it shows up on webui
-      URL url = new URL("http://" + conf.get(DFSConfigKeys.DFS_NAMENODE_HTTP_ADDRESS_KEY) + 
-                        "/dfshealth.jsp");
-      String dfsFrontPage = DFSTestUtil.urlGet(url);
-      String warnStr = "WARNING : There are ";
-      assertTrue("HDFS Front page does not contain expected warning", 
-                 dfsFrontPage.contains(warnStr + "1 missing blocks"));
+      MBeanServer mbs = ManagementFactory.getPlatformMBeanServer();
+      ObjectName mxbeanName = new ObjectName(
+              "Hadoop:service=NameNode,name=NameNodeInfo");
+      Assert.assertEquals(1, (long)(Long) mbs.getAttribute(mxbeanName,
+                      "NumberOfMissingBlocks"));
 
       // now do the reverse : remove the file expect the number of missing 
       // blocks to go to zero
@@ -116,11 +117,8 @@ public class TestMissingBlocksAlert {
       assertEquals(2, dfs.getUnderReplicatedBlocksCount());
       assertEquals(2, bm.getUnderReplicatedNotMissingBlocks());
 
-      // and make sure WARNING disappears
-      // Now verify that it shows up on webui
-      dfsFrontPage = DFSTestUtil.urlGet(url);
-      assertFalse("HDFS Front page contains unexpected warning", 
-                  dfsFrontPage.contains(warnStr));
+      Assert.assertEquals(0, (long)(Long) mbs.getAttribute(mxbeanName,
+              "NumberOfMissingBlocks"));
     } finally {
       if (cluster != null) {
         cluster.shutdown();

+ 1 - 55
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/qjournal/TestNNWithQJM.java

@@ -21,16 +21,12 @@ import static org.junit.Assert.*;
 
 import java.io.File;
 import java.io.IOException;
-import java.net.URL;
-import java.util.regex.Pattern;
 
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.FileUtil;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
-import org.apache.hadoop.hdfs.DFSTestUtil;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.server.namenode.NameNode;
@@ -41,6 +37,7 @@ import org.junit.After;
 import org.junit.Before;
 import org.junit.Test;
 
+
 public class TestNNWithQJM {
   final Configuration conf = new HdfsConfiguration();
   private MiniJournalCluster mjc = null;
@@ -204,55 +201,4 @@ public class TestNNWithQJM {
           "Unable to start log segment 1: too few journals", ioe);
     }
   }
-
-  @Test (timeout = 30000)
-  public void testWebPageHasQjmInfo() throws Exception {
-    conf.set(DFSConfigKeys.DFS_NAMENODE_NAME_DIR_KEY,
-        MiniDFSCluster.getBaseDirectory() + "/TestNNWithQJM/image");
-    conf.set(DFSConfigKeys.DFS_NAMENODE_EDITS_DIR_KEY,
-        mjc.getQuorumJournalURI("myjournal").toString());
-    // Speed up the test
-    conf.setInt(
-        CommonConfigurationKeysPublic.IPC_CLIENT_CONNECT_MAX_RETRIES_KEY, 1);
-    
-    MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf)
-      .numDataNodes(0)
-      .manageNameDfsDirs(false)
-      .build();
-    try {
-      URL url = new URL("http://localhost:"
-          + NameNode.getHttpAddress(cluster.getConfiguration(0)).getPort()
-          + "/dfshealth.jsp");
-      
-      cluster.getFileSystem().mkdirs(TEST_PATH);
-      
-      String contents = DFSTestUtil.urlGet(url); 
-      assertTrue(contents.contains("QJM to ["));
-      assertTrue(contents.contains("Written txid 2"));
-
-      // Stop one JN, do another txn, and make sure it shows as behind
-      // stuck behind the others.
-      mjc.getJournalNode(0).stopAndJoin(0);
-      
-      cluster.getFileSystem().delete(TEST_PATH, true);
-      
-      contents = DFSTestUtil.urlGet(url); 
-      System.out.println(contents);
-      assertTrue(Pattern.compile("1 txns/\\d+ms behind").matcher(contents)
-          .find());
-
-      // Restart NN while JN0 is still down.
-      cluster.restartNameNode();
-
-      contents = DFSTestUtil.urlGet(url); 
-      System.out.println(contents);
-      assertTrue(Pattern.compile("never written").matcher(contents)
-          .find());
-      
-
-    } finally {
-      cluster.shutdown();
-    }
-
-  }
 }

+ 0 - 5
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/qjournal/server/TestJournalNode.java

@@ -170,11 +170,6 @@ public class TestJournalNode {
     assertTrue("Bad contents: " + pageContents,
         pageContents.contains(
             "Hadoop:service=JournalNode,name=JvmMetrics"));
-    
-    // Check JSP page.
-    pageContents = DFSTestUtil.urlGet(
-        new URL(urlRoot + "/journalstatus.jsp"));
-    assertTrue(pageContents.contains("JournalNode"));
 
     // Create some edits on server side
     byte[] EDITS_DATA = QJMTestUtil.createTxnData(1, 3);

+ 0 - 200
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/common/TestJspHelper.java

@@ -158,25 +158,6 @@ public class TestJspHelper {
         .next();
     Assert.assertEquals(expected, tokenInUgi.getService().toString());
   }
-  
-  
-  @Test
-  public void testDelegationTokenUrlParam() {
-    conf.set(DFSConfigKeys.HADOOP_SECURITY_AUTHENTICATION, "kerberos");
-    UserGroupInformation.setConfiguration(conf);
-    String tokenString = "xyzabc";
-    String delegationTokenParam = JspHelper
-        .getDelegationTokenUrlParam(tokenString);
-    //Security is enabled
-    Assert.assertEquals(JspHelper.SET_DELEGATION + "xyzabc",
-        delegationTokenParam);
-    conf.set(DFSConfigKeys.HADOOP_SECURITY_AUTHENTICATION, "simple");
-    UserGroupInformation.setConfiguration(conf);
-    delegationTokenParam = JspHelper
-        .getDelegationTokenUrlParam(tokenString);
-    //Empty string must be returned because security is disabled.
-    Assert.assertEquals("", delegationTokenParam);
-  }
 
   @Test
   public void testGetUgiFromToken() throws IOException {
@@ -403,32 +384,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);
@@ -463,146 +418,6 @@ public class TestJspHelper {
     }
   }
 
-  @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);
-
-    // Test sorting by failed volumes
-    JspHelper.sortNodeList(live, "volfails", "ASC");
-    Assert.assertEquals(dnDesc2, live.get(0));
-    Assert.assertEquals(dnDesc1, live.get(1));
-    JspHelper.sortNodeList(live, "volfails", "DSC");
-    Assert.assertEquals(dnDesc1, live.get(0));
-    Assert.assertEquals(dnDesc2, live.get(1));
-
-    // Test sorting by Blockpool used
-    JspHelper.sortNodeList(live, "bpused", "ASC");
-    Assert.assertEquals(dnDesc1, live.get(0));
-    Assert.assertEquals(dnDesc2, live.get(1));
-    JspHelper.sortNodeList(live, "bpused", "DSC");
-    Assert.assertEquals(dnDesc2, live.get(0));
-    Assert.assertEquals(dnDesc1, live.get(1));
-
-    // Test sorting by Percentage Blockpool used
-    JspHelper.sortNodeList(live, "pcbpused", "ASC");
-    Assert.assertEquals(dnDesc2, live.get(0));
-    Assert.assertEquals(dnDesc1, live.get(1));
-    JspHelper.sortNodeList(live, "pcbpused", "DSC");
-    Assert.assertEquals(dnDesc1, live.get(0));
-    Assert.assertEquals(dnDesc2, live.get(1));
-    
-    //unexisted field comparition is d1.getHostName().compareTo(d2.getHostName());    
-    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<Object>() {      
-      @Override
-      public Object answer(InvocationOnMock invok) {
-        Object[] args = invok.getArguments();
-        buffer.append((String)args[0]);
-        return null;
-      }
-    }).when(out).print(arg.capture());
-    
-    
-    JspHelper.createTitle(out, req, "testfile.txt");
-    Mockito.verify(out, Mockito.times(1)).print(Mockito.anyString());
-    
-    JspHelper.addTableHeader(out);
-    Mockito.verify(out, Mockito.times(1 + 2)).print(Mockito.anyString());                  
-     
-    JspHelper.addTableRow(out, new String[] {" row11", "row12 "});
-    Mockito.verify(out, Mockito.times(1 + 2 + 4)).print(Mockito.anyString());      
-    
-    JspHelper.addTableRow(out, new String[] {" row11", "row12 "}, 3);
-    Mockito.verify(out, Mockito.times(1 + 2 + 4 + 4)).print(Mockito.anyString());
-      
-    JspHelper.addTableRow(out, new String[] {" row21", "row22"});
-    Mockito.verify(out, Mockito.times(1 + 2 + 4 + 4 + 4)).print(Mockito.anyString());      
-      
-    JspHelper.addTableFooter(out);
-    Mockito.verify(out, Mockito.times(1 + 2 + 4 + 4 + 4 + 1)).print(Mockito.anyString());
-    
-    assertFalse(Strings.isNullOrEmpty(buffer.toString()));               
-  }
-  
   @Test
   public void testReadWriteReplicaState() {
     try {
@@ -622,21 +437,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;
-  }
-
-}

+ 0 - 59
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestClusterJspHelper.java

@@ -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.
- */
-package org.apache.hadoop.hdfs.server.namenode;
-
-import static org.junit.Assert.*;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hdfs.MiniDFSCluster;
-import org.apache.hadoop.hdfs.server.namenode.ClusterJspHelper.ClusterStatus;
-import org.apache.hadoop.hdfs.server.namenode.ClusterJspHelper.DecommissionStatus;
-import org.junit.After;
-import org.junit.Before;
-import org.junit.Test;
-
-public class TestClusterJspHelper {
-
-  private MiniDFSCluster cluster;
-  private Configuration conf;
-    
-  @Before
-  public void setUp() throws Exception {
-    conf = new Configuration();  
-    cluster = new MiniDFSCluster.Builder(conf).build();
-    cluster.waitClusterUp();
-  }
-
-  @After
-  public void tearDown() throws Exception {
-    if (cluster != null)
-      cluster.shutdown();    
-  }
-  
-  @Test(timeout = 15000)
-  public void testClusterJspHelperReports() {
-    ClusterJspHelper clusterJspHelper = new ClusterJspHelper();
-    ClusterStatus clusterStatus = clusterJspHelper
-     .generateClusterHealthReport();
-    assertNotNull("testClusterJspHelperReports ClusterStatus is null",
-        clusterStatus);       
-    DecommissionStatus decommissionStatus = clusterJspHelper
-        .generateDecommissioningReport();
-    assertNotNull("testClusterJspHelperReports DecommissionStatus is null",
-        decommissionStatus);    
-  }
-}

+ 0 - 131
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestCorruptFilesJsp.java

@@ -1,131 +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.namenode;
-
-import static org.junit.Assert.assertTrue;
-
-import java.net.URL;
-import java.util.Collection;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.ChecksumException;
-import org.apache.hadoop.fs.FSDataInputStream;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.hdfs.DFSConfigKeys;
-import org.apache.hadoop.hdfs.DFSTestUtil;
-import org.apache.hadoop.hdfs.HdfsConfiguration;
-import org.apache.hadoop.hdfs.MiniDFSCluster;
-import org.apache.hadoop.hdfs.TestDatanodeBlockScanner;
-import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
-import org.junit.Test;
-
-/** A JUnit test for corrupt_files.jsp */
-public class TestCorruptFilesJsp  {
-
-  @Test
-  public void testCorruptFilesJsp() throws Exception {
-    MiniDFSCluster cluster = null;
-    try {
-
-      final int FILE_SIZE = 512;
-
-      Path[] filepaths = { new Path("/audiobook"), new Path("/audio/audio1"),
-          new Path("/audio/audio2"), new Path("/audio/audio") };
-
-      Configuration conf = new HdfsConfiguration();
-      // datanode scans directories
-      conf.setInt(DFSConfigKeys.DFS_DATANODE_DIRECTORYSCAN_INTERVAL_KEY, 1);
-      // datanode sends block reports
-      conf.setInt(DFSConfigKeys.DFS_BLOCKREPORT_INTERVAL_MSEC_KEY, 3 * 1000);
-      cluster = new MiniDFSCluster.Builder(conf).build();
-      cluster.waitActive();
-
-      FileSystem fs = cluster.getFileSystem();
-
-      // create files
-      for (Path filepath : filepaths) {
-        DFSTestUtil.createFile(fs, filepath, FILE_SIZE, (short) 1, 0L);
-        DFSTestUtil.waitReplication(fs, filepath, (short) 1);
-      }
-
-      // verify there are not corrupt files
-      final NameNode namenode = cluster.getNameNode();
-      Collection<FSNamesystem.CorruptFileBlockInfo> badFiles = namenode.
-        getNamesystem().listCorruptFileBlocks("/", null);
-      assertTrue("There are " + badFiles.size()
-          + " corrupt files, but expecting none", badFiles.size() == 0);
-
-      // Check if webui agrees
-      URL url = new URL("http://"
-          + conf.get(DFSConfigKeys.DFS_NAMENODE_HTTP_ADDRESS_KEY)
-          + "/corrupt_files.jsp");
-      String corruptFilesPage = DFSTestUtil.urlGet(url);
-      assertTrue("Corrupt files page is not showing a healthy filesystem",
-          corruptFilesPage.contains("No missing blocks found at the moment."));
-
-      // Now corrupt all the files except for the last one
-      for (int idx = 0; idx < filepaths.length - 1; idx++) {
-        ExtendedBlock blk = DFSTestUtil.getFirstBlock(fs, filepaths[idx]);
-        assertTrue(TestDatanodeBlockScanner.corruptReplica(blk, 0));
-
-        // read the file so that the corrupt block is reported to NN
-        FSDataInputStream in = fs.open(filepaths[idx]);
-        try {
-          in.readFully(new byte[FILE_SIZE]);
-        } catch (ChecksumException ignored) { // checksum error is expected.
-        }
-        in.close();
-      }
-
-      try {
-        Thread.sleep(3000); // Wait for block reports. They shouldn't matter.
-      } catch (InterruptedException ie) {}
-
-      // verify if all corrupt files were reported to NN
-      badFiles = namenode.getNamesystem().listCorruptFileBlocks("/", null);
-      assertTrue("Expecting 3 corrupt files, but got " + badFiles.size(),
-          badFiles.size() == 3);
-
-      // Check if webui agrees
-      url = new URL("http://"
-          + conf.get(DFSConfigKeys.DFS_NAMENODE_HTTP_ADDRESS_KEY)
-          + "/corrupt_files.jsp");
-      corruptFilesPage = DFSTestUtil.urlGet(url);
-      assertTrue("'/audiobook' should be corrupt", corruptFilesPage
-          .contains("/audiobook"));
-      assertTrue("'/audio/audio1' should be corrupt", corruptFilesPage
-          .contains("/audio/audio1"));
-      assertTrue("'/audio/audio2' should be corrupt", corruptFilesPage
-          .contains("/audio/audio2"));
-      assertTrue("Summary message shall report 3 corrupt files",
-          corruptFilesPage.contains("At least 3 corrupt file(s)"));
-
-      // clean up
-      for (Path filepath : filepaths) {
-        fs.delete(filepath, false);
-      }
-    } finally {
-      if (cluster != null) {
-        cluster.shutdown();
-      }
-    }
-
-  }
-
-}

+ 11 - 14
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestHostsFiles.java

@@ -19,10 +19,8 @@ package org.apache.hadoop.hdfs.server.namenode;
 
 import static org.junit.Assert.assertTrue;
 
-import java.net.InetSocketAddress;
-import java.net.URL;
+import java.lang.management.ManagementFactory;
 
-import org.apache.commons.lang.StringEscapeUtils;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
@@ -36,6 +34,9 @@ import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 import org.junit.Test;
 
+import javax.management.MBeanServer;
+import javax.management.ObjectName;
+
 /**
  * DFS_HOSTS and DFS_HOSTS_EXCLUDE tests
  * 
@@ -73,7 +74,7 @@ public class TestHostsFiles {
   }
 
   @Test
-  public void testHostsExcludeDfshealthJsp() throws Exception {
+  public void testHostsExcludeInUI() throws Exception {
     Configuration conf = getConf();
     short REPLICATION_FACTOR = 2;
     final Path filePath = new Path("/testFile");
@@ -117,17 +118,13 @@ public class TestHostsFiles {
 
       // Check the block still has sufficient # replicas across racks
       DFSTestUtil.waitForReplication(cluster, b, 2, REPLICATION_FACTOR, 0);
-      
-      InetSocketAddress nnHttpAddress = cluster.getNameNode().getHttpAddress();
-      LOG.info("nnaddr = '" + nnHttpAddress + "'");
-      String nnHostName = nnHttpAddress.getHostName();
-      URL nnjsp = new URL("http://" + nnHostName + ":" + nnHttpAddress.getPort() + "/dfshealth.jsp");
-      LOG.info("fetching " + nnjsp);
-      String dfshealthPage = StringEscapeUtils.unescapeHtml(DFSTestUtil.urlGet(nnjsp));
-      LOG.info("got " + dfshealthPage);
-      assertTrue("dfshealth should contain " + nnHostName + ", got:" + dfshealthPage,
-          dfshealthPage.contains(nnHostName));
 
+      MBeanServer mbs = ManagementFactory.getPlatformMBeanServer();
+      ObjectName mxbeanName = new ObjectName(
+              "Hadoop:service=NameNode,name=NameNodeInfo");
+      String nodes = (String) mbs.getAttribute(mxbeanName, "LiveNodes");
+      assertTrue("Live nodes should contain the decommissioned node",
+              nodes.contains("Decommissioned"));
     } finally {
       cluster.shutdown();
     }

+ 0 - 372
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNameNodeJspHelper.java

@@ -1,372 +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.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 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;
-
-public class TestNameNodeJspHelper {
-
-  private static final int DATA_NODES_AMOUNT = 2;
-  
-  private static MiniDFSCluster cluster;
-  private static Configuration conf;
-  private static final String NAMENODE_ATTRIBUTE_KEY = "name.node";  
-  
-  @BeforeClass
-  public static void setUp() throws Exception {
-    conf = new HdfsConfiguration();
-    cluster = new MiniDFSCluster.Builder(conf)
-        .numDataNodes(DATA_NODES_AMOUNT).build();
-    cluster.waitClusterUp();
-  }
-
-  @AfterClass
-  public static void tearDown() throws Exception {
-    if (cluster != null)
-      cluster.shutdown();
-  }
-
-  @Test
-  public void testDelegationToken() throws IOException, InterruptedException {
-    NamenodeProtocols nn = cluster.getNameNodeRpc();
-    HttpServletRequest request = mock(HttpServletRequest.class);
-    UserGroupInformation ugi = UserGroupInformation.createRemoteUser("auser");
-    String tokenString = NamenodeJspHelper.getDelegationToken(nn, request,
-        conf, ugi);
-    // tokenString returned must be null because security is disabled
-    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();
-
-    for (int i = 0; i < 10; i++) {
-      DatanodeDescriptor dnDescriptor = NamenodeJspHelper
-          .getRandomDatanode(nameNode);
-      assertTrue("testGetRandomDatanode error",
-          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));
-    }
-  }  
-}

+ 0 - 14
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestSecondaryWebUi.java

@@ -78,18 +78,4 @@ public class TestSecondaryWebUi {
     Assert.assertArrayEquals(checkpointEditlogDir,
             snn.getCheckpointEditlogDirectories());
   }
-
-  @Test
-  public void testSecondaryWebUiJsp()
-          throws IOException, MalformedObjectNameException,
-                 AttributeNotFoundException, MBeanException,
-                 ReflectionException, InstanceNotFoundException {
-    String pageContents = DFSTestUtil.urlGet(new URL("http://localhost:" +
-        SecondaryNameNode.getHttpAddress(conf).getPort() + "/status.jsp"));
-    Assert.assertTrue("Didn't find \"Last Checkpoint\"",
-        pageContents.contains("Last Checkpoint"));
-    Assert.assertTrue("Didn't find Checkpoint Transactions: 500",
-        pageContents.contains("Checkpoint Transactions: 500"));
-
-  }
 }

+ 0 - 74
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestHAWebUI.java

@@ -1,74 +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.namenode.ha;
-
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertTrue;
-
-import java.net.URL;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hdfs.DFSTestUtil;
-import org.apache.hadoop.hdfs.MiniDFSCluster;
-import org.apache.hadoop.hdfs.MiniDFSNNTopology;
-import org.apache.hadoop.hdfs.server.namenode.NameNode;
-import org.junit.Test;
-
-public class TestHAWebUI {
-
-  /**
-   * Tests that the web UI of the name node provides a link to browse the file
-   * system and summary of under-replicated blocks only in active state
-   * 
-   */
-  @Test
-  public void testLinkAndClusterSummary() throws Exception {
-    Configuration conf = new Configuration();
-
-    MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf)
-        .nnTopology(MiniDFSNNTopology.simpleHATopology()).numDataNodes(0)
-        .build();
-    try {
-      cluster.waitActive();
-
-      cluster.transitionToActive(0);
-      String pageContents = DFSTestUtil.urlGet(new URL("http://localhost:"
-          + NameNode.getHttpAddress(cluster.getConfiguration(0)).getPort()
-          + "/dfshealth.jsp"));
-      assertTrue(pageContents.contains("Browse the filesystem"));
-      assertTrue(pageContents.contains("Number of Under-Replicated Blocks"));
-
-      cluster.transitionToStandby(0);
-      pageContents = DFSTestUtil.urlGet(new URL("http://localhost:"
-          + NameNode.getHttpAddress(cluster.getConfiguration(0)).getPort()
-          + "/dfshealth.jsp"));
-      assertFalse(pageContents.contains("Browse the filesystem"));
-      assertFalse(pageContents.contains("Number of Under-Replicated Blocks"));
-
-      cluster.transitionToActive(0);
-      pageContents = DFSTestUtil.urlGet(new URL("http://localhost:"
-          + NameNode.getHttpAddress(cluster.getConfiguration(0)).getPort()
-          + "/dfshealth.jsp"));
-      assertTrue(pageContents.contains("Browse the filesystem"));
-      assertTrue(pageContents.contains("Number of Under-Replicated Blocks"));
-
-    } finally {
-      cluster.shutdown();
-    }
-  }
-}