浏览代码

HDFS-1296. using delegation token over hftp for long running clients (spawn from hdfs-1007). borya via jghoman

git-svn-id: https://svn.apache.org/repos/asf/hadoop/hdfs/trunk@982004 13f79535-47bb-0310-9956-ffa450edef68
Jakob Homan 14 年之前
父节点
当前提交
72c0488fd3

+ 3 - 0
CHANGES.txt

@@ -176,6 +176,9 @@ Trunk (unreleased changes)
 
     HDFS-912. sed in build.xml on Solaris fails. (Allen Wittenauer via jghoman)
 
+    HDFS-1296. using delegation token over hftp for long running 
+    clients (boryas)
+
 Release 0.21.0 - Unreleased
 
   INCOMPATIBLE CHANGES

+ 31 - 5
src/java/org/apache/hadoop/hdfs/DFSClient.java

@@ -23,6 +23,7 @@ import static org.apache.hadoop.hdfs.protocol.DataTransferProtocol.Status.ERROR_
 import static org.apache.hadoop.hdfs.protocol.DataTransferProtocol.Status.SUCCESS;
 
 import java.io.BufferedOutputStream;
+import java.io.ByteArrayInputStream;
 import java.io.DataInputStream;
 import java.io.DataOutputStream;
 import java.io.FileNotFoundException;
@@ -56,9 +57,9 @@ import org.apache.hadoop.fs.FsServerDefaults;
 import org.apache.hadoop.fs.FsStatus;
 import org.apache.hadoop.fs.InvalidPathException;
 import org.apache.hadoop.fs.MD5MD5CRC32FileChecksum;
+import org.apache.hadoop.fs.Options;
 import org.apache.hadoop.fs.ParentNotDirectoryException;
 import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.fs.Options;
 import org.apache.hadoop.fs.UnresolvedLinkException;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.hdfs.protocol.AlreadyBeingCreatedException;
@@ -76,6 +77,7 @@ import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
 import org.apache.hadoop.hdfs.protocol.NSQuotaExceededException;
 import org.apache.hadoop.hdfs.protocol.UnresolvedPathException;
+import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier;
 import org.apache.hadoop.hdfs.server.common.HdfsConstants;
 import org.apache.hadoop.hdfs.server.common.UpgradeStatusReport;
 import org.apache.hadoop.hdfs.server.datanode.DataNode;
@@ -96,13 +98,11 @@ import org.apache.hadoop.net.NetUtils;
 import org.apache.hadoop.net.NodeBase;
 import org.apache.hadoop.security.AccessControlException;
 import org.apache.hadoop.security.UserGroupInformation;
-import org.apache.hadoop.security.token.Token;
 import org.apache.hadoop.security.token.SecretManager.InvalidToken;
-import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier;
+import org.apache.hadoop.security.token.Token;
 import org.apache.hadoop.util.Daemon;
 import org.apache.hadoop.util.Progressable;
 import org.apache.hadoop.util.StringUtils;
-import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier;
 
 /********************************************************
  * DFSClient can connect to a Hadoop Filesystem and 
@@ -360,12 +360,36 @@ public class DFSClient implements FSConstants, java.io.Closeable {
     return serverDefaults;
   }
 
+  /**
+   *  A test method for printing out tokens 
+   *  @param token
+   *  @return Stringify version of the token
+   */
+  public static String stringifyToken(Token<DelegationTokenIdentifier> token)
+  throws IOException {
+    DelegationTokenIdentifier ident = new DelegationTokenIdentifier();
+    ByteArrayInputStream buf = new ByteArrayInputStream(token.getIdentifier());
+    DataInputStream in = new DataInputStream(buf);  
+    ident.readFields(in);
+    String str = ident.getKind() + " token " + ident.getSequenceNumber() + 
+    " for " + ident.getUser().getShortUserName();
+    if (token.getService().getLength() > 0) {
+      return (str + " on " + token.getService());
+    } else {
+      return str;
+    }
+  }
+
+  
   /**
    * @see ClientProtocol#getDelegationToken(Text)
    */
   public Token<DelegationTokenIdentifier> getDelegationToken(Text renewer)
       throws IOException {
-    return namenode.getDelegationToken(renewer);
+    Token<DelegationTokenIdentifier> result =
+      namenode.getDelegationToken(renewer);
+    LOG.info("Created " + stringifyToken(result));
+    return result;
   }
 
   /**
@@ -373,6 +397,7 @@ public class DFSClient implements FSConstants, java.io.Closeable {
    */
   public long renewDelegationToken(Token<DelegationTokenIdentifier> token)
       throws InvalidToken, IOException {
+    LOG.info("Renewing " + stringifyToken(token));
     try {
       return namenode.renewDelegationToken(token);
     } catch (RemoteException re) {
@@ -386,6 +411,7 @@ public class DFSClient implements FSConstants, java.io.Closeable {
    */
   public void cancelDelegationToken(Token<DelegationTokenIdentifier> token)
       throws InvalidToken, IOException {
+    LOG.info("Cancelling " + stringifyToken(token));
     try {
       namenode.cancelDelegationToken(token);
     } catch (RemoteException re) {

+ 43 - 13
src/java/org/apache/hadoop/hdfs/DistributedFileSystem.java

@@ -18,34 +18,46 @@
 
 package org.apache.hadoop.hdfs;
 
-import java.io.*;
-import java.net.*;
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.net.InetSocketAddress;
+import java.net.URI;
 import java.util.ArrayList;
 import java.util.EnumSet;
 
-import org.apache.hadoop.fs.permission.FsPermission;
-import org.apache.hadoop.fs.*;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
-import org.apache.hadoop.hdfs.protocol.FSConstants;
+import org.apache.hadoop.fs.BlockLocation;
+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.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.FsServerDefaults;
+import org.apache.hadoop.fs.FsStatus;
+import org.apache.hadoop.fs.MD5MD5CRC32FileChecksum;
+import org.apache.hadoop.fs.Options;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.hdfs.DFSClient.DFSDataInputStream;
 import org.apache.hadoop.hdfs.protocol.Block;
-import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
-import org.apache.hadoop.hdfs.protocol.LocatedBlock;
+import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.DirectoryListing;
+import org.apache.hadoop.hdfs.protocol.FSConstants;
 import org.apache.hadoop.hdfs.protocol.FSConstants.DatanodeReportType;
 import org.apache.hadoop.hdfs.protocol.FSConstants.UpgradeAction;
+import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
+import org.apache.hadoop.hdfs.protocol.LocatedBlock;
+import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier;
 import org.apache.hadoop.hdfs.server.common.UpgradeStatusReport;
 import org.apache.hadoop.hdfs.server.namenode.NameNode;
-import org.apache.hadoop.hdfs.DFSClient.DFSDataInputStream;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.security.AccessControlException;
-import org.apache.hadoop.security.token.Token;
 import org.apache.hadoop.security.token.SecretManager.InvalidToken;
-import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier;
+import org.apache.hadoop.security.token.Token;
 import org.apache.hadoop.util.Progressable;
-import org.apache.hadoop.fs.Options;
 
 
 /****************************************************************
@@ -677,7 +689,23 @@ public class DistributedFileSystem extends FileSystem {
     dfs.setTimes(getPathName(p), mtime, atime);
   }
   
-  /** 
+
+  @Override
+  protected int getDefaultPort() {
+    return NameNode.DEFAULT_PORT;
+  }
+
+  @Override
+  public 
+  Token<DelegationTokenIdentifier> getDelegationToken(String renewer
+  ) throws IOException {
+    Token<DelegationTokenIdentifier> result =
+      dfs.getDelegationToken(renewer == null ? null : new Text(renewer));
+    result.setService(new Text(getCanonicalServiceName()));
+    return result;
+  }
+
+  /*
    * Delegation Token Operations
    * These are DFS only operations.
    */
@@ -688,7 +716,9 @@ public class DistributedFileSystem extends FileSystem {
    * @param renewer Name of the designated renewer for the token
    * @return Token<DelegationTokenIdentifier>
    * @throws IOException
+   * @deprecated use {@link #getDelegationToken(String)}
    */
+  @Deprecated
   public Token<DelegationTokenIdentifier> getDelegationToken(Text renewer)
       throws IOException {
     return dfs.getDelegationToken(renewer);

+ 256 - 60
src/java/org/apache/hadoop/hdfs/HftpFileSystem.java

@@ -21,6 +21,7 @@ package org.apache.hadoop.hdfs;
 import java.io.FileNotFoundException;
 import java.io.IOException;
 import java.io.InputStream;
+import java.lang.ref.WeakReference;
 import java.net.HttpURLConnection;
 import java.net.InetSocketAddress;
 import java.net.URI;
@@ -30,9 +31,11 @@ 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 java.util.concurrent.DelayQueue;
+import java.util.concurrent.Delayed;
+import java.util.concurrent.TimeUnit;
 
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
@@ -46,6 +49,7 @@ 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.security.token.delegation.DelegationTokenIdentifier;
 import org.apache.hadoop.hdfs.server.common.JspHelper;
 import org.apache.hadoop.hdfs.server.namenode.NameNode;
 import org.apache.hadoop.hdfs.tools.DelegationTokenFetcher;
@@ -80,16 +84,18 @@ public class HftpFileSystem extends FileSystem {
     HttpURLConnection.setFollowRedirects(true);
   }
 
+  private static final int DEFAULT_PORT = 50470;
+  private String nnHttpUrl;
+  private URI hdfsURI;
   protected InetSocketAddress nnAddr;
   protected UserGroupInformation ugi; 
   protected final Random ran = new Random();
 
   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";
+  private Token<DelegationTokenIdentifier> delegationToken;
   public static final String HFTP_SERVICE_NAME_KEY = "hdfs.service.host_";
-
+  
   public static final SimpleDateFormat getDateFormat() {
     final SimpleDateFormat df = new SimpleDateFormat(HFTP_DATE_FORMAT);
     df.setTimeZone(TimeZone.getTimeZone(HFTP_TIMEZONE));
@@ -98,11 +104,33 @@ public class HftpFileSystem extends FileSystem {
 
   protected static final ThreadLocal<SimpleDateFormat> df =
     new ThreadLocal<SimpleDateFormat>() {
-      protected SimpleDateFormat initialValue() {
-        return getDateFormat();
-      }
-    };
+    protected SimpleDateFormat initialValue() {
+      return getDateFormat();
+    }
+  };
+
+  private static RenewerThread renewer = new RenewerThread();
+  static {
+    renewer.start();
+  }
+
+  @Override
+  protected int getDefaultPort() {
+    return DEFAULT_PORT;
+  }
+
+  @Override
+  public String getCanonicalServiceName() {
+    return SecurityUtil.buildDTServiceName(hdfsURI, getDefaultPort());
+  }
+  
+  private String buildUri(String schema, String host, int port) {
+    StringBuilder sb = new StringBuilder(schema);
+    return sb.append(host).append(":").append(port).toString();
+  }
+
 
+  @SuppressWarnings("unchecked")
   @Override
   public void initialize(final URI name, final Configuration conf)
   throws IOException {
@@ -110,66 +138,79 @@ public class HftpFileSystem extends FileSystem {
     setConf(conf);
     this.ugi = UserGroupInformation.getCurrentUser(); 
     nnAddr = NetUtils.createSocketAddr(name.toString());
-    
-    if (UserGroupInformation.isSecurityEnabled()) {
-      // configuration has the actual service name for this url. Build the key 
-      // and get it.
-      final String key = HftpFileSystem.HFTP_SERVICE_NAME_KEY +
-        SecurityUtil.buildDTServiceName(name, NameNode.DEFAULT_PORT);
+   
+    nnHttpUrl = buildUri("https://", NetUtils.normalizeHostName(name.getHost()), 
+        conf.getInt("dfs.https.port", DEFAULT_PORT));
+
+    // if one uses RPC port different from the Default one,  
+    // one should specify what is the setvice name for this delegation token
+    // otherwise it is hostname:RPC_PORT
+    String key = HftpFileSystem.HFTP_SERVICE_NAME_KEY+
+    SecurityUtil.buildDTServiceName(name, DEFAULT_PORT);
+    LOG.debug("Trying to find DT for " + name + " using key=" + key + 
+        "; conf=" + conf.get(key, ""));
+    String nnServiceName = conf.get(key);
+    int nnPort = NameNode.DEFAULT_PORT;
+    if (nnServiceName != null) { // get the real port
+      nnPort = NetUtils.createSocketAddr(nnServiceName, 
+          NameNode.DEFAULT_PORT).getPort();
+    }
 
-      LOG.debug("Trying to find DT for " + name + " using key=" + key + 
-          "; conf=" + conf.get(key, ""));
-      Text nnServiceNameText = new Text(conf.get(key, ""));
+    try {
+      hdfsURI = new URI(buildUri("hdfs://", nnAddr.getHostName(), nnPort));
+    } catch (URISyntaxException ue) {
+      throw new IOException("bad uri for hdfs", ue);
+    }
 
-      Collection<Token<? extends TokenIdentifier>> tokens = ugi.getTokens();
+    if (UserGroupInformation.isSecurityEnabled()) {
       //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)) {
+      String canonicalName = getCanonicalServiceName();
+      for (Token<? extends TokenIdentifier> t : ugi.getTokens()) {
+        if (DelegationTokenIdentifier.HDFS_DELEGATION_KIND.equals(t.getKind()) &&
+            t.getService().toString().equals(canonicalName)) {
           LOG.debug("Found existing DT for " + name);
-          delegationToken = t;
-          return;
+          delegationToken = (Token<DelegationTokenIdentifier>) t;
+          break;
         }
       }
       //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);
+      if (delegationToken == null) {
+        delegationToken = 
+          (Token<DelegationTokenIdentifier>) getDelegationToken(null);
+        renewer.addTokenToRenew(this);
       }
     }
   }
   
-  
-  public Token<? extends TokenIdentifier> getDelegationToken() {
-    return delegationToken;
+
+  @Override
+  public Token<?> getDelegationToken(final String renewer) throws IOException {
+    try {
+      return ugi.doAs(new PrivilegedExceptionAction<Token<?>>() {
+        public Token<?> run() throws IOException {
+          Credentials c;
+          try {
+            c = DelegationTokenFetcher.getDTfromRemote(nnHttpUrl, renewer);
+          } catch (Exception e) {
+            LOG.info("Couldn't get a delegation token from " + nnHttpUrl + 
+            " using https.");
+            LOG.debug("error was ", e);
+            //Maybe the server is in unsecure mode (that's bad but okay)
+            return null;
+          }
+          for (Token<? extends TokenIdentifier> t : c.getAllTokens()) {
+            LOG.debug("Got dt for " + getUri() + ";t.service="
+                +t.getService());
+            t.setService(new Text(getCanonicalServiceName()));
+            return t;
+          }
+          return null;
+        }
+      });
+    } catch (InterruptedException e) {
+      throw new RuntimeException(e);
+    }
   }
 
   @Override
@@ -238,10 +279,12 @@ public class HftpFileSystem extends FileSystem {
   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
+      synchronized (this) {
+        if (delegationToken != null) {
+          tokenString = delegationToken.encodeToUrlString();
+          return (query + JspHelper.SET_DELEGATION + tokenString);
+        } // else we are talking to an insecure cluster
+      }
     }
     return query;
   }
@@ -520,4 +563,157 @@ public class HftpFileSystem extends FileSystem {
     final ContentSummary cs = new ContentSummaryParser().getContentSummary(s);
     return cs != null? cs: super.getContentSummary(f);
   }
+
+
+  /**
+   * An action that will renew and replace the hftp file system's delegation 
+   * tokens automatically.
+   */
+  private static class RenewAction implements Delayed {
+    // when should the renew happen
+    private long timestamp;
+    // a weak reference to the file system so that it can be garbage collected
+    private final WeakReference<HftpFileSystem> weakFs;
+
+    RenewAction(long timestamp, HftpFileSystem fs) {
+      this.timestamp = timestamp;
+      this.weakFs = new WeakReference<HftpFileSystem>(fs);
+    }
+
+    /**
+     * Get the delay until this event should happen.
+     */
+    @Override
+    public long getDelay(TimeUnit unit) {
+      long millisLeft = timestamp - System.currentTimeMillis();
+      return unit.convert(millisLeft, TimeUnit.MILLISECONDS);
+    }
+
+    /**
+     * Compare two events in the same queue.
+     */
+    @Override
+    public int compareTo(Delayed o) {
+      if (o.getClass() != RenewAction.class) {
+        throw new IllegalArgumentException("Illegal comparision to non-RenewAction");
+      }
+      RenewAction other = (RenewAction) o;
+      return timestamp < other.timestamp ? -1 :
+        (timestamp == other.timestamp ? 0 : 1);
+    }
+    
+    @Override
+    public int hashCode() {
+      assert false : "hashCode not designed";
+    return 33;  
+    }
+    /**
+     * equals
+     */
+    @Override
+    public boolean equals(Object o) {
+      if(!( o instanceof Delayed))
+        return false;
+      
+      return compareTo((Delayed) o) == 0;
+    }
+
+    /**
+     * Set a new time for the renewal. Can only be called when the action
+     * is not in the queue.
+     * @param newTime the new time
+     */
+    public void setNewTime(long newTime) {
+      timestamp = newTime;
+    }
+
+    /**
+     * Renew or replace the delegation token for this file system.
+     * @return
+     * @throws IOException
+     */
+    @SuppressWarnings("unchecked")
+    public boolean renew() throws IOException, InterruptedException {
+      final HftpFileSystem fs = weakFs.get();
+      if (fs != null) {
+        synchronized (fs) {
+          fs.ugi.doAs(new PrivilegedExceptionAction<Void>() {
+
+            @Override
+            public Void run() throws Exception {
+              try {
+                DelegationTokenFetcher.renewDelegationToken(fs.nnHttpUrl, 
+                    fs.delegationToken);
+              } catch (IOException ie) {
+                try {
+                  fs.delegationToken = 
+                    (Token<DelegationTokenIdentifier>) fs.getDelegationToken(null);
+                } catch (IOException ie2) {
+                  throw new IOException("Can't renew or get new delegation token ", 
+                      ie);
+                }
+              }
+              return null;
+            } 
+          });
+        }
+      }
+      return fs != null;
+    }
+
+    public String toString() {
+      StringBuilder result = new StringBuilder();
+      HftpFileSystem fs = weakFs.get();
+      if (fs == null) {
+        return "evaporated token renew";
+      }
+      synchronized (fs) {
+        result.append(fs.delegationToken);
+      }
+      result.append(" renew in ");
+      result.append(getDelay(TimeUnit.SECONDS));
+      result.append(" secs");
+      return result.toString();
+    }
+  }
+
+  /**
+   * A daemon thread that waits for the next file system to renew.
+   */
+  private static class RenewerThread extends Thread {
+    private DelayQueue<RenewAction> queue = new DelayQueue<RenewAction>();
+    // wait for 95% of a day between renewals
+    private static final int RENEW_CYCLE = (int) (0.95 * 24 * 60 * 60 * 1000);
+
+    public RenewerThread() {
+      super("HFTP Delegation Token Renewer");
+      setDaemon(true);
+    }
+
+    public void addTokenToRenew(HftpFileSystem fs) {
+      queue.add(new RenewAction(RENEW_CYCLE + System.currentTimeMillis(),fs));
+    }
+
+    public void run() {
+      RenewAction action = null;
+      while (true) {
+        try {
+          action = queue.take();
+          if (action.renew()) {
+            action.setNewTime(RENEW_CYCLE + System.currentTimeMillis());
+            queue.add(action);
+          }
+          action = null;
+        } catch (InterruptedException ie) {
+          return;
+        } catch (Exception ie) {
+          if (action != null) {
+            LOG.warn("Failure to renew token " + action, ie);
+          } else {
+            LOG.warn("Failure in renew queue", ie);
+          }
+        }
+      }
+    }
+  }
 }

+ 83 - 0
src/java/org/apache/hadoop/hdfs/server/namenode/CancelDelegationTokenServlet.java

@@ -0,0 +1,83 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with this
+ * work for additional information regarding copyright ownership. The ASF
+ * licenses this file to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ * 
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
+ * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
+ * License for the specific language governing permissions and limitations under
+ * the License.
+ */
+package org.apache.hadoop.hdfs.server.namenode;
+
+import java.io.IOException;
+import java.security.PrivilegedExceptionAction;
+
+import javax.servlet.ServletContext;
+import javax.servlet.ServletException;
+import javax.servlet.http.HttpServletRequest;
+import javax.servlet.http.HttpServletResponse;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier;
+import org.apache.hadoop.hdfs.server.common.JspHelper;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.security.token.Token;
+
+/**
+ * Cancel delegation tokens over http for use in hftp.
+ */
+@SuppressWarnings("serial")
+public class CancelDelegationTokenServlet extends DfsServlet {
+  private static final Log LOG = LogFactory.getLog(CancelDelegationTokenServlet.class);
+  public static final String PATH_SPEC = "/cancelDelegationToken";
+  public static final String TOKEN = "token";
+  
+  @Override
+  protected void doGet(final HttpServletRequest req, final HttpServletResponse resp)
+      throws ServletException, IOException {
+    final UserGroupInformation ugi;
+    final ServletContext context = getServletContext();
+    final Configuration conf = 
+      (Configuration) context.getAttribute(JspHelper.CURRENT_CONF);
+    try {
+      ugi = getUGI(req, conf);
+    } catch(IOException ioe) {
+      LOG.info("Request for token received with no authentication from "
+          + req.getRemoteAddr(), ioe);
+      resp.sendError(HttpServletResponse.SC_FORBIDDEN, 
+          "Unable to identify or authenticate user");
+      return;
+    }
+    final NameNode nn = (NameNode) context.getAttribute("name.node");
+    String tokenString = req.getParameter(TOKEN);
+    if (tokenString == null) {
+      resp.sendError(HttpServletResponse.SC_MULTIPLE_CHOICES,
+                     "Token to renew not specified");
+    }
+    final Token<DelegationTokenIdentifier> token = 
+      new Token<DelegationTokenIdentifier>();
+    token.decodeFromUrlString(tokenString);
+    
+    try {
+      ugi.doAs(new PrivilegedExceptionAction<Void>() {
+        public Void run() throws Exception {
+          nn.cancelDelegationToken(token);
+          return null;
+        }
+      });
+    } catch(Exception e) {
+      LOG.info("Exception while cancelling token. Re-throwing. ", e);
+      resp.sendError(HttpServletResponse.SC_INTERNAL_SERVER_ERROR,
+                     e.getMessage());
+    }
+  }
+}

+ 13 - 10
src/java/org/apache/hadoop/hdfs/server/namenode/DelegationTokenServlet.java → src/java/org/apache/hadoop/hdfs/server/namenode/GetDelegationTokenServlet.java

@@ -27,9 +27,9 @@ import javax.servlet.http.HttpServletResponse;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier;
+import org.apache.hadoop.hdfs.server.common.JspHelper;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.security.Credentials;
 import org.apache.hadoop.security.UserGroupInformation;
@@ -38,10 +38,9 @@ import org.apache.hadoop.security.token.Token;
 /**
  * Serve delegation tokens over http for use in hftp.
  */
-@InterfaceAudience.Private
 @SuppressWarnings("serial")
-public class DelegationTokenServlet extends DfsServlet {
-  private static final Log LOG = LogFactory.getLog(DelegationTokenServlet.class);
+public class GetDelegationTokenServlet extends DfsServlet {
+  private static final Log LOG = LogFactory.getLog(GetDelegationTokenServlet.class);
   public static final String PATH_SPEC = "/getDelegationToken";
   public static final String RENEWER = "renewer";
   
@@ -49,8 +48,11 @@ public class DelegationTokenServlet extends DfsServlet {
   protected void doGet(final HttpServletRequest req, final HttpServletResponse resp)
       throws ServletException, IOException {
     final UserGroupInformation ugi;
+    final ServletContext context = getServletContext();
+    final Configuration conf = 
+      (Configuration) context.getAttribute(JspHelper.CURRENT_CONF);
     try {
-      ugi = getUGI(req, new Configuration());
+      ugi = getUGI(req, conf);
     } catch(IOException ioe) {
       LOG.info("Request for token received with no authentication from "
           + req.getRemoteAddr(), ioe);
@@ -59,7 +61,6 @@ public class DelegationTokenServlet extends DfsServlet {
       return;
     }
     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) ? 
@@ -72,16 +73,18 @@ public class DelegationTokenServlet extends DfsServlet {
       ugi.doAs(new PrivilegedExceptionAction<Void>() {
         @Override
         public Void run() throws Exception {
-          
+          String s = NameNode.getAddress(conf).getAddress().getHostAddress()
+          + ":" + NameNode.getAddress(conf).getPort();
+
           Token<DelegationTokenIdentifier> token = 
             nn.getDelegationToken(new Text(renewerFinal));
-          String s = nn.rpcAddress.getAddress().getHostAddress() 
-                     + ":" + nn.rpcAddress.getPort();
+          if(token == null) {
+            throw new Exception("couldn't get the token for " +s);
+          }
           token.setService(new Text(s));
           Credentials ts = new Credentials();
           ts.addToken(new Text(ugi.getShortUserName()), token);
           ts.write(dosFinal);
-          dosFinal.close();
           return null;
         }
       });

+ 15 - 7
src/java/org/apache/hadoop/hdfs/server/namenode/NameNode.java

@@ -43,7 +43,6 @@ import org.apache.hadoop.fs.UnresolvedLinkException;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.fs.permission.PermissionStatus;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
-import org.apache.hadoop.hdfs.DFSUtil;
 import org.apache.hadoop.hdfs.HDFSPolicyProvider;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.protocol.Block;
@@ -60,11 +59,11 @@ import org.apache.hadoop.hdfs.protocol.UnregisteredNodeException;
 import org.apache.hadoop.hdfs.protocol.UnresolvedPathException;
 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.HdfsConstants.NamenodeRole;
+import org.apache.hadoop.hdfs.server.common.HdfsConstants.StartupOption;
 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;
 import org.apache.hadoop.hdfs.server.namenode.metrics.NameNodeMetrics;
 import org.apache.hadoop.hdfs.server.protocol.BlocksWithLocations;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeCommand;
@@ -77,6 +76,9 @@ import org.apache.hadoop.hdfs.server.protocol.NamenodeRegistration;
 import org.apache.hadoop.hdfs.server.protocol.NamespaceInfo;
 import org.apache.hadoop.hdfs.server.protocol.NodeRegistration;
 import org.apache.hadoop.hdfs.server.protocol.UpgradeCommand;
+import org.apache.hadoop.hdfs.server.namenode.GetDelegationTokenServlet;
+import org.apache.hadoop.hdfs.server.namenode.CancelDelegationTokenServlet;
+import org.apache.hadoop.hdfs.server.namenode.RenewDelegationTokenServlet;
 import org.apache.hadoop.http.HttpServer;
 import org.apache.hadoop.io.EnumSetWritable;
 import org.apache.hadoop.io.Text;
@@ -88,14 +90,14 @@ import org.apache.hadoop.net.Node;
 import org.apache.hadoop.security.AccessControlException;
 import org.apache.hadoop.security.Groups;
 import org.apache.hadoop.security.RefreshUserMappingsProtocol;
+import org.apache.hadoop.security.SecurityUtil;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.authorize.AuthorizationException;
 import org.apache.hadoop.security.authorize.ProxyUsers;
 import org.apache.hadoop.security.authorize.RefreshAuthorizationPolicyProtocol;
 import org.apache.hadoop.security.authorize.ServiceAuthorizationManager;
-import org.apache.hadoop.security.token.Token;
 import org.apache.hadoop.security.token.SecretManager.InvalidToken;
-import org.apache.hadoop.security.SecurityUtil;
+import org.apache.hadoop.security.token.Token;
 import org.apache.hadoop.util.ServicePlugin;
 import org.apache.hadoop.util.StringUtils;
 
@@ -482,8 +484,14 @@ public class NameNode implements NamenodeProtocols, FSConstants {
           httpServer.setAttribute("name.system.image", getFSImage());
           httpServer.setAttribute(JspHelper.CURRENT_CONF, conf);
           httpServer.addInternalServlet("getDelegationToken",
-              DelegationTokenServlet.PATH_SPEC, DelegationTokenServlet.class,
-              true);
+              GetDelegationTokenServlet.PATH_SPEC, 
+              GetDelegationTokenServlet.class, true);
+          httpServer.addInternalServlet("renewDelegationToken", 
+              RenewDelegationTokenServlet.PATH_SPEC, 
+              RenewDelegationTokenServlet.class, true);
+          httpServer.addInternalServlet("cancelDelegationToken", 
+              CancelDelegationTokenServlet.PATH_SPEC, 
+              CancelDelegationTokenServlet.class, true);
           httpServer.addInternalServlet("fsck", "/fsck", FsckServlet.class,
               true);
           httpServer.addInternalServlet("getimage", "/getimage",

+ 86 - 0
src/java/org/apache/hadoop/hdfs/server/namenode/RenewDelegationTokenServlet.java

@@ -0,0 +1,86 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with this
+ * work for additional information regarding copyright ownership. The ASF
+ * licenses this file to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ * 
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
+ * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
+ * License for the specific language governing permissions and limitations under
+ * the License.
+ */
+package org.apache.hadoop.hdfs.server.namenode;
+
+import java.io.IOException;
+import java.io.PrintStream;
+import java.security.PrivilegedExceptionAction;
+
+import javax.servlet.ServletContext;
+import javax.servlet.ServletException;
+import javax.servlet.http.HttpServletRequest;
+import javax.servlet.http.HttpServletResponse;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier;
+import org.apache.hadoop.hdfs.server.common.JspHelper;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.security.token.Token;
+
+/**
+ * Renew delegation tokens over http for use in hftp.
+ */
+@SuppressWarnings("serial")
+public class RenewDelegationTokenServlet extends DfsServlet {
+  private static final Log LOG = LogFactory.getLog(RenewDelegationTokenServlet.class);
+  public static final String PATH_SPEC = "/renewDelegationToken";
+  public static final String TOKEN = "token";
+  
+  @Override
+  protected void doGet(final HttpServletRequest req, final HttpServletResponse resp)
+      throws ServletException, IOException {
+    final UserGroupInformation ugi;
+    final ServletContext context = getServletContext();
+    final Configuration conf = 
+      (Configuration) context.getAttribute(JspHelper.CURRENT_CONF);
+    try {
+      ugi = getUGI(req, conf);
+    } catch(IOException ioe) {
+      LOG.info("Request for token received with no authentication from "
+          + req.getRemoteAddr(), ioe);
+      resp.sendError(HttpServletResponse.SC_FORBIDDEN, 
+          "Unable to identify or authenticate user");
+      return;
+    }
+    final NameNode nn = (NameNode) context.getAttribute("name.node");
+    String tokenString = req.getParameter(TOKEN);
+    if (tokenString == null) {
+      resp.sendError(HttpServletResponse.SC_MULTIPLE_CHOICES,
+                     "Token to renew not specified");
+    }
+    final Token<DelegationTokenIdentifier> token = 
+      new Token<DelegationTokenIdentifier>();
+    token.decodeFromUrlString(tokenString);
+    
+    try {
+      long result = ugi.doAs(new PrivilegedExceptionAction<Long>() {
+        public Long run() throws Exception {
+          return nn.renewDelegationToken(token);
+        }
+      });
+      PrintStream os = new PrintStream(resp.getOutputStream());
+      os.println(result);
+      os.close();
+    } catch(Exception e) {
+      LOG.info("Exception while renewing token. Re-throwing. ", e);
+      resp.sendError(HttpServletResponse.SC_INTERNAL_SERVER_ERROR,
+                     e.getMessage());
+    }
+  }
+}

+ 91 - 15
src/java/org/apache/hadoop/hdfs/tools/DelegationTokenFetcher.java

@@ -17,27 +17,35 @@
  */
 package org.apache.hadoop.hdfs.tools;
 
+import java.io.BufferedReader;
 import java.io.DataInputStream;
 import java.io.DataOutputStream;
 import java.io.FileOutputStream;
 import java.io.IOException;
 import java.io.InputStream;
+import java.io.InputStreamReader;
+import java.net.HttpURLConnection;
 import java.net.InetAddress;
 import java.net.URI;
 import java.net.URL;
 import java.net.URLConnection;
 import java.security.PrivilegedExceptionAction;
 
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.hdfs.DistributedFileSystem;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier;
-import org.apache.hadoop.hdfs.server.namenode.DelegationTokenServlet;
+import org.apache.hadoop.hdfs.server.namenode.CancelDelegationTokenServlet;
+import org.apache.hadoop.hdfs.server.namenode.GetDelegationTokenServlet;
+import org.apache.hadoop.hdfs.server.namenode.RenewDelegationTokenServlet;
+import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.io.Text;
-import org.apache.hadoop.security.SecurityUtil;
 import org.apache.hadoop.security.Credentials;
+import org.apache.hadoop.security.SecurityUtil;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.token.Token;
 import org.apache.hadoop.util.StringUtils;
@@ -57,6 +65,14 @@ public class DelegationTokenFetcher {
   private final UserGroupInformation ugi;
   private final DataOutputStream out;
   private final Configuration conf;
+  
+  private static final Log LOG = 
+    LogFactory.getLog(DelegationTokenFetcher.class);
+
+  static {
+    // Enable Kerberos sockets
+    System.setProperty("https.cipherSuites", "TLS_KRB5_WITH_3DES_EDE_CBC_SHA");
+  }
 
   /**
    * Command-line interface
@@ -122,7 +138,7 @@ public class DelegationTokenFetcher {
     String fullName = ugi.getUserName();
     String shortName = ugi.getShortUserName();
     Token<DelegationTokenIdentifier> token = 
-      dfs.getDelegationToken(new Text(fullName));
+      dfs.getDelegationToken(fullName);
     
     // Reconstruct the ip:port of the Namenode
     URI uri = FileSystem.getDefaultUri(conf);
@@ -135,27 +151,21 @@ public class DelegationTokenFetcher {
     ts.writeTokenStorageToStream(out);
   }
   
-
-  /**
-   * Utility method to obtain a delegation token over http
-   * @param nnAddr Namenode http addr, such as http://namenode:50070
-   */
-  static public Credentials getDTfromRemote(String nnAddr, String renewer) 
-  throws IOException {
-    // Enable Kerberos sockets
-    System.setProperty("https.cipherSuites", "TLS_KRB5_WITH_3DES_EDE_CBC_SHA");
+  static public Credentials getDTfromRemote(String nnAddr, 
+      String renewer) throws IOException {
     DataInputStream dis = null;
 
     try {
       StringBuffer url = new StringBuffer();
       if (renewer != null) {
-        url.append(nnAddr).append(DelegationTokenServlet.PATH_SPEC).append("?").
-        append(DelegationTokenServlet.RENEWER).append("=").append(renewer);
+        url.append(nnAddr).append(GetDelegationTokenServlet.PATH_SPEC).append("?").
+        append(GetDelegationTokenServlet.RENEWER).append("=").append(renewer);
       } else {
-        url.append(nnAddr).append(DelegationTokenServlet.PATH_SPEC);
+        url.append(nnAddr).append(GetDelegationTokenServlet.PATH_SPEC);
       }
       System.out.println("Retrieving token from: " + url);
       URL remoteURL = new URL(url.toString());
+      SecurityUtil.fetchServiceTicket(remoteURL);
       URLConnection connection = remoteURL.openConnection();
 
       InputStream in = connection.getInputStream();
@@ -170,6 +180,72 @@ public class DelegationTokenFetcher {
     }
   }
 
+  /**
+   * Renew a Delegation Token.
+   * @param nnAddr the NameNode's address
+   * @param tok the token to renew
+   * @return the Date that the token will expire next.
+   * @throws IOException
+   */
+  static public long renewDelegationToken(String nnAddr,
+      Token<DelegationTokenIdentifier> tok
+  ) throws IOException {
+    StringBuilder buf = new StringBuilder();
+    buf.append(nnAddr);
+    buf.append(RenewDelegationTokenServlet.PATH_SPEC);
+    buf.append("?");
+    buf.append(RenewDelegationTokenServlet.TOKEN);
+    buf.append("=");
+    buf.append(tok.encodeToUrlString());
+    BufferedReader in = null;
+    try {
+      URL url = new URL(buf.toString());
+      SecurityUtil.fetchServiceTicket(url);
+      URLConnection connection = url.openConnection();
+      in = new BufferedReader(new InputStreamReader
+          (connection.getInputStream()));
+      long result = Long.parseLong(in.readLine());
+      in.close();
+      return result;
+    } catch (IOException ie) {
+      IOUtils.cleanup(LOG, in);
+      throw ie;
+    }
+  }
+
+  /**
+   * Cancel a Delegation Token.
+   * @param nnAddr the NameNode's address
+   * @param tok the token to cancel
+   * @throws IOException
+   */
+  static public void cancelDelegationToken(String nnAddr,
+      Token<DelegationTokenIdentifier> tok
+  ) throws IOException {
+    StringBuilder buf = new StringBuilder();
+    buf.append(nnAddr);
+    buf.append(CancelDelegationTokenServlet.PATH_SPEC);
+    buf.append("?");
+    buf.append(CancelDelegationTokenServlet.TOKEN);
+    buf.append("=");
+    buf.append(tok.encodeToUrlString());
+    BufferedReader in = null;
+    try {
+      URL url = new URL(buf.toString());
+      SecurityUtil.fetchServiceTicket(url);
+      HttpURLConnection connection = (HttpURLConnection) url.openConnection();
+      if (connection.getResponseCode() != HttpURLConnection.HTTP_OK) {
+        throw new IOException("Error cancelling token:" + 
+            connection.getResponseMessage());
+      }
+    } catch (IOException ie) {
+      IOUtils.cleanup(LOG, in);
+      throw ie;
+    }
+  }
+
+
+
   /**
    * Utility method to obtain a delegation token over http
    * @param nnHttpAddr Namenode http addr, such as http://namenode:50070

+ 3 - 3
src/test/hdfs/org/apache/hadoop/hdfs/security/TestDelegationToken.java

@@ -144,7 +144,7 @@ public class TestDelegationToken {
   public void testDelegationTokenDFSApi() throws Exception {
     DelegationTokenSecretManager dtSecretManager = cluster.getNamesystem().getDelegationTokenSecretManager();
     DistributedFileSystem dfs = (DistributedFileSystem) cluster.getFileSystem();
-    Token<DelegationTokenIdentifier> token = dfs.getDelegationToken(new Text("JobTracker"));
+    Token<DelegationTokenIdentifier> token = dfs.getDelegationToken("JobTracker");
     DelegationTokenIdentifier identifier = new DelegationTokenIdentifier();
     byte[] tokenId = token.getIdentifier();
     identifier.readFields(new DataInputStream(
@@ -157,8 +157,8 @@ public class TestDelegationToken {
   @Test
   public void testDelegationTokenWithDoAs() throws Exception {
     final DistributedFileSystem dfs = (DistributedFileSystem) cluster.getFileSystem();
-    final Token<DelegationTokenIdentifier> token = dfs.getDelegationToken(new Text(
-        "JobTracker"));
+    final Token<DelegationTokenIdentifier> token = 
+      dfs.getDelegationToken("JobTracker");
     final UserGroupInformation longUgi = UserGroupInformation
         .createRemoteUser("JobTracker/foo.com@FOO.COM");
     final UserGroupInformation shortUgi = UserGroupInformation

+ 1 - 1
src/test/hdfs/org/apache/hadoop/hdfs/security/TestDelegationTokenForProxyUser.java

@@ -122,7 +122,7 @@ public class TestDelegationTokenForProxyUser {
             public Token<DelegationTokenIdentifier> run() throws IOException {
               DistributedFileSystem dfs = (DistributedFileSystem) cluster
                   .getFileSystem();
-              return dfs.getDelegationToken(new Text("RenewerUser"));
+              return dfs.getDelegationToken("RenewerUser");
             }
           });
       DelegationTokenIdentifier identifier = new DelegationTokenIdentifier();

+ 1 - 1
src/test/hdfs/org/apache/hadoop/tools/TestDelegationTokenFetcher.java

@@ -74,7 +74,7 @@ public class TestDelegationTokenFetcher {
     // Create a token for the fetcher to fetch, wire NN to return it when asked
     // for this particular user.
     Token<DelegationTokenIdentifier> t = new Token<DelegationTokenIdentifier>();
-    when(dfs.getDelegationToken(eq(new Text(LONG_NAME)))).thenReturn(t);
+    when(dfs.getDelegationToken(eq(LONG_NAME))).thenReturn(t);
     
     // Now, actually let the TokenFetcher go fetch the token.
     final ByteArrayOutputStream baos = new ByteArrayOutputStream();