فهرست منبع

MAPREDUCE-2452. Moves the cancellation of delegation tokens to a separate
thread. (ddas)


git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/branches/branch-0.20-security-204@1136736 13f79535-47bb-0310-9956-ffa450edef68

Owen O'Malley 14 سال پیش
والد
کامیت
6ae35982e4
2فایلهای تغییر یافته به همراه71 افزوده شده و 9 حذف شده
  1. 3 0
      CHANGES.txt
  2. 68 9
      src/mapred/org/apache/hadoop/mapreduce/security/token/DelegationTokenRenewal.java

+ 3 - 0
CHANGES.txt

@@ -14,6 +14,9 @@ Release 0.20.204.0 - unreleased
     MAPREDUCE-2495. exit() the TaskTracker when the distributed cache cleanup
     thread dies. (Robert Joseph Evans via cdouglas)
 
+    MAPREDUCE-2452. Moves the cancellation of delegation tokens to a separate
+    thread. (ddas)
+
     MAPREDUCE-2555. Avoid sprious logging from completedtasks. (Thomas Graves
     via cdouglas)
 

+ 68 - 9
src/mapred/org/apache/hadoop/mapreduce/security/token/DelegationTokenRenewal.java

@@ -33,6 +33,7 @@ import java.util.Iterator;
 import java.util.Set;
 import java.util.Timer;
 import java.util.TimerTask;
+import java.util.concurrent.LinkedBlockingQueue;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
@@ -48,6 +49,7 @@ import org.apache.hadoop.security.Credentials;
 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.util.StringUtils;
 
 
 //@InterfaceAudience.Private
@@ -100,10 +102,75 @@ public class DelegationTokenRenewal {
   // global single timer (daemon)
   private static Timer renewalTimer = new Timer(true);
   
+  //delegation token canceler thread
+  private static DelegationTokenCancelThread dtCancelThread =
+    new DelegationTokenCancelThread();
+  static {
+    dtCancelThread.start();
+  }
+
+  
   //managing the list of tokens using Map
   // jobId=>List<tokens>
   private static Set<DelegationTokenToRenew> delegationTokens = 
     Collections.synchronizedSet(new HashSet<DelegationTokenToRenew>());
+  
+  private static class DelegationTokenCancelThread extends Thread {
+    private static class TokenWithConf {
+      Token<DelegationTokenIdentifier> token;
+      Configuration conf;
+      TokenWithConf(Token<DelegationTokenIdentifier> token,  
+          Configuration conf) {
+        this.token = token;
+        this.conf = conf;
+      }
+    }
+    private LinkedBlockingQueue<TokenWithConf> queue =  
+      new LinkedBlockingQueue<TokenWithConf>();
+     
+    public DelegationTokenCancelThread() {
+      super("Delegation Token Canceler");
+      setDaemon(true);
+    }
+    public void cancelToken(Token<DelegationTokenIdentifier> token,  
+        Configuration conf) {
+      TokenWithConf tokenWithConf = new TokenWithConf(token, conf);
+      while (!queue.offer(tokenWithConf)) {
+        LOG.warn("Unable to add token " + token + " for cancellation. " +
+        		 "Will retry..");
+        try {
+          Thread.sleep(100);
+        } catch (InterruptedException e) {
+          throw new RuntimeException(e);
+        }
+      }
+    }
+
+    public void run() {
+      while (true) {
+        TokenWithConf tokenWithConf = null;
+        try {
+          tokenWithConf = queue.take();
+          DistributedFileSystem dfs = getDFSForToken(tokenWithConf.token,  
+              tokenWithConf.conf);
+          if (LOG.isDebugEnabled()) {
+            LOG.debug("Canceling token " + tokenWithConf.token.getService() +  
+                " for dfs=" + dfs);
+          }
+          dfs.cancelDelegationToken(tokenWithConf.token);
+        } catch (IOException e) {
+          LOG.warn("Failed to cancel token " + tokenWithConf.token + " " +  
+              StringUtils.stringifyException(e));
+        } catch (InterruptedException ie) {
+          return;
+        } catch (Throwable t) {
+          LOG.warn("Got exception " + StringUtils.stringifyException(t) + 
+                   ". Exiting..");
+          System.exit(-1);
+        }
+      }
+    }
+  }
   //adding token
   private static void addTokenToList(DelegationTokenToRenew t) {
     delegationTokens.add(t);
@@ -309,15 +376,7 @@ public class DelegationTokenRenewal {
     Configuration conf = t.conf;
     
     if(token.getKind().equals(kindHdfs)) {
-      try {
-        DistributedFileSystem dfs = getDFSForToken(token, conf);
-        if (LOG.isDebugEnabled())
-          LOG.debug("canceling token " + token.getService() + " for dfs=" +
-              dfs);
-        dfs.cancelDelegationToken(token);
-      } catch (Exception e) {
-        LOG.warn("Failed to cancel " + token, e);
-      }
+      dtCancelThread.cancelToken(token, conf);
     }
   }