You are viewing a plain text version of this content. The canonical link for it is here.
Posted to common-commits@hadoop.apache.org by ji...@apache.org on 2011/10/15 03:16:36 UTC

svn commit: r1183576 - in /hadoop/common/branches/branch-0.23/hadoop-common-project/hadoop-common: ./ src/main/java/org/apache/hadoop/security/ src/main/java/org/apache/hadoop/security/token/

Author: jitendra
Date: Sat Oct 15 01:16:34 2011
New Revision: 1183576

URL: http://svn.apache.org/viewvc?rev=1183576&view=rev
Log:
Merged r1183187 from trunk for MAPREDUCE-2764.

Added:
    hadoop/common/branches/branch-0.23/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/token/TokenRenewer.java
      - copied unchanged from r1183187, hadoop/common/trunk/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/token/TokenRenewer.java
Modified:
    hadoop/common/branches/branch-0.23/hadoop-common-project/hadoop-common/CHANGES.txt
    hadoop/common/branches/branch-0.23/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/SecurityUtil.java
    hadoop/common/branches/branch-0.23/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/UserGroupInformation.java
    hadoop/common/branches/branch-0.23/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/token/Token.java

Modified: hadoop/common/branches/branch-0.23/hadoop-common-project/hadoop-common/CHANGES.txt
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-0.23/hadoop-common-project/hadoop-common/CHANGES.txt?rev=1183576&r1=1183575&r2=1183576&view=diff
==============================================================================
--- hadoop/common/branches/branch-0.23/hadoop-common-project/hadoop-common/CHANGES.txt (original)
+++ hadoop/common/branches/branch-0.23/hadoop-common-project/hadoop-common/CHANGES.txt Sat Oct 15 01:16:34 2011
@@ -1,5 +1,7 @@
 Hadoop Change Log
 
+    MAPREDUCE-2764. Fix renewal of dfs delegation tokens. (Owen via jitendra)
+
 Release 0.23.0 - Unreleased
 
   INCOMPATIBLE CHANGES

Modified: hadoop/common/branches/branch-0.23/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/SecurityUtil.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-0.23/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/SecurityUtil.java?rev=1183576&r1=1183575&r2=1183576&view=diff
==============================================================================
--- hadoop/common/branches/branch-0.23/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/SecurityUtil.java (original)
+++ hadoop/common/branches/branch-0.23/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/SecurityUtil.java Sat Oct 15 01:16:34 2011
@@ -18,6 +18,7 @@ package org.apache.hadoop.security;
 
 import java.io.IOException;
 import java.net.InetAddress;
+import java.net.InetSocketAddress;
 import java.net.URI;
 import java.net.URL;
 import java.net.UnknownHostException;
@@ -34,7 +35,9 @@ import org.apache.commons.logging.LogFac
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.io.Text;
 import org.apache.hadoop.net.NetUtils;
+import org.apache.hadoop.security.token.Token;
 import org.apache.hadoop.security.token.TokenInfo;
 
 import sun.security.jgss.krb5.Krb5Util;
@@ -352,4 +355,22 @@ public class SecurityUtil {
     return null;
   }
 
+  /**
+   * Set the given token's service to the format expected by the RPC client 
+   * @param token a delegation token
+   * @param addr the socket for the rpc connection
+   */
+  public static void setTokenService(Token<?> token, InetSocketAddress addr) {
+    token.setService(buildTokenService(addr));
+  }
+  
+  /**
+   * Construct the service key for a token
+   * @param addr InetSocketAddress of remote connection with a token
+   * @return "ip:port"
+   */
+  public static Text buildTokenService(InetSocketAddress addr) {
+    String host = addr.getAddress().getHostAddress();
+    return new Text(host + ":" + addr.getPort());
+  }
 }

Modified: hadoop/common/branches/branch-0.23/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/UserGroupInformation.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-0.23/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/UserGroupInformation.java?rev=1183576&r1=1183575&r2=1183576&view=diff
==============================================================================
--- hadoop/common/branches/branch-0.23/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/UserGroupInformation.java (original)
+++ hadoop/common/branches/branch-0.23/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/UserGroupInformation.java Sat Oct 15 01:16:34 2011
@@ -635,6 +635,23 @@ public class UserGroupInformation {
   }
   
   /**
+   * Re-login a user from keytab if TGT is expired or is close to expiry.
+   * 
+   * @throws IOException
+   */
+  public synchronized void checkTGTAndReloginFromKeytab() throws IOException {
+    if (!isSecurityEnabled()
+        || user.getAuthenticationMethod() != AuthenticationMethod.KERBEROS
+        || !isKeytab)
+      return;
+    KerberosTicket tgt = getTGT();
+    if (tgt != null && System.currentTimeMillis() < getRefreshTime(tgt)) {
+      return;
+    }
+    reloginFromKeytab();
+  }
+
+  /**
    * Re-Login a user in from a keytab file. Loads a user identity from a keytab
    * file and logs them in. They become the currently logged-in user. This
    * method assumes that {@link #loginUserFromKeytab(String, String)} had 

Modified: hadoop/common/branches/branch-0.23/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/token/Token.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-0.23/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/token/Token.java?rev=1183576&r1=1183575&r2=1183576&view=diff
==============================================================================
--- hadoop/common/branches/branch-0.23/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/token/Token.java (original)
+++ hadoop/common/branches/branch-0.23/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/token/Token.java Sat Oct 15 01:16:34 2011
@@ -22,11 +22,15 @@ import java.io.DataInput;
 import java.io.DataOutput;
 import java.io.IOException;
 import java.util.Arrays;
+import java.util.ServiceLoader;
 
 import org.apache.commons.codec.binary.Base64;
-
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+  
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.io.DataInputBuffer;
 import org.apache.hadoop.io.DataOutputBuffer;
 import org.apache.hadoop.io.Text;
@@ -40,10 +44,12 @@ import org.apache.hadoop.io.WritableUtil
 @InterfaceAudience.LimitedPrivate({"HDFS", "MapReduce"})
 @InterfaceStability.Evolving
 public class Token<T extends TokenIdentifier> implements Writable {
+  public static final Log LOG = LogFactory.getLog(Token.class);
   private byte[] identifier;
   private byte[] password;
   private Text kind;
   private Text service;
+  private TokenRenewer renewer;
   
   /**
    * Construct a token given a token identifier and a secret manager for the
@@ -83,6 +89,17 @@ public class Token<T extends TokenIdenti
   }
 
   /**
+   * Clone a token.
+   * @param other the token to clone
+   */
+  public Token(Token<T> other) {
+    this.identifier = other.identifier;
+    this.password = other.password;
+    this.kind = other.kind;
+    this.service = other.service;
+  }
+
+  /**
    * Get the token identifier
    * @return the token identifier
    */
@@ -107,6 +124,17 @@ public class Token<T extends TokenIdenti
   }
 
   /**
+   * Set the token kind. This is only intended to be used by services that
+   * wrap another service's token, such as HFTP wrapping HDFS.
+   * @param newKind
+   */
+  @InterfaceAudience.Private
+  public synchronized void setKind(Text newKind) {
+    kind = newKind;
+    renewer = null;
+  }
+
+  /**
    * Get the service on which the token is supposed to be used
    * @return the service name
    */
@@ -244,4 +272,90 @@ public class Token<T extends TokenIdenti
     buffer.append(service.toString());
     return buffer.toString();
   }
+  
+  private static ServiceLoader<TokenRenewer> renewers =
+      ServiceLoader.load(TokenRenewer.class);
+
+  private synchronized TokenRenewer getRenewer() throws IOException {
+    if (renewer != null) {
+      return renewer;
+    }
+    renewer = TRIVIAL_RENEWER;
+    for (TokenRenewer canidate: renewers) {
+      if (canidate.handleKind(this.kind)) {
+        renewer = canidate;
+        return renewer;
+      }
+    }
+    LOG.warn("No TokenRenewer defined for token kind " + this.kind);
+    return renewer;
+  }
+
+  /**
+   * Is this token managed so that it can be renewed or cancelled?
+   * @return true, if it can be renewed and cancelled.
+   */
+  public boolean isManaged() throws IOException {
+    return getRenewer().isManaged(this);
+  }
+
+  /**
+   * Renew this delegation token
+   * @return the new expiration time
+   * @throws IOException
+   * @throws InterruptedException
+   */
+  public long renew(Configuration conf
+                    ) throws IOException, InterruptedException {
+    return getRenewer().renew(this, conf);
+  }
+  
+  /**
+   * Cancel this delegation token
+   * @throws IOException
+   * @throws InterruptedException
+   */
+  public void cancel(Configuration conf
+                     ) throws IOException, InterruptedException {
+    getRenewer().cancel(this, conf);
+  }
+  
+  /**
+   * A trivial renewer for token kinds that aren't managed. Sub-classes need
+   * to implement getKind for their token kind.
+   */
+  @InterfaceAudience.LimitedPrivate({"HDFS", "MapReduce"})
+  @InterfaceStability.Evolving
+  public static class TrivialRenewer extends TokenRenewer {
+    
+    // define the kind for this renewer
+    protected Text getKind() {
+      return null;
+    }
+
+    @Override
+    public boolean handleKind(Text kind) {
+      return kind.equals(getKind());
+    }
+
+    @Override
+    public boolean isManaged(Token<?> token) {
+      return false;
+    }
+
+    @Override
+    public long renew(Token<?> token, Configuration conf) {
+      throw new UnsupportedOperationException("Token renewal is not supported "+
+                                              " for " + token.kind + " tokens");
+    }
+
+    @Override
+    public void cancel(Token<?> token, Configuration conf) throws IOException,
+        InterruptedException {
+      throw new UnsupportedOperationException("Token cancel is not supported " +
+          " for " + token.kind + " tokens");
+    }
+
+  }
+  private static final TokenRenewer TRIVIAL_RENEWER = new TrivialRenewer();
 }