소스 검색

HADOOP-6551. Delegation token renewing and cancelling should provide
meaningful exceptions when there are failures instead of returning
false. (omalley)


git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/trunk@911743 13f79535-47bb-0310-9956-ffa450edef68

Owen O'Malley 15 년 전
부모
커밋
a417a8faea

+ 4 - 0
CHANGES.txt

@@ -148,6 +148,10 @@ Trunk (unreleased changes)
     connection setup fails. This is applicable only to keytab based logins.
     connection setup fails. This is applicable only to keytab based logins.
     (Devaraj Das)
     (Devaraj Das)
 
 
+    HADOOP-6551. Delegation token renewing and cancelling should provide
+    meaningful exceptions when there are failures instead of returning 
+    false. (omalley)
+
   OPTIMIZATIONS
   OPTIMIZATIONS
 
 
   BUG FIXES
   BUG FIXES

+ 45 - 33
src/java/org/apache/hadoop/security/token/delegation/AbstractDelegationTokenSecretManager.java

@@ -19,6 +19,8 @@
 package org.apache.hadoop.security.token.delegation;
 package org.apache.hadoop.security.token.delegation;
 
 
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.io.Text;
+
 import static org.apache.hadoop.classification.InterfaceAudience.LimitedPrivate.Project.HDFS;
 import static org.apache.hadoop.classification.InterfaceAudience.LimitedPrivate.Project.HDFS;
 import static org.apache.hadoop.classification.InterfaceAudience.LimitedPrivate.Project.MAPREDUCE;
 import static org.apache.hadoop.classification.InterfaceAudience.LimitedPrivate.Project.MAPREDUCE;
 
 
@@ -34,6 +36,7 @@ import javax.crypto.SecretKey;
 
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.security.AccessControlException;
 import org.apache.hadoop.security.token.Token;
 import org.apache.hadoop.security.token.Token;
 import org.apache.hadoop.security.token.SecretManager;
 import org.apache.hadoop.security.token.SecretManager;
 import org.apache.hadoop.util.Daemon;
 import org.apache.hadoop.util.Daemon;
@@ -185,11 +188,15 @@ extends AbstractDelegationTokenIdentifier>
   }
   }
 
 
   /**
   /**
-   * Renew a delegation token. Canceled tokens are not renewed. Return true if
-   * the token is successfully renewed; false otherwise.
+   * Renew a delegation token.
+   * @param token the token to renew
+   * @param renewer the full principal name of the user doing the renewal
+   * @return the new expiration time
+   * @throws InvalidToken if the token is invalid
+   * @throws AccessControlException if the user can't renew token
    */
    */
-  public Boolean renewToken(Token<TokenIdent> token,
-      String renewer) throws InvalidToken, IOException {
+  public long renewToken(Token<TokenIdent> token,
+                         String renewer) throws InvalidToken, IOException {
     long now = System.currentTimeMillis();
     long now = System.currentTimeMillis();
     ByteArrayInputStream buf = new ByteArrayInputStream(token.getIdentifier());
     ByteArrayInputStream buf = new ByteArrayInputStream(token.getIdentifier());
     DataInputStream in = new DataInputStream(buf);
     DataInputStream in = new DataInputStream(buf);
@@ -197,71 +204,76 @@ extends AbstractDelegationTokenIdentifier>
     id.readFields(in);
     id.readFields(in);
     synchronized (currentTokens) {
     synchronized (currentTokens) {
       if (currentTokens.get(id) == null) {
       if (currentTokens.get(id) == null) {
-        LOG.warn("Renewal request for unknown token");
-        return false;
+        throw new InvalidToken("Renewal request for unknown token");
       }
       }
     }
     }
     if (id.getMaxDate() < now) {
     if (id.getMaxDate() < now) {
-      LOG.warn("Client " + renewer + " tries to renew an expired token");
-      return false;
+      throw new InvalidToken("User " + renewer + 
+                             " tried to renew an expired token");
+    }
+    if (id.getRenewer() == null) {
+      throw new AccessControlException("User " + renewer + 
+                                       " tried to renew a token without " +
+                                       "a renewer");
     }
     }
-    if (id.getRenewer() == null || !id.getRenewer().toString().equals(renewer)) {
-      LOG.warn("Client " + renewer + " tries to renew a token with "
-          + "renewer specified as " + id.getRenewer());
-      return false;
+    if (!id.getRenewer().toString().equals(renewer)) {
+      throw new AccessControlException("Client " + renewer + 
+                                       " tries to renew a token with " +
+                                       "renewer specified as " + 
+                                       id.getRenewer());
     }
     }
     DelegationKey key = null;
     DelegationKey key = null;
     synchronized (this) {
     synchronized (this) {
       key = allKeys.get(id.getMasterKeyId());
       key = allKeys.get(id.getMasterKeyId());
     }
     }
     if (key == null) {
     if (key == null) {
-      LOG.warn("Unable to find master key for keyId=" + id.getMasterKeyId() 
-          + " from cache. Failed to renew an unexpired token with sequenceNumber=" 
-          + id.getSequenceNumber() + ", issued by this key");
-      return false;
+      throw new InvalidToken("Unable to find master key for keyId=" + 
+                             id.getMasterKeyId() +
+                             " from cache. Failed to renew an unexpired token"+
+                             " with sequenceNumber=" + id.getSequenceNumber());
     }
     }
     byte[] password = createPassword(token.getIdentifier(), key.getKey());
     byte[] password = createPassword(token.getIdentifier(), key.getKey());
     if (!Arrays.equals(password, token.getPassword())) {
     if (!Arrays.equals(password, token.getPassword())) {
-      LOG.warn("Client " + renewer + " is trying to renew a token with wrong password");
-      return false;
+      throw new AccessControlException("Client " + renewer + 
+                                       " is trying to renew a token with " +
+                                       "wrong password");
     }
     }
     DelegationTokenInformation info = new DelegationTokenInformation(
     DelegationTokenInformation info = new DelegationTokenInformation(
         Math.min(id.getMaxDate(), now + tokenRenewInterval), password);
         Math.min(id.getMaxDate(), now + tokenRenewInterval), password);
     synchronized (currentTokens) {
     synchronized (currentTokens) {
       currentTokens.put(id, info);
       currentTokens.put(id, info);
     }
     }
-    return true;
+    return info.getRenewDate();
   }
   }
   
   
   /**
   /**
-   * Cancel a token by removing it from cache. Return true if 
-   * token exists in cache; false otherwise.
+   * Cancel a token by removing it from cache.
+   * @throws InvalidToken for invalid token
+   * @throws AccessControlException if the user isn't allowed to cancel
    */
    */
-  public Boolean cancelToken(Token<TokenIdent> token,
+  public void cancelToken(Token<TokenIdent> token,
       String canceller) throws IOException {
       String canceller) throws IOException {
     ByteArrayInputStream buf = new ByteArrayInputStream(token.getIdentifier());
     ByteArrayInputStream buf = new ByteArrayInputStream(token.getIdentifier());
     DataInputStream in = new DataInputStream(buf);
     DataInputStream in = new DataInputStream(buf);
     TokenIdent id = createIdentifier();
     TokenIdent id = createIdentifier();
     id.readFields(in);
     id.readFields(in);
-    if (id.getRenewer() == null) {
-      LOG.warn("Renewer is null: Invalid Identifier");
-      return false;
-    }
     if (id.getUser() == null) {
     if (id.getUser() == null) {
-      LOG.warn("owner is null: Invalid Identifier");
-      return false;
+      throw new InvalidToken("Token with no owner");
     }
     }
     String owner = id.getUser().getUserName();
     String owner = id.getUser().getUserName();
-    String renewer = id.getRenewer().toString();
-    if (!canceller.equals(owner) && !canceller.equals(renewer)) {
-      LOG.warn(canceller + " is not authorized to cancel the token");
-      return false;
+    Text renewer = id.getRenewer();
+    if (!canceller.equals(owner) && 
+        (renewer == null || !canceller.equals(renewer.toString()))) {
+      throw new AccessControlException(canceller + 
+                                      " is not authorized to cancel the token");
     }
     }
     DelegationTokenInformation info = null;
     DelegationTokenInformation info = null;
     synchronized (currentTokens) {
     synchronized (currentTokens) {
       info = currentTokens.remove(id);
       info = currentTokens.remove(id);
     }
     }
-    return info != null;
+    if (info == null) {
+      throw new InvalidToken("Token not found");
+    }
   }
   }
   
   
   /**
   /**

+ 58 - 18
src/test/core/org/apache/hadoop/security/token/delegation/TestDelegationToken.java

@@ -23,26 +23,28 @@ import java.io.DataInput;
 import java.io.DataInputStream;
 import java.io.DataInputStream;
 import java.io.DataOutput;
 import java.io.DataOutput;
 import java.io.IOException;
 import java.io.IOException;
+import java.security.PrivilegedExceptionAction;
 import java.util.ArrayList;
 import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Collection;
 import java.util.List;
 import java.util.List;
 
 
 import junit.framework.Assert;
 import junit.framework.Assert;
 
 
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.io.DataInputBuffer;
 import org.apache.hadoop.io.DataInputBuffer;
 import org.apache.hadoop.io.DataOutputBuffer;
 import org.apache.hadoop.io.DataOutputBuffer;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.io.Writable;
 import org.apache.hadoop.io.Writable;
+import org.apache.hadoop.security.AccessControlException;
 import org.apache.hadoop.security.token.Token;
 import org.apache.hadoop.security.token.Token;
-import org.apache.hadoop.security.token.TokenIdentifier;
 import org.apache.hadoop.security.token.SecretManager.InvalidToken;
 import org.apache.hadoop.security.token.SecretManager.InvalidToken;
+import org.apache.hadoop.util.StringUtils;
 import org.junit.Test;
 import org.junit.Test;
-import org.mortbay.log.Log;
 
 
 import static org.junit.Assert.*;
 import static org.junit.Assert.*;
 
 
 public class TestDelegationToken {
 public class TestDelegationToken {
+  private static final Log LOG = LogFactory.getLog(TestDelegationToken.class);
   private static final Text KIND = new Text("MY KIND");
   private static final Text KIND = new Text("MY KIND");
 
 
   public static class TestDelegationTokenIdentifier 
   public static class TestDelegationTokenIdentifier 
@@ -137,25 +139,44 @@ public class TestDelegationToken {
         owner), new Text(renewer), null);
         owner), new Text(renewer), null);
     return new Token<TestDelegationTokenIdentifier>(dtId, dtSecretManager);
     return new Token<TestDelegationTokenIdentifier>(dtId, dtSecretManager);
   }
   }
+  
+  private void shouldThrow(PrivilegedExceptionAction<Object> action,
+                           Class<? extends Throwable> except) {
+    try {
+      action.run();
+      Assert.fail("action did not throw " + except);
+    } catch (Throwable th) {
+      LOG.info("Caught an exception: " + StringUtils.stringifyException(th));
+      assertEquals("action threw wrong exception", except, th.getClass());
+    }
+  }
+
   @Test
   @Test
   public void testDelegationTokenSecretManager() throws Exception {
   public void testDelegationTokenSecretManager() throws Exception {
-    TestDelegationTokenSecretManager dtSecretManager = 
+    final TestDelegationTokenSecretManager dtSecretManager = 
       new TestDelegationTokenSecretManager(24*60*60*1000,
       new TestDelegationTokenSecretManager(24*60*60*1000,
           3*1000,1*1000,3600000);
           3*1000,1*1000,3600000);
     try {
     try {
       dtSecretManager.startThreads();
       dtSecretManager.startThreads();
-      Token<TestDelegationTokenIdentifier> token = generateDelegationToken(
+      final Token<TestDelegationTokenIdentifier> token = 
+        generateDelegationToken(
           dtSecretManager, "SomeUser", "JobTracker");
           dtSecretManager, "SomeUser", "JobTracker");
       // Fake renewer should not be able to renew
       // Fake renewer should not be able to renew
-      Assert.assertFalse(dtSecretManager.renewToken(token, "FakeRenewer"));
-      Assert.assertTrue(dtSecretManager.renewToken(token, "JobTracker"));
+      shouldThrow(new PrivilegedExceptionAction<Object>() {
+        public Object run() throws Exception {
+          dtSecretManager.renewToken(token, "FakeRenewer");
+          return null;
+        }
+      }, AccessControlException.class);
+      long time = dtSecretManager.renewToken(token, "JobTracker");
+      assertTrue("renew time is in future", time > System.currentTimeMillis());
       TestDelegationTokenIdentifier identifier = 
       TestDelegationTokenIdentifier identifier = 
         new TestDelegationTokenIdentifier();
         new TestDelegationTokenIdentifier();
       byte[] tokenId = token.getIdentifier();
       byte[] tokenId = token.getIdentifier();
       identifier.readFields(new DataInputStream(
       identifier.readFields(new DataInputStream(
           new ByteArrayInputStream(tokenId)));
           new ByteArrayInputStream(tokenId)));
       Assert.assertTrue(null != dtSecretManager.retrievePassword(identifier));
       Assert.assertTrue(null != dtSecretManager.retrievePassword(identifier));
-      Log.info("Sleep to expire the token");
+      LOG.info("Sleep to expire the token");
       Thread.sleep(2000);
       Thread.sleep(2000);
       //Token should be expired
       //Token should be expired
       try {
       try {
@@ -165,31 +186,49 @@ public class TestDelegationToken {
       } catch (InvalidToken e) {
       } catch (InvalidToken e) {
         //Success
         //Success
       }
       }
-      Assert.assertTrue(dtSecretManager.renewToken(token, "JobTracker"));
-      Log.info("Sleep beyond the max lifetime");
+      dtSecretManager.renewToken(token, "JobTracker");
+      LOG.info("Sleep beyond the max lifetime");
       Thread.sleep(2000);
       Thread.sleep(2000);
-      Assert.assertFalse(dtSecretManager.renewToken(token, "JobTracker"));
+      
+      shouldThrow(new PrivilegedExceptionAction<Object>() {
+        public Object run() throws Exception {
+          dtSecretManager.renewToken(token, "JobTracker");
+          return null;
+        }
+      }, InvalidToken.class);
     } finally {
     } finally {
       dtSecretManager.stopThreads();
       dtSecretManager.stopThreads();
     }
     }
   }
   }
+
   @Test 
   @Test 
   public void testCancelDelegationToken() throws Exception {
   public void testCancelDelegationToken() throws Exception {
-    TestDelegationTokenSecretManager dtSecretManager = 
+    final TestDelegationTokenSecretManager dtSecretManager = 
       new TestDelegationTokenSecretManager(24*60*60*1000,
       new TestDelegationTokenSecretManager(24*60*60*1000,
         10*1000,1*1000,3600000);
         10*1000,1*1000,3600000);
     try {
     try {
       dtSecretManager.startThreads();
       dtSecretManager.startThreads();
-      Token<TestDelegationTokenIdentifier> token = generateDelegationToken(
-          dtSecretManager, "SomeUser", "JobTracker");
+      final Token<TestDelegationTokenIdentifier> token = 
+        generateDelegationToken(dtSecretManager, "SomeUser", "JobTracker");
       //Fake renewer should not be able to renew
       //Fake renewer should not be able to renew
-      Assert.assertFalse(dtSecretManager.cancelToken(token, "FakeCanceller"));
-      Assert.assertTrue(dtSecretManager.cancelToken(token, "JobTracker"));
-      Assert.assertFalse(dtSecretManager.renewToken(token, "JobTracker"));
+      shouldThrow(new PrivilegedExceptionAction<Object>() {
+        public Object run() throws Exception {
+          dtSecretManager.renewToken(token, "FakeCanceller");
+          return null;
+        }
+      }, AccessControlException.class);
+      dtSecretManager.cancelToken(token, "JobTracker");
+      shouldThrow(new PrivilegedExceptionAction<Object>() {
+        public Object run() throws Exception {
+          dtSecretManager.renewToken(token, "JobTracker");
+          return null;
+        }
+      }, InvalidToken.class);
     } finally {
     } finally {
       dtSecretManager.stopThreads();
       dtSecretManager.stopThreads();
     }
     }
   }
   }
+
   @Test
   @Test
   public void testRollMasterKey() throws Exception {
   public void testRollMasterKey() throws Exception {
     TestDelegationTokenSecretManager dtSecretManager = 
     TestDelegationTokenSecretManager dtSecretManager = 
@@ -226,6 +265,7 @@ public class TestDelegationToken {
       dtSecretManager.stopThreads();
       dtSecretManager.stopThreads();
     }
     }
   }
   }
+
   @Test
   @Test
   @SuppressWarnings("unchecked")
   @SuppressWarnings("unchecked")
   public void testDelegationTokenSelector() throws Exception {
   public void testDelegationTokenSelector() throws Exception {