Ver Fonte

HDFS-4477. Secondary namenode may retain old tokens. Contributed by Daryn Sharp.

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/branches/branch-0.23@1483513 13f79535-47bb-0310-9956-ffa450edef68
Kihwal Lee há 12 anos atrás
pai
commit
745b073473

+ 22 - 6
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/token/delegation/AbstractDelegationTokenSecretManager.java

@@ -27,8 +27,10 @@ import java.io.DataInputStream;
 import java.io.IOException;
 import java.util.Arrays;
 import java.util.HashMap;
+import java.util.HashSet;
 import java.util.Iterator;
 import java.util.Map;
+import java.util.Set;
 
 import javax.crypto.SecretKey;
 
@@ -140,6 +142,10 @@ extends AbstractDelegationTokenIdentifier>
     return;
   }
   
+  protected void logExpireToken(TokenIdent ident) throws IOException {
+    return;
+  }
+
   /** 
    * Update the current master key 
    * This is called once by startThreads before tokenRemoverThread is created, 
@@ -359,15 +365,25 @@ extends AbstractDelegationTokenIdentifier>
   }
   
   /** Remove expired delegation tokens from cache */
-  private synchronized void removeExpiredToken() {
+  private void removeExpiredToken() throws IOException {
     long now = System.currentTimeMillis();
-    Iterator<DelegationTokenInformation> i = currentTokens.values().iterator();
-    while (i.hasNext()) {
-      long renewDate = i.next().getRenewDate();
-      if (now > renewDate) {
-        i.remove();
+    Set<TokenIdent> expiredTokens = new HashSet<TokenIdent>();
+    synchronized (this) {
+      Iterator<Map.Entry<TokenIdent, DelegationTokenInformation>> i =
+          currentTokens.entrySet().iterator();
+      while (i.hasNext()) {
+        Map.Entry<TokenIdent, DelegationTokenInformation> entry = i.next();
+        long renewDate = entry.getValue().getRenewDate();
+        if (renewDate < now) {
+          expiredTokens.add(entry.getKey());
+          i.remove();
+        }
       }
     }
+    // don't hold lock on 'this' to avoid edit log updates blocking token ops
+    for (TokenIdent ident : expiredTokens) {
+      logExpireToken(ident);
+    }
   }
 
   public void stopThreads() {

+ 2 - 0
hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt

@@ -21,6 +21,8 @@ Release 0.23.8 - UNRELEASED
     use the stored generation stamp to check if the block is valid.  (Vinay
     via szetszwo)
 
+    HDFS-4477. Secondary namenode may retain old tokens. (daryn via kihwal)
+
 Release 0.23.7 - 2013-04-18
 
   INCOMPATIBLE CHANGES

+ 17 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/security/token/delegation/DelegationTokenSecretManager.java

@@ -297,6 +297,23 @@ public class DelegationTokenSecretManager
       namesystem.logUpdateMasterKey(key);
     }
   }
+  
+  @Override //AbstractDelegationTokenManager
+  protected void logExpireToken(final DelegationTokenIdentifier dtId)
+      throws IOException {
+    synchronized (noInterruptsLock) {
+      // The edit logging code will fail catastrophically if it
+      // is interrupted during a logSync, since the interrupt
+      // closes the edit log files. Doing this inside the
+      // above lock and then checking interruption status
+      // prevents this bug.
+      if (Thread.interrupted()) {
+        throw new InterruptedIOException(
+            "Interrupted before expiring delegation token");
+      }
+      namesystem.logExpireDelegationToken(dtId);
+    }
+  }
 
   /** A utility method for creating credentials. */
   public static Credentials createCredentials(final NameNode namenode,

+ 15 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java

@@ -4220,6 +4220,21 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
     getEditLog().logSync();
   }
   
+  /**
+   * Log the cancellation of expired tokens to edit logs
+   * 
+   * @param id token identifier to cancel
+   */
+  public void logExpireDelegationToken(DelegationTokenIdentifier id) {
+    assert !isInSafeMode() :
+      "this should never be called while in safemode, since we stop " +
+      "the DT manager before entering safemode!";
+    // No need to hold FSN lock since we don't access any internal
+    // structures, and this is stopped before the FSN shuts itself
+    // down, etc.
+    getEditLog().logCancelDelegationToken(id);
+  }  
+  
   private void logReassignLease(String leaseHolder, String src,
       String newHolder) throws IOException {
     assert hasWriteLock();

+ 78 - 0
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestSecurityTokenEditLog.java

@@ -17,6 +17,8 @@
  */
 package org.apache.hadoop.hdfs.server.namenode;
 
+import static org.apache.hadoop.hdfs.DFSConfigKeys.*;
+
 import junit.framework.TestCase;
 import java.io.*;
 import java.net.URI;
@@ -27,6 +29,7 @@ import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier;
+import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenSecretManager;
 import org.apache.hadoop.hdfs.server.namenode.EditLogFileInputStream;
 import org.apache.hadoop.hdfs.server.common.Storage.StorageDirectory;
 import org.apache.hadoop.hdfs.server.namenode.NNStorage.NameNodeDirType;
@@ -34,6 +37,9 @@ import org.apache.hadoop.io.Text;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.token.Token;
 
+import static org.apache.hadoop.hdfs.DFSConfigKeys.*;
+import static org.mockito.Mockito.*;
+
 /**
  * This class tests the creation and validation of a checkpoint.
  */
@@ -150,4 +156,76 @@ public class TestSecurityTokenEditLog extends TestCase {
       if(cluster != null) cluster.shutdown();
     }
   }
+
+  public void testEditsForCancelOnTokenExpire() throws IOException,
+  InterruptedException {
+    long renewInterval = 2000;
+    Configuration conf = new Configuration();
+    conf.setBoolean(DFS_NAMENODE_DELEGATION_TOKEN_ALWAYS_USE_KEY, true);
+    conf.setLong(DFS_NAMENODE_DELEGATION_TOKEN_RENEW_INTERVAL_KEY, renewInterval);
+    conf.setLong(DFS_NAMENODE_DELEGATION_TOKEN_MAX_LIFETIME_KEY, renewInterval*2);
+
+    Text renewer = new Text(UserGroupInformation.getCurrentUser().getUserName());
+    FSImage fsImage = mock(FSImage.class);
+    FSEditLog log = mock(FSEditLog.class);
+    doReturn(log).when(fsImage).getEditLog();   
+    FSNamesystem fsn = new FSNamesystem(fsImage, conf);
+    
+    DelegationTokenSecretManager dtsm = fsn.getDelegationTokenSecretManager();
+    try {
+      dtsm.startThreads();
+      
+      // get two tokens
+      Token<DelegationTokenIdentifier> token1 = fsn.getDelegationToken(renewer);
+      Token<DelegationTokenIdentifier> token2 = fsn.getDelegationToken(renewer);
+      DelegationTokenIdentifier ident1 = decodeIdentifier(token1);
+      DelegationTokenIdentifier ident2 = decodeIdentifier(token2);
+      
+      // verify we got the tokens
+      verify(log, times(1)).logGetDelegationToken(eq(ident1), anyLong());
+      verify(log, times(1)).logGetDelegationToken(eq(ident2), anyLong());
+      
+      // this is a little tricky because DTSM doesn't let us set scan interval
+      // so need to periodically sleep, then stop/start threads to force scan
+      
+      // renew first token 1/2 to expire
+      Thread.sleep(renewInterval/2);
+      fsn.renewDelegationToken(token2);
+      verify(log, times(1)).logRenewDelegationToken(eq(ident2), anyLong());
+      // force scan and give it a little time to complete
+      dtsm.stopThreads(); dtsm.startThreads();
+      Thread.sleep(250);
+      // no token has expired yet 
+      verify(log, times(0)).logCancelDelegationToken(eq(ident1));
+      verify(log, times(0)).logCancelDelegationToken(eq(ident2));
+      
+      // sleep past expiration of 1st non-renewed token
+      Thread.sleep(renewInterval/2);
+      dtsm.stopThreads(); dtsm.startThreads();
+      Thread.sleep(250);
+      // non-renewed token should have implicitly been cancelled
+      verify(log, times(1)).logCancelDelegationToken(eq(ident1));
+      verify(log, times(0)).logCancelDelegationToken(eq(ident2));
+      
+      // sleep past expiration of 2nd renewed token
+      Thread.sleep(renewInterval/2);
+      dtsm.stopThreads(); dtsm.startThreads();
+      Thread.sleep(250);
+      // both tokens should have been implicitly cancelled by now
+      verify(log, times(1)).logCancelDelegationToken(eq(ident1));
+      verify(log, times(1)).logCancelDelegationToken(eq(ident2));
+    } finally {
+      dtsm.stopThreads();
+    }
+  }
+
+  private static DelegationTokenIdentifier decodeIdentifier(Token<?> token)
+      throws IOException {
+    DelegationTokenIdentifier ident = new DelegationTokenIdentifier();
+    ByteArrayInputStream buf = new ByteArrayInputStream(token.getIdentifier());
+    DataInputStream in = new DataInputStream(buf);  
+    ident.readFields(in);
+    in.close();
+    return ident;
+  }
 }