Browse Source

YARN-1915. Fixed a race condition that client could use the ClientToAMToken to contact with AM before AM actually receives the ClientToAMTokenMasterKey. Contributed by Jason Lowe

Jian He 10 năm trước cách đây
mục cha
commit
5864dd99a4

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

@@ -742,6 +742,10 @@ Release 2.6.0 - UNRELEASED
     YARN-2724. Skipped uploading a local log file to HDFS if exception is raised
     when opening it. (Xuan Gong via zjshen)
 
+    YARN-1915. Fixed a race condition that client could use the ClientToAMToken
+    to contact with AM before AM actually receives the ClientToAMTokenMasterKey.
+    (Jason Lowe via jianhe)
+
 Release 2.5.1 - 2014-09-05
 
   INCOMPATIBLE CHANGES

+ 23 - 2
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/security/client/ClientToAMTokenSecretManager.java

@@ -36,9 +36,10 @@ import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
 @Evolving
 public class ClientToAMTokenSecretManager extends
     BaseClientToAMTokenSecretManager {
+  private static final int MASTER_KEY_WAIT_MSEC = 10 * 1000;
 
   // Only one master-key for AM
-  private SecretKey masterKey;
+  private volatile SecretKey masterKey;
 
   public ClientToAMTokenSecretManager(
       ApplicationAttemptId applicationAttemptID, byte[] key) {
@@ -51,6 +52,23 @@ public class ClientToAMTokenSecretManager extends
     
   }
 
+  @Override
+  public byte[] retrievePassword(ClientToAMTokenIdentifier identifier)
+      throws InvalidToken {
+    if (this.masterKey == null) {
+      synchronized (this) {
+        while (masterKey == null) {
+          try {
+            wait(MASTER_KEY_WAIT_MSEC);
+            break;
+          } catch (InterruptedException e) {
+          }
+        }
+      }
+    }
+    return super.retrievePassword(identifier);
+  }
+
   @Override
   public SecretKey getMasterKey(ApplicationAttemptId applicationAttemptID) {
     // Only one master-key for AM, just return that.
@@ -58,6 +76,9 @@ public class ClientToAMTokenSecretManager extends
   }
 
   public void setMasterKey(byte[] key) {
-    this.masterKey = SecretManager.createSecretKey(key);
+    synchronized (this) {
+      this.masterKey = SecretManager.createSecretKey(key);
+      notifyAll();
+    }
   }
 }

+ 121 - 1
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/security/TestClientToAMTokens.java

@@ -30,9 +30,12 @@ import static org.mockito.Mockito.when;
 import java.io.IOException;
 import java.lang.annotation.Annotation;
 import java.net.InetSocketAddress;
+import java.nio.ByteBuffer;
 import java.security.PrivilegedAction;
 import java.security.PrivilegedExceptionAction;
 import java.util.Arrays;
+import java.util.Timer;
+import java.util.TimerTask;
 
 import javax.security.sasl.SaslException;
 
@@ -169,6 +172,10 @@ public class TestClientToAMTokens extends ParameterizedSchedulerTestBase {
       this.address = NetUtils.getConnectAddress(server);
       super.serviceStart();
     }
+
+    public void setClientSecretKey(byte[] key) {
+      secretMgr.setMasterKey(key);
+    }
   }
 
   @Test
@@ -291,7 +298,7 @@ public class TestClientToAMTokens extends ParameterizedSchedulerTestBase {
     // Verify for a new version token
     verifyNewVersionToken(conf, am, token, rm);
 
-
+    am.stop();
     rm.stop();
   }
 
@@ -410,4 +417,117 @@ public class TestClientToAMTokens extends ParameterizedSchedulerTestBase {
       }
     });
   }
+
+  @Test(timeout=20000)
+  public void testClientTokenRace() throws Exception {
+
+    final Configuration conf = new Configuration();
+    conf.set(CommonConfigurationKeysPublic.HADOOP_SECURITY_AUTHENTICATION,
+      "kerberos");
+    UserGroupInformation.setConfiguration(conf);
+
+    ContainerManagementProtocol containerManager =
+        mock(ContainerManagementProtocol.class);
+    StartContainersResponse mockResponse = mock(StartContainersResponse.class);
+    when(containerManager.startContainers((StartContainersRequest) any()))
+      .thenReturn(mockResponse);
+    final DrainDispatcher dispatcher = new DrainDispatcher();
+
+    MockRM rm = new MockRMWithCustomAMLauncher(conf, containerManager) {
+      protected ClientRMService createClientRMService() {
+        return new ClientRMService(this.rmContext, scheduler,
+          this.rmAppManager, this.applicationACLsManager, this.queueACLsManager,
+          getRMContext().getRMDelegationTokenSecretManager());
+      };
+
+      @Override
+      protected Dispatcher createDispatcher() {
+        return dispatcher;
+      }
+
+      @Override
+      protected void doSecureLogin() throws IOException {
+      }
+    };
+    rm.start();
+
+    // Submit an app
+    RMApp app = rm.submitApp(1024);
+
+    // Set up a node.
+    MockNM nm1 = rm.registerNode("localhost:1234", 3072);
+    nm1.nodeHeartbeat(true);
+    dispatcher.await();
+
+    nm1.nodeHeartbeat(true);
+    dispatcher.await();
+
+    ApplicationAttemptId appAttempt = app.getCurrentAppAttempt().getAppAttemptId();
+    final MockAM mockAM =
+        new MockAM(rm.getRMContext(), rm.getApplicationMasterService(),
+            app.getCurrentAppAttempt().getAppAttemptId());
+    UserGroupInformation appUgi =
+        UserGroupInformation.createRemoteUser(appAttempt.toString());
+    RegisterApplicationMasterResponse response =
+        appUgi.doAs(new PrivilegedAction<RegisterApplicationMasterResponse>() {
+
+          @Override
+          public RegisterApplicationMasterResponse run() {
+            RegisterApplicationMasterResponse response = null;
+            try {
+              response = mockAM.registerAppAttempt();
+            } catch (Exception e) {
+              Assert.fail("Exception was not expected");
+            }
+            return response;
+          }
+        });
+
+    // Get the app-report.
+    GetApplicationReportRequest request =
+        Records.newRecord(GetApplicationReportRequest.class);
+    request.setApplicationId(app.getApplicationId());
+    GetApplicationReportResponse reportResponse =
+        rm.getClientRMService().getApplicationReport(request);
+    ApplicationReport appReport = reportResponse.getApplicationReport();
+    org.apache.hadoop.yarn.api.records.Token originalClientToAMToken =
+        appReport.getClientToAMToken();
+
+    // ClientToAMToken master key should have been received on register
+    // application master response.
+    final ByteBuffer clientMasterKey = response.getClientToAMTokenMasterKey();
+    Assert.assertNotNull(clientMasterKey);
+    Assert.assertTrue(clientMasterKey.array().length > 0);
+
+    // Start the AM with the correct shared-secret.
+    ApplicationAttemptId appAttemptId =
+        app.getAppAttempts().keySet().iterator().next();
+    Assert.assertNotNull(appAttemptId);
+    final CustomAM am = new CustomAM(appAttemptId, null);
+    am.init(conf);
+    am.start();
+
+    // Now the real test!
+    // Set up clients to be able to pick up correct tokens.
+    SecurityUtil.setSecurityInfoProviders(new CustomSecurityInfo());
+
+    Token<ClientToAMTokenIdentifier> token =
+        ConverterUtils.convertFromYarn(originalClientToAMToken, am.address);
+
+    // Schedule the key to be set after a significant delay
+    Timer timer = new Timer();
+    TimerTask timerTask = new TimerTask() {
+      @Override
+      public void run() {
+        am.setClientSecretKey(clientMasterKey.array());
+      }
+    };
+    timer.schedule(timerTask, 250);
+
+    // connect should pause waiting for the master key to arrive
+    verifyValidToken(conf, am, token);
+
+    am.stop();
+    rm.stop();
+  }
 }