You are viewing a plain text version of this content. The canonical link for it is here.
Posted to common-commits@hadoop.apache.org by om...@apache.org on 2011/03/04 05:08:18 UTC

svn commit: r1077371 - in /hadoop/common/branches/branch-0.20-security-patches/src: contrib/hdfsproxy/src/java/org/apache/hadoop/hdfsproxy/ hdfs/org/apache/hadoop/hdfs/ hdfs/org/apache/hadoop/hdfs/server/datanode/ hdfs/org/apache/hadoop/hdfs/server/nam...

Author: omalley
Date: Fri Mar  4 04:08:17 2011
New Revision: 1077371

URL: http://svn.apache.org/viewvc?rev=1077371&view=rev
Log:
commit 096325a12be9507ec8f1dcb9158fe4cd30826a10
Author: Boris Shkolnik <bo...@yahoo-inc.com>
Date:   Tue Apr 6 17:01:51 2010 -0700

    HDFS:1007 from https://issues.apache.org/jira/secure/attachment/12440931/HDFS-1007-BP20-fix-3.patch
    
    +++ b/YAHOO-CHANGES.txt
    +
    +    HDFS-1007. Fix. ServiceName for delegation token for Hftp has hftp
    +    port and not RPC port.

Modified:
    hadoop/common/branches/branch-0.20-security-patches/src/contrib/hdfsproxy/src/java/org/apache/hadoop/hdfsproxy/HdfsProxy.java
    hadoop/common/branches/branch-0.20-security-patches/src/contrib/hdfsproxy/src/java/org/apache/hadoop/hdfsproxy/ProxyFileDataServlet.java
    hadoop/common/branches/branch-0.20-security-patches/src/hdfs/org/apache/hadoop/hdfs/HftpFileSystem.java
    hadoop/common/branches/branch-0.20-security-patches/src/hdfs/org/apache/hadoop/hdfs/server/datanode/DataNode.java
    hadoop/common/branches/branch-0.20-security-patches/src/hdfs/org/apache/hadoop/hdfs/server/namenode/ContentSummaryServlet.java
    hadoop/common/branches/branch-0.20-security-patches/src/hdfs/org/apache/hadoop/hdfs/server/namenode/DfsServlet.java
    hadoop/common/branches/branch-0.20-security-patches/src/hdfs/org/apache/hadoop/hdfs/server/namenode/FileChecksumServlets.java
    hadoop/common/branches/branch-0.20-security-patches/src/hdfs/org/apache/hadoop/hdfs/server/namenode/FileDataServlet.java
    hadoop/common/branches/branch-0.20-security-patches/src/hdfs/org/apache/hadoop/hdfs/server/namenode/FsckServlet.java
    hadoop/common/branches/branch-0.20-security-patches/src/hdfs/org/apache/hadoop/hdfs/server/namenode/GetImageServlet.java
    hadoop/common/branches/branch-0.20-security-patches/src/hdfs/org/apache/hadoop/hdfs/server/namenode/JspHelper.java
    hadoop/common/branches/branch-0.20-security-patches/src/hdfs/org/apache/hadoop/hdfs/server/namenode/ListPathsServlet.java
    hadoop/common/branches/branch-0.20-security-patches/src/hdfs/org/apache/hadoop/hdfs/server/namenode/NameNode.java
    hadoop/common/branches/branch-0.20-security-patches/src/hdfs/org/apache/hadoop/hdfs/server/namenode/SecondaryNameNode.java
    hadoop/common/branches/branch-0.20-security-patches/src/hdfs/org/apache/hadoop/hdfs/server/namenode/StreamFile.java
    hadoop/common/branches/branch-0.20-security-patches/src/mapred/org/apache/hadoop/mapred/JobClient.java
    hadoop/common/branches/branch-0.20-security-patches/src/mapred/org/apache/hadoop/mapreduce/security/TokenCache.java
    hadoop/common/branches/branch-0.20-security-patches/src/tools/org/apache/hadoop/tools/DistCp.java
    hadoop/common/branches/branch-0.20-security-patches/src/webapps/datanode/browseBlock.jsp
    hadoop/common/branches/branch-0.20-security-patches/src/webapps/datanode/browseDirectory.jsp
    hadoop/common/branches/branch-0.20-security-patches/src/webapps/datanode/tail.jsp
    hadoop/common/branches/branch-0.20-security-patches/src/webapps/hdfs/nn_browsedfscontent.jsp

Modified: hadoop/common/branches/branch-0.20-security-patches/src/contrib/hdfsproxy/src/java/org/apache/hadoop/hdfsproxy/HdfsProxy.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-0.20-security-patches/src/contrib/hdfsproxy/src/java/org/apache/hadoop/hdfsproxy/HdfsProxy.java?rev=1077371&r1=1077370&r2=1077371&view=diff
==============================================================================
--- hadoop/common/branches/branch-0.20-security-patches/src/contrib/hdfsproxy/src/java/org/apache/hadoop/hdfsproxy/HdfsProxy.java (original)
+++ hadoop/common/branches/branch-0.20-security-patches/src/contrib/hdfsproxy/src/java/org/apache/hadoop/hdfsproxy/HdfsProxy.java Fri Mar  4 04:08:17 2011
@@ -29,6 +29,7 @@ import javax.net.ssl.HttpsURLConnection;
 import javax.net.ssl.HostnameVerifier;
 import javax.net.ssl.SSLSession;
 import javax.servlet.http.HttpServletResponse;
+import org.apache.hadoop.hdfs.server.namenode.JspHelper;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
@@ -74,7 +75,7 @@ public class HdfsProxy {
     this.server = new ProxyHttpServer(sslAddr, sslConf);
     this.server.setAttribute("proxy.https.port", server.getPort());
     this.server.setAttribute("name.node.address", nnAddr);
-    this.server.setAttribute("name.conf", new Configuration());
+    this.server.setAttribute(JspHelper.CURRENT_CONF, new Configuration());
     this.server.addGlobalFilter("ProxyFilter", ProxyFilter.class.getName(), null);
     this.server.addServlet("listPaths", "/listPaths/*", ProxyListPathsServlet.class);
     this.server.addServlet("data", "/data/*", ProxyFileDataServlet.class);

Modified: hadoop/common/branches/branch-0.20-security-patches/src/contrib/hdfsproxy/src/java/org/apache/hadoop/hdfsproxy/ProxyFileDataServlet.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-0.20-security-patches/src/contrib/hdfsproxy/src/java/org/apache/hadoop/hdfsproxy/ProxyFileDataServlet.java?rev=1077371&r1=1077370&r2=1077371&view=diff
==============================================================================
--- hadoop/common/branches/branch-0.20-security-patches/src/contrib/hdfsproxy/src/java/org/apache/hadoop/hdfsproxy/ProxyFileDataServlet.java (original)
+++ hadoop/common/branches/branch-0.20-security-patches/src/contrib/hdfsproxy/src/java/org/apache/hadoop/hdfsproxy/ProxyFileDataServlet.java Fri Mar  4 04:08:17 2011
@@ -27,6 +27,7 @@ import org.apache.hadoop.conf.Configurat
 import org.apache.hadoop.hdfs.protocol.ClientProtocol;
 import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
 import org.apache.hadoop.hdfs.server.namenode.FileDataServlet;
+import org.apache.hadoop.hdfs.server.namenode.JspHelper;
 import org.apache.hadoop.security.UserGroupInformation;
 
 /** {@inheritDoc} */
@@ -37,11 +38,18 @@ public class ProxyFileDataServlet extend
   /** {@inheritDoc} */
   @Override
   protected URI createUri(String parent, HdfsFileStatus i, UserGroupInformation ugi,
-      ClientProtocol nnproxy, HttpServletRequest request) throws IOException,
+      ClientProtocol nnproxy, HttpServletRequest request, String dt) throws IOException,
       URISyntaxException {
+    
+    String dtParam="";
+    if (dt != null) {
+      StringBuilder sb = new StringBuilder(JspHelper.SET_DELEGATION).append(dt);
+      dtParam=sb.toString();
+    }
+    
     return new URI(request.getScheme(), null, request.getServerName(), request
         .getServerPort(), "/streamFile", "filename=" + i.getFullName(parent) 
-        + "&ugi=" + ugi.getShortUserName(), null);
+        + "&ugi=" + ugi.getShortUserName() + dtParam, null);
   }
 
   /** {@inheritDoc} */

Modified: hadoop/common/branches/branch-0.20-security-patches/src/hdfs/org/apache/hadoop/hdfs/HftpFileSystem.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-0.20-security-patches/src/hdfs/org/apache/hadoop/hdfs/HftpFileSystem.java?rev=1077371&r1=1077370&r2=1077371&view=diff
==============================================================================
--- hadoop/common/branches/branch-0.20-security-patches/src/hdfs/org/apache/hadoop/hdfs/HftpFileSystem.java (original)
+++ hadoop/common/branches/branch-0.20-security-patches/src/hdfs/org/apache/hadoop/hdfs/HftpFileSystem.java Fri Mar  4 04:08:17 2011
@@ -19,52 +19,48 @@
 package org.apache.hadoop.hdfs;
 
 import java.io.FileNotFoundException;
-import java.io.InputStream;
 import java.io.IOException;
-
+import java.io.InputStream;
 import java.net.HttpURLConnection;
 import java.net.InetSocketAddress;
 import java.net.URI;
 import java.net.URISyntaxException;
 import java.net.URL;
-
 import java.security.PrivilegedExceptionAction;
 import java.text.ParseException;
 import java.text.SimpleDateFormat;
-
 import java.util.ArrayList;
 import java.util.Collection;
 import java.util.Random;
 import java.util.TimeZone;
 
-import org.xml.sax.Attributes;
-import org.xml.sax.InputSource;
-import org.xml.sax.SAXException;
-import org.xml.sax.XMLReader;
-import org.xml.sax.helpers.DefaultHandler;
-import org.xml.sax.helpers.XMLReaderFactory;
-
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.net.NetUtils;
 import org.apache.hadoop.fs.ContentSummary;
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.FSInputStream;
 import org.apache.hadoop.fs.FileChecksum;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.FSInputStream;
-import org.apache.hadoop.fs.FSDataInputStream;
-import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.MD5MD5CRC32FileChecksum;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.hdfs.server.namenode.JspHelper;
-import org.apache.hadoop.hdfs.server.namenode.ListPathsServlet;
 import org.apache.hadoop.hdfs.tools.DelegationTokenFetcher;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.ipc.RemoteException;
-import org.apache.hadoop.security.*;
+import org.apache.hadoop.net.NetUtils;
+import org.apache.hadoop.security.Credentials;
+import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.token.Token;
 import org.apache.hadoop.security.token.TokenIdentifier;
 import org.apache.hadoop.util.Progressable;
+import org.xml.sax.Attributes;
+import org.xml.sax.InputSource;
+import org.xml.sax.SAXException;
+import org.xml.sax.XMLReader;
+import org.xml.sax.helpers.DefaultHandler;
+import org.xml.sax.helpers.XMLReaderFactory;
 
 /** An implementation of a protocol for accessing filesystems over HTTP.
  * The following implementation provides a limited, read-only interface
@@ -84,6 +80,7 @@ public class HftpFileSystem extends File
   public static final String HFTP_DATE_FORMAT = "yyyy-MM-dd'T'HH:mm:ssZ";
   private Token<? extends TokenIdentifier> delegationToken;
   public static final String HFTP_RENEWER = "fs.hftp.renewer";
+  public static final String HFTP_SERVICE_NAME_KEY = "hdfs.service.host_";
 
   public static final SimpleDateFormat getDateFormat() {
     final SimpleDateFormat df = new SimpleDateFormat(HFTP_DATE_FORMAT);
@@ -108,17 +105,22 @@ public class HftpFileSystem extends File
     nnAddr = NetUtils.createSocketAddr(name.toString());
     
     if (UserGroupInformation.isSecurityEnabled()) {
-      StringBuffer sb = new StringBuffer();
-      final String nnServiceName = 
-        (sb.append(NetUtils.normalizeHostName(name.getHost()))
-                .append(":").append(name.getPort())).toString();
-      Text nnServiceNameText = new Text(nnServiceName);
+      StringBuffer sb = new StringBuffer(HFTP_SERVICE_NAME_KEY);
+      // configuration has the actual service name for this url. Build the key 
+      // and get it.
+      final String key = sb.append(NetUtils.normalizeHostName(name.getHost())).
+        append(".").append(name.getPort()).toString();
+      
+      LOG.debug("Trying to find DT for " + name + " using key=" + key + "; conf=" + conf.get(key, ""));
+      Text nnServiceNameText = new Text(conf.get(key, ""));
+      
       Collection<Token<? extends TokenIdentifier>> tokens =
         ugi.getTokens();
       //try finding a token for this namenode (esp applicable for tasks
       //using hftp). If there exists one, just set the delegationField
       for (Token<? extends TokenIdentifier> t : tokens) {
         if ((t.getService()).equals(nnServiceNameText)) {
+          LOG.debug("Found existing DT for " + name);
           delegationToken = t;
           return;
         }

Modified: hadoop/common/branches/branch-0.20-security-patches/src/hdfs/org/apache/hadoop/hdfs/server/datanode/DataNode.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-0.20-security-patches/src/hdfs/org/apache/hadoop/hdfs/server/datanode/DataNode.java?rev=1077371&r1=1077370&r2=1077371&view=diff
==============================================================================
--- hadoop/common/branches/branch-0.20-security-patches/src/hdfs/org/apache/hadoop/hdfs/server/datanode/DataNode.java (original)
+++ hadoop/common/branches/branch-0.20-security-patches/src/hdfs/org/apache/hadoop/hdfs/server/datanode/DataNode.java Fri Mar  4 04:08:17 2011
@@ -17,8 +17,6 @@
  */
 package org.apache.hadoop.hdfs.server.datanode;
 
-import static org.junit.Assert.assertTrue;
-
 import java.io.BufferedOutputStream;
 import java.io.DataOutputStream;
 import java.io.File;
@@ -50,13 +48,11 @@ import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configured;
-import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.LocalFileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
-import org.apache.hadoop.hdfs.DFSUtil;
 import org.apache.hadoop.hdfs.HDFSPolicyProvider;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.BlockListAsLongs;
@@ -70,14 +66,15 @@ import org.apache.hadoop.hdfs.protocol.U
 import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier;
 import org.apache.hadoop.hdfs.security.token.block.BlockTokenSecretManager;
 import org.apache.hadoop.hdfs.security.token.block.ExportedBlockKeys;
-import org.apache.hadoop.hdfs.server.common.HdfsConstants.StartupOption;
-import org.apache.hadoop.hdfs.server.common.HdfsConstants;
 import org.apache.hadoop.hdfs.server.common.GenerationStamp;
+import org.apache.hadoop.hdfs.server.common.HdfsConstants;
 import org.apache.hadoop.hdfs.server.common.IncorrectVersionException;
 import org.apache.hadoop.hdfs.server.common.Storage;
+import org.apache.hadoop.hdfs.server.common.HdfsConstants.StartupOption;
 import org.apache.hadoop.hdfs.server.datanode.metrics.DataNodeMetrics;
 import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
 import org.apache.hadoop.hdfs.server.namenode.FileChecksumServlets;
+import org.apache.hadoop.hdfs.server.namenode.JspHelper;
 import org.apache.hadoop.hdfs.server.namenode.NameNode;
 import org.apache.hadoop.hdfs.server.namenode.StreamFile;
 import org.apache.hadoop.hdfs.server.protocol.BlockCommand;
@@ -86,8 +83,8 @@ import org.apache.hadoop.hdfs.server.pro
 import org.apache.hadoop.hdfs.server.protocol.DatanodeProtocol;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeRegistration;
 import org.apache.hadoop.hdfs.server.protocol.DisallowedDatanodeException;
-import org.apache.hadoop.hdfs.server.protocol.KeyUpdateCommand;
 import org.apache.hadoop.hdfs.server.protocol.InterDatanodeProtocol;
+import org.apache.hadoop.hdfs.server.protocol.KeyUpdateCommand;
 import org.apache.hadoop.hdfs.server.protocol.NamespaceInfo;
 import org.apache.hadoop.hdfs.server.protocol.UpgradeCommand;
 import org.apache.hadoop.http.HttpServer;
@@ -100,15 +97,15 @@ import org.apache.hadoop.net.DNS;
 import org.apache.hadoop.net.NetUtils;
 import org.apache.hadoop.security.SecurityUtil;
 import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.security.authorize.ServiceAuthorizationManager;
+import org.apache.hadoop.security.token.Token;
+import org.apache.hadoop.security.token.TokenIdentifier;
 import org.apache.hadoop.util.Daemon;
 import org.apache.hadoop.util.DiskChecker;
 import org.apache.hadoop.util.ReflectionUtils;
 import org.apache.hadoop.util.StringUtils;
 import org.apache.hadoop.util.DiskChecker.DiskErrorException;
 import org.apache.hadoop.util.DiskChecker.DiskOutOfSpaceException;
-import org.apache.hadoop.security.authorize.ServiceAuthorizationManager;
-import org.apache.hadoop.security.token.Token;
-import org.apache.hadoop.security.token.TokenIdentifier;
 
 /**********************************************************
  * DataNode is a class (and program) that stores a set of
@@ -402,7 +399,7 @@ public class DataNode extends Configured
     this.infoServer.addInternalServlet(null, "/getFileChecksum/*",
         FileChecksumServlets.GetServlet.class);
     this.infoServer.setAttribute("datanode.blockScanner", blockScanner);
-    this.infoServer.setAttribute("datanode.conf", conf);
+    this.infoServer.setAttribute(JspHelper.CURRENT_CONF, conf);
     this.infoServer.addServlet(null, "/blockScannerReport", 
                                DataBlockScanner.Servlet.class);
     this.infoServer.start();

Modified: hadoop/common/branches/branch-0.20-security-patches/src/hdfs/org/apache/hadoop/hdfs/server/namenode/ContentSummaryServlet.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-0.20-security-patches/src/hdfs/org/apache/hadoop/hdfs/server/namenode/ContentSummaryServlet.java?rev=1077371&r1=1077370&r2=1077371&view=diff
==============================================================================
--- hadoop/common/branches/branch-0.20-security-patches/src/hdfs/org/apache/hadoop/hdfs/server/namenode/ContentSummaryServlet.java (original)
+++ hadoop/common/branches/branch-0.20-security-patches/src/hdfs/org/apache/hadoop/hdfs/server/namenode/ContentSummaryServlet.java Fri Mar  4 04:08:17 2011
@@ -41,7 +41,7 @@ public class ContentSummaryServlet exten
   public void doGet(final HttpServletRequest request,
       final HttpServletResponse response) throws ServletException, IOException {
     final Configuration conf = 
-      (Configuration) getServletContext().getAttribute("name.conf");
+      (Configuration) getServletContext().getAttribute(JspHelper.CURRENT_CONF);
     final UserGroupInformation ugi = getUGI(request, conf);
     try {
       ugi.doAs(new PrivilegedExceptionAction<Object>() {

Modified: hadoop/common/branches/branch-0.20-security-patches/src/hdfs/org/apache/hadoop/hdfs/server/namenode/DfsServlet.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-0.20-security-patches/src/hdfs/org/apache/hadoop/hdfs/server/namenode/DfsServlet.java?rev=1077371&r1=1077370&r2=1077371&view=diff
==============================================================================
--- hadoop/common/branches/branch-0.20-security-patches/src/hdfs/org/apache/hadoop/hdfs/server/namenode/DfsServlet.java (original)
+++ hadoop/common/branches/branch-0.20-security-patches/src/hdfs/org/apache/hadoop/hdfs/server/namenode/DfsServlet.java Fri Mar  4 04:08:17 2011
@@ -60,7 +60,7 @@ abstract class DfsServlet extends HttpSe
     ServletContext context = getServletContext();
     InetSocketAddress nnAddr = (InetSocketAddress)context.getAttribute("name.node.address");
     Configuration conf = new Configuration(
-        (Configuration)context.getAttribute("name.conf"));
+        (Configuration)context.getAttribute(JspHelper.CURRENT_CONF));
     return DFSClient.createNamenode(nnAddr, conf);
   }
 

Modified: hadoop/common/branches/branch-0.20-security-patches/src/hdfs/org/apache/hadoop/hdfs/server/namenode/FileChecksumServlets.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-0.20-security-patches/src/hdfs/org/apache/hadoop/hdfs/server/namenode/FileChecksumServlets.java?rev=1077371&r1=1077370&r2=1077371&view=diff
==============================================================================
--- hadoop/common/branches/branch-0.20-security-patches/src/hdfs/org/apache/hadoop/hdfs/server/namenode/FileChecksumServlets.java (original)
+++ hadoop/common/branches/branch-0.20-security-patches/src/hdfs/org/apache/hadoop/hdfs/server/namenode/FileChecksumServlets.java Fri Mar  4 04:08:17 2011
@@ -53,7 +53,7 @@ public class FileChecksumServlets {
     public void doGet(HttpServletRequest request, HttpServletResponse response
         ) throws ServletException, IOException {
       final ServletContext context = getServletContext();
-      Configuration conf = (Configuration) context.getAttribute("name.conf");
+      Configuration conf = (Configuration) context.getAttribute(JspHelper.CURRENT_CONF);
       final UserGroupInformation ugi = getUGI(request, conf);
       final NameNode namenode = (NameNode)context.getAttribute("name.node");
       final DatanodeID datanode = namenode.namesystem.getRandomDatanode();

Modified: hadoop/common/branches/branch-0.20-security-patches/src/hdfs/org/apache/hadoop/hdfs/server/namenode/FileDataServlet.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-0.20-security-patches/src/hdfs/org/apache/hadoop/hdfs/server/namenode/FileDataServlet.java?rev=1077371&r1=1077370&r2=1077371&view=diff
==============================================================================
--- hadoop/common/branches/branch-0.20-security-patches/src/hdfs/org/apache/hadoop/hdfs/server/namenode/FileDataServlet.java (original)
+++ hadoop/common/branches/branch-0.20-security-patches/src/hdfs/org/apache/hadoop/hdfs/server/namenode/FileDataServlet.java Fri Mar  4 04:08:17 2011
@@ -41,7 +41,7 @@ public class FileDataServlet extends Dfs
 
   /** Create a redirection URI */
   protected URI createUri(String parent, HdfsFileStatus i, UserGroupInformation ugi,
-      ClientProtocol nnproxy, HttpServletRequest request)
+      ClientProtocol nnproxy, HttpServletRequest request, String dt)
       throws IOException, URISyntaxException {
     String scheme = request.getScheme();
     final DatanodeID host = pickSrcDatanode(parent, i, nnproxy);
@@ -51,12 +51,19 @@ public class FileDataServlet extends Dfs
     } else {
       hostname = host.getHost();
     }
+    
+    String dtParam="";
+    if (dt != null) {
+      StringBuilder sb = new StringBuilder(JspHelper.SET_DELEGATION).append(dt);
+      dtParam=sb.toString();
+    }
+    
     return new URI(scheme, null, hostname,
         "https".equals(scheme)
           ? (Integer)getServletContext().getAttribute("datanode.https.port")
           : host.getInfoPort(),
         "/streamFile", "filename=" + i.getFullName(parent) + 
-        "&ugi=" + ugi.getShortUserName(), null);
+        "&ugi=" + ugi.getShortUserName() + dtParam, null);
   }
 
   private static JspHelper jspHelper = null;
@@ -91,7 +98,7 @@ public class FileDataServlet extends Dfs
   public void doGet(HttpServletRequest request, HttpServletResponse response)
     throws IOException {
     Configuration conf =
-	(Configuration) getServletContext().getAttribute("name.conf");
+	(Configuration) getServletContext().getAttribute(JspHelper.CURRENT_CONF);
     final UserGroupInformation ugi = getUGI(request, conf);
 
     try {
@@ -103,13 +110,16 @@ public class FileDataServlet extends Dfs
             }
           });
 
-      final String path = request.getPathInfo() != null ? 
-                                                    request.getPathInfo() : "/";
+      final String path = 
+        request.getPathInfo() != null ? request.getPathInfo() : "/";
+      
+      String delegationToken = 
+        request.getParameter(JspHelper.DELEGATION_PARAMETER_NAME);
       
       HdfsFileStatus info = nnproxy.getFileInfo(path);
       if ((info != null) && !info.isDir()) {
         response.sendRedirect(createUri(path, info, ugi, nnproxy,
-              request).toURL().toString());
+              request, delegationToken).toURL().toString());
       } else if (info == null){
         response.sendError(400, "cat: File not found " + path);
       } else {

Modified: hadoop/common/branches/branch-0.20-security-patches/src/hdfs/org/apache/hadoop/hdfs/server/namenode/FsckServlet.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-0.20-security-patches/src/hdfs/org/apache/hadoop/hdfs/server/namenode/FsckServlet.java?rev=1077371&r1=1077370&r2=1077371&view=diff
==============================================================================
--- hadoop/common/branches/branch-0.20-security-patches/src/hdfs/org/apache/hadoop/hdfs/server/namenode/FsckServlet.java (original)
+++ hadoop/common/branches/branch-0.20-security-patches/src/hdfs/org/apache/hadoop/hdfs/server/namenode/FsckServlet.java Fri Mar  4 04:08:17 2011
@@ -45,7 +45,7 @@ public class FsckServlet extends DfsServ
     final PrintWriter out = response.getWriter();
     final ServletContext context = getServletContext();
     final Configuration conf = 
-      (Configuration) context.getAttribute("name.conf");
+      (Configuration) context.getAttribute(JspHelper.CURRENT_CONF);
     final UserGroupInformation ugi = getUGI(request, conf);
     try {
       ugi.doAs(new PrivilegedExceptionAction<Object>() {

Modified: hadoop/common/branches/branch-0.20-security-patches/src/hdfs/org/apache/hadoop/hdfs/server/namenode/GetImageServlet.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-0.20-security-patches/src/hdfs/org/apache/hadoop/hdfs/server/namenode/GetImageServlet.java?rev=1077371&r1=1077370&r2=1077371&view=diff
==============================================================================
--- hadoop/common/branches/branch-0.20-security-patches/src/hdfs/org/apache/hadoop/hdfs/server/namenode/GetImageServlet.java (original)
+++ hadoop/common/branches/branch-0.20-security-patches/src/hdfs/org/apache/hadoop/hdfs/server/namenode/GetImageServlet.java Fri Mar  4 04:08:17 2011
@@ -57,7 +57,7 @@ public class GetImageServlet extends Htt
       ServletContext context = getServletContext();
       final FSImage nnImage = (FSImage)context.getAttribute("name.system.image");
       final TransferFsImage ff = new TransferFsImage(pmap, request, response);
-      final Configuration conf = (Configuration)getServletContext().getAttribute("name.conf");
+      final Configuration conf = (Configuration)getServletContext().getAttribute(JspHelper.CURRENT_CONF);
       if(UserGroupInformation.isSecurityEnabled() && 
           !isValidRequestor(request.getRemoteUser(), conf)) {
         response.sendError(HttpServletResponse.SC_FORBIDDEN, 

Modified: hadoop/common/branches/branch-0.20-security-patches/src/hdfs/org/apache/hadoop/hdfs/server/namenode/JspHelper.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-0.20-security-patches/src/hdfs/org/apache/hadoop/hdfs/server/namenode/JspHelper.java?rev=1077371&r1=1077370&r2=1077371&view=diff
==============================================================================
--- hadoop/common/branches/branch-0.20-security-patches/src/hdfs/org/apache/hadoop/hdfs/server/namenode/JspHelper.java (original)
+++ hadoop/common/branches/branch-0.20-security-patches/src/hdfs/org/apache/hadoop/hdfs/server/namenode/JspHelper.java Fri Mar  4 04:08:17 2011
@@ -58,6 +58,7 @@ import org.apache.hadoop.util.StringUtil
 import org.apache.hadoop.net.NetUtils;
 
 public class JspHelper {
+  public static final String CURRENT_CONF = "current.conf";
   final static public String WEB_UGI_PROPERTY_NAME = "dfs.web.ugi";
   public static final String DELEGATION_PARAMETER_NAME = "delegation";
   public static final String SET_DELEGATION = "&" + DELEGATION_PARAMETER_NAME +

Modified: hadoop/common/branches/branch-0.20-security-patches/src/hdfs/org/apache/hadoop/hdfs/server/namenode/ListPathsServlet.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-0.20-security-patches/src/hdfs/org/apache/hadoop/hdfs/server/namenode/ListPathsServlet.java?rev=1077371&r1=1077370&r2=1077371&view=diff
==============================================================================
--- hadoop/common/branches/branch-0.20-security-patches/src/hdfs/org/apache/hadoop/hdfs/server/namenode/ListPathsServlet.java (original)
+++ hadoop/common/branches/branch-0.20-security-patches/src/hdfs/org/apache/hadoop/hdfs/server/namenode/ListPathsServlet.java Fri Mar  4 04:08:17 2011
@@ -136,7 +136,7 @@ public class ListPathsServlet extends Df
       final Pattern filter = Pattern.compile(root.get("filter"));
       final Pattern exclude = Pattern.compile(root.get("exclude"));
       final Configuration conf = 
-        (Configuration) getServletContext().getAttribute("name.conf");
+        (Configuration) getServletContext().getAttribute(JspHelper.CURRENT_CONF);
       
       ClientProtocol nnproxy = getUGI(request, conf).doAs
         (new PrivilegedExceptionAction<ClientProtocol>() {

Modified: hadoop/common/branches/branch-0.20-security-patches/src/hdfs/org/apache/hadoop/hdfs/server/namenode/NameNode.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-0.20-security-patches/src/hdfs/org/apache/hadoop/hdfs/server/namenode/NameNode.java?rev=1077371&r1=1077370&r2=1077371&view=diff
==============================================================================
--- hadoop/common/branches/branch-0.20-security-patches/src/hdfs/org/apache/hadoop/hdfs/server/namenode/NameNode.java (original)
+++ hadoop/common/branches/branch-0.20-security-patches/src/hdfs/org/apache/hadoop/hdfs/server/namenode/NameNode.java Fri Mar  4 04:08:17 2011
@@ -288,7 +288,7 @@ public class NameNode implements ClientP
           httpServer.setAttribute("name.node", NameNode.this);
           httpServer.setAttribute("name.node.address", getNameNodeAddress());
           httpServer.setAttribute("name.system.image", getFSImage());
-          httpServer.setAttribute("name.conf", conf);
+          httpServer.setAttribute(JspHelper.CURRENT_CONF, conf);
           httpServer.addInternalServlet("getDelegationToken", 
               DelegationTokenServlet.PATH_SPEC, DelegationTokenServlet.class, true);
           httpServer.addInternalServlet("fsck", "/fsck", FsckServlet.class, true);

Modified: hadoop/common/branches/branch-0.20-security-patches/src/hdfs/org/apache/hadoop/hdfs/server/namenode/SecondaryNameNode.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-0.20-security-patches/src/hdfs/org/apache/hadoop/hdfs/server/namenode/SecondaryNameNode.java?rev=1077371&r1=1077370&r2=1077371&view=diff
==============================================================================
--- hadoop/common/branches/branch-0.20-security-patches/src/hdfs/org/apache/hadoop/hdfs/server/namenode/SecondaryNameNode.java (original)
+++ hadoop/common/branches/branch-0.20-security-patches/src/hdfs/org/apache/hadoop/hdfs/server/namenode/SecondaryNameNode.java Fri Mar  4 04:08:17 2011
@@ -210,7 +210,7 @@ public class SecondaryNameNode implement
           }
           
           infoServer.setAttribute("name.system.image", checkpointImage);
-          infoServer.setAttribute("name.conf", conf);
+          infoServer.setAttribute(JspHelper.CURRENT_CONF, conf);
           infoServer.addInternalServlet("getimage", "/getimage",
               GetImageServlet.class, true);
           infoServer.start();

Modified: hadoop/common/branches/branch-0.20-security-patches/src/hdfs/org/apache/hadoop/hdfs/server/namenode/StreamFile.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-0.20-security-patches/src/hdfs/org/apache/hadoop/hdfs/server/namenode/StreamFile.java?rev=1077371&r1=1077370&r2=1077371&view=diff
==============================================================================
--- hadoop/common/branches/branch-0.20-security-patches/src/hdfs/org/apache/hadoop/hdfs/server/namenode/StreamFile.java (original)
+++ hadoop/common/branches/branch-0.20-security-patches/src/hdfs/org/apache/hadoop/hdfs/server/namenode/StreamFile.java Fri Mar  4 04:08:17 2011
@@ -40,7 +40,7 @@ public class StreamFile extends DfsServl
   public void doGet(HttpServletRequest request, HttpServletResponse response)
     throws ServletException, IOException {
     Configuration conf = 
-      (Configuration) getServletContext().getAttribute("name.conf");
+      (Configuration) getServletContext().getAttribute(JspHelper.CURRENT_CONF);
     String filename = request.getParameter("filename");
     if (filename == null || filename.length() == 0) {
       response.setContentType("text/plain");
@@ -52,7 +52,7 @@ public class StreamFile extends DfsServl
     DFSClient dfs;
     UserGroupInformation ugi = getUGI(request, conf);
     try {
-	dfs = JspHelper.getDFSClient(ugi, nameNodeAddr, conf);
+      dfs = JspHelper.getDFSClient(ugi, nameNodeAddr, conf);
     } catch (InterruptedException e) {
       response.sendError(400, e.getMessage());
       return;

Modified: hadoop/common/branches/branch-0.20-security-patches/src/mapred/org/apache/hadoop/mapred/JobClient.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-0.20-security-patches/src/mapred/org/apache/hadoop/mapred/JobClient.java?rev=1077371&r1=1077370&r2=1077371&view=diff
==============================================================================
--- hadoop/common/branches/branch-0.20-security-patches/src/mapred/org/apache/hadoop/mapred/JobClient.java (original)
+++ hadoop/common/branches/branch-0.20-security-patches/src/mapred/org/apache/hadoop/mapred/JobClient.java Fri Mar  4 04:08:17 2011
@@ -1919,7 +1919,7 @@ public class JobClient extends Configure
  
     // add the delegation tokens from configuration
     String [] nameNodes = conf.getStrings(JobContext.JOB_NAMENODES);
-    LOG.info("adding the following namenodes' delegation tokens:" + Arrays.toString(nameNodes));
+    LOG.debug("adding the following namenodes' delegation tokens:" + Arrays.toString(nameNodes));
     if(nameNodes != null) {
       Path [] ps = new Path[nameNodes.length];
       for(int i=0; i< nameNodes.length; i++) {

Modified: hadoop/common/branches/branch-0.20-security-patches/src/mapred/org/apache/hadoop/mapreduce/security/TokenCache.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-0.20-security-patches/src/mapred/org/apache/hadoop/mapreduce/security/TokenCache.java?rev=1077371&r1=1077370&r2=1077371&view=diff
==============================================================================
--- hadoop/common/branches/branch-0.20-security-patches/src/mapred/org/apache/hadoop/mapreduce/security/TokenCache.java (original)
+++ hadoop/common/branches/branch-0.20-security-patches/src/mapred/org/apache/hadoop/mapreduce/security/TokenCache.java Fri Mar  4 04:08:17 2011
@@ -20,30 +20,26 @@ package org.apache.hadoop.mapreduce.secu
 
 import java.io.IOException;
 import java.net.URI;
-import java.util.Collection;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.FSDataInputStream;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hdfs.DistributedFileSystem;
 import org.apache.hadoop.hdfs.HftpFileSystem;
 import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier;
 import org.apache.hadoop.hdfs.server.namenode.NameNode;
-import org.apache.hadoop.hdfs.tools.DelegationTokenFetcher;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.mapred.JobConf;
 import org.apache.hadoop.mapred.JobTracker;
-import org.apache.hadoop.mapreduce.JobContext;
 import org.apache.hadoop.mapreduce.security.token.JobTokenIdentifier;
 import org.apache.hadoop.net.NetUtils;
 import org.apache.hadoop.security.Credentials;
 import org.apache.hadoop.security.KerberosName;
+import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.token.Token;
 import org.apache.hadoop.security.token.TokenIdentifier;
-import org.apache.hadoop.security.UserGroupInformation;
 
 /**
  * This class provides user facing APIs for transferring secrets from
@@ -130,7 +126,7 @@ public class TokenCache {
 
         token.setService(new Text(fs_addr));
         credentials.addToken(new Text(fs_addr), token);
-        LOG.info("Got dt for " + p.toString() + ";uri="+ fs_addr + 
+        LOG.info("Got dt for " + p + ";uri="+ fs_addr + 
             ";t.service="+token.getService());
       } else if (fs instanceof HftpFileSystem) {
         String fs_addr = buildDTServiceName(fs.getUri());
@@ -142,8 +138,17 @@ public class TokenCache {
         }
         //the initialize method of hftp, called via FileSystem.get() done
         //earlier gets a delegation token
-        credentials.addToken(new Text(fs_addr), 
-            ((HftpFileSystem) fs).getDelegationToken());
+        Token<? extends TokenIdentifier> t = ((HftpFileSystem) fs).getDelegationToken(); 
+        credentials.addToken(new Text(fs_addr), t);
+        
+        // in this case port in fs_addr is port for hftp request, but
+        // token's port is for RPC
+        // to find the correct DT we need to know the mapping between Hftp port 
+        // and RPC one. hence this new setting in the conf.
+        URI uri = ((HftpFileSystem) fs).getUri();
+        String key = HftpFileSystem.HFTP_SERVICE_NAME_KEY+uri.getHost() + "." + uri.getPort();
+        conf.set(key, t.getService().toString());
+        LOG.info("GOT dt for " + p + " and stored in conf as " + key + "=" + t.getService());
       }
     }
   }

Modified: hadoop/common/branches/branch-0.20-security-patches/src/tools/org/apache/hadoop/tools/DistCp.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-0.20-security-patches/src/tools/org/apache/hadoop/tools/DistCp.java?rev=1077371&r1=1077370&r2=1077371&view=diff
==============================================================================
--- hadoop/common/branches/branch-0.20-security-patches/src/tools/org/apache/hadoop/tools/DistCp.java (original)
+++ hadoop/common/branches/branch-0.20-security-patches/src/tools/org/apache/hadoop/tools/DistCp.java Fri Mar  4 04:08:17 2011
@@ -623,21 +623,20 @@ public class DistCp implements Tool {
   }
 
   /** Sanity check for srcPath */
-  private static void checkSrcPath(Configuration conf, 
-                                   List<Path> srcPaths, JobConf jobConf)
+  private static void checkSrcPath(JobConf jobConf, List<Path> srcPaths)
   throws IOException {
     List<IOException> rslt = new ArrayList<IOException>();
     
     // get tokens for all the required FileSystems..
     // also set the renewer as the JobTracker for the hftp case
-    conf.set(HftpFileSystem.HFTP_RENEWER, 
-        conf.get(JobTracker.JT_USER_NAME, ""));
+    jobConf.set(HftpFileSystem.HFTP_RENEWER, 
+        jobConf.get(JobTracker.JT_USER_NAME, ""));
     Path[] ps = new Path[srcPaths.size()];
     ps = srcPaths.toArray(ps);
-    TokenCache.obtainTokensForNamenodes(jobConf.getCredentials(), ps, conf);
+    TokenCache.obtainTokensForNamenodes(jobConf.getCredentials(), ps, jobConf);
 
     for (Path p : srcPaths) {
-      FileSystem fs = p.getFileSystem(conf);
+      FileSystem fs = p.getFileSystem(jobConf);
       if (!fs.exists(p)) {
         rslt.add(new IOException("Input source " + p + " does not exist."));
       }
@@ -658,7 +657,7 @@ public class DistCp implements Tool {
 
     JobConf job = createJobConf(conf);
     
-    checkSrcPath(conf, args.srcs, job);
+    checkSrcPath(job, args.srcs);
     if (args.preservedAttributes != null) {
       job.set(PRESERVE_STATUS_LABEL, args.preservedAttributes);
     }

Modified: hadoop/common/branches/branch-0.20-security-patches/src/webapps/datanode/browseBlock.jsp
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-0.20-security-patches/src/webapps/datanode/browseBlock.jsp?rev=1077371&r1=1077370&r2=1077371&view=diff
==============================================================================
--- hadoop/common/branches/branch-0.20-security-patches/src/webapps/datanode/browseBlock.jsp (original)
+++ hadoop/common/branches/branch-0.20-security-patches/src/webapps/datanode/browseBlock.jsp Fri Mar  4 04:08:17 2011
@@ -411,7 +411,7 @@
 <body onload="document.goto.dir.focus()">
 <% 
    Configuration conf = 
-     (Configuration) getServletContext().getAttribute("datanode.conf");
+     (Configuration) getServletContext().getAttribute(JspHelper.CURRENT_CONF);
    generateFileChunks(out, request, conf);
 %>
 <hr>

Modified: hadoop/common/branches/branch-0.20-security-patches/src/webapps/datanode/browseDirectory.jsp
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-0.20-security-patches/src/webapps/datanode/browseDirectory.jsp?rev=1077371&r1=1077370&r2=1077371&view=diff
==============================================================================
--- hadoop/common/branches/branch-0.20-security-patches/src/webapps/datanode/browseDirectory.jsp (original)
+++ hadoop/common/branches/branch-0.20-security-patches/src/webapps/datanode/browseDirectory.jsp Fri Mar  4 04:08:17 2011
@@ -174,7 +174,7 @@ body 
 <body onload="document.goto.dir.focus()">
 <% 
   Configuration conf = 
-    (Configuration) getServletContext().getAttribute("datanode.conf");
+    (Configuration) getServletContext().getAttribute(JspHelper.CURRENT_CONF);
   try {
     generateDirectoryStructure(out,request,response,conf);
   }

Modified: hadoop/common/branches/branch-0.20-security-patches/src/webapps/datanode/tail.jsp
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-0.20-security-patches/src/webapps/datanode/tail.jsp?rev=1077371&r1=1077370&r2=1077371&view=diff
==============================================================================
--- hadoop/common/branches/branch-0.20-security-patches/src/webapps/datanode/tail.jsp (original)
+++ hadoop/common/branches/branch-0.20-security-patches/src/webapps/datanode/tail.jsp Fri Mar  4 04:08:17 2011
@@ -125,7 +125,7 @@
 <form action="/tail.jsp" method="GET">
 <% 
    Configuration conf = 
-     (Configuration) application.getAttribute("datanode.conf");
+     (Configuration) application.getAttribute(JspHelper.CURRENT_CONF);
    generateFileChunks(out, request, conf);
 %>
 </form>

Modified: hadoop/common/branches/branch-0.20-security-patches/src/webapps/hdfs/nn_browsedfscontent.jsp
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-0.20-security-patches/src/webapps/hdfs/nn_browsedfscontent.jsp?rev=1077371&r1=1077370&r2=1077371&view=diff
==============================================================================
--- hadoop/common/branches/branch-0.20-security-patches/src/webapps/hdfs/nn_browsedfscontent.jsp (original)
+++ hadoop/common/branches/branch-0.20-security-patches/src/webapps/hdfs/nn_browsedfscontent.jsp Fri Mar  4 04:08:17 2011
@@ -77,7 +77,7 @@
 <body>
 <% 
   NameNode nn = (NameNode)application.getAttribute("name.node");
-  Configuration conf = (Configuration) application.getAttribute("name.conf");
+  Configuration conf = (Configuration) application.getAttribute(JspHelper.CURRENT_CONF);
   redirectToRandomDataNode(nn, request, response, conf); 
 %>
 <hr>