You are viewing a plain text version of this content. The canonical link for it is here.
Posted to hdfs-commits@hadoop.apache.org by br...@apache.org on 2013/10/07 21:15:30 UTC

svn commit: r1530033 - in /hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs: ./ src/main/java/org/apache/hadoop/hdfs/server/common/ src/main/java/org/apache/hadoop/hdfs/server/datanode/ src/main/java/org/apache/hadoop/hdfs/server/namenod...

Author: brandonli
Date: Mon Oct  7 19:15:29 2013
New Revision: 1530033

URL: http://svn.apache.org/r1530033
Log:
HDFS-5307. Merging change r1530027 from trunk

Modified:
    hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
    hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/JspHelper.java
    hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DatanodeJspHelper.java
    hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ClusterJspHelper.java
    hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NamenodeJspHelper.java
    hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDatanodeJsp.java

Modified: hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt?rev=1530033&r1=1530032&r2=1530033&view=diff
==============================================================================
--- hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt (original)
+++ hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt Mon Oct  7 19:15:29 2013
@@ -153,6 +153,9 @@ Release 2.2.0 - 2013-10-13
     HDFS-5259. Support client which combines appended data with old data
     before sends it to NFS server. (brandonli)
 
+    HDFS-5307. Support both HTTP and HTTPS in jsp pages (Haohui Mai via
+    branconli)
+
 Release 2.1.1-beta - 2013-09-23
 
   INCOMPATIBLE CHANGES

Modified: hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/JspHelper.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/JspHelper.java?rev=1530033&r1=1530032&r2=1530033&view=diff
==============================================================================
--- hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/JspHelper.java (original)
+++ hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/JspHelper.java Mon Oct  7 19:15:29 2013
@@ -19,6 +19,7 @@
 package org.apache.hadoop.hdfs.server.common;
 
 import com.google.common.base.Charsets;
+
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.classification.InterfaceAudience;
@@ -55,6 +56,7 @@ import org.apache.hadoop.util.VersionInf
 import javax.servlet.ServletContext;
 import javax.servlet.http.HttpServletRequest;
 import javax.servlet.jsp.JspWriter;
+
 import java.io.ByteArrayInputStream;
 import java.io.DataInputStream;
 import java.io.IOException;
@@ -101,7 +103,7 @@ public class JspHelper {
       return super.hashCode();
     }
   }
- 
+
   // compare two records based on their frequency
   private static class NodeRecordComparator implements Comparator<NodeRecord> {
 
@@ -115,6 +117,27 @@ public class JspHelper {
       return 0;
     }
   }
+
+  /**
+   * A helper class that generates the correct URL for different schema.
+   *
+   */
+  public static final class Url {
+    public static String authority(String scheme, DatanodeID d) {
+      if (scheme.equals("http")) {
+        return d.getInfoAddr();
+      } else if (scheme.equals("https")) {
+        return d.getInfoSecureAddr();
+      } 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 =

Modified: hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DatanodeJspHelper.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DatanodeJspHelper.java?rev=1530033&r1=1530032&r2=1530033&view=diff
==============================================================================
--- hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DatanodeJspHelper.java (original)
+++ hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DatanodeJspHelper.java Mon Oct  7 19:15:29 2013
@@ -18,6 +18,7 @@
 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;
@@ -35,6 +36,7 @@ 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;
@@ -45,16 +47,22 @@ import org.apache.hadoop.hdfs.security.t
 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.http.HttpConfig;
 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
@@ -137,10 +145,10 @@ public class DatanodeJspHelper {
           out.print("Empty file");
         } else {
           DatanodeInfo chosenNode = JspHelper.bestNode(firstBlock, conf);
-          String fqdn = canonicalize(chosenNode.getIpAddr());
           int datanodePort = chosenNode.getXferPort();
-          String redirectLocation = HttpConfig.getSchemePrefix() + fqdn + ":"
-              + chosenNode.getInfoPort() + "/browseBlock.jsp?blockId="
+          String redirectLocation = JspHelper.Url.url(req.getScheme(),
+              chosenNode)
+              + "/browseBlock.jsp?blockId="
               + firstBlock.getBlock().getBlockId() + "&blockSize="
               + firstBlock.getBlock().getNumBytes() + "&genstamp="
               + firstBlock.getBlock().getGenerationStamp() + "&filename="
@@ -312,8 +320,8 @@ public class DatanodeJspHelper {
       dfs.close();
       return;
     }
-    String fqdn = canonicalize(chosenNode.getIpAddr());
-    String tailUrl = "///" + fqdn + ":" + chosenNode.getInfoPort()
+
+    String tailUrl = "///" + JspHelper.Url.authority(req.getScheme(), chosenNode)
         + "/tail.jsp?filename=" + URLEncoder.encode(filename, "UTF-8")
         + "&namenodeInfoPort=" + namenodeInfoPort
         + "&chunkSizeToView=" + chunkSizeToView
@@ -361,8 +369,7 @@ public class DatanodeJspHelper {
       for (int j = 0; j < locs.length; j++) {
         String datanodeAddr = locs[j].getXferAddr();
         datanodePort = locs[j].getXferPort();
-        fqdn = canonicalize(locs[j].getIpAddr());
-        String blockUrl = "///" + fqdn + ":" + locs[j].getInfoPort()
+        String blockUrl = "///" + JspHelper.Url.authority(req.getScheme(), locs[j])
             + "/browseBlock.jsp?blockId=" + blockidstring
             + "&blockSize=" + blockSize
             + "&filename=" + URLEncoder.encode(filename, "UTF-8")
@@ -492,112 +499,23 @@ public class DatanodeJspHelper {
     out.print("<a href=\"#viewOptions\">Advanced view/download options</a><br>");
     out.print("<hr>");
 
-    // Determine the prev & next blocks
-    long nextStartOffset = 0;
-    long nextBlockSize = 0;
-    String nextBlockIdStr = null;
-    String nextGenStamp = null;
-    String nextHost = req.getServerName();
-    int nextPort = req.getServerPort();
-    int nextDatanodePort = datanodePort;
-    // determine data for the next link
-    if (startOffset + chunkSizeToView >= blockSize) {
-      // we have to go to the next block from this point onwards
-      List<LocatedBlock> blocks = dfs.getNamenode().getBlockLocations(filename, 0,
-          Long.MAX_VALUE).getLocatedBlocks();
-      for (int i = 0; i < blocks.size(); i++) {
-        if (blocks.get(i).getBlock().getBlockId() == blockId) {
-          if (i != blocks.size() - 1) {
-            LocatedBlock nextBlock = blocks.get(i + 1);
-            nextBlockIdStr = Long.toString(nextBlock.getBlock().getBlockId());
-            nextGenStamp = Long.toString(nextBlock.getBlock()
-                .getGenerationStamp());
-            nextStartOffset = 0;
-            nextBlockSize = nextBlock.getBlock().getNumBytes();
-            DatanodeInfo d = JspHelper.bestNode(nextBlock, conf);
-            nextDatanodePort = d.getXferPort();
-            nextHost = d.getIpAddr();
-            nextPort = d.getInfoPort();
-          }
-        }
-      }
-    } else {
-      // we are in the same block
-      nextBlockIdStr = blockId.toString();
-      nextStartOffset = startOffset + chunkSizeToView;
-      nextBlockSize = blockSize;
-      nextGenStamp = genStamp.toString();
-    }
-    String nextUrl = null;
-    if (nextBlockIdStr != null) {
-      nextUrl = "///" + canonicalize(nextHost) + ":" + nextPort
-          + "/browseBlock.jsp?blockId=" + nextBlockIdStr
-          + "&blockSize=" + nextBlockSize
-          + "&startOffset=" + nextStartOffset
-          + "&genstamp=" + nextGenStamp
-          + "&filename=" + URLEncoder.encode(filename, "UTF-8")
-          + "&chunkSizeToView=" + chunkSizeToView
-          + "&datanodePort=" + nextDatanodePort
-          + "&namenodeInfoPort=" + namenodeInfoPort
-          + JspHelper.getDelegationTokenUrlParam(tokenString)
-          + JspHelper.getUrlParam(JspHelper.NAMENODE_ADDRESS, nnAddr);
+    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;");
     }
-    // determine data for the prev link
-    String prevBlockIdStr = null;
-    String prevGenStamp = null;
-    long prevStartOffset = 0;
-    long prevBlockSize = 0;
-    String prevHost = req.getServerName();
-    int prevPort = req.getServerPort();
-    int prevDatanodePort = datanodePort;
-    if (startOffset == 0) {
-      List<LocatedBlock> blocks = dfs.getNamenode().getBlockLocations(filename, 0,
-          Long.MAX_VALUE).getLocatedBlocks();
-      for (int i = 0; i < blocks.size(); i++) {
-        if (blocks.get(i).getBlock().getBlockId() == blockId) {
-          if (i != 0) {
-            LocatedBlock prevBlock = blocks.get(i - 1);
-            prevBlockIdStr = Long.toString(prevBlock.getBlock().getBlockId());
-            prevGenStamp = Long.toString(prevBlock.getBlock()
-                .getGenerationStamp());
-            prevStartOffset = prevBlock.getBlock().getNumBytes()
-                - chunkSizeToView;
-            if (prevStartOffset < 0)
-              prevStartOffset = 0;
-            prevBlockSize = prevBlock.getBlock().getNumBytes();
-            DatanodeInfo d = JspHelper.bestNode(prevBlock, conf);
-            prevDatanodePort = d.getXferPort();
-            prevHost = d.getIpAddr();
-            prevPort = d.getInfoPort();
-          }
-        }
-      }
-    } else {
-      // we are in the same block
-      prevBlockIdStr = blockId.toString();
-      prevStartOffset = startOffset - chunkSizeToView;
-      if (prevStartOffset < 0)
-        prevStartOffset = 0;
-      prevBlockSize = blockSize;
-      prevGenStamp = genStamp.toString();
-    }
 
-    String prevUrl = null;
-    if (prevBlockIdStr != null) {
-      prevUrl = "///" + canonicalize(prevHost) + ":" + prevPort
-          + "/browseBlock.jsp?blockId=" + prevBlockIdStr
-          + "&blockSize=" + prevBlockSize
-          + "&startOffset=" + prevStartOffset
-          + "&filename=" + URLEncoder.encode(filename, "UTF-8")
-          + "&chunkSizeToView=" + chunkSizeToView
-          + "&genstamp=" + prevGenStamp
-          + "&datanodePort=" + prevDatanodePort
-          + "&namenodeInfoPort=" + namenodeInfoPort
-          + JspHelper.getDelegationTokenUrlParam(tokenString)
-          + JspHelper.getUrlParam(JspHelper.NAMENODE_ADDRESS, nnAddr);
+    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 {
@@ -612,6 +530,71 @@ public class DatanodeJspHelper {
     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,

Modified: hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ClusterJspHelper.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ClusterJspHelper.java?rev=1530033&r1=1530032&r2=1530033&view=diff
==============================================================================
--- hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ClusterJspHelper.java (original)
+++ hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ClusterJspHelper.java Mon Oct  7 19:15:29 2013
@@ -887,7 +887,16 @@ class ClusterJspHelper {
 
   private static String queryMbean(String httpAddress, Configuration conf) 
     throws IOException {
-    URL url = new URL(HttpConfig.getSchemePrefix() + httpAddress+JMX_QRY);
+    /**
+     * 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("http://" + httpAddress + JMX_QRY);
     return readOutput(url);
   }
   /**

Modified: hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NamenodeJspHelper.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NamenodeJspHelper.java?rev=1530033&r1=1530032&r2=1530033&view=diff
==============================================================================
--- hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NamenodeJspHelper.java (original)
+++ hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NamenodeJspHelper.java Mon Oct  7 19:15:29 2013
@@ -23,7 +23,6 @@ import java.io.IOException;
 import java.lang.management.ManagementFactory;
 import java.lang.management.MemoryMXBean;
 import java.lang.management.MemoryUsage;
-import java.net.InetAddress;
 import java.net.InetSocketAddress;
 import java.net.URI;
 import java.net.URLEncoder;
@@ -61,7 +60,6 @@ import org.apache.hadoop.hdfs.server.nam
 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.http.HttpConfig;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.net.NodeBase;
 import org.apache.hadoop.security.UserGroupInformation;
@@ -656,25 +654,21 @@ class NamenodeJspHelper {
     // We can't redirect if there isn't a DN to redirect to.
     // Lets instead show a proper error message.
     FSNamesystem fsn = nn.getNamesystem();
-    if (fsn == null || fsn.getNumLiveDataNodes() < 1) {
+
+    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.");
     }
-    final DatanodeID datanode = getRandomDatanode(nn);;
+
     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);
-    // if the user is defined, get a delegation token and stringify it
-    final String redirectLocation;
-    final String nodeToRedirect;
-    int redirectPort;
-    if (datanode != null) {
-      nodeToRedirect = datanode.getIpAddr();
-      redirectPort = datanode.getInfoPort();
-    } else {
-      nodeToRedirect = nn.getHttpAddress().getHostName();
-      redirectPort = nn.getHttpAddress().getPort();
-    }
 
     InetSocketAddress rpcAddr = nn.getNameNodeAddress();
     String rpcHost = rpcAddr.getAddress().isAnyLocalAddress()
@@ -682,13 +676,14 @@ class NamenodeJspHelper {
       : rpcAddr.getAddress().getHostAddress();
     String addr = rpcHost + ":" + rpcAddr.getPort();
 
-    String fqdn = InetAddress.getByName(nodeToRedirect).getCanonicalHostName();
-    redirectLocation = HttpConfig.getSchemePrefix() + fqdn + ":" + redirectPort
+    final String redirectLocation =
+        JspHelper.Url.url(request.getScheme(), datanode)
         + "/browseDirectory.jsp?namenodeInfoPort="
-        + nn.getHttpAddress().getPort() + "&dir=/"
+        + request.getServerPort() + "&dir=/"
         + (tokenString == null ? "" :
            JspHelper.getDelegationTokenUrlParam(tokenString))
         + JspHelper.getUrlParam(JspHelper.NAMENODE_ADDRESS, addr);
+
     resp.sendRedirect(redirectLocation);
   }
 
@@ -743,11 +738,10 @@ class NamenodeJspHelper {
     }
 
     private void generateNodeDataHeader(JspWriter out, DatanodeDescriptor d,
-        String suffix, boolean alive, int nnHttpPort, String nnaddr)
+        String suffix, boolean alive, int nnHttpPort, String nnaddr, String scheme)
         throws IOException {
       // from nn_browsedfscontent.jsp:
-      String url = "///" + d.getHostName() + ":"
-          + d.getInfoPort()
+      String url = "///" + JspHelper.Url.authority(scheme, d)
           + "/browseDirectory.jsp?namenodeInfoPort=" + nnHttpPort + "&dir="
           + URLEncoder.encode("/", "UTF-8")
           + JspHelper.getUrlParam(JspHelper.NAMENODE_ADDRESS, nnaddr);
@@ -765,9 +759,9 @@ class NamenodeJspHelper {
     }
 
     void generateDecommissioningNodeData(JspWriter out, DatanodeDescriptor d,
-        String suffix, boolean alive, int nnHttpPort, String nnaddr)
+        String suffix, boolean alive, int nnHttpPort, String nnaddr, String scheme)
         throws IOException {
-      generateNodeDataHeader(out, d, suffix, alive, nnHttpPort, nnaddr);
+      generateNodeDataHeader(out, d, suffix, alive, nnHttpPort, nnaddr, scheme);
       if (!alive) {
         return;
       }
@@ -791,7 +785,7 @@ class NamenodeJspHelper {
     }
     
     void generateNodeData(JspWriter out, DatanodeDescriptor d, String suffix,
-        boolean alive, int nnHttpPort, String nnaddr) throws IOException {
+        boolean alive, int nnHttpPort, 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
@@ -803,7 +797,7 @@ class NamenodeJspHelper {
        * interact with datanodes.
        */
 
-      generateNodeDataHeader(out, d, suffix, alive, nnHttpPort, nnaddr);
+      generateNodeDataHeader(out, d, suffix, alive, nnHttpPort, nnaddr, scheme);
       if (!alive) {
         out.print("<td class=\"decommissioned\"> " + 
             d.isDecommissioned() + "\n");
@@ -961,7 +955,7 @@ class NamenodeJspHelper {
             JspHelper.sortNodeList(live, sorterField, sorterOrder);
             for (int i = 0; i < live.size(); i++) {
               generateNodeData(out, live.get(i), port_suffix, true, nnHttpPort,
-                  nnaddr);
+                  nnaddr, request.getScheme());
             }
           }
           out.print("</table>\n");
@@ -981,7 +975,7 @@ class NamenodeJspHelper {
             JspHelper.sortNodeList(dead, sorterField, sorterOrder);
             for (int i = 0; i < dead.size(); i++) {
               generateNodeData(out, dead.get(i), port_suffix, false,
-                  nnHttpPort, nnaddr);
+                  nnHttpPort, nnaddr, request.getScheme());
             }
 
             out.print("</table>\n");
@@ -1012,7 +1006,7 @@ class NamenodeJspHelper {
             JspHelper.sortNodeList(decommissioning, "name", "ASC");
             for (int i = 0; i < decommissioning.size(); i++) {
               generateDecommissioningNodeData(out, decommissioning.get(i),
-                  port_suffix, true, nnHttpPort, nnaddr);
+                  port_suffix, true, nnHttpPort, nnaddr, request.getScheme());
             }
             out.print("</table>\n");
           }

Modified: hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDatanodeJsp.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDatanodeJsp.java?rev=1530033&r1=1530032&r2=1530033&view=diff
==============================================================================
--- hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDatanodeJsp.java (original)
+++ hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDatanodeJsp.java Mon Oct  7 19:15:29 2013
@@ -86,6 +86,9 @@ public class TestDatanodeJsp {
     // 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=\"///localhost:" + 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)
@@ -166,6 +169,7 @@ public class TestDatanodeJsp {
     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 {