浏览代码

Reverting HDFS-1296. Broke the build.

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

+ 0 - 3
CHANGES.txt

@@ -174,9 +174,6 @@ Trunk (unreleased changes)
     HDFS-1317. Remove the FILEPATH_PATTERN from hdfsproxy.AuthorizationFilter.
     (Rohini Palaniswamy via szetszwo)
 
-    HDFS-1296. using delegation token over hftp for long running 
-    clients (boryas)
-
 Release 0.21.0 - Unreleased
 
   INCOMPATIBLE CHANGES

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

@@ -23,7 +23,6 @@ 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;
@@ -57,9 +56,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;
@@ -77,7 +76,6 @@ 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;
@@ -98,11 +96,13 @@ 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.SecretManager.InvalidToken;
 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.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,36 +360,12 @@ 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 {
-    Token<DelegationTokenIdentifier> result =
-      namenode.getDelegationToken(renewer);
-    LOG.info("Created " + stringifyToken(result));
-    return result;
+    return namenode.getDelegationToken(renewer);
   }
 
   /**
@@ -397,7 +373,6 @@ 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) {
@@ -411,7 +386,6 @@ 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) {

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

@@ -18,46 +18,34 @@
 
 package org.apache.hadoop.hdfs;
 
-import java.io.FileNotFoundException;
-import java.io.IOException;
-import java.net.InetSocketAddress;
-import java.net.URI;
+import java.io.*;
+import java.net.*;
 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.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.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.Block;
 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.protocol.DirectoryListing;
+import org.apache.hadoop.hdfs.protocol.FSConstants.DatanodeReportType;
+import org.apache.hadoop.hdfs.protocol.FSConstants.UpgradeAction;
 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.SecretManager.InvalidToken;
 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.util.Progressable;
+import org.apache.hadoop.fs.Options;
 
 
 /****************************************************************
@@ -689,23 +677,7 @@ 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.
    */
@@ -716,9 +688,7 @@ 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);

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

@@ -21,7 +21,6 @@ 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;
@@ -31,11 +30,9 @@ 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;
@@ -49,7 +46,6 @@ 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;
@@ -84,18 +80,16 @@ 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<DelegationTokenIdentifier> delegationToken;
+  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);
     df.setTimeZone(TimeZone.getTimeZone(HFTP_TIMEZONE));
@@ -104,33 +98,11 @@ public class HftpFileSystem extends FileSystem {
 
   protected static final ThreadLocal<SimpleDateFormat> df =
     new ThreadLocal<SimpleDateFormat>() {
-    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();
-  }
-
+      protected SimpleDateFormat initialValue() {
+        return getDateFormat();
+      }
+    };
 
-  @SuppressWarnings("unchecked")
   @Override
   public void initialize(final URI name, final Configuration conf)
   throws IOException {
@@ -138,79 +110,66 @@ public class HftpFileSystem extends FileSystem {
     setConf(conf);
     this.ugi = UserGroupInformation.getCurrentUser(); 
     nnAddr = NetUtils.createSocketAddr(name.toString());
-   
-    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();
-    }
+    
+    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);
 
-    try {
-      hdfsURI = new URI(buildUri("hdfs://", nnAddr.getHostName(), nnPort));
-    } catch (URISyntaxException ue) {
-      throw new IOException("bad uri for hdfs", ue);
-    }
+      LOG.debug("Trying to find DT for " + name + " using key=" + key + 
+          "; conf=" + conf.get(key, ""));
+      Text nnServiceNameText = new Text(conf.get(key, ""));
 
-    if (UserGroupInformation.isSecurityEnabled()) {
+      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
-      String canonicalName = getCanonicalServiceName();
-      for (Token<? extends TokenIdentifier> t : ugi.getTokens()) {
-        if (DelegationTokenIdentifier.HDFS_DELEGATION_KIND.equals(t.getKind()) &&
-            t.getService().toString().equals(canonicalName)) {
+      for (Token<? extends TokenIdentifier> t : tokens) {
+        if ((t.getService()).equals(nnServiceNameText)) {
           LOG.debug("Found existing DT for " + name);
-          delegationToken = (Token<DelegationTokenIdentifier>) t;
-          break;
+          delegationToken = t;
+          return;
         }
       }
       //since we don't already have a token, go get one over https
-      if (delegationToken == null) {
-        delegationToken = 
-          (Token<DelegationTokenIdentifier>) getDelegationToken(null);
-        renewer.addTokenToRenew(this);
+      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);
       }
     }
   }
   
-
-  @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);
-    }
+  
+  public Token<? extends TokenIdentifier> getDelegationToken() {
+    return delegationToken;
   }
 
   @Override
@@ -279,12 +238,10 @@ public class HftpFileSystem extends FileSystem {
   protected String updateQuery(String query) throws IOException {
     String tokenString = null;
     if (UserGroupInformation.isSecurityEnabled()) {
-      synchronized (this) {
-        if (delegationToken != null) {
-          tokenString = delegationToken.encodeToUrlString();
-          return (query + JspHelper.SET_DELEGATION + tokenString);
-        } // else we are talking to an insecure cluster
-      }
+      if (delegationToken != null) {
+        tokenString = delegationToken.encodeToUrlString();
+        return (query + JspHelper.SET_DELEGATION + tokenString);
+      } // else we are talking to an unsecure cluster
     }
     return query;
   }
@@ -563,157 +520,4 @@ 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);
-          }
-        }
-      }
-    }
-  }
 }

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

@@ -1,83 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements. See the NOTICE file distributed with this
- * work for additional information regarding copyright ownership. The ASF
- * licenses this file to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance with the License.
- * You may obtain a copy of the License at
- * 
- * http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
- * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
- * License for the specific language governing permissions and limitations under
- * the License.
- */
-package org.apache.hadoop.hdfs.server.namenode;
-
-import java.io.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());
-    }
-  }
-}

+ 10 - 13
src/java/org/apache/hadoop/hdfs/server/namenode/GetDelegationTokenServlet.java → src/java/org/apache/hadoop/hdfs/server/namenode/DelegationTokenServlet.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,9 +38,10 @@ import org.apache.hadoop.security.token.Token;
 /**
  * Serve delegation tokens over http for use in hftp.
  */
+@InterfaceAudience.Private
 @SuppressWarnings("serial")
-public class GetDelegationTokenServlet extends DfsServlet {
-  private static final Log LOG = LogFactory.getLog(GetDelegationTokenServlet.class);
+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";
   
@@ -48,11 +49,8 @@ public class GetDelegationTokenServlet 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, conf);
+      ugi = getUGI(req, new Configuration());
     } catch(IOException ioe) {
       LOG.info("Request for token received with no authentication from "
           + req.getRemoteAddr(), ioe);
@@ -61,6 +59,7 @@ public class GetDelegationTokenServlet 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) ? 
@@ -73,18 +72,16 @@ public class GetDelegationTokenServlet 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));
-          if(token == null) {
-            throw new Exception("couldn't get the token for " +s);
-          }
+          String s = nn.rpcAddress.getAddress().getHostAddress() 
+                     + ":" + nn.rpcAddress.getPort();
           token.setService(new Text(s));
           Credentials ts = new Credentials();
           ts.addToken(new Text(ugi.getShortUserName()), token);
           ts.write(dosFinal);
+          dosFinal.close();
           return null;
         }
       });

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

@@ -43,6 +43,7 @@ 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;
@@ -59,11 +60,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;
@@ -76,9 +77,6 @@ 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;
@@ -90,14 +88,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.SecretManager.InvalidToken;
 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.util.ServicePlugin;
 import org.apache.hadoop.util.StringUtils;
 
@@ -484,14 +482,8 @@ public class NameNode implements NamenodeProtocols, FSConstants {
           httpServer.setAttribute("name.system.image", getFSImage());
           httpServer.setAttribute(JspHelper.CURRENT_CONF, conf);
           httpServer.addInternalServlet("getDelegationToken",
-              GetDelegationTokenServlet.PATH_SPEC, 
-              GetDelegationTokenServlet.class, true);
-          httpServer.addInternalServlet("renewDelegationToken", 
-              RenewDelegationTokenServlet.PATH_SPEC, 
-              RenewDelegationTokenServlet.class, true);
-          httpServer.addInternalServlet("cancelDelegationToken", 
-              CancelDelegationTokenServlet.PATH_SPEC, 
-              CancelDelegationTokenServlet.class, true);
+              DelegationTokenServlet.PATH_SPEC, DelegationTokenServlet.class,
+              true);
           httpServer.addInternalServlet("fsck", "/fsck", FsckServlet.class,
               true);
           httpServer.addInternalServlet("getimage", "/getimage",

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

@@ -1,86 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements. See the NOTICE file distributed with this
- * work for additional information regarding copyright ownership. The ASF
- * licenses this file to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance with the License.
- * You may obtain a copy of the License at
- * 
- * http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
- * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
- * License for the specific language governing permissions and limitations under
- * the License.
- */
-package org.apache.hadoop.hdfs.server.namenode;
-
-import java.io.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());
-    }
-  }
-}

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

@@ -17,35 +17,27 @@
  */
 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.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.hdfs.server.namenode.DelegationTokenServlet;
 import org.apache.hadoop.io.Text;
-import org.apache.hadoop.security.Credentials;
 import org.apache.hadoop.security.SecurityUtil;
+import org.apache.hadoop.security.Credentials;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.token.Token;
 import org.apache.hadoop.util.StringUtils;
@@ -65,14 +57,6 @@ 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
@@ -138,7 +122,7 @@ public class DelegationTokenFetcher {
     String fullName = ugi.getUserName();
     String shortName = ugi.getShortUserName();
     Token<DelegationTokenIdentifier> token = 
-      dfs.getDelegationToken(fullName);
+      dfs.getDelegationToken(new Text(fullName));
     
     // Reconstruct the ip:port of the Namenode
     URI uri = FileSystem.getDefaultUri(conf);
@@ -151,21 +135,27 @@ public class DelegationTokenFetcher {
     ts.writeTokenStorageToStream(out);
   }
   
-  static public Credentials getDTfromRemote(String nnAddr, 
-      String renewer) throws IOException {
+
+  /**
+   * 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");
     DataInputStream dis = null;
 
     try {
       StringBuffer url = new StringBuffer();
       if (renewer != null) {
-        url.append(nnAddr).append(GetDelegationTokenServlet.PATH_SPEC).append("?").
-        append(GetDelegationTokenServlet.RENEWER).append("=").append(renewer);
+        url.append(nnAddr).append(DelegationTokenServlet.PATH_SPEC).append("?").
+        append(DelegationTokenServlet.RENEWER).append("=").append(renewer);
       } else {
-        url.append(nnAddr).append(GetDelegationTokenServlet.PATH_SPEC);
+        url.append(nnAddr).append(DelegationTokenServlet.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();
@@ -180,72 +170,6 @@ 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("JobTracker");
+    Token<DelegationTokenIdentifier> token = dfs.getDelegationToken(new Text("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("JobTracker");
+    final Token<DelegationTokenIdentifier> token = dfs.getDelegationToken(new Text(
+        "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("RenewerUser");
+              return dfs.getDelegationToken(new Text("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(LONG_NAME))).thenReturn(t);
+    when(dfs.getDelegationToken(eq(new Text(LONG_NAME)))).thenReturn(t);
     
     // Now, actually let the TokenFetcher go fetch the token.
     final ByteArrayOutputStream baos = new ByteArrayOutputStream();