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 bo...@apache.org on 2010/07/15 02:29:32 UTC

svn commit: r964266 - in /hadoop/hdfs/trunk: ./ src/contrib/hdfsproxy/src/java/org/apache/hadoop/hdfsproxy/ src/java/org/apache/hadoop/hdfs/ src/java/org/apache/hadoop/hdfs/server/common/ src/java/org/apache/hadoop/hdfs/server/datanode/ src/java/org/ap...

Author: boryas
Date: Thu Jul 15 00:29:31 2010
New Revision: 964266

URL: http://svn.apache.org/viewvc?rev=964266&view=rev
Log:
HDFS-1007.  HFTP needs to be updated to use delegation tokens

Modified:
    hadoop/hdfs/trunk/CHANGES.txt
    hadoop/hdfs/trunk/src/contrib/hdfsproxy/src/java/org/apache/hadoop/hdfsproxy/HdfsProxy.java
    hadoop/hdfs/trunk/src/contrib/hdfsproxy/src/java/org/apache/hadoop/hdfsproxy/ProxyFileDataServlet.java
    hadoop/hdfs/trunk/src/java/org/apache/hadoop/hdfs/HftpFileSystem.java
    hadoop/hdfs/trunk/src/java/org/apache/hadoop/hdfs/HsftpFileSystem.java
    hadoop/hdfs/trunk/src/java/org/apache/hadoop/hdfs/server/common/JspHelper.java
    hadoop/hdfs/trunk/src/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java
    hadoop/hdfs/trunk/src/java/org/apache/hadoop/hdfs/server/namenode/ContentSummaryServlet.java
    hadoop/hdfs/trunk/src/java/org/apache/hadoop/hdfs/server/namenode/DelegationTokenServlet.java
    hadoop/hdfs/trunk/src/java/org/apache/hadoop/hdfs/server/namenode/DfsServlet.java
    hadoop/hdfs/trunk/src/java/org/apache/hadoop/hdfs/server/namenode/FileChecksumServlets.java
    hadoop/hdfs/trunk/src/java/org/apache/hadoop/hdfs/server/namenode/FileDataServlet.java
    hadoop/hdfs/trunk/src/java/org/apache/hadoop/hdfs/server/namenode/FsckServlet.java
    hadoop/hdfs/trunk/src/java/org/apache/hadoop/hdfs/server/namenode/GetImageServlet.java
    hadoop/hdfs/trunk/src/java/org/apache/hadoop/hdfs/server/namenode/ListPathsServlet.java
    hadoop/hdfs/trunk/src/java/org/apache/hadoop/hdfs/server/namenode/NameNode.java
    hadoop/hdfs/trunk/src/java/org/apache/hadoop/hdfs/server/namenode/SecondaryNameNode.java
    hadoop/hdfs/trunk/src/java/org/apache/hadoop/hdfs/server/namenode/StreamFile.java
    hadoop/hdfs/trunk/src/java/org/apache/hadoop/hdfs/tools/DelegationTokenFetcher.java
    hadoop/hdfs/trunk/src/webapps/datanode/browseBlock.jsp
    hadoop/hdfs/trunk/src/webapps/datanode/browseDirectory.jsp
    hadoop/hdfs/trunk/src/webapps/datanode/tail.jsp
    hadoop/hdfs/trunk/src/webapps/hdfs/nn_browsedfscontent.jsp

Modified: hadoop/hdfs/trunk/CHANGES.txt
URL: http://svn.apache.org/viewvc/hadoop/hdfs/trunk/CHANGES.txt?rev=964266&r1=964265&r2=964266&view=diff
==============================================================================
--- hadoop/hdfs/trunk/CHANGES.txt (original)
+++ hadoop/hdfs/trunk/CHANGES.txt Thu Jul 15 00:29:31 2010
@@ -139,6 +139,8 @@ Trunk (unreleased changes)
 
     HDFS-1289. Datanode secure mode is broken. (Kan Zhang via jghoman)
 
+    HDFS-1007. HFTP needs to be updated to use delegation tokens (boryas)
+
 Release 0.21.0 - Unreleased
 
   INCOMPATIBLE CHANGES

Modified: hadoop/hdfs/trunk/src/contrib/hdfsproxy/src/java/org/apache/hadoop/hdfsproxy/HdfsProxy.java
URL: http://svn.apache.org/viewvc/hadoop/hdfs/trunk/src/contrib/hdfsproxy/src/java/org/apache/hadoop/hdfsproxy/HdfsProxy.java?rev=964266&r1=964265&r2=964266&view=diff
==============================================================================
--- hadoop/hdfs/trunk/src/contrib/hdfsproxy/src/java/org/apache/hadoop/hdfsproxy/HdfsProxy.java (original)
+++ hadoop/hdfs/trunk/src/contrib/hdfsproxy/src/java/org/apache/hadoop/hdfsproxy/HdfsProxy.java Thu Jul 15 00:29:31 2010
@@ -19,6 +19,7 @@
 package org.apache.hadoop.hdfsproxy;
 
 import java.io.IOException;
+
 import java.net.InetSocketAddress;
 import java.net.URI;
 import java.net.URISyntaxException;
@@ -30,6 +31,7 @@ import org.apache.hadoop.conf.Configurat
 import org.apache.hadoop.net.NetUtils;
 import org.apache.hadoop.util.StringUtils;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
+import org.apache.hadoop.hdfs.server.common.JspHelper;
 
 /**
  * A HTTPS/SSL proxy to HDFS, implementing certificate based access control.
@@ -68,7 +70,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 HdfsConfiguration());
+    this.server.setAttribute(JspHelper.CURRENT_CONF, new HdfsConfiguration());
     this.server.addGlobalFilter("ProxyFilter", ProxyFilter.class.getName(), null);
     this.server.addServlet("listPaths", "/listPaths/*", ProxyListPathsServlet.class);
     this.server.addServlet("data", "/data/*", ProxyFileDataServlet.class);

Modified: hadoop/hdfs/trunk/src/contrib/hdfsproxy/src/java/org/apache/hadoop/hdfsproxy/ProxyFileDataServlet.java
URL: http://svn.apache.org/viewvc/hadoop/hdfs/trunk/src/contrib/hdfsproxy/src/java/org/apache/hadoop/hdfsproxy/ProxyFileDataServlet.java?rev=964266&r1=964265&r2=964266&view=diff
==============================================================================
--- hadoop/hdfs/trunk/src/contrib/hdfsproxy/src/java/org/apache/hadoop/hdfsproxy/ProxyFileDataServlet.java (original)
+++ hadoop/hdfs/trunk/src/contrib/hdfsproxy/src/java/org/apache/hadoop/hdfsproxy/ProxyFileDataServlet.java Thu Jul 15 00:29:31 2010
@@ -26,6 +26,7 @@ import javax.servlet.http.HttpServletReq
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hdfs.protocol.ClientProtocol;
 import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
+import org.apache.hadoop.hdfs.server.common.JspHelper;
 import org.apache.hadoop.hdfs.server.namenode.FileDataServlet;
 import org.apache.hadoop.security.UserGroupInformation;
 
@@ -37,11 +38,17 @@ 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" + i.getFullName(parent),
-        "&ugi=" + ugi.getShortUserName(), null);
+        "&ugi=" + ugi.getShortUserName() + dtParam, null);
   }
 
   /** {@inheritDoc} */

Modified: hadoop/hdfs/trunk/src/java/org/apache/hadoop/hdfs/HftpFileSystem.java
URL: http://svn.apache.org/viewvc/hadoop/hdfs/trunk/src/java/org/apache/hadoop/hdfs/HftpFileSystem.java?rev=964266&r1=964265&r2=964266&view=diff
==============================================================================
--- hadoop/hdfs/trunk/src/java/org/apache/hadoop/hdfs/HftpFileSystem.java (original)
+++ hadoop/hdfs/trunk/src/java/org/apache/hadoop/hdfs/HftpFileSystem.java Thu Jul 15 00:29:31 2010
@@ -26,10 +26,11 @@ 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.EnumSet;
+import java.util.Collection;
 import java.util.Random;
 import java.util.TimeZone;
 
@@ -37,7 +38,6 @@ import org.apache.hadoop.classification.
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.ContentSummary;
-import org.apache.hadoop.fs.CreateFlag;
 import org.apache.hadoop.fs.FSDataInputStream;
 import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.FileChecksum;
@@ -46,9 +46,15 @@ import org.apache.hadoop.fs.FileSystem;
 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.common.JspHelper;
+import org.apache.hadoop.hdfs.tools.DelegationTokenFetcher;
+import org.apache.hadoop.io.Text;
 import org.apache.hadoop.ipc.RemoteException;
 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;
@@ -78,6 +84,9 @@ public class HftpFileSystem extends File
 
   public static final String HFTP_TIMEZONE = "UTC";
   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);
@@ -93,13 +102,74 @@ public class HftpFileSystem extends File
     };
 
   @Override
-  public void initialize(URI name, Configuration conf) throws IOException {
+  public void initialize(final URI name, final Configuration conf)
+  throws IOException {
     super.initialize(name, conf);
     setConf(conf);
     this.ugi = UserGroupInformation.getCurrentUser(); 
     nnAddr = NetUtils.createSocketAddr(name.toString());
+    
+    if (UserGroupInformation.isSecurityEnabled()) {
+      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;
+        }
+      }
+      //since we don't already have a token, go get one over https
+      try {
+        ugi.doAs(new PrivilegedExceptionAction<Object>() {
+          public Object run() throws IOException {
+            StringBuffer sb = new StringBuffer();
+            //try https (on http we NEVER get a delegation token)
+            String nnHttpUrl = "https://" + 
+            (sb.append(NetUtils.normalizeHostName(name.getHost()))
+                .append(":").append(conf.getInt("dfs.https.port", 50470))).
+                toString();
+            Credentials c;
+            try {
+              c = DelegationTokenFetcher.getDTfromRemote(nnHttpUrl, 
+                  conf.get(HFTP_RENEWER));
+            } catch (Exception e) {
+              LOG.info("Couldn't get a delegation token from " + nnHttpUrl + 
+              " using https.");
+              //Maybe the server is in unsecure mode (that's bad but okay)
+              return null;
+            }
+            for (Token<? extends TokenIdentifier> t : c.getAllTokens()) {
+              //the service field is already set and so setService 
+              //is not required
+              delegationToken = t;
+              LOG.debug("Got dt for " + getUri() + ";t.service="
+                  +t.getService());
+            }
+            return null;
+          }
+        });
+      } catch (InterruptedException e) {
+        throw new RuntimeException(e);
+      }
+    }
   }
   
+  
+  public Token<? extends TokenIdentifier> getDelegationToken() {
+    return delegationToken;
+  }
 
   @Override
   public URI getUri() {
@@ -116,7 +186,7 @@ public class HftpFileSystem extends File
     Construct URL pointing to file on namenode
   */
   URL getNamenodeFileURL(Path f) throws IOException {
-    return getNamenodeURL("/data" + f.toUri().getPath(), "ugi=" + ugi.getShortUserName());
+    return getNamenodeURL("/data" + f.toUri().getPath(), "ugi=" + getUgiParameter());
   }
 
   /* 
@@ -136,12 +206,27 @@ public class HftpFileSystem extends File
   }
 
   /**
+   * ugi parameter for http connection
+   * 
+   * @return user_shortname,group1,group2...
+   */
+  private String getUgiParameter() {
+    StringBuilder ugiParamenter = new StringBuilder(ugi.getShortUserName());
+    for(String g: ugi.getGroupNames()) {
+      ugiParamenter.append(",");
+      ugiParamenter.append(g);
+    }
+    return ugiParamenter.toString();
+  }
+  
+  /**
    * Open an HTTP connection to the namenode to read file data and metadata.
    * @param path The path component of the URL
    * @param query The query component of the URL
    */
   protected HttpURLConnection openConnection(String path, String query)
       throws IOException {
+    query = updateQuery(query);
     final URL url = getNamenodeURL(path, query);
     HttpURLConnection connection = (HttpURLConnection)url.openConnection();
     connection.setRequestMethod("GET");
@@ -149,9 +234,21 @@ public class HftpFileSystem extends File
     return connection;
   }
 
+  protected String updateQuery(String query) throws IOException {
+    String tokenString = null;
+    if (UserGroupInformation.isSecurityEnabled()) {
+      if (delegationToken != null) {
+        tokenString = delegationToken.encodeToUrlString();
+        return (query + JspHelper.SET_DELEGATION + tokenString);
+      } // else we are talking to an unsecure cluster
+    }
+    return query;
+  }
+
   @Override
   public FSDataInputStream open(Path f, int buffersize) throws IOException {
-    URL u = getNamenodeURL("/data" + f.toUri().getPath(), "ugi=" + ugi.getShortUserName());
+    URL u = getNamenodeURL("/data" + f.toUri().getPath(), 
+        "ugi=" + getUgiParameter());
     return new FSDataInputStream(new ByteRangeInputStream(u));
   }
 
@@ -201,7 +298,7 @@ public class HftpFileSystem extends File
         XMLReader xr = XMLReaderFactory.createXMLReader();
         xr.setContentHandler(this);
         HttpURLConnection connection = openConnection("/listPaths" + path,
-            "ugi=" + ugi.getShortUserName() + (recur? "&recursive=yes" : ""));
+            "ugi=" + getUgiParameter() + (recur? "&recursive=yes" : ""));
 
         InputStream resp = connection.getInputStream();
         xr.parse(new InputSource(resp));
@@ -265,7 +362,7 @@ public class HftpFileSystem extends File
 
     private FileChecksum getFileChecksum(String f) throws IOException {
       final HttpURLConnection connection = openConnection(
-          "/fileChecksum" + f, "ugi=" + ugi.getShortUserName());
+          "/fileChecksum" + f, "ugi=" + getUgiParameter());
       try {
         final XMLReader xr = XMLReaderFactory.createXMLReader();
         xr.setContentHandler(this);
@@ -352,7 +449,7 @@ public class HftpFileSystem extends File
      */
     private ContentSummary getContentSummary(String path) throws IOException {
       final HttpURLConnection connection = openConnection(
-          "/contentSummary" + path, "ugi=" + ugi);
+          "/contentSummary" + path, "ugi=" + getUgiParameter());
       InputStream in = null;
       try {
         in = connection.getInputStream();        

Modified: hadoop/hdfs/trunk/src/java/org/apache/hadoop/hdfs/HsftpFileSystem.java
URL: http://svn.apache.org/viewvc/hadoop/hdfs/trunk/src/java/org/apache/hadoop/hdfs/HsftpFileSystem.java?rev=964266&r1=964265&r2=964266&view=diff
==============================================================================
--- hadoop/hdfs/trunk/src/java/org/apache/hadoop/hdfs/HsftpFileSystem.java (original)
+++ hadoop/hdfs/trunk/src/java/org/apache/hadoop/hdfs/HsftpFileSystem.java Thu Jul 15 00:29:31 2010
@@ -124,6 +124,7 @@ public class HsftpFileSystem extends Hft
   protected HttpURLConnection openConnection(String path, String query)
       throws IOException {
     try {
+      query = updateQuery(query);
       final URL url = new URI("https", null, nnAddr.getHostName(), nnAddr
           .getPort(), path, query, null).toURL();
       HttpsURLConnection conn = (HttpsURLConnection) url.openConnection();

Modified: hadoop/hdfs/trunk/src/java/org/apache/hadoop/hdfs/server/common/JspHelper.java
URL: http://svn.apache.org/viewvc/hadoop/hdfs/trunk/src/java/org/apache/hadoop/hdfs/server/common/JspHelper.java?rev=964266&r1=964265&r2=964266&view=diff
==============================================================================
--- hadoop/hdfs/trunk/src/java/org/apache/hadoop/hdfs/server/common/JspHelper.java (original)
+++ hadoop/hdfs/trunk/src/java/org/apache/hadoop/hdfs/server/common/JspHelper.java Thu Jul 15 00:29:31 2010
@@ -61,6 +61,7 @@ import org.apache.hadoop.util.VersionInf
 
 @InterfaceAudience.Private
 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 +
@@ -469,6 +470,7 @@ public class JspHelper {
                                             Configuration conf
                                            ) throws IOException {
     UserGroupInformation ugi = null;
+    final String RANDOM_USER = "webuser1234";
     if(UserGroupInformation.isSecurityEnabled()) {
       String user = request.getRemoteUser();
       String tokenString = request.getParameter(DELEGATION_PARAMETER_NAME);
@@ -476,6 +478,12 @@ public class JspHelper {
         Token<DelegationTokenIdentifier> token = 
           new Token<DelegationTokenIdentifier>();
         token.decodeFromUrlString(tokenString);
+        if (user == null) {
+          //this really doesn't break any security since we use the 
+          //delegation token for authentication in
+          //the back end.
+          user = RANDOM_USER;
+        }
         InetSocketAddress serviceAddr = NameNode.getAddress(conf);
         LOG.info("Setting service in token: "
             + new Text(serviceAddr.getAddress().getHostAddress() + ":"
@@ -501,7 +509,7 @@ public class JspHelper {
       if(user == null) { // not specified in request
         ugi = getDefaultWebUser(conf);
       } else {
-        ugi = UserGroupInformation.createRemoteUser(user);
+        ugi = UserGroupInformation.createRemoteUser(user.split(",")[0]);
       }
       ugi.setAuthenticationMethod(AuthenticationMethod.SIMPLE);
     }

Modified: hadoop/hdfs/trunk/src/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java
URL: http://svn.apache.org/viewvc/hadoop/hdfs/trunk/src/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java?rev=964266&r1=964265&r2=964266&view=diff
==============================================================================
--- hadoop/hdfs/trunk/src/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java (original)
+++ hadoop/hdfs/trunk/src/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java Thu Jul 15 00:29:31 2010
@@ -17,7 +17,6 @@
  */
 package org.apache.hadoop.hdfs.server.datanode;
 
-import static org.junit.Assert.assertTrue;
 
 import java.io.BufferedOutputStream;
 import java.io.DataOutputStream;
@@ -72,6 +71,7 @@ import org.apache.hadoop.hdfs.security.t
 import org.apache.hadoop.hdfs.security.token.block.ExportedBlockKeys;
 import org.apache.hadoop.hdfs.server.common.HdfsConstants;
 import org.apache.hadoop.hdfs.server.common.IncorrectVersionException;
+import org.apache.hadoop.hdfs.server.common.JspHelper;
 import org.apache.hadoop.hdfs.server.common.Storage;
 import org.apache.hadoop.hdfs.server.common.Util;
 import static org.apache.hadoop.hdfs.server.common.Util.now;
@@ -400,7 +400,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/hdfs/trunk/src/java/org/apache/hadoop/hdfs/server/namenode/ContentSummaryServlet.java
URL: http://svn.apache.org/viewvc/hadoop/hdfs/trunk/src/java/org/apache/hadoop/hdfs/server/namenode/ContentSummaryServlet.java?rev=964266&r1=964265&r2=964266&view=diff
==============================================================================
--- hadoop/hdfs/trunk/src/java/org/apache/hadoop/hdfs/server/namenode/ContentSummaryServlet.java (original)
+++ hadoop/hdfs/trunk/src/java/org/apache/hadoop/hdfs/server/namenode/ContentSummaryServlet.java Thu Jul 15 00:29:31 2010
@@ -29,6 +29,7 @@ import org.apache.hadoop.classification.
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.ContentSummary;
 import org.apache.hadoop.hdfs.protocol.ClientProtocol;
+import org.apache.hadoop.hdfs.server.common.JspHelper;
 import org.apache.hadoop.ipc.RemoteException;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.znerd.xmlenc.XMLOutputter;
@@ -43,7 +44,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/hdfs/trunk/src/java/org/apache/hadoop/hdfs/server/namenode/DelegationTokenServlet.java
URL: http://svn.apache.org/viewvc/hadoop/hdfs/trunk/src/java/org/apache/hadoop/hdfs/server/namenode/DelegationTokenServlet.java?rev=964266&r1=964265&r2=964266&view=diff
==============================================================================
--- hadoop/hdfs/trunk/src/java/org/apache/hadoop/hdfs/server/namenode/DelegationTokenServlet.java (original)
+++ hadoop/hdfs/trunk/src/java/org/apache/hadoop/hdfs/server/namenode/DelegationTokenServlet.java Thu Jul 15 00:29:31 2010
@@ -43,6 +43,7 @@ import org.apache.hadoop.security.token.
 public class DelegationTokenServlet extends DfsServlet {
   private static final Log LOG = LogFactory.getLog(DelegationTokenServlet.class);
   public static final String PATH_SPEC = "/getDelegationToken";
+  public static final String RENEWER = "renewer";
   
   @Override
   protected void doGet(final HttpServletRequest req, final HttpServletResponse resp)
@@ -60,6 +61,9 @@ public class DelegationTokenServlet exte
     LOG.info("Sending token: {" + ugi.getUserName() + "," + req.getRemoteAddr() +"}");
     final ServletContext context = getServletContext();
     final NameNode nn = (NameNode) context.getAttribute("name.node");
+    String renewer = req.getParameter(RENEWER);
+    final String renewerFinal = (renewer == null) ? 
+        req.getUserPrincipal().getName() : renewer;
     
     DataOutputStream dos = null;
     try {
@@ -70,7 +74,7 @@ public class DelegationTokenServlet exte
         public Void run() throws Exception {
           
           Token<DelegationTokenIdentifier> token = 
-            nn.getDelegationToken(new Text(req.getUserPrincipal().getName()));
+            nn.getDelegationToken(new Text(renewerFinal));
           String s = nn.rpcAddress.getAddress().getHostAddress() 
                      + ":" + nn.rpcAddress.getPort();
           token.setService(new Text(s));

Modified: hadoop/hdfs/trunk/src/java/org/apache/hadoop/hdfs/server/namenode/DfsServlet.java
URL: http://svn.apache.org/viewvc/hadoop/hdfs/trunk/src/java/org/apache/hadoop/hdfs/server/namenode/DfsServlet.java?rev=964266&r1=964265&r2=964266&view=diff
==============================================================================
--- hadoop/hdfs/trunk/src/java/org/apache/hadoop/hdfs/server/namenode/DfsServlet.java (original)
+++ hadoop/hdfs/trunk/src/java/org/apache/hadoop/hdfs/server/namenode/DfsServlet.java Thu Jul 15 00:29:31 2010
@@ -54,7 +54,7 @@ abstract class DfsServlet extends HttpSe
     ServletContext context = getServletContext();
     InetSocketAddress nnAddr = (InetSocketAddress)context.getAttribute("name.node.address");
     Configuration conf = new HdfsConfiguration(
-        (Configuration)context.getAttribute("name.conf"));
+        (Configuration)context.getAttribute(JspHelper.CURRENT_CONF));
     return DFSClient.createNamenode(nnAddr, conf);
   }
 

Modified: hadoop/hdfs/trunk/src/java/org/apache/hadoop/hdfs/server/namenode/FileChecksumServlets.java
URL: http://svn.apache.org/viewvc/hadoop/hdfs/trunk/src/java/org/apache/hadoop/hdfs/server/namenode/FileChecksumServlets.java?rev=964266&r1=964265&r2=964266&view=diff
==============================================================================
--- hadoop/hdfs/trunk/src/java/org/apache/hadoop/hdfs/server/namenode/FileChecksumServlets.java (original)
+++ hadoop/hdfs/trunk/src/java/org/apache/hadoop/hdfs/server/namenode/FileChecksumServlets.java Thu Jul 15 00:29:31 2010
@@ -36,6 +36,7 @@ import org.apache.hadoop.hdfs.DFSClient;
 import org.apache.hadoop.hdfs.protocol.ClientProtocol;
 import org.apache.hadoop.hdfs.protocol.DatanodeID;
 import org.apache.hadoop.hdfs.server.common.HdfsConstants;
+import org.apache.hadoop.hdfs.server.common.JspHelper;
 import org.apache.hadoop.hdfs.server.datanode.DataNode;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
@@ -58,7 +59,7 @@ public class FileChecksumServlets {
         ) throws ServletException, IOException {
       final ServletContext context = getServletContext();
       final Configuration conf = 
-        (Configuration) context.getAttribute("name.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.getNamesystem().getRandomDatanode();

Modified: hadoop/hdfs/trunk/src/java/org/apache/hadoop/hdfs/server/namenode/FileDataServlet.java
URL: http://svn.apache.org/viewvc/hadoop/hdfs/trunk/src/java/org/apache/hadoop/hdfs/server/namenode/FileDataServlet.java?rev=964266&r1=964265&r2=964266&view=diff
==============================================================================
--- hadoop/hdfs/trunk/src/java/org/apache/hadoop/hdfs/server/namenode/FileDataServlet.java (original)
+++ hadoop/hdfs/trunk/src/java/org/apache/hadoop/hdfs/server/namenode/FileDataServlet.java Thu Jul 15 00:29:31 2010
@@ -46,7 +46,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);
@@ -56,12 +56,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" + i.getFullName(parent), 
-            "ugi=" + ugi.getShortUserName(), null);
+            "ugi=" + ugi.getShortUserName() + dtParam, null);
   }
 
   /** Select a datanode to service this request.
@@ -90,7 +97,7 @@ public class FileDataServlet extends Dfs
   public void doGet(HttpServletRequest request, HttpServletResponse response)
       throws IOException {
     final Configuration conf = 
-      (Configuration) getServletContext().getAttribute("name.conf");
+      (Configuration) getServletContext().getAttribute(JspHelper.CURRENT_CONF);
     final UserGroupInformation ugi = getUGI(request, conf);
 
     try {
@@ -102,12 +109,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()) {
         String redirect = createUri(path, info, ugi, nnproxy,
-              request).toURL().toString();
+              request, delegationToken).toURL().toString();
         response.sendRedirect(redirect);
       } else if (info == null){
         response.sendError(400, "cat: File not found " + path);

Modified: hadoop/hdfs/trunk/src/java/org/apache/hadoop/hdfs/server/namenode/FsckServlet.java
URL: http://svn.apache.org/viewvc/hadoop/hdfs/trunk/src/java/org/apache/hadoop/hdfs/server/namenode/FsckServlet.java?rev=964266&r1=964265&r2=964266&view=diff
==============================================================================
--- hadoop/hdfs/trunk/src/java/org/apache/hadoop/hdfs/server/namenode/FsckServlet.java (original)
+++ hadoop/hdfs/trunk/src/java/org/apache/hadoop/hdfs/server/namenode/FsckServlet.java Thu Jul 15 00:29:31 2010
@@ -29,6 +29,7 @@ import javax.servlet.http.HttpServletRes
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hdfs.protocol.FSConstants.DatanodeReportType;
+import org.apache.hadoop.hdfs.server.common.JspHelper;
 import org.apache.hadoop.security.UserGroupInformation;
 
 /**
@@ -46,7 +47,7 @@ public class FsckServlet extends DfsServ
     final Map<String,String[]> pmap = request.getParameterMap();
     final PrintWriter out = response.getWriter();
     final Configuration conf = 
-      (Configuration) getServletContext().getAttribute("name.conf");
+      (Configuration) getServletContext().getAttribute(JspHelper.CURRENT_CONF);
 
     final UserGroupInformation ugi = getUGI(request, conf);
     try {

Modified: hadoop/hdfs/trunk/src/java/org/apache/hadoop/hdfs/server/namenode/GetImageServlet.java
URL: http://svn.apache.org/viewvc/hadoop/hdfs/trunk/src/java/org/apache/hadoop/hdfs/server/namenode/GetImageServlet.java?rev=964266&r1=964265&r2=964266&view=diff
==============================================================================
--- hadoop/hdfs/trunk/src/java/org/apache/hadoop/hdfs/server/namenode/GetImageServlet.java (original)
+++ hadoop/hdfs/trunk/src/java/org/apache/hadoop/hdfs/server/namenode/GetImageServlet.java Thu Jul 15 00:29:31 2010
@@ -37,6 +37,7 @@ import org.apache.commons.logging.LogFac
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
+import org.apache.hadoop.hdfs.server.common.JspHelper;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.util.StringUtils;
 
@@ -60,7 +61,8 @@ 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)) {

Modified: hadoop/hdfs/trunk/src/java/org/apache/hadoop/hdfs/server/namenode/ListPathsServlet.java
URL: http://svn.apache.org/viewvc/hadoop/hdfs/trunk/src/java/org/apache/hadoop/hdfs/server/namenode/ListPathsServlet.java?rev=964266&r1=964265&r2=964266&view=diff
==============================================================================
--- hadoop/hdfs/trunk/src/java/org/apache/hadoop/hdfs/server/namenode/ListPathsServlet.java (original)
+++ hadoop/hdfs/trunk/src/java/org/apache/hadoop/hdfs/server/namenode/ListPathsServlet.java Thu Jul 15 00:29:31 2010
@@ -24,6 +24,7 @@ import org.apache.hadoop.hdfs.HftpFileSy
 import org.apache.hadoop.hdfs.protocol.ClientProtocol;
 import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
 import org.apache.hadoop.hdfs.protocol.DirectoryListing;
+import org.apache.hadoop.hdfs.server.common.JspHelper;
 import org.apache.hadoop.ipc.RemoteException;
 import org.apache.hadoop.util.VersionInfo;
 
@@ -138,7 +139,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/hdfs/trunk/src/java/org/apache/hadoop/hdfs/server/namenode/NameNode.java
URL: http://svn.apache.org/viewvc/hadoop/hdfs/trunk/src/java/org/apache/hadoop/hdfs/server/namenode/NameNode.java?rev=964266&r1=964265&r2=964266&view=diff
==============================================================================
--- hadoop/hdfs/trunk/src/java/org/apache/hadoop/hdfs/server/namenode/NameNode.java (original)
+++ hadoop/hdfs/trunk/src/java/org/apache/hadoop/hdfs/server/namenode/NameNode.java Thu Jul 15 00:29:31 2010
@@ -61,6 +61,7 @@ import org.apache.hadoop.hdfs.protocol.U
 import org.apache.hadoop.hdfs.security.token.block.ExportedBlockKeys;
 import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier;
 import org.apache.hadoop.hdfs.server.common.IncorrectVersionException;
+import org.apache.hadoop.hdfs.server.common.JspHelper;
 import org.apache.hadoop.hdfs.server.common.UpgradeStatusReport;
 import org.apache.hadoop.hdfs.server.common.HdfsConstants.NamenodeRole;
 import org.apache.hadoop.hdfs.server.common.HdfsConstants.StartupOption;
@@ -464,7 +465,7 @@ public class NameNode implements Namenod
           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);
@@ -473,13 +474,13 @@ public class NameNode implements Namenod
           httpServer.addInternalServlet("getimage", "/getimage",
               GetImageServlet.class, true);
           httpServer.addInternalServlet("listPaths", "/listPaths/*",
-              ListPathsServlet.class, true);
+              ListPathsServlet.class, false);
           httpServer.addInternalServlet("data", "/data/*",
-              FileDataServlet.class, true);
+              FileDataServlet.class, false);
           httpServer.addInternalServlet("checksum", "/fileChecksum/*",
-              FileChecksumServlets.RedirectServlet.class, true);
+              FileChecksumServlets.RedirectServlet.class, false);
           httpServer.addInternalServlet("contentSummary", "/contentSummary/*",
-              ContentSummaryServlet.class, true);
+              ContentSummaryServlet.class, false);
           httpServer.start();
 
           // The web-server port can be ephemeral... ensure we have the correct

Modified: hadoop/hdfs/trunk/src/java/org/apache/hadoop/hdfs/server/namenode/SecondaryNameNode.java
URL: http://svn.apache.org/viewvc/hadoop/hdfs/trunk/src/java/org/apache/hadoop/hdfs/server/namenode/SecondaryNameNode.java?rev=964266&r1=964265&r2=964266&view=diff
==============================================================================
--- hadoop/hdfs/trunk/src/java/org/apache/hadoop/hdfs/server/namenode/SecondaryNameNode.java (original)
+++ hadoop/hdfs/trunk/src/java/org/apache/hadoop/hdfs/server/namenode/SecondaryNameNode.java Thu Jul 15 00:29:31 2010
@@ -36,6 +36,7 @@ import org.apache.hadoop.hdfs.DFSUtil.Er
 import org.apache.hadoop.hdfs.protocol.FSConstants;
 import org.apache.hadoop.hdfs.server.common.HdfsConstants;
 import org.apache.hadoop.hdfs.server.common.InconsistentFSStateException;
+import org.apache.hadoop.hdfs.server.common.JspHelper;
 import org.apache.hadoop.hdfs.server.namenode.FSImage.NameNodeDirType;
 import org.apache.hadoop.hdfs.server.namenode.FSImage.NameNodeFile;
 import org.apache.hadoop.hdfs.server.protocol.NamenodeProtocol;
@@ -190,7 +191,7 @@ public class SecondaryNameNode implement
           
           infoServer.setAttribute("secondary.name.node", this);
           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/hdfs/trunk/src/java/org/apache/hadoop/hdfs/server/namenode/StreamFile.java
URL: http://svn.apache.org/viewvc/hadoop/hdfs/trunk/src/java/org/apache/hadoop/hdfs/server/namenode/StreamFile.java?rev=964266&r1=964265&r2=964266&view=diff
==============================================================================
--- hadoop/hdfs/trunk/src/java/org/apache/hadoop/hdfs/server/namenode/StreamFile.java (original)
+++ hadoop/hdfs/trunk/src/java/org/apache/hadoop/hdfs/server/namenode/StreamFile.java Thu Jul 15 00:29:31 2010
@@ -54,7 +54,7 @@ public class StreamFile extends DfsServl
   protected DFSClient getDFSClient(HttpServletRequest request)
       throws IOException, InterruptedException {
     final Configuration conf =
-      (Configuration) getServletContext().getAttribute("datanode.conf");
+      (Configuration) getServletContext().getAttribute(JspHelper.CURRENT_CONF);
     
     UserGroupInformation ugi = getUGI(request, conf);
     DFSClient client = ugi.doAs(new PrivilegedExceptionAction<DFSClient>() {

Modified: hadoop/hdfs/trunk/src/java/org/apache/hadoop/hdfs/tools/DelegationTokenFetcher.java
URL: http://svn.apache.org/viewvc/hadoop/hdfs/trunk/src/java/org/apache/hadoop/hdfs/tools/DelegationTokenFetcher.java?rev=964266&r1=964265&r2=964266&view=diff
==============================================================================
--- hadoop/hdfs/trunk/src/java/org/apache/hadoop/hdfs/tools/DelegationTokenFetcher.java (original)
+++ hadoop/hdfs/trunk/src/java/org/apache/hadoop/hdfs/tools/DelegationTokenFetcher.java Thu Jul 15 00:29:31 2010
@@ -68,7 +68,7 @@ public class DelegationTokenFetcher {
       public Object run() throws Exception {
         
         if(args.length == 3 && "--webservice".equals(args[0])) {
-          getDTfromRemote(args[1], args[2]);
+          getDTfromRemoteIntoFile(args[1], args[2]);
           return null;
         }
         // avoid annoying mistake
@@ -135,43 +135,53 @@ public class DelegationTokenFetcher {
     ts.write(out);
   }
   
+
   /**
    * Utility method to obtain a delegation token over http
-   * @param nnHttpAddr Namenode http addr, such as http://namenode:50070
-   * @param filename Name of file to store token in
+   * @param nnAddr Namenode http addr, such as http://namenode:50070
    */
-   static private void getDTfromRemote(String nnAddr, String filename) 
-   throws IOException {
-     // Enable Kerberos sockets
+  static public Credentials getDTfromRemote(String nnAddr, String renewer) 
+  throws IOException {
+    // Enable Kerberos sockets
     System.setProperty("https.cipherSuites", "TLS_KRB5_WITH_3DES_EDE_CBC_SHA");
-    String ugiPostfix = "";
-    DataOutputStream file = null;
     DataInputStream dis = null;
-    
-    if(nnAddr.startsWith("http:"))
-      ugiPostfix = "?ugi=" + UserGroupInformation.getCurrentUser().getShortUserName();
-    
+
     try {
-      System.out.println("Retrieving token from: " + 
-          nnAddr + DelegationTokenServlet.PATH_SPEC + ugiPostfix);
-      URL remoteURL = new URL(nnAddr + DelegationTokenServlet.PATH_SPEC + ugiPostfix);
-      SecurityUtil.fetchServiceTicket(remoteURL);
+      StringBuffer url = new StringBuffer();
+      if (renewer != null) {
+        url.append(nnAddr).append(DelegationTokenServlet.PATH_SPEC).append("?").
+        append(DelegationTokenServlet.RENEWER).append("=").append(renewer);
+      } else {
+        url.append(nnAddr).append(DelegationTokenServlet.PATH_SPEC);
+      }
+      System.out.println("Retrieving token from: " + url);
+      URL remoteURL = new URL(url.toString());
       URLConnection connection = remoteURL.openConnection();
-      
+
       InputStream in = connection.getInputStream();
       Credentials ts = new Credentials();
       dis = new DataInputStream(in);
       ts.readFields(dis);
-      file = new DataOutputStream(new FileOutputStream(filename));
-      ts.write(file);
-      file.flush();
-      System.out.println("Successfully wrote token of " + file.size() 
-          + " bytes  to " + filename);
+      return ts;
     } catch (Exception e) {
       throw new IOException("Unable to obtain remote token", e);
     } finally {
       if(dis != null) dis.close();
-      if(file != null) file.close();
     }
   }
+
+  /**
+   * Utility method to obtain a delegation token over http
+   * @param nnHttpAddr Namenode http addr, such as http://namenode:50070
+   * @param filename Name of file to store token in
+   */
+  static private void getDTfromRemoteIntoFile(String nnAddr, String filename) 
+  throws IOException {
+    Credentials ts = getDTfromRemote(nnAddr, null); 
+    DataOutputStream file = new DataOutputStream(new FileOutputStream(filename));
+    ts.write(file);
+    file.flush();
+    System.out.println("Successfully wrote token of " + file.size() 
+        + " bytes  to " + filename);
+  }
 }

Modified: hadoop/hdfs/trunk/src/webapps/datanode/browseBlock.jsp
URL: http://svn.apache.org/viewvc/hadoop/hdfs/trunk/src/webapps/datanode/browseBlock.jsp?rev=964266&r1=964265&r2=964266&view=diff
==============================================================================
--- hadoop/hdfs/trunk/src/webapps/datanode/browseBlock.jsp (original)
+++ hadoop/hdfs/trunk/src/webapps/datanode/browseBlock.jsp Thu Jul 15 00:29:31 2010
@@ -35,7 +35,7 @@
 <body onload="document.goto.dir.focus()">
 <% 
   Configuration conf = 
-     (Configuration) application.getAttribute("datanode.conf");
+     (Configuration) application.getAttribute(JspHelper.CURRENT_CONF);
   DatanodeJspHelper.generateFileChunks(out, request, conf); 
 %>
 <hr>

Modified: hadoop/hdfs/trunk/src/webapps/datanode/browseDirectory.jsp
URL: http://svn.apache.org/viewvc/hadoop/hdfs/trunk/src/webapps/datanode/browseDirectory.jsp?rev=964266&r1=964265&r2=964266&view=diff
==============================================================================
--- hadoop/hdfs/trunk/src/webapps/datanode/browseDirectory.jsp (original)
+++ hadoop/hdfs/trunk/src/webapps/datanode/browseDirectory.jsp Thu Jul 15 00:29:31 2010
@@ -47,7 +47,7 @@ body 
 <% 
   try {
     Configuration conf = 
-      (Configuration) application.getAttribute("datanode.conf");
+      (Configuration) application.getAttribute(JspHelper.CURRENT_CONF);
     DatanodeJspHelper.generateDirectoryStructure(out,request,response, conf);
   }
   catch(IOException ioe) {

Modified: hadoop/hdfs/trunk/src/webapps/datanode/tail.jsp
URL: http://svn.apache.org/viewvc/hadoop/hdfs/trunk/src/webapps/datanode/tail.jsp?rev=964266&r1=964265&r2=964266&view=diff
==============================================================================
--- hadoop/hdfs/trunk/src/webapps/datanode/tail.jsp (original)
+++ hadoop/hdfs/trunk/src/webapps/datanode/tail.jsp Thu Jul 15 00:29:31 2010
@@ -35,7 +35,7 @@
 <form action="/tail.jsp" method="GET">
 <% 
    Configuration conf = 
-     (Configuration) application.getAttribute("datanode.conf");
+     (Configuration) application.getAttribute(JspHelper.CURRENT_CONF);
    DatanodeJspHelper.generateFileChunksForTail(out,request, conf); 
 %>
 </form>

Modified: hadoop/hdfs/trunk/src/webapps/hdfs/nn_browsedfscontent.jsp
URL: http://svn.apache.org/viewvc/hadoop/hdfs/trunk/src/webapps/hdfs/nn_browsedfscontent.jsp?rev=964266&r1=964265&r2=964266&view=diff
==============================================================================
--- hadoop/hdfs/trunk/src/webapps/hdfs/nn_browsedfscontent.jsp (original)
+++ hadoop/hdfs/trunk/src/webapps/hdfs/nn_browsedfscontent.jsp Thu Jul 15 00:29:31 2010
@@ -21,6 +21,7 @@
   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
@@ -34,7 +35,7 @@
 <% 
   NameNode nn = (NameNode)application.getAttribute("name.node");
   Configuration conf = 
-     (Configuration) application.getAttribute("name.conf");
+     (Configuration) application.getAttribute(JspHelper.CURRENT_CONF);
   NamenodeJspHelper.redirectToRandomDataNode(nn, request, response, conf); 
 %>
 <hr>