Explorar o código

YARN-11478. [Federation] SQLFederationStateStore Support Store ApplicationSubmitData. (#5663)

slfan1989 hai 1 ano
pai
achega
b977065cc4
Modificáronse 10 ficheiros con 126 adicións e 101 borrados
  1. 12 7
      hadoop-yarn-project/hadoop-yarn/bin/FederationStateStore/MySQL/FederationStateStoreStoredProcs.sql
  2. 2 1
      hadoop-yarn-project/hadoop-yarn/bin/FederationStateStore/MySQL/FederationStateStoreTables.sql
  3. 16 5
      hadoop-yarn-project/hadoop-yarn/bin/FederationStateStore/SQLServer/FederationStateStoreStoredProcs.sql
  4. 1 1
      hadoop-yarn-project/hadoop-yarn/bin/FederationStateStore/SQLServer/FederationStateStoreTables.sql
  5. 53 13
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/impl/SQLFederationStateStore.java
  6. 2 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/records/impl/pb/ApplicationHomeSubClusterPBImpl.java
  7. 26 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/store/impl/FederationStateStoreBaseTest.java
  8. 14 6
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/store/impl/HSQLDBFederationStateStore.java
  9. 0 34
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/store/impl/TestMemoryFederationStateStore.java
  10. 0 33
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/store/impl/TestZookeeperFederationStateStore.java

+ 12 - 7
hadoop-yarn-project/hadoop-yarn/bin/FederationStateStore/MySQL/FederationStateStoreStoredProcs.sql

@@ -18,7 +18,7 @@
 
 -- Script to generate all the stored procedures for the Federation StateStore in MySQL
 
-USE FederationStateStore
+USE FederationStateStore;
 
 DELIMITER //
 
@@ -89,11 +89,12 @@ END //
 
 CREATE PROCEDURE sp_addApplicationHomeSubCluster(
    IN applicationId_IN varchar(64), IN homeSubCluster_IN varchar(256),
+   IN applicationContext_IN BLOB,
    OUT storedHomeSubCluster_OUT varchar(256), OUT rowCount_OUT int)
 BEGIN
    INSERT INTO applicationsHomeSubCluster
-      (applicationId,homeSubCluster)
-      (SELECT applicationId_IN, homeSubCluster_IN
+      (applicationId, homeSubCluster, createTime, applicationContext)
+      (SELECT applicationId_IN, homeSubCluster_IN, NOW(), applicationContext_IN
        FROM applicationsHomeSubCluster
        WHERE applicationId = applicationId_IN
        HAVING COUNT(*) = 0 );
@@ -105,19 +106,23 @@ END //
 
 CREATE PROCEDURE sp_updateApplicationHomeSubCluster(
    IN applicationId_IN varchar(64),
-   IN homeSubCluster_IN varchar(256), OUT rowCount_OUT int)
+   IN homeSubCluster_IN varchar(256), IN applicationContext_IN BLOB, OUT rowCount_OUT int)
 BEGIN
    UPDATE applicationsHomeSubCluster
-     SET homeSubCluster = homeSubCluster_IN
+     SET homeSubCluster = homeSubCluster_IN,
+         applicationContext = applicationContext_IN
    WHERE applicationId = applicationId_IN;
    SELECT ROW_COUNT() INTO rowCount_OUT;
 END //
 
 CREATE PROCEDURE sp_getApplicationHomeSubCluster(
    IN applicationId_IN varchar(64),
-   OUT homeSubCluster_OUT varchar(256))
+   OUT homeSubCluster_OUT varchar(256),
+   OUT createTime_OUT datetime,
+   OUT applicationContext_OUT BLOB)
 BEGIN
-   SELECT homeSubCluster INTO homeSubCluster_OUT
+   SELECT homeSubCluster, applicationContext, createTime
+       INTO homeSubCluster_OUT, applicationContext_OUT, createTime_OUT
    FROM applicationsHomeSubCluster
    WHERE applicationId = applicationID_IN;
 END //

+ 2 - 1
hadoop-yarn-project/hadoop-yarn/bin/FederationStateStore/MySQL/FederationStateStoreTables.sql

@@ -18,12 +18,13 @@
 
 -- Script to generate all the tables for the Federation StateStore in MySQL
 
-USE FederationStateStore
+USE FederationStateStore;
 
 CREATE TABLE applicationsHomeSubCluster(
    applicationId varchar(64) NOT NULL,
    homeSubCluster varchar(256) NOT NULL,
    createTime datetime NOT NULL,
+   applicationContext BLOB NULL,
    CONSTRAINT pk_applicationId PRIMARY KEY (applicationId)
 );
 

+ 16 - 5
hadoop-yarn-project/hadoop-yarn/bin/FederationStateStore/SQLServer/FederationStateStoreStoredProcs.sql

@@ -26,6 +26,7 @@ GO
 CREATE PROCEDURE [dbo].[sp_addApplicationHomeSubCluster]
     @applicationId_IN VARCHAR(64),
     @homeSubCluster_IN VARCHAR(256),
+    @applicationContext_IN VARBINARY(MAX),
     @storedHomeSubCluster_OUT VARCHAR(256) OUTPUT,
     @rowCount_OUT int OUTPUT
 AS BEGIN
@@ -41,10 +42,14 @@ AS BEGIN
 
                 INSERT INTO [dbo].[applicationsHomeSubCluster] (
                     [applicationId],
-                    [homeSubCluster])
+                    [homeSubCluster],
+                    [createTime],
+                    [applicationContext])
                 VALUES (
                     @applicationId_IN,
-                    @homeSubCluster_IN);
+                    @homeSubCluster_IN,
+                    GETUTCDATE(),
+                    @applicationContext_IN);
             -- End of the IF block
 
             SELECT @rowCount_OUT = @@ROWCOUNT;
@@ -77,6 +82,7 @@ GO
 CREATE PROCEDURE [dbo].[sp_updateApplicationHomeSubCluster]
     @applicationId_IN VARCHAR(64),
     @homeSubCluster_IN VARCHAR(256),
+    @applicationContext_IN VARBINARY(MAX),
     @rowCount_OUT int OUTPUT
 AS BEGIN
     DECLARE @errorMessage nvarchar(4000)
@@ -85,7 +91,8 @@ AS BEGIN
         BEGIN TRAN
 
             UPDATE [dbo].[applicationsHomeSubCluster]
-            SET [homeSubCluster] = @homeSubCluster_IN
+            SET [homeSubCluster] = @homeSubCluster_IN,
+                [applicationContext] = @applicationContext_IN
             WHERE [applicationId] = @applicationId_IN;
             SELECT @rowCount_OUT = @@ROWCOUNT;
 
@@ -151,13 +158,17 @@ GO
 
 CREATE PROCEDURE [dbo].[sp_getApplicationHomeSubCluster]
     @applicationId_IN VARCHAR(64),
-    @homeSubCluster_OUT VARCHAR(256) OUTPUT
+    @homeSubCluster_OUT VARCHAR(256) OUTPUT,
+    @createTime_OUT datetime OUT,
+    @applicationContext_OUT VARBINARY(MAX) OUTPUT
 AS BEGIN
     DECLARE @errorMessage nvarchar(4000)
 
     BEGIN TRY
 
-        SELECT @homeSubCluster_OUT = [homeSubCluster]
+        SELECT @homeSubCluster_OUT = [homeSubCluster],
+            @createTime_OUT = [createTime],
+            @applicationContext_OUT = [applicationContext]
         FROM [dbo].[applicationsHomeSubCluster]
         WHERE [applicationId] = @applicationId_IN;
 

+ 1 - 1
hadoop-yarn-project/hadoop-yarn/bin/FederationStateStore/SQLServer/FederationStateStoreTables.sql

@@ -35,7 +35,7 @@ IF NOT EXISTS ( SELECT * FROM [FederationStateStore].sys.tables
             applicationId   VARCHAR(64) COLLATE Latin1_General_100_BIN2 NOT NULL,
             homeSubCluster  VARCHAR(256) NOT NULL,
             createTime      DATETIME2 NOT NULL CONSTRAINT ts_createAppTime DEFAULT GETUTCDATE(),
-
+            applicationContext VARBINARY(MAX) NULL,
             CONSTRAINT [pk_applicationId] PRIMARY KEY
             (
                 [applicationId]

+ 53 - 13
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/impl/SQLFederationStateStore.java

@@ -18,13 +18,16 @@
 
 package org.apache.hadoop.yarn.server.federation.store.impl;
 
+import java.io.ByteArrayInputStream;
 import java.io.IOException;
 import java.nio.ByteBuffer;
 import java.sql.CallableStatement;
 import java.sql.Connection;
 import java.sql.ResultSet;
 import java.sql.SQLException;
+import java.sql.Blob;
 import java.sql.Timestamp;
+import java.sql.Types;
 import java.util.ArrayList;
 import java.util.Calendar;
 import java.util.List;
@@ -35,10 +38,13 @@ import org.apache.commons.lang3.StringUtils;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.security.token.delegation.DelegationKey;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
+import org.apache.hadoop.yarn.api.records.ApplicationSubmissionContext;
 import org.apache.hadoop.yarn.api.records.ReservationId;
+import org.apache.hadoop.yarn.api.records.impl.pb.ApplicationSubmissionContextPBImpl;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.exceptions.YarnException;
 import org.apache.hadoop.yarn.proto.YarnServerCommonProtos;
+import org.apache.hadoop.yarn.proto.YarnProtos.ApplicationSubmissionContextProto;
 import org.apache.hadoop.yarn.security.client.YARNDelegationTokenIdentifier;
 import org.apache.hadoop.yarn.server.federation.store.FederationStateStore;
 import org.apache.hadoop.yarn.server.federation.store.exception.FederationStateStoreInvalidInputException;
@@ -145,16 +151,16 @@ public class SQLFederationStateStore implements FederationStateStore {
       "{call sp_subClusterHeartbeat(?, ?, ?, ?)}";
 
   private static final String CALL_SP_ADD_APPLICATION_HOME_SUBCLUSTER =
-      "{call sp_addApplicationHomeSubCluster(?, ?, ?, ?)}";
+      "{call sp_addApplicationHomeSubCluster(?, ?, ?, ?, ?)}";
 
   private static final String CALL_SP_UPDATE_APPLICATION_HOME_SUBCLUSTER =
-      "{call sp_updateApplicationHomeSubCluster(?, ?, ?)}";
+      "{call sp_updateApplicationHomeSubCluster(?, ?, ?, ?)}";
 
   private static final String CALL_SP_DELETE_APPLICATION_HOME_SUBCLUSTER =
       "{call sp_deleteApplicationHomeSubCluster(?, ?)}";
 
   private static final String CALL_SP_GET_APPLICATION_HOME_SUBCLUSTER =
-      "{call sp_getApplicationHomeSubCluster(?, ?)}";
+      "{call sp_getApplicationHomeSubCluster(?, ?, ?, ?)}";
 
   private static final String CALL_SP_GET_APPLICATIONS_HOME_SUBCLUSTER =
       "{call sp_getApplicationsHomeSubCluster(?, ?)}";
@@ -610,10 +616,12 @@ public class SQLFederationStateStore implements FederationStateStore {
     CallableStatement cstmt = null;
 
     String subClusterHome = null;
-    ApplicationId appId =
-        request.getApplicationHomeSubCluster().getApplicationId();
-    SubClusterId subClusterId =
-        request.getApplicationHomeSubCluster().getHomeSubCluster();
+    ApplicationHomeSubCluster applicationHomeSubCluster =
+        request.getApplicationHomeSubCluster();
+    ApplicationId appId = applicationHomeSubCluster.getApplicationId();
+    SubClusterId subClusterId = applicationHomeSubCluster.getHomeSubCluster();
+    ApplicationSubmissionContext appSubmissionContext =
+        applicationHomeSubCluster.getApplicationSubmissionContext();
 
     try {
       cstmt = getCallableStatement(CALL_SP_ADD_APPLICATION_HOME_SUBCLUSTER);
@@ -621,6 +629,12 @@ public class SQLFederationStateStore implements FederationStateStore {
       // Set the parameters for the stored procedure
       cstmt.setString("applicationId_IN", appId.toString());
       cstmt.setString("homeSubCluster_IN", subClusterId.getId());
+      if (appSubmissionContext != null) {
+        cstmt.setBlob("applicationContext_IN", new ByteArrayInputStream(
+            ((ApplicationSubmissionContextPBImpl) appSubmissionContext).getProto().toByteArray()));
+      } else {
+        cstmt.setNull("applicationContext_IN", Types.BLOB);
+      }
       cstmt.registerOutParameter("storedHomeSubCluster_OUT", VARCHAR);
       cstmt.registerOutParameter("rowCount_OUT", INTEGER);
 
@@ -687,10 +701,12 @@ public class SQLFederationStateStore implements FederationStateStore {
 
     CallableStatement cstmt = null;
 
-    ApplicationId appId =
-        request.getApplicationHomeSubCluster().getApplicationId();
-    SubClusterId subClusterId =
-        request.getApplicationHomeSubCluster().getHomeSubCluster();
+    ApplicationHomeSubCluster applicationHomeSubCluster =
+        request.getApplicationHomeSubCluster();
+    ApplicationId appId = applicationHomeSubCluster.getApplicationId();
+    SubClusterId subClusterId = applicationHomeSubCluster.getHomeSubCluster();
+    ApplicationSubmissionContext appSubmissionContext =
+        applicationHomeSubCluster.getApplicationSubmissionContext();
 
     try {
       cstmt = getCallableStatement(CALL_SP_UPDATE_APPLICATION_HOME_SUBCLUSTER);
@@ -698,6 +714,12 @@ public class SQLFederationStateStore implements FederationStateStore {
       // Set the parameters for the stored procedure
       cstmt.setString("applicationId_IN", appId.toString());
       cstmt.setString("homeSubCluster_IN", subClusterId.getId());
+      if (appSubmissionContext != null) {
+        cstmt.setBlob("applicationContext_IN", new ByteArrayInputStream(
+            ((ApplicationSubmissionContextPBImpl) appSubmissionContext).getProto().toByteArray()));
+      } else {
+        cstmt.setNull("applicationContext_IN", Types.BLOB);
+      }
       cstmt.registerOutParameter("rowCount_OUT", INTEGER);
 
       // Execute the query
@@ -742,8 +764,9 @@ public class SQLFederationStateStore implements FederationStateStore {
     CallableStatement cstmt = null;
 
     SubClusterId homeRM = null;
-
+    Long createTime = 0L;
     ApplicationId applicationId = request.getApplicationId();
+    ApplicationSubmissionContext appSubmissionContext = null;
 
     try {
       cstmt = getCallableStatement(CALL_SP_GET_APPLICATION_HOME_SUBCLUSTER);
@@ -751,6 +774,8 @@ public class SQLFederationStateStore implements FederationStateStore {
       // Set the parameters for the stored procedure
       cstmt.setString("applicationId_IN", applicationId.toString());
       cstmt.registerOutParameter("homeSubCluster_OUT", VARCHAR);
+      cstmt.registerOutParameter("createTime_OUT", java.sql.Types.TIMESTAMP);
+      cstmt.registerOutParameter("applicationContext_OUT", Types.BLOB);
 
       // Execute the query
       long startTime = clock.getTime();
@@ -765,6 +790,15 @@ public class SQLFederationStateStore implements FederationStateStore {
             "Application %s does not exist.", applicationId);
       }
 
+      Timestamp createTimeStamp = cstmt.getTimestamp("createTime_OUT", utcCalendar);
+      createTime = createTimeStamp != null ? createTimeStamp.getTime() : 0;
+
+      Blob blobAppContextData = cstmt.getBlob("applicationContext_OUT");
+      if (blobAppContextData != null && request.getContainsAppSubmissionContext()) {
+        appSubmissionContext = new ApplicationSubmissionContextPBImpl(
+            ApplicationSubmissionContextProto.parseFrom(blobAppContextData.getBinaryStream()));
+      }
+
       LOG.debug("Got the information about the specified application {}."
           + " The AM is running in {}", applicationId, homeRM);
 
@@ -775,11 +809,17 @@ public class SQLFederationStateStore implements FederationStateStore {
       FederationStateStoreUtils.logAndThrowRetriableException(e, LOG,
           "Unable to obtain the application information for the specified application %s.",
           applicationId);
+    } catch (IOException e) {
+      FederationStateStoreClientMetrics.failedStateStoreCall();
+      FederationStateStoreUtils.logAndThrowRetriableException(e, LOG,
+          "Unable to obtain the application information for the specified application %s.",
+          applicationId);
     } finally {
       // Return to the pool the CallableStatement
       FederationStateStoreUtils.returnToPool(LOG, cstmt);
     }
-    return GetApplicationHomeSubClusterResponse.newInstance(request.getApplicationId(), homeRM);
+    return GetApplicationHomeSubClusterResponse.newInstance(applicationId, homeRM,
+        createTime, appSubmissionContext);
   }
 
   @Override

+ 2 - 1
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/records/impl/pb/ApplicationHomeSubClusterPBImpl.java

@@ -183,8 +183,9 @@ public class ApplicationHomeSubClusterPBImpl extends ApplicationHomeSubCluster {
   @Override
   public void setApplicationSubmissionContext(ApplicationSubmissionContext context) {
     maybeInitBuilder();
-    if (applicationSubmissionContext == null) {
+    if (context == null) {
       builder.clearAppSubmitContext();
+      return;
     }
     this.applicationSubmissionContext = context;
     builder.setAppSubmitContext(convertToProtoFormat(context));

+ 26 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/store/impl/FederationStateStoreBaseTest.java

@@ -34,6 +34,8 @@ import org.apache.hadoop.util.Time;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.api.records.ReservationId;
 import org.apache.hadoop.yarn.api.records.ApplicationSubmissionContext;
+import org.apache.hadoop.yarn.api.records.Priority;
+import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.exceptions.YarnException;
 import org.apache.hadoop.yarn.security.client.RMDelegationTokenIdentifier;
 import org.apache.hadoop.yarn.server.federation.store.FederationStateStore;
@@ -1086,4 +1088,28 @@ public abstract class FederationStateStoreBaseTest {
   public void testCheckVersion() throws Exception {
     stateStore.checkVersion();
   }
+
+  @Test
+  public void testGetApplicationHomeSubClusterWithContext() throws Exception {
+    FederationStateStore federationStateStore = this.getStateStore();
+
+    ApplicationId appId = ApplicationId.newInstance(1, 3);
+    SubClusterId subClusterId = SubClusterId.newInstance("SC");
+    ApplicationSubmissionContext context =
+        ApplicationSubmissionContext.newInstance(appId, "test", "default",
+        Priority.newInstance(0), null, true, true,
+        2, Resource.newInstance(10, 2), "test");
+    addApplicationHomeSC(appId, subClusterId, context);
+
+    GetApplicationHomeSubClusterRequest getRequest =
+         GetApplicationHomeSubClusterRequest.newInstance(appId, true);
+    GetApplicationHomeSubClusterResponse result =
+         federationStateStore.getApplicationHomeSubCluster(getRequest);
+
+    ApplicationHomeSubCluster applicationHomeSubCluster = result.getApplicationHomeSubCluster();
+
+    assertEquals(appId, applicationHomeSubCluster.getApplicationId());
+    assertEquals(subClusterId, applicationHomeSubCluster.getHomeSubCluster());
+    assertEquals(context, applicationHomeSubCluster.getApplicationSubmissionContext());
+  }
 }

+ 14 - 6
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/store/impl/HSQLDBFederationStateStore.java

@@ -52,6 +52,7 @@ public class HSQLDBFederationStateStore extends SQLFederationStateStore {
           + " applicationId varchar(64) NOT NULL,"
           + " homeSubCluster varchar(256) NOT NULL,"
           + " createTime datetime NOT NULL,"
+          + " applicationContext BLOB NULL,"
           + " CONSTRAINT pk_applicationId PRIMARY KEY (applicationId))";
 
   private static final String TABLE_MEMBERSHIP =
@@ -173,12 +174,14 @@ public class HSQLDBFederationStateStore extends SQLFederationStateStore {
       "CREATE PROCEDURE sp_addApplicationHomeSubCluster("
           + " IN applicationId_IN varchar(64),"
           + " IN homeSubCluster_IN varchar(256),"
+          + " IN applicationContext_IN BLOB,"
           + " OUT storedHomeSubCluster_OUT varchar(256), OUT rowCount_OUT int)"
           + " MODIFIES SQL DATA BEGIN ATOMIC"
           + " INSERT INTO applicationsHomeSubCluster "
-          + " (applicationId,homeSubCluster,createTime) "
+          + " (applicationId,homeSubCluster,createTime,applicationContext) "
           + " (SELECT applicationId_IN, homeSubCluster_IN, "
-          + " NOW() AT TIME ZONE INTERVAL '0:00' HOUR TO MINUTE"
+          + " NOW() AT TIME ZONE INTERVAL '0:00' HOUR TO MINUTE, "
+          + " applicationContext_IN "
           + " FROM applicationsHomeSubCluster"
           + " WHERE applicationId = applicationId_IN"
           + " HAVING COUNT(*) = 0 );"
@@ -190,19 +193,24 @@ public class HSQLDBFederationStateStore extends SQLFederationStateStore {
   private static final String SP_UPDATEAPPLICATIONHOMESUBCLUSTER =
       "CREATE PROCEDURE sp_updateApplicationHomeSubCluster("
           + " IN applicationId_IN varchar(64),"
-          + " IN homeSubCluster_IN varchar(256), OUT rowCount_OUT int)"
+          + " IN homeSubCluster_IN varchar(256), "
+          + " IN applicationContext_IN BLOB, OUT rowCount_OUT int)"
           + " MODIFIES SQL DATA BEGIN ATOMIC"
           + " UPDATE applicationsHomeSubCluster"
-          + " SET homeSubCluster = homeSubCluster_IN"
+          + " SET homeSubCluster = homeSubCluster_IN, "
+          + " applicationContext = applicationContext_IN "
           + " WHERE applicationId = applicationId_IN;"
           + " GET DIAGNOSTICS rowCount_OUT = ROW_COUNT; END";
 
   private static final String SP_GETAPPLICATIONHOMESUBCLUSTER =
       "CREATE PROCEDURE sp_getApplicationHomeSubCluster("
           + " IN applicationId_IN varchar(64),"
-          + " OUT homeSubCluster_OUT varchar(256))"
+          + " OUT homeSubCluster_OUT varchar(256),"
+          + " OUT createTime_OUT datetime,"
+          + " OUT applicationContext_OUT BLOB)"
           + " MODIFIES SQL DATA BEGIN ATOMIC"
-          + " SELECT homeSubCluster INTO homeSubCluster_OUT"
+          + " SELECT homeSubCluster, applicationContext, createTime "
+          + " INTO homeSubCluster_OUT, applicationContext_OUT, createTime_OUT "
           + " FROM applicationsHomeSubCluster"
           + " WHERE applicationId = applicationID_IN; END";
 

+ 0 - 34
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/store/impl/TestMemoryFederationStateStore.java

@@ -19,10 +19,6 @@ package org.apache.hadoop.yarn.server.federation.store.impl;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.security.token.delegation.DelegationKey;
-import org.apache.hadoop.yarn.api.records.ApplicationId;
-import org.apache.hadoop.yarn.api.records.ApplicationSubmissionContext;
-import org.apache.hadoop.yarn.api.records.Priority;
-import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.exceptions.YarnException;
 import org.apache.hadoop.yarn.security.client.RMDelegationTokenIdentifier;
@@ -31,10 +27,6 @@ import org.apache.hadoop.yarn.server.federation.store.FederationStateStore;
 import org.apache.hadoop.yarn.server.federation.store.records.RouterMasterKey;
 import org.apache.hadoop.yarn.server.federation.store.records.RouterRMDTSecretManagerState;
 import org.apache.hadoop.yarn.server.federation.store.records.RouterStoreToken;
-import org.apache.hadoop.yarn.server.federation.store.records.SubClusterId;
-import org.apache.hadoop.yarn.server.federation.store.records.GetApplicationHomeSubClusterRequest;
-import org.apache.hadoop.yarn.server.federation.store.records.GetApplicationHomeSubClusterResponse;
-import org.junit.Test;
 
 import java.io.IOException;
 import java.nio.ByteBuffer;
@@ -96,30 +88,4 @@ public class TestMemoryFederationStateStore extends FederationStateStoreBaseTest
     assertTrue(tokenIdentifier instanceof RMDelegationTokenIdentifier);
     assertEquals(identifier, tokenIdentifier);
   }
-
-  @Test
-  public void testGetApplicationHomeSubClusterWithContext() throws Exception {
-    MemoryFederationStateStore memoryStateStore =
-        MemoryFederationStateStore.class.cast(this.getStateStore());
-
-    ApplicationId appId = ApplicationId.newInstance(1, 3);
-    SubClusterId subClusterId = SubClusterId.newInstance("SC");
-    ApplicationSubmissionContext context =
-        ApplicationSubmissionContext.newInstance(appId, "test", "default",
-        Priority.newInstance(0), null, true, true,
-        2, Resource.newInstance(10, 2), "test");
-    addApplicationHomeSC(appId, subClusterId, context);
-
-    GetApplicationHomeSubClusterRequest getRequest =
-        GetApplicationHomeSubClusterRequest.newInstance(appId, true);
-    GetApplicationHomeSubClusterResponse result =
-        memoryStateStore.getApplicationHomeSubCluster(getRequest);
-
-    assertEquals(appId,
-        result.getApplicationHomeSubCluster().getApplicationId());
-    assertEquals(subClusterId,
-        result.getApplicationHomeSubCluster().getHomeSubCluster());
-    assertEquals(context,
-        result.getApplicationHomeSubCluster().getApplicationSubmissionContext());
-  }
 }

+ 0 - 33
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/store/impl/TestZookeeperFederationStateStore.java

@@ -33,10 +33,6 @@ import org.apache.hadoop.metrics2.impl.MetricsCollectorImpl;
 import org.apache.hadoop.metrics2.impl.MetricsRecords;
 import org.apache.hadoop.security.token.delegation.DelegationKey;
 import org.apache.hadoop.util.Time;
-import org.apache.hadoop.yarn.api.records.ApplicationId;
-import org.apache.hadoop.yarn.api.records.ApplicationSubmissionContext;
-import org.apache.hadoop.yarn.api.records.Priority;
-import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.exceptions.YarnException;
 import org.apache.hadoop.yarn.security.client.RMDelegationTokenIdentifier;
@@ -45,10 +41,6 @@ import org.apache.hadoop.yarn.server.federation.store.records.RouterMasterKey;
 import org.apache.hadoop.yarn.server.federation.store.records.RouterMasterKeyRequest;
 import org.apache.hadoop.yarn.server.federation.store.records.RouterMasterKeyResponse;
 import org.apache.hadoop.yarn.server.federation.store.records.RouterStoreToken;
-import org.apache.hadoop.yarn.server.federation.store.records.SubClusterId;
-import org.apache.hadoop.yarn.server.federation.store.records.ApplicationHomeSubCluster;
-import org.apache.hadoop.yarn.server.federation.store.records.GetApplicationHomeSubClusterRequest;
-import org.apache.hadoop.yarn.server.federation.store.records.GetApplicationHomeSubClusterResponse;
 import org.apache.hadoop.yarn.util.Records;
 import org.junit.After;
 import org.junit.Before;
@@ -284,29 +276,4 @@ public class TestZookeeperFederationStateStore extends FederationStateStoreBaseT
     assertNotNull(zkRouterStoreToken);
     assertEquals(token, zkRouterStoreToken);
   }
-
-  @Test
-  public void testGetApplicationHomeSubClusterWithContext() throws Exception {
-    ZookeeperFederationStateStore zkFederationStateStore =
-        (ZookeeperFederationStateStore) this.getStateStore();
-
-    ApplicationId appId = ApplicationId.newInstance(1, 3);
-    SubClusterId subClusterId = SubClusterId.newInstance("SC");
-    ApplicationSubmissionContext context =
-        ApplicationSubmissionContext.newInstance(appId, "test", "default",
-        Priority.newInstance(0), null, true, true,
-        2, Resource.newInstance(10, 2), "test");
-    addApplicationHomeSC(appId, subClusterId, context);
-
-    GetApplicationHomeSubClusterRequest getRequest =
-        GetApplicationHomeSubClusterRequest.newInstance(appId, true);
-    GetApplicationHomeSubClusterResponse result =
-        zkFederationStateStore.getApplicationHomeSubCluster(getRequest);
-
-    ApplicationHomeSubCluster applicationHomeSubCluster = result.getApplicationHomeSubCluster();
-
-    assertEquals(appId, applicationHomeSubCluster.getApplicationId());
-    assertEquals(subClusterId, applicationHomeSubCluster.getHomeSubCluster());
-    assertEquals(context, applicationHomeSubCluster.getApplicationSubmissionContext());
-  }
 }