Selaa lähdekoodia

Merge r1523149 from branch-2 to branch-2.1-beta YARN-1116. Populate AMRMTokens back to AMRMTokenSecretManager after RM restarts (Jian He via bikas)

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/branches/branch-2.1-beta@1523151 13f79535-47bb-0310-9956-ffa450edef68
Bikas Saha 11 vuotta sitten
vanhempi
commit
3dfefe5f9f

+ 0 - 9
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/rm/RMCommunicator.java

@@ -237,15 +237,6 @@ public abstract class RMCommunicator extends AbstractService
             } catch (YarnRuntimeException e) {
               LOG.error("Error communicating with RM: " + e.getMessage() , e);
               return;
-            } catch (InvalidToken e) {
-              // This can happen if the RM has been restarted, since currently
-              // when RM restarts AMRMToken is not populated back to
-              // AMRMTokenSecretManager yet. Once this is fixed, no need
-              // to send JOB_AM_REBOOT event in this method any more.
-              eventHandler.handle(new JobEvent(job.getID(),
-                JobEventType.JOB_AM_REBOOT));
-              LOG.error("Error in authencating with RM: " ,e);
-              return;
             } catch (Exception e) {
               LOG.error("ERROR IN CONTACTING RM. ", e);
               continue;

+ 3 - 0
hadoop-yarn-project/CHANGES.txt

@@ -159,6 +159,9 @@ Release 2.1.1-beta - UNRELEASED
     YARN-1194. TestContainerLogsPage fails with native builds (Roman Shaposhnik
     via jlowe)
 
+    YARN-1116. Populate AMRMTokens back to AMRMTokenSecretManager after RM
+    restarts (Jian He via bikas)
+
 Release 2.1.0-beta - 2013-08-22
 
   INCOMPATIBLE CHANGES

+ 1 - 1
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/RMAppImpl.java

@@ -569,7 +569,7 @@ public class RMAppImpl implements RMApp, Recoverable {
   }
   
   @Override
-  public void recover(RMState state) {
+  public void recover(RMState state) throws Exception{
     ApplicationState appState = state.getApplicationState().get(getApplicationId());
     LOG.info("Recovering app: " + getApplicationId() + " with " + 
             + appState.getAttemptCount() + " attempts");

+ 5 - 7
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/attempt/RMAppAttemptImpl.java

@@ -20,6 +20,7 @@ package org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt;
 
 import static org.apache.hadoop.yarn.util.StringHelper.pjoin;
 
+import java.io.IOException;
 import java.net.URI;
 import java.net.URISyntaxException;
 import java.util.ArrayList;
@@ -675,7 +676,7 @@ public class RMAppAttemptImpl implements RMAppAttempt, Recoverable {
   }
 
   @Override
-  public void recover(RMState state) {
+  public void recover(RMState state) throws Exception{
     ApplicationState appState = 
         state.getApplicationState().get(getAppAttemptId().getApplicationId());
     ApplicationAttemptState attemptState = appState.getAttempt(getAppAttemptId());
@@ -690,7 +691,8 @@ public class RMAppAttemptImpl implements RMAppAttempt, Recoverable {
                                  RMAppAttemptEventType.RECOVER));
   }
 
-  private void recoverAppAttemptCredentials(Credentials appAttemptTokens) {
+  private void recoverAppAttemptCredentials(Credentials appAttemptTokens)
+      throws IOException {
     if (appAttemptTokens == null) {
       return;
     }
@@ -707,11 +709,7 @@ public class RMAppAttemptImpl implements RMAppAttempt, Recoverable {
     this.amrmToken =
         (Token<AMRMTokenIdentifier>) appAttemptTokens
           .getToken(RMStateStore.AM_RM_TOKEN_SERVICE);
-
-    // For now, no need to populate tokens back to AMRMTokenSecretManager,
-    // because running attempts are rebooted. Later in work-preserve restart,
-    // we'll create NEW->RUNNING transition in which the restored tokens will be
-    // added to the secret manager
+    rmContext.getAMRMTokenSecretManager().addPersistedPassword(this.amrmToken);
   }
 
   private static class BaseTransition implements

+ 15 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/security/AMRMTokenSecretManager.java

@@ -18,6 +18,7 @@
 
 package org.apache.hadoop.yarn.server.resourcemanager.security;
 
+import java.io.IOException;
 import java.util.HashMap;
 import java.util.Map;
 import java.util.Timer;
@@ -30,6 +31,7 @@ import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.classification.InterfaceAudience.Private;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.security.token.SecretManager;
+import org.apache.hadoop.security.token.Token;
 import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.security.AMRMTokenIdentifier;
@@ -123,6 +125,19 @@ public class AMRMTokenSecretManager extends
     return password;
   }
 
+  /**
+   * Populate persisted password of AMRMToken back to AMRMTokenSecretManager.
+   */
+  public synchronized void
+      addPersistedPassword(Token<AMRMTokenIdentifier> token) throws IOException {
+    AMRMTokenIdentifier identifier = token.decodeIdentifier();
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("Adding password for " + identifier.getApplicationAttemptId());
+    }
+    this.passwords.put(identifier.getApplicationAttemptId(),
+      token.getPassword());
+  }
+
   /**
    * Retrieve the password for the given {@link AMRMTokenIdentifier}.
    * Used by RPC layer to validate a remote {@link AMRMTokenIdentifier}.

+ 7 - 4
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRMRestart.java

@@ -52,6 +52,7 @@ import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.hadoop.yarn.api.records.ContainerState;
 import org.apache.hadoop.yarn.api.records.ResourceRequest;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
+import org.apache.hadoop.yarn.security.AMRMTokenIdentifier;
 import org.apache.hadoop.yarn.security.client.RMDelegationTokenIdentifier;
 import org.apache.hadoop.yarn.server.api.protocolrecords.NodeHeartbeatResponse;
 import org.apache.hadoop.yarn.server.api.records.NodeAction;
@@ -577,14 +578,16 @@ public class TestRMRestart {
         attempt1.getClientTokenMasterKey(),
         loadedAttempt1.getClientTokenMasterKey());
 
-    // assert secret manager also knows about the key
+    // assert ClientTokenSecretManager also knows about the key
     Assert.assertArrayEquals(clientTokenMasterKey,
         rm2.getClientToAMTokenSecretManager().getMasterKey(attemptId1)
             .getEncoded());
 
-    // Not testing ApplicationTokenSecretManager has the password populated back,
-    // that is needed in work-preserving restart
-
+    // assert AMRMTokenSecretManager also knows about the AMRMToken password
+    Token<AMRMTokenIdentifier> amrmToken = loadedAttempt1.getAMRMToken();
+    Assert.assertArrayEquals(amrmToken.getPassword(),
+      rm2.getAMRMTokenSecretManager().retrievePassword(
+        amrmToken.decodeIdentifier()));
     rm1.stop();
     rm2.stop();
   }